From 93af70438b92049226dfd130e04dd83a9863f1a9 Mon Sep 17 00:00:00 2001 From: advancedxy <xianjin@apache.org> Date: Fri, 24 May 2024 23:13:24 +0800 Subject: [PATCH 01/19] fix: Compute murmur3 hash with dictionary input correctly (#433) * fix: Handle compute murmur3 hash with dictionary input correctly * add unit tests * spotless apply * apply scala fix * address comment * another style issue * Update spark/src/test/scala/org/apache/comet/CometExpressionSuite.scala Co-authored-by: Liang-Chi Hsieh <viirya@gmail.com> --------- Co-authored-by: Liang-Chi Hsieh <viirya@gmail.com> --- core/src/execution/datafusion/spark_hash.rs | 270 ++++++++---------- .../apache/comet/CometExpressionSuite.scala | 58 +++- 2 files changed, 163 insertions(+), 165 deletions(-) diff --git a/core/src/execution/datafusion/spark_hash.rs b/core/src/execution/datafusion/spark_hash.rs index aa4269dd0..6d25a72f6 100644 --- a/core/src/execution/datafusion/spark_hash.rs +++ b/core/src/execution/datafusion/spark_hash.rs @@ -17,7 +17,10 @@ //! This includes utilities for hashing and murmur3 hashing. -use arrow::datatypes::{ArrowNativeTypeOp, UInt16Type, UInt32Type, UInt64Type, UInt8Type}; +use arrow::{ + compute::take, + datatypes::{ArrowNativeTypeOp, UInt16Type, UInt32Type, UInt64Type, UInt8Type}, +}; use std::sync::Arc; use datafusion::{ @@ -95,19 +98,8 @@ pub(crate) fn spark_compatible_murmur3_hash<T: AsRef<[u8]>>(data: T, seed: u32) } } -#[test] -fn test_murmur3() { - let _hashes = ["", "a", "ab", "abc", "abcd", "abcde"] - .into_iter() - .map(|s| spark_compatible_murmur3_hash(s.as_bytes(), 42) as i32) - .collect::<Vec<_>>(); - let _expected = vec![ - 142593372, 1485273170, -97053317, 1322437556, -396302900, 814637928, - ]; -} - macro_rules! hash_array { - ($array_type:ident, $column: ident, $hashes: ident) => { + ($array_type: ident, $column: ident, $hashes: ident) => { let array = $column.as_any().downcast_ref::<$array_type>().unwrap(); if array.null_count() == 0 { for (i, hash) in $hashes.iter_mut().enumerate() { @@ -123,8 +115,31 @@ macro_rules! hash_array { }; } +macro_rules! hash_array_boolean { + ($array_type: ident, $column: ident, $hash_input_type: ident, $hashes: ident) => { + let array = $column.as_any().downcast_ref::<$array_type>().unwrap(); + if array.null_count() == 0 { + for (i, hash) in $hashes.iter_mut().enumerate() { + *hash = spark_compatible_murmur3_hash( + $hash_input_type::from(array.value(i)).to_le_bytes(), + *hash, + ); + } + } else { + for (i, hash) in $hashes.iter_mut().enumerate() { + if !array.is_null(i) { + *hash = spark_compatible_murmur3_hash( + $hash_input_type::from(array.value(i)).to_le_bytes(), + *hash, + ); + } + } + } + }; +} + macro_rules! hash_array_primitive { - ($array_type:ident, $column: ident, $ty: ident, $hashes: ident) => { + ($array_type: ident, $column: ident, $ty: ident, $hashes: ident) => { let array = $column.as_any().downcast_ref::<$array_type>().unwrap(); let values = array.values(); @@ -143,7 +158,7 @@ macro_rules! hash_array_primitive { } macro_rules! hash_array_primitive_float { - ($array_type:ident, $column: ident, $ty: ident, $ty2: ident, $hashes: ident) => { + ($array_type: ident, $column: ident, $ty: ident, $ty2: ident, $hashes: ident) => { let array = $column.as_any().downcast_ref::<$array_type>().unwrap(); let values = array.values(); @@ -172,7 +187,7 @@ macro_rules! hash_array_primitive_float { } macro_rules! hash_array_decimal { - ($array_type:ident, $column: ident, $hashes: ident) => { + ($array_type: ident, $column: ident, $hashes: ident) => { let array = $column.as_any().downcast_ref::<$array_type>().unwrap(); if array.null_count() == 0 { @@ -193,27 +208,33 @@ macro_rules! hash_array_decimal { fn create_hashes_dictionary<K: ArrowDictionaryKeyType>( array: &ArrayRef, hashes_buffer: &mut [u32], + first_col: bool, ) -> Result<()> { let dict_array = array.as_any().downcast_ref::<DictionaryArray<K>>().unwrap(); - - // Hash each dictionary value once, and then use that computed - // hash for each key value to avoid a potentially expensive - // redundant hashing for large dictionary elements (e.g. strings) - let dict_values = Arc::clone(dict_array.values()); - let mut dict_hashes = vec![0; dict_values.len()]; - create_hashes(&[dict_values], &mut dict_hashes)?; - - for (hash, key) in hashes_buffer.iter_mut().zip(dict_array.keys().iter()) { - if let Some(key) = key { - let idx = key.to_usize().ok_or_else(|| { - DataFusionError::Internal(format!( - "Can not convert key value {:?} to usize in dictionary of type {:?}", - key, - dict_array.data_type() - )) - })?; - *hash = dict_hashes[idx] - } // no update for Null, consistent with other hashes + if !first_col { + // unpack the dictionary array as each row may have a different hash input + let unpacked = take(dict_array.values().as_ref(), dict_array.keys(), None)?; + create_hashes(&[unpacked], hashes_buffer)?; + } else { + // For the first column, hash each dictionary value once, and then use + // that computed hash for each key value to avoid a potentially + // expensive redundant hashing for large dictionary elements (e.g. strings) + let dict_values = Arc::clone(dict_array.values()); + // same initial seed as Spark + let mut dict_hashes = vec![42; dict_values.len()]; + create_hashes(&[dict_values], &mut dict_hashes)?; + for (hash, key) in hashes_buffer.iter_mut().zip(dict_array.keys().iter()) { + if let Some(key) = key { + let idx = key.to_usize().ok_or_else(|| { + DataFusionError::Internal(format!( + "Can not convert key value {:?} to usize in dictionary of type {:?}", + key, + dict_array.data_type() + )) + })?; + *hash = dict_hashes[idx] + } // no update for Null, consistent with other hashes + } } Ok(()) } @@ -227,27 +248,11 @@ pub fn create_hashes<'a>( arrays: &[ArrayRef], hashes_buffer: &'a mut [u32], ) -> Result<&'a mut [u32]> { - for col in arrays { + for (i, col) in arrays.iter().enumerate() { + let first_col = i == 0; match col.data_type() { DataType::Boolean => { - let array = col.as_any().downcast_ref::<BooleanArray>().unwrap(); - if array.null_count() == 0 { - for (i, hash) in hashes_buffer.iter_mut().enumerate() { - *hash = spark_compatible_murmur3_hash( - i32::from(array.value(i)).to_le_bytes(), - *hash, - ); - } - } else { - for (i, hash) in hashes_buffer.iter_mut().enumerate() { - if !array.is_null(i) { - *hash = spark_compatible_murmur3_hash( - i32::from(array.value(i)).to_le_bytes(), - *hash, - ); - } - } - } + hash_array_boolean!(BooleanArray, col, i32, hashes_buffer); } DataType::Int8 => { hash_array_primitive!(Int8Array, col, i32, hashes_buffer); @@ -305,28 +310,28 @@ pub fn create_hashes<'a>( } DataType::Dictionary(index_type, _) => match **index_type { DataType::Int8 => { - create_hashes_dictionary::<Int8Type>(col, hashes_buffer)?; + create_hashes_dictionary::<Int8Type>(col, hashes_buffer, first_col)?; } DataType::Int16 => { - create_hashes_dictionary::<Int16Type>(col, hashes_buffer)?; + create_hashes_dictionary::<Int16Type>(col, hashes_buffer, first_col)?; } DataType::Int32 => { - create_hashes_dictionary::<Int32Type>(col, hashes_buffer)?; + create_hashes_dictionary::<Int32Type>(col, hashes_buffer, first_col)?; } DataType::Int64 => { - create_hashes_dictionary::<Int64Type>(col, hashes_buffer)?; + create_hashes_dictionary::<Int64Type>(col, hashes_buffer, first_col)?; } DataType::UInt8 => { - create_hashes_dictionary::<UInt8Type>(col, hashes_buffer)?; + create_hashes_dictionary::<UInt8Type>(col, hashes_buffer, first_col)?; } DataType::UInt16 => { - create_hashes_dictionary::<UInt16Type>(col, hashes_buffer)?; + create_hashes_dictionary::<UInt16Type>(col, hashes_buffer, first_col)?; } DataType::UInt32 => { - create_hashes_dictionary::<UInt32Type>(col, hashes_buffer)?; + create_hashes_dictionary::<UInt32Type>(col, hashes_buffer, first_col)?; } DataType::UInt64 => { - create_hashes_dictionary::<UInt64Type>(col, hashes_buffer)?; + create_hashes_dictionary::<UInt64Type>(col, hashes_buffer, first_col)?; } _ => { return Err(DataFusionError::Internal(format!( @@ -363,78 +368,64 @@ mod tests { use crate::execution::datafusion::spark_hash::{create_hashes, pmod}; use datafusion::arrow::array::{ArrayRef, Int32Array, Int64Array, Int8Array, StringArray}; - macro_rules! test_hashes { - ($ty:ty, $values:expr, $expected:expr) => { - let i = Arc::new(<$ty>::from($values)) as ArrayRef; - let mut hashes = vec![42; $values.len()]; + macro_rules! test_hashes_internal { + ($input: expr, $len: expr, $expected: expr) => { + let i = $input as ArrayRef; + let mut hashes = vec![42; $len]; create_hashes(&[i], &mut hashes).unwrap(); assert_eq!(hashes, $expected); }; } + fn test_murmur3_hash<I: Clone, T: arrow_array::Array + From<Vec<Option<I>>> + 'static>( + values: Vec<Option<I>>, + expected: Vec<u32>, + ) { + // copied before inserting nulls + let mut input_with_nulls = values.clone(); + let mut expected_with_nulls = expected.clone(); + let len = values.len(); + let i = Arc::new(T::from(values)) as ArrayRef; + test_hashes_internal!(i, len, expected); + + // test with nulls + let median = len / 2; + input_with_nulls.insert(0, None); + input_with_nulls.insert(median, None); + expected_with_nulls.insert(0, 42); + expected_with_nulls.insert(median, 42); + let with_nulls_len = len + 2; + let nullable_input = Arc::new(T::from(input_with_nulls)) as ArrayRef; + test_hashes_internal!(nullable_input, with_nulls_len, expected_with_nulls); + } + #[test] fn test_i8() { - test_hashes!( - Int8Array, + test_murmur3_hash::<i8, Int8Array>( vec![Some(1), Some(0), Some(-1), Some(i8::MAX), Some(i8::MIN)], - vec![0xdea578e3, 0x379fae8f, 0xa0590e3d, 0x43b4d8ed, 0x422a1365] - ); - // with null input - test_hashes!( - Int8Array, - vec![Some(1), None, Some(-1), Some(i8::MAX), Some(i8::MIN)], - vec![0xdea578e3, 42, 0xa0590e3d, 0x43b4d8ed, 0x422a1365] + vec![0xdea578e3, 0x379fae8f, 0xa0590e3d, 0x43b4d8ed, 0x422a1365], ); } #[test] fn test_i32() { - test_hashes!( - Int32Array, + test_murmur3_hash::<i32, Int32Array>( vec![Some(1), Some(0), Some(-1), Some(i32::MAX), Some(i32::MIN)], - vec![0xdea578e3, 0x379fae8f, 0xa0590e3d, 0x07fb67e7, 0x2b1f0fc6] - ); - // with null input - test_hashes!( - Int32Array, - vec![ - Some(1), - Some(0), - Some(-1), - None, - Some(i32::MAX), - Some(i32::MIN) - ], - vec![0xdea578e3, 0x379fae8f, 0xa0590e3d, 42, 0x07fb67e7, 0x2b1f0fc6] + vec![0xdea578e3, 0x379fae8f, 0xa0590e3d, 0x07fb67e7, 0x2b1f0fc6], ); } #[test] fn test_i64() { - test_hashes!( - Int64Array, + test_murmur3_hash::<i64, Int64Array>( vec![Some(1), Some(0), Some(-1), Some(i64::MAX), Some(i64::MIN)], - vec![0x99f0149d, 0x9c67b85d, 0xc8008529, 0xa05b5d7b, 0xcd1e64fb] - ); - // with null input - test_hashes!( - Int64Array, - vec![ - Some(1), - Some(0), - Some(-1), - None, - Some(i64::MAX), - Some(i64::MIN) - ], - vec![0x99f0149d, 0x9c67b85d, 0xc8008529, 42, 0xa05b5d7b, 0xcd1e64fb] + vec![0x99f0149d, 0x9c67b85d, 0xc8008529, 0xa05b5d7b, 0xcd1e64fb], ); } #[test] fn test_f32() { - test_hashes!( - Float32Array, + test_murmur3_hash::<f32, Float32Array>( vec![ Some(1.0), Some(0.0), @@ -443,28 +434,15 @@ mod tests { Some(99999999999.99999999999), Some(-99999999999.99999999999), ], - vec![0xe434cc39, 0x379fae8f, 0x379fae8f, 0xdc0da8eb, 0xcbdc340f, 0xc0361c86] - ); - // with null input - test_hashes!( - Float32Array, vec![ - Some(1.0), - Some(0.0), - Some(-0.0), - Some(-1.0), - None, - Some(99999999999.99999999999), - Some(-99999999999.99999999999) + 0xe434cc39, 0x379fae8f, 0x379fae8f, 0xdc0da8eb, 0xcbdc340f, 0xc0361c86, ], - vec![0xe434cc39, 0x379fae8f, 0x379fae8f, 0xdc0da8eb, 42, 0xcbdc340f, 0xc0361c86] ); } #[test] fn test_f64() { - test_hashes!( - Float64Array, + test_murmur3_hash::<f64, Float64Array>( vec![ Some(1.0), Some(0.0), @@ -473,44 +451,26 @@ mod tests { Some(99999999999.99999999999), Some(-99999999999.99999999999), ], - vec![0xe4876492, 0x9c67b85d, 0x9c67b85d, 0x13d81357, 0xb87e1595, 0xa0eef9f9] - ); - // with null input - test_hashes!( - Float64Array, vec![ - Some(1.0), - Some(0.0), - Some(-0.0), - Some(-1.0), - None, - Some(99999999999.99999999999), - Some(-99999999999.99999999999) + 0xe4876492, 0x9c67b85d, 0x9c67b85d, 0x13d81357, 0xb87e1595, 0xa0eef9f9, ], - vec![0xe4876492, 0x9c67b85d, 0x9c67b85d, 0x13d81357, 42, 0xb87e1595, 0xa0eef9f9] ); } #[test] fn test_str() { - test_hashes!( - StringArray, - vec!["hello", "bar", "", "😁", "天地"], - vec![3286402344, 2486176763, 142593372, 885025535, 2395000894] - ); - // test with null input - test_hashes!( - StringArray, - vec![ - Some("hello"), - Some("bar"), - None, - Some(""), - Some("😁"), - Some("天地") - ], - vec![3286402344, 2486176763, 42, 142593372, 885025535, 2395000894] - ); + let input = vec![ + "hello", "bar", "", "😁", "天地", "a", "ab", "abc", "abcd", "abcde", + ] + .iter() + .map(|s| Some(s.to_string())) + .collect::<Vec<Option<String>>>(); + let expected: Vec<u32> = vec![ + 3286402344, 2486176763, 142593372, 885025535, 2395000894, 1485273170, 0xfa37157b, + 1322437556, 0xe860e5cc, 814637928, + ]; + + test_murmur3_hash::<String, StringArray>(input, expected); } #[test] diff --git a/spark/src/test/scala/org/apache/comet/CometExpressionSuite.scala b/spark/src/test/scala/org/apache/comet/CometExpressionSuite.scala index 6ca4baf60..992615084 100644 --- a/spark/src/test/scala/org/apache/comet/CometExpressionSuite.scala +++ b/spark/src/test/scala/org/apache/comet/CometExpressionSuite.scala @@ -1454,17 +1454,55 @@ class CometExpressionSuite extends CometTestBase with AdaptiveSparkPlanHelper { withTable(table) { sql(s"create table $table(col string, a int, b float) using parquet") sql(s""" - |insert into $table values - |('Spark SQL ', 10, 1.2), (NULL, NULL, NULL), ('', 0, 0.0), ('苹果手机', NULL, 3.999999) - |, ('Spark SQL ', 10, 1.2), (NULL, NULL, NULL), ('', 0, 0.0), ('苹果手机', NULL, 3.999999) - |""".stripMargin) + |insert into $table values + |('Spark SQL ', 10, 1.2), (NULL, NULL, NULL), ('', 0, 0.0), ('苹果手机', NULL, 3.999999) + |, ('Spark SQL ', 10, 1.2), (NULL, NULL, NULL), ('', 0, 0.0), ('苹果手机', NULL, 3.999999) + |""".stripMargin) checkSparkAnswerAndOperator(""" - |select - |md5(col), md5(cast(a as string)), md5(cast(b as string)), - |hash(col), hash(col, 1), hash(col, 0), hash(col, a, b), hash(b, a, col), - |sha2(col, 0), sha2(col, 256), sha2(col, 224), sha2(col, 384), sha2(col, 512), sha2(col, 128) - |from test - |""".stripMargin) + |select + |md5(col), md5(cast(a as string)), md5(cast(b as string)), + |hash(col), hash(col, 1), hash(col, 0), hash(col, a, b), hash(b, a, col), + |sha2(col, 0), sha2(col, 256), sha2(col, 224), sha2(col, 384), sha2(col, 512), sha2(col, 128) + |from test + |""".stripMargin) + } + } + } + } + + test("hash functions with random input") { + val dataGen = DataGenerator.DEFAULT + // sufficient number of rows to create dictionary encoded ArrowArray. + val randomNumRows = 1000 + + val whitespaceChars = " \t\r\n" + val timestampPattern = "0123456789/:T" + whitespaceChars + Seq(true, false).foreach { dictionary => + withSQLConf( + "parquet.enable.dictionary" -> dictionary.toString, + CometConf.COMET_CAST_ALLOW_INCOMPATIBLE.key -> "true") { + val table = "test" + withTable(table) { + sql(s"create table $table(col string, a int, b float) using parquet") + // TODO: Add a Row generator in the data gen class and replace th following code + val col = dataGen.generateStrings(randomNumRows, timestampPattern, 6) + val colA = dataGen.generateInts(randomNumRows) + val colB = dataGen.generateFloats(randomNumRows) + val data = col.zip(colA).zip(colB).map { case ((a, b), c) => (a, b, c) } + data + .toDF("col", "a", "b") + .write + .mode("append") + .insertInto(table) + // with random generated data + // disable cast(b as string) for now, as the cast from float to string may produce incompatible result + checkSparkAnswerAndOperator(""" + |select + |md5(col), md5(cast(a as string)), --md5(cast(b as string)), + |hash(col), hash(col, 1), hash(col, 0), hash(col, a, b), hash(b, a, col), + |sha2(col, 0), sha2(col, 256), sha2(col, 224), sha2(col, 384), sha2(col, 512), sha2(col, 128) + |from test + |""".stripMargin) } } } From 79431f8837f21f91cb24f3a0b4c11a364dcd70f9 Mon Sep 17 00:00:00 2001 From: Andy Grove <agrove@apache.org> Date: Sat, 25 May 2024 12:48:23 -0600 Subject: [PATCH 02/19] fix: Only delegate to DataFusion cast when we know that it is compatible with Spark (#461) * only delegate to DataFusion cast when we know that it is compatible with Spark * add more supported casts * improve support for dictionary-encoded string arrays * clippy * fix merge conflict * fix a regression * fix a regression * fix a regression * fix regression * fix regression * fix regression * remove TODO comment now that issue has been filed * remove cast int32/int64 -> decimal from datafusion compatible list * Revert "remove cast int32/int64 -> decimal from datafusion compatible list" This reverts commit 340e00007575e7f91affa018caa7cd9e2d8964f9. * add comment --- .../execution/datafusion/expressions/cast.rs | 182 +++++++++++------- 1 file changed, 115 insertions(+), 67 deletions(-) diff --git a/core/src/execution/datafusion/expressions/cast.rs b/core/src/execution/datafusion/expressions/cast.rs index fd1f9166d..7e8a96f28 100644 --- a/core/src/execution/datafusion/expressions/cast.rs +++ b/core/src/execution/datafusion/expressions/cast.rs @@ -503,41 +503,37 @@ impl Cast { fn cast_array(&self, array: ArrayRef) -> DataFusionResult<ArrayRef> { let to_type = &self.data_type; let array = array_with_timezone(array, self.timezone.clone(), Some(to_type)); + let from_type = array.data_type().clone(); + + // unpack dictionary string arrays first + // TODO: we are unpacking a dictionary-encoded array and then performing + // the cast. We could potentially improve performance here by casting the + // dictionary values directly without unpacking the array first, although this + // would add more complexity to the code + let array = match &from_type { + DataType::Dictionary(key_type, value_type) + if key_type.as_ref() == &DataType::Int32 + && (value_type.as_ref() == &DataType::Utf8 + || value_type.as_ref() == &DataType::LargeUtf8) => + { + cast_with_options(&array, value_type.as_ref(), &CAST_OPTIONS)? + } + _ => array, + }; let from_type = array.data_type(); + let cast_result = match (from_type, to_type) { (DataType::Utf8, DataType::Boolean) => { - Self::spark_cast_utf8_to_boolean::<i32>(&array, self.eval_mode)? + Self::spark_cast_utf8_to_boolean::<i32>(&array, self.eval_mode) } (DataType::LargeUtf8, DataType::Boolean) => { - Self::spark_cast_utf8_to_boolean::<i64>(&array, self.eval_mode)? + Self::spark_cast_utf8_to_boolean::<i64>(&array, self.eval_mode) } (DataType::Utf8, DataType::Timestamp(_, _)) => { - Self::cast_string_to_timestamp(&array, to_type, self.eval_mode)? + Self::cast_string_to_timestamp(&array, to_type, self.eval_mode) } (DataType::Utf8, DataType::Date32) => { - Self::cast_string_to_date(&array, to_type, self.eval_mode)? - } - (DataType::Dictionary(key_type, value_type), DataType::Date32) - if key_type.as_ref() == &DataType::Int32 - && (value_type.as_ref() == &DataType::Utf8 - || value_type.as_ref() == &DataType::LargeUtf8) => - { - match value_type.as_ref() { - DataType::Utf8 => { - let unpacked_array = - cast_with_options(&array, &DataType::Utf8, &CAST_OPTIONS)?; - Self::cast_string_to_date(&unpacked_array, to_type, self.eval_mode)? - } - DataType::LargeUtf8 => { - let unpacked_array = - cast_with_options(&array, &DataType::LargeUtf8, &CAST_OPTIONS)?; - Self::cast_string_to_date(&unpacked_array, to_type, self.eval_mode)? - } - dt => unreachable!( - "{}", - format!("invalid value type {dt} for dictionary-encoded string array") - ), - } + Self::cast_string_to_date(&array, to_type, self.eval_mode) } (DataType::Int64, DataType::Int32) | (DataType::Int64, DataType::Int16) @@ -547,61 +543,33 @@ impl Cast { | (DataType::Int16, DataType::Int8) if self.eval_mode != EvalMode::Try => { - Self::spark_cast_int_to_int(&array, self.eval_mode, from_type, to_type)? + Self::spark_cast_int_to_int(&array, self.eval_mode, from_type, to_type) } ( DataType::Utf8, DataType::Int8 | DataType::Int16 | DataType::Int32 | DataType::Int64, - ) => Self::cast_string_to_int::<i32>(to_type, &array, self.eval_mode)?, + ) => Self::cast_string_to_int::<i32>(to_type, &array, self.eval_mode), ( DataType::LargeUtf8, DataType::Int8 | DataType::Int16 | DataType::Int32 | DataType::Int64, - ) => Self::cast_string_to_int::<i64>(to_type, &array, self.eval_mode)?, - ( - DataType::Dictionary(key_type, value_type), - DataType::Int8 | DataType::Int16 | DataType::Int32 | DataType::Int64, - ) if key_type.as_ref() == &DataType::Int32 - && (value_type.as_ref() == &DataType::Utf8 - || value_type.as_ref() == &DataType::LargeUtf8) => - { - // TODO: we are unpacking a dictionary-encoded array and then performing - // the cast. We could potentially improve performance here by casting the - // dictionary values directly without unpacking the array first, although this - // would add more complexity to the code - match value_type.as_ref() { - DataType::Utf8 => { - let unpacked_array = - cast_with_options(&array, &DataType::Utf8, &CAST_OPTIONS)?; - Self::cast_string_to_int::<i32>(to_type, &unpacked_array, self.eval_mode)? - } - DataType::LargeUtf8 => { - let unpacked_array = - cast_with_options(&array, &DataType::LargeUtf8, &CAST_OPTIONS)?; - Self::cast_string_to_int::<i64>(to_type, &unpacked_array, self.eval_mode)? - } - dt => unreachable!( - "{}", - format!("invalid value type {dt} for dictionary-encoded string array") - ), - } - } + ) => Self::cast_string_to_int::<i64>(to_type, &array, self.eval_mode), (DataType::Float64, DataType::Utf8) => { - Self::spark_cast_float64_to_utf8::<i32>(&array, self.eval_mode)? + Self::spark_cast_float64_to_utf8::<i32>(&array, self.eval_mode) } (DataType::Float64, DataType::LargeUtf8) => { - Self::spark_cast_float64_to_utf8::<i64>(&array, self.eval_mode)? + Self::spark_cast_float64_to_utf8::<i64>(&array, self.eval_mode) } (DataType::Float32, DataType::Utf8) => { - Self::spark_cast_float32_to_utf8::<i32>(&array, self.eval_mode)? + Self::spark_cast_float32_to_utf8::<i32>(&array, self.eval_mode) } (DataType::Float32, DataType::LargeUtf8) => { - Self::spark_cast_float32_to_utf8::<i64>(&array, self.eval_mode)? + Self::spark_cast_float32_to_utf8::<i64>(&array, self.eval_mode) } (DataType::Float32, DataType::Decimal128(precision, scale)) => { - Self::cast_float32_to_decimal128(&array, *precision, *scale, self.eval_mode)? + Self::cast_float32_to_decimal128(&array, *precision, *scale, self.eval_mode) } (DataType::Float64, DataType::Decimal128(precision, scale)) => { - Self::cast_float64_to_decimal128(&array, *precision, *scale, self.eval_mode)? + Self::cast_float64_to_decimal128(&array, *precision, *scale, self.eval_mode) } (DataType::Float32, DataType::Int8) | (DataType::Float32, DataType::Int16) @@ -622,14 +590,94 @@ impl Cast { self.eval_mode, from_type, to_type, - )? + ) + } + _ if Self::is_datafusion_spark_compatible(from_type, to_type) => { + // use DataFusion cast only when we know that it is compatible with Spark + Ok(cast_with_options(&array, to_type, &CAST_OPTIONS)?) } _ => { - // when we have no Spark-specific casting we delegate to DataFusion - cast_with_options(&array, to_type, &CAST_OPTIONS)? + // we should never reach this code because the Scala code should be checking + // for supported cast operations and falling back to Spark for anything that + // is not yet supported + Err(CometError::Internal(format!( + "Native cast invoked for unsupported cast from {from_type:?} to {to_type:?}" + ))) } }; - Ok(spark_cast(cast_result, from_type, to_type)) + Ok(spark_cast(cast_result?, from_type, to_type)) + } + + /// Determines if DataFusion supports the given cast in a way that is + /// compatible with Spark + fn is_datafusion_spark_compatible(from_type: &DataType, to_type: &DataType) -> bool { + if from_type == to_type { + return true; + } + match from_type { + DataType::Boolean => matches!( + to_type, + DataType::Int8 + | DataType::Int16 + | DataType::Int32 + | DataType::Int64 + | DataType::Float32 + | DataType::Float64 + | DataType::Utf8 + ), + DataType::Int8 | DataType::Int16 | DataType::Int32 | DataType::Int64 => { + // note that the cast from Int32/Int64 -> Decimal128 here is actually + // not compatible with Spark (no overflow checks) but we have tests that + // rely on this cast working so we have to leave it here for now + matches!( + to_type, + DataType::Boolean + | DataType::Int8 + | DataType::Int16 + | DataType::Int32 + | DataType::Int64 + | DataType::Float32 + | DataType::Float64 + | DataType::Decimal128(_, _) + | DataType::Utf8 + ) + } + DataType::Float32 | DataType::Float64 => matches!( + to_type, + DataType::Boolean + | DataType::Int8 + | DataType::Int16 + | DataType::Int32 + | DataType::Int64 + | DataType::Float32 + | DataType::Float64 + ), + DataType::Decimal128(_, _) | DataType::Decimal256(_, _) => matches!( + to_type, + DataType::Int8 + | DataType::Int16 + | DataType::Int32 + | DataType::Int64 + | DataType::Float32 + | DataType::Float64 + | DataType::Decimal128(_, _) + | DataType::Decimal256(_, _) + ), + DataType::Utf8 => matches!(to_type, DataType::Binary), + DataType::Date32 => matches!(to_type, DataType::Utf8), + DataType::Timestamp(_, _) => { + matches!( + to_type, + DataType::Int64 | DataType::Date32 | DataType::Utf8 | DataType::Timestamp(_, _) + ) + } + DataType::Binary => { + // note that this is not completely Spark compatible because + // DataFusion only supports binary data containing valid UTF-8 strings + matches!(to_type, DataType::Utf8) + } + _ => false, + } } fn cast_string_to_int<OffsetSize: OffsetSizeTrait>( From 5bfe46d4f1841985a85b74dc8e8ceaf10a8e3266 Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh <viirya@gmail.com> Date: Sat, 25 May 2024 14:05:02 -0700 Subject: [PATCH 03/19] fix: `ColumnReader.loadVector` should initiate `CometDictionary` after re-import arrays (#473) --- .../org/apache/comet/parquet/ColumnReader.java | 14 +++++++------- 1 file changed, 7 insertions(+), 7 deletions(-) diff --git a/common/src/main/java/org/apache/comet/parquet/ColumnReader.java b/common/src/main/java/org/apache/comet/parquet/ColumnReader.java index 46fd87f6b..50991d5b0 100644 --- a/common/src/main/java/org/apache/comet/parquet/ColumnReader.java +++ b/common/src/main/java/org/apache/comet/parquet/ColumnReader.java @@ -230,15 +230,15 @@ public CometDecodedVector loadVector() { // return plain vector. currentVector = cometVector; return currentVector; - } else if (dictionary == null) { - // There is dictionary from native side but the Java side dictionary hasn't been - // initialized yet. - Dictionary arrowDictionary = dictionaryProvider.lookup(dictionaryEncoding.getId()); - CometPlainVector dictionaryVector = - new CometPlainVector(arrowDictionary.getVector(), useDecimal128, isUuid); - dictionary = new CometDictionary(dictionaryVector); } + // We should already re-initiate `CometDictionary` here because `Data.importVector` API will + // release the previous dictionary vector and create a new one. + Dictionary arrowDictionary = dictionaryProvider.lookup(dictionaryEncoding.getId()); + CometPlainVector dictionaryVector = + new CometPlainVector(arrowDictionary.getVector(), useDecimal128, isUuid); + dictionary = new CometDictionary(dictionaryVector); + currentVector = new CometDictionaryVector( cometVector, dictionary, dictionaryProvider, useDecimal128, false, isUuid); From 7ba569357cadade64b49b6bdf5f0946f81f95301 Mon Sep 17 00:00:00 2001 From: Son <14060682+sonhmai@users.noreply.github.com> Date: Mon, 27 May 2024 03:13:19 +0700 Subject: [PATCH 04/19] fix: substring with negative indices should produce correct result (#470) --- core/src/execution/datafusion/planner.rs | 3 ++- .../test/scala/org/apache/comet/CometExpressionSuite.scala | 6 ++++++ 2 files changed, 8 insertions(+), 1 deletion(-) diff --git a/core/src/execution/datafusion/planner.rs b/core/src/execution/datafusion/planner.rs index 01d892381..7a37e3aae 100644 --- a/core/src/execution/datafusion/planner.rs +++ b/core/src/execution/datafusion/planner.rs @@ -395,7 +395,8 @@ impl PhysicalPlanner { let child = self.create_expr(expr.child.as_ref().unwrap(), input_schema)?; // Spark Substring's start is 1-based when start > 0 let start = expr.start - i32::from(expr.start > 0); - let len = expr.len; + // substring negative len is treated as 0 in Spark + let len = std::cmp::max(expr.len, 0); Ok(Arc::new(SubstringExec::new( child, diff --git a/spark/src/test/scala/org/apache/comet/CometExpressionSuite.scala b/spark/src/test/scala/org/apache/comet/CometExpressionSuite.scala index 992615084..1afdd78ec 100644 --- a/spark/src/test/scala/org/apache/comet/CometExpressionSuite.scala +++ b/spark/src/test/scala/org/apache/comet/CometExpressionSuite.scala @@ -156,6 +156,12 @@ class CometExpressionSuite extends CometTestBase with AdaptiveSparkPlanHelper { test("string type and substring") { withParquetTable((0 until 5).map(i => (i.toString, (i + 100).toString)), "tbl") { checkSparkAnswerAndOperator("SELECT _1, substring(_2, 2, 2) FROM tbl") + checkSparkAnswerAndOperator("SELECT _1, substring(_2, 2, -2) FROM tbl") + checkSparkAnswerAndOperator("SELECT _1, substring(_2, -2, 2) FROM tbl") + checkSparkAnswerAndOperator("SELECT _1, substring(_2, -2, -2) FROM tbl") + checkSparkAnswerAndOperator("SELECT _1, substring(_2, -2, 10) FROM tbl") + checkSparkAnswerAndOperator("SELECT _1, substring(_2, 0, 0) FROM tbl") + checkSparkAnswerAndOperator("SELECT _1, substring(_2, 1, 0) FROM tbl") } } From 479a97a7a40f13609d591a5a08c1265b1cc74fdd Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh <viirya@gmail.com> Date: Mon, 27 May 2024 18:55:19 -0700 Subject: [PATCH 05/19] fix: CometReader.loadVector should not overwrite dictionary ids (#476) * fix: CometReader.loadVector should not overwrite dictionary ids * For review --- .../apache/arrow/c/CometSchemaImporter.java | 73 +++++++++++++++++++ .../org/apache/comet/parquet/BatchReader.java | 13 ++++ .../apache/comet/parquet/ColumnReader.java | 23 +++--- .../comet/parquet/LazyColumnReader.java | 4 +- .../java/org/apache/comet/parquet/Utils.java | 10 ++- 5 files changed, 107 insertions(+), 16 deletions(-) create mode 100644 common/src/main/java/org/apache/arrow/c/CometSchemaImporter.java diff --git a/common/src/main/java/org/apache/arrow/c/CometSchemaImporter.java b/common/src/main/java/org/apache/arrow/c/CometSchemaImporter.java new file mode 100644 index 000000000..32955f1ac --- /dev/null +++ b/common/src/main/java/org/apache/arrow/c/CometSchemaImporter.java @@ -0,0 +1,73 @@ +/* + * 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.arrow.c; + +import org.apache.arrow.memory.BufferAllocator; +import org.apache.arrow.vector.FieldVector; +import org.apache.arrow.vector.types.pojo.Field; + +/** This is a simple wrapper around SchemaImporter to make it accessible from Java Arrow. */ +public class CometSchemaImporter { + private final BufferAllocator allocator; + private final SchemaImporter importer; + private final CDataDictionaryProvider provider = new CDataDictionaryProvider(); + + public CometSchemaImporter(BufferAllocator allocator) { + this.allocator = allocator; + this.importer = new SchemaImporter(allocator); + } + + public BufferAllocator getAllocator() { + return allocator; + } + + public CDataDictionaryProvider getProvider() { + return provider; + } + + public Field importField(ArrowSchema schema) { + try { + return importer.importField(schema, provider); + } finally { + schema.release(); + schema.close(); + } + } + + /** + * Imports data from ArrowArray/ArrowSchema into a FieldVector. This is basically the same as Java + * Arrow `Data.importVector`. `Data.importVector` initiates `SchemaImporter` internally which is + * used to fill dictionary ids for dictionary encoded vectors. Every call to `importVector` will + * begin with dictionary ids starting from 0. So, separate calls to `importVector` will overwrite + * dictionary ids. To avoid this, we need to use the same `SchemaImporter` instance for all calls + * to `importVector`. + */ + public FieldVector importVector(ArrowArray array, ArrowSchema schema) { + Field field = importField(schema); + FieldVector vector = field.createVector(allocator); + Data.importIntoVector(allocator, array, vector, provider); + + return vector; + } + + public void close() { + provider.close(); + } +} diff --git a/common/src/main/java/org/apache/comet/parquet/BatchReader.java b/common/src/main/java/org/apache/comet/parquet/BatchReader.java index 9940390dc..bf8e6e550 100644 --- a/common/src/main/java/org/apache/comet/parquet/BatchReader.java +++ b/common/src/main/java/org/apache/comet/parquet/BatchReader.java @@ -37,6 +37,9 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import org.apache.arrow.c.CometSchemaImporter; +import org.apache.arrow.memory.BufferAllocator; +import org.apache.arrow.memory.RootAllocator; import org.apache.commons.lang3.tuple.Pair; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; @@ -88,6 +91,7 @@ */ public class BatchReader extends RecordReader<Void, ColumnarBatch> implements Closeable { private static final Logger LOG = LoggerFactory.getLogger(FileReader.class); + protected static final BufferAllocator ALLOCATOR = new RootAllocator(); private Configuration conf; private int capacity; @@ -104,6 +108,7 @@ public class BatchReader extends RecordReader<Void, ColumnarBatch> implements Cl private MessageType requestedSchema; private CometVector[] vectors; private AbstractColumnReader[] columnReaders; + private CometSchemaImporter importer; private ColumnarBatch currentBatch; private Future<Option<Throwable>> prefetchTask; private LinkedBlockingQueue<Pair<PageReadStore, Long>> prefetchQueue; @@ -515,6 +520,10 @@ public void close() throws IOException { fileReader.close(); fileReader = null; } + if (importer != null) { + importer.close(); + importer = null; + } } @SuppressWarnings("deprecation") @@ -552,6 +561,9 @@ private boolean loadNextRowGroupIfNecessary() throws Throwable { numRowGroupsMetric.add(1); } + if (importer != null) importer.close(); + importer = new CometSchemaImporter(ALLOCATOR); + List<ColumnDescriptor> columns = requestedSchema.getColumns(); for (int i = 0; i < columns.size(); i++) { if (missingColumns[i]) continue; @@ -564,6 +576,7 @@ private boolean loadNextRowGroupIfNecessary() throws Throwable { Utils.getColumnReader( dataType, columns.get(i), + importer, capacity, useDecimal128, useLazyMaterialization, diff --git a/common/src/main/java/org/apache/comet/parquet/ColumnReader.java b/common/src/main/java/org/apache/comet/parquet/ColumnReader.java index 50991d5b0..3d4cb3aa5 100644 --- a/common/src/main/java/org/apache/comet/parquet/ColumnReader.java +++ b/common/src/main/java/org/apache/comet/parquet/ColumnReader.java @@ -27,10 +27,7 @@ import org.apache.arrow.c.ArrowArray; import org.apache.arrow.c.ArrowSchema; -import org.apache.arrow.c.CDataDictionaryProvider; -import org.apache.arrow.c.Data; -import org.apache.arrow.memory.BufferAllocator; -import org.apache.arrow.memory.RootAllocator; +import org.apache.arrow.c.CometSchemaImporter; import org.apache.arrow.vector.FieldVector; import org.apache.arrow.vector.dictionary.Dictionary; import org.apache.arrow.vector.types.pojo.DictionaryEncoding; @@ -53,7 +50,6 @@ public class ColumnReader extends AbstractColumnReader { protected static final Logger LOG = LoggerFactory.getLogger(ColumnReader.class); - protected static final BufferAllocator ALLOCATOR = new RootAllocator(); /** * The current Comet vector holding all the values read by this column reader. Owned by this @@ -89,18 +85,19 @@ public class ColumnReader extends AbstractColumnReader { */ boolean hadNull; - /** Dictionary provider for this column. */ - private final CDataDictionaryProvider dictionaryProvider = new CDataDictionaryProvider(); + private final CometSchemaImporter importer; public ColumnReader( DataType type, ColumnDescriptor descriptor, + CometSchemaImporter importer, int batchSize, boolean useDecimal128, boolean useLegacyDateTimestamp) { super(type, descriptor, useDecimal128, useLegacyDateTimestamp); assert batchSize > 0 : "Batch size must be positive, found " + batchSize; this.batchSize = batchSize; + this.importer = importer; initNative(); } @@ -164,7 +161,6 @@ public void close() { currentVector.close(); currentVector = null; } - dictionaryProvider.close(); super.close(); } @@ -209,10 +205,11 @@ public CometDecodedVector loadVector() { try (ArrowArray array = ArrowArray.wrap(addresses[0]); ArrowSchema schema = ArrowSchema.wrap(addresses[1])) { - FieldVector vector = Data.importVector(ALLOCATOR, array, schema, dictionaryProvider); + FieldVector vector = importer.importVector(array, schema); + DictionaryEncoding dictionaryEncoding = vector.getField().getDictionary(); - CometPlainVector cometVector = new CometPlainVector(vector, useDecimal128, isUuid); + CometPlainVector cometVector = new CometPlainVector(vector, useDecimal128); // Update whether the current vector contains any null values. This is used in the following // batch(s) to determine whether we can skip loading the native vector. @@ -234,15 +231,17 @@ public CometDecodedVector loadVector() { // We should already re-initiate `CometDictionary` here because `Data.importVector` API will // release the previous dictionary vector and create a new one. - Dictionary arrowDictionary = dictionaryProvider.lookup(dictionaryEncoding.getId()); + Dictionary arrowDictionary = importer.getProvider().lookup(dictionaryEncoding.getId()); CometPlainVector dictionaryVector = new CometPlainVector(arrowDictionary.getVector(), useDecimal128, isUuid); dictionary = new CometDictionary(dictionaryVector); currentVector = new CometDictionaryVector( - cometVector, dictionary, dictionaryProvider, useDecimal128, false, isUuid); + cometVector, dictionary, importer.getProvider(), useDecimal128, false, isUuid); + currentVector = + new CometDictionaryVector(cometVector, dictionary, importer.getProvider(), useDecimal128); return currentVector; } } diff --git a/common/src/main/java/org/apache/comet/parquet/LazyColumnReader.java b/common/src/main/java/org/apache/comet/parquet/LazyColumnReader.java index a15d84192..dd08a88ab 100644 --- a/common/src/main/java/org/apache/comet/parquet/LazyColumnReader.java +++ b/common/src/main/java/org/apache/comet/parquet/LazyColumnReader.java @@ -21,6 +21,7 @@ import java.io.IOException; +import org.apache.arrow.c.CometSchemaImporter; import org.apache.parquet.column.ColumnDescriptor; import org.apache.parquet.column.page.PageReader; import org.apache.spark.sql.types.DataType; @@ -45,10 +46,11 @@ public class LazyColumnReader extends ColumnReader { public LazyColumnReader( DataType sparkReadType, ColumnDescriptor descriptor, + CometSchemaImporter importer, int batchSize, boolean useDecimal128, boolean useLegacyDateTimestamp) { - super(sparkReadType, descriptor, batchSize, useDecimal128, useLegacyDateTimestamp); + super(sparkReadType, descriptor, importer, batchSize, useDecimal128, useLegacyDateTimestamp); this.batchSize = 0; // the batch size is set later in `readBatch` this.vector = new CometLazyVector(sparkReadType, this, useDecimal128); } diff --git a/common/src/main/java/org/apache/comet/parquet/Utils.java b/common/src/main/java/org/apache/comet/parquet/Utils.java index 95ca06cda..99f3a4edd 100644 --- a/common/src/main/java/org/apache/comet/parquet/Utils.java +++ b/common/src/main/java/org/apache/comet/parquet/Utils.java @@ -19,6 +19,7 @@ package org.apache.comet.parquet; +import org.apache.arrow.c.CometSchemaImporter; import org.apache.parquet.column.ColumnDescriptor; import org.apache.parquet.schema.LogicalTypeAnnotation; import org.apache.parquet.schema.PrimitiveType; @@ -28,26 +29,29 @@ public class Utils { public static ColumnReader getColumnReader( DataType type, ColumnDescriptor descriptor, + CometSchemaImporter importer, int batchSize, boolean useDecimal128, boolean useLazyMaterialization) { // TODO: support `useLegacyDateTimestamp` for Iceberg return getColumnReader( - type, descriptor, batchSize, useDecimal128, useLazyMaterialization, true); + type, descriptor, importer, batchSize, useDecimal128, useLazyMaterialization, true); } public static ColumnReader getColumnReader( DataType type, ColumnDescriptor descriptor, + CometSchemaImporter importer, int batchSize, boolean useDecimal128, boolean useLazyMaterialization, boolean useLegacyDateTimestamp) { if (useLazyMaterialization && supportLazyMaterialization(type)) { return new LazyColumnReader( - type, descriptor, batchSize, useDecimal128, useLegacyDateTimestamp); + type, descriptor, importer, batchSize, useDecimal128, useLegacyDateTimestamp); } else { - return new ColumnReader(type, descriptor, batchSize, useDecimal128, useLegacyDateTimestamp); + return new ColumnReader( + type, descriptor, importer, batchSize, useDecimal128, useLegacyDateTimestamp); } } From 9b3e87b5e8b616ac6663411090513f29b98330e7 Mon Sep 17 00:00:00 2001 From: KAZUYUKI TANIMURA <ktanimura@apple.com> Date: Tue, 28 May 2024 12:09:03 -0700 Subject: [PATCH 06/19] build: Add spark-4.0 profile and shims (#407) This PR adds the spark-4.0 profile and shims This is an initial commit. Tests with the spark-4.0 profile do not pass yet. Tests for spark-3.x should pass. --- .github/workflows/pr_build.yml | 113 ++++++++++++++++++ .../comet/parquet/CometParquetUtils.scala | 61 +--------- .../comet/shims/ShimCometParquetUtils.scala | 81 +++++++++++++ .../apache/comet/shims/ShimBatchReader.scala | 37 ++++++ .../apache/comet/shims/ShimFileFormat.scala | 31 +++++ .../shims/ShimResolveDefaultColumns.scala | 29 +++++ .../comet/shims/ShimCometParquetUtils.scala | 38 ++++++ dev/ensure-jars-have-correct-contents.sh | 2 + pom.xml | 27 ++++- spark/pom.xml | 2 + .../parquet/CometParquetFileFormat.scala | 9 +- .../CometParquetPartitionReaderFactory.scala | 3 +- .../apache/comet/parquet/ParquetFilters.scala | 9 +- .../apache/comet/serde/QueryPlanSerde.scala | 23 ++-- .../main/scala/org/apache/spark/Plugins.scala | 24 +--- .../comet/CometBroadcastExchangeExec.scala | 2 +- .../spark/sql/comet/CometScanExec.scala | 9 +- .../spark/sql/comet/DecimalPrecision.scala | 9 +- .../shuffle/CometShuffleExchangeExec.scala | 36 +++--- .../apache/comet/shims/CometExprShim.scala | 4 +- .../apache/comet/shims/CometExprShim.scala | 4 +- .../apache/comet/shims/CometExprShim.scala | 4 +- .../shims/ShimCometShuffleExchangeExec.scala | 10 ++ .../ShimCometSparkSessionExtensions.scala | 4 - .../org/apache/comet/shims/ShimSQLConf.scala | 4 + .../ShimCometBroadcastExchangeExec.scala | 2 +- .../comet/shims/ShimCometDriverPlugin.scala | 38 ++++++ .../sql}/comet/shims/ShimCometScanExec.scala | 29 +++-- .../ShimCometShuffleWriteProcessor.scala | 44 +++++++ .../apache/comet/shims/CometExprShim.scala | 33 +++++ .../comet/shims/ShimCometBatchScanExec.scala | 34 ++++++ .../ShimCometBroadcastHashJoinExec.scala | 31 +++++ .../shims/ShimCometShuffleExchangeExec.scala | 43 +++++++ .../ShimCometSparkSessionExtensions.scala | 37 ++++++ .../ShimCometTakeOrderedAndProjectExec.scala | 26 ++++ .../comet/shims/ShimQueryPlanSerde.scala | 37 ++++++ .../org/apache/comet/shims/ShimSQLConf.scala | 31 +++++ .../ShimCometBroadcastExchangeExec.scala | 30 +++++ .../comet/shims/ShimCometDriverPlugin.scala | 32 +++++ .../sql/comet/shims/ShimCometScanExec.scala | 83 +++++++++++++ .../ShimCometShuffleWriteProcessor.scala | 26 ++++ .../spark/sql/CometTPCDSQuerySuite.scala | 7 +- .../spark/sql/CometTPCHQuerySuite.scala | 5 +- .../org/apache/spark/sql/CometTestBase.scala | 1 - .../sql/benchmark/CometReadBenchmark.scala | 5 +- .../comet/exec/CometExec3_4PlusSuite.scala} | 4 +- .../comet/shims/ShimCometTPCHQuerySuite.scala | 28 +++++ .../spark/comet/shims/ShimTestUtils.scala} | 5 +- .../shims/ShimCometTPCDSQuerySuite.scala | 25 ++++ .../comet/shims/ShimCometTPCHQuerySuite.scala | 25 ++++ .../spark/comet/shims/ShimTestUtils.scala | 27 +++++ .../shims/ShimCometTPCDSQuerySuite.scala | 24 ++++ 52 files changed, 1125 insertions(+), 162 deletions(-) create mode 100644 common/src/main/spark-3.x/org/apache/spark/sql/comet/shims/ShimCometParquetUtils.scala create mode 100644 common/src/main/spark-4.0/org/apache/comet/shims/ShimBatchReader.scala create mode 100644 common/src/main/spark-4.0/org/apache/comet/shims/ShimFileFormat.scala create mode 100644 common/src/main/spark-4.0/org/apache/comet/shims/ShimResolveDefaultColumns.scala create mode 100644 common/src/main/spark-4.0/org/apache/spark/sql/comet/shims/ShimCometParquetUtils.scala rename spark/src/main/spark-3.x/org/apache/{ => spark}/comet/shims/ShimCometBroadcastExchangeExec.scala (98%) create mode 100644 spark/src/main/spark-3.x/org/apache/spark/comet/shims/ShimCometDriverPlugin.scala rename spark/src/main/spark-3.x/org/apache/{ => spark/sql}/comet/shims/ShimCometScanExec.scala (81%) create mode 100644 spark/src/main/spark-3.x/org/apache/spark/sql/comet/shims/ShimCometShuffleWriteProcessor.scala create mode 100644 spark/src/main/spark-4.0/org/apache/comet/shims/CometExprShim.scala create mode 100644 spark/src/main/spark-4.0/org/apache/comet/shims/ShimCometBatchScanExec.scala create mode 100644 spark/src/main/spark-4.0/org/apache/comet/shims/ShimCometBroadcastHashJoinExec.scala create mode 100644 spark/src/main/spark-4.0/org/apache/comet/shims/ShimCometShuffleExchangeExec.scala create mode 100644 spark/src/main/spark-4.0/org/apache/comet/shims/ShimCometSparkSessionExtensions.scala create mode 100644 spark/src/main/spark-4.0/org/apache/comet/shims/ShimCometTakeOrderedAndProjectExec.scala create mode 100644 spark/src/main/spark-4.0/org/apache/comet/shims/ShimQueryPlanSerde.scala create mode 100644 spark/src/main/spark-4.0/org/apache/comet/shims/ShimSQLConf.scala create mode 100644 spark/src/main/spark-4.0/org/apache/spark/comet/shims/ShimCometBroadcastExchangeExec.scala create mode 100644 spark/src/main/spark-4.0/org/apache/spark/comet/shims/ShimCometDriverPlugin.scala create mode 100644 spark/src/main/spark-4.0/org/apache/spark/sql/comet/shims/ShimCometScanExec.scala create mode 100644 spark/src/main/spark-4.0/org/apache/spark/sql/comet/shims/ShimCometShuffleWriteProcessor.scala rename spark/src/test/{spark-3.4/org/apache/comet/exec/CometExec3_4Suite.scala => spark-3.4-plus/org/apache/comet/exec/CometExec3_4PlusSuite.scala} (98%) create mode 100644 spark/src/test/spark-3.x/org/apache/comet/shims/ShimCometTPCHQuerySuite.scala rename spark/src/test/{scala/org/apache/comet/TestUtils.scala => spark-3.x/org/apache/spark/comet/shims/ShimTestUtils.scala} (96%) create mode 100644 spark/src/test/spark-3.x/org/apache/spark/sql/comet/shims/ShimCometTPCDSQuerySuite.scala create mode 100644 spark/src/test/spark-4.0/org/apache/comet/shims/ShimCometTPCHQuerySuite.scala create mode 100644 spark/src/test/spark-4.0/org/apache/spark/comet/shims/ShimTestUtils.scala create mode 100644 spark/src/test/spark-4.0/org/apache/spark/sql/comet/shims/ShimCometTPCDSQuerySuite.scala diff --git a/.github/workflows/pr_build.yml b/.github/workflows/pr_build.yml index 1e347250f..410f1e1fe 100644 --- a/.github/workflows/pr_build.yml +++ b/.github/workflows/pr_build.yml @@ -76,6 +76,44 @@ jobs: # upload test reports only for java 17 upload-test-reports: ${{ matrix.java_version == '17' }} + linux-test-with-spark4_0: + strategy: + matrix: + os: [ubuntu-latest] + java_version: [17] + test-target: [java] + spark-version: ['4.0'] + is_push_event: + - ${{ github.event_name == 'push' }} + fail-fast: false + name: ${{ matrix.os }}/java ${{ matrix.java_version }}-spark-${{matrix.spark-version}}/${{ matrix.test-target }} + runs-on: ${{ matrix.os }} + container: + image: amd64/rust + steps: + - uses: actions/checkout@v4 + - name: Setup Rust & Java toolchain + uses: ./.github/actions/setup-builder + with: + rust-version: ${{env.RUST_VERSION}} + jdk-version: ${{ matrix.java_version }} + - name: Clone Spark + uses: actions/checkout@v4 + with: + repository: "apache/spark" + path: "apache-spark" + - name: Install Spark + shell: bash + working-directory: ./apache-spark + run: build/mvn install -Phive -Phadoop-cloud -DskipTests + - name: Java test steps + uses: ./.github/actions/java-test + with: + # TODO: remove -DskipTests after fixing tests + maven_opts: "-Pspark-${{ matrix.spark-version }} -DskipTests" + # TODO: upload test reports after enabling tests + upload-test-reports: false + linux-test-with-old-spark: strategy: matrix: @@ -169,6 +207,81 @@ jobs: with: maven_opts: -Pspark-${{ matrix.spark-version }},scala-${{ matrix.scala-version }} + macos-test-with-spark4_0: + strategy: + matrix: + os: [macos-13] + java_version: [17] + test-target: [java] + spark-version: ['4.0'] + fail-fast: false + if: github.event_name == 'push' + name: ${{ matrix.os }}/java ${{ matrix.java_version }}-spark-${{matrix.spark-version}}/${{ matrix.test-target }} + runs-on: ${{ matrix.os }} + steps: + - uses: actions/checkout@v4 + - name: Setup Rust & Java toolchain + uses: ./.github/actions/setup-macos-builder + with: + rust-version: ${{env.RUST_VERSION}} + jdk-version: ${{ matrix.java_version }} + - name: Clone Spark + uses: actions/checkout@v4 + with: + repository: "apache/spark" + path: "apache-spark" + - name: Install Spark + shell: bash + working-directory: ./apache-spark + run: build/mvn install -Phive -Phadoop-cloud -DskipTests + - name: Java test steps + uses: ./.github/actions/java-test + with: + # TODO: remove -DskipTests after fixing tests + maven_opts: "-Pspark-${{ matrix.spark-version }} -DskipTests" + # TODO: upload test reports after enabling tests + upload-test-reports: false + + macos-aarch64-test-with-spark4_0: + strategy: + matrix: + java_version: [17] + test-target: [java] + spark-version: ['4.0'] + is_push_event: + - ${{ github.event_name == 'push' }} + exclude: # exclude java 11 for pull_request event + - java_version: 11 + is_push_event: false + fail-fast: false + name: macos-14(Silicon)/java ${{ matrix.java_version }}-spark-${{matrix.spark-version}}/${{ matrix.test-target }} + runs-on: macos-14 + steps: + - uses: actions/checkout@v4 + - name: Setup Rust & Java toolchain + uses: ./.github/actions/setup-macos-builder + with: + rust-version: ${{env.RUST_VERSION}} + jdk-version: ${{ matrix.java_version }} + jdk-architecture: aarch64 + protoc-architecture: aarch_64 + - name: Clone Spark + uses: actions/checkout@v4 + with: + repository: "apache/spark" + path: "apache-spark" + - name: Install Spark + shell: bash + working-directory: ./apache-spark + run: build/mvn install -Phive -Phadoop-cloud -DskipTests + - name: Java test steps + uses: ./.github/actions/java-test + with: + # TODO: remove -DskipTests after fixing tests + maven_opts: "-Pspark-${{ matrix.spark-version }} -DskipTests" + # TODO: upload test reports after enabling tests + upload-test-reports: false + macos-aarch64-test-with-old-spark: strategy: matrix: diff --git a/common/src/main/scala/org/apache/comet/parquet/CometParquetUtils.scala b/common/src/main/scala/org/apache/comet/parquet/CometParquetUtils.scala index d851067b5..d03252d06 100644 --- a/common/src/main/scala/org/apache/comet/parquet/CometParquetUtils.scala +++ b/common/src/main/scala/org/apache/comet/parquet/CometParquetUtils.scala @@ -20,10 +20,10 @@ package org.apache.comet.parquet import org.apache.hadoop.conf.Configuration +import org.apache.spark.sql.comet.shims.ShimCometParquetUtils import org.apache.spark.sql.internal.SQLConf -import org.apache.spark.sql.types._ -object CometParquetUtils { +object CometParquetUtils extends ShimCometParquetUtils { private val PARQUET_FIELD_ID_WRITE_ENABLED = "spark.sql.parquet.fieldId.write.enabled" private val PARQUET_FIELD_ID_READ_ENABLED = "spark.sql.parquet.fieldId.read.enabled" private val IGNORE_MISSING_PARQUET_FIELD_ID = "spark.sql.parquet.fieldId.read.ignoreMissing" @@ -39,61 +39,4 @@ object CometParquetUtils { def ignoreMissingIds(conf: SQLConf): Boolean = conf.getConfString(IGNORE_MISSING_PARQUET_FIELD_ID, "false").toBoolean - - // The following is copied from QueryExecutionErrors - // TODO: remove after dropping Spark 3.2.0 support and directly use - // QueryExecutionErrors.foundDuplicateFieldInFieldIdLookupModeError - def foundDuplicateFieldInFieldIdLookupModeError( - requiredId: Int, - matchedFields: String): Throwable = { - new RuntimeException(s""" - |Found duplicate field(s) "$requiredId": $matchedFields - |in id mapping mode - """.stripMargin.replaceAll("\n", " ")) - } - - // The followings are copied from org.apache.spark.sql.execution.datasources.parquet.ParquetUtils - // TODO: remove after dropping Spark 3.2.0 support and directly use ParquetUtils - /** - * A StructField metadata key used to set the field id of a column in the Parquet schema. - */ - val FIELD_ID_METADATA_KEY = "parquet.field.id" - - /** - * Whether there exists a field in the schema, whether inner or leaf, has the parquet field ID - * metadata. - */ - def hasFieldIds(schema: StructType): Boolean = { - def recursiveCheck(schema: DataType): Boolean = { - schema match { - case st: StructType => - st.exists(field => hasFieldId(field) || recursiveCheck(field.dataType)) - - case at: ArrayType => recursiveCheck(at.elementType) - - case mt: MapType => recursiveCheck(mt.keyType) || recursiveCheck(mt.valueType) - - case _ => - // No need to really check primitive types, just to terminate the recursion - false - } - } - if (schema.isEmpty) false else recursiveCheck(schema) - } - - def hasFieldId(field: StructField): Boolean = - field.metadata.contains(FIELD_ID_METADATA_KEY) - - def getFieldId(field: StructField): Int = { - require( - hasFieldId(field), - s"The key `$FIELD_ID_METADATA_KEY` doesn't exist in the metadata of " + field) - try { - Math.toIntExact(field.metadata.getLong(FIELD_ID_METADATA_KEY)) - } catch { - case _: ArithmeticException | _: ClassCastException => - throw new IllegalArgumentException( - s"The key `$FIELD_ID_METADATA_KEY` must be a 32-bit integer") - } - } } diff --git a/common/src/main/spark-3.x/org/apache/spark/sql/comet/shims/ShimCometParquetUtils.scala b/common/src/main/spark-3.x/org/apache/spark/sql/comet/shims/ShimCometParquetUtils.scala new file mode 100644 index 000000000..f22ac4060 --- /dev/null +++ b/common/src/main/spark-3.x/org/apache/spark/sql/comet/shims/ShimCometParquetUtils.scala @@ -0,0 +1,81 @@ +/* + * 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.shims + +import org.apache.spark.sql.types._ + +trait ShimCometParquetUtils { + // The following is copied from QueryExecutionErrors + // TODO: remove after dropping Spark 3.2.0 support and directly use + // QueryExecutionErrors.foundDuplicateFieldInFieldIdLookupModeError + def foundDuplicateFieldInFieldIdLookupModeError( + requiredId: Int, + matchedFields: String): Throwable = { + new RuntimeException(s""" + |Found duplicate field(s) "$requiredId": $matchedFields + |in id mapping mode + """.stripMargin.replaceAll("\n", " ")) + } + + // The followings are copied from org.apache.spark.sql.execution.datasources.parquet.ParquetUtils + // TODO: remove after dropping Spark 3.2.0 support and directly use ParquetUtils + /** + * A StructField metadata key used to set the field id of a column in the Parquet schema. + */ + val FIELD_ID_METADATA_KEY = "parquet.field.id" + + /** + * Whether there exists a field in the schema, whether inner or leaf, has the parquet field ID + * metadata. + */ + def hasFieldIds(schema: StructType): Boolean = { + def recursiveCheck(schema: DataType): Boolean = { + schema match { + case st: StructType => + st.exists(field => hasFieldId(field) || recursiveCheck(field.dataType)) + + case at: ArrayType => recursiveCheck(at.elementType) + + case mt: MapType => recursiveCheck(mt.keyType) || recursiveCheck(mt.valueType) + + case _ => + // No need to really check primitive types, just to terminate the recursion + false + } + } + if (schema.isEmpty) false else recursiveCheck(schema) + } + + def hasFieldId(field: StructField): Boolean = + field.metadata.contains(FIELD_ID_METADATA_KEY) + + def getFieldId(field: StructField): Int = { + require( + hasFieldId(field), + s"The key `$FIELD_ID_METADATA_KEY` doesn't exist in the metadata of " + field) + try { + Math.toIntExact(field.metadata.getLong(FIELD_ID_METADATA_KEY)) + } catch { + case _: ArithmeticException | _: ClassCastException => + throw new IllegalArgumentException( + s"The key `$FIELD_ID_METADATA_KEY` must be a 32-bit integer") + } + } +} diff --git a/common/src/main/spark-4.0/org/apache/comet/shims/ShimBatchReader.scala b/common/src/main/spark-4.0/org/apache/comet/shims/ShimBatchReader.scala new file mode 100644 index 000000000..448d0886c --- /dev/null +++ b/common/src/main/spark-4.0/org/apache/comet/shims/ShimBatchReader.scala @@ -0,0 +1,37 @@ +/* + * 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.shims + +import org.apache.spark.paths.SparkPath +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.execution.datasources.PartitionedFile + +object ShimBatchReader { + def newPartitionedFile(partitionValues: InternalRow, file: String): PartitionedFile = + PartitionedFile( + partitionValues, + SparkPath.fromUrlString(file), + -1, // -1 means we read the entire file + -1, + Array.empty[String], + 0, + 0 + ) +} diff --git a/common/src/main/spark-4.0/org/apache/comet/shims/ShimFileFormat.scala b/common/src/main/spark-4.0/org/apache/comet/shims/ShimFileFormat.scala new file mode 100644 index 000000000..2f386869a --- /dev/null +++ b/common/src/main/spark-4.0/org/apache/comet/shims/ShimFileFormat.scala @@ -0,0 +1,31 @@ +/* + * 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.shims + +import org.apache.spark.sql.execution.datasources.FileFormat +import org.apache.spark.sql.execution.datasources.parquet.ParquetFileFormat + +object ShimFileFormat { + // A name for a temporary column that holds row indexes computed by the file format reader + // until they can be placed in the _metadata struct. + val ROW_INDEX_TEMPORARY_COLUMN_NAME = ParquetFileFormat.ROW_INDEX_TEMPORARY_COLUMN_NAME + + val OPTION_RETURNING_BATCH = FileFormat.OPTION_RETURNING_BATCH +} diff --git a/common/src/main/spark-4.0/org/apache/comet/shims/ShimResolveDefaultColumns.scala b/common/src/main/spark-4.0/org/apache/comet/shims/ShimResolveDefaultColumns.scala new file mode 100644 index 000000000..60e21765b --- /dev/null +++ b/common/src/main/spark-4.0/org/apache/comet/shims/ShimResolveDefaultColumns.scala @@ -0,0 +1,29 @@ +/* + * 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.shims + + +import org.apache.spark.sql.catalyst.util.ResolveDefaultColumns +import org.apache.spark.sql.types.{StructField, StructType} + +object ShimResolveDefaultColumns { + def getExistenceDefaultValue(field: StructField): Any = + ResolveDefaultColumns.getExistenceDefaultValues(StructType(Seq(field))).head +} diff --git a/common/src/main/spark-4.0/org/apache/spark/sql/comet/shims/ShimCometParquetUtils.scala b/common/src/main/spark-4.0/org/apache/spark/sql/comet/shims/ShimCometParquetUtils.scala new file mode 100644 index 000000000..d402cd786 --- /dev/null +++ b/common/src/main/spark-4.0/org/apache/spark/sql/comet/shims/ShimCometParquetUtils.scala @@ -0,0 +1,38 @@ +/* + * 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.shims + +import org.apache.spark.sql.errors.QueryExecutionErrors +import org.apache.spark.sql.execution.datasources.parquet.ParquetUtils +import org.apache.spark.sql.types._ + +trait ShimCometParquetUtils { + def foundDuplicateFieldInFieldIdLookupModeError( + requiredId: Int, + matchedFields: String): Throwable = { + QueryExecutionErrors.foundDuplicateFieldInFieldIdLookupModeError(requiredId, matchedFields) + } + + def hasFieldIds(schema: StructType): Boolean = ParquetUtils.hasFieldIds(schema) + + def hasFieldId(field: StructField): Boolean = ParquetUtils.hasFieldId(field) + + def getFieldId(field: StructField): Int = ParquetUtils.getFieldId (field) +} diff --git a/dev/ensure-jars-have-correct-contents.sh b/dev/ensure-jars-have-correct-contents.sh index 1f97d2d4a..12f555b8e 100755 --- a/dev/ensure-jars-have-correct-contents.sh +++ b/dev/ensure-jars-have-correct-contents.sh @@ -40,9 +40,11 @@ allowed_expr="(^org/$|^org/apache/$" # we have to allow the directories that lead to the org/apache/comet dir # We allow all the classes under the following packages: # * org.apache.comet +# * org.apache.spark.comet # * org.apache.spark.sql.comet # * org.apache.arrow.c allowed_expr+="|^org/apache/comet/" +allowed_expr+="|^org/apache/spark/comet/" allowed_expr+="|^org/apache/spark/sql/comet/" allowed_expr+="|^org/apache/arrow/c/" # * whatever in the "META-INF" directory diff --git a/pom.xml b/pom.xml index 59e0569ff..57b4206cf 100644 --- a/pom.xml +++ b/pom.xml @@ -87,7 +87,7 @@ under the License. </extraJavaTestArgs> <argLine>-ea -Xmx4g -Xss4m ${extraJavaTestArgs}</argLine> <additional.3_3.test.source>spark-3.3-plus</additional.3_3.test.source> - <additional.3_4.test.source>spark-3.4</additional.3_4.test.source> + <additional.3_4.test.source>spark-3.4-plus</additional.3_4.test.source> <shims.majorVerSrc>spark-3.x</shims.majorVerSrc> <shims.minorVerSrc>spark-3.4</shims.minorVerSrc> </properties> @@ -512,7 +512,6 @@ under the License. <spark.version>3.3.2</spark.version> <spark.version.short>3.3</spark.version.short> <parquet.version>1.12.0</parquet.version> - <additional.3_3.test.source>spark-3.3-plus</additional.3_3.test.source> <additional.3_4.test.source>not-needed-yet</additional.3_4.test.source> <shims.minorVerSrc>spark-3.3</shims.minorVerSrc> </properties> @@ -524,9 +523,25 @@ under the License. <scala.version>2.12.17</scala.version> <spark.version.short>3.4</spark.version.short> <parquet.version>1.13.1</parquet.version> - <additional.3_3.test.source>spark-3.3-plus</additional.3_3.test.source> - <additional.3_4.test.source>spark-3.4</additional.3_4.test.source> - <shims.minorVerSrc>spark-3.4</shims.minorVerSrc> + </properties> + </profile> + + <profile> + <!-- FIXME: this is WIP. Tests may fail --> + <id>spark-4.0</id> + <properties> + <!-- Use Scala 2.13 by default --> + <scala.version>2.13.13</scala.version> + <scala.binary.version>2.13</scala.binary.version> + <spark.version>4.0.0-SNAPSHOT</spark.version> + <spark.version.short>4.0</spark.version.short> + <parquet.version>1.13.1</parquet.version> + <shims.majorVerSrc>spark-4.0</shims.majorVerSrc> + <shims.minorVerSrc>not-needed-yet</shims.minorVerSrc> + <!-- Use jdk17 by default --> + <java.version>17</java.version> + <maven.compiler.source>${java.version}</maven.compiler.source> + <maven.compiler.target>${java.version}</maven.compiler.target> </properties> </profile> @@ -605,7 +620,7 @@ under the License. <compilerPlugin> <groupId>org.scalameta</groupId> <artifactId>semanticdb-scalac_${scala.version}</artifactId> - <version>4.7.5</version> + <version>4.8.8</version> </compilerPlugin> </compilerPlugins> </configuration> diff --git a/spark/pom.xml b/spark/pom.xml index 21fa09fc2..84e2e501f 100644 --- a/spark/pom.xml +++ b/spark/pom.xml @@ -252,6 +252,8 @@ under the License. <sources> <source>src/test/${additional.3_3.test.source}</source> <source>src/test/${additional.3_4.test.source}</source> + <source>src/test/${shims.majorVerSrc}</source> + <source>src/test/${shims.minorVerSrc}</source> </sources> </configuration> </execution> diff --git a/spark/src/main/scala/org/apache/comet/parquet/CometParquetFileFormat.scala b/spark/src/main/scala/org/apache/comet/parquet/CometParquetFileFormat.scala index ac871cf60..52d8d09a0 100644 --- a/spark/src/main/scala/org/apache/comet/parquet/CometParquetFileFormat.scala +++ b/spark/src/main/scala/org/apache/comet/parquet/CometParquetFileFormat.scala @@ -37,7 +37,6 @@ import org.apache.spark.sql.execution.datasources.parquet.ParquetFileFormat import org.apache.spark.sql.execution.datasources.parquet.ParquetOptions import org.apache.spark.sql.execution.datasources.parquet.ParquetReadSupport import org.apache.spark.sql.internal.SQLConf -import org.apache.spark.sql.internal.SQLConf.LegacyBehaviorPolicy import org.apache.spark.sql.sources.Filter import org.apache.spark.sql.types.{DateType, StructType, TimestampType} import org.apache.spark.util.SerializableConfiguration @@ -144,7 +143,7 @@ class CometParquetFileFormat extends ParquetFileFormat with MetricsSupport with isCaseSensitive, useFieldId, ignoreMissingIds, - datetimeRebaseSpec.mode == LegacyBehaviorPolicy.CORRECTED, + datetimeRebaseSpec.mode == CORRECTED, partitionSchema, file.partitionValues, JavaConverters.mapAsJavaMap(metrics)) @@ -161,7 +160,7 @@ class CometParquetFileFormat extends ParquetFileFormat with MetricsSupport with } } -object CometParquetFileFormat extends Logging { +object CometParquetFileFormat extends Logging with ShimSQLConf { /** * Populates Parquet related configurations from the input `sqlConf` to the `hadoopConf` @@ -210,7 +209,7 @@ object CometParquetFileFormat extends Logging { case _ => false }) - if (hasDateOrTimestamp && datetimeRebaseSpec.mode == LegacyBehaviorPolicy.LEGACY) { + if (hasDateOrTimestamp && datetimeRebaseSpec.mode == LEGACY) { if (exceptionOnRebase) { logWarning( s"""Found Parquet file $file that could potentially contain dates/timestamps that were @@ -222,7 +221,7 @@ object CometParquetFileFormat extends Logging { calendar, please disable Comet for this query.""") } else { // do not throw exception on rebase - read as it is - datetimeRebaseSpec = datetimeRebaseSpec.copy(LegacyBehaviorPolicy.CORRECTED) + datetimeRebaseSpec = datetimeRebaseSpec.copy(CORRECTED) } } diff --git a/spark/src/main/scala/org/apache/comet/parquet/CometParquetPartitionReaderFactory.scala b/spark/src/main/scala/org/apache/comet/parquet/CometParquetPartitionReaderFactory.scala index 693af125b..e48d76384 100644 --- a/spark/src/main/scala/org/apache/comet/parquet/CometParquetPartitionReaderFactory.scala +++ b/spark/src/main/scala/org/apache/comet/parquet/CometParquetPartitionReaderFactory.scala @@ -37,7 +37,6 @@ import org.apache.spark.sql.execution.datasources.parquet.ParquetOptions import org.apache.spark.sql.execution.datasources.v2.FilePartitionReaderFactory import org.apache.spark.sql.execution.metric.SQLMetric import org.apache.spark.sql.internal.SQLConf -import org.apache.spark.sql.internal.SQLConf.LegacyBehaviorPolicy import org.apache.spark.sql.sources.Filter import org.apache.spark.sql.types.StructType import org.apache.spark.sql.vectorized.ColumnarBatch @@ -135,7 +134,7 @@ case class CometParquetPartitionReaderFactory( isCaseSensitive, useFieldId, ignoreMissingIds, - datetimeRebaseSpec.mode == LegacyBehaviorPolicy.CORRECTED, + datetimeRebaseSpec.mode == CORRECTED, partitionSchema, file.partitionValues, JavaConverters.mapAsJavaMap(metrics)) diff --git a/spark/src/main/scala/org/apache/comet/parquet/ParquetFilters.scala b/spark/src/main/scala/org/apache/comet/parquet/ParquetFilters.scala index 5994dfb41..58c2aeb41 100644 --- a/spark/src/main/scala/org/apache/comet/parquet/ParquetFilters.scala +++ b/spark/src/main/scala/org/apache/comet/parquet/ParquetFilters.scala @@ -38,11 +38,11 @@ import org.apache.parquet.schema.PrimitiveType.PrimitiveTypeName._ import org.apache.parquet.schema.Type.Repetition import org.apache.spark.sql.catalyst.util.{quoteIfNeeded, CaseInsensitiveMap, DateTimeUtils, IntervalUtils} import org.apache.spark.sql.catalyst.util.RebaseDateTime.{rebaseGregorianToJulianDays, rebaseGregorianToJulianMicros, RebaseSpec} -import org.apache.spark.sql.internal.SQLConf.LegacyBehaviorPolicy import org.apache.spark.sql.sources import org.apache.spark.unsafe.types.UTF8String import org.apache.comet.CometSparkSessionExtensions.isSpark34Plus +import org.apache.comet.shims.ShimSQLConf /** * Copied from Spark 3.2 & 3.4, in order to fix Parquet shading issue. TODO: find a way to remove @@ -58,7 +58,8 @@ class ParquetFilters( pushDownStringPredicate: Boolean, pushDownInFilterThreshold: Int, caseSensitive: Boolean, - datetimeRebaseSpec: RebaseSpec) { + datetimeRebaseSpec: RebaseSpec) + extends ShimSQLConf { // A map which contains parquet field name and data type, if predicate push down applies. // // Each key in `nameToParquetField` represents a column; `dots` are used as separators for @@ -153,7 +154,7 @@ class ParquetFilters( case ld: LocalDate => DateTimeUtils.localDateToDays(ld) } datetimeRebaseSpec.mode match { - case LegacyBehaviorPolicy.LEGACY => rebaseGregorianToJulianDays(gregorianDays) + case LEGACY => rebaseGregorianToJulianDays(gregorianDays) case _ => gregorianDays } } @@ -164,7 +165,7 @@ class ParquetFilters( case t: Timestamp => DateTimeUtils.fromJavaTimestamp(t) } datetimeRebaseSpec.mode match { - case LegacyBehaviorPolicy.LEGACY => + case LEGACY => rebaseGregorianToJulianMicros(datetimeRebaseSpec.timeZone, gregorianMicros) case _ => gregorianMicros } diff --git a/spark/src/main/scala/org/apache/comet/serde/QueryPlanSerde.scala b/spark/src/main/scala/org/apache/comet/serde/QueryPlanSerde.scala index 6333650dd..a717e066e 100644 --- a/spark/src/main/scala/org/apache/comet/serde/QueryPlanSerde.scala +++ b/spark/src/main/scala/org/apache/comet/serde/QueryPlanSerde.scala @@ -1987,18 +1987,17 @@ object QueryPlanSerde extends Logging with ShimQueryPlanSerde with CometExprShim // With Spark 3.4, CharVarcharCodegenUtils.readSidePadding gets called to pad spaces for // char types. Use rpad to achieve the behavior. // See https://github.com/apache/spark/pull/38151 - case StaticInvoke( - _: Class[CharVarcharCodegenUtils], - _: StringType, - "readSidePadding", - arguments, - _, - true, - false, - true) if arguments.size == 2 => + case s: StaticInvoke + if s.staticObject.isInstanceOf[Class[CharVarcharCodegenUtils]] && + s.dataType.isInstanceOf[StringType] && + s.functionName == "readSidePadding" && + s.arguments.size == 2 && + s.propagateNull && + !s.returnNullable && + s.isDeterministic => val argsExpr = Seq( - exprToProtoInternal(Cast(arguments(0), StringType), inputs), - exprToProtoInternal(arguments(1), inputs)) + exprToProtoInternal(Cast(s.arguments(0), StringType), inputs), + exprToProtoInternal(s.arguments(1), inputs)) if (argsExpr.forall(_.isDefined)) { val builder = ExprOuterClass.ScalarFunc.newBuilder() @@ -2007,7 +2006,7 @@ object QueryPlanSerde extends Logging with ShimQueryPlanSerde with CometExprShim Some(ExprOuterClass.Expr.newBuilder().setScalarFunc(builder).build()) } else { - withInfo(expr, arguments: _*) + withInfo(expr, s.arguments: _*) None } diff --git a/spark/src/main/scala/org/apache/spark/Plugins.scala b/spark/src/main/scala/org/apache/spark/Plugins.scala index 97838448a..dcc00f66c 100644 --- a/spark/src/main/scala/org/apache/spark/Plugins.scala +++ b/spark/src/main/scala/org/apache/spark/Plugins.scala @@ -23,9 +23,9 @@ import java.{util => ju} import java.util.Collections import org.apache.spark.api.plugin.{DriverPlugin, ExecutorPlugin, PluginContext, SparkPlugin} +import org.apache.spark.comet.shims.ShimCometDriverPlugin import org.apache.spark.internal.Logging import org.apache.spark.internal.config.{EXECUTOR_MEMORY, EXECUTOR_MEMORY_OVERHEAD} -import org.apache.spark.resource.ResourceProfile import org.apache.comet.{CometConf, CometSparkSessionExtensions} @@ -40,9 +40,7 @@ import org.apache.comet.{CometConf, CometSparkSessionExtensions} * * To enable this plugin, set the config "spark.plugins" to `org.apache.spark.CometPlugin`. */ -class CometDriverPlugin extends DriverPlugin with Logging { - import CometDriverPlugin._ - +class CometDriverPlugin extends DriverPlugin with Logging with ShimCometDriverPlugin { override def init(sc: SparkContext, pluginContext: PluginContext): ju.Map[String, String] = { logInfo("CometDriverPlugin init") @@ -52,14 +50,10 @@ class CometDriverPlugin extends DriverPlugin with Logging { } else { // By default, executorMemory * spark.executor.memoryOverheadFactor, with minimum of 384MB val executorMemory = sc.getConf.getSizeAsMb(EXECUTOR_MEMORY.key) - val memoryOverheadFactor = - sc.getConf.getDouble( - EXECUTOR_MEMORY_OVERHEAD_FACTOR, - EXECUTOR_MEMORY_OVERHEAD_FACTOR_DEFAULT) - - Math.max( - (executorMemory * memoryOverheadFactor).toInt, - ResourceProfile.MEMORY_OVERHEAD_MIN_MIB) + val memoryOverheadFactor = getMemoryOverheadFactor(sc.getConf) + val memoryOverheadMinMib = getMemoryOverheadMinMib(sc.getConf) + + Math.max((executorMemory * memoryOverheadFactor).toLong, memoryOverheadMinMib) } val cometMemOverhead = CometSparkSessionExtensions.getCometMemoryOverheadInMiB(sc.getConf) @@ -100,12 +94,6 @@ class CometDriverPlugin extends DriverPlugin with Logging { } } -object CometDriverPlugin { - // `org.apache.spark.internal.config.EXECUTOR_MEMORY_OVERHEAD_FACTOR` was added since Spark 3.3.0 - val EXECUTOR_MEMORY_OVERHEAD_FACTOR = "spark.executor.memoryOverheadFactor" - val EXECUTOR_MEMORY_OVERHEAD_FACTOR_DEFAULT = 0.1 -} - /** * The Comet plugin for Spark. To enable this plugin, set the config "spark.plugins" to * `org.apache.spark.CometPlugin` diff --git a/spark/src/main/scala/org/apache/spark/sql/comet/CometBroadcastExchangeExec.scala b/spark/src/main/scala/org/apache/spark/sql/comet/CometBroadcastExchangeExec.scala index 7bd34debb..38247b2c4 100644 --- a/spark/src/main/scala/org/apache/spark/sql/comet/CometBroadcastExchangeExec.scala +++ b/spark/src/main/scala/org/apache/spark/sql/comet/CometBroadcastExchangeExec.scala @@ -27,6 +27,7 @@ import scala.concurrent.duration.NANOSECONDS import scala.util.control.NonFatal import org.apache.spark.{broadcast, Partition, SparkContext, TaskContext} +import org.apache.spark.comet.shims.ShimCometBroadcastExchangeExec import org.apache.spark.launcher.SparkLauncher import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.InternalRow @@ -44,7 +45,6 @@ import org.apache.spark.util.io.ChunkedByteBuffer import com.google.common.base.Objects import org.apache.comet.CometRuntimeException -import org.apache.comet.shims.ShimCometBroadcastExchangeExec /** * A [[CometBroadcastExchangeExec]] collects, transforms and finally broadcasts the result of a diff --git a/spark/src/main/scala/org/apache/spark/sql/comet/CometScanExec.scala b/spark/src/main/scala/org/apache/spark/sql/comet/CometScanExec.scala index 14a664108..9a5b55d65 100644 --- a/spark/src/main/scala/org/apache/spark/sql/comet/CometScanExec.scala +++ b/spark/src/main/scala/org/apache/spark/sql/comet/CometScanExec.scala @@ -32,6 +32,7 @@ import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.plans.QueryPlan import org.apache.spark.sql.catalyst.plans.physical._ import org.apache.spark.sql.catalyst.util.CaseInsensitiveMap +import org.apache.spark.sql.comet.shims.ShimCometScanExec import org.apache.spark.sql.execution._ import org.apache.spark.sql.execution.datasources._ import org.apache.spark.sql.execution.datasources.parquet.ParquetOptions @@ -43,7 +44,7 @@ import org.apache.spark.util.collection._ import org.apache.comet.{CometConf, MetricsSupport} import org.apache.comet.parquet.{CometParquetFileFormat, CometParquetPartitionReaderFactory} -import org.apache.comet.shims.{ShimCometScanExec, ShimFileFormat} +import org.apache.comet.shims.ShimFileFormat /** * Comet physical scan node for DataSource V1. Most of the code here follow Spark's @@ -271,7 +272,7 @@ case class CometScanExec( selectedPartitions .flatMap { p => p.files.map { f => - PartitionedFileUtil.getPartitionedFile(f, f.getPath, p.values) + getPartitionedFile(f, p) } } .groupBy { f => @@ -358,7 +359,7 @@ case class CometScanExec( // SPARK-39634: Allow file splitting in combination with row index generation once // the fix for PARQUET-2161 is available. !isNeededForSchema(requiredSchema) - PartitionedFileUtil.splitFiles( + super.splitFiles( sparkSession = relation.sparkSession, file = file, filePath = filePath, @@ -409,7 +410,7 @@ case class CometScanExec( Map.empty) } else { newFileScanRDD( - fsRelation.sparkSession, + fsRelation, readFile, partitions, new StructType(requiredSchema.fields ++ fsRelation.partitionSchema.fields), diff --git a/spark/src/main/scala/org/apache/spark/sql/comet/DecimalPrecision.scala b/spark/src/main/scala/org/apache/spark/sql/comet/DecimalPrecision.scala index 13f26ce58..a2cdf421c 100644 --- a/spark/src/main/scala/org/apache/spark/sql/comet/DecimalPrecision.scala +++ b/spark/src/main/scala/org/apache/spark/sql/comet/DecimalPrecision.scala @@ -59,7 +59,7 @@ object DecimalPrecision { } CheckOverflow(add, resultType, nullOnOverflow) - case sub @ Subtract(DecimalType.Expression(p1, s1), DecimalType.Expression(p2, s2), _) => + case sub @ Subtract(DecimalExpression(p1, s1), DecimalExpression(p2, s2), _) => val resultScale = max(s1, s2) val resultType = if (allowPrecisionLoss) { DecimalType.adjustPrecisionScale(max(p1 - s1, p2 - s2) + resultScale + 1, resultScale) @@ -68,7 +68,7 @@ object DecimalPrecision { } CheckOverflow(sub, resultType, nullOnOverflow) - case mul @ Multiply(DecimalType.Expression(p1, s1), DecimalType.Expression(p2, s2), _) => + case mul @ Multiply(DecimalExpression(p1, s1), DecimalExpression(p2, s2), _) => val resultType = if (allowPrecisionLoss) { DecimalType.adjustPrecisionScale(p1 + p2 + 1, s1 + s2) } else { @@ -76,7 +76,7 @@ object DecimalPrecision { } CheckOverflow(mul, resultType, nullOnOverflow) - case div @ Divide(DecimalType.Expression(p1, s1), DecimalType.Expression(p2, s2), _) => + case div @ Divide(DecimalExpression(p1, s1), DecimalExpression(p2, s2), _) => val resultType = if (allowPrecisionLoss) { // Precision: p1 - s1 + s2 + max(6, s1 + p2 + 1) // Scale: max(6, s1 + p2 + 1) @@ -96,7 +96,7 @@ object DecimalPrecision { } CheckOverflow(div, resultType, nullOnOverflow) - case rem @ Remainder(DecimalType.Expression(p1, s1), DecimalType.Expression(p2, s2), _) => + case rem @ Remainder(DecimalExpression(p1, s1), DecimalExpression(p2, s2), _) => val resultType = if (allowPrecisionLoss) { DecimalType.adjustPrecisionScale(min(p1 - s1, p2 - s2) + max(s1, s2), max(s1, s2)) } else { @@ -108,6 +108,7 @@ object DecimalPrecision { } } + // TODO: consider to use `org.apache.spark.sql.types.DecimalExpression` for Spark 3.5+ object DecimalExpression { def unapply(e: Expression): Option[(Int, Int)] = e.dataType match { case t: DecimalType => Some((t.precision, t.scale)) diff --git a/spark/src/main/scala/org/apache/spark/sql/comet/execution/shuffle/CometShuffleExchangeExec.scala b/spark/src/main/scala/org/apache/spark/sql/comet/execution/shuffle/CometShuffleExchangeExec.scala index 49c263f3f..3f4d7bfd3 100644 --- a/spark/src/main/scala/org/apache/spark/sql/comet/execution/shuffle/CometShuffleExchangeExec.scala +++ b/spark/src/main/scala/org/apache/spark/sql/comet/execution/shuffle/CometShuffleExchangeExec.scala @@ -28,10 +28,10 @@ import scala.concurrent.Future import org.apache.spark._ import org.apache.spark.internal.config -import org.apache.spark.rdd.{MapPartitionsRDD, RDD} +import org.apache.spark.rdd.RDD import org.apache.spark.scheduler.MapStatus import org.apache.spark.serializer.Serializer -import org.apache.spark.shuffle.{IndexShuffleBlockResolver, ShuffleWriteMetricsReporter, ShuffleWriteProcessor} +import org.apache.spark.shuffle.{IndexShuffleBlockResolver, ShuffleWriteMetricsReporter} import org.apache.spark.shuffle.sort.SortShuffleManager import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.{Attribute, BoundReference, UnsafeProjection, UnsafeRow} @@ -39,12 +39,12 @@ import org.apache.spark.sql.catalyst.expressions.codegen.LazilyGeneratedOrdering import org.apache.spark.sql.catalyst.plans.logical.Statistics import org.apache.spark.sql.catalyst.plans.physical._ import org.apache.spark.sql.comet.{CometExec, CometMetricNode, CometPlan} +import org.apache.spark.sql.comet.shims.ShimCometShuffleWriteProcessor import org.apache.spark.sql.execution._ import org.apache.spark.sql.execution.exchange.{ENSURE_REQUIREMENTS, ShuffleExchangeLike, ShuffleOrigin} import org.apache.spark.sql.execution.exchange.ShuffleExchangeExec import org.apache.spark.sql.execution.metric.{SQLMetric, SQLMetrics, SQLShuffleReadMetricsReporter, SQLShuffleWriteMetricsReporter} import org.apache.spark.sql.internal.SQLConf -import org.apache.spark.sql.types.StructType import org.apache.spark.sql.vectorized.ColumnarBatch import org.apache.spark.util.MutablePair import org.apache.spark.util.collection.unsafe.sort.{PrefixComparators, RecordComparator} @@ -68,7 +68,8 @@ case class CometShuffleExchangeExec( shuffleType: ShuffleType = CometNativeShuffle, advisoryPartitionSize: Option[Long] = None) extends ShuffleExchangeLike - with CometPlan { + with CometPlan + with ShimCometShuffleExchangeExec { private lazy val writeMetrics = SQLShuffleWriteMetricsReporter.createShuffleWriteMetrics(sparkContext) @@ -127,6 +128,9 @@ case class CometShuffleExchangeExec( Statistics(dataSize, Some(rowCount)) } + // TODO: add `override` keyword after dropping Spark-3.x supports + def shuffleId: Int = getShuffleId(shuffleDependency) + /** * A [[ShuffleDependency]] that will partition rows of its child based on the partitioning * scheme defined in `newPartitioning`. Those partitions of the returned ShuffleDependency will @@ -386,7 +390,7 @@ object CometShuffleExchangeExec extends ShimCometShuffleExchangeExec { val pageSize = SparkEnv.get.memoryManager.pageSizeBytes val sorter = UnsafeExternalRowSorter.createWithRecordComparator( - StructType.fromAttributes(outputAttributes), + fromAttributes(outputAttributes), recordComparatorSupplier, prefixComparator, prefixComputer, @@ -430,7 +434,7 @@ object CometShuffleExchangeExec extends ShimCometShuffleExchangeExec { serializer, shuffleWriterProcessor = ShuffleExchangeExec.createShuffleWriteProcessor(writeMetrics), shuffleType = CometColumnarShuffle, - schema = Some(StructType.fromAttributes(outputAttributes))) + schema = Some(fromAttributes(outputAttributes))) dependency } @@ -445,7 +449,7 @@ class CometShuffleWriteProcessor( outputPartitioning: Partitioning, outputAttributes: Seq[Attribute], metrics: Map[String, SQLMetric]) - extends ShuffleWriteProcessor { + extends ShimCometShuffleWriteProcessor { private val OFFSET_LENGTH = 8 @@ -455,11 +459,11 @@ class CometShuffleWriteProcessor( } override def write( - rdd: RDD[_], + inputs: Iterator[_], dep: ShuffleDependency[_, _, _], mapId: Long, - context: TaskContext, - partition: Partition): MapStatus = { + mapIndex: Int, + context: TaskContext): MapStatus = { val shuffleBlockResolver = SparkEnv.get.shuffleManager.shuffleBlockResolver.asInstanceOf[IndexShuffleBlockResolver] val dataFile = shuffleBlockResolver.getDataFile(dep.shuffleId, mapId) @@ -469,10 +473,6 @@ class CometShuffleWriteProcessor( val tempDataFilePath = Paths.get(tempDataFilename) val tempIndexFilePath = Paths.get(tempIndexFilename) - // Getting rid of the fake partitionId - val cometRDD = - rdd.asInstanceOf[MapPartitionsRDD[_, _]].prev.asInstanceOf[RDD[ColumnarBatch]] - // Call native shuffle write val nativePlan = getNativePlan(tempDataFilename, tempIndexFilename) @@ -482,8 +482,12 @@ class CometShuffleWriteProcessor( "elapsed_compute" -> metrics("shuffleReadElapsedCompute")) val nativeMetrics = CometMetricNode(nativeSQLMetrics) - val rawIter = cometRDD.iterator(partition, context) - val cometIter = CometExec.getCometIterator(Seq(rawIter), nativePlan, nativeMetrics) + // Getting rid of the fake partitionId + val newInputs = inputs.asInstanceOf[Iterator[_ <: Product2[Any, Any]]].map(_._2) + val cometIter = CometExec.getCometIterator( + Seq(newInputs.asInstanceOf[Iterator[ColumnarBatch]]), + nativePlan, + nativeMetrics) while (cometIter.hasNext) { cometIter.next() diff --git a/spark/src/main/spark-3.2/org/apache/comet/shims/CometExprShim.scala b/spark/src/main/spark-3.2/org/apache/comet/shims/CometExprShim.scala index 0c45a9c2c..f5a578f82 100644 --- a/spark/src/main/spark-3.2/org/apache/comet/shims/CometExprShim.scala +++ b/spark/src/main/spark-3.2/org/apache/comet/shims/CometExprShim.scala @@ -25,8 +25,8 @@ import org.apache.spark.sql.catalyst.expressions._ */ trait CometExprShim { /** - * Returns a tuple of expressions for the `unhex` function. - */ + * Returns a tuple of expressions for the `unhex` function. + */ def unhexSerde(unhex: Unhex): (Expression, Expression) = { (unhex.child, Literal(false)) } diff --git a/spark/src/main/spark-3.3/org/apache/comet/shims/CometExprShim.scala b/spark/src/main/spark-3.3/org/apache/comet/shims/CometExprShim.scala index 0c45a9c2c..f5a578f82 100644 --- a/spark/src/main/spark-3.3/org/apache/comet/shims/CometExprShim.scala +++ b/spark/src/main/spark-3.3/org/apache/comet/shims/CometExprShim.scala @@ -25,8 +25,8 @@ import org.apache.spark.sql.catalyst.expressions._ */ trait CometExprShim { /** - * Returns a tuple of expressions for the `unhex` function. - */ + * Returns a tuple of expressions for the `unhex` function. + */ def unhexSerde(unhex: Unhex): (Expression, Expression) = { (unhex.child, Literal(false)) } diff --git a/spark/src/main/spark-3.4/org/apache/comet/shims/CometExprShim.scala b/spark/src/main/spark-3.4/org/apache/comet/shims/CometExprShim.scala index 409e1c94b..3f2301f0a 100644 --- a/spark/src/main/spark-3.4/org/apache/comet/shims/CometExprShim.scala +++ b/spark/src/main/spark-3.4/org/apache/comet/shims/CometExprShim.scala @@ -25,8 +25,8 @@ import org.apache.spark.sql.catalyst.expressions._ */ trait CometExprShim { /** - * Returns a tuple of expressions for the `unhex` function. - */ + * Returns a tuple of expressions for the `unhex` function. + */ def unhexSerde(unhex: Unhex): (Expression, Expression) = { (unhex.child, Literal(unhex.failOnError)) } diff --git a/spark/src/main/spark-3.x/org/apache/comet/shims/ShimCometShuffleExchangeExec.scala b/spark/src/main/spark-3.x/org/apache/comet/shims/ShimCometShuffleExchangeExec.scala index 6b4fad974..350aeb9f0 100644 --- a/spark/src/main/spark-3.x/org/apache/comet/shims/ShimCometShuffleExchangeExec.scala +++ b/spark/src/main/spark-3.x/org/apache/comet/shims/ShimCometShuffleExchangeExec.scala @@ -19,8 +19,11 @@ package org.apache.comet.shims +import org.apache.spark.ShuffleDependency +import org.apache.spark.sql.catalyst.expressions.Attribute import org.apache.spark.sql.comet.execution.shuffle.{CometShuffleExchangeExec, ShuffleType} import org.apache.spark.sql.execution.exchange.ShuffleExchangeExec +import org.apache.spark.sql.types.{StructField, StructType} trait ShimCometShuffleExchangeExec { // TODO: remove after dropping Spark 3.2 and 3.3 support @@ -37,4 +40,11 @@ trait ShimCometShuffleExchangeExec { shuffleType, advisoryPartitionSize) } + + // TODO: remove after dropping Spark 3.x support + protected def fromAttributes(attributes: Seq[Attribute]): StructType = + StructType(attributes.map(a => StructField(a.name, a.dataType, a.nullable, a.metadata))) + + // TODO: remove after dropping Spark 3.x support + protected def getShuffleId(shuffleDependency: ShuffleDependency[Int, _, _]): Int = 0 } diff --git a/spark/src/main/spark-3.x/org/apache/comet/shims/ShimCometSparkSessionExtensions.scala b/spark/src/main/spark-3.x/org/apache/comet/shims/ShimCometSparkSessionExtensions.scala index eb04c68ab..377485335 100644 --- a/spark/src/main/spark-3.x/org/apache/comet/shims/ShimCometSparkSessionExtensions.scala +++ b/spark/src/main/spark-3.x/org/apache/comet/shims/ShimCometSparkSessionExtensions.scala @@ -24,8 +24,6 @@ import org.apache.spark.sql.execution.{LimitExec, QueryExecution, SparkPlan} import org.apache.spark.sql.execution.datasources.v2.parquet.ParquetScan trait ShimCometSparkSessionExtensions { - import org.apache.comet.shims.ShimCometSparkSessionExtensions._ - /** * TODO: delete after dropping Spark 3.2.0 support and directly call scan.pushedAggregate */ @@ -45,9 +43,7 @@ trait ShimCometSparkSessionExtensions { * SQLConf.EXTENDED_EXPLAIN_PROVIDERS.key */ protected val EXTENDED_EXPLAIN_PROVIDERS_KEY = "spark.sql.extendedExplainProviders" -} -object ShimCometSparkSessionExtensions { private def getOffsetOpt(plan: SparkPlan): Option[Int] = plan.getClass.getDeclaredFields .filter(_.getName == "offset") .map { a => a.setAccessible(true); a.get(plan) } diff --git a/spark/src/main/spark-3.x/org/apache/comet/shims/ShimSQLConf.scala b/spark/src/main/spark-3.x/org/apache/comet/shims/ShimSQLConf.scala index ff60ef964..c3d0c56e5 100644 --- a/spark/src/main/spark-3.x/org/apache/comet/shims/ShimSQLConf.scala +++ b/spark/src/main/spark-3.x/org/apache/comet/shims/ShimSQLConf.scala @@ -20,6 +20,7 @@ package org.apache.comet.shims import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.internal.SQLConf.LegacyBehaviorPolicy trait ShimSQLConf { @@ -39,4 +40,7 @@ trait ShimSQLConf { case _ => None }) .head + + protected val LEGACY = LegacyBehaviorPolicy.LEGACY + protected val CORRECTED = LegacyBehaviorPolicy.CORRECTED } diff --git a/spark/src/main/spark-3.x/org/apache/comet/shims/ShimCometBroadcastExchangeExec.scala b/spark/src/main/spark-3.x/org/apache/spark/comet/shims/ShimCometBroadcastExchangeExec.scala similarity index 98% rename from spark/src/main/spark-3.x/org/apache/comet/shims/ShimCometBroadcastExchangeExec.scala rename to spark/src/main/spark-3.x/org/apache/spark/comet/shims/ShimCometBroadcastExchangeExec.scala index 63ff2a2c1..aede47951 100644 --- a/spark/src/main/spark-3.x/org/apache/comet/shims/ShimCometBroadcastExchangeExec.scala +++ b/spark/src/main/spark-3.x/org/apache/spark/comet/shims/ShimCometBroadcastExchangeExec.scala @@ -17,7 +17,7 @@ * under the License. */ -package org.apache.comet.shims +package org.apache.spark.comet.shims import scala.reflect.ClassTag diff --git a/spark/src/main/spark-3.x/org/apache/spark/comet/shims/ShimCometDriverPlugin.scala b/spark/src/main/spark-3.x/org/apache/spark/comet/shims/ShimCometDriverPlugin.scala new file mode 100644 index 000000000..cfb6a0088 --- /dev/null +++ b/spark/src/main/spark-3.x/org/apache/spark/comet/shims/ShimCometDriverPlugin.scala @@ -0,0 +1,38 @@ +/* + * 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.comet.shims + +import org.apache.spark.SparkConf + +trait ShimCometDriverPlugin { + // `org.apache.spark.internal.config.EXECUTOR_MEMORY_OVERHEAD_FACTOR` was added since Spark 3.3.0 + private val EXECUTOR_MEMORY_OVERHEAD_FACTOR = "spark.executor.memoryOverheadFactor" + private val EXECUTOR_MEMORY_OVERHEAD_FACTOR_DEFAULT = 0.1 + // `org.apache.spark.internal.config.EXECUTOR_MIN_MEMORY_OVERHEAD` was added since Spark 4.0.0 + private val EXECUTOR_MIN_MEMORY_OVERHEAD = "spark.executor.minMemoryOverhead" + private val EXECUTOR_MIN_MEMORY_OVERHEAD_DEFAULT = 384L + + def getMemoryOverheadFactor(sc: SparkConf): Double = + sc.getDouble( + EXECUTOR_MEMORY_OVERHEAD_FACTOR, + EXECUTOR_MEMORY_OVERHEAD_FACTOR_DEFAULT) + def getMemoryOverheadMinMib(sc: SparkConf): Long = + sc.getLong(EXECUTOR_MIN_MEMORY_OVERHEAD, EXECUTOR_MIN_MEMORY_OVERHEAD_DEFAULT) +} diff --git a/spark/src/main/spark-3.x/org/apache/comet/shims/ShimCometScanExec.scala b/spark/src/main/spark-3.x/org/apache/spark/sql/comet/shims/ShimCometScanExec.scala similarity index 81% rename from spark/src/main/spark-3.x/org/apache/comet/shims/ShimCometScanExec.scala rename to spark/src/main/spark-3.x/org/apache/spark/sql/comet/shims/ShimCometScanExec.scala index 544a67385..02b97f9fb 100644 --- a/spark/src/main/spark-3.x/org/apache/comet/shims/ShimCometScanExec.scala +++ b/spark/src/main/spark-3.x/org/apache/spark/sql/comet/shims/ShimCometScanExec.scala @@ -17,17 +17,21 @@ * under the License. */ -package org.apache.comet.shims +package org.apache.spark.sql.comet.shims + +import org.apache.comet.shims.ShimFileFormat import scala.language.implicitConversions +import org.apache.hadoop.fs.{FileStatus, Path} + import org.apache.spark.{SparkContext, SparkException} import org.apache.spark.sql.SparkSession import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.AttributeReference import org.apache.spark.sql.connector.read.{InputPartition, PartitionReaderFactory} -import org.apache.spark.sql.execution.FileSourceScanExec -import org.apache.spark.sql.execution.datasources.{FilePartition, FileScanRDD, PartitionedFile} +import org.apache.spark.sql.execution.{FileSourceScanExec, PartitionedFileUtil} +import org.apache.spark.sql.execution.datasources.{FilePartition, FileScanRDD, HadoopFsRelation, PartitionDirectory, PartitionedFile} import org.apache.spark.sql.execution.datasources.parquet.ParquetOptions import org.apache.spark.sql.execution.datasources.v2.DataSourceRDD import org.apache.spark.sql.execution.metric.SQLMetric @@ -63,7 +67,7 @@ trait ShimCometScanExec { // TODO: remove after dropping Spark 3.2 support and directly call new FileScanRDD protected def newFileScanRDD( - sparkSession: SparkSession, + fsRelation: HadoopFsRelation, readFunction: PartitionedFile => Iterator[InternalRow], filePartitions: Seq[FilePartition], readSchema: StructType, @@ -73,12 +77,12 @@ trait ShimCometScanExec { .filter(c => List(3, 5, 6).contains(c.getParameterCount()) ) .map { c => c.getParameterCount match { - case 3 => c.newInstance(sparkSession, readFunction, filePartitions) + case 3 => c.newInstance(fsRelation.sparkSession, readFunction, filePartitions) case 5 => - c.newInstance(sparkSession, readFunction, filePartitions, readSchema, metadataColumns) + c.newInstance(fsRelation.sparkSession, readFunction, filePartitions, readSchema, metadataColumns) case 6 => c.newInstance( - sparkSession, + fsRelation.sparkSession, readFunction, filePartitions, readSchema, @@ -123,4 +127,15 @@ trait ShimCometScanExec { protected def isNeededForSchema(sparkSchema: StructType): Boolean = { findRowIndexColumnIndexInSchema(sparkSchema) >= 0 } + + protected def getPartitionedFile(f: FileStatus, p: PartitionDirectory): PartitionedFile = + PartitionedFileUtil.getPartitionedFile(f, f.getPath, p.values) + + protected def splitFiles(sparkSession: SparkSession, + file: FileStatus, + filePath: Path, + isSplitable: Boolean, + maxSplitBytes: Long, + partitionValues: InternalRow): Seq[PartitionedFile] = + PartitionedFileUtil.splitFiles(sparkSession, file, filePath, isSplitable, maxSplitBytes, partitionValues) } diff --git a/spark/src/main/spark-3.x/org/apache/spark/sql/comet/shims/ShimCometShuffleWriteProcessor.scala b/spark/src/main/spark-3.x/org/apache/spark/sql/comet/shims/ShimCometShuffleWriteProcessor.scala new file mode 100644 index 000000000..9100b90c2 --- /dev/null +++ b/spark/src/main/spark-3.x/org/apache/spark/sql/comet/shims/ShimCometShuffleWriteProcessor.scala @@ -0,0 +1,44 @@ +/* + * 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.shims + +import org.apache.spark.{Partition, ShuffleDependency, TaskContext} +import org.apache.spark.rdd.RDD +import org.apache.spark.scheduler.MapStatus +import org.apache.spark.shuffle.ShuffleWriteProcessor + +trait ShimCometShuffleWriteProcessor extends ShuffleWriteProcessor { + override def write( + rdd: RDD[_], + dep: ShuffleDependency[_, _, _], + mapId: Long, + context: TaskContext, + partition: Partition): MapStatus = { + val rawIter = rdd.iterator(partition, context) + write(rawIter, dep, mapId, partition.index, context) + } + + def write( + inputs: Iterator[_], + dep: ShuffleDependency[_, _, _], + mapId: Long, + mapIndex: Int, + context: TaskContext): MapStatus +} diff --git a/spark/src/main/spark-4.0/org/apache/comet/shims/CometExprShim.scala b/spark/src/main/spark-4.0/org/apache/comet/shims/CometExprShim.scala new file mode 100644 index 000000000..01f923206 --- /dev/null +++ b/spark/src/main/spark-4.0/org/apache/comet/shims/CometExprShim.scala @@ -0,0 +1,33 @@ +/* + * 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.shims + +import org.apache.spark.sql.catalyst.expressions._ + +/** + * `CometExprShim` acts as a shim for for parsing expressions from different Spark versions. + */ +trait CometExprShim { + /** + * Returns a tuple of expressions for the `unhex` function. + */ + protected def unhexSerde(unhex: Unhex): (Expression, Expression) = { + (unhex.child, Literal(unhex.failOnError)) + } +} diff --git a/spark/src/main/spark-4.0/org/apache/comet/shims/ShimCometBatchScanExec.scala b/spark/src/main/spark-4.0/org/apache/comet/shims/ShimCometBatchScanExec.scala new file mode 100644 index 000000000..167b539f8 --- /dev/null +++ b/spark/src/main/spark-4.0/org/apache/comet/shims/ShimCometBatchScanExec.scala @@ -0,0 +1,34 @@ +/* + * 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.shims + +import org.apache.spark.sql.catalyst.expressions.{Expression, SortOrder} +import org.apache.spark.sql.connector.read.InputPartition +import org.apache.spark.sql.execution.datasources.v2.BatchScanExec + +trait ShimCometBatchScanExec { + def wrapped: BatchScanExec + + def keyGroupedPartitioning: Option[Seq[Expression]] = wrapped.keyGroupedPartitioning + + def inputPartitions: Seq[InputPartition] = wrapped.inputPartitions + + def ordering: Option[Seq[SortOrder]] = wrapped.ordering +} diff --git a/spark/src/main/spark-4.0/org/apache/comet/shims/ShimCometBroadcastHashJoinExec.scala b/spark/src/main/spark-4.0/org/apache/comet/shims/ShimCometBroadcastHashJoinExec.scala new file mode 100644 index 000000000..1f689b400 --- /dev/null +++ b/spark/src/main/spark-4.0/org/apache/comet/shims/ShimCometBroadcastHashJoinExec.scala @@ -0,0 +1,31 @@ +/* + * 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.shims + +import org.apache.spark.sql.catalyst.expressions.Expression +import org.apache.spark.sql.catalyst.plans.physical.{HashPartitioningLike, Partitioning} + +trait ShimCometBroadcastHashJoinExec { + protected def getHashPartitioningLikeExpressions(partitioning: Partitioning): Seq[Expression] = + partitioning match { + case p: HashPartitioningLike => p.expressions + case _ => Seq() + } +} diff --git a/spark/src/main/spark-4.0/org/apache/comet/shims/ShimCometShuffleExchangeExec.scala b/spark/src/main/spark-4.0/org/apache/comet/shims/ShimCometShuffleExchangeExec.scala new file mode 100644 index 000000000..559e327b4 --- /dev/null +++ b/spark/src/main/spark-4.0/org/apache/comet/shims/ShimCometShuffleExchangeExec.scala @@ -0,0 +1,43 @@ +/* + * 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.shims + +import org.apache.spark.ShuffleDependency +import org.apache.spark.sql.catalyst.expressions.Attribute +import org.apache.spark.sql.catalyst.types.DataTypeUtils +import org.apache.spark.sql.comet.execution.shuffle.{CometShuffleExchangeExec, ShuffleType} +import org.apache.spark.sql.execution.exchange.ShuffleExchangeExec +import org.apache.spark.sql.types.StructType + +trait ShimCometShuffleExchangeExec { + def apply(s: ShuffleExchangeExec, shuffleType: ShuffleType): CometShuffleExchangeExec = { + CometShuffleExchangeExec( + s.outputPartitioning, + s.child, + s, + s.shuffleOrigin, + shuffleType, + s.advisoryPartitionSize) + } + + protected def fromAttributes(attributes: Seq[Attribute]): StructType = DataTypeUtils.fromAttributes(attributes) + + protected def getShuffleId(shuffleDependency: ShuffleDependency[Int, _, _]): Int = shuffleDependency.shuffleId +} diff --git a/spark/src/main/spark-4.0/org/apache/comet/shims/ShimCometSparkSessionExtensions.scala b/spark/src/main/spark-4.0/org/apache/comet/shims/ShimCometSparkSessionExtensions.scala new file mode 100644 index 000000000..9fb7355ee --- /dev/null +++ b/spark/src/main/spark-4.0/org/apache/comet/shims/ShimCometSparkSessionExtensions.scala @@ -0,0 +1,37 @@ +/* + * 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.shims + +import org.apache.spark.sql.connector.expressions.aggregate.Aggregation +import org.apache.spark.sql.execution.{CollectLimitExec, GlobalLimitExec, LocalLimitExec, QueryExecution} +import org.apache.spark.sql.execution.datasources.v2.parquet.ParquetScan +import org.apache.spark.sql.internal.SQLConf + +trait ShimCometSparkSessionExtensions { + protected def getPushedAggregate(scan: ParquetScan): Option[Aggregation] = scan.pushedAggregate + + protected def getOffset(limit: LocalLimitExec): Int = 0 + protected def getOffset(limit: GlobalLimitExec): Int = limit.offset + protected def getOffset(limit: CollectLimitExec): Int = limit.offset + + protected def supportsExtendedExplainInfo(qe: QueryExecution): Boolean = true + + protected val EXTENDED_EXPLAIN_PROVIDERS_KEY = SQLConf.EXTENDED_EXPLAIN_PROVIDERS.key +} diff --git a/spark/src/main/spark-4.0/org/apache/comet/shims/ShimCometTakeOrderedAndProjectExec.scala b/spark/src/main/spark-4.0/org/apache/comet/shims/ShimCometTakeOrderedAndProjectExec.scala new file mode 100644 index 000000000..5a8ac97b3 --- /dev/null +++ b/spark/src/main/spark-4.0/org/apache/comet/shims/ShimCometTakeOrderedAndProjectExec.scala @@ -0,0 +1,26 @@ +/* + * 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.shims + +import org.apache.spark.sql.execution.TakeOrderedAndProjectExec + +trait ShimCometTakeOrderedAndProjectExec { + protected def getOffset(plan: TakeOrderedAndProjectExec): Option[Int] = Some(plan.offset) +} diff --git a/spark/src/main/spark-4.0/org/apache/comet/shims/ShimQueryPlanSerde.scala b/spark/src/main/spark-4.0/org/apache/comet/shims/ShimQueryPlanSerde.scala new file mode 100644 index 000000000..4d261f3c2 --- /dev/null +++ b/spark/src/main/spark-4.0/org/apache/comet/shims/ShimQueryPlanSerde.scala @@ -0,0 +1,37 @@ +/* + * 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.shims + +import org.apache.spark.sql.catalyst.expressions.{BinaryArithmetic, BinaryExpression, BloomFilterMightContain, EvalMode} +import org.apache.spark.sql.catalyst.expressions.aggregate.{Average, Sum} + +trait ShimQueryPlanSerde { + protected def getFailOnError(b: BinaryArithmetic): Boolean = + b.getClass.getMethod("failOnError").invoke(b).asInstanceOf[Boolean] + + protected def getFailOnError(aggregate: Sum): Boolean = aggregate.initQueryContext().isDefined + protected def getFailOnError(aggregate: Average): Boolean = aggregate.initQueryContext().isDefined + + protected def isLegacyMode(aggregate: Sum): Boolean = aggregate.evalMode.equals(EvalMode.LEGACY) + protected def isLegacyMode(aggregate: Average): Boolean = aggregate.evalMode.equals(EvalMode.LEGACY) + + protected def isBloomFilterMightContain(binary: BinaryExpression): Boolean = + binary.isInstanceOf[BloomFilterMightContain] +} diff --git a/spark/src/main/spark-4.0/org/apache/comet/shims/ShimSQLConf.scala b/spark/src/main/spark-4.0/org/apache/comet/shims/ShimSQLConf.scala new file mode 100644 index 000000000..574967767 --- /dev/null +++ b/spark/src/main/spark-4.0/org/apache/comet/shims/ShimSQLConf.scala @@ -0,0 +1,31 @@ +/* + * 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.shims + +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.internal.LegacyBehaviorPolicy + +trait ShimSQLConf { + protected def getPushDownStringPredicate(sqlConf: SQLConf): Boolean = + sqlConf.parquetFilterPushDownStringPredicate + + protected val LEGACY = LegacyBehaviorPolicy.LEGACY + protected val CORRECTED = LegacyBehaviorPolicy.CORRECTED +} diff --git a/spark/src/main/spark-4.0/org/apache/spark/comet/shims/ShimCometBroadcastExchangeExec.scala b/spark/src/main/spark-4.0/org/apache/spark/comet/shims/ShimCometBroadcastExchangeExec.scala new file mode 100644 index 000000000..ba87a2515 --- /dev/null +++ b/spark/src/main/spark-4.0/org/apache/spark/comet/shims/ShimCometBroadcastExchangeExec.scala @@ -0,0 +1,30 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.spark.comet.shims + +import scala.reflect.ClassTag + +import org.apache.spark.SparkContext +import org.apache.spark.broadcast.Broadcast + +trait ShimCometBroadcastExchangeExec { + protected def doBroadcast[T: ClassTag](sparkContext: SparkContext, value: T): Broadcast[Any] = + sparkContext.broadcastInternal(value, true) +} diff --git a/spark/src/main/spark-4.0/org/apache/spark/comet/shims/ShimCometDriverPlugin.scala b/spark/src/main/spark-4.0/org/apache/spark/comet/shims/ShimCometDriverPlugin.scala new file mode 100644 index 000000000..f7a57a642 --- /dev/null +++ b/spark/src/main/spark-4.0/org/apache/spark/comet/shims/ShimCometDriverPlugin.scala @@ -0,0 +1,32 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.spark.comet.shims + +import org.apache.spark.SparkConf +import org.apache.spark.internal.config.EXECUTOR_MEMORY_OVERHEAD_FACTOR +import org.apache.spark.internal.config.EXECUTOR_MIN_MEMORY_OVERHEAD + +trait ShimCometDriverPlugin { + protected def getMemoryOverheadFactor(sparkConf: SparkConf): Double = sparkConf.get( + EXECUTOR_MEMORY_OVERHEAD_FACTOR) + + protected def getMemoryOverheadMinMib(sparkConf: SparkConf): Long = sparkConf.get( + EXECUTOR_MIN_MEMORY_OVERHEAD) +} diff --git a/spark/src/main/spark-4.0/org/apache/spark/sql/comet/shims/ShimCometScanExec.scala b/spark/src/main/spark-4.0/org/apache/spark/sql/comet/shims/ShimCometScanExec.scala new file mode 100644 index 000000000..543116c10 --- /dev/null +++ b/spark/src/main/spark-4.0/org/apache/spark/sql/comet/shims/ShimCometScanExec.scala @@ -0,0 +1,83 @@ +/* + * 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.shims + +import org.apache.hadoop.fs.Path + +import org.apache.spark.sql.SparkSession +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.expressions.AttributeReference +import org.apache.spark.sql.connector.read.{InputPartition, PartitionReaderFactory} +import org.apache.spark.sql.errors.QueryExecutionErrors +import org.apache.spark.sql.execution.datasources.parquet.ParquetOptions +import org.apache.spark.sql.execution.datasources.v2.DataSourceRDD +import org.apache.spark.sql.execution.datasources._ +import org.apache.spark.sql.execution.metric.SQLMetric +import org.apache.spark.sql.execution.{FileSourceScanExec, PartitionedFileUtil} +import org.apache.spark.sql.types.StructType +import org.apache.spark.SparkContext + +trait ShimCometScanExec { + def wrapped: FileSourceScanExec + + lazy val fileConstantMetadataColumns: Seq[AttributeReference] = + wrapped.fileConstantMetadataColumns + + protected def newDataSourceRDD( + sc: SparkContext, + inputPartitions: Seq[Seq[InputPartition]], + partitionReaderFactory: PartitionReaderFactory, + columnarReads: Boolean, + customMetrics: Map[String, SQLMetric]): DataSourceRDD = + new DataSourceRDD(sc, inputPartitions, partitionReaderFactory, columnarReads, customMetrics) + + protected def newFileScanRDD( + fsRelation: HadoopFsRelation, + readFunction: PartitionedFile => Iterator[InternalRow], + filePartitions: Seq[FilePartition], + readSchema: StructType, + options: ParquetOptions): FileScanRDD = { + new FileScanRDD( + fsRelation.sparkSession, + readFunction, + filePartitions, + readSchema, + fileConstantMetadataColumns, + fsRelation.fileFormat.fileConstantMetadataExtractors, + options) + } + + protected def invalidBucketFile(path: String, sparkVersion: String): Throwable = + QueryExecutionErrors.invalidBucketFile(path) + + // see SPARK-39634 + protected def isNeededForSchema(sparkSchema: StructType): Boolean = false + + protected def getPartitionedFile(f: FileStatusWithMetadata, p: PartitionDirectory): PartitionedFile = + PartitionedFileUtil.getPartitionedFile(f, p.values, 0, f.getLen) + + protected def splitFiles(sparkSession: SparkSession, + file: FileStatusWithMetadata, + filePath: Path, + isSplitable: Boolean, + maxSplitBytes: Long, + partitionValues: InternalRow): Seq[PartitionedFile] = + PartitionedFileUtil.splitFiles(file, isSplitable, maxSplitBytes, partitionValues) +} diff --git a/spark/src/main/spark-4.0/org/apache/spark/sql/comet/shims/ShimCometShuffleWriteProcessor.scala b/spark/src/main/spark-4.0/org/apache/spark/sql/comet/shims/ShimCometShuffleWriteProcessor.scala new file mode 100644 index 000000000..f875e3f38 --- /dev/null +++ b/spark/src/main/spark-4.0/org/apache/spark/sql/comet/shims/ShimCometShuffleWriteProcessor.scala @@ -0,0 +1,26 @@ +/* + * 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.shims + +import org.apache.spark.shuffle.ShuffleWriteProcessor + +trait ShimCometShuffleWriteProcessor extends ShuffleWriteProcessor { + +} diff --git a/spark/src/test/scala/org/apache/spark/sql/CometTPCDSQuerySuite.scala b/spark/src/test/scala/org/apache/spark/sql/CometTPCDSQuerySuite.scala index 1357d6548..53186b131 100644 --- a/spark/src/test/scala/org/apache/spark/sql/CometTPCDSQuerySuite.scala +++ b/spark/src/test/scala/org/apache/spark/sql/CometTPCDSQuerySuite.scala @@ -21,13 +21,13 @@ package org.apache.spark.sql import org.apache.spark.SparkConf import org.apache.spark.internal.config.{MEMORY_OFFHEAP_ENABLED, MEMORY_OFFHEAP_SIZE} +import org.apache.spark.sql.comet.shims.ShimCometTPCDSQuerySuite import org.apache.comet.CometConf class CometTPCDSQuerySuite extends { - // This is private in `TPCDSBase`. - val excludedTpcdsQueries: Seq[String] = Seq() + override val excludedTpcdsQueries: Set[String] = Set() // This is private in `TPCDSBase` and `excludedTpcdsQueries` is private too. // So we cannot override `excludedTpcdsQueries` to exclude the queries. @@ -145,7 +145,8 @@ class CometTPCDSQuerySuite override val tpcdsQueries: Seq[String] = tpcdsAllQueries.filterNot(excludedTpcdsQueries.contains) } - with TPCDSQueryTestSuite { + with TPCDSQueryTestSuite + with ShimCometTPCDSQuerySuite { override def sparkConf: SparkConf = { val conf = super.sparkConf conf.set("spark.sql.extensions", "org.apache.comet.CometSparkSessionExtensions") diff --git a/spark/src/test/scala/org/apache/spark/sql/CometTPCHQuerySuite.scala b/spark/src/test/scala/org/apache/spark/sql/CometTPCHQuerySuite.scala index 1abe5faeb..ec87f19e9 100644 --- a/spark/src/test/scala/org/apache/spark/sql/CometTPCHQuerySuite.scala +++ b/spark/src/test/scala/org/apache/spark/sql/CometTPCHQuerySuite.scala @@ -33,6 +33,7 @@ import org.apache.spark.sql.test.{SharedSparkSession, TestSparkSession} import org.apache.comet.CometConf import org.apache.comet.CometSparkSessionExtensions.isSpark34Plus +import org.apache.comet.shims.ShimCometTPCHQuerySuite /** * End-to-end tests to check TPCH query results. @@ -49,7 +50,7 @@ import org.apache.comet.CometSparkSessionExtensions.isSpark34Plus * ./mvnw -Dsuites=org.apache.spark.sql.CometTPCHQuerySuite test * }}} */ -class CometTPCHQuerySuite extends QueryTest with CometTPCBase with SQLQueryTestHelper { +class CometTPCHQuerySuite extends QueryTest with CometTPCBase with ShimCometTPCHQuerySuite { private val tpchDataPath = sys.env.get("SPARK_TPCH_DATA") @@ -142,7 +143,7 @@ class CometTPCHQuerySuite extends QueryTest with CometTPCBase with SQLQueryTestH val shouldSortResults = sortMergeJoinConf != conf // Sort for other joins withSQLConf(conf.toSeq: _*) { try { - val (schema, output) = handleExceptions(getNormalizedResult(spark, query)) + val (schema, output) = handleExceptions(getNormalizedQueryExecutionResult(spark, query)) val queryString = query.trim val outputString = output.mkString("\n").replaceAll("\\s+$", "") if (shouldRegenerateGoldenFiles) { diff --git a/spark/src/test/scala/org/apache/spark/sql/CometTestBase.scala b/spark/src/test/scala/org/apache/spark/sql/CometTestBase.scala index 0530d764c..d8c82f12b 100644 --- a/spark/src/test/scala/org/apache/spark/sql/CometTestBase.scala +++ b/spark/src/test/scala/org/apache/spark/sql/CometTestBase.scala @@ -48,7 +48,6 @@ import org.apache.spark.sql.types.StructType import org.apache.comet._ import org.apache.comet.CometSparkSessionExtensions.isSpark34Plus import org.apache.comet.shims.ShimCometSparkSessionExtensions -import org.apache.comet.shims.ShimCometSparkSessionExtensions.supportsExtendedExplainInfo /** * Base class for testing. This exists in `org.apache.spark.sql` since [[SQLTestUtils]] is diff --git a/spark/src/test/scala/org/apache/spark/sql/benchmark/CometReadBenchmark.scala b/spark/src/test/scala/org/apache/spark/sql/benchmark/CometReadBenchmark.scala index 4c2f832af..fc4549445 100644 --- a/spark/src/test/scala/org/apache/spark/sql/benchmark/CometReadBenchmark.scala +++ b/spark/src/test/scala/org/apache/spark/sql/benchmark/CometReadBenchmark.scala @@ -25,11 +25,12 @@ import scala.collection.JavaConverters._ import scala.util.Random import org.apache.spark.benchmark.Benchmark +import org.apache.spark.comet.shims.ShimTestUtils import org.apache.spark.sql.execution.datasources.parquet.VectorizedParquetRecordReader import org.apache.spark.sql.types._ import org.apache.spark.sql.vectorized.ColumnVector -import org.apache.comet.{CometConf, TestUtils} +import org.apache.comet.CometConf import org.apache.comet.parquet.BatchReader /** @@ -123,7 +124,7 @@ object CometReadBenchmark extends CometBenchmarkBase { (col: ColumnVector, i: Int) => longSum += col.getUTF8String(i).toLongExact } - val files = TestUtils.listDirectory(new File(dir, "parquetV1")) + val files = ShimTestUtils.listDirectory(new File(dir, "parquetV1")) sqlBenchmark.addCase("ParquetReader Spark") { _ => files.map(_.asInstanceOf[String]).foreach { p => diff --git a/spark/src/test/spark-3.4/org/apache/comet/exec/CometExec3_4Suite.scala b/spark/src/test/spark-3.4-plus/org/apache/comet/exec/CometExec3_4PlusSuite.scala similarity index 98% rename from spark/src/test/spark-3.4/org/apache/comet/exec/CometExec3_4Suite.scala rename to spark/src/test/spark-3.4-plus/org/apache/comet/exec/CometExec3_4PlusSuite.scala index 019b4f030..31d1ffbf7 100644 --- a/spark/src/test/spark-3.4/org/apache/comet/exec/CometExec3_4Suite.scala +++ b/spark/src/test/spark-3.4-plus/org/apache/comet/exec/CometExec3_4PlusSuite.scala @@ -27,9 +27,9 @@ import org.apache.spark.sql.CometTestBase import org.apache.comet.CometConf /** - * This test suite contains tests for only Spark 3.4. + * This test suite contains tests for only Spark 3.4+. */ -class CometExec3_4Suite extends CometTestBase { +class CometExec3_4PlusSuite extends CometTestBase { import testImplicits._ override protected def test(testName: String, testTags: Tag*)(testFun: => Any)(implicit diff --git a/spark/src/test/spark-3.x/org/apache/comet/shims/ShimCometTPCHQuerySuite.scala b/spark/src/test/spark-3.x/org/apache/comet/shims/ShimCometTPCHQuerySuite.scala new file mode 100644 index 000000000..caa943c26 --- /dev/null +++ b/spark/src/test/spark-3.x/org/apache/comet/shims/ShimCometTPCHQuerySuite.scala @@ -0,0 +1,28 @@ +/* + * 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.shims + +import org.apache.spark.sql.{SQLQueryTestHelper, SparkSession} + +trait ShimCometTPCHQuerySuite extends SQLQueryTestHelper { + protected def getNormalizedQueryExecutionResult(session: SparkSession, sql: String): (String, Seq[String]) = { + getNormalizedResult(session, sql) + } +} diff --git a/spark/src/test/scala/org/apache/comet/TestUtils.scala b/spark/src/test/spark-3.x/org/apache/spark/comet/shims/ShimTestUtils.scala similarity index 96% rename from spark/src/test/scala/org/apache/comet/TestUtils.scala rename to spark/src/test/spark-3.x/org/apache/spark/comet/shims/ShimTestUtils.scala index d4e771568..fcb543f9b 100644 --- a/spark/src/test/scala/org/apache/comet/TestUtils.scala +++ b/spark/src/test/spark-3.x/org/apache/spark/comet/shims/ShimTestUtils.scala @@ -17,13 +17,12 @@ * under the License. */ -package org.apache.comet +package org.apache.spark.comet.shims import java.io.File - import scala.collection.mutable.ArrayBuffer -object TestUtils { +object ShimTestUtils { /** * Spark 3.3.0 moved {{{SpecificParquetRecordReaderBase.listDirectory}}} to diff --git a/spark/src/test/spark-3.x/org/apache/spark/sql/comet/shims/ShimCometTPCDSQuerySuite.scala b/spark/src/test/spark-3.x/org/apache/spark/sql/comet/shims/ShimCometTPCDSQuerySuite.scala new file mode 100644 index 000000000..f8d621c7e --- /dev/null +++ b/spark/src/test/spark-3.x/org/apache/spark/sql/comet/shims/ShimCometTPCDSQuerySuite.scala @@ -0,0 +1,25 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.spark.sql.comet.shims + +trait ShimCometTPCDSQuerySuite { + // This is private in `TPCDSBase`. + val excludedTpcdsQueries: Set[String] = Set() +} diff --git a/spark/src/test/spark-4.0/org/apache/comet/shims/ShimCometTPCHQuerySuite.scala b/spark/src/test/spark-4.0/org/apache/comet/shims/ShimCometTPCHQuerySuite.scala new file mode 100644 index 000000000..ec9823e52 --- /dev/null +++ b/spark/src/test/spark-4.0/org/apache/comet/shims/ShimCometTPCHQuerySuite.scala @@ -0,0 +1,25 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.comet.shims + +import org.apache.spark.sql.SQLQueryTestHelper + +trait ShimCometTPCHQuerySuite extends SQLQueryTestHelper { +} diff --git a/spark/src/test/spark-4.0/org/apache/spark/comet/shims/ShimTestUtils.scala b/spark/src/test/spark-4.0/org/apache/spark/comet/shims/ShimTestUtils.scala new file mode 100644 index 000000000..923ae68f2 --- /dev/null +++ b/spark/src/test/spark-4.0/org/apache/spark/comet/shims/ShimTestUtils.scala @@ -0,0 +1,27 @@ +/* + * 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.comet.shims + +import java.io.File + +object ShimTestUtils { + def listDirectory(path: File): Array[String] = + org.apache.spark.TestUtils.listDirectory(path) +} diff --git a/spark/src/test/spark-4.0/org/apache/spark/sql/comet/shims/ShimCometTPCDSQuerySuite.scala b/spark/src/test/spark-4.0/org/apache/spark/sql/comet/shims/ShimCometTPCDSQuerySuite.scala new file mode 100644 index 000000000..43917df63 --- /dev/null +++ b/spark/src/test/spark-4.0/org/apache/spark/sql/comet/shims/ShimCometTPCDSQuerySuite.scala @@ -0,0 +1,24 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.spark.sql.comet.shims + +trait ShimCometTPCDSQuerySuite { + +} From ee154606e2380a76e7e4f5cbceb7a7b59b33877f Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh <viirya@gmail.com> Date: Tue, 28 May 2024 13:50:42 -0700 Subject: [PATCH 07/19] fix: Reuse previous CometDictionary Java arrays (#489) --- .../main/java/org/apache/comet/parquet/ColumnReader.java | 6 +++++- .../java/org/apache/comet/vector/CometDictionary.java | 9 ++++++++- 2 files changed, 13 insertions(+), 2 deletions(-) diff --git a/common/src/main/java/org/apache/comet/parquet/ColumnReader.java b/common/src/main/java/org/apache/comet/parquet/ColumnReader.java index 3d4cb3aa5..9e594804f 100644 --- a/common/src/main/java/org/apache/comet/parquet/ColumnReader.java +++ b/common/src/main/java/org/apache/comet/parquet/ColumnReader.java @@ -234,7 +234,11 @@ public CometDecodedVector loadVector() { Dictionary arrowDictionary = importer.getProvider().lookup(dictionaryEncoding.getId()); CometPlainVector dictionaryVector = new CometPlainVector(arrowDictionary.getVector(), useDecimal128, isUuid); - dictionary = new CometDictionary(dictionaryVector); + if (dictionary != null) { + dictionary.setDictionaryVector(dictionaryVector); + } else { + dictionary = new CometDictionary(dictionaryVector); + } currentVector = new CometDictionaryVector( diff --git a/common/src/main/java/org/apache/comet/vector/CometDictionary.java b/common/src/main/java/org/apache/comet/vector/CometDictionary.java index 9aa42bd68..b213b75d8 100644 --- a/common/src/main/java/org/apache/comet/vector/CometDictionary.java +++ b/common/src/main/java/org/apache/comet/vector/CometDictionary.java @@ -26,7 +26,7 @@ public class CometDictionary implements AutoCloseable { private static final int DECIMAL_BYTE_WIDTH = 16; - private final CometPlainVector values; + private CometPlainVector values; private final int numValues; /** Decoded dictionary values. Only one of the following is set. */ @@ -47,6 +47,13 @@ public CometDictionary(CometPlainVector values) { initialize(); } + public void setDictionaryVector(CometPlainVector values) { + this.values = values; + if (values.numValues() != numValues) { + throw new IllegalArgumentException("Mismatched dictionary size"); + } + } + public ValueVector getValueVector() { return values.getValueVector(); } From fd90a284995d7b518c048127773c16605ed5fd44 Mon Sep 17 00:00:00 2001 From: Sujith Jay Nair <sujith@sujithjay.com> Date: Tue, 28 May 2024 19:53:13 -0400 Subject: [PATCH 08/19] fix: Fallback to Spark for LIKE with custom escape character (#478) * Fallback to Spark for LIKE with custom escape character Currently, LIKE with custom escape character produces incorrect results. * For custom escape character, provide user with specific info message * Test case for default escape char with checkSparkAnswerAndOperator --- .../apache/comet/serde/QueryPlanSerde.scala | 33 +++++++++++-------- .../apache/comet/CometExpressionSuite.scala | 18 ++++++++++ 2 files changed, 37 insertions(+), 14 deletions(-) diff --git a/spark/src/main/scala/org/apache/comet/serde/QueryPlanSerde.scala b/spark/src/main/scala/org/apache/comet/serde/QueryPlanSerde.scala index a717e066e..06b9bc7f4 100644 --- a/spark/src/main/scala/org/apache/comet/serde/QueryPlanSerde.scala +++ b/spark/src/main/scala/org/apache/comet/serde/QueryPlanSerde.scala @@ -1074,23 +1074,28 @@ object QueryPlanSerde extends Logging with ShimQueryPlanSerde with CometExprShim None } - case Like(left, right, _) => - // TODO escapeChar - val leftExpr = exprToProtoInternal(left, inputs) - val rightExpr = exprToProtoInternal(right, inputs) + case Like(left, right, escapeChar) => + if (escapeChar == '\\') { + val leftExpr = exprToProtoInternal(left, inputs) + val rightExpr = exprToProtoInternal(right, inputs) - if (leftExpr.isDefined && rightExpr.isDefined) { - val builder = ExprOuterClass.Like.newBuilder() - builder.setLeft(leftExpr.get) - builder.setRight(rightExpr.get) + if (leftExpr.isDefined && rightExpr.isDefined) { + val builder = ExprOuterClass.Like.newBuilder() + builder.setLeft(leftExpr.get) + builder.setRight(rightExpr.get) - Some( - ExprOuterClass.Expr - .newBuilder() - .setLike(builder) - .build()) + Some( + ExprOuterClass.Expr + .newBuilder() + .setLike(builder) + .build()) + } else { + withInfo(expr, left, right) + None + } } else { - withInfo(expr, left, right) + // TODO custom escape char + withInfo(expr, s"custom escape character $escapeChar not supported in LIKE") None } diff --git a/spark/src/test/scala/org/apache/comet/CometExpressionSuite.scala b/spark/src/test/scala/org/apache/comet/CometExpressionSuite.scala index 1afdd78ec..34c794eb1 100644 --- a/spark/src/test/scala/org/apache/comet/CometExpressionSuite.scala +++ b/spark/src/test/scala/org/apache/comet/CometExpressionSuite.scala @@ -594,6 +594,24 @@ class CometExpressionSuite extends CometTestBase with AdaptiveSparkPlanHelper { } } + test("like with custom escape") { + val table = "names" + withTable(table) { + sql(s"create table $table(id int, name varchar(20)) using parquet") + sql(s"insert into $table values(1,'James Smith')") + sql(s"insert into $table values(2,'Michael_Rose')") + sql(s"insert into $table values(3,'Robert_R_Williams')") + + // Filter column having values that include underscores + val queryDefaultEscape = sql("select id from names where name like '%\\_%'") + checkSparkAnswerAndOperator(queryDefaultEscape) + + val queryCustomEscape = sql("select id from names where name like '%$_%' escape '$'") + checkAnswer(queryCustomEscape, Row(2) :: Row(3) :: Nil) + + } + } + test("contains") { assume(!isSpark32) From 8f8a0d927daf146015bee3306342e1844e1b4758 Mon Sep 17 00:00:00 2001 From: Huaxin Gao <huaxin_gao@apple.com> Date: Tue, 28 May 2024 23:17:57 -0700 Subject: [PATCH 09/19] build: bump spark version to 3.4.3 (#292) * build: bump spark version to 3.4.3 * add 3.4.4 diff * fix 3.4.3.diff --------- Co-authored-by: Huaxin Gao <huaxin.gao@apple.com> --- .../actions/setup-spark-builder/action.yaml | 4 +- .github/workflows/spark_sql_test.yml | 2 +- dev/diffs/3.4.3.diff | 2564 +++++++++++++++++ pom.xml | 2 +- 4 files changed, 2568 insertions(+), 4 deletions(-) create mode 100644 dev/diffs/3.4.3.diff diff --git a/.github/actions/setup-spark-builder/action.yaml b/.github/actions/setup-spark-builder/action.yaml index 9293dee90..10cdbff59 100644 --- a/.github/actions/setup-spark-builder/action.yaml +++ b/.github/actions/setup-spark-builder/action.yaml @@ -23,9 +23,9 @@ inputs: required: true default: '3.4' spark-version: - description: 'The Apache Spark version (e.g., 3.4.2) to build' + description: 'The Apache Spark version (e.g., 3.4.3) to build' required: true - default: '3.4.2' + default: '3.4.3' comet-version: description: 'The Comet version to use for Spark' required: true diff --git a/.github/workflows/spark_sql_test.yml b/.github/workflows/spark_sql_test.yml index 94958b46c..997136ded 100644 --- a/.github/workflows/spark_sql_test.yml +++ b/.github/workflows/spark_sql_test.yml @@ -45,7 +45,7 @@ jobs: matrix: os: [ubuntu-latest] java-version: [11] - spark-version: [{short: '3.4', full: '3.4.2'}] + spark-version: [{short: '3.4', full: '3.4.3'}] module: - {name: "catalyst", args1: "catalyst/test", args2: ""} - {name: "sql/core-1", args1: "", args2: sql/testOnly * -- -l org.apache.spark.tags.ExtendedSQLTest -l org.apache.spark.tags.SlowSQLTest} diff --git a/dev/diffs/3.4.3.diff b/dev/diffs/3.4.3.diff new file mode 100644 index 000000000..ae46e916b --- /dev/null +++ b/dev/diffs/3.4.3.diff @@ -0,0 +1,2564 @@ +diff --git a/pom.xml b/pom.xml +index d3544881af1..47382e29b5a 100644 +--- a/pom.xml ++++ b/pom.xml +@@ -148,6 +148,8 @@ + <chill.version>0.10.0</chill.version> + <ivy.version>2.5.1</ivy.version> + <oro.version>2.0.8</oro.version> ++ <spark.version.short>3.4</spark.version.short> ++ <comet.version>0.1.0-SNAPSHOT</comet.version> + <!-- + If you changes codahale.metrics.version, you also need to change + the link to metrics.dropwizard.io in docs/monitoring.md. +@@ -2784,6 +2786,25 @@ + <artifactId>arpack</artifactId> + <version>${netlib.ludovic.dev.version}</version> + </dependency> ++ <dependency> ++ <groupId>org.apache.comet</groupId> ++ <artifactId>comet-spark-spark${spark.version.short}_${scala.binary.version}</artifactId> ++ <version>${comet.version}</version> ++ <exclusions> ++ <exclusion> ++ <groupId>org.apache.spark</groupId> ++ <artifactId>spark-sql_${scala.binary.version}</artifactId> ++ </exclusion> ++ <exclusion> ++ <groupId>org.apache.spark</groupId> ++ <artifactId>spark-core_${scala.binary.version}</artifactId> ++ </exclusion> ++ <exclusion> ++ <groupId>org.apache.spark</groupId> ++ <artifactId>spark-catalyst_${scala.binary.version}</artifactId> ++ </exclusion> ++ </exclusions> ++ </dependency> + </dependencies> + </dependencyManagement> + +diff --git a/sql/core/pom.xml b/sql/core/pom.xml +index b386d135da1..854aec17c2d 100644 +--- a/sql/core/pom.xml ++++ b/sql/core/pom.xml +@@ -77,6 +77,10 @@ + <groupId>org.apache.spark</groupId> + <artifactId>spark-tags_${scala.binary.version}</artifactId> + </dependency> ++ <dependency> ++ <groupId>org.apache.comet</groupId> ++ <artifactId>comet-spark-spark${spark.version.short}_${scala.binary.version}</artifactId> ++ </dependency> + + <!-- + This spark-tags test-dep is needed even though it isn't used in this module, otherwise testing-cmds that exclude +diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala b/sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala +index c595b50950b..6b60213e775 100644 +--- a/sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala ++++ b/sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala +@@ -102,7 +102,7 @@ class SparkSession private( + sc: SparkContext, + initialSessionOptions: java.util.HashMap[String, String]) = { + this(sc, None, None, +- SparkSession.applyExtensions( ++ SparkSession.applyExtensions(sc, + sc.getConf.get(StaticSQLConf.SPARK_SESSION_EXTENSIONS).getOrElse(Seq.empty), + new SparkSessionExtensions), initialSessionOptions.asScala.toMap) + } +@@ -1028,7 +1028,7 @@ object SparkSession extends Logging { + } + + loadExtensions(extensions) +- applyExtensions( ++ applyExtensions(sparkContext, + sparkContext.getConf.get(StaticSQLConf.SPARK_SESSION_EXTENSIONS).getOrElse(Seq.empty), + extensions) + +@@ -1282,14 +1282,24 @@ object SparkSession extends Logging { + } + } + ++ private def loadCometExtension(sparkContext: SparkContext): Seq[String] = { ++ if (sparkContext.getConf.getBoolean("spark.comet.enabled", false)) { ++ Seq("org.apache.comet.CometSparkSessionExtensions") ++ } else { ++ Seq.empty ++ } ++ } ++ + /** + * Initialize extensions for given extension classnames. The classes will be applied to the + * extensions passed into this function. + */ + private def applyExtensions( ++ sparkContext: SparkContext, + extensionConfClassNames: Seq[String], + extensions: SparkSessionExtensions): SparkSessionExtensions = { +- extensionConfClassNames.foreach { extensionConfClassName => ++ val extensionClassNames = extensionConfClassNames ++ loadCometExtension(sparkContext) ++ extensionClassNames.foreach { extensionConfClassName => + try { + val extensionConfClass = Utils.classForName(extensionConfClassName) + val extensionConf = extensionConfClass.getConstructor().newInstance() +diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlanInfo.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlanInfo.scala +index db587dd9868..aac7295a53d 100644 +--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlanInfo.scala ++++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlanInfo.scala +@@ -18,6 +18,7 @@ + package org.apache.spark.sql.execution + + import org.apache.spark.annotation.DeveloperApi ++import org.apache.spark.sql.comet.CometScanExec + import org.apache.spark.sql.execution.adaptive.{AdaptiveSparkPlanExec, QueryStageExec} + import org.apache.spark.sql.execution.columnar.InMemoryTableScanExec + import org.apache.spark.sql.execution.exchange.ReusedExchangeExec +@@ -67,6 +68,7 @@ private[execution] object SparkPlanInfo { + // dump the file scan metadata (e.g file path) to event log + val metadata = plan match { + case fileScan: FileSourceScanExec => fileScan.metadata ++ case cometScan: CometScanExec => cometScan.metadata + case _ => Map[String, String]() + } + new SparkPlanInfo( +diff --git a/sql/core/src/test/resources/sql-tests/inputs/explain-aqe.sql b/sql/core/src/test/resources/sql-tests/inputs/explain-aqe.sql +index 7aef901da4f..f3d6e18926d 100644 +--- a/sql/core/src/test/resources/sql-tests/inputs/explain-aqe.sql ++++ b/sql/core/src/test/resources/sql-tests/inputs/explain-aqe.sql +@@ -2,3 +2,4 @@ + + --SET spark.sql.adaptive.enabled=true + --SET spark.sql.maxMetadataStringLength = 500 ++--SET spark.comet.enabled = false +diff --git a/sql/core/src/test/resources/sql-tests/inputs/explain-cbo.sql b/sql/core/src/test/resources/sql-tests/inputs/explain-cbo.sql +index eeb2180f7a5..afd1b5ec289 100644 +--- a/sql/core/src/test/resources/sql-tests/inputs/explain-cbo.sql ++++ b/sql/core/src/test/resources/sql-tests/inputs/explain-cbo.sql +@@ -1,5 +1,6 @@ + --SET spark.sql.cbo.enabled=true + --SET spark.sql.maxMetadataStringLength = 500 ++--SET spark.comet.enabled = false + + CREATE TABLE explain_temp1(a INT, b INT) USING PARQUET; + CREATE TABLE explain_temp2(c INT, d INT) USING PARQUET; +diff --git a/sql/core/src/test/resources/sql-tests/inputs/explain.sql b/sql/core/src/test/resources/sql-tests/inputs/explain.sql +index 698ca009b4f..57d774a3617 100644 +--- a/sql/core/src/test/resources/sql-tests/inputs/explain.sql ++++ b/sql/core/src/test/resources/sql-tests/inputs/explain.sql +@@ -1,6 +1,7 @@ + --SET spark.sql.codegen.wholeStage = true + --SET spark.sql.adaptive.enabled = false + --SET spark.sql.maxMetadataStringLength = 500 ++--SET spark.comet.enabled = false + + -- Test tables + CREATE table explain_temp1 (key int, val int) USING PARQUET; +diff --git a/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/aggregates_part1.sql b/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/aggregates_part1.sql +index 1152d77da0c..f77493f690b 100644 +--- a/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/aggregates_part1.sql ++++ b/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/aggregates_part1.sql +@@ -7,6 +7,9 @@ + + -- avoid bit-exact output here because operations may not be bit-exact. + -- SET extra_float_digits = 0; ++-- Disable Comet exec due to floating point precision difference ++--SET spark.comet.exec.enabled = false ++ + + -- Test aggregate operator with codegen on and off. + --CONFIG_DIM1 spark.sql.codegen.wholeStage=true +diff --git a/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/aggregates_part3.sql b/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/aggregates_part3.sql +index 41fd4de2a09..44cd244d3b0 100644 +--- a/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/aggregates_part3.sql ++++ b/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/aggregates_part3.sql +@@ -5,6 +5,9 @@ + -- AGGREGATES [Part 3] + -- https://github.com/postgres/postgres/blob/REL_12_BETA2/src/test/regress/sql/aggregates.sql#L352-L605 + ++-- Disable Comet exec due to floating point precision difference ++--SET spark.comet.exec.enabled = false ++ + -- Test aggregate operator with codegen on and off. + --CONFIG_DIM1 spark.sql.codegen.wholeStage=true + --CONFIG_DIM1 spark.sql.codegen.wholeStage=false,spark.sql.codegen.factoryMode=CODEGEN_ONLY +diff --git a/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/int8.sql b/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/int8.sql +index fac23b4a26f..2b73732c33f 100644 +--- a/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/int8.sql ++++ b/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/int8.sql +@@ -1,6 +1,10 @@ + -- + -- Portions Copyright (c) 1996-2019, PostgreSQL Global Development Group + -- ++ ++-- Disable Comet exec due to floating point precision difference ++--SET spark.comet.exec.enabled = false ++ + -- + -- INT8 + -- Test int8 64-bit integers. +diff --git a/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/select_having.sql b/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/select_having.sql +index 0efe0877e9b..423d3b3d76d 100644 +--- a/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/select_having.sql ++++ b/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/select_having.sql +@@ -1,6 +1,10 @@ + -- + -- Portions Copyright (c) 1996-2019, PostgreSQL Global Development Group + -- ++ ++-- Disable Comet exec due to floating point precision difference ++--SET spark.comet.exec.enabled = false ++ + -- + -- SELECT_HAVING + -- https://github.com/postgres/postgres/blob/REL_12_BETA2/src/test/regress/sql/select_having.sql +diff --git a/sql/core/src/test/scala/org/apache/spark/sql/CachedTableSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/CachedTableSuite.scala +index cf40e944c09..bdd5be4f462 100644 +--- a/sql/core/src/test/scala/org/apache/spark/sql/CachedTableSuite.scala ++++ b/sql/core/src/test/scala/org/apache/spark/sql/CachedTableSuite.scala +@@ -38,7 +38,7 @@ import org.apache.spark.sql.catalyst.util.DateTimeConstants + import org.apache.spark.sql.execution.{ColumnarToRowExec, ExecSubqueryExpression, RDDScanExec, SparkPlan} + import org.apache.spark.sql.execution.adaptive.AdaptiveSparkPlanHelper + import org.apache.spark.sql.execution.columnar._ +-import org.apache.spark.sql.execution.exchange.ShuffleExchangeExec ++import org.apache.spark.sql.execution.exchange.ShuffleExchangeLike + import org.apache.spark.sql.execution.ui.SparkListenerSQLAdaptiveExecutionUpdate + import org.apache.spark.sql.functions._ + import org.apache.spark.sql.internal.SQLConf +@@ -516,7 +516,8 @@ class CachedTableSuite extends QueryTest with SQLTestUtils + */ + private def verifyNumExchanges(df: DataFrame, expected: Int): Unit = { + assert( +- collect(df.queryExecution.executedPlan) { case e: ShuffleExchangeExec => e }.size == expected) ++ collect(df.queryExecution.executedPlan) { ++ case _: ShuffleExchangeLike => 1 }.size == expected) + } + + test("A cached table preserves the partitioning and ordering of its cached SparkPlan") { +diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameAggregateSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameAggregateSuite.scala +index 1cc09c3d7fc..f031fa45c33 100644 +--- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameAggregateSuite.scala ++++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameAggregateSuite.scala +@@ -27,7 +27,7 @@ import org.apache.spark.SparkException + import org.apache.spark.sql.execution.WholeStageCodegenExec + import org.apache.spark.sql.execution.adaptive.AdaptiveSparkPlanHelper + import org.apache.spark.sql.execution.aggregate.{HashAggregateExec, ObjectHashAggregateExec, SortAggregateExec} +-import org.apache.spark.sql.execution.exchange.ShuffleExchangeExec ++import org.apache.spark.sql.execution.exchange.ShuffleExchangeLike + import org.apache.spark.sql.expressions.Window + import org.apache.spark.sql.functions._ + import org.apache.spark.sql.internal.SQLConf +@@ -755,7 +755,7 @@ class DataFrameAggregateSuite extends QueryTest + assert(objHashAggPlans.nonEmpty) + + val exchangePlans = collect(aggPlan) { +- case shuffle: ShuffleExchangeExec => shuffle ++ case shuffle: ShuffleExchangeLike => shuffle + } + assert(exchangePlans.length == 1) + } +diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameJoinSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameJoinSuite.scala +index 56e9520fdab..917932336df 100644 +--- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameJoinSuite.scala ++++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameJoinSuite.scala +@@ -435,7 +435,9 @@ class DataFrameJoinSuite extends QueryTest + + withTempDatabase { dbName => + withTable(table1Name, table2Name) { +- withSQLConf(SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "-1") { ++ withSQLConf( ++ SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "-1", ++ "spark.comet.enabled" -> "false") { + spark.range(50).write.saveAsTable(s"$dbName.$table1Name") + spark.range(100).write.saveAsTable(s"$dbName.$table2Name") + +diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala +index a9f69ab28a1..4056f14c893 100644 +--- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala ++++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala +@@ -43,7 +43,7 @@ import org.apache.spark.sql.connector.FakeV2Provider + import org.apache.spark.sql.execution.{FilterExec, LogicalRDD, QueryExecution, SortExec, WholeStageCodegenExec} + import org.apache.spark.sql.execution.adaptive.AdaptiveSparkPlanHelper + import org.apache.spark.sql.execution.aggregate.HashAggregateExec +-import org.apache.spark.sql.execution.exchange.{BroadcastExchangeExec, ReusedExchangeExec, ShuffleExchangeExec, ShuffleExchangeLike} ++import org.apache.spark.sql.execution.exchange.{BroadcastExchangeExec, ReusedExchangeExec, ShuffleExchangeLike} + import org.apache.spark.sql.expressions.{Aggregator, Window} + import org.apache.spark.sql.functions._ + import org.apache.spark.sql.internal.SQLConf +@@ -1981,7 +1981,7 @@ class DataFrameSuite extends QueryTest + fail("Should not have back to back Aggregates") + } + atFirstAgg = true +- case e: ShuffleExchangeExec => atFirstAgg = false ++ case e: ShuffleExchangeLike => atFirstAgg = false + case _ => + } + } +@@ -2305,7 +2305,7 @@ class DataFrameSuite extends QueryTest + checkAnswer(join, df) + assert( + collect(join.queryExecution.executedPlan) { +- case e: ShuffleExchangeExec => true }.size === 1) ++ case _: ShuffleExchangeLike => true }.size === 1) + assert( + collect(join.queryExecution.executedPlan) { case e: ReusedExchangeExec => true }.size === 1) + val broadcasted = broadcast(join) +@@ -2313,7 +2313,7 @@ class DataFrameSuite extends QueryTest + checkAnswer(join2, df) + assert( + collect(join2.queryExecution.executedPlan) { +- case e: ShuffleExchangeExec => true }.size == 1) ++ case _: ShuffleExchangeLike => true }.size == 1) + assert( + collect(join2.queryExecution.executedPlan) { + case e: BroadcastExchangeExec => true }.size === 1) +@@ -2876,7 +2876,7 @@ class DataFrameSuite extends QueryTest + + // Assert that no extra shuffle introduced by cogroup. + val exchanges = collect(df3.queryExecution.executedPlan) { +- case h: ShuffleExchangeExec => h ++ case h: ShuffleExchangeLike => h + } + assert(exchanges.size == 2) + } +@@ -3325,7 +3325,8 @@ class DataFrameSuite extends QueryTest + assert(df2.isLocal) + } + +- test("SPARK-35886: PromotePrecision should be subexpr replaced") { ++ test("SPARK-35886: PromotePrecision should be subexpr replaced", ++ IgnoreComet("TODO: fix Comet for this test")) { + withTable("tbl") { + sql( + """ +diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala +index daef11ae4d6..9f3cc9181f2 100644 +--- a/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala ++++ b/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala +@@ -39,7 +39,7 @@ import org.apache.spark.sql.catalyst.plans.{LeftAnti, LeftSemi} + import org.apache.spark.sql.catalyst.util.sideBySide + import org.apache.spark.sql.execution.{LogicalRDD, RDDScanExec, SQLExecution} + import org.apache.spark.sql.execution.adaptive.AdaptiveSparkPlanHelper +-import org.apache.spark.sql.execution.exchange.{BroadcastExchangeExec, ShuffleExchangeExec} ++import org.apache.spark.sql.execution.exchange.{BroadcastExchangeExec, ShuffleExchangeExec, ShuffleExchangeLike} + import org.apache.spark.sql.execution.streaming.MemoryStream + import org.apache.spark.sql.expressions.UserDefinedFunction + import org.apache.spark.sql.functions._ +@@ -2254,7 +2254,7 @@ class DatasetSuite extends QueryTest + + // Assert that no extra shuffle introduced by cogroup. + val exchanges = collect(df3.queryExecution.executedPlan) { +- case h: ShuffleExchangeExec => h ++ case h: ShuffleExchangeLike => h + } + assert(exchanges.size == 2) + } +diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DynamicPartitionPruningSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DynamicPartitionPruningSuite.scala +index f33432ddb6f..060f874ea72 100644 +--- a/sql/core/src/test/scala/org/apache/spark/sql/DynamicPartitionPruningSuite.scala ++++ b/sql/core/src/test/scala/org/apache/spark/sql/DynamicPartitionPruningSuite.scala +@@ -22,6 +22,7 @@ import org.scalatest.GivenWhenThen + import org.apache.spark.sql.catalyst.expressions.{DynamicPruningExpression, Expression} + import org.apache.spark.sql.catalyst.expressions.CodegenObjectFactoryMode._ + import org.apache.spark.sql.catalyst.plans.ExistenceJoin ++import org.apache.spark.sql.comet.CometScanExec + import org.apache.spark.sql.connector.catalog.{InMemoryTableCatalog, InMemoryTableWithV2FilterCatalog} + import org.apache.spark.sql.execution._ + import org.apache.spark.sql.execution.adaptive._ +@@ -262,6 +263,9 @@ abstract class DynamicPartitionPruningSuiteBase + case s: BatchScanExec => s.runtimeFilters.collect { + case d: DynamicPruningExpression => d.child + } ++ case s: CometScanExec => s.partitionFilters.collect { ++ case d: DynamicPruningExpression => d.child ++ } + case _ => Nil + } + } +@@ -1187,7 +1191,8 @@ abstract class DynamicPartitionPruningSuiteBase + } + } + +- test("Make sure dynamic pruning works on uncorrelated queries") { ++ test("Make sure dynamic pruning works on uncorrelated queries", ++ IgnoreComet("TODO: Support SubqueryBroadcastExec in Comet: #242")) { + withSQLConf(SQLConf.DYNAMIC_PARTITION_PRUNING_REUSE_BROADCAST_ONLY.key -> "true") { + val df = sql( + """ +@@ -1238,7 +1243,8 @@ abstract class DynamicPartitionPruningSuiteBase + } + } + +- test("Plan broadcast pruning only when the broadcast can be reused") { ++ test("Plan broadcast pruning only when the broadcast can be reused", ++ IgnoreComet("TODO: Support SubqueryBroadcastExec in Comet: #242")) { + Given("dynamic pruning filter on the build side") + withSQLConf(SQLConf.DYNAMIC_PARTITION_PRUNING_REUSE_BROADCAST_ONLY.key -> "true") { + val df = sql( +@@ -1485,7 +1491,7 @@ abstract class DynamicPartitionPruningSuiteBase + } + + test("SPARK-38148: Do not add dynamic partition pruning if there exists static partition " + +- "pruning") { ++ "pruning", IgnoreComet("TODO: Support SubqueryBroadcastExec in Comet: #242")) { + withSQLConf(SQLConf.DYNAMIC_PARTITION_PRUNING_ENABLED.key -> "true") { + Seq( + "f.store_id = 1" -> false, +@@ -1729,6 +1735,8 @@ abstract class DynamicPartitionPruningV1Suite extends DynamicPartitionPruningDat + case s: BatchScanExec => + // we use f1 col for v2 tables due to schema pruning + s.output.exists(_.exists(_.argString(maxFields = 100).contains("f1"))) ++ case s: CometScanExec => ++ s.output.exists(_.exists(_.argString(maxFields = 100).contains("fid"))) + case _ => false + } + assert(scanOption.isDefined) +diff --git a/sql/core/src/test/scala/org/apache/spark/sql/ExplainSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/ExplainSuite.scala +index a6b295578d6..91acca4306f 100644 +--- a/sql/core/src/test/scala/org/apache/spark/sql/ExplainSuite.scala ++++ b/sql/core/src/test/scala/org/apache/spark/sql/ExplainSuite.scala +@@ -463,7 +463,8 @@ class ExplainSuite extends ExplainSuiteHelper with DisableAdaptiveExecutionSuite + } + } + +- test("Explain formatted output for scan operator for datasource V2") { ++ test("Explain formatted output for scan operator for datasource V2", ++ IgnoreComet("Comet explain output is different")) { + withTempDir { dir => + Seq("parquet", "orc", "csv", "json").foreach { fmt => + val basePath = dir.getCanonicalPath + "/" + fmt +@@ -541,7 +542,9 @@ class ExplainSuite extends ExplainSuiteHelper with DisableAdaptiveExecutionSuite + } + } + +-class ExplainSuiteAE extends ExplainSuiteHelper with EnableAdaptiveExecutionSuite { ++// Ignored when Comet is enabled. Comet changes expected query plans. ++class ExplainSuiteAE extends ExplainSuiteHelper with EnableAdaptiveExecutionSuite ++ with IgnoreCometSuite { + import testImplicits._ + + test("SPARK-35884: Explain Formatted") { +diff --git a/sql/core/src/test/scala/org/apache/spark/sql/FileBasedDataSourceSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/FileBasedDataSourceSuite.scala +index 2796b1cf154..be7078b38f4 100644 +--- a/sql/core/src/test/scala/org/apache/spark/sql/FileBasedDataSourceSuite.scala ++++ b/sql/core/src/test/scala/org/apache/spark/sql/FileBasedDataSourceSuite.scala +@@ -33,6 +33,7 @@ import org.apache.spark.sql.TestingUDT.{IntervalUDT, NullData, NullUDT} + import org.apache.spark.sql.catalyst.expressions.{AttributeReference, GreaterThan, Literal} + import org.apache.spark.sql.catalyst.expressions.IntegralLiteralTestUtils.{negativeInt, positiveInt} + import org.apache.spark.sql.catalyst.plans.logical.Filter ++import org.apache.spark.sql.comet.{CometBatchScanExec, CometScanExec, CometSortMergeJoinExec} + import org.apache.spark.sql.execution.{FileSourceScanLike, SimpleMode} + import org.apache.spark.sql.execution.adaptive.AdaptiveSparkPlanHelper + import org.apache.spark.sql.execution.datasources.FilePartition +@@ -815,6 +816,7 @@ class FileBasedDataSourceSuite extends QueryTest + assert(bJoinExec.isEmpty) + val smJoinExec = collect(joinedDF.queryExecution.executedPlan) { + case smJoin: SortMergeJoinExec => smJoin ++ case smJoin: CometSortMergeJoinExec => smJoin + } + assert(smJoinExec.nonEmpty) + } +@@ -875,6 +877,7 @@ class FileBasedDataSourceSuite extends QueryTest + + val fileScan = df.queryExecution.executedPlan collectFirst { + case BatchScanExec(_, f: FileScan, _, _, _, _, _, _, _) => f ++ case CometBatchScanExec(BatchScanExec(_, f: FileScan, _, _, _, _, _, _, _), _) => f + } + assert(fileScan.nonEmpty) + assert(fileScan.get.partitionFilters.nonEmpty) +@@ -916,6 +919,7 @@ class FileBasedDataSourceSuite extends QueryTest + + val fileScan = df.queryExecution.executedPlan collectFirst { + case BatchScanExec(_, f: FileScan, _, _, _, _, _, _, _) => f ++ case CometBatchScanExec(BatchScanExec(_, f: FileScan, _, _, _, _, _, _, _), _) => f + } + assert(fileScan.nonEmpty) + assert(fileScan.get.partitionFilters.isEmpty) +@@ -1100,6 +1104,8 @@ class FileBasedDataSourceSuite extends QueryTest + val filters = df.queryExecution.executedPlan.collect { + case f: FileSourceScanLike => f.dataFilters + case b: BatchScanExec => b.scan.asInstanceOf[FileScan].dataFilters ++ case b: CometScanExec => b.dataFilters ++ case b: CometBatchScanExec => b.scan.asInstanceOf[FileScan].dataFilters + }.flatten + assert(filters.contains(GreaterThan(scan.logicalPlan.output.head, Literal(5L)))) + } +diff --git a/sql/core/src/test/scala/org/apache/spark/sql/IgnoreComet.scala b/sql/core/src/test/scala/org/apache/spark/sql/IgnoreComet.scala +new file mode 100644 +index 00000000000..4b31bea33de +--- /dev/null ++++ b/sql/core/src/test/scala/org/apache/spark/sql/IgnoreComet.scala +@@ -0,0 +1,42 @@ ++/* ++ * 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 ++ ++import org.scalactic.source.Position ++import org.scalatest.Tag ++ ++import org.apache.spark.sql.test.SQLTestUtils ++ ++/** ++ * Tests with this tag will be ignored when Comet is enabled (e.g., via `ENABLE_COMET`). ++ */ ++case class IgnoreComet(reason: String) extends Tag("DisableComet") ++ ++/** ++ * Helper trait that disables Comet for all tests regardless of default config values. ++ */ ++trait IgnoreCometSuite extends SQLTestUtils { ++ override protected def test(testName: String, testTags: Tag*)(testFun: => Any) ++ (implicit pos: Position): Unit = { ++ if (isCometEnabled) { ++ ignore(testName + " (disabled when Comet is on)", testTags: _*)(testFun) ++ } else { ++ super.test(testName, testTags: _*)(testFun) ++ } ++ } ++} +diff --git a/sql/core/src/test/scala/org/apache/spark/sql/JoinHintSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/JoinHintSuite.scala +index 1792b4c32eb..1616e6f39bd 100644 +--- a/sql/core/src/test/scala/org/apache/spark/sql/JoinHintSuite.scala ++++ b/sql/core/src/test/scala/org/apache/spark/sql/JoinHintSuite.scala +@@ -23,6 +23,7 @@ import org.apache.spark.sql.catalyst.optimizer.{BuildLeft, BuildRight, BuildSide + import org.apache.spark.sql.catalyst.plans.PlanTest + import org.apache.spark.sql.catalyst.plans.logical._ + import org.apache.spark.sql.catalyst.rules.RuleExecutor ++import org.apache.spark.sql.comet.{CometHashJoinExec, CometSortMergeJoinExec} + import org.apache.spark.sql.execution.adaptive.AdaptiveSparkPlanHelper + import org.apache.spark.sql.execution.joins._ + import org.apache.spark.sql.internal.SQLConf +@@ -362,6 +363,7 @@ class JoinHintSuite extends PlanTest with SharedSparkSession with AdaptiveSparkP + val executedPlan = df.queryExecution.executedPlan + val shuffleHashJoins = collect(executedPlan) { + case s: ShuffledHashJoinExec => s ++ case c: CometHashJoinExec => c.originalPlan.asInstanceOf[ShuffledHashJoinExec] + } + assert(shuffleHashJoins.size == 1) + assert(shuffleHashJoins.head.buildSide == buildSide) +@@ -371,6 +373,7 @@ class JoinHintSuite extends PlanTest with SharedSparkSession with AdaptiveSparkP + val executedPlan = df.queryExecution.executedPlan + val shuffleMergeJoins = collect(executedPlan) { + case s: SortMergeJoinExec => s ++ case c: CometSortMergeJoinExec => c + } + assert(shuffleMergeJoins.size == 1) + } +diff --git a/sql/core/src/test/scala/org/apache/spark/sql/JoinSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/JoinSuite.scala +index 7f062bfb899..400e939468f 100644 +--- a/sql/core/src/test/scala/org/apache/spark/sql/JoinSuite.scala ++++ b/sql/core/src/test/scala/org/apache/spark/sql/JoinSuite.scala +@@ -30,6 +30,7 @@ import org.apache.spark.sql.catalyst.TableIdentifier + import org.apache.spark.sql.catalyst.analysis.UnresolvedRelation + import org.apache.spark.sql.catalyst.expressions.{Ascending, GenericRow, SortOrder} + import org.apache.spark.sql.catalyst.plans.logical.Filter ++import org.apache.spark.sql.comet._ + import org.apache.spark.sql.execution.{BinaryExecNode, FilterExec, ProjectExec, SortExec, SparkPlan, WholeStageCodegenExec} + import org.apache.spark.sql.execution.adaptive.AdaptiveSparkPlanHelper + import org.apache.spark.sql.execution.exchange.{ShuffleExchangeExec, ShuffleExchangeLike} +@@ -740,7 +741,8 @@ class JoinSuite extends QueryTest with SharedSparkSession with AdaptiveSparkPlan + } + } + +- test("test SortMergeJoin (with spill)") { ++ test("test SortMergeJoin (with spill)", ++ IgnoreComet("TODO: Comet SMJ doesn't support spill yet")) { + withSQLConf(SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "1", + SQLConf.SORT_MERGE_JOIN_EXEC_BUFFER_IN_MEMORY_THRESHOLD.key -> "0", + SQLConf.SORT_MERGE_JOIN_EXEC_BUFFER_SPILL_THRESHOLD.key -> "1") { +@@ -1115,9 +1117,11 @@ class JoinSuite extends QueryTest with SharedSparkSession with AdaptiveSparkPlan + val plan = df1.join(df2.hint("SHUFFLE_HASH"), $"k1" === $"k2", joinType) + .groupBy($"k1").count() + .queryExecution.executedPlan +- assert(collect(plan) { case _: ShuffledHashJoinExec => true }.size === 1) ++ assert(collect(plan) { ++ case _: ShuffledHashJoinExec | _: CometHashJoinExec => true }.size === 1) + // No extra shuffle before aggregate +- assert(collect(plan) { case _: ShuffleExchangeExec => true }.size === 2) ++ assert(collect(plan) { ++ case _: ShuffleExchangeLike => true }.size === 2) + }) + } + +@@ -1134,10 +1138,11 @@ class JoinSuite extends QueryTest with SharedSparkSession with AdaptiveSparkPlan + .join(df4.hint("SHUFFLE_MERGE"), $"k1" === $"k4", joinType) + .queryExecution + .executedPlan +- assert(collect(plan) { case _: SortMergeJoinExec => true }.size === 2) ++ assert(collect(plan) { ++ case _: SortMergeJoinExec | _: CometSortMergeJoinExec => true }.size === 2) + assert(collect(plan) { case _: BroadcastHashJoinExec => true }.size === 1) + // No extra sort before last sort merge join +- assert(collect(plan) { case _: SortExec => true }.size === 3) ++ assert(collect(plan) { case _: SortExec | _: CometSortExec => true }.size === 3) + }) + + // Test shuffled hash join +@@ -1147,10 +1152,13 @@ class JoinSuite extends QueryTest with SharedSparkSession with AdaptiveSparkPlan + .join(df4.hint("SHUFFLE_MERGE"), $"k1" === $"k4", joinType) + .queryExecution + .executedPlan +- assert(collect(plan) { case _: SortMergeJoinExec => true }.size === 2) +- assert(collect(plan) { case _: ShuffledHashJoinExec => true }.size === 1) ++ assert(collect(plan) { ++ case _: SortMergeJoinExec | _: CometSortMergeJoinExec => true }.size === 2) ++ assert(collect(plan) { ++ case _: ShuffledHashJoinExec | _: CometHashJoinExec => true }.size === 1) + // No extra sort before last sort merge join +- assert(collect(plan) { case _: SortExec => true }.size === 3) ++ assert(collect(plan) { ++ case _: SortExec | _: CometSortExec => true }.size === 3) + }) + } + +@@ -1241,12 +1249,12 @@ class JoinSuite extends QueryTest with SharedSparkSession with AdaptiveSparkPlan + inputDFs.foreach { case (df1, df2, joinExprs) => + val smjDF = df1.join(df2.hint("SHUFFLE_MERGE"), joinExprs, "full") + assert(collect(smjDF.queryExecution.executedPlan) { +- case _: SortMergeJoinExec => true }.size === 1) ++ case _: SortMergeJoinExec | _: CometSortMergeJoinExec => true }.size === 1) + val smjResult = smjDF.collect() + + val shjDF = df1.join(df2.hint("SHUFFLE_HASH"), joinExprs, "full") + assert(collect(shjDF.queryExecution.executedPlan) { +- case _: ShuffledHashJoinExec => true }.size === 1) ++ case _: ShuffledHashJoinExec | _: CometHashJoinExec => true }.size === 1) + // Same result between shuffled hash join and sort merge join + checkAnswer(shjDF, smjResult) + } +@@ -1341,7 +1349,8 @@ class JoinSuite extends QueryTest with SharedSparkSession with AdaptiveSparkPlan + val plan = sql(getAggQuery(selectExpr, joinType)).queryExecution.executedPlan + assert(collect(plan) { case _: BroadcastNestedLoopJoinExec => true }.size === 1) + // Have shuffle before aggregation +- assert(collect(plan) { case _: ShuffleExchangeExec => true }.size === 1) ++ assert(collect(plan) { ++ case _: ShuffleExchangeLike => true }.size === 1) + } + + def getJoinQuery(selectExpr: String, joinType: String): String = { +@@ -1370,9 +1379,12 @@ class JoinSuite extends QueryTest with SharedSparkSession with AdaptiveSparkPlan + } + val plan = sql(getJoinQuery(selectExpr, joinType)).queryExecution.executedPlan + assert(collect(plan) { case _: BroadcastNestedLoopJoinExec => true }.size === 1) +- assert(collect(plan) { case _: SortMergeJoinExec => true }.size === 3) ++ assert(collect(plan) { ++ case _: SortMergeJoinExec => true ++ case _: CometSortMergeJoinExec => true ++ }.size === 3) + // No extra sort on left side before last sort merge join +- assert(collect(plan) { case _: SortExec => true }.size === 5) ++ assert(collect(plan) { case _: SortExec | _: CometSortExec => true }.size === 5) + } + + // Test output ordering is not preserved +@@ -1381,9 +1393,12 @@ class JoinSuite extends QueryTest with SharedSparkSession with AdaptiveSparkPlan + val selectExpr = "/*+ BROADCAST(left_t) */ k1 as k0" + val plan = sql(getJoinQuery(selectExpr, joinType)).queryExecution.executedPlan + assert(collect(plan) { case _: BroadcastNestedLoopJoinExec => true }.size === 1) +- assert(collect(plan) { case _: SortMergeJoinExec => true }.size === 3) ++ assert(collect(plan) { ++ case _: SortMergeJoinExec => true ++ case _: CometSortMergeJoinExec => true ++ }.size === 3) + // Have sort on left side before last sort merge join +- assert(collect(plan) { case _: SortExec => true }.size === 6) ++ assert(collect(plan) { case _: SortExec | _: CometSortExec => true }.size === 6) + } + + // Test singe partition +@@ -1393,7 +1408,8 @@ class JoinSuite extends QueryTest with SharedSparkSession with AdaptiveSparkPlan + |FROM range(0, 10, 1, 1) t1 FULL OUTER JOIN range(0, 10, 1, 1) t2 + |""".stripMargin) + val plan = fullJoinDF.queryExecution.executedPlan +- assert(collect(plan) { case _: ShuffleExchangeExec => true}.size == 1) ++ assert(collect(plan) { ++ case _: ShuffleExchangeLike => true}.size == 1) + checkAnswer(fullJoinDF, Row(100)) + } + } +diff --git a/sql/core/src/test/scala/org/apache/spark/sql/PlanStabilitySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/PlanStabilitySuite.scala +index b5b34922694..a72403780c4 100644 +--- a/sql/core/src/test/scala/org/apache/spark/sql/PlanStabilitySuite.scala ++++ b/sql/core/src/test/scala/org/apache/spark/sql/PlanStabilitySuite.scala +@@ -69,7 +69,7 @@ import org.apache.spark.tags.ExtendedSQLTest + * }}} + */ + // scalastyle:on line.size.limit +-trait PlanStabilitySuite extends DisableAdaptiveExecutionSuite { ++trait PlanStabilitySuite extends DisableAdaptiveExecutionSuite with IgnoreCometSuite { + + protected val baseResourcePath = { + // use the same way as `SQLQueryTestSuite` to get the resource path +diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala +index 525d97e4998..8a3e7457618 100644 +--- a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala ++++ b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala +@@ -1508,7 +1508,8 @@ class SQLQuerySuite extends QueryTest with SharedSparkSession with AdaptiveSpark + checkAnswer(sql("select -0.001"), Row(BigDecimal("-0.001"))) + } + +- test("external sorting updates peak execution memory") { ++ test("external sorting updates peak execution memory", ++ IgnoreComet("TODO: native CometSort does not update peak execution memory")) { + AccumulatorSuite.verifyPeakExecutionMemorySet(sparkContext, "external sort") { + sql("SELECT * FROM testData2 ORDER BY a ASC, b ASC").collect() + } +diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SubquerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SubquerySuite.scala +index 75eabcb96f2..36e3318ad7e 100644 +--- a/sql/core/src/test/scala/org/apache/spark/sql/SubquerySuite.scala ++++ b/sql/core/src/test/scala/org/apache/spark/sql/SubquerySuite.scala +@@ -21,10 +21,11 @@ import scala.collection.mutable.ArrayBuffer + + import org.apache.spark.sql.catalyst.expressions.SubqueryExpression + import org.apache.spark.sql.catalyst.plans.logical.{Aggregate, Join, LogicalPlan, Project, Sort, Union} ++import org.apache.spark.sql.comet.CometScanExec + import org.apache.spark.sql.execution._ + import org.apache.spark.sql.execution.adaptive.{AdaptiveSparkPlanHelper, DisableAdaptiveExecution} + import org.apache.spark.sql.execution.datasources.FileScanRDD +-import org.apache.spark.sql.execution.exchange.ShuffleExchangeExec ++import org.apache.spark.sql.execution.exchange.ShuffleExchangeLike + import org.apache.spark.sql.execution.joins.{BaseJoinExec, BroadcastHashJoinExec, BroadcastNestedLoopJoinExec} + import org.apache.spark.sql.internal.SQLConf + import org.apache.spark.sql.test.SharedSparkSession +@@ -1543,6 +1544,12 @@ class SubquerySuite extends QueryTest + fs.inputRDDs().forall( + _.asInstanceOf[FileScanRDD].filePartitions.forall( + _.files.forall(_.urlEncodedPath.contains("p=0")))) ++ case WholeStageCodegenExec(ColumnarToRowExec(InputAdapter( ++ fs @ CometScanExec(_, _, _, partitionFilters, _, _, _, _, _, _)))) => ++ partitionFilters.exists(ExecSubqueryExpression.hasSubquery) && ++ fs.inputRDDs().forall( ++ _.asInstanceOf[FileScanRDD].filePartitions.forall( ++ _.files.forall(_.urlEncodedPath.contains("p=0")))) + case _ => false + }) + } +@@ -2108,7 +2115,7 @@ class SubquerySuite extends QueryTest + + df.collect() + val exchanges = collect(df.queryExecution.executedPlan) { +- case s: ShuffleExchangeExec => s ++ case s: ShuffleExchangeLike => s + } + assert(exchanges.size === 1) + } +diff --git a/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2Suite.scala b/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2Suite.scala +index 02990a7a40d..bddf5e1ccc2 100644 +--- a/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2Suite.scala ++++ b/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2Suite.scala +@@ -24,6 +24,7 @@ import test.org.apache.spark.sql.connector._ + + import org.apache.spark.sql.{AnalysisException, DataFrame, QueryTest, Row} + import org.apache.spark.sql.catalyst.InternalRow ++import org.apache.spark.sql.comet.CometSortExec + import org.apache.spark.sql.connector.catalog.{PartitionInternalRow, SupportsRead, Table, TableCapability, TableProvider} + import org.apache.spark.sql.connector.catalog.TableCapability._ + import org.apache.spark.sql.connector.expressions.{Expression, FieldReference, Literal, NamedReference, NullOrdering, SortDirection, SortOrder, Transform} +@@ -33,7 +34,7 @@ import org.apache.spark.sql.connector.read.partitioning.{KeyGroupedPartitioning, + import org.apache.spark.sql.execution.SortExec + import org.apache.spark.sql.execution.adaptive.AdaptiveSparkPlanHelper + import org.apache.spark.sql.execution.datasources.v2.{BatchScanExec, DataSourceV2Relation, DataSourceV2ScanRelation} +-import org.apache.spark.sql.execution.exchange.{Exchange, ShuffleExchangeExec} ++import org.apache.spark.sql.execution.exchange.{Exchange, ShuffleExchangeExec, ShuffleExchangeLike} + import org.apache.spark.sql.execution.vectorized.OnHeapColumnVector + import org.apache.spark.sql.expressions.Window + import org.apache.spark.sql.functions._ +@@ -268,13 +269,13 @@ class DataSourceV2Suite extends QueryTest with SharedSparkSession with AdaptiveS + val groupByColJ = df.groupBy($"j").agg(sum($"i")) + checkAnswer(groupByColJ, Seq(Row(2, 8), Row(4, 2), Row(6, 5))) + assert(collectFirst(groupByColJ.queryExecution.executedPlan) { +- case e: ShuffleExchangeExec => e ++ case e: ShuffleExchangeLike => e + }.isDefined) + + val groupByIPlusJ = df.groupBy($"i" + $"j").agg(count("*")) + checkAnswer(groupByIPlusJ, Seq(Row(5, 2), Row(6, 2), Row(8, 1), Row(9, 1))) + assert(collectFirst(groupByIPlusJ.queryExecution.executedPlan) { +- case e: ShuffleExchangeExec => e ++ case e: ShuffleExchangeLike => e + }.isDefined) + } + } +@@ -334,10 +335,11 @@ class DataSourceV2Suite extends QueryTest with SharedSparkSession with AdaptiveS + + val (shuffleExpected, sortExpected) = groupByExpects + assert(collectFirst(groupBy.queryExecution.executedPlan) { +- case e: ShuffleExchangeExec => e ++ case e: ShuffleExchangeLike => e + }.isDefined === shuffleExpected) + assert(collectFirst(groupBy.queryExecution.executedPlan) { + case e: SortExec => e ++ case c: CometSortExec => c + }.isDefined === sortExpected) + } + +@@ -352,10 +354,11 @@ class DataSourceV2Suite extends QueryTest with SharedSparkSession with AdaptiveS + + val (shuffleExpected, sortExpected) = windowFuncExpects + assert(collectFirst(windowPartByColIOrderByColJ.queryExecution.executedPlan) { +- case e: ShuffleExchangeExec => e ++ case e: ShuffleExchangeLike => e + }.isDefined === shuffleExpected) + assert(collectFirst(windowPartByColIOrderByColJ.queryExecution.executedPlan) { + case e: SortExec => e ++ case c: CometSortExec => c + }.isDefined === sortExpected) + } + } +diff --git a/sql/core/src/test/scala/org/apache/spark/sql/connector/FileDataSourceV2FallBackSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/connector/FileDataSourceV2FallBackSuite.scala +index cfc8b2cc845..c6fcfd7bd08 100644 +--- a/sql/core/src/test/scala/org/apache/spark/sql/connector/FileDataSourceV2FallBackSuite.scala ++++ b/sql/core/src/test/scala/org/apache/spark/sql/connector/FileDataSourceV2FallBackSuite.scala +@@ -21,6 +21,7 @@ import scala.collection.mutable.ArrayBuffer + import org.apache.spark.SparkConf + import org.apache.spark.sql.{AnalysisException, QueryTest} + import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan ++import org.apache.spark.sql.comet.CometScanExec + import org.apache.spark.sql.connector.catalog.{SupportsRead, SupportsWrite, Table, TableCapability} + import org.apache.spark.sql.connector.read.ScanBuilder + import org.apache.spark.sql.connector.write.{LogicalWriteInfo, WriteBuilder} +@@ -184,7 +185,11 @@ class FileDataSourceV2FallBackSuite extends QueryTest with SharedSparkSession { + val df = spark.read.format(format).load(path.getCanonicalPath) + checkAnswer(df, inputData.toDF()) + assert( +- df.queryExecution.executedPlan.exists(_.isInstanceOf[FileSourceScanExec])) ++ df.queryExecution.executedPlan.exists { ++ case _: FileSourceScanExec | _: CometScanExec => true ++ case _ => false ++ } ++ ) + } + } finally { + spark.listenerManager.unregister(listener) +diff --git a/sql/core/src/test/scala/org/apache/spark/sql/connector/KeyGroupedPartitioningSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/connector/KeyGroupedPartitioningSuite.scala +index cf76f6ca32c..f454128af06 100644 +--- a/sql/core/src/test/scala/org/apache/spark/sql/connector/KeyGroupedPartitioningSuite.scala ++++ b/sql/core/src/test/scala/org/apache/spark/sql/connector/KeyGroupedPartitioningSuite.scala +@@ -22,6 +22,7 @@ import org.apache.spark.sql.{DataFrame, Row} + import org.apache.spark.sql.catalyst.InternalRow + import org.apache.spark.sql.catalyst.expressions.{Literal, TransformExpression} + import org.apache.spark.sql.catalyst.plans.physical ++import org.apache.spark.sql.comet.CometSortMergeJoinExec + import org.apache.spark.sql.connector.catalog.Identifier + import org.apache.spark.sql.connector.catalog.InMemoryTableCatalog + import org.apache.spark.sql.connector.catalog.functions._ +@@ -31,7 +32,7 @@ import org.apache.spark.sql.connector.expressions.Expressions._ + import org.apache.spark.sql.execution.SparkPlan + import org.apache.spark.sql.execution.datasources.v2.BatchScanExec + import org.apache.spark.sql.execution.datasources.v2.DataSourceV2ScanRelation +-import org.apache.spark.sql.execution.exchange.ShuffleExchangeExec ++import org.apache.spark.sql.execution.exchange.ShuffleExchangeLike + import org.apache.spark.sql.execution.joins.SortMergeJoinExec + import org.apache.spark.sql.internal.SQLConf + import org.apache.spark.sql.internal.SQLConf._ +@@ -279,13 +280,14 @@ class KeyGroupedPartitioningSuite extends DistributionAndOrderingSuiteBase { + Row("bbb", 20, 250.0), Row("bbb", 20, 350.0), Row("ccc", 30, 400.50))) + } + +- private def collectShuffles(plan: SparkPlan): Seq[ShuffleExchangeExec] = { ++ private def collectShuffles(plan: SparkPlan): Seq[ShuffleExchangeLike] = { + // here we skip collecting shuffle operators that are not associated with SMJ + collect(plan) { + case s: SortMergeJoinExec => s ++ case c: CometSortMergeJoinExec => c.originalPlan + }.flatMap(smj => + collect(smj) { +- case s: ShuffleExchangeExec => s ++ case s: ShuffleExchangeLike => s + }) + } + +diff --git a/sql/core/src/test/scala/org/apache/spark/sql/errors/QueryExecutionErrorsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/errors/QueryExecutionErrorsSuite.scala +index c0ec8a58bd5..4e8bc6ed3c5 100644 +--- a/sql/core/src/test/scala/org/apache/spark/sql/errors/QueryExecutionErrorsSuite.scala ++++ b/sql/core/src/test/scala/org/apache/spark/sql/errors/QueryExecutionErrorsSuite.scala +@@ -27,7 +27,7 @@ import org.apache.hadoop.fs.permission.FsPermission + import org.mockito.Mockito.{mock, spy, when} + + import org.apache.spark._ +-import org.apache.spark.sql.{AnalysisException, DataFrame, Dataset, QueryTest, Row, SaveMode} ++import org.apache.spark.sql.{AnalysisException, DataFrame, Dataset, IgnoreComet, QueryTest, Row, SaveMode} + import org.apache.spark.sql.catalyst.expressions.CodegenObjectFactoryMode._ + import org.apache.spark.sql.catalyst.util.BadRecordException + import org.apache.spark.sql.execution.datasources.jdbc.{DriverRegistry, JDBCOptions} +@@ -248,7 +248,8 @@ class QueryExecutionErrorsSuite + } + + test("INCONSISTENT_BEHAVIOR_CROSS_VERSION: " + +- "compatibility with Spark 2.4/3.2 in reading/writing dates") { ++ "compatibility with Spark 2.4/3.2 in reading/writing dates", ++ IgnoreComet("Comet doesn't completely support datetime rebase mode yet")) { + + // Fail to read ancient datetime values. + withSQLConf(SQLConf.PARQUET_REBASE_MODE_IN_READ.key -> EXCEPTION.toString) { +diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/DataSourceScanExecRedactionSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/DataSourceScanExecRedactionSuite.scala +index 418ca3430bb..eb8267192f8 100644 +--- a/sql/core/src/test/scala/org/apache/spark/sql/execution/DataSourceScanExecRedactionSuite.scala ++++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/DataSourceScanExecRedactionSuite.scala +@@ -23,7 +23,7 @@ import scala.util.Random + import org.apache.hadoop.fs.Path + + import org.apache.spark.SparkConf +-import org.apache.spark.sql.{DataFrame, QueryTest} ++import org.apache.spark.sql.{DataFrame, IgnoreComet, QueryTest} + import org.apache.spark.sql.execution.datasources.v2.BatchScanExec + import org.apache.spark.sql.execution.datasources.v2.orc.OrcScan + import org.apache.spark.sql.internal.SQLConf +@@ -195,7 +195,7 @@ class DataSourceV2ScanExecRedactionSuite extends DataSourceScanRedactionTest { + } + } + +- test("FileScan description") { ++ test("FileScan description", IgnoreComet("Comet doesn't use BatchScan")) { + Seq("json", "orc", "parquet").foreach { format => + withTempPath { path => + val dir = path.getCanonicalPath +diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/LogicalPlanTagInSparkPlanSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/LogicalPlanTagInSparkPlanSuite.scala +index 743ec41dbe7..9f30d6c8e04 100644 +--- a/sql/core/src/test/scala/org/apache/spark/sql/execution/LogicalPlanTagInSparkPlanSuite.scala ++++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/LogicalPlanTagInSparkPlanSuite.scala +@@ -53,6 +53,10 @@ class LogicalPlanTagInSparkPlanSuite extends TPCDSQuerySuite with DisableAdaptiv + case ColumnarToRowExec(i: InputAdapter) => isScanPlanTree(i.child) + case p: ProjectExec => isScanPlanTree(p.child) + case f: FilterExec => isScanPlanTree(f.child) ++ // Comet produces scan plan tree like: ++ // ColumnarToRow ++ // +- ReusedExchange ++ case _: ReusedExchangeExec => false + case _: LeafExecNode => true + case _ => false + } +diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/PlannerSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/PlannerSuite.scala +index 4b3d3a4b805..56e1e0e6f16 100644 +--- a/sql/core/src/test/scala/org/apache/spark/sql/execution/PlannerSuite.scala ++++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/PlannerSuite.scala +@@ -18,7 +18,7 @@ + package org.apache.spark.sql.execution + + import org.apache.spark.rdd.RDD +-import org.apache.spark.sql.{execution, DataFrame, Row} ++import org.apache.spark.sql.{execution, DataFrame, IgnoreCometSuite, Row} + import org.apache.spark.sql.catalyst.InternalRow + import org.apache.spark.sql.catalyst.expressions._ + import org.apache.spark.sql.catalyst.plans._ +@@ -35,7 +35,9 @@ import org.apache.spark.sql.internal.SQLConf + import org.apache.spark.sql.test.SharedSparkSession + import org.apache.spark.sql.types._ + +-class PlannerSuite extends SharedSparkSession with AdaptiveSparkPlanHelper { ++// Ignore this suite when Comet is enabled. This suite tests the Spark planner and Comet planner ++// comes out with too many difference. Simply ignoring this suite for now. ++class PlannerSuite extends SharedSparkSession with AdaptiveSparkPlanHelper with IgnoreCometSuite { + import testImplicits._ + + setupTestData() +diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/RemoveRedundantProjectsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/RemoveRedundantProjectsSuite.scala +index 9e9d717db3b..91a4f9a38d5 100644 +--- a/sql/core/src/test/scala/org/apache/spark/sql/execution/RemoveRedundantProjectsSuite.scala ++++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/RemoveRedundantProjectsSuite.scala +@@ -18,6 +18,7 @@ + package org.apache.spark.sql.execution + + import org.apache.spark.sql.{DataFrame, QueryTest, Row} ++import org.apache.spark.sql.comet.CometProjectExec + import org.apache.spark.sql.connector.SimpleWritableDataSource + import org.apache.spark.sql.execution.adaptive.{AdaptiveSparkPlanHelper, DisableAdaptiveExecutionSuite, EnableAdaptiveExecutionSuite} + import org.apache.spark.sql.internal.SQLConf +@@ -34,7 +35,10 @@ abstract class RemoveRedundantProjectsSuiteBase + private def assertProjectExecCount(df: DataFrame, expected: Int): Unit = { + withClue(df.queryExecution) { + val plan = df.queryExecution.executedPlan +- val actual = collectWithSubqueries(plan) { case p: ProjectExec => p }.size ++ val actual = collectWithSubqueries(plan) { ++ case p: ProjectExec => p ++ case p: CometProjectExec => p ++ }.size + assert(actual == expected) + } + } +diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/RemoveRedundantSortsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/RemoveRedundantSortsSuite.scala +index 30ce940b032..0d3f6c6c934 100644 +--- a/sql/core/src/test/scala/org/apache/spark/sql/execution/RemoveRedundantSortsSuite.scala ++++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/RemoveRedundantSortsSuite.scala +@@ -19,6 +19,7 @@ package org.apache.spark.sql.execution + + import org.apache.spark.sql.{DataFrame, QueryTest} + import org.apache.spark.sql.catalyst.plans.physical.{RangePartitioning, UnknownPartitioning} ++import org.apache.spark.sql.comet.CometSortExec + import org.apache.spark.sql.execution.adaptive.{AdaptiveSparkPlanHelper, DisableAdaptiveExecutionSuite, EnableAdaptiveExecutionSuite} + import org.apache.spark.sql.execution.joins.ShuffledJoin + import org.apache.spark.sql.internal.SQLConf +@@ -33,7 +34,7 @@ abstract class RemoveRedundantSortsSuiteBase + + private def checkNumSorts(df: DataFrame, count: Int): Unit = { + val plan = df.queryExecution.executedPlan +- assert(collectWithSubqueries(plan) { case s: SortExec => s }.length == count) ++ assert(collectWithSubqueries(plan) { case _: SortExec | _: CometSortExec => 1 }.length == count) + } + + private def checkSorts(query: String, enabledCount: Int, disabledCount: Int): Unit = { +diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/ReplaceHashWithSortAggSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/ReplaceHashWithSortAggSuite.scala +index 47679ed7865..9ffbaecb98e 100644 +--- a/sql/core/src/test/scala/org/apache/spark/sql/execution/ReplaceHashWithSortAggSuite.scala ++++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/ReplaceHashWithSortAggSuite.scala +@@ -18,6 +18,7 @@ + package org.apache.spark.sql.execution + + import org.apache.spark.sql.{DataFrame, QueryTest} ++import org.apache.spark.sql.comet.CometHashAggregateExec + import org.apache.spark.sql.execution.adaptive.{AdaptiveSparkPlanHelper, DisableAdaptiveExecutionSuite, EnableAdaptiveExecutionSuite} + import org.apache.spark.sql.execution.aggregate.{HashAggregateExec, ObjectHashAggregateExec, SortAggregateExec} + import org.apache.spark.sql.internal.SQLConf +@@ -31,7 +32,7 @@ abstract class ReplaceHashWithSortAggSuiteBase + private def checkNumAggs(df: DataFrame, hashAggCount: Int, sortAggCount: Int): Unit = { + val plan = df.queryExecution.executedPlan + assert(collectWithSubqueries(plan) { +- case s @ (_: HashAggregateExec | _: ObjectHashAggregateExec) => s ++ case s @ (_: HashAggregateExec | _: ObjectHashAggregateExec | _: CometHashAggregateExec ) => s + }.length == hashAggCount) + assert(collectWithSubqueries(plan) { case s: SortAggregateExec => s }.length == sortAggCount) + } +diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/WholeStageCodegenSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/WholeStageCodegenSuite.scala +index ac710c32296..e163c1a6a76 100644 +--- a/sql/core/src/test/scala/org/apache/spark/sql/execution/WholeStageCodegenSuite.scala ++++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/WholeStageCodegenSuite.scala +@@ -19,6 +19,7 @@ package org.apache.spark.sql.execution + + import org.apache.spark.sql.{Dataset, QueryTest, Row, SaveMode} + import org.apache.spark.sql.catalyst.expressions.codegen.{ByteCodeStats, CodeAndComment, CodeGenerator} ++import org.apache.spark.sql.comet.CometSortMergeJoinExec + import org.apache.spark.sql.execution.adaptive.DisableAdaptiveExecutionSuite + import org.apache.spark.sql.execution.aggregate.{HashAggregateExec, SortAggregateExec} + import org.apache.spark.sql.execution.columnar.InMemoryTableScanExec +@@ -224,6 +225,7 @@ class WholeStageCodegenSuite extends QueryTest with SharedSparkSession + assert(twoJoinsDF.queryExecution.executedPlan.collect { + case WholeStageCodegenExec(_ : ShuffledHashJoinExec) if hint == "SHUFFLE_HASH" => true + case WholeStageCodegenExec(_ : SortMergeJoinExec) if hint == "SHUFFLE_MERGE" => true ++ case _: CometSortMergeJoinExec if hint == "SHUFFLE_MERGE" => true + }.size === 2) + checkAnswer(twoJoinsDF, + Seq(Row(0, 0, 0), Row(1, 1, null), Row(2, 2, 2), Row(3, 3, null), Row(4, 4, null), +@@ -258,6 +260,7 @@ class WholeStageCodegenSuite extends QueryTest with SharedSparkSession + .join(df1.hint("SHUFFLE_MERGE"), $"k3" === $"k1", "right_outer") + assert(twoJoinsDF.queryExecution.executedPlan.collect { + case WholeStageCodegenExec(_ : SortMergeJoinExec) => true ++ case _: CometSortMergeJoinExec => true + }.size === 2) + checkAnswer(twoJoinsDF, + Seq(Row(0, 0, 0), Row(1, 1, 1), Row(2, 2, 2), Row(3, 3, 3), Row(4, null, 4), Row(5, null, 5), +@@ -280,8 +283,7 @@ class WholeStageCodegenSuite extends QueryTest with SharedSparkSession + val twoJoinsDF = df3.join(df2.hint("SHUFFLE_MERGE"), $"k3" === $"k2", "left_semi") + .join(df1.hint("SHUFFLE_MERGE"), $"k3" === $"k1", "left_semi") + assert(twoJoinsDF.queryExecution.executedPlan.collect { +- case WholeStageCodegenExec(ProjectExec(_, _ : SortMergeJoinExec)) | +- WholeStageCodegenExec(_ : SortMergeJoinExec) => true ++ case _: SortMergeJoinExec => true + }.size === 2) + checkAnswer(twoJoinsDF, Seq(Row(0), Row(1), Row(2), Row(3))) + } +@@ -302,8 +304,7 @@ class WholeStageCodegenSuite extends QueryTest with SharedSparkSession + val twoJoinsDF = df1.join(df2.hint("SHUFFLE_MERGE"), $"k1" === $"k2", "left_anti") + .join(df3.hint("SHUFFLE_MERGE"), $"k1" === $"k3", "left_anti") + assert(twoJoinsDF.queryExecution.executedPlan.collect { +- case WholeStageCodegenExec(ProjectExec(_, _ : SortMergeJoinExec)) | +- WholeStageCodegenExec(_ : SortMergeJoinExec) => true ++ case _: SortMergeJoinExec => true + }.size === 2) + checkAnswer(twoJoinsDF, Seq(Row(6), Row(7), Row(8), Row(9))) + } +@@ -436,7 +437,9 @@ class WholeStageCodegenSuite extends QueryTest with SharedSparkSession + val plan = df.queryExecution.executedPlan + assert(plan.exists(p => + p.isInstanceOf[WholeStageCodegenExec] && +- p.asInstanceOf[WholeStageCodegenExec].child.isInstanceOf[SortExec])) ++ p.asInstanceOf[WholeStageCodegenExec].collect { ++ case _: SortExec => true ++ }.nonEmpty)) + assert(df.collect() === Array(Row(1), Row(2), Row(3))) + } + +@@ -616,7 +619,9 @@ class WholeStageCodegenSuite extends QueryTest with SharedSparkSession + .write.mode(SaveMode.Overwrite).parquet(path) + + withSQLConf(SQLConf.WHOLESTAGE_MAX_NUM_FIELDS.key -> "255", +- SQLConf.WHOLESTAGE_SPLIT_CONSUME_FUNC_BY_OPERATOR.key -> "true") { ++ SQLConf.WHOLESTAGE_SPLIT_CONSUME_FUNC_BY_OPERATOR.key -> "true", ++ // Disable Comet native execution because this checks wholestage codegen. ++ "spark.comet.exec.enabled" -> "false") { + val projection = Seq.tabulate(columnNum)(i => s"c$i + c$i as newC$i") + val df = spark.read.parquet(path).selectExpr(projection: _*) + +diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/adaptive/AdaptiveQueryExecSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/adaptive/AdaptiveQueryExecSuite.scala +index 593bd7bb4ba..7ad55e3ab20 100644 +--- a/sql/core/src/test/scala/org/apache/spark/sql/execution/adaptive/AdaptiveQueryExecSuite.scala ++++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/adaptive/AdaptiveQueryExecSuite.scala +@@ -26,9 +26,11 @@ import org.scalatest.time.SpanSugar._ + + import org.apache.spark.SparkException + import org.apache.spark.scheduler.{SparkListener, SparkListenerEvent, SparkListenerJobStart} +-import org.apache.spark.sql.{Dataset, QueryTest, Row, SparkSession, Strategy} ++import org.apache.spark.sql.{Dataset, IgnoreComet, QueryTest, Row, SparkSession, Strategy} + import org.apache.spark.sql.catalyst.optimizer.{BuildLeft, BuildRight} + import org.apache.spark.sql.catalyst.plans.logical.{Aggregate, LogicalPlan} ++import org.apache.spark.sql.comet._ ++import org.apache.spark.sql.comet.execution.shuffle.CometShuffleExchangeExec + import org.apache.spark.sql.execution.{CollectLimitExec, LocalTableScanExec, PartialReducerPartitionSpec, QueryExecution, ReusedSubqueryExec, ShuffledRowRDD, SortExec, SparkPlan, SparkPlanInfo, UnionExec} + import org.apache.spark.sql.execution.aggregate.BaseAggregateExec + import org.apache.spark.sql.execution.command.DataWritingCommandExec +@@ -104,6 +106,7 @@ class AdaptiveQueryExecSuite + private def findTopLevelBroadcastHashJoin(plan: SparkPlan): Seq[BroadcastHashJoinExec] = { + collect(plan) { + case j: BroadcastHashJoinExec => j ++ case j: CometBroadcastHashJoinExec => j.originalPlan.asInstanceOf[BroadcastHashJoinExec] + } + } + +@@ -116,30 +119,38 @@ class AdaptiveQueryExecSuite + private def findTopLevelSortMergeJoin(plan: SparkPlan): Seq[SortMergeJoinExec] = { + collect(plan) { + case j: SortMergeJoinExec => j ++ case j: CometSortMergeJoinExec => ++ assert(j.originalPlan.isInstanceOf[SortMergeJoinExec]) ++ j.originalPlan.asInstanceOf[SortMergeJoinExec] + } + } + + private def findTopLevelShuffledHashJoin(plan: SparkPlan): Seq[ShuffledHashJoinExec] = { + collect(plan) { + case j: ShuffledHashJoinExec => j ++ case j: CometHashJoinExec => j.originalPlan.asInstanceOf[ShuffledHashJoinExec] + } + } + + private def findTopLevelBaseJoin(plan: SparkPlan): Seq[BaseJoinExec] = { + collect(plan) { + case j: BaseJoinExec => j ++ case c: CometHashJoinExec => c.originalPlan.asInstanceOf[BaseJoinExec] ++ case c: CometSortMergeJoinExec => c.originalPlan.asInstanceOf[BaseJoinExec] + } + } + + private def findTopLevelSort(plan: SparkPlan): Seq[SortExec] = { + collect(plan) { + case s: SortExec => s ++ case s: CometSortExec => s.originalPlan.asInstanceOf[SortExec] + } + } + + private def findTopLevelAggregate(plan: SparkPlan): Seq[BaseAggregateExec] = { + collect(plan) { + case agg: BaseAggregateExec => agg ++ case agg: CometHashAggregateExec => agg.originalPlan.asInstanceOf[BaseAggregateExec] + } + } + +@@ -176,6 +187,7 @@ class AdaptiveQueryExecSuite + val parts = rdd.partitions + assert(parts.forall(rdd.preferredLocations(_).nonEmpty)) + } ++ + assert(numShuffles === (numLocalReads.length + numShufflesWithoutLocalRead)) + } + +@@ -184,7 +196,7 @@ class AdaptiveQueryExecSuite + val plan = df.queryExecution.executedPlan + assert(plan.isInstanceOf[AdaptiveSparkPlanExec]) + val shuffle = plan.asInstanceOf[AdaptiveSparkPlanExec].executedPlan.collect { +- case s: ShuffleExchangeExec => s ++ case s: ShuffleExchangeLike => s + } + assert(shuffle.size == 1) + assert(shuffle(0).outputPartitioning.numPartitions == numPartition) +@@ -200,7 +212,8 @@ class AdaptiveQueryExecSuite + assert(smj.size == 1) + val bhj = findTopLevelBroadcastHashJoin(adaptivePlan) + assert(bhj.size == 1) +- checkNumLocalShuffleReads(adaptivePlan) ++ // Comet shuffle changes shuffle metrics ++ // checkNumLocalShuffleReads(adaptivePlan) + } + } + +@@ -227,7 +240,8 @@ class AdaptiveQueryExecSuite + } + } + +- test("Reuse the parallelism of coalesced shuffle in local shuffle read") { ++ test("Reuse the parallelism of coalesced shuffle in local shuffle read", ++ IgnoreComet("Comet shuffle changes shuffle partition size")) { + withSQLConf( + SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "true", + SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "80", +@@ -259,7 +273,8 @@ class AdaptiveQueryExecSuite + } + } + +- test("Reuse the default parallelism in local shuffle read") { ++ test("Reuse the default parallelism in local shuffle read", ++ IgnoreComet("Comet shuffle changes shuffle partition size")) { + withSQLConf( + SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "true", + SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "80", +@@ -273,7 +288,8 @@ class AdaptiveQueryExecSuite + val localReads = collect(adaptivePlan) { + case read: AQEShuffleReadExec if read.isLocalRead => read + } +- assert(localReads.length == 2) ++ // Comet shuffle changes shuffle metrics ++ assert(localReads.length == 1) + val localShuffleRDD0 = localReads(0).execute().asInstanceOf[ShuffledRowRDD] + val localShuffleRDD1 = localReads(1).execute().asInstanceOf[ShuffledRowRDD] + // the final parallelism is math.max(1, numReduces / numMappers): math.max(1, 5/2) = 2 +@@ -322,7 +338,7 @@ class AdaptiveQueryExecSuite + } + } + +- test("Scalar subquery") { ++ test("Scalar subquery", IgnoreComet("Comet shuffle changes shuffle metrics")) { + withSQLConf( + SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "true", + SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "80") { +@@ -337,7 +353,7 @@ class AdaptiveQueryExecSuite + } + } + +- test("Scalar subquery in later stages") { ++ test("Scalar subquery in later stages", IgnoreComet("Comet shuffle changes shuffle metrics")) { + withSQLConf( + SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "true", + SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "80") { +@@ -353,7 +369,7 @@ class AdaptiveQueryExecSuite + } + } + +- test("multiple joins") { ++ test("multiple joins", IgnoreComet("Comet shuffle changes shuffle metrics")) { + withSQLConf( + SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "true", + SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "80") { +@@ -398,7 +414,7 @@ class AdaptiveQueryExecSuite + } + } + +- test("multiple joins with aggregate") { ++ test("multiple joins with aggregate", IgnoreComet("Comet shuffle changes shuffle metrics")) { + withSQLConf( + SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "true", + SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "80") { +@@ -443,7 +459,7 @@ class AdaptiveQueryExecSuite + } + } + +- test("multiple joins with aggregate 2") { ++ test("multiple joins with aggregate 2", IgnoreComet("Comet shuffle changes shuffle metrics")) { + withSQLConf( + SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "true", + SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "500") { +@@ -508,7 +524,7 @@ class AdaptiveQueryExecSuite + } + } + +- test("Exchange reuse with subqueries") { ++ test("Exchange reuse with subqueries", IgnoreComet("Comet shuffle changes shuffle metrics")) { + withSQLConf( + SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "true", + SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "80") { +@@ -539,7 +555,9 @@ class AdaptiveQueryExecSuite + assert(smj.size == 1) + val bhj = findTopLevelBroadcastHashJoin(adaptivePlan) + assert(bhj.size == 1) +- checkNumLocalShuffleReads(adaptivePlan) ++ // Comet shuffle changes shuffle metrics, ++ // so we can't check the number of local shuffle reads. ++ // checkNumLocalShuffleReads(adaptivePlan) + // Even with local shuffle read, the query stage reuse can also work. + val ex = findReusedExchange(adaptivePlan) + assert(ex.nonEmpty) +@@ -560,7 +578,9 @@ class AdaptiveQueryExecSuite + assert(smj.size == 1) + val bhj = findTopLevelBroadcastHashJoin(adaptivePlan) + assert(bhj.size == 1) +- checkNumLocalShuffleReads(adaptivePlan) ++ // Comet shuffle changes shuffle metrics, ++ // so we can't check the number of local shuffle reads. ++ // checkNumLocalShuffleReads(adaptivePlan) + // Even with local shuffle read, the query stage reuse can also work. + val ex = findReusedExchange(adaptivePlan) + assert(ex.isEmpty) +@@ -569,7 +589,8 @@ class AdaptiveQueryExecSuite + } + } + +- test("Broadcast exchange reuse across subqueries") { ++ test("Broadcast exchange reuse across subqueries", ++ IgnoreComet("Comet shuffle changes shuffle metrics")) { + withSQLConf( + SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "true", + SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "20000000", +@@ -664,7 +685,8 @@ class AdaptiveQueryExecSuite + val bhj = findTopLevelBroadcastHashJoin(adaptivePlan) + assert(bhj.size == 1) + // There is still a SMJ, and its two shuffles can't apply local read. +- checkNumLocalShuffleReads(adaptivePlan, 2) ++ // Comet shuffle changes shuffle metrics ++ // checkNumLocalShuffleReads(adaptivePlan, 2) + } + } + +@@ -786,7 +808,8 @@ class AdaptiveQueryExecSuite + } + } + +- test("SPARK-29544: adaptive skew join with different join types") { ++ test("SPARK-29544: adaptive skew join with different join types", ++ IgnoreComet("Comet shuffle has different partition metrics")) { + Seq("SHUFFLE_MERGE", "SHUFFLE_HASH").foreach { joinHint => + def getJoinNode(plan: SparkPlan): Seq[ShuffledJoin] = if (joinHint == "SHUFFLE_MERGE") { + findTopLevelSortMergeJoin(plan) +@@ -1004,7 +1027,8 @@ class AdaptiveQueryExecSuite + } + } + +- test("metrics of the shuffle read") { ++ test("metrics of the shuffle read", ++ IgnoreComet("Comet shuffle changes the metrics")) { + withSQLConf(SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "true") { + val (_, adaptivePlan) = runAdaptiveAndVerifyResult( + "SELECT key FROM testData GROUP BY key") +@@ -1599,7 +1623,7 @@ class AdaptiveQueryExecSuite + val (_, adaptivePlan) = runAdaptiveAndVerifyResult( + "SELECT id FROM v1 GROUP BY id DISTRIBUTE BY id") + assert(collect(adaptivePlan) { +- case s: ShuffleExchangeExec => s ++ case s: ShuffleExchangeLike => s + }.length == 1) + } + } +@@ -1679,7 +1703,8 @@ class AdaptiveQueryExecSuite + } + } + +- test("SPARK-33551: Do not use AQE shuffle read for repartition") { ++ test("SPARK-33551: Do not use AQE shuffle read for repartition", ++ IgnoreComet("Comet shuffle changes partition size")) { + def hasRepartitionShuffle(plan: SparkPlan): Boolean = { + find(plan) { + case s: ShuffleExchangeLike => +@@ -1864,6 +1889,9 @@ class AdaptiveQueryExecSuite + def checkNoCoalescePartitions(ds: Dataset[Row], origin: ShuffleOrigin): Unit = { + assert(collect(ds.queryExecution.executedPlan) { + case s: ShuffleExchangeExec if s.shuffleOrigin == origin && s.numPartitions == 2 => s ++ case c: CometShuffleExchangeExec ++ if c.originalPlan.shuffleOrigin == origin && ++ c.originalPlan.numPartitions == 2 => c + }.size == 1) + ds.collect() + val plan = ds.queryExecution.executedPlan +@@ -1872,6 +1900,9 @@ class AdaptiveQueryExecSuite + }.isEmpty) + assert(collect(plan) { + case s: ShuffleExchangeExec if s.shuffleOrigin == origin && s.numPartitions == 2 => s ++ case c: CometShuffleExchangeExec ++ if c.originalPlan.shuffleOrigin == origin && ++ c.originalPlan.numPartitions == 2 => c + }.size == 1) + checkAnswer(ds, testData) + } +@@ -2028,7 +2059,8 @@ class AdaptiveQueryExecSuite + } + } + +- test("SPARK-35264: Support AQE side shuffled hash join formula") { ++ test("SPARK-35264: Support AQE side shuffled hash join formula", ++ IgnoreComet("Comet shuffle changes the partition size")) { + withTempView("t1", "t2") { + def checkJoinStrategy(shouldShuffleHashJoin: Boolean): Unit = { + Seq("100", "100000").foreach { size => +@@ -2114,7 +2146,8 @@ class AdaptiveQueryExecSuite + } + } + +- test("SPARK-35725: Support optimize skewed partitions in RebalancePartitions") { ++ test("SPARK-35725: Support optimize skewed partitions in RebalancePartitions", ++ IgnoreComet("Comet shuffle changes shuffle metrics")) { + withTempView("v") { + withSQLConf( + SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "true", +@@ -2213,7 +2246,7 @@ class AdaptiveQueryExecSuite + runAdaptiveAndVerifyResult(s"SELECT $repartition key1 FROM skewData1 " + + s"JOIN skewData2 ON key1 = key2 GROUP BY key1") + val shuffles1 = collect(adaptive1) { +- case s: ShuffleExchangeExec => s ++ case s: ShuffleExchangeLike => s + } + assert(shuffles1.size == 3) + // shuffles1.head is the top-level shuffle under the Aggregate operator +@@ -2226,7 +2259,7 @@ class AdaptiveQueryExecSuite + runAdaptiveAndVerifyResult(s"SELECT $repartition key1 FROM skewData1 " + + s"JOIN skewData2 ON key1 = key2") + val shuffles2 = collect(adaptive2) { +- case s: ShuffleExchangeExec => s ++ case s: ShuffleExchangeLike => s + } + if (hasRequiredDistribution) { + assert(shuffles2.size == 3) +@@ -2260,7 +2293,8 @@ class AdaptiveQueryExecSuite + } + } + +- test("SPARK-35794: Allow custom plugin for cost evaluator") { ++ test("SPARK-35794: Allow custom plugin for cost evaluator", ++ IgnoreComet("Comet shuffle changes shuffle metrics")) { + CostEvaluator.instantiate( + classOf[SimpleShuffleSortCostEvaluator].getCanonicalName, spark.sparkContext.getConf) + intercept[IllegalArgumentException] { +@@ -2404,6 +2438,7 @@ class AdaptiveQueryExecSuite + val (_, adaptive) = runAdaptiveAndVerifyResult(query) + assert(adaptive.collect { + case sort: SortExec => sort ++ case sort: CometSortExec => sort + }.size == 1) + val read = collect(adaptive) { + case read: AQEShuffleReadExec => read +@@ -2421,7 +2456,8 @@ class AdaptiveQueryExecSuite + } + } + +- test("SPARK-37357: Add small partition factor for rebalance partitions") { ++ test("SPARK-37357: Add small partition factor for rebalance partitions", ++ IgnoreComet("Comet shuffle changes shuffle metrics")) { + withTempView("v") { + withSQLConf( + SQLConf.ADAPTIVE_OPTIMIZE_SKEWS_IN_REBALANCE_PARTITIONS_ENABLED.key -> "true", +@@ -2533,7 +2569,7 @@ class AdaptiveQueryExecSuite + runAdaptiveAndVerifyResult("SELECT key1 FROM skewData1 JOIN skewData2 ON key1 = key2 " + + "JOIN skewData3 ON value2 = value3") + val shuffles1 = collect(adaptive1) { +- case s: ShuffleExchangeExec => s ++ case s: ShuffleExchangeLike => s + } + assert(shuffles1.size == 4) + val smj1 = findTopLevelSortMergeJoin(adaptive1) +@@ -2544,7 +2580,7 @@ class AdaptiveQueryExecSuite + runAdaptiveAndVerifyResult("SELECT key1 FROM skewData1 JOIN skewData2 ON key1 = key2 " + + "JOIN skewData3 ON value1 = value3") + val shuffles2 = collect(adaptive2) { +- case s: ShuffleExchangeExec => s ++ case s: ShuffleExchangeLike => s + } + assert(shuffles2.size == 4) + val smj2 = findTopLevelSortMergeJoin(adaptive2) +diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/SchemaPruningSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/SchemaPruningSuite.scala +index bd9c79e5b96..ab7584e768e 100644 +--- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/SchemaPruningSuite.scala ++++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/SchemaPruningSuite.scala +@@ -27,6 +27,7 @@ import org.apache.spark.sql.catalyst.SchemaPruningTest + import org.apache.spark.sql.catalyst.expressions.Concat + import org.apache.spark.sql.catalyst.parser.CatalystSqlParser + import org.apache.spark.sql.catalyst.plans.logical.Expand ++import org.apache.spark.sql.comet.CometScanExec + import org.apache.spark.sql.execution.FileSourceScanExec + import org.apache.spark.sql.execution.adaptive.AdaptiveSparkPlanHelper + import org.apache.spark.sql.functions._ +@@ -867,6 +868,7 @@ abstract class SchemaPruningSuite + val fileSourceScanSchemata = + collect(df.queryExecution.executedPlan) { + case scan: FileSourceScanExec => scan.requiredSchema ++ case scan: CometScanExec => scan.requiredSchema + } + assert(fileSourceScanSchemata.size === expectedSchemaCatalogStrings.size, + s"Found ${fileSourceScanSchemata.size} file sources in dataframe, " + +diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/V1WriteCommandSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/V1WriteCommandSuite.scala +index ce43edb79c1..c414b19eda7 100644 +--- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/V1WriteCommandSuite.scala ++++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/V1WriteCommandSuite.scala +@@ -17,7 +17,7 @@ + + package org.apache.spark.sql.execution.datasources + +-import org.apache.spark.sql.{QueryTest, Row} ++import org.apache.spark.sql.{IgnoreComet, QueryTest, Row} + import org.apache.spark.sql.catalyst.expressions.{Ascending, AttributeReference, NullsFirst, SortOrder} + import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, Sort} + import org.apache.spark.sql.execution.{QueryExecution, SortExec} +@@ -305,7 +305,8 @@ class V1WriteCommandSuite extends QueryTest with SharedSparkSession with V1Write + } + } + +- test("v1 write with AQE changing SMJ to BHJ") { ++ test("v1 write with AQE changing SMJ to BHJ", ++ IgnoreComet("TODO: Comet SMJ to BHJ by AQE")) { + withPlannedWrite { enabled => + withTable("t") { + sql( +diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/binaryfile/BinaryFileFormatSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/binaryfile/BinaryFileFormatSuite.scala +index 1d2e467c94c..3ea82cd1a3f 100644 +--- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/binaryfile/BinaryFileFormatSuite.scala ++++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/binaryfile/BinaryFileFormatSuite.scala +@@ -28,7 +28,7 @@ import org.apache.hadoop.fs.{FileStatus, FileSystem, GlobFilter, Path} + import org.mockito.Mockito.{mock, when} + + import org.apache.spark.SparkException +-import org.apache.spark.sql.{DataFrame, QueryTest, Row} ++import org.apache.spark.sql.{DataFrame, IgnoreCometSuite, QueryTest, Row} + import org.apache.spark.sql.catalyst.encoders.RowEncoder + import org.apache.spark.sql.execution.datasources.PartitionedFile + import org.apache.spark.sql.functions.col +@@ -38,7 +38,9 @@ import org.apache.spark.sql.test.SharedSparkSession + import org.apache.spark.sql.types._ + import org.apache.spark.util.Utils + +-class BinaryFileFormatSuite extends QueryTest with SharedSparkSession { ++// For some reason this suite is flaky w/ or w/o Comet when running in Github workflow. ++// Since it isn't related to Comet, we disable it for now. ++class BinaryFileFormatSuite extends QueryTest with SharedSparkSession with IgnoreCometSuite { + import BinaryFileFormat._ + + private var testDir: String = _ +diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetEncodingSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetEncodingSuite.scala +index 07e2849ce6f..3e73645b638 100644 +--- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetEncodingSuite.scala ++++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetEncodingSuite.scala +@@ -28,7 +28,7 @@ import org.apache.parquet.hadoop.ParquetOutputFormat + + import org.apache.spark.TestUtils + import org.apache.spark.memory.MemoryMode +-import org.apache.spark.sql.Row ++import org.apache.spark.sql.{IgnoreComet, Row} + import org.apache.spark.sql.catalyst.util.DateTimeUtils + import org.apache.spark.sql.internal.SQLConf + import org.apache.spark.sql.test.SharedSparkSession +@@ -201,7 +201,8 @@ class ParquetEncodingSuite extends ParquetCompatibilityTest with SharedSparkSess + } + } + +- test("parquet v2 pages - rle encoding for boolean value columns") { ++ test("parquet v2 pages - rle encoding for boolean value columns", ++ IgnoreComet("Comet doesn't support RLE encoding yet")) { + val extraOptions = Map[String, String]( + ParquetOutputFormat.WRITER_VERSION -> ParquetProperties.WriterVersion.PARQUET_2_0.toString + ) +diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFilterSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFilterSuite.scala +index 104b4e416cd..034f56b6230 100644 +--- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFilterSuite.scala ++++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFilterSuite.scala +@@ -1096,7 +1096,11 @@ abstract class ParquetFilterSuite extends QueryTest with ParquetTest with Shared + // When a filter is pushed to Parquet, Parquet can apply it to every row. + // So, we can check the number of rows returned from the Parquet + // to make sure our filter pushdown work. +- assert(stripSparkFilter(df).count == 1) ++ // Similar to Spark's vectorized reader, Comet doesn't do row-level filtering but relies ++ // on Spark to apply the data filters after columnar batches are returned ++ if (!isCometEnabled) { ++ assert(stripSparkFilter(df).count == 1) ++ } + } + } + } +@@ -1581,7 +1585,11 @@ abstract class ParquetFilterSuite extends QueryTest with ParquetTest with Shared + // than the total length but should not be a single record. + // Note that, if record level filtering is enabled, it should be a single record. + // If no filter is pushed down to Parquet, it should be the total length of data. +- assert(actual > 1 && actual < data.length) ++ // Only enable Comet test iff it's scan only, since with native execution ++ // `stripSparkFilter` can't remove the native filter ++ if (!isCometEnabled || isCometScanOnly) { ++ assert(actual > 1 && actual < data.length) ++ } + } + } + } +@@ -1608,7 +1616,11 @@ abstract class ParquetFilterSuite extends QueryTest with ParquetTest with Shared + // than the total length but should not be a single record. + // Note that, if record level filtering is enabled, it should be a single record. + // If no filter is pushed down to Parquet, it should be the total length of data. +- assert(actual > 1 && actual < data.length) ++ // Only enable Comet test iff it's scan only, since with native execution ++ // `stripSparkFilter` can't remove the native filter ++ if (!isCometEnabled || isCometScanOnly) { ++ assert(actual > 1 && actual < data.length) ++ } + } + } + } +@@ -1744,7 +1756,8 @@ abstract class ParquetFilterSuite extends QueryTest with ParquetTest with Shared + } + } + +- test("SPARK-17091: Convert IN predicate to Parquet filter push-down") { ++ test("SPARK-17091: Convert IN predicate to Parquet filter push-down", ++ IgnoreComet("IN predicate is not yet supported in Comet, see issue #36")) { + val schema = StructType(Seq( + StructField("a", IntegerType, nullable = false) + )) +@@ -1985,7 +1998,8 @@ abstract class ParquetFilterSuite extends QueryTest with ParquetTest with Shared + } + } + +- test("Support Parquet column index") { ++ test("Support Parquet column index", ++ IgnoreComet("Comet doesn't support Parquet column index yet")) { + // block 1: + // null count min max + // page-0 0 0 99 +@@ -2277,7 +2291,11 @@ class ParquetV1FilterSuite extends ParquetFilterSuite { + assert(pushedParquetFilters.exists(_.getClass === filterClass), + s"${pushedParquetFilters.map(_.getClass).toList} did not contain ${filterClass}.") + +- checker(stripSparkFilter(query), expected) ++ // Similar to Spark's vectorized reader, Comet doesn't do row-level filtering but relies ++ // on Spark to apply the data filters after columnar batches are returned ++ if (!isCometEnabled) { ++ checker(stripSparkFilter(query), expected) ++ } + } else { + assert(selectedFilters.isEmpty, "There is filter pushed down") + } +@@ -2337,7 +2355,11 @@ class ParquetV2FilterSuite extends ParquetFilterSuite { + assert(pushedParquetFilters.exists(_.getClass === filterClass), + s"${pushedParquetFilters.map(_.getClass).toList} did not contain ${filterClass}.") + +- checker(stripSparkFilter(query), expected) ++ // Similar to Spark's vectorized reader, Comet doesn't do row-level filtering but relies ++ // on Spark to apply the data filters after columnar batches are returned ++ if (!isCometEnabled) { ++ checker(stripSparkFilter(query), expected) ++ } + + case _ => + throw new AnalysisException("Can not match ParquetTable in the query.") +diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetIOSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetIOSuite.scala +index 8670d95c65e..b624c3811dd 100644 +--- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetIOSuite.scala ++++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetIOSuite.scala +@@ -1335,7 +1335,8 @@ class ParquetIOSuite extends QueryTest with ParquetTest with SharedSparkSession + } + } + +- test("SPARK-40128 read DELTA_LENGTH_BYTE_ARRAY encoded strings") { ++ test("SPARK-40128 read DELTA_LENGTH_BYTE_ARRAY encoded strings", ++ IgnoreComet("Comet doesn't support DELTA encoding yet")) { + withAllParquetReaders { + checkAnswer( + // "fruit" column in this file is encoded using DELTA_LENGTH_BYTE_ARRAY. +diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetQuerySuite.scala +index 29cb224c878..7a57d29d9a8 100644 +--- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetQuerySuite.scala ++++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetQuerySuite.scala +@@ -1047,7 +1047,8 @@ abstract class ParquetQuerySuite extends QueryTest with ParquetTest with SharedS + checkAnswer(readParquet(schema, path), df) + } + +- withSQLConf(SQLConf.PARQUET_VECTORIZED_READER_ENABLED.key -> "false") { ++ withSQLConf(SQLConf.PARQUET_VECTORIZED_READER_ENABLED.key -> "false", ++ "spark.comet.enabled" -> "false") { + val schema1 = "a DECIMAL(3, 2), b DECIMAL(18, 3), c DECIMAL(37, 3)" + checkAnswer(readParquet(schema1, path), df) + val schema2 = "a DECIMAL(3, 0), b DECIMAL(18, 1), c DECIMAL(37, 1)" +@@ -1069,7 +1070,8 @@ abstract class ParquetQuerySuite extends QueryTest with ParquetTest with SharedS + val df = sql(s"SELECT 1 a, 123456 b, ${Int.MaxValue.toLong * 10} c, CAST('1.2' AS BINARY) d") + df.write.parquet(path.toString) + +- withSQLConf(SQLConf.PARQUET_VECTORIZED_READER_ENABLED.key -> "false") { ++ withSQLConf(SQLConf.PARQUET_VECTORIZED_READER_ENABLED.key -> "false", ++ "spark.comet.enabled" -> "false") { + checkAnswer(readParquet("a DECIMAL(3, 2)", path), sql("SELECT 1.00")) + checkAnswer(readParquet("b DECIMAL(3, 2)", path), Row(null)) + checkAnswer(readParquet("b DECIMAL(11, 1)", path), sql("SELECT 123456.0")) +@@ -1128,7 +1130,7 @@ abstract class ParquetQuerySuite extends QueryTest with ParquetTest with SharedS + .where(s"a < ${Long.MaxValue}") + .collect() + } +- assert(exception.getCause.getCause.isInstanceOf[SchemaColumnConvertNotSupportedException]) ++ assert(exception.getMessage.contains("Column: [a], Expected: bigint, Found: INT32")) + } + } + } +diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetRebaseDatetimeSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetRebaseDatetimeSuite.scala +index 240bb4e6dcb..8287ffa03ca 100644 +--- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetRebaseDatetimeSuite.scala ++++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetRebaseDatetimeSuite.scala +@@ -21,7 +21,7 @@ import java.nio.file.{Files, Paths, StandardCopyOption} + import java.sql.{Date, Timestamp} + + import org.apache.spark.{SPARK_VERSION_SHORT, SparkConf, SparkException, SparkUpgradeException} +-import org.apache.spark.sql.{QueryTest, Row, SPARK_LEGACY_DATETIME_METADATA_KEY, SPARK_LEGACY_INT96_METADATA_KEY, SPARK_TIMEZONE_METADATA_KEY} ++import org.apache.spark.sql.{IgnoreCometSuite, QueryTest, Row, SPARK_LEGACY_DATETIME_METADATA_KEY, SPARK_LEGACY_INT96_METADATA_KEY, SPARK_TIMEZONE_METADATA_KEY} + import org.apache.spark.sql.catalyst.util.DateTimeTestUtils + import org.apache.spark.sql.internal.SQLConf + import org.apache.spark.sql.internal.SQLConf.{LegacyBehaviorPolicy, ParquetOutputTimestampType} +@@ -30,9 +30,11 @@ import org.apache.spark.sql.internal.SQLConf.ParquetOutputTimestampType.{INT96, + import org.apache.spark.sql.test.SharedSparkSession + import org.apache.spark.tags.SlowSQLTest + ++// Comet is disabled for this suite because it doesn't support datetime rebase mode + abstract class ParquetRebaseDatetimeSuite + extends QueryTest + with ParquetTest ++ with IgnoreCometSuite + with SharedSparkSession { + + import testImplicits._ +diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetRowIndexSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetRowIndexSuite.scala +index 351c6d698fc..36492fe936d 100644 +--- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetRowIndexSuite.scala ++++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetRowIndexSuite.scala +@@ -26,6 +26,7 @@ import org.apache.parquet.hadoop.{ParquetFileReader, ParquetOutputFormat} + import org.apache.parquet.hadoop.ParquetWriter.DEFAULT_BLOCK_SIZE + + import org.apache.spark.sql.QueryTest ++import org.apache.spark.sql.comet.{CometBatchScanExec, CometScanExec} + import org.apache.spark.sql.execution.FileSourceScanExec + import org.apache.spark.sql.execution.datasources.FileFormat + import org.apache.spark.sql.execution.datasources.v2.BatchScanExec +@@ -230,6 +231,12 @@ class ParquetRowIndexSuite extends QueryTest with SharedSparkSession { + case f: FileSourceScanExec => + numPartitions += f.inputRDD.partitions.length + numOutputRows += f.metrics("numOutputRows").value ++ case b: CometScanExec => ++ numPartitions += b.inputRDD.partitions.length ++ numOutputRows += b.metrics("numOutputRows").value ++ case b: CometBatchScanExec => ++ numPartitions += b.inputRDD.partitions.length ++ numOutputRows += b.metrics("numOutputRows").value + case _ => + } + assert(numPartitions > 0) +diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetSchemaPruningSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetSchemaPruningSuite.scala +index 5c0b7def039..151184bc98c 100644 +--- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetSchemaPruningSuite.scala ++++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetSchemaPruningSuite.scala +@@ -20,6 +20,7 @@ package org.apache.spark.sql.execution.datasources.parquet + import org.apache.spark.SparkConf + import org.apache.spark.sql.DataFrame + import org.apache.spark.sql.catalyst.parser.CatalystSqlParser ++import org.apache.spark.sql.comet.CometBatchScanExec + import org.apache.spark.sql.execution.adaptive.AdaptiveSparkPlanHelper + import org.apache.spark.sql.execution.datasources.SchemaPruningSuite + import org.apache.spark.sql.execution.datasources.v2.BatchScanExec +@@ -56,6 +57,7 @@ class ParquetV2SchemaPruningSuite extends ParquetSchemaPruningSuite { + val fileSourceScanSchemata = + collect(df.queryExecution.executedPlan) { + case scan: BatchScanExec => scan.scan.asInstanceOf[ParquetScan].readDataSchema ++ case scan: CometBatchScanExec => scan.scan.asInstanceOf[ParquetScan].readDataSchema + } + assert(fileSourceScanSchemata.size === expectedSchemaCatalogStrings.size, + s"Found ${fileSourceScanSchemata.size} file sources in dataframe, " + +diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetSchemaSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetSchemaSuite.scala +index bf5c51b89bb..ca22370ca3b 100644 +--- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetSchemaSuite.scala ++++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetSchemaSuite.scala +@@ -27,6 +27,7 @@ import org.apache.parquet.schema.PrimitiveType.PrimitiveTypeName + import org.apache.parquet.schema.Type._ + + import org.apache.spark.SparkException ++import org.apache.spark.sql.IgnoreComet + import org.apache.spark.sql.catalyst.ScalaReflection + import org.apache.spark.sql.execution.datasources.SchemaColumnConvertNotSupportedException + import org.apache.spark.sql.functions.desc +@@ -1016,7 +1017,8 @@ class ParquetSchemaSuite extends ParquetSchemaTest { + e + } + +- test("schema mismatch failure error message for parquet reader") { ++ test("schema mismatch failure error message for parquet reader", ++ IgnoreComet("Comet doesn't work with vectorizedReaderEnabled = false")) { + withTempPath { dir => + val e = testSchemaMismatch(dir.getCanonicalPath, vectorizedReaderEnabled = false) + val expectedMessage = "Encountered error while reading file" +diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/debug/DebuggingSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/debug/DebuggingSuite.scala +index 3a0bd35cb70..b28f06a757f 100644 +--- a/sql/core/src/test/scala/org/apache/spark/sql/execution/debug/DebuggingSuite.scala ++++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/debug/DebuggingSuite.scala +@@ -20,6 +20,7 @@ package org.apache.spark.sql.execution.debug + import java.io.ByteArrayOutputStream + + import org.apache.spark.rdd.RDD ++import org.apache.spark.sql.IgnoreComet + import org.apache.spark.sql.catalyst.InternalRow + import org.apache.spark.sql.catalyst.expressions.Attribute + import org.apache.spark.sql.catalyst.expressions.codegen.CodegenContext +@@ -124,7 +125,8 @@ class DebuggingSuite extends DebuggingSuiteBase with DisableAdaptiveExecutionSui + | id LongType: {}""".stripMargin)) + } + +- test("SPARK-28537: DebugExec cannot debug columnar related queries") { ++ test("SPARK-28537: DebugExec cannot debug columnar related queries", ++ IgnoreComet("Comet does not use FileScan")) { + withTempPath { workDir => + val workDirPath = workDir.getAbsolutePath + val input = spark.range(5).toDF("id") +diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/metric/SQLMetricsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/metric/SQLMetricsSuite.scala +index 26e61c6b58d..cb09d7e116a 100644 +--- a/sql/core/src/test/scala/org/apache/spark/sql/execution/metric/SQLMetricsSuite.scala ++++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/metric/SQLMetricsSuite.scala +@@ -45,8 +45,10 @@ import org.apache.spark.sql.util.QueryExecutionListener + import org.apache.spark.util.{AccumulatorContext, JsonProtocol} + + // Disable AQE because metric info is different with AQE on/off ++// This test suite runs tests against the metrics of physical operators. ++// Disabling it for Comet because the metrics are different with Comet enabled. + class SQLMetricsSuite extends SharedSparkSession with SQLMetricsTestUtils +- with DisableAdaptiveExecutionSuite { ++ with DisableAdaptiveExecutionSuite with IgnoreCometSuite { + import testImplicits._ + + /** +@@ -737,7 +739,8 @@ class SQLMetricsSuite extends SharedSparkSession with SQLMetricsTestUtils + } + } + +- test("SPARK-26327: FileSourceScanExec metrics") { ++ test("SPARK-26327: FileSourceScanExec metrics", ++ IgnoreComet("Spark uses row-based Parquet reader while Comet is vectorized")) { + withTable("testDataForScan") { + spark.range(10).selectExpr("id", "id % 3 as p") + .write.partitionBy("p").saveAsTable("testDataForScan") +diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/python/ExtractPythonUDFsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/python/ExtractPythonUDFsSuite.scala +index 0ab8691801d..d9125f658ad 100644 +--- a/sql/core/src/test/scala/org/apache/spark/sql/execution/python/ExtractPythonUDFsSuite.scala ++++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/python/ExtractPythonUDFsSuite.scala +@@ -18,6 +18,7 @@ + package org.apache.spark.sql.execution.python + + import org.apache.spark.sql.catalyst.plans.logical.{ArrowEvalPython, BatchEvalPython, Limit, LocalLimit} ++import org.apache.spark.sql.comet._ + import org.apache.spark.sql.execution.{FileSourceScanExec, SparkPlan, SparkPlanTest} + import org.apache.spark.sql.execution.datasources.v2.BatchScanExec + import org.apache.spark.sql.execution.datasources.v2.parquet.ParquetScan +@@ -108,6 +109,7 @@ class ExtractPythonUDFsSuite extends SparkPlanTest with SharedSparkSession { + + val scanNodes = query.queryExecution.executedPlan.collect { + case scan: FileSourceScanExec => scan ++ case scan: CometScanExec => scan + } + assert(scanNodes.length == 1) + assert(scanNodes.head.output.map(_.name) == Seq("a")) +@@ -120,11 +122,16 @@ class ExtractPythonUDFsSuite extends SparkPlanTest with SharedSparkSession { + + val scanNodes = query.queryExecution.executedPlan.collect { + case scan: FileSourceScanExec => scan ++ case scan: CometScanExec => scan + } + assert(scanNodes.length == 1) + // $"a" is not null and $"a" > 1 +- assert(scanNodes.head.dataFilters.length == 2) +- assert(scanNodes.head.dataFilters.flatMap(_.references.map(_.name)).distinct == Seq("a")) ++ val dataFilters = scanNodes.head match { ++ case scan: FileSourceScanExec => scan.dataFilters ++ case scan: CometScanExec => scan.dataFilters ++ } ++ assert(dataFilters.length == 2) ++ assert(dataFilters.flatMap(_.references.map(_.name)).distinct == Seq("a")) + } + } + } +@@ -145,6 +152,7 @@ class ExtractPythonUDFsSuite extends SparkPlanTest with SharedSparkSession { + + val scanNodes = query.queryExecution.executedPlan.collect { + case scan: BatchScanExec => scan ++ case scan: CometBatchScanExec => scan + } + assert(scanNodes.length == 1) + assert(scanNodes.head.output.map(_.name) == Seq("a")) +@@ -157,6 +165,7 @@ class ExtractPythonUDFsSuite extends SparkPlanTest with SharedSparkSession { + + val scanNodes = query.queryExecution.executedPlan.collect { + case scan: BatchScanExec => scan ++ case scan: CometBatchScanExec => scan + } + assert(scanNodes.length == 1) + // $"a" is not null and $"a" > 1 +diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/AsyncProgressTrackingMicroBatchExecutionSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/AsyncProgressTrackingMicroBatchExecutionSuite.scala +index d083cac48ff..3c11bcde807 100644 +--- a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/AsyncProgressTrackingMicroBatchExecutionSuite.scala ++++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/AsyncProgressTrackingMicroBatchExecutionSuite.scala +@@ -37,8 +37,10 @@ import org.apache.spark.sql.streaming.{StreamingQuery, StreamingQueryException, + import org.apache.spark.sql.streaming.util.StreamManualClock + import org.apache.spark.util.Utils + ++// For some reason this suite is flaky w/ or w/o Comet when running in Github workflow. ++// Since it isn't related to Comet, we disable it for now. + class AsyncProgressTrackingMicroBatchExecutionSuite +- extends StreamTest with BeforeAndAfter with Matchers { ++ extends StreamTest with BeforeAndAfter with Matchers with IgnoreCometSuite { + + import testImplicits._ + +diff --git a/sql/core/src/test/scala/org/apache/spark/sql/sources/BucketedReadSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/sources/BucketedReadSuite.scala +index 266bb343526..a426d8396be 100644 +--- a/sql/core/src/test/scala/org/apache/spark/sql/sources/BucketedReadSuite.scala ++++ b/sql/core/src/test/scala/org/apache/spark/sql/sources/BucketedReadSuite.scala +@@ -24,10 +24,11 @@ import org.apache.spark.sql.catalyst.catalog.BucketSpec + import org.apache.spark.sql.catalyst.expressions + import org.apache.spark.sql.catalyst.expressions._ + import org.apache.spark.sql.catalyst.plans.physical.HashPartitioning +-import org.apache.spark.sql.execution.{FileSourceScanExec, SortExec, SparkPlan} ++import org.apache.spark.sql.comet._ ++import org.apache.spark.sql.execution.{ColumnarToRowExec, FileSourceScanExec, SortExec, SparkPlan} + import org.apache.spark.sql.execution.adaptive.{AdaptiveSparkPlanExec, AdaptiveSparkPlanHelper, DisableAdaptiveExecution} + import org.apache.spark.sql.execution.datasources.BucketingUtils +-import org.apache.spark.sql.execution.exchange.ShuffleExchangeExec ++import org.apache.spark.sql.execution.exchange.{ShuffleExchangeExec, ShuffleExchangeLike} + import org.apache.spark.sql.execution.joins.SortMergeJoinExec + import org.apache.spark.sql.functions._ + import org.apache.spark.sql.internal.SQLConf +@@ -101,12 +102,20 @@ abstract class BucketedReadSuite extends QueryTest with SQLTestUtils with Adapti + } + } + +- private def getFileScan(plan: SparkPlan): FileSourceScanExec = { +- val fileScan = collect(plan) { case f: FileSourceScanExec => f } ++ private def getFileScan(plan: SparkPlan): SparkPlan = { ++ val fileScan = collect(plan) { ++ case f: FileSourceScanExec => f ++ case f: CometScanExec => f ++ } + assert(fileScan.nonEmpty, plan) + fileScan.head + } + ++ private def getBucketScan(plan: SparkPlan): Boolean = getFileScan(plan) match { ++ case fs: FileSourceScanExec => fs.bucketedScan ++ case bs: CometScanExec => bs.bucketedScan ++ } ++ + // To verify if the bucket pruning works, this function checks two conditions: + // 1) Check if the pruned buckets (before filtering) are empty. + // 2) Verify the final result is the same as the expected one +@@ -155,7 +164,8 @@ abstract class BucketedReadSuite extends QueryTest with SQLTestUtils with Adapti + val planWithoutBucketedScan = bucketedDataFrame.filter(filterCondition) + .queryExecution.executedPlan + val fileScan = getFileScan(planWithoutBucketedScan) +- assert(!fileScan.bucketedScan, s"except no bucketed scan but found\n$fileScan") ++ val bucketedScan = getBucketScan(planWithoutBucketedScan) ++ assert(!bucketedScan, s"except no bucketed scan but found\n$fileScan") + + val bucketColumnType = bucketedDataFrame.schema.apply(bucketColumnIndex).dataType + val rowsWithInvalidBuckets = fileScan.execute().filter(row => { +@@ -451,28 +461,44 @@ abstract class BucketedReadSuite extends QueryTest with SQLTestUtils with Adapti + val joinOperator = if (joined.sqlContext.conf.adaptiveExecutionEnabled) { + val executedPlan = + joined.queryExecution.executedPlan.asInstanceOf[AdaptiveSparkPlanExec].executedPlan +- assert(executedPlan.isInstanceOf[SortMergeJoinExec]) +- executedPlan.asInstanceOf[SortMergeJoinExec] ++ executedPlan match { ++ case s: SortMergeJoinExec => s ++ case b: CometSortMergeJoinExec => ++ b.originalPlan match { ++ case s: SortMergeJoinExec => s ++ case o => fail(s"expected SortMergeJoinExec, but found\n$o") ++ } ++ case o => fail(s"expected SortMergeJoinExec, but found\n$o") ++ } + } else { + val executedPlan = joined.queryExecution.executedPlan +- assert(executedPlan.isInstanceOf[SortMergeJoinExec]) +- executedPlan.asInstanceOf[SortMergeJoinExec] ++ executedPlan match { ++ case s: SortMergeJoinExec => s ++ case ColumnarToRowExec(child) => ++ child.asInstanceOf[CometSortMergeJoinExec].originalPlan match { ++ case s: SortMergeJoinExec => s ++ case o => fail(s"expected SortMergeJoinExec, but found\n$o") ++ } ++ case o => fail(s"expected SortMergeJoinExec, but found\n$o") ++ } + } + + // check existence of shuffle + assert( +- joinOperator.left.exists(_.isInstanceOf[ShuffleExchangeExec]) == shuffleLeft, ++ joinOperator.left.exists(op => op.isInstanceOf[ShuffleExchangeLike]) == shuffleLeft, + s"expected shuffle in plan to be $shuffleLeft but found\n${joinOperator.left}") + assert( +- joinOperator.right.exists(_.isInstanceOf[ShuffleExchangeExec]) == shuffleRight, ++ joinOperator.right.exists(op => op.isInstanceOf[ShuffleExchangeLike]) == shuffleRight, + s"expected shuffle in plan to be $shuffleRight but found\n${joinOperator.right}") + + // check existence of sort + assert( +- joinOperator.left.exists(_.isInstanceOf[SortExec]) == sortLeft, ++ joinOperator.left.exists(op => op.isInstanceOf[SortExec] || op.isInstanceOf[CometExec] && ++ op.asInstanceOf[CometExec].originalPlan.isInstanceOf[SortExec]) == sortLeft, + s"expected sort in the left child to be $sortLeft but found\n${joinOperator.left}") + assert( +- joinOperator.right.exists(_.isInstanceOf[SortExec]) == sortRight, ++ joinOperator.right.exists(op => op.isInstanceOf[SortExec] || op.isInstanceOf[CometExec] && ++ op.asInstanceOf[CometExec].originalPlan.isInstanceOf[SortExec]) == sortRight, + s"expected sort in the right child to be $sortRight but found\n${joinOperator.right}") + + // check the output partitioning +@@ -835,11 +861,11 @@ abstract class BucketedReadSuite extends QueryTest with SQLTestUtils with Adapti + df1.write.format("parquet").bucketBy(8, "i").saveAsTable("bucketed_table") + + val scanDF = spark.table("bucketed_table").select("j") +- assert(!getFileScan(scanDF.queryExecution.executedPlan).bucketedScan) ++ assert(!getBucketScan(scanDF.queryExecution.executedPlan)) + checkAnswer(scanDF, df1.select("j")) + + val aggDF = spark.table("bucketed_table").groupBy("j").agg(max("k")) +- assert(!getFileScan(aggDF.queryExecution.executedPlan).bucketedScan) ++ assert(!getBucketScan(aggDF.queryExecution.executedPlan)) + checkAnswer(aggDF, df1.groupBy("j").agg(max("k"))) + } + } +@@ -1026,15 +1052,23 @@ abstract class BucketedReadSuite extends QueryTest with SQLTestUtils with Adapti + expectedNumShuffles: Int, + expectedCoalescedNumBuckets: Option[Int]): Unit = { + val plan = sql(query).queryExecution.executedPlan +- val shuffles = plan.collect { case s: ShuffleExchangeExec => s } ++ val shuffles = plan.collect { ++ case s: ShuffleExchangeLike => s ++ } + assert(shuffles.length == expectedNumShuffles) + + val scans = plan.collect { + case f: FileSourceScanExec if f.optionalNumCoalescedBuckets.isDefined => f ++ case b: CometScanExec if b.optionalNumCoalescedBuckets.isDefined => b + } + if (expectedCoalescedNumBuckets.isDefined) { + assert(scans.length == 1) +- assert(scans.head.optionalNumCoalescedBuckets == expectedCoalescedNumBuckets) ++ scans.head match { ++ case f: FileSourceScanExec => ++ assert(f.optionalNumCoalescedBuckets == expectedCoalescedNumBuckets) ++ case b: CometScanExec => ++ assert(b.optionalNumCoalescedBuckets == expectedCoalescedNumBuckets) ++ } + } else { + assert(scans.isEmpty) + } +diff --git a/sql/core/src/test/scala/org/apache/spark/sql/sources/CreateTableAsSelectSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/sources/CreateTableAsSelectSuite.scala +index b5f6d2f9f68..277784a92af 100644 +--- a/sql/core/src/test/scala/org/apache/spark/sql/sources/CreateTableAsSelectSuite.scala ++++ b/sql/core/src/test/scala/org/apache/spark/sql/sources/CreateTableAsSelectSuite.scala +@@ -20,7 +20,7 @@ package org.apache.spark.sql.sources + import java.io.File + + import org.apache.spark.SparkException +-import org.apache.spark.sql.AnalysisException ++import org.apache.spark.sql.{AnalysisException, IgnoreCometSuite} + import org.apache.spark.sql.catalyst.TableIdentifier + import org.apache.spark.sql.catalyst.catalog.{BucketSpec, CatalogTableType} + import org.apache.spark.sql.catalyst.parser.ParseException +@@ -28,7 +28,10 @@ import org.apache.spark.sql.internal.SQLConf.BUCKETING_MAX_BUCKETS + import org.apache.spark.sql.test.SharedSparkSession + import org.apache.spark.util.Utils + +-class CreateTableAsSelectSuite extends DataSourceTest with SharedSparkSession { ++// For some reason this suite is flaky w/ or w/o Comet when running in Github workflow. ++// Since it isn't related to Comet, we disable it for now. ++class CreateTableAsSelectSuite extends DataSourceTest with SharedSparkSession ++ with IgnoreCometSuite { + import testImplicits._ + + protected override lazy val sql = spark.sql _ +diff --git a/sql/core/src/test/scala/org/apache/spark/sql/sources/DisableUnnecessaryBucketedScanSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/sources/DisableUnnecessaryBucketedScanSuite.scala +index 1f55742cd67..42377f7cf26 100644 +--- a/sql/core/src/test/scala/org/apache/spark/sql/sources/DisableUnnecessaryBucketedScanSuite.scala ++++ b/sql/core/src/test/scala/org/apache/spark/sql/sources/DisableUnnecessaryBucketedScanSuite.scala +@@ -20,6 +20,7 @@ package org.apache.spark.sql.sources + import org.apache.spark.sql.QueryTest + import org.apache.spark.sql.catalyst.expressions.AttributeReference + import org.apache.spark.sql.catalyst.plans.physical.HashPartitioning ++import org.apache.spark.sql.comet.CometScanExec + import org.apache.spark.sql.execution.FileSourceScanExec + import org.apache.spark.sql.execution.adaptive.{AdaptiveSparkPlanHelper, DisableAdaptiveExecutionSuite, EnableAdaptiveExecutionSuite} + import org.apache.spark.sql.execution.exchange.ShuffleExchangeExec +@@ -71,7 +72,10 @@ abstract class DisableUnnecessaryBucketedScanSuite + + def checkNumBucketedScan(query: String, expectedNumBucketedScan: Int): Unit = { + val plan = sql(query).queryExecution.executedPlan +- val bucketedScan = collect(plan) { case s: FileSourceScanExec if s.bucketedScan => s } ++ val bucketedScan = collect(plan) { ++ case s: FileSourceScanExec if s.bucketedScan => s ++ case s: CometScanExec if s.bucketedScan => s ++ } + assert(bucketedScan.length == expectedNumBucketedScan) + } + +diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/FileStreamSinkSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/FileStreamSinkSuite.scala +index 75f440caefc..36b1146bc3a 100644 +--- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/FileStreamSinkSuite.scala ++++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/FileStreamSinkSuite.scala +@@ -34,6 +34,7 @@ import org.apache.spark.paths.SparkPath + import org.apache.spark.scheduler.{SparkListener, SparkListenerTaskEnd} + import org.apache.spark.sql.{AnalysisException, DataFrame} + import org.apache.spark.sql.catalyst.util.stringToFile ++import org.apache.spark.sql.comet.CometBatchScanExec + import org.apache.spark.sql.execution.DataSourceScanExec + import org.apache.spark.sql.execution.datasources._ + import org.apache.spark.sql.execution.datasources.v2.{BatchScanExec, DataSourceV2Relation, FileScan, FileTable} +@@ -748,6 +749,8 @@ class FileStreamSinkV2Suite extends FileStreamSinkSuite { + val fileScan = df.queryExecution.executedPlan.collect { + case batch: BatchScanExec if batch.scan.isInstanceOf[FileScan] => + batch.scan.asInstanceOf[FileScan] ++ case batch: CometBatchScanExec if batch.scan.isInstanceOf[FileScan] => ++ batch.scan.asInstanceOf[FileScan] + }.headOption.getOrElse { + fail(s"No FileScan in query\n${df.queryExecution}") + } +diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/FlatMapGroupsWithStateDistributionSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/FlatMapGroupsWithStateDistributionSuite.scala +index b597a244710..b2e8be41065 100644 +--- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/FlatMapGroupsWithStateDistributionSuite.scala ++++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/FlatMapGroupsWithStateDistributionSuite.scala +@@ -21,6 +21,7 @@ import java.io.File + + import org.apache.commons.io.FileUtils + ++import org.apache.spark.sql.IgnoreComet + import org.apache.spark.sql.catalyst.streaming.InternalOutputModes.Update + import org.apache.spark.sql.execution.streaming.{FlatMapGroupsWithStateExec, MemoryStream} + import org.apache.spark.sql.internal.SQLConf +@@ -91,7 +92,7 @@ class FlatMapGroupsWithStateDistributionSuite extends StreamTest + } + + test("SPARK-38204: flatMapGroupsWithState should require StatefulOpClusteredDistribution " + +- "from children - without initial state") { ++ "from children - without initial state", IgnoreComet("TODO: fix Comet for this test")) { + // function will return -1 on timeout and returns count of the state otherwise + val stateFunc = + (key: (String, String), values: Iterator[(String, String, Long)], +@@ -243,7 +244,8 @@ class FlatMapGroupsWithStateDistributionSuite extends StreamTest + } + + test("SPARK-38204: flatMapGroupsWithState should require ClusteredDistribution " + +- "from children if the query starts from checkpoint in 3.2.x - without initial state") { ++ "from children if the query starts from checkpoint in 3.2.x - without initial state", ++ IgnoreComet("TODO: fix Comet for this test")) { + // function will return -1 on timeout and returns count of the state otherwise + val stateFunc = + (key: (String, String), values: Iterator[(String, String, Long)], +@@ -335,7 +337,8 @@ class FlatMapGroupsWithStateDistributionSuite extends StreamTest + } + + test("SPARK-38204: flatMapGroupsWithState should require ClusteredDistribution " + +- "from children if the query starts from checkpoint in prior to 3.2") { ++ "from children if the query starts from checkpoint in prior to 3.2", ++ IgnoreComet("TODO: fix Comet for this test")) { + // function will return -1 on timeout and returns count of the state otherwise + val stateFunc = + (key: (String, String), values: Iterator[(String, String, Long)], +diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/FlatMapGroupsWithStateSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/FlatMapGroupsWithStateSuite.scala +index 6aa7d0945c7..38523536154 100644 +--- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/FlatMapGroupsWithStateSuite.scala ++++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/FlatMapGroupsWithStateSuite.scala +@@ -25,7 +25,7 @@ import org.scalatest.exceptions.TestFailedException + + import org.apache.spark.SparkException + import org.apache.spark.api.java.function.FlatMapGroupsWithStateFunction +-import org.apache.spark.sql.{DataFrame, Encoder} ++import org.apache.spark.sql.{DataFrame, Encoder, IgnoreCometSuite} + import org.apache.spark.sql.catalyst.InternalRow + import org.apache.spark.sql.catalyst.expressions.{GenericInternalRow, UnsafeProjection, UnsafeRow} + import org.apache.spark.sql.catalyst.plans.logical.FlatMapGroupsWithState +@@ -46,8 +46,9 @@ case class RunningCount(count: Long) + + case class Result(key: Long, count: Int) + ++// TODO: fix Comet to enable this suite + @SlowSQLTest +-class FlatMapGroupsWithStateSuite extends StateStoreMetricsTest { ++class FlatMapGroupsWithStateSuite extends StateStoreMetricsTest with IgnoreCometSuite { + + import testImplicits._ + +diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/FlatMapGroupsWithStateWithInitialStateSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/FlatMapGroupsWithStateWithInitialStateSuite.scala +index 2a2a83d35e1..e3b7b290b3e 100644 +--- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/FlatMapGroupsWithStateWithInitialStateSuite.scala ++++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/FlatMapGroupsWithStateWithInitialStateSuite.scala +@@ -18,7 +18,7 @@ + package org.apache.spark.sql.streaming + + import org.apache.spark.SparkException +-import org.apache.spark.sql.{AnalysisException, Dataset, KeyValueGroupedDataset} ++import org.apache.spark.sql.{AnalysisException, Dataset, IgnoreComet, KeyValueGroupedDataset} + import org.apache.spark.sql.catalyst.streaming.InternalOutputModes.Update + import org.apache.spark.sql.execution.streaming.MemoryStream + import org.apache.spark.sql.execution.streaming.state.FlatMapGroupsWithStateExecHelper +@@ -253,7 +253,8 @@ class FlatMapGroupsWithStateWithInitialStateSuite extends StateStoreMetricsTest + assert(e.message.contains(expectedError)) + } + +- test("flatMapGroupsWithState - initial state - initial state has flatMapGroupsWithState") { ++ test("flatMapGroupsWithState - initial state - initial state has flatMapGroupsWithState", ++ IgnoreComet("TODO: fix Comet for this test")) { + val initialStateDS = Seq(("keyInStateAndData", new RunningCount(1))).toDS() + val initialState: KeyValueGroupedDataset[String, RunningCount] = + initialStateDS.groupByKey(_._1).mapValues(_._2) +diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamSuite.scala +index ef5b8a769fe..84fe1bfabc9 100644 +--- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamSuite.scala ++++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamSuite.scala +@@ -37,6 +37,7 @@ import org.apache.spark.sql._ + import org.apache.spark.sql.catalyst.plans.logical.{Range, RepartitionByExpression} + import org.apache.spark.sql.catalyst.streaming.{InternalOutputModes, StreamingRelationV2} + import org.apache.spark.sql.catalyst.util.DateTimeUtils ++import org.apache.spark.sql.comet.CometLocalLimitExec + import org.apache.spark.sql.execution.{LocalLimitExec, SimpleMode, SparkPlan} + import org.apache.spark.sql.execution.command.ExplainCommand + import org.apache.spark.sql.execution.streaming._ +@@ -1103,11 +1104,12 @@ class StreamSuite extends StreamTest { + val localLimits = execPlan.collect { + case l: LocalLimitExec => l + case l: StreamingLocalLimitExec => l ++ case l: CometLocalLimitExec => l + } + + require( + localLimits.size == 1, +- s"Cant verify local limit optimization with this plan:\n$execPlan") ++ s"Cant verify local limit optimization ${localLimits.size} with this plan:\n$execPlan") + + if (expectStreamingLimit) { + assert( +@@ -1115,7 +1117,8 @@ class StreamSuite extends StreamTest { + s"Local limit was not StreamingLocalLimitExec:\n$execPlan") + } else { + assert( +- localLimits.head.isInstanceOf[LocalLimitExec], ++ localLimits.head.isInstanceOf[LocalLimitExec] || ++ localLimits.head.isInstanceOf[CometLocalLimitExec], + s"Local limit was not LocalLimitExec:\n$execPlan") + } + } +diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingAggregationDistributionSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingAggregationDistributionSuite.scala +index b4c4ec7acbf..20579284856 100644 +--- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingAggregationDistributionSuite.scala ++++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingAggregationDistributionSuite.scala +@@ -23,6 +23,7 @@ import org.apache.commons.io.FileUtils + import org.scalatest.Assertions + + import org.apache.spark.sql.catalyst.plans.physical.UnspecifiedDistribution ++import org.apache.spark.sql.comet.CometHashAggregateExec + import org.apache.spark.sql.execution.aggregate.BaseAggregateExec + import org.apache.spark.sql.execution.streaming.{MemoryStream, StateStoreRestoreExec, StateStoreSaveExec} + import org.apache.spark.sql.functions.count +@@ -67,6 +68,7 @@ class StreamingAggregationDistributionSuite extends StreamTest + // verify aggregations in between, except partial aggregation + val allAggregateExecs = query.lastExecution.executedPlan.collect { + case a: BaseAggregateExec => a ++ case c: CometHashAggregateExec => c.originalPlan + } + + val aggregateExecsWithoutPartialAgg = allAggregateExecs.filter { +@@ -201,6 +203,7 @@ class StreamingAggregationDistributionSuite extends StreamTest + // verify aggregations in between, except partial aggregation + val allAggregateExecs = executedPlan.collect { + case a: BaseAggregateExec => a ++ case c: CometHashAggregateExec => c.originalPlan + } + + val aggregateExecsWithoutPartialAgg = allAggregateExecs.filter { +diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingJoinSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingJoinSuite.scala +index 4d92e270539..33f1c2eb75e 100644 +--- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingJoinSuite.scala ++++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingJoinSuite.scala +@@ -31,7 +31,7 @@ import org.apache.spark.scheduler.ExecutorCacheTaskLocation + import org.apache.spark.sql.{DataFrame, Row, SparkSession} + import org.apache.spark.sql.catalyst.expressions.{AttributeReference, Expression} + import org.apache.spark.sql.catalyst.plans.physical.HashPartitioning +-import org.apache.spark.sql.execution.exchange.ShuffleExchangeExec ++import org.apache.spark.sql.execution.exchange.ShuffleExchangeLike + import org.apache.spark.sql.execution.streaming.{MemoryStream, StatefulOperatorStateInfo, StreamingSymmetricHashJoinExec, StreamingSymmetricHashJoinHelper} + import org.apache.spark.sql.execution.streaming.state.{RocksDBStateStoreProvider, StateStore, StateStoreProviderId} + import org.apache.spark.sql.functions._ +@@ -619,14 +619,28 @@ class StreamingInnerJoinSuite extends StreamingJoinSuite { + + val numPartitions = spark.sqlContext.conf.getConf(SQLConf.SHUFFLE_PARTITIONS) + +- assert(query.lastExecution.executedPlan.collect { +- case j @ StreamingSymmetricHashJoinExec(_, _, _, _, _, _, _, _, _, +- ShuffleExchangeExec(opA: HashPartitioning, _, _), +- ShuffleExchangeExec(opB: HashPartitioning, _, _)) +- if partitionExpressionsColumns(opA.expressions) === Seq("a", "b") +- && partitionExpressionsColumns(opB.expressions) === Seq("a", "b") +- && opA.numPartitions == numPartitions && opB.numPartitions == numPartitions => j +- }.size == 1) ++ val join = query.lastExecution.executedPlan.collect { ++ case j: StreamingSymmetricHashJoinExec => j ++ }.head ++ val opA = join.left.collect { ++ case s: ShuffleExchangeLike ++ if s.outputPartitioning.isInstanceOf[HashPartitioning] && ++ partitionExpressionsColumns( ++ s.outputPartitioning ++ .asInstanceOf[HashPartitioning].expressions) === Seq("a", "b") => ++ s.outputPartitioning ++ .asInstanceOf[HashPartitioning] ++ }.head ++ val opB = join.right.collect { ++ case s: ShuffleExchangeLike ++ if s.outputPartitioning.isInstanceOf[HashPartitioning] && ++ partitionExpressionsColumns( ++ s.outputPartitioning ++ .asInstanceOf[HashPartitioning].expressions) === Seq("a", "b") => ++ s.outputPartitioning ++ .asInstanceOf[HashPartitioning] ++ }.head ++ assert(opA.numPartitions == numPartitions && opB.numPartitions == numPartitions) + }) + } + +diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/test/DataStreamTableAPISuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/test/DataStreamTableAPISuite.scala +index abe606ad9c1..2d930b64cca 100644 +--- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/test/DataStreamTableAPISuite.scala ++++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/test/DataStreamTableAPISuite.scala +@@ -22,7 +22,7 @@ import java.util + + import org.scalatest.BeforeAndAfter + +-import org.apache.spark.sql.{AnalysisException, Row, SaveMode} ++import org.apache.spark.sql.{AnalysisException, IgnoreComet, Row, SaveMode} + import org.apache.spark.sql.catalyst.TableIdentifier + import org.apache.spark.sql.catalyst.analysis.TableAlreadyExistsException + import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTable, CatalogTableType} +@@ -327,7 +327,8 @@ class DataStreamTableAPISuite extends StreamTest with BeforeAndAfter { + } + } + +- test("explain with table on DSv1 data source") { ++ test("explain with table on DSv1 data source", ++ IgnoreComet("Comet explain output is different")) { + val tblSourceName = "tbl_src" + val tblTargetName = "tbl_target" + val tblSourceQualified = s"default.$tblSourceName" +diff --git a/sql/core/src/test/scala/org/apache/spark/sql/test/SQLTestUtils.scala b/sql/core/src/test/scala/org/apache/spark/sql/test/SQLTestUtils.scala +index dd55fcfe42c..293e9dc2986 100644 +--- a/sql/core/src/test/scala/org/apache/spark/sql/test/SQLTestUtils.scala ++++ b/sql/core/src/test/scala/org/apache/spark/sql/test/SQLTestUtils.scala +@@ -41,6 +41,7 @@ import org.apache.spark.sql.catalyst.plans.PlanTest + import org.apache.spark.sql.catalyst.plans.PlanTestBase + import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan + import org.apache.spark.sql.catalyst.util._ ++import org.apache.spark.sql.comet._ + import org.apache.spark.sql.execution.FilterExec + import org.apache.spark.sql.execution.adaptive.DisableAdaptiveExecution + import org.apache.spark.sql.execution.datasources.DataSourceUtils +@@ -126,7 +127,11 @@ private[sql] trait SQLTestUtils extends SparkFunSuite with SQLTestUtilsBase with + } + } + } else { +- super.test(testName, testTags: _*)(testFun) ++ if (isCometEnabled && testTags.exists(_.isInstanceOf[IgnoreComet])) { ++ ignore(testName + " (disabled when Comet is on)", testTags: _*)(testFun) ++ } else { ++ super.test(testName, testTags: _*)(testFun) ++ } + } + } + +@@ -242,6 +247,32 @@ private[sql] trait SQLTestUtilsBase + protected override def _sqlContext: SQLContext = self.spark.sqlContext + } + ++ /** ++ * Whether Comet extension is enabled ++ */ ++ protected def isCometEnabled: Boolean = { ++ val v = System.getenv("ENABLE_COMET") ++ v != null && v.toBoolean ++ } ++ ++ /** ++ * Whether to enable ansi mode This is only effective when ++ * [[isCometEnabled]] returns true. ++ */ ++ protected def enableCometAnsiMode: Boolean = { ++ val v = System.getenv("ENABLE_COMET_ANSI_MODE") ++ v != null && v.toBoolean ++ } ++ ++ /** ++ * Whether Spark should only apply Comet scan optimization. This is only effective when ++ * [[isCometEnabled]] returns true. ++ */ ++ protected def isCometScanOnly: Boolean = { ++ val v = System.getenv("ENABLE_COMET_SCAN_ONLY") ++ v != null && v.toBoolean ++ } ++ + protected override def withSQLConf(pairs: (String, String)*)(f: => Unit): Unit = { + SparkSession.setActiveSession(spark) + super.withSQLConf(pairs: _*)(f) +@@ -434,6 +465,8 @@ private[sql] trait SQLTestUtilsBase + val schema = df.schema + val withoutFilters = df.queryExecution.executedPlan.transform { + case FilterExec(_, child) => child ++ case CometFilterExec(_, _, _, child, _) => child ++ case CometProjectExec(_, _, _, _, CometFilterExec(_, _, _, child, _), _) => child + } + + spark.internalCreateDataFrame(withoutFilters.execute(), schema) +diff --git a/sql/core/src/test/scala/org/apache/spark/sql/test/SharedSparkSession.scala b/sql/core/src/test/scala/org/apache/spark/sql/test/SharedSparkSession.scala +index ed2e309fa07..e071fc44960 100644 +--- a/sql/core/src/test/scala/org/apache/spark/sql/test/SharedSparkSession.scala ++++ b/sql/core/src/test/scala/org/apache/spark/sql/test/SharedSparkSession.scala +@@ -74,6 +74,28 @@ trait SharedSparkSessionBase + // this rule may potentially block testing of other optimization rules such as + // ConstantPropagation etc. + .set(SQLConf.OPTIMIZER_EXCLUDED_RULES.key, ConvertToLocalRelation.ruleName) ++ // Enable Comet if `ENABLE_COMET` environment variable is set ++ if (isCometEnabled) { ++ conf ++ .set("spark.sql.extensions", "org.apache.comet.CometSparkSessionExtensions") ++ .set("spark.comet.enabled", "true") ++ ++ if (!isCometScanOnly) { ++ conf ++ .set("spark.comet.exec.enabled", "true") ++ .set("spark.comet.exec.all.enabled", "true") ++ .set("spark.shuffle.manager", ++ "org.apache.spark.sql.comet.execution.shuffle.CometShuffleManager") ++ .set("spark.comet.exec.shuffle.enabled", "true") ++ .set("spark.comet.memoryOverhead", "10g") ++ } ++ ++ if (enableCometAnsiMode) { ++ conf ++ .set("spark.sql.ansi.enabled", "true") ++ .set("spark.comet.ansi.enabled", "true") ++ } ++ } + conf.set( + StaticSQLConf.WAREHOUSE_PATH, + conf.get(StaticSQLConf.WAREHOUSE_PATH) + "/" + getClass.getCanonicalName) +diff --git a/sql/core/src/test/scala/org/apache/spark/status/api/v1/sql/SqlResourceWithActualMetricsSuite.scala b/sql/core/src/test/scala/org/apache/spark/status/api/v1/sql/SqlResourceWithActualMetricsSuite.scala +index 1510e8957f9..7618419d8ff 100644 +--- a/sql/core/src/test/scala/org/apache/spark/status/api/v1/sql/SqlResourceWithActualMetricsSuite.scala ++++ b/sql/core/src/test/scala/org/apache/spark/status/api/v1/sql/SqlResourceWithActualMetricsSuite.scala +@@ -43,7 +43,7 @@ class SqlResourceWithActualMetricsSuite + import testImplicits._ + + // Exclude nodes which may not have the metrics +- val excludedNodes = List("WholeStageCodegen", "Project", "SerializeFromObject") ++ val excludedNodes = List("WholeStageCodegen", "Project", "SerializeFromObject", "RowToColumnar") + + implicit val formats = new DefaultFormats { + override def dateFormatter = new SimpleDateFormat("yyyy-MM-dd'T'HH:mm:ss") +diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/DynamicPartitionPruningHiveScanSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/DynamicPartitionPruningHiveScanSuite.scala +index 52abd248f3a..7a199931a08 100644 +--- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/DynamicPartitionPruningHiveScanSuite.scala ++++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/DynamicPartitionPruningHiveScanSuite.scala +@@ -19,6 +19,7 @@ package org.apache.spark.sql.hive + + import org.apache.spark.sql._ + import org.apache.spark.sql.catalyst.expressions.{DynamicPruningExpression, Expression} ++import org.apache.spark.sql.comet._ + import org.apache.spark.sql.execution._ + import org.apache.spark.sql.execution.adaptive.{DisableAdaptiveExecutionSuite, EnableAdaptiveExecutionSuite} + import org.apache.spark.sql.hive.execution.HiveTableScanExec +@@ -35,6 +36,9 @@ abstract class DynamicPartitionPruningHiveScanSuiteBase + case s: FileSourceScanExec => s.partitionFilters.collect { + case d: DynamicPruningExpression => d.child + } ++ case s: CometScanExec => s.partitionFilters.collect { ++ case d: DynamicPruningExpression => d.child ++ } + case h: HiveTableScanExec => h.partitionPruningPred.collect { + case d: DynamicPruningExpression => d.child + } +diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/AggregationQuerySuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/AggregationQuerySuite.scala +index 1966e1e64fd..cde97a0aafe 100644 +--- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/AggregationQuerySuite.scala ++++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/AggregationQuerySuite.scala +@@ -656,7 +656,8 @@ abstract class AggregationQuerySuite extends QueryTest with SQLTestUtils with Te + Row(3, 4, 4, 3, null) :: Nil) + } + +- test("single distinct multiple columns set") { ++ test("single distinct multiple columns set", ++ IgnoreComet("TODO: fix Comet for this test")) { + checkAnswer( + spark.sql( + """ +diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/test/TestHive.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/test/TestHive.scala +index 07361cfdce9..25b0dc3ef7e 100644 +--- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/test/TestHive.scala ++++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/test/TestHive.scala +@@ -55,25 +55,53 @@ object TestHive + new SparkContext( + System.getProperty("spark.sql.test.master", "local[1]"), + "TestSQLContext", +- new SparkConf() +- .set("spark.sql.test", "") +- .set(SQLConf.CODEGEN_FALLBACK.key, "false") +- .set(SQLConf.CODEGEN_FACTORY_MODE.key, CodegenObjectFactoryMode.CODEGEN_ONLY.toString) +- .set(HiveUtils.HIVE_METASTORE_BARRIER_PREFIXES.key, +- "org.apache.spark.sql.hive.execution.PairSerDe") +- .set(WAREHOUSE_PATH.key, TestHiveContext.makeWarehouseDir().toURI.getPath) +- // SPARK-8910 +- .set(UI_ENABLED, false) +- .set(config.UNSAFE_EXCEPTION_ON_MEMORY_LEAK, true) +- // Hive changed the default of hive.metastore.disallow.incompatible.col.type.changes +- // from false to true. For details, see the JIRA HIVE-12320 and HIVE-17764. +- .set("spark.hadoop.hive.metastore.disallow.incompatible.col.type.changes", "false") +- // Disable ConvertToLocalRelation for better test coverage. Test cases built on +- // LocalRelation will exercise the optimization rules better by disabling it as +- // this rule may potentially block testing of other optimization rules such as +- // ConstantPropagation etc. +- .set(SQLConf.OPTIMIZER_EXCLUDED_RULES.key, ConvertToLocalRelation.ruleName))) ++ { ++ val conf = new SparkConf() ++ .set("spark.sql.test", "") ++ .set(SQLConf.CODEGEN_FALLBACK.key, "false") ++ .set(SQLConf.CODEGEN_FACTORY_MODE.key, CodegenObjectFactoryMode.CODEGEN_ONLY.toString) ++ .set(HiveUtils.HIVE_METASTORE_BARRIER_PREFIXES.key, ++ "org.apache.spark.sql.hive.execution.PairSerDe") ++ .set(WAREHOUSE_PATH.key, TestHiveContext.makeWarehouseDir().toURI.getPath) ++ // SPARK-8910 ++ .set(UI_ENABLED, false) ++ .set(config.UNSAFE_EXCEPTION_ON_MEMORY_LEAK, true) ++ // Hive changed the default of hive.metastore.disallow.incompatible.col.type.changes ++ // from false to true. For details, see the JIRA HIVE-12320 and HIVE-17764. ++ .set("spark.hadoop.hive.metastore.disallow.incompatible.col.type.changes", "false") ++ // Disable ConvertToLocalRelation for better test coverage. Test cases built on ++ // LocalRelation will exercise the optimization rules better by disabling it as ++ // this rule may potentially block testing of other optimization rules such as ++ // ConstantPropagation etc. ++ .set(SQLConf.OPTIMIZER_EXCLUDED_RULES.key, ConvertToLocalRelation.ruleName) ++ ++ val v = System.getenv("ENABLE_COMET") ++ if (v != null && v.toBoolean) { ++ conf ++ .set("spark.sql.extensions", "org.apache.spark.CometSparkSessionExtensions") ++ .set("spark.comet.enabled", "true") ++ ++ val v = System.getenv("ENABLE_COMET_SCAN_ONLY") ++ if (v == null || !v.toBoolean) { ++ conf ++ .set("spark.comet.exec.enabled", "true") ++ .set("spark.comet.exec.all.enabled", "true") ++ .set("spark.shuffle.manager", ++ "org.apache.spark.sql.comet.execution.shuffle.CometShuffleManager") ++ .set("spark.comet.exec.shuffle.enabled", "true") ++ } ++ ++ val a = System.getenv("ENABLE_COMET_ANSI_MODE") ++ if (a != null && a.toBoolean) { ++ conf ++ .set("spark.sql.ansi.enabled", "true") ++ .set("spark.comet.ansi.enabled", "true") ++ } ++ } + ++ conf ++ } ++ )) + + case class TestHiveVersion(hiveClient: HiveClient) + extends TestHiveContext(TestHive.sparkContext, hiveClient) diff --git a/pom.xml b/pom.xml index 57b4206cf..0ec834982 100644 --- a/pom.xml +++ b/pom.xml @@ -46,7 +46,7 @@ under the License. <scala.plugin.version>4.7.2</scala.plugin.version> <scalatest.version>3.2.9</scalatest.version> <scalatest-maven-plugin.version>2.0.2</scalatest-maven-plugin.version> - <spark.version>3.4.2</spark.version> + <spark.version>3.4.3</spark.version> <spark.version.short>3.4</spark.version.short> <spark.maven.scope>provided</spark.maven.scope> <protobuf.version>3.19.6</protobuf.version> From 8c532be3a49eb6bcb37736190dd28da828ff1788 Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh <viirya@gmail.com> Date: Wed, 29 May 2024 21:05:02 -0700 Subject: [PATCH 10/19] chore: Removing copying data from dictionary values into CometDictionary (#490) * chore: Removing copying data from dictionary values into CometDictionary * For review --- .../apache/comet/vector/CometDictionary.java | 102 ++++-------------- 1 file changed, 22 insertions(+), 80 deletions(-) diff --git a/common/src/main/java/org/apache/comet/vector/CometDictionary.java b/common/src/main/java/org/apache/comet/vector/CometDictionary.java index b213b75d8..24a6d6d8c 100644 --- a/common/src/main/java/org/apache/comet/vector/CometDictionary.java +++ b/common/src/main/java/org/apache/comet/vector/CometDictionary.java @@ -29,17 +29,8 @@ public class CometDictionary implements AutoCloseable { private CometPlainVector values; private final int numValues; - /** Decoded dictionary values. Only one of the following is set. */ - private byte[] bytes; - - private short[] shorts; - private int[] ints; - private long[] longs; - private float[] floats; - private double[] doubles; - private boolean[] booleans; + /** Decoded dictionary values. We only need to copy values for decimal type. */ private ByteArrayWrapper[] binaries; - private UTF8String[] strings; public CometDictionary(CometPlainVector values) { this.values = values; @@ -59,39 +50,48 @@ public ValueVector getValueVector() { } public boolean decodeToBoolean(int index) { - return booleans[index]; + return values.getBoolean(index); } public byte decodeToByte(int index) { - return bytes[index]; + return values.getByte(index); } public short decodeToShort(int index) { - return shorts[index]; + return values.getShort(index); } public int decodeToInt(int index) { - return ints[index]; + return values.getInt(index); } public long decodeToLong(int index) { - return longs[index]; + return values.getLong(index); } public float decodeToFloat(int index) { - return floats[index]; + return values.getFloat(index); } public double decodeToDouble(int index) { - return doubles[index]; + return values.getDouble(index); } public byte[] decodeToBinary(int index) { - return binaries[index].bytes; + switch (values.getValueVector().getMinorType()) { + case VARBINARY: + case FIXEDSIZEBINARY: + return values.getBinary(index); + case DECIMAL: + return binaries[index].bytes; + default: + throw new IllegalArgumentException( + "Invalid Arrow minor type: " + values.getValueVector().getMinorType()); + } } public UTF8String decodeToUTF8String(int index) { - return strings[index]; + return values.getUTF8String(index); } @Override @@ -101,65 +101,10 @@ public void close() { private void initialize() { switch (values.getValueVector().getMinorType()) { - case BIT: - booleans = new boolean[numValues]; - for (int i = 0; i < numValues; i++) { - booleans[i] = values.getBoolean(i); - } - break; - case TINYINT: - bytes = new byte[numValues]; - for (int i = 0; i < numValues; i++) { - bytes[i] = values.getByte(i); - } - break; - case SMALLINT: - shorts = new short[numValues]; - for (int i = 0; i < numValues; i++) { - shorts[i] = values.getShort(i); - } - break; - case INT: - case DATEDAY: - ints = new int[numValues]; - for (int i = 0; i < numValues; i++) { - ints[i] = values.getInt(i); - } - break; - case BIGINT: - case TIMESTAMPMICRO: - case TIMESTAMPMICROTZ: - longs = new long[numValues]; - for (int i = 0; i < numValues; i++) { - longs[i] = values.getLong(i); - } - break; - case FLOAT4: - floats = new float[numValues]; - for (int i = 0; i < numValues; i++) { - floats[i] = values.getFloat(i); - } - break; - case FLOAT8: - doubles = new double[numValues]; - for (int i = 0; i < numValues; i++) { - doubles[i] = values.getDouble(i); - } - break; - case VARBINARY: - case FIXEDSIZEBINARY: - binaries = new ByteArrayWrapper[numValues]; - for (int i = 0; i < numValues; i++) { - binaries[i] = new ByteArrayWrapper(values.getBinary(i)); - } - break; - case VARCHAR: - strings = new UTF8String[numValues]; - for (int i = 0; i < numValues; i++) { - strings[i] = values.getUTF8String(i); - } - break; case DECIMAL: + // We only need to copy values for decimal type as random access + // to the dictionary is not efficient for decimal (it needs to copy + // the value to a new byte array everytime). binaries = new ByteArrayWrapper[numValues]; for (int i = 0; i < numValues; i++) { // Need copying here since we re-use byte array for decimal @@ -168,9 +113,6 @@ private void initialize() { binaries[i] = new ByteArrayWrapper(bytes); } break; - default: - throw new IllegalArgumentException( - "Invalid Arrow minor type: " + values.getValueVector().getMinorType()); } } From 1f81c3812490d35b488795ad597e4e6d3f9e114d Mon Sep 17 00:00:00 2001 From: Andy Grove <agrove@apache.org> Date: Fri, 31 May 2024 10:36:16 -0600 Subject: [PATCH 11/19] chore: Update README to highlight Comet benefits (#497) * Update README to show current performance info * Update README.md Co-authored-by: Oleks V <comphead@users.noreply.github.com> * address feedback * Add hardware info to benchmarking page * add raw benchmark results --------- Co-authored-by: Oleks V <comphead@users.noreply.github.com> --- README.md | 98 ++++++--- .../source/_static/images/tpch_allqueries.png | Bin 0 -> 23677 bytes .../_static/images/tpch_queries_compare.png | Bin 0 -> 27388 bytes .../_static/images/tpch_queries_speedup.png | Bin 0 -> 38821 bytes .../2024-05-30/comet-8-exec-5-runs.json | 201 ++++++++++++++++++ .../2024-05-30/datafusion-python-8-cores.json | 73 +++++++ .../2024-05-30/spark-8-exec-5-runs.json | 184 ++++++++++++++++ docs/source/contributor-guide/benchmarking.md | 113 +++++++--- 8 files changed, 599 insertions(+), 70 deletions(-) create mode 100644 docs/source/_static/images/tpch_allqueries.png create mode 100644 docs/source/_static/images/tpch_queries_compare.png create mode 100644 docs/source/_static/images/tpch_queries_speedup.png create mode 100644 docs/source/contributor-guide/benchmark-results/2024-05-30/comet-8-exec-5-runs.json create mode 100644 docs/source/contributor-guide/benchmark-results/2024-05-30/datafusion-python-8-cores.json create mode 100644 docs/source/contributor-guide/benchmark-results/2024-05-30/spark-8-exec-5-runs.json diff --git a/README.md b/README.md index fb17535aa..b0b72fdb6 100644 --- a/README.md +++ b/README.md @@ -19,58 +19,86 @@ under the License. # Apache DataFusion Comet -Apache DataFusion Comet is an Apache Spark plugin that uses [Apache DataFusion](https://datafusion.apache.org/) -as native runtime to achieve improvement in terms of query efficiency and query runtime. +Apache DataFusion Comet is a high-performance accelerator for Apache Spark, built on top of the powerful +[Apache DataFusion](https://datafusion.apache.org) query engine. Comet is designed to significantly enhance the +performance of Apache Spark workloads while leveraging commodity hardware and seamlessly integrating with the +Spark ecosystem without requiring any code changes. -Comet runs Spark SQL queries using the native DataFusion runtime, which is -typically faster and more resource efficient than JVM based runtimes. +# Benefits of Using Comet -<a href="docs/source/_static/images/comet-overview.png"><img src="docs/source/_static/images/comet-system-diagram.png" align="center" width="500" ></a> +## Run Spark Queries at DataFusion Speeds -Comet aims to support: +Comet delivers a performance speedup for many queries, enabling faster data processing and shorter time-to-insights. -- a native Parquet implementation, including both reader and writer -- full implementation of Spark operators, including - Filter/Project/Aggregation/Join/Exchange etc. -- full implementation of Spark built-in expressions -- a UDF framework for users to migrate their existing UDF to native +The following chart shows the time it takes to run the 22 TPC-H queries against 100 GB of data in Parquet format +using a single executor with 8 cores. See the [Comet Benchmarking Guide](https://datafusion.apache.org/comet/contributor-guide/benchmarking.html) +for details of the environment used for these benchmarks. -## Architecture +When using Comet, the overall run time is reduced from 649 seconds to 440 seconds, a 1.5x speedup. -The following diagram illustrates the architecture of Comet: +Running the same queries with DataFusion standalone (without Spark) using the same number of cores results in a 3.9x +speedup compared to Spark. -<a href="docs/source/_static/images/comet-overview.png"><img src="docs/source/_static/images/comet-overview.png" align="center" height="600" width="750" ></a> +Comet is not yet achieving full DataFusion speeds in all cases, but with future work we aim to provide a 2x-4x speedup +for many use cases. -## Current Status +![](docs/source/_static/images/tpch_allqueries.png) -The project is currently integrated into Apache Spark 3.2, 3.3, and 3.4. +Here is a breakdown showing relative performance of Spark, Comet, and DataFusion for each TPC-H query. -## Feature Parity with Apache Spark +![](docs/source/_static/images/tpch_queries_compare.png) -The project strives to keep feature parity with Apache Spark, that is, -users should expect the same behavior (w.r.t features, configurations, -query results, etc) with Comet turned on or turned off in their Spark -jobs. In addition, Comet extension should automatically detect unsupported -features and fallback to Spark engine. +The following chart shows how much Comet currently accelerates each query from the benchmark. Performance optimization +is an ongoing task, and we welcome contributions from the community to help achieve even greater speedups in the future. -To achieve this, besides unit tests within Comet itself, we also re-use -Spark SQL tests and make sure they all pass with Comet extension -enabled. +![](docs/source/_static/images/tpch_queries_speedup.png) -## Supported Platforms +These benchmarks can be reproduced in any environment using the documentation in the +[Comet Benchmarking Guide](https://datafusion.apache.org/comet/contributor-guide/benchmarking.html). We encourage +you to run your own benchmarks. -Linux, Apple OSX (Intel and M1) +## Use Commodity Hardware -## Requirements +Comet leverages commodity hardware, eliminating the need for costly hardware upgrades or +specialized hardware accelerators, such as GPUs or FGPA. By maximizing the utilization of commodity hardware, Comet +ensures cost-effectiveness and scalability for your Spark deployments. -- Apache Spark 3.2, 3.3, or 3.4 -- JDK 8, 11 and 17 (JDK 11 recommended because Spark 3.2 doesn't support 17) -- GLIBC 2.17 (Centos 7) and up +## Spark Compatibility -## Getting started +Comet aims for 100% compatibility with all supported versions of Apache Spark, allowing you to integrate Comet into +your existing Spark deployments and workflows seamlessly. With no code changes required, you can immediately harness +the benefits of Comet's acceleration capabilities without disrupting your Spark applications. -See the [DataFusion Comet User Guide](https://datafusion.apache.org/comet/user-guide/installation.html) for installation instructions. +## Tight Integration with Apache DataFusion + +Comet tightly integrates with the core Apache DataFusion project, leveraging its powerful execution engine. With +seamless interoperability between Comet and DataFusion, you can achieve optimal performance and efficiency in your +Spark workloads. + +## Active Community + +Comet boasts a vibrant and active community of developers, contributors, and users dedicated to advancing the +capabilities of Apache DataFusion and accelerating the performance of Apache Spark. + +## Getting Started + +To get started with Apache DataFusion Comet, follow the +[installation instructions](https://datafusion.apache.org/comet/user-guide/installation.html). Join the +[DataFusion Slack and Discord channels](https://datafusion.apache.org/contributor-guide/communication.html) to connect +with other users, ask questions, and share your experiences with Comet. ## Contributing -See the [DataFusion Comet Contribution Guide](https://datafusion.apache.org/comet/contributor-guide/contributing.html) -for information on how to get started contributing to the project. + +We welcome contributions from the community to help improve and enhance Apache DataFusion Comet. Whether it's fixing +bugs, adding new features, writing documentation, or optimizing performance, your contributions are invaluable in +shaping the future of Comet. Check out our +[contributor guide](https://datafusion.apache.org/comet/contributor-guide/contributing.html) to get started. + +## License + +Apache DataFusion Comet is licensed under the Apache License 2.0. See the [LICENSE.txt](LICENSE.txt) file for details. + +## Acknowledgments + +We would like to express our gratitude to the Apache DataFusion community for their support and contributions to +Comet. Together, we're building a faster, more efficient future for big data processing with Apache Spark. diff --git a/docs/source/_static/images/tpch_allqueries.png b/docs/source/_static/images/tpch_allqueries.png new file mode 100644 index 0000000000000000000000000000000000000000..a6788d5a402647856d59eab220c68b80b1a84e81 GIT binary patch literal 23677 zcmdtKc{tYV`!@P9k7bUONT?)~p~;jeLoy^G8Ov10gv<(&L?n%7MG*~>GG!`MhEzf* zlu{W|B*Q*$ty=5-9q+sMaqR#0?pWW|cNw1N^W67!U)On^=Xu>x20OORV-;kjC~BUz zmWB~U(S}eIO%F38zM|0F*@wS2cxjq??R2yE^0D#UPwClsxgT`%I_P98;=SMVkdvFM zob(21St$`mFE95)8)al%{{Dcpo2P?}JiXRkya<cCmf0bS;;<qAqsdmi>O@g{K51)g zHuk;r<&BT2@wN`eVgHYTTv=NUZ0T}M(@ir{S9nL=)NB^eV>aPUY*XN>G0iw#U46EC zw_VH{8a8k7)YRIemzgCs#S3V4l)Hs%zZKKOgbP)a2EQtNHTdkwGo|H2mcz0io}5&V z7h}=DpG*a>`i+z*{%}XLh;VUnojz+rt3|%C{~X1Q|8`m^5kh{@JmUZF-)`nvqZJUK zr0CKfma#4N-TS8(AJ^4|78WXWzIpTZ%a?gpR#pOnf+v{6BO@cV2A#YZWu>L(TUuJW z`}jP%Y4@n9<jA3ui`LCQ^x;_zlW+i~9WQ(MlkxiX>#ZLiWqADb>5F~qHH_<dS!>uz z=L>BNE1#Lv&D*|vH~)?uI|kl7SoiGGhMl=X+hUj4)WwTCc9m>6qa%Lu<Vk}E-YzEv z6<BxQu=v^77BOFlRZi~qhgucq*6i8O4ntj~0xALPKfSolw0G}b<9y5M^DfaRPH+ke zvN||82rOKfw)q(2MEC5>ZrhHD316?ToeS;l?F(+*s_c1q%wzoP8}5!PSy?vqXI5rg zm8(2?^2B=IzOV}l2W;C4Y}R{!K4;!>c<?3PuOT-(yJJ7TbSzU+no`YAioA1ds{<7n z7|6xT8<w}HsIsMH^!%6f^z?#*uh&YM-(gM({Bd4li)gyhSQUqAc4>Ek4IO@acW&p4 z7knJTD)bb)fGnk=qOw9tN&_oyl4l-1JnTL-@<`aRHJgj-=_>Uf=q#4AwX<u^+p~^+ z@p?`?^}T?fYsucYl}|4?a^#5m4UgX1n6{E5a)Rqz*_oJ*f4F;j<6+77jVZBQ%eJPP z-L{|TPqLDA?@?R9%Q`mHZRP4S_;U3fmyQKGS=`^=Kiy(tvY>2gRQJ**pZROntdW(K zjq9kbu1-Dh>W1%d56$HGP^`YnPx^?62rVtGif5NM-E@7kaj{B(fJ?{SUD5loUGCoA z5n|d}dU`eY4-fYA_8zkep5}XV|M03evsxO)BU*xIl9H}A|Ijclqo=3WYMi_o9~UP= z6SqLxBHN_wz$<B%5P8o&t>Rp~WBjR8r|w{>ZKsv|#<VWT+EfPDzWeZD`S}YMRBkT` z;pOBUPMPj33*0mFy0!4Y?i}+%XYsoa4iBEq%+Jq%>0oGhy7h|Qq8#TJ*NirA4z>6? zG$gZd(IVY-Ze3>@u=2}k;>v!$UsNomti1PDQBl(8&!00VCMIm{7Vz<DFIwk%#@5!h z?^fY~S2|Q?!gBp<g?jq>Nn_u?XNA%+&3sVcX}+kRQ55q1`*#}Z=g*(RV`J4XuJ5+W zR_1bwMJQ7gH#av$H8(d;d@1Ud4VZABn4Da2z|GMyQl@;$$TVR7{P{9tw?8IX1#J!r zQc;;6+gZFl&7h*YTe!&ejk>Y%8H+NP4n?oQm-BEIJkKX3@jJh|5t9-;V<yeU#<s=N zbM0B_y{!EFNw#*9ysRX4o?cM+)i@!wdbQ2LgY3%6OJjq03^y$ATOL|grZO}2O`mHE zmm$NU>B+;CT7P{a4fVkL3oXUIMCqRKWVz3fWzN_)s~ssS^LFmsdGG%HYl-FGo^A3B z9DJGQ@#}|6W>!|3)AK7elfEA=`wagYDsLDHp8mdDV9}xs{S*}$A$@y$iLY-SFloof zojn_oUpC2VW@a|gbBx3CzKp2fH|AeEhmUcnRE&2A$Nv2GAzCm#vOew7rNx~Od^p0x z!|!>0<gSes=AZ^Tiey3_MK7aLgQh3*l})_86?60QtRAHBpT2a7eb1gf2-`wWe}8|f z;_@b+K5zY`*#|Cuej7ax9kRj7>Ly92?YO+L#$H5N$v5?`<FmFlfh$+8IJ=aTl(cp& z5)`z>8V$7F>|JHKl&e%bOgw~vfx+~){k$JT-LVpS7kGc!6#9<X8m4g?F1)bekc9oS zODoHZ3JNNYGO+cTNhj(sP_paR1%J7T95g(bUw+nfcY9P*)o~8p&S&+HA8XwRo(!J7 zo|l%k)L3#?)T;AL97|4G1^v_v66WL#;ag>rU1(P9DUN-dN0Ac}t2z@It1`*_{{8!W z`3x-$w*@vVJy@RRdmeVwio4-e<+HXD>FZuSKE15D$jxeCV4$L<g&QYgmu`fVloX|u z-0#=*m8POsW2Kzj&b%kjp6!?^x^ctx=6PAYHHwPT9;>b+b=G~?pB;+6qp)t>HWL$* zo8E`sKRI1gaKuFP*RxGS*>CKZ$XZiK=4+f=S0LK-T(Z`1ISJ!I9v)-ToTsVLZ<=Rb zn0)b!ve{?%ubmt2I6Z$~9O%+jval**-p*ACVjsUB3t;&1<Hvg6VG%KHqhA?6>Q;9z z-}6|h`1`NSDW|%+I%L*(9$S8X{!7aK3vg7Ni$WL4I}W#ud>j}^4gB@P^zOlhH!Mp} zAYqhw32ThK=M3WQ#JMYQXk7ABQOj()Kq7rK$AA1Q6(SrwHKN+#=kIUVT^8t^f2$?i za9Lkp-{VGE6)qv6)vxF(_PCWY95`^`(#FHrOofrcJU3ATFY^{HQ3;3?+T?vk=OlKV zGI*CO%H_n54>|p7{O!k&QSF5XsvbSsHZ$lmG5q$Ntj*DE*~sim--d_PK0LcT^2Vpz z27lLl`ZhW$>+w#5igkESBFb`qmP2E5)2mmHURedNPSTDKD=3gH`TlZ`<Lg^l7GDPk zqu)I6S+;VewsiE#lhui<cK6lDQawXM90=r-!m4G}&-E?ea|}#P1$=(<fS-|(vBpG_ z=kyOQZ#Lhli4nZ+He1{Dc8(OBg{88xoY4zbbMvusB_3L&cu?)?j>}C~^@Y8Lx`dLG zlVueYYLI<yIz5*VTfUsFgKfA`b@oirm16HNf@|gEMt*)sp`!#953Xo5MP1=Tji{;& zXFHve5^MkT0{Lv~{l-=VO@3TH@aa>1V9<}(4sEygv$Yqw#iE9bu3E*$#xJcgJ^B5G ztJ5AcGn+@Jga*1wH{k^%XQqE;psbLGH8SG9d+;?a<@}39g=@)@Q#g$@z5!=CYhxC9 zeE+m-`}wt+LC@kN@0`mPy6WQMk|pQxOe<bY8&$03``{~a?fA^WSNW<wMM0w1)?4<A z#W^5$PLpbsgExpjfByVFYip4g*LLX&D*N5>`qExFI~Cw`nSQiyUrfAZP*9Mw3ld}2 zk#Fy{@k0sh?ChC{C>_<A3A?E2WYu8HPpvr#`)qAR&CJbpjxo-QH!?EPIe78XB`Ji2 zwyUda{LPy;_c=PQD4qV%(Gs_GOXjj|r=}cAwr<^Ol(97?bJukfZB0$h*&|O8qC2(X z&mA9r^7JVeKmTzQo`;>CixCSo9d6|@JFo6|*wL|Qe5jkBi;;lGmpnh%tuHFN6xdWT z{w!5bQ!_N=Qs1(rxh&A%(1w%1No5<kkPnS1$FE*pj>3JCj%j`>PVPPTx3n$KpL<MA zc(i64ZqX2!gPi)Nw9HB>7d>lE@U8d%A$9B4ttR9PRMRuQVg#FVIYkuE%dT0&h4Lw$ zJbC)uIn#HNiUiC`23B&b0TLlZs%mSu=1pPq3U1!qY-5uaDGU4-c_+C*Q>3N*)HXlt zxvmOznS4=Rzjx7QCPqeD-;z&0lVcz8(x(Sm+1PO4^1R2$8*(|Ft>vkBepR2&SBcIS zNkTwC;PGL^vtwI9T0rhj&7H=^VViuuR&;ijRKM0EPZBJqU3rX&qo=2b5)!gmx21v4 zrjeSzNRhqa>}vmvQzF>81&f7*8aA@yr~MeTwY9Uc%4Ww6CBiN<<QlBdxg_53>{;zV zbH+{kr-FuQX9N!&Iz+{mtGOB3>S}0=N$kuqcK7l+6C8UWW33fKz)+WR+w0e9si|rj z0$Dm)Mw@etMdjoY2zobUc-Y-dYftTx<P+15A1YEUeTS6KSw1`Ce={>~ZUwgW>Z_QW zn^)G?v$BTMA~kKz(n;*wohZ3$`0a6FQ4tZf=Qaim-l&rS!w;FVEsEEkjEONU4+<n{ zsQPY7z$gnZD+B8FHIpTXms3KUfDXBl5NeUyPbVgR@AMGYt~|L=&f3k5!_Lmm_sEgA zfUjAH&9LM9xAw|~ppt}%X%94|(cN|FU|h9oRq?Zm3JP(441cavrLuk+DxckY-6XFG zK^1oVcvXA5;O=}&O+4p`!Z$bVm}Vw=V{N;yl3i$MV8c<aI>EOp)hI{SwR1hY&_>=p zMXrJG+tSj~Ov?f`yv(=a094+dq9WAQ)pgHzM0EUym}okyUN4I^Py?DY78Z9;&v)J{ z$BY(8nNr)fZOh&Jz(o3xx3~1mufSFU%6{@cKEFP8WAB4&FTQk@E}ER2G%fbbD(O^} zl474HC|`j^G`Ma5)OH$KGfOwgw*8LN!0X$~DQlFKg2KWYV9Vz6;Mtav?FfLhFqfd6 zdFBd>H+i!Q2Mp7g-#r-BQRGHP1x>tV!WXw23m8BA++N70cTrIjmBGf&j?Tx&C)cvn zJWmgYjS+P?HC3b)u-sI*xVRX&)}Ujs4(NbN{`p2%OG`~hhZPyUZ*7)Q6k42Rl9Jct zrDGx^v!5{QQbkyCH#aw6on`24@at5_lPBAsTwJG5-nR3e2OZX39aWMPtchR0?4F!k zV_F_m+G&p-r|J3gs&i}XbUtTnIX+)YTl*o&yGcpM5nhXSUfE7V;V>pmoIi7hmkP<t z%QG*=zbv+CXc!x^=u)b)Q-{LB!)?BQdVvI+)Uc<}IsD$e^@a#Ga-@)0?Yc^ikX8$E zw8plY89z)zp+mTdo+Ecp(bw~_{P@uP$g7XiHKVO<ZD}`c+O&StrmB}OO%C{kEEg61 zBJsJXyT2i+8F6LXk1k;0!iDLTsjRhKT}zM?o%i3kaYJkSc6w^DvLCOLlM}Z6XmawB zv9U4tZ}0WCpOd9Q-APiFLramFnc0KAzZX2Kf~pe!z-Q<kps%!t1>TcVb9TOFegyye z`{BNP{_Kt<ZrhJ);{RU8CLNXO?P!04^M15-n+u&^P!uvmEf8-MD#c?NCMG6&1_qnP zWaXO4+l7VU=g%)xbn9YYwrrW#=eC>G9+xj)wsCMc4j5LvJ<Dk6#*KnYmMqEg4aX6B z$FhmlwK|eZV3W7h%Um<MaP?3e1QAI|R-WZMs>eP)*9_mx_he$ExB1q7U2H4|4wDMt z|1h5a9a1_2WqiY8A<nDGuaf}N=oAdD?hq<Ih!l)cFOC+mz^z+_Jl)LH&w@L5&UD<n zcTYr2jK0)=JVHIK;+{wEFiLO+KH;dMERg@ei|a{|x>)`e0f3M*D|Z?g8`JCS>jTTA zqLq@Bm#?ml;#uQ2=1BJXd9IlpJ|f-2hY!!mIn2Lw`SMM#k4Ef*@&j3uQBhG3pPZLN z5@7(s3i$SPoq&>Wf%(OTB$;IZg+Z!LEn@o|9PIiYpML!Ku^Qk!^7TVyDk?I&lE-_& zFb}%Mdq4>&BR%imGs?-yovuofOcn>*VLUQCUR1HLVmV5wwX5s=Lx;YxUY;WPewT6O zf>2&wUTc&C1Iff}%hHY4cIVH>;X#kER8`f=_@7eM#@<6xbl@-Vu!xBI>nk$S(`665 zl<F)G7RH(3=F<sZGdos#Ykp{b6Ysd>ej!>euF^EXa<pI7SM)A6W$E%4=b~FjDz6Tq zq5Dm~u!w$MP=HL;UR$D1V!5J1?a%v}(yj-LeR6rzZWTHK3SGfS4ZoE&if4MI2=Yja zpI|9NCty?I_^<JyZAhQZ1vXo#ir2Rta3-0s0eTUeNs-lWcBIWVmVD9mYjW~hsgIXe zYu_;Wx-o5xL<m~b7yHJ^e~z(+Dbo>NNTZ*QM-6Wvy1I&<lnku~SsF3&hHLj<pzQbU zJJUb+>*m^~wD+tZ(jM$8F$<cV`8AZ`*<bHFm{;Wf<FnacD8OQEYI?3@2zl%m2`gmt z3_vMnR#tAl-*DpjfJ%tIuD14Ro(cfW-s$n~ZGeLYrGAIy-MTFPtVJ8UHL`!)nKSeh zo+tzjB&8NF#=Bs_3D#re5eHc=(tLh?m4k8q;z-oiw*UhJEm>?-56(lj+3hQU!1x15 zK<^Inh3s*Bd^{mh5F?Zt`cKlc{`=6WI2aN;E}7;A0K2BEljRn5Tu%ca&wt<RBQ1y# zB<R6n7yq$=v^oRw(gVVRdX)e7PvJ;@ZEbC-s8@{#uNthl#~Ft2YH?zzfw#nGE|MRv z-x03NHi7g)no5IW&;6wJ1TbEI_|q8+8EtO+C+BF;7vif`sBd_wU4zS%m4$%wj)sRb zpebI17JT3S{ZVDS<jK+`_t8$E8_fW!BiPl~cfGV_Nu*}<f;6Kn9nXtRO$Jg{Ws7ZW zY$8vcipa@XSyEa$GSIxN_-X3qWA1M&89sLuaUjFho?f<1*0b;Nt$Sp<MQwNiX!i?M zH(Xe|pY@cm>M3w$XHD^Vix)2je?fnw-vf>EK^GUon}B~29vK;NKXj<7E@64|9jBdC z?7Hk9BO?z{r+|!xQ#=rZD-w0~+1poER;G_h4-0?O?_Tv?CFk9_ca~YXTSP?tmN`w1 z`)C?0b)bB{w1?x&q`P&O0n~*)c(BPN-*WY<<ziwY;^LYrF7y=0vkU9oPGx7Wpwx_w zd9f({eH;|(UTQ|hv4H_c5B@C(WwCQ{^aZJ@6zMXYot<;We0po=<(e0=A|+QQEBm8? ziCtq?YYhxRElJtLPTjqG*Et0lIdL#KcSq0#REON$+&;zGvllP2CF*4TAPaNNF-$DP z&(Ck7FG0UGx!hj}JL}kaZw-nq6Dsw|#T&SnZ9S=GYHFG@HY>Gs=~ASX(T3(`!;D^% zaGZ`7?5P+k`8J;rf|7x+J36YKKGn;60_qG2kfKP1S)tf_?!pDhSFc~cUb=d<US1hK z!^e*wJue2c&7Z&3-u}vzj*gCG;^0y|4!3rjp?gZu<VspPy3=RQgrWDZXl*slD?_f4 z>D-On<o<JXf8niFyKki8Y)}-?5b)#DwQIR!1Qlq9Cqf&sUa0kSl#-HC-09PyZEdFV zPA&A5drt)oFCSkT+7R(m2OJ%lw`{56Kb3$8&Q{_+OMz$*rey7(2oSPIEj}|zg=HVk z3#lSlisOh#(zH(e=x`fB?WvzW1@06Ta05szhJO$l%ExGw7|Jr(z}o?X>s3{S(VnQ! z&P-c5GRR`F3^m22Xtrx<ttc!5YZ-yIS7iNq0kr+0w{Ob<gr2~MtaHIf4*UH1AX#~o ze8)HU6|gQ~I>m~9e!Q9@i!|!%=cm4H8&g?X88G3IoEmKehw+kKj{~U&wy$(r7penM zMdXkDqFB3_hK~yesbSd=wjmeSxzSSOeF*ij?{_88L*jFOAxY23So!D?6W(xCF8J)( zv!?gGa{kRT!`Ji6DuH{~fFdT1Xg}awP4S@*8sQ`<XMRUvAi&edN5;ezY!9CPKk4W8 z?L$$EHUl|Tym+zeZw^$H4Gj%tW@fIyK@`#ECN;f$cA5hLHu3er(0^W7IB<*$pnZ)? zyF7}pbx%d8ySsbEt^Eysdg)Ol9eu;QQ?iatBB;B}8#ZisTwi}QCMM?bTk>6I+7ud# z+x9U5f{{^CED5Su%Jj9OQZ(!=ESo_&z3uA@1D(?QV5qDLz*l3QK0c4eEazFuveb`6 zFf$8FD1MPNyRA7pnUItSTP`9=x*yTJ-4YHoANu-8DJ8V^aRe?)yclU_yae)N9B2|G zMx?AoMW6PVRnYQ>5-#ERe4)y;Eyvv_$DF|BrSE;<-3AJ_{jSTZe_nFA6wPOl$spI( zAT3Ry6alvID#(+=rmWc}7ZS}cL)r=Cd&0}uVm(Mudi2-x87?I)D;K>)2%!;>kv-5{ z;p3$4Z#avkP%HKOZcymF>(ADtE!wObABN_zw<&E4A&JoaM5L#Sn(o;X%|2I%t^?^J zm7eN-6uTwMD9xrS!gu_Q4}r5Frq4Fgk*1bKOp3<K!{gy2;oy_tyVY`xPY9BCi=5F! zWVydpH{3!Vs`(J}9(r2pIEV07@c4$e?e$u7&3Kgk#wwaKwrNJF{RtzG0?Uy10Emy0 zR0hB<tEd=(?px8dbH$$}`n+QO?%lh|0l|~)US(C{y9OCG29g1n%^Arh4BIYLXY`;y zKDO;4%O#o;k6y<6piy&6(7A&h=lrKw^PHtNZrmukcI_?GpD3|!2L}GW!9l*^Fcuz? z4K8gs#JGCTU7&2$pvAywRS3NQ1n11mOu_BjRe<or-x^ive5WjWkS&^TJLvyi<!ukY ze|@tEcyqx9F|rcV(M{xdBJF&JC{uu<X0U=+rd1JScS0K3z-CqX3zYQU`^E`xc*uXQ zYI39<`0(Ls-B<Ek1ALBS3oCHKV(tQ^il&3Jr!N4iKc^b35}%Ne@_K8$7%72!?|VH2 zW>;7^pS<bjs-h4M$T*EEzu3{-siT)8Wesvq#pB1!!L!rrNRdVT8Tj~dDRL6%yqgkL zBq0;1kh}Bhd~_iZ)zumYU*BTDGS)=#EFT_zyTv%ih~ewkucoDb3It1l!qAIQpF2VZ zV2~0IbMm3i95u?(Kyd{zs#7}q^XgEdUy8BEc8`3cmk8L>)AG~59*FjAZ6d$+|0LUO zHKi5c;tI8UbgFI$%}CVN7(p9y0>FEtQ}`1^n>TMZ$T6lupl#VyhOrEfjBHj{r$t5i z&(Dy2b?6Wsb+F^^Jg6X5B}cv$c=qc6)0G=rVTq2ary2Mc-@Ettd+ZWrbu~3ga`z2( z04#_Fq6!MT@?x<Vpt|@ZBqRt^V_?9^%F1dvt4{W)enU(${ir-09UU?jMM_KT(k&4b zMXK-lhe?J-u6sXsms<tpUmc%KS{pRwJB1TWcs<|%e4=obz{wu;JXA<!r8;>JZ*TAQ z9`ER}y=(S8<{TItB+X_1q4Q+L`T+wAz&tavvulvFz{7Lw>BV())D4T`qt|v_FY^jU zaYu}<GD$vvJ__yA5<qwjq#=p|Tn@i}eH91@jysOcwBT`TYNGjX2G0hywYR4mW^6sT z$wyj3KlvnBzK0bRJ>R9t!OmXH0~!<!7HObg=IyzO>OrVgDg>e!B)gwkzaIE~mjtdF z#;3AtO5HryrB>J0ilQJwfb`sherWxM4NKAGQJj2y;gjD7N!#n%3)M1FCmf8X(xE4v zofg1Yh5QI!L(OFe8Jp_rSdfdPP0IGw#V38ejxugq<SGQ}@K|5pz9-4bi(K08JOoF7 zIzApdO=oIuel4)}<;w*tSFWT&0AvgMhP0;>?mJ-PAAWv)+iIgFIX<HrAnmAE`Bsm> zG<!gTk=k>YT~SeynS<kaLITgHmwCS`r@cP5XwLWmu@ceL_`1nFwEuFbNeWyWDO7M% zmzrCM@{Ej(+S+A!(H43eoHk*VllWlE(CvZ{0l~k3>cQKd+i>Xpf$uI(PK4R+#s9E1 zTP!V?tX;ddwLSC7mHI~Z_SdiZ_V7*h1!_0)sX#no26_5t9i(V30l%t%eZU#lcR8(z zjT}bmEivhataC4DdgJFWUz#(uc@QMa)~+?WQ4SfBn}_G^+qX2-@bGX2iZ*m0iK5p< zZo)W_eR<B<(#S63N>`c`NQ;OKDHB)7rwE2pV=FY_8YEqV;pr?s#Ds1??8XggMs@)T z9p&4q<I6yPL%Jw{5Q7qkHwj&^+MJ>`9{xnXZrwU4SOVw<3AX}qf|HYz0^~9({=M!x zrnHV&1?(L<#p{VFDuL#fTvO?;Z|;ZF&@uKPAs5`eyA>H1GHY-JNVAIGaB<}8;qTuy zU0tOK>i_=Asv+goix)M}x88!h#~<L#%Em?xK&m6tY~t7p$qxkhi!xqV?;)C=k#Tc> z!-Ccudlz|lco3ZjT(a!G$8oQM<`+o)MfA>nM=E|^wQ{vT(<QaD^ZJ>?%o-YQVtTVP zH<-~>u6OHF0*6r1+{}d)M+(`}ao2^g6WcE;>it1C1BUL`cjwolU-OumI)dir;hQ%? zK<V6k=%bO1hgCu<dgrBGzH~{01EeSr2XZ{P*+^_pCE%_LU4OikWr^v33PQ3<pdcg~ zDg-ps3f>dk9m>})k*(87*~(J^I;MVB#cLo=;y6CDXga^xK0bGdmxK($18`N5cSQ-- zrl5)v%!9=8_lq4&?l?sgpKmwDB_~TeYmrxw3@vc6Hpw*&MUDWuK|}b=oSYo<KnsgY z*>gSOoeRi2=;qQR=HBrGAd^Y;?I4o=x-=@2Y3M^vo@5UITwtC*|7;4w$>`{+FC9gi zb6tGq3qZEunV-vvaLT~8pr=8mTx082M#}ovk1zQ71*p)?2hO9Sft&@GwmWx6yciB4 z12|sLL&n#2w{O>Ybz`p$XtPC&7J=RY<mxH5%es1%<Qs%zNLH431c&e`WL%>XLD3eI zO704<5rj|0Gl!o(ecIwHByi{mk?6Sx5-%$q;qDIj2L990z;?PNOIKIvbvC$W;4@8} z&K;N5he8RlL!kg=SbOLZWHL@^i+dpu$!u^y)_e9bZ#}ixO!AewMmrF-?`WTP01N^( z_1%7ht3kj0eisCS4W5rwOUYf=jB{te51HO^<O1`xGuIXF2tw!vO^s}mcWz}S1ssR& zCW=yDe}Ce7OIQ!E)WgucNDo8$g0jg^s}8o`Ve$Fexe}>H^b3Hkyu5svPL`xe5C9=T za&sRK8bm5~a(Hx9(j+M%;TVK+TPV$QWkLAg3@rWa$Bzh|ED)jmI*VnHp3w+6w%^II zc;Da8AS5KDv29yYB_G#mML?$?1I=_eW<(;V)Y1$@AZtL2kAT2W+6K#ZG`yT()Z^mf zxCbA_3df#Zy(b+yjQGo|nVH$*!BpRP<9#~t7Gwhp*D5P3o0^%8++g7<RRr$_JQWAt z4Bt`;nBW8#xdm9=#?|$dOxc7wIb<^8D_61r1=Q%PPG|m@8s!1`U5QUs0Qzm@A-F10 zn_rY<q7FeqU@@x*SDBkDCGXmKuexI;s;1__gHkOmEkvLx&c#M=JGJ;Bcz4o5gOjal zX)*e82>q4g^DFb<8K6fSTU}k9{o?ACE3|4+MJH3vT+PbLN#x_@6;W26KA*Bw)Xxwh zw%okPH3l^?ddXP?7233S*g@z0KwHP6quEF)#lG|I=`n!<A!T-p9wZSH<?HL)6D?Dw zW^R5VV2=;TB@(|EFYbxz_4D-=S+<N0>X&0nCJP0iL$^XwQXO^>-Ct{?v>HHxe|U2K zTj%6#`~1+zY^ie>FJ_=`ao!JU9a_gN3(J<1mr%vPOVUuVn;gA(aS>tM!J!}~t-<W0 z0SkdBPKiiJFm8!p-;6G^3lU3N6M!dyHMaENH{zg10u7`E&(5IrY8u`Jhs6$nd%H*T z3&)X@fT=^kEx_aR5FA^V>HYin+e-b{O-)aevIEz`#Lu5O(>8dLV<0~p*!roSt0_U1 z*@hWCA3w4nu7Ig*e~$L=_h8W&i!da>@kNhhA}e@#O&p3fFu2@e?M=-2FXhtuD<yc+ zGG|<y@6kW}Wb-2J=jG+AXww{MW-o$SvTsa|`S|>5(*EClZq{}lHR|z`CqSb1X2<8C zq5{(S#Lx#5F>Q5_W<;?dg&yHG#yuf$)VtS?S;I{l(HJkT+uAh<jUQDAYUV!)YEC{p zOG$`{MGBrEnK(J)Ny;3zD0(yODfI)OpEdklc)M$N+1{#V0_>e+ASQyyNA{T`#}+JF z6b(VKqOy_}!dDQ<(Wl=tU!=JMl?8w{7MeAhZCc-e)Jd!tggrJXi+cvfD@egG*8_tK z5`n1R8-CI)5su|W`A|6Ul4P1W4pEn4V`X*!*pg-2pCr9k9j2SV{rsK^L}b<Q+bRNh zO5i4;LO_~4Nl_KX*LnVJkAi;oC;ijj#KGl36%-Y11JIS&e(p~LjBd5Hx%pwHc7iPE zY?M$c1g??nxz>9?7$@TDnh^qtBsVFs%yEZ*>DTRZJ<uKYf7scK5ls;UM{t_AUG2#u z{cDIZs;9^Lq|l}Xpja6w7>$^DB(_%|t;B)7g6b16{rwg6s6^GIbLW_YgM$|<`4ZpN z@V9SUw6vJV#_IREu#*+#6m0Z5D`l4cLj@@Vb_i$fv+8NS{q)0qbqN#iFS;Qi(-n@6 z^gik>>JF+xno2d6B$f*lZzd!zrs0Xj@1T}dK6^HQ`HoA3UX5U1NLW+I5krZ#q({2O zET2pQA-fc)=Miwx2#{~NH-TgVC={eOh3ao1-T355H8I-6ELxY2;HE-AIaGovgLi_- z{ElNNn#6bSYNFf9gH^#qHp4;z!kiyzwKmu6wzakOaQg@elEiI%lM5?CLMYN308l&k zBSSoHY}~!d$HOBO=N;ZQiZV1bgiEfheYFi13p-<R&poYr1k8JyQBJERi6K6{@W3{g z=U4P1QJg_^M!(3)%tV=_koKMXK^_9Ro$gS&Jf)lyY_uD8Fiybi2fXQ|@08rc=gBDh zZxW`5+GEqp$iy_KZHSADvn(R)aP}R?0LJ0&B5gia1_&7^63d^AxZ`QHbal6Yr0Bl| zAhLLaC;cju+zK$5anRoY8w$+`=LcGkhJvDlmL}t+8xg2>b8p$Q1*XO__e<pY`%l;M zsOjqJ!g}<xR7@N6({BHKgB1p*rW2Kc=z9KV@@0n_`WqK>Behc@M2SJ!f*1uyD+fma zBndp%v6z^7l=a^<+_B0^25WA3<mw3hFswIOT~~{XH-HPfYGUu~e4;*aRYN~~M%%2d zMF2LeZEQk8XjD~I(XBSWLs4FzT3I00LHUt|uK=ydQKG|$h=kk@zy}!H`rMbc?Z)0s zdK`x7;<{(mev^96i#DsKUB6B|Dphcu;rZYq2~SB8e*fhBPKgo1x<sYNiD`2yI|2j| z6U@V|F3Yi|f93OUS;;m5hW{%i-Q#1>ka&DZ<g|q295F3%JtTJVhNEC)&er_yz4O&w z=?KrPk$+=5cLj*e%~n?BQV|l^?iRoOp4^YoT{YO6;y*=?hAgZA!nLgkp%G8McoCf? zitjr>wF~n}`iY!L8=(go9}am)$|EtxC?_l<V;Qj$ew@=TATa1tq(g#|qH=o%I>{sa zzPu1oQMjhJK+J}}CZN?^`<iji(acOSy}5)9l<Z5yA7riX4lk`9c<UWlXG7S1NjAJ3 zV0bj_ZJ*A6$G&cZspG#d=k6|BSyeUC<v+9(l@5*-QVG1g<O2c&Z$cZUXJR@qXJ`<e zJYG9a;|1LoReS^%0$ZR5=flJhfD?QYAMo$b#l@t;(ij0w6;D8mjW{NzoxS&gB58hj zczC$x$hc-zu<P++Z}Dp6d!V7_E0?a_5JGZcq)5-gim)(xdB-M7T3Xt%DU}8#G!!Nk zC%?Hx_G64$9s(|7#flXb#E}N^AsxFSu)#C)%5l=%q&qZCl(qo|*`QA%fRIXsP69uM z<gMGcZE#{f{!YYJ@+x%a;Qj+)UN`jR%NN)XPu{sBug9TBc3Zq@f;|=97qqD8Y5Uir zAS?L}6Z9nA>m)*c!q!oVCg4_oLwN)2X#K&fcv&G<Id7cPYmSD@9g7P~j*^dLc@%yV zEJ?amc@VVH)-eO}`(w27Wz73v(#5K`j>*n_=_pdBPJHIarR3KJ#PJUFOY-SStfh}` zew<bU@6_y(88`GBk-z(m<T&~!n0JukMSr&(vm#<N?5wPW>y1pCdlgm3sfB7Pv!I&c zVs+<}MKGN{+w4nwE-sFO^-ParF1ls3dK)Q|>n3{7w~(sbtjl?egILMA5{3UqQCcQN zBPA*6XMY6I_@{Ux<$$9gasWUU@k78If{e4`Rd=aB|FdV${yhM@i1FqT6wDFWiV2P* za>2WI?+PHgqSvE(EJMjkh&lx@y2I#2YSAzP^le70N@dQ@tM21p_YxBsu{HRbGn3Dv zg)j#ox_+168V;8N5H)0@;7o?|yV(nEDwdiTF#}&}WGskLIaDK<lSe?*jeUKy=lSoD z+=-R<MqWZgu-e4c-abOv{|8t=t-s&KUcn-A@6d-GQHSAxYy(w*$Nu*?$%WwE)zJ&3 zn7ILydw*#!HBJHdqz;Az36KUr1(;SuNy*|y@YnzsEiEmP1<##7|M1l-esqz}fnpL8 z>3Dk>mO_zjj~+d0U|=xp$v?MoG%T-~Cs9LT=oSDeX`1^L2^FzZrTcw1*lU7PP(T2n zcsF_tgrI1=L%%|WkOB^~UkwVl`^2yfp(_F>$4;wSnVLqTe2dD+aG-owfRZ5!A9&^7 z@6u-wHQBTVT7q3+LrCNUtvNj4O41BgG~GXZfHYQs3seY75qQ#Um-*9@$nXWAv`faj z{J~DtA{~&FLPOYEdN9%&nwn`ifZ&d{!77YR6#|h)O94X59vR)&S?a&LD`mL+FVTRn zzbkwvQ#bW|T&Gn3+WIT^cCQ@doDgdD?0>sSE0g8I%=BI^)KB-(zPjw)`Ag8R5b=wk zeCQBa-cx&QY?cy_IoM#b;EIaG#EB*^p12$;K$rmoP$6J-Tl1}~cFa%}I6pYn$(mAd z)kzymCbmbD6-dVs@qJg#5*8Mgs*@-QiCH2D7Ght2{|x9nut?Od8uy2JjO<`j6DKiE zd4F!B0oFkm@dqz_fpL&fZxld-q3%p_t7?w_SwIG;q(A!y1}5r92$`M8hwwz=4@>Tv zl~FWV!E0LR%!jN<g6J)_AivcOeTw`0CkxSQlHv!vEv{l`dg17?W2eubUs;ZhQGVkR zau_5W0~QGo@9M_GtkB$?`+*@SYHXmnH*r!yR#puiBMqf@LH;3-4RPI!4Zh-lr2<|H zgNqE?_4L9oUlyV&fa)OYIZrE_iBDzfX|vCp2gYqeXRgR5@(ZTWK(tvmGhRNMXj)`j zGw0^cx;Z~9JDVBjl5}P8opUW*xboHBvVeuqi@>o_a~&N55MUyqgLI$^7Ayc!<p0H# zWT|?&n0(roVF0&;(ZEq9{(Yz$ksv~>;h}*GdKp@$5p)2AZ}^8>qfrW4!sT1yo{2^b zO_&<okZ?)L`VLzMj5TjN2kSiBqlddHGPV0{7$nM%wdW}XFh$9SUJqgD#G4`6voBmr z58EcxTNIkmOH(uzXlo!Z(_v1e@BRBqga!rz3a5^M!~YCl7?cfsmJkT+Q&Us24$m&# zy8l~9m+WJ=KJsv(T1)1r?61n9RPR+B-J3=Ade~D_6V%8D%$~CTA0s(2(zLGub=isn zUrf)z{(*i+d795JuD@a{+_{1ma}}-o=DM0Oj^EaT3!jt8D_PR~Zm3$;3SOqMC=2+> zRwN{6f0A(-0p01#iA-X4t1rBLB`@pmsh^{GS?H@xe@q|~?7wG(ng`f>%CVSipb6A% zZAC{%M_WHFkg=3nJ6FT<Aj`ryt^uZ>aMr!xf5>^8nH-eSiYI0+!lKMIRSy`zYoCPp zL9nIl+O?ICMPTFzL-?ja-v47}LE{54iYz&mdiCmYV64g~PuM}1z<0*>ANdF+Wb{xl zvJ9Bp`xZWs>;A|_FkRjSClB;-3>1ApKmai*kl_zPCc|(1pBEv5AfcD7k+h+GRziR> zyl%4k21SY~x0Wd_(FYr56%`aTVPDBkn^Tcib~0ZeAL;+pkNt7WFz;coKn8`tKqPk9 z%s@aGT~~GQ3Hg@>bcY3saPjxOznt*<{%LcXL8Ksg@yKNGQJr_3c!FkrI$(e!62k@f zqa*Tpp&A%^2!K0V-RFuHI~I(EN5cB`t^L5Yt#Raw#<XD3V<Aa`+!R$(T8u{s2W&T= zhOns_PDG|T8WFt;=8=lB-;$ReSO{Vfg@sRn7b${EnQ7fovIs3}EA1(`&{7_Zccb^x z`1iNbgY5KA0l$G#jXsqYXzR}&s_WCPtV2rj_|&@cpIs?k8_ZaMcC_uT%bq_RLcJ1A zClp*%hWg(KGH)qNP!AqFXcy(g%molXvD;xJfsmR=TQ3QB_UVQ0m(o+G=0oKjf!bYJ zQ$uJ6{4v+pqBS)zAg%_aU!sas&n?CnyLvKBD+HJ2DFH{JzeNe0BA+*TOPgq%PuCHq ztNJMm6c*2Y=3@o)!_W0qYan12AY-8>y#;hN_Afn&Y%G7^B|AzwQLOSUOXsW<ivN3k zOAp5c8_EhO3k#6QiWST-W9J9vk*DlG4>vLl`b-qS&sJ$1;W#lCaI<LrWjbG>c5`8> z0-7XPSqj5d3jvA45T#(Yc`4cdmvUE@Z~xp|M`ILFyIKtiFTgbrl)7c{5Ei(yww94- z%V+{nCRhOW4tCz-8Y&%MLiOM@yB|KR311EAv5829(H&5xr}?iC3nz-}J^yiOSm|aC zSzwR^$_7lN@Vk_FPBpHD3C?IJLGskfue<6u(g!gWS{#{}&+T;dN^tOQdPJ4g@`t}< z&`(ezZYzYq=U`+29^pC{3k!>$zCN+#A%EKNYx~fK!<W(vw1Q6cm1zn2IQ!u{%U%h~ zjleqB#enDGI6pKLGCR6`3axD+y71p^_>{lyh2=vjcYYQ@8AHOZAKE3k%W3aiOv&!Y zT*e?P%oOO*$f!X~EFU*FZI;o>^2I_z3Mvu^3`3{uO)Nd&+zR~1T?tnL2$9$TtLLAd z`v;I2$ex6bDgRx=3FG?hQ+v=gVMd{UH=bvu<vbF-0(@&}&WN$V)=~+#0I}2%_2$Jo z7;Io9jy!qtOoOyl*-2=l03;N#;E>T2%qZ2l;Mg(&8Qv={PF#NjoFmkye`tu9VFwD{ zYA#{S84*DRhtlePEcgwwhvieFY;g1UJNzIMhA}Z|U}s<$BYu_odRA0c;%LVEG^J~@ zQDpc6X2}RxI+n@H^FsPNr|2RG0BY^%XzItm9V>4946NyMFu&!Xj5c~{SzZDi7mA0o z0}%pH7xcCaW>5x-I2~TpqpgdET?4KeDKpyNOr_|$b^KjjT@#ao`TckQcT}ehtWf!f zFV98mWw@hYaH&NOBi^*kt5=EljSLik@#|PTsi_rj{BBTtl!?MH1AN%xm?}7Z`Em@T z5`O=?`G|-KG{>jGoRTa8nItUPGTcf4eVf}kh!?4l>3{}8;ADj@JGw9`HWtZ284MP> zzY{otOV_M1%+o{Y11nQtViDHF#_Ee<nnRB)@qWY|3hD;-#KpitSx20Vw<3jNtI6kk z`{|Qp5^#<Uv;u%yev-iq!xQj)5%U%i0YUsF`aB`@r4gXF!IOtU80A4KJ&rZ)lz@PM zS+K3x6QW9k#<GRqv#I%rw`9SQZ}!+hTM!!Hp0<E>#Kdaek13^H+bgd;IC0`cYJR>| zu7#DAFpkDtLcZn>f5B1UY^B>lP^*Zf@93CqVUn?RUZEmfAdjHS$nLM_EzZT5S)-88 z$h#WCUgJ3#dq$$n%BO!Ug36Ndl4F+=PVFXi4w#>Lheh*zjkR8bqy)Bp&VM-&V+03x zchIkSfGiP3Me7b8JV<ok!j5W`KVmvW%L1P}3ucAx9C$UKn56B?fh;3qVs>rXr3(%R zoLuCI6D@t>rzsRRjF+jwJ%f{t)5nMwp9a|-O+di+7rW;^n_mVF$WqMmeEZP24V~ak zm{riPEkghD5XqE=TIbqHQBN)?un{Moh6X(uA%!K@qSUXjI%w$4gX8EVxb>g<y^Jz5 z_F#U%@OfY$5;7q?2Ew3zX(`b8*QWOq&Rga~ot-iIy=vGCXec=5DuHD<7c7VXfT?p) z!9>na?;9{EiY{A5#y8LroY5OLwns=1(_wkgbkj|{ZE$rPN;ft%RDq0ze9DQbJSZ+} zz}3(yE0@|H7r^_~7x{u{ph>M(oV8pz<yDS@BA$HVdoIuI?01~-5Bw$rq)cXd9@72H z=R%S8+d$@0Ugj=D0?D0|uZYr45%VnCFQp<q?P*1)7L6~)dFkn*Pz&p?IvduAA7nI$ z7lR&Z+mDV5MibtDVl*)^95|qEh=-e-bQ2Jl9-<#Yc^`O{FATd?G)_Dz_?T;6MFbuS zCL9=fE1}2{=P?oBh&LYt5HMGm6kB1r$(A7T(NZM$Kx8H!xJ^SP$M7?vn>JnKS?5Nf zxq%x7FNJ?FcWT_>SPX3mjsxNdyix`0?X?2LXE@%xVF(b2hC+Wspp)PD*BH?KSZ`u^ z%Kpc11J<Y^Wg#lV5@o+IG`J*p%v+>b4c4snZger!ekk9|%ZKohzIK&HFO;)~p_Bo; zO3=~G@TJBhD=;`Czo7!ggZ|mevNp6Z#P@B)8)Nob2w)217ESvE1i=U+Wst(j#ujZf z6g>0MfA$T8oZssA)O_dS(P1I67Wz3299+`+L+`g!u_>xp%aryTeI(O~B^NkQ(^FF# z7)->U#-&n33h^@wVBh>;WDWgOuDkOH8_$)<@ejwc{k%Al3h+|wGcdW`VhshJ(OLz^ z*`t_6$B9p+qPH&}`YSP`qKfs$ND|;W=k~)$j3#mh4Mlbj)4WhQ#4)zF1k@SdzbLEg zfsek5(j=&wZD4Z^F_|7P@%H$7pRbqxa5)9^O)!Y?%UEilkwyrT;HK1e?3nKbyzs|P z^o4Hy!8X|vn8G1{99K`9^J|O1)&b7$4_3q}hesw2+MAY+jym3r*fyvpSQ9Wo@E5AX z69l$^Z&fds2U#OqC;9(lQ={=^8tgCfJ{>>+r3P($`}XbDc6R^3a?Y3=v^%u4;;gI! zMi>MYoWsy<{8L-tTb{G;KD&Iq?l;%Evhr@|d6mG8L@kDgahvpYgcs@=p`xSyG&Z_a zLPCNeXqem*L@c1dzaLEvu}{QoWa@+ljIh9JAv46HMLxm>=s$2*&I3!|1Unh1`AQH3 z#10Jof)LYxU%!K?3F=w-?93)&Od<n_6d)40F5r)A(T#o{g^X+8GRFc%Amt!D$k{+H zIt}s?r@n9!a#<wn+>Hj;Nu-<-ibh+IhLcYEHD1>6Kiv&B1jqC3hm_!m*x1<Yr6hE+ z=_^fW76}NXqAsRK5LGJO0j`A|n3Tb68`#_Q2-4xEb0dQWd~RDqKoSJZ>^()&X!2it zDst<Vmi*l>N=VV%xpPMYT%6`S^0*p}B3OtA-g()-iO3CMZuEiR@6fX@q{u`HQQ{%d zB7U#4?4gfG1?5nk;zZ-;*l~9?(h(aI6BE0z%3|18|6O?ESXhh^eiG&I2N5$fjJpiL zxwQcIf5S>V1gN))=t?M$L`(!ILVbJ)jp8O&C+X85W{}8KC|cK4gB1nf^$PuP-Gf~% zdmHT9n6*N^y65-(n)$`CF)w1G#S6f6S`C4ajKJMbn)4Tb#eyI5^t3*3U{7vmZSDIk zH&VwkU<w6p*&p@&p9wXTB2FP8GOLaTrWe$m=S8@hAqB7z9?^TKD+aj58bdEKK89rF z_~Ot@K;k%uscyxCZ4|%=RRKX|jsFiqYLysU!3eP#WlSeGL5u}~Qv)wi6AYGA1&TYK zCK?zc9nv%yPizJKj%1}M)FoPqggcr};w=d~cC6}v#Y&_f!=Dld8Qsxlt`C}8f6XM1 z6|s%NUkNu4VFzLE)N#24PM#234Gp@|J3*J=a*IEL8krzUP^Cbz9YyyBKauY01terf zwJ_5Gd#yXBX{W}X>yxp0${OMbk^cZ%bzLrzIkFYr4_>3CyMYN2@SZW|rGFgx;&o4- z!qs$R$}-C+(WH&6OC|Fgrl#s@dX#+54f}Fff4kOxk6os}!k~>iRL(i$NC22F(*o<w zIWcw)4oqZzeb*{|J-x(1oJx4Tu9#RE7=&PFi}kes!=mY|cEyv68seN0;|mVsiNsf! zNW)-7mf|V+(TVZ~Vk-%<JGi)pXgz@JfBf{tPodTr040HHP`&L2QSJ!a#pN<&B~EN@ zTirv(Mmjq9R~5m*<>@)9`#tf0Uyg}Q8AtRKVD7`$uNM;Fc<dM*fD!{mIyOL|e@5pp zx%9_MUrc)r$EeE}1GA9xh$|PQyK_I3+Tf3wgUD}=-DRayDln!|Js&<C#{sZu$<%&~ zkyJ9#i6exfinFiY6~su9xoUD-3ABc$=H}4+e3`Q{Rx^UTNp3S<gFa!~Nr6z%pXBX@ zHhNJMV23SUO(u}H6OM5<?f7If2##RK76UhYIPXIEVf=iOU=p5jKo3a)0!1Y{<q<gd z2?b4&8%sb)E`w|EPsfiA5T_EuM`WrK=^KXUt(aLv%d$d}p6t=lid_}@h)7sXgcO`x zVc2+k$gQ8qfQBEzCUjwD3!i}KOW=w663cOVf<HR^CfZ(*G>HowpjU%*uS8=eP2d7v z^>6-Arn?8g35Y=4?Ymax@4CLgwX0<1AG7sAM%hi9_PD$sib4!Y(~~+$NToL1;Bm06 zU~>>rToDTJloaY+o-5I}!`{%&pEoZYu5AK#N=lTrUyx_vm$p#Dg%Fs%k^c9k3s5re zdG@ms-wddH;HWv&3*QEK&Ut^yFu4$lsPdRTIK72@?&fmXBu+*~Rsh8Jjbj!>1Hg=2 zo-&1Q=|2fHHB|&an<B19P5}XCVPRo04BcSH^v~KOSaI6XlhRPc;0!oLthK0g$4J?j ztDl&&nxm`!T6yuAXntFHH<R6IQG%-jEznjdz@&pecTNCPWD6;G^hdT=j}eJ!&edUY z)0P2#JOU@l%#K^>_XQL*lEp}0f2kowx&fH;Q2F^+S>4(fmCcDT!U)6fz03~ypS<e- zxax@OE-Sk6MWl;>j|XZ2_9fFK7(+sH2%XDG0cMtM!hvB35Ibnuzwr#G!auVGuvX)4 z0Mhs%`nVP5%<741m^gsia}+1|KNK6eGzipBTtdRKzkb0z6`&<6;gYjKPDbt70%AE& zMOu2tAHxb_=K5}#zvBS96iDTj&MU|o`ptKxCt76Ye))}bj{EnAlkpxRMHuI_92%d| z)z^RZAA(r-tW+p8c*+{)mJ5sLyf>C{TY1z}Rm-_&=OoSj#~c6eZ-WY39)i}s3L!^a z>{JMfKW4VDna)5Cm_Q_>YNY*(jpcyrnufxPJ^Ej3E_%-6NZ-(l;6fBl8i%@oo&Fnp z&u9TZWyqv8OxV{FUtQb92Fhx#J0TpPBHg*mm-7_AjcedYKy>(vHpN4TjTx1#Zybag zx?g5|6Rt?jaKk4cBMs3Fk$aM00K`1Y%G+~_f(9C@^`JYD1c)pKa+{cO;P;@xlobly zy=>W55|U)#1{c)8K1QZgNqNSA0I0x~2_&UP%EMALSH$2PyyN;t5N)Ccu6e;E=`mzx z95BK3N`u?Dh2^Bc8d|{bV{nt9+LKWY_>RtKE0MA#ii1f{+`&*VPb?likB8v|Ee5rt zc>CYZSaL)TtO72XO6+#9x2L6`L~XSVIP+a?*|bqjs?kDNZqamX2x7{~b^5evqHp}J z;*h44JWT-9L~>sax$`DFTPyDd$PUph4`G{u0?UkkywDT>vcTw4bt7q|u>3XkU~Lq* zmpK)^>y{V2IU$sJ>&nNW9`G?pC<;nCKQX>S5(%8Z?tpW)hkw(U>FP)J!}khP{h4ll zKE9>Myzl^EVt7T><1Qt5({P0Yp#IVDa80x0Tg)T7H%lJCBcBq}CTb3VR{h2^Yat(F zHd}u3#~0F?@m@T<iVd#MP&lZKcJ6#p4~T@Z=-uxeu>a96W_C_aIXRpxK)npX!-nVI z`NY)=mLI#2mM0gikJ*{{3l~lR5PF`Tq6rBNrMV#Qbd(HK=R%1>RfAN6C!5<vT@e06 zM#fN2dW8Q{Y&!9q;^Kf<GzlOemmx)9_f`!y&HXs3*I+18p<a-yMn1jDC$*w*5;oUJ zR!#hxDs)SJIr}GM5}&jh_$^O4-JGX*>NV&ClexF-kNExZ<0V##Z9b3+knbNbTDKhX zh~!;Ij8XJJK@qK-J6?V$+wf8%cQ9hcKJ?V7f=!JiY{q0N*WN^-fJhfxCrm2aLY}iR zP%~{1t$)G#UFJ+QCjgplw;ec$Rres*WB8?@2DlaYbO-ViJ_BrT1Dpxzp1t|yGe?1A zaC1@&VmI;ddgJ*BP9>kOi{WBO%g8u}MA-|M*XSUMHx4chdT2Hn$uOEgWKIlS4FC91 zi?+LAqr6`gtWnkw7kz(yDI(1sum{BXy)NaAx5(V|OiLa`ZaWKCr@=p1Kr%rAV+O7; z(!1HKo8kcRDc?)00uq~e?CjVeu3Yl{ST^$%xD%ornwPEv<C<=as3f>gf>(hzaSf5> z`ulBQbs=wg8oQ6}#(1`sogLh`4=YkXbKJOk>l-BRG?Z-u*8#D~%@%zhKG;D<gtOE4 z{laby{XtBVy#$R3rxoT4kmG2HZx$%%CNA|F_~<nA>OIW%WG3Zj>CKxX2gY-*U;p)F z=M61gZH=q}G~#C|Ele5s*)O76$Za}@zjm6T8;OUhGz*ha*@frjo#H{&tj4Vkrk>v3 zXMw&Y`uqFqzGGbA47u7S54SyBvn;)zX+Y?ROi)eQc!b0bTtBhrRzX1=9M6whN)04^ zK=Fx|VwA*)T!(ZYakK|q|NSH1uC!)s<EVoUzjocaBpjzz*o?<tk^}O~5NdUpMO~|) zkO<8*>*!HhCrE7ZxT;~Fi_7Z3nV%nOCWTc3?ipw4@YR7hcLq500w1IuFXjYWF*jBC z3NsiAKe|ILFGB`uz3tFw+mG|aO*!Fe2fTX=tjL)_q}rJCF@g(T90BqX^fLJt{)(QS zRmLG8FNZT_lpvueh8dwcfP6t&>yR^;dW}t(yG9LBxK}D7{!{W)iL&o-D%Rl)%=YsZ zuQ#}6lyzbs>h%lAVdfY-l2Z*HXj(dVvkuNKZU|Vem#kz~-`JQm*im?3dCU@Jb4(h= z!$>Uly1l*rl{#O_*4Z;aOdQx#!muNxKNP#5TfI?dpv8&Dfo@N35^`)#uaR3Pdl^&! zaaUp}5dNkhHH@cvfTIUzM@xZdC+-OFZ9UL834w~QtU)m;jzy1cgFdsZEN~Opk?5G4 z9=%)1DQ~;yu?%yFSI)Ct@g!*yXLsUVw2Za0vw3b8?ovSsBiDz3cp|n8TumS%B0?dm zcW%lWfEj~$CD66quqr<VQw0Z@PBrX{2Y&R@dJkIRpoyiJ1|;+=rABBK7?&X6z*9i( zQy@3HAuBf_M_p14CZ+)5oTQRBn>Tw6G|>>I3`E;V^ETnlC_Qk69Y;EV13VO_ab9dy zCBiudei2*X2++rjARgI!MjACKcqt#@^BbaZC(JB*3Ai<Da1<;hJ+P5NgfYTgIx(kV zS>A%96~9{=_=~`AS+4;Dz+xIStd)^m%Lv>DGgSA5<X^P&=%jFk$(*wph8gfpkBp+u z*T+fRZh*>w8EBoE@o$!6?=cg49iw&PbvX0RxU!0O@nWgV%Kj!dm_!1x=FTZjT_weR zLqoGqQ~;}2D1a9YQYCSStf8Z;yYs;KK1O$pXvG`l1bp2tR=NSLsyNkx)kt}5k1`hz z!^D|$=T;=`z30A}R-~r7S}T5R!hb)I%xot@d-TVTvrq=4`4%jw-+%a1>uGg$b#ine zW$1t$Xu;pDwbu<ZBF3Js>wY|qzJ0hbKS6%bOb%u@ZR~D>#ooQTxgEFmM;n72_&JS3 zCZ1*VGo=@U7DlupjZ0L{XUQolo`nyif47B2C)-mB)WBY33)3jO>0jT-m5jz%{v4Ef z*V`C@sDmaqbED|Ms~yg7?iZS?qUTM3pMEOk)z<zLT<=JP8jMz1zpIWi1rH-}Jl(Sz zZX@uAHQ*q~3=zTFC>><Fm~0Xia{T!5b_X=ItqbVLt!aplM?kM&Ji?JiAe3YWUICtm zBqXgkgQG<_D1bh4r&d-_pz(~BOhHN@W-=BF36arJqLVyCyrY2|^~plUz<qQ{9Vi3p zFb27B>;y1DEYVQUl9i(|iGXWM%v?%9<-Efo$UnT-I0&})6zf8Gx`14WW{C+#V*JBW z17_3l@FY0=-~y7u+rLGxMkXHn!^Pn^fnt`>fe&C-wup9WmiE-_JB&gh4ytQv(m;sf zLKA+d6=ricYJyP7zFdzqwhE&t#FU6jV`wQD5Y@gM0vR>xlOm}IcM4#!BHX(2%It}9 z(4~kY<%)jFDNKV|<Dw%lHSgI16%oed4hJA)Lh1k^+Ci6j+|aP|!0whj6$%>Xaa=|U zhT@xg6wR&xNVP`+pAy5D%yy-JsyopXoTU7-|1zdLa5)5~Y`1<n3~v)gc}l&Nj8&>1 zJ*omDfSU~Ykz67e=LuRP`9NW*0SKl^ZEMWFy5GT}3Z@l=;{=*(c%-ypL&&IZ+^}(D zIvBpu26Ut|bbTLFoB$)7QwDK^2HM&v;uC}HLvjZ$a<Lk`pECaD>IvMRBoa)+Gwgds z9L;XfEaD~m#Vo>R)T)HIzVY<rdQ^Frc3%A7N=#T<`Q`!*^}rE4cX6lR*g#7$CHV>r zK+l&iF-S|-0vnTIHVT1#1EwXIM8uLXs0^>rLg(8uGS@*9z$q09%-$~Z1xEgKv^x|9 zY97|gvQYu!yYm;XCj+d+GY7vLY!{U%MMUK!^J|c=4G&ng%yyGJ1Abvm@Js-BF-CCG zG71Sl`dlA9mkSklNwarxnWw6%I)yu?FwwuV3>T<z|2j0MhM#F48bCZ(;eIp#h`_h^ zvfpvv;^=w4^U0X~1Kbou+;tcO<|1=i;MfZ=Iftua$PAC?1Jr$MoIujK<0i%^w7i5y zspx%u2?PT%&XIviG#CBlXRM4#J|FJcOa^U1_^f)Wo9XF{-WaZ*Y9gONjwDxVfxaWZ zK_*6IWMp7i6Gn8YTUac{%>YDXB2pVD-N1g2(vp%MOd5hYut6%DfVILM=C$pYH}XMg z#0?O4Ju&PH4VKI>E?u{-=#KP79W(gdpklk@A~2i_8}#_3D3Q>@U@jqqNy^`C6&Q{I z$R1J^;gm5K#sx}ba1#VHnWP5+Ks<F&sYX8l!-RuWD08={t*u32Wgrz5_?pnQ(2;i@ zkOR9qHQFFUZtx+4V(96wztIV5#~{ZE7@AKmQ5GayhZ{$ao|khF#dWUR?mDySsqO<8 zBnHku2Xo<#fz21+B73z!M92h&)`e!g7I(bioGrsr=+)yEKCzD@qH%~e9dd^vAbA#+ zQWpkXlfmUHR%l(r5D@Mtkw9M4HwU56zU4)uU-nFQM}<X+ui>qmH&0_!R>7m#_!-LG zHW&!v(5*@=6cEq?J#m(xuAfVP^1oqx?<BsOePVnZ*EeY+A)Q5oBL<L5pci0xTuDzK z9y1=ZU;CV#>Rhbr*C6JO3@*S_S6zF%IeNf&BEH^bc7!Y&Us_ta57Spi{QY$yD#lTd zJLUmDk-BDtuy}Dd^z}WD3}ZC9bvVJJ!^3e{?5w-8y+=AKv|4&o96%P63jv`eFLOcw zI7h3!e|C9Q@%{Tc*o$}&6+m(NAF*DIpMPoI4LvgHFRdL<zE;=RIJ9f#o6&}hII*W) z880xnf;>BWsDloVi~EK8BcY?i*3blJ`>pv<86Zb(V95lxj3$|D&77rZnklAx;KR3I z(zNtP*E&!N0msiviUci3TU39x_P1XxeKh<)DWSstj<@`=Q~kMo=zkcE|3C2UDdC*r Xlh?PuG~J4;z$tCb9U3>(t&jd+Y>naM literal 0 HcmV?d00001 diff --git a/docs/source/_static/images/tpch_queries_compare.png b/docs/source/_static/images/tpch_queries_compare.png new file mode 100644 index 0000000000000000000000000000000000000000..927680612fda7033541b09f9b158d2cc2e98e292 GIT binary patch literal 27388 zcmeIb2UJz();78mPxPo!PZAALK@tT;5KvH%CPrh~pnxb<iiILYP!Nz#VhG2EaU;E1 z=xjtmP!K^yMY_^aiU=E&svu3u{-3!p9&>Z<_nmS7Z`^VJF>b~<iAOhkt@W<=o%5N` z^UUc<Esbq+W-Xe<VzK71xBsNgVtvuTVtuav_v!eZ<LM$I_>aP|E&9iH+gluS+3#S^ z+PVMO5nKCXwpItecQ$u$w6eF8l~Ryez3Tfz$BrFwRFsxJ{P6=)_70ZPYyY-A6JIj( z$aVuq7Hi&q`rl^>s_|AVmbDf8ryutmkLYQ14GA=FQJoyVmOAa1yFS0X_~k{eL}$T* z&s+l520JX1d^ks~vP7%WsABDcj2%C!SKerF+4cAfc_Rs<&AaLwd{_p*42<PR4;=pd z*mq?m3*QW8Uv(^WjdG6~$X)oRBqyuGt7w+*rXO$@EY@8IE3HoESFCF?TDPXrFE@TA z*6<nq;^N=u_tJk^T7R46!u&$)b6+Lq7xRC<FU0)f)W(*%%rCZFJTilR;j{Gr|C;}I z*XD)yg}^|at~GA&UgeDS-q!J7XIr<rG25y5X6|50&Uk<EX=C@^hODf273%4OHGAWC zhRa#ia1Gpsx<gyfscx{(9c{0*P<r{#Wfe&}$*Cn?<8KSRdz_YdjkJ8ZXsy^{+iO2A zlCurzZM-e*)O~r&X@{p<1Y)<HTSDI<_w3M>@z<K(hA->hwia04_4&-II!WhQhD~sv zXP^7%t1IV%);mADVWrS>qcW}Kx5aTf$-4A&H74#og05}nUHc1HB%cvdaA-S_#}`s^ zE-6#$i<q(G_zRnk*({dRrCo{GksCb+{8Yy;6%8!5&slR$x&QHj0{7wG$Z}P_GTY(J zy&0D0RE8ek_nj^4H9lhGIauxb;^!ItC9>sa0XyZ`91lFs!qxjWK03QN_|DKX2|q!H z?S>iFf$~kqm)&e|Q1=?X-E2NNKI$Y>5N@BVaI?wHM((+7`mxs)Jc)v+qmLG5z4&Fe z!@H;3EQ=*?mKeJ<25z3KAYyKAuJG#O3Go6?+(7+)KcR}ao#EsC!FeTIgLH?t4+T0h zYE6nA%T<Dx?+n}H@7bHxk=7CA-rmfgbeXI8>f-ntKh+(TaXV%A-~YA2wIgz&f`i$k zGYbQRoa-*B1v!=}tWa@#xBZgV<Eq5nvJP#JyRR24$-)<;)tb037?z*OVg;S2T{c<Z zk~h{UULNh*9*UJ&c3Mc2<I?OY&rYki2-i`_Yjl2Y9_=|)SgmIo**rOPlbx#{Ia#4D z%^P}dUw-%JX~{+%0(m1_%GS4@Sz2!P+H17fwEdY)>dw|jXVY;hUY|FA?CS#0kruzK zHjcW?@i#AwTzP?P4*h=p(E=gIqLuM&KGWuI&;M<XM5==OQ1_X|@>1A*>KxmQYTf$7 zHydSe`+|-oYuKFf4P&dudTyF7T4#MB*wjnmj@w{Te;A(1&Mu1=wAQ}K<GS+gzFhar zx&x<+I!;WVqZY3b*&nYu!A6YPSXi%(C~SRC>UK9rBy{!uO`NuR%ji<i;l|(}PfVB8 z&0N98f;wXDYYlRxjI(2owmvdd>b-iqx9FEIW;-+-EznsaAt6zWl?<P^VdVRR1^y|@ zlfil8qV7Ezdgk>}BA#PIJIkX~20B#vS^ahEYK;O416FjT6-({q+VcC~&k*)FlG)%; zoLX9&dH6YQSNZHB1NFbLHohKBnooD8)Rz+=V%41M$!Y7o-KW9v=yPwDC>m6q9BI95 z)X|lL187yBX6`oJYs?$b-FsNa)eFl}?kDUOVpSQZW8gX2e6TN%KUvf<+~BAgCT|}# zG2HAOI#3d_W|^?}_)$b{?6M$<tGAY>Ta+5U)D*_yFy4)*z45D(P}vpqnQx_xLTXLD zRW|?jC7UC7<nCwe+*_fCIJUQ1ek-0ZOt&o8@S7_XG^SX+#<jogSZCEr>|`B(m)-^i zwv1IpOwm!k=eT!)*ZbG;q4M@7VRDDV=-axmiU%aR`4fHKmQ6XXf#u=2xgg2Cx(h3B z+Bb`rhvRcysd*c2_hjy!WBGR$>){n-1D~be$=T}vHfOQ!XiiR!vVh6){XFsBP+806 zY{Q#&cN$WwaabB~NYs_}x?heCl&fmu4F-p_6c6wxno}1)zhPBb5}_cQVwhoQ+c2Cv ze5b*w);O@7@7_EjJx6$hvWolA4$h89#dY4}t-=GiIn|WCH?KZD<;r4-EvHO1|K6Gc z!GLLV*MC1}(c10LGY_XvzVFgo8C_gl9N=;N)a}F1GfiDnD3Zuplq5KCFJHdgUpr^9 zoRMv;%V8^eQk@RdFeMi|-Y9?4-sw@MU85o`y={X+M??Tc&%z3R%f954%%rr#rT1nC zN9>euTw5UQ?J<N?v=$5c%yAma=fKoE;-;4=ni~A#?dtdhXG@Yl?9Cqztj$JDc5FMp zO;<rzY<`fc*_>l|8Rt|dkK=U*9yZTy*c7K8EEU)~IX2=QuI%c7=&Cn5?;kikMG~nw zoG)g7yQf~);>uY$!*&lm<zot=&ExIzu3afN)9|>`1M85=^%q@GM0WQ0ru)ut6Pu$t z`SzTtQl>Y55>a?tlCqjPf@NWZf}{BGMSewk`G&o&8`#*Q8OL5d`9V78&HV$l#}UKC z3q%VMx&vv&TpEsU+a)uJ!@}l>UJb+R&W@p%uJ|NgGy2Q#R)3#|?c&%G=_2Lqh4?!9 z_MCS_+v)0;IAtmQVu3if@@E&zm+9yk>$q~Y#M;7at`=G+rvxEnWcH6BHaYg)8E(f3 zNL7}zu1d%mZ4VEWKev-}@SKWVq@q*N?>}trZx6SZ8VV~;3-iAB%NMSQQk606fOox( z?gsGyBU{YX7qVCh%Mj`hz8~xg#&@Z@_uX-EZgi@>VOuX_e`ipaBF~N}Wd#9KkE4-h zu7lNjj>C;EE7(mqnsn|cI&sWe&&pplI-mExGr6NJK(j2>YcP40{<ZJ-+GUyd<_wf6 zzkTpc*6X<NifjAxi)<#}#jBR$DS{$hniN(<9@Dlc4GT1#=+vt}l0PTe_I%}4<J*Wv zLiM<p^tX=|cU(L%w<u_@T_{pi2kx<)epcjlP0I5FS;EI({G5ZA$mSqI@bI{0d+qN8 zR%$4vI7EA3moK1QUUh!1kYd~G_-I<pKzv}GWyH~~?d8$Y>uU6ujdiDMYAC*rQ7rC% zd0M(1iIMBjUEG1FbKJZoN<_fu$d9@P=7HlwH}gs#pI?SO+SPW>d;H*Bg|^?tg_|>M zYR@c`UB20GZmCs#WDwGsS-Hw^iD<BqHg9@bp@76Be{wjlqc(5S2|+Kn!L2LB;Krd{ zw0CUtCWb~E`@C*GxYO%BIS{QknQf46@wz%$?`rP*&hZyBRP}qCzhSZTP0W3@8$FTK z@Q?1&q5mg9^2aOMGkmhs4n4iRBW&$TgaHi$1B1j?`bMp#K)ZC4m9gr<YY*+)DTC<L zvG`y`4Er?>Q{qA9Lz+4K$uR@K2X&znf&aW8X@90BW$%j7m)<cv9#2$*knzSuiblq_ zom09`U(}!9>v7N{>+B9!*Q_bL_k8Z|V{&4j)3^OQ5ZuiG<-2P3E%$v5uI>9V@u1eW zMe=qLh>t;v)0;O=L2O+^-0beS9fj{+SE>v7D%K#??Af#D)*5jqwGUs=Rh)JQ$;;~L zW#9h6#TT{2%*rAZLgfw{a$+1i%0x9D$e(0x$S1G@s7RuCXsG9Tu;N^&1lNIfW*^1k zr!#{umEE!ur<;|Cl^(XWe|mY#$pEccrd~}wjhe>VOet_pJN;~ld_cKU><4h4REIs_ z0MHh!IGuZBe(!|_T*0wD)H03$A>w6?p-sy8OMyH<mjuDxV)Oe4qWQ{^N-i1$?WL3| zauDo9cg(?6wQg^C|LTgMd-rwh^u3g5PMtdSuCG~DQ)O^Z;+4<TTjCNgH24abge0ft zD#|w;*<54lJ%LCxTLCx+c_KXG_{)=Q&S|M^F@OnIJ7l<bubwWmjB>SjKin5h8;+pn zNlSW0?xE~xXVb{srrQGDStY`<M_$}5iB?r*zShXGlyb<Gt!Il?M0;q@o|(7WdvctD zcg@v#5=s&cE3@LGy%Yhh?T3)9j`cOE5R|G+HO*5-q6VaD^r|-x=2@za84NYLG~3>I zx-|>uS{C0x*e-U?JbK>QahlOlE254sr99}|<l4c#{%BD>BAx@1ttCJjyZ_-5o}Q_v zlXJ7@Fu@C(t63IyI78yKqlKDYoG2CdxGkr@HE=9lM@z1ob@G*gWn|z^xq1-_wUt-4 zor{&S&FRY>BQV-<ta4|Dyp{e;G2A~QiX1&35N@al0GwVw!k7gDohD~T$ZFbeY`_)? zRs5V>Jx;K!b9Y^=i3?t6sDh(KAQg6ZPhBMN4tP%yuLJ-fv9(NjpyW%z^{Z2J`qk#D zyn7;e?8%P{RlPiTqj;s3ZI927B0GdqGRi_ebbK8ftef@dxqV(_-Z)Qsw9AwqMsMzV z?JdN1M*!)LxNy6oOMXiXZz$X^+TNm380q}Xig0#CvR>+1yN31QcG*@M8%EoT_&;lg z%80Tn5_iXo3)kY`MULtYlcOC}+ytIOfURkwTAg(&Vb_(d1g>d&+&KJP?)l-H#$N9$ zquYUqQ)GGpT^|o)_1<;Y%jS(dlump2?cTOi5?2$H`RUOUtDf84KAi2+7loXuL@ASj zhDGZrv{Erbw?ie6c*k$^gri#hm81x`QR;sdzM3mS=Odaw-a)S-3y(NJ@%OjIc8?zl zm_{tNyR|>-syn@V0RO;qKze!GPg31&-HhbNY#!uDDqXgoTM}6+_k7i|ZT?)rN@Q1Y zVdmcw4ai_Mwl^D{6B81)C+nv4(>;H)LWA4C#)f-+tKzX&*Xp{Ok;xXym~CHzOy7O0 z)&F6nAUB3?ZDo9fc2<#R{v7!d!d71{lo9EO_V(h$T)w=NE#uT#B`z$Fe`94U0K_f? z2kQY>;V5knTW1r5!a|acS`?t7;L!(PEd<ib*nn+wUE$r6&FKY8+}oPm`%2ZN?_7{A zTfc0eqdZmSs(Epc`uGSzBq7CPj|Xc_qc>f;*5LTFJ!l0{dCRhhwTH9@&mBN5k&Zwv z!alb|MSIt-^LT|iQPJ3lQJzEf4<{E0;4b!=XEwQYnGbc>2QtNvcn6h0Nc~Fe{&$s{ zmS@-1JYHxSab7aBZfO@cT?03B$2yO~Qy}1(vP+|rp2qr?UtjKE={lTM6JibF0}(Dw zt`v-zKm)KoI+*GmdBtB*yimZU0lO~15fI2C(y4}S%z>gFfLXnL-h|rXCAg)F-+CZ2 zD!lNS=1{e3n>i4fT31cV!2Q|MZI@3=73Gfg?p5f`d~R3#?OxmSJvW;-$TuF_9p&De zWrVw7bDU}nR^Z*0NZxGV;D0hWou<8bU*<(m0o#>PMARerR$xOLIoF#%j9AV+MsIO} zp3!;Pa%I^9Ar~X64p8~D%}^cr(TftUzrtYEZdp`iRDgwU@N~<>ZcrY1ae^Y_%z$8% z;mY3Rnp-Z?b?HF?^l~3ofdjC`t|GQz08r8lz>2b@Kce+GfK7^^%TT>#2jEH}AWU#s zq*8~b_xs&7Kpi558^*c=e%Sn5+equV@*LLzrs4(OC_$LY*kMqE;#qm9?jZ9RKuSgq zg`y85#JPKlX36b)x%fbPNhm<hPK%O|>lNJ{R|y69#D)WWFW(XJ`w|tmRhfrUvzvDd zs2}#QH__D%QtG{xKbNKcRwtStczwlXuWM6T&tC9!rO&e$af$RipssY8RCagPV~wX; zeEt6kwyHC6<?gt)U%%0K49pseMUS(oy-H|K7C4pDu894XR=7O;!rEz7+h}jmKb;K* zhdW0GEmjTq5lK^Tce|$oseQatFRut8Ap|hK4Uw<t#k~^)@4HfKH)-v2{cy8`PiUpn z($Xl~EL?wNA2ByLJgS7r@ah9ZLf0QXK$+);b+tIqio%<_(`4!7+IVDrJ%Ml2ylK}+ z<c=UPyTPp^`=zp9T?VL#0S-{;%Na<@83Nj1>L;Ab?XIq_*kHjq{Y4UN&iQ5A+Pl-s z%c#`AEr4k<M_hLr8vA@ObOFN-YL$B+h`AzsJ0hvY;*6h;v`4A+t)$@|gUoZlj{MPb zZ=w!7I+U6VUp>7dQILzR#n3b_PRwfuOfI~cH_<bWgrJ7{8>Lu?^2EDJh0cU46~GL% z9oZZz;!4Bi%csp#{$1|yvo`$1VFL107mSv<<OWa<q`MdZ>$IHx>_%d1^mjgPQ=o}8 zSVC!M)1}`{?zl-hmEaKdBTAd!|8*v9Zcf{y#r7pwsAvSh4TS1BX23&P0N$0^PF-A( zLiI@G1Nc<i<XE33sEM@R?5=hk?GhA$Q7Jc@6g4-Dn$cn25iTc*B(#%b<ni`l7M{}) zHGW6PL4P&^G@V{7eAT4=bneC}a#gE*{w&{AAtJAi^*qZ+v(@APBT1t&EggB=DlvN) z#9O>mS54qSMW@aS0E=;8J9PYU2}w5IP^uYDGI0)pwPWK0<#c3M)#mIok7q!%j~f%N z0EKtioq2ZaXsdX6aZypwaa%kcn*%T&3&g0C89S<t7Z$8Ek6Y5Pb^)_pL!En`#iBf1 zxWVJNMS0Yny3yB_@$oKwO|JZj;Q=gy-V#0~cl@p;-cRyd8~e<9i%=PJHDY%_e%&rQ zSgoVClLH(jGFw23`_|@e(H}44$(xq^^dl|zi6R2yG|9mNOSBf8E9V89Ms4tXA7kH~ zOU07ot4n_ENR}l;q~*_5T7@Hjkl3L`d3JbYN`l=to4u&;8$j{VP9+cpeITqK3I(25 z5w?-e8+X`%;0eT%s$corg0LyD-gyc9j;wU<$N_?4<FB@<7ENDl`#YsI0!qY}VY?IG zfZ9kW4Hu`IVz3wLYN^_;puj*LmZ5yEQupu6&MC`)izxYVhR}uKCU<F+r)nGmM$@K$ z$<*UipHY}x6ZfS=PTD4?-n%S`pFxBW-U?lFaBER<vAR&a7-7_mM|a%sMP)?%je<sW zmScwnZPKS#wjt~4cCN3wp89?~^^D3_gD0kr-~Ti@f3b|>(@GF!#nO33onl(YrV5h; zOaJvR|B3S-%jK>@;jRI0E)DpIhhtL?S~$rw60sr<`?h%ey=l-({@6&bOWy60+V`fY zm`d9U1JdtJU%YBK<6^GWo+-&et5z;&f4-vUo!YtHheK5@Kp!t$bKobUuz={=L8p|k zS43XG$yEWT>9S~D(u8&2(`~|~xTNVQ5=?usXDOYC%a;#irA`|>{~;jFTX*=`+64<1 z7#z;1CLWXr3egJZ^{O%gBY{Cq8<nea4M*~c2B6!c2#ez&$|kHuuao#7qf1^j29~1i zNszT>rzJDDBc1OC9w7o4iE8hv&0|A7_>EnohbrSrzrOh^W!R#kA{j98B87nTL7V5P z5Y*X@>WAuTvj_kDiXD^_3u575TD9)pSme9c4OxyFN<BBmahej9^HAFJ9(<!&k{BNq zMn@2gl!KI3^pqmA?~CijQn!hXC^O~1uN7e<5IKzUf;)uyBijh!QhKG7A0R@=6-;Rw zV4G!sTVW`OBQKC-E7;$z(*NBQB^p%$4q#{6kvB@e+9tT`Mg-xxNyO^*pJyz2jNs!4 zHpCK#*!X4LJk{~lDAl$R|4E4<3si_BqAw2x5nwbj*!JQe$pK`FcEUQuj%{1w8gnT0 z(s#Bisdi>bQ0V31T<zpU04D_&+y<`3xt3QQZ<*(atHiPcmQ()60)G4aJ}=>lf719M zC0Oi3Ws1o}qv)m>>O?(vetx*};ywP}^f4@zS)h*Y=(95folri60q|Xxd0t`URINjy z{VML3HmzQ&N$6TDZLj`-=r*jhE3eBQWg4MYju{YLVW6H#C~&4Z!1UNNWi7p_;C$`# zmGfueEU2S2>mqP$;yp1)Bmk4rz?u^6DrtB_zjGbw3wX!Kll^MILFq+{uWLlCUxQ#_ z!n;MB%pLdPs~(fQIhuK8L_z;NZ3cz59eG|8X#i#BRG2U-3WC~(Ku0xBPv&zeaE(Qv zoqxA$%oZKJe1vb0QrhID+<GlF=1}SvI;r;LTAy_v{zb(`VW!rXh@qOmru_(p7O8iJ zis(W}3DQU`#Q~ZdLXkR1>rj_$uq=;?oSYboM~G*bt4o{xcHnj=#)iwsM+cJ6;3sTI zr4cX4H9Be{HedXlr2e&wMIqQND&XZzdh_^`BskO~LdudXfNZxe$_2VXva+;<MDn=b z=LynNlm|=VpC|`b;eHIK&lL;-EpJZA6?#G%s+t(w3B8%Xw>!hu7J^Pl7IdE&tQ{y^ z5#9F7Y^ld!g8B6Jif?G=0PwK2&11*Bn^}W-1wv4$Mq7pXAvhDOpn~)T9lz6b$K4V& zh|a9>_$W7V&_HFK$*CqiMq0vCC+Ol5(CQ{}J3=|MqbdB8<Y9q=yOelM$}u*ncbw^v z5Y3K5me&vQ>4#b)$DM&TRQl5^A^7mNKo|v!0z+Ql_D7FWi1K=S+H{YS9vyc)8gu_R zM+9ibyDe3dW!h=$Q6$Ql7<xWhpgMUTsPSe|&02mBkbd!4UtaV#8tZ>Z&6{+iPkp&q zZWW$6CCyuz`}gsaU*H(|xV=JL+|f}MS&0orG%HcXQhDS3l&h@juI-m=I^H^t`z|By zWR6_j_f%$-sq)9gYwK`u2J(d`S2qJORp6By9ecXv^^;4RxB3f)B4S9=Zk4+A;urI# zPA0AsiDC<VP~k)IqpQ#if;9p)U;UvgXPK<HKmRvSsC$Esry!t7qKfq<6l9z8_R(m| zJns-jnt7Rj>LT||vug5Z64(El^}6#R80_)u6B>yls{I4nk0Es8zgcK3shnR__suTV ze+S)nJ2c*Q(KIlqKXC5ei5Xz)3cRP!TM_}nD;{J8VdXfDNW+&^6o6PdE5U>izzbNR zdDVk9?-Z(J%a}d>on(kC961@hD`F7wD!f-NV<Ok8xLt|UHbOQ@QqF@Gn1&+K9Mx6{ zXi*0WfyBx(q7)@Bh&z#pqst*7?p(Qa>gTt%Xxxzjr8Eu-yacPg8^sHq0xGSZ-?Zn2 zATq}8y=k{l$(aju5_ECQYFNjn0aTSiF&ikrZ__2M&c9#qVR*QB69Sd8p6}mL+0Evl z6u~Q5ZUq1rc$9eY3oCc$_8Vdg79&H7N0&z^q}Lq<ui)dx#YNd11;_Rnq)t(d&Bo!f zmKq%Xa0jdNm-~@!JnXn)M2;wwkjT@tWg<{;Z`n@!j{8p%fl;woK+a`z0PbVBT<(L! zgu~z~9otI^K@m_vv(~omTGjcH>AP<2w$A|;a)oj_fNY!5ufJu>79vi781)Qz8&IDB z348DcHB=|weJP}Pqkj~^v=Vgl<qmRNbf!v=osd>{$yIokd3Y>i;;Z@LxW|LYps(w$ z9~^2J;P<S!4eTF}hg!JKdJlyb5m%%}nWHZspq%NDNUg;^tv&qghOR8Vhrzvo!HO=u z5&l9-7Fc@PTviRYt_Sv9_!Zy0A2Equ;9S-5H*2lm@h9~&ttyuz!4~0-5oM1|+JTyH zd-Mj+hSda;6EcB4#P|Jd7=I+*<T##F7FQN@s@ior@;0jH%8NgqAPk5%q0XP2pcEJQ zy(oT;b+=+=L!tmF5}-dGIF^R3D+VBz^%!}hs6lJZ((;`1p=@O7PXGUe^<q&VsYk54 zwXeONc~&nP?)}2O#^Gz0zM{nc`Bsr92~{{AV&~JJ?KxO-=2<!Z&o#~znE(RuKk>KF z>WCDQjvGDRwt@;<8RP~o>;|->s9+IcL(2!};n0&&Yw`ZwYpVYp2wnkutUT&~rQNrs zV&cYF(z@Qp*+X2O$lapHvs~1Ap~Mv<Xa|!tM5s&^z=L=bDoVr)D12wZ-~mRsmc;O# zz+jbm-u#kCrJKJN5|_xD6ej+9wsg*Fg1P_vYMJ=CL6qSAwK%rH#?A&_L+R1xNPjyy z4aZy0zJ}d^=v^XmouVL~u~;v>4M4;;f{Ls^a9Yyg?q@TIHmC=s>4<01jf#d6Px&<0 zb4+)M4e%TBa%bS>NUV)Ss<kMKFyJwFqP3Uks=Ud;)S`(xVYgS8i2GN7a^?sIQiM(4 zAwt&=JlBPP^Rh%0_~QfkzVM=LwO5U8?7ybFxho%aKa#T-0uz}u7{djDzJvY>%eM)8 z9=n>=9>&Y=MI9KmP}XA4h9+<R#8OcTCoKI4{9qTrHc?gL1r(=GezRgZTU=b6BeFYQ zBSprdvnnyLBPa$SbfvU(YCruHD?t*k0WT}F7=-UGUL7&$q^cDP$ZMq}Bs>Nb5HClJ zm4&&%7B6;pyaDsHL6kJ^LZeJWQfY_Fz<e&M4`Ogg5esgf=yhQep+Z<#0l!P-K@3C` zg@F=T(y6Gfoe~aMZ}_`!?RH?aMt}rLzJS+`6`=cx2qd~yq9DSpGci=f-PvEEJ!D7p z1{Ksyo=ZI5+*@l?!~M7ue3b_86UxE9v`Op$5#Ug!(9ss*SeBT-hk4j)%X$9?`@D9~ zN2rx>qQaTw!_mIAd)YkYHybIQbFnQD>W>=UICQ?hzn@re;=Rc5puLUxe8TQoE<uGt zE)D0Qvb3Z2Py-8qT47wod!oy9;K}BBJjeu|&d_a@AzKj-zyK*>&v&~Lr(i<W$*r+- z@1EjO{SzB(5=KaQ&Z1S5ed}Oo+Q(_jpQl<vWG(3k#K#*KOkLTLu8fqHyv`1ENbwf} zMq0&I%<Jf!x3@@ZEy$A+lt2;KVBD%*Q-6THyhw>K8Sn=R1KVcQuj#0CV?jhNRB=0I zmCtO#sx`h!>zu(qSt2iLa0ug&afXT*V*5!YK3O^SNlAQi2$(jW$bScLJRRif0UPHC zE}I8FN8%`x0erG!kHkMBDvGcnk%_d;x<PACAq&7kQ4P|ISXwtJEJ|FZYMNGnQZF|A zyPoIr?0yuAaRBunKRUm+s*c1g!mF^rc+Fk^!Uw5)Et@3u2&h1xd~#K=z4zGj(?Uw) zZ}QcDXBP3|jrqN+0<cOs&kkxrjW^uE$go;@Vhsk%gH7D50z2qUfng2+Mp@1#DUKiv zBH2`WeX%(9$HZyZ4$$)OS`h(@mxtSO;A!iq)Ko2n{1-<4Hh}tKX<&TqZA_Y`O=I0n z1UJ1`!0(&=b-+;J5Ixqaj@6f|g?vd1aj_Ta{yJHSD0&|#5SZdOg1-NX{&eGOG5jg0 z@pY;r_vVpAue)Tiyq%H1%1|BF3UQo1#p*=dTXY+QBoPoKY7obW=$KA40>alQ+%Kk` z$C=MMB_Z})d)W!#LS-5;q^>Bb0z}{$A%ays;^Tc8*nzj&lg8lG=HqxATd31Sk!(G? zC=h{=Dm9%lRnQ8eYD^f_8t!h<I2(Dag01j6CLoPSyleNS*CTf%D<^^YE>i=_3fz`z zatC2CQc3>AjCmShNh-lU6BwdEdv@j2mJ3;cmriO2ElWLY9YJG;FcEnz@qG8RsN?LS zi9FRX*q92*;YGy>kX#{lXf)Y4AQD$O<Vn%vZoEzlNWKK*BdMYc;vXR;KlCq$eWTXc z#n=u7OgCIyY|dB9w#E>J3F(GP&-uK;saPE>X`XOhc;eNz(NcS_0MevtA%fnlKpdvf zK_Ql~ok$@V)U<u~exdi`V2fw0K!r)UF3XfODY$I7I2=(3=_J<>4j=;#Ayl0*KE$My z8Yd<@3M$CJQDwJWGb#cJUWtrA!9-sZeHymbii16>I~t-9sb~%~Z|d3zxNZ(DHn1G1 z!r%@g60)j%upw-K`eE|4mmFq9<04ZcHAuisf3(P^t=xOkk;>iHt|;%xQ3a3^!Ah`< zVac3m$*>w#MNasJaEf^lq{@)zf<^~AERn}_bh9*3O`LSk<=?~+P@yu^V*`Lbk<_{} z%!aAUkK_%3kK>_MFClv2+=lmuvQp)db8v!`h}n{B_81`H7SS|GnGgQl?B#=#r~u0K zOyd79c~<w<?t>7z%4V*J_?^Vn)89&}aZpaI_7if}zk2KCsT<G)4Qm-65$Um{aN@0B zK*Wi~{}iBV5n3S6DsFb?ZRZ|s6O$+^r%@{v0xs+huVijpe>q(WpD6_m=KwQ{8J?G@ zJ1V)!>!SMp={eAuWD+FLAL&+t@Zk{49)-O_mMFFZ#B7FQfmWMN>@bMBXYyuKaY2GE z;0<EIfnslbq;ryTCMXapyGWZMUQMq{5$@Asy(oL|_oN=T<oii$8qwSFF*HXomczS- zKhNrVEBRvK%u@RuWis{{RKF^H6d^LGv<T?+*P*73!af?au|Gu4b=I}C>#<+18wzh8 zUru}u7?7y5@=fa*xq!-QSJ3Ke5sYp7q8-I1in@#$D*JWZl-uaUybbI9mbe|F?ep>& zeJRUHy`{_nrjsos1!@;CzbB^PjwdBz>}uZR`-lE;Xbfh-qM`oDz1C5*e%D!TbG?;L zH$k+oMVRm`CPdGPJr2j4E1naYFlgdWe^wvJZ=h<orlj*Y2vN0QDdUf;(6BaekrGm| zBQ#e_lygw5j5#92-4k_6hxwC~QB?=An*)hPgk(~xLPbacW!ONd#nPYVHt>%QExrO| zlqvyvQ;8~?jC)q7dtpnV$F9Otr#BE_)l{8WYm<_pOZF8i(zi3K;-rBP>52Oya;y@2 zn%uj@z;eu<T>SBay(#yo?aIw(=O7J}3+}<mZz@5E7PS&HBZD1AAYXLH4A#2JPpbla zJ@*jhN`Q$7ydN=mW84B;{}8ypj-6W9Ilv8cmNamNzEzq89L_$<Sbg4{y@CKW*~g!h zCQ}@5010`O;SDhX<72x>2l;wAb3zi_GcR0vL@Hr7j)W^#gXjywm<(q{PtCY=TBnv{ zC0=7D$o;lb`Q{QLQpx>+<*f($CT<X~H>>KckNtP@*_5r-cG5<|StZx;pEeRs+&+q) zMIHHm^U5g$6V(oD47BR0OR3;e&hQ+YA(gW^<f5c=yu`ix{LF8bo4tB+>ElbM)!hmG zhbS#ho3!_+I!XLY#c*0WHw4-B%7fR;>*@?bBGM07zLSbJk-{tIHiUUiyccDI0oBXu zXXM)#;rA&wl3N!^H61Du4{q#Y=zd`$t^=j=>7c`d*E1j8*qBC{8XS2gYJ&!tFNiBo ze`cL5%7(H1XHa$*|8!w!vkqWXrJTp%%a;6c5#5td1-`^w2Ex6e5SfQq*Dh<{O5Z_* ztkIEsUtllwYZpK_WEfcFM@NVNwfm<4S_KY&u^FsTqcG-&5haaidX$P#c-lpYY#?QV z&>Q$BTgsLc-wAxE2{9GlMGZntl%OVIXs3Wp#8^M5cE)2!(FU5pIx@dpMJdq@V*~My ziLK1$t6GRxM0t_X)XsUmyP~~~-kn(P$NHM9^??{>{i|u@S}b9bH@=;=m(JI1oyH2n zg42VuK5aRP<S}<~yLrM6Av=I5TsA|`lK6V^!xmwXEZMpd`!rDz3}9I)d$Ktw3EQX= z`xNs!!_v`C)>A;-a8%iPOPY}d35Er&W>#ZkBdV{1_Bn62I@j&@d+2`?tPq69)Ul1s z?TN<VK%s{0hoN5Dd7LsKzRlp5pO`<5rQ=5NG6I0f6~+mo{=nAp$mj%<Wh>RdvHnwd zIerQ+LjjV+xh;a_It*N$8PeZ&9F_qv-RoCG9NCok$ElM*<faM|Jqn`;Vgo2pRLYeH zc95@*Jb3z7yG*cINXo-)8M)fx7%o&ke&pFr`)KGW<}dF4910-sP{!Qq#p8g`Hwi{T z&Mk(0le8Ae+6;mYj5l=a;=_A*BdBO8Www!z1?iJYsBU1);z2{^TOOLJFSMo$WxYHl z(GMSpCBi28K?4Usw!f4yrgTSmipYa@5|5NO9DH<!95CdNN(YrM!rpr3yYnFOJ3=j^ zC~v2aPrX@v4LXB}pp4$zWACj5kEssAbx@ZZU}hP3=TKrHRyGtsi-Dnn6x+Z8x53Up z=qKio1*W#tMH4N&QE-#?gpSERRqUC9@>-PS0D9bqWjkRmgB-yII7$}m108{2pH^j4 z7A`N19hJ-gAzs-`dgN<shwzTc?s~(n(Jd>LwDFFZ0F5HpoGw7JTJS@?08y#nBIEj> z61{1>n11esH)URRsVosyxcmAMn<Q(UNJJvkjY2~msf_Fb1W3pwM+cP<A><pvDuL(1 z6`icH@q*kM>HLyj=|L9O;y^fR@%71d+P^J&4$4LbG8Iz*BXz@XYQ!5yIb)d8#e{ae zg(N{GP~nwu*nzuJWh3KO0_L7sv`&25jJZ2FAo3TJzXS&^y{Ze&i?ygcj2}(0993ti z5C!rYXz>$fSPpEcJ+0w!WR$1Qp}>QZdoy>S0FXF0@81Hmqf^ww#U^?uNG!)5`3r>u zt=y_H?ZUy`2?ADz%$itv!ZPF*?@FD`Qg_k~w>}fPYfJ&pnjlY*2yncAv?IE<C<8Q# zG0zo3jSMK&FNHVc=$bG=bf0UMmH0a6dtV5&ZBu=}i=<wH1hST8+$pkWWchVG&th2a zIMQ}*@0;Pk$M&KkNBhB4W}S)PHiT^*tjPTTjdw2L<r6EQy^U6#xs_D`FmbM9`li+% z`W?R>lqj$RE_b@XptFnbs8%J{CfP0v{Dsot6#|*Q%6HTL>qq}U2UheU<g^&%x4=B6 z^m;Lyn1V$%-1Fd_Eg<R|96Y;DO^68|7n=^DkB@si=!P#-+ufCqoboxaPTdWXsd`NP zI&cSdf+>P)*69NCg>)xXkEzcEUnn)`VPFtC^+O|DLEkTf?SGS^bGLrycg&*={-)Ak zV1ag#SbygKYxN;bSom(bRK?V>tWKc0Vqnt!rtVYk)W?1ia8e1rTep*euB<`-|H`v~ zm4_TlM!gT9AVGkj4z*lm)EG+K?#;HLQjUxqRp*&oP=m5^_6`V8S6Dx(wSx|x8gvFK zo|Qo##`;ectsPj+VDcT(Nur_w*-c8=<IRr<W4Fn#%M^ad4;K`d(l>96#CDtyigfn3 zs~$rSg_Ak0NIba>ew!UQAdUbPL5k%0V{J^?eJkCF&MkF=K%Oll&JVo+7ULst@r66U zhNh6QkeE(&A?9petYJX6e8Xx|iAe9F+acP6v^S!S0}syKF!I6!nu6#KCdz2ZPbCMq zpH^=Za@s@jCf45oA~!iD6?aU@28&R4ETdb7k6#wG+EnIcG3&S;(yS~(!5?c&XQmEr zGe^R@sK7N7uh8?bE<~sxE|E+|SekI=9NZ{QG1p--)qBF`B<Ej9jLhk%>aSI;VScpp z3!DzQ0}uZB>0t<l0alebDZtA}L5IgbSfM+NUIB<W?8v05x2$hnaG3qWcipDL>=WpZ zGl)$vEVJ*f4{6efxW!bN0#$a9;|hS>0fhI*S4ADH+*{P*V0-=Hx1*qzD)13N>Gw4^ zW#GvB2x_S+13UB+c~;5ZUGx^-st!Ejs$C2^_NlmrOB1;a3(Bp!w^RTE$VmOj><OO= z9EyVz?+1%fX`rJCCziBe@Kb51?Jh_$r)1+y3cI(VZC8MYC*xQsBu#3iAm=;1uK}1* z67kUVPv-@?9(oQ2u(8_6QwcmDh6>FbS&p(~a>GxI9qUE-M#9Wh3A=Jh=T>BP^~OTl zMgVtjCCY^XPMj`Gaba|->tLMO!y8588-ZH#V8NyEp)t2v4C<24f?B{pUEUfh))C_C z0aSxu$0d^goGi^yyW-KaMWr+LoR+KCzXcE0ho`pZ#Ygjd=U2n{7%*Gv*1qf2eWvE9 zam&GF{tohk2POJ3DWpKE^8AqkRgMVpIx52r0mSPd{})2SOhfXjtwAFl{;_@#Bs1R_ z8<6mY^|@wjhh$|{P`0F*Ft~t<S~@lI%^r^ARun0Owo!o+3YK(PFrkM}g)wmaVjQgo zxL9NaNHC}`9btU20HL4!bR98qbO%%_QAKUnaEsb{s6$DR65_G08c9%u6Gitu0U)U% zjyC}Px<-sS##1Z`g#6SZs-uV&fZ+xiy^Uj@T>~3I5UgxwSU1Kr56+pkgDa3aH7UW1 zo>?^Vk>V*u^P4JO$dC+=S%3tlroz#_Y-|ILx0DEAg0IxJiBuc4=HR1Mwd;^C|16#I z&Q1{kKOs{*htD|z2Q3D@caLfhq1%PZ2a!U^)lqc3^APj;Bl|p7z?)qT!8Doc5IqGc za{I$Ru!}YkO!V1juvW~8rMm;Tw;UlOy)!8}4on`2P~^`h{DEj=(Ay=AT9S?{T>~-E z5*s$K1Ac_FbdIr?40>ys=ppv41ae2T*L$l!a6FL>q{)adEd%7eK&S6^Dh6{<J`rQ* z@bcseuPM2hN)9q!OoH_6Ao(4N7IE~kxb21TkpDJUh%rxL0~2D#H3kCZr($G+4VI>! z{7G1vHds&D@UeZXREuT~YS%2LfQQ8%w@H-;-zbP<MTI*+Y`|H$XJYeI-f2OOyD3sg z4FB{wi_8c_V_#6mN%Ga~B;EF@ghO&dJ-$7ZIz>(g_ONN4s8xkbRLF=yxDKuX*jww# z8wE@>A9>S4UdywLR?VuCqtr<RA&kD>i5>EF`%bJq^yGV3<C8y#HL<ZrEJIVWJCj`D z6c2XuBAXy&gvKuzWRfrgDK8xkpN=!KWt74Ml;rK3aw4#j)KV~WDvDq5r<@KoNrUP` zqB*EQAjAx-EV187$ybXg-a<bJQd~}3b7Leb(Gp#t8W5jyM5mB|k6KiO4d#~?hdPAm z?VQ}r+({J`WTaM+`;MY>KNL7oW;5P^T=Afg+3;$c5#SiBB?#@ne+9Uf@E2UUYC-uc z&W`lCyJ}qU^vqA(E&4B{*{Rm1YyIn!cdjcaO=uKkc`P&d-|~J_gBe{n!DeyOwU%0c z*Pb`Eo!`%7NqqBB0_VbW+u3ArIBAW=)}9W0)@NzjM-+zI<08|91cT>uZqNz*@y40| z&eOs?-T&AOhPC2OtQX-ae|D5WZ65u2>3zORcOBByh#2;H@JG=->pz<Fu-k#CKk@{k z_E99BlClVxr7J}t<XnC8;vSKv)NT*E8|kZ}Y79KRww@Y3V72*jiSl}?S*baNLNBG) z?)r3z0@A0bnFfhJlS&{Q*CJF_ruRR1dDh{%j>GMptSk(*0x*^Fm7!LC{NLMh4oqWd zl{<avoMW+?=K~$|QO75{0%ngtQ~f*sMfGQjEMy|sWH3l0398dkTStH&ks`p)F#}<b zl*NW;Wv{s{LW>OO%~%tgKcx>f3#>KE=cg786{5$+5yi?upJz4HDgv`Qf+i67VF61^ zI<h)0tkt5*4os~NKN=c^ygz!$8;}fLG1P!KC%q-9(4cX;$qyvnitAAqU93qT_<(XO zgt#!F%EMcjoeN0o21rZY$<$XuUQzND2M$2fcBM{4>a$tm{x1uSWnwL2pcTx?$;x!c zEbG&f=3-fJS}{SFfH{u2UT<*|cz0@8BZUU87ve8*w56yOq2<?aD&T0eMMW#B!Bk;y zfD%PrkjGcBq3gwq7i>ehzs!F<WJhr7<ZdV5R(ol9Fhwr}prqCqw7HY<#8j!h(ZcB; zfAbN2_6C7)U2jIdX);fQMLOTgtWB5RWE`a4Bv9#=2&Ya{B=o-e`uwRLjQ_~X@t2V& zzd+Vyo?hbe9Q(R!hOH(_;~9n;pH~S?b(VZOFNN{CW{`i&vnPbQYUXnAqfQV1I98t{ zR&*92LMYg6?P;fPy{P-~AbuR60xs;@fonqQDM@8#C%%>+rid^i{@1Jh<yQYXo#i5w zKuEFPhmYs~2HVcFmegw4c33{|%Rjv(AGg9HYg|Nk?Z=BSceOF)&vqZ5XO{m_T=;K8 z*dJ?pSFQbHbgx?Tr$CCOG5p|%XToLG?myq#s$EXHYjR<gcOnT`_sfre`X3ChVVZ?S zMxQrN22o%3La-u~YSCS=3>2>^Zxq=Nk`fOsBBek^4BrdD35y1IclHw&tGMfryWZ&e zqHh}O+Se!?-1egxjf7A#a*0Oey}Epw@n#db2z4eL;}YV$`IA%+2(4i1HGLUScmmYz zNq`1J5X9y?eA-AVQV1f1WI!IeI;go|Pw*}#_yoZl)Om*(Z-g7hLU<j-j2WuZG9ZB9 z_eTkX!A<w^*y8T&@liGhU#6qb^zZs5tu<&nz6b|7uh#Xa@R7KTxu^up1cN3ht~q81 z<U0P42Q&H`IHc0!pzAQzFe_sVypFX<_VD(Z5&A2D0Vou}Rt^Db<$f8uh7Ug;yK*C^ zBy61on&+H(19*})w0>wnvm@sCzx2e+rsiSR8&kk~xSHN<BH5y-11!l-jvx%dq+M%3 z94?7cQ7RC0+(Ak|_1clNKphlGk0GI~Dzn~QdQ{e09&u4ZnQSp|;9r35TQxv~U4Vzk z8B?`?sw(VE_xoe$(G-1g;lXe6wj&t_DG(@b|6EJ(uY2#K>Ag3;Q7`3apUE{#$rYbX z2HzL*=~ViNNvedK7Qf)X9sZ1!`<4zfw<^g|HuJ%pdRD@=@zyzaXB98O@8Tb(i!G4L zq5{t+Ed5hZq5n(g_yfP_p%+~vg+Jg=?E$Ts_w~v2rpycYorLMnfBLLA`2!LtwO05$ zE923}DDd%IOPb>vy2mUt65DSr%m_ayYQFd%tc%%%ZRq0vX1WtY=4)xzD+#jpJv{L7 zi<q_IuBH|bIMjibtQBUm?!Nr%7uC^R7uLQn_C!hs*`~gmLu>#;?Y&t15sxr|bD{j_ z6j}7w{3l3`8ksszepr4vr5HB>Wt&G1F|;&6f%k><6K($-5?QQVaBf8C44ivOgEE*n zlKaDW{tVW}9_okRs1h&o1<Pl_<`Yv$5LCs_%tLDVjaSWLadYtF^NpQyPrjRD)-&Kr ztCh`6pun%h=mGHG7mJJJ^t6orx?w(v<+@@}zckVJRf+K^)cocOjY3U`sl>iNc|UT& zW73`A-nIbQ=8AbYG(>`o{4@jLgG32x?kHmdhdj^pvyco%(kn#trKl)8KYT8RPNEaF z!h;?vhEQnY%0&jIm;xg9_Jea&+#nb0je4L<na!c<*RD?S#-XMN>$Bkox(PM6e50kg zO@pUK+z-t?Jr-ATsm@!VbpKI__kwC~U4B^O$JdGO(2C~gg@xqv-^b<TiMGGVE7^av zT}w7s*Dyk%szjb=6C2}IeyJ)qLi3W=9o1wlWAiV(lk`6$8#X4Z7!qBBC>-BELdq^G zU$QzHcORs1Oa^7Ag;b9FEO<=zc{G^Zsam6dQvZ)I2E9%qn(9)jHc9$Gss8oy<h@{h ziOM6o9z;h@)lusQf}JxQ6NF1Hd9C)V*!y%~MZ1=1$mWTCa|cf-RLyJGQVq#3O2wd( zX3TKPKK5F>Q;cgiznA+DU1O(e@+pwF#lZuy5#8AOFAFFzq8?3g;G)#eSa7{bLR#9i zrFmZ|egFG=bTP3z3sk%h%pLUo@ckby#%%!V+THng?ynHJ7i<FbO@lVV6w?K8EnOg- zR)2sF4eR0_?h#AhH2_=`{K%n6eMLODT?j_gSeYQjDMb<crobh4Ad*PMQ8f8P!8lG6 zvI-&fQ;rE#G`v#-kpevjw%2={bVcZaKitrTrc+U>Yy95d|73;m1U_Nn5-9w%{pe2c z0wPuZ5_$d_U+xR#{xFI5ggDbYFDec|u=n>DO9v}f$k0>_lH|=`2;D(eR`_q23LO0v ziB}{8NXsM(6aFY2Y%Iw;q^YS3B|q{S$rp|TBS)6j#4D=;2v5>vqb2CT6LaF4%b~pz zrRN5coeeW>L#e?U9Yd>}xYlFM<@bWm5e|s;*Oeqnk4#3`M=I2A3idV_a~R^Pt_KxD z2XcJy)wURPeUlGj`j_AR4n`-r>UyrWQ&V|+S!6h%aXi=~B7De?jyzlASr?m7Cdf<~ zgAqgx&9!)98DvAn@=={!yScP2sS}+Gew!v}px-p;8%^ak5Ol<Lz7S0)>u8~NZt7eo zh)pI5fQ4|J;XA$gQV-KSJOLZh(6Seg#|O`B8EHPQSRI;AJ!N(759fK0<Y&*j{kQ4~ zL8{k@prYp|7v#14xua+q?UL)(_GlDr0y!6(YuStnK6d7WYSMsf2;4}R3($uiw58yG zOF*gx(clIrVds@evVOSM3Zhu4kA%GOcc;zc%J<n<Pk8l!5GqA8a~M&)n7KeEdW15y z3)xAXWi4c`1MMD8{yUtE*qOKI<90DRCLDo>87zlYpvR@|7jRS3`yKSN#2Q)_A0T{% z0+Sf)AVsya)O$se9ZjLg8F_h{87P6`lbLu!`-N;xAXZC=io;|N8R`{5yH!78VxZzP zncEoHKy%{gea8B0FQaBn@PiV<-k1-eCS(;#CMLQdQGsWcs3@l9zPF}}k*T)oX;Kdu z-W7n2H24a8jjEQ!D>}spI0MwyOfz&IofD2mWv9T@TzGBT+meHTEx{Q|Q)`g}>}fnO zNkt^;K)STR6UO@A+*9qUx)=^og#D-nybqEQh%txpEpP5_uZ+%;{pdU*8iyuX)a*~| z9LrXqXh}sf&6Xf9837@(X2K7Z;2$7m?6hGX)p?|?60w8VqjpJaJ9VEy-csC=eXO{D zuo2955aOfa_6*x3B|kVJvWwgwZ4ofuvu$Rcp;P}k>>#WX{Zv1+TOQ=Layl&2=X4g_ zY&DhlL**<rL{l0pssE-PTdHe)4@akC;0UyH9?<M#FtL$j+o1j$azxWmG^dbbuW=HH zg~c~(Ln5bsF>;uYQ-{1>sI!w=Z=6o<9kwB&5Dr>zgtjGx__J_yj@!fad~j-e?T0?f zj9`H0CP?gcg^uN1>ZYQwNmLx;Um<@cHFVvX2%Dz6{fm9EtMNkb(+rGF2raYGIqD0G z$gOMGAK#L!Al?d%Mr*tFanz@3oPp#$E^|%Y@N#CcXF;YgX`DVWt60$wZ)rYfKA9J@ z=e%El0|Ll-@fv&|3nnppNT5dl;Mx0l*M@01>itfR>tSA<jZ<sSmi3BE4bP|0e4l3d zRs1%-sQCTxepCgL<XUuE+%Mr_JP8^w?+8HYp3x9E7r93sEwsGweGkX8T9D|~O`QA4 zO2b8OVHRG6WN_&Sgx#9r68>SUK1;B+((vHjIs$G^XUSsBXZeA>A8NSM>Ju+>SI~4i zUlCt&7wgLMQ=g4wza-z_CAbuPp`ll4yyEX#?06icIUbkbde5(u!Vkmpv50&0(O^3Z z-BeN{D`}urM%0A*!;pNvJ1$k32ilquI%ks&wsZNLud5z4`$F=JIRZCvwZ;oku}6Xk zLk72iT#^OB0vbo&>5%LbT!OZMO$ZmWNX}NKnqd=2#TLBk#1jiw@rP|lOq$J}h52J1 zyb!pGFKX%P>W;k0pVxI@)|QDVN3;j|q7<sKL*zW$7cLD8(<U@TTCZH=F;I2RTYE;( zb8><ah@ioM#P)&$>A;~uDlkvuI>OVi6#=@l`rKQ|VnNz}WYwp%L49?K8rjFN%`t_C z=iWSKqM^XMz=iQm6vwA}RGDG~kznvlh0~O>XpesO$qK1@y;5o}%A&zM*yA+gh&}}t z)HpTs@7sfV`tm8W;DuKwSErf=d*eNo(m5f23kVx(@N4oID1FyeoBC^F^7|wOQp4Fv z<9mL>QqgP{Wmv&Q3&U6O$Lc?EsP>a2{=uP|DYsO*J$xj}NC+!S6CPkE*b{TXG^Nfl z1Z@Kp6)<5V25({uo&_%vhC^s|erDiN0A~_nW7PGy<@w}kAny-xT}1mL1jm0{A!X+} zpow_|*h99&#L=`Nct?x;m3sB#L0k<r`;rQT3h!#=aQA_d{q%r@VaZ5F6dIX;i7zCl zWaJb6LD!De;Hs%@7YiR)K%5S^_Nw~mc;1B9VQgJ17&C~mE?Pq~8?g$Vw&A7r&xRw| zgYB!KClRsyZN+{0J&B9G_l-`BHS?uN@OuLXHYPjLl&3>YIis`E?|T}wYl+RDg=tZ) z`$17jkU;4a(x0s3GMCuLOPFTii-5QZ&cj)-y%^1`kTP&;e}kC?Y1g0}EkIn^gP$L? z^L58>leBjPF@1pvG`;)iCfM}~IcGlxhpwXb?v=YblMv@juWHf!g%;{TZ$U&5kAUZd zu5uqThI+_2-=HS1l7>X>M`!a&4@lciAr7yT0=yw5`NC?-{XOOhj3I;zXVCzwlV3~s zcD2*Ys!i~c&6**2WMd05up|lP@c#%PDxud*Pg<6LGxkn&BRY3{kxf=P_h8x6j&vr* z*KF+;KQ$3$bZS9w##^U{Xr$Tm3dCLuwarV4m>8|;SPe>piH+=8<O_>Tv6HS5q+XNz zp!>w5<LBz#60n&|k$5&0+ATm&lGN?Eiktb&+SlH*Yi~wG?)p)f9rkY5K7`}mf);*h zO4`0ed_I97b|2X(Yg0*!w-6)}U)X<YH$4-Ts1c-I>P<)s;>|ZPwQ)8nFp}~7O>5d$ zgKgY|o!kP=UmWHBZ6=B%9hq*|UXOgxSuHrrSJXR?2!aI|nUYmkhcOKEuR<{~ESzXQ zHKAg;QukKvqv2ArzFtBN@CL@Rt8z>8<+U4ENq=TtP4hg-4vpIBs&Wl-8(}oj9T~Pk z=5tA9M?+Q!vN<%{Y^j4@v7Kf6(NAASQXP)$dSto7l;Mg3HDiaI9KQp}<TVi9+=f`g z4IaC@yn0NT+N=WenlR}i{pr@l<|!F9Mc4xASmGM1CehJs1&=Jv4nlhvsw2Xdt2Yav zxhSaOY|@aMYCK=@uXFjP*)TrPq>P$yl~>6Dd`z!;OjBWis2GnlvR<A!nE2PQvj<jJ z$0GS0LN5$qa0|%HR(KnW(Ar7xg9d_Oa9ePw-rw4<-9oKrL8JZ%0t0#{Z(o^bxTfO> zu4F-;DUwoY&@FIZ$!<**oi2W!_#?qy+%7T8qR+2n>qq<Xo0?q-&7)tL9*ddv0%)gm zBZr^--DsXi`Q*KMUd;RqHe;FqWkgd~9I@#I<lphdhMGHQ)+DLCguD)H88<}kfia8* zk95#12oS_!cu}&{ViE;!3r#wG=x@AbO*?>?AHgRD+zuHe<AE>9B~Fbp)rAwRTy}G+ zXmf;u2gZjGn4+E&L8R}DpvAdXAaJ-^x#O2x05OWu!omTT)!t@kK|mO^VC3P<`sLJI zK(;JA*ww1*NxYLQq8=j}Q^;LwT_tvQ(YhjV^}*ytevJ*C*jil&GBA_^C?(RwD|?FR z?$K&tP9szta?qPVR;!cR)eR^C&J&iUw5~1``)mW00h+&53J*pX&G4iKYzN2yn5i?W zAq8)pc0qC`47kcP@Pt?|YB-@08i;lrczn{Uo?2tR>C!*cUO^Ij*_o9iabqwJMt(MJ zdJLw_WMN-bAcxSS5}QwwK4udCfNnH*Y8#?7AM39piIL0tWauCnyWdlZjcP~V^7)~v z-=t5+Eokw)jj=N9w2B-{8a{@7q=6|Z*R;18EQf@zK_eNA@LlANChEcxGzL|!<kq5x zfcch3Gl8&_)Mej+5@z!Tz$LfA+zFlLR2?vjsE>6X(fpZ0^0K8iUD5lkPa#>>51_@I z&LGtnl($8<&y<k%RpS1QDW8CWigH~bvKvr3{W{K24uwas=~}X8P@i%o&Vcq$#z|pb zODL{q1v?~qHVRS+L~|=hgEty|g;PXnUm5O<*i*(^c5awQJ|db9R%Cxly{D_4f|)H^ z;;9!o$)n*z!{8N-bg9xbk0n4I+0;=#*l98ysUF64RK52ONEG1=>Su*ruL|NhK7pJ< zZUTx+CdXTRyK$7A7G`L#X)6CyKgH0$`YAMb4M-Kr^Hj1FiEqUU(lm`M;LEOeEA*|q z29V^m#pZXmdEWlcaXp4{f9u|?!66*HZ0i{@=eCDcH61O*u2n;0&TIOy3ssD>kFBhC zcfPJVgEl$vd^Dj-{jc(L*R;RkMe$?Znx%$1_S1wPq~d|PgXhfgPVh0$Qwy9DPraQm zBtHf48^%Fj?Z+Rh1OYVZ7+g078)S_mOwF>G?zi$oO|ol+s_19UBAFL{4%~=2Niv6? zT#RriT1f-}AqtF;yHXV1PJLKt+O&Yc#?~I);tewTj{eR!PLBFu4swxw2*{OCQQ$jU zz=8<x+>x)csdSR@;-p6B6D5LOOB1o#h)4t2T_hBZ0zW#UvGv0G2?6A1H20Wx&(4H@ z0nKoin{0z|fe=u~Wk^NT!-V<(JKx0tnbQI=TLaDA>JgK(t4EbQX&N@2N3sfcqkLqh zVo`?)?AWohCrwcmw4L6>41uEos+VKbhnz-{*3xn8+Ar+ZV1u}))*Uklku*Ybpv#n> z{or+;R%H1-nhJ{q7K1?nuDGm21$%`#hC@o!pEW)G)~n{fY<)@JRPqDUVGGh{&(#;6 z`1uMD4e+<5)Yx0=o>A~H$Sg-WO>>mVI7TB*FhFT3b@Mlsw_w<eo*hCI5U4Z}5Hykm zMjvJ#7rghol1ERk^NaP2tVQI*lOFU9$*0_jcdI~t2V&jG5=o{X8kYnoMM6^tri2s| z7KhMcj-ygWc7F^-WQJeR)JwxsQ?7!?YwAm)%B=y-1Q}7c>l|NUek?gzXnrg+NfFZ? zI!GX)D2;gQ;v2G=bLP7>OEDjVhF!~GC?VOespm4%L&)SzUo9C;5b)YCDXfg_k2HLZ zX4cXCCJaB|{=;!Q2m4IdD;GvE{n2wiG!q0WwoYHi8}s^H>aBu7yo`X7>%{vfbMC5L z`gN>e+Lm{)S>Ap_6K|hjBGM+vv9qYNVn19G+?=CDa45}Uf9nojgBz2@f38>BQXD=* z^7UKBPwEz%8kP?^-DITZX08r}8KwlofIR^^er`^cRhwG6_^T#T-#gPUlRH+!*+s}` z7G8-M6?oThWAn*vejhU+)P!!I>YR!<_mQxS55ai7AL}f|j3k|%^Ebpk>de`MEjOD= zMB^v1BiZ9r&Cy^9U#!D?5=`{rQJEcV<iO_5@$EW+O4S}H=04Og6G9P}up51&C%P?t z(c4trRRZRM#*ob>UkC$VscGv?ioIv@YRns7U_BQGmc1`S(0YbN9_hD8Rd>UXuKn0B z2nhOhAPi=sHd@6z4fJgRa?afc!D6yv^z<4(QMORi+6*oINd@?cu7kf)noms{hY))9 zV@pd>$nZrS&i%(|-Z6p@WK-FM3S$<jPc)_3iIW%NxE>&CHrnAOR~g;9R+c8<I2V5h zg2jF`y-FWLaY66yKDphgI@L>1AY1fGv|=SbL1Mg58eoU1%ly`{448*wj=+N;KwCph zT3VX^N&{awZJQ;k1gY<s<7Jwi<EZ~EdfkE_=4%>U(z<;6z-JkDD07Nvx5Mm?j$Z4M zKUmdwy6?sdPbv5<STn$vVJKq11)g&y+$O<H&n9O2GcmdFwT!s)i9h6n)Wq5XBL_po z{>OhehmSbTHf8Pt2F`uH%*=K-8o6D*YnxCYb^ub@a&j3{u;BrzCkE7_R_VYn$%3jv zo6lHxi+&<g&ER8d>nHam4IQV6{VNIJklsK(ax6ZRr)^SIP^5^kaVRUW#)j^Y<@F6# zf;7IkgnUQT`iO0?o~FDJ4N4u<)KW;vpSm2FhI!h)IvZd-%_N44H-L#Vs)YUExCqC< zEOV66rO20iT_^v!%>Oa@QNZF!BcBygS#I`L(wx8%`Jrj>4ZIE1bI^var9@>ujnhM9 z*PuCH=pAIx6ePx{-u7VHbD1ITDsLf@w9}+U8W~2)h=LxC1jC>&aKSdSeDBwq*wA2? z<~(l<xsRlQzr<vb2L>T$&s8%`4)}Y93;QErovc;F&`=rREwV_F8ckgf<j)UG?m@xx zgo|<6WO0_oA*W_&^s>Ko7}tOu>Qo{l@Uu*hW8snE$pI-E)rYE+>L%j)P2E#y`V^Jf z^w21(G2{&8K332IvHCTfZ=4vExzs=$O?Bbh|9+HVA==gGZ+?hD3pX>k)H1XK+dyI( z?Z7Ho)0@9|TJc7_-D%J;9E&u*P{*ITfeHTc24E<DBau2xtBs&UIX<wJDsY|?8TaTe zM=_-6cc8I$n$L@RVK4)(n|<a}8}tArUC=^Au2N}vI#L*%7KYvaj%R`hV0-<57v4Z% zUKGL?0VbNid)hdjanjq>r}6YGD6mmKI=CNc=d0LDYz~PF^ml{M5Od}=0FTjdc+oN( zI9&@8G7?r$CZiF7dd5yVzkRt-+gx}U8MTeNdEko-@Yofm8Ks=nKaLd-MDNa?4b3pV zIN;U9!A43$C62~{y<0`be`@bQhKlv?s!zA1iJmk#6})j7vIL2N9<7Xrt!kN`iL3b^ zZWsjYy)@H;M!41k5D=d<4r)P@LsVfFn$D=(iSgSK@h>hfj@`4F{F=;&D=I3;HGua{ zcJu+12YZw>vHpnv|M!y=>D7?&i0CciJEPDCNalL=Mn8!DAC_htCgnxa?=mA2(fMpa z6L@Kw5V7#oya`yx5urqq?c5W_AIswx|KKOc!Q{FE5>t3iG@OAL1hNheHGA`EmM<v> z=pqibdgzW9PVG9RS}+5^@#IuKyW);2ptFzwM|>L<8Z^wALIOD}m<dMwJD<Vwa+UkW z$;AwwL^W$v9L#WE%%~F~4?Z&}Ui}g@Zl48r@d9MPAyK1*)swsPl0L`ZKtq4C0cl)I zudQWtxBqM|%qOBQ6PZ8lW&7W;B>mOr^IupY|LTX~3(M(ze%SeMhBkCp>@6BUUEQ?r Gm;VL$ZQ!r~ literal 0 HcmV?d00001 diff --git a/docs/source/_static/images/tpch_queries_speedup.png b/docs/source/_static/images/tpch_queries_speedup.png new file mode 100644 index 0000000000000000000000000000000000000000..fb417ff1db0f08ac92f69b1f084a15fde9147c97 GIT binary patch literal 38821 zcmeFZXIPZk_a$0tV-{LPF##$dNkEaDO+*37l0^_GGLkd43W@?!<X9*<OU|GoNhyIM zNlub;js>$1_V0i1GxyHi`Euve^wZkaQ1#aPp0M{?Yp;FYJd~3-arDAb3<h&TO7fl} z26JE>gW3P+@Im;=0!;%0{D<H6{v%r@OG8@+y=PA`vU;{wW|p>Q*e93mpFXp}T3Xy< z=V!lh{j#yGt(A=c2Z#B8y@B2GnGpxqUy=oIkt0@;>NXe*g&z8|FIhARi@}7RlDc<W z*)eLm*TGh~t*>Um=oC(D{}Jw1%)a<T&tsRbsfhUy|4KOQbK%}=ru#lz3b*%_UzI%> zY<RCsF`nuE-!EBD-HsssMRrGi|MS0bOFPzm$KM^f-_Q_lP=U`kylI5Lm5V>pTy423 z;?jM*>c(Gi8yL*>k*9WhFEC=y$<Yf>%~-KR=*8y~4`UCY7Xt^Gn)aa=RtG3yG~l12 zeMfEgUi4td1omE7c<!g(d%^pk%Ut|Fm-+v5Z0d5PL$0Ujlsl_aA6Sc3h!aoMDqgH~ z_E*i&n3zby|BSu&Nl{S|cjwL>F1JnFv^b2Xl0xi(&y`l=O;=W?I^+|igB4e&JI|au zmt0~uWzv<0EqONL)3#uNo$D)ip2_d9T^?6Ri~D@yx@N(H6Bl`ut*kz6lSnG5%FK4N zJ?d$y894%7!drvhw0VYv3n$KV-otG?^(7dV4i69ei-yp9^Kfu*oMBeHM~LL@@H%mk zxEvf6m0oN!mJ0X!J(4$TdwW|s$3SXjwpUY>wA@0STd~?@n$u|};Ie8~GA08L$79~r zv*1=>*`GOv_s|&}99&#j2!L;xS)p}b|G=W096Q~aJ5{kY+u|};9y)91L0SnFaF|W+ zDYjv~apU1kSANT2m3K=y4?DZu>1z)j`d;9sq@<)56%`F;Qn;gCV%zL<hV@cr?<c*x z5qOWjGAETZRW<@%w0}^8mX`LuxVXvJmq$vRS0<@F&SKuHKh1@uZlKDln*K0JF8cS_ zSQ3q}bMmv1?<Mv#5j>WC>7lyr>MpC(&FSj7>3-pv<t}R`4PiG;T9X4{s<3vK9vLtG zu6N!T32lRKgdu&fv$G@MI3HU<TGeFJ$dhE#EXWfW5)S9W;v%@PA32|Xm&7H?MW@Py zvUfISX%#CTqhfCjVpa+g^>BA_i59S%Zj9tMO@zCsyK$VF|Jl8kB>9fJ6vFXmSXIYi zVnXP(i=W;17uigPMUimu#0BH^`9U=Fel)_VjiS3+OQVFAZ2em0^hY<7JVt|A)l7ay z@uw;#_(%t{B=oy)&4hHo50n~3w$f=_rf;GVX-Sm5aO1J(sZ*z>bxkCR2{NIrMb`S& zH8m!1ZJ3{CxW86-y!#RothyDhtELn*Ldj>@G{*f!cOH6^Q@6%RP>eK$E7khbc6<+J zZA+BB=(bqT=nQ|TbB>?8Ax0rI?@2!aQ>anqm>uIdbVR$%QQ_ptldDtCYqJ?2?s}Vi z`G?GTG9`&~vi2Zmwi<)k*7v5M<#{&RfDPi%?c=i^x)-rjxw}((({57v7_K)uIhpA$ zF58{toEFEY^5zu7yBJ{?4eBaIW@ct_*d8TgI2`WX=g(qs>bZs!G5&AfSR3Zq!s@8M zalFd-*QXB(DT)a$&i-Lx=jb#G%xa?eZ7iG!@lt`urD_7`B#6Qs^{-#QwleO{Hx0c~ zZ`@Z_T+Vj&>eu%M55hSa2rgQc?!93!qt@H&3*DpC!p?bD>iKL&AHWh1V$#dA=-n)s z(>v#7J=w~<)o*8#xlB(_e{N%OL}n?bx72}m|Ni~&mc|;pyZc<1WZ^N~w(MQHE8Mn@ zTUuJ`|N5l2S`Z&k9|wz2TujU}+i7Xkc1ye5IY0Z^i1gON=E~FwDRE`xpenmJZ{CPE zf4uK2U^f}><PSUR90Y;NHh&C;Q#i5cz-M8{sf<F~2_l2r`k*(9R*_Z2($nuj&2Kp> zj72NlNhLMD7n5MNyOS!(X+^Z)YUEim(ceq-st&d#E8OLO_WR+ZM`3OnGq49zK7G3J zv@u2-Hf&e1jZs9!+FXB@AGIKhWnbC%7~xXb5FfHCce>nnAW*2nR-%{ky~tZ2&n9a* zX_2awB$wha*T<ro9)==Bk(FM)X~$1JA6JuAZqxQ;leScJ4fu63N98srVGAvf1Bb|! zB_&^@_a{C$U%oLKDJ>;sUBu47a33zG0--KQ*v0<L^+yj$n=37^j!|98ymkEUaIL>u z_LJ)0b8}hnbf%3l!q`pOXz6VST955)GoEf?kC<hx3RiWG@?{+9RtT$_Tp*)-h~BwX zQx>(%bTlAN=F-wWK4;ld;pujU@%K$;+tSp!J-iKnMul6{<Qg@yK;*HTYL{Jeh3H9g znza30{g%oUb_xWq_b|1s1?HO6qV7F|c9A^hiZB$)WscSx8yn2@^!HvIJRxn<Es>>D z9(2n{j#|K0GD9O@F;lC!6#|Rq_Pqocj=KJSUD#MfnYxwkKc-aEAAOkW$V$VuBo>y9 z>b!jQirHglTj1I6*Yp#x-CG;NZ?&bX-#T%L?*?I|udnZdkmDyA@kWM}$6x;W9za)1 zouygnUj8#$FbU#q-v{BVSMP*!=-P+*>{L_fjz2#{^U2>zMiJI#TfEe1x1Ei#)dJXC zq|K?U3$Os#Jn#{`mKJy?E5VA2itmAp{y9%&1e<eVS9O&+<^IHrdepA%^K5txKw<Vv z&eI>tkV?EKJ{7hUTIxb*mWHt_l&6U1v;K$z>S))CLlmi+g%-}g10=X$Q)?Gl-S!t+ zd_CHT&$|8M;KUoon5M9s`nHhIAR%>rNbb!w@`vS-;I_4j9dG*JyfssJ=I_5_S7*9S zCx{8ScoYCXd+x)<LsFP(kr!=C!n>y)J;Tz&xvTFloCu*T6&7BEh+E0Q<L6@6*Bz+^ z?HdM5`aO2FQLd<>5KVt`nvn%!z+`JmV0)&vY2~#mS8hWfZ#BZZp)fq&>(H;#TVls; z_Ui@&OMx<&7Si(K5037+skz?LLUij83z?Xi>)m^k72>#_{g&+Y*d;-<$DxorN{F0V z9Qom}vS!2Tu;fJ4Kj+m|V4lXTl$iYVM~L%Sw1aY{7JvEn;tz6sHe`j(ovk@L2prxW zu(i|~<)W<CNk!fGSM=0;R$^#fL_m6g)sO<iGBJ9boLtrK5`UgqXHF@*<FyB2v5+T> z>)u~M0UaJEEm1a7fU;jfK|xwlvdN}VIBBiVMII7U6Qoy^>*Ad%(ek?g_Uut|^7h~L zfh)iC{S&(9;FjB9A;X<W;oVl%OR6<fA(b&f_-9f0_)Xg_vD9wxD;dn+7m<vuNXv?5 z2t6$iIxE!18u1;SxkfEQJ3G~FmBE|UI<O8dbyZbWjf{*u6~*IlAGJ&E6N=?<G_Lcv z(H2-(Sim*xGD;Iq%L^^|FMX@6?G~M~tRy9N=(xyM?rf$xuTF=LcFoUQo%i|xH?ckJ zj=8M#$sR+C;l%&?_2@3HIa9l|v_wQepzFQC@=SNf2e<iwFJAQeFm0;^rx>Kq;oiaL zes!y6>6lA$KKos#x4lqHySf^F%P28Cu@6>0-Sz7YMb0a!<!lTL)yYape%AeNYn|OY z^^|LnpyBgUf!BIhOIoJ^!(<miYP<juhbL7f&Cv29uSL|!zyE%`z1nR_o<%EQn>;u( zV}1$N8RVK@dDy@!^<knO6;*Zsubvr;x^3nSPEDCYlIW6@lFHWQbzQf*i>snsYfbs) z!Rs*F!_^W!x4INRmX=k)Nq-&!nuGD&uk1Lz+|GBfdoFnCojXK+=?4Azx$?D`Dm#b) zWdRUNwrodAN$NdOE^VnQg@iDO#|mEbbeAuGc7)jPd*t-BbAtBMk~Z29u0Gy>d%@b; zdUX{(+XE>rl84Y2(igM|8{q;^Bw^5@$fz*}Mi2HIHnY;nXA8$5a~#0aFhEcqTwvhb zdiZ2pn$b&d@BY<@((s|aBzf9lh(k3KElIxS-38}mBe-tTpNAPNfFyX+xal(FIGeGb zf`%p0t^Gf`g=Y<Ywg?uy)ewxn&abY@t!X{qPxcPx5IvmH&nWn?eyyLZP4CuRrKKe2 z>Ri9&IWIlsG*v=fw4g)86uWkb0c7l&o!7p;`FI&p^yXI8j~_q6Zs^@9*^rCk)05<c z%u49q#b7Q6KKY9pv-6>-h))W|+1bfWfJD-Y5zT|4I?hr!V!D}D_LKW<ncCg%TDgX@ zQbdHchN|BtVdb@DH?}vHGzu-E0f0{5xC#-8Ki|0d4#FWj5Fkr;qEcWjUD4ImHUIVL z3c+@=l2tMOwSevTV^Q+s$CdqQg!631<hGcsAzM#&WMv6CE%LoP1$kPzzP=vq^O+7Z zGO`p1rsN&3yuDfB(KR4g(@%6;R<=QgT>g5DgYSW^yyf==Q9E8dqcE#lW>8^ap<Cje zRCM=VdgHM30_cO;G*toSw!v-BbkH+0PC<rmqVpGeg3oj<6n(4w;ll@$=8x6dZZ$Xc zYm5PErK)B`z+T8u({)pUjGn4fUIGy{baWE}0^jzm!}Ko&Sl$J@F*47>4Ba*spGFJ2 zs6r?R&^tHL7qZY})7Z<||Mk_ewged}#5-;o{R|52h~Axe217JbA2{aM(Od6L>z=%y z>?ji--y_IstrVi$v2}HlTXXHAB#$rA*VS|GnRaB#E-^j&^w_J+BNd*luA_su*9L+| zVWXEMsim=zqj$=p;u_>zegIcCy|5!v;flX#VPDJG5Fpr2<LwrPs-0msuS}+8A;N>r zlqI>8xB0AL-gjpE2x7MHK7alkXMpc=V)WQv@GdJWn@*ETLaF-M*v|~no%N~@q|vKd zxPSinhhFN+l`9hkNoNU%C}@K8&Ori<7x%jqZV>_)Jk7ZIqeHM@FdmObsI7dND8usT zQ!L6&Frmp3{xs;CRrvt0q-^{hI&+>fNWl+PKHPb=49gfERh#8u>}R%H(xc0Z5VX9v zVp~etSH>J5&hSTZz^^LViL;}y?-8gDxO$&RX^~QRvS-_&4+i-J;4{R85H5k-G`>91 zGD%E$ONxL!AAj`3iSi%v#(Sf%KJ3p5<JDbY-cn2Jkxq8(?0A3#O*23<6ONkrJW-=a zLLzVwzMe{t>0dWdhSv2U*+p>JJ<->v_RD?x!~Ta$p8qBOkBl*ncYuwQjGTR3_@u`+ zYq#8_BQtt)ZH@!3pjG?~@U!&M!-qfNcNup%itJ}JBDxPBrDVDc5O&KNV9q7gAbGo` zMp3%obpdzb;bqo(01M3Mev04OSj*>B%qrc8fXg6KfX#FP+%cp1t=DAJ1rTSI^FG+- zs%B>CUgXrN7QH3>+m%;W=2ql1>-LEm-Umfv_u=Emu|AZ{AK~U#Xv=3Uc5gc^iT;{P z6kefCPgzSLrx98YiA(a>#SK^hGJ1$0M#bhNYlcp_2Azao>$f+j^DTM>V9(6Zma75Y zLpfDV$BChLDTT}m{{NpBFIuL(qb60!c4m4e=PvW}Z_rk%i38+#3$!;85E8m9-ead2 z*3G2?-mN6jx25V#*M(uRnY5>e!}YE~!~~iw4^yF3<#l3iQy4-gf@;yC9y)iujw=A> zLQn=0oJ_g%inNVrJ|5rCCQ%Lf#*7yBQWBn6P9-1=Zz~-(^Dk9Mf0W(L&ZSo2>Lf*+ z?Jc!*!h4{U9IyWOd|d!th?&hJz|{<}`(bgA>uze5+RKjpj8=lBL|<Q#v{fI-*it58 zqB)sA@*`Bu);5pyHc=T6Nd7J_Nxs6^m>B;ighMxTrn@i=(CzAKYh}lBMIFX79f?4* zw*kv?nYLdy?=5N56L5Bx4r2Nc6hujaAVz*HyZYO=aR@%=3*M{1x}cX55fO3FD&N!* z$4iH@XP{7p@V5Id$w4JULj@%Uk*(=lTihve>%5g7lR6D4uOwk??ck4~;dSkjJj9;c zGBno;$c!c%=hpf?3U`mwPe6{r`sB5qTzN~y!2+3bb+xkb$vPu`Of?*Rpt`nJ@ZHC` z{>uI<xdwjrqU!@M5Z{C~n+3E&5uVH;Oce#hcgk_%Ctz+kWUW_WZwg+z==A{sQ-Eu@ zh9@h9{q8MgPD_{0zAA0`>O4^o)C7%;H;pr5^Q}M<&{GOlWV&uFYUCKyi_XsBogvGP z%*<o}t)BKfR*e84*8N~%#PdP36y0mv%PsN{DT2#M08bTReUZvSLqorQ{R%%9O>pvu zq?w|eqPW^W2eAYK&HV^bj~%v3*DXdxwTVinIcKTWGs~8R$qX0+!=4x9n6-gmH5~;Y zhnuT2X<=buTw@f}0tS^HoGY}SfgITtjYd241<-ppfAFA#@C)0V2xHemy&zyS`VJ@y zS1*iU--6p|VEirWO2O>f8pS{^Z>_Ar&RA_5VpytZl^5B#2OA`xiVikodwYgvK8Pk< zZa~eNcPN&CP_Xjzs}I-s%GmTm3Yh7H1;PSLz8vr+c}v9Z&XOE~t==79YS2d~3`o{i zbgS1jW;xHKO&TS?MvtB0CXuJ#DLc&yxHM&9=I=u+juY7&r-NM6uIF58yC(Y9g9WfS z&;{8wtx31Vk?+Aa#=E}*<u4`LdMs2^2{<jrlWryaTrS#}`+DP@?Hpsp#oM=UYXDV- z{fo95pv!T;tQ76i0(PyU$GZ40>YxoQ!=RfIBDg2yzQ{{SHAAk=5Z&EYa$5YY073{g z&3R!vl7;Yc(_N&hh^!AB@Lr`5Sr-FchkRFl5|G%=TIDW3<piLxCQ#X?ySb%D=Zvd; z&Tw1w@R1+@PpodqTL?a?a4n`^dg~iG^DFx<PvnfB?z{bBhSi4!Fg-7TbelpNMj;$2 zUXXSFBRLhAb$*2CvNG8wgWI5J096CRiF%3S!p93dX34OGO+nH!>924r*?Qcc1{*^c z-l_yaGsb-r8_c32ee>qct`fVno$V!g5|9*J!$~d($%sDt`1+#kMdLjF!FJAiXQr^f z6@c%S=x2!fh$yjKyLOGhV+|oe1ux>ZT)9h<hxE}ZLF*O|G6^ACu*043gQGGXa*2j{ zSDpfpSV|h2%pai~rfajkn?r9okh(b1h}T8RB9J6R@IW$30|5hxNOP6Dneb?`kvy_M z!ju8>aXHREmW|?5Kvx5BjKGrH5`h<+_yv?VEnwAZ=tkfvesp(h(z-1NK!SO1_@k(l zBsAd!3=*kS<(rdf0?y0Hvz5E<8S1&>yQDb}ZmR*YEh@4Uq#J&DamW<N0L-OjL?@ZU zR_=Nd=v>N4a?}8nWne!ek%3*OEIF%UQyLNWt<{;9p0DqDEp;GY$pbA{eemENQtTjY zjzdrmG4Rs~g$)A%qaD&u0*Fj(>N&r4W%(QNqLqkp!9Ga=f(8V6$-G0+xVI!9h;brB zJ=I2`r3T?O7v4ViEpx<t4jeq*gv1rt^-S_HLIU>FVIMz!%=6gYu`N`x^@1I77w^8A zLf?68a$&euhSSJA-fwXX5-RLFrF-{YBH;(Pr!lPC&FS1mF1@dRjSwO`c481KFl>y; zfNhJO9u2?XTv^;dWXHOwN`e}~1%YU)ygkdNoTVcyyg441s#^&W)C)~*tsk|_+Oc|= zrq0U0ggkc2Dt0za&3lRzfVJiKi@!Gk2^J`PrdygIrtaBwbwL_T&TG&W367f+Nxnc8 zE`U-X4p93YY>o?iBt_LoN=m8*c>TMkriWFvwY7(j9JvX3bkxdMFHF-Qp)eXpPX}D! z!we*{2@4N<p#O0Ip2Pt>A>t7SAyFLW<{d}H`q#k{0Eibs{^2YuFSjfm>4N|Def|1y z97JQJmz|`fG-yp%Z&}}7YHT4?IIoz*L0tEOm9GG)la7Ujh2T<CTYC|5@~!Vd%=+t_ zn-?~g#(07M>pzGQ>I5tkQI(%-NZ|ZhRpon$|JgYRjhqmx-vL0V1|{+(DdP8U1AS0| z#RtE>+It5CSWb7+idx*gH>b$E)6aY9P5;WSN><NJL{yKGL#M|Pc)mq3p{v(DW*G?X zML<ZL0AQZ1udkn5>vh22#1iSfGX^YF3oUwW3x^S&&?J#3FoLQOChStsT;b+|E{+~! zaBj{LaUmo?284wfSMu1&qyt>PfSw2f-`*W%XcpT45+K|-t`Ew;IIIs}AWVDDi2}ti zEIJ@rz{&D;J93TWfhS(>HRH9+4VfJH@?!6?Ktn-Pta!%Aw7Vdq8KhCon94}=_MM&n zo{T;H^OljHLZ!PK9TQXC{KCRbKm>Zrv%OoH1&?KV4QEvG6BfI2migx4^l^GsUf4`L zw`qt1C>>QD9UV-N;%@>{w)GJ8rxmrZo&ME)`1o-yB=_cFTWZ!xYyB+*tA(Na`Y?{- z`6g}e(A}eT1)H@wTVFyS@U=t36bvqshY|6M*rn2f)HJ`c!adp$Zc~dV16iM^s?PJN z+&&CVmO(uZYAHExCMJfVZ6b}8`5<yaynY12$QqZ%oC(_bapJyc>v6F*(%D1^*roP4 zPqD!+t$S*E6XTg?O#zjHx@U;TAxS5rurEj8D5n^dWBQ!NsD01f{AcKrSV)a96w@xt zNNbOihvWVI<x8gSVZ>CcAnK!kprF=!|C(G>@*x^WF<4fLpdfY?T4sUP$*0wltU%Xk z#^YU^+bA++B|CXE8Bz`sWgzI9_2Uj>(jmS!U$?8YSKSiYRfW4XI47Q1w0jBT+2F%G z%bsIBEPnsqJ*4S1K}O~VU4a|IsLR@H{+z^hDS1&+UX35Msp#t3TA`@CoSf=ZdxlE# z!z<>UIrkamV-Q{aJv<!Oq3h9`fZr<?zR!MC-Fp>Qfdbqok3~;1sBMy*5{y6}Q&W_Z z61O5OD>lR-dN+e~p#16aZ7Jfr^S8JR>i&-Aw-JZw%JiW?k6ripU92WO505G?2*RrO zDr~nYct|D0jq2$9je#PpDlEg2WdRjNCrU;UBGWREOf1N^@Ld`a?x#+l4u+*JDhCN0 zJ{`=VD*{7h<jRFEAMUAl&d1+h4Uq2dk?P*YSs3O0PzbodXB9iyo{<J6iSF)FyQxG_ zf|4Zzu6BW_?{)nAM;P^S7~3Fv;5pwv@P!(Pia>Rt#0{k+{hD{yL`GB~ecuUUt`?;C zT!Dl4o?E<uP=nM_B!AxBy45J8??H3+Y$A~1G?0h@Z@dS{>n87pN<jeu0jtBn_0DkU zXi4~8g4JI<JF{1Pty>-%9zJv7g0%L4I8&2$IxIU4zzY?-J6nS8GAesQ1Y|Ci5?kyU z7G=rqLQ97tNYCOp<>ZGSZ2PMk8|B^I-LYLz>`D9l`LoN$VKfjq7zmb9L+xt3Uwsd< zKRM@xpf6;06X45;Bo3Q?jEiHCf>;qk4~mCosl!}5Xf`t)>Dr~Lz+6%vWq(w9ws(VC zFHYK0!Lv3&%2@8--BE+pn$v%ahLSQKLLw+bcQrmeey)*kk^o^B3;gZ^m+^b}6U(BK zDv1NlI1|cnll6@FR!z(D1i+-&Z?8}KynZd|5OfLM>U*b8gow92CS7$v03p$Gpg{)t z?<ESO+91#a6`ySS@Iw6ExxpJ}NWh8r#7h`erpuSbgk4r!FItv;FzcM1Ee!!}2V@jx z9-a{9S-7<4p*S8ZQOa=|uie9F0wr4y&c1f}KgI4pMu7h<75q9|Kbqjx>1+WqG}i{{ zW~zrUJ8#zd0z=q?&3IC@__0vEK8+?vtR_~>TeG`FB~^Q?W?_^-EU_D1Vo^!2-!n4& z1m8g+k7)>(N5%4Jq@tLoa;ztpeLDX?F}zuA6X@EPhpi|Cdb)aw6-(_E(^QE$Pl-Xy zirG&TXM0AK9nd@KSj<3GRQ2|p`c_@d%)}&_ot=I3`0+e}pW3C*?t=ibImLmURA%P; zHF*!-E;bsXZu%nq9rLV}X{~?cv8s(t>8!G=)PmRSW?1az*jFa>?e>Cs9p3|MZ_O#< z{iDZjCFQS8Gr#hdG3}5Z`ze*7sTsXD*qr2a<5s5D*2F-0={IkHf*D2@7V<~*Wyj@r zqFcL6rx06}VK;F#n1drBUNRuSZ%jHk=h|uXJH7*Nzd4096#aeAwT$0ncFxl76#B$e z-MjqTs<py!-O3%7k?;4d(KlV5ii1&1Q@mfY>ps!zkgznCA-A`Uf8K=wR87GZ?{1{3 zC@u_*o;B@vH??Cy-%#gBInK@r7+rY&^8tvFp@=jA&VV|@=fQ|HwJa8>6@mJo1m$8> z6kxt~t!6L+gxM6(2Nu`X)F3J7xNkhwhGJ+iZFPdHYkVzDplW8&>P#SuTFB-~pma!3 zjbFaNo`=NX(otKg&LZo$t5>fsLyZu@NNH(lMTlT_ATVxm10sUi`$2i;OafF6CxM7M zFE>lfj0J*fQetNc=u;M|G|d32mqEKX1$|r9l|J;?kpS^`@5!lgBHIr)yVowF2Ya8A z?NfRm?mBK@;4wn3CoXV31T_iRLo6_w<l$Y*`Erk4cSwGST!JK#zf9omPX@><0PybU z+3QSF5jXX{P>uj)6I97HKaycV@CDO0ocnh3Bqe-@SbJcn$pQJ#BI9S@0I#A4*sX@k z)2}6GU|>Mtf7SGRC`T4(BHhRSmlB)b+qZ%A@L5P(^~`uSI`c02r~ibZFG>tNKe<o- z?sJRhptm5^2}WN9s4#;RlooMWSy`ZC&RCfPokUfmEpA?_v6L~T#ss6GZ*MN#GJFt$ zO00FFfz#UdE2)L9u8O~B%$Y>`B)lmOr5HEMe8=ymwl%{Sc|Ksha^=h9;q{A0ZI9gp z905R}4UkTBU&klCD$4Wcv7&I3N&vthy2mglX1i{t$euje_Z_dhIbUTUycSjHbC%oo z_iNdfM20jQquD07C}#Hl<g`W+1<d_ae`O_-QGf>Opln)51&Iw3M+@*gYVl_MCr==^ zUD@JBxz4e&Ri#fUYeNuh9Z)$<z^;Kw#NWTMlEf#+YnPhmQAv4`mD~8eY`;5eTe4ou zmls5R|EEf8v#N|JZg3_wJfv{Epla=Kvg$@_rH2Qbc8M~e9VKJq<f;v*BW<j&kC)C> z<QAfu8W0{OSd0u(pt!e#ux7WpjNRe}!Bz!CMyN@TM^Fqs>FZN`b4oFo1>%aL?*&7b za84XmWjT7noWI4JwCQ2196Nix$zyj*7v|=Nd!ysShwq^{@F6z#^2$Vu1V8~KC+t6P zU<E||Wc!&es2zpSrvNubSvSUYQL22UT@wgh5Tv{hU_P*wsL$Qc!GUe0nfOXJvG{bt zn>N10j-ase+aRswPw3Q(<fh8*G@B@Bj22V_4G~8VHUSkRwVh?xO8)lk9;#M=cI^;W zu>)%Gc=1?F3(zh~P#r;7nDpiCdstxRu%<0z*e$I$B0W*hNVwzHd=<OxIPqI*>84F$ zYkpeW5%@082IkzhgE4@ssti_l4+jP&j2ko4-#bcPAaDYP;Jwq+)vn8|L+m;REHF@x zMe0$ZY^HV>(D2mpfnZfwVA_$<SY2DnzK1LME|LlKXtbp&*5eeNd7nfHj!o_Xxd9<c zh-(bel99YJ_kH4psZI@T3ZaV(Tt^paFUYW6x7r-jD6}^}i+*f%P1SnsMtUm(KVlzw ztR&w3FL>L!)4wQMP76)p+`UwP%;DV8g$8tP{`0>vJ)*DtAK2}R4oUb+yR1Hn-VbjP zQ^gS`F=B>GeJ(^Ob`!1zvogx!E$ptM&#Uhv6R<j@Sty-v+Rq1-VvWL|t~v=m+~Me# z2EsHdvAW=*0G<zaoXBOEekRb;$c|Dz1Ul9F@&2*46s6k9*H(9P4W*XGWGpL~$^Mcg zJ1VJP8$mbjw4||@!NgPtPp-w{`K<3i{appvF1P!(Bk~`CFsKrEO%|DqGIT3-KuuGE zJ)Q5k@FZ>cE1@BpHQ$sm--Llp^O|h<)ibQuMt+*h*`c}jd^d2CZXBc+CBXLi5HD5W z61}E?j6Wg)27qE4C>`7&3{fS$6f|vrkg6sRIa;IWVT6DRT5o62Mbf{~9bE=#h6T#e zh2_@(TmPr{F$HjvUD~T*VBOkAk55;3B3Lb6zcxNoJ7KmrLBuUx4&B7%wE5S}vnkb1 zK;B}UCz-)!^*b~35Pgaa74!9msP?|xBuX0g3~knELB@Ow^FR4s{gnjec#5m5Yo0;< zsWkn|fp6Xh#<(g@_G$_^5zUrXc_do;T^hU0;QE*xa(|X4h7=eyP^Yre{hmLFm!V2= zb+Vx>*A=a`Nsv|#JOXbMYafQ;?Vm<&w_n-%E8s?85)#tdw{IW$*rDU+9xHGbr3KI> zJU>8Cu<`InYeAQ;OZ@|9tP+}<nlxi_!CU9L{^?jtk&c#D6ci&UFJ1&jAeJ^f?i+2S zO>As3+1PEHTASlt6QUsJ!SBw$P%h3<&NVzph9ad_L$ApJ$AQC}(Pv*D*s&neV;2qT z8|9rzcGz{R6je6t^dn|YMdyY4X{XVGtUD@{DJE|>)5VRPX}~O03(RP?^gnB_uj|{i zZW9@nbe45Jg74f-$Tdn3_x<BAH8}3CNzV?8&uw!#pt7<O7y?X*pA=5VageNhq2}V* zq!)sVOM#2O1Mb|3-`*e$X>sDW)*nTp&?448Xic^W31}IZnlUIG$S71^4jC3n`>*N; zGPk$E8W-Ov5@^Q5ywYJna<&=dgAZcr4%6@!Zb8YA2w51`D;p3-7Z=ctkh$bo;1})k zt<2?yTe#3r;5X8iWmi_FnMWF!-FAxqAWMTYG-|PU;IUu6e8F~c=vGWW{B$4%czajz zQq98GSHxnQ(L4tZ|Hv0;WIn44WtKrf(hK}LdZKb{-^Uul<A7*to;eoYK&AUI3c<0u zM>M?upvv(S82Q@!%3bV8n|4SK1&-bp!ls!A3_fl6%LzA<T~|SKT*B8^gfa8(5*|dN zJ=5!lk1m6>$SfeBVOh4|1&S0hX{u!DXd@5<5^&qXfLB*xs7XhNtD9wBb&a%AV%Xan zFo^yc4YgPqMJTXvgB24k-}U*s*q@gsq8;K*+8PR)9@?Dw1DBa-9H=s&$Sj8!iB2TJ zc3L!SydEFtV2S;*AMK`W{e<5JHox~+94D^!%($0ce0NM(7*l`b*o_0c%nElja>b7x zem~g}dg^qB99m_U(-y**XXh%mGU;vFR-u;51e(IC&A&x(6_Dl9hOV6c)EOVDgVQJo z;xW(JODmq~HTp-F!6*ZoLz7WM*x-%m-i{762APmnkQvf_ydT6XGAvz@diq1Ul4NL3 z1K)&Ee^E<04)Gv>9;rLJ?i&eU&(ycRYH+sHzIRj0wQTAcdRtwWP@<T8{*I-LB_ksv z(yzhQ^4EPqqDp?Ma!!0u-y+%A&*FdY;`{K~Hy1%m5ZRu;D`jZi7km2v8g*c(U7z$` z<PQ#nNC&;=u?+FMB(=r#sA+P~MV)ihpS3CO<44<F(wYnoCN~ijO_SXn5+cevZYwHD zD_bCQO+j^50fY;K+|K1THC_Iwxwf=WGoHw*IMCtAJEEXg2#!HDKq9DIE#kI04z&Zh zAIFut3y6bN6L{0!u6%X$L<ZVuxe7YSy*hdZN1p-4QcyJ%K_**iRg%ZT1IkUUfv2Yg z6owTj+Vw9+z&%ZZ-)FTJv<vE=;F8itK37<?`K}w!+~BsIN5hSxLe7KPDBgL6C|X<c z5BefbpW5CRIw1DU^p>iDl+X@9X8Fq@T1gz}>X%gQ8;zmT|B1$ZeXv_+cx=pQcX412 z%mqoX1t$O=;lY`5Mamg;;nq|YR@oRK_4_`Q_45tr(wL5gqW$G?CzCzJxkstkuS2OB zl9x2Dz`WaVLuC9SkJ&>Y9WbMDs2GR5eig18^ipX3&-N70R2S28gSkRSM+Y2Mkw}Y~ zbph3c<>t+1_e)e%j8cejK+HcaO6XmXE}q4KNiP@e-r1KPbfvT~VY)6e+$xz`>fmHm z`{M^@lv-R}Z3n%B?MEAU-;h`iQc4?mR=Gf66W*S$Lgf~aq7nKgJ9J2}>>AJ%g!SZL z0)DI^>s!sc^MQ$OkR*XMHO@CK$Oqdfh_de5DoDV0i(LzN|NgPvU)#R0mvF6M-Q_X| zeIxI3{RyPFi&7jrrUZo@WsrivWpW$5HN|6V)I4TkSLx_xYQe{I^vDqfU{PTDUeDq* zddXa>l`G#F%d(+G^iR7a%%WX!T_x3Ek3AZwrsSFqiU{S1Rl08EAVCk}@C3{_`53t4 z9xJd&AOrc%_J-a3z-?|U@gA%tA_zp(nZ@81MP|PrTkTLATy0g~&%3k-$>qJqc9Im! zw3b-M$?@EAWot2cwkZa=QX}7`T9P!Q5i$!`P~GHxri$V?R1G2Zvog@rkHcC-gY+{- znAO9>0|u^C_W2H26k+OZz^eL*J_Rr`s(aZ_|M~!sT?&+esUR?qL1|s%Ja>zW{+N2w z+RZ$T9T)eR_6)_DE?z%g%aHbrTX?mUMUEE;AzNLfDI|2o-hP2p#&KBYV2Jc~-(HX) z0fRhBLD2!rKe~Mm@)#vGb;kI(u@n)^zvwj@7uaaX*S!K{b#lBpZgaMDt~FCzFqm0M z9Q<?d=@ETI^<XIBW<X(X3j8h)VSj^hM@`kLambK>+g#&K%9BxizMuFjn^<J6*jLtW zh=TEZYe=L1etzvT9Od-jsCRO4VZC`%0RoCM=*nnaL2!V}rhvYNtUC0}oScd<aG(&! z{e~hltcw&V8G~yveRFvt4n_ww*m^zZO|UDWXo{u|`AUHjpqhQsQ!ke=K{b;OIViqr z6g-IFyw9Tg;Cpm;)a0IcU^-vKJPRfJ#YN7_(8C}&Bm4M=JD{nNO{^~r%S{!6yPxy; zaLAZimX2HPS{=;W+3Szq2V}pO;xfI~md5CNfq_knNj92=WUk-=4~Pkmg2jyPcz0(_ z6bzJZCg8ai_q_nPpm7qO3=!B2zeDlX+eO7UqO!7<zq1mv^%XlZ71w4J@gk~8d&8t4 zWZ|<ll~rk>>u3(HePln4ahc@@aoKBP$Vx`LL{d+0G;cLVIAPujJEJkt!=GYj`iC6V zG11)oXJ@5D7p2+|e}8`nJ9?HB@Nf{Wu0J@~qplUT{5w$b$ww}m#Lm$fbH7A~X=8I9 zbo1wH>BO4^ZO3DwNG^?ot=%%sNM4l7t{uNPLQq(CCtYd`W_c=mZSTE}fJDKQG!6he zi2l6Sl}Li2_0@q{-%HF;h*Y%ATb|BUO+t%t;5yCCYEq{WUZvE22Esl3i^N<_oye%D z>8}Q#d+UQ`z~S8OKKM@%qyN+rx_X@>jY4cCv`|cT<)?wEB^d&t2}HW@(Sn6-*`B3~ zE%IP6CcZg6sxuBap6qDYSOiyq`%Y(<EBZL67eH2=GOP#?%yl6DAY(RIqb=>|2QrUe zpeK!qDrRVm?u>z~4@~+D8_RjGf58u>KXa3kFO6{5<l)bv&nAezGvIp&{2m-g<6L*0 zs17w1xH|z@#)mi~U;eeUW0SS3OI&!vNKZ>^<LvU};~#;`;FABNYU;X%$@XB?Oj|P$ z%PxrO09=lEuy=bjWC8=DBAb#XE!WIl#(om>O*0$O#DC^zYO=)bMMd;JPwz!Y3!tt8 za;hZeS@u_GLo%^*yF5S+!p{ZX5b0ptRl0;UHOr_p?d}q-y=#ZS0))z-8idX~pe0hs zb)f^%9Tjz9Z{+Y%zi(e^l%W=sR!|^z=8g_oEi8-{N50+bn+lJt%^0e9Qhl@>(k1Fk z*ptAaBH0Eu@(<vjK{ajF+5<#BHw~ZHH}D>dZ<aTw3<E0c*}Buw(5Syb!JP8s$q&d{ z09BGiR00O|IvKKpYVCfa-H6Zd!u%lUH^Y)hPV{VO98N6KMRsW6)iC5bAV8A_vOz&; z=iY1f**B5;@U}+&_iXd|$0?e;OFMfw0|1Y2tQw>&F!_NqKk3fhyDP}xfNXK#s>+9l z$as|fb@H|6i~Dbhmj3U!*6tz>^q$+X=kO*}VbYKw10P;309)la6b75Qd4jzzE-u}o zoxt||`hza`{NY=`7RK4T`lQ+i3vvm&+<~6XOzlDuqiW&tsSX*Avdawgl5yfv<<7+- zd%O3Hlh*SX9PTPdR^I7Dkam|Lx~DoW44sYNZ;@f5l42lLe}fn!=<sucR9ho``!=s% zl6+`^`7ONY`sKYQtM>{*hCVJpYfv5E(-VWd{!sN&eD*Adv<=qXONr$NW<TCLf?YD4 zs0+yR|A#EP-ve+Fm2MB;>Pj$d1K0I+zDg93J}WsscrF6YY~+d9gz@2qQaS2TL8i7U zw=r?&yFs4ST?N79v~s&zQ|&+JJO~$<1vJ3@EMWb$$J=ey4BN%EBoCLuoUhVPEb>@s zQ$sq~1#T01Dez;IgH1F;g2w3{SOW6t>kV_&!~T}+&%+jq@alUlZLKW@*7%3o&xJUz z1_{`{7uoo>K|;VIrbb`9sS%V)C{6wHhn|ouWD@{GG;#|8p1S_vP*GM8C3Zxb`C3hn zzc90uNvC30W&flBd|0e~=qBAb$Y&kUJ926ERwHP*U>1o3(Jt+-H~HDb13hYEKS!@x zA2~t{)s1-jE#H#k#3f_)$GpyY)%#$XdS3quUk}cROlaXs#5RWCN<ywsXu#5Zf7Ht3 z5QQ|YI}-$U^Ufy)!Aw`!v{{1LO!s2sW!8nTlgr>H9S5CuBIXDBW^g`$A+EDJ1+zwV zfn{aJ)Gt<aJ$AUbmbP$D(tdx&>A~|DMtK@!5Dgi#8Tp<7VpPeElkfTO-Iq#Ji_}cz zHZ|N+jV@0uhV7@eQu{BR#zQ{qM+n=Dy#*D!WZ_1|F?jmV!t-HJ)#-^GH@3N1G})FQ z?9%Y0+*l#S(Hh&q{wE8lA0iX*><0Qek>3%&??Ak21}Fi9t>%6=wF^SO_JF!r`|aoC z0|i4v4d!j2pdG*KyeO4h?&dPFj6JpoBGiwgYjD21*m#k*Vuc^R8MEsraduo!Pfq~U zkccr1k>yd-rTd}rPD>Vfcp>JejTU>k;WF!Dok|Z>Ra4{R<GC95#pgTQZO-iMjGo=o z2p~A09mnhxS9vGs*EWn*zm-Xlmb!hr-4LydoKw(?S(xvTRV>!CFl?yD9{$Q%@->%3 zrg91{VFBLYdHuzH>d)pYvur2MUxkRFzeS}RHdbI+c#R5v`@YTb_f>1mY+v<T>{}bD z$zKXSzIINCvR(f1QS6YF=Ejl?-kl{OzQ<Tb>ZY1)V`$@E*c4NRdci=!C?QZ}=h62_ zwbX}STz?Z>;*f$bRx@Ac(geK%Sx^9scOt)>!TB_Pt|AcZ+EhM)4P-+{DL?W2`{f4D zE1-MH7Jr~9e?J-V?8qO6CW{yvaKSpb^WN_KsrIp<VeQaiLr}azIdZF!cAs+=J@+3H zW^fJ*x+X}2M3*@!nNVq-Naf5ZF1M1YTku8<Dh&-3{2^XGlu|D2^suEYds*5t-G&{P z46(feOHRQoKj&!Sy;f&bt>K}~yQRG;k~}i)OvlzlO+#8MEBO<&Ip<!`3aVb<{i(Hg zdvz=zTJ*H%VF5f%;3IEVZIBxO$7&Xkyr`iVXn{mURxoAZ;=wkN46@Jn5CWaYCC_ft z_J(cT<5ni@;GKVKOv-RSVl|6JKDHo$nSiwd5<&tDmJ|-ziq9}T6oZAX1n`JW?(l%& z|EvOadDyXsat*dVH1|nwg4K2f{!BrTZ3hJn`lHSWFE!kPBsI_b-vcEZGAFg314vaU z@`1Q6At5n^e2FnaPSfUKgKK_y<n;3A0~C+Kj*B@D4aGC!gHD}J2){+(v9z?|D%Y1V z%|$uxeEqO^6Wr{c0dXct9<LhC`SiNvUjz0fwgKNmc_o^+R&0dSQiJ|Tl1B4ICzrt| zU;^zrs1+3~ENoIlDC{N!+?<5^i!wxdQWsRSk58?EE3*SEC#awjE##z#+)Cifni)G$ z)dWx+Nf@X_3@RaQhiF}6w`NNl-3j0eK-#)7aNfnmMJNaeqc$04kQ&a1oQ@g$FF9;( z!j4Wt`oN)xxY3=R^N~o1vtk7DT!?KE$F)Lp8wf=}^RG)G*D&dqVJ<U2KY!I`IZ!5K zG7|<ujVe<uQ62!+mw^?SfF%!cLNG72fv}2rF(8jndOM)hV612bf)yMT6c2jORJc)0 zGgQ<$uQLw#?QH(&S?cN6?W)w>96Xfr>}iUjY+B&n$ku|?0~QWV;v8yF0%F0}YvZM7 z3>KveH=n$O>oH1gf)5f|F5DmyA$0ivMX{|rjrM7Q$>@JB@SU%MZ-%I>L4Cy+jMC$+ zDN10fPbp`Ex+ya8gP#oX8*l-uK*}hf8pE~bVU?VmoUmYXhI;+olDX)xFjRWjvYw>e zfi^~{Y4J~-?iJ1onqr?Av&XlcIiEheYmfet8~sU0>@CfjA~G;vzaD8nTb9}MaAH~= z>St!oJ5C+H0jJcN^n9T4qyuVR2~ey%#<D5)`CD9%6_9NyMW}gEcR|_3Sk&QP@*UL} z!WOwWN`Njd;nR$At>8H$=4F8`aL2;xt|dR!O75b(>8(e-yEB~dS2X=K9(bgS3{v~{ z5dc?e%yb*Yls-%Npqkkeb{iEwgk<jA`KRsVJTI&dUXxZPS<T7vaUADzg^k0FKmA}` z{=-8<l&4O8gsf{ce${c!<a?$K`-IWMVAl}CPbItEPvI}YaK2_uMN{pf%NtuRQ^bV! z$)3(8uM<2Y^}`9^Z+U9=5EdbY_0s-MD+>4~fPr5I`Iwod1RZ8uVO1fpBEyM{AUMj? zr{jRgc6XzCS-y$N7B_4>dq-r!wcFWt+L~$s&4p1^w32HEF}j6?de7xuIiFBdcQVU0 zb?tyJlRK87@YmdLC=<Jr%6^Y9kk%@R7cC8x@FI#oqlja884(ehLQQ%IQ(jiy&KN%c z`4mTwO1A*CxV5pl$pTyyT8XTj62Z)e?g<(gqRTJ)zkS;R>;&0%q;P<$GJrgpKxqwV zb|^hqd*q<9nOd*dUiX#8^<h_8rxrcCCeMWEZUBu-gVq;e`fq=-agE;i#4ZAQ;I6gm z+R`ntm1=Ho_VM!zF<?JV^@o`Qy21{8ettpJeb1XX(^sx3?7WP6CZ=5y)c$9zTq0z( zo)H8NMdRc=i)hNT*UDEylrtjo9JBXmkLSB#Yr1iuXV3U}j^9OF7fdF4wB`0*;KJPN zXQfT*qv`|0oEvnUIFAIY*+S_Wg;^w2Ku;7BxjNUxLgOJ8rGcl%1WfhNak326b12LA z+i4hnrhX7(HtR-%%9oejwlb9Cylm%V`Eh*-Z~s)<-hocy49xi{FvN3ScU@3%x><sj zKcgSf=ZPglo=90xC)%K1VCf_akthH0m;FZH4)lZ+7Ak*yxgq3Wk;@kv&-s*|V-y1` z`0_E()JU+1$ZJ6tgBBFeDoJk4+TaMoOeY;INRWp}P!lN3<WG&M=Fx_A=;^%#bnD*a z!%LnI^m_PgOiDvT^Na(yp1ibOjNb_3{_*cjExg`O!uKQzKmLqiytZ#%9_l^OrP#Dp zao>1=WMpdtkJA+Qq8XydpyKjfLTdVvBS#Q>rJn#*{*Q^^24aC_<q5m}*uUZH>lB5a zyh(uI^NSYs_rp-+TlR~frU}$p2<c9_+}REs0zW5{F`Uo8-D4K%+pEaenFxIptL%g) zUFma~OM%{!)dI+?SnznFp!upEV{j1fVc|C*I`0W&ZI6_TsSbfE1)UKf#IJD8fFSMj z=1unHlNg89LSI_SGYm6bPcyGKW_qLhl0-28loix&4E|a(%kFG_Z=^?n=Ggkx0Mnyz z?<JR0P7v-1&DeJax7Bd<s6&7KsbZ+V1v<EAU_QeV50s(qkAKT~{>vd1h~b3rTeXvp zn4X63!Kl8xG&(xD5XdNVh$5nqian1H;kFz~VA3Q*sNXe#RbvHA>sZj5aG*&1NQ2(b zvy2CXIxY$}3Y7bQXj|-GGTmAY?#c@|cmn4`6<YHyAjhcjvlW_Kp-*D+DsVyrbW(!8 z#j45UPr8XIda>>}(IU^FtBal0!mgO4+&_k0HlFz4y;m-SY8y~fMtb^WdAtakGE_r> z<{>HKF^;lW-af~wAzMt3tT*{yMFgoa_A~KMf5bD$#Q$4xLsvMu3@*?KL@yz|1d$T? zQ=#yII@Z`89Q;fjEy4zhiEI=TX*II*>^Y|LU6DNs$Qe>gnAw3E8shM<6l9=3u}>#O zkpcNK_r^S0K(QqeWE>~j&Hl!{<yH;f1-1?7JL8kx0<KPd)E@Ib{`{>5;YK%C`gR;t zBs_090d#_iK(_tW;nt6?=!^tt6Hv*<6lF^X$G`k*A}uR^<VQl4_g)FZGrbPL4sbiW zJ;k@wAq%>*GXY+pgEhKxp1uV!)!0r5V`13Ap5o##@#`jZ-KRXcKao`#e@!-?LX>Mx z0<mt+e*kZFfCK?StpqgHe~UOEf0}|n0b*sL01bwBX*!n(f_io~RDBhPYltaIip9@h zv>)viWkFl;{lAxTFm->OIt2=S1JsJ7hik%JKKj!(3L|&aUcHH33QlMszCKCUnxV-X zE#jtuLuMCaK%cN^1}wm}L)s#YiS96w{nf)aHe)V(tzJt-M5M1|2S)S#0!06QaP6QQ zHSgvDr~5rHzvJi;kG=rs9#l_YiXyFt8%{1SjwDFX1k}8{*)1Q9o9rmO)&i+yaFBR_ zLJ|}_S#;Kpfe_?zxSLb}sHUJHEUv8N7m7ls#U|*nd+Q+IjG0=9iGb&Tb{6(i(0OTa zj+m~VS#S0G^{};aP_bigj4<&Zvdnt88fFaQlp9nw5blUr1+Ne49Qo%SrbpeM2DGHn z*DLa(;bfalABd#2$nMPIJO7XU*)KGLI0?u-XT|}~w4)xWyt$rYC1{7v-wOjjI25fu zZMZg^1AP&@We+gtVoPtdIq-1@ixejtW`VBj?G!Xzcn+=ch~sc6D-nOq>T)oTT;Mc# z4PJgw`H*N0XPexHh5=PzgXw;UJ`*514LjZWM>#)rLx@XRTFzeEufh!)3FB|SfaX&I z$Dte}2W@CyMc2D{8=BlYkgogc<x5olL52}zwFI+>F}NxL;tv>tAg$x70NK){tF?@( z_eMPQg)WXc$a<|ig4Yha@eFo$iX8Q=juN}f3B-vqA<vY6dkA$2fLMS9qZeu+00XqU z8yNW)3!XL7V=N|S3K=*!Qj8U_%~z2<#H}ewHc(`{iTgaL7(b)`7~cSnLZP2c{p!lZ z@P7s&awrCAJy4qnpa(qtwgqFQR#5@oJmSkEqb#!>S&>Q&OCvVgUM)!p0d$0bt;se* z!Dq1v(pmzWrdB8tkbaIpGR7@~ewzw$9Mydxie*CdGlk6x8rne3HYwrLjT8;;j<aJp z{)BR@Qp)WEOCrmac=)(j`v80#<RT(!e4x^CybboA5PF1Mk<$#7S?s4Dy*<6(Gw$Ee z5TEn(k892dWy}{8B26HKtUy0ccQ<O-hqCqfD=Hm0014`SAwk>Q4P#@CGJ+1$z}>2S zibZ~GuD})zOY8E#z_2novWZrMdO)qLwLBD^PzP3?x3~8coZfMke!pk;*6-6c)$34o z9a;SzoS;|Lz}A13!`^*2JKL|eB`I?2^IRXBY{c(vv<Zb?!SaN5tay+GUvSMpYXYd` zy=;<yVG4Pja=;9g_vF>`cpR}qSEjpA26l4!PRY`iWw$Ey<Xo(UQEwEDq@bl8TLZ>7 zzzmvtDcW05J4OK&hqg8J<2*nR%D^C=RN~O*RGa!5^LT8AEtpA9q%m4kI{Gi=#COGH zQMnoFbiNmj{MzLs4c(b<Jd5YA24(C&7tR#h1;Jf~w7bTPYH#$+a8Si}yeM=6t*C9; znUx=O^y9UUv6B+Ln>JFJ)Zye5{`T8{hd`pDzFcV;9AosQwUooob++_`k^<$U`Z*wE zC&8?^LGD$Rz<7EX04{mU4els#D}6gW@$JY&D>JtJ`q&yCcNOrooVjkkH1wFohp;82 zC?zb8ME8rLOU%Q>&J7I>LG!%;lEg<gLa$+XTT_E=XM22y_khcutqCR~T2SWp?a`oB zj!G`7H>UkM7x<dAk}Z~v$BbZJ=l`N9l+=Rm$`$<Xwgk?0<og+PVhWOLp$;E%UHyr* z07jCPWwK@oj1Z6xy=gd)jp^4$_3wgJ&6`&G7?reUL6|vMon%6DI0Rw2ii5^Y0@|wv zkH`RW2YayIIfN?v+}5;xX+`@W$G&g>_J%ix;-aBNuKj>#amnUe4uQ(uriUq(x}XP7 zd^tqjD7;g&v;~P~o{|RI>%nmkI(0BTU`S~sE5lJw$V!RMnmC(uz%%doaLv+8H(`bV z=&Mc=+b1-(Mtg=KL&w}A1$tPQwqP60D+B8Be)-ZE(i5uJp+kotI7;E5p$?tYq!~qy zA(l9|Y2EI<K_5xD{!^*=N!1&(&iM8W$erNiBoqu)CnzRF_4C36FrqmG=sww3UJA_~ zDaeoywIy6b${fgP?Oq2wZMBM@^f|-`;Gh<WY`kXarmMI6OVuXZeoXVye=~zq6<kI3 zWU!@M+uBb)+Rj7-v4&1<L3*(|3)uw}NGIt0Q$nUb*eJhOc6UqIR8lF|Vh>o!NcboA z&koZSKQaH&SDtX?F8Eb#%yx&4E%mI<{A{$!Z4?cy@e9t>4Rv0A??vHYjsoz{A?vr_ zq!b+;i(pO1LK{j~H<Egx7^x9<6ti>Q!^37DuTiPvO>=8@DQdxpxb>g*f09+6X=t92 zf8OKgdoyBn#{W8a&>c8D+yQDO1O;6*XJ#JV58hY)%H7w8v5UVilKwB)$XB_mDfJ*| zXgFgEg0+D#9MR_49l<NVxvtM_sgrVt49rB#@-Y`@eil4S2ofx8Ei0y^Ha|P{2Qu2a z?@nxN+b|1hIS=fcSn0?>=`x4Lzw{6_X1+0qnGP76?dUt(?V?WAvLgPRSr)e~Y4a}0 z7YD=L{Lj-tZUXXu$&yhr+R*YGBu^V!ApjHSkh{);mF4?(uGqd@J{o>S7Htiao?lvo z-p--ngo&1fkskz=;i=XDNJi}$Htzi?J=PJzO4Iu=b<RTef5F0@ir}|N<T3jd3k_7z z-m3;jKuy0Q|2)$7%c-4#bUee}@^0U0M(JpQ@J8<wfnUEO;PkeIN;fEko$&en#CX$f zlV1-@T-7H#l{CtJZ9jYb>jrkm_14?m3@v6txV~I2UZ~UcL9~fs&XWjX*3JV^<!~~4 zE&>dN1c!4V)dmSMsIwJLCks5QVRx4fQ<Tz5>R_;u$;W$y?Cpak5nCfcx2{Y1yvI|V z?Q+>}Japc5!~Qr6!R7mR?COK;LzvIEpqx-)JE3Sd-t@)23_>X~lJ@oW;ZQpsI+;tg z`T(ZTZY4F(rBMBNmzkk;%y&Z@_g%Y5N!iE<rIPMEDVLvThK32j({64A>?)|=Ez=VI zAy8f0AW7T{Bi$31*3scj{MBvNo@oGa8-Q@niya_>$W(@0%JeDV1#({PGy*#x3zVTB zWR)%Y&^Qk4d#kkHa|<kbCLKRrztu>4>|hIOYa*&Y<`TlL$VM8<J*DM^ic3P$-9u0> zwOLt@5hmcP!8VU$fWqs~xqfQ390(xi8=*1~3a%9>FQma1bsl_0g=)z_yyU>Bmr9~p zhSOOHlVO+vdp{c4%HG=A>kqX*C?+g-H%8kKbf#x-5c(#AVeUPboEi?7(4!7tu=aL< z!4`>R!VBL{cL9Hbh7@E+NfUd%Usd8S&j(fg{R<VwtjcMl5$=jfoxH}630y7?kLq{k z9^Iz__uv>*JW`Mq2A%7L!X+T8R`6xOefhO@@B1w4Mq0^e)GnWlIDX4e(q%O)_u%&s zJ6Sxhjd$?0Wev+BIP1M@-^J2G#``y1mpII2f`TXyqQ-l$ry-3Gb)l+Cw3=p>UyCG+ z&cmrUWu$GW9G-!K9`w5)P0@KeyB5S4eZ40bs|s@J1~x+C1F0>mk;4`V(u#Nf0W;62 z<mzBm=9HkF59!+*qGe^cd5Y4swl&3e0G(fLN?P4qr@lp@(W{lI1V8|a3Gx1-1=k?r zo+rS_yDX1GbJS5RF<uI_GsDhF%7WfZ=t@!m%XC|ot|;{5s-g1-68KrcPX|?_cK0Z@ zZr{p}aEuZbSdEX5k0Mx*Q3nuo@G6UL&%ZdMu<tViJSg;7rx?`djorDU6{z?B##b7e zr#_2~ty-=~ug-q*aWzY)q_iD&1Q^-Pv$XMoLK;eZ*I<qO{C$&b6WYq+!De+InE|0~ zi_*EpHxN{zCTL9t-OW%vtSN|1lLvmnB<lhmeB_~Gd-O?+g0veW0{w?7U@<}M06F(B zd0tO5lAoCFTt2pJgtwFRGQtN#+jXj1W7M<VnZi-&N&Q-Bj<SJm(m+PclgeiaTt<Gy zKHI<RNA)L_ZN8VMsRmefw{NRM8J}irQyK`m$l8K6;vG>0{iDlZl0c_jpw1!a?@#;m z=@VB6bmQJSOK*eD=SYLTFa$A?as?ta!mem}!YOh{;N03fGXYAf`5-xKo*jpy(xe|e z$fK>Tu9kwg;DUmJe)mw=P?afWmd4||-Bi^K{+0eb)R6v%L_iM*UdK-xV}u2x##LgA zeA0NZ+NJL3k;M_SRQ07294;Pw3N?N!A%u{0u>OHT9y$SvJ-ne$pQ#~Z({#GCMJvo= z{;}n&m**}B3shJ5l(BMVJh~zqHjwb@m^qvyW#D_Bu6#A35Xc~fh@^rdZgBf=%=>DC zc8`k&Pid55J<ySS6A7VQa3Ig#0aXx-+X9Hm(4{J}v#t-v;>aP%3nWu$-Ug&Km4)2s zu5NC@f({nm9h4x-fqkH4#05^+LH%)3Efo;>Hnz5mmL=9}zZWOtgFQ4N|06JJ2mj|) zZ|~|1*655Z-W`s*n4*#*zM$ya+<ktY;mU{ShJ;P^Y9DRl^8-${D<=0dw6J|Twkwl& zHHwMFKT;f~*I%cq`9oX2N3yu@xzK4*k6c!@94u+`$>M^PLam>N$=J`w(8;EuU2^GR zFpH6PNSrEL%EJ#@FpW7T?LsMX(SFjLbIrQN#HTl*_9ZDP`AJ~^PL1KYgSz~sDIig% zLQ^d|gDZ$mf*ek*?AGJb+B=sDxd)+aZ&)%H(0AAwnjXNYlqP1>&*emwI4v-Sb3dq3 zydqwoS@+m(x9|u2mHB<|)xZR){|#Q3r!DnqUB{X@sNy_r>DW5M!1i>hpRq3s3!L+1 z%blHAbwwt&ov_Ux4Y-XTj{|mXIA`fL|J>%5a%zyQ+g76StQH3tID1QzlGU;(LuDN- zJBgYrQx_rk2{^YVk{!dHp<?tsHwdN&^9b%A8y<%MO!U9V%7Sw#KTNcg=fT$m!cicJ zE6V#YyJApTS%lg$v?3NJpBT6eArw`ez(Uk15q%6KDRlgaOMhV-3ansMKuy5t6cad+ zDG|m<;N8dCZ{K=&N}>4}j^iWdZLP1@)t<2El3930$QqatNn=H7I+D~Y)Be12bl!DX z9P}ne@$<-rKhy7DYI=yRI@rCdJGedg8))u}LkV}i8}3i-KadOE4K@KK{5DbeD@Jl) zFfCgFFHi^@@xQhA-SJqrf8S?`x*D{VLW_#*QJF17Q7L3MWM+lzQ7R2GOPOUPyX;L# zA!LMXA+qT-vY+>%>-s&v`+4ry{d(?yo<E+~<B#jrWt`{t{C<z)b9}~o$gtYR-ZVB< ziYKFF6RYmidKD)TkGI~tkedabDtL!wGr!WW^LYlxUzM@&&bYJ8g4qR02slV|^kaZa zRsXC~@I(#vSoAB6Sr=bZ9HA8wis9elencV*Eg(Oz?q37cfburfGJ8J@+8fnC_Ysjo zXfl2N^ht&^xaVeX(JUujFQOFn-jC9*k!v5QZkzk-Vdg&d8%ddV8Jtowr#zTIv*luz z<W4o=0-9zM5BFf}VB-O+zGvDigTxG48HXN@%b!>AHmxiOQ0ii%uJS>G!x-6=RCJva zxY?|)0e6A>F{1<cM%!Uj${0u&$Rd3uClC7mP+=ha4)PsLd&VKSpcY^bc7G%KGmkN5 zMi%fIr98naBBCbzPv!{`l_D@YOrT<cs*7u<-|?BeU-HN1>R2a7i)=m>%*ZX4*%nZu zl{lk~R4n#@lOE}>bpwfHi~HuamxgZT*(OtGLivsd2$vu5`s88Z<lOt$8X>(^mi@XW zu{V?d9`fVxdX>S0CJ_--4bFG09R^&D(51sYB^1H27R2Q4(Rw5$Q&B9=v*tK~6$m8( zKs9m*#2^dsi9oBk`19veq+kK<mm~}Fp%N`gIh1ycfK^wgU*B)s*wvWV*%2`@Sl$D~ z$r%X)3DdUL1I@vMuS(9}1VjKjTlvEmo^WI)B#!+Xwt%okf{Jh$fJrsgn)AN04dj0g z2)^BYP_34tUuFCS!e^Sm!{oI(FE}!?Eu~DJkM`+*R1_%lW^wqhci2h8?&r>u&!?ux z&Rb+HOpjxjQ2px%h(Q&D3za)9#-=?b%XrwdoHZx7>Hx$}GqEyR@bTjcv^7E@uloGc zX>-$>R~{j@-CGBnGX=^saR>ScEr+Oe9p$IEWOM#ImcTartktU}G$!Ij5>}x$dKijX zc&EMIM`mQQ8_@U`vmTG3&g5?xI=Y+!yt=^cc~Xyp?Ry^e3W3R@DJDx0oKVS<B4nV2 zsV~k46#k-^$;7HewSiT*Ile-7eB%tdF!xtbr}l<E+x=5M;G|J~mdU$3&1@4#?(xsS zBk+r2MN;v4gzrrRGBCFN`KDYAJv^llc@kofYD}34$3}hA&g`%ebI9oG??2=-bX_S{ zj_uT|1zY5V|2l~;AQ?D|V?XZk5=w#Qso*QDTlC8}&aB)Np%hz2JFU2e<$CLe;fMPt zd@_9&y2};4cNu)SdnbeBMit%XXs3xV8lch!s7S%D1nl-4CN?J7_t7R-erxw|E=<ZG zDoHzIhm^FOW)|Q-vW88{A`XY<mIfDf;u+ipKpgsB7m%!2c6|7Y6pJ7Q`sr}5J$Uuw zYN#ikZgZe|5M>B+MaqcSzaS`}3TYH6ogbJS_P3ij?YwCUtb;JA;LyvBDJjO$>gDCn zzLRc^8Lg*k(HtHr5oNh@<w^>K2Dr3xE+UTBq4Q1VcaWplzsrgi*L%r2Td7Oiu0&|7 z(^OFo0|pzQds%$V`GaDaG^q#7r=S5)^#ydn1c5f8rK5Xwm=W^M=A9qWeU{(6hlWy+ z8=0`HE!IEtWK(%;aj3*#!0Mq9ws&o8>`Ax9C_Ap&mTjB_sJ0)2b4VcE$m$`G6nrCF zcq65Ni??WgB2G(6E6FJ~Gk<2*0*sH|X{%@%28(O<6v{l8=hmacP3ehtNJz0U^^tvI zL~;(2C<r8x_zun|Edmsed;AwlR9Q{DlI|wkl4+j59&a}nd;E}pQnxKxM(s_()Eo^( z%oVA(C$p$BC@w@FQ_HTGtzq8zuv#H6A^#D<X;Q`%?O#F(DQ=M&iU0e;?ytZwqU=e3 zZcc<oCb%8~D(MU>%$!i4e`)8V&Dul$5RzI|K<{o(Q(d)YP3oboXzO|+C-je7@VWFu z4paxUYcc$$@>#`LP9>$7y(DG14IPL&=kMOTr|5@bkv8MksoVebVasYeKe?N?Y+NkF z6Ep>6#+Fc?hoTB|+Dmkf&z}*$W`!PK`@Mf^-^fghND(U(fDu;xai7PgdO(B&jH0?f zQ62qEFY|qs3vHMS3JrG$HliM%*It~MRKIgo{1O88GNCRn)?f)G1;?x?UgPAsA?cq^ z>tSG(7zo2ac1QAIM>VFy)f#+x|6Z8Z>s)#%LMficIsrB^7wiG|rhy`#tR(c63d*6{ z7a%M&JY`bDK6qH=U%$x%jP<KCp$X(M+Cw>04LX3_3bYg#d$Y;-);Qs-_YOa4{v^Yp zUYE6fDB~<MnAWIySs?g>W??fB<TXEmTQmzP8=*Z~;^Pw<IyH-grbhwf?&yne-Aiun z9UZNX3YE*t>po_M@^)3N2aB<SBjS`3OY+HPA|xb|{@hg=+P+|rwcdSz9+>XvKhxRF z8}>g+tKVEE$9P9kLP<OQs@f%#pwB3cfIxC?qGACNNb#d;bd}0<e(4x2IPDi!Hmp1{ zJ)s^gy(amh(*PgnTs?YbZg=}gFPV?blL~TwO@OSy8`@K0jGi%RYKCQTW)$LnQ+j@3 zo(XtvEk-352DnYutAk)fraY)vvv*)Q6|c_8DU<5p*5Fl<UjSgkMAPDe_~8bjt<BeP z`(3CuCvKrTyaZzEvILldh#rRYf(TBK!h-b6z*sW`oD)ZlHxaKR<q<PpIKvwQl;}Y% zt?Do-3~H)Ne7mo9pRWJ$`a;R5P50?&ZOQs^28tQC=f28M5GbCXn;{)=qG>xZ)Y5Mz z0rV#UY3P;*M1s?y<_Ds92IQ2=Xd4xQPDEO=b0>xvM*I%{kyeBfuD}*EQ~wAcQu+{4 zYl*%h)aQO;woG*6Ny><E!)JgQnb)BUwSJM?IhAu=wKu&js7-HX0;v0LVWa3PR}l&J z1bRRyE@>%F3e%^5P=25KwqViveoOrW_455*$PjthA_CH6F-ihW$Bwhzd_e0GSGao> zkgP7wVT+hhi$}@%*|vSz0)V!Ok9mDfPrQ(yH`W*1U3?fc_eJ}G4A=E+5#@oA&`1nf z^_RKvV<Drmb7Rxv<y`fM?|}T_K{KZqb23W=Io)c_N;OLUcD?!(k1UtET%d4fnkmJ_ zVv!Wp{O)gjX8(=mIK{|q7!etgCuxD6krPP3q^ILU#Cd764LvvF67Bz4hL|uyN?3eh zmP(!_bp6<5(flEDUL<h<ZeBGAqwF1A))!yXBURK=UT&6($u;0!#zf|W$W0M=1ev)& zQR*wo`Pfe62N9fBFSDRNF0pcR3Rv?yb`yv$CcIEyGei#frL_*pP<e>BIx@W!UaXvg z_{^HcwTs5KIJl6#3ReNuAV4nQuqc$ciH{Y@z@q4<&Q8v%S!>{8yxtN_J(f5K-H>om zXlNpp0LHSan7A*#CSKLo7Vk@dRYVjIl`w}^PR5>ItcqV$Y7Kz5f>7g}Nd>YwS&0vq zO`))S#2rCSs?NOlzb=B98nR+KSc_$YuakwHBtyYh0W)4Y3)**(;^z7S>Qy!`Sbi@1 z1>*KBu5ivuq~CpJLFm#UGa|q$P}tq2qRHP<LMresBwiWxLxqjd2uX7aLunw2$<J*> zhK?>cqaRic!hs$M&Uk4zEO64F&qa%eLoMZX?K$#o3QH+w+C3${pDueJD&*bB%E|*4 ziX+--n@OI5*WyPrIqLNDj<YZ9ZEd5Gn8u(^CS0=1{N(2@dkIJaXCKB9<ZH(o>`v>! zMyvXI<!HsC2eOObJsAoBJ}@Cq%5{A&Z}I8kEt(s~YSG+80IU>>D4FR4us|4`5OEG8 zDuqB|lbwBYs5zS40q{FGAeKZ%3RVO<NWk#-*NqXla40}evV>FzmdG{JSSb7xUkG5F zftP5#eTV!q<xJ1I-9HaD<=UqM7haR%cJH1c2G*EBkslV&tQrGeQ6EA9M4USp*_=@T z=t_{JyYz3L(h>9}(u9Jb3t?l$AnzbT?gS~kRY3P5^Ke2ApjjNMNfBf|hf>lI8KSbx z;#<s~Vlii!E?-o7mT<TTfO9><7n>}!x6eW83QY7%=2@a7(t^z-g#3%7V_23%o6NGk zU<n4}>da_=;-vA!n(~iuaiRQyHD>G`&Y?agVpha#!sqbrOK5t5oaHHK`Sa|fMS?tK z=m{XUL_iJN?a_s93}nU*p5L8Ur=N!bGYMV5o64WEBQdQtz`Otgd(U#qQF#zx0)TEc zj`CNogrNjkWLfz{;Nm$+Pv+L}`_zn80JAk84vhQ<%+{v#=U-yE&ep|f0%RYjJ2Tv- z$xj(M05rF<OhMaL7F|3{X|St2F#g^Kgy2DXOt~R!Tn1@9eH89Ry(Ysr@FDsP2AR3O zGss6C*oC|;_3j{IczwWbWD&DL!AJhlD*?q;&2ts)+$2a4F>kzLESkf1ogN1V289g2 zR^k`APIqqyY4$uB>>YwGEGR}qIv(@XYvc*HE3__*=bV~;@H(iTYsOlME_Nng6aV@1 zuk&;&kX1Tky~^++0VE<oyv9;9Xq=m!CAg<m>sxX`I3n775FMl+#DB1I&SoW!QmMY7 zq1pix(CXme<*nR*cnM@xI=AU(M1-PyntAP?X30No6YJ7;IiS_ZU9!=*#lplC11=Wq zJl?-_#E8-)pMeajp~f~23>E2r`$V9Cb3}>S2Th68@5mfp38$zz#qOrKe8RRA2SN># z1(<xqtS+oXSV}efevF7ZS#+1Z6=`LLihB$SJ$RPfBMVbLSVReu_-ToR4b{F9LM^FA ziqb87J@ndZGmg_yLdx;s5xqG;D<w!G!14){g4m!S13!<9IR3G<3{jIOp%EZK@c!hm z>2c|?1fdxlj-bdeI@Lp1VB`IY04I!mSKhnSCGK74N=sQX3nMM)J9n1hHDtjyR+gET zLErX^-%VT(zs%#42NQR1B3LB0t@l7D#ViQS2xMTukC0gkO8#gtAblbLc-#kz6!2?P z4gs<4V*lX27if|X-Tjcb7z~CwvNv3ZhK|tlx<r4>!{ttTNqhL%jhD{O2^6uo_}4Dc z7(`u1JRcAUqM&m{RJ?p3ykn|cVqAwkI3NFnGqO1olJ?MlKrY?eev~@({%%6a+HL(d zVvboU#(|i32pTBX4YRrc@4Ivk9s?K9cD005{EuSq!q(Q8$Cr}_w-{y3v>j&v!Zpn^ zkhsU--|D2!p;Rg~mu}4II9lX!72N%v&~i7<n-7z+7V_2ewPG|HW~bod&_A=heLJR- zrj04GyCMF9wQ=0O#D@*DXJ=JExI7g&7JUlb{qeK^DT^LCOkB9sPWIPjau)^&b>DYf zq)Q<uKb5R^s^-;mXWjAnt?!njFlliGStYEit4nZ@v?p#{`zo;$pIP;!8@*!Tz3LyI zk)|UTC3u`Sm#b%|Fvb;;bPWv^bjnfnniw$9o$lJuGuh9L4`9?vZG9VV8zgd@M^)f9 zL&0G&lV!P>#1c_C0w^ZbLIR#4fm@RoZ*tITIz{kCgrIrYSd*!)`lT0R^D~&nB$Rmh zTxng!x?y*`8gR5Efti`KmtIizY^O0>Lx3T$Lm&+jh$x)HS)_!JN}Ac67`&_fj8Rg+ zOY?i9CV3?v`Qb#gd|*nC#!Cym{rieFgyuf;l<#jHY&u1=9E(*hTt;tjKAz>>@^&WN zT1HgC`Ln+egw-MtB+2#m)kJw>nGU_tCvddWe}*gPqoE(ne!MSc)JJvgDJY}>u#})G z+uApOUa`#EUR#^`WnuQy_IIHoN*vm)r~q-*oRw_*5#Q`O9_%F@0E(Bvg%};<g_ke8 z<$i&v;>sWrVm~_?*Op(q`!_`c$>A8$X@jh64ffz3S<*ZJ@=YlukrjVjeaN!-fWpG> z!UHR{oi$5CTEtYc(^9NrsRDo6KH<O10}kg)TKBj0t#crlmq#B6gbBUwI@o&<ZgxMW zPxXNF#lheNKl%V19(4M8(5SC13=fOI_?}(9EWUnEZ=X`VPVrND!xlBeR*j^Fvb0r< z8a3&nW1%9q+u{R-7B3co0wR%8h`2R4K*lRXhkrtdw+dM~A$8{dD&+)N2WOdR)R|N3 z)}cPk<W=pj)AeTAW*>ghDh;s7t_(XXs?%>w&%>l5F*(=FPI6i}cdq-rS+1R|fSyRc zB&CjQEHGF5s={RgCWuZq+V&|xD`CfH-zXZj`E5oG`57ov9XwdX7+J67OSjrfE!8LM zrqJ*wr%6qNk@mPYXFFW);zT$X!#&8R`=a5|S(#@`)x~hBEsUfZL$SwbEq;tV<uXlK z!*amZHuKe?ClMKksPopc7^HwEL$#fm(ET1%JL8Ee|A>EC(`vArz=KC}uP3yTXMT8Q ziyGHLzVimk!b2cf&~C2v+IgH{`WJyGq~ixQPC1Mcs<3qj&A9LhqSRCN88_wFD(Xah zjeKOpzvZJxkH@yQX$|E;=Z+8Ch!lk4#b*Eo0R`6mbG|yV;tQo<3aJ71Y!4ea$7kvx ze_bQKR~9%~;5oU=F#u|RjIMPWhM$i+HgjPzNV_T^{M)Tt-TpMu3L!af!yJ@{7P@(s zq?!6V&8z;J5YFF|dDxRPUYjR(<o4lO=db&y^EM#;{0R{^R2Uvw;7Zjkie8VFD<=v+ z>I^Io+)KOcSxtVj-GWM@T())Ks_KTtnXl4h#HWuUEN4Qw!C>zW5JacPB@SsU3LeU= zy|=gWUS63|8XAG_c_ZuLh0;yYnxfQo+c@W@q8eDX870uRuLPz~+rhl^TYdfZl7%Tf z3ujzO2wGT(__Zhvm>VUgx;CJJ!2z9T$jiDTU*VENuI-f>!R@2VrsCP5-mp*=4zU1m zo4dm=@>wivt5;D=`XIEmE{-ofu%9s8juF9+iR;uy4;zv-eU}vXiFLD#0U||gz2FYz zv4;o<kO+X8b0kR_F_+|k)q$2p5q}Uq%J2F(mZJ2mg|AkWW5+dXFegA%1R_iCjD_~Z z8Hti_5*KAzLMl+bhlhs~qnVjem{354M4nVL&_=SunJ_-p6RQ4<K!6wS)Q107o*eGm zx@`acYJg%-yJrq+Df(oNx6aQclrO<JOW?K>&J+N}E1hznnn6&R6P-j=6TO-AT}*h9 z5zkD!W%+-iT`GxoQXLS-priMTQGO!rdDEChR_?T4f~N0{A2%G^Tm)>&R@e~@t*+vA zvPb~x$3UiOnmp<sfkJQUO%@Td<Nr&=vz=8;e<(N4>}ATeM-9qhcQTjTHzXyL?~)C5 z=DzE8ib3pBD+7ZLEyiu>AY0>TCh9wapbB747V9)OJwZjYQV{`_>onoUXC(gb@STN) zx5C-4_gOS$irYuft_o_CY+uPDP~_O0VNMVX#Al3{ZK09ii=%IDmIySX#s25knQ0?f zIE_e1C?@MB3`_1hhIX}uVZOir;rO}aFJ+GHG)px?mz&WAUcQ)n^sW(%OgJ#i#>n8N zJ|xF;0Ltk-o1!O0WCc)2Vqgo!s`$sGBx+7Y=`QssqQtj+t|v<23AeWJqe;a`r1SZ2 zsRQ}WzS-^Ev9;&7Ww4@UcxKbajeaKfsNXb$P>52B#lphM#4Fj*xweQ+MePpFk-@r$ zjMh%~{?%`rU&JX}JLS8?+LRF)RfM3HKZqyxyLV-m)1Sm%;K}<rJKqW1M5jNGe|N<b zbhrLNo4CaOu{O*I4>Wae$<qwrJ<4&RIA`zd$mO2NwwJD!LqVAV4*#*&27VYCwt*dl z2j3o=ZKu?j+jQWov2brUTkv(^o=NA$bGgBJyzlQ*`CytO4ZrB({-s!3{&;xUF!<lF zN|`ZP$+gkDg^W4a*aIeCF3fHB+Vsywh0D~KPLA?LJr`|q-`XA0gGbbmR58r(X6Yi* z*~PwlnCI@jtXGKu(t2z&J#7HdP{VJP=2_xi&;Mw+{8lM_`UFDDnU2>R_gM9_W5NW0 zUK+DFNARJYz|0Eiw>Qr(QTb!hXeRTw;0blc^S&2!7Y)CfU0-Nhh=>Z|XMFQNKLa4B z5IpI;X%t-%41kJC;w!ESw%%LI%kFmmk^Uz|JIJ5Ltp_&k_nFW*d7nDNqp8Lus%$r* z3em*tqK<0Q?KJf*Gz>)63-{4Q^q+=%E@9d&l<cjpe5v~q*=hsGIdEAT>y6r4^9jO4 z0^J@i_BsOYW9-p8Vjn_W4pRhFHBhq9w<1}F&J}96thjwvUK_QiT$0*t<=CeBSL&^q z(IoOdX#9bbYcvhP>*)@MIXl9jy0E{u(=)#Ob{yD|_>BT_^B)?`$K4`6dX(Qj#9yr; z$gMQkSTQUqTsQg$&j0|7tS`2h7|q{LVg1(Lth^9sEy9_6=u;8$1bQ29bDPSGvXaqK z`uEJ#(~~;mruX;VJ`>_lZRHp&VLWt~g~fz>OMO<VDndUdR@52oWa)>Gg&8A7|B15} z0Sf~SSb8D1l$w#mP%?5kRknM&F&vb6Ysw;P%`Pbk2{sEkd#qJA`ff(rBhtjg1|PU7 z_J3i3>z$JIhT`ItD`nedWfea^t)8E&7~U4*YFsC_T75tMWQTy<X~BgtYIfDrtnwyp zXIZI7BIyf{nIm89*uV2=sqZI^Jwu+75w~a}W0WGR!h@Ug1<qm%!#xgvY-WFuu{-*e zyud0;Jy5+sRIz2q(%IbQ1EL9JJ{EiE|J(tu{h2qo&GcWRVR!iUe7o!89v0_jtHWNY zf9ntVX=IkGU+}|VIx|uYv8kUi<nH1#I`XQk)VMY2nn3!^?1sCmy61i#k-q##izvhO zr#`in`oD8~cgFs??JlA={|UFZzIkX(@rpmLre@2{%^~Ton5fm=VHsW1-<0mJ5Sqrj zC28ZIJ)GehsN!zg{JXBl#re;gp$xsabkt&_rcZZyc9qkw2tV;`q32%XXJ2G{L)^Sn zH~Fwwt4-$`HzU%C0x*XYsyTo;S3&3Y+Br4cwE~McEiPKRSH_c*2h%Un@GotO%Y05a zj%f9zvGzb6a9#|hIQ%D6P(PBU2VpRs5g+#><9EFf+m8=3$ix05*P_d{uefhdF~^pp z)&fK>8knE~PqaX0GBHT#N0mkz9x#qzgCv&{5t9ejnIMwz-21Nv3T50b=e<~6py_u& zRgshjkiL1y$ddp^6@$tbN5N4I`dJC<#-gGkupRP}@^t~xgFAqmC75I&Y40unclDeo zFR!)42Iy9Ac@<f-UednotxUW-q(D>n_O=z2P95>lJ1-aJ>t8(=$W6I%Eva_h!dqzb z5=Q`Nu70I6xxIhLL<jUw;x`16UM$*}@M647d>kC_%z|qO7NwPqJdWYPo@&lYc%Qun z#XQ15J6YEQ*wx?>vZ7t$dcH_*EB&Mw{;S7-{rZ6Yrmd-JnTo$gz3Ex5M?B#?c#(OO z_0%7r$Gvryd$+-m3)I^3Caq-fh=_8u{0uoVaTt@j1}#l;ltPO+2IYHWVgtAxq_v1% zeGEZhwL%CP@7mrLdv152({yjvp5ExFobf7|gVJS%g~@0f^(#wd>$$(kW|giC9>CBX zYc1ErULF=9)y9B8y&^LNXm60&8?gw&OmKUGtT;Dml?;FXk7T{fM`!R}low}r_Lgp9 zrKOW2ByH{<E3CrZZu~#;;{~pE$30%!QbMFmp*qrBV}yhXzG{(V(2H(-DzQQ0K$mKs zwFb)rQbw7UQR!qa7jMDD*BE{@eU-P2>%7dN>@9yCV0faqhrVMNX^){dMbxQ?L8tv_ z3wj)&`^BKY+6WWA%{*Fhgig!PU)?c=*K_H4E?nNcE!W08)^|tjrDZMhzo(`EXpth^ zUO#X^!48D(YVph8^eb0Zqbo?nFaW|Ls{yF|D-7hW7A%eHzU%&RmpKbDd4hMxfn{WA z%@T6D_*6vg{vl?B`^&I#!dJ6vnMP2d_@8+4|NV<8|F4=d5A7nnKQz&Zi6`dNQTFAC zo?ii6M-I)2G6%~w37cxjtQ-S%!UlvyY&48nr@_?-W?htlYqx~9@eSw{0v;gyMSM~t z>%Y(uo-iWKcNFZgn5u(nT32w3rZ(MF8*WjJN~~n_P}=th7xJ!-J#OQJ4%tOCB)$^W z+Q$5;?ro`{jU!T>Mbu)G70@lKM2&0NUA7f_#}Gb@YHzRHT_iyCqsJaYramZst!teD z$@-UCW5Mbf4e~DhjD|FmzL5OnB{UTv;wL&Zd@x~p*%M-~fMezyW*aG1{mO)J_}z+_ zIO0l8r}F*<NBsCv&IdAGi4dG{pCgqNh2I19i|qQVX+y3{TjYcOP3ZosapD7TcF_s} z_U*CYuqc$EJ+=z4%iHVg>q|?KagEmlSSoSC-=S~ITH5J+estD}kxY87F0>Ayy_xf9 zedOP^n2nz26%`lHjnDpPK+;Rnm?S=_q~Gi`o55CN7B|$!%nw-Dkk`AK8Ed!IA+7nM zIwNEAEjwMKCWq;_E;i255MF>8>)&GjagT2=$lRLe$${4z$hGyxiTjmezG~*G;22DS zXRccs5K%LkUnS2s!6lBDm$#+LLKg8H&F9eLP4UX!UrwWX;hGD5QTum4{=d8eFw)yD zbv5UneX+2LfRbQP!{0?*&*I^pNF3MT#qe|w#JB#D%AX6s%@fHnLD&U%@a8=oa>FSX z-Hhurx-cvwD+vqN5AqObXTt~w#P3r+q*RLq+>Dqbj}uKx`1g>mGGws_cDEB(W_a<& zkuK8A<S^-Fkh0ha$F;JPAbI&CbcnT5gbR#@VuNR_QY^bH!FOTQyaz@<T-tdWSnomP zvXLl^4>X(yTR1hJT~P1>Fj;sVS3%=OK>~9@H~btrd>Zq%aUJlzMmH(ydD`&)|Fc`g zs>(3)kJ#?B#qx>b@?T5K7yISS{yIe*DKPf*E!RB{3$h<`q*qR;O=v1a;ChL!A{;i& zRTq$ooX7e61h`Bhih+AvgHlr<VnNgvQ#!Dt1>w~YmE(+@hUj#odq=x$85y>tE#^i% zAoIHccp*ItOuopg5R)q$!*!_b`S%nC&(i}k#}QcmbO@F(rEFX06Lal7ACFcnQ07C5 zS7$yu;qzbxJNM2oD^-6f1r-fgLVt)nwqHu_QA(*vmb#(Rm$z!$zH)2U`lR(M{t&z~ zB)x8D>N$?8N{{<*xV=|ASeAK`Beadx>G`I2VzMDy!)@~`N)DHl34EEMI`qt(4jC3Z zNV%A9!VKL#O0E`zhqUYa_s5iyo^&y8+}Mszw#<<ul%SxXOM*-gNHu-&`t|G92Dxbp z1-6OHa88=tym>D>+dkKV3z{2p9+RSbK%mcb`6YbL*GJ;er2-cw<6n(NiK_9hFbZYF zO}_o;(JCCm_jVq?*NzTxx>~ENNn4%=3>o3bwNgPr;V0l|#QrvU7Rv8e&<%u>%H8B- z0n8M=o<BE8cA$994$OSkp1+dJR9#cUcjd#1jLgh;MMbp8ls+OFDt$SZ)rzwJ62^&@ z{o`iV)-rG?nySkhkJJ%Az{yz%w$S(9-oI2-RG{8TNAgiNZG$yRCzfI4S*b6G9phI1 zD>*5N7bBD{tLc7@+reQpwvJBDU-Izbtsw1@&oE{uj?su`b+w|B;R|hU`pVvD=!Uo> zK+>~1jNEGJf}-SBeBFQas1O+}pWT=~US+qCP%Zluu-_+6pI(itf2;R}<^GD(>>GxV zc@NKif`oe!utV+elSy^=@mY`fc?oLA@MnF8&ZLLC`)c<seDB4|fuuX9s`>zDz!nfl zHYg`(Jiv0gg1jl_>!n_CSUm~dxN#$9?toU#mV|_a6#yd1t|`b(Lw#`0IMc}28G~q~ zfwW-%9<u7&NKEix1_bM)us=`N&)=Uo-aI>Op_?o41b-Dhk!RYFRK%Vs*K|T&PSPG? z4_fi^=qKx@1^~BuAE<y%w1R&_g`^U$>(5TJ!}hP-@cAS}omgNo@)o_Zh^VMeFn&+y z=<EUYEiQy>*OtRK_CSi!4Fl#~LPDG1B(jBHX9WuI_z>+M@4n<gr|d|0)g?F_3732+ zEqxE$U^l=7cJlG9GcYiK&EQUAWpDSidfWsL`#0*EH`c>i1FgB+K<<5yr+Xp#Xo=k+ z>miMS#?(~+JrrVOX^zj1Ii}hZhe)9dQzL~8FRSz~d{*|+*3ek(Te1v0028q75)NCC z&Tf%#IK;up86U#Xye$aqvjQFd@7T?UZVqh0ocMct!FNX09Vc%7#B7frnkML0t%LK) z1E@qG4@MfVklKr5kcYBINND^)Pi96&F{8G_xj{Y#28Jo<rd;<~cmW}|bChDtOt}=) z>jSv8^A4aI@k2KJ0H;SBtb!C&<)}l}a5P)8gOOXqrb`TL7<Tlqm4xzzhmWEkdJS#! zhrenQzT*&Hdc>qTWAwzYSO>_lgw^y$goPbTo#pWCr@>UTqq|#|FF8+$@b1=c-mE+v zl+`VHvc&GCbZh5yL1!P@ph}=?EOQxXO8>$>MY^~-aM7#PszGt43Y(Q$ssKO#G8b}s zi_Jj}w&)V?zUel27&2_wyqO(4BT02W!@RQ##Yd8At1FAf@H{niVFl$d{KRYnoag$r z!>aW6h>EJG&dS8Q-MxD^tk&=v9!vKp>;+mEIL5{{(mkDLjd=A+4uU@Lkd&dEMDI`j z*W^IMZhCrp#S0g7oG>9dE6C(A(2`x>H??aQQF8`WRHo|J`W8>^?_Qet2=DQn;q<HE zfVtGk)2B~Bb5FAeaG1*&yzkq$@5r@lY~&E|d-CKm%0S_gd~I`cJ`_ZiaGC}uGfRQ` z=E8aW61#lB9azN?euh<o5^6BK_KPZg7W*w0=gzGY6BCQ`F)hIq-5kqLZBRVTDnDJy z2|qz}*v`zX^hKq5dEY5ndE`Z~7nVjS1pwf|?5smzcUxkfahNhIY?(Jem}3Nt=MEU* zNyDEkxG&mR9&3+8<>uz*N@$jd=+&`H?%THl>#Mk;;@*8Q7ikVzbgw?m?psR*O`L{O zJF!vp(4iN|8_5>DP?CP3&^19T_aVW_KH073*QKJY+yRf?qreyNp$lwdZy%0=YFxxG zE3aSk#tr`FmX<2G-%tesU%ch&>KazV)0fS{BI&e!)22<>mG`IKV&}cn7u*H6#VWC{ zzYoA@{2c_E-hg`n^RTadspIAO7CrQWOlz19np}5^gHEEEnc3MCldlIQd3YG$bS@4F zOSq!lLFPf+AZy2coZf|u9b9&T0W8L+KP^S3P&Af#{^;p>1hWmLi5Ia?tUEuh;d_31 z=@Hycsl@uvunV~UlxJ5G-jr-M%KEiy&-Uc@$Qe#!p&dY*`$=i3+&XW&!J~=t+wR}@ zd-_xkC{rDXc6tWs>FT~hFjobeKW0j9rWsC@Y(NEENlP_+J;-8dl=2QS^n<=&DN!#i zXMEed{OdQ@goszzkHv^mY+PLOXbR#_&e}~5aRGH?k(>0SyX_1%kTTgoP~IdM3!kS? z*<r=)hu~NWQfZu84d@H7e38PM1<bn?`Axr{ivMg<6902C+oY*v?Wmd(sAZY>&i?v! zgOACBAAE55!+1e-rx-^h59Nane?!E*<&IG4fCz!WelD(5S*O|%RXsgDRb=(pEv3ll zq{6FC0w>aE%V77-E*;wN1SsWUu!hdn0p0qTACiXR&6uzKw9I0n0!o;k1DglWa+O%R zb>4u_oFr$Y>Aq&y1ZgKNtq|NNgs;aN85uuAQkD%s21+r}x(*2q3OYdfP*`{z6aO8M z()op{{6mE(UkF`XyiQ~I?Z9yLj-<D7aR+dOkQLVb3A<!1;5GHOK=hrse0e8g<#K4v z;&`C})bpgA+%Z6{;t*j)dyL`dR9sSW)Y4KAC=2u7Bj43R-SnbgL16hRsudiA=YHfQ zHKrVulA^^g?A-Nxrfp%qa_X9%-tIknmLb6`LZYfvt`e4#vJofGM<^6+%Kun5*V;IB z(6se~sq>6F7}i?}SP4sD+J(vQD?u|;hk<xF3IGMMx7)+ceg)VT<u8fw`g>iwd`R+3 z8feh4{-dR1U~$JsgF;g)hY&1K)}r1jfZ`G0qM<v)#F}#^V<h~#76wh73vr6CWn^3q z-;nJhmMfq(doMB3G*$drU2(l(>ThtG01R1zV*ogf71(k|_4W6ih|p?~GqQmm(>7kz zK(!SMX@1&P66O~#Zt1QFUPbxYnkzaFG1sfeL7|STEHvy?_xpfd^}?#JF*~`r>7i(2 zl|}E#G)3pSgq(F)-=p>U^P3PiZQxRhb;t0i$0gQVyOu{Ay5FaR;bAZmt39_83H~R% z=^OA!i>P7u+}tuYMDO6_T!jSeBe@uuAXxU-twD}{3@s(H;R9>GcXz`SS>y%8g}&uE zibAB_1FW7}3y{#f{QZ}QSFM1W+aVE=4m^V2EyGT3!Z4=}QHeCXO7xGvWn1%%**SU` zpf7YAgqB?hZV1iEYFwVlD5yYVy99{anLP`#uuI#F)UOicw+|mb9*eDw5jJIWY?nn{ z!HmFZHZh<Yc&*Vs@PcudTC%)`hDQ7FuqOueSCF@xSz3NTV>JG$N+i|;U{jv}9pl9S z+;XsS6NY_T4w`HS?~|PF*++hYZ3oE7{h*-jBo&gBWW{YxmW8$rK9RTgEmT59y3Zk9 zgjYQQ#(k_f3&sOaVlz~1o9%i!J6&-#hBZK9e!&6rdi-Up)pvXo0&q;$qZvQ|$^Tli zW9QB#s;a8UNDrFsr4)eRwlR1*iq5y>4i;6{k00N0vA;h_&fgjsxE>g{4Rtkz5;%Ek z`dT|Bct8PQwwJYrvB6PRh>75Te>9=SR3ti#s5hVpQ_WF(o%&Se#ExCv78{KU73E=y zaZF8(i4-97qvPXJ7cOj^7-$H$V;RZ!cv)6<8b>@sNJxlKw{LK;#}XR4XNmfWV{*}k z`OAT8{sdg+CEhI?7q^ZY**bShbZ#p|9Y?dEBUj&We2GwPq&#vJcbsL^kIz%7wbwTo zjlYX&V!-$N)NpiO3znEM48#!d7LH62$B9^PXUT&H4<et|(besMlH8Gt7cb^gjnoVA zp{t-|h2^f|lUUPRnQd*VqoZ@dz6AlO9rd6c%O%@fTomidRjbTrf16XUVxhl|i;GKB zNlTb|IyTU-K`nkW*6=~er+7~2Y=mj&J8jlZlwRXJ+sSHl9k&`{9qYi69iN?nMBKZU z7G}HzxIM`>QLBW0HKiEPXlrZNPO}Le5c?23@HjS>6IJlqO`B-XoH>J&@+w*?wZbkK zL=r1Vs-sM-1jW(8!EkQy4U;XYY4Y<WiIV;a8JWVGnhoIZ^0sLeHBMob6r!7Rng2O% zrWufG1fOVj<Hmc8bhTqA&MNvwMd^90Ss$;wij_4~g!!J{{=~4br93$!OCeMH#(vVI zIoF;sGc%Ji#C-ErO>dd21>y<bMJ2n)CsN|2Zm6jJ!~yKWJI=_nV$yC^Gu^PT@D2<# zYE_4PqAUvRjnd(xZyC$Z@6I<g(^6Et4SI41j^7MhD!Yrl+csfv&ngfIVZZ^;N{t@m zbmiOb!tT+`%1Y^%^1g&#MC*$=Hhh3y(jc?dxpwV43@6D4Mqp5mJz$jfK+V?WhlJOY zCwDPO=tPgeEh=hnb#*l_vhK0rHdZtwlycV~r+<qh3DwI|ENc=o-y)3Y=;@I%Bef#d zI8Q54#>|WtN;U0WUCT(=6th_i?inrdu^4xtd`mOlwwjK{U3#}dh}Z)_%PzsJF-9{& zfz&lTJy3kRN92%w)B|Ucu(@wLNIy3-`3ZRv5^_&m18RnI<6><6nwKu^Ks@1foVJA7 z;ASK@_ubt;KwX@x)#rl8cx8F{8bs!2s9JDty{PXWMW_zPdin{@cy7}X*)^VxECNg( z`Vh>lr7qdQ&Q1rvLf$Q_Pbw&U#HJ*l`ccz<!!9F1rV~<9m(hp*31L3~mL3D!vmIsA zN8C}?z%w5neel*KX%A^Cbqov~z}@pFmdDw-QXsWZUXLc@#M%4Q8Xdf?fM{VEK1xnb zM&wHUXj~unBs4gf3-xNp;2@V;yfOe>Y-m<`5!gwzxZl^Y7t^h}gXXqMjF)Y{eL;Xd z(N$#}F?LlUUSbjg1g8*de;@}~Tr$8`ja3>T`Hc(Fwz{0<HV3bVJS}oM7UwpFs0O?e zW>GmWEBgR79p9GK$k>Y@8V1G&Hx{N<X^e4RweUI4GaqiSW<VtbPh1-}E&_>Y$M7%@ zTup0DtRi&fBO@dEQBETgZ3<S{izVQ-ZnyO7o8E^`A%lmkB|MEc;)hR=aG!*C?g8z* zW``y6Mm>p83?Uy5pV3`9Jcn8#irEx1`V)0KZVY$0pkJVP@#1U4wCY(bsYhVV3qlwT zx0M~6@QqwZ^*Fbrq_!3)jOJEWn{u^yP**(xVkJ5$M{b@G*<op%YH|el59A488sLNI zuhe+NU=71DE|cU}((|nIBg+k1b29Z))q0`s6km5L^{F%3LDg^%Fu!^8DTX()==_8^ z7=9Dn#i1#6r??cI6c^mvPAF#1N_97=&liq}>)Qwkj?KE+RtpKGjE`Inv6NpU-o|36 zuiyX9w<6t*iOEhvXuH_*VAyTptC-%*MFGxd_|jQ!b6*q-5?|iT?JSmvOXN$mV2Q}T zzDxPf{_N0^y?=f+iDvtY4eQp4OF>op4hARh>gyRrg6UCfKNHZSN0&0x(g1IC;mn_3 zV<YvrR<7Mn0f7xb$Gt;g`u66a+CW=AD^AqoM~~jBP5S)#bG)!AUOvl>23Hgo^#LuF zmo>d3s9<&-I<!SPd>xkNw{%n1q~zqckOic;NQb`z&MhXl4zV{Bl5|tUNU3&;iESSm z8X6fH!C|mv9=TjVXej@+mC}#5KY+PP0j?DE8XH)2kou_Zbu8>@Q1rhP+Wh?f{ntmo zlMnuL|Jk!=UC>dv1X)PL)|iPPPfrF|+uldrbnL`|idaNc;NXs9#OdwlcL(F8!rIzR z=m*zM_zmMzp+`fqk(<M<9=JO@#l@LXA_<)F4G4Ick|KyCNh$X(+pb-PFSdtTE+J1u zRack_x921f~A-n9o!nhQ`j9H=0mZX8P=pkTYwLz;$i|G|S>+1aANLE!+PP`J6d ze?#vZx2Y@<t{GcMwTYvS+7GwGDpUy5+`HDTJLd{VMZC$Gb*Jo$hgnZcmZHUTAK7CV zt_?ByE{-N_ud~X^_mDDBEs?fH^W~!(bQR_DK2FZ#wzh|1S?-SCt~I`lrSzI<=qMI~ z-OArHGsnZLcHrY-na7Me*x897oTBPO_rq{?7bgctSw5QS+i)j804Zcx_URJuGiPga z^XSppk?)07a5yLVS5(c*KPS{N$IURizCX{peDQjh;G|s5EV2>Wg6oj1st<~dH8*Qq zZ%p|M7f)wq>J{wkd(}w-iwgMB?tI8f+`{vD-_*3tE;moviX@6Z{I#Xg6|)p!{^-f7 z-5F$q1s1c7r2{)*%<p2Zoe=y6T%q0f9s4lDvUfeYN_-dufD66OdAcT)LP7uGJ=*i$ zcyf?ATauVN!IYvFZz};f@*RJFMkvR5VF4fw+>QPboQvp?z+8X(7>OM%(t-P)o;}|O zaql@g{;aDiK!mr8-dv4{d)&t65D4dMnV2486$d1Kpr^oXUIq_*G_Y-!4w1KD*m=SJ zkf>;2Pv^jZJL*tW%2(i$3^!@K<g(Jz(v5@XhAUsvVAfI@eSR;>luw4APn|gNwNn2? z$=pS<>$wpYSD-gu0D5#4#y){M4_83z#|7c(G5}1%B{26hTwz0xonLkSIz%8~x4s4w zqubgG)Ag{U8G2`q5~dIwng_V|tusMzxxLoxN+QWa13@MmTieShqTo^e-j~}AJBRVE zeI=ZAPx3h>p24~n&pEmMzZX-;Z#OKVCoe+21}KYb0dIHWrV9xSi(mi!Q1yT5WeXOT Y7*H}ec+^tKsFxxurEntQ*rnV511F5s9{>OV literal 0 HcmV?d00001 diff --git a/docs/source/contributor-guide/benchmark-results/2024-05-30/comet-8-exec-5-runs.json b/docs/source/contributor-guide/benchmark-results/2024-05-30/comet-8-exec-5-runs.json new file mode 100644 index 000000000..38142151d --- /dev/null +++ b/docs/source/contributor-guide/benchmark-results/2024-05-30/comet-8-exec-5-runs.json @@ -0,0 +1,201 @@ +{ + "engine": "datafusion-comet", + "benchmark": "tpch", + "data_path": "/mnt/bigdata/tpch/sf100/", + "query_path": "../../tpch/queries", + "spark_conf": { + "spark.comet.explainFallback.enabled": "true", + "spark.jars": "file:///home/andy/git/apache/datafusion-comet/spark/target/comet-spark-spark3.4_2.12-0.1.0-SNAPSHOT.jar", + "spark.comet.cast.allowIncompatible": "true", + "spark.executor.extraClassPath": "/home/andy/git/apache/datafusion-comet/spark/target/comet-spark-spark3.4_2.12-0.1.0-SNAPSHOT.jar", + "spark.executor.memory": "8G", + "spark.comet.exec.shuffle.enabled": "true", + "spark.app.name": "DataFusion Comet Benchmark derived from TPC-H / TPC-DS", + "spark.driver.port": "36573", + "spark.sql.adaptive.coalescePartitions.enabled": "false", + "spark.app.startTime": "1716923498046", + "spark.comet.batchSize": "8192", + "spark.app.id": "app-20240528131138-0043", + "spark.serializer.objectStreamReset": "100", + "spark.app.initial.jar.urls": "spark://woody.lan:36573/jars/comet-spark-spark3.4_2.12-0.1.0-SNAPSHOT.jar", + "spark.submit.deployMode": "client", + "spark.sql.autoBroadcastJoinThreshold": "-1", + "spark.comet.exec.all.enabled": "true", + "spark.eventLog.enabled": "false", + "spark.driver.host": "woody.lan", + "spark.driver.extraJavaOptions": "-Djava.net.preferIPv6Addresses=false -XX:+IgnoreUnrecognizedVMOptions --add-opens=java.base/java.lang=ALL-UNNAMED --add-opens=java.base/java.lang.invoke=ALL-UNNAMED --add-opens=java.base/java.lang.reflect=ALL-UNNAMED --add-opens=java.base/java.io=ALL-UNNAMED --add-opens=java.base/java.net=ALL-UNNAMED --add-opens=java.base/java.nio=ALL-UNNAMED --add-opens=java.base/java.util=ALL-UNNAMED --add-opens=java.base/java.util.concurrent=ALL-UNNAMED --add-opens=java.base/java.util.concurrent.atomic=ALL-UNNAMED --add-opens=java.base/jdk.internal.ref=ALL-UNNAMED --add-opens=java.base/sun.nio.ch=ALL-UNNAMED --add-opens=java.base/sun.nio.cs=ALL-UNNAMED --add-opens=java.base/sun.security.action=ALL-UNNAMED --add-opens=java.base/sun.util.calendar=ALL-UNNAMED --add-opens=java.security.jgss/sun.security.krb5=ALL-UNNAMED -Djdk.reflect.useDirectMethodHandle=false", + "spark.sql.warehouse.dir": "file:/home/andy/git/apache/datafusion-benchmarks/runners/datafusion-comet/spark-warehouse", + "spark.shuffle.manager": "org.apache.spark.sql.comet.execution.shuffle.CometShuffleManager", + "spark.comet.exec.enabled": "true", + "spark.repl.local.jars": "file:///home/andy/git/apache/datafusion-comet/spark/target/comet-spark-spark3.4_2.12-0.1.0-SNAPSHOT.jar", + "spark.executor.id": "driver", + "spark.master": "spark://woody:7077", + "spark.executor.instances": "8", + "spark.comet.exec.shuffle.mode": "auto", + "spark.sql.extensions": "org.apache.comet.CometSparkSessionExtensions", + "spark.driver.memory": "8G", + "spark.driver.extraClassPath": "/home/andy/git/apache/datafusion-comet/spark/target/comet-spark-spark3.4_2.12-0.1.0-SNAPSHOT.jar", + "spark.rdd.compress": "True", + "spark.executor.extraJavaOptions": "-Djava.net.preferIPv6Addresses=false -XX:+IgnoreUnrecognizedVMOptions --add-opens=java.base/java.lang=ALL-UNNAMED --add-opens=java.base/java.lang.invoke=ALL-UNNAMED --add-opens=java.base/java.lang.reflect=ALL-UNNAMED --add-opens=java.base/java.io=ALL-UNNAMED --add-opens=java.base/java.net=ALL-UNNAMED --add-opens=java.base/java.nio=ALL-UNNAMED --add-opens=java.base/java.util=ALL-UNNAMED --add-opens=java.base/java.util.concurrent=ALL-UNNAMED --add-opens=java.base/java.util.concurrent.atomic=ALL-UNNAMED --add-opens=java.base/jdk.internal.ref=ALL-UNNAMED --add-opens=java.base/sun.nio.ch=ALL-UNNAMED --add-opens=java.base/sun.nio.cs=ALL-UNNAMED --add-opens=java.base/sun.security.action=ALL-UNNAMED --add-opens=java.base/sun.util.calendar=ALL-UNNAMED --add-opens=java.security.jgss/sun.security.krb5=ALL-UNNAMED -Djdk.reflect.useDirectMethodHandle=false", + "spark.cores.max": "8", + "spark.comet.enabled": "true", + "spark.app.submitTime": "1716923497738", + "spark.submit.pyFiles": "", + "spark.executor.cores": "1", + "spark.comet.parquet.io.enabled": "false" + }, + "1": [ + 32.121661901474, + 27.997092485427856, + 27.756758451461792, + 28.55236315727234, + 28.332542181015015 + ], + "2": [ + 18.269107580184937, + 16.200955629348755, + 16.194639682769775, + 16.745808839797974, + 16.59864115715027 + ], + "3": [ + 17.265466690063477, + 17.069786310195923, + 17.12887978553772, + 19.33678102493286, + 18.182055234909058 + ], + "4": [ + 8.367004156112671, + 8.172023296356201, + 8.023266077041626, + 8.350765228271484, + 8.258736610412598 + ], + "5": [ + 34.10048794746399, + 32.69314408302307, + 33.21383595466614, + 36.391114473342896, + 39.00048065185547 + ], + "6": [ + 3.1693499088287354, + 3.044705390930176, + 3.047694206237793, + 3.2817511558532715, + 3.274174928665161 + ], + "7": [ + 25.369214296340942, + 24.020941257476807, + 24.0787034034729, + 28.47402787208557, + 28.23443365097046 + ], + "8": [ + 40.06126809120178, + 39.828824281692505, + 45.250510454177856, + 44.406742572784424, + 48.98451232910156 + ], + "9": [ + 62.822797775268555, + 61.26328158378601, + 64.95581865310669, + 69.51708793640137, + 73.52380013465881 + ], + "10": [ + 20.55334782600403, + 20.546096324920654, + 20.57452392578125, + 22.84211039543152, + 23.724371671676636 + ], + "11": [ + 11.068235158920288, + 10.715423822402954, + 11.353424310684204, + 11.37632942199707, + 11.530814170837402 + ], + "12": [ + 10.264788389205933, + 8.67864990234375, + 8.845952033996582, + 8.593009233474731, + 8.540803909301758 + ], + "13": [ + 9.603406190872192, + 9.648627042770386, + 13.040799140930176, + 10.154011249542236, + 9.716034412384033 + ], + "14": [ + 6.20926308631897, + 6.0385496616363525, + 7.674488544464111, + 10.53052043914795, + 7.661675691604614 + ], + "15": [ + 11.466301918029785, + 11.473632097244263, + 11.279382228851318, + 13.291078329086304, + 12.81026816368103 + ], + "16": [ + 8.096073865890503, + 7.73410701751709, + 7.742897272109985, + 8.477537631988525, + 7.821273326873779 + ], + "17": [ + 43.69264578819275, + 43.33040428161621, + 46.291987657547, + 54.654345989227295, + 54.37124800682068 + ], + "18": [ + 27.205485105514526, + 26.785916090011597, + 27.331408262252808, + 29.946768760681152, + 28.037617444992065 + ], + "19": [ + 8.100102186203003, + 7.845783472061157, + 8.52329158782959, + 8.907397985458374, + 9.13755488395691 + ], + "20": [ + 13.09695029258728, + 12.683861255645752, + 15.612725019454956, + 13.361177206039429, + 16.614356517791748 + ], + "21": [ + 43.69623780250549, + 43.26758122444153, + 46.91650056838989, + 47.875754833221436, + 57.9763662815094 + ], + "22": [ + 4.5090577602386475, + 4.420571804046631, + 4.639787673950195, + 5.118046998977661, + 5.017346143722534 + ] +} \ No newline at end of file diff --git a/docs/source/contributor-guide/benchmark-results/2024-05-30/datafusion-python-8-cores.json b/docs/source/contributor-guide/benchmark-results/2024-05-30/datafusion-python-8-cores.json new file mode 100644 index 000000000..f032d536d --- /dev/null +++ b/docs/source/contributor-guide/benchmark-results/2024-05-30/datafusion-python-8-cores.json @@ -0,0 +1,73 @@ +{ + "engine": "datafusion-python", + "datafusion-version": "38.0.1", + "benchmark": "tpch", + "data_path": "/mnt/bigdata/tpch/sf100/", + "query_path": "../../tpch/queries/", + "1": [ + 7.410699844360352 + ], + "2": [ + 2.966364622116089 + ], + "3": [ + 3.988652467727661 + ], + "4": [ + 1.8821499347686768 + ], + "5": [ + 6.957948684692383 + ], + "6": [ + 1.779731273651123 + ], + "7": [ + 14.559604167938232 + ], + "8": [ + 7.062309265136719 + ], + "9": [ + 14.908353805541992 + ], + "10": [ + 7.73533296585083 + ], + "11": [ + 2.346423387527466 + ], + "12": [ + 2.7248904705047607 + ], + "13": [ + 6.38663387298584 + ], + "14": [ + 2.4675676822662354 + ], + "15": [ + 4.799000024795532 + ], + "16": [ + 1.9091999530792236 + ], + "17": [ + 19.230653762817383 + ], + "18": [ + 25.15683078765869 + ], + "19": [ + 4.2268781661987305 + ], + "20": [ + 8.66620659828186 + ], + "21": [ + 17.696006059646606 + ], + "22": [ + 1.3805692195892334 + ] +} \ No newline at end of file diff --git a/docs/source/contributor-guide/benchmark-results/2024-05-30/spark-8-exec-5-runs.json b/docs/source/contributor-guide/benchmark-results/2024-05-30/spark-8-exec-5-runs.json new file mode 100644 index 000000000..012b05c3a --- /dev/null +++ b/docs/source/contributor-guide/benchmark-results/2024-05-30/spark-8-exec-5-runs.json @@ -0,0 +1,184 @@ +{ + "engine": "datafusion-comet", + "benchmark": "tpch", + "data_path": "/mnt/bigdata/tpch/sf100/", + "query_path": "../../tpch/queries", + "spark_conf": { + "spark.driver.extraJavaOptions": "-Djava.net.preferIPv6Addresses=false -XX:+IgnoreUnrecognizedVMOptions --add-opens=java.base/java.lang=ALL-UNNAMED --add-opens=java.base/java.lang.invoke=ALL-UNNAMED --add-opens=java.base/java.lang.reflect=ALL-UNNAMED --add-opens=java.base/java.io=ALL-UNNAMED --add-opens=java.base/java.net=ALL-UNNAMED --add-opens=java.base/java.nio=ALL-UNNAMED --add-opens=java.base/java.util=ALL-UNNAMED --add-opens=java.base/java.util.concurrent=ALL-UNNAMED --add-opens=java.base/java.util.concurrent.atomic=ALL-UNNAMED --add-opens=java.base/jdk.internal.ref=ALL-UNNAMED --add-opens=java.base/sun.nio.ch=ALL-UNNAMED --add-opens=java.base/sun.nio.cs=ALL-UNNAMED --add-opens=java.base/sun.security.action=ALL-UNNAMED --add-opens=java.base/sun.util.calendar=ALL-UNNAMED --add-opens=java.security.jgss/sun.security.krb5=ALL-UNNAMED -Djdk.reflect.useDirectMethodHandle=false", + "spark.sql.warehouse.dir": "file:/home/andy/git/apache/datafusion-benchmarks/runners/datafusion-comet/spark-warehouse", + "spark.app.id": "app-20240528090804-0041", + "spark.app.submitTime": "1716908883258", + "spark.executor.memory": "8G", + "spark.master": "spark://woody:7077", + "spark.executor.id": "driver", + "spark.executor.instances": "8", + "spark.app.name": "DataFusion Comet Benchmark derived from TPC-H / TPC-DS", + "spark.driver.memory": "8G", + "spark.rdd.compress": "True", + "spark.executor.extraJavaOptions": "-Djava.net.preferIPv6Addresses=false -XX:+IgnoreUnrecognizedVMOptions --add-opens=java.base/java.lang=ALL-UNNAMED --add-opens=java.base/java.lang.invoke=ALL-UNNAMED --add-opens=java.base/java.lang.reflect=ALL-UNNAMED --add-opens=java.base/java.io=ALL-UNNAMED --add-opens=java.base/java.net=ALL-UNNAMED --add-opens=java.base/java.nio=ALL-UNNAMED --add-opens=java.base/java.util=ALL-UNNAMED --add-opens=java.base/java.util.concurrent=ALL-UNNAMED --add-opens=java.base/java.util.concurrent.atomic=ALL-UNNAMED --add-opens=java.base/jdk.internal.ref=ALL-UNNAMED --add-opens=java.base/sun.nio.ch=ALL-UNNAMED --add-opens=java.base/sun.nio.cs=ALL-UNNAMED --add-opens=java.base/sun.security.action=ALL-UNNAMED --add-opens=java.base/sun.util.calendar=ALL-UNNAMED --add-opens=java.security.jgss/sun.security.krb5=ALL-UNNAMED -Djdk.reflect.useDirectMethodHandle=false", + "spark.serializer.objectStreamReset": "100", + "spark.cores.max": "8", + "spark.submit.pyFiles": "", + "spark.executor.cores": "1", + "spark.submit.deployMode": "client", + "spark.sql.autoBroadcastJoinThreshold": "-1", + "spark.eventLog.enabled": "false", + "spark.app.startTime": "1716908883579", + "spark.driver.port": "33725", + "spark.driver.host": "woody.lan" + }, + "1": [ + 76.91316103935242, + 79.55859923362732, + 81.10397529602051, + 79.01998662948608, + 79.1286551952362 + ], + "2": [ + 23.977370262145996, + 22.214473247528076, + 22.686659812927246, + 22.016682386398315, + 21.766324520111084 + ], + "3": [ + 22.700742721557617, + 21.980144739151, + 21.876065969467163, + 21.661516189575195, + 21.69345998764038 + ], + "4": [ + 17.377647638320923, + 16.249598264694214, + 16.15747308731079, + 16.128843069076538, + 16.04338026046753 + ], + "5": [ + 44.38863182067871, + 45.47764492034912, + 45.76063895225525, + 45.16393995285034, + 60.848369121551514 + ], + "6": [ + 3.2041075229644775, + 2.970944881439209, + 2.891291856765747, + 2.9719409942626953, + 3.0702600479125977 + ], + "7": [ + 24.369274377822876, + 24.684266567230225, + 24.146574020385742, + 24.023175716400146, + 30.56047773361206 + ], + "8": [ + 46.46081209182739, + 45.9838604927063, + 46.341185092926025, + 45.833823919296265, + 46.61182403564453 + ], + "9": [ + 67.67960548400879, + 67.34667444229126, + 70.34601259231567, + 71.24095153808594, + 84.38811421394348 + ], + "10": [ + 19.16477870941162, + 19.081010580062866, + 19.501060009002686, + 19.165698528289795, + 20.216782331466675 + ], + "11": [ + 17.158706426620483, + 17.05184030532837, + 17.714542150497437, + 17.004602909088135, + 17.700096130371094 + ], + "12": [ + 11.654477834701538, + 11.805298805236816, + 11.822469234466553, + 12.79678750038147, + 13.64478850364685 + ], + "13": [ + 20.430822372436523, + 20.18759250640869, + 21.26596975326538, + 21.234288454055786, + 20.189200162887573 + ], + "14": [ + 5.60215950012207, + 5.160705089569092, + 5.080057382583618, + 4.937625408172607, + 5.853632688522339 + ], + "15": [ + 14.17775845527649, + 13.898571729660034, + 14.215840578079224, + 14.316090106964111, + 14.356236457824707 + ], + "16": [ + 6.252386808395386, + 6.010213375091553, + 6.054978370666504, + 5.886059522628784, + 5.923115253448486 + ], + "17": [ + 71.41593313217163, + 70.25399804115295, + 72.07622528076172, + 72.27566242218018, + 72.20579051971436 + ], + "18": [ + 65.72738265991211, + 65.47461080551147, + 67.14260482788086, + 65.95489883422852, + 69.51795554161072 + ], + "19": [ + 7.1520891189575195, + 6.516514301300049, + 6.580992698669434, + 6.486274242401123, + 6.418147087097168 + ], + "20": [ + 12.619760036468506, + 12.235978126525879, + 12.116347551345825, + 12.161245584487915, + 12.30910348892212 + ], + "21": [ + 60.795483350753784, + 60.484593629837036, + 61.27316427230835, + 60.475560426712036, + 81.21473670005798 + ], + "22": [ + 8.926804065704346, + 8.113754034042358, + 8.029133796691895, + 7.99291467666626, + 8.439452648162842 + ] +} \ No newline at end of file diff --git a/docs/source/contributor-guide/benchmarking.md b/docs/source/contributor-guide/benchmarking.md index 502b35c29..3e9a61efb 100644 --- a/docs/source/contributor-guide/benchmarking.md +++ b/docs/source/contributor-guide/benchmarking.md @@ -19,44 +19,87 @@ under the License. # Comet Benchmarking Guide -To track progress on performance, we regularly run benchmarks derived from TPC-H and TPC-DS. Benchmarking scripts are -available in the [DataFusion Benchmarks](https://github.com/apache/datafusion-benchmarks) GitHub repository. +To track progress on performance, we regularly run benchmarks derived from TPC-H and TPC-DS. Data generation and +benchmarking documentation and scripts are available in the [DataFusion Benchmarks](https://github.com/apache/datafusion-benchmarks) GitHub repository. -Here is an example command for running the benchmarks. This command will need to be adapted based on the Spark -environment and location of data files. +Here are example commands for running the benchmarks against a Spark cluster. This command will need to be +adapted based on the Spark environment and location of data files. -This command assumes that `datafusion-benchmarks` is checked out in a parallel directory to `datafusion-comet`. +These commands are intended to be run from the `runners/datafusion-comet` directory in the `datafusion-benchmarks` +repository. + +## Running Benchmarks Against Apache Spark ```shell -$SPARK_HOME/bin/spark-submit \ - --master "local[*]" \ - --conf spark.driver.memory=8G \ - --conf spark.executor.memory=64G \ - --conf spark.executor.cores=16 \ - --conf spark.cores.max=16 \ - --conf spark.eventLog.enabled=true \ - --conf spark.sql.autoBroadcastJoinThreshold=-1 \ - --jars $COMET_JAR \ - --conf spark.driver.extraClassPath=$COMET_JAR \ - --conf spark.executor.extraClassPath=$COMET_JAR \ - --conf spark.sql.extensions=org.apache.comet.CometSparkSessionExtensions \ - --conf spark.comet.enabled=true \ - --conf spark.comet.exec.enabled=true \ - --conf spark.comet.exec.all.enabled=true \ - --conf spark.comet.cast.allowIncompatible=true \ - --conf spark.comet.explainFallback.enabled=true \ - --conf spark.comet.parquet.io.enabled=false \ - --conf spark.comet.batchSize=8192 \ - --conf spark.comet.columnar.shuffle.enabled=false \ - --conf spark.comet.exec.shuffle.enabled=true \ - --conf spark.shuffle.manager=org.apache.spark.sql.comet.execution.shuffle.CometShuffleManager \ - --conf spark.sql.adaptive.coalescePartitions.enabled=false \ - --conf spark.comet.shuffle.enforceMode.enabled=true \ - ../datafusion-benchmarks/runners/datafusion-comet/tpcbench.py \ - --benchmark tpch \ - --data /mnt/bigdata/tpch/sf100-parquet/ \ - --queries ../datafusion-benchmarks/tpch/queries +$SPARK_HOME/bin/spark-submit \ + --master $SPARK_MASTER \ + --conf spark.driver.memory=8G \ + --conf spark.executor.memory=32G \ + --conf spark.executor.cores=8 \ + --conf spark.cores.max=8 \ + --conf spark.sql.autoBroadcastJoinThreshold=-1 \ + tpcbench.py \ + --benchmark tpch \ + --data /mnt/bigdata/tpch/sf100/ \ + --queries ../../tpch/queries \ + --iterations 5 ``` -Comet performance can be compared to regular Spark performance by running the benchmark twice, once with -`spark.comet.enabled` set to `true` and once with it set to `false`. \ No newline at end of file +## Running Benchmarks Against Apache Spark with Apache DataFusion Comet Enabled + +```shell +$SPARK_HOME/bin/spark-submit \ + --master $SPARK_MASTER \ + --conf spark.driver.memory=8G \ + --conf spark.executor.memory=64G \ + --conf spark.executor.cores=8 \ + --conf spark.cores.max=8 \ + --conf spark.sql.autoBroadcastJoinThreshold=-1 \ + --jars $COMET_JAR \ + --conf spark.driver.extraClassPath=$COMET_JAR \ + --conf spark.executor.extraClassPath=$COMET_JAR \ + --conf spark.sql.extensions=org.apache.comet.CometSparkSessionExtensions \ + --conf spark.comet.enabled=true \ + --conf spark.comet.exec.enabled=true \ + --conf spark.comet.exec.all.enabled=true \ + --conf spark.comet.cast.allowIncompatible=true \ + --conf spark.comet.explainFallback.enabled=true \ + --conf spark.comet.parquet.io.enabled=false \ + --conf spark.comet.batchSize=8192 \ + --conf spark.comet.exec.shuffle.enabled=true \ + --conf spark.comet.exec.shuffle.mode=auto \ + --conf spark.shuffle.manager=org.apache.spark.sql.comet.execution.shuffle.CometShuffleManager \ + --conf spark.sql.adaptive.coalescePartitions.enabled=false \ + tpcbench.py \ + --benchmark tpch \ + --data /mnt/bigdata/tpch/sf100/ \ + --queries ../../tpch/queries \ + --iterations 5 +``` + +## Current Performance + +Comet is not yet achieving full DataFusion speeds in all cases, but with future work we aim to provide a 2x-4x speedup +for many use cases. + +The following benchmarks were performed on a Linux workstation with PCIe 5, AMD 7950X CPU (16 cores), 128 GB RAM, and +data stored locally on NVMe storage. Performance characteristics will vary in different environments and we encourage +you to run these benchmarks in your own environments. + +![](../../_static/images/tpch_allqueries.png) + +Here is a breakdown showing relative performance of Spark, Comet, and DataFusion for each TPC-H query. + +![](../../_static/images/tpch_queries_compare.png) + +The following chart shows how much Comet currently accelerates each query from the benchmark. Performance optimization +is an ongoing task, and we welcome contributions from the community to help achieve even greater speedups in the future. + +![](../../_static/images/tpch_queries_speedup.png) + +The raw results of these benchmarks in JSON format is available here: + +- [Spark](./benchmark-results/2024-05-30/spark-8-exec-5-runs.json) +- [Comet](./benchmark-results/2024-05-30/comet-8-exec-5-runs.json) +- [DataFusion](./benchmark-results/2024-05-30/datafusion-python-8-cores.json) + From 6ea5a427eab1e93a8fc10e7cf76a6b14ba2f0c18 Mon Sep 17 00:00:00 2001 From: KAZUYUKI TANIMURA <ktanimura@apple.com> Date: Fri, 31 May 2024 15:11:35 -0700 Subject: [PATCH 12/19] test: fix ClassNotFoundException for Hive tests (#499) ## Rationale for this change To fix `java.lang.ClassNotFoundException: org.apache.spark.CometSparkSessionExtensions` in Hive tests ## What changes are included in this PR? fixed diffs ## How are these changes tested? existing tests --- dev/diffs/3.4.2.diff | 2 +- dev/diffs/3.4.3.diff | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/dev/diffs/3.4.2.diff b/dev/diffs/3.4.2.diff index 19bf6dd41..cd02970da 100644 --- a/dev/diffs/3.4.2.diff +++ b/dev/diffs/3.4.2.diff @@ -2526,7 +2526,7 @@ index 07361cfdce9..25b0dc3ef7e 100644 + val v = System.getenv("ENABLE_COMET") + if (v != null && v.toBoolean) { + conf -+ .set("spark.sql.extensions", "org.apache.spark.CometSparkSessionExtensions") ++ .set("spark.sql.extensions", "org.apache.comet.CometSparkSessionExtensions") + .set("spark.comet.enabled", "true") + + val v = System.getenv("ENABLE_COMET_SCAN_ONLY") diff --git a/dev/diffs/3.4.3.diff b/dev/diffs/3.4.3.diff index ae46e916b..c6d10c2ec 100644 --- a/dev/diffs/3.4.3.diff +++ b/dev/diffs/3.4.3.diff @@ -2535,7 +2535,7 @@ index 07361cfdce9..25b0dc3ef7e 100644 + val v = System.getenv("ENABLE_COMET") + if (v != null && v.toBoolean) { + conf -+ .set("spark.sql.extensions", "org.apache.spark.CometSparkSessionExtensions") ++ .set("spark.sql.extensions", "org.apache.comet.CometSparkSessionExtensions") + .set("spark.comet.enabled", "true") + + val v = System.getenv("ENABLE_COMET_SCAN_ONLY") From 24781fb7b3966f787cf72c97f42e2613f24fb2ac Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh <viirya@gmail.com> Date: Sat, 1 Jun 2024 14:05:24 -0700 Subject: [PATCH 13/19] fix: Incorrect input schema when preparing result expressions for HashAggregation (#501) --- core/src/execution/datafusion/planner.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/execution/datafusion/planner.rs b/core/src/execution/datafusion/planner.rs index 7a37e3aae..a0b75de3f 100644 --- a/core/src/execution/datafusion/planner.rs +++ b/core/src/execution/datafusion/planner.rs @@ -779,7 +779,7 @@ impl PhysicalPlanner { .iter() .enumerate() .map(|(idx, expr)| { - self.create_expr(expr, child.schema()) + self.create_expr(expr, aggregate.schema()) .map(|r| (r, format!("col_{}", idx))) }) .collect(); From c79bd5cc71be81d67309467a455026324aa05e44 Mon Sep 17 00:00:00 2001 From: advancedxy <xianjin@apache.org> Date: Mon, 3 Jun 2024 23:57:53 +0800 Subject: [PATCH 14/19] feat: Add xxhash64 function support (#424) * feat: Add xxhash64 function support * Update related docs * Update core/src/execution/datafusion/spark_hash.rs Co-authored-by: Liang-Chi Hsieh <viirya@gmail.com> * Update QueriesList results --------- Co-authored-by: Liang-Chi Hsieh <viirya@gmail.com> Co-authored-by: Parth Chandra <parthc@apple.com> --- core/Cargo.lock | 2 + core/Cargo.toml | 1 + .../datafusion/expressions/scalar_funcs.rs | 51 +- .../execution/datafusion/shuffle_writer.rs | 4 +- core/src/execution/datafusion/spark_hash.rs | 519 ++++++++++++---- docs/source/user-guide/expressions.md | 5 + .../CometTPCDSQueriesList-results.txt | 583 +++++++++++------- .../CometTPCHQueriesList-results.txt | 113 ++-- .../apache/comet/serde/QueryPlanSerde.scala | 15 + .../apache/comet/CometExpressionSuite.scala | 17 +- 10 files changed, 862 insertions(+), 448 deletions(-) diff --git a/core/Cargo.lock b/core/Cargo.lock index 52f105591..105bcaf7c 100644 --- a/core/Cargo.lock +++ b/core/Cargo.lock @@ -637,6 +637,7 @@ dependencies = [ "thrift 0.17.0", "tokio", "tokio-stream", + "twox-hash", "unicode-segmentation", "zstd", ] @@ -2823,6 +2824,7 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "97fee6b57c6a41524a810daee9286c02d7752c4253064d0b05472833a438f675" dependencies = [ "cfg-if", + "rand", "static_assertions", ] diff --git a/core/Cargo.toml b/core/Cargo.toml index 5e3e0ee74..6a179a6ee 100644 --- a/core/Cargo.toml +++ b/core/Cargo.toml @@ -75,6 +75,7 @@ once_cell = "1.18.0" regex = "1.9.6" crc32fast = "1.3.2" simd-adler32 = "0.3.7" +twox-hash = "1.6.3" [build-dependencies] prost-build = "0.9.0" diff --git a/core/src/execution/datafusion/expressions/scalar_funcs.rs b/core/src/execution/datafusion/expressions/scalar_funcs.rs index 8c5e1f391..0f254004b 100644 --- a/core/src/execution/datafusion/expressions/scalar_funcs.rs +++ b/core/src/execution/datafusion/expressions/scalar_funcs.rs @@ -23,7 +23,7 @@ use std::{ sync::Arc, }; -use crate::execution::datafusion::spark_hash::create_hashes; +use crate::execution::datafusion::spark_hash::{create_murmur3_hashes, create_xxhash64_hashes}; use arrow::{ array::{ ArrayRef, AsArray, Decimal128Builder, Float32Array, Float64Array, GenericStringArray, @@ -119,6 +119,10 @@ pub fn create_comet_physical_fun( let func = Arc::new(spark_murmur3_hash); make_comet_scalar_udf!("murmur3_hash", func, without data_type) } + "xxhash64" => { + let func = Arc::new(spark_xxhash64); + make_comet_scalar_udf!("xxhash64", func, without data_type) + } sha if sha2_functions.contains(&sha) => { // Spark requires hex string as the result of sha2 functions, we have to wrap the // result of digest functions as hex string @@ -653,7 +657,7 @@ fn spark_murmur3_hash(args: &[ColumnarValue]) -> Result<ColumnarValue, DataFusio } }) .collect::<Vec<ArrayRef>>(); - create_hashes(&arrays, &mut hashes)?; + create_murmur3_hashes(&arrays, &mut hashes)?; if num_rows == 1 { Ok(ColumnarValue::Scalar(ScalarValue::Int32(Some( hashes[0] as i32, @@ -672,6 +676,49 @@ fn spark_murmur3_hash(args: &[ColumnarValue]) -> Result<ColumnarValue, DataFusio } } +fn spark_xxhash64(args: &[ColumnarValue]) -> Result<ColumnarValue, DataFusionError> { + let length = args.len(); + let seed = &args[length - 1]; + match seed { + ColumnarValue::Scalar(ScalarValue::Int64(Some(seed))) => { + // iterate over the arguments to find out the length of the array + let num_rows = args[0..args.len() - 1] + .iter() + .find_map(|arg| match arg { + ColumnarValue::Array(array) => Some(array.len()), + ColumnarValue::Scalar(_) => None, + }) + .unwrap_or(1); + let mut hashes: Vec<u64> = vec![0_u64; num_rows]; + hashes.fill(*seed as u64); + let arrays = args[0..args.len() - 1] + .iter() + .map(|arg| match arg { + ColumnarValue::Array(array) => array.clone(), + ColumnarValue::Scalar(scalar) => { + scalar.clone().to_array_of_size(num_rows).unwrap() + } + }) + .collect::<Vec<ArrayRef>>(); + create_xxhash64_hashes(&arrays, &mut hashes)?; + if num_rows == 1 { + Ok(ColumnarValue::Scalar(ScalarValue::Int64(Some( + hashes[0] as i64, + )))) + } else { + let hashes: Vec<i64> = hashes.into_iter().map(|x| x as i64).collect(); + Ok(ColumnarValue::Array(Arc::new(Int64Array::from(hashes)))) + } + } + _ => { + internal_err!( + "The seed of function xxhash64 must be an Int64 scalar value, but got: {:?}.", + seed + ) + } + } +} + #[inline] fn hex_encode<T: AsRef<[u8]>>(data: T) -> String { let mut s = String::with_capacity(data.as_ref().len() * 2); diff --git a/core/src/execution/datafusion/shuffle_writer.rs b/core/src/execution/datafusion/shuffle_writer.rs index 3b92abbde..967340979 100644 --- a/core/src/execution/datafusion/shuffle_writer.rs +++ b/core/src/execution/datafusion/shuffle_writer.rs @@ -62,7 +62,7 @@ use tokio::task; use crate::{ common::bit::ceil, errors::{CometError, CometResult}, - execution::datafusion::spark_hash::{create_hashes, pmod}, + execution::datafusion::spark_hash::{create_murmur3_hashes, pmod}, }; /// The shuffle writer operator maps each input partition to M output partitions based on a @@ -673,7 +673,7 @@ impl ShuffleRepartitioner { // Hash arrays and compute buckets based on number of partitions let partition_ids = &mut self.partition_ids[..arrays[0].len()]; - create_hashes(&arrays, hashes_buf)? + create_murmur3_hashes(&arrays, hashes_buf)? .iter() .enumerate() .for_each(|(idx, hash)| { diff --git a/core/src/execution/datafusion/spark_hash.rs b/core/src/execution/datafusion/spark_hash.rs index 6d25a72f6..15d3a5021 100644 --- a/core/src/execution/datafusion/spark_hash.rs +++ b/core/src/execution/datafusion/spark_hash.rs @@ -21,7 +21,8 @@ use arrow::{ compute::take, datatypes::{ArrowNativeTypeOp, UInt16Type, UInt32Type, UInt64Type, UInt8Type}, }; -use std::sync::Arc; +use std::{hash::Hasher, sync::Arc}; +use twox_hash::XxHash64; use datafusion::{ arrow::{ @@ -98,17 +99,25 @@ pub(crate) fn spark_compatible_murmur3_hash<T: AsRef<[u8]>>(data: T, seed: u32) } } +#[inline] +pub(crate) fn spark_compatible_xxhash64<T: AsRef<[u8]>>(data: T, seed: u64) -> u64 { + // TODO: Rewrite with a stateless hasher to reduce stack allocation? + let mut hasher = XxHash64::with_seed(seed); + hasher.write(data.as_ref()); + hasher.finish() +} + macro_rules! hash_array { - ($array_type: ident, $column: ident, $hashes: ident) => { + ($array_type: ident, $column: ident, $hashes: ident, $hash_method: ident) => { let array = $column.as_any().downcast_ref::<$array_type>().unwrap(); if array.null_count() == 0 { for (i, hash) in $hashes.iter_mut().enumerate() { - *hash = spark_compatible_murmur3_hash(&array.value(i), *hash); + *hash = $hash_method(&array.value(i), *hash); } } else { for (i, hash) in $hashes.iter_mut().enumerate() { if !array.is_null(i) { - *hash = spark_compatible_murmur3_hash(&array.value(i), *hash); + *hash = $hash_method(&array.value(i), *hash); } } } @@ -116,22 +125,17 @@ macro_rules! hash_array { } macro_rules! hash_array_boolean { - ($array_type: ident, $column: ident, $hash_input_type: ident, $hashes: ident) => { + ($array_type: ident, $column: ident, $hash_input_type: ident, $hashes: ident, $hash_method: ident) => { let array = $column.as_any().downcast_ref::<$array_type>().unwrap(); if array.null_count() == 0 { for (i, hash) in $hashes.iter_mut().enumerate() { - *hash = spark_compatible_murmur3_hash( - $hash_input_type::from(array.value(i)).to_le_bytes(), - *hash, - ); + *hash = $hash_method($hash_input_type::from(array.value(i)).to_le_bytes(), *hash); } } else { for (i, hash) in $hashes.iter_mut().enumerate() { if !array.is_null(i) { - *hash = spark_compatible_murmur3_hash( - $hash_input_type::from(array.value(i)).to_le_bytes(), - *hash, - ); + *hash = + $hash_method($hash_input_type::from(array.value(i)).to_le_bytes(), *hash); } } } @@ -139,18 +143,18 @@ macro_rules! hash_array_boolean { } macro_rules! hash_array_primitive { - ($array_type: ident, $column: ident, $ty: ident, $hashes: ident) => { + ($array_type: ident, $column: ident, $ty: ident, $hashes: ident, $hash_method: ident) => { let array = $column.as_any().downcast_ref::<$array_type>().unwrap(); let values = array.values(); if array.null_count() == 0 { for (hash, value) in $hashes.iter_mut().zip(values.iter()) { - *hash = spark_compatible_murmur3_hash((*value as $ty).to_le_bytes(), *hash); + *hash = $hash_method((*value as $ty).to_le_bytes(), *hash); } } else { for (i, (hash, value)) in $hashes.iter_mut().zip(values.iter()).enumerate() { if !array.is_null(i) { - *hash = spark_compatible_murmur3_hash((*value as $ty).to_le_bytes(), *hash); + *hash = $hash_method((*value as $ty).to_le_bytes(), *hash); } } } @@ -158,7 +162,7 @@ macro_rules! hash_array_primitive { } macro_rules! hash_array_primitive_float { - ($array_type: ident, $column: ident, $ty: ident, $ty2: ident, $hashes: ident) => { + ($array_type: ident, $column: ident, $ty: ident, $ty2: ident, $hashes: ident, $hash_method: ident) => { let array = $column.as_any().downcast_ref::<$array_type>().unwrap(); let values = array.values(); @@ -166,9 +170,9 @@ macro_rules! hash_array_primitive_float { for (hash, value) in $hashes.iter_mut().zip(values.iter()) { // Spark uses 0 as hash for -0.0, see `Murmur3Hash` expression. if *value == 0.0 && value.is_sign_negative() { - *hash = spark_compatible_murmur3_hash((0 as $ty2).to_le_bytes(), *hash); + *hash = $hash_method((0 as $ty2).to_le_bytes(), *hash); } else { - *hash = spark_compatible_murmur3_hash((*value as $ty).to_le_bytes(), *hash); + *hash = $hash_method((*value as $ty).to_le_bytes(), *hash); } } } else { @@ -176,9 +180,9 @@ macro_rules! hash_array_primitive_float { if !array.is_null(i) { // Spark uses 0 as hash for -0.0, see `Murmur3Hash` expression. if *value == 0.0 && value.is_sign_negative() { - *hash = spark_compatible_murmur3_hash((0 as $ty2).to_le_bytes(), *hash); + *hash = $hash_method((0 as $ty2).to_le_bytes(), *hash); } else { - *hash = spark_compatible_murmur3_hash((*value as $ty).to_le_bytes(), *hash); + *hash = $hash_method((*value as $ty).to_le_bytes(), *hash); } } } @@ -187,17 +191,17 @@ macro_rules! hash_array_primitive_float { } macro_rules! hash_array_decimal { - ($array_type: ident, $column: ident, $hashes: ident) => { + ($array_type:ident, $column: ident, $hashes: ident, $hash_method: ident) => { let array = $column.as_any().downcast_ref::<$array_type>().unwrap(); if array.null_count() == 0 { for (i, hash) in $hashes.iter_mut().enumerate() { - *hash = spark_compatible_murmur3_hash(array.value(i).to_le_bytes(), *hash); + *hash = $hash_method(array.value(i).to_le_bytes(), *hash); } } else { for (i, hash) in $hashes.iter_mut().enumerate() { if !array.is_null(i) { - *hash = spark_compatible_murmur3_hash(array.value(i).to_le_bytes(), *hash); + *hash = $hash_method(array.value(i).to_le_bytes(), *hash); } } } @@ -214,7 +218,7 @@ fn create_hashes_dictionary<K: ArrowDictionaryKeyType>( if !first_col { // unpack the dictionary array as each row may have a different hash input let unpacked = take(dict_array.values().as_ref(), dict_array.keys(), None)?; - create_hashes(&[unpacked], hashes_buffer)?; + create_murmur3_hashes(&[unpacked], hashes_buffer)?; } else { // For the first column, hash each dictionary value once, and then use // that computed hash for each key value to avoid a potentially @@ -222,7 +226,42 @@ fn create_hashes_dictionary<K: ArrowDictionaryKeyType>( let dict_values = Arc::clone(dict_array.values()); // same initial seed as Spark let mut dict_hashes = vec![42; dict_values.len()]; - create_hashes(&[dict_values], &mut dict_hashes)?; + create_murmur3_hashes(&[dict_values], &mut dict_hashes)?; + for (hash, key) in hashes_buffer.iter_mut().zip(dict_array.keys().iter()) { + if let Some(key) = key { + let idx = key.to_usize().ok_or_else(|| { + DataFusionError::Internal(format!( + "Can not convert key value {:?} to usize in dictionary of type {:?}", + key, + dict_array.data_type() + )) + })?; + *hash = dict_hashes[idx] + } // no update for Null, consistent with other hashes + } + } + Ok(()) +} + +// Hash the values in a dictionary array using xxhash64 +fn create_xxhash64_hashes_dictionary<K: ArrowDictionaryKeyType>( + array: &ArrayRef, + hashes_buffer: &mut [u64], + first_col: bool, +) -> Result<()> { + let dict_array = array.as_any().downcast_ref::<DictionaryArray<K>>().unwrap(); + if !first_col { + let unpacked = take(dict_array.values().as_ref(), dict_array.keys(), None)?; + create_xxhash64_hashes(&[unpacked], hashes_buffer)?; + } else { + // Hash each dictionary value once, and then use that computed + // hash for each key value to avoid a potentially expensive + // redundant hashing for large dictionary elements (e.g. strings) + let dict_values = Arc::clone(dict_array.values()); + // same initial seed as Spark + let mut dict_hashes = vec![42u64; dict_values.len()]; + create_xxhash64_hashes(&[dict_values], &mut dict_hashes)?; + for (hash, key) in hashes_buffer.iter_mut().zip(dict_array.keys().iter()) { if let Some(key) = key { let idx = key.to_usize().ok_or_else(|| { @@ -244,111 +283,214 @@ fn create_hashes_dictionary<K: ArrowDictionaryKeyType>( /// /// The number of rows to hash is determined by `hashes_buffer.len()`. /// `hashes_buffer` should be pre-sized appropriately -pub fn create_hashes<'a>( - arrays: &[ArrayRef], - hashes_buffer: &'a mut [u32], -) -> Result<&'a mut [u32]> { - for (i, col) in arrays.iter().enumerate() { - let first_col = i == 0; - match col.data_type() { - DataType::Boolean => { - hash_array_boolean!(BooleanArray, col, i32, hashes_buffer); - } - DataType::Int8 => { - hash_array_primitive!(Int8Array, col, i32, hashes_buffer); - } - DataType::Int16 => { - hash_array_primitive!(Int16Array, col, i32, hashes_buffer); - } - DataType::Int32 => { - hash_array_primitive!(Int32Array, col, i32, hashes_buffer); - } - DataType::Int64 => { - hash_array_primitive!(Int64Array, col, i64, hashes_buffer); - } - DataType::Float32 => { - hash_array_primitive_float!(Float32Array, col, f32, i32, hashes_buffer); - } - DataType::Float64 => { - hash_array_primitive_float!(Float64Array, col, f64, i64, hashes_buffer); - } - DataType::Timestamp(TimeUnit::Second, _) => { - hash_array_primitive!(TimestampSecondArray, col, i64, hashes_buffer); - } - DataType::Timestamp(TimeUnit::Millisecond, _) => { - hash_array_primitive!(TimestampMillisecondArray, col, i64, hashes_buffer); - } - DataType::Timestamp(TimeUnit::Microsecond, _) => { - hash_array_primitive!(TimestampMicrosecondArray, col, i64, hashes_buffer); - } - DataType::Timestamp(TimeUnit::Nanosecond, _) => { - hash_array_primitive!(TimestampNanosecondArray, col, i64, hashes_buffer); - } - DataType::Date32 => { - hash_array_primitive!(Date32Array, col, i32, hashes_buffer); - } - DataType::Date64 => { - hash_array_primitive!(Date64Array, col, i64, hashes_buffer); - } - DataType::Utf8 => { - hash_array!(StringArray, col, hashes_buffer); - } - DataType::LargeUtf8 => { - hash_array!(LargeStringArray, col, hashes_buffer); - } - DataType::Binary => { - hash_array!(BinaryArray, col, hashes_buffer); - } - DataType::LargeBinary => { - hash_array!(LargeBinaryArray, col, hashes_buffer); - } - DataType::FixedSizeBinary(_) => { - hash_array!(FixedSizeBinaryArray, col, hashes_buffer); - } - DataType::Decimal128(_, _) => { - hash_array_decimal!(Decimal128Array, col, hashes_buffer); - } - DataType::Dictionary(index_type, _) => match **index_type { +/// +/// `hash_method` is the hash function to use. +/// `create_dictionary_hash_method` is the function to create hashes for dictionary arrays input. +macro_rules! create_hashes_internal { + ($arrays: ident, $hashes_buffer: ident, $hash_method: ident, $create_dictionary_hash_method: ident) => { + for (i, col) in $arrays.iter().enumerate() { + let first_col = i == 0; + match col.data_type() { + DataType::Boolean => { + hash_array_boolean!(BooleanArray, col, i32, $hashes_buffer, $hash_method); + } DataType::Int8 => { - create_hashes_dictionary::<Int8Type>(col, hashes_buffer, first_col)?; + hash_array_primitive!(Int8Array, col, i32, $hashes_buffer, $hash_method); } DataType::Int16 => { - create_hashes_dictionary::<Int16Type>(col, hashes_buffer, first_col)?; + hash_array_primitive!(Int16Array, col, i32, $hashes_buffer, $hash_method); } DataType::Int32 => { - create_hashes_dictionary::<Int32Type>(col, hashes_buffer, first_col)?; + hash_array_primitive!(Int32Array, col, i32, $hashes_buffer, $hash_method); } DataType::Int64 => { - create_hashes_dictionary::<Int64Type>(col, hashes_buffer, first_col)?; + hash_array_primitive!(Int64Array, col, i64, $hashes_buffer, $hash_method); } - DataType::UInt8 => { - create_hashes_dictionary::<UInt8Type>(col, hashes_buffer, first_col)?; + DataType::Float32 => { + hash_array_primitive_float!( + Float32Array, + col, + f32, + i32, + $hashes_buffer, + $hash_method + ); } - DataType::UInt16 => { - create_hashes_dictionary::<UInt16Type>(col, hashes_buffer, first_col)?; + DataType::Float64 => { + hash_array_primitive_float!( + Float64Array, + col, + f64, + i64, + $hashes_buffer, + $hash_method + ); } - DataType::UInt32 => { - create_hashes_dictionary::<UInt32Type>(col, hashes_buffer, first_col)?; + DataType::Timestamp(TimeUnit::Second, _) => { + hash_array_primitive!( + TimestampSecondArray, + col, + i64, + $hashes_buffer, + $hash_method + ); + } + DataType::Timestamp(TimeUnit::Millisecond, _) => { + hash_array_primitive!( + TimestampMillisecondArray, + col, + i64, + $hashes_buffer, + $hash_method + ); + } + DataType::Timestamp(TimeUnit::Microsecond, _) => { + hash_array_primitive!( + TimestampMicrosecondArray, + col, + i64, + $hashes_buffer, + $hash_method + ); + } + DataType::Timestamp(TimeUnit::Nanosecond, _) => { + hash_array_primitive!( + TimestampNanosecondArray, + col, + i64, + $hashes_buffer, + $hash_method + ); } - DataType::UInt64 => { - create_hashes_dictionary::<UInt64Type>(col, hashes_buffer, first_col)?; + DataType::Date32 => { + hash_array_primitive!(Date32Array, col, i32, $hashes_buffer, $hash_method); } + DataType::Date64 => { + hash_array_primitive!(Date64Array, col, i64, $hashes_buffer, $hash_method); + } + DataType::Utf8 => { + hash_array!(StringArray, col, $hashes_buffer, $hash_method); + } + DataType::LargeUtf8 => { + hash_array!(LargeStringArray, col, $hashes_buffer, $hash_method); + } + DataType::Binary => { + hash_array!(BinaryArray, col, $hashes_buffer, $hash_method); + } + DataType::LargeBinary => { + hash_array!(LargeBinaryArray, col, $hashes_buffer, $hash_method); + } + DataType::FixedSizeBinary(_) => { + hash_array!(FixedSizeBinaryArray, col, $hashes_buffer, $hash_method); + } + DataType::Decimal128(_, _) => { + hash_array_decimal!(Decimal128Array, col, $hashes_buffer, $hash_method); + } + DataType::Dictionary(index_type, _) => match **index_type { + DataType::Int8 => { + $create_dictionary_hash_method::<Int8Type>(col, $hashes_buffer, first_col)?; + } + DataType::Int16 => { + $create_dictionary_hash_method::<Int16Type>( + col, + $hashes_buffer, + first_col, + )?; + } + DataType::Int32 => { + $create_dictionary_hash_method::<Int32Type>( + col, + $hashes_buffer, + first_col, + )?; + } + DataType::Int64 => { + $create_dictionary_hash_method::<Int64Type>( + col, + $hashes_buffer, + first_col, + )?; + } + DataType::UInt8 => { + $create_dictionary_hash_method::<UInt8Type>( + col, + $hashes_buffer, + first_col, + )?; + } + DataType::UInt16 => { + $create_dictionary_hash_method::<UInt16Type>( + col, + $hashes_buffer, + first_col, + )?; + } + DataType::UInt32 => { + $create_dictionary_hash_method::<UInt32Type>( + col, + $hashes_buffer, + first_col, + )?; + } + DataType::UInt64 => { + $create_dictionary_hash_method::<UInt64Type>( + col, + $hashes_buffer, + first_col, + )?; + } + _ => { + return Err(DataFusionError::Internal(format!( + "Unsupported dictionary type in hasher hashing: {}", + col.data_type(), + ))) + } + }, _ => { + // This is internal because we should have caught this before. return Err(DataFusionError::Internal(format!( - "Unsupported dictionary type in hasher hashing: {}", - col.data_type(), - ))) + "Unsupported data type in hasher: {}", + col.data_type() + ))); } - }, - _ => { - // This is internal because we should have caught this before. - return Err(DataFusionError::Internal(format!( - "Unsupported data type in hasher: {}", - col.data_type() - ))); } } - } + }; +} + +/// Creates hash values for every row, based on the values in the +/// columns. +/// +/// The number of rows to hash is determined by `hashes_buffer.len()`. +/// `hashes_buffer` should be pre-sized appropriately +pub(crate) fn create_murmur3_hashes<'a>( + arrays: &[ArrayRef], + hashes_buffer: &'a mut [u32], +) -> Result<&'a mut [u32]> { + create_hashes_internal!( + arrays, + hashes_buffer, + spark_compatible_murmur3_hash, + create_hashes_dictionary + ); + Ok(hashes_buffer) +} + +/// Creates xxhash64 hash values for every row, based on the values in the +/// columns. +/// +/// The number of rows to hash is determined by `hashes_buffer.len()`. +/// `hashes_buffer` should be pre-sized appropriately +pub(crate) fn create_xxhash64_hashes<'a>( + arrays: &[ArrayRef], + hashes_buffer: &'a mut [u64], +) -> Result<&'a mut [u64]> { + create_hashes_internal!( + arrays, + hashes_buffer, + spark_compatible_xxhash64, + create_xxhash64_hashes_dictionary + ); Ok(hashes_buffer) } @@ -365,38 +507,61 @@ mod tests { use arrow::array::{Float32Array, Float64Array}; use std::sync::Arc; - use crate::execution::datafusion::spark_hash::{create_hashes, pmod}; + use crate::execution::datafusion::spark_hash::{ + create_murmur3_hashes, create_xxhash64_hashes, pmod, + }; use datafusion::arrow::array::{ArrayRef, Int32Array, Int64Array, Int8Array, StringArray}; macro_rules! test_hashes_internal { - ($input: expr, $len: expr, $expected: expr) => { - let i = $input as ArrayRef; - let mut hashes = vec![42; $len]; - create_hashes(&[i], &mut hashes).unwrap(); + ($hash_method: ident, $input: expr, $initial_seeds: expr, $expected: expr) => { + let i = $input; + let mut hashes = $initial_seeds.clone(); + $hash_method(&[i], &mut hashes).unwrap(); assert_eq!(hashes, $expected); }; } + macro_rules! test_hashes_with_nulls { + ($method: ident, $t: ty, $values: ident, $expected: ident, $seed_type: ty) => { + // copied before inserting nulls + let mut input_with_nulls = $values.clone(); + let mut expected_with_nulls = $expected.clone(); + // test before inserting nulls + let len = $values.len(); + let initial_seeds = vec![42 as $seed_type; len]; + let i = Arc::new(<$t>::from($values)) as ArrayRef; + test_hashes_internal!($method, i, initial_seeds, $expected); + + // test with nulls + let median = len / 2; + input_with_nulls.insert(0, None); + input_with_nulls.insert(median, None); + expected_with_nulls.insert(0, 42 as $seed_type); + expected_with_nulls.insert(median, 42 as $seed_type); + let len_with_nulls = len + 2; + let initial_seeds_with_nulls = vec![42 as $seed_type; len_with_nulls]; + let nullable_input = Arc::new(<$t>::from(input_with_nulls)) as ArrayRef; + test_hashes_internal!( + $method, + nullable_input, + initial_seeds_with_nulls, + expected_with_nulls + ); + }; + } + fn test_murmur3_hash<I: Clone, T: arrow_array::Array + From<Vec<Option<I>>> + 'static>( values: Vec<Option<I>>, expected: Vec<u32>, ) { - // copied before inserting nulls - let mut input_with_nulls = values.clone(); - let mut expected_with_nulls = expected.clone(); - let len = values.len(); - let i = Arc::new(T::from(values)) as ArrayRef; - test_hashes_internal!(i, len, expected); - - // test with nulls - let median = len / 2; - input_with_nulls.insert(0, None); - input_with_nulls.insert(median, None); - expected_with_nulls.insert(0, 42); - expected_with_nulls.insert(median, 42); - let with_nulls_len = len + 2; - let nullable_input = Arc::new(T::from(input_with_nulls)) as ArrayRef; - test_hashes_internal!(nullable_input, with_nulls_len, expected_with_nulls); + test_hashes_with_nulls!(create_murmur3_hashes, T, values, expected, u32); + } + + fn test_xxhash64_hash<I: Clone, T: arrow_array::Array + From<Vec<Option<I>>> + 'static>( + values: Vec<Option<I>>, + expected: Vec<u64>, + ) { + test_hashes_with_nulls!(create_xxhash64_hashes, T, values, expected, u64); } #[test] @@ -405,6 +570,16 @@ mod tests { vec![Some(1), Some(0), Some(-1), Some(i8::MAX), Some(i8::MIN)], vec![0xdea578e3, 0x379fae8f, 0xa0590e3d, 0x43b4d8ed, 0x422a1365], ); + test_xxhash64_hash::<i8, Int8Array>( + vec![Some(1), Some(0), Some(-1), Some(i8::MAX), Some(i8::MIN)], + vec![ + 0xa309b38455455929, + 0x3229fbc4681e48f3, + 0x1bfdda8861c06e45, + 0x77cc15d9f9f2cdc2, + 0x39bc22b9e94d81d0, + ], + ); } #[test] @@ -413,6 +588,16 @@ mod tests { vec![Some(1), Some(0), Some(-1), Some(i32::MAX), Some(i32::MIN)], vec![0xdea578e3, 0x379fae8f, 0xa0590e3d, 0x07fb67e7, 0x2b1f0fc6], ); + test_xxhash64_hash::<i32, Int32Array>( + vec![Some(1), Some(0), Some(-1), Some(i32::MAX), Some(i32::MIN)], + vec![ + 0xa309b38455455929, + 0x3229fbc4681e48f3, + 0x1bfdda8861c06e45, + 0x14f0ac009c21721c, + 0x1cc7cb8d034769cd, + ], + ); } #[test] @@ -421,6 +606,16 @@ mod tests { vec![Some(1), Some(0), Some(-1), Some(i64::MAX), Some(i64::MIN)], vec![0x99f0149d, 0x9c67b85d, 0xc8008529, 0xa05b5d7b, 0xcd1e64fb], ); + test_xxhash64_hash::<i64, Int64Array>( + vec![Some(1), Some(0), Some(-1), Some(i64::MAX), Some(i64::MIN)], + vec![ + 0x9ed50fd59358d232, + 0xb71b47ebda15746c, + 0x358ae035bfb46fd2, + 0xd2f1c616ae7eb306, + 0x88608019c494c1f4, + ], + ); } #[test] @@ -438,6 +633,24 @@ mod tests { 0xe434cc39, 0x379fae8f, 0x379fae8f, 0xdc0da8eb, 0xcbdc340f, 0xc0361c86, ], ); + test_xxhash64_hash::<f32, Float32Array>( + vec![ + Some(1.0), + Some(0.0), + Some(-0.0), + Some(-1.0), + Some(99999999999.99999999999), + Some(-99999999999.99999999999), + ], + vec![ + 0x9b92689757fcdbd, + 0x3229fbc4681e48f3, + 0x3229fbc4681e48f3, + 0xa2becc0e61bb3823, + 0x8f20ab82d4f3687f, + 0xdce4982d97f7ac4, + ], + ) } #[test] @@ -455,6 +668,25 @@ mod tests { 0xe4876492, 0x9c67b85d, 0x9c67b85d, 0x13d81357, 0xb87e1595, 0xa0eef9f9, ], ); + + test_xxhash64_hash::<f64, Float64Array>( + vec![ + Some(1.0), + Some(0.0), + Some(-0.0), + Some(-1.0), + Some(99999999999.99999999999), + Some(-99999999999.99999999999), + ], + vec![ + 0xe1fd6e07fee8ad53, + 0xb71b47ebda15746c, + 0xb71b47ebda15746c, + 0x8cdde022746f8f1f, + 0x793c5c88d313eac7, + 0xc5e60e7b75d9b232, + ], + ) } #[test] @@ -470,7 +702,22 @@ mod tests { 1322437556, 0xe860e5cc, 814637928, ]; - test_murmur3_hash::<String, StringArray>(input, expected); + test_murmur3_hash::<String, StringArray>(input.clone(), expected); + test_xxhash64_hash::<String, StringArray>( + input, + vec![ + 0xc3629e6318d53932, + 0xe7097b6a54378d8a, + 0x98b1582b0977e704, + 0xa80d9d5a6a523bd5, + 0xfcba5f61ac666c61, + 0x88e4fe59adf7b0cc, + 0x259dd873209a3fe3, + 0x13c1d910702770e6, + 0xa17b5eb5dc364dff, + 0xf241303e4a90f299, + ], + ) } #[test] diff --git a/docs/source/user-guide/expressions.md b/docs/source/user-guide/expressions.md index 521699d34..775ebedb6 100644 --- a/docs/source/user-guide/expressions.md +++ b/docs/source/user-guide/expressions.md @@ -71,6 +71,11 @@ The following Spark expressions are currently available: - Shiftright/Shiftleft - Date/Time functions - Year/Hour/Minute/Second +- Hash functions + - Md5 + - Sha2 + - Hash + - Xxhash64 - Math functions - Abs - Acos diff --git a/spark/inspections/CometTPCDSQueriesList-results.txt b/spark/inspections/CometTPCDSQueriesList-results.txt index 13f99a1ac..cf1af9bed 100644 --- a/spark/inspections/CometTPCDSQueriesList-results.txt +++ b/spark/inspections/CometTPCDSQueriesList-results.txt @@ -1,838 +1,941 @@ Query: q1. Comet Exec: Enabled (CometFilter, CometProject) Query: q1: ExplainInfo: -ObjectHashAggregate is not supported BroadcastExchange is not supported -BroadcastHashJoin disabled because not all child plans are native -xxhash64 is not supported +Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled +ObjectHashAggregate is not supported SortMergeJoin disabled because not all child plans are native +TakeOrderedAndProject requires shuffle to be enabled Query: q2. Comet Exec: Enabled (CometFilter, CometProject, CometUnion) Query: q2: ExplainInfo: -ObjectHashAggregate is not supported -xxhash64 is not supported BroadcastExchange is not supported -BroadcastHashJoin disabled because not all child plans are native +Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled +ObjectHashAggregate is not supported SortMergeJoin disabled because not all child plans are native -Shuffle: unsupported Spark partitioning: org.apache.spark.sql.catalyst.plans.physical.RangePartitioning -Query: q3. Comet Exec: Enabled (CometFilter, CometProject) +Query: q3. Comet Exec: Enabled (CometBroadcastHashJoin, CometFilter, CometProject) Query: q3: ExplainInfo: BroadcastExchange is not supported -BroadcastHashJoin disabled because not all child plans are native +Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled +TakeOrderedAndProject requires shuffle to be enabled Query: q4. Comet Exec: Enabled (CometFilter, CometProject) Query: q4: ExplainInfo: BroadcastExchange is not supported -BroadcastHashJoin disabled because not all child plans are native +Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled SortMergeJoin disabled because not all child plans are native +TakeOrderedAndProject requires shuffle to be enabled Query: q5. Comet Exec: Enabled (CometFilter, CometProject, CometUnion) Query: q5: ExplainInfo: BroadcastExchange is not supported -BroadcastHashJoin disabled because not all child plans are native +Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled +SortMergeJoin disabled because not all child plans are native +TakeOrderedAndProject requires shuffle to be enabled Union disabled because not all child plans are native -Query: q6. Comet Exec: Enabled (CometHashAggregate, CometFilter, CometProject) +Query: q6. Comet Exec: Enabled (CometHashAggregate, CometBroadcastHashJoin, CometFilter, CometProject) Query: q6: ExplainInfo: BroadcastExchange is not supported -BroadcastHashJoin disabled because not all child plans are native +Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled +SortMergeJoin disabled because not all child plans are native +TakeOrderedAndProject requires shuffle to be enabled Query: q7. Comet Exec: Enabled (CometFilter, CometProject) Query: q7: ExplainInfo: BroadcastExchange is not supported -BroadcastHashJoin disabled because not all child plans are native +Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled +TakeOrderedAndProject requires shuffle to be enabled -Query: q8. Comet Exec: Enabled (CometFilter, CometProject, CometSort) +Query: q8. Comet Exec: Enabled (CometHashAggregate, CometBroadcastHashJoin, CometFilter, CometProject) Query: q8: ExplainInfo: -ObjectHashAggregate is not supported -getstructfield is not supported -xxhash64 is not supported BroadcastExchange is not supported -BroadcastHashJoin disabled because not all child plans are native +Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled SortMergeJoin disabled because not all child plans are native +TakeOrderedAndProject requires shuffle to be enabled Query: q9. Comet Exec: Enabled (CometFilter) Query: q9: ExplainInfo: -named_struct is not supported +Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled getstructfield is not supported Query: q10. Comet Exec: Enabled (CometFilter, CometProject) Query: q10: ExplainInfo: -ObjectHashAggregate is not supported BroadcastExchange is not supported -BroadcastHashJoin disabled because not all child plans are native -xxhash64 is not supported +Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled SortMergeJoin disabled because not all child plans are native +TakeOrderedAndProject requires shuffle to be enabled Query: q11. Comet Exec: Enabled (CometFilter, CometProject) Query: q11: ExplainInfo: BroadcastExchange is not supported -BroadcastHashJoin disabled because not all child plans are native +Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled SortMergeJoin disabled because not all child plans are native +TakeOrderedAndProject requires shuffle to be enabled Query: q12. Comet Exec: Enabled (CometFilter, CometProject) Query: q12: ExplainInfo: BroadcastExchange is not supported -BroadcastHashJoin disabled because not all child plans are native +Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled +TakeOrderedAndProject requires shuffle to be enabled Window is not supported Query: q13. Comet Exec: Enabled (CometFilter, CometProject) Query: q13: ExplainInfo: BroadcastExchange is not supported -BroadcastHashJoin disabled because not all child plans are native +Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled -Query: q14a. Comet Exec: Enabled (CometFilter, CometProject, CometSort) +Query: q14a. Comet Exec: Enabled (CometFilter, CometProject) Query: q14a: ExplainInfo: BroadcastExchange is not supported -BroadcastHashJoin disabled because not all child plans are native +Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled SortMergeJoin disabled because not all child plans are native -ObjectHashAggregate is not supported -xxhash64 is not supported +TakeOrderedAndProject requires shuffle to be enabled Union disabled because not all child plans are native -Query: q14b. Comet Exec: Enabled (CometFilter, CometProject, CometSort) +Query: q14b. Comet Exec: Enabled (CometFilter, CometProject) Query: q14b: ExplainInfo: BroadcastExchange is not supported -BroadcastHashJoin disabled because not all child plans are native +Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled SortMergeJoin disabled because not all child plans are native -ObjectHashAggregate is not supported -xxhash64 is not supported +TakeOrderedAndProject requires shuffle to be enabled Union disabled because not all child plans are native Query: q15. Comet Exec: Enabled (CometFilter, CometProject) Query: q15: ExplainInfo: BroadcastExchange is not supported -BroadcastHashJoin disabled because not all child plans are native +Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled +SortMergeJoin disabled because not all child plans are native +TakeOrderedAndProject requires shuffle to be enabled Query: q16. Comet Exec: Enabled (CometFilter, CometProject) Query: q16: ExplainInfo: BroadcastExchange is not supported -BroadcastHashJoin disabled because not all child plans are native +Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled +ObjectHashAggregate is not supported +SortMergeJoin disabled because not all child plans are native Query: q17. Comet Exec: Enabled (CometFilter, CometProject) Query: q17: ExplainInfo: BroadcastExchange is not supported -BroadcastHashJoin disabled because not all child plans are native +Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled +SortMergeJoin disabled because not all child plans are native +TakeOrderedAndProject requires shuffle to be enabled Query: q18. Comet Exec: Enabled (CometFilter, CometProject) Query: q18: ExplainInfo: BroadcastExchange is not supported -BroadcastHashJoin disabled because not all child plans are native +Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled +SortMergeJoin disabled because not all child plans are native +TakeOrderedAndProject requires shuffle to be enabled -Query: q19. Comet Exec: Enabled (CometFilter, CometProject) +Query: q19. Comet Exec: Enabled (CometBroadcastHashJoin, CometFilter, CometProject) Query: q19: ExplainInfo: BroadcastExchange is not supported -BroadcastHashJoin disabled because not all child plans are native +Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled +SortMergeJoin disabled because not all child plans are native +TakeOrderedAndProject requires shuffle to be enabled Query: q20. Comet Exec: Enabled (CometFilter, CometProject) Query: q20: ExplainInfo: BroadcastExchange is not supported -BroadcastHashJoin disabled because not all child plans are native +Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled +TakeOrderedAndProject requires shuffle to be enabled Window is not supported Query: q21. Comet Exec: Enabled (CometFilter, CometProject) Query: q21: ExplainInfo: BroadcastExchange is not supported -BroadcastHashJoin disabled because not all child plans are native +Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled +TakeOrderedAndProject requires shuffle to be enabled Query: q22. Comet Exec: Enabled (CometFilter, CometProject) Query: q22: ExplainInfo: BroadcastExchange is not supported -BroadcastHashJoin disabled because not all child plans are native +Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled +TakeOrderedAndProject requires shuffle to be enabled Query: q23a. Comet Exec: Enabled (CometFilter, CometProject) Query: q23a: ExplainInfo: BroadcastExchange is not supported -BroadcastHashJoin disabled because not all child plans are native -ObjectHashAggregate is not supported -xxhash64 is not supported +Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled SortMergeJoin disabled because not all child plans are native Union disabled because not all child plans are native -Query: q23b. Comet Exec: Enabled (CometFilter, CometProject, CometSort) +Query: q23b. Comet Exec: Enabled (CometFilter, CometProject) Query: q23b: ExplainInfo: BroadcastExchange is not supported -BroadcastHashJoin disabled because not all child plans are native -ObjectHashAggregate is not supported -xxhash64 is not supported +Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled SortMergeJoin disabled because not all child plans are native +TakeOrderedAndProject requires shuffle to be enabled Union disabled because not all child plans are native Query: q24a. Comet Exec: Enabled (CometFilter, CometProject) Query: q24a: ExplainInfo: BroadcastExchange is not supported -BroadcastHashJoin disabled because not all child plans are native +Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled +ObjectHashAggregate is not supported +SortMergeJoin disabled because not all child plans are native Query: q24b. Comet Exec: Enabled (CometFilter, CometProject) Query: q24b: ExplainInfo: BroadcastExchange is not supported -BroadcastHashJoin disabled because not all child plans are native +Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled +ObjectHashAggregate is not supported +SortMergeJoin disabled because not all child plans are native Query: q25. Comet Exec: Enabled (CometFilter, CometProject) Query: q25: ExplainInfo: BroadcastExchange is not supported -BroadcastHashJoin disabled because not all child plans are native +Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled +SortMergeJoin disabled because not all child plans are native +TakeOrderedAndProject requires shuffle to be enabled Query: q26. Comet Exec: Enabled (CometFilter, CometProject) Query: q26: ExplainInfo: BroadcastExchange is not supported -BroadcastHashJoin disabled because not all child plans are native +Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled +TakeOrderedAndProject requires shuffle to be enabled Query: q27. Comet Exec: Enabled (CometFilter, CometProject) Query: q27: ExplainInfo: BroadcastExchange is not supported -BroadcastHashJoin disabled because not all child plans are native +Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled +TakeOrderedAndProject requires shuffle to be enabled Query: q28. Comet Exec: Enabled (CometHashAggregate, CometFilter, CometProject) Query: q28: ExplainInfo: -Unsupported aggregation mode PartialMerge BroadcastExchange is not supported +Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled Query: q29. Comet Exec: Enabled (CometFilter, CometProject) Query: q29: ExplainInfo: BroadcastExchange is not supported -BroadcastHashJoin disabled because not all child plans are native +Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled +SortMergeJoin disabled because not all child plans are native +TakeOrderedAndProject requires shuffle to be enabled Query: q30. Comet Exec: Enabled (CometFilter, CometProject) Query: q30: ExplainInfo: BroadcastExchange is not supported -BroadcastHashJoin disabled because not all child plans are native -ObjectHashAggregate is not supported -xxhash64 is not supported +Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled SortMergeJoin disabled because not all child plans are native +TakeOrderedAndProject requires shuffle to be enabled Query: q31. Comet Exec: Enabled (CometFilter) Query: q31: ExplainInfo: BroadcastExchange is not supported -BroadcastHashJoin disabled because not all child plans are native +Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled SortMergeJoin disabled because not all child plans are native -Shuffle: unsupported Spark partitioning: org.apache.spark.sql.catalyst.plans.physical.RangePartitioning Query: q32. Comet Exec: Enabled (CometFilter, CometProject) Query: q32: ExplainInfo: -ObjectHashAggregate is not supported BroadcastExchange is not supported -BroadcastHashJoin disabled because not all child plans are native -xxhash64 is not supported +Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled SortMergeJoin disabled because not all child plans are native Query: q33. Comet Exec: Enabled (CometFilter, CometProject) Query: q33: ExplainInfo: BroadcastExchange is not supported -BroadcastHashJoin disabled because not all child plans are native +Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled +TakeOrderedAndProject requires shuffle to be enabled Union disabled because not all child plans are native Query: q34. Comet Exec: Enabled (CometFilter, CometProject) Query: q34: ExplainInfo: BroadcastExchange is not supported -BroadcastHashJoin disabled because not all child plans are native -Shuffle: unsupported Spark partitioning: org.apache.spark.sql.catalyst.plans.physical.RangePartitioning +Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled +SortMergeJoin disabled because not all child plans are native -Query: q35. Comet Exec: Enabled (CometFilter, CometProject, CometSort) +Query: q35. Comet Exec: Enabled (CometFilter, CometProject) Query: q35: ExplainInfo: BroadcastExchange is not supported -BroadcastHashJoin disabled because not all child plans are native +Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled SortMergeJoin disabled because not all child plans are native +TakeOrderedAndProject requires shuffle to be enabled Query: q36. Comet Exec: Enabled (CometFilter, CometProject) Query: q36: ExplainInfo: BroadcastExchange is not supported -BroadcastHashJoin disabled because not all child plans are native +Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled +TakeOrderedAndProject requires shuffle to be enabled Window is not supported -Query: q37. Comet Exec: Enabled (CometFilter, CometProject) +Query: q37. Comet Exec: Enabled (CometBroadcastHashJoin, CometFilter, CometProject) Query: q37: ExplainInfo: BroadcastExchange is not supported -BroadcastHashJoin disabled because not all child plans are native +Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled +SortMergeJoin disabled because not all child plans are native +TakeOrderedAndProject requires shuffle to be enabled Query: q38. Comet Exec: Enabled (CometFilter, CometProject) Query: q38: ExplainInfo: BroadcastExchange is not supported -BroadcastHashJoin disabled because not all child plans are native +Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled SortMergeJoin disabled because not all child plans are native Query: q39a. Comet Exec: Enabled (CometFilter, CometProject) Query: q39a: ExplainInfo: BroadcastExchange is not supported -BroadcastHashJoin disabled because not all child plans are native +Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled SortMergeJoin disabled because not all child plans are native -Shuffle: unsupported Spark partitioning: org.apache.spark.sql.catalyst.plans.physical.RangePartitioning Query: q39b. Comet Exec: Enabled (CometFilter, CometProject) Query: q39b: ExplainInfo: BroadcastExchange is not supported -BroadcastHashJoin disabled because not all child plans are native +Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled SortMergeJoin disabled because not all child plans are native -Shuffle: unsupported Spark partitioning: org.apache.spark.sql.catalyst.plans.physical.RangePartitioning Query: q40. Comet Exec: Enabled (CometFilter, CometProject) Query: q40: ExplainInfo: BroadcastExchange is not supported -BroadcastHashJoin disabled because not all child plans are native +Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled +SortMergeJoin disabled because not all child plans are native +TakeOrderedAndProject requires shuffle to be enabled -Query: q41. Comet Exec: Enabled (CometFilter, CometProject) +Query: q41. Comet Exec: Enabled (CometHashAggregate, CometFilter, CometProject) Query: q41: ExplainInfo: -ObjectHashAggregate is not supported -xxhash64 is not supported BroadcastExchange is not supported -BroadcastHashJoin disabled because not all child plans are native +Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled +TakeOrderedAndProject requires shuffle to be enabled -Query: q42. Comet Exec: Enabled (CometFilter, CometProject) +Query: q42. Comet Exec: Enabled (CometBroadcastHashJoin, CometFilter, CometProject) Query: q42: ExplainInfo: BroadcastExchange is not supported -BroadcastHashJoin disabled because not all child plans are native +Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled +TakeOrderedAndProject requires shuffle to be enabled -Query: q43. Comet Exec: Enabled (CometFilter, CometProject) +Query: q43. Comet Exec: Enabled (CometBroadcastHashJoin, CometFilter, CometProject) Query: q43: ExplainInfo: BroadcastExchange is not supported -BroadcastHashJoin disabled because not all child plans are native +Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled +TakeOrderedAndProject requires shuffle to be enabled -Query: q44. Comet Exec: Enabled (CometHashAggregate, CometFilter, CometProject, CometSort) +Query: q44. Comet Exec: Enabled (CometHashAggregate, CometFilter, CometProject) Query: q44: ExplainInfo: -Window is not supported BroadcastExchange is not supported -BroadcastHashJoin disabled because not all child plans are native +Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled +SortMergeJoin disabled because not all child plans are native +TakeOrderedAndProject requires shuffle to be enabled +Window is not supported Query: q45. Comet Exec: Enabled (CometFilter, CometProject) Query: q45: ExplainInfo: BroadcastExchange is not supported -BroadcastHashJoin disabled because not all child plans are native +Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled +SortMergeJoin disabled because not all child plans are native +TakeOrderedAndProject requires shuffle to be enabled Query: q46. Comet Exec: Enabled (CometFilter, CometProject) Query: q46: ExplainInfo: BroadcastExchange is not supported -BroadcastHashJoin disabled because not all child plans are native +Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled +SortMergeJoin disabled because not all child plans are native +TakeOrderedAndProject requires shuffle to be enabled -Query: q47. Comet Exec: Enabled (CometFilter, CometProject) +Query: q47. Comet Exec: Enabled (CometBroadcastHashJoin, CometFilter, CometProject) Query: q47: ExplainInfo: BroadcastExchange is not supported -BroadcastHashJoin disabled because not all child plans are native -Window is not supported +Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled SortMergeJoin disabled because not all child plans are native +TakeOrderedAndProject requires shuffle to be enabled +Window is not supported Query: q48. Comet Exec: Enabled (CometFilter, CometProject) Query: q48: ExplainInfo: BroadcastExchange is not supported -BroadcastHashJoin disabled because not all child plans are native +Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled Query: q49. Comet Exec: Enabled (CometFilter, CometProject) Query: q49: ExplainInfo: BroadcastExchange is not supported -BroadcastHashJoin disabled because not all child plans are native -Window is not supported +Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled +SortMergeJoin disabled because not all child plans are native +TakeOrderedAndProject requires shuffle to be enabled Union disabled because not all child plans are native +Window is not supported Query: q50. Comet Exec: Enabled (CometFilter, CometProject) Query: q50: ExplainInfo: BroadcastExchange is not supported -BroadcastHashJoin disabled because not all child plans are native +Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled +SortMergeJoin disabled because not all child plans are native +TakeOrderedAndProject requires shuffle to be enabled Query: q51. Comet Exec: Enabled (CometFilter, CometProject) Query: q51: ExplainInfo: BroadcastExchange is not supported -BroadcastHashJoin disabled because not all child plans are native -Window is not supported +Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled SortMergeJoin disabled because not all child plans are native +TakeOrderedAndProject requires shuffle to be enabled +Window is not supported -Query: q52. Comet Exec: Enabled (CometFilter, CometProject) +Query: q52. Comet Exec: Enabled (CometBroadcastHashJoin, CometFilter, CometProject) Query: q52: ExplainInfo: BroadcastExchange is not supported -BroadcastHashJoin disabled because not all child plans are native +Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled +TakeOrderedAndProject requires shuffle to be enabled -Query: q53. Comet Exec: Enabled (CometFilter, CometProject) +Query: q53. Comet Exec: Enabled (CometBroadcastHashJoin, CometFilter, CometProject) Query: q53: ExplainInfo: BroadcastExchange is not supported -BroadcastHashJoin disabled because not all child plans are native +Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled +TakeOrderedAndProject requires shuffle to be enabled Window is not supported Query: q54. Comet Exec: Enabled (CometFilter, CometProject, CometUnion) Query: q54: ExplainInfo: BroadcastExchange is not supported -BroadcastHashJoin disabled because not all child plans are native -ObjectHashAggregate is not supported -xxhash64 is not supported +Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled SortMergeJoin disabled because not all child plans are native +TakeOrderedAndProject requires shuffle to be enabled -Query: q55. Comet Exec: Enabled (CometFilter, CometProject) +Query: q55. Comet Exec: Enabled (CometBroadcastHashJoin, CometFilter, CometProject) Query: q55: ExplainInfo: BroadcastExchange is not supported -BroadcastHashJoin disabled because not all child plans are native +Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled +TakeOrderedAndProject requires shuffle to be enabled Query: q56. Comet Exec: Enabled (CometFilter, CometProject) Query: q56: ExplainInfo: BroadcastExchange is not supported -BroadcastHashJoin disabled because not all child plans are native +Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled +TakeOrderedAndProject requires shuffle to be enabled Union disabled because not all child plans are native -Query: q57. Comet Exec: Enabled (CometFilter, CometProject) +Query: q57. Comet Exec: Enabled (CometBroadcastHashJoin, CometFilter, CometProject) Query: q57: ExplainInfo: BroadcastExchange is not supported -BroadcastHashJoin disabled because not all child plans are native -Window is not supported +Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled SortMergeJoin disabled because not all child plans are native +TakeOrderedAndProject requires shuffle to be enabled +Window is not supported Query: q58. Comet Exec: Enabled (CometFilter, CometProject) Query: q58: ExplainInfo: BroadcastExchange is not supported -BroadcastHashJoin disabled because not all child plans are native +Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled SortMergeJoin disabled because not all child plans are native +TakeOrderedAndProject requires shuffle to be enabled Query: q59. Comet Exec: Enabled (CometFilter, CometProject) Query: q59: ExplainInfo: -ObjectHashAggregate is not supported -xxhash64 is not supported BroadcastExchange is not supported -BroadcastHashJoin disabled because not all child plans are native +Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled +ObjectHashAggregate is not supported SortMergeJoin disabled because not all child plans are native +TakeOrderedAndProject requires shuffle to be enabled Query: q60. Comet Exec: Enabled (CometFilter, CometProject) Query: q60: ExplainInfo: BroadcastExchange is not supported -BroadcastHashJoin disabled because not all child plans are native +Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled +TakeOrderedAndProject requires shuffle to be enabled Union disabled because not all child plans are native Query: q61. Comet Exec: Enabled (CometFilter, CometProject) Query: q61: ExplainInfo: BroadcastExchange is not supported -BroadcastHashJoin disabled because not all child plans are native +Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled +ObjectHashAggregate is not supported +SortMergeJoin disabled because not all child plans are native Query: q62. Comet Exec: Enabled (CometFilter, CometProject) Query: q62: ExplainInfo: BroadcastExchange is not supported -BroadcastHashJoin disabled because not all child plans are native +Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled +TakeOrderedAndProject requires shuffle to be enabled -Query: q63. Comet Exec: Enabled (CometFilter, CometProject) +Query: q63. Comet Exec: Enabled (CometBroadcastHashJoin, CometFilter, CometProject) Query: q63: ExplainInfo: BroadcastExchange is not supported -BroadcastHashJoin disabled because not all child plans are native +Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled +TakeOrderedAndProject requires shuffle to be enabled Window is not supported Query: q64. Comet Exec: Enabled (CometFilter, CometProject) Query: q64: ExplainInfo: BroadcastExchange is not supported -ObjectHashAggregate is not supported -BroadcastHashJoin disabled because not all child plans are native -xxhash64 is not supported +Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled SortMergeJoin disabled because not all child plans are native -Shuffle: unsupported Spark partitioning: org.apache.spark.sql.catalyst.plans.physical.RangePartitioning Query: q65. Comet Exec: Enabled (CometFilter, CometProject) Query: q65: ExplainInfo: BroadcastExchange is not supported -BroadcastHashJoin disabled because not all child plans are native +Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled SortMergeJoin disabled because not all child plans are native +TakeOrderedAndProject requires shuffle to be enabled Query: q66. Comet Exec: Enabled (CometFilter, CometProject) Query: q66: ExplainInfo: BroadcastExchange is not supported -BroadcastHashJoin disabled because not all child plans are native +Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled +TakeOrderedAndProject requires shuffle to be enabled Union disabled because not all child plans are native Query: q67. Comet Exec: Enabled (CometFilter, CometProject) Query: q67: ExplainInfo: BroadcastExchange is not supported -BroadcastHashJoin disabled because not all child plans are native +Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled +TakeOrderedAndProject requires shuffle to be enabled Window is not supported Query: q68. Comet Exec: Enabled (CometFilter, CometProject) Query: q68: ExplainInfo: BroadcastExchange is not supported -BroadcastHashJoin disabled because not all child plans are native +Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled +SortMergeJoin disabled because not all child plans are native +TakeOrderedAndProject requires shuffle to be enabled -Query: q69. Comet Exec: Enabled (CometFilter, CometProject, CometSort) +Query: q69. Comet Exec: Enabled (CometFilter, CometProject) Query: q69: ExplainInfo: BroadcastExchange is not supported -BroadcastHashJoin disabled because not all child plans are native +Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled SortMergeJoin disabled because not all child plans are native +TakeOrderedAndProject requires shuffle to be enabled -Query: q70. Comet Exec: Enabled (CometFilter, CometProject, CometSort) +Query: q70. Comet Exec: Enabled (CometFilter, CometProject) Query: q70: ExplainInfo: BroadcastExchange is not supported -BroadcastHashJoin disabled because not all child plans are native -Window is not supported +Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled SortMergeJoin disabled because not all child plans are native +TakeOrderedAndProject requires shuffle to be enabled +Window is not supported Query: q71. Comet Exec: Enabled (CometFilter, CometProject) Query: q71: ExplainInfo: BroadcastExchange is not supported -BroadcastHashJoin disabled because not all child plans are native +Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled Union disabled because not all child plans are native -Shuffle: unsupported Spark partitioning: org.apache.spark.sql.catalyst.plans.physical.RangePartitioning Query: q72. Comet Exec: Enabled (CometFilter, CometProject) Query: q72: ExplainInfo: BroadcastExchange is not supported -BroadcastHashJoin disabled because not all child plans are native +Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled +ObjectHashAggregate is not supported +SortMergeJoin disabled because not all child plans are native +TakeOrderedAndProject requires shuffle to be enabled Query: q73. Comet Exec: Enabled (CometFilter, CometProject) Query: q73: ExplainInfo: BroadcastExchange is not supported -BroadcastHashJoin disabled because not all child plans are native -Shuffle: unsupported Spark partitioning: org.apache.spark.sql.catalyst.plans.physical.RangePartitioning +Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled +SortMergeJoin disabled because not all child plans are native Query: q74. Comet Exec: Enabled (CometFilter, CometProject) Query: q74: ExplainInfo: BroadcastExchange is not supported -BroadcastHashJoin disabled because not all child plans are native +Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled SortMergeJoin disabled because not all child plans are native +TakeOrderedAndProject requires shuffle to be enabled Query: q75. Comet Exec: Enabled (CometFilter, CometProject) Query: q75: ExplainInfo: BroadcastExchange is not supported -BroadcastHashJoin disabled because not all child plans are native -Union disabled because not all child plans are native +Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled SortMergeJoin disabled because not all child plans are native +TakeOrderedAndProject requires shuffle to be enabled +Union disabled because not all child plans are native Query: q76. Comet Exec: Enabled (CometFilter, CometProject) Query: q76: ExplainInfo: BroadcastExchange is not supported -BroadcastHashJoin disabled because not all child plans are native +Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled +TakeOrderedAndProject requires shuffle to be enabled Union disabled because not all child plans are native Query: q77. Comet Exec: Enabled (CometFilter, CometProject) Query: q77: ExplainInfo: BroadcastExchange is not supported -BroadcastHashJoin disabled because not all child plans are native +Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled SortMergeJoin disabled because not all child plans are native +TakeOrderedAndProject requires shuffle to be enabled Union disabled because not all child plans are native -Query: q78. Comet Exec: Enabled (CometFilter) +Query: q78. Comet Exec: Enabled (CometFilter, CometProject) Query: q78: ExplainInfo: BroadcastExchange is not supported -BroadcastHashJoin disabled because not all child plans are native +Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled SortMergeJoin disabled because not all child plans are native +TakeOrderedAndProject requires shuffle to be enabled Query: q79. Comet Exec: Enabled (CometFilter, CometProject) Query: q79: ExplainInfo: BroadcastExchange is not supported -BroadcastHashJoin disabled because not all child plans are native +Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled +SortMergeJoin disabled because not all child plans are native +TakeOrderedAndProject requires shuffle to be enabled Query: q80. Comet Exec: Enabled (CometFilter, CometProject) Query: q80: ExplainInfo: BroadcastExchange is not supported -BroadcastHashJoin disabled because not all child plans are native +Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled +ObjectHashAggregate is not supported +SortMergeJoin disabled because not all child plans are native +TakeOrderedAndProject requires shuffle to be enabled Union disabled because not all child plans are native Query: q81. Comet Exec: Enabled (CometFilter, CometProject) Query: q81: ExplainInfo: BroadcastExchange is not supported -BroadcastHashJoin disabled because not all child plans are native -ObjectHashAggregate is not supported -xxhash64 is not supported +Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled SortMergeJoin disabled because not all child plans are native +TakeOrderedAndProject requires shuffle to be enabled -Query: q82. Comet Exec: Enabled (CometFilter, CometProject) +Query: q82. Comet Exec: Enabled (CometBroadcastHashJoin, CometFilter, CometProject) Query: q82: ExplainInfo: BroadcastExchange is not supported -BroadcastHashJoin disabled because not all child plans are native +Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled +SortMergeJoin disabled because not all child plans are native +TakeOrderedAndProject requires shuffle to be enabled Query: q83. Comet Exec: Enabled (CometFilter, CometProject) Query: q83: ExplainInfo: BroadcastExchange is not supported -BroadcastHashJoin disabled because not all child plans are native +Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled SortMergeJoin disabled because not all child plans are native +TakeOrderedAndProject requires shuffle to be enabled Query: q84. Comet Exec: Enabled (CometFilter, CometProject) Query: q84: ExplainInfo: BroadcastExchange is not supported -BroadcastHashJoin disabled because not all child plans are native +Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled +SortMergeJoin disabled because not all child plans are native +TakeOrderedAndProject requires shuffle to be enabled Query: q85. Comet Exec: Enabled (CometFilter, CometProject) Query: q85: ExplainInfo: BroadcastExchange is not supported -BroadcastHashJoin disabled because not all child plans are native +Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled +SortMergeJoin disabled because not all child plans are native +TakeOrderedAndProject requires shuffle to be enabled Query: q86. Comet Exec: Enabled (CometFilter, CometProject) Query: q86: ExplainInfo: BroadcastExchange is not supported -BroadcastHashJoin disabled because not all child plans are native +Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled +TakeOrderedAndProject requires shuffle to be enabled Window is not supported Query: q87. Comet Exec: Enabled (CometFilter, CometProject) Query: q87: ExplainInfo: BroadcastExchange is not supported -BroadcastHashJoin disabled because not all child plans are native +Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled SortMergeJoin disabled because not all child plans are native Query: q88. Comet Exec: Enabled (CometFilter, CometProject) Query: q88: ExplainInfo: BroadcastExchange is not supported -BroadcastHashJoin disabled because not all child plans are native +Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled -Query: q89. Comet Exec: Enabled (CometFilter, CometProject) +Query: q89. Comet Exec: Enabled (CometBroadcastHashJoin, CometFilter, CometProject) Query: q89: ExplainInfo: BroadcastExchange is not supported -BroadcastHashJoin disabled because not all child plans are native +Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled +TakeOrderedAndProject requires shuffle to be enabled Window is not supported Query: q90. Comet Exec: Enabled (CometFilter, CometProject) Query: q90: ExplainInfo: BroadcastExchange is not supported -BroadcastHashJoin disabled because not all child plans are native +Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled -Query: q91. Comet Exec: Enabled (CometFilter, CometProject) +Query: q91. Comet Exec: Enabled (CometBroadcastHashJoin, CometFilter, CometProject) Query: q91: ExplainInfo: BroadcastExchange is not supported -BroadcastHashJoin disabled because not all child plans are native -Shuffle: unsupported Spark partitioning: org.apache.spark.sql.catalyst.plans.physical.RangePartitioning +Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled +ObjectHashAggregate is not supported +SortMergeJoin disabled because not all child plans are native Query: q92. Comet Exec: Enabled (CometFilter, CometProject) Query: q92: ExplainInfo: -ObjectHashAggregate is not supported BroadcastExchange is not supported -BroadcastHashJoin disabled because not all child plans are native -xxhash64 is not supported +Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled SortMergeJoin disabled because not all child plans are native Query: q93. Comet Exec: Enabled (CometFilter, CometProject) Query: q93: ExplainInfo: BroadcastExchange is not supported -BroadcastHashJoin disabled because not all child plans are native +Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled +ObjectHashAggregate is not supported +SortMergeJoin disabled because not all child plans are native +TakeOrderedAndProject requires shuffle to be enabled Query: q94. Comet Exec: Enabled (CometFilter, CometProject) Query: q94: ExplainInfo: BroadcastExchange is not supported -BroadcastHashJoin disabled because not all child plans are native +Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled +ObjectHashAggregate is not supported +SortMergeJoin disabled because not all child plans are native -Query: q95. Comet Exec: Enabled (CometFilter, CometProject, CometSort) +Query: q95. Comet Exec: Enabled (CometFilter, CometProject) Query: q95: ExplainInfo: BroadcastExchange is not supported -BroadcastHashJoin disabled because not all child plans are native +Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled +ObjectHashAggregate is not supported SortMergeJoin disabled because not all child plans are native Query: q96. Comet Exec: Enabled (CometFilter, CometProject) Query: q96: ExplainInfo: BroadcastExchange is not supported -BroadcastHashJoin disabled because not all child plans are native +Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled Query: q97. Comet Exec: Enabled (CometFilter, CometProject) Query: q97: ExplainInfo: BroadcastExchange is not supported -BroadcastHashJoin disabled because not all child plans are native +Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled SortMergeJoin disabled because not all child plans are native Query: q98. Comet Exec: Enabled (CometFilter, CometProject) Query: q98: ExplainInfo: BroadcastExchange is not supported -BroadcastHashJoin disabled because not all child plans are native +Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled Window is not supported -Shuffle: unsupported Spark partitioning: org.apache.spark.sql.catalyst.plans.physical.RangePartitioning Query: q99. Comet Exec: Enabled (CometFilter, CometProject) Query: q99: ExplainInfo: BroadcastExchange is not supported -BroadcastHashJoin disabled because not all child plans are native +Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled +TakeOrderedAndProject requires shuffle to be enabled Query: q5a-v2.7. Comet Exec: Enabled (CometFilter, CometProject, CometUnion) Query: q5a-v2.7: ExplainInfo: BroadcastExchange is not supported -BroadcastHashJoin disabled because not all child plans are native +Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled +SortMergeJoin disabled because not all child plans are native +TakeOrderedAndProject requires shuffle to be enabled Union disabled because not all child plans are native -Query: q6-v2.7. Comet Exec: Enabled (CometHashAggregate, CometFilter, CometProject) +Query: q6-v2.7. Comet Exec: Enabled (CometHashAggregate, CometBroadcastHashJoin, CometFilter, CometProject) Query: q6-v2.7: ExplainInfo: BroadcastExchange is not supported -BroadcastHashJoin disabled because not all child plans are native +Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled +SortMergeJoin disabled because not all child plans are native +TakeOrderedAndProject requires shuffle to be enabled -Query: q10a-v2.7. Comet Exec: Enabled (CometFilter, CometProject, CometSort) +Query: q10a-v2.7. Comet Exec: Enabled (CometFilter, CometProject) Query: q10a-v2.7: ExplainInfo: BroadcastExchange is not supported -BroadcastHashJoin disabled because not all child plans are native -Union disabled because not all child plans are native +Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled SortMergeJoin disabled because not all child plans are native +TakeOrderedAndProject requires shuffle to be enabled +Union disabled because not all child plans are native Query: q11-v2.7. Comet Exec: Enabled (CometFilter, CometProject) Query: q11-v2.7: ExplainInfo: BroadcastExchange is not supported -BroadcastHashJoin disabled because not all child plans are native +Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled SortMergeJoin disabled because not all child plans are native +TakeOrderedAndProject requires shuffle to be enabled Query: q12-v2.7. Comet Exec: Enabled (CometFilter, CometProject) Query: q12-v2.7: ExplainInfo: BroadcastExchange is not supported -BroadcastHashJoin disabled because not all child plans are native +Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled +TakeOrderedAndProject requires shuffle to be enabled Window is not supported -Query: q14-v2.7. Comet Exec: Enabled (CometFilter, CometProject, CometSort) +Query: q14-v2.7. Comet Exec: Enabled (CometFilter, CometProject) Query: q14-v2.7: ExplainInfo: BroadcastExchange is not supported -BroadcastHashJoin disabled because not all child plans are native +Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled SortMergeJoin disabled because not all child plans are native -ObjectHashAggregate is not supported -xxhash64 is not supported +TakeOrderedAndProject requires shuffle to be enabled Union disabled because not all child plans are native -Query: q14a-v2.7. Comet Exec: Enabled (CometFilter, CometProject, CometSort) +Query: q14a-v2.7. Comet Exec: Enabled (CometFilter, CometProject) Query: q14a-v2.7: ExplainInfo: BroadcastExchange is not supported -BroadcastHashJoin disabled because not all child plans are native +Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled SortMergeJoin disabled because not all child plans are native -ObjectHashAggregate is not supported -xxhash64 is not supported +TakeOrderedAndProject requires shuffle to be enabled Union disabled because not all child plans are native Query: q18a-v2.7. Comet Exec: Enabled (CometFilter, CometProject) Query: q18a-v2.7: ExplainInfo: BroadcastExchange is not supported -BroadcastHashJoin disabled because not all child plans are native +Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled +SortMergeJoin disabled because not all child plans are native +TakeOrderedAndProject requires shuffle to be enabled Union disabled because not all child plans are native Query: q20-v2.7. Comet Exec: Enabled (CometFilter, CometProject) Query: q20-v2.7: ExplainInfo: BroadcastExchange is not supported -BroadcastHashJoin disabled because not all child plans are native +Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled +TakeOrderedAndProject requires shuffle to be enabled Window is not supported Query: q22-v2.7. Comet Exec: Enabled (CometFilter, CometProject) Query: q22-v2.7: ExplainInfo: BroadcastExchange is not supported -BroadcastHashJoin disabled because not all child plans are native -Native Broadcast is not enabled +Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled +TakeOrderedAndProject requires shuffle to be enabled Query: q22a-v2.7. Comet Exec: Enabled (CometFilter, CometProject) Query: q22a-v2.7: ExplainInfo: BroadcastExchange is not supported -BroadcastHashJoin disabled because not all child plans are native +Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled +TakeOrderedAndProject requires shuffle to be enabled Union disabled because not all child plans are native Query: q24-v2.7. Comet Exec: Enabled (CometFilter, CometProject) Query: q24-v2.7: ExplainInfo: BroadcastExchange is not supported -BroadcastHashJoin disabled because not all child plans are native -Shuffle: unsupported Spark partitioning: org.apache.spark.sql.catalyst.plans.physical.RangePartitioning +Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled +ObjectHashAggregate is not supported +SortMergeJoin disabled because not all child plans are native Query: q27a-v2.7. Comet Exec: Enabled (CometFilter, CometProject) Query: q27a-v2.7: ExplainInfo: BroadcastExchange is not supported -BroadcastHashJoin disabled because not all child plans are native +Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled +TakeOrderedAndProject requires shuffle to be enabled Union disabled because not all child plans are native Query: q34-v2.7. Comet Exec: Enabled (CometFilter, CometProject) Query: q34-v2.7: ExplainInfo: BroadcastExchange is not supported -BroadcastHashJoin disabled because not all child plans are native -Shuffle: unsupported Spark partitioning: org.apache.spark.sql.catalyst.plans.physical.RangePartitioning +Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled +SortMergeJoin disabled because not all child plans are native -Query: q35-v2.7. Comet Exec: Enabled (CometFilter, CometProject, CometSort) +Query: q35-v2.7. Comet Exec: Enabled (CometFilter, CometProject) Query: q35-v2.7: ExplainInfo: BroadcastExchange is not supported -BroadcastHashJoin disabled because not all child plans are native +Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled SortMergeJoin disabled because not all child plans are native +TakeOrderedAndProject requires shuffle to be enabled -Query: q35a-v2.7. Comet Exec: Enabled (CometFilter, CometProject, CometSort) +Query: q35a-v2.7. Comet Exec: Enabled (CometFilter, CometProject) Query: q35a-v2.7: ExplainInfo: BroadcastExchange is not supported -BroadcastHashJoin disabled because not all child plans are native -Union disabled because not all child plans are native +Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled SortMergeJoin disabled because not all child plans are native +TakeOrderedAndProject requires shuffle to be enabled +Union disabled because not all child plans are native Query: q36a-v2.7. Comet Exec: Enabled (CometFilter, CometProject) Query: q36a-v2.7: ExplainInfo: BroadcastExchange is not supported -BroadcastHashJoin disabled because not all child plans are native +Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled +TakeOrderedAndProject requires shuffle to be enabled Union disabled because not all child plans are native Window is not supported -Query: q47-v2.7. Comet Exec: Enabled (CometFilter, CometProject) +Query: q47-v2.7. Comet Exec: Enabled (CometBroadcastHashJoin, CometFilter, CometProject) Query: q47-v2.7: ExplainInfo: BroadcastExchange is not supported -BroadcastHashJoin disabled because not all child plans are native -Window is not supported +Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled SortMergeJoin disabled because not all child plans are native +TakeOrderedAndProject requires shuffle to be enabled +Window is not supported Query: q49-v2.7. Comet Exec: Enabled (CometFilter, CometProject) Query: q49-v2.7: ExplainInfo: BroadcastExchange is not supported -BroadcastHashJoin disabled because not all child plans are native -Window is not supported +Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled +SortMergeJoin disabled because not all child plans are native +TakeOrderedAndProject requires shuffle to be enabled Union disabled because not all child plans are native +Window is not supported Query: q51a-v2.7. Comet Exec: Enabled (CometFilter, CometProject) Query: q51a-v2.7: ExplainInfo: BroadcastExchange is not supported -BroadcastHashJoin disabled because not all child plans are native -Window is not supported +Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled SortMergeJoin disabled because not all child plans are native +TakeOrderedAndProject requires shuffle to be enabled +Window is not supported -Query: q57-v2.7. Comet Exec: Enabled (CometFilter, CometProject) +Query: q57-v2.7. Comet Exec: Enabled (CometBroadcastHashJoin, CometFilter, CometProject) Query: q57-v2.7: ExplainInfo: BroadcastExchange is not supported -BroadcastHashJoin disabled because not all child plans are native -Window is not supported +Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled SortMergeJoin disabled because not all child plans are native +TakeOrderedAndProject requires shuffle to be enabled +Window is not supported Query: q64-v2.7. Comet Exec: Enabled (CometFilter, CometProject) Query: q64-v2.7: ExplainInfo: BroadcastExchange is not supported -ObjectHashAggregate is not supported -BroadcastHashJoin disabled because not all child plans are native -xxhash64 is not supported +Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled SortMergeJoin disabled because not all child plans are native -Shuffle: unsupported Spark partitioning: org.apache.spark.sql.catalyst.plans.physical.RangePartitioning Query: q67a-v2.7. Comet Exec: Enabled (CometFilter, CometProject) Query: q67a-v2.7: ExplainInfo: BroadcastExchange is not supported -BroadcastHashJoin disabled because not all child plans are native +Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled +TakeOrderedAndProject requires shuffle to be enabled Union disabled because not all child plans are native Window is not supported -Query: q70a-v2.7. Comet Exec: Enabled (CometFilter, CometProject, CometSort) +Query: q70a-v2.7. Comet Exec: Enabled (CometFilter, CometProject) Query: q70a-v2.7: ExplainInfo: BroadcastExchange is not supported -BroadcastHashJoin disabled because not all child plans are native -Window is not supported +Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled SortMergeJoin disabled because not all child plans are native +TakeOrderedAndProject requires shuffle to be enabled Union disabled because not all child plans are native +Window is not supported Query: q72-v2.7. Comet Exec: Enabled (CometFilter, CometProject) Query: q72-v2.7: ExplainInfo: BroadcastExchange is not supported -BroadcastHashJoin disabled because not all child plans are native +Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled +ObjectHashAggregate is not supported +SortMergeJoin disabled because not all child plans are native +TakeOrderedAndProject requires shuffle to be enabled Query: q74-v2.7. Comet Exec: Enabled (CometFilter, CometProject) Query: q74-v2.7: ExplainInfo: BroadcastExchange is not supported -BroadcastHashJoin disabled because not all child plans are native +Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled SortMergeJoin disabled because not all child plans are native +TakeOrderedAndProject requires shuffle to be enabled Query: q75-v2.7. Comet Exec: Enabled (CometFilter, CometProject) Query: q75-v2.7: ExplainInfo: BroadcastExchange is not supported -BroadcastHashJoin disabled because not all child plans are native -Union disabled because not all child plans are native +Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled SortMergeJoin disabled because not all child plans are native +TakeOrderedAndProject requires shuffle to be enabled +Union disabled because not all child plans are native Query: q77a-v2.7. Comet Exec: Enabled (CometFilter, CometProject) Query: q77a-v2.7: ExplainInfo: BroadcastExchange is not supported -BroadcastHashJoin disabled because not all child plans are native +Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled SortMergeJoin disabled because not all child plans are native +TakeOrderedAndProject requires shuffle to be enabled Union disabled because not all child plans are native -Query: q78-v2.7. Comet Exec: Enabled (CometFilter) +Query: q78-v2.7. Comet Exec: Enabled (CometFilter, CometProject) Query: q78-v2.7: ExplainInfo: BroadcastExchange is not supported -BroadcastHashJoin disabled because not all child plans are native +Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled SortMergeJoin disabled because not all child plans are native +TakeOrderedAndProject requires shuffle to be enabled Query: q80a-v2.7. Comet Exec: Enabled (CometFilter, CometProject) Query: q80a-v2.7: ExplainInfo: BroadcastExchange is not supported -BroadcastHashJoin disabled because not all child plans are native +Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled +ObjectHashAggregate is not supported +SortMergeJoin disabled because not all child plans are native +TakeOrderedAndProject requires shuffle to be enabled Union disabled because not all child plans are native Query: q86a-v2.7. Comet Exec: Enabled (CometFilter, CometProject) Query: q86a-v2.7: ExplainInfo: BroadcastExchange is not supported -BroadcastHashJoin disabled because not all child plans are native +Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled +TakeOrderedAndProject requires shuffle to be enabled Union disabled because not all child plans are native Window is not supported Query: q98-v2.7. Comet Exec: Enabled (CometFilter, CometProject) Query: q98-v2.7: ExplainInfo: BroadcastExchange is not supported -BroadcastHashJoin disabled because not all child plans are native +Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled Window is not supported -Shuffle: unsupported Spark partitioning: org.apache.spark.sql.catalyst.plans.physical.RangePartitioning diff --git a/spark/inspections/CometTPCHQueriesList-results.txt b/spark/inspections/CometTPCHQueriesList-results.txt index b51286d80..ce896ba82 100644 --- a/spark/inspections/CometTPCHQueriesList-results.txt +++ b/spark/inspections/CometTPCHQueriesList-results.txt @@ -1,142 +1,135 @@ -Query: q1 TPCH Snappy. Comet Exec: Enabled (CometHashAggregate, CometFilter, CometProject) +Query: q1 TPCH Snappy. Comet Exec: Enabled (CometHashAggregate, CometProject) Query: q1 TPCH Snappy: ExplainInfo: -Shuffle: unsupported Spark partitioning: org.apache.spark.sql.catalyst.plans.physical.RangePartitioning +Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled Query: q2 TPCH Snappy. Comet Exec: Enabled (CometFilter, CometProject) Query: q2 TPCH Snappy: ExplainInfo: BroadcastExchange is not supported -BroadcastHashJoin disabled because not all child plans are native +Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled ObjectHashAggregate is not supported SortMergeJoin disabled because not all child plans are native -xxhash64 is not supported +TakeOrderedAndProject requires shuffle to be enabled -Query: q3 TPCH Snappy. Comet Exec: Enabled (CometFilter, CometProject, CometSort) +Query: q3 TPCH Snappy. Comet Exec: Enabled (CometFilter, CometProject) Query: q3 TPCH Snappy: ExplainInfo: -BroadcastExchange is not supported -BroadcastHashJoin disabled because not all child plans are native +Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled SortMergeJoin disabled because not all child plans are native +TakeOrderedAndProject requires shuffle to be enabled Query: q4 TPCH Snappy. Comet Exec: Enabled (CometFilter, CometProject) Query: q4 TPCH Snappy: ExplainInfo: -BroadcastExchange is not supported -BroadcastHashJoin disabled because not all child plans are native -Shuffle: unsupported Spark partitioning: org.apache.spark.sql.catalyst.plans.physical.RangePartitioning +Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled +SortMergeJoin disabled because not all child plans are native -Query: q5 TPCH Snappy. Comet Exec: Enabled (CometFilter, CometProject, CometSort) +Query: q5 TPCH Snappy. Comet Exec: Enabled (CometFilter, CometProject) Query: q5 TPCH Snappy: ExplainInfo: BroadcastExchange is not supported -BroadcastHashJoin disabled because not all child plans are native +Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled ObjectHashAggregate is not supported SortMergeJoin disabled because not all child plans are native -xxhash64 is not supported -Shuffle: unsupported Spark partitioning: org.apache.spark.sql.catalyst.plans.physical.RangePartitioning Query: q6 TPCH Snappy. Comet Exec: Enabled (CometHashAggregate, CometFilter, CometProject) Query: q6 TPCH Snappy: ExplainInfo: +Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled - -Query: q7 TPCH Snappy. Comet Exec: Enabled (CometFilter) +Query: q7 TPCH Snappy. Comet Exec: Enabled (CometFilter, CometProject) Query: q7 TPCH Snappy: ExplainInfo: BroadcastExchange is not supported -BroadcastHashJoin disabled because not all child plans are native -Shuffle: unsupported Spark partitioning: org.apache.spark.sql.catalyst.plans.physical.RangePartitioning +Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled +ObjectHashAggregate is not supported +SortMergeJoin disabled because not all child plans are native Query: q8 TPCH Snappy. Comet Exec: Enabled (CometFilter, CometProject) Query: q8 TPCH Snappy: ExplainInfo: BroadcastExchange is not supported -BroadcastHashJoin disabled because not all child plans are native -Shuffle: unsupported Spark partitioning: org.apache.spark.sql.catalyst.plans.physical.RangePartitioning +Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled +ObjectHashAggregate is not supported +SortMergeJoin disabled because not all child plans are native -Query: q9 TPCH Snappy. Comet Exec: Enabled (CometFilter, CometProject, CometSort) +Query: q9 TPCH Snappy. Comet Exec: Enabled (CometFilter, CometProject) Query: q9 TPCH Snappy: ExplainInfo: BroadcastExchange is not supported -BroadcastHashJoin disabled because not all child plans are native +Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled SortMergeJoin disabled because not all child plans are native -Shuffle: unsupported Spark partitioning: org.apache.spark.sql.catalyst.plans.physical.RangePartitioning -Query: q10 TPCH Snappy. Comet Exec: Enabled (CometFilter, CometProject, CometSort) +Query: q10 TPCH Snappy. Comet Exec: Enabled (CometFilter, CometProject) Query: q10 TPCH Snappy: ExplainInfo: BroadcastExchange is not supported -BroadcastHashJoin disabled because not all child plans are native +Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled SortMergeJoin disabled because not all child plans are native +TakeOrderedAndProject requires shuffle to be enabled Query: q11 TPCH Snappy. Comet Exec: Enabled (CometFilter, CometProject) Query: q11 TPCH Snappy: ExplainInfo: BroadcastExchange is not supported -BroadcastHashJoin disabled because not all child plans are native -Shuffle: unsupported Spark partitioning: org.apache.spark.sql.catalyst.plans.physical.RangePartitioning +Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled +ObjectHashAggregate is not supported +SortMergeJoin disabled because not all child plans are native Query: q12 TPCH Snappy. Comet Exec: Enabled (CometFilter, CometProject) Query: q12 TPCH Snappy: ExplainInfo: -BroadcastExchange is not supported -BroadcastHashJoin disabled because not all child plans are native -Shuffle: unsupported Spark partitioning: org.apache.spark.sql.catalyst.plans.physical.RangePartitioning +Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled +SortMergeJoin disabled because not all child plans are native Query: q13 TPCH Snappy. Comet Exec: Enabled (CometFilter, CometProject) Query: q13 TPCH Snappy: ExplainInfo: -BroadcastExchange is not supported -BroadcastHashJoin disabled because not all child plans are native -Shuffle: unsupported Spark partitioning: org.apache.spark.sql.catalyst.plans.physical.RangePartitioning +Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled +SortMergeJoin disabled because not all child plans are native Query: q14 TPCH Snappy. Comet Exec: Enabled (CometFilter, CometProject) Query: q14 TPCH Snappy: ExplainInfo: -BroadcastExchange is not supported -BroadcastHashJoin disabled because not all child plans are native +Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled +SortMergeJoin disabled because not all child plans are native Query: q15 TPCH Snappy. Comet Exec: Enabled (CometHashAggregate, CometFilter, CometProject) Query: q15 TPCH Snappy: ExplainInfo: -BroadcastExchange is not supported -BroadcastHashJoin disabled because not all child plans are native -Shuffle: unsupported Spark partitioning: org.apache.spark.sql.catalyst.plans.physical.RangePartitioning +Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled +SortMergeJoin disabled because not all child plans are native Query: q16 TPCH Snappy. Comet Exec: Enabled (CometFilter, CometProject) Query: q16 TPCH Snappy: ExplainInfo: BroadcastExchange is not supported -BroadcastHashJoin disabled because not all child plans are native -Shuffle: unsupported Spark partitioning: org.apache.spark.sql.catalyst.plans.physical.RangePartitioning +Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled +SortMergeJoin disabled because not all child plans are native -Query: q17 TPCH Snappy. Comet Exec: Enabled (CometHashAggregate, CometFilter, CometProject, CometSort) +Query: q17 TPCH Snappy. Comet Exec: Enabled (CometHashAggregate, CometFilter, CometProject) Query: q17 TPCH Snappy: ExplainInfo: -BroadcastExchange is not supported -BroadcastHashJoin disabled because not all child plans are native +Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled SortMergeJoin disabled because not all child plans are native Query: q18 TPCH Snappy. Comet Exec: Enabled (CometHashAggregate, CometFilter, CometProject) Query: q18 TPCH Snappy: ExplainInfo: -BroadcastExchange is not supported -BroadcastHashJoin disabled because not all child plans are native +Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled SortMergeJoin disabled because not all child plans are native +TakeOrderedAndProject requires shuffle to be enabled Query: q19 TPCH Snappy. Comet Exec: Enabled (CometFilter, CometProject) Query: q19 TPCH Snappy: ExplainInfo: -BroadcastExchange is not supported -BroadcastHashJoin disabled because not all child plans are native +Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled +SortMergeJoin disabled because not all child plans are native Query: q20 TPCH Snappy. Comet Exec: Enabled (CometFilter, CometProject) Query: q20 TPCH Snappy: ExplainInfo: BroadcastExchange is not supported -BroadcastHashJoin disabled because not all child plans are native +Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled ObjectHashAggregate is not supported SortMergeJoin disabled because not all child plans are native -xxhash64 is not supported -Shuffle: unsupported Spark partitioning: org.apache.spark.sql.catalyst.plans.physical.RangePartitioning -Query: q21 TPCH Snappy. Comet Exec: Enabled (CometFilter, CometProject, CometSort) +Query: q21 TPCH Snappy. Comet Exec: Enabled (CometFilter, CometProject) Query: q21 TPCH Snappy: ExplainInfo: +BroadcastExchange is not supported +Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled ObjectHashAggregate is not supported -Sort merge join with a join condition is not supported -xxhash64 is not supported SortMergeJoin disabled because not all child plans are native -BroadcastExchange is not supported -BroadcastHashJoin disabled because not all child plans are native +TakeOrderedAndProject requires shuffle to be enabled -Query: q22 TPCH Snappy. Comet Exec: Enabled (CometFilter) +Query: q22 TPCH Snappy. Comet Exec: Enabled (CometProject) Query: q22 TPCH Snappy: ExplainInfo: -BroadcastExchange is not supported -BroadcastHashJoin disabled because not all child plans are native -Shuffle: unsupported Spark partitioning: org.apache.spark.sql.catalyst.plans.physical.RangePartitioning +Comet does not guarantee correct results for cast from DecimalType(12,2) to DecimalType(16,6) with timezone Some(America/Los_Angeles) and evalMode LEGACY. To enable all incompatible casts, set spark.comet.cast.allowIncompatible=true +Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled +SortMergeJoin disabled because not all child plans are native Query: q1 TPCH Extended Snappy. Comet Exec: Enabled (CometHashAggregate, CometFilter, CometProject) Query: q1 TPCH Extended Snappy: ExplainInfo: - +Comet shuffle is not enabled: spark.sql.adaptive.coalescePartitions.enabled is enabled and spark.comet.shuffle.enforceMode.enabled is not enabled diff --git a/spark/src/main/scala/org/apache/comet/serde/QueryPlanSerde.scala b/spark/src/main/scala/org/apache/comet/serde/QueryPlanSerde.scala index 06b9bc7f4..fe60e9ba3 100644 --- a/spark/src/main/scala/org/apache/comet/serde/QueryPlanSerde.scala +++ b/spark/src/main/scala/org/apache/comet/serde/QueryPlanSerde.scala @@ -2090,6 +2090,21 @@ object QueryPlanSerde extends Logging with ShimQueryPlanSerde with CometExprShim // the seed is put at the end of the arguments scalarExprToProtoWithReturnType("murmur3_hash", IntegerType, exprs :+ seedExpr: _*) + case XxHash64(children, seed) => + val firstUnSupportedInput = children.find(c => !supportedDataType(c.dataType)) + if (firstUnSupportedInput.isDefined) { + withInfo(expr, s"Unsupported datatype ${firstUnSupportedInput.get.dataType}") + return None + } + val exprs = children.map(exprToProtoInternal(_, inputs)) + val seedBuilder = ExprOuterClass.Literal + .newBuilder() + .setDatatype(serializeDataType(LongType).get) + .setLongVal(seed) + val seedExpr = Some(ExprOuterClass.Expr.newBuilder().setLiteral(seedBuilder).build()) + // the seed is put at the end of the arguments + scalarExprToProtoWithReturnType("xxhash64", LongType, exprs :+ seedExpr: _*) + case Sha2(left, numBits) => if (!numBits.foldable) { withInfo(expr, "non literal numBits is not supported") diff --git a/spark/src/test/scala/org/apache/comet/CometExpressionSuite.scala b/spark/src/test/scala/org/apache/comet/CometExpressionSuite.scala index 34c794eb1..d418ec870 100644 --- a/spark/src/test/scala/org/apache/comet/CometExpressionSuite.scala +++ b/spark/src/test/scala/org/apache/comet/CometExpressionSuite.scala @@ -1486,6 +1486,7 @@ class CometExpressionSuite extends CometTestBase with AdaptiveSparkPlanHelper { |select |md5(col), md5(cast(a as string)), md5(cast(b as string)), |hash(col), hash(col, 1), hash(col, 0), hash(col, a, b), hash(b, a, col), + |xxhash64(col), xxhash64(col, 1), xxhash64(col, 0), xxhash64(col, a, b), xxhash64(b, a, col), |sha2(col, 0), sha2(col, 256), sha2(col, 224), sha2(col, 384), sha2(col, 512), sha2(col, 128) |from test |""".stripMargin) @@ -1508,14 +1509,13 @@ class CometExpressionSuite extends CometTestBase with AdaptiveSparkPlanHelper { val table = "test" withTable(table) { sql(s"create table $table(col string, a int, b float) using parquet") - // TODO: Add a Row generator in the data gen class and replace th following code - val col = dataGen.generateStrings(randomNumRows, timestampPattern, 6) - val colA = dataGen.generateInts(randomNumRows) - val colB = dataGen.generateFloats(randomNumRows) - val data = col.zip(colA).zip(colB).map { case ((a, b), c) => (a, b, c) } - data - .toDF("col", "a", "b") - .write + val tableSchema = spark.table(table).schema + val rows = dataGen.generateRows( + randomNumRows, + tableSchema, + Some(() => dataGen.generateString(timestampPattern, 6))) + val data = spark.createDataFrame(spark.sparkContext.parallelize(rows), tableSchema) + data.write .mode("append") .insertInto(table) // with random generated data @@ -1524,6 +1524,7 @@ class CometExpressionSuite extends CometTestBase with AdaptiveSparkPlanHelper { |select |md5(col), md5(cast(a as string)), --md5(cast(b as string)), |hash(col), hash(col, 1), hash(col, 0), hash(col, a, b), hash(b, a, col), + |xxhash64(col), xxhash64(col, 1), xxhash64(col, 0), xxhash64(col, a, b), xxhash64(b, a, col), |sha2(col, 0), sha2(col, 256), sha2(col, 224), sha2(col, 384), sha2(col, 512), sha2(col, 128) |from test |""".stripMargin) From a71f68b46754025310b70c4aeb789e470c810052 Mon Sep 17 00:00:00 2001 From: Trent Hauck <trent@trenthauck.com> Date: Mon, 3 Jun 2024 09:04:51 -0700 Subject: [PATCH 15/19] feat: add hex scalar function (#449) * feat: add hex scalar function * test: change hex test to use makeParquetFileAllTypes, support more types * test: add more columns to spark test * refactor: remove extra rust code * feat: support dictionary * fix: simplify hex_int64 * refactor: combine functions for hex byte/string * refactor: update vec collection * refactor: refactor hex to support byte ref * style: fix clippy * refactor: remove scalar handling * style: new lines in expression test file * fix: handle large strings --- .../datafusion/expressions/scalar_funcs.rs | 7 + .../expressions/scalar_funcs/hex.rs | 306 ++++++++++++++++++ .../apache/comet/serde/QueryPlanSerde.scala | 7 + .../apache/comet/CometExpressionSuite.scala | 16 + 4 files changed, 336 insertions(+) create mode 100644 core/src/execution/datafusion/expressions/scalar_funcs/hex.rs diff --git a/core/src/execution/datafusion/expressions/scalar_funcs.rs b/core/src/execution/datafusion/expressions/scalar_funcs.rs index 0f254004b..dc333e8be 100644 --- a/core/src/execution/datafusion/expressions/scalar_funcs.rs +++ b/core/src/execution/datafusion/expressions/scalar_funcs.rs @@ -55,6 +55,9 @@ use unicode_segmentation::UnicodeSegmentation; mod unhex; use unhex::spark_unhex; +mod hex; +use hex::spark_hex; + macro_rules! make_comet_scalar_udf { ($name:expr, $func:ident, $data_type:ident) => {{ let scalar_func = CometScalarFunction::new( @@ -108,6 +111,10 @@ pub fn create_comet_physical_fun( "make_decimal" => { make_comet_scalar_udf!("make_decimal", spark_make_decimal, data_type) } + "hex" => { + let func = Arc::new(spark_hex); + make_comet_scalar_udf!("hex", func, without data_type) + } "unhex" => { let func = Arc::new(spark_unhex); make_comet_scalar_udf!("unhex", func, without data_type) diff --git a/core/src/execution/datafusion/expressions/scalar_funcs/hex.rs b/core/src/execution/datafusion/expressions/scalar_funcs/hex.rs new file mode 100644 index 000000000..ea572574a --- /dev/null +++ b/core/src/execution/datafusion/expressions/scalar_funcs/hex.rs @@ -0,0 +1,306 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +use std::sync::Arc; + +use arrow::{ + array::{as_dictionary_array, as_largestring_array, as_string_array}, + datatypes::Int32Type, +}; +use arrow_array::StringArray; +use arrow_schema::DataType; +use datafusion::logical_expr::ColumnarValue; +use datafusion_common::{ + cast::{as_binary_array, as_fixed_size_binary_array, as_int64_array}, + exec_err, DataFusionError, +}; +use std::fmt::Write; + +fn hex_int64(num: i64) -> String { + format!("{:X}", num) +} + +fn hex_bytes<T: AsRef<[u8]>>(bytes: T) -> Result<String, std::fmt::Error> { + let bytes = bytes.as_ref(); + let length = bytes.len(); + let mut hex_string = String::with_capacity(length * 2); + for &byte in bytes { + write!(&mut hex_string, "{:02X}", byte)?; + } + Ok(hex_string) +} + +pub(super) fn spark_hex(args: &[ColumnarValue]) -> Result<ColumnarValue, DataFusionError> { + if args.len() != 1 { + return Err(DataFusionError::Internal( + "hex expects exactly one argument".to_string(), + )); + } + + match &args[0] { + ColumnarValue::Array(array) => match array.data_type() { + DataType::Int64 => { + let array = as_int64_array(array)?; + + let hexed_array: StringArray = array.iter().map(|v| v.map(hex_int64)).collect(); + + Ok(ColumnarValue::Array(Arc::new(hexed_array))) + } + DataType::Utf8 => { + let array = as_string_array(array); + + let hexed: StringArray = array + .iter() + .map(|v| v.map(hex_bytes).transpose()) + .collect::<Result<_, _>>()?; + + Ok(ColumnarValue::Array(Arc::new(hexed))) + } + DataType::LargeUtf8 => { + let array = as_largestring_array(array); + + let hexed: StringArray = array + .iter() + .map(|v| v.map(hex_bytes).transpose()) + .collect::<Result<_, _>>()?; + + Ok(ColumnarValue::Array(Arc::new(hexed))) + } + DataType::Binary => { + let array = as_binary_array(array)?; + + let hexed: StringArray = array + .iter() + .map(|v| v.map(hex_bytes).transpose()) + .collect::<Result<_, _>>()?; + + Ok(ColumnarValue::Array(Arc::new(hexed))) + } + DataType::FixedSizeBinary(_) => { + let array = as_fixed_size_binary_array(array)?; + + let hexed: StringArray = array + .iter() + .map(|v| v.map(hex_bytes).transpose()) + .collect::<Result<_, _>>()?; + + Ok(ColumnarValue::Array(Arc::new(hexed))) + } + DataType::Dictionary(_, value_type) if matches!(**value_type, DataType::Int64) => { + let dict = as_dictionary_array::<Int32Type>(&array); + + let hexed_values = as_int64_array(dict.values())?; + let values = hexed_values + .iter() + .map(|v| v.map(hex_int64)) + .collect::<Vec<_>>(); + + let keys = dict.keys().clone(); + let mut new_keys = Vec::with_capacity(values.len()); + + for key in keys.iter() { + let key = key.map(|k| values[k as usize].clone()).unwrap_or(None); + new_keys.push(key); + } + + let string_array_values = StringArray::from(new_keys); + Ok(ColumnarValue::Array(Arc::new(string_array_values))) + } + DataType::Dictionary(_, value_type) if matches!(**value_type, DataType::Utf8) => { + let dict = as_dictionary_array::<Int32Type>(&array); + + let hexed_values = as_string_array(dict.values()); + let values: Vec<Option<String>> = hexed_values + .iter() + .map(|v| v.map(hex_bytes).transpose()) + .collect::<Result<_, _>>()?; + + let keys = dict.keys().clone(); + + let mut new_keys = Vec::with_capacity(values.len()); + + for key in keys.iter() { + let key = key.map(|k| values[k as usize].clone()).unwrap_or(None); + new_keys.push(key); + } + + let string_array_values = StringArray::from(new_keys); + Ok(ColumnarValue::Array(Arc::new(string_array_values))) + } + DataType::Dictionary(_, value_type) if matches!(**value_type, DataType::Binary) => { + let dict = as_dictionary_array::<Int32Type>(&array); + + let hexed_values = as_binary_array(dict.values())?; + let values: Vec<Option<String>> = hexed_values + .iter() + .map(|v| v.map(hex_bytes).transpose()) + .collect::<Result<_, _>>()?; + + let keys = dict.keys().clone(); + let mut new_keys = Vec::with_capacity(values.len()); + + for key in keys.iter() { + let key = key.map(|k| values[k as usize].clone()).unwrap_or(None); + new_keys.push(key); + } + + let string_array_values = StringArray::from(new_keys); + Ok(ColumnarValue::Array(Arc::new(string_array_values))) + } + _ => exec_err!( + "hex got an unexpected argument type: {:?}", + array.data_type() + ), + }, + _ => exec_err!("native hex does not support scalar values at this time"), + } +} + +#[cfg(test)] +mod test { + use std::sync::Arc; + + use arrow::{ + array::{ + as_string_array, BinaryDictionaryBuilder, PrimitiveDictionaryBuilder, StringBuilder, + StringDictionaryBuilder, + }, + datatypes::{Int32Type, Int64Type}, + }; + use arrow_array::{Int64Array, StringArray}; + use datafusion::logical_expr::ColumnarValue; + + #[test] + fn test_dictionary_hex_utf8() { + let mut input_builder = StringDictionaryBuilder::<Int32Type>::new(); + input_builder.append_value("hi"); + input_builder.append_value("bye"); + input_builder.append_null(); + input_builder.append_value("rust"); + let input = input_builder.finish(); + + let mut string_builder = StringBuilder::new(); + string_builder.append_value("6869"); + string_builder.append_value("627965"); + string_builder.append_null(); + string_builder.append_value("72757374"); + let expected = string_builder.finish(); + + let columnar_value = ColumnarValue::Array(Arc::new(input)); + let result = super::spark_hex(&[columnar_value]).unwrap(); + + let result = match result { + ColumnarValue::Array(array) => array, + _ => panic!("Expected array"), + }; + + let result = as_string_array(&result); + + assert_eq!(result, &expected); + } + + #[test] + fn test_dictionary_hex_int64() { + let mut input_builder = PrimitiveDictionaryBuilder::<Int32Type, Int64Type>::new(); + input_builder.append_value(1); + input_builder.append_value(2); + input_builder.append_null(); + input_builder.append_value(3); + let input = input_builder.finish(); + + let mut string_builder = StringBuilder::new(); + string_builder.append_value("1"); + string_builder.append_value("2"); + string_builder.append_null(); + string_builder.append_value("3"); + let expected = string_builder.finish(); + + let columnar_value = ColumnarValue::Array(Arc::new(input)); + let result = super::spark_hex(&[columnar_value]).unwrap(); + + let result = match result { + ColumnarValue::Array(array) => array, + _ => panic!("Expected array"), + }; + + let result = as_string_array(&result); + + assert_eq!(result, &expected); + } + + #[test] + fn test_dictionary_hex_binary() { + let mut input_builder = BinaryDictionaryBuilder::<Int32Type>::new(); + input_builder.append_value("1"); + input_builder.append_value("1"); + input_builder.append_null(); + input_builder.append_value("3"); + let input = input_builder.finish(); + + let mut expected_builder = StringBuilder::new(); + expected_builder.append_value("31"); + expected_builder.append_value("31"); + expected_builder.append_null(); + expected_builder.append_value("33"); + let expected = expected_builder.finish(); + + let columnar_value = ColumnarValue::Array(Arc::new(input)); + let result = super::spark_hex(&[columnar_value]).unwrap(); + + let result = match result { + ColumnarValue::Array(array) => array, + _ => panic!("Expected array"), + }; + + let result = as_string_array(&result); + + assert_eq!(result, &expected); + } + + #[test] + fn test_hex_int64() { + let num = 1234; + let hexed = super::hex_int64(num); + assert_eq!(hexed, "4D2".to_string()); + + let num = -1; + let hexed = super::hex_int64(num); + assert_eq!(hexed, "FFFFFFFFFFFFFFFF".to_string()); + } + + #[test] + fn test_spark_hex_int64() { + let int_array = Int64Array::from(vec![Some(1), Some(2), None, Some(3)]); + let columnar_value = ColumnarValue::Array(Arc::new(int_array)); + + let result = super::spark_hex(&[columnar_value]).unwrap(); + let result = match result { + ColumnarValue::Array(array) => array, + _ => panic!("Expected array"), + }; + + let string_array = as_string_array(&result); + let expected_array = StringArray::from(vec![ + Some("1".to_string()), + Some("2".to_string()), + None, + Some("3".to_string()), + ]); + + assert_eq!(string_array, &expected_array); + } +} diff --git a/spark/src/main/scala/org/apache/comet/serde/QueryPlanSerde.scala b/spark/src/main/scala/org/apache/comet/serde/QueryPlanSerde.scala index fe60e9ba3..c1188e193 100644 --- a/spark/src/main/scala/org/apache/comet/serde/QueryPlanSerde.scala +++ b/spark/src/main/scala/org/apache/comet/serde/QueryPlanSerde.scala @@ -1509,6 +1509,13 @@ object QueryPlanSerde extends Logging with ShimQueryPlanSerde with CometExprShim val optExpr = scalarExprToProto("atan2", leftExpr, rightExpr) optExprWithInfo(optExpr, expr, left, right) + case Hex(child) => + val childExpr = exprToProtoInternal(child, inputs) + val optExpr = + scalarExprToProtoWithReturnType("hex", StringType, childExpr) + + optExprWithInfo(optExpr, expr, child) + case e: Unhex if !isSpark32 => val unHex = unhexSerde(e) diff --git a/spark/src/test/scala/org/apache/comet/CometExpressionSuite.scala b/spark/src/test/scala/org/apache/comet/CometExpressionSuite.scala index d418ec870..e15b09ca2 100644 --- a/spark/src/test/scala/org/apache/comet/CometExpressionSuite.scala +++ b/spark/src/test/scala/org/apache/comet/CometExpressionSuite.scala @@ -1062,6 +1062,22 @@ class CometExpressionSuite extends CometTestBase with AdaptiveSparkPlanHelper { } } } + + test("hex") { + Seq(true, false).foreach { dictionaryEnabled => + withTempDir { dir => + val path = new Path(dir.toURI.toString, "hex.parquet") + makeParquetFileAllTypes(path, dictionaryEnabled = dictionaryEnabled, 10000) + + withParquetTable(path.toString, "tbl") { + // _9 and _10 (uint8 and uint16) not supported + checkSparkAnswerAndOperator( + "SELECT hex(_1), hex(_2), hex(_3), hex(_4), hex(_5), hex(_6), hex(_7), hex(_8), hex(_11), hex(_12), hex(_13), hex(_14), hex(_15), hex(_16), hex(_17), hex(_18), hex(_19), hex(_20) FROM tbl") + } + } + } + } + test("unhex") { // When running against Spark 3.2, we include a bug fix for https://issues.apache.org/jira/browse/SPARK-40924 that // was added in Spark 3.3, so although Comet's behavior is more correct when running against Spark 3.2, it is not From 8e0114907bc60a98583f751c1ac25bd306833544 Mon Sep 17 00:00:00 2001 From: Andy Grove <agrove@apache.org> Date: Mon, 3 Jun 2024 10:30:17 -0600 Subject: [PATCH 16/19] feat: Add "Comet Fuzz" fuzz-testing utility (#472) * Add Comet Fuzz utility * update git ignore * asf headers * make comet-fuzz a submodule * spotless * improve comparison logic, make queries deterministic * make tests more deterministic * lint * fix for scala 2.13 * Move TODO comments to README and address review comments * refactor and formatting * more refactoring * fix decimal * comment out some types for now * fix CI failure * enable boolean and binary types * improve formatting of SQL in report * enable decimal types * trigger tests * trigger CI * remove hard-coded master --- dev/scalastyle-config.xml | 2 +- fuzz-testing/.gitignore | 6 + fuzz-testing/README.md | 100 +++++++++++ fuzz-testing/pom.xml | 105 +++++++++++ .../scala/org/apache/comet/fuzz/DataGen.scala | 151 ++++++++++++++++ .../scala/org/apache/comet/fuzz/Main.scala | 87 +++++++++ .../scala/org/apache/comet/fuzz/Meta.scala | 109 +++++++++++ .../org/apache/comet/fuzz/QueryGen.scala | 121 +++++++++++++ .../org/apache/comet/fuzz/QueryRunner.scala | 170 ++++++++++++++++++ .../scala/org/apache/comet/fuzz/Utils.scala | 46 +++++ pom.xml | 7 + 11 files changed, 903 insertions(+), 1 deletion(-) create mode 100644 fuzz-testing/.gitignore create mode 100644 fuzz-testing/README.md create mode 100644 fuzz-testing/pom.xml create mode 100644 fuzz-testing/src/main/scala/org/apache/comet/fuzz/DataGen.scala create mode 100644 fuzz-testing/src/main/scala/org/apache/comet/fuzz/Main.scala create mode 100644 fuzz-testing/src/main/scala/org/apache/comet/fuzz/Meta.scala create mode 100644 fuzz-testing/src/main/scala/org/apache/comet/fuzz/QueryGen.scala create mode 100644 fuzz-testing/src/main/scala/org/apache/comet/fuzz/QueryRunner.scala create mode 100644 fuzz-testing/src/main/scala/org/apache/comet/fuzz/Utils.scala diff --git a/dev/scalastyle-config.xml b/dev/scalastyle-config.xml index 92ba690ad..9de6df51e 100644 --- a/dev/scalastyle-config.xml +++ b/dev/scalastyle-config.xml @@ -242,7 +242,7 @@ This file is divided into 3 sections: <parameter name="groups">java,scala,org,apache,3rdParty,comet</parameter> <parameter name="group.java">javax?\..*</parameter> <parameter name="group.scala">scala\..*</parameter> - <parameter name="group.org">org\.(?!apache\.comet).*</parameter> + <parameter name="group.org">org\.(?!apache).*</parameter> <parameter name="group.apache">org\.apache\.(?!comet).*</parameter> <parameter name="group.3rdParty">(?!(javax?\.|scala\.|org\.apache\.comet\.)).*</parameter> <parameter name="group.comet">org\.apache\.comet\..*</parameter> diff --git a/fuzz-testing/.gitignore b/fuzz-testing/.gitignore new file mode 100644 index 000000000..570ff02a7 --- /dev/null +++ b/fuzz-testing/.gitignore @@ -0,0 +1,6 @@ +.idea +target +spark-warehouse +queries.sql +results*.md +test*.parquet \ No newline at end of file diff --git a/fuzz-testing/README.md b/fuzz-testing/README.md new file mode 100644 index 000000000..56af359f2 --- /dev/null +++ b/fuzz-testing/README.md @@ -0,0 +1,100 @@ +<!-- +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. +--> + +# Comet Fuzz + +Comet Fuzz is a standalone project for generating random data and queries and executing queries against Spark +with Comet disabled and enabled and checking for incompatibilities. + +Although it is a simple tool it has already been useful in finding many bugs. + +Comet Fuzz is inspired by the [SparkFuzz](https://ir.cwi.nl/pub/30222) paper from Databricks and CWI. + +## Roadmap + +Planned areas of improvement: + +- Support for all data types, expressions, and operators supported by Comet +- Explicit casts +- Unary and binary arithmetic expressions +- IF and CASE WHEN expressions +- Complex (nested) expressions +- Literal scalar values in queries +- Add option to avoid grouping and sorting on floating-point columns +- Improve join query support: + - Support joins without join keys + - Support composite join keys + - Support multiple join keys + - Support join conditions that use expressions + +## Usage + +Build the jar file first. + +```shell +mvn package +``` + +Set appropriate values for `SPARK_HOME`, `SPARK_MASTER`, and `COMET_JAR` environment variables and then use +`spark-submit` to run CometFuzz against a Spark cluster. + +### Generating Data Files + +```shell +$SPARK_HOME/bin/spark-submit \ + --master $SPARK_MASTER \ + --class org.apache.comet.fuzz.Main \ + target/comet-fuzz-spark3.4_2.12-0.1.0-SNAPSHOT-jar-with-dependencies.jar \ + data --num-files=2 --num-rows=200 --num-columns=100 +``` + +### Generating Queries + +Generate random queries that are based on the available test files. + +```shell +$SPARK_HOME/bin/spark-submit \ + --master $SPARK_MASTER \ + --class org.apache.comet.fuzz.Main \ + target/comet-fuzz-spark3.4_2.12-0.1.0-SNAPSHOT-jar-with-dependencies.jar \ + queries --num-files=2 --num-queries=500 +``` + +Note that the output filename is currently hard-coded as `queries.sql` + +### Execute Queries + +```shell +$SPARK_HOME/bin/spark-submit \ + --master $SPARK_MASTER \ + --conf spark.sql.extensions=org.apache.comet.CometSparkSessionExtensions \ + --conf spark.comet.enabled=true \ + --conf spark.comet.exec.enabled=true \ + --conf spark.comet.exec.all.enabled=true \ + --conf spark.shuffle.manager=org.apache.spark.sql.comet.execution.shuffle.CometShuffleManager \ + --conf spark.comet.exec.shuffle.enabled=true \ + --conf spark.comet.exec.shuffle.mode=auto \ + --jars $COMET_JAR \ + --driver-class-path $COMET_JAR \ + --class org.apache.comet.fuzz.Main \ + target/comet-fuzz-spark3.4_2.12-0.1.0-SNAPSHOT-jar-with-dependencies.jar \ + run --num-files=2 --filename=queries.sql +``` + +Note that the output filename is currently hard-coded as `results-${System.currentTimeMillis()}.md` diff --git a/fuzz-testing/pom.xml b/fuzz-testing/pom.xml new file mode 100644 index 000000000..f69d959f9 --- /dev/null +++ b/fuzz-testing/pom.xml @@ -0,0 +1,105 @@ +<!-- +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. +--> + +<project xmlns="http://maven.apache.org/POM/4.0.0" + xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" + xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd"> + <modelVersion>4.0.0</modelVersion> + + <parent> + <groupId>org.apache.comet</groupId> + <artifactId>comet-parent-spark${spark.version.short}_${scala.binary.version}</artifactId> + <version>0.1.0-SNAPSHOT</version> + <relativePath>../pom.xml</relativePath> + </parent> + + <artifactId>comet-fuzz-spark${spark.version.short}_${scala.binary.version}</artifactId> + <name>comet-fuzz</name> + <url>http://maven.apache.org</url> + <packaging>jar</packaging> + + <properties> + <!-- Reverse default (skip installation), and then enable only for child modules --> + <maven.deploy.skip>false</maven.deploy.skip> + </properties> + + <dependencies> + <dependency> + <groupId>org.scala-lang</groupId> + <artifactId>scala-library</artifactId> + <version>${scala.version}</version> + <scope>provided</scope> + </dependency> + <dependency> + <groupId>org.apache.spark</groupId> + <artifactId>spark-sql_${scala.binary.version}</artifactId> + <scope>provided</scope> + </dependency> + <dependency> + <groupId>org.rogach</groupId> + <artifactId>scallop_${scala.binary.version}</artifactId> + </dependency> + </dependencies> + + <build> + <sourceDirectory>src/main/scala</sourceDirectory> + <plugins> + <plugin> + <groupId>org.apache.maven.plugins</groupId> + <artifactId>maven-compiler-plugin</artifactId> + <version>3.1</version> + <configuration> + <source>${java.version}</source> + <target>${java.version}</target> + </configuration> + </plugin> + <plugin> + <groupId>net.alchim31.maven</groupId> + <artifactId>scala-maven-plugin</artifactId> + <version>4.7.2</version> + <executions> + <execution> + <goals> + <goal>compile</goal> + <goal>testCompile</goal> + </goals> + </execution> + </executions> + </plugin> + <plugin> + <artifactId>maven-assembly-plugin</artifactId> + <version>3.3.0</version> + <configuration> + <descriptorRefs> + <descriptorRef>jar-with-dependencies</descriptorRef> + </descriptorRefs> + </configuration> + <executions> + <execution> + <id>make-assembly</id> + <phase>package</phase> + <goals> + <goal>single</goal> + </goals> + </execution> + </executions> + </plugin> + </plugins> + </build> +</project> diff --git a/fuzz-testing/src/main/scala/org/apache/comet/fuzz/DataGen.scala b/fuzz-testing/src/main/scala/org/apache/comet/fuzz/DataGen.scala new file mode 100644 index 000000000..47a6bd879 --- /dev/null +++ b/fuzz-testing/src/main/scala/org/apache/comet/fuzz/DataGen.scala @@ -0,0 +1,151 @@ +/* + * 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.fuzz + +import java.math.{BigDecimal, RoundingMode} +import java.nio.charset.Charset +import java.sql.Timestamp + +import scala.util.Random + +import org.apache.spark.sql.{Row, SaveMode, SparkSession} +import org.apache.spark.sql.types.{DataType, DataTypes, DecimalType, StructField, StructType} + +object DataGen { + + def generateRandomFiles( + r: Random, + spark: SparkSession, + numFiles: Int, + numRows: Int, + numColumns: Int): Unit = { + for (i <- 0 until numFiles) { + generateRandomParquetFile(r, spark, s"test$i.parquet", numRows, numColumns) + } + } + + def generateRandomParquetFile( + r: Random, + spark: SparkSession, + filename: String, + numRows: Int, + numColumns: Int): Unit = { + + // generate schema using random data types + val fields = Range(0, numColumns) + .map(i => StructField(s"c$i", Utils.randomWeightedChoice(Meta.dataTypes), nullable = true)) + val schema = StructType(fields) + + // generate columnar data + val cols: Seq[Seq[Any]] = fields.map(f => generateColumn(r, f.dataType, numRows)) + + // convert to rows + val rows = Range(0, numRows).map(rowIndex => { + Row.fromSeq(cols.map(_(rowIndex))) + }) + + val df = spark.createDataFrame(spark.sparkContext.parallelize(rows), schema) + df.write.mode(SaveMode.Overwrite).parquet(filename) + } + + def generateColumn(r: Random, dataType: DataType, numRows: Int): Seq[Any] = { + dataType match { + case DataTypes.BooleanType => + generateColumn(r, DataTypes.LongType, numRows) + .map(_.asInstanceOf[Long].toShort) + .map(s => s % 2 == 0) + case DataTypes.ByteType => + generateColumn(r, DataTypes.LongType, numRows).map(_.asInstanceOf[Long].toByte) + case DataTypes.ShortType => + generateColumn(r, DataTypes.LongType, numRows).map(_.asInstanceOf[Long].toShort) + case DataTypes.IntegerType => + generateColumn(r, DataTypes.LongType, numRows).map(_.asInstanceOf[Long].toInt) + case DataTypes.LongType => + Range(0, numRows).map(_ => { + r.nextInt(50) match { + case 0 => null + case 1 => 0L + case 2 => Byte.MinValue.toLong + case 3 => Byte.MaxValue.toLong + case 4 => Short.MinValue.toLong + case 5 => Short.MaxValue.toLong + case 6 => Int.MinValue.toLong + case 7 => Int.MaxValue.toLong + case 8 => Long.MinValue + case 9 => Long.MaxValue + case _ => r.nextLong() + } + }) + case DataTypes.FloatType => + Range(0, numRows).map(_ => { + r.nextInt(20) match { + case 0 => null + case 1 => Float.NegativeInfinity + case 2 => Float.PositiveInfinity + case 3 => Float.MinValue + case 4 => Float.MaxValue + case 5 => 0.0f + case 6 => -0.0f + case _ => r.nextFloat() + } + }) + case DataTypes.DoubleType => + Range(0, numRows).map(_ => { + r.nextInt(20) match { + case 0 => null + case 1 => Double.NegativeInfinity + case 2 => Double.PositiveInfinity + case 3 => Double.MinValue + case 4 => Double.MaxValue + case 5 => 0.0 + case 6 => -0.0 + case _ => r.nextDouble() + } + }) + case dt: DecimalType => + Range(0, numRows).map(_ => + new BigDecimal(r.nextDouble()).setScale(dt.scale, RoundingMode.HALF_UP)) + case DataTypes.StringType => + Range(0, numRows).map(_ => { + r.nextInt(10) match { + case 0 => null + case 1 => r.nextInt().toByte.toString + case 2 => r.nextLong().toString + case 3 => r.nextDouble().toString + case _ => r.nextString(8) + } + }) + case DataTypes.BinaryType => + generateColumn(r, DataTypes.StringType, numRows) + .map { + case x: String => + x.getBytes(Charset.defaultCharset()) + case _ => + null + } + case DataTypes.DateType => + Range(0, numRows).map(_ => new java.sql.Date(1716645600011L + r.nextInt())) + case DataTypes.TimestampType => + Range(0, numRows).map(_ => new Timestamp(1716645600011L + r.nextInt())) + case _ => throw new IllegalStateException(s"Cannot generate data for $dataType yet") + } + } + +} diff --git a/fuzz-testing/src/main/scala/org/apache/comet/fuzz/Main.scala b/fuzz-testing/src/main/scala/org/apache/comet/fuzz/Main.scala new file mode 100644 index 000000000..799885d65 --- /dev/null +++ b/fuzz-testing/src/main/scala/org/apache/comet/fuzz/Main.scala @@ -0,0 +1,87 @@ +/* + * 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.fuzz + +import scala.util.Random + +import org.rogach.scallop.{ScallopConf, Subcommand} +import org.rogach.scallop.ScallopOption + +import org.apache.spark.sql.SparkSession + +class Conf(arguments: Seq[String]) extends ScallopConf(arguments) { + object generateData extends Subcommand("data") { + val numFiles: ScallopOption[Int] = opt[Int](required = true) + val numRows: ScallopOption[Int] = opt[Int](required = true) + val numColumns: ScallopOption[Int] = opt[Int](required = true) + } + addSubcommand(generateData) + object generateQueries extends Subcommand("queries") { + val numFiles: ScallopOption[Int] = opt[Int](required = false) + val numQueries: ScallopOption[Int] = opt[Int](required = true) + } + addSubcommand(generateQueries) + object runQueries extends Subcommand("run") { + val filename: ScallopOption[String] = opt[String](required = true) + val numFiles: ScallopOption[Int] = opt[Int](required = false) + val showMatchingResults: ScallopOption[Boolean] = opt[Boolean](required = false) + } + addSubcommand(runQueries) + verify() +} + +object Main { + + lazy val spark: SparkSession = SparkSession + .builder() + .getOrCreate() + + def main(args: Array[String]): Unit = { + val r = new Random(42) + + val conf = new Conf(args.toIndexedSeq) + conf.subcommand match { + case Some(conf.generateData) => + DataGen.generateRandomFiles( + r, + spark, + numFiles = conf.generateData.numFiles(), + numRows = conf.generateData.numRows(), + numColumns = conf.generateData.numColumns()) + case Some(conf.generateQueries) => + QueryGen.generateRandomQueries( + r, + spark, + numFiles = conf.generateQueries.numFiles(), + conf.generateQueries.numQueries()) + case Some(conf.runQueries) => + QueryRunner.runQueries( + spark, + conf.runQueries.numFiles(), + conf.runQueries.filename(), + conf.runQueries.showMatchingResults()) + case _ => + // scalastyle:off println + println("Invalid subcommand") + // scalastyle:on println + sys.exit(-1) + } + } +} diff --git a/fuzz-testing/src/main/scala/org/apache/comet/fuzz/Meta.scala b/fuzz-testing/src/main/scala/org/apache/comet/fuzz/Meta.scala new file mode 100644 index 000000000..13ebbf9ed --- /dev/null +++ b/fuzz-testing/src/main/scala/org/apache/comet/fuzz/Meta.scala @@ -0,0 +1,109 @@ +/* + * 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.fuzz + +import org.apache.spark.sql.types.DataType +import org.apache.spark.sql.types.DataTypes + +object Meta { + + val dataTypes: Seq[(DataType, Double)] = Seq( + (DataTypes.BooleanType, 0.1), + (DataTypes.ByteType, 0.2), + (DataTypes.ShortType, 0.2), + (DataTypes.IntegerType, 0.2), + (DataTypes.LongType, 0.2), + (DataTypes.FloatType, 0.2), + (DataTypes.DoubleType, 0.2), + (DataTypes.createDecimalType(10, 2), 0.2), + (DataTypes.DateType, 0.2), + (DataTypes.TimestampType, 0.2), + // TimestampNTZType only in Spark 3.4+ + // (DataTypes.TimestampNTZType, 0.2), + (DataTypes.StringType, 0.2), + (DataTypes.BinaryType, 0.1)) + + val stringScalarFunc: Seq[Function] = Seq( + Function("substring", 3), + Function("coalesce", 1), + Function("starts_with", 2), + Function("ends_with", 2), + Function("contains", 2), + Function("ascii", 1), + Function("bit_length", 1), + Function("octet_length", 1), + Function("upper", 1), + Function("lower", 1), + Function("chr", 1), + Function("init_cap", 1), + Function("trim", 1), + Function("ltrim", 1), + Function("rtrim", 1), + Function("btrim", 1), + Function("concat_ws", 2), + Function("repeat", 2), + Function("length", 1), + Function("reverse", 1), + Function("in_str", 2), + Function("replace", 2), + Function("translate", 2)) + + val dateScalarFunc: Seq[Function] = + Seq(Function("year", 1), Function("hour", 1), Function("minute", 1), Function("second", 1)) + + val mathScalarFunc: Seq[Function] = Seq( + Function("abs", 1), + Function("acos", 1), + Function("asin", 1), + Function("atan", 1), + Function("Atan2", 1), + Function("Cos", 1), + Function("Exp", 2), + Function("Ln", 1), + Function("Log10", 1), + Function("Log2", 1), + Function("Pow", 2), + Function("Round", 1), + Function("Signum", 1), + Function("Sin", 1), + Function("Sqrt", 1), + Function("Tan", 1), + Function("Ceil", 1), + Function("Floor", 1)) + + val scalarFunc: Seq[Function] = stringScalarFunc ++ dateScalarFunc ++ mathScalarFunc + + val aggFunc: Seq[Function] = Seq( + Function("min", 1), + Function("max", 1), + Function("count", 1), + Function("avg", 1), + Function("sum", 1), + Function("first", 1), + Function("last", 1), + Function("var_pop", 1), + Function("var_samp", 1), + Function("covar_pop", 1), + Function("covar_samp", 1), + Function("stddev_pop", 1), + Function("stddev_samp", 1), + Function("corr", 2)) + +} diff --git a/fuzz-testing/src/main/scala/org/apache/comet/fuzz/QueryGen.scala b/fuzz-testing/src/main/scala/org/apache/comet/fuzz/QueryGen.scala new file mode 100644 index 000000000..1daa26200 --- /dev/null +++ b/fuzz-testing/src/main/scala/org/apache/comet/fuzz/QueryGen.scala @@ -0,0 +1,121 @@ +/* + * 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.fuzz + +import java.io.{BufferedWriter, FileWriter} + +import scala.collection.mutable +import scala.util.Random + +import org.apache.spark.sql.SparkSession + +object QueryGen { + + def generateRandomQueries( + r: Random, + spark: SparkSession, + numFiles: Int, + numQueries: Int): Unit = { + for (i <- 0 until numFiles) { + spark.read.parquet(s"test$i.parquet").createTempView(s"test$i") + } + + val w = new BufferedWriter(new FileWriter("queries.sql")) + + val uniqueQueries = mutable.HashSet[String]() + + for (_ <- 0 until numQueries) { + val sql = r.nextInt().abs % 3 match { + case 0 => generateJoin(r, spark, numFiles) + case 1 => generateAggregate(r, spark, numFiles) + case 2 => generateScalar(r, spark, numFiles) + } + if (!uniqueQueries.contains(sql)) { + uniqueQueries += sql + w.write(sql + "\n") + } + } + w.close() + } + + private def generateAggregate(r: Random, spark: SparkSession, numFiles: Int): String = { + val tableName = s"test${r.nextInt(numFiles)}" + val table = spark.table(tableName) + + val func = Utils.randomChoice(Meta.aggFunc, r) + val args = Range(0, func.num_args) + .map(_ => Utils.randomChoice(table.columns, r)) + + val groupingCols = Range(0, 2).map(_ => Utils.randomChoice(table.columns, r)) + + if (groupingCols.isEmpty) { + s"SELECT ${args.mkString(", ")}, ${func.name}(${args.mkString(", ")}) AS x " + + s"FROM $tableName " + + s"ORDER BY ${args.mkString(", ")};" + } else { + s"SELECT ${groupingCols.mkString(", ")}, ${func.name}(${args.mkString(", ")}) " + + s"FROM $tableName " + + s"GROUP BY ${groupingCols.mkString(",")} " + + s"ORDER BY ${groupingCols.mkString(", ")};" + } + } + + private def generateScalar(r: Random, spark: SparkSession, numFiles: Int): String = { + val tableName = s"test${r.nextInt(numFiles)}" + val table = spark.table(tableName) + + val func = Utils.randomChoice(Meta.scalarFunc, r) + val args = Range(0, func.num_args) + .map(_ => Utils.randomChoice(table.columns, r)) + + // Example SELECT c0, log(c0) as x FROM test0 + s"SELECT ${args.mkString(", ")}, ${func.name}(${args.mkString(", ")}) AS x " + + s"FROM $tableName " + + s"ORDER BY ${args.mkString(", ")};" + } + + private def generateJoin(r: Random, spark: SparkSession, numFiles: Int): String = { + val leftTableName = s"test${r.nextInt(numFiles)}" + val rightTableName = s"test${r.nextInt(numFiles)}" + val leftTable = spark.table(leftTableName) + val rightTable = spark.table(rightTableName) + + val leftCol = Utils.randomChoice(leftTable.columns, r) + val rightCol = Utils.randomChoice(rightTable.columns, r) + + val joinTypes = Seq(("INNER", 0.4), ("LEFT", 0.3), ("RIGHT", 0.3)) + val joinType = Utils.randomWeightedChoice(joinTypes) + + val leftColProjection = leftTable.columns.map(c => s"l.$c").mkString(", ") + val rightColProjection = rightTable.columns.map(c => s"r.$c").mkString(", ") + "SELECT " + + s"$leftColProjection, " + + s"$rightColProjection " + + s"FROM $leftTableName l " + + s"$joinType JOIN $rightTableName r " + + s"ON l.$leftCol = r.$rightCol " + + "ORDER BY " + + s"$leftColProjection, " + + s"$rightColProjection;" + } + +} + +case class Function(name: String, num_args: Int) diff --git a/fuzz-testing/src/main/scala/org/apache/comet/fuzz/QueryRunner.scala b/fuzz-testing/src/main/scala/org/apache/comet/fuzz/QueryRunner.scala new file mode 100644 index 000000000..49f9fc3bd --- /dev/null +++ b/fuzz-testing/src/main/scala/org/apache/comet/fuzz/QueryRunner.scala @@ -0,0 +1,170 @@ +/* + * 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.fuzz + +import java.io.{BufferedWriter, FileWriter} + +import scala.io.Source + +import org.apache.spark.sql.{Row, SparkSession} + +object QueryRunner { + + def runQueries( + spark: SparkSession, + numFiles: Int, + filename: String, + showMatchingResults: Boolean, + showFailedSparkQueries: Boolean = false): Unit = { + + val outputFilename = s"results-${System.currentTimeMillis()}.md" + // scalastyle:off println + println(s"Writing results to $outputFilename") + // scalastyle:on println + + val w = new BufferedWriter(new FileWriter(outputFilename)) + + // register input files + for (i <- 0 until numFiles) { + val table = spark.read.parquet(s"test$i.parquet") + val tableName = s"test$i" + table.createTempView(tableName) + w.write( + s"Created table $tableName with schema:\n\t" + + s"${table.schema.fields.map(f => s"${f.name}: ${f.dataType}").mkString("\n\t")}\n\n") + } + + val querySource = Source.fromFile(filename) + try { + querySource + .getLines() + .foreach(sql => { + + try { + // execute with Spark + spark.conf.set("spark.comet.enabled", "false") + val df = spark.sql(sql) + val sparkRows = df.collect() + val sparkPlan = df.queryExecution.executedPlan.toString + + try { + spark.conf.set("spark.comet.enabled", "true") + val df = spark.sql(sql) + val cometRows = df.collect() + val cometPlan = df.queryExecution.executedPlan.toString + + if (sparkRows.length == cometRows.length) { + var i = 0 + while (i < sparkRows.length) { + val l = sparkRows(i) + val r = cometRows(i) + assert(l.length == r.length) + for (j <- 0 until l.length) { + val same = (l(j), r(j)) match { + case (a: Float, b: Float) if a.isInfinity => b.isInfinity + case (a: Float, b: Float) if a.isNaN => b.isNaN + case (a: Float, b: Float) => (a - b).abs <= 0.000001f + case (a: Double, b: Double) if a.isInfinity => b.isInfinity + case (a: Double, b: Double) if a.isNaN => b.isNaN + case (a: Double, b: Double) => (a - b).abs <= 0.000001 + case (a: Array[Byte], b: Array[Byte]) => a.sameElements(b) + case (a, b) => a == b + } + if (!same) { + showSQL(w, sql) + showPlans(w, sparkPlan, cometPlan) + w.write(s"First difference at row $i:\n") + w.write("Spark: `" + formatRow(l) + "`\n") + w.write("Comet: `" + formatRow(r) + "`\n") + i = sparkRows.length + } + } + i += 1 + } + } else { + showSQL(w, sql) + showPlans(w, sparkPlan, cometPlan) + w.write( + s"[ERROR] Spark produced ${sparkRows.length} rows and " + + s"Comet produced ${cometRows.length} rows.\n") + } + } catch { + case e: Exception => + // the query worked in Spark but failed in Comet, so this is likely a bug in Comet + showSQL(w, sql) + w.write(s"[ERROR] Query failed in Comet: ${e.getMessage}\n") + } + + // flush after every query so that results are saved in the event of the driver crashing + w.flush() + + } catch { + case e: Exception => + // we expect many generated queries to be invalid + if (showFailedSparkQueries) { + showSQL(w, sql) + w.write(s"Query failed in Spark: ${e.getMessage}\n") + } + } + }) + + } finally { + w.close() + querySource.close() + } + } + + private def formatRow(row: Row): String = { + row.toSeq + .map { + case v: Array[Byte] => v.mkString + case other => other.toString + } + .mkString(",") + } + + private def showSQL(w: BufferedWriter, sql: String, maxLength: Int = 120): Unit = { + w.write("## SQL\n") + w.write("```\n") + val words = sql.split(" ") + val currentLine = new StringBuilder + for (word <- words) { + if (currentLine.length + word.length + 1 > maxLength) { + w.write(currentLine.toString.trim) + w.write("\n") + currentLine.setLength(0) + } + currentLine.append(word).append(" ") + } + if (currentLine.nonEmpty) { + w.write(currentLine.toString.trim) + w.write("\n") + } + w.write("```\n") + } + + private def showPlans(w: BufferedWriter, sparkPlan: String, cometPlan: String): Unit = { + w.write("### Spark Plan\n") + w.write(s"```\n$sparkPlan\n```\n") + w.write("### Comet Plan\n") + w.write(s"```\n$cometPlan\n```\n") + } + +} diff --git a/fuzz-testing/src/main/scala/org/apache/comet/fuzz/Utils.scala b/fuzz-testing/src/main/scala/org/apache/comet/fuzz/Utils.scala new file mode 100644 index 000000000..19f9695a9 --- /dev/null +++ b/fuzz-testing/src/main/scala/org/apache/comet/fuzz/Utils.scala @@ -0,0 +1,46 @@ +/* + * 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.fuzz + +import scala.util.Random + +object Utils { + + def randomChoice[T](list: Seq[T], r: Random): T = { + list(r.nextInt(list.length)) + } + + def randomWeightedChoice[T](valuesWithWeights: Seq[(T, Double)]): T = { + val totalWeight = valuesWithWeights.map(_._2).sum + val randomValue = Random.nextDouble() * totalWeight + var cumulativeWeight = 0.0 + + for ((value, weight) <- valuesWithWeights) { + cumulativeWeight += weight + if (cumulativeWeight >= randomValue) { + return value + } + } + + // If for some reason the loop doesn't return, return the last value + valuesWithWeights.last._1 + } + +} diff --git a/pom.xml b/pom.xml index 0ec834982..8c322bae0 100644 --- a/pom.xml +++ b/pom.xml @@ -33,6 +33,7 @@ under the License. <module>common</module> <module>spark</module> <module>spark-integration</module> + <module>fuzz-testing</module> </modules> <properties> @@ -409,6 +410,12 @@ under the License. <scope>test</scope> </dependency> + <dependency> + <groupId>org.rogach</groupId> + <artifactId>scallop_${scala.binary.version}</artifactId> + <version>5.1.0</version> + </dependency> + </dependencies> </dependencyManagement> From 3a831339688641f431c4430d01fc6eda0309159e Mon Sep 17 00:00:00 2001 From: "Prashant K. Sharma" <prashaantsharmaa@gmail.com> Date: Tue, 4 Jun 2024 01:54:15 +0900 Subject: [PATCH 17/19] feat: Use enum to represent CAST eval_mode in expr.proto (#415) * Fixes Issue #361: Use enum to represent CAST eval_mode in expr.proto * Update expr.proto and QueryPlanSerde.scala for handling enum EvalMode for cast message * issue 361 fixed type issue for eval_mode in planner.rs * issue 361 refactored QueryPlanSerde.scala for enhanced type safety and localization compliance, including a new string-to-enum conversion function and improved import organization. * Updated planner.rs, expr.proto, QueryPlanSerde.scala for enum support * Update spark/src/main/scala/org/apache/comet/serde/QueryPlanSerde.scala --------- Co-authored-by: Prashant K. Sharma <prakush@foundation.local> Co-authored-by: Andy Grove <andygrove73@gmail.com> --- core/src/execution/datafusion/planner.rs | 14 +++++--------- core/src/execution/proto/expr.proto | 10 ++++++++-- .../apache/comet/serde/QueryPlanSerde.scala | 19 +++++++++++++++++-- 3 files changed, 30 insertions(+), 13 deletions(-) diff --git a/core/src/execution/datafusion/planner.rs b/core/src/execution/datafusion/planner.rs index a0b75de3f..20119b13a 100644 --- a/core/src/execution/datafusion/planner.rs +++ b/core/src/execution/datafusion/planner.rs @@ -348,16 +348,12 @@ impl PhysicalPlanner { let child = self.create_expr(expr.child.as_ref().unwrap(), input_schema)?; let datatype = to_arrow_datatype(expr.datatype.as_ref().unwrap()); let timezone = expr.timezone.clone(); - let eval_mode = match expr.eval_mode.as_str() { - "ANSI" => EvalMode::Ansi, - "TRY" => EvalMode::Try, - "LEGACY" => EvalMode::Legacy, - other => { - return Err(ExecutionError::GeneralError(format!( - "Invalid Cast EvalMode: \"{other}\"" - ))) - } + let eval_mode = match spark_expression::EvalMode::try_from(expr.eval_mode)? { + spark_expression::EvalMode::Legacy => EvalMode::Legacy, + spark_expression::EvalMode::Try => EvalMode::Try, + spark_expression::EvalMode::Ansi => EvalMode::Ansi, }; + Ok(Arc::new(Cast::new(child, datatype, eval_mode, timezone))) } ExprStruct::Hour(expr) => { diff --git a/core/src/execution/proto/expr.proto b/core/src/execution/proto/expr.proto index be85e8a92..bcd983875 100644 --- a/core/src/execution/proto/expr.proto +++ b/core/src/execution/proto/expr.proto @@ -249,12 +249,18 @@ message Remainder { DataType return_type = 4; } +enum EvalMode { + LEGACY = 0; + TRY = 1; + ANSI = 2; +} + message Cast { Expr child = 1; DataType datatype = 2; string timezone = 3; - // LEGACY, ANSI, or TRY - string eval_mode = 4; + EvalMode eval_mode = 4; + } message Equal { diff --git a/spark/src/main/scala/org/apache/comet/serde/QueryPlanSerde.scala b/spark/src/main/scala/org/apache/comet/serde/QueryPlanSerde.scala index c1188e193..79a75102d 100644 --- a/spark/src/main/scala/org/apache/comet/serde/QueryPlanSerde.scala +++ b/spark/src/main/scala/org/apache/comet/serde/QueryPlanSerde.scala @@ -19,6 +19,8 @@ package org.apache.comet.serde +import java.util.Locale + import scala.collection.JavaConverters._ import org.apache.spark.internal.Logging @@ -588,6 +590,18 @@ object QueryPlanSerde extends Logging with ShimQueryPlanSerde with CometExprShim * @return * The protobuf representation of the expression, or None if the expression is not supported */ + + def stringToEvalMode(evalModeStr: String): ExprOuterClass.EvalMode = + evalModeStr.toUpperCase(Locale.ROOT) match { + case "LEGACY" => ExprOuterClass.EvalMode.LEGACY + case "TRY" => ExprOuterClass.EvalMode.TRY + case "ANSI" => ExprOuterClass.EvalMode.ANSI + case invalid => + throw new IllegalArgumentException( + s"Invalid eval mode '$invalid' " + ) // Assuming we want to catch errors strictly + } + def exprToProto( expr: Expression, input: Seq[Attribute], @@ -598,12 +612,13 @@ object QueryPlanSerde extends Logging with ShimQueryPlanSerde with CometExprShim childExpr: Option[Expr], evalMode: String): Option[Expr] = { val dataType = serializeDataType(dt) + val evalModeEnum = stringToEvalMode(evalMode) // Convert string to enum if (childExpr.isDefined && dataType.isDefined) { val castBuilder = ExprOuterClass.Cast.newBuilder() castBuilder.setChild(childExpr.get) castBuilder.setDatatype(dataType.get) - castBuilder.setEvalMode(evalMode) + castBuilder.setEvalMode(evalModeEnum) // Set the enum in protobuf val timeZone = timeZoneId.getOrElse("UTC") castBuilder.setTimezone(timeZone) @@ -1305,7 +1320,7 @@ object QueryPlanSerde extends Logging with ShimQueryPlanSerde with CometExprShim .newBuilder() .setChild(e) .setDatatype(serializeDataType(IntegerType).get) - .setEvalMode("LEGACY") // year is not affected by ANSI mode + .setEvalMode(ExprOuterClass.EvalMode.LEGACY) .build()) .build() }) From edd63efb6965537dfc564d2b0eda5aa6c23398e7 Mon Sep 17 00:00:00 2001 From: Vipul Vaibhaw <vaibhaw.vipul@gmail.com> Date: Tue, 4 Jun 2024 02:37:51 +0530 Subject: [PATCH 18/19] feat: Implement ANSI support for UnaryMinus (#471) * checking for invalid inputs for unary minus * adding eval mode to expressions and proto message * extending evaluate function for negative expression * remove print statements * fix format errors * removing units * fix clippy errors * expect instead of unwrap, map_err instead of match and removing Float16 * adding test case for unary negative integer overflow * added a function to make the code more readable * adding comet sql ansi config * using withTempDir and checkSparkAnswerAndOperator * adding macros to improve code readability * using withParquetTable * adding scalar tests * adding more test cases and bug fix * using failonerror and removing eval_mode * bug fix * removing checks for float64 and monthdaynano * removing checks of float and monthday nano * adding checks while evalute bounds * IntervalDayTime splitting i64 and then checking * Adding interval test * fix ci errors --- core/src/errors.rs | 3 + .../execution/datafusion/expressions/mod.rs | 1 + .../datafusion/expressions/negative.rs | 270 ++++++++++++++++++ core/src/execution/datafusion/planner.rs | 9 +- core/src/execution/proto/expr.proto | 1 + .../apache/comet/serde/QueryPlanSerde.scala | 3 +- .../apache/comet/CometExpressionSuite.scala | 98 +++++++ 7 files changed, 381 insertions(+), 4 deletions(-) create mode 100644 core/src/execution/datafusion/expressions/negative.rs diff --git a/core/src/errors.rs b/core/src/errors.rs index 04a1629d5..af4fd2697 100644 --- a/core/src/errors.rs +++ b/core/src/errors.rs @@ -88,6 +88,9 @@ pub enum CometError { to_type: String, }, + #[error("[ARITHMETIC_OVERFLOW] {from_type} overflow. If necessary set \"spark.sql.ansi.enabled\" to \"false\" to bypass this error.")] + ArithmeticOverflow { from_type: String }, + #[error(transparent)] Arrow { #[from] diff --git a/core/src/execution/datafusion/expressions/mod.rs b/core/src/execution/datafusion/expressions/mod.rs index 9db4b65b3..084fef2df 100644 --- a/core/src/execution/datafusion/expressions/mod.rs +++ b/core/src/execution/datafusion/expressions/mod.rs @@ -29,6 +29,7 @@ pub mod avg_decimal; pub mod bloom_filter_might_contain; pub mod correlation; pub mod covariance; +pub mod negative; pub mod stats; pub mod stddev; pub mod strings; diff --git a/core/src/execution/datafusion/expressions/negative.rs b/core/src/execution/datafusion/expressions/negative.rs new file mode 100644 index 000000000..e7aa2ac64 --- /dev/null +++ b/core/src/execution/datafusion/expressions/negative.rs @@ -0,0 +1,270 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +use crate::errors::CometError; +use arrow::{compute::kernels::numeric::neg_wrapping, datatypes::IntervalDayTimeType}; +use arrow_array::RecordBatch; +use arrow_schema::{DataType, Schema}; +use datafusion::{ + logical_expr::{interval_arithmetic::Interval, ColumnarValue}, + physical_expr::PhysicalExpr, +}; +use datafusion_common::{Result, ScalarValue}; +use datafusion_physical_expr::{ + aggregate::utils::down_cast_any_ref, sort_properties::SortProperties, +}; +use std::{ + any::Any, + hash::{Hash, Hasher}, + sync::Arc, +}; + +pub fn create_negate_expr( + expr: Arc<dyn PhysicalExpr>, + fail_on_error: bool, +) -> Result<Arc<dyn PhysicalExpr>, CometError> { + Ok(Arc::new(NegativeExpr::new(expr, fail_on_error))) +} + +/// Negative expression +#[derive(Debug, Hash)] +pub struct NegativeExpr { + /// Input expression + arg: Arc<dyn PhysicalExpr>, + fail_on_error: bool, +} + +fn arithmetic_overflow_error(from_type: &str) -> CometError { + CometError::ArithmeticOverflow { + from_type: from_type.to_string(), + } +} + +macro_rules! check_overflow { + ($array:expr, $array_type:ty, $min_val:expr, $type_name:expr) => {{ + let typed_array = $array + .as_any() + .downcast_ref::<$array_type>() + .expect(concat!(stringify!($array_type), " expected")); + for i in 0..typed_array.len() { + if typed_array.value(i) == $min_val { + if $type_name == "byte" || $type_name == "short" { + let value = typed_array.value(i).to_string() + " caused"; + return Err(arithmetic_overflow_error(value.as_str()).into()); + } + return Err(arithmetic_overflow_error($type_name).into()); + } + } + }}; +} + +impl NegativeExpr { + /// Create new not expression + pub fn new(arg: Arc<dyn PhysicalExpr>, fail_on_error: bool) -> Self { + Self { arg, fail_on_error } + } + + /// Get the input expression + pub fn arg(&self) -> &Arc<dyn PhysicalExpr> { + &self.arg + } +} + +impl std::fmt::Display for NegativeExpr { + fn fmt(&self, f: &mut std::fmt::Formatter) -> std::fmt::Result { + write!(f, "(- {})", self.arg) + } +} + +impl PhysicalExpr for NegativeExpr { + /// Return a reference to Any that can be used for downcasting + fn as_any(&self) -> &dyn Any { + self + } + + fn data_type(&self, input_schema: &Schema) -> Result<DataType> { + self.arg.data_type(input_schema) + } + + fn nullable(&self, input_schema: &Schema) -> Result<bool> { + self.arg.nullable(input_schema) + } + + fn evaluate(&self, batch: &RecordBatch) -> Result<ColumnarValue> { + let arg = self.arg.evaluate(batch)?; + + // overflow checks only apply in ANSI mode + // datatypes supported are byte, short, integer, long, float, interval + match arg { + ColumnarValue::Array(array) => { + if self.fail_on_error { + match array.data_type() { + DataType::Int8 => { + check_overflow!(array, arrow::array::Int8Array, i8::MIN, "byte") + } + DataType::Int16 => { + check_overflow!(array, arrow::array::Int16Array, i16::MIN, "short") + } + DataType::Int32 => { + check_overflow!(array, arrow::array::Int32Array, i32::MIN, "integer") + } + DataType::Int64 => { + check_overflow!(array, arrow::array::Int64Array, i64::MIN, "long") + } + DataType::Interval(value) => match value { + arrow::datatypes::IntervalUnit::YearMonth => check_overflow!( + array, + arrow::array::IntervalYearMonthArray, + i32::MIN, + "interval" + ), + arrow::datatypes::IntervalUnit::DayTime => check_overflow!( + array, + arrow::array::IntervalDayTimeArray, + i64::MIN, + "interval" + ), + arrow::datatypes::IntervalUnit::MonthDayNano => { + // Overflow checks are not supported + } + }, + _ => { + // Overflow checks are not supported for other datatypes + } + } + } + let result = neg_wrapping(array.as_ref())?; + Ok(ColumnarValue::Array(result)) + } + ColumnarValue::Scalar(scalar) => { + if self.fail_on_error { + match scalar { + ScalarValue::Int8(value) => { + if value == Some(i8::MIN) { + return Err(arithmetic_overflow_error(" caused").into()); + } + } + ScalarValue::Int16(value) => { + if value == Some(i16::MIN) { + return Err(arithmetic_overflow_error(" caused").into()); + } + } + ScalarValue::Int32(value) => { + if value == Some(i32::MIN) { + return Err(arithmetic_overflow_error("integer").into()); + } + } + ScalarValue::Int64(value) => { + if value == Some(i64::MIN) { + return Err(arithmetic_overflow_error("long").into()); + } + } + ScalarValue::IntervalDayTime(value) => { + let (days, ms) = + IntervalDayTimeType::to_parts(value.unwrap_or_default()); + if days == i32::MIN || ms == i32::MIN { + return Err(arithmetic_overflow_error("interval").into()); + } + } + ScalarValue::IntervalYearMonth(value) => { + if value == Some(i32::MIN) { + return Err(arithmetic_overflow_error("interval").into()); + } + } + _ => { + // Overflow checks are not supported for other datatypes + } + } + } + Ok(ColumnarValue::Scalar((scalar.arithmetic_negate())?)) + } + } + } + + fn children(&self) -> Vec<Arc<dyn PhysicalExpr>> { + vec![self.arg.clone()] + } + + fn with_new_children( + self: Arc<Self>, + children: Vec<Arc<dyn PhysicalExpr>>, + ) -> Result<Arc<dyn PhysicalExpr>> { + Ok(Arc::new(NegativeExpr::new( + children[0].clone(), + self.fail_on_error, + ))) + } + + fn dyn_hash(&self, state: &mut dyn Hasher) { + let mut s = state; + self.hash(&mut s); + } + + /// Given the child interval of a NegativeExpr, it calculates the NegativeExpr's interval. + /// It replaces the upper and lower bounds after multiplying them with -1. + /// Ex: `(a, b]` => `[-b, -a)` + fn evaluate_bounds(&self, children: &[&Interval]) -> Result<Interval> { + Interval::try_new( + children[0].upper().arithmetic_negate()?, + children[0].lower().arithmetic_negate()?, + ) + } + + /// Returns a new [`Interval`] of a NegativeExpr that has the existing `interval` given that + /// given the input interval is known to be `children`. + fn propagate_constraints( + &self, + interval: &Interval, + children: &[&Interval], + ) -> Result<Option<Vec<Interval>>> { + let child_interval = children[0]; + + if child_interval.lower() == &ScalarValue::Int32(Some(i32::MIN)) + || child_interval.upper() == &ScalarValue::Int32(Some(i32::MIN)) + || child_interval.lower() == &ScalarValue::Int64(Some(i64::MIN)) + || child_interval.upper() == &ScalarValue::Int64(Some(i64::MIN)) + { + return Err(CometError::ArithmeticOverflow { + from_type: "long".to_string(), + } + .into()); + } + + let negated_interval = Interval::try_new( + interval.upper().arithmetic_negate()?, + interval.lower().arithmetic_negate()?, + )?; + + Ok(child_interval + .intersect(negated_interval)? + .map(|result| vec![result])) + } + + /// The ordering of a [`NegativeExpr`] is simply the reverse of its child. + fn get_ordering(&self, children: &[SortProperties]) -> SortProperties { + -children[0] + } +} + +impl PartialEq<dyn Any> for NegativeExpr { + fn eq(&self, other: &dyn Any) -> bool { + down_cast_any_ref(other) + .downcast_ref::<Self>() + .map(|x| self.arg.eq(&x.arg)) + .unwrap_or(false) + } +} diff --git a/core/src/execution/datafusion/planner.rs b/core/src/execution/datafusion/planner.rs index 20119b13a..3a8548f77 100644 --- a/core/src/execution/datafusion/planner.rs +++ b/core/src/execution/datafusion/planner.rs @@ -33,7 +33,7 @@ use datafusion::{ expressions::{ in_list, BinaryExpr, BitAnd, BitOr, BitXor, CaseExpr, CastExpr, Column, Count, FirstValue, InListExpr, IsNotNullExpr, IsNullExpr, LastValue, - Literal as DataFusionLiteral, Max, Min, NegativeExpr, NotExpr, Sum, UnKnownColumn, + Literal as DataFusionLiteral, Max, Min, NotExpr, Sum, UnKnownColumn, }, AggregateExpr, PhysicalExpr, PhysicalSortExpr, ScalarFunctionExpr, }, @@ -70,6 +70,7 @@ use crate::{ correlation::Correlation, covariance::Covariance, if_expr::IfExpr, + negative, scalar_funcs::create_comet_physical_fun, stats::StatsType, stddev::Stddev, @@ -563,8 +564,10 @@ impl PhysicalPlanner { Ok(Arc::new(NotExpr::new(child))) } ExprStruct::Negative(expr) => { - let child = self.create_expr(expr.child.as_ref().unwrap(), input_schema)?; - Ok(Arc::new(NegativeExpr::new(child))) + let child: Arc<dyn PhysicalExpr> = + self.create_expr(expr.child.as_ref().unwrap(), input_schema.clone())?; + let result = negative::create_negate_expr(child, expr.fail_on_error); + result.map_err(|e| ExecutionError::GeneralError(e.to_string())) } ExprStruct::NormalizeNanAndZero(expr) => { let child = self.create_expr(expr.child.as_ref().unwrap(), input_schema)?; diff --git a/core/src/execution/proto/expr.proto b/core/src/execution/proto/expr.proto index bcd983875..9c6049013 100644 --- a/core/src/execution/proto/expr.proto +++ b/core/src/execution/proto/expr.proto @@ -454,6 +454,7 @@ message Not { message Negative { Expr child = 1; + bool fail_on_error = 2; } message IfExpr { diff --git a/spark/src/main/scala/org/apache/comet/serde/QueryPlanSerde.scala b/spark/src/main/scala/org/apache/comet/serde/QueryPlanSerde.scala index 79a75102d..5fe290cf6 100644 --- a/spark/src/main/scala/org/apache/comet/serde/QueryPlanSerde.scala +++ b/spark/src/main/scala/org/apache/comet/serde/QueryPlanSerde.scala @@ -1984,11 +1984,12 @@ object QueryPlanSerde extends Logging with ShimQueryPlanSerde with CometExprShim None } - case UnaryMinus(child, _) => + case UnaryMinus(child, failOnError) => val childExpr = exprToProtoInternal(child, inputs) if (childExpr.isDefined) { val builder = ExprOuterClass.Negative.newBuilder() builder.setChild(childExpr.get) + builder.setFailOnError(failOnError) Some( ExprOuterClass.Expr .newBuilder() diff --git a/spark/src/test/scala/org/apache/comet/CometExpressionSuite.scala b/spark/src/test/scala/org/apache/comet/CometExpressionSuite.scala index e15b09ca2..e69054dbd 100644 --- a/spark/src/test/scala/org/apache/comet/CometExpressionSuite.scala +++ b/spark/src/test/scala/org/apache/comet/CometExpressionSuite.scala @@ -1548,5 +1548,103 @@ class CometExpressionSuite extends CometTestBase with AdaptiveSparkPlanHelper { } } } + test("unary negative integer overflow test") { + def withAnsiMode(enabled: Boolean)(f: => Unit): Unit = { + withSQLConf( + SQLConf.ANSI_ENABLED.key -> enabled.toString, + CometConf.COMET_ANSI_MODE_ENABLED.key -> enabled.toString, + CometConf.COMET_ENABLED.key -> "true", + CometConf.COMET_EXEC_ENABLED.key -> "true")(f) + } + + def checkOverflow(query: String, dtype: String): Unit = { + checkSparkMaybeThrows(sql(query)) match { + case (Some(sparkException), Some(cometException)) => + assert(sparkException.getMessage.contains(dtype + " overflow")) + assert(cometException.getMessage.contains(dtype + " overflow")) + case (None, None) => assert(true) // got same outputs + case (None, Some(ex)) => + fail("Comet threw an exception but Spark did not " + ex.getMessage) + case (Some(_), None) => + fail("Spark threw an exception but Comet did not") + } + } + + def runArrayTest(query: String, dtype: String, path: String): Unit = { + withParquetTable(path, "t") { + withAnsiMode(enabled = false) { + checkSparkAnswerAndOperator(sql(query)) + } + withAnsiMode(enabled = true) { + checkOverflow(query, dtype) + } + } + } + withTempDir { dir => + // Array values test + val arrayPath = new Path(dir.toURI.toString, "array_test.parquet").toString + Seq(Int.MaxValue, Int.MinValue).toDF("a").write.mode("overwrite").parquet(arrayPath) + val arrayQuery = "select a, -a from t" + runArrayTest(arrayQuery, "integer", arrayPath) + + // long values test + val longArrayPath = new Path(dir.toURI.toString, "long_array_test.parquet").toString + Seq(Long.MaxValue, Long.MinValue) + .toDF("a") + .write + .mode("overwrite") + .parquet(longArrayPath) + val longArrayQuery = "select a, -a from t" + runArrayTest(longArrayQuery, "long", longArrayPath) + + // short values test + val shortArrayPath = new Path(dir.toURI.toString, "short_array_test.parquet").toString + Seq(Short.MaxValue, Short.MinValue) + .toDF("a") + .write + .mode("overwrite") + .parquet(shortArrayPath) + val shortArrayQuery = "select a, -a from t" + runArrayTest(shortArrayQuery, " caused", shortArrayPath) + + // byte values test + val byteArrayPath = new Path(dir.toURI.toString, "byte_array_test.parquet").toString + Seq(Byte.MaxValue, Byte.MinValue) + .toDF("a") + .write + .mode("overwrite") + .parquet(byteArrayPath) + val byteArrayQuery = "select a, -a from t" + runArrayTest(byteArrayQuery, " caused", byteArrayPath) + + // interval values test + withTable("t_interval") { + spark.sql("CREATE TABLE t_interval(a STRING) USING PARQUET") + spark.sql("INSERT INTO t_interval VALUES ('INTERVAL 10000000000 YEAR')") + withAnsiMode(enabled = true) { + spark + .sql("SELECT CAST(a AS INTERVAL) AS a FROM t_interval") + .createOrReplaceTempView("t_interval_casted") + checkOverflow("SELECT a, -a FROM t_interval_casted", "interval") + } + } + + withTable("t") { + sql("create table t(a int) using parquet") + sql("insert into t values (-2147483648)") + withAnsiMode(enabled = true) { + checkOverflow("select a, -a from t", "integer") + } + } + + withTable("t_float") { + sql("create table t_float(a float) using parquet") + sql("insert into t_float values (3.4128235E38)") + withAnsiMode(enabled = true) { + checkOverflow("select a, -a from t_float", "float") + } + } + } + } } From 77c9a6cc03b98e60d6c1b3d2805293826b5d3c2f Mon Sep 17 00:00:00 2001 From: KAZUYUKI TANIMURA <ktanimura@apple.com> Date: Mon, 3 Jun 2024 16:03:00 -0700 Subject: [PATCH 19/19] build: Enable comet tests with spark-4.0 profile (#493) ## Rationale for this change To be ready for Spark 4.0 ## What changes are included in this PR? This PR enables the comet tests with the spark-4.0 profile ## How are these changes tested? Tests with the spark-4.0 profile now should pass. (But Spark tests do not yet) --- .github/workflows/pr_build.yml | 18 +- .../scala/org/apache/comet/CometConf.scala | 9 +- .../apache/comet/shims/ShimCometConf.scala | 25 + .../apache/comet/shims/ShimCometConf.scala | 25 + docs/source/contributor-guide/development.md | 4 + docs/source/user-guide/configs.md | 1 - .../comet/CometSparkSessionExtensions.scala | 4 + .../apache/comet/serde/QueryPlanSerde.scala | 3 +- .../spark/sql/ExtendedExplainGenerator.scala | 0 .../q1/explain.txt | 274 +++++ .../q1/simplified.txt | 68 ++ .../q10/explain.txt | 286 +++++ .../q10/simplified.txt | 75 ++ .../q11/explain.txt | 482 ++++++++ .../q11/simplified.txt | 123 ++ .../q12/explain.txt | 150 +++ .../q12/simplified.txt | 40 + .../q13/explain.txt | 232 ++++ .../q13/simplified.txt | 59 + .../q14a/explain.txt | 800 +++++++++++++ .../q14a/simplified.txt | 214 ++++ .../q14b/explain.txt | 759 ++++++++++++ .../q14b/simplified.txt | 204 ++++ .../q15/explain.txt | 164 +++ .../q15/simplified.txt | 41 + .../q16/explain.txt | 260 ++++ .../q16/simplified.txt | 74 ++ .../q17/explain.txt | 298 +++++ .../q17/simplified.txt | 76 ++ .../q18/explain.txt | 281 +++++ .../q18/simplified.txt | 71 ++ .../q19/explain.txt | 227 ++++ .../q19/simplified.txt | 58 + .../q2/explain.txt | 210 ++++ .../q2/simplified.txt | 54 + .../q20/explain.txt | 150 +++ .../q20/simplified.txt | 40 + .../q21/explain.txt | 169 +++ .../q21/simplified.txt | 42 + .../q22/explain.txt | 169 +++ .../q22/simplified.txt | 42 + .../q23a/explain.txt | 570 +++++++++ .../q23a/simplified.txt | 155 +++ .../q23b/explain.txt | 694 +++++++++++ .../q23b/simplified.txt | 188 +++ .../q24a/explain.txt | 427 +++++++ .../q24a/simplified.txt | 118 ++ .../q24b/explain.txt | 427 +++++++ .../q24b/simplified.txt | 118 ++ .../q25/explain.txt | 298 +++++ .../q25/simplified.txt | 76 ++ .../q26/explain.txt | 208 ++++ .../q26/simplified.txt | 52 + .../q27/explain.txt | 208 ++++ .../q27/simplified.txt | 52 + .../q28/explain.txt | 437 +++++++ .../q28/simplified.txt | 111 ++ .../q29/explain.txt | 326 +++++ .../q29/simplified.txt | 83 ++ .../q3/explain.txt | 125 ++ .../q3/simplified.txt | 31 + .../q30/explain.txt | 324 +++++ .../q30/simplified.txt | 81 ++ .../q31/explain.txt | 616 ++++++++++ .../q31/simplified.txt | 159 +++ .../q32/explain.txt | 209 ++++ .../q32/simplified.txt | 52 + .../q33/explain.txt | 405 +++++++ .../q33/simplified.txt | 105 ++ .../q34/explain.txt | 218 ++++ .../q34/simplified.txt | 56 + .../q35/explain.txt | 281 +++++ .../q35/simplified.txt | 74 ++ .../q36/explain.txt | 194 +++ .../q36/simplified.txt | 51 + .../q37/explain.txt | 179 +++ .../q37/simplified.txt | 44 + .../q38/explain.txt | 321 +++++ .../q38/simplified.txt | 81 ++ .../q39a/explain.txt | 318 +++++ .../q39a/simplified.txt | 81 ++ .../q39b/explain.txt | 318 +++++ .../q39b/simplified.txt | 81 ++ .../q4/explain.txt | 698 +++++++++++ .../q4/simplified.txt | 179 +++ .../q40/explain.txt | 218 ++++ .../q40/simplified.txt | 60 + .../q41/explain.txt | 119 ++ .../q41/simplified.txt | 29 + .../q42/explain.txt | 125 ++ .../q42/simplified.txt | 31 + .../q43/explain.txt | 125 ++ .../q43/simplified.txt | 31 + .../q44/explain.txt | 286 +++++ .../q44/simplified.txt | 81 ++ .../q45/explain.txt | 242 ++++ .../q45/simplified.txt | 61 + .../q46/explain.txt | 258 ++++ .../q46/simplified.txt | 65 + .../q47/explain.txt | 279 +++++ .../q47/simplified.txt | 81 ++ .../q48/explain.txt | 198 +++ .../q48/simplified.txt | 50 + .../q49/explain.txt | 456 +++++++ .../q49/simplified.txt | 121 ++ .../q5/explain.txt | 457 +++++++ .../q5/simplified.txt | 111 ++ .../q50/explain.txt | 199 +++ .../q50/simplified.txt | 50 + .../q51/explain.txt | 245 ++++ .../q51/simplified.txt | 75 ++ .../q52/explain.txt | 125 ++ .../q52/simplified.txt | 31 + .../q53/explain.txt | 194 +++ .../q53/simplified.txt | 51 + .../q54/explain.txt | 483 ++++++++ .../q54/simplified.txt | 123 ++ .../q55/explain.txt | 125 ++ .../q55/simplified.txt | 31 + .../q56/explain.txt | 405 +++++++ .../q56/simplified.txt | 105 ++ .../q57/explain.txt | 279 +++++ .../q57/simplified.txt | 81 ++ .../q58/explain.txt | 386 ++++++ .../q58/simplified.txt | 98 ++ .../q59/explain.txt | 256 ++++ .../q59/simplified.txt | 66 + .../q6/explain.txt | 309 +++++ .../q6/simplified.txt | 79 ++ .../q60/explain.txt | 405 +++++++ .../q60/simplified.txt | 105 ++ .../q61/explain.txt | 417 +++++++ .../q61/simplified.txt | 106 ++ .../q62/explain.txt | 187 +++ .../q62/simplified.txt | 48 + .../q63/explain.txt | 194 +++ .../q63/simplified.txt | 51 + .../q64/explain.txt | 1064 +++++++++++++++++ .../q64/simplified.txt | 281 +++++ .../q65/explain.txt | 269 +++++ .../q65/simplified.txt | 67 ++ .../q66/explain.txt | 332 +++++ .../q66/simplified.txt | 86 ++ .../q67/explain.txt | 204 ++++ .../q67/simplified.txt | 53 + .../q68/explain.txt | 258 ++++ .../q68/simplified.txt | 65 + .../q69/explain.txt | 281 +++++ .../q69/simplified.txt | 74 ++ .../q7/explain.txt | 208 ++++ .../q7/simplified.txt | 52 + .../q70/explain.txt | 283 +++++ .../q70/simplified.txt | 75 ++ .../q71/explain.txt | 254 ++++ .../q71/simplified.txt | 69 ++ .../q72/explain.txt | 433 +++++++ .../q72/simplified.txt | 116 ++ .../q73/explain.txt | 218 ++++ .../q73/simplified.txt | 56 + .../q74/explain.txt | 477 ++++++++ .../q74/simplified.txt | 122 ++ .../q75/explain.txt | 779 ++++++++++++ .../q75/simplified.txt | 240 ++++ .../q76/explain.txt | 218 ++++ .../q76/simplified.txt | 58 + .../q77/explain.txt | 547 +++++++++ .../q77/simplified.txt | 143 +++ .../q78/explain.txt | 431 +++++++ .../q78/simplified.txt | 127 ++ .../q79/explain.txt | 208 ++++ .../q79/simplified.txt | 52 + .../q8/explain.txt | 288 +++++ .../q8/simplified.txt | 72 ++ .../q80/explain.txt | 645 ++++++++++ .../q80/simplified.txt | 182 +++ .../q81/explain.txt | 319 +++++ .../q81/simplified.txt | 80 ++ .../q82/explain.txt | 179 +++ .../q82/simplified.txt | 44 + .../q83.ansi/explain.txt | 372 ++++++ .../q83.ansi/simplified.txt | 95 ++ .../q84/explain.txt | 210 ++++ .../q84/simplified.txt | 54 + .../q85/explain.txt | 305 +++++ .../q85/simplified.txt | 75 ++ .../q86/explain.txt | 155 +++ .../q86/simplified.txt | 41 + .../q87/explain.txt | 321 +++++ .../q87/simplified.txt | 81 ++ .../q88/explain.txt | 1031 ++++++++++++++++ .../q88/simplified.txt | 265 ++++ .../q89/explain.txt | 189 +++ .../q89/simplified.txt | 50 + .../q9/explain.txt | 303 +++++ .../q9/simplified.txt | 81 ++ .../q90/explain.txt | 292 +++++ .../q90/simplified.txt | 74 ++ .../q91/explain.txt | 281 +++++ .../q91/simplified.txt | 73 ++ .../q92/explain.txt | 209 ++++ .../q92/simplified.txt | 52 + .../q93/explain.txt | 138 +++ .../q93/simplified.txt | 40 + .../q94/explain.txt | 260 ++++ .../q94/simplified.txt | 74 ++ .../q95/explain.txt | 330 +++++ .../q95/simplified.txt | 102 ++ .../q96/explain.txt | 163 +++ .../q96/simplified.txt | 41 + .../q97/explain.txt | 179 +++ .../q97/simplified.txt | 47 + .../q98/explain.txt | 160 +++ .../q98/simplified.txt | 44 + .../q99/explain.txt | 187 +++ .../q99/simplified.txt | 48 + .../q10a/explain.txt | 272 +++++ .../q10a/simplified.txt | 72 ++ .../q11/explain.txt | 477 ++++++++ .../q11/simplified.txt | 122 ++ .../q12/explain.txt | 150 +++ .../q12/simplified.txt | 40 + .../q14/explain.txt | 759 ++++++++++++ .../q14/simplified.txt | 204 ++++ .../q14a/explain.txt | 964 +++++++++++++++ .../q14a/simplified.txt | 261 ++++ .../q18a/explain.txt | 909 ++++++++++++++ .../q18a/simplified.txt | 233 ++++ .../q20/explain.txt | 150 +++ .../q20/simplified.txt | 40 + .../q22/explain.txt | 161 +++ .../q22/simplified.txt | 41 + .../q22a/explain.txt | 315 +++++ .../q22a/simplified.txt | 80 ++ .../q24/explain.txt | 437 +++++++ .../q24/simplified.txt | 122 ++ .../q27a/explain.txt | 457 +++++++ .../q27a/simplified.txt | 117 ++ .../q34/explain.txt | 218 ++++ .../q34/simplified.txt | 56 + .../q35/explain.txt | 281 +++++ .../q35/simplified.txt | 74 ++ .../q35a/explain.txt | 267 +++++ .../q35a/simplified.txt | 71 ++ .../q36a/explain.txt | 279 +++++ .../q36a/simplified.txt | 76 ++ .../q47/explain.txt | 279 +++++ .../q47/simplified.txt | 81 ++ .../q49/explain.txt | 456 +++++++ .../q49/simplified.txt | 121 ++ .../q51a/explain.txt | 416 +++++++ .../q51a/simplified.txt | 124 ++ .../q57/explain.txt | 279 +++++ .../q57/simplified.txt | 81 ++ .../q5a/explain.txt | 542 +++++++++ .../q5a/simplified.txt | 136 +++ .../q6/explain.txt | 309 +++++ .../q6/simplified.txt | 79 ++ .../q64/explain.txt | 1064 +++++++++++++++++ .../q64/simplified.txt | 281 +++++ .../q67a/explain.txt | 466 ++++++++ .../q67a/simplified.txt | 127 ++ .../q70a/explain.txt | 368 ++++++ .../q70a/simplified.txt | 100 ++ .../q72/explain.txt | 433 +++++++ .../q72/simplified.txt | 116 ++ .../q74/explain.txt | 477 ++++++++ .../q74/simplified.txt | 122 ++ .../q75/explain.txt | 779 ++++++++++++ .../q75/simplified.txt | 240 ++++ .../q77a/explain.txt | 632 ++++++++++ .../q77a/simplified.txt | 168 +++ .../q78/explain.txt | 431 +++++++ .../q78/simplified.txt | 127 ++ .../q80a/explain.txt | 730 +++++++++++ .../q80a/simplified.txt | 207 ++++ .../q86a/explain.txt | 240 ++++ .../q86a/simplified.txt | 66 + .../q98/explain.txt | 155 +++ .../q98/simplified.txt | 43 + .../org/apache/comet/CometCastSuite.scala | 14 +- .../apache/comet/exec/CometExecSuite.scala | 8 +- .../comet/parquet/ParquetReadSuite.scala | 10 +- .../sql/comet/CometPlanStabilitySuite.scala | 10 +- .../comet/ParquetDatetimeRebaseSuite.scala | 10 +- .../comet/CometExpression3_3PlusSuite.scala | 1 - .../comet/exec/CometExec3_4PlusSuite.scala | 1 - 286 files changed, 59332 insertions(+), 36 deletions(-) create mode 100644 common/src/main/spark-3.x/org/apache/comet/shims/ShimCometConf.scala create mode 100644 common/src/main/spark-4.0/org/apache/comet/shims/ShimCometConf.scala rename spark/src/main/{scala => spark-3.x}/org/apache/spark/sql/ExtendedExplainGenerator.scala (100%) create mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q1/explain.txt create mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q1/simplified.txt create mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q10/explain.txt create mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q10/simplified.txt create mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q11/explain.txt create mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q11/simplified.txt create mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q12/explain.txt create mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q12/simplified.txt create mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q13/explain.txt create mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q13/simplified.txt create mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q14a/explain.txt create mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q14a/simplified.txt create mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q14b/explain.txt create mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q14b/simplified.txt create mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q15/explain.txt create mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q15/simplified.txt create mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q16/explain.txt create mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q16/simplified.txt create mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q17/explain.txt create mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q17/simplified.txt create mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q18/explain.txt create mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q18/simplified.txt create mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q19/explain.txt create mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q19/simplified.txt create mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q2/explain.txt create mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q2/simplified.txt create mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q20/explain.txt create mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q20/simplified.txt create mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q21/explain.txt create mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q21/simplified.txt create mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q22/explain.txt create mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q22/simplified.txt create mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q23a/explain.txt create mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q23a/simplified.txt create mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q23b/explain.txt create mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q23b/simplified.txt create mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q24a/explain.txt create mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q24a/simplified.txt create mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q24b/explain.txt create mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q24b/simplified.txt create mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q25/explain.txt create mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q25/simplified.txt create mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q26/explain.txt create mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q26/simplified.txt create mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q27/explain.txt create mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q27/simplified.txt create mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q28/explain.txt create mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q28/simplified.txt create mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q29/explain.txt create mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q29/simplified.txt create mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q3/explain.txt create mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q3/simplified.txt create mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q30/explain.txt create mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q30/simplified.txt create mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q31/explain.txt create mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q31/simplified.txt create mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q32/explain.txt create mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q32/simplified.txt create mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q33/explain.txt create mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q33/simplified.txt create mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q34/explain.txt create mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q34/simplified.txt create mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q35/explain.txt create mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q35/simplified.txt create mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q36/explain.txt create mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q36/simplified.txt create mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q37/explain.txt create mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q37/simplified.txt create mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q38/explain.txt create mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q38/simplified.txt create mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q39a/explain.txt create mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q39a/simplified.txt create mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q39b/explain.txt create mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q39b/simplified.txt create mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q4/explain.txt create mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q4/simplified.txt create mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q40/explain.txt create mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q40/simplified.txt create mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q41/explain.txt create mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q41/simplified.txt create mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q42/explain.txt create mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q42/simplified.txt create mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q43/explain.txt create mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q43/simplified.txt create mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q44/explain.txt create mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q44/simplified.txt create mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q45/explain.txt create mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q45/simplified.txt create mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q46/explain.txt create mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q46/simplified.txt create mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q47/explain.txt create mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q47/simplified.txt create mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q48/explain.txt create mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q48/simplified.txt create mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q49/explain.txt create mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q49/simplified.txt create mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q5/explain.txt create mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q5/simplified.txt create mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q50/explain.txt create mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q50/simplified.txt create mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q51/explain.txt create mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q51/simplified.txt create mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q52/explain.txt create mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q52/simplified.txt create mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q53/explain.txt create mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q53/simplified.txt create mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q54/explain.txt create mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q54/simplified.txt create mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q55/explain.txt create mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q55/simplified.txt create mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q56/explain.txt create mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q56/simplified.txt create mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q57/explain.txt create mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q57/simplified.txt create mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q58/explain.txt create mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q58/simplified.txt create mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q59/explain.txt create mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q59/simplified.txt create mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q6/explain.txt create mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q6/simplified.txt create mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q60/explain.txt create mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q60/simplified.txt create mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q61/explain.txt create mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q61/simplified.txt create mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q62/explain.txt create mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q62/simplified.txt create mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q63/explain.txt create mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q63/simplified.txt create mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q64/explain.txt create mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q64/simplified.txt create mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q65/explain.txt create mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q65/simplified.txt create mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q66/explain.txt create mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q66/simplified.txt create mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q67/explain.txt create mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q67/simplified.txt create mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q68/explain.txt create mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q68/simplified.txt create mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q69/explain.txt create mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q69/simplified.txt create mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q7/explain.txt create mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q7/simplified.txt create mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q70/explain.txt create mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q70/simplified.txt create mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q71/explain.txt create mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q71/simplified.txt create mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q72/explain.txt create mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q72/simplified.txt create mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q73/explain.txt create mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q73/simplified.txt create mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q74/explain.txt create mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q74/simplified.txt create mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q75/explain.txt create mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q75/simplified.txt create mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q76/explain.txt create mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q76/simplified.txt create mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q77/explain.txt create mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q77/simplified.txt create mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q78/explain.txt create mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q78/simplified.txt create mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q79/explain.txt create mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q79/simplified.txt create mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q8/explain.txt create mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q8/simplified.txt create mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q80/explain.txt create mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q80/simplified.txt create mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q81/explain.txt create mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q81/simplified.txt create mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q82/explain.txt create mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q82/simplified.txt create mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q83.ansi/explain.txt create mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q83.ansi/simplified.txt create mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q84/explain.txt create mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q84/simplified.txt create mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q85/explain.txt create mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q85/simplified.txt create mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q86/explain.txt create mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q86/simplified.txt create mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q87/explain.txt create mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q87/simplified.txt create mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q88/explain.txt create mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q88/simplified.txt create mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q89/explain.txt create mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q89/simplified.txt create mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q9/explain.txt create mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q9/simplified.txt create mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q90/explain.txt create mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q90/simplified.txt create mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q91/explain.txt create mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q91/simplified.txt create mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q92/explain.txt create mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q92/simplified.txt create mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q93/explain.txt create mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q93/simplified.txt create mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q94/explain.txt create mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q94/simplified.txt create mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q95/explain.txt create mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q95/simplified.txt create mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q96/explain.txt create mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q96/simplified.txt create mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q97/explain.txt create mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q97/simplified.txt create mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q98/explain.txt create mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q98/simplified.txt create mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q99/explain.txt create mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q99/simplified.txt create mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q10a/explain.txt create mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q10a/simplified.txt create mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q11/explain.txt create mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q11/simplified.txt create mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q12/explain.txt create mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q12/simplified.txt create mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q14/explain.txt create mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q14/simplified.txt create mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q14a/explain.txt create mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q14a/simplified.txt create mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q18a/explain.txt create mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q18a/simplified.txt create mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q20/explain.txt create mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q20/simplified.txt create mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q22/explain.txt create mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q22/simplified.txt create mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q22a/explain.txt create mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q22a/simplified.txt create mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q24/explain.txt create mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q24/simplified.txt create mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q27a/explain.txt create mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q27a/simplified.txt create mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q34/explain.txt create mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q34/simplified.txt create mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q35/explain.txt create mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q35/simplified.txt create mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q35a/explain.txt create mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q35a/simplified.txt create mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q36a/explain.txt create mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q36a/simplified.txt create mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q47/explain.txt create mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q47/simplified.txt create mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q49/explain.txt create mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q49/simplified.txt create mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q51a/explain.txt create mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q51a/simplified.txt create mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q57/explain.txt create mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q57/simplified.txt create mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q5a/explain.txt create mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q5a/simplified.txt create mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q6/explain.txt create mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q6/simplified.txt create mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q64/explain.txt create mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q64/simplified.txt create mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q67a/explain.txt create mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q67a/simplified.txt create mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q70a/explain.txt create mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q70a/simplified.txt create mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q72/explain.txt create mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q72/simplified.txt create mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q74/explain.txt create mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q74/simplified.txt create mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q75/explain.txt create mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q75/simplified.txt create mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q77a/explain.txt create mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q77a/simplified.txt create mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q78/explain.txt create mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q78/simplified.txt create mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q80a/explain.txt create mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q80a/simplified.txt create mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q86a/explain.txt create mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q86a/simplified.txt create mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q98/explain.txt create mode 100644 spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q98/simplified.txt diff --git a/.github/workflows/pr_build.yml b/.github/workflows/pr_build.yml index 410f1e1fe..7de198a47 100644 --- a/.github/workflows/pr_build.yml +++ b/.github/workflows/pr_build.yml @@ -109,10 +109,8 @@ jobs: - name: Java test steps uses: ./.github/actions/java-test with: - # TODO: remove -DskipTests after fixing tests - maven_opts: "-Pspark-${{ matrix.spark-version }} -DskipTests" - # TODO: upload test reports after enabling tests - upload-test-reports: false + maven_opts: -Pspark-${{ matrix.spark-version }} + upload-test-reports: true linux-test-with-old-spark: strategy: @@ -237,10 +235,8 @@ jobs: - name: Java test steps uses: ./.github/actions/java-test with: - # TODO: remove -DskipTests after fixing tests - maven_opts: "-Pspark-${{ matrix.spark-version }} -DskipTests" - # TODO: upload test reports after enabling tests - upload-test-reports: false + maven_opts: -Pspark-${{ matrix.spark-version }} + upload-test-reports: true macos-aarch64-test-with-spark4_0: strategy: @@ -277,10 +273,8 @@ jobs: - name: Java test steps uses: ./.github/actions/java-test with: - # TODO: remove -DskipTests after fixing tests - maven_opts: "-Pspark-${{ matrix.spark-version }} -DskipTests" - # TODO: upload test reports after enabling tests - upload-test-reports: false + maven_opts: -Pspark-${{ matrix.spark-version }} + upload-test-reports: true macos-aarch64-test-with-old-spark: strategy: diff --git a/common/src/main/scala/org/apache/comet/CometConf.scala b/common/src/main/scala/org/apache/comet/CometConf.scala index 5aee02f11..42fb5fb4c 100644 --- a/common/src/main/scala/org/apache/comet/CometConf.scala +++ b/common/src/main/scala/org/apache/comet/CometConf.scala @@ -29,6 +29,8 @@ import org.apache.spark.network.util.JavaUtils import org.apache.spark.sql.comet.util.Utils import org.apache.spark.sql.internal.SQLConf +import org.apache.comet.shims.ShimCometConf + /** * Configurations for a Comet application. Mostly inspired by [[SQLConf]] in Spark. * @@ -41,7 +43,7 @@ import org.apache.spark.sql.internal.SQLConf * which retrieves the config value from the thread-local [[SQLConf]] object. Alternatively, you * can also explicitly pass a [[SQLConf]] object to the `get` method. */ -object CometConf { +object CometConf extends ShimCometConf { /** List of all configs that is used for generating documentation */ val allConfs = new ListBuffer[ConfigEntry[_]] @@ -361,7 +363,7 @@ object CometConf { "column to a long column, a float column to a double column, etc. This is automatically" + "enabled when reading from Iceberg tables.") .booleanConf - .createWithDefault(false) + .createWithDefault(COMET_SCHEMA_EVOLUTION_ENABLED_DEFAULT) val COMET_ROW_TO_COLUMNAR_ENABLED: ConfigEntry[Boolean] = conf("spark.comet.rowToColumnar.enabled") @@ -382,12 +384,13 @@ object CometConf { .createWithDefault(Seq("Range,InMemoryTableScan")) val COMET_ANSI_MODE_ENABLED: ConfigEntry[Boolean] = conf("spark.comet.ansi.enabled") + .internal() .doc( "Comet does not respect ANSI mode in most cases and by default will not accelerate " + "queries when ansi mode is enabled. Enable this setting to test Comet's experimental " + "support for ANSI mode. This should not be used in production.") .booleanConf - .createWithDefault(false) + .createWithDefault(COMET_ANSI_MODE_ENABLED_DEFAULT) val COMET_CAST_ALLOW_INCOMPATIBLE: ConfigEntry[Boolean] = conf("spark.comet.cast.allowIncompatible") diff --git a/common/src/main/spark-3.x/org/apache/comet/shims/ShimCometConf.scala b/common/src/main/spark-3.x/org/apache/comet/shims/ShimCometConf.scala new file mode 100644 index 000000000..dc84a7525 --- /dev/null +++ b/common/src/main/spark-3.x/org/apache/comet/shims/ShimCometConf.scala @@ -0,0 +1,25 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.comet.shims + +trait ShimCometConf { + protected val COMET_SCHEMA_EVOLUTION_ENABLED_DEFAULT = false + protected val COMET_ANSI_MODE_ENABLED_DEFAULT = false +} diff --git a/common/src/main/spark-4.0/org/apache/comet/shims/ShimCometConf.scala b/common/src/main/spark-4.0/org/apache/comet/shims/ShimCometConf.scala new file mode 100644 index 000000000..13da6bc10 --- /dev/null +++ b/common/src/main/spark-4.0/org/apache/comet/shims/ShimCometConf.scala @@ -0,0 +1,25 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.comet.shims + +trait ShimCometConf { + protected val COMET_SCHEMA_EVOLUTION_ENABLED_DEFAULT = true + protected val COMET_ANSI_MODE_ENABLED_DEFAULT = true +} diff --git a/docs/source/contributor-guide/development.md b/docs/source/contributor-guide/development.md index 0121d9f46..913eea408 100644 --- a/docs/source/contributor-guide/development.md +++ b/docs/source/contributor-guide/development.md @@ -92,11 +92,13 @@ The plan stability testing framework is located in the `spark` module and can be ```sh ./mvnw -pl spark -Dsuites="org.apache.spark.sql.comet.CometTPCDSV1_4_PlanStabilitySuite" test +./mvnw -pl spark -Dsuites="org.apache.spark.sql.comet.CometTPCDSV1_4_PlanStabilitySuite" -Pspark-4.0 -nsu test ``` and ```sh ./mvnw -pl spark -Dsuites="org.apache.spark.sql.comet.CometTPCDSV2_7_PlanStabilitySuite" test +./mvnw -pl spark -Dsuites="org.apache.spark.sql.comet.CometTPCDSV2_7_PlanStabilitySuite" -Pspark-4.0 -nsu test ``` If your pull request changes the query plans generated by Comet, you should regenerate the golden files. @@ -104,11 +106,13 @@ To regenerate the golden files, you can run the following command: ```sh SPARK_GENERATE_GOLDEN_FILES=1 ./mvnw -pl spark -Dsuites="org.apache.spark.sql.comet.CometTPCDSV1_4_PlanStabilitySuite" test +SPARK_GENERATE_GOLDEN_FILES=1 ./mvnw -pl spark -Dsuites="org.apache.spark.sql.comet.CometTPCDSV1_4_PlanStabilitySuite" -Pspark-4.0 -nsu test ``` and ```sh SPARK_GENERATE_GOLDEN_FILES=1 ./mvnw -pl spark -Dsuites="org.apache.spark.sql.comet.CometTPCDSV2_7_PlanStabilitySuite" test +SPARK_GENERATE_GOLDEN_FILES=1 ./mvnw -pl spark -Dsuites="org.apache.spark.sql.comet.CometTPCDSV2_7_PlanStabilitySuite" -Pspark-4.0 -nsu test ``` ## Benchmark diff --git a/docs/source/user-guide/configs.md b/docs/source/user-guide/configs.md index eb349b349..104f29ce8 100644 --- a/docs/source/user-guide/configs.md +++ b/docs/source/user-guide/configs.md @@ -23,7 +23,6 @@ Comet provides the following configuration settings. | Config | Description | Default Value | |--------|-------------|---------------| -| spark.comet.ansi.enabled | Comet does not respect ANSI mode in most cases and by default will not accelerate queries when ansi mode is enabled. Enable this setting to test Comet's experimental support for ANSI mode. This should not be used in production. | false | | spark.comet.batchSize | The columnar batch size, i.e., the maximum number of rows that a batch can contain. | 8192 | | spark.comet.cast.allowIncompatible | Comet is not currently fully compatible with Spark for all cast operations. Set this config to true to allow them anyway. See compatibility guide for more information. | false | | spark.comet.columnar.shuffle.async.enabled | Whether to enable asynchronous shuffle for Arrow-based shuffle. By default, this config is false. | false | diff --git a/spark/src/main/scala/org/apache/comet/CometSparkSessionExtensions.scala b/spark/src/main/scala/org/apache/comet/CometSparkSessionExtensions.scala index 168d2bb52..d6ec85f5b 100644 --- a/spark/src/main/scala/org/apache/comet/CometSparkSessionExtensions.scala +++ b/spark/src/main/scala/org/apache/comet/CometSparkSessionExtensions.scala @@ -1030,6 +1030,10 @@ object CometSparkSessionExtensions extends Logging { org.apache.spark.SPARK_VERSION >= "3.4" } + def isSpark40Plus: Boolean = { + org.apache.spark.SPARK_VERSION >= "4.0" + } + /** Calculates required memory overhead in MB per executor process for Comet. */ def getCometMemoryOverheadInMiB(sparkConf: SparkConf): Long = { // `spark.executor.memory` default value is 1g diff --git a/spark/src/main/scala/org/apache/comet/serde/QueryPlanSerde.scala b/spark/src/main/scala/org/apache/comet/serde/QueryPlanSerde.scala index 5fe290cf6..439ec4ebb 100644 --- a/spark/src/main/scala/org/apache/comet/serde/QueryPlanSerde.scala +++ b/spark/src/main/scala/org/apache/comet/serde/QueryPlanSerde.scala @@ -2244,7 +2244,8 @@ object QueryPlanSerde extends Logging with ShimQueryPlanSerde with CometExprShim } def nullIfWhenPrimitive(expression: Expression): Expression = if (isPrimitive(expression)) { - new NullIf(expression, Literal.default(expression.dataType)).child + val zero = Literal.default(expression.dataType) + If(EqualTo(expression, zero), Literal.create(null, expression.dataType), expression) } else { expression } diff --git a/spark/src/main/scala/org/apache/spark/sql/ExtendedExplainGenerator.scala b/spark/src/main/spark-3.x/org/apache/spark/sql/ExtendedExplainGenerator.scala similarity index 100% rename from spark/src/main/scala/org/apache/spark/sql/ExtendedExplainGenerator.scala rename to spark/src/main/spark-3.x/org/apache/spark/sql/ExtendedExplainGenerator.scala 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 new file mode 100644 index 000000000..762f3a4f6 --- /dev/null +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q1/explain.txt @@ -0,0 +1,274 @@ +== Physical Plan == +TakeOrderedAndProject (40) ++- * Project (39) + +- * BroadcastHashJoin Inner BuildRight (38) + :- * Project (33) + : +- * BroadcastHashJoin Inner BuildRight (32) + : :- * Project (26) + : : +- * BroadcastHashJoin Inner BuildRight (25) + : : :- * Filter (10) + : : : +- * HashAggregate (9) + : : : +- Exchange (8) + : : : +- * HashAggregate (7) + : : : +- * Project (6) + : : : +- * BroadcastHashJoin Inner BuildRight (5) + : : : :- * ColumnarToRow (3) + : : : : +- CometFilter (2) + : : : : +- CometScan parquet spark_catalog.default.store_returns (1) + : : : +- ReusedExchange (4) + : : +- BroadcastExchange (24) + : : +- * Filter (23) + : : +- * HashAggregate (22) + : : +- Exchange (21) + : : +- * HashAggregate (20) + : : +- * HashAggregate (19) + : : +- Exchange (18) + : : +- * HashAggregate (17) + : : +- * Project (16) + : : +- * BroadcastHashJoin Inner BuildRight (15) + : : :- * ColumnarToRow (13) + : : : +- CometFilter (12) + : : : +- CometScan parquet spark_catalog.default.store_returns (11) + : : +- ReusedExchange (14) + : +- BroadcastExchange (31) + : +- * ColumnarToRow (30) + : +- CometProject (29) + : +- CometFilter (28) + : +- CometScan parquet spark_catalog.default.store (27) + +- BroadcastExchange (37) + +- * ColumnarToRow (36) + +- CometFilter (35) + +- CometScan parquet spark_catalog.default.customer (34) + + +(1) Scan parquet spark_catalog.default.store_returns +Output [4]: [sr_customer_sk#1, sr_store_sk#2, sr_return_amt#3, sr_returned_date_sk#4] +Batched: true +Location: InMemoryFileIndex [] +PartitionFilters: [isnotnull(sr_returned_date_sk#4), dynamicpruningexpression(sr_returned_date_sk#4 IN dynamicpruning#5)] +PushedFilters: [IsNotNull(sr_store_sk), IsNotNull(sr_customer_sk)] +ReadSchema: struct<sr_customer_sk:int,sr_store_sk:int,sr_return_amt:decimal(7,2)> + +(2) CometFilter +Input [4]: [sr_customer_sk#1, sr_store_sk#2, sr_return_amt#3, sr_returned_date_sk#4] +Condition : (isnotnull(sr_store_sk#2) AND isnotnull(sr_customer_sk#1)) + +(3) ColumnarToRow [codegen id : 2] +Input [4]: [sr_customer_sk#1, sr_store_sk#2, sr_return_amt#3, sr_returned_date_sk#4] + +(4) ReusedExchange [Reuses operator id: 45] +Output [1]: [d_date_sk#6] + +(5) BroadcastHashJoin [codegen id : 2] +Left keys [1]: [sr_returned_date_sk#4] +Right keys [1]: [d_date_sk#6] +Join type: Inner +Join condition: None + +(6) 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] + +(7) 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))] +Aggregate Attributes [1]: [sum#7] +Results [3]: [sr_customer_sk#1, sr_store_sk#2, sum#8] + +(8) Exchange +Input [3]: [sr_customer_sk#1, sr_store_sk#2, sum#8] +Arguments: hashpartitioning(sr_customer_sk#1, sr_store_sk#2, 5), ENSURE_REQUIREMENTS, [plan_id=1] + +(9) HashAggregate [codegen id : 9] +Input [3]: [sr_customer_sk#1, sr_store_sk#2, sum#8] +Keys [2]: [sr_customer_sk#1, sr_store_sk#2] +Functions [1]: [sum(UnscaledValue(sr_return_amt#3))] +Aggregate Attributes [1]: [sum(UnscaledValue(sr_return_amt#3))#9] +Results [3]: [sr_customer_sk#1 AS ctr_customer_sk#10, sr_store_sk#2 AS ctr_store_sk#11, MakeDecimal(sum(UnscaledValue(sr_return_amt#3))#9,17,2) AS ctr_total_return#12] + +(10) Filter [codegen id : 9] +Input [3]: [ctr_customer_sk#10, ctr_store_sk#11, ctr_total_return#12] +Condition : isnotnull(ctr_total_return#12) + +(11) Scan parquet spark_catalog.default.store_returns +Output [4]: [sr_customer_sk#13, sr_store_sk#14, sr_return_amt#15, sr_returned_date_sk#16] +Batched: true +Location: InMemoryFileIndex [] +PartitionFilters: [isnotnull(sr_returned_date_sk#16), dynamicpruningexpression(sr_returned_date_sk#16 IN dynamicpruning#17)] +PushedFilters: [IsNotNull(sr_store_sk)] +ReadSchema: struct<sr_customer_sk:int,sr_store_sk:int,sr_return_amt:decimal(7,2)> + +(12) CometFilter +Input [4]: [sr_customer_sk#13, sr_store_sk#14, sr_return_amt#15, sr_returned_date_sk#16] +Condition : isnotnull(sr_store_sk#14) + +(13) ColumnarToRow [codegen id : 4] +Input [4]: [sr_customer_sk#13, sr_store_sk#14, sr_return_amt#15, sr_returned_date_sk#16] + +(14) ReusedExchange [Reuses operator id: 45] +Output [1]: [d_date_sk#18] + +(15) BroadcastHashJoin [codegen id : 4] +Left keys [1]: [sr_returned_date_sk#16] +Right keys [1]: [d_date_sk#18] +Join type: Inner +Join condition: None + +(16) Project [codegen id : 4] +Output [3]: [sr_customer_sk#13, sr_store_sk#14, sr_return_amt#15] +Input [5]: [sr_customer_sk#13, sr_store_sk#14, sr_return_amt#15, sr_returned_date_sk#16, d_date_sk#18] + +(17) HashAggregate [codegen id : 4] +Input [3]: [sr_customer_sk#13, sr_store_sk#14, sr_return_amt#15] +Keys [2]: [sr_customer_sk#13, sr_store_sk#14] +Functions [1]: [partial_sum(UnscaledValue(sr_return_amt#15))] +Aggregate Attributes [1]: [sum#19] +Results [3]: [sr_customer_sk#13, sr_store_sk#14, sum#20] + +(18) Exchange +Input [3]: [sr_customer_sk#13, sr_store_sk#14, sum#20] +Arguments: hashpartitioning(sr_customer_sk#13, sr_store_sk#14, 5), ENSURE_REQUIREMENTS, [plan_id=2] + +(19) HashAggregate [codegen id : 5] +Input [3]: [sr_customer_sk#13, sr_store_sk#14, sum#20] +Keys [2]: [sr_customer_sk#13, sr_store_sk#14] +Functions [1]: [sum(UnscaledValue(sr_return_amt#15))] +Aggregate Attributes [1]: [sum(UnscaledValue(sr_return_amt#15))#9] +Results [2]: [sr_store_sk#14 AS ctr_store_sk#21, MakeDecimal(sum(UnscaledValue(sr_return_amt#15))#9,17,2) AS ctr_total_return#22] + +(20) HashAggregate [codegen id : 5] +Input [2]: [ctr_store_sk#21, ctr_total_return#22] +Keys [1]: [ctr_store_sk#21] +Functions [1]: [partial_avg(ctr_total_return#22)] +Aggregate Attributes [2]: [sum#23, count#24] +Results [3]: [ctr_store_sk#21, sum#25, count#26] + +(21) Exchange +Input [3]: [ctr_store_sk#21, sum#25, count#26] +Arguments: hashpartitioning(ctr_store_sk#21, 5), ENSURE_REQUIREMENTS, [plan_id=3] + +(22) HashAggregate [codegen id : 6] +Input [3]: [ctr_store_sk#21, sum#25, count#26] +Keys [1]: [ctr_store_sk#21] +Functions [1]: [avg(ctr_total_return#22)] +Aggregate Attributes [1]: [avg(ctr_total_return#22)#27] +Results [2]: [(avg(ctr_total_return#22)#27 * 1.2) AS (avg(ctr_total_return) * 1.2)#28, ctr_store_sk#21] + +(23) Filter [codegen id : 6] +Input [2]: [(avg(ctr_total_return) * 1.2)#28, ctr_store_sk#21] +Condition : isnotnull((avg(ctr_total_return) * 1.2)#28) + +(24) BroadcastExchange +Input [2]: [(avg(ctr_total_return) * 1.2)#28, ctr_store_sk#21] +Arguments: HashedRelationBroadcastMode(List(cast(input[1, int, true] as bigint)),false), [plan_id=4] + +(25) BroadcastHashJoin [codegen id : 9] +Left keys [1]: [ctr_store_sk#11] +Right keys [1]: [ctr_store_sk#21] +Join type: Inner +Join condition: (cast(ctr_total_return#12 as decimal(24,7)) > (avg(ctr_total_return) * 1.2)#28) + +(26) Project [codegen id : 9] +Output [2]: [ctr_customer_sk#10, ctr_store_sk#11] +Input [5]: [ctr_customer_sk#10, ctr_store_sk#11, ctr_total_return#12, (avg(ctr_total_return) * 1.2)#28, ctr_store_sk#21] + +(27) Scan parquet spark_catalog.default.store +Output [2]: [s_store_sk#29, s_state#30] +Batched: true +Location [not included in comparison]/{warehouse_dir}/store] +PushedFilters: [IsNotNull(s_state), EqualTo(s_state,TN), IsNotNull(s_store_sk)] +ReadSchema: struct<s_store_sk:int,s_state:string> + +(28) CometFilter +Input [2]: [s_store_sk#29, s_state#30] +Condition : ((isnotnull(s_state#30) AND (s_state#30 = TN)) AND isnotnull(s_store_sk#29)) + +(29) CometProject +Input [2]: [s_store_sk#29, s_state#30] +Arguments: [s_store_sk#29], [s_store_sk#29] + +(30) ColumnarToRow [codegen id : 7] +Input [1]: [s_store_sk#29] + +(31) BroadcastExchange +Input [1]: [s_store_sk#29] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] + +(32) BroadcastHashJoin [codegen id : 9] +Left keys [1]: [ctr_store_sk#11] +Right keys [1]: [s_store_sk#29] +Join type: Inner +Join condition: None + +(33) Project [codegen id : 9] +Output [1]: [ctr_customer_sk#10] +Input [3]: [ctr_customer_sk#10, ctr_store_sk#11, s_store_sk#29] + +(34) Scan parquet spark_catalog.default.customer +Output [2]: [c_customer_sk#31, c_customer_id#32] +Batched: true +Location [not included in comparison]/{warehouse_dir}/customer] +PushedFilters: [IsNotNull(c_customer_sk)] +ReadSchema: struct<c_customer_sk:int,c_customer_id:string> + +(35) CometFilter +Input [2]: [c_customer_sk#31, c_customer_id#32] +Condition : isnotnull(c_customer_sk#31) + +(36) ColumnarToRow [codegen id : 8] +Input [2]: [c_customer_sk#31, c_customer_id#32] + +(37) BroadcastExchange +Input [2]: [c_customer_sk#31, c_customer_id#32] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=6] + +(38) BroadcastHashJoin [codegen id : 9] +Left keys [1]: [ctr_customer_sk#10] +Right keys [1]: [c_customer_sk#31] +Join type: Inner +Join condition: None + +(39) Project [codegen id : 9] +Output [1]: [c_customer_id#32] +Input [3]: [ctr_customer_sk#10, c_customer_sk#31, c_customer_id#32] + +(40) TakeOrderedAndProject +Input [1]: [c_customer_id#32] +Arguments: 100, [c_customer_id#32 ASC NULLS FIRST], [c_customer_id#32] + +===== Subqueries ===== + +Subquery:1 Hosting operator id = 1 Hosting Expression = sr_returned_date_sk#4 IN dynamicpruning#5 +BroadcastExchange (45) ++- * ColumnarToRow (44) + +- CometProject (43) + +- CometFilter (42) + +- CometScan parquet spark_catalog.default.date_dim (41) + + +(41) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#6, d_year#33] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2000), IsNotNull(d_date_sk)] +ReadSchema: struct<d_date_sk:int,d_year:int> + +(42) CometFilter +Input [2]: [d_date_sk#6, d_year#33] +Condition : ((isnotnull(d_year#33) AND (d_year#33 = 2000)) AND isnotnull(d_date_sk#6)) + +(43) CometProject +Input [2]: [d_date_sk#6, d_year#33] +Arguments: [d_date_sk#6], [d_date_sk#6] + +(44) ColumnarToRow [codegen id : 1] +Input [1]: [d_date_sk#6] + +(45) BroadcastExchange +Input [1]: [d_date_sk#6] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=7] + +Subquery:2 Hosting operator id = 11 Hosting Expression = sr_returned_date_sk#16 IN dynamicpruning#5 + + 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 new file mode 100644 index 000000000..688fb69a8 --- /dev/null +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q1/simplified.txt @@ -0,0 +1,68 @@ +TakeOrderedAndProject [c_customer_id] + WholeStageCodegen (9) + Project [c_customer_id] + BroadcastHashJoin [ctr_customer_sk,c_customer_sk] + Project [ctr_customer_sk] + BroadcastHashJoin [ctr_store_sk,s_store_sk] + Project [ctr_customer_sk,ctr_store_sk] + BroadcastHashJoin [ctr_store_sk,ctr_store_sk,ctr_total_return,(avg(ctr_total_return) * 1.2)] + Filter [ctr_total_return] + HashAggregate [sr_customer_sk,sr_store_sk,sum] [sum(UnscaledValue(sr_return_amt)),ctr_customer_sk,ctr_store_sk,ctr_total_return,sum] + InputAdapter + Exchange [sr_customer_sk,sr_store_sk] #1 + WholeStageCodegen (2) + HashAggregate [sr_customer_sk,sr_store_sk,sr_return_amt] [sum,sum] + Project [sr_customer_sk,sr_store_sk,sr_return_amt] + BroadcastHashJoin [sr_returned_date_sk,d_date_sk] + ColumnarToRow + InputAdapter + CometFilter [sr_store_sk,sr_customer_sk] + CometScan parquet spark_catalog.default.store_returns [sr_customer_sk,sr_store_sk,sr_return_amt,sr_returned_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #2 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_year,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + InputAdapter + ReusedExchange [d_date_sk] #2 + InputAdapter + BroadcastExchange #3 + WholeStageCodegen (6) + Filter [(avg(ctr_total_return) * 1.2)] + HashAggregate [ctr_store_sk,sum,count] [avg(ctr_total_return),(avg(ctr_total_return) * 1.2),sum,count] + InputAdapter + Exchange [ctr_store_sk] #4 + WholeStageCodegen (5) + HashAggregate [ctr_store_sk,ctr_total_return] [sum,count,sum,count] + HashAggregate [sr_customer_sk,sr_store_sk,sum] [sum(UnscaledValue(sr_return_amt)),ctr_store_sk,ctr_total_return,sum] + InputAdapter + Exchange [sr_customer_sk,sr_store_sk] #5 + WholeStageCodegen (4) + HashAggregate [sr_customer_sk,sr_store_sk,sr_return_amt] [sum,sum] + Project [sr_customer_sk,sr_store_sk,sr_return_amt] + BroadcastHashJoin [sr_returned_date_sk,d_date_sk] + ColumnarToRow + InputAdapter + CometFilter [sr_store_sk] + CometScan parquet spark_catalog.default.store_returns [sr_customer_sk,sr_store_sk,sr_return_amt,sr_returned_date_sk] + ReusedSubquery [d_date_sk] #1 + InputAdapter + ReusedExchange [d_date_sk] #2 + InputAdapter + BroadcastExchange #6 + WholeStageCodegen (7) + ColumnarToRow + InputAdapter + CometProject [s_store_sk] + CometFilter [s_state,s_store_sk] + CometScan parquet spark_catalog.default.store [s_store_sk,s_state] + InputAdapter + BroadcastExchange #7 + WholeStageCodegen (8) + ColumnarToRow + InputAdapter + CometFilter [c_customer_sk] + CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q10/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q10/explain.txt new file mode 100644 index 000000000..15490b87d --- /dev/null +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q10/explain.txt @@ -0,0 +1,286 @@ +== Physical Plan == +TakeOrderedAndProject (43) ++- * HashAggregate (42) + +- Exchange (41) + +- * HashAggregate (40) + +- * Project (39) + +- * BroadcastHashJoin Inner BuildRight (38) + :- * Project (33) + : +- * BroadcastHashJoin Inner BuildRight (32) + : :- * Project (26) + : : +- * Filter (25) + : : +- * BroadcastHashJoin ExistenceJoin(exists#1) BuildRight (24) + : : :- * BroadcastHashJoin ExistenceJoin(exists#2) BuildRight (17) + : : : :- * BroadcastHashJoin LeftSemi BuildRight (10) + : : : : :- * ColumnarToRow (3) + : : : : : +- CometFilter (2) + : : : : : +- CometScan parquet spark_catalog.default.customer (1) + : : : : +- BroadcastExchange (9) + : : : : +- * Project (8) + : : : : +- * BroadcastHashJoin Inner BuildRight (7) + : : : : :- * ColumnarToRow (5) + : : : : : +- CometScan parquet spark_catalog.default.store_sales (4) + : : : : +- ReusedExchange (6) + : : : +- BroadcastExchange (16) + : : : +- * Project (15) + : : : +- * BroadcastHashJoin Inner BuildRight (14) + : : : :- * ColumnarToRow (12) + : : : : +- CometScan parquet spark_catalog.default.web_sales (11) + : : : +- ReusedExchange (13) + : : +- BroadcastExchange (23) + : : +- * Project (22) + : : +- * BroadcastHashJoin Inner BuildRight (21) + : : :- * ColumnarToRow (19) + : : : +- CometScan parquet spark_catalog.default.catalog_sales (18) + : : +- ReusedExchange (20) + : +- BroadcastExchange (31) + : +- * ColumnarToRow (30) + : +- CometProject (29) + : +- CometFilter (28) + : +- CometScan parquet spark_catalog.default.customer_address (27) + +- BroadcastExchange (37) + +- * ColumnarToRow (36) + +- CometFilter (35) + +- CometScan parquet spark_catalog.default.customer_demographics (34) + + +(1) Scan parquet spark_catalog.default.customer +Output [3]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5] +Batched: true +Location [not included in comparison]/{warehouse_dir}/customer] +PushedFilters: [IsNotNull(c_current_addr_sk), IsNotNull(c_current_cdemo_sk)] +ReadSchema: struct<c_customer_sk:int,c_current_cdemo_sk:int,c_current_addr_sk:int> + +(2) CometFilter +Input [3]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5] +Condition : (isnotnull(c_current_addr_sk#5) AND isnotnull(c_current_cdemo_sk#4)) + +(3) ColumnarToRow [codegen id : 9] +Input [3]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5] + +(4) Scan parquet spark_catalog.default.store_sales +Output [2]: [ss_customer_sk#6, ss_sold_date_sk#7] +Batched: true +Location: InMemoryFileIndex [] +PartitionFilters: [isnotnull(ss_sold_date_sk#7), dynamicpruningexpression(ss_sold_date_sk#7 IN dynamicpruning#8)] +ReadSchema: struct<ss_customer_sk:int> + +(5) ColumnarToRow [codegen id : 2] +Input [2]: [ss_customer_sk#6, ss_sold_date_sk#7] + +(6) ReusedExchange [Reuses operator id: 48] +Output [1]: [d_date_sk#9] + +(7) BroadcastHashJoin [codegen id : 2] +Left keys [1]: [ss_sold_date_sk#7] +Right keys [1]: [d_date_sk#9] +Join type: Inner +Join condition: None + +(8) Project [codegen id : 2] +Output [1]: [ss_customer_sk#6] +Input [3]: [ss_customer_sk#6, ss_sold_date_sk#7, d_date_sk#9] + +(9) BroadcastExchange +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] +Left keys [1]: [c_customer_sk#3] +Right keys [1]: [ss_customer_sk#6] +Join type: LeftSemi +Join condition: None + +(11) Scan parquet spark_catalog.default.web_sales +Output [2]: [ws_bill_customer_sk#10, ws_sold_date_sk#11] +Batched: true +Location: InMemoryFileIndex [] +PartitionFilters: [isnotnull(ws_sold_date_sk#11), dynamicpruningexpression(ws_sold_date_sk#11 IN dynamicpruning#12)] +ReadSchema: struct<ws_bill_customer_sk:int> + +(12) ColumnarToRow [codegen id : 4] +Input [2]: [ws_bill_customer_sk#10, ws_sold_date_sk#11] + +(13) ReusedExchange [Reuses operator id: 48] +Output [1]: [d_date_sk#13] + +(14) BroadcastHashJoin [codegen id : 4] +Left keys [1]: [ws_sold_date_sk#11] +Right keys [1]: [d_date_sk#13] +Join type: Inner +Join condition: None + +(15) Project [codegen id : 4] +Output [1]: [ws_bill_customer_sk#10] +Input [3]: [ws_bill_customer_sk#10, ws_sold_date_sk#11, d_date_sk#13] + +(16) BroadcastExchange +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] +Left keys [1]: [c_customer_sk#3] +Right keys [1]: [ws_bill_customer_sk#10] +Join type: ExistenceJoin(exists#2) +Join condition: None + +(18) Scan parquet spark_catalog.default.catalog_sales +Output [2]: [cs_ship_customer_sk#14, cs_sold_date_sk#15] +Batched: true +Location: InMemoryFileIndex [] +PartitionFilters: [isnotnull(cs_sold_date_sk#15), dynamicpruningexpression(cs_sold_date_sk#15 IN dynamicpruning#16)] +ReadSchema: struct<cs_ship_customer_sk:int> + +(19) ColumnarToRow [codegen id : 6] +Input [2]: [cs_ship_customer_sk#14, cs_sold_date_sk#15] + +(20) ReusedExchange [Reuses operator id: 48] +Output [1]: [d_date_sk#17] + +(21) BroadcastHashJoin [codegen id : 6] +Left keys [1]: [cs_sold_date_sk#15] +Right keys [1]: [d_date_sk#17] +Join type: Inner +Join condition: None + +(22) Project [codegen id : 6] +Output [1]: [cs_ship_customer_sk#14] +Input [3]: [cs_ship_customer_sk#14, cs_sold_date_sk#15, d_date_sk#17] + +(23) BroadcastExchange +Input [1]: [cs_ship_customer_sk#14] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] + +(24) BroadcastHashJoin [codegen id : 9] +Left keys [1]: [c_customer_sk#3] +Right keys [1]: [cs_ship_customer_sk#14] +Join type: ExistenceJoin(exists#1) +Join condition: None + +(25) Filter [codegen id : 9] +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] +Output [2]: [c_current_cdemo_sk#4, c_current_addr_sk#5] +Input [5]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5, exists#2, exists#1] + +(27) Scan parquet spark_catalog.default.customer_address +Output [2]: [ca_address_sk#18, ca_county#19] +Batched: true +Location [not included in comparison]/{warehouse_dir}/customer_address] +PushedFilters: [In(ca_county, [Dona Ana County,Jefferson County,La Porte County,Rush County,Toole County]), IsNotNull(ca_address_sk)] +ReadSchema: struct<ca_address_sk:int,ca_county:string> + +(28) CometFilter +Input [2]: [ca_address_sk#18, ca_county#19] +Condition : (ca_county#19 IN (Rush County,Toole County,Jefferson County,Dona Ana County,La Porte County) AND isnotnull(ca_address_sk#18)) + +(29) CometProject +Input [2]: [ca_address_sk#18, ca_county#19] +Arguments: [ca_address_sk#18], [ca_address_sk#18] + +(30) ColumnarToRow [codegen id : 7] +Input [1]: [ca_address_sk#18] + +(31) BroadcastExchange +Input [1]: [ca_address_sk#18] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] + +(32) BroadcastHashJoin [codegen id : 9] +Left keys [1]: [c_current_addr_sk#5] +Right keys [1]: [ca_address_sk#18] +Join type: Inner +Join condition: None + +(33) Project [codegen id : 9] +Output [1]: [c_current_cdemo_sk#4] +Input [3]: [c_current_cdemo_sk#4, c_current_addr_sk#5, ca_address_sk#18] + +(34) Scan parquet spark_catalog.default.customer_demographics +Output [9]: [cd_demo_sk#20, cd_gender#21, cd_marital_status#22, cd_education_status#23, cd_purchase_estimate#24, cd_credit_rating#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] +Batched: true +Location [not included in comparison]/{warehouse_dir}/customer_demographics] +PushedFilters: [IsNotNull(cd_demo_sk)] +ReadSchema: struct<cd_demo_sk:int,cd_gender:string,cd_marital_status:string,cd_education_status:string,cd_purchase_estimate:int,cd_credit_rating:string,cd_dep_count:int,cd_dep_employed_count:int,cd_dep_college_count:int> + +(35) CometFilter +Input [9]: [cd_demo_sk#20, cd_gender#21, cd_marital_status#22, cd_education_status#23, cd_purchase_estimate#24, cd_credit_rating#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] +Condition : isnotnull(cd_demo_sk#20) + +(36) ColumnarToRow [codegen id : 8] +Input [9]: [cd_demo_sk#20, cd_gender#21, cd_marital_status#22, cd_education_status#23, cd_purchase_estimate#24, cd_credit_rating#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] + +(37) BroadcastExchange +Input [9]: [cd_demo_sk#20, cd_gender#21, cd_marital_status#22, cd_education_status#23, cd_purchase_estimate#24, cd_credit_rating#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=5] + +(38) BroadcastHashJoin [codegen id : 9] +Left keys [1]: [c_current_cdemo_sk#4] +Right keys [1]: [cd_demo_sk#20] +Join type: Inner +Join condition: None + +(39) Project [codegen id : 9] +Output [8]: [cd_gender#21, cd_marital_status#22, cd_education_status#23, cd_purchase_estimate#24, cd_credit_rating#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] +Input [10]: [c_current_cdemo_sk#4, cd_demo_sk#20, cd_gender#21, cd_marital_status#22, cd_education_status#23, cd_purchase_estimate#24, cd_credit_rating#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] + +(40) HashAggregate [codegen id : 9] +Input [8]: [cd_gender#21, cd_marital_status#22, cd_education_status#23, cd_purchase_estimate#24, cd_credit_rating#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] +Keys [8]: [cd_gender#21, cd_marital_status#22, cd_education_status#23, cd_purchase_estimate#24, cd_credit_rating#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] +Functions [1]: [partial_count(1)] +Aggregate Attributes [1]: [count#29] +Results [9]: [cd_gender#21, cd_marital_status#22, cd_education_status#23, cd_purchase_estimate#24, cd_credit_rating#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28, count#30] + +(41) Exchange +Input [9]: [cd_gender#21, cd_marital_status#22, cd_education_status#23, cd_purchase_estimate#24, cd_credit_rating#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28, count#30] +Arguments: hashpartitioning(cd_gender#21, cd_marital_status#22, cd_education_status#23, cd_purchase_estimate#24, cd_credit_rating#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28, 5), ENSURE_REQUIREMENTS, [plan_id=6] + +(42) HashAggregate [codegen id : 10] +Input [9]: [cd_gender#21, cd_marital_status#22, cd_education_status#23, cd_purchase_estimate#24, cd_credit_rating#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28, count#30] +Keys [8]: [cd_gender#21, cd_marital_status#22, cd_education_status#23, cd_purchase_estimate#24, cd_credit_rating#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] +Functions [1]: [count(1)] +Aggregate Attributes [1]: [count(1)#31] +Results [14]: [cd_gender#21, cd_marital_status#22, cd_education_status#23, count(1)#31 AS cnt1#32, cd_purchase_estimate#24, count(1)#31 AS cnt2#33, cd_credit_rating#25, count(1)#31 AS cnt3#34, cd_dep_count#26, count(1)#31 AS cnt4#35, cd_dep_employed_count#27, count(1)#31 AS cnt5#36, cd_dep_college_count#28, count(1)#31 AS cnt6#37] + +(43) TakeOrderedAndProject +Input [14]: [cd_gender#21, cd_marital_status#22, cd_education_status#23, cnt1#32, cd_purchase_estimate#24, cnt2#33, cd_credit_rating#25, cnt3#34, cd_dep_count#26, cnt4#35, cd_dep_employed_count#27, cnt5#36, cd_dep_college_count#28, cnt6#37] +Arguments: 100, [cd_gender#21 ASC NULLS FIRST, cd_marital_status#22 ASC NULLS FIRST, cd_education_status#23 ASC NULLS FIRST, cd_purchase_estimate#24 ASC NULLS FIRST, cd_credit_rating#25 ASC NULLS FIRST, cd_dep_count#26 ASC NULLS FIRST, cd_dep_employed_count#27 ASC NULLS FIRST, cd_dep_college_count#28 ASC NULLS FIRST], [cd_gender#21, cd_marital_status#22, cd_education_status#23, cnt1#32, cd_purchase_estimate#24, cnt2#33, cd_credit_rating#25, cnt3#34, cd_dep_count#26, cnt4#35, cd_dep_employed_count#27, cnt5#36, cd_dep_college_count#28, cnt6#37] + +===== Subqueries ===== + +Subquery:1 Hosting operator id = 4 Hosting Expression = ss_sold_date_sk#7 IN dynamicpruning#8 +BroadcastExchange (48) ++- * ColumnarToRow (47) + +- CometProject (46) + +- CometFilter (45) + +- CometScan parquet spark_catalog.default.date_dim (44) + + +(44) Scan parquet spark_catalog.default.date_dim +Output [3]: [d_date_sk#9, d_year#38, d_moy#39] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2002), GreaterThanOrEqual(d_moy,1), LessThanOrEqual(d_moy,4), IsNotNull(d_date_sk)] +ReadSchema: struct<d_date_sk:int,d_year:int,d_moy:int> + +(45) CometFilter +Input [3]: [d_date_sk#9, d_year#38, d_moy#39] +Condition : (((((isnotnull(d_year#38) AND isnotnull(d_moy#39)) AND (d_year#38 = 2002)) AND (d_moy#39 >= 1)) AND (d_moy#39 <= 4)) AND isnotnull(d_date_sk#9)) + +(46) CometProject +Input [3]: [d_date_sk#9, d_year#38, d_moy#39] +Arguments: [d_date_sk#9], [d_date_sk#9] + +(47) ColumnarToRow [codegen id : 1] +Input [1]: [d_date_sk#9] + +(48) BroadcastExchange +Input [1]: [d_date_sk#9] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=7] + +Subquery:2 Hosting operator id = 11 Hosting Expression = ws_sold_date_sk#11 IN dynamicpruning#8 + +Subquery:3 Hosting operator id = 18 Hosting Expression = cs_sold_date_sk#15 IN dynamicpruning#8 + + diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q10/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q10/simplified.txt new file mode 100644 index 000000000..89893c831 --- /dev/null +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q10/simplified.txt @@ -0,0 +1,75 @@ +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) + HashAggregate [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count,count] [count(1),cnt1,cnt2,cnt3,cnt4,cnt5,cnt6,count] + InputAdapter + Exchange [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] #1 + WholeStageCodegen (9) + HashAggregate [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] [count,count] + Project [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] + BroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] + Project [c_current_cdemo_sk] + BroadcastHashJoin [c_current_addr_sk,ca_address_sk] + Project [c_current_cdemo_sk,c_current_addr_sk] + Filter [exists,exists] + BroadcastHashJoin [c_customer_sk,cs_ship_customer_sk] + BroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] + BroadcastHashJoin [c_customer_sk,ss_customer_sk] + ColumnarToRow + InputAdapter + CometFilter [c_current_addr_sk,c_current_cdemo_sk] + CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] + InputAdapter + BroadcastExchange #2 + WholeStageCodegen (2) + Project [ss_customer_sk] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + ColumnarToRow + InputAdapter + CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_year,d_moy,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + InputAdapter + ReusedExchange [d_date_sk] #3 + InputAdapter + BroadcastExchange #4 + WholeStageCodegen (4) + Project [ws_bill_customer_sk] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + ColumnarToRow + InputAdapter + CometScan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + InputAdapter + ReusedExchange [d_date_sk] #3 + InputAdapter + BroadcastExchange #5 + WholeStageCodegen (6) + Project [cs_ship_customer_sk] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + ColumnarToRow + InputAdapter + CometScan parquet spark_catalog.default.catalog_sales [cs_ship_customer_sk,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + InputAdapter + ReusedExchange [d_date_sk] #3 + InputAdapter + BroadcastExchange #6 + WholeStageCodegen (7) + ColumnarToRow + InputAdapter + CometProject [ca_address_sk] + CometFilter [ca_county,ca_address_sk] + CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_county] + InputAdapter + BroadcastExchange #7 + WholeStageCodegen (8) + ColumnarToRow + InputAdapter + CometFilter [cd_demo_sk] + CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] 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 new file mode 100644 index 000000000..c663d4688 --- /dev/null +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q11/explain.txt @@ -0,0 +1,482 @@ +== Physical Plan == +TakeOrderedAndProject (72) ++- * Project (71) + +- * BroadcastHashJoin Inner BuildRight (70) + :- * Project (53) + : +- * BroadcastHashJoin Inner BuildRight (52) + : :- * Project (34) + : : +- * BroadcastHashJoin Inner BuildRight (33) + : : :- * Filter (16) + : : : +- * HashAggregate (15) + : : : +- Exchange (14) + : : : +- * HashAggregate (13) + : : : +- * Project (12) + : : : +- * BroadcastHashJoin Inner BuildRight (11) + : : : :- * Project (9) + : : : : +- * BroadcastHashJoin Inner BuildRight (8) + : : : : :- * ColumnarToRow (3) + : : : : : +- CometFilter (2) + : : : : : +- CometScan parquet spark_catalog.default.customer (1) + : : : : +- BroadcastExchange (7) + : : : : +- * ColumnarToRow (6) + : : : : +- CometFilter (5) + : : : : +- CometScan parquet spark_catalog.default.store_sales (4) + : : : +- ReusedExchange (10) + : : +- BroadcastExchange (32) + : : +- * HashAggregate (31) + : : +- Exchange (30) + : : +- * HashAggregate (29) + : : +- * Project (28) + : : +- * BroadcastHashJoin Inner BuildRight (27) + : : :- * Project (25) + : : : +- * BroadcastHashJoin Inner BuildRight (24) + : : : :- * ColumnarToRow (19) + : : : : +- CometFilter (18) + : : : : +- CometScan parquet spark_catalog.default.customer (17) + : : : +- BroadcastExchange (23) + : : : +- * ColumnarToRow (22) + : : : +- CometFilter (21) + : : : +- CometScan parquet spark_catalog.default.store_sales (20) + : : +- ReusedExchange (26) + : +- BroadcastExchange (51) + : +- * Filter (50) + : +- * HashAggregate (49) + : +- Exchange (48) + : +- * HashAggregate (47) + : +- * Project (46) + : +- * BroadcastHashJoin Inner BuildRight (45) + : :- * Project (43) + : : +- * BroadcastHashJoin Inner BuildRight (42) + : : :- * ColumnarToRow (37) + : : : +- CometFilter (36) + : : : +- CometScan parquet spark_catalog.default.customer (35) + : : +- BroadcastExchange (41) + : : +- * ColumnarToRow (40) + : : +- CometFilter (39) + : : +- CometScan parquet spark_catalog.default.web_sales (38) + : +- ReusedExchange (44) + +- BroadcastExchange (69) + +- * HashAggregate (68) + +- Exchange (67) + +- * HashAggregate (66) + +- * Project (65) + +- * BroadcastHashJoin Inner BuildRight (64) + :- * Project (62) + : +- * BroadcastHashJoin Inner BuildRight (61) + : :- * ColumnarToRow (56) + : : +- CometFilter (55) + : : +- CometScan parquet spark_catalog.default.customer (54) + : +- BroadcastExchange (60) + : +- * ColumnarToRow (59) + : +- CometFilter (58) + : +- CometScan parquet spark_catalog.default.web_sales (57) + +- ReusedExchange (63) + + +(1) Scan parquet spark_catalog.default.customer +Output [8]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8] +Batched: true +Location [not included in comparison]/{warehouse_dir}/customer] +PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_customer_id)] +ReadSchema: struct<c_customer_sk:int,c_customer_id:string,c_first_name:string,c_last_name:string,c_preferred_cust_flag:string,c_birth_country:string,c_login:string,c_email_address:string> + +(2) CometFilter +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] +Condition : (isnotnull(c_customer_sk#1) AND isnotnull(c_customer_id#2)) + +(3) ColumnarToRow [codegen id : 3] +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] + +(4) Scan parquet spark_catalog.default.store_sales +Output [4]: [ss_customer_sk#9, ss_ext_discount_amt#10, ss_ext_list_price#11, ss_sold_date_sk#12] +Batched: true +Location: InMemoryFileIndex [] +PartitionFilters: [isnotnull(ss_sold_date_sk#12), dynamicpruningexpression(ss_sold_date_sk#12 IN dynamicpruning#13)] +PushedFilters: [IsNotNull(ss_customer_sk)] +ReadSchema: struct<ss_customer_sk:int,ss_ext_discount_amt:decimal(7,2),ss_ext_list_price:decimal(7,2)> + +(5) CometFilter +Input [4]: [ss_customer_sk#9, ss_ext_discount_amt#10, ss_ext_list_price#11, ss_sold_date_sk#12] +Condition : isnotnull(ss_customer_sk#9) + +(6) ColumnarToRow [codegen id : 1] +Input [4]: [ss_customer_sk#9, ss_ext_discount_amt#10, ss_ext_list_price#11, ss_sold_date_sk#12] + +(7) BroadcastExchange +Input [4]: [ss_customer_sk#9, ss_ext_discount_amt#10, ss_ext_list_price#11, ss_sold_date_sk#12] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=1] + +(8) BroadcastHashJoin [codegen id : 3] +Left keys [1]: [c_customer_sk#1] +Right keys [1]: [ss_customer_sk#9] +Join type: Inner +Join condition: None + +(9) Project [codegen id : 3] +Output [10]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, ss_ext_discount_amt#10, ss_ext_list_price#11, ss_sold_date_sk#12] +Input [12]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, ss_customer_sk#9, ss_ext_discount_amt#10, ss_ext_list_price#11, ss_sold_date_sk#12] + +(10) ReusedExchange [Reuses operator id: 76] +Output [2]: [d_date_sk#14, d_year#15] + +(11) BroadcastHashJoin [codegen id : 3] +Left keys [1]: [ss_sold_date_sk#12] +Right keys [1]: [d_date_sk#14] +Join type: Inner +Join condition: None + +(12) Project [codegen id : 3] +Output [10]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, ss_ext_discount_amt#10, ss_ext_list_price#11, d_year#15] +Input [12]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, ss_ext_discount_amt#10, ss_ext_list_price#11, ss_sold_date_sk#12, d_date_sk#14, d_year#15] + +(13) HashAggregate [codegen id : 3] +Input [10]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, ss_ext_discount_amt#10, ss_ext_list_price#11, d_year#15] +Keys [8]: [c_customer_id#2, c_first_name#3, c_last_name#4, d_year#15, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8] +Functions [1]: [partial_sum(UnscaledValue((ss_ext_list_price#11 - ss_ext_discount_amt#10)))] +Aggregate Attributes [1]: [sum#16] +Results [9]: [c_customer_id#2, c_first_name#3, c_last_name#4, d_year#15, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, sum#17] + +(14) Exchange +Input [9]: [c_customer_id#2, c_first_name#3, c_last_name#4, d_year#15, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, sum#17] +Arguments: hashpartitioning(c_customer_id#2, c_first_name#3, c_last_name#4, d_year#15, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, 5), ENSURE_REQUIREMENTS, [plan_id=2] + +(15) HashAggregate [codegen id : 16] +Input [9]: [c_customer_id#2, c_first_name#3, c_last_name#4, d_year#15, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, sum#17] +Keys [8]: [c_customer_id#2, c_first_name#3, c_last_name#4, d_year#15, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8] +Functions [1]: [sum(UnscaledValue((ss_ext_list_price#11 - ss_ext_discount_amt#10)))] +Aggregate Attributes [1]: [sum(UnscaledValue((ss_ext_list_price#11 - ss_ext_discount_amt#10)))#18] +Results [2]: [c_customer_id#2 AS customer_id#19, MakeDecimal(sum(UnscaledValue((ss_ext_list_price#11 - ss_ext_discount_amt#10)))#18,18,2) AS year_total#20] + +(16) Filter [codegen id : 16] +Input [2]: [customer_id#19, year_total#20] +Condition : (isnotnull(year_total#20) AND (year_total#20 > 0.00)) + +(17) Scan parquet spark_catalog.default.customer +Output [8]: [c_customer_sk#21, c_customer_id#22, c_first_name#23, c_last_name#24, c_preferred_cust_flag#25, c_birth_country#26, c_login#27, c_email_address#28] +Batched: true +Location [not included in comparison]/{warehouse_dir}/customer] +PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_customer_id)] +ReadSchema: struct<c_customer_sk:int,c_customer_id:string,c_first_name:string,c_last_name:string,c_preferred_cust_flag:string,c_birth_country:string,c_login:string,c_email_address:string> + +(18) CometFilter +Input [8]: [c_customer_sk#21, c_customer_id#22, c_first_name#23, c_last_name#24, c_preferred_cust_flag#25, c_birth_country#26, c_login#27, c_email_address#28] +Condition : (isnotnull(c_customer_sk#21) AND isnotnull(c_customer_id#22)) + +(19) ColumnarToRow [codegen id : 6] +Input [8]: [c_customer_sk#21, c_customer_id#22, c_first_name#23, c_last_name#24, c_preferred_cust_flag#25, c_birth_country#26, c_login#27, c_email_address#28] + +(20) Scan parquet spark_catalog.default.store_sales +Output [4]: [ss_customer_sk#29, ss_ext_discount_amt#30, ss_ext_list_price#31, ss_sold_date_sk#32] +Batched: true +Location: InMemoryFileIndex [] +PartitionFilters: [isnotnull(ss_sold_date_sk#32), dynamicpruningexpression(ss_sold_date_sk#32 IN dynamicpruning#33)] +PushedFilters: [IsNotNull(ss_customer_sk)] +ReadSchema: struct<ss_customer_sk:int,ss_ext_discount_amt:decimal(7,2),ss_ext_list_price:decimal(7,2)> + +(21) CometFilter +Input [4]: [ss_customer_sk#29, ss_ext_discount_amt#30, ss_ext_list_price#31, ss_sold_date_sk#32] +Condition : isnotnull(ss_customer_sk#29) + +(22) ColumnarToRow [codegen id : 4] +Input [4]: [ss_customer_sk#29, ss_ext_discount_amt#30, ss_ext_list_price#31, ss_sold_date_sk#32] + +(23) BroadcastExchange +Input [4]: [ss_customer_sk#29, ss_ext_discount_amt#30, ss_ext_list_price#31, ss_sold_date_sk#32] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=3] + +(24) BroadcastHashJoin [codegen id : 6] +Left keys [1]: [c_customer_sk#21] +Right keys [1]: [ss_customer_sk#29] +Join type: Inner +Join condition: None + +(25) Project [codegen id : 6] +Output [10]: [c_customer_id#22, c_first_name#23, c_last_name#24, c_preferred_cust_flag#25, c_birth_country#26, c_login#27, c_email_address#28, ss_ext_discount_amt#30, ss_ext_list_price#31, ss_sold_date_sk#32] +Input [12]: [c_customer_sk#21, c_customer_id#22, c_first_name#23, c_last_name#24, c_preferred_cust_flag#25, c_birth_country#26, c_login#27, c_email_address#28, ss_customer_sk#29, ss_ext_discount_amt#30, ss_ext_list_price#31, ss_sold_date_sk#32] + +(26) ReusedExchange [Reuses operator id: 80] +Output [2]: [d_date_sk#34, d_year#35] + +(27) BroadcastHashJoin [codegen id : 6] +Left keys [1]: [ss_sold_date_sk#32] +Right keys [1]: [d_date_sk#34] +Join type: Inner +Join condition: None + +(28) Project [codegen id : 6] +Output [10]: [c_customer_id#22, c_first_name#23, c_last_name#24, c_preferred_cust_flag#25, c_birth_country#26, c_login#27, c_email_address#28, ss_ext_discount_amt#30, ss_ext_list_price#31, d_year#35] +Input [12]: [c_customer_id#22, c_first_name#23, c_last_name#24, c_preferred_cust_flag#25, c_birth_country#26, c_login#27, c_email_address#28, ss_ext_discount_amt#30, ss_ext_list_price#31, ss_sold_date_sk#32, d_date_sk#34, d_year#35] + +(29) HashAggregate [codegen id : 6] +Input [10]: [c_customer_id#22, c_first_name#23, c_last_name#24, c_preferred_cust_flag#25, c_birth_country#26, c_login#27, c_email_address#28, ss_ext_discount_amt#30, ss_ext_list_price#31, d_year#35] +Keys [8]: [c_customer_id#22, c_first_name#23, c_last_name#24, d_year#35, c_preferred_cust_flag#25, c_birth_country#26, c_login#27, c_email_address#28] +Functions [1]: [partial_sum(UnscaledValue((ss_ext_list_price#31 - ss_ext_discount_amt#30)))] +Aggregate Attributes [1]: [sum#36] +Results [9]: [c_customer_id#22, c_first_name#23, c_last_name#24, d_year#35, c_preferred_cust_flag#25, c_birth_country#26, c_login#27, c_email_address#28, sum#37] + +(30) Exchange +Input [9]: [c_customer_id#22, c_first_name#23, c_last_name#24, d_year#35, c_preferred_cust_flag#25, c_birth_country#26, c_login#27, c_email_address#28, sum#37] +Arguments: hashpartitioning(c_customer_id#22, c_first_name#23, c_last_name#24, d_year#35, c_preferred_cust_flag#25, c_birth_country#26, c_login#27, c_email_address#28, 5), ENSURE_REQUIREMENTS, [plan_id=4] + +(31) HashAggregate [codegen id : 7] +Input [9]: [c_customer_id#22, c_first_name#23, c_last_name#24, d_year#35, c_preferred_cust_flag#25, c_birth_country#26, c_login#27, c_email_address#28, sum#37] +Keys [8]: [c_customer_id#22, c_first_name#23, c_last_name#24, d_year#35, c_preferred_cust_flag#25, c_birth_country#26, c_login#27, c_email_address#28] +Functions [1]: [sum(UnscaledValue((ss_ext_list_price#31 - ss_ext_discount_amt#30)))] +Aggregate Attributes [1]: [sum(UnscaledValue((ss_ext_list_price#31 - ss_ext_discount_amt#30)))#18] +Results [3]: [c_customer_id#22 AS customer_id#38, c_preferred_cust_flag#25 AS customer_preferred_cust_flag#39, MakeDecimal(sum(UnscaledValue((ss_ext_list_price#31 - ss_ext_discount_amt#30)))#18,18,2) AS year_total#40] + +(32) BroadcastExchange +Input [3]: [customer_id#38, customer_preferred_cust_flag#39, year_total#40] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=5] + +(33) BroadcastHashJoin [codegen id : 16] +Left keys [1]: [customer_id#19] +Right keys [1]: [customer_id#38] +Join type: Inner +Join condition: None + +(34) Project [codegen id : 16] +Output [4]: [customer_id#19, year_total#20, customer_preferred_cust_flag#39, year_total#40] +Input [5]: [customer_id#19, year_total#20, customer_id#38, customer_preferred_cust_flag#39, year_total#40] + +(35) Scan parquet spark_catalog.default.customer +Output [8]: [c_customer_sk#41, c_customer_id#42, c_first_name#43, c_last_name#44, c_preferred_cust_flag#45, c_birth_country#46, c_login#47, c_email_address#48] +Batched: true +Location [not included in comparison]/{warehouse_dir}/customer] +PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_customer_id)] +ReadSchema: struct<c_customer_sk:int,c_customer_id:string,c_first_name:string,c_last_name:string,c_preferred_cust_flag:string,c_birth_country:string,c_login:string,c_email_address:string> + +(36) CometFilter +Input [8]: [c_customer_sk#41, c_customer_id#42, c_first_name#43, c_last_name#44, c_preferred_cust_flag#45, c_birth_country#46, c_login#47, c_email_address#48] +Condition : (isnotnull(c_customer_sk#41) AND isnotnull(c_customer_id#42)) + +(37) ColumnarToRow [codegen id : 10] +Input [8]: [c_customer_sk#41, c_customer_id#42, c_first_name#43, c_last_name#44, c_preferred_cust_flag#45, c_birth_country#46, c_login#47, c_email_address#48] + +(38) Scan parquet spark_catalog.default.web_sales +Output [4]: [ws_bill_customer_sk#49, ws_ext_discount_amt#50, ws_ext_list_price#51, ws_sold_date_sk#52] +Batched: true +Location: InMemoryFileIndex [] +PartitionFilters: [isnotnull(ws_sold_date_sk#52), dynamicpruningexpression(ws_sold_date_sk#52 IN dynamicpruning#53)] +PushedFilters: [IsNotNull(ws_bill_customer_sk)] +ReadSchema: struct<ws_bill_customer_sk:int,ws_ext_discount_amt:decimal(7,2),ws_ext_list_price:decimal(7,2)> + +(39) CometFilter +Input [4]: [ws_bill_customer_sk#49, ws_ext_discount_amt#50, ws_ext_list_price#51, ws_sold_date_sk#52] +Condition : isnotnull(ws_bill_customer_sk#49) + +(40) ColumnarToRow [codegen id : 8] +Input [4]: [ws_bill_customer_sk#49, ws_ext_discount_amt#50, ws_ext_list_price#51, ws_sold_date_sk#52] + +(41) BroadcastExchange +Input [4]: [ws_bill_customer_sk#49, ws_ext_discount_amt#50, ws_ext_list_price#51, ws_sold_date_sk#52] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=6] + +(42) BroadcastHashJoin [codegen id : 10] +Left keys [1]: [c_customer_sk#41] +Right keys [1]: [ws_bill_customer_sk#49] +Join type: Inner +Join condition: None + +(43) Project [codegen id : 10] +Output [10]: [c_customer_id#42, c_first_name#43, c_last_name#44, c_preferred_cust_flag#45, c_birth_country#46, c_login#47, c_email_address#48, ws_ext_discount_amt#50, ws_ext_list_price#51, ws_sold_date_sk#52] +Input [12]: [c_customer_sk#41, c_customer_id#42, c_first_name#43, c_last_name#44, c_preferred_cust_flag#45, c_birth_country#46, c_login#47, c_email_address#48, ws_bill_customer_sk#49, ws_ext_discount_amt#50, ws_ext_list_price#51, ws_sold_date_sk#52] + +(44) ReusedExchange [Reuses operator id: 76] +Output [2]: [d_date_sk#54, d_year#55] + +(45) BroadcastHashJoin [codegen id : 10] +Left keys [1]: [ws_sold_date_sk#52] +Right keys [1]: [d_date_sk#54] +Join type: Inner +Join condition: None + +(46) Project [codegen id : 10] +Output [10]: [c_customer_id#42, c_first_name#43, c_last_name#44, c_preferred_cust_flag#45, c_birth_country#46, c_login#47, c_email_address#48, ws_ext_discount_amt#50, ws_ext_list_price#51, d_year#55] +Input [12]: [c_customer_id#42, c_first_name#43, c_last_name#44, c_preferred_cust_flag#45, c_birth_country#46, c_login#47, c_email_address#48, ws_ext_discount_amt#50, ws_ext_list_price#51, ws_sold_date_sk#52, d_date_sk#54, d_year#55] + +(47) HashAggregate [codegen id : 10] +Input [10]: [c_customer_id#42, c_first_name#43, c_last_name#44, c_preferred_cust_flag#45, c_birth_country#46, c_login#47, c_email_address#48, ws_ext_discount_amt#50, ws_ext_list_price#51, d_year#55] +Keys [8]: [c_customer_id#42, c_first_name#43, c_last_name#44, c_preferred_cust_flag#45, c_birth_country#46, c_login#47, c_email_address#48, d_year#55] +Functions [1]: [partial_sum(UnscaledValue((ws_ext_list_price#51 - ws_ext_discount_amt#50)))] +Aggregate Attributes [1]: [sum#56] +Results [9]: [c_customer_id#42, c_first_name#43, c_last_name#44, c_preferred_cust_flag#45, c_birth_country#46, c_login#47, c_email_address#48, d_year#55, sum#57] + +(48) Exchange +Input [9]: [c_customer_id#42, c_first_name#43, c_last_name#44, c_preferred_cust_flag#45, c_birth_country#46, c_login#47, c_email_address#48, d_year#55, sum#57] +Arguments: hashpartitioning(c_customer_id#42, c_first_name#43, c_last_name#44, c_preferred_cust_flag#45, c_birth_country#46, c_login#47, c_email_address#48, d_year#55, 5), ENSURE_REQUIREMENTS, [plan_id=7] + +(49) HashAggregate [codegen id : 11] +Input [9]: [c_customer_id#42, c_first_name#43, c_last_name#44, c_preferred_cust_flag#45, c_birth_country#46, c_login#47, c_email_address#48, d_year#55, sum#57] +Keys [8]: [c_customer_id#42, c_first_name#43, c_last_name#44, c_preferred_cust_flag#45, c_birth_country#46, c_login#47, c_email_address#48, d_year#55] +Functions [1]: [sum(UnscaledValue((ws_ext_list_price#51 - ws_ext_discount_amt#50)))] +Aggregate Attributes [1]: [sum(UnscaledValue((ws_ext_list_price#51 - ws_ext_discount_amt#50)))#58] +Results [2]: [c_customer_id#42 AS customer_id#59, MakeDecimal(sum(UnscaledValue((ws_ext_list_price#51 - ws_ext_discount_amt#50)))#58,18,2) AS year_total#60] + +(50) Filter [codegen id : 11] +Input [2]: [customer_id#59, year_total#60] +Condition : (isnotnull(year_total#60) AND (year_total#60 > 0.00)) + +(51) BroadcastExchange +Input [2]: [customer_id#59, year_total#60] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=8] + +(52) BroadcastHashJoin [codegen id : 16] +Left keys [1]: [customer_id#19] +Right keys [1]: [customer_id#59] +Join type: Inner +Join condition: None + +(53) Project [codegen id : 16] +Output [5]: [customer_id#19, year_total#20, customer_preferred_cust_flag#39, year_total#40, year_total#60] +Input [6]: [customer_id#19, year_total#20, customer_preferred_cust_flag#39, year_total#40, customer_id#59, year_total#60] + +(54) Scan parquet spark_catalog.default.customer +Output [8]: [c_customer_sk#61, c_customer_id#62, c_first_name#63, c_last_name#64, c_preferred_cust_flag#65, c_birth_country#66, c_login#67, c_email_address#68] +Batched: true +Location [not included in comparison]/{warehouse_dir}/customer] +PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_customer_id)] +ReadSchema: struct<c_customer_sk:int,c_customer_id:string,c_first_name:string,c_last_name:string,c_preferred_cust_flag:string,c_birth_country:string,c_login:string,c_email_address:string> + +(55) CometFilter +Input [8]: [c_customer_sk#61, c_customer_id#62, c_first_name#63, c_last_name#64, c_preferred_cust_flag#65, c_birth_country#66, c_login#67, c_email_address#68] +Condition : (isnotnull(c_customer_sk#61) AND isnotnull(c_customer_id#62)) + +(56) ColumnarToRow [codegen id : 14] +Input [8]: [c_customer_sk#61, c_customer_id#62, c_first_name#63, c_last_name#64, c_preferred_cust_flag#65, c_birth_country#66, c_login#67, c_email_address#68] + +(57) Scan parquet spark_catalog.default.web_sales +Output [4]: [ws_bill_customer_sk#69, ws_ext_discount_amt#70, ws_ext_list_price#71, ws_sold_date_sk#72] +Batched: true +Location: InMemoryFileIndex [] +PartitionFilters: [isnotnull(ws_sold_date_sk#72), dynamicpruningexpression(ws_sold_date_sk#72 IN dynamicpruning#73)] +PushedFilters: [IsNotNull(ws_bill_customer_sk)] +ReadSchema: struct<ws_bill_customer_sk:int,ws_ext_discount_amt:decimal(7,2),ws_ext_list_price:decimal(7,2)> + +(58) CometFilter +Input [4]: [ws_bill_customer_sk#69, ws_ext_discount_amt#70, ws_ext_list_price#71, ws_sold_date_sk#72] +Condition : isnotnull(ws_bill_customer_sk#69) + +(59) ColumnarToRow [codegen id : 12] +Input [4]: [ws_bill_customer_sk#69, ws_ext_discount_amt#70, ws_ext_list_price#71, ws_sold_date_sk#72] + +(60) BroadcastExchange +Input [4]: [ws_bill_customer_sk#69, ws_ext_discount_amt#70, ws_ext_list_price#71, ws_sold_date_sk#72] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=9] + +(61) BroadcastHashJoin [codegen id : 14] +Left keys [1]: [c_customer_sk#61] +Right keys [1]: [ws_bill_customer_sk#69] +Join type: Inner +Join condition: None + +(62) Project [codegen id : 14] +Output [10]: [c_customer_id#62, c_first_name#63, c_last_name#64, c_preferred_cust_flag#65, c_birth_country#66, c_login#67, c_email_address#68, ws_ext_discount_amt#70, ws_ext_list_price#71, ws_sold_date_sk#72] +Input [12]: [c_customer_sk#61, c_customer_id#62, c_first_name#63, c_last_name#64, c_preferred_cust_flag#65, c_birth_country#66, c_login#67, c_email_address#68, ws_bill_customer_sk#69, ws_ext_discount_amt#70, ws_ext_list_price#71, ws_sold_date_sk#72] + +(63) ReusedExchange [Reuses operator id: 80] +Output [2]: [d_date_sk#74, d_year#75] + +(64) BroadcastHashJoin [codegen id : 14] +Left keys [1]: [ws_sold_date_sk#72] +Right keys [1]: [d_date_sk#74] +Join type: Inner +Join condition: None + +(65) Project [codegen id : 14] +Output [10]: [c_customer_id#62, c_first_name#63, c_last_name#64, c_preferred_cust_flag#65, c_birth_country#66, c_login#67, c_email_address#68, ws_ext_discount_amt#70, ws_ext_list_price#71, d_year#75] +Input [12]: [c_customer_id#62, c_first_name#63, c_last_name#64, c_preferred_cust_flag#65, c_birth_country#66, c_login#67, c_email_address#68, ws_ext_discount_amt#70, ws_ext_list_price#71, ws_sold_date_sk#72, d_date_sk#74, d_year#75] + +(66) HashAggregate [codegen id : 14] +Input [10]: [c_customer_id#62, c_first_name#63, c_last_name#64, c_preferred_cust_flag#65, c_birth_country#66, c_login#67, c_email_address#68, ws_ext_discount_amt#70, ws_ext_list_price#71, d_year#75] +Keys [8]: [c_customer_id#62, c_first_name#63, c_last_name#64, c_preferred_cust_flag#65, c_birth_country#66, c_login#67, c_email_address#68, d_year#75] +Functions [1]: [partial_sum(UnscaledValue((ws_ext_list_price#71 - ws_ext_discount_amt#70)))] +Aggregate Attributes [1]: [sum#76] +Results [9]: [c_customer_id#62, c_first_name#63, c_last_name#64, c_preferred_cust_flag#65, c_birth_country#66, c_login#67, c_email_address#68, d_year#75, sum#77] + +(67) Exchange +Input [9]: [c_customer_id#62, c_first_name#63, c_last_name#64, c_preferred_cust_flag#65, c_birth_country#66, c_login#67, c_email_address#68, d_year#75, sum#77] +Arguments: hashpartitioning(c_customer_id#62, c_first_name#63, c_last_name#64, c_preferred_cust_flag#65, c_birth_country#66, c_login#67, c_email_address#68, d_year#75, 5), ENSURE_REQUIREMENTS, [plan_id=10] + +(68) HashAggregate [codegen id : 15] +Input [9]: [c_customer_id#62, c_first_name#63, c_last_name#64, c_preferred_cust_flag#65, c_birth_country#66, c_login#67, c_email_address#68, d_year#75, sum#77] +Keys [8]: [c_customer_id#62, c_first_name#63, c_last_name#64, c_preferred_cust_flag#65, c_birth_country#66, c_login#67, c_email_address#68, d_year#75] +Functions [1]: [sum(UnscaledValue((ws_ext_list_price#71 - ws_ext_discount_amt#70)))] +Aggregate Attributes [1]: [sum(UnscaledValue((ws_ext_list_price#71 - ws_ext_discount_amt#70)))#58] +Results [2]: [c_customer_id#62 AS customer_id#78, MakeDecimal(sum(UnscaledValue((ws_ext_list_price#71 - ws_ext_discount_amt#70)))#58,18,2) AS year_total#79] + +(69) BroadcastExchange +Input [2]: [customer_id#78, year_total#79] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=11] + +(70) BroadcastHashJoin [codegen id : 16] +Left keys [1]: [customer_id#19] +Right keys [1]: [customer_id#78] +Join type: Inner +Join condition: (CASE WHEN (year_total#60 > 0.00) THEN (year_total#79 / year_total#60) END > CASE WHEN (year_total#20 > 0.00) THEN (year_total#40 / year_total#20) END) + +(71) Project [codegen id : 16] +Output [1]: [customer_preferred_cust_flag#39] +Input [7]: [customer_id#19, year_total#20, customer_preferred_cust_flag#39, year_total#40, year_total#60, customer_id#78, year_total#79] + +(72) TakeOrderedAndProject +Input [1]: [customer_preferred_cust_flag#39] +Arguments: 100, [customer_preferred_cust_flag#39 ASC NULLS FIRST], [customer_preferred_cust_flag#39] + +===== Subqueries ===== + +Subquery:1 Hosting operator id = 4 Hosting Expression = ss_sold_date_sk#12 IN dynamicpruning#13 +BroadcastExchange (76) ++- * ColumnarToRow (75) + +- CometFilter (74) + +- CometScan parquet spark_catalog.default.date_dim (73) + + +(73) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#14, d_year#15] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), IsNotNull(d_date_sk)] +ReadSchema: struct<d_date_sk:int,d_year:int> + +(74) CometFilter +Input [2]: [d_date_sk#14, d_year#15] +Condition : ((isnotnull(d_year#15) AND (d_year#15 = 2001)) AND isnotnull(d_date_sk#14)) + +(75) ColumnarToRow [codegen id : 1] +Input [2]: [d_date_sk#14, d_year#15] + +(76) BroadcastExchange +Input [2]: [d_date_sk#14, d_year#15] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=12] + +Subquery:2 Hosting operator id = 20 Hosting Expression = ss_sold_date_sk#32 IN dynamicpruning#33 +BroadcastExchange (80) ++- * ColumnarToRow (79) + +- CometFilter (78) + +- CometScan parquet spark_catalog.default.date_dim (77) + + +(77) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#34, d_year#35] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2002), IsNotNull(d_date_sk)] +ReadSchema: struct<d_date_sk:int,d_year:int> + +(78) CometFilter +Input [2]: [d_date_sk#34, d_year#35] +Condition : ((isnotnull(d_year#35) AND (d_year#35 = 2002)) AND isnotnull(d_date_sk#34)) + +(79) ColumnarToRow [codegen id : 1] +Input [2]: [d_date_sk#34, d_year#35] + +(80) BroadcastExchange +Input [2]: [d_date_sk#34, d_year#35] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=13] + +Subquery:3 Hosting operator id = 38 Hosting Expression = ws_sold_date_sk#52 IN dynamicpruning#13 + +Subquery:4 Hosting operator id = 57 Hosting Expression = ws_sold_date_sk#72 IN dynamicpruning#33 + + diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q11/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q11/simplified.txt new file mode 100644 index 000000000..562b5fdf2 --- /dev/null +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q11/simplified.txt @@ -0,0 +1,123 @@ +TakeOrderedAndProject [customer_preferred_cust_flag] + WholeStageCodegen (16) + Project [customer_preferred_cust_flag] + BroadcastHashJoin [customer_id,customer_id,year_total,year_total,year_total,year_total] + Project [customer_id,year_total,customer_preferred_cust_flag,year_total,year_total] + BroadcastHashJoin [customer_id,customer_id] + Project [customer_id,year_total,customer_preferred_cust_flag,year_total] + BroadcastHashJoin [customer_id,customer_id] + Filter [year_total] + HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,sum] [sum(UnscaledValue((ss_ext_list_price - ss_ext_discount_amt))),customer_id,year_total,sum] + InputAdapter + Exchange [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] #1 + WholeStageCodegen (3) + HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_list_price,ss_ext_discount_amt] [sum,sum] + Project [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_discount_amt,ss_ext_list_price,d_year] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] + BroadcastHashJoin [c_customer_sk,ss_customer_sk] + ColumnarToRow + InputAdapter + CometFilter [c_customer_sk,c_customer_id] + CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] + InputAdapter + BroadcastExchange #2 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometFilter [ss_customer_sk] + CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometFilter [d_year,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + InputAdapter + ReusedExchange [d_date_sk,d_year] #3 + InputAdapter + BroadcastExchange #4 + WholeStageCodegen (7) + HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,sum] [sum(UnscaledValue((ss_ext_list_price - ss_ext_discount_amt))),customer_id,customer_preferred_cust_flag,year_total,sum] + InputAdapter + Exchange [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] #5 + WholeStageCodegen (6) + HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_list_price,ss_ext_discount_amt] [sum,sum] + Project [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_discount_amt,ss_ext_list_price,d_year] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] + BroadcastHashJoin [c_customer_sk,ss_customer_sk] + ColumnarToRow + InputAdapter + CometFilter [c_customer_sk,c_customer_id] + CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] + InputAdapter + BroadcastExchange #6 + WholeStageCodegen (4) + ColumnarToRow + InputAdapter + CometFilter [ss_customer_sk] + CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #2 + BroadcastExchange #7 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometFilter [d_year,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + InputAdapter + ReusedExchange [d_date_sk,d_year] #7 + InputAdapter + BroadcastExchange #8 + WholeStageCodegen (11) + Filter [year_total] + HashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum] [sum(UnscaledValue((ws_ext_list_price - ws_ext_discount_amt))),customer_id,year_total,sum] + InputAdapter + Exchange [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year] #9 + WholeStageCodegen (10) + HashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,ws_ext_list_price,ws_ext_discount_amt] [sum,sum] + Project [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_list_price,d_year] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + Project [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] + BroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] + ColumnarToRow + InputAdapter + CometFilter [c_customer_sk,c_customer_id] + CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] + InputAdapter + BroadcastExchange #10 + WholeStageCodegen (8) + ColumnarToRow + InputAdapter + CometFilter [ws_bill_customer_sk] + CometScan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + InputAdapter + ReusedExchange [d_date_sk,d_year] #3 + InputAdapter + BroadcastExchange #11 + WholeStageCodegen (15) + HashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum] [sum(UnscaledValue((ws_ext_list_price - ws_ext_discount_amt))),customer_id,year_total,sum] + InputAdapter + Exchange [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year] #12 + WholeStageCodegen (14) + HashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,ws_ext_list_price,ws_ext_discount_amt] [sum,sum] + Project [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_list_price,d_year] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + Project [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] + BroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] + ColumnarToRow + InputAdapter + CometFilter [c_customer_sk,c_customer_id] + CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] + InputAdapter + BroadcastExchange #13 + WholeStageCodegen (12) + ColumnarToRow + InputAdapter + CometFilter [ws_bill_customer_sk] + CometScan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #2 + InputAdapter + ReusedExchange [d_date_sk,d_year] #7 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 new file mode 100644 index 000000000..6cf7f4b08 --- /dev/null +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q12/explain.txt @@ -0,0 +1,150 @@ +== Physical Plan == +TakeOrderedAndProject (20) ++- * Project (19) + +- Window (18) + +- * Sort (17) + +- Exchange (16) + +- * HashAggregate (15) + +- Exchange (14) + +- * HashAggregate (13) + +- * Project (12) + +- * BroadcastHashJoin Inner BuildRight (11) + :- * Project (9) + : +- * BroadcastHashJoin Inner BuildRight (8) + : :- * ColumnarToRow (3) + : : +- CometFilter (2) + : : +- CometScan parquet spark_catalog.default.web_sales (1) + : +- BroadcastExchange (7) + : +- * ColumnarToRow (6) + : +- CometFilter (5) + : +- CometScan parquet spark_catalog.default.item (4) + +- ReusedExchange (10) + + +(1) Scan parquet spark_catalog.default.web_sales +Output [3]: [ws_item_sk#1, ws_ext_sales_price#2, ws_sold_date_sk#3] +Batched: true +Location: InMemoryFileIndex [] +PartitionFilters: [isnotnull(ws_sold_date_sk#3), dynamicpruningexpression(ws_sold_date_sk#3 IN dynamicpruning#4)] +PushedFilters: [IsNotNull(ws_item_sk)] +ReadSchema: struct<ws_item_sk:int,ws_ext_sales_price:decimal(7,2)> + +(2) CometFilter +Input [3]: [ws_item_sk#1, ws_ext_sales_price#2, ws_sold_date_sk#3] +Condition : isnotnull(ws_item_sk#1) + +(3) ColumnarToRow [codegen id : 3] +Input [3]: [ws_item_sk#1, ws_ext_sales_price#2, ws_sold_date_sk#3] + +(4) Scan parquet spark_catalog.default.item +Output [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] +Batched: true +Location [not included in comparison]/{warehouse_dir}/item] +PushedFilters: [In(i_category, [Books ,Home ,Sports ]), IsNotNull(i_item_sk)] +ReadSchema: struct<i_item_sk:int,i_item_id:string,i_item_desc:string,i_current_price:decimal(7,2),i_class:string,i_category:string> + +(5) CometFilter +Input [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] +Condition : (i_category#10 IN (Sports ,Books ,Home ) AND isnotnull(i_item_sk#5)) + +(6) ColumnarToRow [codegen id : 1] +Input [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] + +(7) BroadcastExchange +Input [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=1] + +(8) BroadcastHashJoin [codegen id : 3] +Left keys [1]: [ws_item_sk#1] +Right keys [1]: [i_item_sk#5] +Join type: Inner +Join condition: None + +(9) Project [codegen id : 3] +Output [7]: [ws_ext_sales_price#2, ws_sold_date_sk#3, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] +Input [9]: [ws_item_sk#1, ws_ext_sales_price#2, ws_sold_date_sk#3, i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] + +(10) ReusedExchange [Reuses operator id: 25] +Output [1]: [d_date_sk#11] + +(11) BroadcastHashJoin [codegen id : 3] +Left keys [1]: [ws_sold_date_sk#3] +Right keys [1]: [d_date_sk#11] +Join type: Inner +Join condition: None + +(12) Project [codegen id : 3] +Output [6]: [ws_ext_sales_price#2, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] +Input [8]: [ws_ext_sales_price#2, ws_sold_date_sk#3, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10, d_date_sk#11] + +(13) HashAggregate [codegen id : 3] +Input [6]: [ws_ext_sales_price#2, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] +Keys [5]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8] +Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#2))] +Aggregate Attributes [1]: [sum#12] +Results [6]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, sum#13] + +(14) Exchange +Input [6]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, sum#13] +Arguments: hashpartitioning(i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, 5), ENSURE_REQUIREMENTS, [plan_id=2] + +(15) HashAggregate [codegen id : 4] +Input [6]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, sum#13] +Keys [5]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8] +Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#2))] +Aggregate Attributes [1]: [sum(UnscaledValue(ws_ext_sales_price#2))#14] +Results [7]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#2))#14,17,2) AS itemrevenue#15, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#2))#14,17,2) AS _w0#16, i_item_id#6] + +(16) Exchange +Input [7]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#15, _w0#16, i_item_id#6] +Arguments: hashpartitioning(i_class#9, 5), ENSURE_REQUIREMENTS, [plan_id=3] + +(17) Sort [codegen id : 5] +Input [7]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#15, _w0#16, i_item_id#6] +Arguments: [i_class#9 ASC NULLS FIRST], false, 0 + +(18) Window +Input [7]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#15, _w0#16, i_item_id#6] +Arguments: [sum(_w0#16) windowspecdefinition(i_class#9, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS _we0#17], [i_class#9] + +(19) Project [codegen id : 6] +Output [7]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#15, ((_w0#16 * 100) / _we0#17) AS revenueratio#18, i_item_id#6] +Input [8]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#15, _w0#16, i_item_id#6, _we0#17] + +(20) TakeOrderedAndProject +Input [7]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#15, revenueratio#18, i_item_id#6] +Arguments: 100, [i_category#10 ASC NULLS FIRST, i_class#9 ASC NULLS FIRST, i_item_id#6 ASC NULLS FIRST, i_item_desc#7 ASC NULLS FIRST, revenueratio#18 ASC NULLS FIRST], [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#15, revenueratio#18] + +===== Subqueries ===== + +Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#3 IN dynamicpruning#4 +BroadcastExchange (25) ++- * ColumnarToRow (24) + +- CometProject (23) + +- CometFilter (22) + +- CometScan parquet spark_catalog.default.date_dim (21) + + +(21) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#11, d_date#19] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,1999-02-22), LessThanOrEqual(d_date,1999-03-24), IsNotNull(d_date_sk)] +ReadSchema: struct<d_date_sk:int,d_date:date> + +(22) CometFilter +Input [2]: [d_date_sk#11, d_date#19] +Condition : (((isnotnull(d_date#19) AND (d_date#19 >= 1999-02-22)) AND (d_date#19 <= 1999-03-24)) AND isnotnull(d_date_sk#11)) + +(23) CometProject +Input [2]: [d_date_sk#11, d_date#19] +Arguments: [d_date_sk#11], [d_date_sk#11] + +(24) ColumnarToRow [codegen id : 1] +Input [1]: [d_date_sk#11] + +(25) BroadcastExchange +Input [1]: [d_date_sk#11] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] + + diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q12/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q12/simplified.txt new file mode 100644 index 000000000..fae1c6dba --- /dev/null +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q12/simplified.txt @@ -0,0 +1,40 @@ +TakeOrderedAndProject [i_category,i_class,i_item_id,i_item_desc,revenueratio,i_current_price,itemrevenue] + 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) + Sort [i_class] + InputAdapter + Exchange [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] + InputAdapter + Exchange [i_item_id,i_item_desc,i_category,i_class,i_current_price] #2 + WholeStageCodegen (3) + HashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,ws_ext_sales_price] [sum,sum] + Project [ws_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + Project [ws_ext_sales_price,ws_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + BroadcastHashJoin [ws_item_sk,i_item_sk] + ColumnarToRow + InputAdapter + CometFilter [ws_item_sk] + CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_ext_sales_price,ws_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] + InputAdapter + BroadcastExchange #4 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometFilter [i_category,i_item_sk] + CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + InputAdapter + ReusedExchange [d_date_sk] #3 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 new file mode 100644 index 000000000..a647b1f05 --- /dev/null +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q13/explain.txt @@ -0,0 +1,232 @@ +== Physical Plan == +* HashAggregate (34) ++- Exchange (33) + +- * HashAggregate (32) + +- * Project (31) + +- * BroadcastHashJoin Inner BuildRight (30) + :- * Project (25) + : +- * BroadcastHashJoin Inner BuildRight (24) + : :- * Project (19) + : : +- * BroadcastHashJoin Inner BuildRight (18) + : : :- * Project (16) + : : : +- * BroadcastHashJoin Inner BuildRight (15) + : : : :- * Project (9) + : : : : +- * BroadcastHashJoin Inner BuildRight (8) + : : : : :- * ColumnarToRow (3) + : : : : : +- CometFilter (2) + : : : : : +- CometScan parquet spark_catalog.default.store_sales (1) + : : : : +- BroadcastExchange (7) + : : : : +- * ColumnarToRow (6) + : : : : +- CometFilter (5) + : : : : +- CometScan parquet spark_catalog.default.store (4) + : : : +- BroadcastExchange (14) + : : : +- * ColumnarToRow (13) + : : : +- CometProject (12) + : : : +- CometFilter (11) + : : : +- CometScan parquet spark_catalog.default.customer_address (10) + : : +- ReusedExchange (17) + : +- BroadcastExchange (23) + : +- * ColumnarToRow (22) + : +- CometFilter (21) + : +- CometScan parquet spark_catalog.default.customer_demographics (20) + +- BroadcastExchange (29) + +- * ColumnarToRow (28) + +- CometFilter (27) + +- CometScan parquet spark_catalog.default.household_demographics (26) + + +(1) Scan parquet spark_catalog.default.store_sales +Output [10]: [ss_cdemo_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_store_sk#4, ss_quantity#5, ss_sales_price#6, ss_ext_sales_price#7, ss_ext_wholesale_cost#8, ss_net_profit#9, ss_sold_date_sk#10] +Batched: true +Location: InMemoryFileIndex [] +PartitionFilters: [isnotnull(ss_sold_date_sk#10), dynamicpruningexpression(ss_sold_date_sk#10 IN dynamicpruning#11)] +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<ss_cdemo_sk:int,ss_hdemo_sk:int,ss_addr_sk:int,ss_store_sk:int,ss_quantity:int,ss_sales_price:decimal(7,2),ss_ext_sales_price:decimal(7,2),ss_ext_wholesale_cost:decimal(7,2),ss_net_profit:decimal(7,2)> + +(2) CometFilter +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)))) + +(3) ColumnarToRow [codegen id : 6] +Input [10]: [ss_cdemo_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_store_sk#4, ss_quantity#5, ss_sales_price#6, ss_ext_sales_price#7, ss_ext_wholesale_cost#8, ss_net_profit#9, ss_sold_date_sk#10] + +(4) Scan parquet spark_catalog.default.store +Output [1]: [s_store_sk#12] +Batched: true +Location [not included in comparison]/{warehouse_dir}/store] +PushedFilters: [IsNotNull(s_store_sk)] +ReadSchema: struct<s_store_sk:int> + +(5) CometFilter +Input [1]: [s_store_sk#12] +Condition : isnotnull(s_store_sk#12) + +(6) ColumnarToRow [codegen id : 1] +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] +Left keys [1]: [ss_store_sk#4] +Right keys [1]: [s_store_sk#12] +Join type: Inner +Join condition: None + +(9) Project [codegen id : 6] +Output [9]: [ss_cdemo_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_quantity#5, ss_sales_price#6, ss_ext_sales_price#7, ss_ext_wholesale_cost#8, ss_net_profit#9, ss_sold_date_sk#10] +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] + +(10) Scan parquet spark_catalog.default.customer_address +Output [3]: [ca_address_sk#13, ca_state#14, ca_country#15] +Batched: true +Location [not included in comparison]/{warehouse_dir}/customer_address] +PushedFilters: [IsNotNull(ca_country), EqualTo(ca_country,United States), IsNotNull(ca_address_sk), Or(Or(In(ca_state, [OH,TX]),In(ca_state, [KY,NM,OR])),In(ca_state, [MS,TX,VA]))] +ReadSchema: struct<ca_address_sk:int,ca_state:string,ca_country:string> + +(11) CometFilter +Input [3]: [ca_address_sk#13, ca_state#14, ca_country#15] +Condition : (((isnotnull(ca_country#15) AND (ca_country#15 = United States)) AND isnotnull(ca_address_sk#13)) AND ((ca_state#14 IN (TX,OH) OR ca_state#14 IN (OR,NM,KY)) OR ca_state#14 IN (VA,TX,MS))) + +(12) CometProject +Input [3]: [ca_address_sk#13, ca_state#14, ca_country#15] +Arguments: [ca_address_sk#13, ca_state#14], [ca_address_sk#13, ca_state#14] + +(13) ColumnarToRow [codegen id : 2] +Input [2]: [ca_address_sk#13, ca_state#14] + +(14) BroadcastExchange +Input [2]: [ca_address_sk#13, ca_state#14] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] + +(15) BroadcastHashJoin [codegen id : 6] +Left keys [1]: [ss_addr_sk#3] +Right keys [1]: [ca_address_sk#13] +Join type: Inner +Join condition: ((((ca_state#14 IN (TX,OH) AND (ss_net_profit#9 >= 100.00)) AND (ss_net_profit#9 <= 200.00)) OR ((ca_state#14 IN (OR,NM,KY) AND (ss_net_profit#9 >= 150.00)) AND (ss_net_profit#9 <= 300.00))) OR ((ca_state#14 IN (VA,TX,MS) AND (ss_net_profit#9 >= 50.00)) AND (ss_net_profit#9 <= 250.00))) + +(16) Project [codegen id : 6] +Output [7]: [ss_cdemo_sk#1, ss_hdemo_sk#2, ss_quantity#5, ss_sales_price#6, ss_ext_sales_price#7, ss_ext_wholesale_cost#8, ss_sold_date_sk#10] +Input [11]: [ss_cdemo_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_quantity#5, ss_sales_price#6, ss_ext_sales_price#7, ss_ext_wholesale_cost#8, ss_net_profit#9, ss_sold_date_sk#10, ca_address_sk#13, ca_state#14] + +(17) ReusedExchange [Reuses operator id: 39] +Output [1]: [d_date_sk#16] + +(18) BroadcastHashJoin [codegen id : 6] +Left keys [1]: [ss_sold_date_sk#10] +Right keys [1]: [d_date_sk#16] +Join type: Inner +Join condition: None + +(19) Project [codegen id : 6] +Output [6]: [ss_cdemo_sk#1, ss_hdemo_sk#2, ss_quantity#5, ss_sales_price#6, ss_ext_sales_price#7, ss_ext_wholesale_cost#8] +Input [8]: [ss_cdemo_sk#1, ss_hdemo_sk#2, ss_quantity#5, ss_sales_price#6, ss_ext_sales_price#7, ss_ext_wholesale_cost#8, ss_sold_date_sk#10, d_date_sk#16] + +(20) Scan parquet spark_catalog.default.customer_demographics +Output [3]: [cd_demo_sk#17, cd_marital_status#18, cd_education_status#19] +Batched: true +Location [not included in comparison]/{warehouse_dir}/customer_demographics] +PushedFilters: [IsNotNull(cd_demo_sk), Or(Or(And(EqualTo(cd_marital_status,M),EqualTo(cd_education_status,Advanced Degree )),And(EqualTo(cd_marital_status,S),EqualTo(cd_education_status,College ))),And(EqualTo(cd_marital_status,W),EqualTo(cd_education_status,2 yr Degree )))] +ReadSchema: struct<cd_demo_sk:int,cd_marital_status:string,cd_education_status:string> + +(21) CometFilter +Input [3]: [cd_demo_sk#17, cd_marital_status#18, cd_education_status#19] +Condition : (isnotnull(cd_demo_sk#17) AND ((((cd_marital_status#18 = M) AND (cd_education_status#19 = Advanced Degree )) OR ((cd_marital_status#18 = S) AND (cd_education_status#19 = College ))) OR ((cd_marital_status#18 = W) AND (cd_education_status#19 = 2 yr Degree )))) + +(22) ColumnarToRow [codegen id : 4] +Input [3]: [cd_demo_sk#17, cd_marital_status#18, cd_education_status#19] + +(23) BroadcastExchange +Input [3]: [cd_demo_sk#17, cd_marital_status#18, cd_education_status#19] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=3] + +(24) BroadcastHashJoin [codegen id : 6] +Left keys [1]: [ss_cdemo_sk#1] +Right keys [1]: [cd_demo_sk#17] +Join type: Inner +Join condition: ((((((cd_marital_status#18 = M) AND (cd_education_status#19 = Advanced Degree )) AND (ss_sales_price#6 >= 100.00)) AND (ss_sales_price#6 <= 150.00)) OR ((((cd_marital_status#18 = S) AND (cd_education_status#19 = College )) AND (ss_sales_price#6 >= 50.00)) AND (ss_sales_price#6 <= 100.00))) OR ((((cd_marital_status#18 = W) AND (cd_education_status#19 = 2 yr Degree )) AND (ss_sales_price#6 >= 150.00)) AND (ss_sales_price#6 <= 200.00))) + +(25) Project [codegen id : 6] +Output [7]: [ss_hdemo_sk#2, ss_quantity#5, ss_sales_price#6, ss_ext_sales_price#7, ss_ext_wholesale_cost#8, cd_marital_status#18, cd_education_status#19] +Input [9]: [ss_cdemo_sk#1, ss_hdemo_sk#2, ss_quantity#5, ss_sales_price#6, ss_ext_sales_price#7, ss_ext_wholesale_cost#8, cd_demo_sk#17, cd_marital_status#18, cd_education_status#19] + +(26) Scan parquet spark_catalog.default.household_demographics +Output [2]: [hd_demo_sk#20, hd_dep_count#21] +Batched: true +Location [not included in comparison]/{warehouse_dir}/household_demographics] +PushedFilters: [IsNotNull(hd_demo_sk), Or(EqualTo(hd_dep_count,3),EqualTo(hd_dep_count,1))] +ReadSchema: struct<hd_demo_sk:int,hd_dep_count:int> + +(27) CometFilter +Input [2]: [hd_demo_sk#20, hd_dep_count#21] +Condition : (isnotnull(hd_demo_sk#20) AND ((hd_dep_count#21 = 3) OR (hd_dep_count#21 = 1))) + +(28) ColumnarToRow [codegen id : 5] +Input [2]: [hd_demo_sk#20, hd_dep_count#21] + +(29) BroadcastExchange +Input [2]: [hd_demo_sk#20, hd_dep_count#21] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=4] + +(30) BroadcastHashJoin [codegen id : 6] +Left keys [1]: [ss_hdemo_sk#2] +Right keys [1]: [hd_demo_sk#20] +Join type: Inner +Join condition: (((((((cd_marital_status#18 = M) AND (cd_education_status#19 = Advanced Degree )) AND (ss_sales_price#6 >= 100.00)) AND (ss_sales_price#6 <= 150.00)) AND (hd_dep_count#21 = 3)) OR (((((cd_marital_status#18 = S) AND (cd_education_status#19 = College )) AND (ss_sales_price#6 >= 50.00)) AND (ss_sales_price#6 <= 100.00)) AND (hd_dep_count#21 = 1))) OR (((((cd_marital_status#18 = W) AND (cd_education_status#19 = 2 yr Degree )) AND (ss_sales_price#6 >= 150.00)) AND (ss_sales_price#6 <= 200.00)) AND (hd_dep_count#21 = 1))) + +(31) Project [codegen id : 6] +Output [3]: [ss_quantity#5, ss_ext_sales_price#7, ss_ext_wholesale_cost#8] +Input [9]: [ss_hdemo_sk#2, ss_quantity#5, ss_sales_price#6, ss_ext_sales_price#7, ss_ext_wholesale_cost#8, cd_marital_status#18, cd_education_status#19, hd_demo_sk#20, hd_dep_count#21] + +(32) HashAggregate [codegen id : 6] +Input [3]: [ss_quantity#5, ss_ext_sales_price#7, ss_ext_wholesale_cost#8] +Keys: [] +Functions [4]: [partial_avg(ss_quantity#5), partial_avg(UnscaledValue(ss_ext_sales_price#7)), partial_avg(UnscaledValue(ss_ext_wholesale_cost#8)), partial_sum(UnscaledValue(ss_ext_wholesale_cost#8))] +Aggregate Attributes [7]: [sum#22, count#23, sum#24, count#25, sum#26, count#27, sum#28] +Results [7]: [sum#29, count#30, sum#31, count#32, sum#33, count#34, sum#35] + +(33) Exchange +Input [7]: [sum#29, count#30, sum#31, count#32, sum#33, count#34, sum#35] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=5] + +(34) HashAggregate [codegen id : 7] +Input [7]: [sum#29, count#30, sum#31, count#32, sum#33, count#34, sum#35] +Keys: [] +Functions [4]: [avg(ss_quantity#5), avg(UnscaledValue(ss_ext_sales_price#7)), avg(UnscaledValue(ss_ext_wholesale_cost#8)), sum(UnscaledValue(ss_ext_wholesale_cost#8))] +Aggregate Attributes [4]: [avg(ss_quantity#5)#36, avg(UnscaledValue(ss_ext_sales_price#7))#37, avg(UnscaledValue(ss_ext_wholesale_cost#8))#38, sum(UnscaledValue(ss_ext_wholesale_cost#8))#39] +Results [4]: [avg(ss_quantity#5)#36 AS avg(ss_quantity)#40, cast((avg(UnscaledValue(ss_ext_sales_price#7))#37 / 100.0) as decimal(11,6)) AS avg(ss_ext_sales_price)#41, cast((avg(UnscaledValue(ss_ext_wholesale_cost#8))#38 / 100.0) as decimal(11,6)) AS avg(ss_ext_wholesale_cost)#42, MakeDecimal(sum(UnscaledValue(ss_ext_wholesale_cost#8))#39,17,2) AS sum(ss_ext_wholesale_cost)#43] + +===== Subqueries ===== + +Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#10 IN dynamicpruning#11 +BroadcastExchange (39) ++- * ColumnarToRow (38) + +- CometProject (37) + +- CometFilter (36) + +- CometScan parquet spark_catalog.default.date_dim (35) + + +(35) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#16, d_year#44] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), IsNotNull(d_date_sk)] +ReadSchema: struct<d_date_sk:int,d_year:int> + +(36) CometFilter +Input [2]: [d_date_sk#16, d_year#44] +Condition : ((isnotnull(d_year#44) AND (d_year#44 = 2001)) AND isnotnull(d_date_sk#16)) + +(37) CometProject +Input [2]: [d_date_sk#16, d_year#44] +Arguments: [d_date_sk#16], [d_date_sk#16] + +(38) ColumnarToRow [codegen id : 1] +Input [1]: [d_date_sk#16] + +(39) BroadcastExchange +Input [1]: [d_date_sk#16] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=6] + + diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q13/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q13/simplified.txt new file mode 100644 index 000000000..5e5fc41f8 --- /dev/null +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q13/simplified.txt @@ -0,0 +1,59 @@ +WholeStageCodegen (7) + HashAggregate [sum,count,sum,count,sum,count,sum] [avg(ss_quantity),avg(UnscaledValue(ss_ext_sales_price)),avg(UnscaledValue(ss_ext_wholesale_cost)),sum(UnscaledValue(ss_ext_wholesale_cost)),avg(ss_quantity),avg(ss_ext_sales_price),avg(ss_ext_wholesale_cost),sum(ss_ext_wholesale_cost),sum,count,sum,count,sum,count,sum] + InputAdapter + Exchange #1 + WholeStageCodegen (6) + HashAggregate [ss_quantity,ss_ext_sales_price,ss_ext_wholesale_cost] [sum,count,sum,count,sum,count,sum,sum,count,sum,count,sum,count,sum] + Project [ss_quantity,ss_ext_sales_price,ss_ext_wholesale_cost] + BroadcastHashJoin [ss_hdemo_sk,hd_demo_sk,cd_marital_status,cd_education_status,ss_sales_price,hd_dep_count] + Project [ss_hdemo_sk,ss_quantity,ss_sales_price,ss_ext_sales_price,ss_ext_wholesale_cost,cd_marital_status,cd_education_status] + BroadcastHashJoin [ss_cdemo_sk,cd_demo_sk,cd_marital_status,cd_education_status,ss_sales_price] + Project [ss_cdemo_sk,ss_hdemo_sk,ss_quantity,ss_sales_price,ss_ext_sales_price,ss_ext_wholesale_cost] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [ss_cdemo_sk,ss_hdemo_sk,ss_quantity,ss_sales_price,ss_ext_sales_price,ss_ext_wholesale_cost,ss_sold_date_sk] + BroadcastHashJoin [ss_addr_sk,ca_address_sk,ca_state,ss_net_profit] + Project [ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_quantity,ss_sales_price,ss_ext_sales_price,ss_ext_wholesale_cost,ss_net_profit,ss_sold_date_sk] + BroadcastHashJoin [ss_store_sk,s_store_sk] + ColumnarToRow + InputAdapter + CometFilter [ss_store_sk,ss_addr_sk,ss_cdemo_sk,ss_hdemo_sk,ss_net_profit,ss_sales_price] + CometScan parquet spark_catalog.default.store_sales [ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_quantity,ss_sales_price,ss_ext_sales_price,ss_ext_wholesale_cost,ss_net_profit,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #2 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_year,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + InputAdapter + BroadcastExchange #3 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometFilter [s_store_sk] + CometScan parquet spark_catalog.default.store [s_store_sk] + InputAdapter + BroadcastExchange #4 + WholeStageCodegen (2) + ColumnarToRow + InputAdapter + CometProject [ca_address_sk,ca_state] + CometFilter [ca_country,ca_address_sk,ca_state] + CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state,ca_country] + InputAdapter + ReusedExchange [d_date_sk] #2 + InputAdapter + BroadcastExchange #5 + WholeStageCodegen (4) + ColumnarToRow + InputAdapter + CometFilter [cd_demo_sk,cd_marital_status,cd_education_status] + CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_marital_status,cd_education_status] + InputAdapter + BroadcastExchange #6 + WholeStageCodegen (5) + ColumnarToRow + InputAdapter + CometFilter [hd_demo_sk,hd_dep_count] + CometScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_dep_count] 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 new file mode 100644 index 000000000..565cb97da --- /dev/null +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q14a/explain.txt @@ -0,0 +1,800 @@ +== Physical Plan == +TakeOrderedAndProject (105) ++- * HashAggregate (104) + +- Exchange (103) + +- * HashAggregate (102) + +- * Expand (101) + +- Union (100) + :- * Project (67) + : +- * Filter (66) + : +- * HashAggregate (65) + : +- Exchange (64) + : +- * HashAggregate (63) + : +- * Project (62) + : +- * BroadcastHashJoin Inner BuildRight (61) + : :- * Project (59) + : : +- * BroadcastHashJoin Inner BuildRight (58) + : : :- * BroadcastHashJoin LeftSemi BuildRight (51) + : : : :- * ColumnarToRow (3) + : : : : +- CometFilter (2) + : : : : +- CometScan parquet spark_catalog.default.store_sales (1) + : : : +- BroadcastExchange (50) + : : : +- * Project (49) + : : : +- * BroadcastHashJoin Inner BuildRight (48) + : : : :- * ColumnarToRow (6) + : : : : +- CometFilter (5) + : : : : +- CometScan parquet spark_catalog.default.item (4) + : : : +- BroadcastExchange (47) + : : : +- * BroadcastHashJoin LeftSemi BuildRight (46) + : : : :- * HashAggregate (35) + : : : : +- Exchange (34) + : : : : +- * HashAggregate (33) + : : : : +- * Project (32) + : : : : +- * BroadcastHashJoin Inner BuildRight (31) + : : : : :- * Project (29) + : : : : : +- * BroadcastHashJoin Inner BuildRight (28) + : : : : : :- * ColumnarToRow (9) + : : : : : : +- CometFilter (8) + : : : : : : +- CometScan parquet spark_catalog.default.store_sales (7) + : : : : : +- BroadcastExchange (27) + : : : : : +- * BroadcastHashJoin LeftSemi BuildRight (26) + : : : : : :- * ColumnarToRow (12) + : : : : : : +- CometFilter (11) + : : : : : : +- CometScan parquet spark_catalog.default.item (10) + : : : : : +- BroadcastExchange (25) + : : : : : +- * Project (24) + : : : : : +- * BroadcastHashJoin Inner BuildRight (23) + : : : : : :- * Project (21) + : : : : : : +- * BroadcastHashJoin Inner BuildRight (20) + : : : : : : :- * ColumnarToRow (15) + : : : : : : : +- CometFilter (14) + : : : : : : : +- CometScan parquet spark_catalog.default.catalog_sales (13) + : : : : : : +- BroadcastExchange (19) + : : : : : : +- * ColumnarToRow (18) + : : : : : : +- CometFilter (17) + : : : : : : +- CometScan parquet spark_catalog.default.item (16) + : : : : : +- ReusedExchange (22) + : : : : +- ReusedExchange (30) + : : : +- BroadcastExchange (45) + : : : +- * Project (44) + : : : +- * BroadcastHashJoin Inner BuildRight (43) + : : : :- * Project (41) + : : : : +- * BroadcastHashJoin Inner BuildRight (40) + : : : : :- * ColumnarToRow (38) + : : : : : +- CometFilter (37) + : : : : : +- CometScan parquet spark_catalog.default.web_sales (36) + : : : : +- ReusedExchange (39) + : : : +- ReusedExchange (42) + : : +- BroadcastExchange (57) + : : +- * BroadcastHashJoin LeftSemi BuildRight (56) + : : :- * ColumnarToRow (54) + : : : +- CometFilter (53) + : : : +- CometScan parquet spark_catalog.default.item (52) + : : +- ReusedExchange (55) + : +- ReusedExchange (60) + :- * Project (83) + : +- * Filter (82) + : +- * HashAggregate (81) + : +- Exchange (80) + : +- * HashAggregate (79) + : +- * Project (78) + : +- * BroadcastHashJoin Inner BuildRight (77) + : :- * Project (75) + : : +- * BroadcastHashJoin Inner BuildRight (74) + : : :- * BroadcastHashJoin LeftSemi BuildRight (72) + : : : :- * ColumnarToRow (70) + : : : : +- CometFilter (69) + : : : : +- CometScan parquet spark_catalog.default.catalog_sales (68) + : : : +- ReusedExchange (71) + : : +- ReusedExchange (73) + : +- ReusedExchange (76) + +- * Project (99) + +- * Filter (98) + +- * HashAggregate (97) + +- Exchange (96) + +- * HashAggregate (95) + +- * Project (94) + +- * BroadcastHashJoin Inner BuildRight (93) + :- * Project (91) + : +- * BroadcastHashJoin Inner BuildRight (90) + : :- * BroadcastHashJoin LeftSemi BuildRight (88) + : : :- * ColumnarToRow (86) + : : : +- CometFilter (85) + : : : +- CometScan parquet spark_catalog.default.web_sales (84) + : : +- ReusedExchange (87) + : +- ReusedExchange (89) + +- ReusedExchange (92) + + +(1) Scan parquet spark_catalog.default.store_sales +Output [4]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] +Batched: true +Location: InMemoryFileIndex [] +PartitionFilters: [isnotnull(ss_sold_date_sk#4), dynamicpruningexpression(ss_sold_date_sk#4 IN dynamicpruning#5)] +PushedFilters: [IsNotNull(ss_item_sk)] +ReadSchema: struct<ss_item_sk:int,ss_quantity:int,ss_list_price:decimal(7,2)> + +(2) CometFilter +Input [4]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] +Condition : isnotnull(ss_item_sk#1) + +(3) ColumnarToRow [codegen id : 25] +Input [4]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] + +(4) Scan parquet spark_catalog.default.item +Output [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] +Batched: true +Location [not included in comparison]/{warehouse_dir}/item] +PushedFilters: [IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id)] +ReadSchema: struct<i_item_sk:int,i_brand_id:int,i_class_id:int,i_category_id:int> + +(5) CometFilter +Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] +Condition : ((isnotnull(i_brand_id#7) AND isnotnull(i_class_id#8)) AND isnotnull(i_category_id#9)) + +(6) ColumnarToRow [codegen id : 11] +Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] + +(7) Scan parquet spark_catalog.default.store_sales +Output [2]: [ss_item_sk#10, ss_sold_date_sk#11] +Batched: true +Location: InMemoryFileIndex [] +PartitionFilters: [isnotnull(ss_sold_date_sk#11), dynamicpruningexpression(ss_sold_date_sk#11 IN dynamicpruning#12)] +PushedFilters: [IsNotNull(ss_item_sk)] +ReadSchema: struct<ss_item_sk:int> + +(8) CometFilter +Input [2]: [ss_item_sk#10, ss_sold_date_sk#11] +Condition : isnotnull(ss_item_sk#10) + +(9) ColumnarToRow [codegen id : 6] +Input [2]: [ss_item_sk#10, ss_sold_date_sk#11] + +(10) Scan parquet spark_catalog.default.item +Output [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] +Batched: true +Location [not included in comparison]/{warehouse_dir}/item] +PushedFilters: [IsNotNull(i_item_sk), IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id)] +ReadSchema: struct<i_item_sk:int,i_brand_id:int,i_class_id:int,i_category_id:int> + +(11) CometFilter +Input [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] +Condition : (((isnotnull(i_item_sk#13) AND isnotnull(i_brand_id#14)) AND isnotnull(i_class_id#15)) AND isnotnull(i_category_id#16)) + +(12) ColumnarToRow [codegen id : 4] +Input [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] + +(13) Scan parquet spark_catalog.default.catalog_sales +Output [2]: [cs_item_sk#17, cs_sold_date_sk#18] +Batched: true +Location: InMemoryFileIndex [] +PartitionFilters: [isnotnull(cs_sold_date_sk#18), dynamicpruningexpression(cs_sold_date_sk#18 IN dynamicpruning#19)] +PushedFilters: [IsNotNull(cs_item_sk)] +ReadSchema: struct<cs_item_sk:int> + +(14) CometFilter +Input [2]: [cs_item_sk#17, cs_sold_date_sk#18] +Condition : isnotnull(cs_item_sk#17) + +(15) ColumnarToRow [codegen id : 3] +Input [2]: [cs_item_sk#17, cs_sold_date_sk#18] + +(16) Scan parquet spark_catalog.default.item +Output [4]: [i_item_sk#20, i_brand_id#21, i_class_id#22, i_category_id#23] +Batched: true +Location [not included in comparison]/{warehouse_dir}/item] +PushedFilters: [IsNotNull(i_item_sk)] +ReadSchema: struct<i_item_sk:int,i_brand_id:int,i_class_id:int,i_category_id:int> + +(17) CometFilter +Input [4]: [i_item_sk#20, i_brand_id#21, i_class_id#22, i_category_id#23] +Condition : isnotnull(i_item_sk#20) + +(18) ColumnarToRow [codegen id : 1] +Input [4]: [i_item_sk#20, i_brand_id#21, i_class_id#22, i_category_id#23] + +(19) BroadcastExchange +Input [4]: [i_item_sk#20, i_brand_id#21, i_class_id#22, i_category_id#23] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=1] + +(20) BroadcastHashJoin [codegen id : 3] +Left keys [1]: [cs_item_sk#17] +Right keys [1]: [i_item_sk#20] +Join type: Inner +Join condition: None + +(21) Project [codegen id : 3] +Output [4]: [cs_sold_date_sk#18, i_brand_id#21, i_class_id#22, i_category_id#23] +Input [6]: [cs_item_sk#17, cs_sold_date_sk#18, i_item_sk#20, i_brand_id#21, i_class_id#22, i_category_id#23] + +(22) ReusedExchange [Reuses operator id: 134] +Output [1]: [d_date_sk#24] + +(23) BroadcastHashJoin [codegen id : 3] +Left keys [1]: [cs_sold_date_sk#18] +Right keys [1]: [d_date_sk#24] +Join type: Inner +Join condition: None + +(24) Project [codegen id : 3] +Output [3]: [i_brand_id#21, i_class_id#22, i_category_id#23] +Input [5]: [cs_sold_date_sk#18, i_brand_id#21, i_class_id#22, i_category_id#23, d_date_sk#24] + +(25) BroadcastExchange +Input [3]: [i_brand_id#21, i_class_id#22, i_category_id#23] +Arguments: HashedRelationBroadcastMode(List(coalesce(input[0, int, true], 0), isnull(input[0, int, true]), coalesce(input[1, int, true], 0), isnull(input[1, int, true]), coalesce(input[2, int, true], 0), isnull(input[2, int, true])),false), [plan_id=2] + +(26) BroadcastHashJoin [codegen id : 4] +Left keys [6]: [coalesce(i_brand_id#14, 0), isnull(i_brand_id#14), coalesce(i_class_id#15, 0), isnull(i_class_id#15), coalesce(i_category_id#16, 0), isnull(i_category_id#16)] +Right keys [6]: [coalesce(i_brand_id#21, 0), isnull(i_brand_id#21), coalesce(i_class_id#22, 0), isnull(i_class_id#22), coalesce(i_category_id#23, 0), isnull(i_category_id#23)] +Join type: LeftSemi +Join condition: None + +(27) BroadcastExchange +Input [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=3] + +(28) BroadcastHashJoin [codegen id : 6] +Left keys [1]: [ss_item_sk#10] +Right keys [1]: [i_item_sk#13] +Join type: Inner +Join condition: None + +(29) Project [codegen id : 6] +Output [4]: [ss_sold_date_sk#11, i_brand_id#14, i_class_id#15, i_category_id#16] +Input [6]: [ss_item_sk#10, ss_sold_date_sk#11, i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] + +(30) ReusedExchange [Reuses operator id: 134] +Output [1]: [d_date_sk#25] + +(31) BroadcastHashJoin [codegen id : 6] +Left keys [1]: [ss_sold_date_sk#11] +Right keys [1]: [d_date_sk#25] +Join type: Inner +Join condition: None + +(32) Project [codegen id : 6] +Output [3]: [i_brand_id#14 AS brand_id#26, i_class_id#15 AS class_id#27, i_category_id#16 AS category_id#28] +Input [5]: [ss_sold_date_sk#11, i_brand_id#14, i_class_id#15, i_category_id#16, d_date_sk#25] + +(33) HashAggregate [codegen id : 6] +Input [3]: [brand_id#26, class_id#27, category_id#28] +Keys [3]: [brand_id#26, class_id#27, category_id#28] +Functions: [] +Aggregate Attributes: [] +Results [3]: [brand_id#26, class_id#27, category_id#28] + +(34) Exchange +Input [3]: [brand_id#26, class_id#27, category_id#28] +Arguments: hashpartitioning(brand_id#26, class_id#27, category_id#28, 5), ENSURE_REQUIREMENTS, [plan_id=4] + +(35) HashAggregate [codegen id : 10] +Input [3]: [brand_id#26, class_id#27, category_id#28] +Keys [3]: [brand_id#26, class_id#27, category_id#28] +Functions: [] +Aggregate Attributes: [] +Results [3]: [brand_id#26, class_id#27, category_id#28] + +(36) Scan parquet spark_catalog.default.web_sales +Output [2]: [ws_item_sk#29, ws_sold_date_sk#30] +Batched: true +Location: InMemoryFileIndex [] +PartitionFilters: [isnotnull(ws_sold_date_sk#30), dynamicpruningexpression(ws_sold_date_sk#30 IN dynamicpruning#31)] +PushedFilters: [IsNotNull(ws_item_sk)] +ReadSchema: struct<ws_item_sk:int> + +(37) CometFilter +Input [2]: [ws_item_sk#29, ws_sold_date_sk#30] +Condition : isnotnull(ws_item_sk#29) + +(38) ColumnarToRow [codegen id : 9] +Input [2]: [ws_item_sk#29, ws_sold_date_sk#30] + +(39) ReusedExchange [Reuses operator id: 19] +Output [4]: [i_item_sk#32, i_brand_id#33, i_class_id#34, i_category_id#35] + +(40) BroadcastHashJoin [codegen id : 9] +Left keys [1]: [ws_item_sk#29] +Right keys [1]: [i_item_sk#32] +Join type: Inner +Join condition: None + +(41) Project [codegen id : 9] +Output [4]: [ws_sold_date_sk#30, i_brand_id#33, i_class_id#34, i_category_id#35] +Input [6]: [ws_item_sk#29, ws_sold_date_sk#30, i_item_sk#32, i_brand_id#33, i_class_id#34, i_category_id#35] + +(42) ReusedExchange [Reuses operator id: 134] +Output [1]: [d_date_sk#36] + +(43) BroadcastHashJoin [codegen id : 9] +Left keys [1]: [ws_sold_date_sk#30] +Right keys [1]: [d_date_sk#36] +Join type: Inner +Join condition: None + +(44) Project [codegen id : 9] +Output [3]: [i_brand_id#33, i_class_id#34, i_category_id#35] +Input [5]: [ws_sold_date_sk#30, i_brand_id#33, i_class_id#34, i_category_id#35, d_date_sk#36] + +(45) BroadcastExchange +Input [3]: [i_brand_id#33, i_class_id#34, i_category_id#35] +Arguments: HashedRelationBroadcastMode(List(coalesce(input[0, int, true], 0), isnull(input[0, int, true]), coalesce(input[1, int, true], 0), isnull(input[1, int, true]), coalesce(input[2, int, true], 0), isnull(input[2, int, true])),false), [plan_id=5] + +(46) BroadcastHashJoin [codegen id : 10] +Left keys [6]: [coalesce(brand_id#26, 0), isnull(brand_id#26), coalesce(class_id#27, 0), isnull(class_id#27), coalesce(category_id#28, 0), isnull(category_id#28)] +Right keys [6]: [coalesce(i_brand_id#33, 0), isnull(i_brand_id#33), coalesce(i_class_id#34, 0), isnull(i_class_id#34), coalesce(i_category_id#35, 0), isnull(i_category_id#35)] +Join type: LeftSemi +Join condition: None + +(47) BroadcastExchange +Input [3]: [brand_id#26, class_id#27, category_id#28] +Arguments: HashedRelationBroadcastMode(List(input[0, int, true], input[1, int, true], input[2, int, true]),false), [plan_id=6] + +(48) BroadcastHashJoin [codegen id : 11] +Left keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] +Right keys [3]: [brand_id#26, class_id#27, category_id#28] +Join type: Inner +Join condition: None + +(49) Project [codegen id : 11] +Output [1]: [i_item_sk#6 AS ss_item_sk#37] +Input [7]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9, brand_id#26, class_id#27, category_id#28] + +(50) BroadcastExchange +Input [1]: [ss_item_sk#37] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=7] + +(51) BroadcastHashJoin [codegen id : 25] +Left keys [1]: [ss_item_sk#1] +Right keys [1]: [ss_item_sk#37] +Join type: LeftSemi +Join condition: None + +(52) Scan parquet spark_catalog.default.item +Output [4]: [i_item_sk#38, i_brand_id#39, i_class_id#40, i_category_id#41] +Batched: true +Location [not included in comparison]/{warehouse_dir}/item] +PushedFilters: [IsNotNull(i_item_sk)] +ReadSchema: struct<i_item_sk:int,i_brand_id:int,i_class_id:int,i_category_id:int> + +(53) CometFilter +Input [4]: [i_item_sk#38, i_brand_id#39, i_class_id#40, i_category_id#41] +Condition : isnotnull(i_item_sk#38) + +(54) ColumnarToRow [codegen id : 23] +Input [4]: [i_item_sk#38, i_brand_id#39, i_class_id#40, i_category_id#41] + +(55) ReusedExchange [Reuses operator id: 50] +Output [1]: [ss_item_sk#37] + +(56) BroadcastHashJoin [codegen id : 23] +Left keys [1]: [i_item_sk#38] +Right keys [1]: [ss_item_sk#37] +Join type: LeftSemi +Join condition: None + +(57) BroadcastExchange +Input [4]: [i_item_sk#38, i_brand_id#39, i_class_id#40, i_category_id#41] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=8] + +(58) BroadcastHashJoin [codegen id : 25] +Left keys [1]: [ss_item_sk#1] +Right keys [1]: [i_item_sk#38] +Join type: Inner +Join condition: None + +(59) Project [codegen id : 25] +Output [6]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#39, i_class_id#40, i_category_id#41] +Input [8]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_item_sk#38, i_brand_id#39, i_class_id#40, i_category_id#41] + +(60) ReusedExchange [Reuses operator id: 129] +Output [1]: [d_date_sk#42] + +(61) BroadcastHashJoin [codegen id : 25] +Left keys [1]: [ss_sold_date_sk#4] +Right keys [1]: [d_date_sk#42] +Join type: Inner +Join condition: None + +(62) Project [codegen id : 25] +Output [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#39, i_class_id#40, i_category_id#41] +Input [7]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#39, i_class_id#40, i_category_id#41, d_date_sk#42] + +(63) HashAggregate [codegen id : 25] +Input [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#39, i_class_id#40, i_category_id#41] +Keys [3]: [i_brand_id#39, i_class_id#40, i_category_id#41] +Functions [2]: [partial_sum((cast(ss_quantity#2 as decimal(10,0)) * ss_list_price#3)), partial_count(1)] +Aggregate Attributes [3]: [sum#43, isEmpty#44, count#45] +Results [6]: [i_brand_id#39, i_class_id#40, i_category_id#41, sum#46, isEmpty#47, count#48] + +(64) Exchange +Input [6]: [i_brand_id#39, i_class_id#40, i_category_id#41, sum#46, isEmpty#47, count#48] +Arguments: hashpartitioning(i_brand_id#39, i_class_id#40, i_category_id#41, 5), ENSURE_REQUIREMENTS, [plan_id=9] + +(65) HashAggregate [codegen id : 26] +Input [6]: [i_brand_id#39, i_class_id#40, i_category_id#41, sum#46, isEmpty#47, count#48] +Keys [3]: [i_brand_id#39, i_class_id#40, i_category_id#41] +Functions [2]: [sum((cast(ss_quantity#2 as decimal(10,0)) * ss_list_price#3)), count(1)] +Aggregate Attributes [2]: [sum((cast(ss_quantity#2 as decimal(10,0)) * ss_list_price#3))#49, count(1)#50] +Results [5]: [i_brand_id#39, i_class_id#40, i_category_id#41, sum((cast(ss_quantity#2 as decimal(10,0)) * ss_list_price#3))#49 AS sales#51, count(1)#50 AS number_sales#52] + +(66) Filter [codegen id : 26] +Input [5]: [i_brand_id#39, i_class_id#40, i_category_id#41, sales#51, number_sales#52] +Condition : (isnotnull(sales#51) AND (cast(sales#51 as decimal(32,6)) > cast(Subquery scalar-subquery#53, [id=#54] as decimal(32,6)))) + +(67) Project [codegen id : 26] +Output [6]: [sales#51, number_sales#52, store AS channel#55, i_brand_id#39 AS i_brand_id#56, i_class_id#40 AS i_class_id#57, i_category_id#41 AS i_category_id#58] +Input [5]: [i_brand_id#39, i_class_id#40, i_category_id#41, sales#51, number_sales#52] + +(68) Scan parquet spark_catalog.default.catalog_sales +Output [4]: [cs_item_sk#59, cs_quantity#60, cs_list_price#61, cs_sold_date_sk#62] +Batched: true +Location: InMemoryFileIndex [] +PartitionFilters: [isnotnull(cs_sold_date_sk#62), dynamicpruningexpression(cs_sold_date_sk#62 IN dynamicpruning#63)] +PushedFilters: [IsNotNull(cs_item_sk)] +ReadSchema: struct<cs_item_sk:int,cs_quantity:int,cs_list_price:decimal(7,2)> + +(69) CometFilter +Input [4]: [cs_item_sk#59, cs_quantity#60, cs_list_price#61, cs_sold_date_sk#62] +Condition : isnotnull(cs_item_sk#59) + +(70) ColumnarToRow [codegen id : 51] +Input [4]: [cs_item_sk#59, cs_quantity#60, cs_list_price#61, cs_sold_date_sk#62] + +(71) ReusedExchange [Reuses operator id: 50] +Output [1]: [ss_item_sk#64] + +(72) BroadcastHashJoin [codegen id : 51] +Left keys [1]: [cs_item_sk#59] +Right keys [1]: [ss_item_sk#64] +Join type: LeftSemi +Join condition: None + +(73) ReusedExchange [Reuses operator id: 57] +Output [4]: [i_item_sk#65, i_brand_id#66, i_class_id#67, i_category_id#68] + +(74) BroadcastHashJoin [codegen id : 51] +Left keys [1]: [cs_item_sk#59] +Right keys [1]: [i_item_sk#65] +Join type: Inner +Join condition: None + +(75) Project [codegen id : 51] +Output [6]: [cs_quantity#60, cs_list_price#61, cs_sold_date_sk#62, i_brand_id#66, i_class_id#67, i_category_id#68] +Input [8]: [cs_item_sk#59, cs_quantity#60, cs_list_price#61, cs_sold_date_sk#62, i_item_sk#65, i_brand_id#66, i_class_id#67, i_category_id#68] + +(76) ReusedExchange [Reuses operator id: 129] +Output [1]: [d_date_sk#69] + +(77) BroadcastHashJoin [codegen id : 51] +Left keys [1]: [cs_sold_date_sk#62] +Right keys [1]: [d_date_sk#69] +Join type: Inner +Join condition: None + +(78) Project [codegen id : 51] +Output [5]: [cs_quantity#60, cs_list_price#61, i_brand_id#66, i_class_id#67, i_category_id#68] +Input [7]: [cs_quantity#60, cs_list_price#61, cs_sold_date_sk#62, i_brand_id#66, i_class_id#67, i_category_id#68, d_date_sk#69] + +(79) HashAggregate [codegen id : 51] +Input [5]: [cs_quantity#60, cs_list_price#61, i_brand_id#66, i_class_id#67, i_category_id#68] +Keys [3]: [i_brand_id#66, i_class_id#67, i_category_id#68] +Functions [2]: [partial_sum((cast(cs_quantity#60 as decimal(10,0)) * cs_list_price#61)), partial_count(1)] +Aggregate Attributes [3]: [sum#70, isEmpty#71, count#72] +Results [6]: [i_brand_id#66, i_class_id#67, i_category_id#68, sum#73, isEmpty#74, count#75] + +(80) Exchange +Input [6]: [i_brand_id#66, i_class_id#67, i_category_id#68, sum#73, isEmpty#74, count#75] +Arguments: hashpartitioning(i_brand_id#66, i_class_id#67, i_category_id#68, 5), ENSURE_REQUIREMENTS, [plan_id=10] + +(81) HashAggregate [codegen id : 52] +Input [6]: [i_brand_id#66, i_class_id#67, i_category_id#68, sum#73, isEmpty#74, count#75] +Keys [3]: [i_brand_id#66, i_class_id#67, i_category_id#68] +Functions [2]: [sum((cast(cs_quantity#60 as decimal(10,0)) * cs_list_price#61)), count(1)] +Aggregate Attributes [2]: [sum((cast(cs_quantity#60 as decimal(10,0)) * cs_list_price#61))#76, count(1)#77] +Results [5]: [i_brand_id#66, i_class_id#67, i_category_id#68, sum((cast(cs_quantity#60 as decimal(10,0)) * cs_list_price#61))#76 AS sales#78, count(1)#77 AS number_sales#79] + +(82) Filter [codegen id : 52] +Input [5]: [i_brand_id#66, i_class_id#67, i_category_id#68, sales#78, number_sales#79] +Condition : (isnotnull(sales#78) AND (cast(sales#78 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#53, [id=#54] as decimal(32,6)))) + +(83) Project [codegen id : 52] +Output [6]: [sales#78, number_sales#79, catalog AS channel#80, i_brand_id#66, i_class_id#67, i_category_id#68] +Input [5]: [i_brand_id#66, i_class_id#67, i_category_id#68, sales#78, number_sales#79] + +(84) Scan parquet spark_catalog.default.web_sales +Output [4]: [ws_item_sk#81, ws_quantity#82, ws_list_price#83, ws_sold_date_sk#84] +Batched: true +Location: InMemoryFileIndex [] +PartitionFilters: [isnotnull(ws_sold_date_sk#84), dynamicpruningexpression(ws_sold_date_sk#84 IN dynamicpruning#85)] +PushedFilters: [IsNotNull(ws_item_sk)] +ReadSchema: struct<ws_item_sk:int,ws_quantity:int,ws_list_price:decimal(7,2)> + +(85) CometFilter +Input [4]: [ws_item_sk#81, ws_quantity#82, ws_list_price#83, ws_sold_date_sk#84] +Condition : isnotnull(ws_item_sk#81) + +(86) ColumnarToRow [codegen id : 77] +Input [4]: [ws_item_sk#81, ws_quantity#82, ws_list_price#83, ws_sold_date_sk#84] + +(87) ReusedExchange [Reuses operator id: 50] +Output [1]: [ss_item_sk#86] + +(88) BroadcastHashJoin [codegen id : 77] +Left keys [1]: [ws_item_sk#81] +Right keys [1]: [ss_item_sk#86] +Join type: LeftSemi +Join condition: None + +(89) ReusedExchange [Reuses operator id: 57] +Output [4]: [i_item_sk#87, i_brand_id#88, i_class_id#89, i_category_id#90] + +(90) BroadcastHashJoin [codegen id : 77] +Left keys [1]: [ws_item_sk#81] +Right keys [1]: [i_item_sk#87] +Join type: Inner +Join condition: None + +(91) Project [codegen id : 77] +Output [6]: [ws_quantity#82, ws_list_price#83, ws_sold_date_sk#84, i_brand_id#88, i_class_id#89, i_category_id#90] +Input [8]: [ws_item_sk#81, ws_quantity#82, ws_list_price#83, ws_sold_date_sk#84, i_item_sk#87, i_brand_id#88, i_class_id#89, i_category_id#90] + +(92) ReusedExchange [Reuses operator id: 129] +Output [1]: [d_date_sk#91] + +(93) BroadcastHashJoin [codegen id : 77] +Left keys [1]: [ws_sold_date_sk#84] +Right keys [1]: [d_date_sk#91] +Join type: Inner +Join condition: None + +(94) Project [codegen id : 77] +Output [5]: [ws_quantity#82, ws_list_price#83, i_brand_id#88, i_class_id#89, i_category_id#90] +Input [7]: [ws_quantity#82, ws_list_price#83, ws_sold_date_sk#84, i_brand_id#88, i_class_id#89, i_category_id#90, d_date_sk#91] + +(95) HashAggregate [codegen id : 77] +Input [5]: [ws_quantity#82, ws_list_price#83, i_brand_id#88, i_class_id#89, i_category_id#90] +Keys [3]: [i_brand_id#88, i_class_id#89, i_category_id#90] +Functions [2]: [partial_sum((cast(ws_quantity#82 as decimal(10,0)) * ws_list_price#83)), partial_count(1)] +Aggregate Attributes [3]: [sum#92, isEmpty#93, count#94] +Results [6]: [i_brand_id#88, i_class_id#89, i_category_id#90, sum#95, isEmpty#96, count#97] + +(96) Exchange +Input [6]: [i_brand_id#88, i_class_id#89, i_category_id#90, sum#95, isEmpty#96, count#97] +Arguments: hashpartitioning(i_brand_id#88, i_class_id#89, i_category_id#90, 5), ENSURE_REQUIREMENTS, [plan_id=11] + +(97) HashAggregate [codegen id : 78] +Input [6]: [i_brand_id#88, i_class_id#89, i_category_id#90, sum#95, isEmpty#96, count#97] +Keys [3]: [i_brand_id#88, i_class_id#89, i_category_id#90] +Functions [2]: [sum((cast(ws_quantity#82 as decimal(10,0)) * ws_list_price#83)), count(1)] +Aggregate Attributes [2]: [sum((cast(ws_quantity#82 as decimal(10,0)) * ws_list_price#83))#98, count(1)#99] +Results [5]: [i_brand_id#88, i_class_id#89, i_category_id#90, sum((cast(ws_quantity#82 as decimal(10,0)) * ws_list_price#83))#98 AS sales#100, count(1)#99 AS number_sales#101] + +(98) Filter [codegen id : 78] +Input [5]: [i_brand_id#88, i_class_id#89, i_category_id#90, sales#100, number_sales#101] +Condition : (isnotnull(sales#100) AND (cast(sales#100 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#53, [id=#54] as decimal(32,6)))) + +(99) Project [codegen id : 78] +Output [6]: [sales#100, number_sales#101, web AS channel#102, i_brand_id#88, i_class_id#89, i_category_id#90] +Input [5]: [i_brand_id#88, i_class_id#89, i_category_id#90, sales#100, number_sales#101] + +(100) Union + +(101) Expand [codegen id : 79] +Input [6]: [sales#51, number_sales#52, channel#55, i_brand_id#56, i_class_id#57, i_category_id#58] +Arguments: [[sales#51, number_sales#52, channel#55, i_brand_id#56, i_class_id#57, i_category_id#58, 0], [sales#51, number_sales#52, channel#55, i_brand_id#56, i_class_id#57, null, 1], [sales#51, number_sales#52, channel#55, i_brand_id#56, null, null, 3], [sales#51, number_sales#52, channel#55, null, null, null, 7], [sales#51, number_sales#52, null, null, null, null, 15]], [sales#51, number_sales#52, channel#103, i_brand_id#104, i_class_id#105, i_category_id#106, spark_grouping_id#107] + +(102) HashAggregate [codegen id : 79] +Input [7]: [sales#51, number_sales#52, channel#103, i_brand_id#104, i_class_id#105, i_category_id#106, spark_grouping_id#107] +Keys [5]: [channel#103, i_brand_id#104, i_class_id#105, i_category_id#106, spark_grouping_id#107] +Functions [2]: [partial_sum(sales#51), partial_sum(number_sales#52)] +Aggregate Attributes [3]: [sum#108, isEmpty#109, sum#110] +Results [8]: [channel#103, i_brand_id#104, i_class_id#105, i_category_id#106, spark_grouping_id#107, sum#111, isEmpty#112, sum#113] + +(103) Exchange +Input [8]: [channel#103, i_brand_id#104, i_class_id#105, i_category_id#106, spark_grouping_id#107, sum#111, isEmpty#112, sum#113] +Arguments: hashpartitioning(channel#103, i_brand_id#104, i_class_id#105, i_category_id#106, spark_grouping_id#107, 5), ENSURE_REQUIREMENTS, [plan_id=12] + +(104) HashAggregate [codegen id : 80] +Input [8]: [channel#103, i_brand_id#104, i_class_id#105, i_category_id#106, spark_grouping_id#107, sum#111, isEmpty#112, sum#113] +Keys [5]: [channel#103, i_brand_id#104, i_class_id#105, i_category_id#106, spark_grouping_id#107] +Functions [2]: [sum(sales#51), sum(number_sales#52)] +Aggregate Attributes [2]: [sum(sales#51)#114, sum(number_sales#52)#115] +Results [6]: [channel#103, i_brand_id#104, i_class_id#105, i_category_id#106, sum(sales#51)#114 AS sum(sales)#116, sum(number_sales#52)#115 AS sum(number_sales)#117] + +(105) TakeOrderedAndProject +Input [6]: [channel#103, i_brand_id#104, i_class_id#105, i_category_id#106, sum(sales)#116, sum(number_sales)#117] +Arguments: 100, [channel#103 ASC NULLS FIRST, i_brand_id#104 ASC NULLS FIRST, i_class_id#105 ASC NULLS FIRST, i_category_id#106 ASC NULLS FIRST], [channel#103, i_brand_id#104, i_class_id#105, i_category_id#106, sum(sales)#116, sum(number_sales)#117] + +===== Subqueries ===== + +Subquery:1 Hosting operator id = 66 Hosting Expression = Subquery scalar-subquery#53, [id=#54] +* HashAggregate (124) ++- Exchange (123) + +- * HashAggregate (122) + +- Union (121) + :- * Project (110) + : +- * BroadcastHashJoin Inner BuildRight (109) + : :- * ColumnarToRow (107) + : : +- CometScan parquet spark_catalog.default.store_sales (106) + : +- ReusedExchange (108) + :- * Project (115) + : +- * BroadcastHashJoin Inner BuildRight (114) + : :- * ColumnarToRow (112) + : : +- CometScan parquet spark_catalog.default.catalog_sales (111) + : +- ReusedExchange (113) + +- * Project (120) + +- * BroadcastHashJoin Inner BuildRight (119) + :- * ColumnarToRow (117) + : +- CometScan parquet spark_catalog.default.web_sales (116) + +- ReusedExchange (118) + + +(106) Scan parquet spark_catalog.default.store_sales +Output [3]: [ss_quantity#118, ss_list_price#119, ss_sold_date_sk#120] +Batched: true +Location: InMemoryFileIndex [] +PartitionFilters: [isnotnull(ss_sold_date_sk#120), dynamicpruningexpression(ss_sold_date_sk#120 IN dynamicpruning#121)] +ReadSchema: struct<ss_quantity:int,ss_list_price:decimal(7,2)> + +(107) ColumnarToRow [codegen id : 2] +Input [3]: [ss_quantity#118, ss_list_price#119, ss_sold_date_sk#120] + +(108) ReusedExchange [Reuses operator id: 134] +Output [1]: [d_date_sk#122] + +(109) BroadcastHashJoin [codegen id : 2] +Left keys [1]: [ss_sold_date_sk#120] +Right keys [1]: [d_date_sk#122] +Join type: Inner +Join condition: None + +(110) Project [codegen id : 2] +Output [2]: [ss_quantity#118 AS quantity#123, ss_list_price#119 AS list_price#124] +Input [4]: [ss_quantity#118, ss_list_price#119, ss_sold_date_sk#120, d_date_sk#122] + +(111) Scan parquet spark_catalog.default.catalog_sales +Output [3]: [cs_quantity#125, cs_list_price#126, cs_sold_date_sk#127] +Batched: true +Location: InMemoryFileIndex [] +PartitionFilters: [isnotnull(cs_sold_date_sk#127), dynamicpruningexpression(cs_sold_date_sk#127 IN dynamicpruning#128)] +ReadSchema: struct<cs_quantity:int,cs_list_price:decimal(7,2)> + +(112) ColumnarToRow [codegen id : 4] +Input [3]: [cs_quantity#125, cs_list_price#126, cs_sold_date_sk#127] + +(113) ReusedExchange [Reuses operator id: 134] +Output [1]: [d_date_sk#129] + +(114) BroadcastHashJoin [codegen id : 4] +Left keys [1]: [cs_sold_date_sk#127] +Right keys [1]: [d_date_sk#129] +Join type: Inner +Join condition: None + +(115) Project [codegen id : 4] +Output [2]: [cs_quantity#125 AS quantity#130, cs_list_price#126 AS list_price#131] +Input [4]: [cs_quantity#125, cs_list_price#126, cs_sold_date_sk#127, d_date_sk#129] + +(116) Scan parquet spark_catalog.default.web_sales +Output [3]: [ws_quantity#132, ws_list_price#133, ws_sold_date_sk#134] +Batched: true +Location: InMemoryFileIndex [] +PartitionFilters: [isnotnull(ws_sold_date_sk#134), dynamicpruningexpression(ws_sold_date_sk#134 IN dynamicpruning#135)] +ReadSchema: struct<ws_quantity:int,ws_list_price:decimal(7,2)> + +(117) ColumnarToRow [codegen id : 6] +Input [3]: [ws_quantity#132, ws_list_price#133, ws_sold_date_sk#134] + +(118) ReusedExchange [Reuses operator id: 134] +Output [1]: [d_date_sk#136] + +(119) BroadcastHashJoin [codegen id : 6] +Left keys [1]: [ws_sold_date_sk#134] +Right keys [1]: [d_date_sk#136] +Join type: Inner +Join condition: None + +(120) Project [codegen id : 6] +Output [2]: [ws_quantity#132 AS quantity#137, ws_list_price#133 AS list_price#138] +Input [4]: [ws_quantity#132, ws_list_price#133, ws_sold_date_sk#134, d_date_sk#136] + +(121) Union + +(122) HashAggregate [codegen id : 7] +Input [2]: [quantity#123, list_price#124] +Keys: [] +Functions [1]: [partial_avg((cast(quantity#123 as decimal(10,0)) * list_price#124))] +Aggregate Attributes [2]: [sum#139, count#140] +Results [2]: [sum#141, count#142] + +(123) Exchange +Input [2]: [sum#141, count#142] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=13] + +(124) HashAggregate [codegen id : 8] +Input [2]: [sum#141, count#142] +Keys: [] +Functions [1]: [avg((cast(quantity#123 as decimal(10,0)) * list_price#124))] +Aggregate Attributes [1]: [avg((cast(quantity#123 as decimal(10,0)) * list_price#124))#143] +Results [1]: [avg((cast(quantity#123 as decimal(10,0)) * list_price#124))#143 AS average_sales#144] + +Subquery:2 Hosting operator id = 106 Hosting Expression = ss_sold_date_sk#120 IN dynamicpruning#12 + +Subquery:3 Hosting operator id = 111 Hosting Expression = cs_sold_date_sk#127 IN dynamicpruning#12 + +Subquery:4 Hosting operator id = 116 Hosting Expression = ws_sold_date_sk#134 IN dynamicpruning#12 + +Subquery:5 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 +BroadcastExchange (129) ++- * ColumnarToRow (128) + +- CometProject (127) + +- CometFilter (126) + +- CometScan parquet spark_catalog.default.date_dim (125) + + +(125) Scan parquet spark_catalog.default.date_dim +Output [3]: [d_date_sk#42, d_year#145, d_moy#146] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2001), EqualTo(d_moy,11), IsNotNull(d_date_sk)] +ReadSchema: struct<d_date_sk:int,d_year:int,d_moy:int> + +(126) CometFilter +Input [3]: [d_date_sk#42, d_year#145, d_moy#146] +Condition : ((((isnotnull(d_year#145) AND isnotnull(d_moy#146)) AND (d_year#145 = 2001)) AND (d_moy#146 = 11)) AND isnotnull(d_date_sk#42)) + +(127) CometProject +Input [3]: [d_date_sk#42, d_year#145, d_moy#146] +Arguments: [d_date_sk#42], [d_date_sk#42] + +(128) ColumnarToRow [codegen id : 1] +Input [1]: [d_date_sk#42] + +(129) BroadcastExchange +Input [1]: [d_date_sk#42] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=14] + +Subquery:6 Hosting operator id = 7 Hosting Expression = ss_sold_date_sk#11 IN dynamicpruning#12 +BroadcastExchange (134) ++- * ColumnarToRow (133) + +- CometProject (132) + +- CometFilter (131) + +- CometScan parquet spark_catalog.default.date_dim (130) + + +(130) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#25, d_year#147] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_year), GreaterThanOrEqual(d_year,1999), LessThanOrEqual(d_year,2001), IsNotNull(d_date_sk)] +ReadSchema: struct<d_date_sk:int,d_year:int> + +(131) CometFilter +Input [2]: [d_date_sk#25, d_year#147] +Condition : (((isnotnull(d_year#147) AND (d_year#147 >= 1999)) AND (d_year#147 <= 2001)) AND isnotnull(d_date_sk#25)) + +(132) CometProject +Input [2]: [d_date_sk#25, d_year#147] +Arguments: [d_date_sk#25], [d_date_sk#25] + +(133) ColumnarToRow [codegen id : 1] +Input [1]: [d_date_sk#25] + +(134) BroadcastExchange +Input [1]: [d_date_sk#25] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=15] + +Subquery:7 Hosting operator id = 13 Hosting Expression = cs_sold_date_sk#18 IN dynamicpruning#12 + +Subquery:8 Hosting operator id = 36 Hosting Expression = ws_sold_date_sk#30 IN dynamicpruning#12 + +Subquery:9 Hosting operator id = 82 Hosting Expression = ReusedSubquery Subquery scalar-subquery#53, [id=#54] + +Subquery:10 Hosting operator id = 68 Hosting Expression = cs_sold_date_sk#62 IN dynamicpruning#5 + +Subquery:11 Hosting operator id = 98 Hosting Expression = ReusedSubquery Subquery scalar-subquery#53, [id=#54] + +Subquery:12 Hosting operator id = 84 Hosting Expression = ws_sold_date_sk#84 IN dynamicpruning#5 + + diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q14a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q14a/simplified.txt new file mode 100644 index 000000000..cf688c448 --- /dev/null +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q14a/simplified.txt @@ -0,0 +1,214 @@ +TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum(sales),sum(number_sales)] + WholeStageCodegen (80) + HashAggregate [channel,i_brand_id,i_class_id,i_category_id,spark_grouping_id,sum,isEmpty,sum] [sum(sales),sum(number_salesL),sum(sales),sum(number_sales),sum,isEmpty,sum] + InputAdapter + Exchange [channel,i_brand_id,i_class_id,i_category_id,spark_grouping_id] #1 + WholeStageCodegen (79) + 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) + Project [sales,number_sales,i_brand_id,i_class_id,i_category_id] + Filter [sales] + Subquery #3 + WholeStageCodegen (8) + HashAggregate [sum,count] [avg((cast(quantity as decimal(10,0)) * list_price)),average_sales,sum,count] + InputAdapter + Exchange #13 + WholeStageCodegen (7) + HashAggregate [quantity,list_price] [sum,count,sum,count] + InputAdapter + Union + WholeStageCodegen (2) + Project [ss_quantity,ss_list_price] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + ColumnarToRow + InputAdapter + CometScan parquet spark_catalog.default.store_sales [ss_quantity,ss_list_price,ss_sold_date_sk] + ReusedSubquery [d_date_sk] #2 + InputAdapter + ReusedExchange [d_date_sk] #7 + WholeStageCodegen (4) + Project [cs_quantity,cs_list_price] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + ColumnarToRow + InputAdapter + CometScan parquet spark_catalog.default.catalog_sales [cs_quantity,cs_list_price,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #2 + InputAdapter + ReusedExchange [d_date_sk] #7 + WholeStageCodegen (6) + Project [ws_quantity,ws_list_price] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + ColumnarToRow + InputAdapter + CometScan parquet spark_catalog.default.web_sales [ws_quantity,ws_list_price,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #2 + InputAdapter + ReusedExchange [d_date_sk] #7 + HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum((cast(ss_quantity as decimal(10,0)) * ss_list_price)),count(1),sales,number_sales,sum,isEmpty,count] + InputAdapter + Exchange [i_brand_id,i_class_id,i_category_id] #2 + WholeStageCodegen (25) + HashAggregate [i_brand_id,i_class_id,i_category_id,ss_quantity,ss_list_price] [sum,isEmpty,count,sum,isEmpty,count] + Project [ss_quantity,ss_list_price,i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [ss_quantity,ss_list_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [ss_item_sk,i_item_sk] + BroadcastHashJoin [ss_item_sk,ss_item_sk] + ColumnarToRow + InputAdapter + CometFilter [ss_item_sk] + CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_year,d_moy,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + InputAdapter + BroadcastExchange #4 + WholeStageCodegen (11) + Project [i_item_sk] + BroadcastHashJoin [i_brand_id,i_class_id,i_category_id,brand_id,class_id,category_id] + ColumnarToRow + InputAdapter + CometFilter [i_brand_id,i_class_id,i_category_id] + CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] + InputAdapter + BroadcastExchange #5 + WholeStageCodegen (10) + BroadcastHashJoin [brand_id,class_id,category_id,i_brand_id,i_class_id,i_category_id] + HashAggregate [brand_id,class_id,category_id] + InputAdapter + Exchange [brand_id,class_id,category_id] #6 + WholeStageCodegen (6) + HashAggregate [brand_id,class_id,category_id] + Project [i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [ss_sold_date_sk,i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [ss_item_sk,i_item_sk] + ColumnarToRow + InputAdapter + CometFilter [ss_item_sk] + CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #2 + BroadcastExchange #7 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_year,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + InputAdapter + BroadcastExchange #8 + WholeStageCodegen (4) + BroadcastHashJoin [i_brand_id,i_class_id,i_category_id,i_brand_id,i_class_id,i_category_id] + ColumnarToRow + InputAdapter + CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] + CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] + InputAdapter + BroadcastExchange #9 + WholeStageCodegen (3) + Project [i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + Project [cs_sold_date_sk,i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [cs_item_sk,i_item_sk] + ColumnarToRow + InputAdapter + CometFilter [cs_item_sk] + CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #2 + InputAdapter + BroadcastExchange #10 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometFilter [i_item_sk] + CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] + InputAdapter + ReusedExchange [d_date_sk] #7 + InputAdapter + ReusedExchange [d_date_sk] #7 + InputAdapter + BroadcastExchange #11 + WholeStageCodegen (9) + Project [i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + Project [ws_sold_date_sk,i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [ws_item_sk,i_item_sk] + ColumnarToRow + InputAdapter + CometFilter [ws_item_sk] + CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #2 + InputAdapter + ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #10 + InputAdapter + ReusedExchange [d_date_sk] #7 + InputAdapter + BroadcastExchange #12 + WholeStageCodegen (23) + BroadcastHashJoin [i_item_sk,ss_item_sk] + ColumnarToRow + InputAdapter + CometFilter [i_item_sk] + CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] + InputAdapter + ReusedExchange [ss_item_sk] #4 + InputAdapter + ReusedExchange [d_date_sk] #3 + WholeStageCodegen (52) + Project [sales,number_sales,i_brand_id,i_class_id,i_category_id] + Filter [sales] + ReusedSubquery [average_sales] #3 + HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum((cast(cs_quantity as decimal(10,0)) * cs_list_price)),count(1),sales,number_sales,sum,isEmpty,count] + InputAdapter + Exchange [i_brand_id,i_class_id,i_category_id] #14 + WholeStageCodegen (51) + HashAggregate [i_brand_id,i_class_id,i_category_id,cs_quantity,cs_list_price] [sum,isEmpty,count,sum,isEmpty,count] + Project [cs_quantity,cs_list_price,i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + Project [cs_quantity,cs_list_price,cs_sold_date_sk,i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [cs_item_sk,i_item_sk] + BroadcastHashJoin [cs_item_sk,ss_item_sk] + ColumnarToRow + InputAdapter + CometFilter [cs_item_sk] + CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_quantity,cs_list_price,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + InputAdapter + ReusedExchange [ss_item_sk] #4 + InputAdapter + ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #12 + InputAdapter + ReusedExchange [d_date_sk] #3 + WholeStageCodegen (78) + Project [sales,number_sales,i_brand_id,i_class_id,i_category_id] + Filter [sales] + ReusedSubquery [average_sales] #3 + HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum((cast(ws_quantity as decimal(10,0)) * ws_list_price)),count(1),sales,number_sales,sum,isEmpty,count] + InputAdapter + Exchange [i_brand_id,i_class_id,i_category_id] #15 + WholeStageCodegen (77) + HashAggregate [i_brand_id,i_class_id,i_category_id,ws_quantity,ws_list_price] [sum,isEmpty,count,sum,isEmpty,count] + Project [ws_quantity,ws_list_price,i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + Project [ws_quantity,ws_list_price,ws_sold_date_sk,i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [ws_item_sk,i_item_sk] + BroadcastHashJoin [ws_item_sk,ss_item_sk] + ColumnarToRow + InputAdapter + CometFilter [ws_item_sk] + CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_quantity,ws_list_price,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + InputAdapter + ReusedExchange [ss_item_sk] #4 + InputAdapter + ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #12 + InputAdapter + ReusedExchange [d_date_sk] #3 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q14b/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q14b/explain.txt new file mode 100644 index 000000000..265909ec2 --- /dev/null +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q14b/explain.txt @@ -0,0 +1,759 @@ +== Physical Plan == +TakeOrderedAndProject (84) ++- * BroadcastHashJoin Inner BuildRight (83) + :- * Filter (66) + : +- * HashAggregate (65) + : +- Exchange (64) + : +- * HashAggregate (63) + : +- * Project (62) + : +- * BroadcastHashJoin Inner BuildRight (61) + : :- * Project (59) + : : +- * BroadcastHashJoin Inner BuildRight (58) + : : :- * BroadcastHashJoin LeftSemi BuildRight (51) + : : : :- * ColumnarToRow (3) + : : : : +- CometFilter (2) + : : : : +- CometScan parquet spark_catalog.default.store_sales (1) + : : : +- BroadcastExchange (50) + : : : +- * Project (49) + : : : +- * BroadcastHashJoin Inner BuildRight (48) + : : : :- * ColumnarToRow (6) + : : : : +- CometFilter (5) + : : : : +- CometScan parquet spark_catalog.default.item (4) + : : : +- BroadcastExchange (47) + : : : +- * BroadcastHashJoin LeftSemi BuildRight (46) + : : : :- * HashAggregate (35) + : : : : +- Exchange (34) + : : : : +- * HashAggregate (33) + : : : : +- * Project (32) + : : : : +- * BroadcastHashJoin Inner BuildRight (31) + : : : : :- * Project (29) + : : : : : +- * BroadcastHashJoin Inner BuildRight (28) + : : : : : :- * ColumnarToRow (9) + : : : : : : +- CometFilter (8) + : : : : : : +- CometScan parquet spark_catalog.default.store_sales (7) + : : : : : +- BroadcastExchange (27) + : : : : : +- * BroadcastHashJoin LeftSemi BuildRight (26) + : : : : : :- * ColumnarToRow (12) + : : : : : : +- CometFilter (11) + : : : : : : +- CometScan parquet spark_catalog.default.item (10) + : : : : : +- BroadcastExchange (25) + : : : : : +- * Project (24) + : : : : : +- * BroadcastHashJoin Inner BuildRight (23) + : : : : : :- * Project (21) + : : : : : : +- * BroadcastHashJoin Inner BuildRight (20) + : : : : : : :- * ColumnarToRow (15) + : : : : : : : +- CometFilter (14) + : : : : : : : +- CometScan parquet spark_catalog.default.catalog_sales (13) + : : : : : : +- BroadcastExchange (19) + : : : : : : +- * ColumnarToRow (18) + : : : : : : +- CometFilter (17) + : : : : : : +- CometScan parquet spark_catalog.default.item (16) + : : : : : +- ReusedExchange (22) + : : : : +- ReusedExchange (30) + : : : +- BroadcastExchange (45) + : : : +- * Project (44) + : : : +- * BroadcastHashJoin Inner BuildRight (43) + : : : :- * Project (41) + : : : : +- * BroadcastHashJoin Inner BuildRight (40) + : : : : :- * ColumnarToRow (38) + : : : : : +- CometFilter (37) + : : : : : +- CometScan parquet spark_catalog.default.web_sales (36) + : : : : +- ReusedExchange (39) + : : : +- ReusedExchange (42) + : : +- BroadcastExchange (57) + : : +- * BroadcastHashJoin LeftSemi BuildRight (56) + : : :- * ColumnarToRow (54) + : : : +- CometFilter (53) + : : : +- CometScan parquet spark_catalog.default.item (52) + : : +- ReusedExchange (55) + : +- ReusedExchange (60) + +- BroadcastExchange (82) + +- * Filter (81) + +- * HashAggregate (80) + +- Exchange (79) + +- * HashAggregate (78) + +- * Project (77) + +- * BroadcastHashJoin Inner BuildRight (76) + :- * Project (74) + : +- * BroadcastHashJoin Inner BuildRight (73) + : :- * BroadcastHashJoin LeftSemi BuildRight (71) + : : :- * ColumnarToRow (69) + : : : +- CometFilter (68) + : : : +- CometScan parquet spark_catalog.default.store_sales (67) + : : +- ReusedExchange (70) + : +- ReusedExchange (72) + +- ReusedExchange (75) + + +(1) Scan parquet spark_catalog.default.store_sales +Output [4]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] +Batched: true +Location: InMemoryFileIndex [] +PartitionFilters: [isnotnull(ss_sold_date_sk#4), dynamicpruningexpression(ss_sold_date_sk#4 IN dynamicpruning#5)] +PushedFilters: [IsNotNull(ss_item_sk)] +ReadSchema: struct<ss_item_sk:int,ss_quantity:int,ss_list_price:decimal(7,2)> + +(2) CometFilter +Input [4]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] +Condition : isnotnull(ss_item_sk#1) + +(3) ColumnarToRow [codegen id : 25] +Input [4]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] + +(4) Scan parquet spark_catalog.default.item +Output [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] +Batched: true +Location [not included in comparison]/{warehouse_dir}/item] +PushedFilters: [IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id)] +ReadSchema: struct<i_item_sk:int,i_brand_id:int,i_class_id:int,i_category_id:int> + +(5) CometFilter +Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] +Condition : ((isnotnull(i_brand_id#7) AND isnotnull(i_class_id#8)) AND isnotnull(i_category_id#9)) + +(6) ColumnarToRow [codegen id : 11] +Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] + +(7) Scan parquet spark_catalog.default.store_sales +Output [2]: [ss_item_sk#10, ss_sold_date_sk#11] +Batched: true +Location: InMemoryFileIndex [] +PartitionFilters: [isnotnull(ss_sold_date_sk#11), dynamicpruningexpression(ss_sold_date_sk#11 IN dynamicpruning#12)] +PushedFilters: [IsNotNull(ss_item_sk)] +ReadSchema: struct<ss_item_sk:int> + +(8) CometFilter +Input [2]: [ss_item_sk#10, ss_sold_date_sk#11] +Condition : isnotnull(ss_item_sk#10) + +(9) ColumnarToRow [codegen id : 6] +Input [2]: [ss_item_sk#10, ss_sold_date_sk#11] + +(10) Scan parquet spark_catalog.default.item +Output [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] +Batched: true +Location [not included in comparison]/{warehouse_dir}/item] +PushedFilters: [IsNotNull(i_item_sk), IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id)] +ReadSchema: struct<i_item_sk:int,i_brand_id:int,i_class_id:int,i_category_id:int> + +(11) CometFilter +Input [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] +Condition : (((isnotnull(i_item_sk#13) AND isnotnull(i_brand_id#14)) AND isnotnull(i_class_id#15)) AND isnotnull(i_category_id#16)) + +(12) ColumnarToRow [codegen id : 4] +Input [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] + +(13) Scan parquet spark_catalog.default.catalog_sales +Output [2]: [cs_item_sk#17, cs_sold_date_sk#18] +Batched: true +Location: InMemoryFileIndex [] +PartitionFilters: [isnotnull(cs_sold_date_sk#18), dynamicpruningexpression(cs_sold_date_sk#18 IN dynamicpruning#19)] +PushedFilters: [IsNotNull(cs_item_sk)] +ReadSchema: struct<cs_item_sk:int> + +(14) CometFilter +Input [2]: [cs_item_sk#17, cs_sold_date_sk#18] +Condition : isnotnull(cs_item_sk#17) + +(15) ColumnarToRow [codegen id : 3] +Input [2]: [cs_item_sk#17, cs_sold_date_sk#18] + +(16) Scan parquet spark_catalog.default.item +Output [4]: [i_item_sk#20, i_brand_id#21, i_class_id#22, i_category_id#23] +Batched: true +Location [not included in comparison]/{warehouse_dir}/item] +PushedFilters: [IsNotNull(i_item_sk)] +ReadSchema: struct<i_item_sk:int,i_brand_id:int,i_class_id:int,i_category_id:int> + +(17) CometFilter +Input [4]: [i_item_sk#20, i_brand_id#21, i_class_id#22, i_category_id#23] +Condition : isnotnull(i_item_sk#20) + +(18) ColumnarToRow [codegen id : 1] +Input [4]: [i_item_sk#20, i_brand_id#21, i_class_id#22, i_category_id#23] + +(19) BroadcastExchange +Input [4]: [i_item_sk#20, i_brand_id#21, i_class_id#22, i_category_id#23] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=1] + +(20) BroadcastHashJoin [codegen id : 3] +Left keys [1]: [cs_item_sk#17] +Right keys [1]: [i_item_sk#20] +Join type: Inner +Join condition: None + +(21) Project [codegen id : 3] +Output [4]: [cs_sold_date_sk#18, i_brand_id#21, i_class_id#22, i_category_id#23] +Input [6]: [cs_item_sk#17, cs_sold_date_sk#18, i_item_sk#20, i_brand_id#21, i_class_id#22, i_category_id#23] + +(22) ReusedExchange [Reuses operator id: 117] +Output [1]: [d_date_sk#24] + +(23) BroadcastHashJoin [codegen id : 3] +Left keys [1]: [cs_sold_date_sk#18] +Right keys [1]: [d_date_sk#24] +Join type: Inner +Join condition: None + +(24) Project [codegen id : 3] +Output [3]: [i_brand_id#21, i_class_id#22, i_category_id#23] +Input [5]: [cs_sold_date_sk#18, i_brand_id#21, i_class_id#22, i_category_id#23, d_date_sk#24] + +(25) BroadcastExchange +Input [3]: [i_brand_id#21, i_class_id#22, i_category_id#23] +Arguments: HashedRelationBroadcastMode(List(coalesce(input[0, int, true], 0), isnull(input[0, int, true]), coalesce(input[1, int, true], 0), isnull(input[1, int, true]), coalesce(input[2, int, true], 0), isnull(input[2, int, true])),false), [plan_id=2] + +(26) BroadcastHashJoin [codegen id : 4] +Left keys [6]: [coalesce(i_brand_id#14, 0), isnull(i_brand_id#14), coalesce(i_class_id#15, 0), isnull(i_class_id#15), coalesce(i_category_id#16, 0), isnull(i_category_id#16)] +Right keys [6]: [coalesce(i_brand_id#21, 0), isnull(i_brand_id#21), coalesce(i_class_id#22, 0), isnull(i_class_id#22), coalesce(i_category_id#23, 0), isnull(i_category_id#23)] +Join type: LeftSemi +Join condition: None + +(27) BroadcastExchange +Input [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=3] + +(28) BroadcastHashJoin [codegen id : 6] +Left keys [1]: [ss_item_sk#10] +Right keys [1]: [i_item_sk#13] +Join type: Inner +Join condition: None + +(29) Project [codegen id : 6] +Output [4]: [ss_sold_date_sk#11, i_brand_id#14, i_class_id#15, i_category_id#16] +Input [6]: [ss_item_sk#10, ss_sold_date_sk#11, i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] + +(30) ReusedExchange [Reuses operator id: 117] +Output [1]: [d_date_sk#25] + +(31) BroadcastHashJoin [codegen id : 6] +Left keys [1]: [ss_sold_date_sk#11] +Right keys [1]: [d_date_sk#25] +Join type: Inner +Join condition: None + +(32) Project [codegen id : 6] +Output [3]: [i_brand_id#14 AS brand_id#26, i_class_id#15 AS class_id#27, i_category_id#16 AS category_id#28] +Input [5]: [ss_sold_date_sk#11, i_brand_id#14, i_class_id#15, i_category_id#16, d_date_sk#25] + +(33) HashAggregate [codegen id : 6] +Input [3]: [brand_id#26, class_id#27, category_id#28] +Keys [3]: [brand_id#26, class_id#27, category_id#28] +Functions: [] +Aggregate Attributes: [] +Results [3]: [brand_id#26, class_id#27, category_id#28] + +(34) Exchange +Input [3]: [brand_id#26, class_id#27, category_id#28] +Arguments: hashpartitioning(brand_id#26, class_id#27, category_id#28, 5), ENSURE_REQUIREMENTS, [plan_id=4] + +(35) HashAggregate [codegen id : 10] +Input [3]: [brand_id#26, class_id#27, category_id#28] +Keys [3]: [brand_id#26, class_id#27, category_id#28] +Functions: [] +Aggregate Attributes: [] +Results [3]: [brand_id#26, class_id#27, category_id#28] + +(36) Scan parquet spark_catalog.default.web_sales +Output [2]: [ws_item_sk#29, ws_sold_date_sk#30] +Batched: true +Location: InMemoryFileIndex [] +PartitionFilters: [isnotnull(ws_sold_date_sk#30), dynamicpruningexpression(ws_sold_date_sk#30 IN dynamicpruning#31)] +PushedFilters: [IsNotNull(ws_item_sk)] +ReadSchema: struct<ws_item_sk:int> + +(37) CometFilter +Input [2]: [ws_item_sk#29, ws_sold_date_sk#30] +Condition : isnotnull(ws_item_sk#29) + +(38) ColumnarToRow [codegen id : 9] +Input [2]: [ws_item_sk#29, ws_sold_date_sk#30] + +(39) ReusedExchange [Reuses operator id: 19] +Output [4]: [i_item_sk#32, i_brand_id#33, i_class_id#34, i_category_id#35] + +(40) BroadcastHashJoin [codegen id : 9] +Left keys [1]: [ws_item_sk#29] +Right keys [1]: [i_item_sk#32] +Join type: Inner +Join condition: None + +(41) Project [codegen id : 9] +Output [4]: [ws_sold_date_sk#30, i_brand_id#33, i_class_id#34, i_category_id#35] +Input [6]: [ws_item_sk#29, ws_sold_date_sk#30, i_item_sk#32, i_brand_id#33, i_class_id#34, i_category_id#35] + +(42) ReusedExchange [Reuses operator id: 117] +Output [1]: [d_date_sk#36] + +(43) BroadcastHashJoin [codegen id : 9] +Left keys [1]: [ws_sold_date_sk#30] +Right keys [1]: [d_date_sk#36] +Join type: Inner +Join condition: None + +(44) Project [codegen id : 9] +Output [3]: [i_brand_id#33, i_class_id#34, i_category_id#35] +Input [5]: [ws_sold_date_sk#30, i_brand_id#33, i_class_id#34, i_category_id#35, d_date_sk#36] + +(45) BroadcastExchange +Input [3]: [i_brand_id#33, i_class_id#34, i_category_id#35] +Arguments: HashedRelationBroadcastMode(List(coalesce(input[0, int, true], 0), isnull(input[0, int, true]), coalesce(input[1, int, true], 0), isnull(input[1, int, true]), coalesce(input[2, int, true], 0), isnull(input[2, int, true])),false), [plan_id=5] + +(46) BroadcastHashJoin [codegen id : 10] +Left keys [6]: [coalesce(brand_id#26, 0), isnull(brand_id#26), coalesce(class_id#27, 0), isnull(class_id#27), coalesce(category_id#28, 0), isnull(category_id#28)] +Right keys [6]: [coalesce(i_brand_id#33, 0), isnull(i_brand_id#33), coalesce(i_class_id#34, 0), isnull(i_class_id#34), coalesce(i_category_id#35, 0), isnull(i_category_id#35)] +Join type: LeftSemi +Join condition: None + +(47) BroadcastExchange +Input [3]: [brand_id#26, class_id#27, category_id#28] +Arguments: HashedRelationBroadcastMode(List(input[0, int, true], input[1, int, true], input[2, int, true]),false), [plan_id=6] + +(48) BroadcastHashJoin [codegen id : 11] +Left keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] +Right keys [3]: [brand_id#26, class_id#27, category_id#28] +Join type: Inner +Join condition: None + +(49) Project [codegen id : 11] +Output [1]: [i_item_sk#6 AS ss_item_sk#37] +Input [7]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9, brand_id#26, class_id#27, category_id#28] + +(50) BroadcastExchange +Input [1]: [ss_item_sk#37] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=7] + +(51) BroadcastHashJoin [codegen id : 25] +Left keys [1]: [ss_item_sk#1] +Right keys [1]: [ss_item_sk#37] +Join type: LeftSemi +Join condition: None + +(52) Scan parquet spark_catalog.default.item +Output [4]: [i_item_sk#38, i_brand_id#39, i_class_id#40, i_category_id#41] +Batched: true +Location [not included in comparison]/{warehouse_dir}/item] +PushedFilters: [IsNotNull(i_item_sk), IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id)] +ReadSchema: struct<i_item_sk:int,i_brand_id:int,i_class_id:int,i_category_id:int> + +(53) CometFilter +Input [4]: [i_item_sk#38, i_brand_id#39, i_class_id#40, i_category_id#41] +Condition : (((isnotnull(i_item_sk#38) AND isnotnull(i_brand_id#39)) AND isnotnull(i_class_id#40)) AND isnotnull(i_category_id#41)) + +(54) ColumnarToRow [codegen id : 23] +Input [4]: [i_item_sk#38, i_brand_id#39, i_class_id#40, i_category_id#41] + +(55) ReusedExchange [Reuses operator id: 50] +Output [1]: [ss_item_sk#37] + +(56) BroadcastHashJoin [codegen id : 23] +Left keys [1]: [i_item_sk#38] +Right keys [1]: [ss_item_sk#37] +Join type: LeftSemi +Join condition: None + +(57) BroadcastExchange +Input [4]: [i_item_sk#38, i_brand_id#39, i_class_id#40, i_category_id#41] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=8] + +(58) BroadcastHashJoin [codegen id : 25] +Left keys [1]: [ss_item_sk#1] +Right keys [1]: [i_item_sk#38] +Join type: Inner +Join condition: None + +(59) Project [codegen id : 25] +Output [6]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#39, i_class_id#40, i_category_id#41] +Input [8]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_item_sk#38, i_brand_id#39, i_class_id#40, i_category_id#41] + +(60) ReusedExchange [Reuses operator id: 108] +Output [1]: [d_date_sk#42] + +(61) BroadcastHashJoin [codegen id : 25] +Left keys [1]: [ss_sold_date_sk#4] +Right keys [1]: [d_date_sk#42] +Join type: Inner +Join condition: None + +(62) Project [codegen id : 25] +Output [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#39, i_class_id#40, i_category_id#41] +Input [7]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#39, i_class_id#40, i_category_id#41, d_date_sk#42] + +(63) HashAggregate [codegen id : 25] +Input [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#39, i_class_id#40, i_category_id#41] +Keys [3]: [i_brand_id#39, i_class_id#40, i_category_id#41] +Functions [2]: [partial_sum((cast(ss_quantity#2 as decimal(10,0)) * ss_list_price#3)), partial_count(1)] +Aggregate Attributes [3]: [sum#43, isEmpty#44, count#45] +Results [6]: [i_brand_id#39, i_class_id#40, i_category_id#41, sum#46, isEmpty#47, count#48] + +(64) Exchange +Input [6]: [i_brand_id#39, i_class_id#40, i_category_id#41, sum#46, isEmpty#47, count#48] +Arguments: hashpartitioning(i_brand_id#39, i_class_id#40, i_category_id#41, 5), ENSURE_REQUIREMENTS, [plan_id=9] + +(65) HashAggregate [codegen id : 52] +Input [6]: [i_brand_id#39, i_class_id#40, i_category_id#41, sum#46, isEmpty#47, count#48] +Keys [3]: [i_brand_id#39, i_class_id#40, i_category_id#41] +Functions [2]: [sum((cast(ss_quantity#2 as decimal(10,0)) * ss_list_price#3)), count(1)] +Aggregate Attributes [2]: [sum((cast(ss_quantity#2 as decimal(10,0)) * ss_list_price#3))#49, count(1)#50] +Results [6]: [store AS channel#51, i_brand_id#39, i_class_id#40, i_category_id#41, sum((cast(ss_quantity#2 as decimal(10,0)) * ss_list_price#3))#49 AS sales#52, count(1)#50 AS number_sales#53] + +(66) Filter [codegen id : 52] +Input [6]: [channel#51, i_brand_id#39, i_class_id#40, i_category_id#41, sales#52, number_sales#53] +Condition : (isnotnull(sales#52) AND (cast(sales#52 as decimal(32,6)) > cast(Subquery scalar-subquery#54, [id=#55] as decimal(32,6)))) + +(67) Scan parquet spark_catalog.default.store_sales +Output [4]: [ss_item_sk#56, ss_quantity#57, ss_list_price#58, ss_sold_date_sk#59] +Batched: true +Location: InMemoryFileIndex [] +PartitionFilters: [isnotnull(ss_sold_date_sk#59), dynamicpruningexpression(ss_sold_date_sk#59 IN dynamicpruning#60)] +PushedFilters: [IsNotNull(ss_item_sk)] +ReadSchema: struct<ss_item_sk:int,ss_quantity:int,ss_list_price:decimal(7,2)> + +(68) CometFilter +Input [4]: [ss_item_sk#56, ss_quantity#57, ss_list_price#58, ss_sold_date_sk#59] +Condition : isnotnull(ss_item_sk#56) + +(69) ColumnarToRow [codegen id : 50] +Input [4]: [ss_item_sk#56, ss_quantity#57, ss_list_price#58, ss_sold_date_sk#59] + +(70) ReusedExchange [Reuses operator id: 50] +Output [1]: [ss_item_sk#61] + +(71) BroadcastHashJoin [codegen id : 50] +Left keys [1]: [ss_item_sk#56] +Right keys [1]: [ss_item_sk#61] +Join type: LeftSemi +Join condition: None + +(72) ReusedExchange [Reuses operator id: 57] +Output [4]: [i_item_sk#62, i_brand_id#63, i_class_id#64, i_category_id#65] + +(73) BroadcastHashJoin [codegen id : 50] +Left keys [1]: [ss_item_sk#56] +Right keys [1]: [i_item_sk#62] +Join type: Inner +Join condition: None + +(74) Project [codegen id : 50] +Output [6]: [ss_quantity#57, ss_list_price#58, ss_sold_date_sk#59, i_brand_id#63, i_class_id#64, i_category_id#65] +Input [8]: [ss_item_sk#56, ss_quantity#57, ss_list_price#58, ss_sold_date_sk#59, i_item_sk#62, i_brand_id#63, i_class_id#64, i_category_id#65] + +(75) ReusedExchange [Reuses operator id: 122] +Output [1]: [d_date_sk#66] + +(76) BroadcastHashJoin [codegen id : 50] +Left keys [1]: [ss_sold_date_sk#59] +Right keys [1]: [d_date_sk#66] +Join type: Inner +Join condition: None + +(77) Project [codegen id : 50] +Output [5]: [ss_quantity#57, ss_list_price#58, i_brand_id#63, i_class_id#64, i_category_id#65] +Input [7]: [ss_quantity#57, ss_list_price#58, ss_sold_date_sk#59, i_brand_id#63, i_class_id#64, i_category_id#65, d_date_sk#66] + +(78) HashAggregate [codegen id : 50] +Input [5]: [ss_quantity#57, ss_list_price#58, i_brand_id#63, i_class_id#64, i_category_id#65] +Keys [3]: [i_brand_id#63, i_class_id#64, i_category_id#65] +Functions [2]: [partial_sum((cast(ss_quantity#57 as decimal(10,0)) * ss_list_price#58)), partial_count(1)] +Aggregate Attributes [3]: [sum#67, isEmpty#68, count#69] +Results [6]: [i_brand_id#63, i_class_id#64, i_category_id#65, sum#70, isEmpty#71, count#72] + +(79) Exchange +Input [6]: [i_brand_id#63, i_class_id#64, i_category_id#65, sum#70, isEmpty#71, count#72] +Arguments: hashpartitioning(i_brand_id#63, i_class_id#64, i_category_id#65, 5), ENSURE_REQUIREMENTS, [plan_id=10] + +(80) HashAggregate [codegen id : 51] +Input [6]: [i_brand_id#63, i_class_id#64, i_category_id#65, sum#70, isEmpty#71, count#72] +Keys [3]: [i_brand_id#63, i_class_id#64, i_category_id#65] +Functions [2]: [sum((cast(ss_quantity#57 as decimal(10,0)) * ss_list_price#58)), count(1)] +Aggregate Attributes [2]: [sum((cast(ss_quantity#57 as decimal(10,0)) * ss_list_price#58))#73, count(1)#74] +Results [6]: [store AS channel#75, i_brand_id#63, i_class_id#64, i_category_id#65, sum((cast(ss_quantity#57 as decimal(10,0)) * ss_list_price#58))#73 AS sales#76, count(1)#74 AS number_sales#77] + +(81) Filter [codegen id : 51] +Input [6]: [channel#75, i_brand_id#63, i_class_id#64, i_category_id#65, sales#76, number_sales#77] +Condition : (isnotnull(sales#76) AND (cast(sales#76 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#54, [id=#55] as decimal(32,6)))) + +(82) BroadcastExchange +Input [6]: [channel#75, i_brand_id#63, i_class_id#64, i_category_id#65, sales#76, number_sales#77] +Arguments: HashedRelationBroadcastMode(List(input[1, int, true], input[2, int, true], input[3, int, true]),false), [plan_id=11] + +(83) BroadcastHashJoin [codegen id : 52] +Left keys [3]: [i_brand_id#39, i_class_id#40, i_category_id#41] +Right keys [3]: [i_brand_id#63, i_class_id#64, i_category_id#65] +Join type: Inner +Join condition: None + +(84) TakeOrderedAndProject +Input [12]: [channel#51, i_brand_id#39, i_class_id#40, i_category_id#41, sales#52, number_sales#53, channel#75, i_brand_id#63, i_class_id#64, i_category_id#65, sales#76, number_sales#77] +Arguments: 100, [i_brand_id#39 ASC NULLS FIRST, i_class_id#40 ASC NULLS FIRST, i_category_id#41 ASC NULLS FIRST], [channel#51, i_brand_id#39, i_class_id#40, i_category_id#41, sales#52, number_sales#53, channel#75, i_brand_id#63, i_class_id#64, i_category_id#65, sales#76, number_sales#77] + +===== Subqueries ===== + +Subquery:1 Hosting operator id = 66 Hosting Expression = Subquery scalar-subquery#54, [id=#55] +* HashAggregate (103) ++- Exchange (102) + +- * HashAggregate (101) + +- Union (100) + :- * Project (89) + : +- * BroadcastHashJoin Inner BuildRight (88) + : :- * ColumnarToRow (86) + : : +- CometScan parquet spark_catalog.default.store_sales (85) + : +- ReusedExchange (87) + :- * Project (94) + : +- * BroadcastHashJoin Inner BuildRight (93) + : :- * ColumnarToRow (91) + : : +- CometScan parquet spark_catalog.default.catalog_sales (90) + : +- ReusedExchange (92) + +- * Project (99) + +- * BroadcastHashJoin Inner BuildRight (98) + :- * ColumnarToRow (96) + : +- CometScan parquet spark_catalog.default.web_sales (95) + +- ReusedExchange (97) + + +(85) Scan parquet spark_catalog.default.store_sales +Output [3]: [ss_quantity#78, ss_list_price#79, ss_sold_date_sk#80] +Batched: true +Location: InMemoryFileIndex [] +PartitionFilters: [isnotnull(ss_sold_date_sk#80), dynamicpruningexpression(ss_sold_date_sk#80 IN dynamicpruning#81)] +ReadSchema: struct<ss_quantity:int,ss_list_price:decimal(7,2)> + +(86) ColumnarToRow [codegen id : 2] +Input [3]: [ss_quantity#78, ss_list_price#79, ss_sold_date_sk#80] + +(87) ReusedExchange [Reuses operator id: 117] +Output [1]: [d_date_sk#82] + +(88) BroadcastHashJoin [codegen id : 2] +Left keys [1]: [ss_sold_date_sk#80] +Right keys [1]: [d_date_sk#82] +Join type: Inner +Join condition: None + +(89) Project [codegen id : 2] +Output [2]: [ss_quantity#78 AS quantity#83, ss_list_price#79 AS list_price#84] +Input [4]: [ss_quantity#78, ss_list_price#79, ss_sold_date_sk#80, d_date_sk#82] + +(90) Scan parquet spark_catalog.default.catalog_sales +Output [3]: [cs_quantity#85, cs_list_price#86, cs_sold_date_sk#87] +Batched: true +Location: InMemoryFileIndex [] +PartitionFilters: [isnotnull(cs_sold_date_sk#87), dynamicpruningexpression(cs_sold_date_sk#87 IN dynamicpruning#88)] +ReadSchema: struct<cs_quantity:int,cs_list_price:decimal(7,2)> + +(91) ColumnarToRow [codegen id : 4] +Input [3]: [cs_quantity#85, cs_list_price#86, cs_sold_date_sk#87] + +(92) ReusedExchange [Reuses operator id: 117] +Output [1]: [d_date_sk#89] + +(93) BroadcastHashJoin [codegen id : 4] +Left keys [1]: [cs_sold_date_sk#87] +Right keys [1]: [d_date_sk#89] +Join type: Inner +Join condition: None + +(94) Project [codegen id : 4] +Output [2]: [cs_quantity#85 AS quantity#90, cs_list_price#86 AS list_price#91] +Input [4]: [cs_quantity#85, cs_list_price#86, cs_sold_date_sk#87, d_date_sk#89] + +(95) Scan parquet spark_catalog.default.web_sales +Output [3]: [ws_quantity#92, ws_list_price#93, ws_sold_date_sk#94] +Batched: true +Location: InMemoryFileIndex [] +PartitionFilters: [isnotnull(ws_sold_date_sk#94), dynamicpruningexpression(ws_sold_date_sk#94 IN dynamicpruning#95)] +ReadSchema: struct<ws_quantity:int,ws_list_price:decimal(7,2)> + +(96) ColumnarToRow [codegen id : 6] +Input [3]: [ws_quantity#92, ws_list_price#93, ws_sold_date_sk#94] + +(97) ReusedExchange [Reuses operator id: 117] +Output [1]: [d_date_sk#96] + +(98) BroadcastHashJoin [codegen id : 6] +Left keys [1]: [ws_sold_date_sk#94] +Right keys [1]: [d_date_sk#96] +Join type: Inner +Join condition: None + +(99) Project [codegen id : 6] +Output [2]: [ws_quantity#92 AS quantity#97, ws_list_price#93 AS list_price#98] +Input [4]: [ws_quantity#92, ws_list_price#93, ws_sold_date_sk#94, d_date_sk#96] + +(100) Union + +(101) HashAggregate [codegen id : 7] +Input [2]: [quantity#83, list_price#84] +Keys: [] +Functions [1]: [partial_avg((cast(quantity#83 as decimal(10,0)) * list_price#84))] +Aggregate Attributes [2]: [sum#99, count#100] +Results [2]: [sum#101, count#102] + +(102) Exchange +Input [2]: [sum#101, count#102] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=12] + +(103) HashAggregate [codegen id : 8] +Input [2]: [sum#101, count#102] +Keys: [] +Functions [1]: [avg((cast(quantity#83 as decimal(10,0)) * list_price#84))] +Aggregate Attributes [1]: [avg((cast(quantity#83 as decimal(10,0)) * list_price#84))#103] +Results [1]: [avg((cast(quantity#83 as decimal(10,0)) * list_price#84))#103 AS average_sales#104] + +Subquery:2 Hosting operator id = 85 Hosting Expression = ss_sold_date_sk#80 IN dynamicpruning#12 + +Subquery:3 Hosting operator id = 90 Hosting Expression = cs_sold_date_sk#87 IN dynamicpruning#12 + +Subquery:4 Hosting operator id = 95 Hosting Expression = ws_sold_date_sk#94 IN dynamicpruning#12 + +Subquery:5 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 +BroadcastExchange (108) ++- * ColumnarToRow (107) + +- CometProject (106) + +- CometFilter (105) + +- CometScan parquet spark_catalog.default.date_dim (104) + + +(104) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#42, d_week_seq#105] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_week_seq), EqualTo(d_week_seq,ScalarSubquery#106), IsNotNull(d_date_sk)] +ReadSchema: struct<d_date_sk:int,d_week_seq:int> + +(105) CometFilter +Input [2]: [d_date_sk#42, d_week_seq#105] +Condition : ((isnotnull(d_week_seq#105) AND (d_week_seq#105 = ReusedSubquery Subquery scalar-subquery#106, [id=#107])) AND isnotnull(d_date_sk#42)) + +(106) CometProject +Input [2]: [d_date_sk#42, d_week_seq#105] +Arguments: [d_date_sk#42], [d_date_sk#42] + +(107) ColumnarToRow [codegen id : 1] +Input [1]: [d_date_sk#42] + +(108) BroadcastExchange +Input [1]: [d_date_sk#42] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=13] + +Subquery:6 Hosting operator id = 105 Hosting Expression = ReusedSubquery Subquery scalar-subquery#106, [id=#107] + +Subquery:7 Hosting operator id = 104 Hosting Expression = Subquery scalar-subquery#106, [id=#107] +* ColumnarToRow (112) ++- CometProject (111) + +- CometFilter (110) + +- CometScan parquet spark_catalog.default.date_dim (109) + + +(109) Scan parquet spark_catalog.default.date_dim +Output [4]: [d_week_seq#108, d_year#109, d_moy#110, d_dom#111] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), IsNotNull(d_dom), EqualTo(d_year,2000), EqualTo(d_moy,12), EqualTo(d_dom,11)] +ReadSchema: struct<d_week_seq:int,d_year:int,d_moy:int,d_dom:int> + +(110) CometFilter +Input [4]: [d_week_seq#108, d_year#109, d_moy#110, d_dom#111] +Condition : (((((isnotnull(d_year#109) AND isnotnull(d_moy#110)) AND isnotnull(d_dom#111)) AND (d_year#109 = 2000)) AND (d_moy#110 = 12)) AND (d_dom#111 = 11)) + +(111) CometProject +Input [4]: [d_week_seq#108, d_year#109, d_moy#110, d_dom#111] +Arguments: [d_week_seq#108], [d_week_seq#108] + +(112) ColumnarToRow [codegen id : 1] +Input [1]: [d_week_seq#108] + +Subquery:8 Hosting operator id = 7 Hosting Expression = ss_sold_date_sk#11 IN dynamicpruning#12 +BroadcastExchange (117) ++- * ColumnarToRow (116) + +- CometProject (115) + +- CometFilter (114) + +- CometScan parquet spark_catalog.default.date_dim (113) + + +(113) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#25, d_year#112] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_year), GreaterThanOrEqual(d_year,1999), LessThanOrEqual(d_year,2001), IsNotNull(d_date_sk)] +ReadSchema: struct<d_date_sk:int,d_year:int> + +(114) CometFilter +Input [2]: [d_date_sk#25, d_year#112] +Condition : (((isnotnull(d_year#112) AND (d_year#112 >= 1999)) AND (d_year#112 <= 2001)) AND isnotnull(d_date_sk#25)) + +(115) CometProject +Input [2]: [d_date_sk#25, d_year#112] +Arguments: [d_date_sk#25], [d_date_sk#25] + +(116) ColumnarToRow [codegen id : 1] +Input [1]: [d_date_sk#25] + +(117) BroadcastExchange +Input [1]: [d_date_sk#25] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=14] + +Subquery:9 Hosting operator id = 13 Hosting Expression = cs_sold_date_sk#18 IN dynamicpruning#12 + +Subquery:10 Hosting operator id = 36 Hosting Expression = ws_sold_date_sk#30 IN dynamicpruning#12 + +Subquery:11 Hosting operator id = 81 Hosting Expression = ReusedSubquery Subquery scalar-subquery#54, [id=#55] + +Subquery:12 Hosting operator id = 67 Hosting Expression = ss_sold_date_sk#59 IN dynamicpruning#60 +BroadcastExchange (122) ++- * ColumnarToRow (121) + +- CometProject (120) + +- CometFilter (119) + +- CometScan parquet spark_catalog.default.date_dim (118) + + +(118) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#66, d_week_seq#113] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_week_seq), EqualTo(d_week_seq,ScalarSubquery#114), IsNotNull(d_date_sk)] +ReadSchema: struct<d_date_sk:int,d_week_seq:int> + +(119) CometFilter +Input [2]: [d_date_sk#66, d_week_seq#113] +Condition : ((isnotnull(d_week_seq#113) AND (d_week_seq#113 = ReusedSubquery Subquery scalar-subquery#114, [id=#115])) AND isnotnull(d_date_sk#66)) + +(120) CometProject +Input [2]: [d_date_sk#66, d_week_seq#113] +Arguments: [d_date_sk#66], [d_date_sk#66] + +(121) ColumnarToRow [codegen id : 1] +Input [1]: [d_date_sk#66] + +(122) BroadcastExchange +Input [1]: [d_date_sk#66] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=15] + +Subquery:13 Hosting operator id = 119 Hosting Expression = ReusedSubquery Subquery scalar-subquery#114, [id=#115] + +Subquery:14 Hosting operator id = 118 Hosting Expression = Subquery scalar-subquery#114, [id=#115] +* ColumnarToRow (126) ++- CometProject (125) + +- CometFilter (124) + +- CometScan parquet spark_catalog.default.date_dim (123) + + +(123) Scan parquet spark_catalog.default.date_dim +Output [4]: [d_week_seq#116, d_year#117, d_moy#118, d_dom#119] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), IsNotNull(d_dom), EqualTo(d_year,1999), EqualTo(d_moy,12), EqualTo(d_dom,11)] +ReadSchema: struct<d_week_seq:int,d_year:int,d_moy:int,d_dom:int> + +(124) CometFilter +Input [4]: [d_week_seq#116, d_year#117, d_moy#118, d_dom#119] +Condition : (((((isnotnull(d_year#117) AND isnotnull(d_moy#118)) AND isnotnull(d_dom#119)) AND (d_year#117 = 1999)) AND (d_moy#118 = 12)) AND (d_dom#119 = 11)) + +(125) CometProject +Input [4]: [d_week_seq#116, d_year#117, d_moy#118, d_dom#119] +Arguments: [d_week_seq#116], [d_week_seq#116] + +(126) ColumnarToRow [codegen id : 1] +Input [1]: [d_week_seq#116] + + 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 new file mode 100644 index 000000000..55aa823ab --- /dev/null +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q14b/simplified.txt @@ -0,0 +1,204 @@ +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) + 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) + HashAggregate [sum,count] [avg((cast(quantity as decimal(10,0)) * list_price)),average_sales,sum,count] + InputAdapter + Exchange #12 + WholeStageCodegen (7) + HashAggregate [quantity,list_price] [sum,count,sum,count] + InputAdapter + Union + WholeStageCodegen (2) + Project [ss_quantity,ss_list_price] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + ColumnarToRow + InputAdapter + CometScan parquet spark_catalog.default.store_sales [ss_quantity,ss_list_price,ss_sold_date_sk] + ReusedSubquery [d_date_sk] #3 + InputAdapter + ReusedExchange [d_date_sk] #6 + WholeStageCodegen (4) + Project [cs_quantity,cs_list_price] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + ColumnarToRow + InputAdapter + CometScan parquet spark_catalog.default.catalog_sales [cs_quantity,cs_list_price,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #3 + InputAdapter + ReusedExchange [d_date_sk] #6 + WholeStageCodegen (6) + Project [ws_quantity,ws_list_price] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + ColumnarToRow + InputAdapter + CometScan parquet spark_catalog.default.web_sales [ws_quantity,ws_list_price,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #3 + InputAdapter + ReusedExchange [d_date_sk] #6 + HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum((cast(ss_quantity as decimal(10,0)) * ss_list_price)),count(1),channel,sales,number_sales,sum,isEmpty,count] + InputAdapter + Exchange [i_brand_id,i_class_id,i_category_id] #1 + WholeStageCodegen (25) + HashAggregate [i_brand_id,i_class_id,i_category_id,ss_quantity,ss_list_price] [sum,isEmpty,count,sum,isEmpty,count] + Project [ss_quantity,ss_list_price,i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [ss_quantity,ss_list_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [ss_item_sk,i_item_sk] + BroadcastHashJoin [ss_item_sk,ss_item_sk] + ColumnarToRow + InputAdapter + CometFilter [ss_item_sk] + CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #2 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_week_seq,d_date_sk] + ReusedSubquery [d_week_seq] #2 + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq] + Subquery #2 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_week_seq] + CometFilter [d_year,d_moy,d_dom] + CometScan parquet spark_catalog.default.date_dim [d_week_seq,d_year,d_moy,d_dom] + InputAdapter + BroadcastExchange #3 + WholeStageCodegen (11) + Project [i_item_sk] + BroadcastHashJoin [i_brand_id,i_class_id,i_category_id,brand_id,class_id,category_id] + ColumnarToRow + InputAdapter + CometFilter [i_brand_id,i_class_id,i_category_id] + CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] + InputAdapter + BroadcastExchange #4 + WholeStageCodegen (10) + BroadcastHashJoin [brand_id,class_id,category_id,i_brand_id,i_class_id,i_category_id] + HashAggregate [brand_id,class_id,category_id] + InputAdapter + Exchange [brand_id,class_id,category_id] #5 + WholeStageCodegen (6) + HashAggregate [brand_id,class_id,category_id] + Project [i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [ss_sold_date_sk,i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [ss_item_sk,i_item_sk] + ColumnarToRow + InputAdapter + CometFilter [ss_item_sk] + CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #3 + BroadcastExchange #6 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_year,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + InputAdapter + BroadcastExchange #7 + WholeStageCodegen (4) + BroadcastHashJoin [i_brand_id,i_class_id,i_category_id,i_brand_id,i_class_id,i_category_id] + ColumnarToRow + InputAdapter + CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] + CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] + InputAdapter + BroadcastExchange #8 + WholeStageCodegen (3) + Project [i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + Project [cs_sold_date_sk,i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [cs_item_sk,i_item_sk] + ColumnarToRow + InputAdapter + CometFilter [cs_item_sk] + CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #3 + InputAdapter + BroadcastExchange #9 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometFilter [i_item_sk] + CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] + InputAdapter + ReusedExchange [d_date_sk] #6 + InputAdapter + ReusedExchange [d_date_sk] #6 + InputAdapter + BroadcastExchange #10 + WholeStageCodegen (9) + Project [i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + Project [ws_sold_date_sk,i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [ws_item_sk,i_item_sk] + ColumnarToRow + InputAdapter + CometFilter [ws_item_sk] + CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #3 + InputAdapter + ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #9 + InputAdapter + ReusedExchange [d_date_sk] #6 + InputAdapter + BroadcastExchange #11 + WholeStageCodegen (23) + BroadcastHashJoin [i_item_sk,ss_item_sk] + ColumnarToRow + InputAdapter + CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] + CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] + InputAdapter + ReusedExchange [ss_item_sk] #3 + InputAdapter + ReusedExchange [d_date_sk] #2 + InputAdapter + BroadcastExchange #13 + WholeStageCodegen (51) + Filter [sales] + ReusedSubquery [average_sales] #4 + HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum((cast(ss_quantity as decimal(10,0)) * ss_list_price)),count(1),channel,sales,number_sales,sum,isEmpty,count] + InputAdapter + Exchange [i_brand_id,i_class_id,i_category_id] #14 + WholeStageCodegen (50) + HashAggregate [i_brand_id,i_class_id,i_category_id,ss_quantity,ss_list_price] [sum,isEmpty,count,sum,isEmpty,count] + Project [ss_quantity,ss_list_price,i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [ss_quantity,ss_list_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [ss_item_sk,i_item_sk] + BroadcastHashJoin [ss_item_sk,ss_item_sk] + ColumnarToRow + InputAdapter + CometFilter [ss_item_sk] + CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #5 + BroadcastExchange #15 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_week_seq,d_date_sk] + ReusedSubquery [d_week_seq] #6 + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq] + Subquery #6 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_week_seq] + CometFilter [d_year,d_moy,d_dom] + CometScan parquet spark_catalog.default.date_dim [d_week_seq,d_year,d_moy,d_dom] + InputAdapter + ReusedExchange [ss_item_sk] #3 + InputAdapter + ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #11 + InputAdapter + ReusedExchange [d_date_sk] #15 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 new file mode 100644 index 000000000..13a621f77 --- /dev/null +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q15/explain.txt @@ -0,0 +1,164 @@ +== Physical Plan == +TakeOrderedAndProject (22) ++- * HashAggregate (21) + +- Exchange (20) + +- * HashAggregate (19) + +- * Project (18) + +- * BroadcastHashJoin Inner BuildRight (17) + :- * Project (15) + : +- * BroadcastHashJoin Inner BuildRight (14) + : :- * Project (9) + : : +- * BroadcastHashJoin Inner BuildRight (8) + : : :- * ColumnarToRow (3) + : : : +- CometFilter (2) + : : : +- CometScan parquet spark_catalog.default.catalog_sales (1) + : : +- BroadcastExchange (7) + : : +- * ColumnarToRow (6) + : : +- CometFilter (5) + : : +- CometScan parquet spark_catalog.default.customer (4) + : +- BroadcastExchange (13) + : +- * ColumnarToRow (12) + : +- CometFilter (11) + : +- CometScan parquet spark_catalog.default.customer_address (10) + +- ReusedExchange (16) + + +(1) Scan parquet spark_catalog.default.catalog_sales +Output [3]: [cs_bill_customer_sk#1, cs_sales_price#2, cs_sold_date_sk#3] +Batched: true +Location: InMemoryFileIndex [] +PartitionFilters: [isnotnull(cs_sold_date_sk#3), dynamicpruningexpression(cs_sold_date_sk#3 IN dynamicpruning#4)] +PushedFilters: [IsNotNull(cs_bill_customer_sk)] +ReadSchema: struct<cs_bill_customer_sk:int,cs_sales_price:decimal(7,2)> + +(2) CometFilter +Input [3]: [cs_bill_customer_sk#1, cs_sales_price#2, cs_sold_date_sk#3] +Condition : isnotnull(cs_bill_customer_sk#1) + +(3) ColumnarToRow [codegen id : 4] +Input [3]: [cs_bill_customer_sk#1, cs_sales_price#2, cs_sold_date_sk#3] + +(4) Scan parquet spark_catalog.default.customer +Output [2]: [c_customer_sk#5, c_current_addr_sk#6] +Batched: true +Location [not included in comparison]/{warehouse_dir}/customer] +PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_current_addr_sk)] +ReadSchema: struct<c_customer_sk:int,c_current_addr_sk:int> + +(5) CometFilter +Input [2]: [c_customer_sk#5, c_current_addr_sk#6] +Condition : (isnotnull(c_customer_sk#5) AND isnotnull(c_current_addr_sk#6)) + +(6) ColumnarToRow [codegen id : 1] +Input [2]: [c_customer_sk#5, c_current_addr_sk#6] + +(7) BroadcastExchange +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] +Left keys [1]: [cs_bill_customer_sk#1] +Right keys [1]: [c_customer_sk#5] +Join type: Inner +Join condition: None + +(9) Project [codegen id : 4] +Output [3]: [cs_sales_price#2, cs_sold_date_sk#3, c_current_addr_sk#6] +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] + +(10) Scan parquet spark_catalog.default.customer_address +Output [3]: [ca_address_sk#7, ca_state#8, ca_zip#9] +Batched: true +Location [not included in comparison]/{warehouse_dir}/customer_address] +PushedFilters: [IsNotNull(ca_address_sk)] +ReadSchema: struct<ca_address_sk:int,ca_state:string,ca_zip:string> + +(11) CometFilter +Input [3]: [ca_address_sk#7, ca_state#8, ca_zip#9] +Condition : isnotnull(ca_address_sk#7) + +(12) ColumnarToRow [codegen id : 2] +Input [3]: [ca_address_sk#7, ca_state#8, ca_zip#9] + +(13) BroadcastExchange +Input [3]: [ca_address_sk#7, ca_state#8, ca_zip#9] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=2] + +(14) BroadcastHashJoin [codegen id : 4] +Left keys [1]: [c_current_addr_sk#6] +Right keys [1]: [ca_address_sk#7] +Join type: Inner +Join condition: ((substr(ca_zip#9, 1, 5) IN (85669,86197,88274,83405,86475,85392,85460,80348,81792) OR ca_state#8 IN (CA,WA,GA)) OR (cs_sales_price#2 > 500.00)) + +(15) Project [codegen id : 4] +Output [3]: [cs_sales_price#2, cs_sold_date_sk#3, ca_zip#9] +Input [6]: [cs_sales_price#2, cs_sold_date_sk#3, c_current_addr_sk#6, ca_address_sk#7, ca_state#8, ca_zip#9] + +(16) ReusedExchange [Reuses operator id: 27] +Output [1]: [d_date_sk#10] + +(17) BroadcastHashJoin [codegen id : 4] +Left keys [1]: [cs_sold_date_sk#3] +Right keys [1]: [d_date_sk#10] +Join type: Inner +Join condition: None + +(18) Project [codegen id : 4] +Output [2]: [cs_sales_price#2, ca_zip#9] +Input [4]: [cs_sales_price#2, cs_sold_date_sk#3, ca_zip#9, d_date_sk#10] + +(19) HashAggregate [codegen id : 4] +Input [2]: [cs_sales_price#2, ca_zip#9] +Keys [1]: [ca_zip#9] +Functions [1]: [partial_sum(UnscaledValue(cs_sales_price#2))] +Aggregate Attributes [1]: [sum#11] +Results [2]: [ca_zip#9, sum#12] + +(20) Exchange +Input [2]: [ca_zip#9, sum#12] +Arguments: hashpartitioning(ca_zip#9, 5), ENSURE_REQUIREMENTS, [plan_id=3] + +(21) HashAggregate [codegen id : 5] +Input [2]: [ca_zip#9, sum#12] +Keys [1]: [ca_zip#9] +Functions [1]: [sum(UnscaledValue(cs_sales_price#2))] +Aggregate Attributes [1]: [sum(UnscaledValue(cs_sales_price#2))#13] +Results [2]: [ca_zip#9, MakeDecimal(sum(UnscaledValue(cs_sales_price#2))#13,17,2) AS sum(cs_sales_price)#14] + +(22) TakeOrderedAndProject +Input [2]: [ca_zip#9, sum(cs_sales_price)#14] +Arguments: 100, [ca_zip#9 ASC NULLS FIRST], [ca_zip#9, sum(cs_sales_price)#14] + +===== Subqueries ===== + +Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#3 IN dynamicpruning#4 +BroadcastExchange (27) ++- * ColumnarToRow (26) + +- CometProject (25) + +- CometFilter (24) + +- CometScan parquet spark_catalog.default.date_dim (23) + + +(23) Scan parquet spark_catalog.default.date_dim +Output [3]: [d_date_sk#10, d_year#15, d_qoy#16] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_qoy), IsNotNull(d_year), EqualTo(d_qoy,2), EqualTo(d_year,2001), IsNotNull(d_date_sk)] +ReadSchema: struct<d_date_sk:int,d_year:int,d_qoy:int> + +(24) CometFilter +Input [3]: [d_date_sk#10, d_year#15, d_qoy#16] +Condition : ((((isnotnull(d_qoy#16) AND isnotnull(d_year#15)) AND (d_qoy#16 = 2)) AND (d_year#15 = 2001)) AND isnotnull(d_date_sk#10)) + +(25) CometProject +Input [3]: [d_date_sk#10, d_year#15, d_qoy#16] +Arguments: [d_date_sk#10], [d_date_sk#10] + +(26) ColumnarToRow [codegen id : 1] +Input [1]: [d_date_sk#10] + +(27) BroadcastExchange +Input [1]: [d_date_sk#10] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] + + diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q15/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q15/simplified.txt new file mode 100644 index 000000000..5c750b2db --- /dev/null +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q15/simplified.txt @@ -0,0 +1,41 @@ +TakeOrderedAndProject [ca_zip,sum(cs_sales_price)] + WholeStageCodegen (5) + HashAggregate [ca_zip,sum] [sum(UnscaledValue(cs_sales_price)),sum(cs_sales_price),sum] + InputAdapter + Exchange [ca_zip] #1 + WholeStageCodegen (4) + HashAggregate [ca_zip,cs_sales_price] [sum,sum] + Project [cs_sales_price,ca_zip] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + Project [cs_sales_price,cs_sold_date_sk,ca_zip] + BroadcastHashJoin [c_current_addr_sk,ca_address_sk,ca_zip,ca_state,cs_sales_price] + Project [cs_sales_price,cs_sold_date_sk,c_current_addr_sk] + BroadcastHashJoin [cs_bill_customer_sk,c_customer_sk] + ColumnarToRow + InputAdapter + CometFilter [cs_bill_customer_sk] + CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_sales_price,cs_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #2 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_qoy,d_year,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] + InputAdapter + BroadcastExchange #3 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometFilter [c_customer_sk,c_current_addr_sk] + CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk] + InputAdapter + BroadcastExchange #4 + WholeStageCodegen (2) + ColumnarToRow + InputAdapter + CometFilter [ca_address_sk] + CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state,ca_zip] + InputAdapter + ReusedExchange [d_date_sk] #2 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q16/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q16/explain.txt new file mode 100644 index 000000000..ccec341ad --- /dev/null +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q16/explain.txt @@ -0,0 +1,260 @@ +== Physical Plan == +* HashAggregate (45) ++- Exchange (44) + +- * HashAggregate (43) + +- * HashAggregate (42) + +- * HashAggregate (41) + +- * Project (40) + +- * BroadcastHashJoin Inner BuildRight (39) + :- * Project (33) + : +- * BroadcastHashJoin Inner BuildRight (32) + : :- * Project (26) + : : +- * BroadcastHashJoin Inner BuildRight (25) + : : :- * SortMergeJoin LeftAnti (19) + : : : :- * Project (13) + : : : : +- * SortMergeJoin LeftSemi (12) + : : : : :- * Sort (6) + : : : : : +- Exchange (5) + : : : : : +- * ColumnarToRow (4) + : : : : : +- CometProject (3) + : : : : : +- CometFilter (2) + : : : : : +- CometScan parquet spark_catalog.default.catalog_sales (1) + : : : : +- * Sort (11) + : : : : +- Exchange (10) + : : : : +- * ColumnarToRow (9) + : : : : +- CometProject (8) + : : : : +- CometScan parquet spark_catalog.default.catalog_sales (7) + : : : +- * Sort (18) + : : : +- Exchange (17) + : : : +- * ColumnarToRow (16) + : : : +- CometProject (15) + : : : +- CometScan parquet spark_catalog.default.catalog_returns (14) + : : +- BroadcastExchange (24) + : : +- * ColumnarToRow (23) + : : +- CometProject (22) + : : +- CometFilter (21) + : : +- CometScan parquet spark_catalog.default.date_dim (20) + : +- BroadcastExchange (31) + : +- * ColumnarToRow (30) + : +- CometProject (29) + : +- CometFilter (28) + : +- CometScan parquet spark_catalog.default.customer_address (27) + +- BroadcastExchange (38) + +- * ColumnarToRow (37) + +- CometProject (36) + +- CometFilter (35) + +- CometScan parquet spark_catalog.default.call_center (34) + + +(1) Scan parquet spark_catalog.default.catalog_sales +Output [8]: [cs_ship_date_sk#1, cs_ship_addr_sk#2, cs_call_center_sk#3, cs_warehouse_sk#4, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7, cs_sold_date_sk#8] +Batched: true +Location [not included in comparison]/{warehouse_dir}/catalog_sales] +PushedFilters: [IsNotNull(cs_ship_date_sk), IsNotNull(cs_ship_addr_sk), IsNotNull(cs_call_center_sk)] +ReadSchema: struct<cs_ship_date_sk:int,cs_ship_addr_sk:int,cs_call_center_sk:int,cs_warehouse_sk:int,cs_order_number:int,cs_ext_ship_cost:decimal(7,2),cs_net_profit:decimal(7,2)> + +(2) CometFilter +Input [8]: [cs_ship_date_sk#1, cs_ship_addr_sk#2, cs_call_center_sk#3, cs_warehouse_sk#4, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7, cs_sold_date_sk#8] +Condition : ((isnotnull(cs_ship_date_sk#1) AND isnotnull(cs_ship_addr_sk#2)) AND isnotnull(cs_call_center_sk#3)) + +(3) CometProject +Input [8]: [cs_ship_date_sk#1, cs_ship_addr_sk#2, cs_call_center_sk#3, cs_warehouse_sk#4, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7, cs_sold_date_sk#8] +Arguments: [cs_ship_date_sk#1, cs_ship_addr_sk#2, cs_call_center_sk#3, cs_warehouse_sk#4, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7], [cs_ship_date_sk#1, cs_ship_addr_sk#2, cs_call_center_sk#3, cs_warehouse_sk#4, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7] + +(4) ColumnarToRow [codegen id : 1] +Input [7]: [cs_ship_date_sk#1, cs_ship_addr_sk#2, cs_call_center_sk#3, cs_warehouse_sk#4, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7] + +(5) Exchange +Input [7]: [cs_ship_date_sk#1, cs_ship_addr_sk#2, cs_call_center_sk#3, cs_warehouse_sk#4, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7] +Arguments: hashpartitioning(cs_order_number#5, 5), ENSURE_REQUIREMENTS, [plan_id=1] + +(6) Sort [codegen id : 2] +Input [7]: [cs_ship_date_sk#1, cs_ship_addr_sk#2, cs_call_center_sk#3, cs_warehouse_sk#4, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7] +Arguments: [cs_order_number#5 ASC NULLS FIRST], false, 0 + +(7) Scan parquet spark_catalog.default.catalog_sales +Output [3]: [cs_warehouse_sk#9, cs_order_number#10, cs_sold_date_sk#11] +Batched: true +Location [not included in comparison]/{warehouse_dir}/catalog_sales] +ReadSchema: struct<cs_warehouse_sk:int,cs_order_number:int> + +(8) CometProject +Input [3]: [cs_warehouse_sk#9, cs_order_number#10, cs_sold_date_sk#11] +Arguments: [cs_warehouse_sk#9, cs_order_number#10], [cs_warehouse_sk#9, cs_order_number#10] + +(9) ColumnarToRow [codegen id : 3] +Input [2]: [cs_warehouse_sk#9, cs_order_number#10] + +(10) Exchange +Input [2]: [cs_warehouse_sk#9, cs_order_number#10] +Arguments: hashpartitioning(cs_order_number#10, 5), ENSURE_REQUIREMENTS, [plan_id=2] + +(11) Sort [codegen id : 4] +Input [2]: [cs_warehouse_sk#9, cs_order_number#10] +Arguments: [cs_order_number#10 ASC NULLS FIRST], false, 0 + +(12) SortMergeJoin [codegen id : 5] +Left keys [1]: [cs_order_number#5] +Right keys [1]: [cs_order_number#10] +Join type: LeftSemi +Join condition: NOT (cs_warehouse_sk#4 = cs_warehouse_sk#9) + +(13) Project [codegen id : 5] +Output [6]: [cs_ship_date_sk#1, cs_ship_addr_sk#2, cs_call_center_sk#3, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7] +Input [7]: [cs_ship_date_sk#1, cs_ship_addr_sk#2, cs_call_center_sk#3, cs_warehouse_sk#4, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7] + +(14) Scan parquet spark_catalog.default.catalog_returns +Output [2]: [cr_order_number#12, cr_returned_date_sk#13] +Batched: true +Location [not included in comparison]/{warehouse_dir}/catalog_returns] +ReadSchema: struct<cr_order_number:int> + +(15) CometProject +Input [2]: [cr_order_number#12, cr_returned_date_sk#13] +Arguments: [cr_order_number#12], [cr_order_number#12] + +(16) ColumnarToRow [codegen id : 6] +Input [1]: [cr_order_number#12] + +(17) Exchange +Input [1]: [cr_order_number#12] +Arguments: hashpartitioning(cr_order_number#12, 5), ENSURE_REQUIREMENTS, [plan_id=3] + +(18) Sort [codegen id : 7] +Input [1]: [cr_order_number#12] +Arguments: [cr_order_number#12 ASC NULLS FIRST], false, 0 + +(19) SortMergeJoin [codegen id : 11] +Left keys [1]: [cs_order_number#5] +Right keys [1]: [cr_order_number#12] +Join type: LeftAnti +Join condition: None + +(20) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#14, d_date#15] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,2002-02-01), LessThanOrEqual(d_date,2002-04-02), IsNotNull(d_date_sk)] +ReadSchema: struct<d_date_sk:int,d_date:date> + +(21) CometFilter +Input [2]: [d_date_sk#14, d_date#15] +Condition : (((isnotnull(d_date#15) AND (d_date#15 >= 2002-02-01)) AND (d_date#15 <= 2002-04-02)) AND isnotnull(d_date_sk#14)) + +(22) CometProject +Input [2]: [d_date_sk#14, d_date#15] +Arguments: [d_date_sk#14], [d_date_sk#14] + +(23) ColumnarToRow [codegen id : 8] +Input [1]: [d_date_sk#14] + +(24) BroadcastExchange +Input [1]: [d_date_sk#14] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] + +(25) BroadcastHashJoin [codegen id : 11] +Left keys [1]: [cs_ship_date_sk#1] +Right keys [1]: [d_date_sk#14] +Join type: Inner +Join condition: None + +(26) Project [codegen id : 11] +Output [5]: [cs_ship_addr_sk#2, cs_call_center_sk#3, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7] +Input [7]: [cs_ship_date_sk#1, cs_ship_addr_sk#2, cs_call_center_sk#3, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7, d_date_sk#14] + +(27) Scan parquet spark_catalog.default.customer_address +Output [2]: [ca_address_sk#16, ca_state#17] +Batched: true +Location [not included in comparison]/{warehouse_dir}/customer_address] +PushedFilters: [IsNotNull(ca_state), EqualTo(ca_state,GA), IsNotNull(ca_address_sk)] +ReadSchema: struct<ca_address_sk:int,ca_state:string> + +(28) CometFilter +Input [2]: [ca_address_sk#16, ca_state#17] +Condition : ((isnotnull(ca_state#17) AND (ca_state#17 = GA)) AND isnotnull(ca_address_sk#16)) + +(29) CometProject +Input [2]: [ca_address_sk#16, ca_state#17] +Arguments: [ca_address_sk#16], [ca_address_sk#16] + +(30) ColumnarToRow [codegen id : 9] +Input [1]: [ca_address_sk#16] + +(31) BroadcastExchange +Input [1]: [ca_address_sk#16] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] + +(32) BroadcastHashJoin [codegen id : 11] +Left keys [1]: [cs_ship_addr_sk#2] +Right keys [1]: [ca_address_sk#16] +Join type: Inner +Join condition: None + +(33) Project [codegen id : 11] +Output [4]: [cs_call_center_sk#3, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7] +Input [6]: [cs_ship_addr_sk#2, cs_call_center_sk#3, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7, ca_address_sk#16] + +(34) Scan parquet spark_catalog.default.call_center +Output [2]: [cc_call_center_sk#18, cc_county#19] +Batched: true +Location [not included in comparison]/{warehouse_dir}/call_center] +PushedFilters: [IsNotNull(cc_county), EqualTo(cc_county,Williamson County), IsNotNull(cc_call_center_sk)] +ReadSchema: struct<cc_call_center_sk:int,cc_county:string> + +(35) CometFilter +Input [2]: [cc_call_center_sk#18, cc_county#19] +Condition : ((isnotnull(cc_county#19) AND (cc_county#19 = Williamson County)) AND isnotnull(cc_call_center_sk#18)) + +(36) CometProject +Input [2]: [cc_call_center_sk#18, cc_county#19] +Arguments: [cc_call_center_sk#18], [cc_call_center_sk#18] + +(37) ColumnarToRow [codegen id : 10] +Input [1]: [cc_call_center_sk#18] + +(38) BroadcastExchange +Input [1]: [cc_call_center_sk#18] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=6] + +(39) BroadcastHashJoin [codegen id : 11] +Left keys [1]: [cs_call_center_sk#3] +Right keys [1]: [cc_call_center_sk#18] +Join type: Inner +Join condition: None + +(40) Project [codegen id : 11] +Output [3]: [cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7] +Input [5]: [cs_call_center_sk#3, cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7, cc_call_center_sk#18] + +(41) HashAggregate [codegen id : 11] +Input [3]: [cs_order_number#5, cs_ext_ship_cost#6, cs_net_profit#7] +Keys [1]: [cs_order_number#5] +Functions [2]: [partial_sum(UnscaledValue(cs_ext_ship_cost#6)), partial_sum(UnscaledValue(cs_net_profit#7))] +Aggregate Attributes [2]: [sum(UnscaledValue(cs_ext_ship_cost#6))#20, sum(UnscaledValue(cs_net_profit#7))#21] +Results [3]: [cs_order_number#5, sum#22, sum#23] + +(42) HashAggregate [codegen id : 11] +Input [3]: [cs_order_number#5, sum#22, sum#23] +Keys [1]: [cs_order_number#5] +Functions [2]: [merge_sum(UnscaledValue(cs_ext_ship_cost#6)), merge_sum(UnscaledValue(cs_net_profit#7))] +Aggregate Attributes [2]: [sum(UnscaledValue(cs_ext_ship_cost#6))#20, sum(UnscaledValue(cs_net_profit#7))#21] +Results [3]: [cs_order_number#5, sum#22, sum#23] + +(43) HashAggregate [codegen id : 11] +Input [3]: [cs_order_number#5, sum#22, sum#23] +Keys: [] +Functions [3]: [merge_sum(UnscaledValue(cs_ext_ship_cost#6)), merge_sum(UnscaledValue(cs_net_profit#7)), partial_count(distinct cs_order_number#5)] +Aggregate Attributes [3]: [sum(UnscaledValue(cs_ext_ship_cost#6))#20, sum(UnscaledValue(cs_net_profit#7))#21, count(cs_order_number#5)#24] +Results [3]: [sum#22, sum#23, count#25] + +(44) Exchange +Input [3]: [sum#22, sum#23, count#25] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=7] + +(45) HashAggregate [codegen id : 12] +Input [3]: [sum#22, sum#23, count#25] +Keys: [] +Functions [3]: [sum(UnscaledValue(cs_ext_ship_cost#6)), sum(UnscaledValue(cs_net_profit#7)), count(distinct cs_order_number#5)] +Aggregate Attributes [3]: [sum(UnscaledValue(cs_ext_ship_cost#6))#20, sum(UnscaledValue(cs_net_profit#7))#21, count(cs_order_number#5)#24] +Results [3]: [count(cs_order_number#5)#24 AS order count #26, MakeDecimal(sum(UnscaledValue(cs_ext_ship_cost#6))#20,17,2) AS total shipping cost #27, MakeDecimal(sum(UnscaledValue(cs_net_profit#7))#21,17,2) AS total net profit #28] + diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q16/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q16/simplified.txt new file mode 100644 index 000000000..a55c182be --- /dev/null +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q16/simplified.txt @@ -0,0 +1,74 @@ +WholeStageCodegen (12) + HashAggregate [sum,sum,count] [sum(UnscaledValue(cs_ext_ship_cost)),sum(UnscaledValue(cs_net_profit)),count(cs_order_number),order count ,total shipping cost ,total net profit ,sum,sum,count] + InputAdapter + Exchange #1 + WholeStageCodegen (11) + HashAggregate [cs_order_number] [sum(UnscaledValue(cs_ext_ship_cost)),sum(UnscaledValue(cs_net_profit)),count(cs_order_number),sum,sum,count,sum,sum,count] + HashAggregate [cs_order_number] [sum(UnscaledValue(cs_ext_ship_cost)),sum(UnscaledValue(cs_net_profit)),sum,sum,sum,sum] + HashAggregate [cs_order_number,cs_ext_ship_cost,cs_net_profit] [sum(UnscaledValue(cs_ext_ship_cost)),sum(UnscaledValue(cs_net_profit)),sum,sum,sum,sum] + Project [cs_order_number,cs_ext_ship_cost,cs_net_profit] + BroadcastHashJoin [cs_call_center_sk,cc_call_center_sk] + Project [cs_call_center_sk,cs_order_number,cs_ext_ship_cost,cs_net_profit] + BroadcastHashJoin [cs_ship_addr_sk,ca_address_sk] + Project [cs_ship_addr_sk,cs_call_center_sk,cs_order_number,cs_ext_ship_cost,cs_net_profit] + BroadcastHashJoin [cs_ship_date_sk,d_date_sk] + SortMergeJoin [cs_order_number,cr_order_number] + InputAdapter + WholeStageCodegen (5) + Project [cs_ship_date_sk,cs_ship_addr_sk,cs_call_center_sk,cs_order_number,cs_ext_ship_cost,cs_net_profit] + SortMergeJoin [cs_order_number,cs_order_number,cs_warehouse_sk,cs_warehouse_sk] + InputAdapter + WholeStageCodegen (2) + Sort [cs_order_number] + InputAdapter + Exchange [cs_order_number] #2 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + 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] + CometScan parquet spark_catalog.default.catalog_sales [cs_ship_date_sk,cs_ship_addr_sk,cs_call_center_sk,cs_warehouse_sk,cs_order_number,cs_ext_ship_cost,cs_net_profit,cs_sold_date_sk] + InputAdapter + WholeStageCodegen (4) + Sort [cs_order_number] + InputAdapter + Exchange [cs_order_number] #3 + WholeStageCodegen (3) + ColumnarToRow + InputAdapter + CometProject [cs_warehouse_sk,cs_order_number] + CometScan parquet spark_catalog.default.catalog_sales [cs_warehouse_sk,cs_order_number,cs_sold_date_sk] + InputAdapter + WholeStageCodegen (7) + Sort [cr_order_number] + InputAdapter + Exchange [cr_order_number] #4 + WholeStageCodegen (6) + ColumnarToRow + InputAdapter + CometProject [cr_order_number] + CometScan parquet spark_catalog.default.catalog_returns [cr_order_number,cr_returned_date_sk] + InputAdapter + BroadcastExchange #5 + WholeStageCodegen (8) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] + InputAdapter + BroadcastExchange #6 + WholeStageCodegen (9) + ColumnarToRow + InputAdapter + CometProject [ca_address_sk] + CometFilter [ca_state,ca_address_sk] + CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] + InputAdapter + BroadcastExchange #7 + WholeStageCodegen (10) + ColumnarToRow + InputAdapter + CometProject [cc_call_center_sk] + CometFilter [cc_county,cc_call_center_sk] + CometScan parquet spark_catalog.default.call_center [cc_call_center_sk,cc_county] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q17/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q17/explain.txt new file mode 100644 index 000000000..0de98cfb0 --- /dev/null +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q17/explain.txt @@ -0,0 +1,298 @@ +== Physical Plan == +TakeOrderedAndProject (40) ++- * HashAggregate (39) + +- Exchange (38) + +- * HashAggregate (37) + +- * Project (36) + +- * BroadcastHashJoin Inner BuildRight (35) + :- * Project (30) + : +- * BroadcastHashJoin Inner BuildRight (29) + : :- * Project (24) + : : +- * BroadcastHashJoin Inner BuildRight (23) + : : :- * Project (21) + : : : +- * BroadcastHashJoin Inner BuildRight (20) + : : : :- * Project (18) + : : : : +- * BroadcastHashJoin Inner BuildRight (17) + : : : : :- * Project (15) + : : : : : +- * BroadcastHashJoin Inner BuildRight (14) + : : : : : :- * Project (9) + : : : : : : +- * BroadcastHashJoin Inner BuildRight (8) + : : : : : : :- * ColumnarToRow (3) + : : : : : : : +- CometFilter (2) + : : : : : : : +- CometScan parquet spark_catalog.default.store_sales (1) + : : : : : : +- BroadcastExchange (7) + : : : : : : +- * ColumnarToRow (6) + : : : : : : +- CometFilter (5) + : : : : : : +- CometScan parquet spark_catalog.default.store_returns (4) + : : : : : +- BroadcastExchange (13) + : : : : : +- * ColumnarToRow (12) + : : : : : +- CometFilter (11) + : : : : : +- CometScan parquet spark_catalog.default.catalog_sales (10) + : : : : +- ReusedExchange (16) + : : : +- ReusedExchange (19) + : : +- ReusedExchange (22) + : +- BroadcastExchange (28) + : +- * ColumnarToRow (27) + : +- CometFilter (26) + : +- CometScan parquet spark_catalog.default.store (25) + +- BroadcastExchange (34) + +- * ColumnarToRow (33) + +- CometFilter (32) + +- CometScan parquet spark_catalog.default.item (31) + + +(1) Scan parquet spark_catalog.default.store_sales +Output [6]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_quantity#5, ss_sold_date_sk#6] +Batched: true +Location: InMemoryFileIndex [] +PartitionFilters: [isnotnull(ss_sold_date_sk#6), dynamicpruningexpression(ss_sold_date_sk#6 IN dynamicpruning#7)] +PushedFilters: [IsNotNull(ss_customer_sk), IsNotNull(ss_item_sk), IsNotNull(ss_ticket_number), IsNotNull(ss_store_sk)] +ReadSchema: struct<ss_item_sk:int,ss_customer_sk:int,ss_store_sk:int,ss_ticket_number:int,ss_quantity:int> + +(2) CometFilter +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)) + +(3) ColumnarToRow [codegen id : 8] +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] + +(4) Scan parquet spark_catalog.default.store_returns +Output [5]: [sr_item_sk#8, sr_customer_sk#9, sr_ticket_number#10, sr_return_quantity#11, sr_returned_date_sk#12] +Batched: true +Location: InMemoryFileIndex [] +PartitionFilters: [isnotnull(sr_returned_date_sk#12), dynamicpruningexpression(sr_returned_date_sk#12 IN dynamicpruning#13)] +PushedFilters: [IsNotNull(sr_customer_sk), IsNotNull(sr_item_sk), IsNotNull(sr_ticket_number)] +ReadSchema: struct<sr_item_sk:int,sr_customer_sk:int,sr_ticket_number:int,sr_return_quantity:int> + +(5) CometFilter +Input [5]: [sr_item_sk#8, sr_customer_sk#9, sr_ticket_number#10, sr_return_quantity#11, sr_returned_date_sk#12] +Condition : ((isnotnull(sr_customer_sk#9) AND isnotnull(sr_item_sk#8)) AND isnotnull(sr_ticket_number#10)) + +(6) ColumnarToRow [codegen id : 1] +Input [5]: [sr_item_sk#8, sr_customer_sk#9, sr_ticket_number#10, sr_return_quantity#11, sr_returned_date_sk#12] + +(7) BroadcastExchange +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] +Left keys [3]: [ss_customer_sk#2, ss_item_sk#1, ss_ticket_number#4] +Right keys [3]: [sr_customer_sk#9, sr_item_sk#8, sr_ticket_number#10] +Join type: Inner +Join condition: None + +(9) Project [codegen id : 8] +Output [8]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, ss_sold_date_sk#6, sr_item_sk#8, sr_customer_sk#9, sr_return_quantity#11, sr_returned_date_sk#12] +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] + +(10) Scan parquet spark_catalog.default.catalog_sales +Output [4]: [cs_bill_customer_sk#14, cs_item_sk#15, cs_quantity#16, cs_sold_date_sk#17] +Batched: true +Location: InMemoryFileIndex [] +PartitionFilters: [isnotnull(cs_sold_date_sk#17), dynamicpruningexpression(cs_sold_date_sk#17 IN dynamicpruning#18)] +PushedFilters: [IsNotNull(cs_bill_customer_sk), IsNotNull(cs_item_sk)] +ReadSchema: struct<cs_bill_customer_sk:int,cs_item_sk:int,cs_quantity:int> + +(11) CometFilter +Input [4]: [cs_bill_customer_sk#14, cs_item_sk#15, cs_quantity#16, cs_sold_date_sk#17] +Condition : (isnotnull(cs_bill_customer_sk#14) AND isnotnull(cs_item_sk#15)) + +(12) ColumnarToRow [codegen id : 2] +Input [4]: [cs_bill_customer_sk#14, cs_item_sk#15, cs_quantity#16, cs_sold_date_sk#17] + +(13) BroadcastExchange +Input [4]: [cs_bill_customer_sk#14, cs_item_sk#15, cs_quantity#16, cs_sold_date_sk#17] +Arguments: HashedRelationBroadcastMode(List((shiftleft(cast(input[0, int, false] as bigint), 32) | (cast(input[1, int, false] as bigint) & 4294967295))),false), [plan_id=2] + +(14) BroadcastHashJoin [codegen id : 8] +Left keys [2]: [sr_customer_sk#9, sr_item_sk#8] +Right keys [2]: [cs_bill_customer_sk#14, cs_item_sk#15] +Join type: Inner +Join condition: None + +(15) Project [codegen id : 8] +Output [8]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, ss_sold_date_sk#6, sr_return_quantity#11, sr_returned_date_sk#12, cs_quantity#16, cs_sold_date_sk#17] +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: 45] +Output [1]: [d_date_sk#19] + +(17) BroadcastHashJoin [codegen id : 8] +Left keys [1]: [ss_sold_date_sk#6] +Right keys [1]: [d_date_sk#19] +Join type: Inner +Join condition: None + +(18) Project [codegen id : 8] +Output [7]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, sr_return_quantity#11, sr_returned_date_sk#12, cs_quantity#16, cs_sold_date_sk#17] +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: 50] +Output [1]: [d_date_sk#20] + +(20) BroadcastHashJoin [codegen id : 8] +Left keys [1]: [sr_returned_date_sk#12] +Right keys [1]: [d_date_sk#20] +Join type: Inner +Join condition: None + +(21) Project [codegen id : 8] +Output [6]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, sr_return_quantity#11, cs_quantity#16, cs_sold_date_sk#17] +Input [8]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, sr_return_quantity#11, sr_returned_date_sk#12, cs_quantity#16, cs_sold_date_sk#17, d_date_sk#20] + +(22) ReusedExchange [Reuses operator id: 50] +Output [1]: [d_date_sk#21] + +(23) BroadcastHashJoin [codegen id : 8] +Left keys [1]: [cs_sold_date_sk#17] +Right keys [1]: [d_date_sk#21] +Join type: Inner +Join condition: None + +(24) Project [codegen id : 8] +Output [5]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, sr_return_quantity#11, cs_quantity#16] +Input [7]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, sr_return_quantity#11, cs_quantity#16, cs_sold_date_sk#17, d_date_sk#21] + +(25) Scan parquet spark_catalog.default.store +Output [2]: [s_store_sk#22, s_state#23] +Batched: true +Location [not included in comparison]/{warehouse_dir}/store] +PushedFilters: [IsNotNull(s_store_sk)] +ReadSchema: struct<s_store_sk:int,s_state:string> + +(26) CometFilter +Input [2]: [s_store_sk#22, s_state#23] +Condition : isnotnull(s_store_sk#22) + +(27) ColumnarToRow [codegen id : 6] +Input [2]: [s_store_sk#22, s_state#23] + +(28) BroadcastExchange +Input [2]: [s_store_sk#22, s_state#23] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=3] + +(29) BroadcastHashJoin [codegen id : 8] +Left keys [1]: [ss_store_sk#3] +Right keys [1]: [s_store_sk#22] +Join type: Inner +Join condition: None + +(30) Project [codegen id : 8] +Output [5]: [ss_item_sk#1, ss_quantity#5, sr_return_quantity#11, cs_quantity#16, s_state#23] +Input [7]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, sr_return_quantity#11, cs_quantity#16, s_store_sk#22, s_state#23] + +(31) Scan parquet spark_catalog.default.item +Output [3]: [i_item_sk#24, i_item_id#25, i_item_desc#26] +Batched: true +Location [not included in comparison]/{warehouse_dir}/item] +PushedFilters: [IsNotNull(i_item_sk)] +ReadSchema: struct<i_item_sk:int,i_item_id:string,i_item_desc:string> + +(32) CometFilter +Input [3]: [i_item_sk#24, i_item_id#25, i_item_desc#26] +Condition : isnotnull(i_item_sk#24) + +(33) ColumnarToRow [codegen id : 7] +Input [3]: [i_item_sk#24, i_item_id#25, i_item_desc#26] + +(34) BroadcastExchange +Input [3]: [i_item_sk#24, i_item_id#25, i_item_desc#26] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=4] + +(35) BroadcastHashJoin [codegen id : 8] +Left keys [1]: [ss_item_sk#1] +Right keys [1]: [i_item_sk#24] +Join type: Inner +Join condition: None + +(36) Project [codegen id : 8] +Output [6]: [ss_quantity#5, sr_return_quantity#11, cs_quantity#16, s_state#23, i_item_id#25, i_item_desc#26] +Input [8]: [ss_item_sk#1, ss_quantity#5, sr_return_quantity#11, cs_quantity#16, s_state#23, i_item_sk#24, i_item_id#25, i_item_desc#26] + +(37) HashAggregate [codegen id : 8] +Input [6]: [ss_quantity#5, sr_return_quantity#11, cs_quantity#16, s_state#23, i_item_id#25, i_item_desc#26] +Keys [3]: [i_item_id#25, i_item_desc#26, s_state#23] +Functions [9]: [partial_count(ss_quantity#5), partial_avg(ss_quantity#5), partial_stddev_samp(cast(ss_quantity#5 as double)), partial_count(sr_return_quantity#11), partial_avg(sr_return_quantity#11), partial_stddev_samp(cast(sr_return_quantity#11 as double)), partial_count(cs_quantity#16), partial_avg(cs_quantity#16), partial_stddev_samp(cast(cs_quantity#16 as double))] +Aggregate Attributes [18]: [count#27, sum#28, count#29, n#30, avg#31, m2#32, count#33, sum#34, count#35, n#36, avg#37, m2#38, count#39, sum#40, count#41, n#42, avg#43, m2#44] +Results [21]: [i_item_id#25, i_item_desc#26, s_state#23, count#45, sum#46, count#47, n#48, avg#49, m2#50, count#51, sum#52, count#53, n#54, avg#55, m2#56, count#57, sum#58, count#59, n#60, avg#61, m2#62] + +(38) Exchange +Input [21]: [i_item_id#25, i_item_desc#26, s_state#23, count#45, sum#46, count#47, n#48, avg#49, m2#50, count#51, sum#52, count#53, n#54, avg#55, m2#56, count#57, sum#58, count#59, n#60, avg#61, m2#62] +Arguments: hashpartitioning(i_item_id#25, i_item_desc#26, s_state#23, 5), ENSURE_REQUIREMENTS, [plan_id=5] + +(39) HashAggregate [codegen id : 9] +Input [21]: [i_item_id#25, i_item_desc#26, s_state#23, count#45, sum#46, count#47, n#48, avg#49, m2#50, count#51, sum#52, count#53, n#54, avg#55, m2#56, count#57, sum#58, count#59, n#60, avg#61, m2#62] +Keys [3]: [i_item_id#25, i_item_desc#26, s_state#23] +Functions [9]: [count(ss_quantity#5), avg(ss_quantity#5), stddev_samp(cast(ss_quantity#5 as double)), count(sr_return_quantity#11), avg(sr_return_quantity#11), stddev_samp(cast(sr_return_quantity#11 as double)), count(cs_quantity#16), avg(cs_quantity#16), stddev_samp(cast(cs_quantity#16 as double))] +Aggregate Attributes [9]: [count(ss_quantity#5)#63, avg(ss_quantity#5)#64, stddev_samp(cast(ss_quantity#5 as double))#65, count(sr_return_quantity#11)#66, avg(sr_return_quantity#11)#67, stddev_samp(cast(sr_return_quantity#11 as double))#68, count(cs_quantity#16)#69, avg(cs_quantity#16)#70, stddev_samp(cast(cs_quantity#16 as double))#71] +Results [15]: [i_item_id#25, i_item_desc#26, s_state#23, count(ss_quantity#5)#63 AS store_sales_quantitycount#72, avg(ss_quantity#5)#64 AS store_sales_quantityave#73, stddev_samp(cast(ss_quantity#5 as double))#65 AS store_sales_quantitystdev#74, (stddev_samp(cast(ss_quantity#5 as double))#65 / avg(ss_quantity#5)#64) AS store_sales_quantitycov#75, count(sr_return_quantity#11)#66 AS as_store_returns_quantitycount#76, avg(sr_return_quantity#11)#67 AS as_store_returns_quantityave#77, stddev_samp(cast(sr_return_quantity#11 as double))#68 AS as_store_returns_quantitystdev#78, (stddev_samp(cast(sr_return_quantity#11 as double))#68 / avg(sr_return_quantity#11)#67) AS store_returns_quantitycov#79, count(cs_quantity#16)#69 AS catalog_sales_quantitycount#80, avg(cs_quantity#16)#70 AS catalog_sales_quantityave#81, (stddev_samp(cast(cs_quantity#16 as double))#71 / avg(cs_quantity#16)#70) AS catalog_sales_quantitystdev#82, (stddev_samp(cast(cs_quantity#16 as double))#71 / avg(cs_quantity#16)#70) AS catalog_sales_quantitycov#83] + +(40) TakeOrderedAndProject +Input [15]: [i_item_id#25, i_item_desc#26, s_state#23, store_sales_quantitycount#72, store_sales_quantityave#73, store_sales_quantitystdev#74, store_sales_quantitycov#75, as_store_returns_quantitycount#76, as_store_returns_quantityave#77, as_store_returns_quantitystdev#78, store_returns_quantitycov#79, catalog_sales_quantitycount#80, catalog_sales_quantityave#81, catalog_sales_quantitystdev#82, catalog_sales_quantitycov#83] +Arguments: 100, [i_item_id#25 ASC NULLS FIRST, i_item_desc#26 ASC NULLS FIRST, s_state#23 ASC NULLS FIRST], [i_item_id#25, i_item_desc#26, s_state#23, store_sales_quantitycount#72, store_sales_quantityave#73, store_sales_quantitystdev#74, store_sales_quantitycov#75, as_store_returns_quantitycount#76, as_store_returns_quantityave#77, as_store_returns_quantitystdev#78, store_returns_quantitycov#79, catalog_sales_quantitycount#80, catalog_sales_quantityave#81, catalog_sales_quantitystdev#82, catalog_sales_quantitycov#83] + +===== Subqueries ===== + +Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#6 IN dynamicpruning#7 +BroadcastExchange (45) ++- * ColumnarToRow (44) + +- CometProject (43) + +- CometFilter (42) + +- CometScan parquet spark_catalog.default.date_dim (41) + + +(41) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#19, d_quarter_name#84] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_quarter_name), EqualTo(d_quarter_name,2001Q1), IsNotNull(d_date_sk)] +ReadSchema: struct<d_date_sk:int,d_quarter_name:string> + +(42) CometFilter +Input [2]: [d_date_sk#19, d_quarter_name#84] +Condition : ((isnotnull(d_quarter_name#84) AND (d_quarter_name#84 = 2001Q1)) AND isnotnull(d_date_sk#19)) + +(43) CometProject +Input [2]: [d_date_sk#19, d_quarter_name#84] +Arguments: [d_date_sk#19], [d_date_sk#19] + +(44) ColumnarToRow [codegen id : 1] +Input [1]: [d_date_sk#19] + +(45) BroadcastExchange +Input [1]: [d_date_sk#19] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=6] + +Subquery:2 Hosting operator id = 4 Hosting Expression = sr_returned_date_sk#12 IN dynamicpruning#13 +BroadcastExchange (50) ++- * ColumnarToRow (49) + +- CometProject (48) + +- CometFilter (47) + +- CometScan parquet spark_catalog.default.date_dim (46) + + +(46) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#20, d_quarter_name#85] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [In(d_quarter_name, [2001Q1,2001Q2,2001Q3]), IsNotNull(d_date_sk)] +ReadSchema: struct<d_date_sk:int,d_quarter_name:string> + +(47) CometFilter +Input [2]: [d_date_sk#20, d_quarter_name#85] +Condition : (d_quarter_name#85 IN (2001Q1,2001Q2,2001Q3) AND isnotnull(d_date_sk#20)) + +(48) CometProject +Input [2]: [d_date_sk#20, d_quarter_name#85] +Arguments: [d_date_sk#20], [d_date_sk#20] + +(49) ColumnarToRow [codegen id : 1] +Input [1]: [d_date_sk#20] + +(50) BroadcastExchange +Input [1]: [d_date_sk#20] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=7] + +Subquery:3 Hosting operator id = 10 Hosting Expression = cs_sold_date_sk#17 IN dynamicpruning#13 + + diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q17/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q17/simplified.txt new file mode 100644 index 000000000..9f4d67dec --- /dev/null +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q17/simplified.txt @@ -0,0 +1,76 @@ +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) + HashAggregate [i_item_id,i_item_desc,s_state,count,sum,count,n,avg,m2,count,sum,count,n,avg,m2,count,sum,count,n,avg,m2] [count(ss_quantity),avg(ss_quantity),stddev_samp(cast(ss_quantity as double)),count(sr_return_quantity),avg(sr_return_quantity),stddev_samp(cast(sr_return_quantity as double)),count(cs_quantity),avg(cs_quantity),stddev_samp(cast(cs_quantity as double)),store_sales_quantitycount,store_sales_quantityave,store_sales_quantitystdev,store_sales_quantitycov,as_store_returns_quantitycount,as_store_returns_quantityave,as_store_returns_quantitystdev,store_returns_quantitycov,catalog_sales_quantitycount,catalog_sales_quantityave,catalog_sales_quantitystdev,catalog_sales_quantitycov,count,sum,count,n,avg,m2,count,sum,count,n,avg,m2,count,sum,count,n,avg,m2] + InputAdapter + Exchange [i_item_id,i_item_desc,s_state] #1 + WholeStageCodegen (8) + HashAggregate [i_item_id,i_item_desc,s_state,ss_quantity,sr_return_quantity,cs_quantity] [count,sum,count,n,avg,m2,count,sum,count,n,avg,m2,count,sum,count,n,avg,m2,count,sum,count,n,avg,m2,count,sum,count,n,avg,m2,count,sum,count,n,avg,m2] + Project [ss_quantity,sr_return_quantity,cs_quantity,s_state,i_item_id,i_item_desc] + BroadcastHashJoin [ss_item_sk,i_item_sk] + Project [ss_item_sk,ss_quantity,sr_return_quantity,cs_quantity,s_state] + BroadcastHashJoin [ss_store_sk,s_store_sk] + Project [ss_item_sk,ss_store_sk,ss_quantity,sr_return_quantity,cs_quantity] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + Project [ss_item_sk,ss_store_sk,ss_quantity,sr_return_quantity,cs_quantity,cs_sold_date_sk] + BroadcastHashJoin [sr_returned_date_sk,d_date_sk] + Project [ss_item_sk,ss_store_sk,ss_quantity,sr_return_quantity,sr_returned_date_sk,cs_quantity,cs_sold_date_sk] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [ss_item_sk,ss_store_sk,ss_quantity,ss_sold_date_sk,sr_return_quantity,sr_returned_date_sk,cs_quantity,cs_sold_date_sk] + BroadcastHashJoin [sr_customer_sk,sr_item_sk,cs_bill_customer_sk,cs_item_sk] + Project [ss_item_sk,ss_store_sk,ss_quantity,ss_sold_date_sk,sr_item_sk,sr_customer_sk,sr_return_quantity,sr_returned_date_sk] + BroadcastHashJoin [ss_customer_sk,ss_item_sk,ss_ticket_number,sr_customer_sk,sr_item_sk,sr_ticket_number] + ColumnarToRow + InputAdapter + CometFilter [ss_customer_sk,ss_item_sk,ss_ticket_number,ss_store_sk] + CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_quantity,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #2 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_quarter_name,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_quarter_name] + InputAdapter + BroadcastExchange #3 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometFilter [sr_customer_sk,sr_item_sk,sr_ticket_number] + CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_customer_sk,sr_ticket_number,sr_return_quantity,sr_returned_date_sk] + SubqueryBroadcast [d_date_sk] #2 + BroadcastExchange #4 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_quarter_name,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_quarter_name] + InputAdapter + BroadcastExchange #5 + WholeStageCodegen (2) + ColumnarToRow + InputAdapter + CometFilter [cs_bill_customer_sk,cs_item_sk] + CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #2 + InputAdapter + ReusedExchange [d_date_sk] #2 + InputAdapter + ReusedExchange [d_date_sk] #4 + InputAdapter + ReusedExchange [d_date_sk] #4 + InputAdapter + BroadcastExchange #6 + WholeStageCodegen (6) + ColumnarToRow + InputAdapter + CometFilter [s_store_sk] + CometScan parquet spark_catalog.default.store [s_store_sk,s_state] + InputAdapter + BroadcastExchange #7 + WholeStageCodegen (7) + ColumnarToRow + InputAdapter + CometFilter [i_item_sk] + CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc] 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 new file mode 100644 index 000000000..1e9c660c5 --- /dev/null +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q18/explain.txt @@ -0,0 +1,281 @@ +== Physical Plan == +TakeOrderedAndProject (43) ++- * HashAggregate (42) + +- Exchange (41) + +- * HashAggregate (40) + +- * Expand (39) + +- * Project (38) + +- * BroadcastHashJoin Inner BuildRight (37) + :- * Project (32) + : +- * BroadcastHashJoin Inner BuildRight (31) + : :- * Project (29) + : : +- * BroadcastHashJoin Inner BuildRight (28) + : : :- * Project (23) + : : : +- * BroadcastHashJoin Inner BuildRight (22) + : : : :- * Project (17) + : : : : +- * BroadcastHashJoin Inner BuildRight (16) + : : : : :- * Project (10) + : : : : : +- * BroadcastHashJoin Inner BuildRight (9) + : : : : : :- * ColumnarToRow (3) + : : : : : : +- CometFilter (2) + : : : : : : +- CometScan parquet spark_catalog.default.catalog_sales (1) + : : : : : +- BroadcastExchange (8) + : : : : : +- * ColumnarToRow (7) + : : : : : +- CometProject (6) + : : : : : +- CometFilter (5) + : : : : : +- CometScan parquet spark_catalog.default.customer_demographics (4) + : : : : +- BroadcastExchange (15) + : : : : +- * ColumnarToRow (14) + : : : : +- CometProject (13) + : : : : +- CometFilter (12) + : : : : +- CometScan parquet spark_catalog.default.customer (11) + : : : +- BroadcastExchange (21) + : : : +- * ColumnarToRow (20) + : : : +- CometFilter (19) + : : : +- CometScan parquet spark_catalog.default.customer_demographics (18) + : : +- BroadcastExchange (27) + : : +- * ColumnarToRow (26) + : : +- CometFilter (25) + : : +- CometScan parquet spark_catalog.default.customer_address (24) + : +- ReusedExchange (30) + +- BroadcastExchange (36) + +- * ColumnarToRow (35) + +- CometFilter (34) + +- CometScan parquet spark_catalog.default.item (33) + + +(1) Scan parquet spark_catalog.default.catalog_sales +Output [9]: [cs_bill_customer_sk#1, cs_bill_cdemo_sk#2, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9] +Batched: true +Location: InMemoryFileIndex [] +PartitionFilters: [isnotnull(cs_sold_date_sk#9), dynamicpruningexpression(cs_sold_date_sk#9 IN dynamicpruning#10)] +PushedFilters: [IsNotNull(cs_bill_cdemo_sk), IsNotNull(cs_bill_customer_sk), IsNotNull(cs_item_sk)] +ReadSchema: struct<cs_bill_customer_sk:int,cs_bill_cdemo_sk:int,cs_item_sk:int,cs_quantity:int,cs_list_price:decimal(7,2),cs_sales_price:decimal(7,2),cs_coupon_amt:decimal(7,2),cs_net_profit:decimal(7,2)> + +(2) CometFilter +Input [9]: [cs_bill_customer_sk#1, cs_bill_cdemo_sk#2, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9] +Condition : ((isnotnull(cs_bill_cdemo_sk#2) AND isnotnull(cs_bill_customer_sk#1)) AND isnotnull(cs_item_sk#3)) + +(3) ColumnarToRow [codegen id : 7] +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] + +(4) Scan parquet spark_catalog.default.customer_demographics +Output [4]: [cd_demo_sk#11, cd_gender#12, cd_education_status#13, cd_dep_count#14] +Batched: true +Location [not included in comparison]/{warehouse_dir}/customer_demographics] +PushedFilters: [IsNotNull(cd_gender), IsNotNull(cd_education_status), EqualTo(cd_gender,F), EqualTo(cd_education_status,Unknown ), IsNotNull(cd_demo_sk)] +ReadSchema: struct<cd_demo_sk:int,cd_gender:string,cd_education_status:string,cd_dep_count:int> + +(5) CometFilter +Input [4]: [cd_demo_sk#11, cd_gender#12, cd_education_status#13, cd_dep_count#14] +Condition : ((((isnotnull(cd_gender#12) AND isnotnull(cd_education_status#13)) AND (cd_gender#12 = F)) AND (cd_education_status#13 = Unknown )) AND isnotnull(cd_demo_sk#11)) + +(6) CometProject +Input [4]: [cd_demo_sk#11, cd_gender#12, cd_education_status#13, cd_dep_count#14] +Arguments: [cd_demo_sk#11, cd_dep_count#14], [cd_demo_sk#11, cd_dep_count#14] + +(7) ColumnarToRow [codegen id : 1] +Input [2]: [cd_demo_sk#11, cd_dep_count#14] + +(8) BroadcastExchange +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] +Left keys [1]: [cs_bill_cdemo_sk#2] +Right keys [1]: [cd_demo_sk#11] +Join type: Inner +Join condition: None + +(10) Project [codegen id : 7] +Output [9]: [cs_bill_customer_sk#1, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14] +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] + +(11) Scan parquet spark_catalog.default.customer +Output [5]: [c_customer_sk#15, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_month#18, c_birth_year#19] +Batched: true +Location [not included in comparison]/{warehouse_dir}/customer] +PushedFilters: [In(c_birth_month, [1,12,2,6,8,9]), IsNotNull(c_customer_sk), IsNotNull(c_current_cdemo_sk), IsNotNull(c_current_addr_sk)] +ReadSchema: struct<c_customer_sk:int,c_current_cdemo_sk:int,c_current_addr_sk:int,c_birth_month:int,c_birth_year:int> + +(12) CometFilter +Input [5]: [c_customer_sk#15, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_month#18, c_birth_year#19] +Condition : (((c_birth_month#18 IN (1,6,8,9,12,2) AND isnotnull(c_customer_sk#15)) AND isnotnull(c_current_cdemo_sk#16)) AND isnotnull(c_current_addr_sk#17)) + +(13) CometProject +Input [5]: [c_customer_sk#15, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_month#18, c_birth_year#19] +Arguments: [c_customer_sk#15, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_year#19], [c_customer_sk#15, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_year#19] + +(14) ColumnarToRow [codegen id : 2] +Input [4]: [c_customer_sk#15, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_year#19] + +(15) BroadcastExchange +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] +Left keys [1]: [cs_bill_customer_sk#1] +Right keys [1]: [c_customer_sk#15] +Join type: Inner +Join condition: None + +(17) Project [codegen id : 7] +Output [11]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_year#19] +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] + +(18) Scan parquet spark_catalog.default.customer_demographics +Output [1]: [cd_demo_sk#20] +Batched: true +Location [not included in comparison]/{warehouse_dir}/customer_demographics] +PushedFilters: [IsNotNull(cd_demo_sk)] +ReadSchema: struct<cd_demo_sk:int> + +(19) CometFilter +Input [1]: [cd_demo_sk#20] +Condition : isnotnull(cd_demo_sk#20) + +(20) ColumnarToRow [codegen id : 3] +Input [1]: [cd_demo_sk#20] + +(21) BroadcastExchange +Input [1]: [cd_demo_sk#20] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=3] + +(22) BroadcastHashJoin [codegen id : 7] +Left keys [1]: [c_current_cdemo_sk#16] +Right keys [1]: [cd_demo_sk#20] +Join type: Inner +Join condition: None + +(23) Project [codegen id : 7] +Output [10]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_current_addr_sk#17, c_birth_year#19] +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] + +(24) Scan parquet spark_catalog.default.customer_address +Output [4]: [ca_address_sk#21, ca_county#22, ca_state#23, ca_country#24] +Batched: true +Location [not included in comparison]/{warehouse_dir}/customer_address] +PushedFilters: [In(ca_state, [IN,MS,ND,NM,OK,VA]), IsNotNull(ca_address_sk)] +ReadSchema: struct<ca_address_sk:int,ca_county:string,ca_state:string,ca_country:string> + +(25) CometFilter +Input [4]: [ca_address_sk#21, ca_county#22, ca_state#23, ca_country#24] +Condition : (ca_state#23 IN (MS,IN,ND,OK,NM,VA) AND isnotnull(ca_address_sk#21)) + +(26) ColumnarToRow [codegen id : 4] +Input [4]: [ca_address_sk#21, ca_county#22, ca_state#23, ca_country#24] + +(27) BroadcastExchange +Input [4]: [ca_address_sk#21, ca_county#22, ca_state#23, ca_country#24] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=4] + +(28) BroadcastHashJoin [codegen id : 7] +Left keys [1]: [c_current_addr_sk#17] +Right keys [1]: [ca_address_sk#21] +Join type: Inner +Join condition: None + +(29) Project [codegen id : 7] +Output [12]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_birth_year#19, ca_county#22, ca_state#23, ca_country#24] +Input [14]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_current_addr_sk#17, c_birth_year#19, ca_address_sk#21, ca_county#22, ca_state#23, ca_country#24] + +(30) ReusedExchange [Reuses operator id: 48] +Output [1]: [d_date_sk#25] + +(31) BroadcastHashJoin [codegen id : 7] +Left keys [1]: [cs_sold_date_sk#9] +Right keys [1]: [d_date_sk#25] +Join type: Inner +Join condition: None + +(32) Project [codegen id : 7] +Output [11]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#19, ca_county#22, ca_state#23, ca_country#24] +Input [13]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_birth_year#19, ca_county#22, ca_state#23, ca_country#24, d_date_sk#25] + +(33) Scan parquet spark_catalog.default.item +Output [2]: [i_item_sk#26, i_item_id#27] +Batched: true +Location [not included in comparison]/{warehouse_dir}/item] +PushedFilters: [IsNotNull(i_item_sk)] +ReadSchema: struct<i_item_sk:int,i_item_id:string> + +(34) CometFilter +Input [2]: [i_item_sk#26, i_item_id#27] +Condition : isnotnull(i_item_sk#26) + +(35) ColumnarToRow [codegen id : 6] +Input [2]: [i_item_sk#26, i_item_id#27] + +(36) BroadcastExchange +Input [2]: [i_item_sk#26, i_item_id#27] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=5] + +(37) BroadcastHashJoin [codegen id : 7] +Left keys [1]: [cs_item_sk#3] +Right keys [1]: [i_item_sk#26] +Join type: Inner +Join condition: None + +(38) Project [codegen id : 7] +Output [11]: [cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#19, i_item_id#27, ca_country#24, ca_state#23, ca_county#22] +Input [13]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#19, ca_county#22, ca_state#23, ca_country#24, i_item_sk#26, i_item_id#27] + +(39) Expand [codegen id : 7] +Input [11]: [cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#19, i_item_id#27, ca_country#24, ca_state#23, ca_county#22] +Arguments: [[cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#19, i_item_id#27, ca_country#24, ca_state#23, ca_county#22, 0], [cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#19, i_item_id#27, ca_country#24, ca_state#23, null, 1], [cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#19, i_item_id#27, ca_country#24, null, null, 3], [cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#19, i_item_id#27, null, null, null, 7], [cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#19, null, null, null, null, 15]], [cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#19, i_item_id#28, ca_country#29, ca_state#30, ca_county#31, spark_grouping_id#32] + +(40) HashAggregate [codegen id : 7] +Input [12]: [cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#19, i_item_id#28, ca_country#29, ca_state#30, ca_county#31, spark_grouping_id#32] +Keys [5]: [i_item_id#28, ca_country#29, ca_state#30, ca_county#31, spark_grouping_id#32] +Functions [7]: [partial_avg(cast(cs_quantity#4 as decimal(12,2))), partial_avg(cast(cs_list_price#5 as decimal(12,2))), partial_avg(cast(cs_coupon_amt#7 as decimal(12,2))), partial_avg(cast(cs_sales_price#6 as decimal(12,2))), partial_avg(cast(cs_net_profit#8 as decimal(12,2))), partial_avg(cast(c_birth_year#19 as decimal(12,2))), partial_avg(cast(cd_dep_count#14 as decimal(12,2)))] +Aggregate Attributes [14]: [sum#33, count#34, sum#35, count#36, sum#37, count#38, sum#39, count#40, sum#41, count#42, sum#43, count#44, sum#45, count#46] +Results [19]: [i_item_id#28, ca_country#29, ca_state#30, ca_county#31, spark_grouping_id#32, sum#47, count#48, sum#49, count#50, sum#51, count#52, sum#53, count#54, sum#55, count#56, sum#57, count#58, sum#59, count#60] + +(41) Exchange +Input [19]: [i_item_id#28, ca_country#29, ca_state#30, ca_county#31, spark_grouping_id#32, sum#47, count#48, sum#49, count#50, sum#51, count#52, sum#53, count#54, sum#55, count#56, sum#57, count#58, sum#59, count#60] +Arguments: hashpartitioning(i_item_id#28, ca_country#29, ca_state#30, ca_county#31, spark_grouping_id#32, 5), ENSURE_REQUIREMENTS, [plan_id=6] + +(42) HashAggregate [codegen id : 8] +Input [19]: [i_item_id#28, ca_country#29, ca_state#30, ca_county#31, spark_grouping_id#32, sum#47, count#48, sum#49, count#50, sum#51, count#52, sum#53, count#54, sum#55, count#56, sum#57, count#58, sum#59, count#60] +Keys [5]: [i_item_id#28, ca_country#29, ca_state#30, ca_county#31, spark_grouping_id#32] +Functions [7]: [avg(cast(cs_quantity#4 as decimal(12,2))), avg(cast(cs_list_price#5 as decimal(12,2))), avg(cast(cs_coupon_amt#7 as decimal(12,2))), avg(cast(cs_sales_price#6 as decimal(12,2))), avg(cast(cs_net_profit#8 as decimal(12,2))), avg(cast(c_birth_year#19 as decimal(12,2))), avg(cast(cd_dep_count#14 as decimal(12,2)))] +Aggregate Attributes [7]: [avg(cast(cs_quantity#4 as decimal(12,2)))#61, avg(cast(cs_list_price#5 as decimal(12,2)))#62, avg(cast(cs_coupon_amt#7 as decimal(12,2)))#63, avg(cast(cs_sales_price#6 as decimal(12,2)))#64, avg(cast(cs_net_profit#8 as decimal(12,2)))#65, avg(cast(c_birth_year#19 as decimal(12,2)))#66, avg(cast(cd_dep_count#14 as decimal(12,2)))#67] +Results [11]: [i_item_id#28, ca_country#29, ca_state#30, ca_county#31, avg(cast(cs_quantity#4 as decimal(12,2)))#61 AS agg1#68, avg(cast(cs_list_price#5 as decimal(12,2)))#62 AS agg2#69, avg(cast(cs_coupon_amt#7 as decimal(12,2)))#63 AS agg3#70, avg(cast(cs_sales_price#6 as decimal(12,2)))#64 AS agg4#71, avg(cast(cs_net_profit#8 as decimal(12,2)))#65 AS agg5#72, avg(cast(c_birth_year#19 as decimal(12,2)))#66 AS agg6#73, avg(cast(cd_dep_count#14 as decimal(12,2)))#67 AS agg7#74] + +(43) TakeOrderedAndProject +Input [11]: [i_item_id#28, ca_country#29, ca_state#30, ca_county#31, agg1#68, agg2#69, agg3#70, agg4#71, agg5#72, agg6#73, agg7#74] +Arguments: 100, [ca_country#29 ASC NULLS FIRST, ca_state#30 ASC NULLS FIRST, ca_county#31 ASC NULLS FIRST, i_item_id#28 ASC NULLS FIRST], [i_item_id#28, ca_country#29, ca_state#30, ca_county#31, agg1#68, agg2#69, agg3#70, agg4#71, agg5#72, agg6#73, agg7#74] + +===== Subqueries ===== + +Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#9 IN dynamicpruning#10 +BroadcastExchange (48) ++- * ColumnarToRow (47) + +- CometProject (46) + +- CometFilter (45) + +- CometScan parquet spark_catalog.default.date_dim (44) + + +(44) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#25, d_year#75] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_year), EqualTo(d_year,1998), IsNotNull(d_date_sk)] +ReadSchema: struct<d_date_sk:int,d_year:int> + +(45) CometFilter +Input [2]: [d_date_sk#25, d_year#75] +Condition : ((isnotnull(d_year#75) AND (d_year#75 = 1998)) AND isnotnull(d_date_sk#25)) + +(46) CometProject +Input [2]: [d_date_sk#25, d_year#75] +Arguments: [d_date_sk#25], [d_date_sk#25] + +(47) ColumnarToRow [codegen id : 1] +Input [1]: [d_date_sk#25] + +(48) BroadcastExchange +Input [1]: [d_date_sk#25] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=7] + + diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q18/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q18/simplified.txt new file mode 100644 index 000000000..47911b9ba --- /dev/null +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q18/simplified.txt @@ -0,0 +1,71 @@ +TakeOrderedAndProject [ca_country,ca_state,ca_county,i_item_id,agg1,agg2,agg3,agg4,agg5,agg6,agg7] + WholeStageCodegen (8) + HashAggregate [i_item_id,ca_country,ca_state,ca_county,spark_grouping_id,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] [avg(cast(cs_quantity as decimal(12,2))),avg(cast(cs_list_price as decimal(12,2))),avg(cast(cs_coupon_amt as decimal(12,2))),avg(cast(cs_sales_price as decimal(12,2))),avg(cast(cs_net_profit as decimal(12,2))),avg(cast(c_birth_year as decimal(12,2))),avg(cast(cd_dep_count as decimal(12,2))),agg1,agg2,agg3,agg4,agg5,agg6,agg7,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] + InputAdapter + Exchange [i_item_id,ca_country,ca_state,ca_county,spark_grouping_id] #1 + WholeStageCodegen (7) + HashAggregate [i_item_id,ca_country,ca_state,ca_county,spark_grouping_id,cs_quantity,cs_list_price,cs_coupon_amt,cs_sales_price,cs_net_profit,c_birth_year,cd_dep_count] [sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] + Expand [cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count,c_birth_year,i_item_id,ca_country,ca_state,ca_county] + Project [cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count,c_birth_year,i_item_id,ca_country,ca_state,ca_county] + BroadcastHashJoin [cs_item_sk,i_item_sk] + Project [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count,c_birth_year,ca_county,ca_state,ca_country] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + Project [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_birth_year,ca_county,ca_state,ca_country] + BroadcastHashJoin [c_current_addr_sk,ca_address_sk] + Project [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_addr_sk,c_birth_year] + BroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] + Project [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] + BroadcastHashJoin [cs_bill_customer_sk,c_customer_sk] + Project [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count] + BroadcastHashJoin [cs_bill_cdemo_sk,cd_demo_sk] + ColumnarToRow + InputAdapter + CometFilter [cs_bill_cdemo_sk,cs_bill_customer_sk,cs_item_sk] + CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_bill_cdemo_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #2 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_year,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + InputAdapter + BroadcastExchange #3 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [cd_demo_sk,cd_dep_count] + CometFilter [cd_gender,cd_education_status,cd_demo_sk] + CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_education_status,cd_dep_count] + InputAdapter + BroadcastExchange #4 + WholeStageCodegen (2) + ColumnarToRow + InputAdapter + CometProject [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] + CometFilter [c_birth_month,c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] + CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_month,c_birth_year] + InputAdapter + BroadcastExchange #5 + WholeStageCodegen (3) + ColumnarToRow + InputAdapter + CometFilter [cd_demo_sk] + CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk] + InputAdapter + BroadcastExchange #6 + WholeStageCodegen (4) + ColumnarToRow + InputAdapter + CometFilter [ca_state,ca_address_sk] + CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_county,ca_state,ca_country] + InputAdapter + ReusedExchange [d_date_sk] #2 + InputAdapter + BroadcastExchange #7 + WholeStageCodegen (6) + ColumnarToRow + InputAdapter + CometFilter [i_item_sk] + CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q19/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q19/explain.txt new file mode 100644 index 000000000..999fec838 --- /dev/null +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q19/explain.txt @@ -0,0 +1,227 @@ +== Physical Plan == +TakeOrderedAndProject (39) ++- * HashAggregate (38) + +- Exchange (37) + +- * HashAggregate (36) + +- * Project (35) + +- * BroadcastHashJoin Inner BuildRight (34) + :- * Project (29) + : +- * BroadcastHashJoin Inner BuildRight (28) + : :- * Project (23) + : : +- * BroadcastHashJoin Inner BuildRight (22) + : : :- * Project (17) + : : : +- * BroadcastHashJoin Inner BuildRight (16) + : : : :- * Project (10) + : : : : +- * BroadcastHashJoin Inner BuildRight (9) + : : : : :- * ColumnarToRow (4) + : : : : : +- CometProject (3) + : : : : : +- CometFilter (2) + : : : : : +- CometScan parquet spark_catalog.default.date_dim (1) + : : : : +- BroadcastExchange (8) + : : : : +- * ColumnarToRow (7) + : : : : +- CometFilter (6) + : : : : +- CometScan parquet spark_catalog.default.store_sales (5) + : : : +- BroadcastExchange (15) + : : : +- * ColumnarToRow (14) + : : : +- CometProject (13) + : : : +- CometFilter (12) + : : : +- CometScan parquet spark_catalog.default.item (11) + : : +- BroadcastExchange (21) + : : +- * ColumnarToRow (20) + : : +- CometFilter (19) + : : +- CometScan parquet spark_catalog.default.customer (18) + : +- BroadcastExchange (27) + : +- * ColumnarToRow (26) + : +- CometFilter (25) + : +- CometScan parquet spark_catalog.default.customer_address (24) + +- BroadcastExchange (33) + +- * ColumnarToRow (32) + +- CometFilter (31) + +- CometScan parquet spark_catalog.default.store (30) + + +(1) Scan parquet spark_catalog.default.date_dim +Output [3]: [d_date_sk#1, d_year#2, d_moy#3] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_moy), IsNotNull(d_year), EqualTo(d_moy,11), EqualTo(d_year,1998), IsNotNull(d_date_sk)] +ReadSchema: struct<d_date_sk:int,d_year:int,d_moy:int> + +(2) CometFilter +Input [3]: [d_date_sk#1, d_year#2, d_moy#3] +Condition : ((((isnotnull(d_moy#3) AND isnotnull(d_year#2)) AND (d_moy#3 = 11)) AND (d_year#2 = 1998)) AND isnotnull(d_date_sk#1)) + +(3) CometProject +Input [3]: [d_date_sk#1, d_year#2, d_moy#3] +Arguments: [d_date_sk#1], [d_date_sk#1] + +(4) ColumnarToRow [codegen id : 6] +Input [1]: [d_date_sk#1] + +(5) Scan parquet spark_catalog.default.store_sales +Output [5]: [ss_item_sk#4, ss_customer_sk#5, ss_store_sk#6, ss_ext_sales_price#7, ss_sold_date_sk#8] +Batched: true +Location: InMemoryFileIndex [] +PartitionFilters: [isnotnull(ss_sold_date_sk#8), dynamicpruningexpression(true)] +PushedFilters: [IsNotNull(ss_item_sk), IsNotNull(ss_customer_sk), IsNotNull(ss_store_sk)] +ReadSchema: struct<ss_item_sk:int,ss_customer_sk:int,ss_store_sk:int,ss_ext_sales_price:decimal(7,2)> + +(6) CometFilter +Input [5]: [ss_item_sk#4, ss_customer_sk#5, ss_store_sk#6, ss_ext_sales_price#7, ss_sold_date_sk#8] +Condition : ((isnotnull(ss_item_sk#4) AND isnotnull(ss_customer_sk#5)) AND isnotnull(ss_store_sk#6)) + +(7) ColumnarToRow [codegen id : 1] +Input [5]: [ss_item_sk#4, ss_customer_sk#5, ss_store_sk#6, ss_ext_sales_price#7, ss_sold_date_sk#8] + +(8) BroadcastExchange +Input [5]: [ss_item_sk#4, ss_customer_sk#5, ss_store_sk#6, ss_ext_sales_price#7, ss_sold_date_sk#8] +Arguments: HashedRelationBroadcastMode(List(cast(input[4, int, true] as bigint)),false), [plan_id=1] + +(9) BroadcastHashJoin [codegen id : 6] +Left keys [1]: [d_date_sk#1] +Right keys [1]: [ss_sold_date_sk#8] +Join type: Inner +Join condition: None + +(10) Project [codegen id : 6] +Output [4]: [ss_item_sk#4, ss_customer_sk#5, ss_store_sk#6, ss_ext_sales_price#7] +Input [6]: [d_date_sk#1, ss_item_sk#4, ss_customer_sk#5, ss_store_sk#6, ss_ext_sales_price#7, ss_sold_date_sk#8] + +(11) Scan parquet spark_catalog.default.item +Output [6]: [i_item_sk#9, i_brand_id#10, i_brand#11, i_manufact_id#12, i_manufact#13, i_manager_id#14] +Batched: true +Location [not included in comparison]/{warehouse_dir}/item] +PushedFilters: [IsNotNull(i_manager_id), EqualTo(i_manager_id,8), IsNotNull(i_item_sk)] +ReadSchema: struct<i_item_sk:int,i_brand_id:int,i_brand:string,i_manufact_id:int,i_manufact:string,i_manager_id:int> + +(12) CometFilter +Input [6]: [i_item_sk#9, i_brand_id#10, i_brand#11, i_manufact_id#12, i_manufact#13, i_manager_id#14] +Condition : ((isnotnull(i_manager_id#14) AND (i_manager_id#14 = 8)) AND isnotnull(i_item_sk#9)) + +(13) CometProject +Input [6]: [i_item_sk#9, i_brand_id#10, i_brand#11, i_manufact_id#12, i_manufact#13, i_manager_id#14] +Arguments: [i_item_sk#9, i_brand_id#10, i_brand#11, i_manufact_id#12, i_manufact#13], [i_item_sk#9, i_brand_id#10, i_brand#11, i_manufact_id#12, i_manufact#13] + +(14) ColumnarToRow [codegen id : 2] +Input [5]: [i_item_sk#9, i_brand_id#10, i_brand#11, i_manufact_id#12, i_manufact#13] + +(15) BroadcastExchange +Input [5]: [i_item_sk#9, i_brand_id#10, i_brand#11, i_manufact_id#12, i_manufact#13] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] + +(16) BroadcastHashJoin [codegen id : 6] +Left keys [1]: [ss_item_sk#4] +Right keys [1]: [i_item_sk#9] +Join type: Inner +Join condition: None + +(17) Project [codegen id : 6] +Output [7]: [ss_customer_sk#5, ss_store_sk#6, ss_ext_sales_price#7, i_brand_id#10, i_brand#11, i_manufact_id#12, i_manufact#13] +Input [9]: [ss_item_sk#4, ss_customer_sk#5, ss_store_sk#6, ss_ext_sales_price#7, i_item_sk#9, i_brand_id#10, i_brand#11, i_manufact_id#12, i_manufact#13] + +(18) Scan parquet spark_catalog.default.customer +Output [2]: [c_customer_sk#15, c_current_addr_sk#16] +Batched: true +Location [not included in comparison]/{warehouse_dir}/customer] +PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_current_addr_sk)] +ReadSchema: struct<c_customer_sk:int,c_current_addr_sk:int> + +(19) CometFilter +Input [2]: [c_customer_sk#15, c_current_addr_sk#16] +Condition : (isnotnull(c_customer_sk#15) AND isnotnull(c_current_addr_sk#16)) + +(20) ColumnarToRow [codegen id : 3] +Input [2]: [c_customer_sk#15, c_current_addr_sk#16] + +(21) BroadcastExchange +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] + +(22) BroadcastHashJoin [codegen id : 6] +Left keys [1]: [ss_customer_sk#5] +Right keys [1]: [c_customer_sk#15] +Join type: Inner +Join condition: None + +(23) Project [codegen id : 6] +Output [7]: [ss_store_sk#6, ss_ext_sales_price#7, i_brand_id#10, i_brand#11, i_manufact_id#12, i_manufact#13, c_current_addr_sk#16] +Input [9]: [ss_customer_sk#5, ss_store_sk#6, ss_ext_sales_price#7, i_brand_id#10, i_brand#11, i_manufact_id#12, i_manufact#13, c_customer_sk#15, c_current_addr_sk#16] + +(24) Scan parquet spark_catalog.default.customer_address +Output [2]: [ca_address_sk#17, ca_zip#18] +Batched: true +Location [not included in comparison]/{warehouse_dir}/customer_address] +PushedFilters: [IsNotNull(ca_address_sk), IsNotNull(ca_zip)] +ReadSchema: struct<ca_address_sk:int,ca_zip:string> + +(25) CometFilter +Input [2]: [ca_address_sk#17, ca_zip#18] +Condition : (isnotnull(ca_address_sk#17) AND isnotnull(ca_zip#18)) + +(26) ColumnarToRow [codegen id : 4] +Input [2]: [ca_address_sk#17, ca_zip#18] + +(27) BroadcastExchange +Input [2]: [ca_address_sk#17, ca_zip#18] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=4] + +(28) BroadcastHashJoin [codegen id : 6] +Left keys [1]: [c_current_addr_sk#16] +Right keys [1]: [ca_address_sk#17] +Join type: Inner +Join condition: None + +(29) Project [codegen id : 6] +Output [7]: [ss_store_sk#6, ss_ext_sales_price#7, i_brand_id#10, i_brand#11, i_manufact_id#12, i_manufact#13, ca_zip#18] +Input [9]: [ss_store_sk#6, ss_ext_sales_price#7, i_brand_id#10, i_brand#11, i_manufact_id#12, i_manufact#13, c_current_addr_sk#16, ca_address_sk#17, ca_zip#18] + +(30) Scan parquet spark_catalog.default.store +Output [2]: [s_store_sk#19, s_zip#20] +Batched: true +Location [not included in comparison]/{warehouse_dir}/store] +PushedFilters: [IsNotNull(s_zip), IsNotNull(s_store_sk)] +ReadSchema: struct<s_store_sk:int,s_zip:string> + +(31) CometFilter +Input [2]: [s_store_sk#19, s_zip#20] +Condition : (isnotnull(s_zip#20) AND isnotnull(s_store_sk#19)) + +(32) ColumnarToRow [codegen id : 5] +Input [2]: [s_store_sk#19, s_zip#20] + +(33) BroadcastExchange +Input [2]: [s_store_sk#19, s_zip#20] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=5] + +(34) BroadcastHashJoin [codegen id : 6] +Left keys [1]: [ss_store_sk#6] +Right keys [1]: [s_store_sk#19] +Join type: Inner +Join condition: NOT (substr(ca_zip#18, 1, 5) = substr(s_zip#20, 1, 5)) + +(35) Project [codegen id : 6] +Output [5]: [ss_ext_sales_price#7, i_brand_id#10, i_brand#11, i_manufact_id#12, i_manufact#13] +Input [9]: [ss_store_sk#6, ss_ext_sales_price#7, i_brand_id#10, i_brand#11, i_manufact_id#12, i_manufact#13, ca_zip#18, s_store_sk#19, s_zip#20] + +(36) HashAggregate [codegen id : 6] +Input [5]: [ss_ext_sales_price#7, i_brand_id#10, i_brand#11, i_manufact_id#12, i_manufact#13] +Keys [4]: [i_brand#11, i_brand_id#10, i_manufact_id#12, i_manufact#13] +Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#7))] +Aggregate Attributes [1]: [sum#21] +Results [5]: [i_brand#11, i_brand_id#10, i_manufact_id#12, i_manufact#13, sum#22] + +(37) Exchange +Input [5]: [i_brand#11, i_brand_id#10, i_manufact_id#12, i_manufact#13, sum#22] +Arguments: hashpartitioning(i_brand#11, i_brand_id#10, i_manufact_id#12, i_manufact#13, 5), ENSURE_REQUIREMENTS, [plan_id=6] + +(38) HashAggregate [codegen id : 7] +Input [5]: [i_brand#11, i_brand_id#10, i_manufact_id#12, i_manufact#13, sum#22] +Keys [4]: [i_brand#11, i_brand_id#10, i_manufact_id#12, i_manufact#13] +Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#7))] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#7))#23] +Results [5]: [i_brand_id#10 AS brand_id#24, i_brand#11 AS brand#25, i_manufact_id#12, i_manufact#13, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#7))#23,17,2) AS ext_price#26] + +(39) TakeOrderedAndProject +Input [5]: [brand_id#24, brand#25, i_manufact_id#12, i_manufact#13, ext_price#26] +Arguments: 100, [ext_price#26 DESC NULLS LAST, brand#25 ASC NULLS FIRST, brand_id#24 ASC NULLS FIRST, i_manufact_id#12 ASC NULLS FIRST, i_manufact#13 ASC NULLS FIRST], [brand_id#24, brand#25, i_manufact_id#12, i_manufact#13, ext_price#26] + diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q19/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q19/simplified.txt new file mode 100644 index 000000000..c2f5d1a87 --- /dev/null +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q19/simplified.txt @@ -0,0 +1,58 @@ +TakeOrderedAndProject [ext_price,brand,brand_id,i_manufact_id,i_manufact] + WholeStageCodegen (7) + HashAggregate [i_brand,i_brand_id,i_manufact_id,i_manufact,sum] [sum(UnscaledValue(ss_ext_sales_price)),brand_id,brand,ext_price,sum] + InputAdapter + Exchange [i_brand,i_brand_id,i_manufact_id,i_manufact] #1 + WholeStageCodegen (6) + HashAggregate [i_brand,i_brand_id,i_manufact_id,i_manufact,ss_ext_sales_price] [sum,sum] + Project [ss_ext_sales_price,i_brand_id,i_brand,i_manufact_id,i_manufact] + BroadcastHashJoin [ss_store_sk,s_store_sk,ca_zip,s_zip] + Project [ss_store_sk,ss_ext_sales_price,i_brand_id,i_brand,i_manufact_id,i_manufact,ca_zip] + BroadcastHashJoin [c_current_addr_sk,ca_address_sk] + Project [ss_store_sk,ss_ext_sales_price,i_brand_id,i_brand,i_manufact_id,i_manufact,c_current_addr_sk] + BroadcastHashJoin [ss_customer_sk,c_customer_sk] + Project [ss_customer_sk,ss_store_sk,ss_ext_sales_price,i_brand_id,i_brand,i_manufact_id,i_manufact] + BroadcastHashJoin [ss_item_sk,i_item_sk] + Project [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ext_sales_price] + BroadcastHashJoin [d_date_sk,ss_sold_date_sk] + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_moy,d_year,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + InputAdapter + BroadcastExchange #2 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometFilter [ss_item_sk,ss_customer_sk,ss_store_sk] + CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ext_sales_price,ss_sold_date_sk] + InputAdapter + BroadcastExchange #3 + WholeStageCodegen (2) + ColumnarToRow + InputAdapter + CometProject [i_item_sk,i_brand_id,i_brand,i_manufact_id,i_manufact] + CometFilter [i_manager_id,i_item_sk] + CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_brand,i_manufact_id,i_manufact,i_manager_id] + InputAdapter + BroadcastExchange #4 + WholeStageCodegen (3) + ColumnarToRow + InputAdapter + CometFilter [c_customer_sk,c_current_addr_sk] + CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk] + InputAdapter + BroadcastExchange #5 + WholeStageCodegen (4) + ColumnarToRow + InputAdapter + CometFilter [ca_address_sk,ca_zip] + CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_zip] + InputAdapter + BroadcastExchange #6 + WholeStageCodegen (5) + ColumnarToRow + InputAdapter + CometFilter [s_zip,s_store_sk] + CometScan parquet spark_catalog.default.store [s_store_sk,s_zip] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q2/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q2/explain.txt new file mode 100644 index 000000000..1215adf3f --- /dev/null +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q2/explain.txt @@ -0,0 +1,210 @@ +== Physical Plan == +* Sort (36) ++- Exchange (35) + +- * Project (34) + +- * BroadcastHashJoin Inner BuildRight (33) + :- * Project (22) + : +- * BroadcastHashJoin Inner BuildRight (21) + : :- * HashAggregate (15) + : : +- Exchange (14) + : : +- * HashAggregate (13) + : : +- * Project (12) + : : +- * BroadcastHashJoin Inner BuildRight (11) + : : :- * ColumnarToRow (6) + : : : +- CometUnion (5) + : : : :- CometProject (2) + : : : : +- CometScan parquet spark_catalog.default.web_sales (1) + : : : +- CometProject (4) + : : : +- CometScan parquet spark_catalog.default.catalog_sales (3) + : : +- BroadcastExchange (10) + : : +- * ColumnarToRow (9) + : : +- CometFilter (8) + : : +- CometScan parquet spark_catalog.default.date_dim (7) + : +- BroadcastExchange (20) + : +- * ColumnarToRow (19) + : +- CometProject (18) + : +- CometFilter (17) + : +- CometScan parquet spark_catalog.default.date_dim (16) + +- BroadcastExchange (32) + +- * Project (31) + +- * BroadcastHashJoin Inner BuildRight (30) + :- * HashAggregate (24) + : +- ReusedExchange (23) + +- BroadcastExchange (29) + +- * ColumnarToRow (28) + +- CometProject (27) + +- CometFilter (26) + +- CometScan parquet spark_catalog.default.date_dim (25) + + +(1) Scan parquet spark_catalog.default.web_sales +Output [2]: [ws_ext_sales_price#1, ws_sold_date_sk#2] +Batched: true +Location: InMemoryFileIndex [] +PartitionFilters: [isnotnull(ws_sold_date_sk#2)] +ReadSchema: struct<ws_ext_sales_price:decimal(7,2)> + +(2) CometProject +Input [2]: [ws_ext_sales_price#1, ws_sold_date_sk#2] +Arguments: [sold_date_sk#3, sales_price#4], [ws_sold_date_sk#2 AS sold_date_sk#3, ws_ext_sales_price#1 AS sales_price#4] + +(3) Scan parquet spark_catalog.default.catalog_sales +Output [2]: [cs_ext_sales_price#5, cs_sold_date_sk#6] +Batched: true +Location: InMemoryFileIndex [] +PartitionFilters: [isnotnull(cs_sold_date_sk#6)] +ReadSchema: struct<cs_ext_sales_price:decimal(7,2)> + +(4) CometProject +Input [2]: [cs_ext_sales_price#5, cs_sold_date_sk#6] +Arguments: [sold_date_sk#7, sales_price#8], [cs_sold_date_sk#6 AS sold_date_sk#7, cs_ext_sales_price#5 AS sales_price#8] + +(5) CometUnion +Child 0 Input [2]: [sold_date_sk#3, sales_price#4] +Child 1 Input [2]: [sold_date_sk#7, sales_price#8] + +(6) ColumnarToRow [codegen id : 2] +Input [2]: [sold_date_sk#3, sales_price#4] + +(7) Scan parquet spark_catalog.default.date_dim +Output [3]: [d_date_sk#9, d_week_seq#10, d_day_name#11] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_date_sk), IsNotNull(d_week_seq)] +ReadSchema: struct<d_date_sk:int,d_week_seq:int,d_day_name:string> + +(8) CometFilter +Input [3]: [d_date_sk#9, d_week_seq#10, d_day_name#11] +Condition : (isnotnull(d_date_sk#9) AND isnotnull(d_week_seq#10)) + +(9) ColumnarToRow [codegen id : 1] +Input [3]: [d_date_sk#9, d_week_seq#10, d_day_name#11] + +(10) BroadcastExchange +Input [3]: [d_date_sk#9, d_week_seq#10, d_day_name#11] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=1] + +(11) BroadcastHashJoin [codegen id : 2] +Left keys [1]: [sold_date_sk#3] +Right keys [1]: [d_date_sk#9] +Join type: Inner +Join condition: None + +(12) Project [codegen id : 2] +Output [3]: [sales_price#4, d_week_seq#10, d_day_name#11] +Input [5]: [sold_date_sk#3, sales_price#4, d_date_sk#9, d_week_seq#10, d_day_name#11] + +(13) HashAggregate [codegen id : 2] +Input [3]: [sales_price#4, d_week_seq#10, d_day_name#11] +Keys [1]: [d_week_seq#10] +Functions [7]: [partial_sum(UnscaledValue(CASE WHEN (d_day_name#11 = Sunday ) THEN sales_price#4 END)), partial_sum(UnscaledValue(CASE WHEN (d_day_name#11 = Monday ) THEN sales_price#4 END)), partial_sum(UnscaledValue(CASE WHEN (d_day_name#11 = Tuesday ) THEN sales_price#4 END)), partial_sum(UnscaledValue(CASE WHEN (d_day_name#11 = Wednesday) THEN sales_price#4 END)), partial_sum(UnscaledValue(CASE WHEN (d_day_name#11 = Thursday ) THEN sales_price#4 END)), partial_sum(UnscaledValue(CASE WHEN (d_day_name#11 = Friday ) THEN sales_price#4 END)), partial_sum(UnscaledValue(CASE WHEN (d_day_name#11 = Saturday ) THEN sales_price#4 END))] +Aggregate Attributes [7]: [sum#12, sum#13, sum#14, sum#15, sum#16, sum#17, sum#18] +Results [8]: [d_week_seq#10, sum#19, sum#20, sum#21, sum#22, sum#23, sum#24, sum#25] + +(14) Exchange +Input [8]: [d_week_seq#10, sum#19, sum#20, sum#21, sum#22, sum#23, sum#24, sum#25] +Arguments: hashpartitioning(d_week_seq#10, 5), ENSURE_REQUIREMENTS, [plan_id=2] + +(15) HashAggregate [codegen id : 8] +Input [8]: [d_week_seq#10, sum#19, sum#20, sum#21, sum#22, sum#23, sum#24, sum#25] +Keys [1]: [d_week_seq#10] +Functions [7]: [sum(UnscaledValue(CASE WHEN (d_day_name#11 = Sunday ) THEN sales_price#4 END)), sum(UnscaledValue(CASE WHEN (d_day_name#11 = Monday ) THEN sales_price#4 END)), sum(UnscaledValue(CASE WHEN (d_day_name#11 = Tuesday ) THEN sales_price#4 END)), sum(UnscaledValue(CASE WHEN (d_day_name#11 = Wednesday) THEN sales_price#4 END)), sum(UnscaledValue(CASE WHEN (d_day_name#11 = Thursday ) THEN sales_price#4 END)), sum(UnscaledValue(CASE WHEN (d_day_name#11 = Friday ) THEN sales_price#4 END)), sum(UnscaledValue(CASE WHEN (d_day_name#11 = Saturday ) THEN sales_price#4 END))] +Aggregate Attributes [7]: [sum(UnscaledValue(CASE WHEN (d_day_name#11 = Sunday ) THEN sales_price#4 END))#26, sum(UnscaledValue(CASE WHEN (d_day_name#11 = Monday ) THEN sales_price#4 END))#27, sum(UnscaledValue(CASE WHEN (d_day_name#11 = Tuesday ) THEN sales_price#4 END))#28, sum(UnscaledValue(CASE WHEN (d_day_name#11 = Wednesday) THEN sales_price#4 END))#29, sum(UnscaledValue(CASE WHEN (d_day_name#11 = Thursday ) THEN sales_price#4 END))#30, sum(UnscaledValue(CASE WHEN (d_day_name#11 = Friday ) THEN sales_price#4 END))#31, sum(UnscaledValue(CASE WHEN (d_day_name#11 = Saturday ) THEN sales_price#4 END))#32] +Results [8]: [d_week_seq#10, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#11 = Sunday ) THEN sales_price#4 END))#26,17,2) AS sun_sales#33, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#11 = Monday ) THEN sales_price#4 END))#27,17,2) AS mon_sales#34, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#11 = Tuesday ) THEN sales_price#4 END))#28,17,2) AS tue_sales#35, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#11 = Wednesday) THEN sales_price#4 END))#29,17,2) AS wed_sales#36, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#11 = Thursday ) THEN sales_price#4 END))#30,17,2) AS thu_sales#37, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#11 = Friday ) THEN sales_price#4 END))#31,17,2) AS fri_sales#38, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#11 = Saturday ) THEN sales_price#4 END))#32,17,2) AS sat_sales#39] + +(16) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_week_seq#40, d_year#41] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), IsNotNull(d_week_seq)] +ReadSchema: struct<d_week_seq:int,d_year:int> + +(17) CometFilter +Input [2]: [d_week_seq#40, d_year#41] +Condition : ((isnotnull(d_year#41) AND (d_year#41 = 2001)) AND isnotnull(d_week_seq#40)) + +(18) CometProject +Input [2]: [d_week_seq#40, d_year#41] +Arguments: [d_week_seq#40], [d_week_seq#40] + +(19) ColumnarToRow [codegen id : 3] +Input [1]: [d_week_seq#40] + +(20) BroadcastExchange +Input [1]: [d_week_seq#40] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] + +(21) BroadcastHashJoin [codegen id : 8] +Left keys [1]: [d_week_seq#10] +Right keys [1]: [d_week_seq#40] +Join type: Inner +Join condition: None + +(22) Project [codegen id : 8] +Output [8]: [d_week_seq#10 AS d_week_seq1#42, sun_sales#33 AS sun_sales1#43, mon_sales#34 AS mon_sales1#44, tue_sales#35 AS tue_sales1#45, wed_sales#36 AS wed_sales1#46, thu_sales#37 AS thu_sales1#47, fri_sales#38 AS fri_sales1#48, sat_sales#39 AS sat_sales1#49] +Input [9]: [d_week_seq#10, sun_sales#33, mon_sales#34, tue_sales#35, wed_sales#36, thu_sales#37, fri_sales#38, sat_sales#39, d_week_seq#40] + +(23) ReusedExchange [Reuses operator id: 14] +Output [8]: [d_week_seq#50, sum#51, sum#52, sum#53, sum#54, sum#55, sum#56, sum#57] + +(24) HashAggregate [codegen id : 7] +Input [8]: [d_week_seq#50, sum#51, sum#52, sum#53, sum#54, sum#55, sum#56, sum#57] +Keys [1]: [d_week_seq#50] +Functions [7]: [sum(UnscaledValue(CASE WHEN (d_day_name#58 = Sunday ) THEN sales_price#59 END)), sum(UnscaledValue(CASE WHEN (d_day_name#58 = Monday ) THEN sales_price#59 END)), sum(UnscaledValue(CASE WHEN (d_day_name#58 = Tuesday ) THEN sales_price#59 END)), sum(UnscaledValue(CASE WHEN (d_day_name#58 = Wednesday) THEN sales_price#59 END)), sum(UnscaledValue(CASE WHEN (d_day_name#58 = Thursday ) THEN sales_price#59 END)), sum(UnscaledValue(CASE WHEN (d_day_name#58 = Friday ) THEN sales_price#59 END)), sum(UnscaledValue(CASE WHEN (d_day_name#58 = Saturday ) THEN sales_price#59 END))] +Aggregate Attributes [7]: [sum(UnscaledValue(CASE WHEN (d_day_name#58 = Sunday ) THEN sales_price#59 END))#26, sum(UnscaledValue(CASE WHEN (d_day_name#58 = Monday ) THEN sales_price#59 END))#27, sum(UnscaledValue(CASE WHEN (d_day_name#58 = Tuesday ) THEN sales_price#59 END))#28, sum(UnscaledValue(CASE WHEN (d_day_name#58 = Wednesday) THEN sales_price#59 END))#29, sum(UnscaledValue(CASE WHEN (d_day_name#58 = Thursday ) THEN sales_price#59 END))#30, sum(UnscaledValue(CASE WHEN (d_day_name#58 = Friday ) THEN sales_price#59 END))#31, sum(UnscaledValue(CASE WHEN (d_day_name#58 = Saturday ) THEN sales_price#59 END))#32] +Results [8]: [d_week_seq#50, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#58 = Sunday ) THEN sales_price#59 END))#26,17,2) AS sun_sales#60, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#58 = Monday ) THEN sales_price#59 END))#27,17,2) AS mon_sales#61, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#58 = Tuesday ) THEN sales_price#59 END))#28,17,2) AS tue_sales#62, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#58 = Wednesday) THEN sales_price#59 END))#29,17,2) AS wed_sales#63, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#58 = Thursday ) THEN sales_price#59 END))#30,17,2) AS thu_sales#64, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#58 = Friday ) THEN sales_price#59 END))#31,17,2) AS fri_sales#65, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#58 = Saturday ) THEN sales_price#59 END))#32,17,2) AS sat_sales#66] + +(25) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_week_seq#67, d_year#68] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2002), IsNotNull(d_week_seq)] +ReadSchema: struct<d_week_seq:int,d_year:int> + +(26) CometFilter +Input [2]: [d_week_seq#67, d_year#68] +Condition : ((isnotnull(d_year#68) AND (d_year#68 = 2002)) AND isnotnull(d_week_seq#67)) + +(27) CometProject +Input [2]: [d_week_seq#67, d_year#68] +Arguments: [d_week_seq#67], [d_week_seq#67] + +(28) ColumnarToRow [codegen id : 6] +Input [1]: [d_week_seq#67] + +(29) BroadcastExchange +Input [1]: [d_week_seq#67] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] + +(30) BroadcastHashJoin [codegen id : 7] +Left keys [1]: [d_week_seq#50] +Right keys [1]: [d_week_seq#67] +Join type: Inner +Join condition: None + +(31) Project [codegen id : 7] +Output [8]: [d_week_seq#50 AS d_week_seq2#69, sun_sales#60 AS sun_sales2#70, mon_sales#61 AS mon_sales2#71, tue_sales#62 AS tue_sales2#72, wed_sales#63 AS wed_sales2#73, thu_sales#64 AS thu_sales2#74, fri_sales#65 AS fri_sales2#75, sat_sales#66 AS sat_sales2#76] +Input [9]: [d_week_seq#50, sun_sales#60, mon_sales#61, tue_sales#62, wed_sales#63, thu_sales#64, fri_sales#65, sat_sales#66, d_week_seq#67] + +(32) BroadcastExchange +Input [8]: [d_week_seq2#69, sun_sales2#70, mon_sales2#71, tue_sales2#72, wed_sales2#73, thu_sales2#74, fri_sales2#75, sat_sales2#76] +Arguments: HashedRelationBroadcastMode(List(cast((input[0, int, true] - 53) as bigint)),false), [plan_id=5] + +(33) BroadcastHashJoin [codegen id : 8] +Left keys [1]: [d_week_seq1#42] +Right keys [1]: [(d_week_seq2#69 - 53)] +Join type: Inner +Join condition: None + +(34) Project [codegen id : 8] +Output [8]: [d_week_seq1#42, round((sun_sales1#43 / sun_sales2#70), 2) AS round((sun_sales1 / sun_sales2), 2)#77, round((mon_sales1#44 / mon_sales2#71), 2) AS round((mon_sales1 / mon_sales2), 2)#78, round((tue_sales1#45 / tue_sales2#72), 2) AS round((tue_sales1 / tue_sales2), 2)#79, round((wed_sales1#46 / wed_sales2#73), 2) AS round((wed_sales1 / wed_sales2), 2)#80, round((thu_sales1#47 / thu_sales2#74), 2) AS round((thu_sales1 / thu_sales2), 2)#81, round((fri_sales1#48 / fri_sales2#75), 2) AS round((fri_sales1 / fri_sales2), 2)#82, round((sat_sales1#49 / sat_sales2#76), 2) AS round((sat_sales1 / sat_sales2), 2)#83] +Input [16]: [d_week_seq1#42, sun_sales1#43, mon_sales1#44, tue_sales1#45, wed_sales1#46, thu_sales1#47, fri_sales1#48, sat_sales1#49, d_week_seq2#69, sun_sales2#70, mon_sales2#71, tue_sales2#72, wed_sales2#73, thu_sales2#74, fri_sales2#75, sat_sales2#76] + +(35) Exchange +Input [8]: [d_week_seq1#42, round((sun_sales1 / sun_sales2), 2)#77, round((mon_sales1 / mon_sales2), 2)#78, round((tue_sales1 / tue_sales2), 2)#79, round((wed_sales1 / wed_sales2), 2)#80, round((thu_sales1 / thu_sales2), 2)#81, round((fri_sales1 / fri_sales2), 2)#82, round((sat_sales1 / sat_sales2), 2)#83] +Arguments: rangepartitioning(d_week_seq1#42 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, [plan_id=6] + +(36) Sort [codegen id : 9] +Input [8]: [d_week_seq1#42, round((sun_sales1 / sun_sales2), 2)#77, round((mon_sales1 / mon_sales2), 2)#78, round((tue_sales1 / tue_sales2), 2)#79, round((wed_sales1 / wed_sales2), 2)#80, round((thu_sales1 / thu_sales2), 2)#81, round((fri_sales1 / fri_sales2), 2)#82, round((sat_sales1 / sat_sales2), 2)#83] +Arguments: [d_week_seq1#42 ASC NULLS FIRST], true, 0 + diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q2/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q2/simplified.txt new file mode 100644 index 000000000..8856ce80d --- /dev/null +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q2/simplified.txt @@ -0,0 +1,54 @@ +WholeStageCodegen (9) + Sort [d_week_seq1] + InputAdapter + Exchange [d_week_seq1] #1 + WholeStageCodegen (8) + Project [d_week_seq1,sun_sales1,sun_sales2,mon_sales1,mon_sales2,tue_sales1,tue_sales2,wed_sales1,wed_sales2,thu_sales1,thu_sales2,fri_sales1,fri_sales2,sat_sales1,sat_sales2] + BroadcastHashJoin [d_week_seq1,d_week_seq2] + Project [d_week_seq,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales] + BroadcastHashJoin [d_week_seq,d_week_seq] + HashAggregate [d_week_seq,sum,sum,sum,sum,sum,sum,sum] [sum(UnscaledValue(CASE WHEN (d_day_name = Sunday ) THEN sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Monday ) THEN sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Tuesday ) THEN sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Wednesday) THEN sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Thursday ) THEN sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Friday ) THEN sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Saturday ) THEN sales_price END)),sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales,sum,sum,sum,sum,sum,sum,sum] + InputAdapter + Exchange [d_week_seq] #2 + WholeStageCodegen (2) + HashAggregate [d_week_seq,d_day_name,sales_price] [sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum] + Project [sales_price,d_week_seq,d_day_name] + BroadcastHashJoin [sold_date_sk,d_date_sk] + ColumnarToRow + InputAdapter + CometUnion + CometProject [ws_sold_date_sk,ws_ext_sales_price] [sold_date_sk,sales_price] + CometScan parquet spark_catalog.default.web_sales [ws_ext_sales_price,ws_sold_date_sk] + CometProject [cs_sold_date_sk,cs_ext_sales_price] [sold_date_sk,sales_price] + CometScan parquet spark_catalog.default.catalog_sales [cs_ext_sales_price,cs_sold_date_sk] + InputAdapter + BroadcastExchange #3 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometFilter [d_date_sk,d_week_seq] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq,d_day_name] + InputAdapter + BroadcastExchange #4 + WholeStageCodegen (3) + ColumnarToRow + InputAdapter + CometProject [d_week_seq] + CometFilter [d_year,d_week_seq] + CometScan parquet spark_catalog.default.date_dim [d_week_seq,d_year] + InputAdapter + BroadcastExchange #5 + WholeStageCodegen (7) + Project [d_week_seq,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales] + BroadcastHashJoin [d_week_seq,d_week_seq] + HashAggregate [d_week_seq,sum,sum,sum,sum,sum,sum,sum] [sum(UnscaledValue(CASE WHEN (d_day_name = Sunday ) THEN sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Monday ) THEN sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Tuesday ) THEN sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Wednesday) THEN sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Thursday ) THEN sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Friday ) THEN sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Saturday ) THEN sales_price END)),sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales,sum,sum,sum,sum,sum,sum,sum] + InputAdapter + ReusedExchange [d_week_seq,sum,sum,sum,sum,sum,sum,sum] #2 + InputAdapter + BroadcastExchange #6 + WholeStageCodegen (6) + ColumnarToRow + InputAdapter + CometProject [d_week_seq] + CometFilter [d_year,d_week_seq] + CometScan parquet spark_catalog.default.date_dim [d_week_seq,d_year] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q20/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q20/explain.txt new file mode 100644 index 000000000..333ef218c --- /dev/null +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q20/explain.txt @@ -0,0 +1,150 @@ +== Physical Plan == +TakeOrderedAndProject (20) ++- * Project (19) + +- Window (18) + +- * Sort (17) + +- Exchange (16) + +- * HashAggregate (15) + +- Exchange (14) + +- * HashAggregate (13) + +- * Project (12) + +- * BroadcastHashJoin Inner BuildRight (11) + :- * Project (9) + : +- * BroadcastHashJoin Inner BuildRight (8) + : :- * ColumnarToRow (3) + : : +- CometFilter (2) + : : +- CometScan parquet spark_catalog.default.catalog_sales (1) + : +- BroadcastExchange (7) + : +- * ColumnarToRow (6) + : +- CometFilter (5) + : +- CometScan parquet spark_catalog.default.item (4) + +- ReusedExchange (10) + + +(1) Scan parquet spark_catalog.default.catalog_sales +Output [3]: [cs_item_sk#1, cs_ext_sales_price#2, cs_sold_date_sk#3] +Batched: true +Location: InMemoryFileIndex [] +PartitionFilters: [isnotnull(cs_sold_date_sk#3), dynamicpruningexpression(cs_sold_date_sk#3 IN dynamicpruning#4)] +PushedFilters: [IsNotNull(cs_item_sk)] +ReadSchema: struct<cs_item_sk:int,cs_ext_sales_price:decimal(7,2)> + +(2) CometFilter +Input [3]: [cs_item_sk#1, cs_ext_sales_price#2, cs_sold_date_sk#3] +Condition : isnotnull(cs_item_sk#1) + +(3) ColumnarToRow [codegen id : 3] +Input [3]: [cs_item_sk#1, cs_ext_sales_price#2, cs_sold_date_sk#3] + +(4) Scan parquet spark_catalog.default.item +Output [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] +Batched: true +Location [not included in comparison]/{warehouse_dir}/item] +PushedFilters: [In(i_category, [Books ,Home ,Sports ]), IsNotNull(i_item_sk)] +ReadSchema: struct<i_item_sk:int,i_item_id:string,i_item_desc:string,i_current_price:decimal(7,2),i_class:string,i_category:string> + +(5) CometFilter +Input [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] +Condition : (i_category#10 IN (Sports ,Books ,Home ) AND isnotnull(i_item_sk#5)) + +(6) ColumnarToRow [codegen id : 1] +Input [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] + +(7) BroadcastExchange +Input [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=1] + +(8) BroadcastHashJoin [codegen id : 3] +Left keys [1]: [cs_item_sk#1] +Right keys [1]: [i_item_sk#5] +Join type: Inner +Join condition: None + +(9) Project [codegen id : 3] +Output [7]: [cs_ext_sales_price#2, cs_sold_date_sk#3, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] +Input [9]: [cs_item_sk#1, cs_ext_sales_price#2, cs_sold_date_sk#3, i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] + +(10) ReusedExchange [Reuses operator id: 25] +Output [1]: [d_date_sk#11] + +(11) BroadcastHashJoin [codegen id : 3] +Left keys [1]: [cs_sold_date_sk#3] +Right keys [1]: [d_date_sk#11] +Join type: Inner +Join condition: None + +(12) Project [codegen id : 3] +Output [6]: [cs_ext_sales_price#2, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] +Input [8]: [cs_ext_sales_price#2, cs_sold_date_sk#3, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10, d_date_sk#11] + +(13) HashAggregate [codegen id : 3] +Input [6]: [cs_ext_sales_price#2, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] +Keys [5]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8] +Functions [1]: [partial_sum(UnscaledValue(cs_ext_sales_price#2))] +Aggregate Attributes [1]: [sum#12] +Results [6]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, sum#13] + +(14) Exchange +Input [6]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, sum#13] +Arguments: hashpartitioning(i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, 5), ENSURE_REQUIREMENTS, [plan_id=2] + +(15) HashAggregate [codegen id : 4] +Input [6]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, sum#13] +Keys [5]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8] +Functions [1]: [sum(UnscaledValue(cs_ext_sales_price#2))] +Aggregate Attributes [1]: [sum(UnscaledValue(cs_ext_sales_price#2))#14] +Results [7]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#2))#14,17,2) AS itemrevenue#15, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#2))#14,17,2) AS _w0#16, i_item_id#6] + +(16) Exchange +Input [7]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#15, _w0#16, i_item_id#6] +Arguments: hashpartitioning(i_class#9, 5), ENSURE_REQUIREMENTS, [plan_id=3] + +(17) Sort [codegen id : 5] +Input [7]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#15, _w0#16, i_item_id#6] +Arguments: [i_class#9 ASC NULLS FIRST], false, 0 + +(18) Window +Input [7]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#15, _w0#16, i_item_id#6] +Arguments: [sum(_w0#16) windowspecdefinition(i_class#9, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS _we0#17], [i_class#9] + +(19) Project [codegen id : 6] +Output [7]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#15, ((_w0#16 * 100) / _we0#17) AS revenueratio#18, i_item_id#6] +Input [8]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#15, _w0#16, i_item_id#6, _we0#17] + +(20) TakeOrderedAndProject +Input [7]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#15, revenueratio#18, i_item_id#6] +Arguments: 100, [i_category#10 ASC NULLS FIRST, i_class#9 ASC NULLS FIRST, i_item_id#6 ASC NULLS FIRST, i_item_desc#7 ASC NULLS FIRST, revenueratio#18 ASC NULLS FIRST], [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#15, revenueratio#18] + +===== Subqueries ===== + +Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#3 IN dynamicpruning#4 +BroadcastExchange (25) ++- * ColumnarToRow (24) + +- CometProject (23) + +- CometFilter (22) + +- CometScan parquet spark_catalog.default.date_dim (21) + + +(21) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#11, d_date#19] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,1999-02-22), LessThanOrEqual(d_date,1999-03-24), IsNotNull(d_date_sk)] +ReadSchema: struct<d_date_sk:int,d_date:date> + +(22) CometFilter +Input [2]: [d_date_sk#11, d_date#19] +Condition : (((isnotnull(d_date#19) AND (d_date#19 >= 1999-02-22)) AND (d_date#19 <= 1999-03-24)) AND isnotnull(d_date_sk#11)) + +(23) CometProject +Input [2]: [d_date_sk#11, d_date#19] +Arguments: [d_date_sk#11], [d_date_sk#11] + +(24) ColumnarToRow [codegen id : 1] +Input [1]: [d_date_sk#11] + +(25) BroadcastExchange +Input [1]: [d_date_sk#11] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] + + diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q20/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q20/simplified.txt new file mode 100644 index 000000000..52c42bdf2 --- /dev/null +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q20/simplified.txt @@ -0,0 +1,40 @@ +TakeOrderedAndProject [i_category,i_class,i_item_id,i_item_desc,revenueratio,i_current_price,itemrevenue] + 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) + Sort [i_class] + InputAdapter + Exchange [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] + InputAdapter + Exchange [i_item_id,i_item_desc,i_category,i_class,i_current_price] #2 + WholeStageCodegen (3) + HashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,cs_ext_sales_price] [sum,sum] + Project [cs_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + Project [cs_ext_sales_price,cs_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + BroadcastHashJoin [cs_item_sk,i_item_sk] + ColumnarToRow + InputAdapter + CometFilter [cs_item_sk] + CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] + InputAdapter + BroadcastExchange #4 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometFilter [i_category,i_item_sk] + CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + InputAdapter + ReusedExchange [d_date_sk] #3 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 new file mode 100644 index 000000000..77f17c7f3 --- /dev/null +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q21/explain.txt @@ -0,0 +1,169 @@ +== Physical Plan == +TakeOrderedAndProject (24) ++- * Filter (23) + +- * HashAggregate (22) + +- Exchange (21) + +- * HashAggregate (20) + +- * Project (19) + +- * BroadcastHashJoin Inner BuildRight (18) + :- * Project (16) + : +- * BroadcastHashJoin Inner BuildRight (15) + : :- * Project (9) + : : +- * BroadcastHashJoin Inner BuildRight (8) + : : :- * ColumnarToRow (3) + : : : +- CometFilter (2) + : : : +- CometScan parquet spark_catalog.default.inventory (1) + : : +- BroadcastExchange (7) + : : +- * ColumnarToRow (6) + : : +- CometFilter (5) + : : +- CometScan parquet spark_catalog.default.warehouse (4) + : +- BroadcastExchange (14) + : +- * ColumnarToRow (13) + : +- CometProject (12) + : +- CometFilter (11) + : +- CometScan parquet spark_catalog.default.item (10) + +- ReusedExchange (17) + + +(1) Scan parquet spark_catalog.default.inventory +Output [4]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4] +Batched: true +Location: InMemoryFileIndex [] +PartitionFilters: [isnotnull(inv_date_sk#4), dynamicpruningexpression(inv_date_sk#4 IN dynamicpruning#5)] +PushedFilters: [IsNotNull(inv_warehouse_sk), IsNotNull(inv_item_sk)] +ReadSchema: struct<inv_item_sk:int,inv_warehouse_sk:int,inv_quantity_on_hand:int> + +(2) CometFilter +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)) + +(3) ColumnarToRow [codegen id : 4] +Input [4]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4] + +(4) Scan parquet spark_catalog.default.warehouse +Output [2]: [w_warehouse_sk#6, w_warehouse_name#7] +Batched: true +Location [not included in comparison]/{warehouse_dir}/warehouse] +PushedFilters: [IsNotNull(w_warehouse_sk)] +ReadSchema: struct<w_warehouse_sk:int,w_warehouse_name:string> + +(5) CometFilter +Input [2]: [w_warehouse_sk#6, w_warehouse_name#7] +Condition : isnotnull(w_warehouse_sk#6) + +(6) ColumnarToRow [codegen id : 1] +Input [2]: [w_warehouse_sk#6, w_warehouse_name#7] + +(7) BroadcastExchange +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] +Left keys [1]: [inv_warehouse_sk#2] +Right keys [1]: [w_warehouse_sk#6] +Join type: Inner +Join condition: None + +(9) Project [codegen id : 4] +Output [4]: [inv_item_sk#1, inv_quantity_on_hand#3, inv_date_sk#4, w_warehouse_name#7] +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] + +(10) Scan parquet spark_catalog.default.item +Output [3]: [i_item_sk#8, i_item_id#9, i_current_price#10] +Batched: true +Location [not included in comparison]/{warehouse_dir}/item] +PushedFilters: [IsNotNull(i_current_price), GreaterThanOrEqual(i_current_price,0.99), LessThanOrEqual(i_current_price,1.49), IsNotNull(i_item_sk)] +ReadSchema: struct<i_item_sk:int,i_item_id:string,i_current_price:decimal(7,2)> + +(11) CometFilter +Input [3]: [i_item_sk#8, i_item_id#9, i_current_price#10] +Condition : (((isnotnull(i_current_price#10) AND (i_current_price#10 >= 0.99)) AND (i_current_price#10 <= 1.49)) AND isnotnull(i_item_sk#8)) + +(12) CometProject +Input [3]: [i_item_sk#8, i_item_id#9, i_current_price#10] +Arguments: [i_item_sk#8, i_item_id#9], [i_item_sk#8, i_item_id#9] + +(13) ColumnarToRow [codegen id : 2] +Input [2]: [i_item_sk#8, i_item_id#9] + +(14) BroadcastExchange +Input [2]: [i_item_sk#8, i_item_id#9] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] + +(15) BroadcastHashJoin [codegen id : 4] +Left keys [1]: [inv_item_sk#1] +Right keys [1]: [i_item_sk#8] +Join type: Inner +Join condition: None + +(16) Project [codegen id : 4] +Output [4]: [inv_quantity_on_hand#3, inv_date_sk#4, w_warehouse_name#7, i_item_id#9] +Input [6]: [inv_item_sk#1, inv_quantity_on_hand#3, inv_date_sk#4, w_warehouse_name#7, i_item_sk#8, i_item_id#9] + +(17) ReusedExchange [Reuses operator id: 28] +Output [2]: [d_date_sk#11, d_date#12] + +(18) BroadcastHashJoin [codegen id : 4] +Left keys [1]: [inv_date_sk#4] +Right keys [1]: [d_date_sk#11] +Join type: Inner +Join condition: None + +(19) Project [codegen id : 4] +Output [4]: [inv_quantity_on_hand#3, w_warehouse_name#7, i_item_id#9, d_date#12] +Input [6]: [inv_quantity_on_hand#3, inv_date_sk#4, w_warehouse_name#7, i_item_id#9, d_date_sk#11, d_date#12] + +(20) HashAggregate [codegen id : 4] +Input [4]: [inv_quantity_on_hand#3, w_warehouse_name#7, i_item_id#9, d_date#12] +Keys [2]: [w_warehouse_name#7, i_item_id#9] +Functions [2]: [partial_sum(CASE WHEN (d_date#12 < 2000-03-11) THEN inv_quantity_on_hand#3 ELSE 0 END), partial_sum(CASE WHEN (d_date#12 >= 2000-03-11) THEN inv_quantity_on_hand#3 ELSE 0 END)] +Aggregate Attributes [2]: [sum#13, sum#14] +Results [4]: [w_warehouse_name#7, i_item_id#9, sum#15, sum#16] + +(21) Exchange +Input [4]: [w_warehouse_name#7, i_item_id#9, sum#15, sum#16] +Arguments: hashpartitioning(w_warehouse_name#7, i_item_id#9, 5), ENSURE_REQUIREMENTS, [plan_id=3] + +(22) HashAggregate [codegen id : 5] +Input [4]: [w_warehouse_name#7, i_item_id#9, sum#15, sum#16] +Keys [2]: [w_warehouse_name#7, i_item_id#9] +Functions [2]: [sum(CASE WHEN (d_date#12 < 2000-03-11) THEN inv_quantity_on_hand#3 ELSE 0 END), sum(CASE WHEN (d_date#12 >= 2000-03-11) THEN inv_quantity_on_hand#3 ELSE 0 END)] +Aggregate Attributes [2]: [sum(CASE WHEN (d_date#12 < 2000-03-11) THEN inv_quantity_on_hand#3 ELSE 0 END)#17, sum(CASE WHEN (d_date#12 >= 2000-03-11) THEN inv_quantity_on_hand#3 ELSE 0 END)#18] +Results [4]: [w_warehouse_name#7, i_item_id#9, sum(CASE WHEN (d_date#12 < 2000-03-11) THEN inv_quantity_on_hand#3 ELSE 0 END)#17 AS inv_before#19, sum(CASE WHEN (d_date#12 >= 2000-03-11) THEN inv_quantity_on_hand#3 ELSE 0 END)#18 AS inv_after#20] + +(23) Filter [codegen id : 5] +Input [4]: [w_warehouse_name#7, i_item_id#9, inv_before#19, inv_after#20] +Condition : (CASE WHEN (inv_before#19 > 0) THEN ((cast(inv_after#20 as double) / cast(inv_before#19 as double)) >= 0.666667) END AND CASE WHEN (inv_before#19 > 0) THEN ((cast(inv_after#20 as double) / cast(inv_before#19 as double)) <= 1.5) END) + +(24) TakeOrderedAndProject +Input [4]: [w_warehouse_name#7, i_item_id#9, inv_before#19, inv_after#20] +Arguments: 100, [w_warehouse_name#7 ASC NULLS FIRST, i_item_id#9 ASC NULLS FIRST], [w_warehouse_name#7, i_item_id#9, inv_before#19, inv_after#20] + +===== Subqueries ===== + +Subquery:1 Hosting operator id = 1 Hosting Expression = inv_date_sk#4 IN dynamicpruning#5 +BroadcastExchange (28) ++- * ColumnarToRow (27) + +- CometFilter (26) + +- CometScan parquet spark_catalog.default.date_dim (25) + + +(25) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#11, d_date#12] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,2000-02-10), LessThanOrEqual(d_date,2000-04-10), IsNotNull(d_date_sk)] +ReadSchema: struct<d_date_sk:int,d_date:date> + +(26) CometFilter +Input [2]: [d_date_sk#11, d_date#12] +Condition : (((isnotnull(d_date#12) AND (d_date#12 >= 2000-02-10)) AND (d_date#12 <= 2000-04-10)) AND isnotnull(d_date_sk#11)) + +(27) ColumnarToRow [codegen id : 1] +Input [2]: [d_date_sk#11, d_date#12] + +(28) BroadcastExchange +Input [2]: [d_date_sk#11, d_date#12] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=4] + + 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 new file mode 100644 index 000000000..e20755e12 --- /dev/null +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q21/simplified.txt @@ -0,0 +1,42 @@ +TakeOrderedAndProject [w_warehouse_name,i_item_id,inv_before,inv_after] + WholeStageCodegen (5) + Filter [inv_before,inv_after] + HashAggregate [w_warehouse_name,i_item_id,sum,sum] [sum(CASE WHEN (d_date < 2000-03-11) THEN inv_quantity_on_hand ELSE 0 END),sum(CASE WHEN (d_date >= 2000-03-11) THEN inv_quantity_on_hand ELSE 0 END),inv_before,inv_after,sum,sum] + InputAdapter + Exchange [w_warehouse_name,i_item_id] #1 + WholeStageCodegen (4) + HashAggregate [w_warehouse_name,i_item_id,d_date,inv_quantity_on_hand] [sum,sum,sum,sum] + Project [inv_quantity_on_hand,w_warehouse_name,i_item_id,d_date] + BroadcastHashJoin [inv_date_sk,d_date_sk] + Project [inv_quantity_on_hand,inv_date_sk,w_warehouse_name,i_item_id] + BroadcastHashJoin [inv_item_sk,i_item_sk] + Project [inv_item_sk,inv_quantity_on_hand,inv_date_sk,w_warehouse_name] + BroadcastHashJoin [inv_warehouse_sk,w_warehouse_sk] + ColumnarToRow + InputAdapter + CometFilter [inv_warehouse_sk,inv_item_sk] + CometScan parquet spark_catalog.default.inventory [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #2 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometFilter [d_date,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] + InputAdapter + BroadcastExchange #3 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometFilter [w_warehouse_sk] + CometScan parquet spark_catalog.default.warehouse [w_warehouse_sk,w_warehouse_name] + InputAdapter + BroadcastExchange #4 + WholeStageCodegen (2) + ColumnarToRow + InputAdapter + CometProject [i_item_sk,i_item_id] + CometFilter [i_current_price,i_item_sk] + CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_current_price] + InputAdapter + ReusedExchange [d_date_sk,d_date] #2 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 new file mode 100644 index 000000000..9f5771fed --- /dev/null +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q22/explain.txt @@ -0,0 +1,169 @@ +== Physical Plan == +TakeOrderedAndProject (23) ++- * HashAggregate (22) + +- Exchange (21) + +- * HashAggregate (20) + +- * Expand (19) + +- * Project (18) + +- * BroadcastHashJoin Inner BuildRight (17) + :- * Project (12) + : +- * BroadcastHashJoin Inner BuildRight (11) + : :- * Project (6) + : : +- * BroadcastHashJoin Inner BuildRight (5) + : : :- * ColumnarToRow (3) + : : : +- CometFilter (2) + : : : +- CometScan parquet spark_catalog.default.inventory (1) + : : +- ReusedExchange (4) + : +- BroadcastExchange (10) + : +- * ColumnarToRow (9) + : +- CometFilter (8) + : +- CometScan parquet spark_catalog.default.item (7) + +- BroadcastExchange (16) + +- * ColumnarToRow (15) + +- CometFilter (14) + +- CometScan parquet spark_catalog.default.warehouse (13) + + +(1) Scan parquet spark_catalog.default.inventory +Output [4]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4] +Batched: true +Location: InMemoryFileIndex [] +PartitionFilters: [isnotnull(inv_date_sk#4), dynamicpruningexpression(inv_date_sk#4 IN dynamicpruning#5)] +PushedFilters: [IsNotNull(inv_item_sk), IsNotNull(inv_warehouse_sk)] +ReadSchema: struct<inv_item_sk:int,inv_warehouse_sk:int,inv_quantity_on_hand:int> + +(2) CometFilter +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)) + +(3) ColumnarToRow [codegen id : 4] +Input [4]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4] + +(4) ReusedExchange [Reuses operator id: 28] +Output [1]: [d_date_sk#6] + +(5) BroadcastHashJoin [codegen id : 4] +Left keys [1]: [inv_date_sk#4] +Right keys [1]: [d_date_sk#6] +Join type: Inner +Join condition: None + +(6) Project [codegen id : 4] +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] + +(7) Scan parquet spark_catalog.default.item +Output [5]: [i_item_sk#7, i_brand#8, i_class#9, i_category#10, i_product_name#11] +Batched: true +Location [not included in comparison]/{warehouse_dir}/item] +PushedFilters: [IsNotNull(i_item_sk)] +ReadSchema: struct<i_item_sk:int,i_brand:string,i_class:string,i_category:string,i_product_name:string> + +(8) CometFilter +Input [5]: [i_item_sk#7, i_brand#8, i_class#9, i_category#10, i_product_name#11] +Condition : isnotnull(i_item_sk#7) + +(9) ColumnarToRow [codegen id : 2] +Input [5]: [i_item_sk#7, i_brand#8, i_class#9, i_category#10, i_product_name#11] + +(10) BroadcastExchange +Input [5]: [i_item_sk#7, i_brand#8, i_class#9, i_category#10, i_product_name#11] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=1] + +(11) BroadcastHashJoin [codegen id : 4] +Left keys [1]: [inv_item_sk#1] +Right keys [1]: [i_item_sk#7] +Join type: Inner +Join condition: None + +(12) Project [codegen id : 4] +Output [6]: [inv_warehouse_sk#2, inv_quantity_on_hand#3, i_brand#8, i_class#9, i_category#10, i_product_name#11] +Input [8]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3, i_item_sk#7, i_brand#8, i_class#9, i_category#10, i_product_name#11] + +(13) Scan parquet spark_catalog.default.warehouse +Output [1]: [w_warehouse_sk#12] +Batched: true +Location [not included in comparison]/{warehouse_dir}/warehouse] +PushedFilters: [IsNotNull(w_warehouse_sk)] +ReadSchema: struct<w_warehouse_sk:int> + +(14) CometFilter +Input [1]: [w_warehouse_sk#12] +Condition : isnotnull(w_warehouse_sk#12) + +(15) ColumnarToRow [codegen id : 3] +Input [1]: [w_warehouse_sk#12] + +(16) BroadcastExchange +Input [1]: [w_warehouse_sk#12] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=2] + +(17) BroadcastHashJoin [codegen id : 4] +Left keys [1]: [inv_warehouse_sk#2] +Right keys [1]: [w_warehouse_sk#12] +Join type: Inner +Join condition: None + +(18) Project [codegen id : 4] +Output [5]: [inv_quantity_on_hand#3, i_product_name#11, i_brand#8, i_class#9, i_category#10] +Input [7]: [inv_warehouse_sk#2, inv_quantity_on_hand#3, i_brand#8, i_class#9, i_category#10, i_product_name#11, w_warehouse_sk#12] + +(19) Expand [codegen id : 4] +Input [5]: [inv_quantity_on_hand#3, i_product_name#11, i_brand#8, i_class#9, i_category#10] +Arguments: [[inv_quantity_on_hand#3, i_product_name#11, i_brand#8, i_class#9, i_category#10, 0], [inv_quantity_on_hand#3, i_product_name#11, i_brand#8, i_class#9, null, 1], [inv_quantity_on_hand#3, i_product_name#11, i_brand#8, null, null, 3], [inv_quantity_on_hand#3, i_product_name#11, null, null, null, 7], [inv_quantity_on_hand#3, null, null, null, null, 15]], [inv_quantity_on_hand#3, i_product_name#13, i_brand#14, i_class#15, i_category#16, spark_grouping_id#17] + +(20) HashAggregate [codegen id : 4] +Input [6]: [inv_quantity_on_hand#3, i_product_name#13, i_brand#14, i_class#15, i_category#16, spark_grouping_id#17] +Keys [5]: [i_product_name#13, i_brand#14, i_class#15, i_category#16, spark_grouping_id#17] +Functions [1]: [partial_avg(inv_quantity_on_hand#3)] +Aggregate Attributes [2]: [sum#18, count#19] +Results [7]: [i_product_name#13, i_brand#14, i_class#15, i_category#16, spark_grouping_id#17, sum#20, count#21] + +(21) Exchange +Input [7]: [i_product_name#13, i_brand#14, i_class#15, i_category#16, spark_grouping_id#17, sum#20, count#21] +Arguments: hashpartitioning(i_product_name#13, i_brand#14, i_class#15, i_category#16, spark_grouping_id#17, 5), ENSURE_REQUIREMENTS, [plan_id=3] + +(22) HashAggregate [codegen id : 5] +Input [7]: [i_product_name#13, i_brand#14, i_class#15, i_category#16, spark_grouping_id#17, sum#20, count#21] +Keys [5]: [i_product_name#13, i_brand#14, i_class#15, i_category#16, spark_grouping_id#17] +Functions [1]: [avg(inv_quantity_on_hand#3)] +Aggregate Attributes [1]: [avg(inv_quantity_on_hand#3)#22] +Results [5]: [i_product_name#13, i_brand#14, i_class#15, i_category#16, avg(inv_quantity_on_hand#3)#22 AS qoh#23] + +(23) TakeOrderedAndProject +Input [5]: [i_product_name#13, i_brand#14, i_class#15, i_category#16, qoh#23] +Arguments: 100, [qoh#23 ASC NULLS FIRST, i_product_name#13 ASC NULLS FIRST, i_brand#14 ASC NULLS FIRST, i_class#15 ASC NULLS FIRST, i_category#16 ASC NULLS FIRST], [i_product_name#13, i_brand#14, i_class#15, i_category#16, qoh#23] + +===== Subqueries ===== + +Subquery:1 Hosting operator id = 1 Hosting Expression = inv_date_sk#4 IN dynamicpruning#5 +BroadcastExchange (28) ++- * ColumnarToRow (27) + +- CometProject (26) + +- CometFilter (25) + +- CometScan parquet spark_catalog.default.date_dim (24) + + +(24) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#6, d_month_seq#24] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1200), LessThanOrEqual(d_month_seq,1211), IsNotNull(d_date_sk)] +ReadSchema: struct<d_date_sk:int,d_month_seq:int> + +(25) CometFilter +Input [2]: [d_date_sk#6, d_month_seq#24] +Condition : (((isnotnull(d_month_seq#24) AND (d_month_seq#24 >= 1200)) AND (d_month_seq#24 <= 1211)) AND isnotnull(d_date_sk#6)) + +(26) CometProject +Input [2]: [d_date_sk#6, d_month_seq#24] +Arguments: [d_date_sk#6], [d_date_sk#6] + +(27) ColumnarToRow [codegen id : 1] +Input [1]: [d_date_sk#6] + +(28) BroadcastExchange +Input [1]: [d_date_sk#6] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] + + diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q22/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q22/simplified.txt new file mode 100644 index 000000000..92714bb02 --- /dev/null +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q22/simplified.txt @@ -0,0 +1,42 @@ +TakeOrderedAndProject [qoh,i_product_name,i_brand,i_class,i_category] + WholeStageCodegen (5) + HashAggregate [i_product_name,i_brand,i_class,i_category,spark_grouping_id,sum,count] [avg(inv_quantity_on_hand),qoh,sum,count] + InputAdapter + Exchange [i_product_name,i_brand,i_class,i_category,spark_grouping_id] #1 + WholeStageCodegen (4) + HashAggregate [i_product_name,i_brand,i_class,i_category,spark_grouping_id,inv_quantity_on_hand] [sum,count,sum,count] + Expand [inv_quantity_on_hand,i_product_name,i_brand,i_class,i_category] + Project [inv_quantity_on_hand,i_product_name,i_brand,i_class,i_category] + BroadcastHashJoin [inv_warehouse_sk,w_warehouse_sk] + Project [inv_warehouse_sk,inv_quantity_on_hand,i_brand,i_class,i_category,i_product_name] + BroadcastHashJoin [inv_item_sk,i_item_sk] + Project [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand] + BroadcastHashJoin [inv_date_sk,d_date_sk] + ColumnarToRow + InputAdapter + CometFilter [inv_item_sk,inv_warehouse_sk] + CometScan parquet spark_catalog.default.inventory [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #2 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_month_seq,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] + InputAdapter + ReusedExchange [d_date_sk] #2 + InputAdapter + BroadcastExchange #3 + WholeStageCodegen (2) + ColumnarToRow + InputAdapter + CometFilter [i_item_sk] + CometScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category,i_product_name] + InputAdapter + BroadcastExchange #4 + WholeStageCodegen (3) + ColumnarToRow + InputAdapter + CometFilter [w_warehouse_sk] + CometScan parquet spark_catalog.default.warehouse [w_warehouse_sk] 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 new file mode 100644 index 000000000..328a8d353 --- /dev/null +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q23a/explain.txt @@ -0,0 +1,570 @@ +== Physical Plan == +* HashAggregate (66) ++- Exchange (65) + +- * HashAggregate (64) + +- Union (63) + :- * Project (45) + : +- * BroadcastHashJoin Inner BuildRight (44) + : :- * Project (42) + : : +- * SortMergeJoin LeftSemi (41) + : : :- * Sort (24) + : : : +- Exchange (23) + : : : +- * Project (22) + : : : +- * BroadcastHashJoin LeftSemi BuildRight (21) + : : : :- * ColumnarToRow (2) + : : : : +- CometScan parquet spark_catalog.default.catalog_sales (1) + : : : +- BroadcastExchange (20) + : : : +- * Project (19) + : : : +- * Filter (18) + : : : +- * HashAggregate (17) + : : : +- Exchange (16) + : : : +- * HashAggregate (15) + : : : +- * Project (14) + : : : +- * BroadcastHashJoin Inner BuildRight (13) + : : : :- * Project (8) + : : : : +- * BroadcastHashJoin Inner BuildRight (7) + : : : : :- * ColumnarToRow (5) + : : : : : +- CometFilter (4) + : : : : : +- CometScan parquet spark_catalog.default.store_sales (3) + : : : : +- ReusedExchange (6) + : : : +- BroadcastExchange (12) + : : : +- * ColumnarToRow (11) + : : : +- CometFilter (10) + : : : +- CometScan parquet spark_catalog.default.item (9) + : : +- * Sort (40) + : : +- * Project (39) + : : +- * Filter (38) + : : +- * HashAggregate (37) + : : +- Exchange (36) + : : +- * HashAggregate (35) + : : +- * Project (34) + : : +- * BroadcastHashJoin Inner BuildRight (33) + : : :- * ColumnarToRow (28) + : : : +- CometProject (27) + : : : +- CometFilter (26) + : : : +- CometScan parquet spark_catalog.default.store_sales (25) + : : +- BroadcastExchange (32) + : : +- * ColumnarToRow (31) + : : +- CometFilter (30) + : : +- CometScan parquet spark_catalog.default.customer (29) + : +- ReusedExchange (43) + +- * Project (62) + +- * BroadcastHashJoin Inner BuildRight (61) + :- * Project (59) + : +- * SortMergeJoin LeftSemi (58) + : :- * Sort (52) + : : +- Exchange (51) + : : +- * Project (50) + : : +- * BroadcastHashJoin LeftSemi BuildRight (49) + : : :- * ColumnarToRow (47) + : : : +- CometScan parquet spark_catalog.default.web_sales (46) + : : +- ReusedExchange (48) + : +- * Sort (57) + : +- * Project (56) + : +- * Filter (55) + : +- * HashAggregate (54) + : +- ReusedExchange (53) + +- ReusedExchange (60) + + +(1) Scan parquet spark_catalog.default.catalog_sales +Output [5]: [cs_bill_customer_sk#1, cs_item_sk#2, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] +Batched: true +Location: InMemoryFileIndex [] +PartitionFilters: [isnotnull(cs_sold_date_sk#5), dynamicpruningexpression(cs_sold_date_sk#5 IN dynamicpruning#6)] +ReadSchema: struct<cs_bill_customer_sk:int,cs_item_sk:int,cs_quantity:int,cs_list_price:decimal(7,2)> + +(2) ColumnarToRow [codegen id : 5] +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 +Output [2]: [ss_item_sk#7, ss_sold_date_sk#8] +Batched: true +Location: InMemoryFileIndex [] +PartitionFilters: [isnotnull(ss_sold_date_sk#8), dynamicpruningexpression(ss_sold_date_sk#8 IN dynamicpruning#9)] +PushedFilters: [IsNotNull(ss_item_sk)] +ReadSchema: struct<ss_item_sk:int> + +(4) CometFilter +Input [2]: [ss_item_sk#7, ss_sold_date_sk#8] +Condition : isnotnull(ss_item_sk#7) + +(5) ColumnarToRow [codegen id : 3] +Input [2]: [ss_item_sk#7, ss_sold_date_sk#8] + +(6) ReusedExchange [Reuses operator id: 76] +Output [2]: [d_date_sk#10, d_date#11] + +(7) BroadcastHashJoin [codegen id : 3] +Left keys [1]: [ss_sold_date_sk#8] +Right keys [1]: [d_date_sk#10] +Join type: Inner +Join condition: None + +(8) Project [codegen id : 3] +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] + +(9) Scan parquet spark_catalog.default.item +Output [2]: [i_item_sk#12, i_item_desc#13] +Batched: true +Location [not included in comparison]/{warehouse_dir}/item] +PushedFilters: [IsNotNull(i_item_sk)] +ReadSchema: struct<i_item_sk:int,i_item_desc:string> + +(10) CometFilter +Input [2]: [i_item_sk#12, i_item_desc#13] +Condition : isnotnull(i_item_sk#12) + +(11) ColumnarToRow [codegen id : 2] +Input [2]: [i_item_sk#12, i_item_desc#13] + +(12) 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] +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] +Output [3]: [d_date#11, i_item_sk#12, substr(i_item_desc#13, 1, 30) AS _groupingexpression#14] +Input [4]: [ss_item_sk#7, d_date#11, i_item_sk#12, i_item_desc#13] + +(15) HashAggregate [codegen id : 3] +Input [3]: [d_date#11, i_item_sk#12, _groupingexpression#14] +Keys [3]: [_groupingexpression#14, i_item_sk#12, d_date#11] +Functions [1]: [partial_count(1)] +Aggregate Attributes [1]: [count#15] +Results [4]: [_groupingexpression#14, i_item_sk#12, d_date#11, count#16] + +(16) Exchange +Input [4]: [_groupingexpression#14, i_item_sk#12, d_date#11, count#16] +Arguments: hashpartitioning(_groupingexpression#14, i_item_sk#12, d_date#11, 5), ENSURE_REQUIREMENTS, [plan_id=2] + +(17) HashAggregate [codegen id : 4] +Input [4]: [_groupingexpression#14, i_item_sk#12, d_date#11, count#16] +Keys [3]: [_groupingexpression#14, i_item_sk#12, d_date#11] +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] + +(18) Filter [codegen id : 4] +Input [2]: [item_sk#18, cnt#19] +Condition : (cnt#19 > 4) + +(19) Project [codegen id : 4] +Output [1]: [item_sk#18] +Input [2]: [item_sk#18, cnt#19] + +(20) BroadcastExchange +Input [1]: [item_sk#18] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] + +(21) BroadcastHashJoin [codegen id : 5] +Left keys [1]: [cs_item_sk#2] +Right keys [1]: [item_sk#18] +Join type: LeftSemi +Join condition: None + +(22) Project [codegen id : 5] +Output [4]: [cs_bill_customer_sk#1, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] +Input [5]: [cs_bill_customer_sk#1, cs_item_sk#2, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] + +(23) Exchange +Input [4]: [cs_bill_customer_sk#1, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] +Arguments: hashpartitioning(cs_bill_customer_sk#1, 5), ENSURE_REQUIREMENTS, [plan_id=4] + +(24) Sort [codegen id : 6] +Input [4]: [cs_bill_customer_sk#1, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] +Arguments: [cs_bill_customer_sk#1 ASC NULLS FIRST], false, 0 + +(25) Scan parquet spark_catalog.default.store_sales +Output [4]: [ss_customer_sk#20, ss_quantity#21, ss_sales_price#22, ss_sold_date_sk#23] +Batched: true +Location [not included in comparison]/{warehouse_dir}/store_sales] +PushedFilters: [IsNotNull(ss_customer_sk)] +ReadSchema: struct<ss_customer_sk:int,ss_quantity:int,ss_sales_price:decimal(7,2)> + +(26) CometFilter +Input [4]: [ss_customer_sk#20, ss_quantity#21, ss_sales_price#22, ss_sold_date_sk#23] +Condition : isnotnull(ss_customer_sk#20) + +(27) CometProject +Input [4]: [ss_customer_sk#20, ss_quantity#21, ss_sales_price#22, ss_sold_date_sk#23] +Arguments: [ss_customer_sk#20, ss_quantity#21, ss_sales_price#22], [ss_customer_sk#20, ss_quantity#21, ss_sales_price#22] + +(28) ColumnarToRow [codegen id : 8] +Input [3]: [ss_customer_sk#20, ss_quantity#21, ss_sales_price#22] + +(29) Scan parquet spark_catalog.default.customer +Output [1]: [c_customer_sk#24] +Batched: true +Location [not included in comparison]/{warehouse_dir}/customer] +PushedFilters: [IsNotNull(c_customer_sk)] +ReadSchema: struct<c_customer_sk:int> + +(30) CometFilter +Input [1]: [c_customer_sk#24] +Condition : isnotnull(c_customer_sk#24) + +(31) ColumnarToRow [codegen id : 7] +Input [1]: [c_customer_sk#24] + +(32) BroadcastExchange +Input [1]: [c_customer_sk#24] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=5] + +(33) BroadcastHashJoin [codegen id : 8] +Left keys [1]: [ss_customer_sk#20] +Right keys [1]: [c_customer_sk#24] +Join type: Inner +Join condition: None + +(34) Project [codegen id : 8] +Output [3]: [ss_quantity#21, ss_sales_price#22, c_customer_sk#24] +Input [4]: [ss_customer_sk#20, ss_quantity#21, ss_sales_price#22, c_customer_sk#24] + +(35) HashAggregate [codegen id : 8] +Input [3]: [ss_quantity#21, ss_sales_price#22, c_customer_sk#24] +Keys [1]: [c_customer_sk#24] +Functions [1]: [partial_sum((cast(ss_quantity#21 as decimal(10,0)) * ss_sales_price#22))] +Aggregate Attributes [2]: [sum#25, isEmpty#26] +Results [3]: [c_customer_sk#24, sum#27, isEmpty#28] + +(36) Exchange +Input [3]: [c_customer_sk#24, sum#27, isEmpty#28] +Arguments: hashpartitioning(c_customer_sk#24, 5), ENSURE_REQUIREMENTS, [plan_id=6] + +(37) HashAggregate [codegen id : 9] +Input [3]: [c_customer_sk#24, sum#27, isEmpty#28] +Keys [1]: [c_customer_sk#24] +Functions [1]: [sum((cast(ss_quantity#21 as decimal(10,0)) * ss_sales_price#22))] +Aggregate Attributes [1]: [sum((cast(ss_quantity#21 as decimal(10,0)) * ss_sales_price#22))#29] +Results [2]: [c_customer_sk#24, sum((cast(ss_quantity#21 as decimal(10,0)) * ss_sales_price#22))#29 AS ssales#30] + +(38) Filter [codegen id : 9] +Input [2]: [c_customer_sk#24, ssales#30] +Condition : (isnotnull(ssales#30) AND (cast(ssales#30 as decimal(38,8)) > (0.500000 * Subquery scalar-subquery#31, [id=#32]))) + +(39) Project [codegen id : 9] +Output [1]: [c_customer_sk#24] +Input [2]: [c_customer_sk#24, ssales#30] + +(40) Sort [codegen id : 9] +Input [1]: [c_customer_sk#24] +Arguments: [c_customer_sk#24 ASC NULLS FIRST], false, 0 + +(41) SortMergeJoin [codegen id : 11] +Left keys [1]: [cs_bill_customer_sk#1] +Right keys [1]: [c_customer_sk#24] +Join type: LeftSemi +Join condition: None + +(42) Project [codegen id : 11] +Output [3]: [cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] +Input [4]: [cs_bill_customer_sk#1, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] + +(43) ReusedExchange [Reuses operator id: 71] +Output [1]: [d_date_sk#33] + +(44) BroadcastHashJoin [codegen id : 11] +Left keys [1]: [cs_sold_date_sk#5] +Right keys [1]: [d_date_sk#33] +Join type: Inner +Join condition: None + +(45) Project [codegen id : 11] +Output [1]: [(cast(cs_quantity#3 as decimal(10,0)) * cs_list_price#4) AS sales#34] +Input [4]: [cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5, d_date_sk#33] + +(46) Scan parquet spark_catalog.default.web_sales +Output [5]: [ws_item_sk#35, ws_bill_customer_sk#36, ws_quantity#37, ws_list_price#38, ws_sold_date_sk#39] +Batched: true +Location: InMemoryFileIndex [] +PartitionFilters: [isnotnull(ws_sold_date_sk#39), dynamicpruningexpression(ws_sold_date_sk#39 IN dynamicpruning#40)] +ReadSchema: struct<ws_item_sk:int,ws_bill_customer_sk:int,ws_quantity:int,ws_list_price:decimal(7,2)> + +(47) ColumnarToRow [codegen id : 16] +Input [5]: [ws_item_sk#35, ws_bill_customer_sk#36, ws_quantity#37, ws_list_price#38, ws_sold_date_sk#39] + +(48) ReusedExchange [Reuses operator id: 20] +Output [1]: [item_sk#41] + +(49) BroadcastHashJoin [codegen id : 16] +Left keys [1]: [ws_item_sk#35] +Right keys [1]: [item_sk#41] +Join type: LeftSemi +Join condition: None + +(50) Project [codegen id : 16] +Output [4]: [ws_bill_customer_sk#36, ws_quantity#37, ws_list_price#38, ws_sold_date_sk#39] +Input [5]: [ws_item_sk#35, ws_bill_customer_sk#36, ws_quantity#37, ws_list_price#38, ws_sold_date_sk#39] + +(51) Exchange +Input [4]: [ws_bill_customer_sk#36, ws_quantity#37, ws_list_price#38, ws_sold_date_sk#39] +Arguments: hashpartitioning(ws_bill_customer_sk#36, 5), ENSURE_REQUIREMENTS, [plan_id=7] + +(52) Sort [codegen id : 17] +Input [4]: [ws_bill_customer_sk#36, ws_quantity#37, ws_list_price#38, ws_sold_date_sk#39] +Arguments: [ws_bill_customer_sk#36 ASC NULLS FIRST], false, 0 + +(53) ReusedExchange [Reuses operator id: 36] +Output [3]: [c_customer_sk#42, sum#43, isEmpty#44] + +(54) HashAggregate [codegen id : 20] +Input [3]: [c_customer_sk#42, sum#43, isEmpty#44] +Keys [1]: [c_customer_sk#42] +Functions [1]: [sum((cast(ss_quantity#45 as decimal(10,0)) * ss_sales_price#46))] +Aggregate Attributes [1]: [sum((cast(ss_quantity#45 as decimal(10,0)) * ss_sales_price#46))#29] +Results [2]: [c_customer_sk#42, sum((cast(ss_quantity#45 as decimal(10,0)) * ss_sales_price#46))#29 AS ssales#47] + +(55) Filter [codegen id : 20] +Input [2]: [c_customer_sk#42, ssales#47] +Condition : (isnotnull(ssales#47) AND (cast(ssales#47 as decimal(38,8)) > (0.500000 * ReusedSubquery Subquery scalar-subquery#31, [id=#32]))) + +(56) Project [codegen id : 20] +Output [1]: [c_customer_sk#42] +Input [2]: [c_customer_sk#42, ssales#47] + +(57) Sort [codegen id : 20] +Input [1]: [c_customer_sk#42] +Arguments: [c_customer_sk#42 ASC NULLS FIRST], false, 0 + +(58) SortMergeJoin [codegen id : 22] +Left keys [1]: [ws_bill_customer_sk#36] +Right keys [1]: [c_customer_sk#42] +Join type: LeftSemi +Join condition: None + +(59) Project [codegen id : 22] +Output [3]: [ws_quantity#37, ws_list_price#38, ws_sold_date_sk#39] +Input [4]: [ws_bill_customer_sk#36, ws_quantity#37, ws_list_price#38, ws_sold_date_sk#39] + +(60) ReusedExchange [Reuses operator id: 71] +Output [1]: [d_date_sk#48] + +(61) BroadcastHashJoin [codegen id : 22] +Left keys [1]: [ws_sold_date_sk#39] +Right keys [1]: [d_date_sk#48] +Join type: Inner +Join condition: None + +(62) Project [codegen id : 22] +Output [1]: [(cast(ws_quantity#37 as decimal(10,0)) * ws_list_price#38) AS sales#49] +Input [4]: [ws_quantity#37, ws_list_price#38, ws_sold_date_sk#39, d_date_sk#48] + +(63) Union + +(64) HashAggregate [codegen id : 23] +Input [1]: [sales#34] +Keys: [] +Functions [1]: [partial_sum(sales#34)] +Aggregate Attributes [2]: [sum#50, isEmpty#51] +Results [2]: [sum#52, isEmpty#53] + +(65) Exchange +Input [2]: [sum#52, isEmpty#53] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=8] + +(66) HashAggregate [codegen id : 24] +Input [2]: [sum#52, isEmpty#53] +Keys: [] +Functions [1]: [sum(sales#34)] +Aggregate Attributes [1]: [sum(sales#34)#54] +Results [1]: [sum(sales#34)#54 AS sum(sales)#55] + +===== Subqueries ===== + +Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#5 IN dynamicpruning#6 +BroadcastExchange (71) ++- * ColumnarToRow (70) + +- CometProject (69) + +- CometFilter (68) + +- CometScan parquet spark_catalog.default.date_dim (67) + + +(67) Scan parquet spark_catalog.default.date_dim +Output [3]: [d_date_sk#33, d_year#56, d_moy#57] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2000), EqualTo(d_moy,2), IsNotNull(d_date_sk)] +ReadSchema: struct<d_date_sk:int,d_year:int,d_moy:int> + +(68) CometFilter +Input [3]: [d_date_sk#33, d_year#56, d_moy#57] +Condition : ((((isnotnull(d_year#56) AND isnotnull(d_moy#57)) AND (d_year#56 = 2000)) AND (d_moy#57 = 2)) AND isnotnull(d_date_sk#33)) + +(69) CometProject +Input [3]: [d_date_sk#33, d_year#56, d_moy#57] +Arguments: [d_date_sk#33], [d_date_sk#33] + +(70) ColumnarToRow [codegen id : 1] +Input [1]: [d_date_sk#33] + +(71) BroadcastExchange +Input [1]: [d_date_sk#33] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=9] + +Subquery:2 Hosting operator id = 3 Hosting Expression = ss_sold_date_sk#8 IN dynamicpruning#9 +BroadcastExchange (76) ++- * ColumnarToRow (75) + +- CometProject (74) + +- CometFilter (73) + +- CometScan parquet spark_catalog.default.date_dim (72) + + +(72) Scan parquet spark_catalog.default.date_dim +Output [3]: [d_date_sk#10, d_date#11, d_year#58] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [In(d_year, [2000,2001,2002,2003]), IsNotNull(d_date_sk)] +ReadSchema: struct<d_date_sk:int,d_date:date,d_year:int> + +(73) CometFilter +Input [3]: [d_date_sk#10, d_date#11, d_year#58] +Condition : (d_year#58 IN (2000,2001,2002,2003) AND isnotnull(d_date_sk#10)) + +(74) CometProject +Input [3]: [d_date_sk#10, d_date#11, d_year#58] +Arguments: [d_date_sk#10, d_date#11], [d_date_sk#10, d_date#11] + +(75) ColumnarToRow [codegen id : 1] +Input [2]: [d_date_sk#10, d_date#11] + +(76) BroadcastExchange +Input [2]: [d_date_sk#10, d_date#11] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=10] + +Subquery:3 Hosting operator id = 38 Hosting Expression = Subquery scalar-subquery#31, [id=#32] +* HashAggregate (91) ++- Exchange (90) + +- * HashAggregate (89) + +- * HashAggregate (88) + +- Exchange (87) + +- * HashAggregate (86) + +- * Project (85) + +- * BroadcastHashJoin Inner BuildRight (84) + :- * Project (82) + : +- * BroadcastHashJoin Inner BuildRight (81) + : :- * ColumnarToRow (79) + : : +- CometFilter (78) + : : +- CometScan parquet spark_catalog.default.store_sales (77) + : +- ReusedExchange (80) + +- ReusedExchange (83) + + +(77) Scan parquet spark_catalog.default.store_sales +Output [4]: [ss_customer_sk#59, ss_quantity#60, ss_sales_price#61, ss_sold_date_sk#62] +Batched: true +Location: InMemoryFileIndex [] +PartitionFilters: [isnotnull(ss_sold_date_sk#62), dynamicpruningexpression(ss_sold_date_sk#62 IN dynamicpruning#63)] +PushedFilters: [IsNotNull(ss_customer_sk)] +ReadSchema: struct<ss_customer_sk:int,ss_quantity:int,ss_sales_price:decimal(7,2)> + +(78) CometFilter +Input [4]: [ss_customer_sk#59, ss_quantity#60, ss_sales_price#61, ss_sold_date_sk#62] +Condition : isnotnull(ss_customer_sk#59) + +(79) ColumnarToRow [codegen id : 3] +Input [4]: [ss_customer_sk#59, ss_quantity#60, ss_sales_price#61, ss_sold_date_sk#62] + +(80) ReusedExchange [Reuses operator id: 32] +Output [1]: [c_customer_sk#64] + +(81) BroadcastHashJoin [codegen id : 3] +Left keys [1]: [ss_customer_sk#59] +Right keys [1]: [c_customer_sk#64] +Join type: Inner +Join condition: None + +(82) Project [codegen id : 3] +Output [4]: [ss_quantity#60, ss_sales_price#61, ss_sold_date_sk#62, c_customer_sk#64] +Input [5]: [ss_customer_sk#59, ss_quantity#60, ss_sales_price#61, ss_sold_date_sk#62, c_customer_sk#64] + +(83) ReusedExchange [Reuses operator id: 96] +Output [1]: [d_date_sk#65] + +(84) BroadcastHashJoin [codegen id : 3] +Left keys [1]: [ss_sold_date_sk#62] +Right keys [1]: [d_date_sk#65] +Join type: Inner +Join condition: None + +(85) Project [codegen id : 3] +Output [3]: [ss_quantity#60, ss_sales_price#61, c_customer_sk#64] +Input [5]: [ss_quantity#60, ss_sales_price#61, ss_sold_date_sk#62, c_customer_sk#64, d_date_sk#65] + +(86) HashAggregate [codegen id : 3] +Input [3]: [ss_quantity#60, ss_sales_price#61, c_customer_sk#64] +Keys [1]: [c_customer_sk#64] +Functions [1]: [partial_sum((cast(ss_quantity#60 as decimal(10,0)) * ss_sales_price#61))] +Aggregate Attributes [2]: [sum#66, isEmpty#67] +Results [3]: [c_customer_sk#64, sum#68, isEmpty#69] + +(87) Exchange +Input [3]: [c_customer_sk#64, sum#68, isEmpty#69] +Arguments: hashpartitioning(c_customer_sk#64, 5), ENSURE_REQUIREMENTS, [plan_id=11] + +(88) HashAggregate [codegen id : 4] +Input [3]: [c_customer_sk#64, sum#68, isEmpty#69] +Keys [1]: [c_customer_sk#64] +Functions [1]: [sum((cast(ss_quantity#60 as decimal(10,0)) * ss_sales_price#61))] +Aggregate Attributes [1]: [sum((cast(ss_quantity#60 as decimal(10,0)) * ss_sales_price#61))#70] +Results [1]: [sum((cast(ss_quantity#60 as decimal(10,0)) * ss_sales_price#61))#70 AS csales#71] + +(89) HashAggregate [codegen id : 4] +Input [1]: [csales#71] +Keys: [] +Functions [1]: [partial_max(csales#71)] +Aggregate Attributes [1]: [max#72] +Results [1]: [max#73] + +(90) Exchange +Input [1]: [max#73] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=12] + +(91) HashAggregate [codegen id : 5] +Input [1]: [max#73] +Keys: [] +Functions [1]: [max(csales#71)] +Aggregate Attributes [1]: [max(csales#71)#74] +Results [1]: [max(csales#71)#74 AS tpcds_cmax#75] + +Subquery:4 Hosting operator id = 77 Hosting Expression = ss_sold_date_sk#62 IN dynamicpruning#63 +BroadcastExchange (96) ++- * ColumnarToRow (95) + +- CometProject (94) + +- CometFilter (93) + +- CometScan parquet spark_catalog.default.date_dim (92) + + +(92) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#65, d_year#76] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [In(d_year, [2000,2001,2002,2003]), IsNotNull(d_date_sk)] +ReadSchema: struct<d_date_sk:int,d_year:int> + +(93) CometFilter +Input [2]: [d_date_sk#65, d_year#76] +Condition : (d_year#76 IN (2000,2001,2002,2003) AND isnotnull(d_date_sk#65)) + +(94) CometProject +Input [2]: [d_date_sk#65, d_year#76] +Arguments: [d_date_sk#65], [d_date_sk#65] + +(95) ColumnarToRow [codegen id : 1] +Input [1]: [d_date_sk#65] + +(96) BroadcastExchange +Input [1]: [d_date_sk#65] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=13] + +Subquery:5 Hosting operator id = 46 Hosting Expression = ws_sold_date_sk#39 IN dynamicpruning#6 + +Subquery:6 Hosting operator id = 55 Hosting Expression = ReusedSubquery Subquery scalar-subquery#31, [id=#32] + + 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 new file mode 100644 index 000000000..0ec56d0e7 --- /dev/null +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q23a/simplified.txt @@ -0,0 +1,155 @@ +WholeStageCodegen (24) + HashAggregate [sum,isEmpty] [sum(sales),sum(sales),sum,isEmpty] + InputAdapter + Exchange #1 + WholeStageCodegen (23) + HashAggregate [sales] [sum,isEmpty,sum,isEmpty] + InputAdapter + Union + WholeStageCodegen (11) + Project [cs_quantity,cs_list_price] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + Project [cs_quantity,cs_list_price,cs_sold_date_sk] + SortMergeJoin [cs_bill_customer_sk,c_customer_sk] + InputAdapter + WholeStageCodegen (6) + Sort [cs_bill_customer_sk] + InputAdapter + Exchange [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 + CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_year,d_moy,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + InputAdapter + BroadcastExchange #4 + WholeStageCodegen (4) + Project [item_sk] + Filter [cnt] + HashAggregate [_groupingexpression,i_item_sk,d_date,count] [count(1),item_sk,cnt,count] + InputAdapter + Exchange [_groupingexpression,i_item_sk,d_date] #5 + WholeStageCodegen (3) + HashAggregate [_groupingexpression,i_item_sk,d_date] [count,count] + Project [d_date,i_item_sk,i_item_desc] + BroadcastHashJoin [ss_item_sk,i_item_sk] + Project [ss_item_sk,d_date] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + ColumnarToRow + InputAdapter + CometFilter [ss_item_sk] + CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #2 + BroadcastExchange #6 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk,d_date] + CometFilter [d_year,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_year] + InputAdapter + ReusedExchange [d_date_sk,d_date] #6 + InputAdapter + BroadcastExchange #7 + WholeStageCodegen (2) + ColumnarToRow + InputAdapter + CometFilter [i_item_sk] + CometScan parquet spark_catalog.default.item [i_item_sk,i_item_desc] + InputAdapter + WholeStageCodegen (9) + Sort [c_customer_sk] + Project [c_customer_sk] + Filter [ssales] + Subquery #3 + WholeStageCodegen (5) + HashAggregate [max] [max(csales),tpcds_cmax,max] + InputAdapter + Exchange #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] + InputAdapter + Exchange [c_customer_sk] #11 + WholeStageCodegen (3) + HashAggregate [c_customer_sk,ss_quantity,ss_sales_price] [sum,isEmpty,sum,isEmpty] + Project [ss_quantity,ss_sales_price,c_customer_sk] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [ss_quantity,ss_sales_price,ss_sold_date_sk,c_customer_sk] + BroadcastHashJoin [ss_customer_sk,c_customer_sk] + ColumnarToRow + InputAdapter + CometFilter [ss_customer_sk] + CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_quantity,ss_sales_price,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #4 + BroadcastExchange #12 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_year,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + InputAdapter + ReusedExchange [c_customer_sk] #9 + InputAdapter + ReusedExchange [d_date_sk] #12 + HashAggregate [c_customer_sk,sum,isEmpty] [sum((cast(ss_quantity as decimal(10,0)) * ss_sales_price)),ssales,sum,isEmpty] + InputAdapter + Exchange [c_customer_sk] #8 + WholeStageCodegen (8) + HashAggregate [c_customer_sk,ss_quantity,ss_sales_price] [sum,isEmpty,sum,isEmpty] + Project [ss_quantity,ss_sales_price,c_customer_sk] + BroadcastHashJoin [ss_customer_sk,c_customer_sk] + ColumnarToRow + InputAdapter + CometProject [ss_customer_sk,ss_quantity,ss_sales_price] + CometFilter [ss_customer_sk] + CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_quantity,ss_sales_price,ss_sold_date_sk] + InputAdapter + BroadcastExchange #9 + WholeStageCodegen (7) + ColumnarToRow + InputAdapter + CometFilter [c_customer_sk] + CometScan parquet spark_catalog.default.customer [c_customer_sk] + InputAdapter + ReusedExchange [d_date_sk] #3 + WholeStageCodegen (22) + Project [ws_quantity,ws_list_price] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + Project [ws_quantity,ws_list_price,ws_sold_date_sk] + SortMergeJoin [ws_bill_customer_sk,c_customer_sk] + InputAdapter + WholeStageCodegen (17) + Sort [ws_bill_customer_sk] + InputAdapter + Exchange [ws_bill_customer_sk] #13 + WholeStageCodegen (16) + Project [ws_bill_customer_sk,ws_quantity,ws_list_price,ws_sold_date_sk] + BroadcastHashJoin [ws_item_sk,item_sk] + ColumnarToRow + InputAdapter + CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_bill_customer_sk,ws_quantity,ws_list_price,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + InputAdapter + ReusedExchange [item_sk] #4 + InputAdapter + WholeStageCodegen (20) + Sort [c_customer_sk] + Project [c_customer_sk] + Filter [ssales] + ReusedSubquery [tpcds_cmax] #3 + HashAggregate [c_customer_sk,sum,isEmpty] [sum((cast(ss_quantity as decimal(10,0)) * ss_sales_price)),ssales,sum,isEmpty] + InputAdapter + ReusedExchange [c_customer_sk,sum,isEmpty] #8 + InputAdapter + ReusedExchange [d_date_sk] #3 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q23b/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q23b/explain.txt new file mode 100644 index 000000000..840f9734a --- /dev/null +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q23b/explain.txt @@ -0,0 +1,694 @@ +== Physical Plan == +TakeOrderedAndProject (87) ++- Union (86) + :- * HashAggregate (62) + : +- Exchange (61) + : +- * HashAggregate (60) + : +- * Project (59) + : +- * BroadcastHashJoin Inner BuildRight (58) + : :- * Project (56) + : : +- * BroadcastHashJoin Inner BuildRight (55) + : : :- * SortMergeJoin LeftSemi (42) + : : : :- * Sort (25) + : : : : +- Exchange (24) + : : : : +- * Project (23) + : : : : +- * BroadcastHashJoin LeftSemi BuildRight (22) + : : : : :- * ColumnarToRow (3) + : : : : : +- CometFilter (2) + : : : : : +- CometScan parquet spark_catalog.default.catalog_sales (1) + : : : : +- BroadcastExchange (21) + : : : : +- * Project (20) + : : : : +- * Filter (19) + : : : : +- * HashAggregate (18) + : : : : +- Exchange (17) + : : : : +- * HashAggregate (16) + : : : : +- * Project (15) + : : : : +- * BroadcastHashJoin Inner BuildRight (14) + : : : : :- * Project (9) + : : : : : +- * BroadcastHashJoin Inner BuildRight (8) + : : : : : :- * ColumnarToRow (6) + : : : : : : +- CometFilter (5) + : : : : : : +- CometScan parquet spark_catalog.default.store_sales (4) + : : : : : +- ReusedExchange (7) + : : : : +- BroadcastExchange (13) + : : : : +- * ColumnarToRow (12) + : : : : +- CometFilter (11) + : : : : +- CometScan parquet spark_catalog.default.item (10) + : : : +- * Sort (41) + : : : +- * Project (40) + : : : +- * Filter (39) + : : : +- * HashAggregate (38) + : : : +- Exchange (37) + : : : +- * HashAggregate (36) + : : : +- * Project (35) + : : : +- * BroadcastHashJoin Inner BuildRight (34) + : : : :- * ColumnarToRow (29) + : : : : +- CometProject (28) + : : : : +- CometFilter (27) + : : : : +- CometScan parquet spark_catalog.default.store_sales (26) + : : : +- BroadcastExchange (33) + : : : +- * ColumnarToRow (32) + : : : +- CometFilter (31) + : : : +- CometScan parquet spark_catalog.default.customer (30) + : : +- BroadcastExchange (54) + : : +- * SortMergeJoin LeftSemi (53) + : : :- * Sort (47) + : : : +- Exchange (46) + : : : +- * ColumnarToRow (45) + : : : +- CometFilter (44) + : : : +- CometScan parquet spark_catalog.default.customer (43) + : : +- * Sort (52) + : : +- * Project (51) + : : +- * Filter (50) + : : +- * HashAggregate (49) + : : +- ReusedExchange (48) + : +- ReusedExchange (57) + +- * HashAggregate (85) + +- Exchange (84) + +- * HashAggregate (83) + +- * Project (82) + +- * BroadcastHashJoin Inner BuildRight (81) + :- * Project (79) + : +- * BroadcastHashJoin Inner BuildRight (78) + : :- * SortMergeJoin LeftSemi (76) + : : :- * Sort (70) + : : : +- Exchange (69) + : : : +- * Project (68) + : : : +- * BroadcastHashJoin LeftSemi BuildRight (67) + : : : :- * ColumnarToRow (65) + : : : : +- CometFilter (64) + : : : : +- CometScan parquet spark_catalog.default.web_sales (63) + : : : +- ReusedExchange (66) + : : +- * Sort (75) + : : +- * Project (74) + : : +- * Filter (73) + : : +- * HashAggregate (72) + : : +- ReusedExchange (71) + : +- ReusedExchange (77) + +- ReusedExchange (80) + + +(1) Scan parquet spark_catalog.default.catalog_sales +Output [5]: [cs_bill_customer_sk#1, cs_item_sk#2, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] +Batched: true +Location: InMemoryFileIndex [] +PartitionFilters: [isnotnull(cs_sold_date_sk#5), dynamicpruningexpression(cs_sold_date_sk#5 IN dynamicpruning#6)] +PushedFilters: [IsNotNull(cs_bill_customer_sk)] +ReadSchema: struct<cs_bill_customer_sk:int,cs_item_sk:int,cs_quantity:int,cs_list_price:decimal(7,2)> + +(2) CometFilter +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) + +(3) ColumnarToRow [codegen id : 5] +Input [5]: [cs_bill_customer_sk#1, cs_item_sk#2, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] + +(4) Scan parquet spark_catalog.default.store_sales +Output [2]: [ss_item_sk#7, ss_sold_date_sk#8] +Batched: true +Location: InMemoryFileIndex [] +PartitionFilters: [isnotnull(ss_sold_date_sk#8), dynamicpruningexpression(ss_sold_date_sk#8 IN dynamicpruning#9)] +PushedFilters: [IsNotNull(ss_item_sk)] +ReadSchema: struct<ss_item_sk:int> + +(5) CometFilter +Input [2]: [ss_item_sk#7, ss_sold_date_sk#8] +Condition : isnotnull(ss_item_sk#7) + +(6) ColumnarToRow [codegen id : 3] +Input [2]: [ss_item_sk#7, ss_sold_date_sk#8] + +(7) ReusedExchange [Reuses operator id: 97] +Output [2]: [d_date_sk#10, d_date#11] + +(8) BroadcastHashJoin [codegen id : 3] +Left keys [1]: [ss_sold_date_sk#8] +Right keys [1]: [d_date_sk#10] +Join type: Inner +Join condition: None + +(9) Project [codegen id : 3] +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] + +(10) Scan parquet spark_catalog.default.item +Output [2]: [i_item_sk#12, i_item_desc#13] +Batched: true +Location [not included in comparison]/{warehouse_dir}/item] +PushedFilters: [IsNotNull(i_item_sk)] +ReadSchema: struct<i_item_sk:int,i_item_desc:string> + +(11) CometFilter +Input [2]: [i_item_sk#12, i_item_desc#13] +Condition : isnotnull(i_item_sk#12) + +(12) ColumnarToRow [codegen id : 2] +Input [2]: [i_item_sk#12, i_item_desc#13] + +(13) 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] +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] +Output [3]: [d_date#11, i_item_sk#12, substr(i_item_desc#13, 1, 30) AS _groupingexpression#14] +Input [4]: [ss_item_sk#7, d_date#11, i_item_sk#12, i_item_desc#13] + +(16) HashAggregate [codegen id : 3] +Input [3]: [d_date#11, i_item_sk#12, _groupingexpression#14] +Keys [3]: [_groupingexpression#14, i_item_sk#12, d_date#11] +Functions [1]: [partial_count(1)] +Aggregate Attributes [1]: [count#15] +Results [4]: [_groupingexpression#14, i_item_sk#12, d_date#11, count#16] + +(17) Exchange +Input [4]: [_groupingexpression#14, i_item_sk#12, d_date#11, count#16] +Arguments: hashpartitioning(_groupingexpression#14, i_item_sk#12, d_date#11, 5), ENSURE_REQUIREMENTS, [plan_id=2] + +(18) HashAggregate [codegen id : 4] +Input [4]: [_groupingexpression#14, i_item_sk#12, d_date#11, count#16] +Keys [3]: [_groupingexpression#14, i_item_sk#12, d_date#11] +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] +Input [2]: [item_sk#18, cnt#19] +Condition : (cnt#19 > 4) + +(20) Project [codegen id : 4] +Output [1]: [item_sk#18] +Input [2]: [item_sk#18, cnt#19] + +(21) BroadcastExchange +Input [1]: [item_sk#18] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] + +(22) BroadcastHashJoin [codegen id : 5] +Left keys [1]: [cs_item_sk#2] +Right keys [1]: [item_sk#18] +Join type: LeftSemi +Join condition: None + +(23) Project [codegen id : 5] +Output [4]: [cs_bill_customer_sk#1, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] +Input [5]: [cs_bill_customer_sk#1, cs_item_sk#2, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] + +(24) Exchange +Input [4]: [cs_bill_customer_sk#1, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] +Arguments: hashpartitioning(cs_bill_customer_sk#1, 5), ENSURE_REQUIREMENTS, [plan_id=4] + +(25) Sort [codegen id : 6] +Input [4]: [cs_bill_customer_sk#1, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5] +Arguments: [cs_bill_customer_sk#1 ASC NULLS FIRST], false, 0 + +(26) Scan parquet spark_catalog.default.store_sales +Output [4]: [ss_customer_sk#20, ss_quantity#21, ss_sales_price#22, ss_sold_date_sk#23] +Batched: true +Location [not included in comparison]/{warehouse_dir}/store_sales] +PushedFilters: [IsNotNull(ss_customer_sk)] +ReadSchema: struct<ss_customer_sk:int,ss_quantity:int,ss_sales_price:decimal(7,2)> + +(27) CometFilter +Input [4]: [ss_customer_sk#20, ss_quantity#21, ss_sales_price#22, ss_sold_date_sk#23] +Condition : isnotnull(ss_customer_sk#20) + +(28) CometProject +Input [4]: [ss_customer_sk#20, ss_quantity#21, ss_sales_price#22, ss_sold_date_sk#23] +Arguments: [ss_customer_sk#20, ss_quantity#21, ss_sales_price#22], [ss_customer_sk#20, ss_quantity#21, ss_sales_price#22] + +(29) ColumnarToRow [codegen id : 8] +Input [3]: [ss_customer_sk#20, ss_quantity#21, ss_sales_price#22] + +(30) Scan parquet spark_catalog.default.customer +Output [1]: [c_customer_sk#24] +Batched: true +Location [not included in comparison]/{warehouse_dir}/customer] +PushedFilters: [IsNotNull(c_customer_sk)] +ReadSchema: struct<c_customer_sk:int> + +(31) CometFilter +Input [1]: [c_customer_sk#24] +Condition : isnotnull(c_customer_sk#24) + +(32) ColumnarToRow [codegen id : 7] +Input [1]: [c_customer_sk#24] + +(33) BroadcastExchange +Input [1]: [c_customer_sk#24] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=5] + +(34) BroadcastHashJoin [codegen id : 8] +Left keys [1]: [ss_customer_sk#20] +Right keys [1]: [c_customer_sk#24] +Join type: Inner +Join condition: None + +(35) Project [codegen id : 8] +Output [3]: [ss_quantity#21, ss_sales_price#22, c_customer_sk#24] +Input [4]: [ss_customer_sk#20, ss_quantity#21, ss_sales_price#22, c_customer_sk#24] + +(36) HashAggregate [codegen id : 8] +Input [3]: [ss_quantity#21, ss_sales_price#22, c_customer_sk#24] +Keys [1]: [c_customer_sk#24] +Functions [1]: [partial_sum((cast(ss_quantity#21 as decimal(10,0)) * ss_sales_price#22))] +Aggregate Attributes [2]: [sum#25, isEmpty#26] +Results [3]: [c_customer_sk#24, sum#27, isEmpty#28] + +(37) Exchange +Input [3]: [c_customer_sk#24, sum#27, isEmpty#28] +Arguments: hashpartitioning(c_customer_sk#24, 5), ENSURE_REQUIREMENTS, [plan_id=6] + +(38) HashAggregate [codegen id : 9] +Input [3]: [c_customer_sk#24, sum#27, isEmpty#28] +Keys [1]: [c_customer_sk#24] +Functions [1]: [sum((cast(ss_quantity#21 as decimal(10,0)) * ss_sales_price#22))] +Aggregate Attributes [1]: [sum((cast(ss_quantity#21 as decimal(10,0)) * ss_sales_price#22))#29] +Results [2]: [c_customer_sk#24, sum((cast(ss_quantity#21 as decimal(10,0)) * ss_sales_price#22))#29 AS ssales#30] + +(39) Filter [codegen id : 9] +Input [2]: [c_customer_sk#24, ssales#30] +Condition : (isnotnull(ssales#30) AND (cast(ssales#30 as decimal(38,8)) > (0.500000 * Subquery scalar-subquery#31, [id=#32]))) + +(40) Project [codegen id : 9] +Output [1]: [c_customer_sk#24] +Input [2]: [c_customer_sk#24, ssales#30] + +(41) Sort [codegen id : 9] +Input [1]: [c_customer_sk#24] +Arguments: [c_customer_sk#24 ASC NULLS FIRST], false, 0 + +(42) SortMergeJoin [codegen id : 17] +Left keys [1]: [cs_bill_customer_sk#1] +Right keys [1]: [c_customer_sk#24] +Join type: LeftSemi +Join condition: None + +(43) Scan parquet spark_catalog.default.customer +Output [3]: [c_customer_sk#33, c_first_name#34, c_last_name#35] +Batched: true +Location [not included in comparison]/{warehouse_dir}/customer] +PushedFilters: [IsNotNull(c_customer_sk)] +ReadSchema: struct<c_customer_sk:int,c_first_name:string,c_last_name:string> + +(44) CometFilter +Input [3]: [c_customer_sk#33, c_first_name#34, c_last_name#35] +Condition : isnotnull(c_customer_sk#33) + +(45) ColumnarToRow [codegen id : 10] +Input [3]: [c_customer_sk#33, c_first_name#34, c_last_name#35] + +(46) Exchange +Input [3]: [c_customer_sk#33, c_first_name#34, c_last_name#35] +Arguments: hashpartitioning(c_customer_sk#33, 5), ENSURE_REQUIREMENTS, [plan_id=7] + +(47) Sort [codegen id : 11] +Input [3]: [c_customer_sk#33, c_first_name#34, c_last_name#35] +Arguments: [c_customer_sk#33 ASC NULLS FIRST], false, 0 + +(48) ReusedExchange [Reuses operator id: 37] +Output [3]: [c_customer_sk#24, sum#27, isEmpty#28] + +(49) HashAggregate [codegen id : 14] +Input [3]: [c_customer_sk#24, sum#27, isEmpty#28] +Keys [1]: [c_customer_sk#24] +Functions [1]: [sum((cast(ss_quantity#21 as decimal(10,0)) * ss_sales_price#22))] +Aggregate Attributes [1]: [sum((cast(ss_quantity#21 as decimal(10,0)) * ss_sales_price#22))#29] +Results [2]: [c_customer_sk#24, sum((cast(ss_quantity#21 as decimal(10,0)) * ss_sales_price#22))#29 AS ssales#30] + +(50) Filter [codegen id : 14] +Input [2]: [c_customer_sk#24, ssales#30] +Condition : (isnotnull(ssales#30) AND (cast(ssales#30 as decimal(38,8)) > (0.500000 * ReusedSubquery Subquery scalar-subquery#31, [id=#32]))) + +(51) Project [codegen id : 14] +Output [1]: [c_customer_sk#24] +Input [2]: [c_customer_sk#24, ssales#30] + +(52) Sort [codegen id : 14] +Input [1]: [c_customer_sk#24] +Arguments: [c_customer_sk#24 ASC NULLS FIRST], false, 0 + +(53) SortMergeJoin [codegen id : 15] +Left keys [1]: [c_customer_sk#33] +Right keys [1]: [c_customer_sk#24] +Join type: LeftSemi +Join condition: None + +(54) BroadcastExchange +Input [3]: [c_customer_sk#33, c_first_name#34, c_last_name#35] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=8] + +(55) BroadcastHashJoin [codegen id : 17] +Left keys [1]: [cs_bill_customer_sk#1] +Right keys [1]: [c_customer_sk#33] +Join type: Inner +Join condition: None + +(56) Project [codegen id : 17] +Output [5]: [cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5, c_first_name#34, c_last_name#35] +Input [7]: [cs_bill_customer_sk#1, cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5, c_customer_sk#33, c_first_name#34, c_last_name#35] + +(57) ReusedExchange [Reuses operator id: 92] +Output [1]: [d_date_sk#36] + +(58) BroadcastHashJoin [codegen id : 17] +Left keys [1]: [cs_sold_date_sk#5] +Right keys [1]: [d_date_sk#36] +Join type: Inner +Join condition: None + +(59) Project [codegen id : 17] +Output [4]: [cs_quantity#3, cs_list_price#4, c_first_name#34, c_last_name#35] +Input [6]: [cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5, c_first_name#34, c_last_name#35, d_date_sk#36] + +(60) HashAggregate [codegen id : 17] +Input [4]: [cs_quantity#3, cs_list_price#4, c_first_name#34, c_last_name#35] +Keys [2]: [c_last_name#35, c_first_name#34] +Functions [1]: [partial_sum((cast(cs_quantity#3 as decimal(10,0)) * cs_list_price#4))] +Aggregate Attributes [2]: [sum#37, isEmpty#38] +Results [4]: [c_last_name#35, c_first_name#34, sum#39, isEmpty#40] + +(61) Exchange +Input [4]: [c_last_name#35, c_first_name#34, sum#39, isEmpty#40] +Arguments: hashpartitioning(c_last_name#35, c_first_name#34, 5), ENSURE_REQUIREMENTS, [plan_id=9] + +(62) HashAggregate [codegen id : 18] +Input [4]: [c_last_name#35, c_first_name#34, sum#39, isEmpty#40] +Keys [2]: [c_last_name#35, c_first_name#34] +Functions [1]: [sum((cast(cs_quantity#3 as decimal(10,0)) * cs_list_price#4))] +Aggregate Attributes [1]: [sum((cast(cs_quantity#3 as decimal(10,0)) * cs_list_price#4))#41] +Results [3]: [c_last_name#35, c_first_name#34, sum((cast(cs_quantity#3 as decimal(10,0)) * cs_list_price#4))#41 AS sales#42] + +(63) Scan parquet spark_catalog.default.web_sales +Output [5]: [ws_item_sk#43, ws_bill_customer_sk#44, ws_quantity#45, ws_list_price#46, ws_sold_date_sk#47] +Batched: true +Location: InMemoryFileIndex [] +PartitionFilters: [isnotnull(ws_sold_date_sk#47), dynamicpruningexpression(ws_sold_date_sk#47 IN dynamicpruning#48)] +PushedFilters: [IsNotNull(ws_bill_customer_sk)] +ReadSchema: struct<ws_item_sk:int,ws_bill_customer_sk:int,ws_quantity:int,ws_list_price:decimal(7,2)> + +(64) CometFilter +Input [5]: [ws_item_sk#43, ws_bill_customer_sk#44, ws_quantity#45, ws_list_price#46, ws_sold_date_sk#47] +Condition : isnotnull(ws_bill_customer_sk#44) + +(65) ColumnarToRow [codegen id : 23] +Input [5]: [ws_item_sk#43, ws_bill_customer_sk#44, ws_quantity#45, ws_list_price#46, ws_sold_date_sk#47] + +(66) ReusedExchange [Reuses operator id: 21] +Output [1]: [item_sk#49] + +(67) BroadcastHashJoin [codegen id : 23] +Left keys [1]: [ws_item_sk#43] +Right keys [1]: [item_sk#49] +Join type: LeftSemi +Join condition: None + +(68) Project [codegen id : 23] +Output [4]: [ws_bill_customer_sk#44, ws_quantity#45, ws_list_price#46, ws_sold_date_sk#47] +Input [5]: [ws_item_sk#43, ws_bill_customer_sk#44, ws_quantity#45, ws_list_price#46, ws_sold_date_sk#47] + +(69) Exchange +Input [4]: [ws_bill_customer_sk#44, ws_quantity#45, ws_list_price#46, ws_sold_date_sk#47] +Arguments: hashpartitioning(ws_bill_customer_sk#44, 5), ENSURE_REQUIREMENTS, [plan_id=10] + +(70) Sort [codegen id : 24] +Input [4]: [ws_bill_customer_sk#44, ws_quantity#45, ws_list_price#46, ws_sold_date_sk#47] +Arguments: [ws_bill_customer_sk#44 ASC NULLS FIRST], false, 0 + +(71) ReusedExchange [Reuses operator id: 37] +Output [3]: [c_customer_sk#50, sum#51, isEmpty#52] + +(72) HashAggregate [codegen id : 27] +Input [3]: [c_customer_sk#50, sum#51, isEmpty#52] +Keys [1]: [c_customer_sk#50] +Functions [1]: [sum((cast(ss_quantity#53 as decimal(10,0)) * ss_sales_price#54))] +Aggregate Attributes [1]: [sum((cast(ss_quantity#53 as decimal(10,0)) * ss_sales_price#54))#29] +Results [2]: [c_customer_sk#50, sum((cast(ss_quantity#53 as decimal(10,0)) * ss_sales_price#54))#29 AS ssales#55] + +(73) Filter [codegen id : 27] +Input [2]: [c_customer_sk#50, ssales#55] +Condition : (isnotnull(ssales#55) AND (cast(ssales#55 as decimal(38,8)) > (0.500000 * ReusedSubquery Subquery scalar-subquery#31, [id=#32]))) + +(74) Project [codegen id : 27] +Output [1]: [c_customer_sk#50] +Input [2]: [c_customer_sk#50, ssales#55] + +(75) Sort [codegen id : 27] +Input [1]: [c_customer_sk#50] +Arguments: [c_customer_sk#50 ASC NULLS FIRST], false, 0 + +(76) SortMergeJoin [codegen id : 35] +Left keys [1]: [ws_bill_customer_sk#44] +Right keys [1]: [c_customer_sk#50] +Join type: LeftSemi +Join condition: None + +(77) ReusedExchange [Reuses operator id: 54] +Output [3]: [c_customer_sk#56, c_first_name#57, c_last_name#58] + +(78) BroadcastHashJoin [codegen id : 35] +Left keys [1]: [ws_bill_customer_sk#44] +Right keys [1]: [c_customer_sk#56] +Join type: Inner +Join condition: None + +(79) Project [codegen id : 35] +Output [5]: [ws_quantity#45, ws_list_price#46, ws_sold_date_sk#47, c_first_name#57, c_last_name#58] +Input [7]: [ws_bill_customer_sk#44, ws_quantity#45, ws_list_price#46, ws_sold_date_sk#47, c_customer_sk#56, c_first_name#57, c_last_name#58] + +(80) ReusedExchange [Reuses operator id: 92] +Output [1]: [d_date_sk#59] + +(81) BroadcastHashJoin [codegen id : 35] +Left keys [1]: [ws_sold_date_sk#47] +Right keys [1]: [d_date_sk#59] +Join type: Inner +Join condition: None + +(82) Project [codegen id : 35] +Output [4]: [ws_quantity#45, ws_list_price#46, c_first_name#57, c_last_name#58] +Input [6]: [ws_quantity#45, ws_list_price#46, ws_sold_date_sk#47, c_first_name#57, c_last_name#58, d_date_sk#59] + +(83) HashAggregate [codegen id : 35] +Input [4]: [ws_quantity#45, ws_list_price#46, c_first_name#57, c_last_name#58] +Keys [2]: [c_last_name#58, c_first_name#57] +Functions [1]: [partial_sum((cast(ws_quantity#45 as decimal(10,0)) * ws_list_price#46))] +Aggregate Attributes [2]: [sum#60, isEmpty#61] +Results [4]: [c_last_name#58, c_first_name#57, sum#62, isEmpty#63] + +(84) Exchange +Input [4]: [c_last_name#58, c_first_name#57, sum#62, isEmpty#63] +Arguments: hashpartitioning(c_last_name#58, c_first_name#57, 5), ENSURE_REQUIREMENTS, [plan_id=11] + +(85) HashAggregate [codegen id : 36] +Input [4]: [c_last_name#58, c_first_name#57, sum#62, isEmpty#63] +Keys [2]: [c_last_name#58, c_first_name#57] +Functions [1]: [sum((cast(ws_quantity#45 as decimal(10,0)) * ws_list_price#46))] +Aggregate Attributes [1]: [sum((cast(ws_quantity#45 as decimal(10,0)) * ws_list_price#46))#64] +Results [3]: [c_last_name#58, c_first_name#57, sum((cast(ws_quantity#45 as decimal(10,0)) * ws_list_price#46))#64 AS sales#65] + +(86) Union + +(87) TakeOrderedAndProject +Input [3]: [c_last_name#35, c_first_name#34, sales#42] +Arguments: 100, [c_last_name#35 ASC NULLS FIRST, c_first_name#34 ASC NULLS FIRST, sales#42 ASC NULLS FIRST], [c_last_name#35, c_first_name#34, sales#42] + +===== Subqueries ===== + +Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#5 IN dynamicpruning#6 +BroadcastExchange (92) ++- * ColumnarToRow (91) + +- CometProject (90) + +- CometFilter (89) + +- CometScan parquet spark_catalog.default.date_dim (88) + + +(88) Scan parquet spark_catalog.default.date_dim +Output [3]: [d_date_sk#36, d_year#66, d_moy#67] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2000), EqualTo(d_moy,2), IsNotNull(d_date_sk)] +ReadSchema: struct<d_date_sk:int,d_year:int,d_moy:int> + +(89) CometFilter +Input [3]: [d_date_sk#36, d_year#66, d_moy#67] +Condition : ((((isnotnull(d_year#66) AND isnotnull(d_moy#67)) AND (d_year#66 = 2000)) AND (d_moy#67 = 2)) AND isnotnull(d_date_sk#36)) + +(90) CometProject +Input [3]: [d_date_sk#36, d_year#66, d_moy#67] +Arguments: [d_date_sk#36], [d_date_sk#36] + +(91) ColumnarToRow [codegen id : 1] +Input [1]: [d_date_sk#36] + +(92) BroadcastExchange +Input [1]: [d_date_sk#36] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=12] + +Subquery:2 Hosting operator id = 4 Hosting Expression = ss_sold_date_sk#8 IN dynamicpruning#9 +BroadcastExchange (97) ++- * ColumnarToRow (96) + +- CometProject (95) + +- CometFilter (94) + +- CometScan parquet spark_catalog.default.date_dim (93) + + +(93) Scan parquet spark_catalog.default.date_dim +Output [3]: [d_date_sk#10, d_date#11, d_year#68] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [In(d_year, [2000,2001,2002,2003]), IsNotNull(d_date_sk)] +ReadSchema: struct<d_date_sk:int,d_date:date,d_year:int> + +(94) CometFilter +Input [3]: [d_date_sk#10, d_date#11, d_year#68] +Condition : (d_year#68 IN (2000,2001,2002,2003) AND isnotnull(d_date_sk#10)) + +(95) CometProject +Input [3]: [d_date_sk#10, d_date#11, d_year#68] +Arguments: [d_date_sk#10, d_date#11], [d_date_sk#10, d_date#11] + +(96) ColumnarToRow [codegen id : 1] +Input [2]: [d_date_sk#10, d_date#11] + +(97) BroadcastExchange +Input [2]: [d_date_sk#10, d_date#11] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=13] + +Subquery:3 Hosting operator id = 39 Hosting Expression = Subquery scalar-subquery#31, [id=#32] +* HashAggregate (112) ++- Exchange (111) + +- * HashAggregate (110) + +- * HashAggregate (109) + +- Exchange (108) + +- * HashAggregate (107) + +- * Project (106) + +- * BroadcastHashJoin Inner BuildRight (105) + :- * Project (103) + : +- * BroadcastHashJoin Inner BuildRight (102) + : :- * ColumnarToRow (100) + : : +- CometFilter (99) + : : +- CometScan parquet spark_catalog.default.store_sales (98) + : +- ReusedExchange (101) + +- ReusedExchange (104) + + +(98) Scan parquet spark_catalog.default.store_sales +Output [4]: [ss_customer_sk#69, ss_quantity#70, ss_sales_price#71, ss_sold_date_sk#72] +Batched: true +Location: InMemoryFileIndex [] +PartitionFilters: [isnotnull(ss_sold_date_sk#72), dynamicpruningexpression(ss_sold_date_sk#72 IN dynamicpruning#73)] +PushedFilters: [IsNotNull(ss_customer_sk)] +ReadSchema: struct<ss_customer_sk:int,ss_quantity:int,ss_sales_price:decimal(7,2)> + +(99) CometFilter +Input [4]: [ss_customer_sk#69, ss_quantity#70, ss_sales_price#71, ss_sold_date_sk#72] +Condition : isnotnull(ss_customer_sk#69) + +(100) ColumnarToRow [codegen id : 3] +Input [4]: [ss_customer_sk#69, ss_quantity#70, ss_sales_price#71, ss_sold_date_sk#72] + +(101) ReusedExchange [Reuses operator id: 33] +Output [1]: [c_customer_sk#74] + +(102) BroadcastHashJoin [codegen id : 3] +Left keys [1]: [ss_customer_sk#69] +Right keys [1]: [c_customer_sk#74] +Join type: Inner +Join condition: None + +(103) Project [codegen id : 3] +Output [4]: [ss_quantity#70, ss_sales_price#71, ss_sold_date_sk#72, c_customer_sk#74] +Input [5]: [ss_customer_sk#69, ss_quantity#70, ss_sales_price#71, ss_sold_date_sk#72, c_customer_sk#74] + +(104) ReusedExchange [Reuses operator id: 117] +Output [1]: [d_date_sk#75] + +(105) BroadcastHashJoin [codegen id : 3] +Left keys [1]: [ss_sold_date_sk#72] +Right keys [1]: [d_date_sk#75] +Join type: Inner +Join condition: None + +(106) Project [codegen id : 3] +Output [3]: [ss_quantity#70, ss_sales_price#71, c_customer_sk#74] +Input [5]: [ss_quantity#70, ss_sales_price#71, ss_sold_date_sk#72, c_customer_sk#74, d_date_sk#75] + +(107) HashAggregate [codegen id : 3] +Input [3]: [ss_quantity#70, ss_sales_price#71, c_customer_sk#74] +Keys [1]: [c_customer_sk#74] +Functions [1]: [partial_sum((cast(ss_quantity#70 as decimal(10,0)) * ss_sales_price#71))] +Aggregate Attributes [2]: [sum#76, isEmpty#77] +Results [3]: [c_customer_sk#74, sum#78, isEmpty#79] + +(108) Exchange +Input [3]: [c_customer_sk#74, sum#78, isEmpty#79] +Arguments: hashpartitioning(c_customer_sk#74, 5), ENSURE_REQUIREMENTS, [plan_id=14] + +(109) HashAggregate [codegen id : 4] +Input [3]: [c_customer_sk#74, sum#78, isEmpty#79] +Keys [1]: [c_customer_sk#74] +Functions [1]: [sum((cast(ss_quantity#70 as decimal(10,0)) * ss_sales_price#71))] +Aggregate Attributes [1]: [sum((cast(ss_quantity#70 as decimal(10,0)) * ss_sales_price#71))#80] +Results [1]: [sum((cast(ss_quantity#70 as decimal(10,0)) * ss_sales_price#71))#80 AS csales#81] + +(110) HashAggregate [codegen id : 4] +Input [1]: [csales#81] +Keys: [] +Functions [1]: [partial_max(csales#81)] +Aggregate Attributes [1]: [max#82] +Results [1]: [max#83] + +(111) Exchange +Input [1]: [max#83] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=15] + +(112) HashAggregate [codegen id : 5] +Input [1]: [max#83] +Keys: [] +Functions [1]: [max(csales#81)] +Aggregate Attributes [1]: [max(csales#81)#84] +Results [1]: [max(csales#81)#84 AS tpcds_cmax#85] + +Subquery:4 Hosting operator id = 98 Hosting Expression = ss_sold_date_sk#72 IN dynamicpruning#73 +BroadcastExchange (117) ++- * ColumnarToRow (116) + +- CometProject (115) + +- CometFilter (114) + +- CometScan parquet spark_catalog.default.date_dim (113) + + +(113) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#75, d_year#86] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [In(d_year, [2000,2001,2002,2003]), IsNotNull(d_date_sk)] +ReadSchema: struct<d_date_sk:int,d_year:int> + +(114) CometFilter +Input [2]: [d_date_sk#75, d_year#86] +Condition : (d_year#86 IN (2000,2001,2002,2003) AND isnotnull(d_date_sk#75)) + +(115) CometProject +Input [2]: [d_date_sk#75, d_year#86] +Arguments: [d_date_sk#75], [d_date_sk#75] + +(116) ColumnarToRow [codegen id : 1] +Input [1]: [d_date_sk#75] + +(117) BroadcastExchange +Input [1]: [d_date_sk#75] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=16] + +Subquery:5 Hosting operator id = 50 Hosting Expression = ReusedSubquery Subquery scalar-subquery#31, [id=#32] + +Subquery:6 Hosting operator id = 63 Hosting Expression = ws_sold_date_sk#47 IN dynamicpruning#6 + +Subquery:7 Hosting operator id = 73 Hosting Expression = ReusedSubquery Subquery scalar-subquery#31, [id=#32] + + diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q23b/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q23b/simplified.txt new file mode 100644 index 000000000..49ddeaef8 --- /dev/null +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q23b/simplified.txt @@ -0,0 +1,188 @@ +TakeOrderedAndProject [c_last_name,c_first_name,sales] + Union + WholeStageCodegen (18) + HashAggregate [c_last_name,c_first_name,sum,isEmpty] [sum((cast(cs_quantity as decimal(10,0)) * cs_list_price)),sales,sum,isEmpty] + InputAdapter + Exchange [c_last_name,c_first_name] #1 + WholeStageCodegen (17) + HashAggregate [c_last_name,c_first_name,cs_quantity,cs_list_price] [sum,isEmpty,sum,isEmpty] + Project [cs_quantity,cs_list_price,c_first_name,c_last_name] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + Project [cs_quantity,cs_list_price,cs_sold_date_sk,c_first_name,c_last_name] + BroadcastHashJoin [cs_bill_customer_sk,c_customer_sk] + SortMergeJoin [cs_bill_customer_sk,c_customer_sk] + InputAdapter + WholeStageCodegen (6) + Sort [cs_bill_customer_sk] + InputAdapter + Exchange [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 + CometFilter [cs_bill_customer_sk] + CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_year,d_moy,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + InputAdapter + BroadcastExchange #4 + WholeStageCodegen (4) + Project [item_sk] + Filter [cnt] + HashAggregate [_groupingexpression,i_item_sk,d_date,count] [count(1),item_sk,cnt,count] + InputAdapter + Exchange [_groupingexpression,i_item_sk,d_date] #5 + WholeStageCodegen (3) + HashAggregate [_groupingexpression,i_item_sk,d_date] [count,count] + Project [d_date,i_item_sk,i_item_desc] + BroadcastHashJoin [ss_item_sk,i_item_sk] + Project [ss_item_sk,d_date] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + ColumnarToRow + InputAdapter + CometFilter [ss_item_sk] + CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #2 + BroadcastExchange #6 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk,d_date] + CometFilter [d_year,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_year] + InputAdapter + ReusedExchange [d_date_sk,d_date] #6 + InputAdapter + BroadcastExchange #7 + WholeStageCodegen (2) + ColumnarToRow + InputAdapter + CometFilter [i_item_sk] + CometScan parquet spark_catalog.default.item [i_item_sk,i_item_desc] + InputAdapter + WholeStageCodegen (9) + Sort [c_customer_sk] + Project [c_customer_sk] + Filter [ssales] + Subquery #3 + WholeStageCodegen (5) + HashAggregate [max] [max(csales),tpcds_cmax,max] + InputAdapter + Exchange #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] + InputAdapter + Exchange [c_customer_sk] #11 + WholeStageCodegen (3) + HashAggregate [c_customer_sk,ss_quantity,ss_sales_price] [sum,isEmpty,sum,isEmpty] + Project [ss_quantity,ss_sales_price,c_customer_sk] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [ss_quantity,ss_sales_price,ss_sold_date_sk,c_customer_sk] + BroadcastHashJoin [ss_customer_sk,c_customer_sk] + ColumnarToRow + InputAdapter + CometFilter [ss_customer_sk] + CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_quantity,ss_sales_price,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #4 + BroadcastExchange #12 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_year,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + InputAdapter + ReusedExchange [c_customer_sk] #9 + InputAdapter + ReusedExchange [d_date_sk] #12 + HashAggregate [c_customer_sk,sum,isEmpty] [sum((cast(ss_quantity as decimal(10,0)) * ss_sales_price)),ssales,sum,isEmpty] + InputAdapter + Exchange [c_customer_sk] #8 + WholeStageCodegen (8) + HashAggregate [c_customer_sk,ss_quantity,ss_sales_price] [sum,isEmpty,sum,isEmpty] + Project [ss_quantity,ss_sales_price,c_customer_sk] + BroadcastHashJoin [ss_customer_sk,c_customer_sk] + ColumnarToRow + InputAdapter + CometProject [ss_customer_sk,ss_quantity,ss_sales_price] + CometFilter [ss_customer_sk] + CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_quantity,ss_sales_price,ss_sold_date_sk] + InputAdapter + BroadcastExchange #9 + WholeStageCodegen (7) + ColumnarToRow + InputAdapter + CometFilter [c_customer_sk] + CometScan parquet spark_catalog.default.customer [c_customer_sk] + InputAdapter + BroadcastExchange #13 + WholeStageCodegen (15) + SortMergeJoin [c_customer_sk,c_customer_sk] + InputAdapter + WholeStageCodegen (11) + Sort [c_customer_sk] + InputAdapter + Exchange [c_customer_sk] #14 + WholeStageCodegen (10) + ColumnarToRow + InputAdapter + CometFilter [c_customer_sk] + CometScan parquet spark_catalog.default.customer [c_customer_sk,c_first_name,c_last_name] + InputAdapter + WholeStageCodegen (14) + Sort [c_customer_sk] + Project [c_customer_sk] + Filter [ssales] + ReusedSubquery [tpcds_cmax] #3 + HashAggregate [c_customer_sk,sum,isEmpty] [sum((cast(ss_quantity as decimal(10,0)) * ss_sales_price)),ssales,sum,isEmpty] + InputAdapter + ReusedExchange [c_customer_sk,sum,isEmpty] #8 + InputAdapter + ReusedExchange [d_date_sk] #3 + WholeStageCodegen (36) + HashAggregate [c_last_name,c_first_name,sum,isEmpty] [sum((cast(ws_quantity as decimal(10,0)) * ws_list_price)),sales,sum,isEmpty] + InputAdapter + Exchange [c_last_name,c_first_name] #15 + WholeStageCodegen (35) + HashAggregate [c_last_name,c_first_name,ws_quantity,ws_list_price] [sum,isEmpty,sum,isEmpty] + Project [ws_quantity,ws_list_price,c_first_name,c_last_name] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + Project [ws_quantity,ws_list_price,ws_sold_date_sk,c_first_name,c_last_name] + BroadcastHashJoin [ws_bill_customer_sk,c_customer_sk] + SortMergeJoin [ws_bill_customer_sk,c_customer_sk] + InputAdapter + WholeStageCodegen (24) + Sort [ws_bill_customer_sk] + InputAdapter + Exchange [ws_bill_customer_sk] #16 + WholeStageCodegen (23) + Project [ws_bill_customer_sk,ws_quantity,ws_list_price,ws_sold_date_sk] + BroadcastHashJoin [ws_item_sk,item_sk] + ColumnarToRow + InputAdapter + CometFilter [ws_bill_customer_sk] + CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_bill_customer_sk,ws_quantity,ws_list_price,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + InputAdapter + ReusedExchange [item_sk] #4 + InputAdapter + WholeStageCodegen (27) + Sort [c_customer_sk] + Project [c_customer_sk] + Filter [ssales] + ReusedSubquery [tpcds_cmax] #3 + HashAggregate [c_customer_sk,sum,isEmpty] [sum((cast(ss_quantity as decimal(10,0)) * ss_sales_price)),ssales,sum,isEmpty] + InputAdapter + ReusedExchange [c_customer_sk,sum,isEmpty] #8 + InputAdapter + ReusedExchange [c_customer_sk,c_first_name,c_last_name] #13 + InputAdapter + ReusedExchange [d_date_sk] #3 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q24a/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q24a/explain.txt new file mode 100644 index 000000000..7241b5ea0 --- /dev/null +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q24a/explain.txt @@ -0,0 +1,427 @@ +== Physical Plan == +* Filter (46) ++- * HashAggregate (45) + +- Exchange (44) + +- * HashAggregate (43) + +- * HashAggregate (42) + +- Exchange (41) + +- * HashAggregate (40) + +- * Project (39) + +- * BroadcastHashJoin Inner BuildRight (38) + :- * Project (33) + : +- * BroadcastHashJoin Inner BuildRight (32) + : :- * Project (27) + : : +- * BroadcastHashJoin Inner BuildRight (26) + : : :- * Project (21) + : : : +- * BroadcastHashJoin Inner BuildRight (20) + : : : :- * Project (14) + : : : : +- * SortMergeJoin Inner (13) + : : : : :- * Sort (6) + : : : : : +- Exchange (5) + : : : : : +- * ColumnarToRow (4) + : : : : : +- CometProject (3) + : : : : : +- CometFilter (2) + : : : : : +- CometScan parquet spark_catalog.default.store_sales (1) + : : : : +- * Sort (12) + : : : : +- Exchange (11) + : : : : +- * ColumnarToRow (10) + : : : : +- CometProject (9) + : : : : +- CometFilter (8) + : : : : +- CometScan parquet spark_catalog.default.store_returns (7) + : : : +- BroadcastExchange (19) + : : : +- * ColumnarToRow (18) + : : : +- CometProject (17) + : : : +- CometFilter (16) + : : : +- CometScan parquet spark_catalog.default.store (15) + : : +- BroadcastExchange (25) + : : +- * ColumnarToRow (24) + : : +- CometFilter (23) + : : +- CometScan parquet spark_catalog.default.item (22) + : +- BroadcastExchange (31) + : +- * ColumnarToRow (30) + : +- CometFilter (29) + : +- CometScan parquet spark_catalog.default.customer (28) + +- BroadcastExchange (37) + +- * ColumnarToRow (36) + +- CometFilter (35) + +- CometScan parquet spark_catalog.default.customer_address (34) + + +(1) Scan parquet spark_catalog.default.store_sales +Output [6]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5, ss_sold_date_sk#6] +Batched: true +Location [not included in comparison]/{warehouse_dir}/store_sales] +PushedFilters: [IsNotNull(ss_ticket_number), IsNotNull(ss_item_sk), IsNotNull(ss_store_sk), IsNotNull(ss_customer_sk)] +ReadSchema: struct<ss_item_sk:int,ss_customer_sk:int,ss_store_sk:int,ss_ticket_number:int,ss_net_paid:decimal(7,2)> + +(2) CometFilter +Input [6]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5, ss_sold_date_sk#6] +Condition : (((isnotnull(ss_ticket_number#4) AND isnotnull(ss_item_sk#1)) AND isnotnull(ss_store_sk#3)) AND isnotnull(ss_customer_sk#2)) + +(3) CometProject +Input [6]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5, ss_sold_date_sk#6] +Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5], [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5] + +(4) ColumnarToRow [codegen id : 1] +Input [5]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5] + +(5) Exchange +Input [5]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5] +Arguments: hashpartitioning(ss_ticket_number#4, ss_item_sk#1, 5), ENSURE_REQUIREMENTS, [plan_id=1] + +(6) Sort [codegen id : 2] +Input [5]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5] +Arguments: [ss_ticket_number#4 ASC NULLS FIRST, ss_item_sk#1 ASC NULLS FIRST], false, 0 + +(7) Scan parquet spark_catalog.default.store_returns +Output [3]: [sr_item_sk#7, sr_ticket_number#8, sr_returned_date_sk#9] +Batched: true +Location [not included in comparison]/{warehouse_dir}/store_returns] +PushedFilters: [IsNotNull(sr_ticket_number), IsNotNull(sr_item_sk)] +ReadSchema: struct<sr_item_sk:int,sr_ticket_number:int> + +(8) CometFilter +Input [3]: [sr_item_sk#7, sr_ticket_number#8, sr_returned_date_sk#9] +Condition : (isnotnull(sr_ticket_number#8) AND isnotnull(sr_item_sk#7)) + +(9) CometProject +Input [3]: [sr_item_sk#7, sr_ticket_number#8, sr_returned_date_sk#9] +Arguments: [sr_item_sk#7, sr_ticket_number#8], [sr_item_sk#7, sr_ticket_number#8] + +(10) ColumnarToRow [codegen id : 3] +Input [2]: [sr_item_sk#7, sr_ticket_number#8] + +(11) Exchange +Input [2]: [sr_item_sk#7, sr_ticket_number#8] +Arguments: hashpartitioning(sr_ticket_number#8, sr_item_sk#7, 5), ENSURE_REQUIREMENTS, [plan_id=2] + +(12) Sort [codegen id : 4] +Input [2]: [sr_item_sk#7, sr_ticket_number#8] +Arguments: [sr_ticket_number#8 ASC NULLS FIRST, sr_item_sk#7 ASC NULLS FIRST], false, 0 + +(13) SortMergeJoin [codegen id : 9] +Left keys [2]: [ss_ticket_number#4, ss_item_sk#1] +Right keys [2]: [sr_ticket_number#8, sr_item_sk#7] +Join type: Inner +Join condition: None + +(14) Project [codegen id : 9] +Output [4]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_net_paid#5] +Input [7]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5, sr_item_sk#7, sr_ticket_number#8] + +(15) Scan parquet spark_catalog.default.store +Output [5]: [s_store_sk#10, s_store_name#11, s_market_id#12, s_state#13, s_zip#14] +Batched: true +Location [not included in comparison]/{warehouse_dir}/store] +PushedFilters: [IsNotNull(s_market_id), EqualTo(s_market_id,8), IsNotNull(s_store_sk), IsNotNull(s_zip)] +ReadSchema: struct<s_store_sk:int,s_store_name:string,s_market_id:int,s_state:string,s_zip:string> + +(16) CometFilter +Input [5]: [s_store_sk#10, s_store_name#11, s_market_id#12, s_state#13, s_zip#14] +Condition : (((isnotnull(s_market_id#12) AND (s_market_id#12 = 8)) AND isnotnull(s_store_sk#10)) AND isnotnull(s_zip#14)) + +(17) CometProject +Input [5]: [s_store_sk#10, s_store_name#11, s_market_id#12, s_state#13, s_zip#14] +Arguments: [s_store_sk#10, s_store_name#11, s_state#13, s_zip#14], [s_store_sk#10, s_store_name#11, s_state#13, s_zip#14] + +(18) ColumnarToRow [codegen id : 5] +Input [4]: [s_store_sk#10, s_store_name#11, s_state#13, s_zip#14] + +(19) BroadcastExchange +Input [4]: [s_store_sk#10, s_store_name#11, s_state#13, s_zip#14] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] + +(20) BroadcastHashJoin [codegen id : 9] +Left keys [1]: [ss_store_sk#3] +Right keys [1]: [s_store_sk#10] +Join type: Inner +Join condition: None + +(21) Project [codegen id : 9] +Output [6]: [ss_item_sk#1, ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#13, s_zip#14] +Input [8]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_net_paid#5, s_store_sk#10, s_store_name#11, s_state#13, s_zip#14] + +(22) Scan parquet spark_catalog.default.item +Output [6]: [i_item_sk#15, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20] +Batched: true +Location [not included in comparison]/{warehouse_dir}/item] +PushedFilters: [IsNotNull(i_color), EqualTo(i_color,pale ), IsNotNull(i_item_sk)] +ReadSchema: struct<i_item_sk:int,i_current_price:decimal(7,2),i_size:string,i_color:string,i_units:string,i_manager_id:int> + +(23) CometFilter +Input [6]: [i_item_sk#15, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20] +Condition : ((isnotnull(i_color#18) AND (i_color#18 = pale )) AND isnotnull(i_item_sk#15)) + +(24) ColumnarToRow [codegen id : 6] +Input [6]: [i_item_sk#15, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20] + +(25) BroadcastExchange +Input [6]: [i_item_sk#15, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=4] + +(26) BroadcastHashJoin [codegen id : 9] +Left keys [1]: [ss_item_sk#1] +Right keys [1]: [i_item_sk#15] +Join type: Inner +Join condition: None + +(27) Project [codegen id : 9] +Output [10]: [ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#13, s_zip#14, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20] +Input [12]: [ss_item_sk#1, ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#13, s_zip#14, i_item_sk#15, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20] + +(28) Scan parquet spark_catalog.default.customer +Output [4]: [c_customer_sk#21, c_first_name#22, c_last_name#23, c_birth_country#24] +Batched: true +Location [not included in comparison]/{warehouse_dir}/customer] +PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_birth_country)] +ReadSchema: struct<c_customer_sk:int,c_first_name:string,c_last_name:string,c_birth_country:string> + +(29) CometFilter +Input [4]: [c_customer_sk#21, c_first_name#22, c_last_name#23, c_birth_country#24] +Condition : (isnotnull(c_customer_sk#21) AND isnotnull(c_birth_country#24)) + +(30) ColumnarToRow [codegen id : 7] +Input [4]: [c_customer_sk#21, c_first_name#22, c_last_name#23, c_birth_country#24] + +(31) BroadcastExchange +Input [4]: [c_customer_sk#21, c_first_name#22, c_last_name#23, c_birth_country#24] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=5] + +(32) BroadcastHashJoin [codegen id : 9] +Left keys [1]: [ss_customer_sk#2] +Right keys [1]: [c_customer_sk#21] +Join type: Inner +Join condition: None + +(33) Project [codegen id : 9] +Output [12]: [ss_net_paid#5, s_store_name#11, s_state#13, s_zip#14, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20, c_first_name#22, c_last_name#23, c_birth_country#24] +Input [14]: [ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#13, s_zip#14, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20, c_customer_sk#21, c_first_name#22, c_last_name#23, c_birth_country#24] + +(34) Scan parquet spark_catalog.default.customer_address +Output [3]: [ca_state#25, ca_zip#26, ca_country#27] +Batched: true +Location [not included in comparison]/{warehouse_dir}/customer_address] +PushedFilters: [IsNotNull(ca_country), IsNotNull(ca_zip)] +ReadSchema: struct<ca_state:string,ca_zip:string,ca_country:string> + +(35) CometFilter +Input [3]: [ca_state#25, ca_zip#26, ca_country#27] +Condition : (isnotnull(ca_country#27) AND isnotnull(ca_zip#26)) + +(36) ColumnarToRow [codegen id : 8] +Input [3]: [ca_state#25, ca_zip#26, ca_country#27] + +(37) BroadcastExchange +Input [3]: [ca_state#25, ca_zip#26, ca_country#27] +Arguments: HashedRelationBroadcastMode(List(upper(input[2, string, false]), input[1, string, false]),false), [plan_id=6] + +(38) BroadcastHashJoin [codegen id : 9] +Left keys [2]: [c_birth_country#24, s_zip#14] +Right keys [2]: [upper(ca_country#27), ca_zip#26] +Join type: Inner +Join condition: None + +(39) Project [codegen id : 9] +Output [11]: [ss_net_paid#5, s_store_name#11, s_state#13, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20, c_first_name#22, c_last_name#23, ca_state#25] +Input [15]: [ss_net_paid#5, s_store_name#11, s_state#13, s_zip#14, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20, c_first_name#22, c_last_name#23, c_birth_country#24, ca_state#25, ca_zip#26, ca_country#27] + +(40) HashAggregate [codegen id : 9] +Input [11]: [ss_net_paid#5, s_store_name#11, s_state#13, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20, c_first_name#22, c_last_name#23, ca_state#25] +Keys [10]: [c_last_name#23, c_first_name#22, s_store_name#11, ca_state#25, s_state#13, i_color#18, i_current_price#16, i_manager_id#20, i_units#19, i_size#17] +Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#5))] +Aggregate Attributes [1]: [sum#28] +Results [11]: [c_last_name#23, c_first_name#22, s_store_name#11, ca_state#25, s_state#13, i_color#18, i_current_price#16, i_manager_id#20, i_units#19, i_size#17, sum#29] + +(41) Exchange +Input [11]: [c_last_name#23, c_first_name#22, s_store_name#11, ca_state#25, s_state#13, i_color#18, i_current_price#16, i_manager_id#20, i_units#19, i_size#17, sum#29] +Arguments: hashpartitioning(c_last_name#23, c_first_name#22, s_store_name#11, ca_state#25, s_state#13, i_color#18, i_current_price#16, i_manager_id#20, i_units#19, i_size#17, 5), ENSURE_REQUIREMENTS, [plan_id=7] + +(42) HashAggregate [codegen id : 10] +Input [11]: [c_last_name#23, c_first_name#22, s_store_name#11, ca_state#25, s_state#13, i_color#18, i_current_price#16, i_manager_id#20, i_units#19, i_size#17, sum#29] +Keys [10]: [c_last_name#23, c_first_name#22, s_store_name#11, ca_state#25, s_state#13, i_color#18, i_current_price#16, i_manager_id#20, i_units#19, i_size#17] +Functions [1]: [sum(UnscaledValue(ss_net_paid#5))] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_paid#5))#30] +Results [4]: [c_last_name#23, c_first_name#22, s_store_name#11, MakeDecimal(sum(UnscaledValue(ss_net_paid#5))#30,17,2) AS netpaid#31] + +(43) HashAggregate [codegen id : 10] +Input [4]: [c_last_name#23, c_first_name#22, s_store_name#11, netpaid#31] +Keys [3]: [c_last_name#23, c_first_name#22, s_store_name#11] +Functions [1]: [partial_sum(netpaid#31)] +Aggregate Attributes [2]: [sum#32, isEmpty#33] +Results [5]: [c_last_name#23, c_first_name#22, s_store_name#11, sum#34, isEmpty#35] + +(44) Exchange +Input [5]: [c_last_name#23, c_first_name#22, s_store_name#11, sum#34, isEmpty#35] +Arguments: hashpartitioning(c_last_name#23, c_first_name#22, s_store_name#11, 5), ENSURE_REQUIREMENTS, [plan_id=8] + +(45) HashAggregate [codegen id : 11] +Input [5]: [c_last_name#23, c_first_name#22, s_store_name#11, sum#34, isEmpty#35] +Keys [3]: [c_last_name#23, c_first_name#22, s_store_name#11] +Functions [1]: [sum(netpaid#31)] +Aggregate Attributes [1]: [sum(netpaid#31)#36] +Results [4]: [c_last_name#23, c_first_name#22, s_store_name#11, sum(netpaid#31)#36 AS paid#37] + +(46) Filter [codegen id : 11] +Input [4]: [c_last_name#23, c_first_name#22, s_store_name#11, paid#37] +Condition : (isnotnull(paid#37) AND (cast(paid#37 as decimal(33,8)) > cast(Subquery scalar-subquery#38, [id=#39] as decimal(33,8)))) + +===== Subqueries ===== + +Subquery:1 Hosting operator id = 46 Hosting Expression = Subquery scalar-subquery#38, [id=#39] +* HashAggregate (73) ++- Exchange (72) + +- * HashAggregate (71) + +- * HashAggregate (70) + +- Exchange (69) + +- * HashAggregate (68) + +- * Project (67) + +- * BroadcastHashJoin Inner BuildRight (66) + :- * Project (64) + : +- * BroadcastHashJoin Inner BuildRight (63) + : :- * Project (61) + : : +- * BroadcastHashJoin Inner BuildRight (60) + : : :- * Project (55) + : : : +- * BroadcastHashJoin Inner BuildRight (54) + : : : :- * Project (52) + : : : : +- * SortMergeJoin Inner (51) + : : : : :- * Sort (48) + : : : : : +- ReusedExchange (47) + : : : : +- * Sort (50) + : : : : +- ReusedExchange (49) + : : : +- ReusedExchange (53) + : : +- BroadcastExchange (59) + : : +- * ColumnarToRow (58) + : : +- CometFilter (57) + : : +- CometScan parquet spark_catalog.default.item (56) + : +- ReusedExchange (62) + +- ReusedExchange (65) + + +(47) ReusedExchange [Reuses operator id: 5] +Output [5]: [ss_item_sk#40, ss_customer_sk#41, ss_store_sk#42, ss_ticket_number#43, ss_net_paid#44] + +(48) Sort [codegen id : 2] +Input [5]: [ss_item_sk#40, ss_customer_sk#41, ss_store_sk#42, ss_ticket_number#43, ss_net_paid#44] +Arguments: [ss_ticket_number#43 ASC NULLS FIRST, ss_item_sk#40 ASC NULLS FIRST], false, 0 + +(49) ReusedExchange [Reuses operator id: 11] +Output [2]: [sr_item_sk#45, sr_ticket_number#46] + +(50) Sort [codegen id : 4] +Input [2]: [sr_item_sk#45, sr_ticket_number#46] +Arguments: [sr_ticket_number#46 ASC NULLS FIRST, sr_item_sk#45 ASC NULLS FIRST], false, 0 + +(51) SortMergeJoin [codegen id : 9] +Left keys [2]: [ss_ticket_number#43, ss_item_sk#40] +Right keys [2]: [sr_ticket_number#46, sr_item_sk#45] +Join type: Inner +Join condition: None + +(52) Project [codegen id : 9] +Output [4]: [ss_item_sk#40, ss_customer_sk#41, ss_store_sk#42, ss_net_paid#44] +Input [7]: [ss_item_sk#40, ss_customer_sk#41, ss_store_sk#42, ss_ticket_number#43, ss_net_paid#44, sr_item_sk#45, sr_ticket_number#46] + +(53) ReusedExchange [Reuses operator id: 19] +Output [4]: [s_store_sk#47, s_store_name#48, s_state#49, s_zip#50] + +(54) BroadcastHashJoin [codegen id : 9] +Left keys [1]: [ss_store_sk#42] +Right keys [1]: [s_store_sk#47] +Join type: Inner +Join condition: None + +(55) Project [codegen id : 9] +Output [6]: [ss_item_sk#40, ss_customer_sk#41, ss_net_paid#44, s_store_name#48, s_state#49, s_zip#50] +Input [8]: [ss_item_sk#40, ss_customer_sk#41, ss_store_sk#42, ss_net_paid#44, s_store_sk#47, s_store_name#48, s_state#49, s_zip#50] + +(56) Scan parquet spark_catalog.default.item +Output [6]: [i_item_sk#51, i_current_price#52, i_size#53, i_color#54, i_units#55, i_manager_id#56] +Batched: true +Location [not included in comparison]/{warehouse_dir}/item] +PushedFilters: [IsNotNull(i_item_sk)] +ReadSchema: struct<i_item_sk:int,i_current_price:decimal(7,2),i_size:string,i_color:string,i_units:string,i_manager_id:int> + +(57) CometFilter +Input [6]: [i_item_sk#51, i_current_price#52, i_size#53, i_color#54, i_units#55, i_manager_id#56] +Condition : isnotnull(i_item_sk#51) + +(58) ColumnarToRow [codegen id : 6] +Input [6]: [i_item_sk#51, i_current_price#52, i_size#53, i_color#54, i_units#55, i_manager_id#56] + +(59) BroadcastExchange +Input [6]: [i_item_sk#51, i_current_price#52, i_size#53, i_color#54, i_units#55, i_manager_id#56] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=9] + +(60) BroadcastHashJoin [codegen id : 9] +Left keys [1]: [ss_item_sk#40] +Right keys [1]: [i_item_sk#51] +Join type: Inner +Join condition: None + +(61) Project [codegen id : 9] +Output [10]: [ss_customer_sk#41, ss_net_paid#44, s_store_name#48, s_state#49, s_zip#50, i_current_price#52, i_size#53, i_color#54, i_units#55, i_manager_id#56] +Input [12]: [ss_item_sk#40, ss_customer_sk#41, ss_net_paid#44, s_store_name#48, s_state#49, s_zip#50, i_item_sk#51, i_current_price#52, i_size#53, i_color#54, i_units#55, i_manager_id#56] + +(62) ReusedExchange [Reuses operator id: 31] +Output [4]: [c_customer_sk#57, c_first_name#58, c_last_name#59, c_birth_country#60] + +(63) BroadcastHashJoin [codegen id : 9] +Left keys [1]: [ss_customer_sk#41] +Right keys [1]: [c_customer_sk#57] +Join type: Inner +Join condition: None + +(64) Project [codegen id : 9] +Output [12]: [ss_net_paid#44, s_store_name#48, s_state#49, s_zip#50, i_current_price#52, i_size#53, i_color#54, i_units#55, i_manager_id#56, c_first_name#58, c_last_name#59, c_birth_country#60] +Input [14]: [ss_customer_sk#41, ss_net_paid#44, s_store_name#48, s_state#49, s_zip#50, i_current_price#52, i_size#53, i_color#54, i_units#55, i_manager_id#56, c_customer_sk#57, c_first_name#58, c_last_name#59, c_birth_country#60] + +(65) ReusedExchange [Reuses operator id: 37] +Output [3]: [ca_state#61, ca_zip#62, ca_country#63] + +(66) BroadcastHashJoin [codegen id : 9] +Left keys [2]: [c_birth_country#60, s_zip#50] +Right keys [2]: [upper(ca_country#63), ca_zip#62] +Join type: Inner +Join condition: None + +(67) Project [codegen id : 9] +Output [11]: [ss_net_paid#44, s_store_name#48, s_state#49, i_current_price#52, i_size#53, i_color#54, i_units#55, i_manager_id#56, c_first_name#58, c_last_name#59, ca_state#61] +Input [15]: [ss_net_paid#44, s_store_name#48, s_state#49, s_zip#50, i_current_price#52, i_size#53, i_color#54, i_units#55, i_manager_id#56, c_first_name#58, c_last_name#59, c_birth_country#60, ca_state#61, ca_zip#62, ca_country#63] + +(68) HashAggregate [codegen id : 9] +Input [11]: [ss_net_paid#44, s_store_name#48, s_state#49, i_current_price#52, i_size#53, i_color#54, i_units#55, i_manager_id#56, c_first_name#58, c_last_name#59, ca_state#61] +Keys [10]: [c_last_name#59, c_first_name#58, s_store_name#48, ca_state#61, s_state#49, i_color#54, i_current_price#52, i_manager_id#56, i_units#55, i_size#53] +Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#44))] +Aggregate Attributes [1]: [sum#64] +Results [11]: [c_last_name#59, c_first_name#58, s_store_name#48, ca_state#61, s_state#49, i_color#54, i_current_price#52, i_manager_id#56, i_units#55, i_size#53, sum#65] + +(69) Exchange +Input [11]: [c_last_name#59, c_first_name#58, s_store_name#48, ca_state#61, s_state#49, i_color#54, i_current_price#52, i_manager_id#56, i_units#55, i_size#53, sum#65] +Arguments: hashpartitioning(c_last_name#59, c_first_name#58, s_store_name#48, ca_state#61, s_state#49, i_color#54, i_current_price#52, i_manager_id#56, i_units#55, i_size#53, 5), ENSURE_REQUIREMENTS, [plan_id=10] + +(70) HashAggregate [codegen id : 10] +Input [11]: [c_last_name#59, c_first_name#58, s_store_name#48, ca_state#61, s_state#49, i_color#54, i_current_price#52, i_manager_id#56, i_units#55, i_size#53, sum#65] +Keys [10]: [c_last_name#59, c_first_name#58, s_store_name#48, ca_state#61, s_state#49, i_color#54, i_current_price#52, i_manager_id#56, i_units#55, i_size#53] +Functions [1]: [sum(UnscaledValue(ss_net_paid#44))] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_paid#44))#30] +Results [1]: [MakeDecimal(sum(UnscaledValue(ss_net_paid#44))#30,17,2) AS netpaid#66] + +(71) HashAggregate [codegen id : 10] +Input [1]: [netpaid#66] +Keys: [] +Functions [1]: [partial_avg(netpaid#66)] +Aggregate Attributes [2]: [sum#67, count#68] +Results [2]: [sum#69, count#70] + +(72) Exchange +Input [2]: [sum#69, count#70] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=11] + +(73) HashAggregate [codegen id : 11] +Input [2]: [sum#69, count#70] +Keys: [] +Functions [1]: [avg(netpaid#66)] +Aggregate Attributes [1]: [avg(netpaid#66)#71] +Results [1]: [(0.05 * avg(netpaid#66)#71) AS (0.05 * avg(netpaid))#72] + + 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 new file mode 100644 index 000000000..8ebd45fd1 --- /dev/null +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q24a/simplified.txt @@ -0,0 +1,118 @@ +WholeStageCodegen (11) + Filter [paid] + Subquery #1 + WholeStageCodegen (11) + HashAggregate [sum,count] [avg(netpaid),(0.05 * avg(netpaid)),sum,count] + InputAdapter + Exchange #9 + WholeStageCodegen (10) + HashAggregate [netpaid] [sum,count,sum,count] + HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,sum] [sum(UnscaledValue(ss_net_paid)),netpaid,sum] + InputAdapter + Exchange [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size] #10 + WholeStageCodegen (9) + HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,ss_net_paid] [sum,sum] + Project [ss_net_paid,s_store_name,s_state,i_current_price,i_size,i_color,i_units,i_manager_id,c_first_name,c_last_name,ca_state] + BroadcastHashJoin [c_birth_country,s_zip,ca_country,ca_zip] + Project [ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id,c_first_name,c_last_name,c_birth_country] + BroadcastHashJoin [ss_customer_sk,c_customer_sk] + Project [ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id] + BroadcastHashJoin [ss_item_sk,i_item_sk] + Project [ss_item_sk,ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip] + BroadcastHashJoin [ss_store_sk,s_store_sk] + Project [ss_item_sk,ss_customer_sk,ss_store_sk,ss_net_paid] + SortMergeJoin [ss_ticket_number,ss_item_sk,sr_ticket_number,sr_item_sk] + InputAdapter + WholeStageCodegen (2) + Sort [ss_ticket_number,ss_item_sk] + InputAdapter + ReusedExchange [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid] #3 + InputAdapter + WholeStageCodegen (4) + Sort [sr_ticket_number,sr_item_sk] + InputAdapter + ReusedExchange [sr_item_sk,sr_ticket_number] #4 + InputAdapter + ReusedExchange [s_store_sk,s_store_name,s_state,s_zip] #5 + InputAdapter + BroadcastExchange #11 + WholeStageCodegen (6) + ColumnarToRow + InputAdapter + CometFilter [i_item_sk] + CometScan parquet spark_catalog.default.item [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] + InputAdapter + ReusedExchange [c_customer_sk,c_first_name,c_last_name,c_birth_country] #7 + InputAdapter + ReusedExchange [ca_state,ca_zip,ca_country] #8 + HashAggregate [c_last_name,c_first_name,s_store_name,sum,isEmpty] [sum(netpaid),paid,sum,isEmpty] + InputAdapter + Exchange [c_last_name,c_first_name,s_store_name] #1 + WholeStageCodegen (10) + HashAggregate [c_last_name,c_first_name,s_store_name,netpaid] [sum,isEmpty,sum,isEmpty] + HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,sum] [sum(UnscaledValue(ss_net_paid)),netpaid,sum] + InputAdapter + Exchange [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size] #2 + WholeStageCodegen (9) + HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,ss_net_paid] [sum,sum] + Project [ss_net_paid,s_store_name,s_state,i_current_price,i_size,i_color,i_units,i_manager_id,c_first_name,c_last_name,ca_state] + BroadcastHashJoin [c_birth_country,s_zip,ca_country,ca_zip] + Project [ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id,c_first_name,c_last_name,c_birth_country] + BroadcastHashJoin [ss_customer_sk,c_customer_sk] + Project [ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id] + BroadcastHashJoin [ss_item_sk,i_item_sk] + Project [ss_item_sk,ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip] + BroadcastHashJoin [ss_store_sk,s_store_sk] + Project [ss_item_sk,ss_customer_sk,ss_store_sk,ss_net_paid] + SortMergeJoin [ss_ticket_number,ss_item_sk,sr_ticket_number,sr_item_sk] + InputAdapter + WholeStageCodegen (2) + Sort [ss_ticket_number,ss_item_sk] + InputAdapter + Exchange [ss_ticket_number,ss_item_sk] #3 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid] + CometFilter [ss_ticket_number,ss_item_sk,ss_store_sk,ss_customer_sk] + CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid,ss_sold_date_sk] + InputAdapter + WholeStageCodegen (4) + Sort [sr_ticket_number,sr_item_sk] + InputAdapter + Exchange [sr_ticket_number,sr_item_sk] #4 + WholeStageCodegen (3) + ColumnarToRow + InputAdapter + CometProject [sr_item_sk,sr_ticket_number] + CometFilter [sr_ticket_number,sr_item_sk] + CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk] + InputAdapter + BroadcastExchange #5 + WholeStageCodegen (5) + ColumnarToRow + InputAdapter + CometProject [s_store_sk,s_store_name,s_state,s_zip] + CometFilter [s_market_id,s_store_sk,s_zip] + CometScan parquet spark_catalog.default.store [s_store_sk,s_store_name,s_market_id,s_state,s_zip] + InputAdapter + BroadcastExchange #6 + WholeStageCodegen (6) + ColumnarToRow + InputAdapter + CometFilter [i_color,i_item_sk] + CometScan parquet spark_catalog.default.item [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] + InputAdapter + BroadcastExchange #7 + WholeStageCodegen (7) + ColumnarToRow + InputAdapter + CometFilter [c_customer_sk,c_birth_country] + CometScan parquet spark_catalog.default.customer [c_customer_sk,c_first_name,c_last_name,c_birth_country] + InputAdapter + BroadcastExchange #8 + WholeStageCodegen (8) + ColumnarToRow + InputAdapter + CometFilter [ca_country,ca_zip] + CometScan parquet spark_catalog.default.customer_address [ca_state,ca_zip,ca_country] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q24b/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q24b/explain.txt new file mode 100644 index 000000000..0ac5639b7 --- /dev/null +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q24b/explain.txt @@ -0,0 +1,427 @@ +== Physical Plan == +* Filter (46) ++- * HashAggregate (45) + +- Exchange (44) + +- * HashAggregate (43) + +- * HashAggregate (42) + +- Exchange (41) + +- * HashAggregate (40) + +- * Project (39) + +- * BroadcastHashJoin Inner BuildRight (38) + :- * Project (33) + : +- * BroadcastHashJoin Inner BuildRight (32) + : :- * Project (27) + : : +- * BroadcastHashJoin Inner BuildRight (26) + : : :- * Project (21) + : : : +- * BroadcastHashJoin Inner BuildRight (20) + : : : :- * Project (14) + : : : : +- * SortMergeJoin Inner (13) + : : : : :- * Sort (6) + : : : : : +- Exchange (5) + : : : : : +- * ColumnarToRow (4) + : : : : : +- CometProject (3) + : : : : : +- CometFilter (2) + : : : : : +- CometScan parquet spark_catalog.default.store_sales (1) + : : : : +- * Sort (12) + : : : : +- Exchange (11) + : : : : +- * ColumnarToRow (10) + : : : : +- CometProject (9) + : : : : +- CometFilter (8) + : : : : +- CometScan parquet spark_catalog.default.store_returns (7) + : : : +- BroadcastExchange (19) + : : : +- * ColumnarToRow (18) + : : : +- CometProject (17) + : : : +- CometFilter (16) + : : : +- CometScan parquet spark_catalog.default.store (15) + : : +- BroadcastExchange (25) + : : +- * ColumnarToRow (24) + : : +- CometFilter (23) + : : +- CometScan parquet spark_catalog.default.item (22) + : +- BroadcastExchange (31) + : +- * ColumnarToRow (30) + : +- CometFilter (29) + : +- CometScan parquet spark_catalog.default.customer (28) + +- BroadcastExchange (37) + +- * ColumnarToRow (36) + +- CometFilter (35) + +- CometScan parquet spark_catalog.default.customer_address (34) + + +(1) Scan parquet spark_catalog.default.store_sales +Output [6]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5, ss_sold_date_sk#6] +Batched: true +Location [not included in comparison]/{warehouse_dir}/store_sales] +PushedFilters: [IsNotNull(ss_ticket_number), IsNotNull(ss_item_sk), IsNotNull(ss_store_sk), IsNotNull(ss_customer_sk)] +ReadSchema: struct<ss_item_sk:int,ss_customer_sk:int,ss_store_sk:int,ss_ticket_number:int,ss_net_paid:decimal(7,2)> + +(2) CometFilter +Input [6]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5, ss_sold_date_sk#6] +Condition : (((isnotnull(ss_ticket_number#4) AND isnotnull(ss_item_sk#1)) AND isnotnull(ss_store_sk#3)) AND isnotnull(ss_customer_sk#2)) + +(3) CometProject +Input [6]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5, ss_sold_date_sk#6] +Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5], [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5] + +(4) ColumnarToRow [codegen id : 1] +Input [5]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5] + +(5) Exchange +Input [5]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5] +Arguments: hashpartitioning(ss_ticket_number#4, ss_item_sk#1, 5), ENSURE_REQUIREMENTS, [plan_id=1] + +(6) Sort [codegen id : 2] +Input [5]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5] +Arguments: [ss_ticket_number#4 ASC NULLS FIRST, ss_item_sk#1 ASC NULLS FIRST], false, 0 + +(7) Scan parquet spark_catalog.default.store_returns +Output [3]: [sr_item_sk#7, sr_ticket_number#8, sr_returned_date_sk#9] +Batched: true +Location [not included in comparison]/{warehouse_dir}/store_returns] +PushedFilters: [IsNotNull(sr_ticket_number), IsNotNull(sr_item_sk)] +ReadSchema: struct<sr_item_sk:int,sr_ticket_number:int> + +(8) CometFilter +Input [3]: [sr_item_sk#7, sr_ticket_number#8, sr_returned_date_sk#9] +Condition : (isnotnull(sr_ticket_number#8) AND isnotnull(sr_item_sk#7)) + +(9) CometProject +Input [3]: [sr_item_sk#7, sr_ticket_number#8, sr_returned_date_sk#9] +Arguments: [sr_item_sk#7, sr_ticket_number#8], [sr_item_sk#7, sr_ticket_number#8] + +(10) ColumnarToRow [codegen id : 3] +Input [2]: [sr_item_sk#7, sr_ticket_number#8] + +(11) Exchange +Input [2]: [sr_item_sk#7, sr_ticket_number#8] +Arguments: hashpartitioning(sr_ticket_number#8, sr_item_sk#7, 5), ENSURE_REQUIREMENTS, [plan_id=2] + +(12) Sort [codegen id : 4] +Input [2]: [sr_item_sk#7, sr_ticket_number#8] +Arguments: [sr_ticket_number#8 ASC NULLS FIRST, sr_item_sk#7 ASC NULLS FIRST], false, 0 + +(13) SortMergeJoin [codegen id : 9] +Left keys [2]: [ss_ticket_number#4, ss_item_sk#1] +Right keys [2]: [sr_ticket_number#8, sr_item_sk#7] +Join type: Inner +Join condition: None + +(14) Project [codegen id : 9] +Output [4]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_net_paid#5] +Input [7]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5, sr_item_sk#7, sr_ticket_number#8] + +(15) Scan parquet spark_catalog.default.store +Output [5]: [s_store_sk#10, s_store_name#11, s_market_id#12, s_state#13, s_zip#14] +Batched: true +Location [not included in comparison]/{warehouse_dir}/store] +PushedFilters: [IsNotNull(s_market_id), EqualTo(s_market_id,8), IsNotNull(s_store_sk), IsNotNull(s_zip)] +ReadSchema: struct<s_store_sk:int,s_store_name:string,s_market_id:int,s_state:string,s_zip:string> + +(16) CometFilter +Input [5]: [s_store_sk#10, s_store_name#11, s_market_id#12, s_state#13, s_zip#14] +Condition : (((isnotnull(s_market_id#12) AND (s_market_id#12 = 8)) AND isnotnull(s_store_sk#10)) AND isnotnull(s_zip#14)) + +(17) CometProject +Input [5]: [s_store_sk#10, s_store_name#11, s_market_id#12, s_state#13, s_zip#14] +Arguments: [s_store_sk#10, s_store_name#11, s_state#13, s_zip#14], [s_store_sk#10, s_store_name#11, s_state#13, s_zip#14] + +(18) ColumnarToRow [codegen id : 5] +Input [4]: [s_store_sk#10, s_store_name#11, s_state#13, s_zip#14] + +(19) BroadcastExchange +Input [4]: [s_store_sk#10, s_store_name#11, s_state#13, s_zip#14] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] + +(20) BroadcastHashJoin [codegen id : 9] +Left keys [1]: [ss_store_sk#3] +Right keys [1]: [s_store_sk#10] +Join type: Inner +Join condition: None + +(21) Project [codegen id : 9] +Output [6]: [ss_item_sk#1, ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#13, s_zip#14] +Input [8]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_net_paid#5, s_store_sk#10, s_store_name#11, s_state#13, s_zip#14] + +(22) Scan parquet spark_catalog.default.item +Output [6]: [i_item_sk#15, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20] +Batched: true +Location [not included in comparison]/{warehouse_dir}/item] +PushedFilters: [IsNotNull(i_color), EqualTo(i_color,chiffon ), IsNotNull(i_item_sk)] +ReadSchema: struct<i_item_sk:int,i_current_price:decimal(7,2),i_size:string,i_color:string,i_units:string,i_manager_id:int> + +(23) CometFilter +Input [6]: [i_item_sk#15, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20] +Condition : ((isnotnull(i_color#18) AND (i_color#18 = chiffon )) AND isnotnull(i_item_sk#15)) + +(24) ColumnarToRow [codegen id : 6] +Input [6]: [i_item_sk#15, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20] + +(25) BroadcastExchange +Input [6]: [i_item_sk#15, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=4] + +(26) BroadcastHashJoin [codegen id : 9] +Left keys [1]: [ss_item_sk#1] +Right keys [1]: [i_item_sk#15] +Join type: Inner +Join condition: None + +(27) Project [codegen id : 9] +Output [10]: [ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#13, s_zip#14, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20] +Input [12]: [ss_item_sk#1, ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#13, s_zip#14, i_item_sk#15, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20] + +(28) Scan parquet spark_catalog.default.customer +Output [4]: [c_customer_sk#21, c_first_name#22, c_last_name#23, c_birth_country#24] +Batched: true +Location [not included in comparison]/{warehouse_dir}/customer] +PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_birth_country)] +ReadSchema: struct<c_customer_sk:int,c_first_name:string,c_last_name:string,c_birth_country:string> + +(29) CometFilter +Input [4]: [c_customer_sk#21, c_first_name#22, c_last_name#23, c_birth_country#24] +Condition : (isnotnull(c_customer_sk#21) AND isnotnull(c_birth_country#24)) + +(30) ColumnarToRow [codegen id : 7] +Input [4]: [c_customer_sk#21, c_first_name#22, c_last_name#23, c_birth_country#24] + +(31) BroadcastExchange +Input [4]: [c_customer_sk#21, c_first_name#22, c_last_name#23, c_birth_country#24] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=5] + +(32) BroadcastHashJoin [codegen id : 9] +Left keys [1]: [ss_customer_sk#2] +Right keys [1]: [c_customer_sk#21] +Join type: Inner +Join condition: None + +(33) Project [codegen id : 9] +Output [12]: [ss_net_paid#5, s_store_name#11, s_state#13, s_zip#14, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20, c_first_name#22, c_last_name#23, c_birth_country#24] +Input [14]: [ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#13, s_zip#14, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20, c_customer_sk#21, c_first_name#22, c_last_name#23, c_birth_country#24] + +(34) Scan parquet spark_catalog.default.customer_address +Output [3]: [ca_state#25, ca_zip#26, ca_country#27] +Batched: true +Location [not included in comparison]/{warehouse_dir}/customer_address] +PushedFilters: [IsNotNull(ca_country), IsNotNull(ca_zip)] +ReadSchema: struct<ca_state:string,ca_zip:string,ca_country:string> + +(35) CometFilter +Input [3]: [ca_state#25, ca_zip#26, ca_country#27] +Condition : (isnotnull(ca_country#27) AND isnotnull(ca_zip#26)) + +(36) ColumnarToRow [codegen id : 8] +Input [3]: [ca_state#25, ca_zip#26, ca_country#27] + +(37) BroadcastExchange +Input [3]: [ca_state#25, ca_zip#26, ca_country#27] +Arguments: HashedRelationBroadcastMode(List(upper(input[2, string, false]), input[1, string, false]),false), [plan_id=6] + +(38) BroadcastHashJoin [codegen id : 9] +Left keys [2]: [c_birth_country#24, s_zip#14] +Right keys [2]: [upper(ca_country#27), ca_zip#26] +Join type: Inner +Join condition: None + +(39) Project [codegen id : 9] +Output [11]: [ss_net_paid#5, s_store_name#11, s_state#13, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20, c_first_name#22, c_last_name#23, ca_state#25] +Input [15]: [ss_net_paid#5, s_store_name#11, s_state#13, s_zip#14, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20, c_first_name#22, c_last_name#23, c_birth_country#24, ca_state#25, ca_zip#26, ca_country#27] + +(40) HashAggregate [codegen id : 9] +Input [11]: [ss_net_paid#5, s_store_name#11, s_state#13, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20, c_first_name#22, c_last_name#23, ca_state#25] +Keys [10]: [c_last_name#23, c_first_name#22, s_store_name#11, ca_state#25, s_state#13, i_color#18, i_current_price#16, i_manager_id#20, i_units#19, i_size#17] +Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#5))] +Aggregate Attributes [1]: [sum#28] +Results [11]: [c_last_name#23, c_first_name#22, s_store_name#11, ca_state#25, s_state#13, i_color#18, i_current_price#16, i_manager_id#20, i_units#19, i_size#17, sum#29] + +(41) Exchange +Input [11]: [c_last_name#23, c_first_name#22, s_store_name#11, ca_state#25, s_state#13, i_color#18, i_current_price#16, i_manager_id#20, i_units#19, i_size#17, sum#29] +Arguments: hashpartitioning(c_last_name#23, c_first_name#22, s_store_name#11, ca_state#25, s_state#13, i_color#18, i_current_price#16, i_manager_id#20, i_units#19, i_size#17, 5), ENSURE_REQUIREMENTS, [plan_id=7] + +(42) HashAggregate [codegen id : 10] +Input [11]: [c_last_name#23, c_first_name#22, s_store_name#11, ca_state#25, s_state#13, i_color#18, i_current_price#16, i_manager_id#20, i_units#19, i_size#17, sum#29] +Keys [10]: [c_last_name#23, c_first_name#22, s_store_name#11, ca_state#25, s_state#13, i_color#18, i_current_price#16, i_manager_id#20, i_units#19, i_size#17] +Functions [1]: [sum(UnscaledValue(ss_net_paid#5))] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_paid#5))#30] +Results [4]: [c_last_name#23, c_first_name#22, s_store_name#11, MakeDecimal(sum(UnscaledValue(ss_net_paid#5))#30,17,2) AS netpaid#31] + +(43) HashAggregate [codegen id : 10] +Input [4]: [c_last_name#23, c_first_name#22, s_store_name#11, netpaid#31] +Keys [3]: [c_last_name#23, c_first_name#22, s_store_name#11] +Functions [1]: [partial_sum(netpaid#31)] +Aggregate Attributes [2]: [sum#32, isEmpty#33] +Results [5]: [c_last_name#23, c_first_name#22, s_store_name#11, sum#34, isEmpty#35] + +(44) Exchange +Input [5]: [c_last_name#23, c_first_name#22, s_store_name#11, sum#34, isEmpty#35] +Arguments: hashpartitioning(c_last_name#23, c_first_name#22, s_store_name#11, 5), ENSURE_REQUIREMENTS, [plan_id=8] + +(45) HashAggregate [codegen id : 11] +Input [5]: [c_last_name#23, c_first_name#22, s_store_name#11, sum#34, isEmpty#35] +Keys [3]: [c_last_name#23, c_first_name#22, s_store_name#11] +Functions [1]: [sum(netpaid#31)] +Aggregate Attributes [1]: [sum(netpaid#31)#36] +Results [4]: [c_last_name#23, c_first_name#22, s_store_name#11, sum(netpaid#31)#36 AS paid#37] + +(46) Filter [codegen id : 11] +Input [4]: [c_last_name#23, c_first_name#22, s_store_name#11, paid#37] +Condition : (isnotnull(paid#37) AND (cast(paid#37 as decimal(33,8)) > cast(Subquery scalar-subquery#38, [id=#39] as decimal(33,8)))) + +===== Subqueries ===== + +Subquery:1 Hosting operator id = 46 Hosting Expression = Subquery scalar-subquery#38, [id=#39] +* HashAggregate (73) ++- Exchange (72) + +- * HashAggregate (71) + +- * HashAggregate (70) + +- Exchange (69) + +- * HashAggregate (68) + +- * Project (67) + +- * BroadcastHashJoin Inner BuildRight (66) + :- * Project (64) + : +- * BroadcastHashJoin Inner BuildRight (63) + : :- * Project (61) + : : +- * BroadcastHashJoin Inner BuildRight (60) + : : :- * Project (55) + : : : +- * BroadcastHashJoin Inner BuildRight (54) + : : : :- * Project (52) + : : : : +- * SortMergeJoin Inner (51) + : : : : :- * Sort (48) + : : : : : +- ReusedExchange (47) + : : : : +- * Sort (50) + : : : : +- ReusedExchange (49) + : : : +- ReusedExchange (53) + : : +- BroadcastExchange (59) + : : +- * ColumnarToRow (58) + : : +- CometFilter (57) + : : +- CometScan parquet spark_catalog.default.item (56) + : +- ReusedExchange (62) + +- ReusedExchange (65) + + +(47) ReusedExchange [Reuses operator id: 5] +Output [5]: [ss_item_sk#40, ss_customer_sk#41, ss_store_sk#42, ss_ticket_number#43, ss_net_paid#44] + +(48) Sort [codegen id : 2] +Input [5]: [ss_item_sk#40, ss_customer_sk#41, ss_store_sk#42, ss_ticket_number#43, ss_net_paid#44] +Arguments: [ss_ticket_number#43 ASC NULLS FIRST, ss_item_sk#40 ASC NULLS FIRST], false, 0 + +(49) ReusedExchange [Reuses operator id: 11] +Output [2]: [sr_item_sk#45, sr_ticket_number#46] + +(50) Sort [codegen id : 4] +Input [2]: [sr_item_sk#45, sr_ticket_number#46] +Arguments: [sr_ticket_number#46 ASC NULLS FIRST, sr_item_sk#45 ASC NULLS FIRST], false, 0 + +(51) SortMergeJoin [codegen id : 9] +Left keys [2]: [ss_ticket_number#43, ss_item_sk#40] +Right keys [2]: [sr_ticket_number#46, sr_item_sk#45] +Join type: Inner +Join condition: None + +(52) Project [codegen id : 9] +Output [4]: [ss_item_sk#40, ss_customer_sk#41, ss_store_sk#42, ss_net_paid#44] +Input [7]: [ss_item_sk#40, ss_customer_sk#41, ss_store_sk#42, ss_ticket_number#43, ss_net_paid#44, sr_item_sk#45, sr_ticket_number#46] + +(53) ReusedExchange [Reuses operator id: 19] +Output [4]: [s_store_sk#47, s_store_name#48, s_state#49, s_zip#50] + +(54) BroadcastHashJoin [codegen id : 9] +Left keys [1]: [ss_store_sk#42] +Right keys [1]: [s_store_sk#47] +Join type: Inner +Join condition: None + +(55) Project [codegen id : 9] +Output [6]: [ss_item_sk#40, ss_customer_sk#41, ss_net_paid#44, s_store_name#48, s_state#49, s_zip#50] +Input [8]: [ss_item_sk#40, ss_customer_sk#41, ss_store_sk#42, ss_net_paid#44, s_store_sk#47, s_store_name#48, s_state#49, s_zip#50] + +(56) Scan parquet spark_catalog.default.item +Output [6]: [i_item_sk#51, i_current_price#52, i_size#53, i_color#54, i_units#55, i_manager_id#56] +Batched: true +Location [not included in comparison]/{warehouse_dir}/item] +PushedFilters: [IsNotNull(i_item_sk)] +ReadSchema: struct<i_item_sk:int,i_current_price:decimal(7,2),i_size:string,i_color:string,i_units:string,i_manager_id:int> + +(57) CometFilter +Input [6]: [i_item_sk#51, i_current_price#52, i_size#53, i_color#54, i_units#55, i_manager_id#56] +Condition : isnotnull(i_item_sk#51) + +(58) ColumnarToRow [codegen id : 6] +Input [6]: [i_item_sk#51, i_current_price#52, i_size#53, i_color#54, i_units#55, i_manager_id#56] + +(59) BroadcastExchange +Input [6]: [i_item_sk#51, i_current_price#52, i_size#53, i_color#54, i_units#55, i_manager_id#56] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=9] + +(60) BroadcastHashJoin [codegen id : 9] +Left keys [1]: [ss_item_sk#40] +Right keys [1]: [i_item_sk#51] +Join type: Inner +Join condition: None + +(61) Project [codegen id : 9] +Output [10]: [ss_customer_sk#41, ss_net_paid#44, s_store_name#48, s_state#49, s_zip#50, i_current_price#52, i_size#53, i_color#54, i_units#55, i_manager_id#56] +Input [12]: [ss_item_sk#40, ss_customer_sk#41, ss_net_paid#44, s_store_name#48, s_state#49, s_zip#50, i_item_sk#51, i_current_price#52, i_size#53, i_color#54, i_units#55, i_manager_id#56] + +(62) ReusedExchange [Reuses operator id: 31] +Output [4]: [c_customer_sk#57, c_first_name#58, c_last_name#59, c_birth_country#60] + +(63) BroadcastHashJoin [codegen id : 9] +Left keys [1]: [ss_customer_sk#41] +Right keys [1]: [c_customer_sk#57] +Join type: Inner +Join condition: None + +(64) Project [codegen id : 9] +Output [12]: [ss_net_paid#44, s_store_name#48, s_state#49, s_zip#50, i_current_price#52, i_size#53, i_color#54, i_units#55, i_manager_id#56, c_first_name#58, c_last_name#59, c_birth_country#60] +Input [14]: [ss_customer_sk#41, ss_net_paid#44, s_store_name#48, s_state#49, s_zip#50, i_current_price#52, i_size#53, i_color#54, i_units#55, i_manager_id#56, c_customer_sk#57, c_first_name#58, c_last_name#59, c_birth_country#60] + +(65) ReusedExchange [Reuses operator id: 37] +Output [3]: [ca_state#61, ca_zip#62, ca_country#63] + +(66) BroadcastHashJoin [codegen id : 9] +Left keys [2]: [c_birth_country#60, s_zip#50] +Right keys [2]: [upper(ca_country#63), ca_zip#62] +Join type: Inner +Join condition: None + +(67) Project [codegen id : 9] +Output [11]: [ss_net_paid#44, s_store_name#48, s_state#49, i_current_price#52, i_size#53, i_color#54, i_units#55, i_manager_id#56, c_first_name#58, c_last_name#59, ca_state#61] +Input [15]: [ss_net_paid#44, s_store_name#48, s_state#49, s_zip#50, i_current_price#52, i_size#53, i_color#54, i_units#55, i_manager_id#56, c_first_name#58, c_last_name#59, c_birth_country#60, ca_state#61, ca_zip#62, ca_country#63] + +(68) HashAggregate [codegen id : 9] +Input [11]: [ss_net_paid#44, s_store_name#48, s_state#49, i_current_price#52, i_size#53, i_color#54, i_units#55, i_manager_id#56, c_first_name#58, c_last_name#59, ca_state#61] +Keys [10]: [c_last_name#59, c_first_name#58, s_store_name#48, ca_state#61, s_state#49, i_color#54, i_current_price#52, i_manager_id#56, i_units#55, i_size#53] +Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#44))] +Aggregate Attributes [1]: [sum#64] +Results [11]: [c_last_name#59, c_first_name#58, s_store_name#48, ca_state#61, s_state#49, i_color#54, i_current_price#52, i_manager_id#56, i_units#55, i_size#53, sum#65] + +(69) Exchange +Input [11]: [c_last_name#59, c_first_name#58, s_store_name#48, ca_state#61, s_state#49, i_color#54, i_current_price#52, i_manager_id#56, i_units#55, i_size#53, sum#65] +Arguments: hashpartitioning(c_last_name#59, c_first_name#58, s_store_name#48, ca_state#61, s_state#49, i_color#54, i_current_price#52, i_manager_id#56, i_units#55, i_size#53, 5), ENSURE_REQUIREMENTS, [plan_id=10] + +(70) HashAggregate [codegen id : 10] +Input [11]: [c_last_name#59, c_first_name#58, s_store_name#48, ca_state#61, s_state#49, i_color#54, i_current_price#52, i_manager_id#56, i_units#55, i_size#53, sum#65] +Keys [10]: [c_last_name#59, c_first_name#58, s_store_name#48, ca_state#61, s_state#49, i_color#54, i_current_price#52, i_manager_id#56, i_units#55, i_size#53] +Functions [1]: [sum(UnscaledValue(ss_net_paid#44))] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_paid#44))#30] +Results [1]: [MakeDecimal(sum(UnscaledValue(ss_net_paid#44))#30,17,2) AS netpaid#66] + +(71) HashAggregate [codegen id : 10] +Input [1]: [netpaid#66] +Keys: [] +Functions [1]: [partial_avg(netpaid#66)] +Aggregate Attributes [2]: [sum#67, count#68] +Results [2]: [sum#69, count#70] + +(72) Exchange +Input [2]: [sum#69, count#70] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=11] + +(73) HashAggregate [codegen id : 11] +Input [2]: [sum#69, count#70] +Keys: [] +Functions [1]: [avg(netpaid#66)] +Aggregate Attributes [1]: [avg(netpaid#66)#71] +Results [1]: [(0.05 * avg(netpaid#66)#71) AS (0.05 * avg(netpaid))#72] + + 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 new file mode 100644 index 000000000..8ebd45fd1 --- /dev/null +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q24b/simplified.txt @@ -0,0 +1,118 @@ +WholeStageCodegen (11) + Filter [paid] + Subquery #1 + WholeStageCodegen (11) + HashAggregate [sum,count] [avg(netpaid),(0.05 * avg(netpaid)),sum,count] + InputAdapter + Exchange #9 + WholeStageCodegen (10) + HashAggregate [netpaid] [sum,count,sum,count] + HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,sum] [sum(UnscaledValue(ss_net_paid)),netpaid,sum] + InputAdapter + Exchange [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size] #10 + WholeStageCodegen (9) + HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,ss_net_paid] [sum,sum] + Project [ss_net_paid,s_store_name,s_state,i_current_price,i_size,i_color,i_units,i_manager_id,c_first_name,c_last_name,ca_state] + BroadcastHashJoin [c_birth_country,s_zip,ca_country,ca_zip] + Project [ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id,c_first_name,c_last_name,c_birth_country] + BroadcastHashJoin [ss_customer_sk,c_customer_sk] + Project [ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id] + BroadcastHashJoin [ss_item_sk,i_item_sk] + Project [ss_item_sk,ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip] + BroadcastHashJoin [ss_store_sk,s_store_sk] + Project [ss_item_sk,ss_customer_sk,ss_store_sk,ss_net_paid] + SortMergeJoin [ss_ticket_number,ss_item_sk,sr_ticket_number,sr_item_sk] + InputAdapter + WholeStageCodegen (2) + Sort [ss_ticket_number,ss_item_sk] + InputAdapter + ReusedExchange [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid] #3 + InputAdapter + WholeStageCodegen (4) + Sort [sr_ticket_number,sr_item_sk] + InputAdapter + ReusedExchange [sr_item_sk,sr_ticket_number] #4 + InputAdapter + ReusedExchange [s_store_sk,s_store_name,s_state,s_zip] #5 + InputAdapter + BroadcastExchange #11 + WholeStageCodegen (6) + ColumnarToRow + InputAdapter + CometFilter [i_item_sk] + CometScan parquet spark_catalog.default.item [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] + InputAdapter + ReusedExchange [c_customer_sk,c_first_name,c_last_name,c_birth_country] #7 + InputAdapter + ReusedExchange [ca_state,ca_zip,ca_country] #8 + HashAggregate [c_last_name,c_first_name,s_store_name,sum,isEmpty] [sum(netpaid),paid,sum,isEmpty] + InputAdapter + Exchange [c_last_name,c_first_name,s_store_name] #1 + WholeStageCodegen (10) + HashAggregate [c_last_name,c_first_name,s_store_name,netpaid] [sum,isEmpty,sum,isEmpty] + HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,sum] [sum(UnscaledValue(ss_net_paid)),netpaid,sum] + InputAdapter + Exchange [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size] #2 + WholeStageCodegen (9) + HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,ss_net_paid] [sum,sum] + Project [ss_net_paid,s_store_name,s_state,i_current_price,i_size,i_color,i_units,i_manager_id,c_first_name,c_last_name,ca_state] + BroadcastHashJoin [c_birth_country,s_zip,ca_country,ca_zip] + Project [ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id,c_first_name,c_last_name,c_birth_country] + BroadcastHashJoin [ss_customer_sk,c_customer_sk] + Project [ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id] + BroadcastHashJoin [ss_item_sk,i_item_sk] + Project [ss_item_sk,ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip] + BroadcastHashJoin [ss_store_sk,s_store_sk] + Project [ss_item_sk,ss_customer_sk,ss_store_sk,ss_net_paid] + SortMergeJoin [ss_ticket_number,ss_item_sk,sr_ticket_number,sr_item_sk] + InputAdapter + WholeStageCodegen (2) + Sort [ss_ticket_number,ss_item_sk] + InputAdapter + Exchange [ss_ticket_number,ss_item_sk] #3 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid] + CometFilter [ss_ticket_number,ss_item_sk,ss_store_sk,ss_customer_sk] + CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid,ss_sold_date_sk] + InputAdapter + WholeStageCodegen (4) + Sort [sr_ticket_number,sr_item_sk] + InputAdapter + Exchange [sr_ticket_number,sr_item_sk] #4 + WholeStageCodegen (3) + ColumnarToRow + InputAdapter + CometProject [sr_item_sk,sr_ticket_number] + CometFilter [sr_ticket_number,sr_item_sk] + CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk] + InputAdapter + BroadcastExchange #5 + WholeStageCodegen (5) + ColumnarToRow + InputAdapter + CometProject [s_store_sk,s_store_name,s_state,s_zip] + CometFilter [s_market_id,s_store_sk,s_zip] + CometScan parquet spark_catalog.default.store [s_store_sk,s_store_name,s_market_id,s_state,s_zip] + InputAdapter + BroadcastExchange #6 + WholeStageCodegen (6) + ColumnarToRow + InputAdapter + CometFilter [i_color,i_item_sk] + CometScan parquet spark_catalog.default.item [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] + InputAdapter + BroadcastExchange #7 + WholeStageCodegen (7) + ColumnarToRow + InputAdapter + CometFilter [c_customer_sk,c_birth_country] + CometScan parquet spark_catalog.default.customer [c_customer_sk,c_first_name,c_last_name,c_birth_country] + InputAdapter + BroadcastExchange #8 + WholeStageCodegen (8) + ColumnarToRow + InputAdapter + CometFilter [ca_country,ca_zip] + CometScan parquet spark_catalog.default.customer_address [ca_state,ca_zip,ca_country] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q25/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q25/explain.txt new file mode 100644 index 000000000..2635546e4 --- /dev/null +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q25/explain.txt @@ -0,0 +1,298 @@ +== Physical Plan == +TakeOrderedAndProject (40) ++- * HashAggregate (39) + +- Exchange (38) + +- * HashAggregate (37) + +- * Project (36) + +- * BroadcastHashJoin Inner BuildRight (35) + :- * Project (30) + : +- * BroadcastHashJoin Inner BuildRight (29) + : :- * Project (24) + : : +- * BroadcastHashJoin Inner BuildRight (23) + : : :- * Project (21) + : : : +- * BroadcastHashJoin Inner BuildRight (20) + : : : :- * Project (18) + : : : : +- * BroadcastHashJoin Inner BuildRight (17) + : : : : :- * Project (15) + : : : : : +- * BroadcastHashJoin Inner BuildRight (14) + : : : : : :- * Project (9) + : : : : : : +- * BroadcastHashJoin Inner BuildRight (8) + : : : : : : :- * ColumnarToRow (3) + : : : : : : : +- CometFilter (2) + : : : : : : : +- CometScan parquet spark_catalog.default.store_sales (1) + : : : : : : +- BroadcastExchange (7) + : : : : : : +- * ColumnarToRow (6) + : : : : : : +- CometFilter (5) + : : : : : : +- CometScan parquet spark_catalog.default.store_returns (4) + : : : : : +- BroadcastExchange (13) + : : : : : +- * ColumnarToRow (12) + : : : : : +- CometFilter (11) + : : : : : +- CometScan parquet spark_catalog.default.catalog_sales (10) + : : : : +- ReusedExchange (16) + : : : +- ReusedExchange (19) + : : +- ReusedExchange (22) + : +- BroadcastExchange (28) + : +- * ColumnarToRow (27) + : +- CometFilter (26) + : +- CometScan parquet spark_catalog.default.store (25) + +- BroadcastExchange (34) + +- * ColumnarToRow (33) + +- CometFilter (32) + +- CometScan parquet spark_catalog.default.item (31) + + +(1) Scan parquet spark_catalog.default.store_sales +Output [6]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_profit#5, ss_sold_date_sk#6] +Batched: true +Location: InMemoryFileIndex [] +PartitionFilters: [isnotnull(ss_sold_date_sk#6), dynamicpruningexpression(ss_sold_date_sk#6 IN dynamicpruning#7)] +PushedFilters: [IsNotNull(ss_customer_sk), IsNotNull(ss_item_sk), IsNotNull(ss_ticket_number), IsNotNull(ss_store_sk)] +ReadSchema: struct<ss_item_sk:int,ss_customer_sk:int,ss_store_sk:int,ss_ticket_number:int,ss_net_profit:decimal(7,2)> + +(2) CometFilter +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)) + +(3) ColumnarToRow [codegen id : 8] +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] + +(4) Scan parquet spark_catalog.default.store_returns +Output [5]: [sr_item_sk#8, sr_customer_sk#9, sr_ticket_number#10, sr_net_loss#11, sr_returned_date_sk#12] +Batched: true +Location: InMemoryFileIndex [] +PartitionFilters: [isnotnull(sr_returned_date_sk#12), dynamicpruningexpression(sr_returned_date_sk#12 IN dynamicpruning#13)] +PushedFilters: [IsNotNull(sr_customer_sk), IsNotNull(sr_item_sk), IsNotNull(sr_ticket_number)] +ReadSchema: struct<sr_item_sk:int,sr_customer_sk:int,sr_ticket_number:int,sr_net_loss:decimal(7,2)> + +(5) CometFilter +Input [5]: [sr_item_sk#8, sr_customer_sk#9, sr_ticket_number#10, sr_net_loss#11, sr_returned_date_sk#12] +Condition : ((isnotnull(sr_customer_sk#9) AND isnotnull(sr_item_sk#8)) AND isnotnull(sr_ticket_number#10)) + +(6) ColumnarToRow [codegen id : 1] +Input [5]: [sr_item_sk#8, sr_customer_sk#9, sr_ticket_number#10, sr_net_loss#11, sr_returned_date_sk#12] + +(7) BroadcastExchange +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] +Left keys [3]: [ss_customer_sk#2, ss_item_sk#1, ss_ticket_number#4] +Right keys [3]: [sr_customer_sk#9, sr_item_sk#8, sr_ticket_number#10] +Join type: Inner +Join condition: None + +(9) Project [codegen id : 8] +Output [8]: [ss_item_sk#1, ss_store_sk#3, ss_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] + +(10) Scan parquet spark_catalog.default.catalog_sales +Output [4]: [cs_bill_customer_sk#14, cs_item_sk#15, cs_net_profit#16, cs_sold_date_sk#17] +Batched: true +Location: InMemoryFileIndex [] +PartitionFilters: [isnotnull(cs_sold_date_sk#17), dynamicpruningexpression(cs_sold_date_sk#17 IN dynamicpruning#18)] +PushedFilters: [IsNotNull(cs_bill_customer_sk), IsNotNull(cs_item_sk)] +ReadSchema: struct<cs_bill_customer_sk:int,cs_item_sk:int,cs_net_profit:decimal(7,2)> + +(11) CometFilter +Input [4]: [cs_bill_customer_sk#14, cs_item_sk#15, cs_net_profit#16, cs_sold_date_sk#17] +Condition : (isnotnull(cs_bill_customer_sk#14) AND isnotnull(cs_item_sk#15)) + +(12) ColumnarToRow [codegen id : 2] +Input [4]: [cs_bill_customer_sk#14, cs_item_sk#15, cs_net_profit#16, cs_sold_date_sk#17] + +(13) BroadcastExchange +Input [4]: [cs_bill_customer_sk#14, cs_item_sk#15, cs_net_profit#16, cs_sold_date_sk#17] +Arguments: HashedRelationBroadcastMode(List((shiftleft(cast(input[0, int, false] as bigint), 32) | (cast(input[1, int, false] as bigint) & 4294967295))),false), [plan_id=2] + +(14) BroadcastHashJoin [codegen id : 8] +Left keys [2]: [sr_customer_sk#9, sr_item_sk#8] +Right keys [2]: [cs_bill_customer_sk#14, cs_item_sk#15] +Join type: Inner +Join condition: None + +(15) Project [codegen id : 8] +Output [8]: [ss_item_sk#1, ss_store_sk#3, ss_net_profit#5, ss_sold_date_sk#6, sr_net_loss#11, sr_returned_date_sk#12, cs_net_profit#16, cs_sold_date_sk#17] +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: 45] +Output [1]: [d_date_sk#19] + +(17) BroadcastHashJoin [codegen id : 8] +Left keys [1]: [ss_sold_date_sk#6] +Right keys [1]: [d_date_sk#19] +Join type: Inner +Join condition: None + +(18) Project [codegen id : 8] +Output [7]: [ss_item_sk#1, ss_store_sk#3, ss_net_profit#5, sr_net_loss#11, sr_returned_date_sk#12, cs_net_profit#16, cs_sold_date_sk#17] +Input [9]: [ss_item_sk#1, ss_store_sk#3, ss_net_profit#5, ss_sold_date_sk#6, sr_net_loss#11, sr_returned_date_sk#12, cs_net_profit#16, cs_sold_date_sk#17, d_date_sk#19] + +(19) ReusedExchange [Reuses operator id: 50] +Output [1]: [d_date_sk#20] + +(20) BroadcastHashJoin [codegen id : 8] +Left keys [1]: [sr_returned_date_sk#12] +Right keys [1]: [d_date_sk#20] +Join type: Inner +Join condition: None + +(21) Project [codegen id : 8] +Output [6]: [ss_item_sk#1, ss_store_sk#3, ss_net_profit#5, sr_net_loss#11, cs_net_profit#16, cs_sold_date_sk#17] +Input [8]: [ss_item_sk#1, ss_store_sk#3, ss_net_profit#5, sr_net_loss#11, sr_returned_date_sk#12, cs_net_profit#16, cs_sold_date_sk#17, d_date_sk#20] + +(22) ReusedExchange [Reuses operator id: 50] +Output [1]: [d_date_sk#21] + +(23) BroadcastHashJoin [codegen id : 8] +Left keys [1]: [cs_sold_date_sk#17] +Right keys [1]: [d_date_sk#21] +Join type: Inner +Join condition: None + +(24) Project [codegen id : 8] +Output [5]: [ss_item_sk#1, ss_store_sk#3, ss_net_profit#5, sr_net_loss#11, cs_net_profit#16] +Input [7]: [ss_item_sk#1, ss_store_sk#3, ss_net_profit#5, sr_net_loss#11, cs_net_profit#16, cs_sold_date_sk#17, d_date_sk#21] + +(25) Scan parquet spark_catalog.default.store +Output [3]: [s_store_sk#22, s_store_id#23, s_store_name#24] +Batched: true +Location [not included in comparison]/{warehouse_dir}/store] +PushedFilters: [IsNotNull(s_store_sk)] +ReadSchema: struct<s_store_sk:int,s_store_id:string,s_store_name:string> + +(26) CometFilter +Input [3]: [s_store_sk#22, s_store_id#23, s_store_name#24] +Condition : isnotnull(s_store_sk#22) + +(27) ColumnarToRow [codegen id : 6] +Input [3]: [s_store_sk#22, s_store_id#23, s_store_name#24] + +(28) BroadcastExchange +Input [3]: [s_store_sk#22, s_store_id#23, s_store_name#24] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=3] + +(29) BroadcastHashJoin [codegen id : 8] +Left keys [1]: [ss_store_sk#3] +Right keys [1]: [s_store_sk#22] +Join type: Inner +Join condition: None + +(30) Project [codegen id : 8] +Output [6]: [ss_item_sk#1, ss_net_profit#5, sr_net_loss#11, cs_net_profit#16, s_store_id#23, s_store_name#24] +Input [8]: [ss_item_sk#1, ss_store_sk#3, ss_net_profit#5, sr_net_loss#11, cs_net_profit#16, s_store_sk#22, s_store_id#23, s_store_name#24] + +(31) Scan parquet spark_catalog.default.item +Output [3]: [i_item_sk#25, i_item_id#26, i_item_desc#27] +Batched: true +Location [not included in comparison]/{warehouse_dir}/item] +PushedFilters: [IsNotNull(i_item_sk)] +ReadSchema: struct<i_item_sk:int,i_item_id:string,i_item_desc:string> + +(32) CometFilter +Input [3]: [i_item_sk#25, i_item_id#26, i_item_desc#27] +Condition : isnotnull(i_item_sk#25) + +(33) ColumnarToRow [codegen id : 7] +Input [3]: [i_item_sk#25, i_item_id#26, i_item_desc#27] + +(34) BroadcastExchange +Input [3]: [i_item_sk#25, i_item_id#26, i_item_desc#27] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=4] + +(35) BroadcastHashJoin [codegen id : 8] +Left keys [1]: [ss_item_sk#1] +Right keys [1]: [i_item_sk#25] +Join type: Inner +Join condition: None + +(36) Project [codegen id : 8] +Output [7]: [ss_net_profit#5, sr_net_loss#11, cs_net_profit#16, s_store_id#23, s_store_name#24, i_item_id#26, i_item_desc#27] +Input [9]: [ss_item_sk#1, ss_net_profit#5, sr_net_loss#11, cs_net_profit#16, s_store_id#23, s_store_name#24, i_item_sk#25, i_item_id#26, i_item_desc#27] + +(37) HashAggregate [codegen id : 8] +Input [7]: [ss_net_profit#5, sr_net_loss#11, cs_net_profit#16, s_store_id#23, s_store_name#24, i_item_id#26, i_item_desc#27] +Keys [4]: [i_item_id#26, i_item_desc#27, s_store_id#23, s_store_name#24] +Functions [3]: [partial_sum(UnscaledValue(ss_net_profit#5)), partial_sum(UnscaledValue(sr_net_loss#11)), partial_sum(UnscaledValue(cs_net_profit#16))] +Aggregate Attributes [3]: [sum#28, sum#29, sum#30] +Results [7]: [i_item_id#26, i_item_desc#27, s_store_id#23, s_store_name#24, sum#31, sum#32, sum#33] + +(38) Exchange +Input [7]: [i_item_id#26, i_item_desc#27, s_store_id#23, s_store_name#24, sum#31, sum#32, sum#33] +Arguments: hashpartitioning(i_item_id#26, i_item_desc#27, s_store_id#23, s_store_name#24, 5), ENSURE_REQUIREMENTS, [plan_id=5] + +(39) HashAggregate [codegen id : 9] +Input [7]: [i_item_id#26, i_item_desc#27, s_store_id#23, s_store_name#24, sum#31, sum#32, sum#33] +Keys [4]: [i_item_id#26, i_item_desc#27, s_store_id#23, s_store_name#24] +Functions [3]: [sum(UnscaledValue(ss_net_profit#5)), sum(UnscaledValue(sr_net_loss#11)), sum(UnscaledValue(cs_net_profit#16))] +Aggregate Attributes [3]: [sum(UnscaledValue(ss_net_profit#5))#34, sum(UnscaledValue(sr_net_loss#11))#35, sum(UnscaledValue(cs_net_profit#16))#36] +Results [7]: [i_item_id#26, i_item_desc#27, s_store_id#23, s_store_name#24, MakeDecimal(sum(UnscaledValue(ss_net_profit#5))#34,17,2) AS store_sales_profit#37, MakeDecimal(sum(UnscaledValue(sr_net_loss#11))#35,17,2) AS store_returns_loss#38, MakeDecimal(sum(UnscaledValue(cs_net_profit#16))#36,17,2) AS catalog_sales_profit#39] + +(40) TakeOrderedAndProject +Input [7]: [i_item_id#26, i_item_desc#27, s_store_id#23, s_store_name#24, store_sales_profit#37, store_returns_loss#38, catalog_sales_profit#39] +Arguments: 100, [i_item_id#26 ASC NULLS FIRST, i_item_desc#27 ASC NULLS FIRST, s_store_id#23 ASC NULLS FIRST, s_store_name#24 ASC NULLS FIRST], [i_item_id#26, i_item_desc#27, s_store_id#23, s_store_name#24, store_sales_profit#37, store_returns_loss#38, catalog_sales_profit#39] + +===== Subqueries ===== + +Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#6 IN dynamicpruning#7 +BroadcastExchange (45) ++- * ColumnarToRow (44) + +- CometProject (43) + +- CometFilter (42) + +- CometScan parquet spark_catalog.default.date_dim (41) + + +(41) Scan parquet spark_catalog.default.date_dim +Output [3]: [d_date_sk#19, d_year#40, d_moy#41] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_moy), IsNotNull(d_year), EqualTo(d_moy,4), EqualTo(d_year,2001), IsNotNull(d_date_sk)] +ReadSchema: struct<d_date_sk:int,d_year:int,d_moy:int> + +(42) CometFilter +Input [3]: [d_date_sk#19, d_year#40, d_moy#41] +Condition : ((((isnotnull(d_moy#41) AND isnotnull(d_year#40)) AND (d_moy#41 = 4)) AND (d_year#40 = 2001)) AND isnotnull(d_date_sk#19)) + +(43) CometProject +Input [3]: [d_date_sk#19, d_year#40, d_moy#41] +Arguments: [d_date_sk#19], [d_date_sk#19] + +(44) ColumnarToRow [codegen id : 1] +Input [1]: [d_date_sk#19] + +(45) BroadcastExchange +Input [1]: [d_date_sk#19] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=6] + +Subquery:2 Hosting operator id = 4 Hosting Expression = sr_returned_date_sk#12 IN dynamicpruning#13 +BroadcastExchange (50) ++- * ColumnarToRow (49) + +- CometProject (48) + +- CometFilter (47) + +- CometScan parquet spark_catalog.default.date_dim (46) + + +(46) Scan parquet spark_catalog.default.date_dim +Output [3]: [d_date_sk#20, d_year#42, d_moy#43] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_moy), IsNotNull(d_year), GreaterThanOrEqual(d_moy,4), LessThanOrEqual(d_moy,10), EqualTo(d_year,2001), IsNotNull(d_date_sk)] +ReadSchema: struct<d_date_sk:int,d_year:int,d_moy:int> + +(47) CometFilter +Input [3]: [d_date_sk#20, d_year#42, d_moy#43] +Condition : (((((isnotnull(d_moy#43) AND isnotnull(d_year#42)) AND (d_moy#43 >= 4)) AND (d_moy#43 <= 10)) AND (d_year#42 = 2001)) AND isnotnull(d_date_sk#20)) + +(48) CometProject +Input [3]: [d_date_sk#20, d_year#42, d_moy#43] +Arguments: [d_date_sk#20], [d_date_sk#20] + +(49) ColumnarToRow [codegen id : 1] +Input [1]: [d_date_sk#20] + +(50) BroadcastExchange +Input [1]: [d_date_sk#20] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=7] + +Subquery:3 Hosting operator id = 10 Hosting Expression = cs_sold_date_sk#17 IN dynamicpruning#13 + + diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q25/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q25/simplified.txt new file mode 100644 index 000000000..eda7f6b64 --- /dev/null +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q25/simplified.txt @@ -0,0 +1,76 @@ +TakeOrderedAndProject [i_item_id,i_item_desc,s_store_id,s_store_name,store_sales_profit,store_returns_loss,catalog_sales_profit] + WholeStageCodegen (9) + HashAggregate [i_item_id,i_item_desc,s_store_id,s_store_name,sum,sum,sum] [sum(UnscaledValue(ss_net_profit)),sum(UnscaledValue(sr_net_loss)),sum(UnscaledValue(cs_net_profit)),store_sales_profit,store_returns_loss,catalog_sales_profit,sum,sum,sum] + InputAdapter + Exchange [i_item_id,i_item_desc,s_store_id,s_store_name] #1 + WholeStageCodegen (8) + HashAggregate [i_item_id,i_item_desc,s_store_id,s_store_name,ss_net_profit,sr_net_loss,cs_net_profit] [sum,sum,sum,sum,sum,sum] + Project [ss_net_profit,sr_net_loss,cs_net_profit,s_store_id,s_store_name,i_item_id,i_item_desc] + BroadcastHashJoin [ss_item_sk,i_item_sk] + Project [ss_item_sk,ss_net_profit,sr_net_loss,cs_net_profit,s_store_id,s_store_name] + BroadcastHashJoin [ss_store_sk,s_store_sk] + Project [ss_item_sk,ss_store_sk,ss_net_profit,sr_net_loss,cs_net_profit] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + Project [ss_item_sk,ss_store_sk,ss_net_profit,sr_net_loss,cs_net_profit,cs_sold_date_sk] + BroadcastHashJoin [sr_returned_date_sk,d_date_sk] + Project [ss_item_sk,ss_store_sk,ss_net_profit,sr_net_loss,sr_returned_date_sk,cs_net_profit,cs_sold_date_sk] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [ss_item_sk,ss_store_sk,ss_net_profit,ss_sold_date_sk,sr_net_loss,sr_returned_date_sk,cs_net_profit,cs_sold_date_sk] + BroadcastHashJoin [sr_customer_sk,sr_item_sk,cs_bill_customer_sk,cs_item_sk] + Project [ss_item_sk,ss_store_sk,ss_net_profit,ss_sold_date_sk,sr_item_sk,sr_customer_sk,sr_net_loss,sr_returned_date_sk] + BroadcastHashJoin [ss_customer_sk,ss_item_sk,ss_ticket_number,sr_customer_sk,sr_item_sk,sr_ticket_number] + ColumnarToRow + InputAdapter + CometFilter [ss_customer_sk,ss_item_sk,ss_ticket_number,ss_store_sk] + CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_profit,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #2 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_moy,d_year,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + InputAdapter + BroadcastExchange #3 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometFilter [sr_customer_sk,sr_item_sk,sr_ticket_number] + CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_customer_sk,sr_ticket_number,sr_net_loss,sr_returned_date_sk] + SubqueryBroadcast [d_date_sk] #2 + BroadcastExchange #4 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_moy,d_year,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + InputAdapter + BroadcastExchange #5 + WholeStageCodegen (2) + ColumnarToRow + InputAdapter + CometFilter [cs_bill_customer_sk,cs_item_sk] + CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_item_sk,cs_net_profit,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #2 + InputAdapter + ReusedExchange [d_date_sk] #2 + InputAdapter + ReusedExchange [d_date_sk] #4 + InputAdapter + ReusedExchange [d_date_sk] #4 + InputAdapter + BroadcastExchange #6 + WholeStageCodegen (6) + ColumnarToRow + InputAdapter + CometFilter [s_store_sk] + CometScan parquet spark_catalog.default.store [s_store_sk,s_store_id,s_store_name] + InputAdapter + BroadcastExchange #7 + WholeStageCodegen (7) + ColumnarToRow + InputAdapter + CometFilter [i_item_sk] + CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc] 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 new file mode 100644 index 000000000..c90dcd024 --- /dev/null +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q26/explain.txt @@ -0,0 +1,208 @@ +== Physical Plan == +TakeOrderedAndProject (30) ++- * HashAggregate (29) + +- Exchange (28) + +- * HashAggregate (27) + +- * Project (26) + +- * BroadcastHashJoin Inner BuildRight (25) + :- * Project (19) + : +- * BroadcastHashJoin Inner BuildRight (18) + : :- * Project (13) + : : +- * BroadcastHashJoin Inner BuildRight (12) + : : :- * Project (10) + : : : +- * BroadcastHashJoin Inner BuildRight (9) + : : : :- * ColumnarToRow (3) + : : : : +- CometFilter (2) + : : : : +- CometScan parquet spark_catalog.default.catalog_sales (1) + : : : +- BroadcastExchange (8) + : : : +- * ColumnarToRow (7) + : : : +- CometProject (6) + : : : +- CometFilter (5) + : : : +- CometScan parquet spark_catalog.default.customer_demographics (4) + : : +- ReusedExchange (11) + : +- BroadcastExchange (17) + : +- * ColumnarToRow (16) + : +- CometFilter (15) + : +- CometScan parquet spark_catalog.default.item (14) + +- BroadcastExchange (24) + +- * ColumnarToRow (23) + +- CometProject (22) + +- CometFilter (21) + +- CometScan parquet spark_catalog.default.promotion (20) + + +(1) Scan parquet spark_catalog.default.catalog_sales +Output [8]: [cs_bill_cdemo_sk#1, cs_item_sk#2, cs_promo_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_sold_date_sk#8] +Batched: true +Location: InMemoryFileIndex [] +PartitionFilters: [isnotnull(cs_sold_date_sk#8), dynamicpruningexpression(cs_sold_date_sk#8 IN dynamicpruning#9)] +PushedFilters: [IsNotNull(cs_bill_cdemo_sk), IsNotNull(cs_item_sk), IsNotNull(cs_promo_sk)] +ReadSchema: struct<cs_bill_cdemo_sk:int,cs_item_sk:int,cs_promo_sk:int,cs_quantity:int,cs_list_price:decimal(7,2),cs_sales_price:decimal(7,2),cs_coupon_amt:decimal(7,2)> + +(2) CometFilter +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)) + +(3) ColumnarToRow [codegen id : 5] +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] + +(4) Scan parquet spark_catalog.default.customer_demographics +Output [4]: [cd_demo_sk#10, cd_gender#11, cd_marital_status#12, cd_education_status#13] +Batched: true +Location [not included in comparison]/{warehouse_dir}/customer_demographics] +PushedFilters: [IsNotNull(cd_gender), IsNotNull(cd_marital_status), IsNotNull(cd_education_status), EqualTo(cd_gender,M), EqualTo(cd_marital_status,S), EqualTo(cd_education_status,College ), IsNotNull(cd_demo_sk)] +ReadSchema: struct<cd_demo_sk:int,cd_gender:string,cd_marital_status:string,cd_education_status:string> + +(5) CometFilter +Input [4]: [cd_demo_sk#10, cd_gender#11, cd_marital_status#12, cd_education_status#13] +Condition : ((((((isnotnull(cd_gender#11) AND isnotnull(cd_marital_status#12)) AND isnotnull(cd_education_status#13)) AND (cd_gender#11 = M)) AND (cd_marital_status#12 = S)) AND (cd_education_status#13 = College )) AND isnotnull(cd_demo_sk#10)) + +(6) CometProject +Input [4]: [cd_demo_sk#10, cd_gender#11, cd_marital_status#12, cd_education_status#13] +Arguments: [cd_demo_sk#10], [cd_demo_sk#10] + +(7) ColumnarToRow [codegen id : 1] +Input [1]: [cd_demo_sk#10] + +(8) BroadcastExchange +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] +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] +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: 35] +Output [1]: [d_date_sk#14] + +(12) BroadcastHashJoin [codegen id : 5] +Left keys [1]: [cs_sold_date_sk#8] +Right keys [1]: [d_date_sk#14] +Join type: Inner +Join condition: None + +(13) Project [codegen id : 5] +Output [6]: [cs_item_sk#2, cs_promo_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7] +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] + +(14) Scan parquet spark_catalog.default.item +Output [2]: [i_item_sk#15, i_item_id#16] +Batched: true +Location [not included in comparison]/{warehouse_dir}/item] +PushedFilters: [IsNotNull(i_item_sk)] +ReadSchema: struct<i_item_sk:int,i_item_id:string> + +(15) CometFilter +Input [2]: [i_item_sk#15, i_item_id#16] +Condition : isnotnull(i_item_sk#15) + +(16) ColumnarToRow [codegen id : 3] +Input [2]: [i_item_sk#15, i_item_id#16] + +(17) BroadcastExchange +Input [2]: [i_item_sk#15, i_item_id#16] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=2] + +(18) BroadcastHashJoin [codegen id : 5] +Left keys [1]: [cs_item_sk#2] +Right keys [1]: [i_item_sk#15] +Join type: Inner +Join condition: None + +(19) Project [codegen id : 5] +Output [6]: [cs_promo_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, i_item_id#16] +Input [8]: [cs_item_sk#2, cs_promo_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, i_item_sk#15, i_item_id#16] + +(20) Scan parquet spark_catalog.default.promotion +Output [3]: [p_promo_sk#17, p_channel_email#18, p_channel_event#19] +Batched: true +Location [not included in comparison]/{warehouse_dir}/promotion] +PushedFilters: [Or(EqualTo(p_channel_email,N),EqualTo(p_channel_event,N)), IsNotNull(p_promo_sk)] +ReadSchema: struct<p_promo_sk:int,p_channel_email:string,p_channel_event:string> + +(21) CometFilter +Input [3]: [p_promo_sk#17, p_channel_email#18, p_channel_event#19] +Condition : (((p_channel_email#18 = N) OR (p_channel_event#19 = N)) AND isnotnull(p_promo_sk#17)) + +(22) CometProject +Input [3]: [p_promo_sk#17, p_channel_email#18, p_channel_event#19] +Arguments: [p_promo_sk#17], [p_promo_sk#17] + +(23) ColumnarToRow [codegen id : 4] +Input [1]: [p_promo_sk#17] + +(24) BroadcastExchange +Input [1]: [p_promo_sk#17] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] + +(25) BroadcastHashJoin [codegen id : 5] +Left keys [1]: [cs_promo_sk#3] +Right keys [1]: [p_promo_sk#17] +Join type: Inner +Join condition: None + +(26) Project [codegen id : 5] +Output [5]: [cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, i_item_id#16] +Input [7]: [cs_promo_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, i_item_id#16, p_promo_sk#17] + +(27) HashAggregate [codegen id : 5] +Input [5]: [cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, i_item_id#16] +Keys [1]: [i_item_id#16] +Functions [4]: [partial_avg(cs_quantity#4), partial_avg(UnscaledValue(cs_list_price#5)), partial_avg(UnscaledValue(cs_coupon_amt#7)), partial_avg(UnscaledValue(cs_sales_price#6))] +Aggregate Attributes [8]: [sum#20, count#21, sum#22, count#23, sum#24, count#25, sum#26, count#27] +Results [9]: [i_item_id#16, sum#28, count#29, sum#30, count#31, sum#32, count#33, sum#34, count#35] + +(28) Exchange +Input [9]: [i_item_id#16, sum#28, count#29, sum#30, count#31, sum#32, count#33, sum#34, count#35] +Arguments: hashpartitioning(i_item_id#16, 5), ENSURE_REQUIREMENTS, [plan_id=4] + +(29) HashAggregate [codegen id : 6] +Input [9]: [i_item_id#16, sum#28, count#29, sum#30, count#31, sum#32, count#33, sum#34, count#35] +Keys [1]: [i_item_id#16] +Functions [4]: [avg(cs_quantity#4), avg(UnscaledValue(cs_list_price#5)), avg(UnscaledValue(cs_coupon_amt#7)), avg(UnscaledValue(cs_sales_price#6))] +Aggregate Attributes [4]: [avg(cs_quantity#4)#36, avg(UnscaledValue(cs_list_price#5))#37, avg(UnscaledValue(cs_coupon_amt#7))#38, avg(UnscaledValue(cs_sales_price#6))#39] +Results [5]: [i_item_id#16, avg(cs_quantity#4)#36 AS agg1#40, cast((avg(UnscaledValue(cs_list_price#5))#37 / 100.0) as decimal(11,6)) AS agg2#41, cast((avg(UnscaledValue(cs_coupon_amt#7))#38 / 100.0) as decimal(11,6)) AS agg3#42, cast((avg(UnscaledValue(cs_sales_price#6))#39 / 100.0) as decimal(11,6)) AS agg4#43] + +(30) TakeOrderedAndProject +Input [5]: [i_item_id#16, agg1#40, agg2#41, agg3#42, agg4#43] +Arguments: 100, [i_item_id#16 ASC NULLS FIRST], [i_item_id#16, agg1#40, agg2#41, agg3#42, agg4#43] + +===== Subqueries ===== + +Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#8 IN dynamicpruning#9 +BroadcastExchange (35) ++- * ColumnarToRow (34) + +- CometProject (33) + +- CometFilter (32) + +- CometScan parquet spark_catalog.default.date_dim (31) + + +(31) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#14, d_year#44] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2000), IsNotNull(d_date_sk)] +ReadSchema: struct<d_date_sk:int,d_year:int> + +(32) CometFilter +Input [2]: [d_date_sk#14, d_year#44] +Condition : ((isnotnull(d_year#44) AND (d_year#44 = 2000)) AND isnotnull(d_date_sk#14)) + +(33) CometProject +Input [2]: [d_date_sk#14, d_year#44] +Arguments: [d_date_sk#14], [d_date_sk#14] + +(34) ColumnarToRow [codegen id : 1] +Input [1]: [d_date_sk#14] + +(35) BroadcastExchange +Input [1]: [d_date_sk#14] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] + + 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 new file mode 100644 index 000000000..7d3893624 --- /dev/null +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q26/simplified.txt @@ -0,0 +1,52 @@ +TakeOrderedAndProject [i_item_id,agg1,agg2,agg3,agg4] + WholeStageCodegen (6) + HashAggregate [i_item_id,sum,count,sum,count,sum,count,sum,count] [avg(cs_quantity),avg(UnscaledValue(cs_list_price)),avg(UnscaledValue(cs_coupon_amt)),avg(UnscaledValue(cs_sales_price)),agg1,agg2,agg3,agg4,sum,count,sum,count,sum,count,sum,count] + InputAdapter + Exchange [i_item_id] #1 + WholeStageCodegen (5) + HashAggregate [i_item_id,cs_quantity,cs_list_price,cs_coupon_amt,cs_sales_price] [sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] + Project [cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,i_item_id] + BroadcastHashJoin [cs_promo_sk,p_promo_sk] + Project [cs_promo_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,i_item_id] + BroadcastHashJoin [cs_item_sk,i_item_sk] + Project [cs_item_sk,cs_promo_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + Project [cs_item_sk,cs_promo_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_sold_date_sk] + BroadcastHashJoin [cs_bill_cdemo_sk,cd_demo_sk] + ColumnarToRow + InputAdapter + CometFilter [cs_bill_cdemo_sk,cs_item_sk,cs_promo_sk] + CometScan parquet spark_catalog.default.catalog_sales [cs_bill_cdemo_sk,cs_item_sk,cs_promo_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #2 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_year,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + InputAdapter + BroadcastExchange #3 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [cd_demo_sk] + CometFilter [cd_gender,cd_marital_status,cd_education_status,cd_demo_sk] + CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status] + InputAdapter + ReusedExchange [d_date_sk] #2 + InputAdapter + BroadcastExchange #4 + WholeStageCodegen (3) + ColumnarToRow + InputAdapter + CometFilter [i_item_sk] + CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id] + InputAdapter + BroadcastExchange #5 + WholeStageCodegen (4) + ColumnarToRow + InputAdapter + CometProject [p_promo_sk] + CometFilter [p_channel_email,p_channel_event,p_promo_sk] + CometScan parquet spark_catalog.default.promotion [p_promo_sk,p_channel_email,p_channel_event] 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 new file mode 100644 index 000000000..e41077ed3 --- /dev/null +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q27/explain.txt @@ -0,0 +1,208 @@ +== Physical Plan == +TakeOrderedAndProject (30) ++- * HashAggregate (29) + +- Exchange (28) + +- * HashAggregate (27) + +- * Expand (26) + +- * Project (25) + +- * BroadcastHashJoin Inner BuildRight (24) + :- * Project (19) + : +- * BroadcastHashJoin Inner BuildRight (18) + : :- * Project (13) + : : +- * BroadcastHashJoin Inner BuildRight (12) + : : :- * Project (10) + : : : +- * BroadcastHashJoin Inner BuildRight (9) + : : : :- * ColumnarToRow (3) + : : : : +- CometFilter (2) + : : : : +- CometScan parquet spark_catalog.default.store_sales (1) + : : : +- BroadcastExchange (8) + : : : +- * ColumnarToRow (7) + : : : +- CometProject (6) + : : : +- CometFilter (5) + : : : +- CometScan parquet spark_catalog.default.customer_demographics (4) + : : +- ReusedExchange (11) + : +- BroadcastExchange (17) + : +- * ColumnarToRow (16) + : +- CometFilter (15) + : +- CometScan parquet spark_catalog.default.store (14) + +- BroadcastExchange (23) + +- * ColumnarToRow (22) + +- CometFilter (21) + +- CometScan parquet spark_catalog.default.item (20) + + +(1) Scan parquet spark_catalog.default.store_sales +Output [8]: [ss_item_sk#1, ss_cdemo_sk#2, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8] +Batched: true +Location: InMemoryFileIndex [] +PartitionFilters: [isnotnull(ss_sold_date_sk#8), dynamicpruningexpression(ss_sold_date_sk#8 IN dynamicpruning#9)] +PushedFilters: [IsNotNull(ss_cdemo_sk), IsNotNull(ss_store_sk), IsNotNull(ss_item_sk)] +ReadSchema: struct<ss_item_sk:int,ss_cdemo_sk:int,ss_store_sk:int,ss_quantity:int,ss_list_price:decimal(7,2),ss_sales_price:decimal(7,2),ss_coupon_amt:decimal(7,2)> + +(2) CometFilter +Input [8]: [ss_item_sk#1, ss_cdemo_sk#2, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8] +Condition : ((isnotnull(ss_cdemo_sk#2) AND isnotnull(ss_store_sk#3)) AND isnotnull(ss_item_sk#1)) + +(3) ColumnarToRow [codegen id : 5] +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] + +(4) Scan parquet spark_catalog.default.customer_demographics +Output [4]: [cd_demo_sk#10, cd_gender#11, cd_marital_status#12, cd_education_status#13] +Batched: true +Location [not included in comparison]/{warehouse_dir}/customer_demographics] +PushedFilters: [IsNotNull(cd_gender), IsNotNull(cd_marital_status), IsNotNull(cd_education_status), EqualTo(cd_gender,M), EqualTo(cd_marital_status,S), EqualTo(cd_education_status,College ), IsNotNull(cd_demo_sk)] +ReadSchema: struct<cd_demo_sk:int,cd_gender:string,cd_marital_status:string,cd_education_status:string> + +(5) CometFilter +Input [4]: [cd_demo_sk#10, cd_gender#11, cd_marital_status#12, cd_education_status#13] +Condition : ((((((isnotnull(cd_gender#11) AND isnotnull(cd_marital_status#12)) AND isnotnull(cd_education_status#13)) AND (cd_gender#11 = M)) AND (cd_marital_status#12 = S)) AND (cd_education_status#13 = College )) AND isnotnull(cd_demo_sk#10)) + +(6) CometProject +Input [4]: [cd_demo_sk#10, cd_gender#11, cd_marital_status#12, cd_education_status#13] +Arguments: [cd_demo_sk#10], [cd_demo_sk#10] + +(7) ColumnarToRow [codegen id : 1] +Input [1]: [cd_demo_sk#10] + +(8) BroadcastExchange +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] +Left keys [1]: [ss_cdemo_sk#2] +Right keys [1]: [cd_demo_sk#10] +Join type: Inner +Join condition: None + +(10) Project [codegen id : 5] +Output [7]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8] +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: 35] +Output [1]: [d_date_sk#14] + +(12) BroadcastHashJoin [codegen id : 5] +Left keys [1]: [ss_sold_date_sk#8] +Right keys [1]: [d_date_sk#14] +Join type: Inner +Join condition: None + +(13) Project [codegen id : 5] +Output [6]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7] +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] + +(14) Scan parquet spark_catalog.default.store +Output [2]: [s_store_sk#15, s_state#16] +Batched: true +Location [not included in comparison]/{warehouse_dir}/store] +PushedFilters: [IsNotNull(s_state), EqualTo(s_state,TN), IsNotNull(s_store_sk)] +ReadSchema: struct<s_store_sk:int,s_state:string> + +(15) CometFilter +Input [2]: [s_store_sk#15, s_state#16] +Condition : ((isnotnull(s_state#16) AND (s_state#16 = TN)) AND isnotnull(s_store_sk#15)) + +(16) ColumnarToRow [codegen id : 3] +Input [2]: [s_store_sk#15, s_state#16] + +(17) BroadcastExchange +Input [2]: [s_store_sk#15, s_state#16] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=2] + +(18) BroadcastHashJoin [codegen id : 5] +Left keys [1]: [ss_store_sk#3] +Right keys [1]: [s_store_sk#15] +Join type: Inner +Join condition: None + +(19) Project [codegen id : 5] +Output [6]: [ss_item_sk#1, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, s_state#16] +Input [8]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, s_store_sk#15, s_state#16] + +(20) Scan parquet spark_catalog.default.item +Output [2]: [i_item_sk#17, i_item_id#18] +Batched: true +Location [not included in comparison]/{warehouse_dir}/item] +PushedFilters: [IsNotNull(i_item_sk)] +ReadSchema: struct<i_item_sk:int,i_item_id:string> + +(21) CometFilter +Input [2]: [i_item_sk#17, i_item_id#18] +Condition : isnotnull(i_item_sk#17) + +(22) ColumnarToRow [codegen id : 4] +Input [2]: [i_item_sk#17, i_item_id#18] + +(23) BroadcastExchange +Input [2]: [i_item_sk#17, i_item_id#18] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=3] + +(24) BroadcastHashJoin [codegen id : 5] +Left keys [1]: [ss_item_sk#1] +Right keys [1]: [i_item_sk#17] +Join type: Inner +Join condition: None + +(25) Project [codegen id : 5] +Output [6]: [ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, i_item_id#18, s_state#16] +Input [8]: [ss_item_sk#1, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, s_state#16, i_item_sk#17, i_item_id#18] + +(26) Expand [codegen id : 5] +Input [6]: [ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, i_item_id#18, s_state#16] +Arguments: [[ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, i_item_id#18, s_state#16, 0], [ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, i_item_id#18, null, 1], [ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, null, null, 3]], [ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, i_item_id#19, s_state#20, spark_grouping_id#21] + +(27) HashAggregate [codegen id : 5] +Input [7]: [ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, i_item_id#19, s_state#20, spark_grouping_id#21] +Keys [3]: [i_item_id#19, s_state#20, spark_grouping_id#21] +Functions [4]: [partial_avg(ss_quantity#4), partial_avg(UnscaledValue(ss_list_price#5)), partial_avg(UnscaledValue(ss_coupon_amt#7)), partial_avg(UnscaledValue(ss_sales_price#6))] +Aggregate Attributes [8]: [sum#22, count#23, sum#24, count#25, sum#26, count#27, sum#28, count#29] +Results [11]: [i_item_id#19, s_state#20, spark_grouping_id#21, sum#30, count#31, sum#32, count#33, sum#34, count#35, sum#36, count#37] + +(28) Exchange +Input [11]: [i_item_id#19, s_state#20, spark_grouping_id#21, sum#30, count#31, sum#32, count#33, sum#34, count#35, sum#36, count#37] +Arguments: hashpartitioning(i_item_id#19, s_state#20, spark_grouping_id#21, 5), ENSURE_REQUIREMENTS, [plan_id=4] + +(29) HashAggregate [codegen id : 6] +Input [11]: [i_item_id#19, s_state#20, spark_grouping_id#21, sum#30, count#31, sum#32, count#33, sum#34, count#35, sum#36, count#37] +Keys [3]: [i_item_id#19, s_state#20, spark_grouping_id#21] +Functions [4]: [avg(ss_quantity#4), avg(UnscaledValue(ss_list_price#5)), avg(UnscaledValue(ss_coupon_amt#7)), avg(UnscaledValue(ss_sales_price#6))] +Aggregate Attributes [4]: [avg(ss_quantity#4)#38, avg(UnscaledValue(ss_list_price#5))#39, avg(UnscaledValue(ss_coupon_amt#7))#40, avg(UnscaledValue(ss_sales_price#6))#41] +Results [7]: [i_item_id#19, s_state#20, cast((shiftright(spark_grouping_id#21, 0) & 1) as tinyint) AS g_state#42, avg(ss_quantity#4)#38 AS agg1#43, cast((avg(UnscaledValue(ss_list_price#5))#39 / 100.0) as decimal(11,6)) AS agg2#44, cast((avg(UnscaledValue(ss_coupon_amt#7))#40 / 100.0) as decimal(11,6)) AS agg3#45, cast((avg(UnscaledValue(ss_sales_price#6))#41 / 100.0) as decimal(11,6)) AS agg4#46] + +(30) TakeOrderedAndProject +Input [7]: [i_item_id#19, s_state#20, g_state#42, agg1#43, agg2#44, agg3#45, agg4#46] +Arguments: 100, [i_item_id#19 ASC NULLS FIRST, s_state#20 ASC NULLS FIRST], [i_item_id#19, s_state#20, g_state#42, agg1#43, agg2#44, agg3#45, agg4#46] + +===== Subqueries ===== + +Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#8 IN dynamicpruning#9 +BroadcastExchange (35) ++- * ColumnarToRow (34) + +- CometProject (33) + +- CometFilter (32) + +- CometScan parquet spark_catalog.default.date_dim (31) + + +(31) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#14, d_year#47] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2002), IsNotNull(d_date_sk)] +ReadSchema: struct<d_date_sk:int,d_year:int> + +(32) CometFilter +Input [2]: [d_date_sk#14, d_year#47] +Condition : ((isnotnull(d_year#47) AND (d_year#47 = 2002)) AND isnotnull(d_date_sk#14)) + +(33) CometProject +Input [2]: [d_date_sk#14, d_year#47] +Arguments: [d_date_sk#14], [d_date_sk#14] + +(34) ColumnarToRow [codegen id : 1] +Input [1]: [d_date_sk#14] + +(35) BroadcastExchange +Input [1]: [d_date_sk#14] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] + + 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 new file mode 100644 index 000000000..9d073ff67 --- /dev/null +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q27/simplified.txt @@ -0,0 +1,52 @@ +TakeOrderedAndProject [i_item_id,s_state,g_state,agg1,agg2,agg3,agg4] + WholeStageCodegen (6) + HashAggregate [i_item_id,s_state,spark_grouping_id,sum,count,sum,count,sum,count,sum,count] [avg(ss_quantity),avg(UnscaledValue(ss_list_price)),avg(UnscaledValue(ss_coupon_amt)),avg(UnscaledValue(ss_sales_price)),g_state,agg1,agg2,agg3,agg4,sum,count,sum,count,sum,count,sum,count] + InputAdapter + Exchange [i_item_id,s_state,spark_grouping_id] #1 + WholeStageCodegen (5) + HashAggregate [i_item_id,s_state,spark_grouping_id,ss_quantity,ss_list_price,ss_coupon_amt,ss_sales_price] [sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] + Expand [ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,i_item_id,s_state] + Project [ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,i_item_id,s_state] + BroadcastHashJoin [ss_item_sk,i_item_sk] + Project [ss_item_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,s_state] + BroadcastHashJoin [ss_store_sk,s_store_sk] + Project [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk] + BroadcastHashJoin [ss_cdemo_sk,cd_demo_sk] + ColumnarToRow + InputAdapter + CometFilter [ss_cdemo_sk,ss_store_sk,ss_item_sk] + CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_cdemo_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #2 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_year,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + InputAdapter + BroadcastExchange #3 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [cd_demo_sk] + CometFilter [cd_gender,cd_marital_status,cd_education_status,cd_demo_sk] + CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status] + InputAdapter + ReusedExchange [d_date_sk] #2 + InputAdapter + BroadcastExchange #4 + WholeStageCodegen (3) + ColumnarToRow + InputAdapter + CometFilter [s_state,s_store_sk] + CometScan parquet spark_catalog.default.store [s_store_sk,s_state] + InputAdapter + BroadcastExchange #5 + WholeStageCodegen (4) + ColumnarToRow + InputAdapter + CometFilter [i_item_sk] + CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q28/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q28/explain.txt new file mode 100644 index 000000000..de4ab3a2c --- /dev/null +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q28/explain.txt @@ -0,0 +1,437 @@ +== Physical Plan == +* BroadcastNestedLoopJoin Inner BuildRight (70) +:- * BroadcastNestedLoopJoin Inner BuildRight (58) +: :- * BroadcastNestedLoopJoin Inner BuildRight (46) +: : :- * BroadcastNestedLoopJoin Inner BuildRight (34) +: : : :- * BroadcastNestedLoopJoin Inner BuildRight (22) +: : : : :- * HashAggregate (10) +: : : : : +- Exchange (9) +: : : : : +- * HashAggregate (8) +: : : : : +- * HashAggregate (7) +: : : : : +- Exchange (6) +: : : : : +- * HashAggregate (5) +: : : : : +- * ColumnarToRow (4) +: : : : : +- CometProject (3) +: : : : : +- CometFilter (2) +: : : : : +- CometScan parquet spark_catalog.default.store_sales (1) +: : : : +- BroadcastExchange (21) +: : : : +- * HashAggregate (20) +: : : : +- Exchange (19) +: : : : +- * HashAggregate (18) +: : : : +- * HashAggregate (17) +: : : : +- Exchange (16) +: : : : +- * HashAggregate (15) +: : : : +- * ColumnarToRow (14) +: : : : +- CometProject (13) +: : : : +- CometFilter (12) +: : : : +- CometScan parquet spark_catalog.default.store_sales (11) +: : : +- BroadcastExchange (33) +: : : +- * HashAggregate (32) +: : : +- Exchange (31) +: : : +- * HashAggregate (30) +: : : +- * HashAggregate (29) +: : : +- Exchange (28) +: : : +- * HashAggregate (27) +: : : +- * ColumnarToRow (26) +: : : +- CometProject (25) +: : : +- CometFilter (24) +: : : +- CometScan parquet spark_catalog.default.store_sales (23) +: : +- BroadcastExchange (45) +: : +- * HashAggregate (44) +: : +- Exchange (43) +: : +- * HashAggregate (42) +: : +- * HashAggregate (41) +: : +- Exchange (40) +: : +- * HashAggregate (39) +: : +- * ColumnarToRow (38) +: : +- CometProject (37) +: : +- CometFilter (36) +: : +- CometScan parquet spark_catalog.default.store_sales (35) +: +- BroadcastExchange (57) +: +- * HashAggregate (56) +: +- Exchange (55) +: +- * HashAggregate (54) +: +- * HashAggregate (53) +: +- Exchange (52) +: +- * HashAggregate (51) +: +- * ColumnarToRow (50) +: +- CometProject (49) +: +- CometFilter (48) +: +- CometScan parquet spark_catalog.default.store_sales (47) ++- BroadcastExchange (69) + +- * HashAggregate (68) + +- Exchange (67) + +- * HashAggregate (66) + +- * HashAggregate (65) + +- Exchange (64) + +- * HashAggregate (63) + +- * ColumnarToRow (62) + +- CometProject (61) + +- CometFilter (60) + +- CometScan parquet spark_catalog.default.store_sales (59) + + +(1) Scan parquet spark_catalog.default.store_sales +Output [5]: [ss_quantity#1, ss_wholesale_cost#2, ss_list_price#3, ss_coupon_amt#4, ss_sold_date_sk#5] +Batched: true +Location [not included in comparison]/{warehouse_dir}/store_sales] +PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,0), LessThanOrEqual(ss_quantity,5), Or(Or(And(GreaterThanOrEqual(ss_list_price,8.00),LessThanOrEqual(ss_list_price,18.00)),And(GreaterThanOrEqual(ss_coupon_amt,459.00),LessThanOrEqual(ss_coupon_amt,1459.00))),And(GreaterThanOrEqual(ss_wholesale_cost,57.00),LessThanOrEqual(ss_wholesale_cost,77.00)))] +ReadSchema: struct<ss_quantity:int,ss_wholesale_cost:decimal(7,2),ss_list_price:decimal(7,2),ss_coupon_amt:decimal(7,2)> + +(2) CometFilter +Input [5]: [ss_quantity#1, ss_wholesale_cost#2, ss_list_price#3, ss_coupon_amt#4, ss_sold_date_sk#5] +Condition : (((isnotnull(ss_quantity#1) AND (ss_quantity#1 >= 0)) AND (ss_quantity#1 <= 5)) AND ((((ss_list_price#3 >= 8.00) AND (ss_list_price#3 <= 18.00)) OR ((ss_coupon_amt#4 >= 459.00) AND (ss_coupon_amt#4 <= 1459.00))) OR ((ss_wholesale_cost#2 >= 57.00) AND (ss_wholesale_cost#2 <= 77.00)))) + +(3) CometProject +Input [5]: [ss_quantity#1, ss_wholesale_cost#2, ss_list_price#3, ss_coupon_amt#4, ss_sold_date_sk#5] +Arguments: [ss_list_price#3], [ss_list_price#3] + +(4) ColumnarToRow [codegen id : 1] +Input [1]: [ss_list_price#3] + +(5) HashAggregate [codegen id : 1] +Input [1]: [ss_list_price#3] +Keys [1]: [ss_list_price#3] +Functions [2]: [partial_avg(UnscaledValue(ss_list_price#3)), partial_count(ss_list_price#3)] +Aggregate Attributes [2]: [avg(UnscaledValue(ss_list_price#3))#6, count(ss_list_price#3)#7] +Results [4]: [ss_list_price#3, sum#8, count#9, count#10] + +(6) Exchange +Input [4]: [ss_list_price#3, sum#8, count#9, count#10] +Arguments: hashpartitioning(ss_list_price#3, 5), ENSURE_REQUIREMENTS, [plan_id=1] + +(7) HashAggregate [codegen id : 2] +Input [4]: [ss_list_price#3, sum#8, count#9, count#10] +Keys [1]: [ss_list_price#3] +Functions [2]: [merge_avg(UnscaledValue(ss_list_price#3)), merge_count(ss_list_price#3)] +Aggregate Attributes [2]: [avg(UnscaledValue(ss_list_price#3))#6, count(ss_list_price#3)#7] +Results [4]: [ss_list_price#3, sum#8, count#9, count#10] + +(8) HashAggregate [codegen id : 2] +Input [4]: [ss_list_price#3, sum#8, count#9, count#10] +Keys: [] +Functions [3]: [merge_avg(UnscaledValue(ss_list_price#3)), merge_count(ss_list_price#3), partial_count(distinct ss_list_price#3)] +Aggregate Attributes [3]: [avg(UnscaledValue(ss_list_price#3))#6, count(ss_list_price#3)#7, count(ss_list_price#3)#11] +Results [4]: [sum#8, count#9, count#10, count#12] + +(9) Exchange +Input [4]: [sum#8, count#9, count#10, count#12] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=2] + +(10) HashAggregate [codegen id : 18] +Input [4]: [sum#8, count#9, count#10, count#12] +Keys: [] +Functions [3]: [avg(UnscaledValue(ss_list_price#3)), count(ss_list_price#3), count(distinct ss_list_price#3)] +Aggregate Attributes [3]: [avg(UnscaledValue(ss_list_price#3))#6, count(ss_list_price#3)#7, count(ss_list_price#3)#11] +Results [3]: [cast((avg(UnscaledValue(ss_list_price#3))#6 / 100.0) as decimal(11,6)) AS B1_LP#13, count(ss_list_price#3)#7 AS B1_CNT#14, count(ss_list_price#3)#11 AS B1_CNTD#15] + +(11) Scan parquet spark_catalog.default.store_sales +Output [5]: [ss_quantity#16, ss_wholesale_cost#17, ss_list_price#18, ss_coupon_amt#19, ss_sold_date_sk#20] +Batched: true +Location [not included in comparison]/{warehouse_dir}/store_sales] +PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,6), LessThanOrEqual(ss_quantity,10), Or(Or(And(GreaterThanOrEqual(ss_list_price,90.00),LessThanOrEqual(ss_list_price,100.00)),And(GreaterThanOrEqual(ss_coupon_amt,2323.00),LessThanOrEqual(ss_coupon_amt,3323.00))),And(GreaterThanOrEqual(ss_wholesale_cost,31.00),LessThanOrEqual(ss_wholesale_cost,51.00)))] +ReadSchema: struct<ss_quantity:int,ss_wholesale_cost:decimal(7,2),ss_list_price:decimal(7,2),ss_coupon_amt:decimal(7,2)> + +(12) CometFilter +Input [5]: [ss_quantity#16, ss_wholesale_cost#17, ss_list_price#18, ss_coupon_amt#19, ss_sold_date_sk#20] +Condition : (((isnotnull(ss_quantity#16) AND (ss_quantity#16 >= 6)) AND (ss_quantity#16 <= 10)) AND ((((ss_list_price#18 >= 90.00) AND (ss_list_price#18 <= 100.00)) OR ((ss_coupon_amt#19 >= 2323.00) AND (ss_coupon_amt#19 <= 3323.00))) OR ((ss_wholesale_cost#17 >= 31.00) AND (ss_wholesale_cost#17 <= 51.00)))) + +(13) CometProject +Input [5]: [ss_quantity#16, ss_wholesale_cost#17, ss_list_price#18, ss_coupon_amt#19, ss_sold_date_sk#20] +Arguments: [ss_list_price#18], [ss_list_price#18] + +(14) ColumnarToRow [codegen id : 3] +Input [1]: [ss_list_price#18] + +(15) HashAggregate [codegen id : 3] +Input [1]: [ss_list_price#18] +Keys [1]: [ss_list_price#18] +Functions [2]: [partial_avg(UnscaledValue(ss_list_price#18)), partial_count(ss_list_price#18)] +Aggregate Attributes [2]: [avg(UnscaledValue(ss_list_price#18))#21, count(ss_list_price#18)#22] +Results [4]: [ss_list_price#18, sum#23, count#24, count#25] + +(16) Exchange +Input [4]: [ss_list_price#18, sum#23, count#24, count#25] +Arguments: hashpartitioning(ss_list_price#18, 5), ENSURE_REQUIREMENTS, [plan_id=3] + +(17) HashAggregate [codegen id : 4] +Input [4]: [ss_list_price#18, sum#23, count#24, count#25] +Keys [1]: [ss_list_price#18] +Functions [2]: [merge_avg(UnscaledValue(ss_list_price#18)), merge_count(ss_list_price#18)] +Aggregate Attributes [2]: [avg(UnscaledValue(ss_list_price#18))#21, count(ss_list_price#18)#22] +Results [4]: [ss_list_price#18, sum#23, count#24, count#25] + +(18) HashAggregate [codegen id : 4] +Input [4]: [ss_list_price#18, sum#23, count#24, count#25] +Keys: [] +Functions [3]: [merge_avg(UnscaledValue(ss_list_price#18)), merge_count(ss_list_price#18), partial_count(distinct ss_list_price#18)] +Aggregate Attributes [3]: [avg(UnscaledValue(ss_list_price#18))#21, count(ss_list_price#18)#22, count(ss_list_price#18)#26] +Results [4]: [sum#23, count#24, count#25, count#27] + +(19) Exchange +Input [4]: [sum#23, count#24, count#25, count#27] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=4] + +(20) HashAggregate [codegen id : 5] +Input [4]: [sum#23, count#24, count#25, count#27] +Keys: [] +Functions [3]: [avg(UnscaledValue(ss_list_price#18)), count(ss_list_price#18), count(distinct ss_list_price#18)] +Aggregate Attributes [3]: [avg(UnscaledValue(ss_list_price#18))#21, count(ss_list_price#18)#22, count(ss_list_price#18)#26] +Results [3]: [cast((avg(UnscaledValue(ss_list_price#18))#21 / 100.0) as decimal(11,6)) AS B2_LP#28, count(ss_list_price#18)#22 AS B2_CNT#29, count(ss_list_price#18)#26 AS B2_CNTD#30] + +(21) BroadcastExchange +Input [3]: [B2_LP#28, B2_CNT#29, B2_CNTD#30] +Arguments: IdentityBroadcastMode, [plan_id=5] + +(22) BroadcastNestedLoopJoin [codegen id : 18] +Join type: Inner +Join condition: None + +(23) Scan parquet spark_catalog.default.store_sales +Output [5]: [ss_quantity#31, ss_wholesale_cost#32, ss_list_price#33, ss_coupon_amt#34, ss_sold_date_sk#35] +Batched: true +Location [not included in comparison]/{warehouse_dir}/store_sales] +PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,11), LessThanOrEqual(ss_quantity,15), Or(Or(And(GreaterThanOrEqual(ss_list_price,142.00),LessThanOrEqual(ss_list_price,152.00)),And(GreaterThanOrEqual(ss_coupon_amt,12214.00),LessThanOrEqual(ss_coupon_amt,13214.00))),And(GreaterThanOrEqual(ss_wholesale_cost,79.00),LessThanOrEqual(ss_wholesale_cost,99.00)))] +ReadSchema: struct<ss_quantity:int,ss_wholesale_cost:decimal(7,2),ss_list_price:decimal(7,2),ss_coupon_amt:decimal(7,2)> + +(24) CometFilter +Input [5]: [ss_quantity#31, ss_wholesale_cost#32, ss_list_price#33, ss_coupon_amt#34, ss_sold_date_sk#35] +Condition : (((isnotnull(ss_quantity#31) AND (ss_quantity#31 >= 11)) AND (ss_quantity#31 <= 15)) AND ((((ss_list_price#33 >= 142.00) AND (ss_list_price#33 <= 152.00)) OR ((ss_coupon_amt#34 >= 12214.00) AND (ss_coupon_amt#34 <= 13214.00))) OR ((ss_wholesale_cost#32 >= 79.00) AND (ss_wholesale_cost#32 <= 99.00)))) + +(25) CometProject +Input [5]: [ss_quantity#31, ss_wholesale_cost#32, ss_list_price#33, ss_coupon_amt#34, ss_sold_date_sk#35] +Arguments: [ss_list_price#33], [ss_list_price#33] + +(26) ColumnarToRow [codegen id : 6] +Input [1]: [ss_list_price#33] + +(27) HashAggregate [codegen id : 6] +Input [1]: [ss_list_price#33] +Keys [1]: [ss_list_price#33] +Functions [2]: [partial_avg(UnscaledValue(ss_list_price#33)), partial_count(ss_list_price#33)] +Aggregate Attributes [2]: [avg(UnscaledValue(ss_list_price#33))#36, count(ss_list_price#33)#37] +Results [4]: [ss_list_price#33, sum#38, count#39, count#40] + +(28) Exchange +Input [4]: [ss_list_price#33, sum#38, count#39, count#40] +Arguments: hashpartitioning(ss_list_price#33, 5), ENSURE_REQUIREMENTS, [plan_id=6] + +(29) HashAggregate [codegen id : 7] +Input [4]: [ss_list_price#33, sum#38, count#39, count#40] +Keys [1]: [ss_list_price#33] +Functions [2]: [merge_avg(UnscaledValue(ss_list_price#33)), merge_count(ss_list_price#33)] +Aggregate Attributes [2]: [avg(UnscaledValue(ss_list_price#33))#36, count(ss_list_price#33)#37] +Results [4]: [ss_list_price#33, sum#38, count#39, count#40] + +(30) HashAggregate [codegen id : 7] +Input [4]: [ss_list_price#33, sum#38, count#39, count#40] +Keys: [] +Functions [3]: [merge_avg(UnscaledValue(ss_list_price#33)), merge_count(ss_list_price#33), partial_count(distinct ss_list_price#33)] +Aggregate Attributes [3]: [avg(UnscaledValue(ss_list_price#33))#36, count(ss_list_price#33)#37, count(ss_list_price#33)#41] +Results [4]: [sum#38, count#39, count#40, count#42] + +(31) Exchange +Input [4]: [sum#38, count#39, count#40, count#42] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=7] + +(32) HashAggregate [codegen id : 8] +Input [4]: [sum#38, count#39, count#40, count#42] +Keys: [] +Functions [3]: [avg(UnscaledValue(ss_list_price#33)), count(ss_list_price#33), count(distinct ss_list_price#33)] +Aggregate Attributes [3]: [avg(UnscaledValue(ss_list_price#33))#36, count(ss_list_price#33)#37, count(ss_list_price#33)#41] +Results [3]: [cast((avg(UnscaledValue(ss_list_price#33))#36 / 100.0) as decimal(11,6)) AS B3_LP#43, count(ss_list_price#33)#37 AS B3_CNT#44, count(ss_list_price#33)#41 AS B3_CNTD#45] + +(33) BroadcastExchange +Input [3]: [B3_LP#43, B3_CNT#44, B3_CNTD#45] +Arguments: IdentityBroadcastMode, [plan_id=8] + +(34) BroadcastNestedLoopJoin [codegen id : 18] +Join type: Inner +Join condition: None + +(35) Scan parquet spark_catalog.default.store_sales +Output [5]: [ss_quantity#46, ss_wholesale_cost#47, ss_list_price#48, ss_coupon_amt#49, ss_sold_date_sk#50] +Batched: true +Location [not included in comparison]/{warehouse_dir}/store_sales] +PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,16), LessThanOrEqual(ss_quantity,20), Or(Or(And(GreaterThanOrEqual(ss_list_price,135.00),LessThanOrEqual(ss_list_price,145.00)),And(GreaterThanOrEqual(ss_coupon_amt,6071.00),LessThanOrEqual(ss_coupon_amt,7071.00))),And(GreaterThanOrEqual(ss_wholesale_cost,38.00),LessThanOrEqual(ss_wholesale_cost,58.00)))] +ReadSchema: struct<ss_quantity:int,ss_wholesale_cost:decimal(7,2),ss_list_price:decimal(7,2),ss_coupon_amt:decimal(7,2)> + +(36) CometFilter +Input [5]: [ss_quantity#46, ss_wholesale_cost#47, ss_list_price#48, ss_coupon_amt#49, ss_sold_date_sk#50] +Condition : (((isnotnull(ss_quantity#46) AND (ss_quantity#46 >= 16)) AND (ss_quantity#46 <= 20)) AND ((((ss_list_price#48 >= 135.00) AND (ss_list_price#48 <= 145.00)) OR ((ss_coupon_amt#49 >= 6071.00) AND (ss_coupon_amt#49 <= 7071.00))) OR ((ss_wholesale_cost#47 >= 38.00) AND (ss_wholesale_cost#47 <= 58.00)))) + +(37) CometProject +Input [5]: [ss_quantity#46, ss_wholesale_cost#47, ss_list_price#48, ss_coupon_amt#49, ss_sold_date_sk#50] +Arguments: [ss_list_price#48], [ss_list_price#48] + +(38) ColumnarToRow [codegen id : 9] +Input [1]: [ss_list_price#48] + +(39) HashAggregate [codegen id : 9] +Input [1]: [ss_list_price#48] +Keys [1]: [ss_list_price#48] +Functions [2]: [partial_avg(UnscaledValue(ss_list_price#48)), partial_count(ss_list_price#48)] +Aggregate Attributes [2]: [avg(UnscaledValue(ss_list_price#48))#51, count(ss_list_price#48)#52] +Results [4]: [ss_list_price#48, sum#53, count#54, count#55] + +(40) Exchange +Input [4]: [ss_list_price#48, sum#53, count#54, count#55] +Arguments: hashpartitioning(ss_list_price#48, 5), ENSURE_REQUIREMENTS, [plan_id=9] + +(41) HashAggregate [codegen id : 10] +Input [4]: [ss_list_price#48, sum#53, count#54, count#55] +Keys [1]: [ss_list_price#48] +Functions [2]: [merge_avg(UnscaledValue(ss_list_price#48)), merge_count(ss_list_price#48)] +Aggregate Attributes [2]: [avg(UnscaledValue(ss_list_price#48))#51, count(ss_list_price#48)#52] +Results [4]: [ss_list_price#48, sum#53, count#54, count#55] + +(42) HashAggregate [codegen id : 10] +Input [4]: [ss_list_price#48, sum#53, count#54, count#55] +Keys: [] +Functions [3]: [merge_avg(UnscaledValue(ss_list_price#48)), merge_count(ss_list_price#48), partial_count(distinct ss_list_price#48)] +Aggregate Attributes [3]: [avg(UnscaledValue(ss_list_price#48))#51, count(ss_list_price#48)#52, count(ss_list_price#48)#56] +Results [4]: [sum#53, count#54, count#55, count#57] + +(43) Exchange +Input [4]: [sum#53, count#54, count#55, count#57] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=10] + +(44) HashAggregate [codegen id : 11] +Input [4]: [sum#53, count#54, count#55, count#57] +Keys: [] +Functions [3]: [avg(UnscaledValue(ss_list_price#48)), count(ss_list_price#48), count(distinct ss_list_price#48)] +Aggregate Attributes [3]: [avg(UnscaledValue(ss_list_price#48))#51, count(ss_list_price#48)#52, count(ss_list_price#48)#56] +Results [3]: [cast((avg(UnscaledValue(ss_list_price#48))#51 / 100.0) as decimal(11,6)) AS B4_LP#58, count(ss_list_price#48)#52 AS B4_CNT#59, count(ss_list_price#48)#56 AS B4_CNTD#60] + +(45) BroadcastExchange +Input [3]: [B4_LP#58, B4_CNT#59, B4_CNTD#60] +Arguments: IdentityBroadcastMode, [plan_id=11] + +(46) BroadcastNestedLoopJoin [codegen id : 18] +Join type: Inner +Join condition: None + +(47) Scan parquet spark_catalog.default.store_sales +Output [5]: [ss_quantity#61, ss_wholesale_cost#62, ss_list_price#63, ss_coupon_amt#64, ss_sold_date_sk#65] +Batched: true +Location [not included in comparison]/{warehouse_dir}/store_sales] +PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,21), LessThanOrEqual(ss_quantity,25), Or(Or(And(GreaterThanOrEqual(ss_list_price,122.00),LessThanOrEqual(ss_list_price,132.00)),And(GreaterThanOrEqual(ss_coupon_amt,836.00),LessThanOrEqual(ss_coupon_amt,1836.00))),And(GreaterThanOrEqual(ss_wholesale_cost,17.00),LessThanOrEqual(ss_wholesale_cost,37.00)))] +ReadSchema: struct<ss_quantity:int,ss_wholesale_cost:decimal(7,2),ss_list_price:decimal(7,2),ss_coupon_amt:decimal(7,2)> + +(48) CometFilter +Input [5]: [ss_quantity#61, ss_wholesale_cost#62, ss_list_price#63, ss_coupon_amt#64, ss_sold_date_sk#65] +Condition : (((isnotnull(ss_quantity#61) AND (ss_quantity#61 >= 21)) AND (ss_quantity#61 <= 25)) AND ((((ss_list_price#63 >= 122.00) AND (ss_list_price#63 <= 132.00)) OR ((ss_coupon_amt#64 >= 836.00) AND (ss_coupon_amt#64 <= 1836.00))) OR ((ss_wholesale_cost#62 >= 17.00) AND (ss_wholesale_cost#62 <= 37.00)))) + +(49) CometProject +Input [5]: [ss_quantity#61, ss_wholesale_cost#62, ss_list_price#63, ss_coupon_amt#64, ss_sold_date_sk#65] +Arguments: [ss_list_price#63], [ss_list_price#63] + +(50) ColumnarToRow [codegen id : 12] +Input [1]: [ss_list_price#63] + +(51) HashAggregate [codegen id : 12] +Input [1]: [ss_list_price#63] +Keys [1]: [ss_list_price#63] +Functions [2]: [partial_avg(UnscaledValue(ss_list_price#63)), partial_count(ss_list_price#63)] +Aggregate Attributes [2]: [avg(UnscaledValue(ss_list_price#63))#66, count(ss_list_price#63)#67] +Results [4]: [ss_list_price#63, sum#68, count#69, count#70] + +(52) Exchange +Input [4]: [ss_list_price#63, sum#68, count#69, count#70] +Arguments: hashpartitioning(ss_list_price#63, 5), ENSURE_REQUIREMENTS, [plan_id=12] + +(53) HashAggregate [codegen id : 13] +Input [4]: [ss_list_price#63, sum#68, count#69, count#70] +Keys [1]: [ss_list_price#63] +Functions [2]: [merge_avg(UnscaledValue(ss_list_price#63)), merge_count(ss_list_price#63)] +Aggregate Attributes [2]: [avg(UnscaledValue(ss_list_price#63))#66, count(ss_list_price#63)#67] +Results [4]: [ss_list_price#63, sum#68, count#69, count#70] + +(54) HashAggregate [codegen id : 13] +Input [4]: [ss_list_price#63, sum#68, count#69, count#70] +Keys: [] +Functions [3]: [merge_avg(UnscaledValue(ss_list_price#63)), merge_count(ss_list_price#63), partial_count(distinct ss_list_price#63)] +Aggregate Attributes [3]: [avg(UnscaledValue(ss_list_price#63))#66, count(ss_list_price#63)#67, count(ss_list_price#63)#71] +Results [4]: [sum#68, count#69, count#70, count#72] + +(55) Exchange +Input [4]: [sum#68, count#69, count#70, count#72] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=13] + +(56) HashAggregate [codegen id : 14] +Input [4]: [sum#68, count#69, count#70, count#72] +Keys: [] +Functions [3]: [avg(UnscaledValue(ss_list_price#63)), count(ss_list_price#63), count(distinct ss_list_price#63)] +Aggregate Attributes [3]: [avg(UnscaledValue(ss_list_price#63))#66, count(ss_list_price#63)#67, count(ss_list_price#63)#71] +Results [3]: [cast((avg(UnscaledValue(ss_list_price#63))#66 / 100.0) as decimal(11,6)) AS B5_LP#73, count(ss_list_price#63)#67 AS B5_CNT#74, count(ss_list_price#63)#71 AS B5_CNTD#75] + +(57) BroadcastExchange +Input [3]: [B5_LP#73, B5_CNT#74, B5_CNTD#75] +Arguments: IdentityBroadcastMode, [plan_id=14] + +(58) BroadcastNestedLoopJoin [codegen id : 18] +Join type: Inner +Join condition: None + +(59) Scan parquet spark_catalog.default.store_sales +Output [5]: [ss_quantity#76, ss_wholesale_cost#77, ss_list_price#78, ss_coupon_amt#79, ss_sold_date_sk#80] +Batched: true +Location [not included in comparison]/{warehouse_dir}/store_sales] +PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,26), LessThanOrEqual(ss_quantity,30), Or(Or(And(GreaterThanOrEqual(ss_list_price,154.00),LessThanOrEqual(ss_list_price,164.00)),And(GreaterThanOrEqual(ss_coupon_amt,7326.00),LessThanOrEqual(ss_coupon_amt,8326.00))),And(GreaterThanOrEqual(ss_wholesale_cost,7.00),LessThanOrEqual(ss_wholesale_cost,27.00)))] +ReadSchema: struct<ss_quantity:int,ss_wholesale_cost:decimal(7,2),ss_list_price:decimal(7,2),ss_coupon_amt:decimal(7,2)> + +(60) CometFilter +Input [5]: [ss_quantity#76, ss_wholesale_cost#77, ss_list_price#78, ss_coupon_amt#79, ss_sold_date_sk#80] +Condition : (((isnotnull(ss_quantity#76) AND (ss_quantity#76 >= 26)) AND (ss_quantity#76 <= 30)) AND ((((ss_list_price#78 >= 154.00) AND (ss_list_price#78 <= 164.00)) OR ((ss_coupon_amt#79 >= 7326.00) AND (ss_coupon_amt#79 <= 8326.00))) OR ((ss_wholesale_cost#77 >= 7.00) AND (ss_wholesale_cost#77 <= 27.00)))) + +(61) CometProject +Input [5]: [ss_quantity#76, ss_wholesale_cost#77, ss_list_price#78, ss_coupon_amt#79, ss_sold_date_sk#80] +Arguments: [ss_list_price#78], [ss_list_price#78] + +(62) ColumnarToRow [codegen id : 15] +Input [1]: [ss_list_price#78] + +(63) HashAggregate [codegen id : 15] +Input [1]: [ss_list_price#78] +Keys [1]: [ss_list_price#78] +Functions [2]: [partial_avg(UnscaledValue(ss_list_price#78)), partial_count(ss_list_price#78)] +Aggregate Attributes [2]: [avg(UnscaledValue(ss_list_price#78))#81, count(ss_list_price#78)#82] +Results [4]: [ss_list_price#78, sum#83, count#84, count#85] + +(64) Exchange +Input [4]: [ss_list_price#78, sum#83, count#84, count#85] +Arguments: hashpartitioning(ss_list_price#78, 5), ENSURE_REQUIREMENTS, [plan_id=15] + +(65) HashAggregate [codegen id : 16] +Input [4]: [ss_list_price#78, sum#83, count#84, count#85] +Keys [1]: [ss_list_price#78] +Functions [2]: [merge_avg(UnscaledValue(ss_list_price#78)), merge_count(ss_list_price#78)] +Aggregate Attributes [2]: [avg(UnscaledValue(ss_list_price#78))#81, count(ss_list_price#78)#82] +Results [4]: [ss_list_price#78, sum#83, count#84, count#85] + +(66) HashAggregate [codegen id : 16] +Input [4]: [ss_list_price#78, sum#83, count#84, count#85] +Keys: [] +Functions [3]: [merge_avg(UnscaledValue(ss_list_price#78)), merge_count(ss_list_price#78), partial_count(distinct ss_list_price#78)] +Aggregate Attributes [3]: [avg(UnscaledValue(ss_list_price#78))#81, count(ss_list_price#78)#82, count(ss_list_price#78)#86] +Results [4]: [sum#83, count#84, count#85, count#87] + +(67) Exchange +Input [4]: [sum#83, count#84, count#85, count#87] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=16] + +(68) HashAggregate [codegen id : 17] +Input [4]: [sum#83, count#84, count#85, count#87] +Keys: [] +Functions [3]: [avg(UnscaledValue(ss_list_price#78)), count(ss_list_price#78), count(distinct ss_list_price#78)] +Aggregate Attributes [3]: [avg(UnscaledValue(ss_list_price#78))#81, count(ss_list_price#78)#82, count(ss_list_price#78)#86] +Results [3]: [cast((avg(UnscaledValue(ss_list_price#78))#81 / 100.0) as decimal(11,6)) AS B6_LP#88, count(ss_list_price#78)#82 AS B6_CNT#89, count(ss_list_price#78)#86 AS B6_CNTD#90] + +(69) BroadcastExchange +Input [3]: [B6_LP#88, B6_CNT#89, B6_CNTD#90] +Arguments: IdentityBroadcastMode, [plan_id=17] + +(70) BroadcastNestedLoopJoin [codegen id : 18] +Join type: Inner +Join condition: None + diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q28/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q28/simplified.txt new file mode 100644 index 000000000..a7a3f9537 --- /dev/null +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q28/simplified.txt @@ -0,0 +1,111 @@ +WholeStageCodegen (18) + BroadcastNestedLoopJoin + BroadcastNestedLoopJoin + BroadcastNestedLoopJoin + BroadcastNestedLoopJoin + BroadcastNestedLoopJoin + HashAggregate [sum,count,count,count] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),count(ss_list_price),B1_LP,B1_CNT,B1_CNTD,sum,count,count,count] + InputAdapter + Exchange #1 + WholeStageCodegen (2) + HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),count(ss_list_price),sum,count,count,count,sum,count,count,count] + HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),sum,count,count,sum,count,count] + InputAdapter + Exchange [ss_list_price] #2 + WholeStageCodegen (1) + HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),sum,count,count,sum,count,count] + ColumnarToRow + InputAdapter + CometProject [ss_list_price] + CometFilter [ss_quantity,ss_list_price,ss_coupon_amt,ss_wholesale_cost] + CometScan parquet spark_catalog.default.store_sales [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] + InputAdapter + BroadcastExchange #3 + WholeStageCodegen (5) + HashAggregate [sum,count,count,count] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),count(ss_list_price),B2_LP,B2_CNT,B2_CNTD,sum,count,count,count] + InputAdapter + Exchange #4 + WholeStageCodegen (4) + HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),count(ss_list_price),sum,count,count,count,sum,count,count,count] + HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),sum,count,count,sum,count,count] + InputAdapter + Exchange [ss_list_price] #5 + WholeStageCodegen (3) + HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),sum,count,count,sum,count,count] + ColumnarToRow + InputAdapter + CometProject [ss_list_price] + CometFilter [ss_quantity,ss_list_price,ss_coupon_amt,ss_wholesale_cost] + CometScan parquet spark_catalog.default.store_sales [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] + InputAdapter + BroadcastExchange #6 + WholeStageCodegen (8) + HashAggregate [sum,count,count,count] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),count(ss_list_price),B3_LP,B3_CNT,B3_CNTD,sum,count,count,count] + InputAdapter + Exchange #7 + WholeStageCodegen (7) + HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),count(ss_list_price),sum,count,count,count,sum,count,count,count] + HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),sum,count,count,sum,count,count] + InputAdapter + Exchange [ss_list_price] #8 + WholeStageCodegen (6) + HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),sum,count,count,sum,count,count] + ColumnarToRow + InputAdapter + CometProject [ss_list_price] + CometFilter [ss_quantity,ss_list_price,ss_coupon_amt,ss_wholesale_cost] + CometScan parquet spark_catalog.default.store_sales [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] + InputAdapter + BroadcastExchange #9 + WholeStageCodegen (11) + HashAggregate [sum,count,count,count] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),count(ss_list_price),B4_LP,B4_CNT,B4_CNTD,sum,count,count,count] + InputAdapter + Exchange #10 + WholeStageCodegen (10) + HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),count(ss_list_price),sum,count,count,count,sum,count,count,count] + HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),sum,count,count,sum,count,count] + InputAdapter + Exchange [ss_list_price] #11 + WholeStageCodegen (9) + HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),sum,count,count,sum,count,count] + ColumnarToRow + InputAdapter + CometProject [ss_list_price] + CometFilter [ss_quantity,ss_list_price,ss_coupon_amt,ss_wholesale_cost] + CometScan parquet spark_catalog.default.store_sales [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] + InputAdapter + BroadcastExchange #12 + WholeStageCodegen (14) + HashAggregate [sum,count,count,count] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),count(ss_list_price),B5_LP,B5_CNT,B5_CNTD,sum,count,count,count] + InputAdapter + Exchange #13 + WholeStageCodegen (13) + HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),count(ss_list_price),sum,count,count,count,sum,count,count,count] + HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),sum,count,count,sum,count,count] + InputAdapter + Exchange [ss_list_price] #14 + WholeStageCodegen (12) + HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),sum,count,count,sum,count,count] + ColumnarToRow + InputAdapter + CometProject [ss_list_price] + CometFilter [ss_quantity,ss_list_price,ss_coupon_amt,ss_wholesale_cost] + CometScan parquet spark_catalog.default.store_sales [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] + InputAdapter + BroadcastExchange #15 + WholeStageCodegen (17) + HashAggregate [sum,count,count,count] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),count(ss_list_price),B6_LP,B6_CNT,B6_CNTD,sum,count,count,count] + InputAdapter + Exchange #16 + WholeStageCodegen (16) + HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),count(ss_list_price),sum,count,count,count,sum,count,count,count] + HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),sum,count,count,sum,count,count] + InputAdapter + Exchange [ss_list_price] #17 + WholeStageCodegen (15) + HashAggregate [ss_list_price] [avg(UnscaledValue(ss_list_price)),count(ss_list_price),sum,count,count,sum,count,count] + ColumnarToRow + InputAdapter + CometProject [ss_list_price] + CometFilter [ss_quantity,ss_list_price,ss_coupon_amt,ss_wholesale_cost] + CometScan parquet spark_catalog.default.store_sales [ss_quantity,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q29/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q29/explain.txt new file mode 100644 index 000000000..522754cbc --- /dev/null +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q29/explain.txt @@ -0,0 +1,326 @@ +== Physical Plan == +TakeOrderedAndProject (40) ++- * HashAggregate (39) + +- Exchange (38) + +- * HashAggregate (37) + +- * Project (36) + +- * BroadcastHashJoin Inner BuildRight (35) + :- * Project (30) + : +- * BroadcastHashJoin Inner BuildRight (29) + : :- * Project (24) + : : +- * BroadcastHashJoin Inner BuildRight (23) + : : :- * Project (21) + : : : +- * BroadcastHashJoin Inner BuildRight (20) + : : : :- * Project (18) + : : : : +- * BroadcastHashJoin Inner BuildRight (17) + : : : : :- * Project (15) + : : : : : +- * BroadcastHashJoin Inner BuildRight (14) + : : : : : :- * Project (9) + : : : : : : +- * BroadcastHashJoin Inner BuildRight (8) + : : : : : : :- * ColumnarToRow (3) + : : : : : : : +- CometFilter (2) + : : : : : : : +- CometScan parquet spark_catalog.default.store_sales (1) + : : : : : : +- BroadcastExchange (7) + : : : : : : +- * ColumnarToRow (6) + : : : : : : +- CometFilter (5) + : : : : : : +- CometScan parquet spark_catalog.default.store_returns (4) + : : : : : +- BroadcastExchange (13) + : : : : : +- * ColumnarToRow (12) + : : : : : +- CometFilter (11) + : : : : : +- CometScan parquet spark_catalog.default.catalog_sales (10) + : : : : +- ReusedExchange (16) + : : : +- ReusedExchange (19) + : : +- ReusedExchange (22) + : +- BroadcastExchange (28) + : +- * ColumnarToRow (27) + : +- CometFilter (26) + : +- CometScan parquet spark_catalog.default.store (25) + +- BroadcastExchange (34) + +- * ColumnarToRow (33) + +- CometFilter (32) + +- CometScan parquet spark_catalog.default.item (31) + + +(1) Scan parquet spark_catalog.default.store_sales +Output [6]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_quantity#5, ss_sold_date_sk#6] +Batched: true +Location: InMemoryFileIndex [] +PartitionFilters: [isnotnull(ss_sold_date_sk#6), dynamicpruningexpression(ss_sold_date_sk#6 IN dynamicpruning#7)] +PushedFilters: [IsNotNull(ss_customer_sk), IsNotNull(ss_item_sk), IsNotNull(ss_ticket_number), IsNotNull(ss_store_sk)] +ReadSchema: struct<ss_item_sk:int,ss_customer_sk:int,ss_store_sk:int,ss_ticket_number:int,ss_quantity:int> + +(2) CometFilter +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)) + +(3) ColumnarToRow [codegen id : 8] +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] + +(4) Scan parquet spark_catalog.default.store_returns +Output [5]: [sr_item_sk#8, sr_customer_sk#9, sr_ticket_number#10, sr_return_quantity#11, sr_returned_date_sk#12] +Batched: true +Location: InMemoryFileIndex [] +PartitionFilters: [isnotnull(sr_returned_date_sk#12), dynamicpruningexpression(sr_returned_date_sk#12 IN dynamicpruning#13)] +PushedFilters: [IsNotNull(sr_customer_sk), IsNotNull(sr_item_sk), IsNotNull(sr_ticket_number)] +ReadSchema: struct<sr_item_sk:int,sr_customer_sk:int,sr_ticket_number:int,sr_return_quantity:int> + +(5) CometFilter +Input [5]: [sr_item_sk#8, sr_customer_sk#9, sr_ticket_number#10, sr_return_quantity#11, sr_returned_date_sk#12] +Condition : ((isnotnull(sr_customer_sk#9) AND isnotnull(sr_item_sk#8)) AND isnotnull(sr_ticket_number#10)) + +(6) ColumnarToRow [codegen id : 1] +Input [5]: [sr_item_sk#8, sr_customer_sk#9, sr_ticket_number#10, sr_return_quantity#11, sr_returned_date_sk#12] + +(7) BroadcastExchange +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] +Left keys [3]: [ss_customer_sk#2, ss_item_sk#1, ss_ticket_number#4] +Right keys [3]: [sr_customer_sk#9, sr_item_sk#8, sr_ticket_number#10] +Join type: Inner +Join condition: None + +(9) Project [codegen id : 8] +Output [8]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, ss_sold_date_sk#6, sr_item_sk#8, sr_customer_sk#9, sr_return_quantity#11, sr_returned_date_sk#12] +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] + +(10) Scan parquet spark_catalog.default.catalog_sales +Output [4]: [cs_bill_customer_sk#14, cs_item_sk#15, cs_quantity#16, cs_sold_date_sk#17] +Batched: true +Location: InMemoryFileIndex [] +PartitionFilters: [isnotnull(cs_sold_date_sk#17), dynamicpruningexpression(cs_sold_date_sk#17 IN dynamicpruning#18)] +PushedFilters: [IsNotNull(cs_bill_customer_sk), IsNotNull(cs_item_sk)] +ReadSchema: struct<cs_bill_customer_sk:int,cs_item_sk:int,cs_quantity:int> + +(11) CometFilter +Input [4]: [cs_bill_customer_sk#14, cs_item_sk#15, cs_quantity#16, cs_sold_date_sk#17] +Condition : (isnotnull(cs_bill_customer_sk#14) AND isnotnull(cs_item_sk#15)) + +(12) ColumnarToRow [codegen id : 2] +Input [4]: [cs_bill_customer_sk#14, cs_item_sk#15, cs_quantity#16, cs_sold_date_sk#17] + +(13) BroadcastExchange +Input [4]: [cs_bill_customer_sk#14, cs_item_sk#15, cs_quantity#16, cs_sold_date_sk#17] +Arguments: HashedRelationBroadcastMode(List((shiftleft(cast(input[0, int, false] as bigint), 32) | (cast(input[1, int, false] as bigint) & 4294967295))),false), [plan_id=2] + +(14) BroadcastHashJoin [codegen id : 8] +Left keys [2]: [sr_customer_sk#9, sr_item_sk#8] +Right keys [2]: [cs_bill_customer_sk#14, cs_item_sk#15] +Join type: Inner +Join condition: None + +(15) Project [codegen id : 8] +Output [8]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, ss_sold_date_sk#6, sr_return_quantity#11, sr_returned_date_sk#12, cs_quantity#16, cs_sold_date_sk#17] +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: 45] +Output [1]: [d_date_sk#19] + +(17) BroadcastHashJoin [codegen id : 8] +Left keys [1]: [ss_sold_date_sk#6] +Right keys [1]: [d_date_sk#19] +Join type: Inner +Join condition: None + +(18) Project [codegen id : 8] +Output [7]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, sr_return_quantity#11, sr_returned_date_sk#12, cs_quantity#16, cs_sold_date_sk#17] +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: 50] +Output [1]: [d_date_sk#20] + +(20) BroadcastHashJoin [codegen id : 8] +Left keys [1]: [sr_returned_date_sk#12] +Right keys [1]: [d_date_sk#20] +Join type: Inner +Join condition: None + +(21) Project [codegen id : 8] +Output [6]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, sr_return_quantity#11, cs_quantity#16, cs_sold_date_sk#17] +Input [8]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, sr_return_quantity#11, sr_returned_date_sk#12, cs_quantity#16, cs_sold_date_sk#17, d_date_sk#20] + +(22) ReusedExchange [Reuses operator id: 55] +Output [1]: [d_date_sk#21] + +(23) BroadcastHashJoin [codegen id : 8] +Left keys [1]: [cs_sold_date_sk#17] +Right keys [1]: [d_date_sk#21] +Join type: Inner +Join condition: None + +(24) Project [codegen id : 8] +Output [5]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, sr_return_quantity#11, cs_quantity#16] +Input [7]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, sr_return_quantity#11, cs_quantity#16, cs_sold_date_sk#17, d_date_sk#21] + +(25) Scan parquet spark_catalog.default.store +Output [3]: [s_store_sk#22, s_store_id#23, s_store_name#24] +Batched: true +Location [not included in comparison]/{warehouse_dir}/store] +PushedFilters: [IsNotNull(s_store_sk)] +ReadSchema: struct<s_store_sk:int,s_store_id:string,s_store_name:string> + +(26) CometFilter +Input [3]: [s_store_sk#22, s_store_id#23, s_store_name#24] +Condition : isnotnull(s_store_sk#22) + +(27) ColumnarToRow [codegen id : 6] +Input [3]: [s_store_sk#22, s_store_id#23, s_store_name#24] + +(28) BroadcastExchange +Input [3]: [s_store_sk#22, s_store_id#23, s_store_name#24] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=3] + +(29) BroadcastHashJoin [codegen id : 8] +Left keys [1]: [ss_store_sk#3] +Right keys [1]: [s_store_sk#22] +Join type: Inner +Join condition: None + +(30) Project [codegen id : 8] +Output [6]: [ss_item_sk#1, ss_quantity#5, sr_return_quantity#11, cs_quantity#16, s_store_id#23, s_store_name#24] +Input [8]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#5, sr_return_quantity#11, cs_quantity#16, s_store_sk#22, s_store_id#23, s_store_name#24] + +(31) Scan parquet spark_catalog.default.item +Output [3]: [i_item_sk#25, i_item_id#26, i_item_desc#27] +Batched: true +Location [not included in comparison]/{warehouse_dir}/item] +PushedFilters: [IsNotNull(i_item_sk)] +ReadSchema: struct<i_item_sk:int,i_item_id:string,i_item_desc:string> + +(32) CometFilter +Input [3]: [i_item_sk#25, i_item_id#26, i_item_desc#27] +Condition : isnotnull(i_item_sk#25) + +(33) ColumnarToRow [codegen id : 7] +Input [3]: [i_item_sk#25, i_item_id#26, i_item_desc#27] + +(34) BroadcastExchange +Input [3]: [i_item_sk#25, i_item_id#26, i_item_desc#27] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=4] + +(35) BroadcastHashJoin [codegen id : 8] +Left keys [1]: [ss_item_sk#1] +Right keys [1]: [i_item_sk#25] +Join type: Inner +Join condition: None + +(36) Project [codegen id : 8] +Output [7]: [ss_quantity#5, sr_return_quantity#11, cs_quantity#16, s_store_id#23, s_store_name#24, i_item_id#26, i_item_desc#27] +Input [9]: [ss_item_sk#1, ss_quantity#5, sr_return_quantity#11, cs_quantity#16, s_store_id#23, s_store_name#24, i_item_sk#25, i_item_id#26, i_item_desc#27] + +(37) HashAggregate [codegen id : 8] +Input [7]: [ss_quantity#5, sr_return_quantity#11, cs_quantity#16, s_store_id#23, s_store_name#24, i_item_id#26, i_item_desc#27] +Keys [4]: [i_item_id#26, i_item_desc#27, s_store_id#23, s_store_name#24] +Functions [3]: [partial_sum(ss_quantity#5), partial_sum(sr_return_quantity#11), partial_sum(cs_quantity#16)] +Aggregate Attributes [3]: [sum#28, sum#29, sum#30] +Results [7]: [i_item_id#26, i_item_desc#27, s_store_id#23, s_store_name#24, sum#31, sum#32, sum#33] + +(38) Exchange +Input [7]: [i_item_id#26, i_item_desc#27, s_store_id#23, s_store_name#24, sum#31, sum#32, sum#33] +Arguments: hashpartitioning(i_item_id#26, i_item_desc#27, s_store_id#23, s_store_name#24, 5), ENSURE_REQUIREMENTS, [plan_id=5] + +(39) HashAggregate [codegen id : 9] +Input [7]: [i_item_id#26, i_item_desc#27, s_store_id#23, s_store_name#24, sum#31, sum#32, sum#33] +Keys [4]: [i_item_id#26, i_item_desc#27, s_store_id#23, s_store_name#24] +Functions [3]: [sum(ss_quantity#5), sum(sr_return_quantity#11), sum(cs_quantity#16)] +Aggregate Attributes [3]: [sum(ss_quantity#5)#34, sum(sr_return_quantity#11)#35, sum(cs_quantity#16)#36] +Results [7]: [i_item_id#26, i_item_desc#27, s_store_id#23, s_store_name#24, sum(ss_quantity#5)#34 AS store_sales_quantity#37, sum(sr_return_quantity#11)#35 AS store_returns_quantity#38, sum(cs_quantity#16)#36 AS catalog_sales_quantity#39] + +(40) TakeOrderedAndProject +Input [7]: [i_item_id#26, i_item_desc#27, s_store_id#23, s_store_name#24, store_sales_quantity#37, store_returns_quantity#38, catalog_sales_quantity#39] +Arguments: 100, [i_item_id#26 ASC NULLS FIRST, i_item_desc#27 ASC NULLS FIRST, s_store_id#23 ASC NULLS FIRST, s_store_name#24 ASC NULLS FIRST], [i_item_id#26, i_item_desc#27, s_store_id#23, s_store_name#24, store_sales_quantity#37, store_returns_quantity#38, catalog_sales_quantity#39] + +===== Subqueries ===== + +Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#6 IN dynamicpruning#7 +BroadcastExchange (45) ++- * ColumnarToRow (44) + +- CometProject (43) + +- CometFilter (42) + +- CometScan parquet spark_catalog.default.date_dim (41) + + +(41) Scan parquet spark_catalog.default.date_dim +Output [3]: [d_date_sk#19, d_year#40, d_moy#41] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_moy), IsNotNull(d_year), EqualTo(d_moy,9), EqualTo(d_year,1999), IsNotNull(d_date_sk)] +ReadSchema: struct<d_date_sk:int,d_year:int,d_moy:int> + +(42) CometFilter +Input [3]: [d_date_sk#19, d_year#40, d_moy#41] +Condition : ((((isnotnull(d_moy#41) AND isnotnull(d_year#40)) AND (d_moy#41 = 9)) AND (d_year#40 = 1999)) AND isnotnull(d_date_sk#19)) + +(43) CometProject +Input [3]: [d_date_sk#19, d_year#40, d_moy#41] +Arguments: [d_date_sk#19], [d_date_sk#19] + +(44) ColumnarToRow [codegen id : 1] +Input [1]: [d_date_sk#19] + +(45) BroadcastExchange +Input [1]: [d_date_sk#19] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=6] + +Subquery:2 Hosting operator id = 4 Hosting Expression = sr_returned_date_sk#12 IN dynamicpruning#13 +BroadcastExchange (50) ++- * ColumnarToRow (49) + +- CometProject (48) + +- CometFilter (47) + +- CometScan parquet spark_catalog.default.date_dim (46) + + +(46) Scan parquet spark_catalog.default.date_dim +Output [3]: [d_date_sk#20, d_year#42, d_moy#43] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_moy), IsNotNull(d_year), GreaterThanOrEqual(d_moy,9), LessThanOrEqual(d_moy,12), EqualTo(d_year,1999), IsNotNull(d_date_sk)] +ReadSchema: struct<d_date_sk:int,d_year:int,d_moy:int> + +(47) CometFilter +Input [3]: [d_date_sk#20, d_year#42, d_moy#43] +Condition : (((((isnotnull(d_moy#43) AND isnotnull(d_year#42)) AND (d_moy#43 >= 9)) AND (d_moy#43 <= 12)) AND (d_year#42 = 1999)) AND isnotnull(d_date_sk#20)) + +(48) CometProject +Input [3]: [d_date_sk#20, d_year#42, d_moy#43] +Arguments: [d_date_sk#20], [d_date_sk#20] + +(49) ColumnarToRow [codegen id : 1] +Input [1]: [d_date_sk#20] + +(50) BroadcastExchange +Input [1]: [d_date_sk#20] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=7] + +Subquery:3 Hosting operator id = 10 Hosting Expression = cs_sold_date_sk#17 IN dynamicpruning#18 +BroadcastExchange (55) ++- * ColumnarToRow (54) + +- CometProject (53) + +- CometFilter (52) + +- CometScan parquet spark_catalog.default.date_dim (51) + + +(51) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#21, d_year#44] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [In(d_year, [1999,2000,2001]), IsNotNull(d_date_sk)] +ReadSchema: struct<d_date_sk:int,d_year:int> + +(52) CometFilter +Input [2]: [d_date_sk#21, d_year#44] +Condition : (d_year#44 IN (1999,2000,2001) AND isnotnull(d_date_sk#21)) + +(53) CometProject +Input [2]: [d_date_sk#21, d_year#44] +Arguments: [d_date_sk#21], [d_date_sk#21] + +(54) ColumnarToRow [codegen id : 1] +Input [1]: [d_date_sk#21] + +(55) BroadcastExchange +Input [1]: [d_date_sk#21] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=8] + + diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q29/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q29/simplified.txt new file mode 100644 index 000000000..68a127d35 --- /dev/null +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q29/simplified.txt @@ -0,0 +1,83 @@ +TakeOrderedAndProject [i_item_id,i_item_desc,s_store_id,s_store_name,store_sales_quantity,store_returns_quantity,catalog_sales_quantity] + WholeStageCodegen (9) + HashAggregate [i_item_id,i_item_desc,s_store_id,s_store_name,sum,sum,sum] [sum(ss_quantity),sum(sr_return_quantity),sum(cs_quantity),store_sales_quantity,store_returns_quantity,catalog_sales_quantity,sum,sum,sum] + InputAdapter + Exchange [i_item_id,i_item_desc,s_store_id,s_store_name] #1 + WholeStageCodegen (8) + HashAggregate [i_item_id,i_item_desc,s_store_id,s_store_name,ss_quantity,sr_return_quantity,cs_quantity] [sum,sum,sum,sum,sum,sum] + Project [ss_quantity,sr_return_quantity,cs_quantity,s_store_id,s_store_name,i_item_id,i_item_desc] + BroadcastHashJoin [ss_item_sk,i_item_sk] + Project [ss_item_sk,ss_quantity,sr_return_quantity,cs_quantity,s_store_id,s_store_name] + BroadcastHashJoin [ss_store_sk,s_store_sk] + Project [ss_item_sk,ss_store_sk,ss_quantity,sr_return_quantity,cs_quantity] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + Project [ss_item_sk,ss_store_sk,ss_quantity,sr_return_quantity,cs_quantity,cs_sold_date_sk] + BroadcastHashJoin [sr_returned_date_sk,d_date_sk] + Project [ss_item_sk,ss_store_sk,ss_quantity,sr_return_quantity,sr_returned_date_sk,cs_quantity,cs_sold_date_sk] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [ss_item_sk,ss_store_sk,ss_quantity,ss_sold_date_sk,sr_return_quantity,sr_returned_date_sk,cs_quantity,cs_sold_date_sk] + BroadcastHashJoin [sr_customer_sk,sr_item_sk,cs_bill_customer_sk,cs_item_sk] + Project [ss_item_sk,ss_store_sk,ss_quantity,ss_sold_date_sk,sr_item_sk,sr_customer_sk,sr_return_quantity,sr_returned_date_sk] + BroadcastHashJoin [ss_customer_sk,ss_item_sk,ss_ticket_number,sr_customer_sk,sr_item_sk,sr_ticket_number] + ColumnarToRow + InputAdapter + CometFilter [ss_customer_sk,ss_item_sk,ss_ticket_number,ss_store_sk] + CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_quantity,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #2 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_moy,d_year,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + InputAdapter + BroadcastExchange #3 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometFilter [sr_customer_sk,sr_item_sk,sr_ticket_number] + CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_customer_sk,sr_ticket_number,sr_return_quantity,sr_returned_date_sk] + SubqueryBroadcast [d_date_sk] #2 + BroadcastExchange #4 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_moy,d_year,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + InputAdapter + BroadcastExchange #5 + WholeStageCodegen (2) + ColumnarToRow + InputAdapter + CometFilter [cs_bill_customer_sk,cs_item_sk] + CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_sold_date_sk] + SubqueryBroadcast [d_date_sk] #3 + BroadcastExchange #6 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_year,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + InputAdapter + ReusedExchange [d_date_sk] #2 + InputAdapter + ReusedExchange [d_date_sk] #4 + InputAdapter + ReusedExchange [d_date_sk] #6 + InputAdapter + BroadcastExchange #7 + WholeStageCodegen (6) + ColumnarToRow + InputAdapter + CometFilter [s_store_sk] + CometScan parquet spark_catalog.default.store [s_store_sk,s_store_id,s_store_name] + InputAdapter + BroadcastExchange #8 + WholeStageCodegen (7) + ColumnarToRow + InputAdapter + CometFilter [i_item_sk] + CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q3/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q3/explain.txt new file mode 100644 index 000000000..e89cfe1ff --- /dev/null +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q3/explain.txt @@ -0,0 +1,125 @@ +== Physical Plan == +TakeOrderedAndProject (21) ++- * HashAggregate (20) + +- Exchange (19) + +- * HashAggregate (18) + +- * Project (17) + +- * BroadcastHashJoin Inner BuildRight (16) + :- * Project (10) + : +- * BroadcastHashJoin Inner BuildRight (9) + : :- * ColumnarToRow (4) + : : +- CometProject (3) + : : +- CometFilter (2) + : : +- CometScan parquet spark_catalog.default.date_dim (1) + : +- BroadcastExchange (8) + : +- * ColumnarToRow (7) + : +- CometFilter (6) + : +- CometScan parquet spark_catalog.default.store_sales (5) + +- BroadcastExchange (15) + +- * ColumnarToRow (14) + +- CometProject (13) + +- CometFilter (12) + +- CometScan parquet spark_catalog.default.item (11) + + +(1) Scan parquet spark_catalog.default.date_dim +Output [3]: [d_date_sk#1, d_year#2, d_moy#3] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_moy), EqualTo(d_moy,11), IsNotNull(d_date_sk)] +ReadSchema: struct<d_date_sk:int,d_year:int,d_moy:int> + +(2) CometFilter +Input [3]: [d_date_sk#1, d_year#2, d_moy#3] +Condition : ((isnotnull(d_moy#3) AND (d_moy#3 = 11)) AND isnotnull(d_date_sk#1)) + +(3) CometProject +Input [3]: [d_date_sk#1, d_year#2, d_moy#3] +Arguments: [d_date_sk#1, d_year#2], [d_date_sk#1, d_year#2] + +(4) ColumnarToRow [codegen id : 3] +Input [2]: [d_date_sk#1, d_year#2] + +(5) Scan parquet spark_catalog.default.store_sales +Output [3]: [ss_item_sk#4, ss_ext_sales_price#5, ss_sold_date_sk#6] +Batched: true +Location: InMemoryFileIndex [] +PartitionFilters: [isnotnull(ss_sold_date_sk#6), dynamicpruningexpression(true)] +PushedFilters: [IsNotNull(ss_item_sk)] +ReadSchema: struct<ss_item_sk:int,ss_ext_sales_price:decimal(7,2)> + +(6) CometFilter +Input [3]: [ss_item_sk#4, ss_ext_sales_price#5, ss_sold_date_sk#6] +Condition : isnotnull(ss_item_sk#4) + +(7) ColumnarToRow [codegen id : 1] +Input [3]: [ss_item_sk#4, ss_ext_sales_price#5, ss_sold_date_sk#6] + +(8) BroadcastExchange +Input [3]: [ss_item_sk#4, ss_ext_sales_price#5, ss_sold_date_sk#6] +Arguments: HashedRelationBroadcastMode(List(cast(input[2, int, true] as bigint)),false), [plan_id=1] + +(9) BroadcastHashJoin [codegen id : 3] +Left keys [1]: [d_date_sk#1] +Right keys [1]: [ss_sold_date_sk#6] +Join type: Inner +Join condition: None + +(10) Project [codegen id : 3] +Output [3]: [d_year#2, ss_item_sk#4, ss_ext_sales_price#5] +Input [5]: [d_date_sk#1, d_year#2, ss_item_sk#4, ss_ext_sales_price#5, ss_sold_date_sk#6] + +(11) Scan parquet spark_catalog.default.item +Output [4]: [i_item_sk#7, i_brand_id#8, i_brand#9, i_manufact_id#10] +Batched: true +Location [not included in comparison]/{warehouse_dir}/item] +PushedFilters: [IsNotNull(i_manufact_id), EqualTo(i_manufact_id,128), IsNotNull(i_item_sk)] +ReadSchema: struct<i_item_sk:int,i_brand_id:int,i_brand:string,i_manufact_id:int> + +(12) CometFilter +Input [4]: [i_item_sk#7, i_brand_id#8, i_brand#9, i_manufact_id#10] +Condition : ((isnotnull(i_manufact_id#10) AND (i_manufact_id#10 = 128)) AND isnotnull(i_item_sk#7)) + +(13) CometProject +Input [4]: [i_item_sk#7, i_brand_id#8, i_brand#9, i_manufact_id#10] +Arguments: [i_item_sk#7, i_brand_id#8, i_brand#9], [i_item_sk#7, i_brand_id#8, i_brand#9] + +(14) ColumnarToRow [codegen id : 2] +Input [3]: [i_item_sk#7, i_brand_id#8, i_brand#9] + +(15) BroadcastExchange +Input [3]: [i_item_sk#7, i_brand_id#8, i_brand#9] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] + +(16) BroadcastHashJoin [codegen id : 3] +Left keys [1]: [ss_item_sk#4] +Right keys [1]: [i_item_sk#7] +Join type: Inner +Join condition: None + +(17) Project [codegen id : 3] +Output [4]: [d_year#2, ss_ext_sales_price#5, i_brand_id#8, i_brand#9] +Input [6]: [d_year#2, ss_item_sk#4, ss_ext_sales_price#5, i_item_sk#7, i_brand_id#8, i_brand#9] + +(18) HashAggregate [codegen id : 3] +Input [4]: [d_year#2, ss_ext_sales_price#5, i_brand_id#8, i_brand#9] +Keys [3]: [d_year#2, i_brand#9, i_brand_id#8] +Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#5))] +Aggregate Attributes [1]: [sum#11] +Results [4]: [d_year#2, i_brand#9, i_brand_id#8, sum#12] + +(19) Exchange +Input [4]: [d_year#2, i_brand#9, i_brand_id#8, sum#12] +Arguments: hashpartitioning(d_year#2, i_brand#9, i_brand_id#8, 5), ENSURE_REQUIREMENTS, [plan_id=3] + +(20) HashAggregate [codegen id : 4] +Input [4]: [d_year#2, i_brand#9, i_brand_id#8, sum#12] +Keys [3]: [d_year#2, i_brand#9, i_brand_id#8] +Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#5))] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#5))#13] +Results [4]: [d_year#2, i_brand_id#8 AS brand_id#14, i_brand#9 AS brand#15, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#5))#13,17,2) AS sum_agg#16] + +(21) TakeOrderedAndProject +Input [4]: [d_year#2, brand_id#14, brand#15, sum_agg#16] +Arguments: 100, [d_year#2 ASC NULLS FIRST, sum_agg#16 DESC NULLS LAST, brand_id#14 ASC NULLS FIRST], [d_year#2, brand_id#14, brand#15, sum_agg#16] + diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q3/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q3/simplified.txt new file mode 100644 index 000000000..3946c0cd8 --- /dev/null +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q3/simplified.txt @@ -0,0 +1,31 @@ +TakeOrderedAndProject [d_year,sum_agg,brand_id,brand] + WholeStageCodegen (4) + HashAggregate [d_year,i_brand,i_brand_id,sum] [sum(UnscaledValue(ss_ext_sales_price)),brand_id,brand,sum_agg,sum] + InputAdapter + Exchange [d_year,i_brand,i_brand_id] #1 + WholeStageCodegen (3) + HashAggregate [d_year,i_brand,i_brand_id,ss_ext_sales_price] [sum,sum] + Project [d_year,ss_ext_sales_price,i_brand_id,i_brand] + BroadcastHashJoin [ss_item_sk,i_item_sk] + Project [d_year,ss_item_sk,ss_ext_sales_price] + BroadcastHashJoin [d_date_sk,ss_sold_date_sk] + ColumnarToRow + InputAdapter + CometProject [d_date_sk,d_year] + CometFilter [d_moy,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + InputAdapter + BroadcastExchange #2 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometFilter [ss_item_sk] + CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] + InputAdapter + BroadcastExchange #3 + WholeStageCodegen (2) + ColumnarToRow + InputAdapter + CometProject [i_item_sk,i_brand_id,i_brand] + CometFilter [i_manufact_id,i_item_sk] + CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_brand,i_manufact_id] 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 new file mode 100644 index 000000000..098d00824 --- /dev/null +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q30/explain.txt @@ -0,0 +1,324 @@ +== Physical Plan == +TakeOrderedAndProject (49) ++- * Project (48) + +- * BroadcastHashJoin Inner BuildRight (47) + :- * Project (41) + : +- * BroadcastHashJoin Inner BuildRight (40) + : :- * Project (35) + : : +- * BroadcastHashJoin Inner BuildRight (34) + : : :- * Filter (16) + : : : +- * HashAggregate (15) + : : : +- Exchange (14) + : : : +- * HashAggregate (13) + : : : +- * Project (12) + : : : +- * BroadcastHashJoin Inner BuildRight (11) + : : : :- * Project (6) + : : : : +- * BroadcastHashJoin Inner BuildRight (5) + : : : : :- * ColumnarToRow (3) + : : : : : +- CometFilter (2) + : : : : : +- CometScan parquet spark_catalog.default.web_returns (1) + : : : : +- ReusedExchange (4) + : : : +- BroadcastExchange (10) + : : : +- * ColumnarToRow (9) + : : : +- CometFilter (8) + : : : +- CometScan parquet spark_catalog.default.customer_address (7) + : : +- BroadcastExchange (33) + : : +- * Filter (32) + : : +- * HashAggregate (31) + : : +- Exchange (30) + : : +- * HashAggregate (29) + : : +- * HashAggregate (28) + : : +- Exchange (27) + : : +- * HashAggregate (26) + : : +- * Project (25) + : : +- * BroadcastHashJoin Inner BuildRight (24) + : : :- * Project (22) + : : : +- * BroadcastHashJoin Inner BuildRight (21) + : : : :- * ColumnarToRow (19) + : : : : +- CometFilter (18) + : : : : +- CometScan parquet spark_catalog.default.web_returns (17) + : : : +- ReusedExchange (20) + : : +- ReusedExchange (23) + : +- BroadcastExchange (39) + : +- * ColumnarToRow (38) + : +- CometFilter (37) + : +- CometScan parquet spark_catalog.default.customer (36) + +- BroadcastExchange (46) + +- * ColumnarToRow (45) + +- CometProject (44) + +- CometFilter (43) + +- CometScan parquet spark_catalog.default.customer_address (42) + + +(1) Scan parquet spark_catalog.default.web_returns +Output [4]: [wr_returning_customer_sk#1, wr_returning_addr_sk#2, wr_return_amt#3, wr_returned_date_sk#4] +Batched: true +Location: InMemoryFileIndex [] +PartitionFilters: [isnotnull(wr_returned_date_sk#4), dynamicpruningexpression(wr_returned_date_sk#4 IN dynamicpruning#5)] +PushedFilters: [IsNotNull(wr_returning_addr_sk), IsNotNull(wr_returning_customer_sk)] +ReadSchema: struct<wr_returning_customer_sk:int,wr_returning_addr_sk:int,wr_return_amt:decimal(7,2)> + +(2) CometFilter +Input [4]: [wr_returning_customer_sk#1, wr_returning_addr_sk#2, wr_return_amt#3, wr_returned_date_sk#4] +Condition : (isnotnull(wr_returning_addr_sk#2) AND isnotnull(wr_returning_customer_sk#1)) + +(3) ColumnarToRow [codegen id : 3] +Input [4]: [wr_returning_customer_sk#1, wr_returning_addr_sk#2, wr_return_amt#3, wr_returned_date_sk#4] + +(4) ReusedExchange [Reuses operator id: 54] +Output [1]: [d_date_sk#6] + +(5) BroadcastHashJoin [codegen id : 3] +Left keys [1]: [wr_returned_date_sk#4] +Right keys [1]: [d_date_sk#6] +Join type: Inner +Join condition: None + +(6) Project [codegen id : 3] +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] + +(7) Scan parquet spark_catalog.default.customer_address +Output [2]: [ca_address_sk#7, ca_state#8] +Batched: true +Location [not included in comparison]/{warehouse_dir}/customer_address] +PushedFilters: [IsNotNull(ca_address_sk), IsNotNull(ca_state)] +ReadSchema: struct<ca_address_sk:int,ca_state:string> + +(8) CometFilter +Input [2]: [ca_address_sk#7, ca_state#8] +Condition : (isnotnull(ca_address_sk#7) AND isnotnull(ca_state#8)) + +(9) ColumnarToRow [codegen id : 2] +Input [2]: [ca_address_sk#7, ca_state#8] + +(10) BroadcastExchange +Input [2]: [ca_address_sk#7, ca_state#8] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=1] + +(11) BroadcastHashJoin [codegen id : 3] +Left keys [1]: [wr_returning_addr_sk#2] +Right keys [1]: [ca_address_sk#7] +Join type: Inner +Join condition: None + +(12) Project [codegen id : 3] +Output [3]: [wr_returning_customer_sk#1, wr_return_amt#3, ca_state#8] +Input [5]: [wr_returning_customer_sk#1, wr_returning_addr_sk#2, wr_return_amt#3, ca_address_sk#7, ca_state#8] + +(13) HashAggregate [codegen id : 3] +Input [3]: [wr_returning_customer_sk#1, wr_return_amt#3, ca_state#8] +Keys [2]: [wr_returning_customer_sk#1, ca_state#8] +Functions [1]: [partial_sum(UnscaledValue(wr_return_amt#3))] +Aggregate Attributes [1]: [sum#9] +Results [3]: [wr_returning_customer_sk#1, ca_state#8, sum#10] + +(14) Exchange +Input [3]: [wr_returning_customer_sk#1, ca_state#8, sum#10] +Arguments: hashpartitioning(wr_returning_customer_sk#1, ca_state#8, 5), ENSURE_REQUIREMENTS, [plan_id=2] + +(15) HashAggregate [codegen id : 11] +Input [3]: [wr_returning_customer_sk#1, ca_state#8, sum#10] +Keys [2]: [wr_returning_customer_sk#1, ca_state#8] +Functions [1]: [sum(UnscaledValue(wr_return_amt#3))] +Aggregate Attributes [1]: [sum(UnscaledValue(wr_return_amt#3))#11] +Results [3]: [wr_returning_customer_sk#1 AS ctr_customer_sk#12, ca_state#8 AS ctr_state#13, MakeDecimal(sum(UnscaledValue(wr_return_amt#3))#11,17,2) AS ctr_total_return#14] + +(16) Filter [codegen id : 11] +Input [3]: [ctr_customer_sk#12, ctr_state#13, ctr_total_return#14] +Condition : isnotnull(ctr_total_return#14) + +(17) Scan parquet spark_catalog.default.web_returns +Output [4]: [wr_returning_customer_sk#15, wr_returning_addr_sk#16, wr_return_amt#17, wr_returned_date_sk#18] +Batched: true +Location: InMemoryFileIndex [] +PartitionFilters: [isnotnull(wr_returned_date_sk#18), dynamicpruningexpression(wr_returned_date_sk#18 IN dynamicpruning#19)] +PushedFilters: [IsNotNull(wr_returning_addr_sk)] +ReadSchema: struct<wr_returning_customer_sk:int,wr_returning_addr_sk:int,wr_return_amt:decimal(7,2)> + +(18) CometFilter +Input [4]: [wr_returning_customer_sk#15, wr_returning_addr_sk#16, wr_return_amt#17, wr_returned_date_sk#18] +Condition : isnotnull(wr_returning_addr_sk#16) + +(19) ColumnarToRow [codegen id : 6] +Input [4]: [wr_returning_customer_sk#15, wr_returning_addr_sk#16, wr_return_amt#17, wr_returned_date_sk#18] + +(20) ReusedExchange [Reuses operator id: 54] +Output [1]: [d_date_sk#20] + +(21) BroadcastHashJoin [codegen id : 6] +Left keys [1]: [wr_returned_date_sk#18] +Right keys [1]: [d_date_sk#20] +Join type: Inner +Join condition: None + +(22) Project [codegen id : 6] +Output [3]: [wr_returning_customer_sk#15, wr_returning_addr_sk#16, wr_return_amt#17] +Input [5]: [wr_returning_customer_sk#15, wr_returning_addr_sk#16, wr_return_amt#17, wr_returned_date_sk#18, d_date_sk#20] + +(23) ReusedExchange [Reuses operator id: 10] +Output [2]: [ca_address_sk#21, ca_state#22] + +(24) BroadcastHashJoin [codegen id : 6] +Left keys [1]: [wr_returning_addr_sk#16] +Right keys [1]: [ca_address_sk#21] +Join type: Inner +Join condition: None + +(25) Project [codegen id : 6] +Output [3]: [wr_returning_customer_sk#15, wr_return_amt#17, ca_state#22] +Input [5]: [wr_returning_customer_sk#15, wr_returning_addr_sk#16, wr_return_amt#17, ca_address_sk#21, ca_state#22] + +(26) HashAggregate [codegen id : 6] +Input [3]: [wr_returning_customer_sk#15, wr_return_amt#17, ca_state#22] +Keys [2]: [wr_returning_customer_sk#15, ca_state#22] +Functions [1]: [partial_sum(UnscaledValue(wr_return_amt#17))] +Aggregate Attributes [1]: [sum#23] +Results [3]: [wr_returning_customer_sk#15, ca_state#22, sum#24] + +(27) Exchange +Input [3]: [wr_returning_customer_sk#15, ca_state#22, sum#24] +Arguments: hashpartitioning(wr_returning_customer_sk#15, ca_state#22, 5), ENSURE_REQUIREMENTS, [plan_id=3] + +(28) HashAggregate [codegen id : 7] +Input [3]: [wr_returning_customer_sk#15, ca_state#22, sum#24] +Keys [2]: [wr_returning_customer_sk#15, ca_state#22] +Functions [1]: [sum(UnscaledValue(wr_return_amt#17))] +Aggregate Attributes [1]: [sum(UnscaledValue(wr_return_amt#17))#11] +Results [2]: [ca_state#22 AS ctr_state#25, MakeDecimal(sum(UnscaledValue(wr_return_amt#17))#11,17,2) AS ctr_total_return#26] + +(29) HashAggregate [codegen id : 7] +Input [2]: [ctr_state#25, ctr_total_return#26] +Keys [1]: [ctr_state#25] +Functions [1]: [partial_avg(ctr_total_return#26)] +Aggregate Attributes [2]: [sum#27, count#28] +Results [3]: [ctr_state#25, sum#29, count#30] + +(30) Exchange +Input [3]: [ctr_state#25, sum#29, count#30] +Arguments: hashpartitioning(ctr_state#25, 5), ENSURE_REQUIREMENTS, [plan_id=4] + +(31) HashAggregate [codegen id : 8] +Input [3]: [ctr_state#25, sum#29, count#30] +Keys [1]: [ctr_state#25] +Functions [1]: [avg(ctr_total_return#26)] +Aggregate Attributes [1]: [avg(ctr_total_return#26)#31] +Results [2]: [(avg(ctr_total_return#26)#31 * 1.2) AS (avg(ctr_total_return) * 1.2)#32, ctr_state#25] + +(32) Filter [codegen id : 8] +Input [2]: [(avg(ctr_total_return) * 1.2)#32, ctr_state#25] +Condition : isnotnull((avg(ctr_total_return) * 1.2)#32) + +(33) BroadcastExchange +Input [2]: [(avg(ctr_total_return) * 1.2)#32, ctr_state#25] +Arguments: HashedRelationBroadcastMode(List(input[1, string, true]),false), [plan_id=5] + +(34) BroadcastHashJoin [codegen id : 11] +Left keys [1]: [ctr_state#13] +Right keys [1]: [ctr_state#25] +Join type: Inner +Join condition: (cast(ctr_total_return#14 as decimal(24,7)) > (avg(ctr_total_return) * 1.2)#32) + +(35) Project [codegen id : 11] +Output [2]: [ctr_customer_sk#12, ctr_total_return#14] +Input [5]: [ctr_customer_sk#12, ctr_state#13, ctr_total_return#14, (avg(ctr_total_return) * 1.2)#32, ctr_state#25] + +(36) Scan parquet spark_catalog.default.customer +Output [14]: [c_customer_sk#33, c_customer_id#34, c_current_addr_sk#35, c_salutation#36, c_first_name#37, c_last_name#38, c_preferred_cust_flag#39, c_birth_day#40, c_birth_month#41, c_birth_year#42, c_birth_country#43, c_login#44, c_email_address#45, c_last_review_date#46] +Batched: true +Location [not included in comparison]/{warehouse_dir}/customer] +PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_current_addr_sk)] +ReadSchema: struct<c_customer_sk:int,c_customer_id:string,c_current_addr_sk:int,c_salutation:string,c_first_name:string,c_last_name:string,c_preferred_cust_flag:string,c_birth_day:int,c_birth_month:int,c_birth_year:int,c_birth_country:string,c_login:string,c_email_address:string,c_last_review_date:int> + +(37) CometFilter +Input [14]: [c_customer_sk#33, c_customer_id#34, c_current_addr_sk#35, c_salutation#36, c_first_name#37, c_last_name#38, c_preferred_cust_flag#39, c_birth_day#40, c_birth_month#41, c_birth_year#42, c_birth_country#43, c_login#44, c_email_address#45, c_last_review_date#46] +Condition : (isnotnull(c_customer_sk#33) AND isnotnull(c_current_addr_sk#35)) + +(38) ColumnarToRow [codegen id : 9] +Input [14]: [c_customer_sk#33, c_customer_id#34, c_current_addr_sk#35, c_salutation#36, c_first_name#37, c_last_name#38, c_preferred_cust_flag#39, c_birth_day#40, c_birth_month#41, c_birth_year#42, c_birth_country#43, c_login#44, c_email_address#45, c_last_review_date#46] + +(39) BroadcastExchange +Input [14]: [c_customer_sk#33, c_customer_id#34, c_current_addr_sk#35, c_salutation#36, c_first_name#37, c_last_name#38, c_preferred_cust_flag#39, c_birth_day#40, c_birth_month#41, c_birth_year#42, c_birth_country#43, c_login#44, c_email_address#45, c_last_review_date#46] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=6] + +(40) BroadcastHashJoin [codegen id : 11] +Left keys [1]: [ctr_customer_sk#12] +Right keys [1]: [c_customer_sk#33] +Join type: Inner +Join condition: None + +(41) Project [codegen id : 11] +Output [14]: [ctr_total_return#14, c_customer_id#34, c_current_addr_sk#35, c_salutation#36, c_first_name#37, c_last_name#38, c_preferred_cust_flag#39, c_birth_day#40, c_birth_month#41, c_birth_year#42, c_birth_country#43, c_login#44, c_email_address#45, c_last_review_date#46] +Input [16]: [ctr_customer_sk#12, ctr_total_return#14, c_customer_sk#33, c_customer_id#34, c_current_addr_sk#35, c_salutation#36, c_first_name#37, c_last_name#38, c_preferred_cust_flag#39, c_birth_day#40, c_birth_month#41, c_birth_year#42, c_birth_country#43, c_login#44, c_email_address#45, c_last_review_date#46] + +(42) Scan parquet spark_catalog.default.customer_address +Output [2]: [ca_address_sk#47, ca_state#48] +Batched: true +Location [not included in comparison]/{warehouse_dir}/customer_address] +PushedFilters: [IsNotNull(ca_state), EqualTo(ca_state,GA), IsNotNull(ca_address_sk)] +ReadSchema: struct<ca_address_sk:int,ca_state:string> + +(43) CometFilter +Input [2]: [ca_address_sk#47, ca_state#48] +Condition : ((isnotnull(ca_state#48) AND (ca_state#48 = GA)) AND isnotnull(ca_address_sk#47)) + +(44) CometProject +Input [2]: [ca_address_sk#47, ca_state#48] +Arguments: [ca_address_sk#47], [ca_address_sk#47] + +(45) ColumnarToRow [codegen id : 10] +Input [1]: [ca_address_sk#47] + +(46) BroadcastExchange +Input [1]: [ca_address_sk#47] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=7] + +(47) BroadcastHashJoin [codegen id : 11] +Left keys [1]: [c_current_addr_sk#35] +Right keys [1]: [ca_address_sk#47] +Join type: Inner +Join condition: None + +(48) Project [codegen id : 11] +Output [13]: [c_customer_id#34, c_salutation#36, c_first_name#37, c_last_name#38, c_preferred_cust_flag#39, c_birth_day#40, c_birth_month#41, c_birth_year#42, c_birth_country#43, c_login#44, c_email_address#45, c_last_review_date#46, ctr_total_return#14] +Input [15]: [ctr_total_return#14, c_customer_id#34, c_current_addr_sk#35, c_salutation#36, c_first_name#37, c_last_name#38, c_preferred_cust_flag#39, c_birth_day#40, c_birth_month#41, c_birth_year#42, c_birth_country#43, c_login#44, c_email_address#45, c_last_review_date#46, ca_address_sk#47] + +(49) TakeOrderedAndProject +Input [13]: [c_customer_id#34, c_salutation#36, c_first_name#37, c_last_name#38, c_preferred_cust_flag#39, c_birth_day#40, c_birth_month#41, c_birth_year#42, c_birth_country#43, c_login#44, c_email_address#45, c_last_review_date#46, ctr_total_return#14] +Arguments: 100, [c_customer_id#34 ASC NULLS FIRST, c_salutation#36 ASC NULLS FIRST, c_first_name#37 ASC NULLS FIRST, c_last_name#38 ASC NULLS FIRST, c_preferred_cust_flag#39 ASC NULLS FIRST, c_birth_day#40 ASC NULLS FIRST, c_birth_month#41 ASC NULLS FIRST, c_birth_year#42 ASC NULLS FIRST, c_birth_country#43 ASC NULLS FIRST, c_login#44 ASC NULLS FIRST, c_email_address#45 ASC NULLS FIRST, c_last_review_date#46 ASC NULLS FIRST, ctr_total_return#14 ASC NULLS FIRST], [c_customer_id#34, c_salutation#36, c_first_name#37, c_last_name#38, c_preferred_cust_flag#39, c_birth_day#40, c_birth_month#41, c_birth_year#42, c_birth_country#43, c_login#44, c_email_address#45, c_last_review_date#46, ctr_total_return#14] + +===== Subqueries ===== + +Subquery:1 Hosting operator id = 1 Hosting Expression = wr_returned_date_sk#4 IN dynamicpruning#5 +BroadcastExchange (54) ++- * ColumnarToRow (53) + +- CometProject (52) + +- CometFilter (51) + +- CometScan parquet spark_catalog.default.date_dim (50) + + +(50) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#6, d_year#49] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2002), IsNotNull(d_date_sk)] +ReadSchema: struct<d_date_sk:int,d_year:int> + +(51) CometFilter +Input [2]: [d_date_sk#6, d_year#49] +Condition : ((isnotnull(d_year#49) AND (d_year#49 = 2002)) AND isnotnull(d_date_sk#6)) + +(52) CometProject +Input [2]: [d_date_sk#6, d_year#49] +Arguments: [d_date_sk#6], [d_date_sk#6] + +(53) ColumnarToRow [codegen id : 1] +Input [1]: [d_date_sk#6] + +(54) BroadcastExchange +Input [1]: [d_date_sk#6] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=8] + +Subquery:2 Hosting operator id = 17 Hosting Expression = wr_returned_date_sk#18 IN dynamicpruning#5 + + diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q30/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q30/simplified.txt new file mode 100644 index 000000000..365f7f973 --- /dev/null +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q30/simplified.txt @@ -0,0 +1,81 @@ +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) + Project [c_customer_id,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_day,c_birth_month,c_birth_year,c_birth_country,c_login,c_email_address,c_last_review_date,ctr_total_return] + BroadcastHashJoin [c_current_addr_sk,ca_address_sk] + Project [ctr_total_return,c_customer_id,c_current_addr_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_day,c_birth_month,c_birth_year,c_birth_country,c_login,c_email_address,c_last_review_date] + BroadcastHashJoin [ctr_customer_sk,c_customer_sk] + Project [ctr_customer_sk,ctr_total_return] + BroadcastHashJoin [ctr_state,ctr_state,ctr_total_return,(avg(ctr_total_return) * 1.2)] + Filter [ctr_total_return] + HashAggregate [wr_returning_customer_sk,ca_state,sum] [sum(UnscaledValue(wr_return_amt)),ctr_customer_sk,ctr_state,ctr_total_return,sum] + InputAdapter + Exchange [wr_returning_customer_sk,ca_state] #1 + WholeStageCodegen (3) + HashAggregate [wr_returning_customer_sk,ca_state,wr_return_amt] [sum,sum] + Project [wr_returning_customer_sk,wr_return_amt,ca_state] + BroadcastHashJoin [wr_returning_addr_sk,ca_address_sk] + Project [wr_returning_customer_sk,wr_returning_addr_sk,wr_return_amt] + BroadcastHashJoin [wr_returned_date_sk,d_date_sk] + ColumnarToRow + InputAdapter + CometFilter [wr_returning_addr_sk,wr_returning_customer_sk] + CometScan parquet spark_catalog.default.web_returns [wr_returning_customer_sk,wr_returning_addr_sk,wr_return_amt,wr_returned_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #2 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_year,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + InputAdapter + ReusedExchange [d_date_sk] #2 + InputAdapter + BroadcastExchange #3 + WholeStageCodegen (2) + ColumnarToRow + InputAdapter + CometFilter [ca_address_sk,ca_state] + CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] + InputAdapter + BroadcastExchange #4 + WholeStageCodegen (8) + Filter [(avg(ctr_total_return) * 1.2)] + HashAggregate [ctr_state,sum,count] [avg(ctr_total_return),(avg(ctr_total_return) * 1.2),sum,count] + InputAdapter + Exchange [ctr_state] #5 + WholeStageCodegen (7) + HashAggregate [ctr_state,ctr_total_return] [sum,count,sum,count] + HashAggregate [wr_returning_customer_sk,ca_state,sum] [sum(UnscaledValue(wr_return_amt)),ctr_state,ctr_total_return,sum] + InputAdapter + Exchange [wr_returning_customer_sk,ca_state] #6 + WholeStageCodegen (6) + HashAggregate [wr_returning_customer_sk,ca_state,wr_return_amt] [sum,sum] + Project [wr_returning_customer_sk,wr_return_amt,ca_state] + BroadcastHashJoin [wr_returning_addr_sk,ca_address_sk] + Project [wr_returning_customer_sk,wr_returning_addr_sk,wr_return_amt] + BroadcastHashJoin [wr_returned_date_sk,d_date_sk] + ColumnarToRow + InputAdapter + CometFilter [wr_returning_addr_sk] + CometScan parquet spark_catalog.default.web_returns [wr_returning_customer_sk,wr_returning_addr_sk,wr_return_amt,wr_returned_date_sk] + ReusedSubquery [d_date_sk] #1 + InputAdapter + ReusedExchange [d_date_sk] #2 + InputAdapter + ReusedExchange [ca_address_sk,ca_state] #3 + InputAdapter + BroadcastExchange #7 + WholeStageCodegen (9) + ColumnarToRow + InputAdapter + CometFilter [c_customer_sk,c_current_addr_sk] + CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_current_addr_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_day,c_birth_month,c_birth_year,c_birth_country,c_login,c_email_address,c_last_review_date] + InputAdapter + BroadcastExchange #8 + WholeStageCodegen (10) + ColumnarToRow + InputAdapter + CometProject [ca_address_sk] + CometFilter [ca_state,ca_address_sk] + CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q31/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q31/explain.txt new file mode 100644 index 000000000..e20d45486 --- /dev/null +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q31/explain.txt @@ -0,0 +1,616 @@ +== Physical Plan == +* Sort (90) ++- Exchange (89) + +- * Project (88) + +- * BroadcastHashJoin Inner BuildRight (87) + :- * Project (73) + : +- * BroadcastHashJoin Inner BuildRight (72) + : :- * BroadcastHashJoin Inner BuildRight (58) + : : :- * Project (44) + : : : +- * BroadcastHashJoin Inner BuildRight (43) + : : : :- * BroadcastHashJoin Inner BuildRight (29) + : : : : :- * HashAggregate (15) + : : : : : +- Exchange (14) + : : : : : +- * HashAggregate (13) + : : : : : +- * Project (12) + : : : : : +- * BroadcastHashJoin Inner BuildRight (11) + : : : : : :- * Project (6) + : : : : : : +- * BroadcastHashJoin Inner BuildRight (5) + : : : : : : :- * ColumnarToRow (3) + : : : : : : : +- CometFilter (2) + : : : : : : : +- CometScan parquet spark_catalog.default.store_sales (1) + : : : : : : +- ReusedExchange (4) + : : : : : +- BroadcastExchange (10) + : : : : : +- * ColumnarToRow (9) + : : : : : +- CometFilter (8) + : : : : : +- CometScan parquet spark_catalog.default.customer_address (7) + : : : : +- BroadcastExchange (28) + : : : : +- * HashAggregate (27) + : : : : +- Exchange (26) + : : : : +- * HashAggregate (25) + : : : : +- * Project (24) + : : : : +- * BroadcastHashJoin Inner BuildRight (23) + : : : : :- * Project (21) + : : : : : +- * BroadcastHashJoin Inner BuildRight (20) + : : : : : :- * ColumnarToRow (18) + : : : : : : +- CometFilter (17) + : : : : : : +- CometScan parquet spark_catalog.default.store_sales (16) + : : : : : +- ReusedExchange (19) + : : : : +- ReusedExchange (22) + : : : +- BroadcastExchange (42) + : : : +- * HashAggregate (41) + : : : +- Exchange (40) + : : : +- * HashAggregate (39) + : : : +- * Project (38) + : : : +- * BroadcastHashJoin Inner BuildRight (37) + : : : :- * Project (35) + : : : : +- * BroadcastHashJoin Inner BuildRight (34) + : : : : :- * ColumnarToRow (32) + : : : : : +- CometFilter (31) + : : : : : +- CometScan parquet spark_catalog.default.store_sales (30) + : : : : +- ReusedExchange (33) + : : : +- ReusedExchange (36) + : : +- BroadcastExchange (57) + : : +- * HashAggregate (56) + : : +- Exchange (55) + : : +- * HashAggregate (54) + : : +- * Project (53) + : : +- * BroadcastHashJoin Inner BuildRight (52) + : : :- * Project (50) + : : : +- * BroadcastHashJoin Inner BuildRight (49) + : : : :- * ColumnarToRow (47) + : : : : +- CometFilter (46) + : : : : +- CometScan parquet spark_catalog.default.web_sales (45) + : : : +- ReusedExchange (48) + : : +- ReusedExchange (51) + : +- BroadcastExchange (71) + : +- * HashAggregate (70) + : +- Exchange (69) + : +- * HashAggregate (68) + : +- * Project (67) + : +- * BroadcastHashJoin Inner BuildRight (66) + : :- * Project (64) + : : +- * BroadcastHashJoin Inner BuildRight (63) + : : :- * ColumnarToRow (61) + : : : +- CometFilter (60) + : : : +- CometScan parquet spark_catalog.default.web_sales (59) + : : +- ReusedExchange (62) + : +- ReusedExchange (65) + +- BroadcastExchange (86) + +- * HashAggregate (85) + +- Exchange (84) + +- * HashAggregate (83) + +- * Project (82) + +- * BroadcastHashJoin Inner BuildRight (81) + :- * Project (79) + : +- * BroadcastHashJoin Inner BuildRight (78) + : :- * ColumnarToRow (76) + : : +- CometFilter (75) + : : +- CometScan parquet spark_catalog.default.web_sales (74) + : +- ReusedExchange (77) + +- ReusedExchange (80) + + +(1) Scan parquet spark_catalog.default.store_sales +Output [3]: [ss_addr_sk#1, ss_ext_sales_price#2, ss_sold_date_sk#3] +Batched: true +Location: InMemoryFileIndex [] +PartitionFilters: [isnotnull(ss_sold_date_sk#3), dynamicpruningexpression(ss_sold_date_sk#3 IN dynamicpruning#4)] +PushedFilters: [IsNotNull(ss_addr_sk)] +ReadSchema: struct<ss_addr_sk:int,ss_ext_sales_price:decimal(7,2)> + +(2) CometFilter +Input [3]: [ss_addr_sk#1, ss_ext_sales_price#2, ss_sold_date_sk#3] +Condition : isnotnull(ss_addr_sk#1) + +(3) ColumnarToRow [codegen id : 3] +Input [3]: [ss_addr_sk#1, ss_ext_sales_price#2, ss_sold_date_sk#3] + +(4) ReusedExchange [Reuses operator id: 94] +Output [3]: [d_date_sk#5, d_year#6, d_qoy#7] + +(5) BroadcastHashJoin [codegen id : 3] +Left keys [1]: [ss_sold_date_sk#3] +Right keys [1]: [d_date_sk#5] +Join type: Inner +Join condition: None + +(6) Project [codegen id : 3] +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] + +(7) Scan parquet spark_catalog.default.customer_address +Output [2]: [ca_address_sk#8, ca_county#9] +Batched: true +Location [not included in comparison]/{warehouse_dir}/customer_address] +PushedFilters: [IsNotNull(ca_address_sk), IsNotNull(ca_county)] +ReadSchema: struct<ca_address_sk:int,ca_county:string> + +(8) CometFilter +Input [2]: [ca_address_sk#8, ca_county#9] +Condition : (isnotnull(ca_address_sk#8) AND isnotnull(ca_county#9)) + +(9) ColumnarToRow [codegen id : 2] +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] +Left keys [1]: [ss_addr_sk#1] +Right keys [1]: [ca_address_sk#8] +Join type: Inner +Join condition: None + +(12) Project [codegen id : 3] +Output [4]: [ss_ext_sales_price#2, d_year#6, d_qoy#7, ca_county#9] +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] +Input [4]: [ss_ext_sales_price#2, d_year#6, d_qoy#7, ca_county#9] +Keys [3]: [ca_county#9, d_qoy#7, d_year#6] +Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#2))] +Aggregate Attributes [1]: [sum#10] +Results [4]: [ca_county#9, d_qoy#7, d_year#6, sum#11] + +(14) Exchange +Input [4]: [ca_county#9, d_qoy#7, d_year#6, sum#11] +Arguments: hashpartitioning(ca_county#9, d_qoy#7, d_year#6, 5), ENSURE_REQUIREMENTS, [plan_id=2] + +(15) HashAggregate [codegen id : 24] +Input [4]: [ca_county#9, d_qoy#7, d_year#6, sum#11] +Keys [3]: [ca_county#9, d_qoy#7, d_year#6] +Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#2))] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#2))#12] +Results [3]: [ca_county#9, d_year#6, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#2))#12,17,2) AS store_sales#13] + +(16) Scan parquet spark_catalog.default.store_sales +Output [3]: [ss_addr_sk#14, ss_ext_sales_price#15, ss_sold_date_sk#16] +Batched: true +Location: InMemoryFileIndex [] +PartitionFilters: [isnotnull(ss_sold_date_sk#16), dynamicpruningexpression(ss_sold_date_sk#16 IN dynamicpruning#17)] +PushedFilters: [IsNotNull(ss_addr_sk)] +ReadSchema: struct<ss_addr_sk:int,ss_ext_sales_price:decimal(7,2)> + +(17) CometFilter +Input [3]: [ss_addr_sk#14, ss_ext_sales_price#15, ss_sold_date_sk#16] +Condition : isnotnull(ss_addr_sk#14) + +(18) ColumnarToRow [codegen id : 6] +Input [3]: [ss_addr_sk#14, ss_ext_sales_price#15, ss_sold_date_sk#16] + +(19) ReusedExchange [Reuses operator id: 98] +Output [3]: [d_date_sk#18, d_year#19, d_qoy#20] + +(20) BroadcastHashJoin [codegen id : 6] +Left keys [1]: [ss_sold_date_sk#16] +Right keys [1]: [d_date_sk#18] +Join type: Inner +Join condition: None + +(21) Project [codegen id : 6] +Output [4]: [ss_addr_sk#14, ss_ext_sales_price#15, d_year#19, d_qoy#20] +Input [6]: [ss_addr_sk#14, ss_ext_sales_price#15, ss_sold_date_sk#16, d_date_sk#18, d_year#19, d_qoy#20] + +(22) ReusedExchange [Reuses operator id: 10] +Output [2]: [ca_address_sk#21, ca_county#22] + +(23) BroadcastHashJoin [codegen id : 6] +Left keys [1]: [ss_addr_sk#14] +Right keys [1]: [ca_address_sk#21] +Join type: Inner +Join condition: None + +(24) Project [codegen id : 6] +Output [4]: [ss_ext_sales_price#15, d_year#19, d_qoy#20, ca_county#22] +Input [6]: [ss_addr_sk#14, ss_ext_sales_price#15, d_year#19, d_qoy#20, ca_address_sk#21, ca_county#22] + +(25) HashAggregate [codegen id : 6] +Input [4]: [ss_ext_sales_price#15, d_year#19, d_qoy#20, ca_county#22] +Keys [3]: [ca_county#22, d_qoy#20, d_year#19] +Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#15))] +Aggregate Attributes [1]: [sum#23] +Results [4]: [ca_county#22, d_qoy#20, d_year#19, sum#24] + +(26) Exchange +Input [4]: [ca_county#22, d_qoy#20, d_year#19, sum#24] +Arguments: hashpartitioning(ca_county#22, d_qoy#20, d_year#19, 5), ENSURE_REQUIREMENTS, [plan_id=3] + +(27) HashAggregate [codegen id : 7] +Input [4]: [ca_county#22, d_qoy#20, d_year#19, sum#24] +Keys [3]: [ca_county#22, d_qoy#20, d_year#19] +Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#15))] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#15))#12] +Results [2]: [ca_county#22, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#15))#12,17,2) AS store_sales#25] + +(28) BroadcastExchange +Input [2]: [ca_county#22, store_sales#25] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=4] + +(29) BroadcastHashJoin [codegen id : 24] +Left keys [1]: [ca_county#9] +Right keys [1]: [ca_county#22] +Join type: Inner +Join condition: None + +(30) Scan parquet spark_catalog.default.store_sales +Output [3]: [ss_addr_sk#26, ss_ext_sales_price#27, ss_sold_date_sk#28] +Batched: true +Location: InMemoryFileIndex [] +PartitionFilters: [isnotnull(ss_sold_date_sk#28), dynamicpruningexpression(ss_sold_date_sk#28 IN dynamicpruning#29)] +PushedFilters: [IsNotNull(ss_addr_sk)] +ReadSchema: struct<ss_addr_sk:int,ss_ext_sales_price:decimal(7,2)> + +(31) CometFilter +Input [3]: [ss_addr_sk#26, ss_ext_sales_price#27, ss_sold_date_sk#28] +Condition : isnotnull(ss_addr_sk#26) + +(32) ColumnarToRow [codegen id : 10] +Input [3]: [ss_addr_sk#26, ss_ext_sales_price#27, ss_sold_date_sk#28] + +(33) ReusedExchange [Reuses operator id: 102] +Output [3]: [d_date_sk#30, d_year#31, d_qoy#32] + +(34) BroadcastHashJoin [codegen id : 10] +Left keys [1]: [ss_sold_date_sk#28] +Right keys [1]: [d_date_sk#30] +Join type: Inner +Join condition: None + +(35) Project [codegen id : 10] +Output [4]: [ss_addr_sk#26, ss_ext_sales_price#27, d_year#31, d_qoy#32] +Input [6]: [ss_addr_sk#26, ss_ext_sales_price#27, ss_sold_date_sk#28, d_date_sk#30, d_year#31, d_qoy#32] + +(36) ReusedExchange [Reuses operator id: 10] +Output [2]: [ca_address_sk#33, ca_county#34] + +(37) BroadcastHashJoin [codegen id : 10] +Left keys [1]: [ss_addr_sk#26] +Right keys [1]: [ca_address_sk#33] +Join type: Inner +Join condition: None + +(38) Project [codegen id : 10] +Output [4]: [ss_ext_sales_price#27, d_year#31, d_qoy#32, ca_county#34] +Input [6]: [ss_addr_sk#26, ss_ext_sales_price#27, d_year#31, d_qoy#32, ca_address_sk#33, ca_county#34] + +(39) HashAggregate [codegen id : 10] +Input [4]: [ss_ext_sales_price#27, d_year#31, d_qoy#32, ca_county#34] +Keys [3]: [ca_county#34, d_qoy#32, d_year#31] +Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#27))] +Aggregate Attributes [1]: [sum#35] +Results [4]: [ca_county#34, d_qoy#32, d_year#31, sum#36] + +(40) Exchange +Input [4]: [ca_county#34, d_qoy#32, d_year#31, sum#36] +Arguments: hashpartitioning(ca_county#34, d_qoy#32, d_year#31, 5), ENSURE_REQUIREMENTS, [plan_id=5] + +(41) HashAggregate [codegen id : 11] +Input [4]: [ca_county#34, d_qoy#32, d_year#31, sum#36] +Keys [3]: [ca_county#34, d_qoy#32, d_year#31] +Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#27))] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#27))#12] +Results [2]: [ca_county#34, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#27))#12,17,2) AS store_sales#37] + +(42) BroadcastExchange +Input [2]: [ca_county#34, store_sales#37] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=6] + +(43) BroadcastHashJoin [codegen id : 24] +Left keys [1]: [ca_county#22] +Right keys [1]: [ca_county#34] +Join type: Inner +Join condition: None + +(44) Project [codegen id : 24] +Output [5]: [ca_county#9, d_year#6, store_sales#13, store_sales#25, store_sales#37] +Input [7]: [ca_county#9, d_year#6, store_sales#13, ca_county#22, store_sales#25, ca_county#34, store_sales#37] + +(45) Scan parquet spark_catalog.default.web_sales +Output [3]: [ws_bill_addr_sk#38, ws_ext_sales_price#39, ws_sold_date_sk#40] +Batched: true +Location: InMemoryFileIndex [] +PartitionFilters: [isnotnull(ws_sold_date_sk#40), dynamicpruningexpression(ws_sold_date_sk#40 IN dynamicpruning#41)] +PushedFilters: [IsNotNull(ws_bill_addr_sk)] +ReadSchema: struct<ws_bill_addr_sk:int,ws_ext_sales_price:decimal(7,2)> + +(46) CometFilter +Input [3]: [ws_bill_addr_sk#38, ws_ext_sales_price#39, ws_sold_date_sk#40] +Condition : isnotnull(ws_bill_addr_sk#38) + +(47) ColumnarToRow [codegen id : 14] +Input [3]: [ws_bill_addr_sk#38, ws_ext_sales_price#39, ws_sold_date_sk#40] + +(48) ReusedExchange [Reuses operator id: 94] +Output [3]: [d_date_sk#42, d_year#43, d_qoy#44] + +(49) BroadcastHashJoin [codegen id : 14] +Left keys [1]: [ws_sold_date_sk#40] +Right keys [1]: [d_date_sk#42] +Join type: Inner +Join condition: None + +(50) Project [codegen id : 14] +Output [4]: [ws_bill_addr_sk#38, ws_ext_sales_price#39, d_year#43, d_qoy#44] +Input [6]: [ws_bill_addr_sk#38, ws_ext_sales_price#39, ws_sold_date_sk#40, d_date_sk#42, d_year#43, d_qoy#44] + +(51) ReusedExchange [Reuses operator id: 10] +Output [2]: [ca_address_sk#45, ca_county#46] + +(52) BroadcastHashJoin [codegen id : 14] +Left keys [1]: [ws_bill_addr_sk#38] +Right keys [1]: [ca_address_sk#45] +Join type: Inner +Join condition: None + +(53) Project [codegen id : 14] +Output [4]: [ws_ext_sales_price#39, d_year#43, d_qoy#44, ca_county#46] +Input [6]: [ws_bill_addr_sk#38, ws_ext_sales_price#39, d_year#43, d_qoy#44, ca_address_sk#45, ca_county#46] + +(54) HashAggregate [codegen id : 14] +Input [4]: [ws_ext_sales_price#39, d_year#43, d_qoy#44, ca_county#46] +Keys [3]: [ca_county#46, d_qoy#44, d_year#43] +Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#39))] +Aggregate Attributes [1]: [sum#47] +Results [4]: [ca_county#46, d_qoy#44, d_year#43, sum#48] + +(55) Exchange +Input [4]: [ca_county#46, d_qoy#44, d_year#43, sum#48] +Arguments: hashpartitioning(ca_county#46, d_qoy#44, d_year#43, 5), ENSURE_REQUIREMENTS, [plan_id=7] + +(56) HashAggregate [codegen id : 15] +Input [4]: [ca_county#46, d_qoy#44, d_year#43, sum#48] +Keys [3]: [ca_county#46, d_qoy#44, d_year#43] +Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#39))] +Aggregate Attributes [1]: [sum(UnscaledValue(ws_ext_sales_price#39))#49] +Results [2]: [ca_county#46, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#39))#49,17,2) AS web_sales#50] + +(57) BroadcastExchange +Input [2]: [ca_county#46, web_sales#50] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=8] + +(58) BroadcastHashJoin [codegen id : 24] +Left keys [1]: [ca_county#9] +Right keys [1]: [ca_county#46] +Join type: Inner +Join condition: None + +(59) Scan parquet spark_catalog.default.web_sales +Output [3]: [ws_bill_addr_sk#51, ws_ext_sales_price#52, ws_sold_date_sk#53] +Batched: true +Location: InMemoryFileIndex [] +PartitionFilters: [isnotnull(ws_sold_date_sk#53), dynamicpruningexpression(ws_sold_date_sk#53 IN dynamicpruning#54)] +PushedFilters: [IsNotNull(ws_bill_addr_sk)] +ReadSchema: struct<ws_bill_addr_sk:int,ws_ext_sales_price:decimal(7,2)> + +(60) CometFilter +Input [3]: [ws_bill_addr_sk#51, ws_ext_sales_price#52, ws_sold_date_sk#53] +Condition : isnotnull(ws_bill_addr_sk#51) + +(61) ColumnarToRow [codegen id : 18] +Input [3]: [ws_bill_addr_sk#51, ws_ext_sales_price#52, ws_sold_date_sk#53] + +(62) ReusedExchange [Reuses operator id: 98] +Output [3]: [d_date_sk#55, d_year#56, d_qoy#57] + +(63) BroadcastHashJoin [codegen id : 18] +Left keys [1]: [ws_sold_date_sk#53] +Right keys [1]: [d_date_sk#55] +Join type: Inner +Join condition: None + +(64) Project [codegen id : 18] +Output [4]: [ws_bill_addr_sk#51, ws_ext_sales_price#52, d_year#56, d_qoy#57] +Input [6]: [ws_bill_addr_sk#51, ws_ext_sales_price#52, ws_sold_date_sk#53, d_date_sk#55, d_year#56, d_qoy#57] + +(65) ReusedExchange [Reuses operator id: 10] +Output [2]: [ca_address_sk#58, ca_county#59] + +(66) BroadcastHashJoin [codegen id : 18] +Left keys [1]: [ws_bill_addr_sk#51] +Right keys [1]: [ca_address_sk#58] +Join type: Inner +Join condition: None + +(67) Project [codegen id : 18] +Output [4]: [ws_ext_sales_price#52, d_year#56, d_qoy#57, ca_county#59] +Input [6]: [ws_bill_addr_sk#51, ws_ext_sales_price#52, d_year#56, d_qoy#57, ca_address_sk#58, ca_county#59] + +(68) HashAggregate [codegen id : 18] +Input [4]: [ws_ext_sales_price#52, d_year#56, d_qoy#57, ca_county#59] +Keys [3]: [ca_county#59, d_qoy#57, d_year#56] +Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#52))] +Aggregate Attributes [1]: [sum#60] +Results [4]: [ca_county#59, d_qoy#57, d_year#56, sum#61] + +(69) Exchange +Input [4]: [ca_county#59, d_qoy#57, d_year#56, sum#61] +Arguments: hashpartitioning(ca_county#59, d_qoy#57, d_year#56, 5), ENSURE_REQUIREMENTS, [plan_id=9] + +(70) HashAggregate [codegen id : 19] +Input [4]: [ca_county#59, d_qoy#57, d_year#56, sum#61] +Keys [3]: [ca_county#59, d_qoy#57, d_year#56] +Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#52))] +Aggregate Attributes [1]: [sum(UnscaledValue(ws_ext_sales_price#52))#49] +Results [2]: [ca_county#59, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#52))#49,17,2) AS web_sales#62] + +(71) BroadcastExchange +Input [2]: [ca_county#59, web_sales#62] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=10] + +(72) BroadcastHashJoin [codegen id : 24] +Left keys [1]: [ca_county#46] +Right keys [1]: [ca_county#59] +Join type: Inner +Join condition: (CASE WHEN (web_sales#50 > 0.00) THEN (web_sales#62 / web_sales#50) END > CASE WHEN (store_sales#13 > 0.00) THEN (store_sales#25 / store_sales#13) END) + +(73) Project [codegen id : 24] +Output [8]: [ca_county#9, d_year#6, store_sales#13, store_sales#25, store_sales#37, ca_county#46, web_sales#50, web_sales#62] +Input [9]: [ca_county#9, d_year#6, store_sales#13, store_sales#25, store_sales#37, ca_county#46, web_sales#50, ca_county#59, web_sales#62] + +(74) Scan parquet spark_catalog.default.web_sales +Output [3]: [ws_bill_addr_sk#63, ws_ext_sales_price#64, ws_sold_date_sk#65] +Batched: true +Location: InMemoryFileIndex [] +PartitionFilters: [isnotnull(ws_sold_date_sk#65), dynamicpruningexpression(ws_sold_date_sk#65 IN dynamicpruning#66)] +PushedFilters: [IsNotNull(ws_bill_addr_sk)] +ReadSchema: struct<ws_bill_addr_sk:int,ws_ext_sales_price:decimal(7,2)> + +(75) CometFilter +Input [3]: [ws_bill_addr_sk#63, ws_ext_sales_price#64, ws_sold_date_sk#65] +Condition : isnotnull(ws_bill_addr_sk#63) + +(76) ColumnarToRow [codegen id : 22] +Input [3]: [ws_bill_addr_sk#63, ws_ext_sales_price#64, ws_sold_date_sk#65] + +(77) ReusedExchange [Reuses operator id: 102] +Output [3]: [d_date_sk#67, d_year#68, d_qoy#69] + +(78) BroadcastHashJoin [codegen id : 22] +Left keys [1]: [ws_sold_date_sk#65] +Right keys [1]: [d_date_sk#67] +Join type: Inner +Join condition: None + +(79) Project [codegen id : 22] +Output [4]: [ws_bill_addr_sk#63, ws_ext_sales_price#64, d_year#68, d_qoy#69] +Input [6]: [ws_bill_addr_sk#63, ws_ext_sales_price#64, ws_sold_date_sk#65, d_date_sk#67, d_year#68, d_qoy#69] + +(80) ReusedExchange [Reuses operator id: 10] +Output [2]: [ca_address_sk#70, ca_county#71] + +(81) BroadcastHashJoin [codegen id : 22] +Left keys [1]: [ws_bill_addr_sk#63] +Right keys [1]: [ca_address_sk#70] +Join type: Inner +Join condition: None + +(82) Project [codegen id : 22] +Output [4]: [ws_ext_sales_price#64, d_year#68, d_qoy#69, ca_county#71] +Input [6]: [ws_bill_addr_sk#63, ws_ext_sales_price#64, d_year#68, d_qoy#69, ca_address_sk#70, ca_county#71] + +(83) HashAggregate [codegen id : 22] +Input [4]: [ws_ext_sales_price#64, d_year#68, d_qoy#69, ca_county#71] +Keys [3]: [ca_county#71, d_qoy#69, d_year#68] +Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#64))] +Aggregate Attributes [1]: [sum#72] +Results [4]: [ca_county#71, d_qoy#69, d_year#68, sum#73] + +(84) Exchange +Input [4]: [ca_county#71, d_qoy#69, d_year#68, sum#73] +Arguments: hashpartitioning(ca_county#71, d_qoy#69, d_year#68, 5), ENSURE_REQUIREMENTS, [plan_id=11] + +(85) HashAggregate [codegen id : 23] +Input [4]: [ca_county#71, d_qoy#69, d_year#68, sum#73] +Keys [3]: [ca_county#71, d_qoy#69, d_year#68] +Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#64))] +Aggregate Attributes [1]: [sum(UnscaledValue(ws_ext_sales_price#64))#49] +Results [2]: [ca_county#71, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#64))#49,17,2) AS web_sales#74] + +(86) BroadcastExchange +Input [2]: [ca_county#71, web_sales#74] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=12] + +(87) BroadcastHashJoin [codegen id : 24] +Left keys [1]: [ca_county#46] +Right keys [1]: [ca_county#71] +Join type: Inner +Join condition: (CASE WHEN (web_sales#62 > 0.00) THEN (web_sales#74 / web_sales#62) END > CASE WHEN (store_sales#25 > 0.00) THEN (store_sales#37 / store_sales#25) END) + +(88) Project [codegen id : 24] +Output [6]: [ca_county#9, d_year#6, (web_sales#62 / web_sales#50) AS web_q1_q2_increase#75, (store_sales#25 / store_sales#13) AS store_q1_q2_increase#76, (web_sales#74 / web_sales#62) AS web_q2_q3_increase#77, (store_sales#37 / store_sales#25) AS store_q2_q3_increase#78] +Input [10]: [ca_county#9, d_year#6, store_sales#13, store_sales#25, store_sales#37, ca_county#46, web_sales#50, web_sales#62, ca_county#71, web_sales#74] + +(89) Exchange +Input [6]: [ca_county#9, d_year#6, web_q1_q2_increase#75, store_q1_q2_increase#76, web_q2_q3_increase#77, store_q2_q3_increase#78] +Arguments: rangepartitioning(ca_county#9 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, [plan_id=13] + +(90) Sort [codegen id : 25] +Input [6]: [ca_county#9, d_year#6, web_q1_q2_increase#75, store_q1_q2_increase#76, web_q2_q3_increase#77, store_q2_q3_increase#78] +Arguments: [ca_county#9 ASC NULLS FIRST], true, 0 + +===== Subqueries ===== + +Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#3 IN dynamicpruning#4 +BroadcastExchange (94) ++- * ColumnarToRow (93) + +- CometFilter (92) + +- CometScan parquet spark_catalog.default.date_dim (91) + + +(91) Scan parquet spark_catalog.default.date_dim +Output [3]: [d_date_sk#5, d_year#6, d_qoy#7] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_qoy), IsNotNull(d_year), EqualTo(d_qoy,1), EqualTo(d_year,2000), IsNotNull(d_date_sk)] +ReadSchema: struct<d_date_sk:int,d_year:int,d_qoy:int> + +(92) CometFilter +Input [3]: [d_date_sk#5, d_year#6, d_qoy#7] +Condition : ((((isnotnull(d_qoy#7) AND isnotnull(d_year#6)) AND (d_qoy#7 = 1)) AND (d_year#6 = 2000)) AND isnotnull(d_date_sk#5)) + +(93) ColumnarToRow [codegen id : 1] +Input [3]: [d_date_sk#5, d_year#6, d_qoy#7] + +(94) BroadcastExchange +Input [3]: [d_date_sk#5, d_year#6, d_qoy#7] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=14] + +Subquery:2 Hosting operator id = 16 Hosting Expression = ss_sold_date_sk#16 IN dynamicpruning#17 +BroadcastExchange (98) ++- * ColumnarToRow (97) + +- CometFilter (96) + +- CometScan parquet spark_catalog.default.date_dim (95) + + +(95) Scan parquet spark_catalog.default.date_dim +Output [3]: [d_date_sk#18, d_year#19, d_qoy#20] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_qoy), IsNotNull(d_year), EqualTo(d_qoy,2), EqualTo(d_year,2000), IsNotNull(d_date_sk)] +ReadSchema: struct<d_date_sk:int,d_year:int,d_qoy:int> + +(96) CometFilter +Input [3]: [d_date_sk#18, d_year#19, d_qoy#20] +Condition : ((((isnotnull(d_qoy#20) AND isnotnull(d_year#19)) AND (d_qoy#20 = 2)) AND (d_year#19 = 2000)) AND isnotnull(d_date_sk#18)) + +(97) ColumnarToRow [codegen id : 1] +Input [3]: [d_date_sk#18, d_year#19, d_qoy#20] + +(98) BroadcastExchange +Input [3]: [d_date_sk#18, d_year#19, d_qoy#20] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=15] + +Subquery:3 Hosting operator id = 30 Hosting Expression = ss_sold_date_sk#28 IN dynamicpruning#29 +BroadcastExchange (102) ++- * ColumnarToRow (101) + +- CometFilter (100) + +- CometScan parquet spark_catalog.default.date_dim (99) + + +(99) Scan parquet spark_catalog.default.date_dim +Output [3]: [d_date_sk#30, d_year#31, d_qoy#32] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_qoy), IsNotNull(d_year), EqualTo(d_qoy,3), EqualTo(d_year,2000), IsNotNull(d_date_sk)] +ReadSchema: struct<d_date_sk:int,d_year:int,d_qoy:int> + +(100) CometFilter +Input [3]: [d_date_sk#30, d_year#31, d_qoy#32] +Condition : ((((isnotnull(d_qoy#32) AND isnotnull(d_year#31)) AND (d_qoy#32 = 3)) AND (d_year#31 = 2000)) AND isnotnull(d_date_sk#30)) + +(101) ColumnarToRow [codegen id : 1] +Input [3]: [d_date_sk#30, d_year#31, d_qoy#32] + +(102) BroadcastExchange +Input [3]: [d_date_sk#30, d_year#31, d_qoy#32] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=16] + +Subquery:4 Hosting operator id = 45 Hosting Expression = ws_sold_date_sk#40 IN dynamicpruning#4 + +Subquery:5 Hosting operator id = 59 Hosting Expression = ws_sold_date_sk#53 IN dynamicpruning#17 + +Subquery:6 Hosting operator id = 74 Hosting Expression = ws_sold_date_sk#65 IN dynamicpruning#29 + + diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q31/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q31/simplified.txt new file mode 100644 index 000000000..f4bf6a89d --- /dev/null +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q31/simplified.txt @@ -0,0 +1,159 @@ +WholeStageCodegen (25) + Sort [ca_county] + InputAdapter + Exchange [ca_county] #1 + WholeStageCodegen (24) + Project [ca_county,d_year,web_sales,web_sales,store_sales,store_sales,web_sales,store_sales] + BroadcastHashJoin [ca_county,ca_county,web_sales,web_sales,store_sales,store_sales] + Project [ca_county,d_year,store_sales,store_sales,store_sales,ca_county,web_sales,web_sales] + BroadcastHashJoin [ca_county,ca_county,web_sales,web_sales,store_sales,store_sales] + BroadcastHashJoin [ca_county,ca_county] + Project [ca_county,d_year,store_sales,store_sales,store_sales] + BroadcastHashJoin [ca_county,ca_county] + BroadcastHashJoin [ca_county,ca_county] + HashAggregate [ca_county,d_qoy,d_year,sum] [sum(UnscaledValue(ss_ext_sales_price)),store_sales,sum] + InputAdapter + Exchange [ca_county,d_qoy,d_year] #2 + WholeStageCodegen (3) + HashAggregate [ca_county,d_qoy,d_year,ss_ext_sales_price] [sum,sum] + Project [ss_ext_sales_price,d_year,d_qoy,ca_county] + BroadcastHashJoin [ss_addr_sk,ca_address_sk] + Project [ss_addr_sk,ss_ext_sales_price,d_year,d_qoy] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + ColumnarToRow + InputAdapter + CometFilter [ss_addr_sk] + CometScan parquet spark_catalog.default.store_sales [ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometFilter [d_qoy,d_year,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] + InputAdapter + ReusedExchange [d_date_sk,d_year,d_qoy] #3 + InputAdapter + BroadcastExchange #4 + WholeStageCodegen (2) + ColumnarToRow + InputAdapter + CometFilter [ca_address_sk,ca_county] + CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_county] + InputAdapter + BroadcastExchange #5 + WholeStageCodegen (7) + HashAggregate [ca_county,d_qoy,d_year,sum] [sum(UnscaledValue(ss_ext_sales_price)),store_sales,sum] + InputAdapter + Exchange [ca_county,d_qoy,d_year] #6 + WholeStageCodegen (6) + HashAggregate [ca_county,d_qoy,d_year,ss_ext_sales_price] [sum,sum] + Project [ss_ext_sales_price,d_year,d_qoy,ca_county] + BroadcastHashJoin [ss_addr_sk,ca_address_sk] + Project [ss_addr_sk,ss_ext_sales_price,d_year,d_qoy] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + ColumnarToRow + InputAdapter + CometFilter [ss_addr_sk] + CometScan parquet spark_catalog.default.store_sales [ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #2 + BroadcastExchange #7 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometFilter [d_qoy,d_year,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] + InputAdapter + ReusedExchange [d_date_sk,d_year,d_qoy] #7 + InputAdapter + ReusedExchange [ca_address_sk,ca_county] #4 + InputAdapter + BroadcastExchange #8 + WholeStageCodegen (11) + HashAggregate [ca_county,d_qoy,d_year,sum] [sum(UnscaledValue(ss_ext_sales_price)),store_sales,sum] + InputAdapter + Exchange [ca_county,d_qoy,d_year] #9 + WholeStageCodegen (10) + HashAggregate [ca_county,d_qoy,d_year,ss_ext_sales_price] [sum,sum] + Project [ss_ext_sales_price,d_year,d_qoy,ca_county] + BroadcastHashJoin [ss_addr_sk,ca_address_sk] + Project [ss_addr_sk,ss_ext_sales_price,d_year,d_qoy] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + ColumnarToRow + InputAdapter + CometFilter [ss_addr_sk] + CometScan parquet spark_catalog.default.store_sales [ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #3 + BroadcastExchange #10 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometFilter [d_qoy,d_year,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] + InputAdapter + ReusedExchange [d_date_sk,d_year,d_qoy] #10 + InputAdapter + ReusedExchange [ca_address_sk,ca_county] #4 + InputAdapter + BroadcastExchange #11 + WholeStageCodegen (15) + HashAggregate [ca_county,d_qoy,d_year,sum] [sum(UnscaledValue(ws_ext_sales_price)),web_sales,sum] + InputAdapter + Exchange [ca_county,d_qoy,d_year] #12 + WholeStageCodegen (14) + HashAggregate [ca_county,d_qoy,d_year,ws_ext_sales_price] [sum,sum] + Project [ws_ext_sales_price,d_year,d_qoy,ca_county] + BroadcastHashJoin [ws_bill_addr_sk,ca_address_sk] + Project [ws_bill_addr_sk,ws_ext_sales_price,d_year,d_qoy] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + ColumnarToRow + InputAdapter + CometFilter [ws_bill_addr_sk] + CometScan parquet spark_catalog.default.web_sales [ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + InputAdapter + ReusedExchange [d_date_sk,d_year,d_qoy] #3 + InputAdapter + ReusedExchange [ca_address_sk,ca_county] #4 + InputAdapter + BroadcastExchange #13 + WholeStageCodegen (19) + HashAggregate [ca_county,d_qoy,d_year,sum] [sum(UnscaledValue(ws_ext_sales_price)),web_sales,sum] + InputAdapter + Exchange [ca_county,d_qoy,d_year] #14 + WholeStageCodegen (18) + HashAggregate [ca_county,d_qoy,d_year,ws_ext_sales_price] [sum,sum] + Project [ws_ext_sales_price,d_year,d_qoy,ca_county] + BroadcastHashJoin [ws_bill_addr_sk,ca_address_sk] + Project [ws_bill_addr_sk,ws_ext_sales_price,d_year,d_qoy] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + ColumnarToRow + InputAdapter + CometFilter [ws_bill_addr_sk] + CometScan parquet spark_catalog.default.web_sales [ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #2 + InputAdapter + ReusedExchange [d_date_sk,d_year,d_qoy] #7 + InputAdapter + ReusedExchange [ca_address_sk,ca_county] #4 + InputAdapter + BroadcastExchange #15 + WholeStageCodegen (23) + HashAggregate [ca_county,d_qoy,d_year,sum] [sum(UnscaledValue(ws_ext_sales_price)),web_sales,sum] + InputAdapter + Exchange [ca_county,d_qoy,d_year] #16 + WholeStageCodegen (22) + HashAggregate [ca_county,d_qoy,d_year,ws_ext_sales_price] [sum,sum] + Project [ws_ext_sales_price,d_year,d_qoy,ca_county] + BroadcastHashJoin [ws_bill_addr_sk,ca_address_sk] + Project [ws_bill_addr_sk,ws_ext_sales_price,d_year,d_qoy] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + ColumnarToRow + InputAdapter + CometFilter [ws_bill_addr_sk] + CometScan parquet spark_catalog.default.web_sales [ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #3 + InputAdapter + ReusedExchange [d_date_sk,d_year,d_qoy] #10 + InputAdapter + ReusedExchange [ca_address_sk,ca_county] #4 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 new file mode 100644 index 000000000..bd7caaaa1 --- /dev/null +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q32/explain.txt @@ -0,0 +1,209 @@ +== Physical Plan == +* HashAggregate (29) ++- Exchange (28) + +- * HashAggregate (27) + +- * Project (26) + +- * BroadcastHashJoin Inner BuildRight (25) + :- * Project (23) + : +- * BroadcastHashJoin Inner BuildRight (22) + : :- * Project (10) + : : +- * BroadcastHashJoin Inner BuildRight (9) + : : :- * ColumnarToRow (3) + : : : +- CometFilter (2) + : : : +- CometScan parquet spark_catalog.default.catalog_sales (1) + : : +- BroadcastExchange (8) + : : +- * ColumnarToRow (7) + : : +- CometProject (6) + : : +- CometFilter (5) + : : +- CometScan parquet spark_catalog.default.item (4) + : +- BroadcastExchange (21) + : +- * Filter (20) + : +- * HashAggregate (19) + : +- Exchange (18) + : +- * HashAggregate (17) + : +- * Project (16) + : +- * BroadcastHashJoin Inner BuildRight (15) + : :- * ColumnarToRow (13) + : : +- CometFilter (12) + : : +- CometScan parquet spark_catalog.default.catalog_sales (11) + : +- ReusedExchange (14) + +- ReusedExchange (24) + + +(1) Scan parquet spark_catalog.default.catalog_sales +Output [3]: [cs_item_sk#1, cs_ext_discount_amt#2, cs_sold_date_sk#3] +Batched: true +Location: InMemoryFileIndex [] +PartitionFilters: [isnotnull(cs_sold_date_sk#3), dynamicpruningexpression(cs_sold_date_sk#3 IN dynamicpruning#4)] +PushedFilters: [IsNotNull(cs_item_sk), IsNotNull(cs_ext_discount_amt)] +ReadSchema: struct<cs_item_sk:int,cs_ext_discount_amt:decimal(7,2)> + +(2) CometFilter +Input [3]: [cs_item_sk#1, cs_ext_discount_amt#2, cs_sold_date_sk#3] +Condition : (isnotnull(cs_item_sk#1) AND isnotnull(cs_ext_discount_amt#2)) + +(3) ColumnarToRow [codegen id : 6] +Input [3]: [cs_item_sk#1, cs_ext_discount_amt#2, cs_sold_date_sk#3] + +(4) Scan parquet spark_catalog.default.item +Output [2]: [i_item_sk#5, i_manufact_id#6] +Batched: true +Location [not included in comparison]/{warehouse_dir}/item] +PushedFilters: [IsNotNull(i_manufact_id), EqualTo(i_manufact_id,977), IsNotNull(i_item_sk)] +ReadSchema: struct<i_item_sk:int,i_manufact_id:int> + +(5) CometFilter +Input [2]: [i_item_sk#5, i_manufact_id#6] +Condition : ((isnotnull(i_manufact_id#6) AND (i_manufact_id#6 = 977)) AND isnotnull(i_item_sk#5)) + +(6) CometProject +Input [2]: [i_item_sk#5, i_manufact_id#6] +Arguments: [i_item_sk#5], [i_item_sk#5] + +(7) ColumnarToRow [codegen id : 1] +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] +Left keys [1]: [cs_item_sk#1] +Right keys [1]: [i_item_sk#5] +Join type: Inner +Join condition: None + +(10) Project [codegen id : 6] +Output [3]: [cs_ext_discount_amt#2, cs_sold_date_sk#3, i_item_sk#5] +Input [4]: [cs_item_sk#1, cs_ext_discount_amt#2, cs_sold_date_sk#3, i_item_sk#5] + +(11) Scan parquet spark_catalog.default.catalog_sales +Output [3]: [cs_item_sk#7, cs_ext_discount_amt#8, cs_sold_date_sk#9] +Batched: true +Location: InMemoryFileIndex [] +PartitionFilters: [isnotnull(cs_sold_date_sk#9), dynamicpruningexpression(cs_sold_date_sk#9 IN dynamicpruning#10)] +PushedFilters: [IsNotNull(cs_item_sk)] +ReadSchema: struct<cs_item_sk:int,cs_ext_discount_amt:decimal(7,2)> + +(12) CometFilter +Input [3]: [cs_item_sk#7, cs_ext_discount_amt#8, cs_sold_date_sk#9] +Condition : isnotnull(cs_item_sk#7) + +(13) ColumnarToRow [codegen id : 3] +Input [3]: [cs_item_sk#7, cs_ext_discount_amt#8, cs_sold_date_sk#9] + +(14) ReusedExchange [Reuses operator id: 34] +Output [1]: [d_date_sk#11] + +(15) BroadcastHashJoin [codegen id : 3] +Left keys [1]: [cs_sold_date_sk#9] +Right keys [1]: [d_date_sk#11] +Join type: Inner +Join condition: None + +(16) Project [codegen id : 3] +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#11] + +(17) HashAggregate [codegen id : 3] +Input [2]: [cs_item_sk#7, cs_ext_discount_amt#8] +Keys [1]: [cs_item_sk#7] +Functions [1]: [partial_avg(UnscaledValue(cs_ext_discount_amt#8))] +Aggregate Attributes [2]: [sum#12, count#13] +Results [3]: [cs_item_sk#7, sum#14, count#15] + +(18) Exchange +Input [3]: [cs_item_sk#7, sum#14, count#15] +Arguments: hashpartitioning(cs_item_sk#7, 5), ENSURE_REQUIREMENTS, [plan_id=2] + +(19) HashAggregate [codegen id : 4] +Input [3]: [cs_item_sk#7, sum#14, count#15] +Keys [1]: [cs_item_sk#7] +Functions [1]: [avg(UnscaledValue(cs_ext_discount_amt#8))] +Aggregate Attributes [1]: [avg(UnscaledValue(cs_ext_discount_amt#8))#16] +Results [2]: [(1.3 * cast((avg(UnscaledValue(cs_ext_discount_amt#8))#16 / 100.0) as decimal(11,6))) AS (1.3 * avg(cs_ext_discount_amt))#17, cs_item_sk#7] + +(20) Filter [codegen id : 4] +Input [2]: [(1.3 * avg(cs_ext_discount_amt))#17, cs_item_sk#7] +Condition : isnotnull((1.3 * avg(cs_ext_discount_amt))#17) + +(21) BroadcastExchange +Input [2]: [(1.3 * avg(cs_ext_discount_amt))#17, cs_item_sk#7] +Arguments: HashedRelationBroadcastMode(List(cast(input[1, int, true] as bigint)),false), [plan_id=3] + +(22) BroadcastHashJoin [codegen id : 6] +Left keys [1]: [i_item_sk#5] +Right keys [1]: [cs_item_sk#7] +Join type: Inner +Join condition: (cast(cs_ext_discount_amt#2 as decimal(14,7)) > (1.3 * avg(cs_ext_discount_amt))#17) + +(23) Project [codegen id : 6] +Output [2]: [cs_ext_discount_amt#2, cs_sold_date_sk#3] +Input [5]: [cs_ext_discount_amt#2, cs_sold_date_sk#3, i_item_sk#5, (1.3 * avg(cs_ext_discount_amt))#17, cs_item_sk#7] + +(24) ReusedExchange [Reuses operator id: 34] +Output [1]: [d_date_sk#18] + +(25) BroadcastHashJoin [codegen id : 6] +Left keys [1]: [cs_sold_date_sk#3] +Right keys [1]: [d_date_sk#18] +Join type: Inner +Join condition: None + +(26) Project [codegen id : 6] +Output [1]: [cs_ext_discount_amt#2] +Input [3]: [cs_ext_discount_amt#2, cs_sold_date_sk#3, d_date_sk#18] + +(27) HashAggregate [codegen id : 6] +Input [1]: [cs_ext_discount_amt#2] +Keys: [] +Functions [1]: [partial_sum(UnscaledValue(cs_ext_discount_amt#2))] +Aggregate Attributes [1]: [sum#19] +Results [1]: [sum#20] + +(28) Exchange +Input [1]: [sum#20] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=4] + +(29) HashAggregate [codegen id : 7] +Input [1]: [sum#20] +Keys: [] +Functions [1]: [sum(UnscaledValue(cs_ext_discount_amt#2))] +Aggregate Attributes [1]: [sum(UnscaledValue(cs_ext_discount_amt#2))#21] +Results [1]: [MakeDecimal(sum(UnscaledValue(cs_ext_discount_amt#2))#21,17,2) AS excess discount amount#22] + +===== Subqueries ===== + +Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#3 IN dynamicpruning#4 +BroadcastExchange (34) ++- * ColumnarToRow (33) + +- CometProject (32) + +- CometFilter (31) + +- CometScan parquet spark_catalog.default.date_dim (30) + + +(30) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#18, d_date#23] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,2000-01-27), LessThanOrEqual(d_date,2000-04-26), IsNotNull(d_date_sk)] +ReadSchema: struct<d_date_sk:int,d_date:date> + +(31) CometFilter +Input [2]: [d_date_sk#18, d_date#23] +Condition : (((isnotnull(d_date#23) AND (d_date#23 >= 2000-01-27)) AND (d_date#23 <= 2000-04-26)) AND isnotnull(d_date_sk#18)) + +(32) CometProject +Input [2]: [d_date_sk#18, d_date#23] +Arguments: [d_date_sk#18], [d_date_sk#18] + +(33) ColumnarToRow [codegen id : 1] +Input [1]: [d_date_sk#18] + +(34) BroadcastExchange +Input [1]: [d_date_sk#18] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] + +Subquery:2 Hosting operator id = 11 Hosting Expression = cs_sold_date_sk#9 IN dynamicpruning#4 + + diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q32/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q32/simplified.txt new file mode 100644 index 000000000..146a33fdd --- /dev/null +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q32/simplified.txt @@ -0,0 +1,52 @@ +WholeStageCodegen (7) + HashAggregate [sum] [sum(UnscaledValue(cs_ext_discount_amt)),excess discount amount,sum] + InputAdapter + Exchange #1 + WholeStageCodegen (6) + HashAggregate [cs_ext_discount_amt] [sum,sum] + Project [cs_ext_discount_amt] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + Project [cs_ext_discount_amt,cs_sold_date_sk] + BroadcastHashJoin [i_item_sk,cs_item_sk,cs_ext_discount_amt,(1.3 * avg(cs_ext_discount_amt))] + Project [cs_ext_discount_amt,cs_sold_date_sk,i_item_sk] + BroadcastHashJoin [cs_item_sk,i_item_sk] + ColumnarToRow + InputAdapter + CometFilter [cs_item_sk,cs_ext_discount_amt] + CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_ext_discount_amt,cs_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #2 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] + InputAdapter + BroadcastExchange #3 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [i_item_sk] + CometFilter [i_manufact_id,i_item_sk] + CometScan parquet spark_catalog.default.item [i_item_sk,i_manufact_id] + InputAdapter + BroadcastExchange #4 + WholeStageCodegen (4) + Filter [(1.3 * avg(cs_ext_discount_amt))] + HashAggregate [cs_item_sk,sum,count] [avg(UnscaledValue(cs_ext_discount_amt)),(1.3 * avg(cs_ext_discount_amt)),sum,count] + InputAdapter + Exchange [cs_item_sk] #5 + WholeStageCodegen (3) + HashAggregate [cs_item_sk,cs_ext_discount_amt] [sum,count,sum,count] + Project [cs_item_sk,cs_ext_discount_amt] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + ColumnarToRow + InputAdapter + CometFilter [cs_item_sk] + CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_ext_discount_amt,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + InputAdapter + ReusedExchange [d_date_sk] #2 + InputAdapter + ReusedExchange [d_date_sk] #2 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q33/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q33/explain.txt new file mode 100644 index 000000000..f9541481c --- /dev/null +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q33/explain.txt @@ -0,0 +1,405 @@ +== Physical Plan == +TakeOrderedAndProject (63) ++- * HashAggregate (62) + +- Exchange (61) + +- * HashAggregate (60) + +- Union (59) + :- * HashAggregate (28) + : +- Exchange (27) + : +- * HashAggregate (26) + : +- * Project (25) + : +- * BroadcastHashJoin Inner BuildRight (24) + : :- * Project (13) + : : +- * BroadcastHashJoin Inner BuildRight (12) + : : :- * Project (6) + : : : +- * BroadcastHashJoin Inner BuildRight (5) + : : : :- * ColumnarToRow (3) + : : : : +- CometFilter (2) + : : : : +- CometScan parquet spark_catalog.default.store_sales (1) + : : : +- ReusedExchange (4) + : : +- BroadcastExchange (11) + : : +- * ColumnarToRow (10) + : : +- CometProject (9) + : : +- CometFilter (8) + : : +- CometScan parquet spark_catalog.default.customer_address (7) + : +- BroadcastExchange (23) + : +- * BroadcastHashJoin LeftSemi BuildRight (22) + : :- * ColumnarToRow (16) + : : +- CometFilter (15) + : : +- CometScan parquet spark_catalog.default.item (14) + : +- BroadcastExchange (21) + : +- * ColumnarToRow (20) + : +- CometProject (19) + : +- CometFilter (18) + : +- CometScan parquet spark_catalog.default.item (17) + :- * HashAggregate (43) + : +- Exchange (42) + : +- * HashAggregate (41) + : +- * Project (40) + : +- * BroadcastHashJoin Inner BuildRight (39) + : :- * Project (37) + : : +- * BroadcastHashJoin Inner BuildRight (36) + : : :- * Project (34) + : : : +- * BroadcastHashJoin Inner BuildRight (33) + : : : :- * ColumnarToRow (31) + : : : : +- CometFilter (30) + : : : : +- CometScan parquet spark_catalog.default.catalog_sales (29) + : : : +- ReusedExchange (32) + : : +- ReusedExchange (35) + : +- ReusedExchange (38) + +- * HashAggregate (58) + +- Exchange (57) + +- * HashAggregate (56) + +- * Project (55) + +- * BroadcastHashJoin Inner BuildRight (54) + :- * Project (52) + : +- * BroadcastHashJoin Inner BuildRight (51) + : :- * Project (49) + : : +- * BroadcastHashJoin Inner BuildRight (48) + : : :- * ColumnarToRow (46) + : : : +- CometFilter (45) + : : : +- CometScan parquet spark_catalog.default.web_sales (44) + : : +- ReusedExchange (47) + : +- ReusedExchange (50) + +- ReusedExchange (53) + + +(1) Scan parquet spark_catalog.default.store_sales +Output [4]: [ss_item_sk#1, ss_addr_sk#2, ss_ext_sales_price#3, ss_sold_date_sk#4] +Batched: true +Location: InMemoryFileIndex [] +PartitionFilters: [isnotnull(ss_sold_date_sk#4), dynamicpruningexpression(ss_sold_date_sk#4 IN dynamicpruning#5)] +PushedFilters: [IsNotNull(ss_addr_sk), IsNotNull(ss_item_sk)] +ReadSchema: struct<ss_item_sk:int,ss_addr_sk:int,ss_ext_sales_price:decimal(7,2)> + +(2) CometFilter +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)) + +(3) ColumnarToRow [codegen id : 5] +Input [4]: [ss_item_sk#1, ss_addr_sk#2, ss_ext_sales_price#3, ss_sold_date_sk#4] + +(4) ReusedExchange [Reuses operator id: 68] +Output [1]: [d_date_sk#6] + +(5) BroadcastHashJoin [codegen id : 5] +Left keys [1]: [ss_sold_date_sk#4] +Right keys [1]: [d_date_sk#6] +Join type: Inner +Join condition: None + +(6) Project [codegen id : 5] +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] + +(7) Scan parquet spark_catalog.default.customer_address +Output [2]: [ca_address_sk#7, ca_gmt_offset#8] +Batched: true +Location [not included in comparison]/{warehouse_dir}/customer_address] +PushedFilters: [IsNotNull(ca_gmt_offset), EqualTo(ca_gmt_offset,-5.00), IsNotNull(ca_address_sk)] +ReadSchema: struct<ca_address_sk:int,ca_gmt_offset:decimal(5,2)> + +(8) CometFilter +Input [2]: [ca_address_sk#7, ca_gmt_offset#8] +Condition : ((isnotnull(ca_gmt_offset#8) AND (ca_gmt_offset#8 = -5.00)) AND isnotnull(ca_address_sk#7)) + +(9) CometProject +Input [2]: [ca_address_sk#7, ca_gmt_offset#8] +Arguments: [ca_address_sk#7], [ca_address_sk#7] + +(10) ColumnarToRow [codegen id : 2] +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 : 5] +Left keys [1]: [ss_addr_sk#2] +Right keys [1]: [ca_address_sk#7] +Join type: Inner +Join condition: None + +(13) Project [codegen id : 5] +Output [2]: [ss_item_sk#1, ss_ext_sales_price#3] +Input [4]: [ss_item_sk#1, ss_addr_sk#2, ss_ext_sales_price#3, ca_address_sk#7] + +(14) Scan parquet spark_catalog.default.item +Output [2]: [i_item_sk#9, i_manufact_id#10] +Batched: true +Location [not included in comparison]/{warehouse_dir}/item] +PushedFilters: [IsNotNull(i_item_sk)] +ReadSchema: struct<i_item_sk:int,i_manufact_id:int> + +(15) CometFilter +Input [2]: [i_item_sk#9, i_manufact_id#10] +Condition : isnotnull(i_item_sk#9) + +(16) ColumnarToRow [codegen id : 4] +Input [2]: [i_item_sk#9, i_manufact_id#10] + +(17) Scan parquet spark_catalog.default.item +Output [2]: [i_category#11, i_manufact_id#12] +Batched: true +Location [not included in comparison]/{warehouse_dir}/item] +PushedFilters: [IsNotNull(i_category), EqualTo(i_category,Electronics )] +ReadSchema: struct<i_category:string,i_manufact_id:int> + +(18) CometFilter +Input [2]: [i_category#11, i_manufact_id#12] +Condition : (isnotnull(i_category#11) AND (i_category#11 = Electronics )) + +(19) CometProject +Input [2]: [i_category#11, i_manufact_id#12] +Arguments: [i_manufact_id#12], [i_manufact_id#12] + +(20) ColumnarToRow [codegen id : 3] +Input [1]: [i_manufact_id#12] + +(21) BroadcastExchange +Input [1]: [i_manufact_id#12] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] + +(22) BroadcastHashJoin [codegen id : 4] +Left keys [1]: [i_manufact_id#10] +Right keys [1]: [i_manufact_id#12] +Join type: LeftSemi +Join condition: None + +(23) BroadcastExchange +Input [2]: [i_item_sk#9, i_manufact_id#10] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=3] + +(24) BroadcastHashJoin [codegen id : 5] +Left keys [1]: [ss_item_sk#1] +Right keys [1]: [i_item_sk#9] +Join type: Inner +Join condition: None + +(25) Project [codegen id : 5] +Output [2]: [ss_ext_sales_price#3, i_manufact_id#10] +Input [4]: [ss_item_sk#1, ss_ext_sales_price#3, i_item_sk#9, i_manufact_id#10] + +(26) HashAggregate [codegen id : 5] +Input [2]: [ss_ext_sales_price#3, i_manufact_id#10] +Keys [1]: [i_manufact_id#10] +Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#3))] +Aggregate Attributes [1]: [sum#13] +Results [2]: [i_manufact_id#10, sum#14] + +(27) Exchange +Input [2]: [i_manufact_id#10, sum#14] +Arguments: hashpartitioning(i_manufact_id#10, 5), ENSURE_REQUIREMENTS, [plan_id=4] + +(28) HashAggregate [codegen id : 6] +Input [2]: [i_manufact_id#10, sum#14] +Keys [1]: [i_manufact_id#10] +Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#3))] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#3))#15] +Results [2]: [i_manufact_id#10, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#3))#15,17,2) AS total_sales#16] + +(29) Scan parquet spark_catalog.default.catalog_sales +Output [4]: [cs_bill_addr_sk#17, cs_item_sk#18, cs_ext_sales_price#19, cs_sold_date_sk#20] +Batched: true +Location: InMemoryFileIndex [] +PartitionFilters: [isnotnull(cs_sold_date_sk#20), dynamicpruningexpression(cs_sold_date_sk#20 IN dynamicpruning#21)] +PushedFilters: [IsNotNull(cs_bill_addr_sk), IsNotNull(cs_item_sk)] +ReadSchema: struct<cs_bill_addr_sk:int,cs_item_sk:int,cs_ext_sales_price:decimal(7,2)> + +(30) CometFilter +Input [4]: [cs_bill_addr_sk#17, cs_item_sk#18, cs_ext_sales_price#19, cs_sold_date_sk#20] +Condition : (isnotnull(cs_bill_addr_sk#17) AND isnotnull(cs_item_sk#18)) + +(31) ColumnarToRow [codegen id : 11] +Input [4]: [cs_bill_addr_sk#17, cs_item_sk#18, cs_ext_sales_price#19, cs_sold_date_sk#20] + +(32) ReusedExchange [Reuses operator id: 68] +Output [1]: [d_date_sk#22] + +(33) BroadcastHashJoin [codegen id : 11] +Left keys [1]: [cs_sold_date_sk#20] +Right keys [1]: [d_date_sk#22] +Join type: Inner +Join condition: None + +(34) Project [codegen id : 11] +Output [3]: [cs_bill_addr_sk#17, cs_item_sk#18, cs_ext_sales_price#19] +Input [5]: [cs_bill_addr_sk#17, cs_item_sk#18, cs_ext_sales_price#19, cs_sold_date_sk#20, d_date_sk#22] + +(35) ReusedExchange [Reuses operator id: 11] +Output [1]: [ca_address_sk#23] + +(36) BroadcastHashJoin [codegen id : 11] +Left keys [1]: [cs_bill_addr_sk#17] +Right keys [1]: [ca_address_sk#23] +Join type: Inner +Join condition: None + +(37) Project [codegen id : 11] +Output [2]: [cs_item_sk#18, cs_ext_sales_price#19] +Input [4]: [cs_bill_addr_sk#17, cs_item_sk#18, cs_ext_sales_price#19, ca_address_sk#23] + +(38) ReusedExchange [Reuses operator id: 23] +Output [2]: [i_item_sk#24, i_manufact_id#25] + +(39) BroadcastHashJoin [codegen id : 11] +Left keys [1]: [cs_item_sk#18] +Right keys [1]: [i_item_sk#24] +Join type: Inner +Join condition: None + +(40) Project [codegen id : 11] +Output [2]: [cs_ext_sales_price#19, i_manufact_id#25] +Input [4]: [cs_item_sk#18, cs_ext_sales_price#19, i_item_sk#24, i_manufact_id#25] + +(41) HashAggregate [codegen id : 11] +Input [2]: [cs_ext_sales_price#19, i_manufact_id#25] +Keys [1]: [i_manufact_id#25] +Functions [1]: [partial_sum(UnscaledValue(cs_ext_sales_price#19))] +Aggregate Attributes [1]: [sum#26] +Results [2]: [i_manufact_id#25, sum#27] + +(42) Exchange +Input [2]: [i_manufact_id#25, sum#27] +Arguments: hashpartitioning(i_manufact_id#25, 5), ENSURE_REQUIREMENTS, [plan_id=5] + +(43) HashAggregate [codegen id : 12] +Input [2]: [i_manufact_id#25, sum#27] +Keys [1]: [i_manufact_id#25] +Functions [1]: [sum(UnscaledValue(cs_ext_sales_price#19))] +Aggregate Attributes [1]: [sum(UnscaledValue(cs_ext_sales_price#19))#28] +Results [2]: [i_manufact_id#25, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#19))#28,17,2) AS total_sales#29] + +(44) Scan parquet spark_catalog.default.web_sales +Output [4]: [ws_item_sk#30, ws_bill_addr_sk#31, ws_ext_sales_price#32, ws_sold_date_sk#33] +Batched: true +Location: InMemoryFileIndex [] +PartitionFilters: [isnotnull(ws_sold_date_sk#33), dynamicpruningexpression(ws_sold_date_sk#33 IN dynamicpruning#34)] +PushedFilters: [IsNotNull(ws_bill_addr_sk), IsNotNull(ws_item_sk)] +ReadSchema: struct<ws_item_sk:int,ws_bill_addr_sk:int,ws_ext_sales_price:decimal(7,2)> + +(45) CometFilter +Input [4]: [ws_item_sk#30, ws_bill_addr_sk#31, ws_ext_sales_price#32, ws_sold_date_sk#33] +Condition : (isnotnull(ws_bill_addr_sk#31) AND isnotnull(ws_item_sk#30)) + +(46) ColumnarToRow [codegen id : 17] +Input [4]: [ws_item_sk#30, ws_bill_addr_sk#31, ws_ext_sales_price#32, ws_sold_date_sk#33] + +(47) ReusedExchange [Reuses operator id: 68] +Output [1]: [d_date_sk#35] + +(48) BroadcastHashJoin [codegen id : 17] +Left keys [1]: [ws_sold_date_sk#33] +Right keys [1]: [d_date_sk#35] +Join type: Inner +Join condition: None + +(49) Project [codegen id : 17] +Output [3]: [ws_item_sk#30, ws_bill_addr_sk#31, ws_ext_sales_price#32] +Input [5]: [ws_item_sk#30, ws_bill_addr_sk#31, ws_ext_sales_price#32, ws_sold_date_sk#33, d_date_sk#35] + +(50) ReusedExchange [Reuses operator id: 11] +Output [1]: [ca_address_sk#36] + +(51) BroadcastHashJoin [codegen id : 17] +Left keys [1]: [ws_bill_addr_sk#31] +Right keys [1]: [ca_address_sk#36] +Join type: Inner +Join condition: None + +(52) Project [codegen id : 17] +Output [2]: [ws_item_sk#30, ws_ext_sales_price#32] +Input [4]: [ws_item_sk#30, ws_bill_addr_sk#31, ws_ext_sales_price#32, ca_address_sk#36] + +(53) ReusedExchange [Reuses operator id: 23] +Output [2]: [i_item_sk#37, i_manufact_id#38] + +(54) BroadcastHashJoin [codegen id : 17] +Left keys [1]: [ws_item_sk#30] +Right keys [1]: [i_item_sk#37] +Join type: Inner +Join condition: None + +(55) Project [codegen id : 17] +Output [2]: [ws_ext_sales_price#32, i_manufact_id#38] +Input [4]: [ws_item_sk#30, ws_ext_sales_price#32, i_item_sk#37, i_manufact_id#38] + +(56) HashAggregate [codegen id : 17] +Input [2]: [ws_ext_sales_price#32, i_manufact_id#38] +Keys [1]: [i_manufact_id#38] +Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#32))] +Aggregate Attributes [1]: [sum#39] +Results [2]: [i_manufact_id#38, sum#40] + +(57) Exchange +Input [2]: [i_manufact_id#38, sum#40] +Arguments: hashpartitioning(i_manufact_id#38, 5), ENSURE_REQUIREMENTS, [plan_id=6] + +(58) HashAggregate [codegen id : 18] +Input [2]: [i_manufact_id#38, sum#40] +Keys [1]: [i_manufact_id#38] +Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#32))] +Aggregate Attributes [1]: [sum(UnscaledValue(ws_ext_sales_price#32))#41] +Results [2]: [i_manufact_id#38, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#32))#41,17,2) AS total_sales#42] + +(59) Union + +(60) HashAggregate [codegen id : 19] +Input [2]: [i_manufact_id#10, total_sales#16] +Keys [1]: [i_manufact_id#10] +Functions [1]: [partial_sum(total_sales#16)] +Aggregate Attributes [2]: [sum#43, isEmpty#44] +Results [3]: [i_manufact_id#10, sum#45, isEmpty#46] + +(61) Exchange +Input [3]: [i_manufact_id#10, sum#45, isEmpty#46] +Arguments: hashpartitioning(i_manufact_id#10, 5), ENSURE_REQUIREMENTS, [plan_id=7] + +(62) HashAggregate [codegen id : 20] +Input [3]: [i_manufact_id#10, sum#45, isEmpty#46] +Keys [1]: [i_manufact_id#10] +Functions [1]: [sum(total_sales#16)] +Aggregate Attributes [1]: [sum(total_sales#16)#47] +Results [2]: [i_manufact_id#10, sum(total_sales#16)#47 AS total_sales#48] + +(63) TakeOrderedAndProject +Input [2]: [i_manufact_id#10, total_sales#48] +Arguments: 100, [total_sales#48 ASC NULLS FIRST], [i_manufact_id#10, total_sales#48] + +===== Subqueries ===== + +Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 +BroadcastExchange (68) ++- * ColumnarToRow (67) + +- CometProject (66) + +- CometFilter (65) + +- CometScan parquet spark_catalog.default.date_dim (64) + + +(64) Scan parquet spark_catalog.default.date_dim +Output [3]: [d_date_sk#6, d_year#49, d_moy#50] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,1998), EqualTo(d_moy,5), IsNotNull(d_date_sk)] +ReadSchema: struct<d_date_sk:int,d_year:int,d_moy:int> + +(65) CometFilter +Input [3]: [d_date_sk#6, d_year#49, d_moy#50] +Condition : ((((isnotnull(d_year#49) AND isnotnull(d_moy#50)) AND (d_year#49 = 1998)) AND (d_moy#50 = 5)) AND isnotnull(d_date_sk#6)) + +(66) CometProject +Input [3]: [d_date_sk#6, d_year#49, d_moy#50] +Arguments: [d_date_sk#6], [d_date_sk#6] + +(67) ColumnarToRow [codegen id : 1] +Input [1]: [d_date_sk#6] + +(68) BroadcastExchange +Input [1]: [d_date_sk#6] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=8] + +Subquery:2 Hosting operator id = 29 Hosting Expression = cs_sold_date_sk#20 IN dynamicpruning#5 + +Subquery:3 Hosting operator id = 44 Hosting Expression = ws_sold_date_sk#33 IN dynamicpruning#5 + + diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q33/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q33/simplified.txt new file mode 100644 index 000000000..4ab82379f --- /dev/null +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q33/simplified.txt @@ -0,0 +1,105 @@ +TakeOrderedAndProject [total_sales,i_manufact_id] + WholeStageCodegen (20) + HashAggregate [i_manufact_id,sum,isEmpty] [sum(total_sales),total_sales,sum,isEmpty] + InputAdapter + Exchange [i_manufact_id] #1 + WholeStageCodegen (19) + HashAggregate [i_manufact_id,total_sales] [sum,isEmpty,sum,isEmpty] + InputAdapter + Union + WholeStageCodegen (6) + HashAggregate [i_manufact_id,sum] [sum(UnscaledValue(ss_ext_sales_price)),total_sales,sum] + InputAdapter + Exchange [i_manufact_id] #2 + WholeStageCodegen (5) + HashAggregate [i_manufact_id,ss_ext_sales_price] [sum,sum] + Project [ss_ext_sales_price,i_manufact_id] + BroadcastHashJoin [ss_item_sk,i_item_sk] + Project [ss_item_sk,ss_ext_sales_price] + BroadcastHashJoin [ss_addr_sk,ca_address_sk] + Project [ss_item_sk,ss_addr_sk,ss_ext_sales_price] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + ColumnarToRow + InputAdapter + CometFilter [ss_addr_sk,ss_item_sk] + CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_year,d_moy,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + InputAdapter + ReusedExchange [d_date_sk] #3 + InputAdapter + BroadcastExchange #4 + WholeStageCodegen (2) + ColumnarToRow + InputAdapter + CometProject [ca_address_sk] + CometFilter [ca_gmt_offset,ca_address_sk] + CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_gmt_offset] + InputAdapter + BroadcastExchange #5 + WholeStageCodegen (4) + BroadcastHashJoin [i_manufact_id,i_manufact_id] + ColumnarToRow + InputAdapter + CometFilter [i_item_sk] + CometScan parquet spark_catalog.default.item [i_item_sk,i_manufact_id] + InputAdapter + BroadcastExchange #6 + WholeStageCodegen (3) + ColumnarToRow + InputAdapter + CometProject [i_manufact_id] + CometFilter [i_category] + CometScan parquet spark_catalog.default.item [i_category,i_manufact_id] + WholeStageCodegen (12) + HashAggregate [i_manufact_id,sum] [sum(UnscaledValue(cs_ext_sales_price)),total_sales,sum] + InputAdapter + Exchange [i_manufact_id] #7 + WholeStageCodegen (11) + HashAggregate [i_manufact_id,cs_ext_sales_price] [sum,sum] + Project [cs_ext_sales_price,i_manufact_id] + BroadcastHashJoin [cs_item_sk,i_item_sk] + Project [cs_item_sk,cs_ext_sales_price] + BroadcastHashJoin [cs_bill_addr_sk,ca_address_sk] + Project [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + ColumnarToRow + InputAdapter + CometFilter [cs_bill_addr_sk,cs_item_sk] + CometScan parquet spark_catalog.default.catalog_sales [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + InputAdapter + ReusedExchange [d_date_sk] #3 + InputAdapter + ReusedExchange [ca_address_sk] #4 + InputAdapter + ReusedExchange [i_item_sk,i_manufact_id] #5 + WholeStageCodegen (18) + HashAggregate [i_manufact_id,sum] [sum(UnscaledValue(ws_ext_sales_price)),total_sales,sum] + InputAdapter + Exchange [i_manufact_id] #8 + WholeStageCodegen (17) + HashAggregate [i_manufact_id,ws_ext_sales_price] [sum,sum] + Project [ws_ext_sales_price,i_manufact_id] + BroadcastHashJoin [ws_item_sk,i_item_sk] + Project [ws_item_sk,ws_ext_sales_price] + BroadcastHashJoin [ws_bill_addr_sk,ca_address_sk] + Project [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + ColumnarToRow + InputAdapter + CometFilter [ws_bill_addr_sk,ws_item_sk] + CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + InputAdapter + ReusedExchange [d_date_sk] #3 + InputAdapter + ReusedExchange [ca_address_sk] #4 + InputAdapter + ReusedExchange [i_item_sk,i_manufact_id] #5 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 new file mode 100644 index 000000000..d0f166fe9 --- /dev/null +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q34/explain.txt @@ -0,0 +1,218 @@ +== Physical Plan == +* Sort (32) ++- Exchange (31) + +- * Project (30) + +- * BroadcastHashJoin Inner BuildRight (29) + :- * Filter (24) + : +- * HashAggregate (23) + : +- Exchange (22) + : +- * HashAggregate (21) + : +- * Project (20) + : +- * BroadcastHashJoin Inner BuildRight (19) + : :- * Project (13) + : : +- * BroadcastHashJoin Inner BuildRight (12) + : : :- * Project (6) + : : : +- * BroadcastHashJoin Inner BuildRight (5) + : : : :- * ColumnarToRow (3) + : : : : +- CometFilter (2) + : : : : +- CometScan parquet spark_catalog.default.store_sales (1) + : : : +- ReusedExchange (4) + : : +- BroadcastExchange (11) + : : +- * ColumnarToRow (10) + : : +- CometProject (9) + : : +- CometFilter (8) + : : +- CometScan parquet spark_catalog.default.store (7) + : +- BroadcastExchange (18) + : +- * ColumnarToRow (17) + : +- CometProject (16) + : +- CometFilter (15) + : +- CometScan parquet spark_catalog.default.household_demographics (14) + +- BroadcastExchange (28) + +- * ColumnarToRow (27) + +- CometFilter (26) + +- CometScan parquet spark_catalog.default.customer (25) + + +(1) Scan parquet spark_catalog.default.store_sales +Output [5]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_sold_date_sk#5] +Batched: true +Location: InMemoryFileIndex [] +PartitionFilters: [isnotnull(ss_sold_date_sk#5), dynamicpruningexpression(ss_sold_date_sk#5 IN dynamicpruning#6)] +PushedFilters: [IsNotNull(ss_store_sk), IsNotNull(ss_hdemo_sk), IsNotNull(ss_customer_sk)] +ReadSchema: struct<ss_customer_sk:int,ss_hdemo_sk:int,ss_store_sk:int,ss_ticket_number:int> + +(2) CometFilter +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)) + +(3) ColumnarToRow [codegen id : 4] +Input [5]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_sold_date_sk#5] + +(4) ReusedExchange [Reuses operator id: 37] +Output [1]: [d_date_sk#7] + +(5) BroadcastHashJoin [codegen id : 4] +Left keys [1]: [ss_sold_date_sk#5] +Right keys [1]: [d_date_sk#7] +Join type: Inner +Join condition: None + +(6) Project [codegen id : 4] +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] + +(7) Scan parquet spark_catalog.default.store +Output [2]: [s_store_sk#8, s_county#9] +Batched: true +Location [not included in comparison]/{warehouse_dir}/store] +PushedFilters: [IsNotNull(s_county), EqualTo(s_county,Williamson County), IsNotNull(s_store_sk)] +ReadSchema: struct<s_store_sk:int,s_county:string> + +(8) CometFilter +Input [2]: [s_store_sk#8, s_county#9] +Condition : ((isnotnull(s_county#9) AND (s_county#9 = Williamson County)) AND isnotnull(s_store_sk#8)) + +(9) CometProject +Input [2]: [s_store_sk#8, s_county#9] +Arguments: [s_store_sk#8], [s_store_sk#8] + +(10) ColumnarToRow [codegen id : 2] +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] +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] +Output [3]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_ticket_number#4] +Input [5]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4, s_store_sk#8] + +(14) Scan parquet spark_catalog.default.household_demographics +Output [4]: [hd_demo_sk#10, hd_buy_potential#11, hd_dep_count#12, hd_vehicle_count#13] +Batched: true +Location [not included in comparison]/{warehouse_dir}/household_demographics] +PushedFilters: [IsNotNull(hd_vehicle_count), Or(EqualTo(hd_buy_potential,>10000 ),EqualTo(hd_buy_potential,unknown )), GreaterThan(hd_vehicle_count,0), IsNotNull(hd_demo_sk)] +ReadSchema: struct<hd_demo_sk:int,hd_buy_potential:string,hd_dep_count:int,hd_vehicle_count:int> + +(15) CometFilter +Input [4]: [hd_demo_sk#10, hd_buy_potential#11, hd_dep_count#12, hd_vehicle_count#13] +Condition : ((((isnotnull(hd_vehicle_count#13) AND ((hd_buy_potential#11 = >10000 ) OR (hd_buy_potential#11 = unknown ))) AND (hd_vehicle_count#13 > 0)) AND CASE WHEN (hd_vehicle_count#13 > 0) THEN ((cast(hd_dep_count#12 as double) / cast(hd_vehicle_count#13 as double)) > 1.2) END) AND isnotnull(hd_demo_sk#10)) + +(16) CometProject +Input [4]: [hd_demo_sk#10, hd_buy_potential#11, hd_dep_count#12, hd_vehicle_count#13] +Arguments: [hd_demo_sk#10], [hd_demo_sk#10] + +(17) ColumnarToRow [codegen id : 3] +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] +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] +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] +Input [2]: [ss_customer_sk#1, ss_ticket_number#4] +Keys [2]: [ss_ticket_number#4, ss_customer_sk#1] +Functions [1]: [partial_count(1)] +Aggregate Attributes [1]: [count#14] +Results [3]: [ss_ticket_number#4, ss_customer_sk#1, count#15] + +(22) Exchange +Input [3]: [ss_ticket_number#4, ss_customer_sk#1, count#15] +Arguments: hashpartitioning(ss_ticket_number#4, ss_customer_sk#1, 5), ENSURE_REQUIREMENTS, [plan_id=3] + +(23) HashAggregate [codegen id : 6] +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] + +(24) Filter [codegen id : 6] +Input [3]: [ss_ticket_number#4, ss_customer_sk#1, cnt#17] +Condition : ((cnt#17 >= 15) AND (cnt#17 <= 20)) + +(25) Scan parquet spark_catalog.default.customer +Output [5]: [c_customer_sk#18, c_salutation#19, c_first_name#20, c_last_name#21, c_preferred_cust_flag#22] +Batched: true +Location [not included in comparison]/{warehouse_dir}/customer] +PushedFilters: [IsNotNull(c_customer_sk)] +ReadSchema: struct<c_customer_sk:int,c_salutation:string,c_first_name:string,c_last_name:string,c_preferred_cust_flag:string> + +(26) CometFilter +Input [5]: [c_customer_sk#18, c_salutation#19, c_first_name#20, c_last_name#21, c_preferred_cust_flag#22] +Condition : isnotnull(c_customer_sk#18) + +(27) ColumnarToRow [codegen id : 5] +Input [5]: [c_customer_sk#18, c_salutation#19, c_first_name#20, c_last_name#21, c_preferred_cust_flag#22] + +(28) BroadcastExchange +Input [5]: [c_customer_sk#18, c_salutation#19, c_first_name#20, c_last_name#21, c_preferred_cust_flag#22] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=4] + +(29) BroadcastHashJoin [codegen id : 6] +Left keys [1]: [ss_customer_sk#1] +Right keys [1]: [c_customer_sk#18] +Join type: Inner +Join condition: None + +(30) Project [codegen id : 6] +Output [6]: [c_last_name#21, c_first_name#20, c_salutation#19, c_preferred_cust_flag#22, ss_ticket_number#4, cnt#17] +Input [8]: [ss_ticket_number#4, ss_customer_sk#1, cnt#17, c_customer_sk#18, c_salutation#19, c_first_name#20, c_last_name#21, c_preferred_cust_flag#22] + +(31) Exchange +Input [6]: [c_last_name#21, c_first_name#20, c_salutation#19, c_preferred_cust_flag#22, ss_ticket_number#4, cnt#17] +Arguments: rangepartitioning(c_last_name#21 ASC NULLS FIRST, c_first_name#20 ASC NULLS FIRST, c_salutation#19 ASC NULLS FIRST, c_preferred_cust_flag#22 DESC NULLS LAST, 5), ENSURE_REQUIREMENTS, [plan_id=5] + +(32) Sort [codegen id : 7] +Input [6]: [c_last_name#21, c_first_name#20, c_salutation#19, c_preferred_cust_flag#22, ss_ticket_number#4, cnt#17] +Arguments: [c_last_name#21 ASC NULLS FIRST, c_first_name#20 ASC NULLS FIRST, c_salutation#19 ASC NULLS FIRST, c_preferred_cust_flag#22 DESC NULLS LAST], true, 0 + +===== Subqueries ===== + +Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#5 IN dynamicpruning#6 +BroadcastExchange (37) ++- * ColumnarToRow (36) + +- CometProject (35) + +- CometFilter (34) + +- CometScan parquet spark_catalog.default.date_dim (33) + + +(33) Scan parquet spark_catalog.default.date_dim +Output [3]: [d_date_sk#7, d_year#23, d_dom#24] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [Or(And(GreaterThanOrEqual(d_dom,1),LessThanOrEqual(d_dom,3)),And(GreaterThanOrEqual(d_dom,25),LessThanOrEqual(d_dom,28))), In(d_year, [1999,2000,2001]), IsNotNull(d_date_sk)] +ReadSchema: struct<d_date_sk:int,d_year:int,d_dom:int> + +(34) CometFilter +Input [3]: [d_date_sk#7, d_year#23, d_dom#24] +Condition : (((((d_dom#24 >= 1) AND (d_dom#24 <= 3)) OR ((d_dom#24 >= 25) AND (d_dom#24 <= 28))) AND d_year#23 IN (1999,2000,2001)) AND isnotnull(d_date_sk#7)) + +(35) CometProject +Input [3]: [d_date_sk#7, d_year#23, d_dom#24] +Arguments: [d_date_sk#7], [d_date_sk#7] + +(36) ColumnarToRow [codegen id : 1] +Input [1]: [d_date_sk#7] + +(37) BroadcastExchange +Input [1]: [d_date_sk#7] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=6] + + diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q34/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q34/simplified.txt new file mode 100644 index 000000000..80405a784 --- /dev/null +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q34/simplified.txt @@ -0,0 +1,56 @@ +WholeStageCodegen (7) + Sort [c_last_name,c_first_name,c_salutation,c_preferred_cust_flag] + InputAdapter + Exchange [c_last_name,c_first_name,c_salutation,c_preferred_cust_flag] #1 + WholeStageCodegen (6) + Project [c_last_name,c_first_name,c_salutation,c_preferred_cust_flag,ss_ticket_number,cnt] + BroadcastHashJoin [ss_customer_sk,c_customer_sk] + Filter [cnt] + HashAggregate [ss_ticket_number,ss_customer_sk,count] [count(1),cnt,count] + InputAdapter + Exchange [ss_ticket_number,ss_customer_sk] #2 + WholeStageCodegen (4) + HashAggregate [ss_ticket_number,ss_customer_sk] [count,count] + Project [ss_customer_sk,ss_ticket_number] + BroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] + Project [ss_customer_sk,ss_hdemo_sk,ss_ticket_number] + BroadcastHashJoin [ss_store_sk,s_store_sk] + Project [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + ColumnarToRow + InputAdapter + CometFilter [ss_store_sk,ss_hdemo_sk,ss_customer_sk] + CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_dom,d_year,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dom] + InputAdapter + ReusedExchange [d_date_sk] #3 + InputAdapter + BroadcastExchange #4 + WholeStageCodegen (2) + ColumnarToRow + InputAdapter + CometProject [s_store_sk] + CometFilter [s_county,s_store_sk] + CometScan parquet spark_catalog.default.store [s_store_sk,s_county] + InputAdapter + BroadcastExchange #5 + WholeStageCodegen (3) + ColumnarToRow + InputAdapter + CometProject [hd_demo_sk] + CometFilter [hd_vehicle_count,hd_buy_potential,hd_dep_count,hd_demo_sk] + CometScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_buy_potential,hd_dep_count,hd_vehicle_count] + InputAdapter + BroadcastExchange #6 + WholeStageCodegen (5) + ColumnarToRow + InputAdapter + CometFilter [c_customer_sk] + CometScan parquet spark_catalog.default.customer [c_customer_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q35/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q35/explain.txt new file mode 100644 index 000000000..a46018cdd --- /dev/null +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q35/explain.txt @@ -0,0 +1,281 @@ +== Physical Plan == +TakeOrderedAndProject (42) ++- * HashAggregate (41) + +- Exchange (40) + +- * HashAggregate (39) + +- * Project (38) + +- * BroadcastHashJoin Inner BuildRight (37) + :- * Project (32) + : +- * BroadcastHashJoin Inner BuildRight (31) + : :- * Project (26) + : : +- * Filter (25) + : : +- * BroadcastHashJoin ExistenceJoin(exists#1) BuildRight (24) + : : :- * BroadcastHashJoin ExistenceJoin(exists#2) BuildRight (17) + : : : :- * BroadcastHashJoin LeftSemi BuildRight (10) + : : : : :- * ColumnarToRow (3) + : : : : : +- CometFilter (2) + : : : : : +- CometScan parquet spark_catalog.default.customer (1) + : : : : +- BroadcastExchange (9) + : : : : +- * Project (8) + : : : : +- * BroadcastHashJoin Inner BuildRight (7) + : : : : :- * ColumnarToRow (5) + : : : : : +- CometScan parquet spark_catalog.default.store_sales (4) + : : : : +- ReusedExchange (6) + : : : +- BroadcastExchange (16) + : : : +- * Project (15) + : : : +- * BroadcastHashJoin Inner BuildRight (14) + : : : :- * ColumnarToRow (12) + : : : : +- CometScan parquet spark_catalog.default.web_sales (11) + : : : +- ReusedExchange (13) + : : +- BroadcastExchange (23) + : : +- * Project (22) + : : +- * BroadcastHashJoin Inner BuildRight (21) + : : :- * ColumnarToRow (19) + : : : +- CometScan parquet spark_catalog.default.catalog_sales (18) + : : +- ReusedExchange (20) + : +- BroadcastExchange (30) + : +- * ColumnarToRow (29) + : +- CometFilter (28) + : +- CometScan parquet spark_catalog.default.customer_address (27) + +- BroadcastExchange (36) + +- * ColumnarToRow (35) + +- CometFilter (34) + +- CometScan parquet spark_catalog.default.customer_demographics (33) + + +(1) Scan parquet spark_catalog.default.customer +Output [3]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5] +Batched: true +Location [not included in comparison]/{warehouse_dir}/customer] +PushedFilters: [IsNotNull(c_current_addr_sk), IsNotNull(c_current_cdemo_sk)] +ReadSchema: struct<c_customer_sk:int,c_current_cdemo_sk:int,c_current_addr_sk:int> + +(2) CometFilter +Input [3]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5] +Condition : (isnotnull(c_current_addr_sk#5) AND isnotnull(c_current_cdemo_sk#4)) + +(3) ColumnarToRow [codegen id : 9] +Input [3]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5] + +(4) Scan parquet spark_catalog.default.store_sales +Output [2]: [ss_customer_sk#6, ss_sold_date_sk#7] +Batched: true +Location: InMemoryFileIndex [] +PartitionFilters: [isnotnull(ss_sold_date_sk#7), dynamicpruningexpression(ss_sold_date_sk#7 IN dynamicpruning#8)] +ReadSchema: struct<ss_customer_sk:int> + +(5) ColumnarToRow [codegen id : 2] +Input [2]: [ss_customer_sk#6, ss_sold_date_sk#7] + +(6) ReusedExchange [Reuses operator id: 47] +Output [1]: [d_date_sk#9] + +(7) BroadcastHashJoin [codegen id : 2] +Left keys [1]: [ss_sold_date_sk#7] +Right keys [1]: [d_date_sk#9] +Join type: Inner +Join condition: None + +(8) Project [codegen id : 2] +Output [1]: [ss_customer_sk#6] +Input [3]: [ss_customer_sk#6, ss_sold_date_sk#7, d_date_sk#9] + +(9) BroadcastExchange +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] +Left keys [1]: [c_customer_sk#3] +Right keys [1]: [ss_customer_sk#6] +Join type: LeftSemi +Join condition: None + +(11) Scan parquet spark_catalog.default.web_sales +Output [2]: [ws_bill_customer_sk#10, ws_sold_date_sk#11] +Batched: true +Location: InMemoryFileIndex [] +PartitionFilters: [isnotnull(ws_sold_date_sk#11), dynamicpruningexpression(ws_sold_date_sk#11 IN dynamicpruning#12)] +ReadSchema: struct<ws_bill_customer_sk:int> + +(12) ColumnarToRow [codegen id : 4] +Input [2]: [ws_bill_customer_sk#10, ws_sold_date_sk#11] + +(13) ReusedExchange [Reuses operator id: 47] +Output [1]: [d_date_sk#13] + +(14) BroadcastHashJoin [codegen id : 4] +Left keys [1]: [ws_sold_date_sk#11] +Right keys [1]: [d_date_sk#13] +Join type: Inner +Join condition: None + +(15) Project [codegen id : 4] +Output [1]: [ws_bill_customer_sk#10] +Input [3]: [ws_bill_customer_sk#10, ws_sold_date_sk#11, d_date_sk#13] + +(16) BroadcastExchange +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] +Left keys [1]: [c_customer_sk#3] +Right keys [1]: [ws_bill_customer_sk#10] +Join type: ExistenceJoin(exists#2) +Join condition: None + +(18) Scan parquet spark_catalog.default.catalog_sales +Output [2]: [cs_ship_customer_sk#14, cs_sold_date_sk#15] +Batched: true +Location: InMemoryFileIndex [] +PartitionFilters: [isnotnull(cs_sold_date_sk#15), dynamicpruningexpression(cs_sold_date_sk#15 IN dynamicpruning#16)] +ReadSchema: struct<cs_ship_customer_sk:int> + +(19) ColumnarToRow [codegen id : 6] +Input [2]: [cs_ship_customer_sk#14, cs_sold_date_sk#15] + +(20) ReusedExchange [Reuses operator id: 47] +Output [1]: [d_date_sk#17] + +(21) BroadcastHashJoin [codegen id : 6] +Left keys [1]: [cs_sold_date_sk#15] +Right keys [1]: [d_date_sk#17] +Join type: Inner +Join condition: None + +(22) Project [codegen id : 6] +Output [1]: [cs_ship_customer_sk#14] +Input [3]: [cs_ship_customer_sk#14, cs_sold_date_sk#15, d_date_sk#17] + +(23) BroadcastExchange +Input [1]: [cs_ship_customer_sk#14] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] + +(24) BroadcastHashJoin [codegen id : 9] +Left keys [1]: [c_customer_sk#3] +Right keys [1]: [cs_ship_customer_sk#14] +Join type: ExistenceJoin(exists#1) +Join condition: None + +(25) Filter [codegen id : 9] +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] +Output [2]: [c_current_cdemo_sk#4, c_current_addr_sk#5] +Input [5]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5, exists#2, exists#1] + +(27) Scan parquet spark_catalog.default.customer_address +Output [2]: [ca_address_sk#18, ca_state#19] +Batched: true +Location [not included in comparison]/{warehouse_dir}/customer_address] +PushedFilters: [IsNotNull(ca_address_sk)] +ReadSchema: struct<ca_address_sk:int,ca_state:string> + +(28) CometFilter +Input [2]: [ca_address_sk#18, ca_state#19] +Condition : isnotnull(ca_address_sk#18) + +(29) ColumnarToRow [codegen id : 7] +Input [2]: [ca_address_sk#18, ca_state#19] + +(30) BroadcastExchange +Input [2]: [ca_address_sk#18, ca_state#19] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=4] + +(31) BroadcastHashJoin [codegen id : 9] +Left keys [1]: [c_current_addr_sk#5] +Right keys [1]: [ca_address_sk#18] +Join type: Inner +Join condition: None + +(32) Project [codegen id : 9] +Output [2]: [c_current_cdemo_sk#4, ca_state#19] +Input [4]: [c_current_cdemo_sk#4, c_current_addr_sk#5, ca_address_sk#18, ca_state#19] + +(33) Scan parquet spark_catalog.default.customer_demographics +Output [6]: [cd_demo_sk#20, cd_gender#21, cd_marital_status#22, cd_dep_count#23, cd_dep_employed_count#24, cd_dep_college_count#25] +Batched: true +Location [not included in comparison]/{warehouse_dir}/customer_demographics] +PushedFilters: [IsNotNull(cd_demo_sk)] +ReadSchema: struct<cd_demo_sk:int,cd_gender:string,cd_marital_status:string,cd_dep_count:int,cd_dep_employed_count:int,cd_dep_college_count:int> + +(34) CometFilter +Input [6]: [cd_demo_sk#20, cd_gender#21, cd_marital_status#22, cd_dep_count#23, cd_dep_employed_count#24, cd_dep_college_count#25] +Condition : isnotnull(cd_demo_sk#20) + +(35) ColumnarToRow [codegen id : 8] +Input [6]: [cd_demo_sk#20, cd_gender#21, cd_marital_status#22, cd_dep_count#23, cd_dep_employed_count#24, cd_dep_college_count#25] + +(36) BroadcastExchange +Input [6]: [cd_demo_sk#20, cd_gender#21, cd_marital_status#22, cd_dep_count#23, cd_dep_employed_count#24, cd_dep_college_count#25] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=5] + +(37) BroadcastHashJoin [codegen id : 9] +Left keys [1]: [c_current_cdemo_sk#4] +Right keys [1]: [cd_demo_sk#20] +Join type: Inner +Join condition: None + +(38) Project [codegen id : 9] +Output [6]: [ca_state#19, cd_gender#21, cd_marital_status#22, cd_dep_count#23, cd_dep_employed_count#24, cd_dep_college_count#25] +Input [8]: [c_current_cdemo_sk#4, ca_state#19, cd_demo_sk#20, cd_gender#21, cd_marital_status#22, cd_dep_count#23, cd_dep_employed_count#24, cd_dep_college_count#25] + +(39) HashAggregate [codegen id : 9] +Input [6]: [ca_state#19, cd_gender#21, cd_marital_status#22, cd_dep_count#23, cd_dep_employed_count#24, cd_dep_college_count#25] +Keys [6]: [ca_state#19, cd_gender#21, cd_marital_status#22, cd_dep_count#23, cd_dep_employed_count#24, cd_dep_college_count#25] +Functions [10]: [partial_count(1), partial_min(cd_dep_count#23), partial_max(cd_dep_count#23), partial_avg(cd_dep_count#23), partial_min(cd_dep_employed_count#24), partial_max(cd_dep_employed_count#24), partial_avg(cd_dep_employed_count#24), partial_min(cd_dep_college_count#25), partial_max(cd_dep_college_count#25), partial_avg(cd_dep_college_count#25)] +Aggregate Attributes [13]: [count#26, min#27, max#28, sum#29, count#30, min#31, max#32, sum#33, count#34, min#35, max#36, sum#37, count#38] +Results [19]: [ca_state#19, cd_gender#21, cd_marital_status#22, cd_dep_count#23, cd_dep_employed_count#24, cd_dep_college_count#25, count#39, min#40, max#41, sum#42, count#43, min#44, max#45, sum#46, count#47, min#48, max#49, sum#50, count#51] + +(40) Exchange +Input [19]: [ca_state#19, cd_gender#21, cd_marital_status#22, cd_dep_count#23, cd_dep_employed_count#24, cd_dep_college_count#25, count#39, min#40, max#41, sum#42, count#43, min#44, max#45, sum#46, count#47, min#48, max#49, sum#50, count#51] +Arguments: hashpartitioning(ca_state#19, cd_gender#21, cd_marital_status#22, cd_dep_count#23, cd_dep_employed_count#24, cd_dep_college_count#25, 5), ENSURE_REQUIREMENTS, [plan_id=6] + +(41) HashAggregate [codegen id : 10] +Input [19]: [ca_state#19, cd_gender#21, cd_marital_status#22, cd_dep_count#23, cd_dep_employed_count#24, cd_dep_college_count#25, count#39, min#40, max#41, sum#42, count#43, min#44, max#45, sum#46, count#47, min#48, max#49, sum#50, count#51] +Keys [6]: [ca_state#19, cd_gender#21, cd_marital_status#22, cd_dep_count#23, cd_dep_employed_count#24, cd_dep_college_count#25] +Functions [10]: [count(1), min(cd_dep_count#23), max(cd_dep_count#23), avg(cd_dep_count#23), min(cd_dep_employed_count#24), max(cd_dep_employed_count#24), avg(cd_dep_employed_count#24), min(cd_dep_college_count#25), max(cd_dep_college_count#25), avg(cd_dep_college_count#25)] +Aggregate Attributes [10]: [count(1)#52, min(cd_dep_count#23)#53, max(cd_dep_count#23)#54, avg(cd_dep_count#23)#55, min(cd_dep_employed_count#24)#56, max(cd_dep_employed_count#24)#57, avg(cd_dep_employed_count#24)#58, min(cd_dep_college_count#25)#59, max(cd_dep_college_count#25)#60, avg(cd_dep_college_count#25)#61] +Results [18]: [ca_state#19, cd_gender#21, cd_marital_status#22, count(1)#52 AS cnt1#62, min(cd_dep_count#23)#53 AS min(cd_dep_count)#63, max(cd_dep_count#23)#54 AS max(cd_dep_count)#64, avg(cd_dep_count#23)#55 AS avg(cd_dep_count)#65, cd_dep_employed_count#24, count(1)#52 AS cnt2#66, min(cd_dep_employed_count#24)#56 AS min(cd_dep_employed_count)#67, max(cd_dep_employed_count#24)#57 AS max(cd_dep_employed_count)#68, avg(cd_dep_employed_count#24)#58 AS avg(cd_dep_employed_count)#69, cd_dep_college_count#25, count(1)#52 AS cnt3#70, min(cd_dep_college_count#25)#59 AS min(cd_dep_college_count)#71, max(cd_dep_college_count#25)#60 AS max(cd_dep_college_count)#72, avg(cd_dep_college_count#25)#61 AS avg(cd_dep_college_count)#73, cd_dep_count#23] + +(42) TakeOrderedAndProject +Input [18]: [ca_state#19, cd_gender#21, cd_marital_status#22, cnt1#62, min(cd_dep_count)#63, max(cd_dep_count)#64, avg(cd_dep_count)#65, cd_dep_employed_count#24, cnt2#66, min(cd_dep_employed_count)#67, max(cd_dep_employed_count)#68, avg(cd_dep_employed_count)#69, cd_dep_college_count#25, cnt3#70, min(cd_dep_college_count)#71, max(cd_dep_college_count)#72, avg(cd_dep_college_count)#73, cd_dep_count#23] +Arguments: 100, [ca_state#19 ASC NULLS FIRST, cd_gender#21 ASC NULLS FIRST, cd_marital_status#22 ASC NULLS FIRST, cd_dep_count#23 ASC NULLS FIRST, cd_dep_employed_count#24 ASC NULLS FIRST, cd_dep_college_count#25 ASC NULLS FIRST], [ca_state#19, cd_gender#21, cd_marital_status#22, cnt1#62, min(cd_dep_count)#63, max(cd_dep_count)#64, avg(cd_dep_count)#65, cd_dep_employed_count#24, cnt2#66, min(cd_dep_employed_count)#67, max(cd_dep_employed_count)#68, avg(cd_dep_employed_count)#69, cd_dep_college_count#25, cnt3#70, min(cd_dep_college_count)#71, max(cd_dep_college_count)#72, avg(cd_dep_college_count)#73] + +===== Subqueries ===== + +Subquery:1 Hosting operator id = 4 Hosting Expression = ss_sold_date_sk#7 IN dynamicpruning#8 +BroadcastExchange (47) ++- * ColumnarToRow (46) + +- CometProject (45) + +- CometFilter (44) + +- CometScan parquet spark_catalog.default.date_dim (43) + + +(43) Scan parquet spark_catalog.default.date_dim +Output [3]: [d_date_sk#9, d_year#74, d_qoy#75] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_year), IsNotNull(d_qoy), EqualTo(d_year,2002), LessThan(d_qoy,4), IsNotNull(d_date_sk)] +ReadSchema: struct<d_date_sk:int,d_year:int,d_qoy:int> + +(44) CometFilter +Input [3]: [d_date_sk#9, d_year#74, d_qoy#75] +Condition : ((((isnotnull(d_year#74) AND isnotnull(d_qoy#75)) AND (d_year#74 = 2002)) AND (d_qoy#75 < 4)) AND isnotnull(d_date_sk#9)) + +(45) CometProject +Input [3]: [d_date_sk#9, d_year#74, d_qoy#75] +Arguments: [d_date_sk#9], [d_date_sk#9] + +(46) ColumnarToRow [codegen id : 1] +Input [1]: [d_date_sk#9] + +(47) BroadcastExchange +Input [1]: [d_date_sk#9] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=7] + +Subquery:2 Hosting operator id = 11 Hosting Expression = ws_sold_date_sk#11 IN dynamicpruning#8 + +Subquery:3 Hosting operator id = 18 Hosting Expression = cs_sold_date_sk#15 IN dynamicpruning#8 + + diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q35/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q35/simplified.txt new file mode 100644 index 000000000..ea0ef274e --- /dev/null +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q35/simplified.txt @@ -0,0 +1,74 @@ +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) + HashAggregate [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count,count,min,max,sum,count,min,max,sum,count,min,max,sum,count] [count(1),min(cd_dep_count),max(cd_dep_count),avg(cd_dep_count),min(cd_dep_employed_count),max(cd_dep_employed_count),avg(cd_dep_employed_count),min(cd_dep_college_count),max(cd_dep_college_count),avg(cd_dep_college_count),cnt1,min(cd_dep_count),max(cd_dep_count),avg(cd_dep_count),cnt2,min(cd_dep_employed_count),max(cd_dep_employed_count),avg(cd_dep_employed_count),cnt3,min(cd_dep_college_count),max(cd_dep_college_count),avg(cd_dep_college_count),count,min,max,sum,count,min,max,sum,count,min,max,sum,count] + InputAdapter + Exchange [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] #1 + WholeStageCodegen (9) + HashAggregate [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] [count,min,max,sum,count,min,max,sum,count,min,max,sum,count,count,min,max,sum,count,min,max,sum,count,min,max,sum,count] + Project [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] + BroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] + Project [c_current_cdemo_sk,ca_state] + BroadcastHashJoin [c_current_addr_sk,ca_address_sk] + Project [c_current_cdemo_sk,c_current_addr_sk] + Filter [exists,exists] + BroadcastHashJoin [c_customer_sk,cs_ship_customer_sk] + BroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] + BroadcastHashJoin [c_customer_sk,ss_customer_sk] + ColumnarToRow + InputAdapter + CometFilter [c_current_addr_sk,c_current_cdemo_sk] + CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] + InputAdapter + BroadcastExchange #2 + WholeStageCodegen (2) + Project [ss_customer_sk] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + ColumnarToRow + InputAdapter + CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_year,d_qoy,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] + InputAdapter + ReusedExchange [d_date_sk] #3 + InputAdapter + BroadcastExchange #4 + WholeStageCodegen (4) + Project [ws_bill_customer_sk] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + ColumnarToRow + InputAdapter + CometScan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + InputAdapter + ReusedExchange [d_date_sk] #3 + InputAdapter + BroadcastExchange #5 + WholeStageCodegen (6) + Project [cs_ship_customer_sk] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + ColumnarToRow + InputAdapter + CometScan parquet spark_catalog.default.catalog_sales [cs_ship_customer_sk,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + InputAdapter + ReusedExchange [d_date_sk] #3 + InputAdapter + BroadcastExchange #6 + WholeStageCodegen (7) + ColumnarToRow + InputAdapter + CometFilter [ca_address_sk] + CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] + InputAdapter + BroadcastExchange #7 + WholeStageCodegen (8) + ColumnarToRow + InputAdapter + CometFilter [cd_demo_sk] + CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] 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 new file mode 100644 index 000000000..39b838157 --- /dev/null +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q36/explain.txt @@ -0,0 +1,194 @@ +== Physical Plan == +TakeOrderedAndProject (28) ++- * Project (27) + +- Window (26) + +- * Sort (25) + +- Exchange (24) + +- * HashAggregate (23) + +- Exchange (22) + +- * HashAggregate (21) + +- * Expand (20) + +- * Project (19) + +- * BroadcastHashJoin Inner BuildRight (18) + :- * Project (12) + : +- * BroadcastHashJoin Inner BuildRight (11) + : :- * Project (6) + : : +- * BroadcastHashJoin Inner BuildRight (5) + : : :- * ColumnarToRow (3) + : : : +- CometFilter (2) + : : : +- CometScan parquet spark_catalog.default.store_sales (1) + : : +- ReusedExchange (4) + : +- BroadcastExchange (10) + : +- * ColumnarToRow (9) + : +- CometFilter (8) + : +- CometScan parquet spark_catalog.default.item (7) + +- BroadcastExchange (17) + +- * ColumnarToRow (16) + +- CometProject (15) + +- CometFilter (14) + +- CometScan parquet spark_catalog.default.store (13) + + +(1) Scan parquet spark_catalog.default.store_sales +Output [5]: [ss_item_sk#1, ss_store_sk#2, ss_ext_sales_price#3, ss_net_profit#4, ss_sold_date_sk#5] +Batched: true +Location: InMemoryFileIndex [] +PartitionFilters: [isnotnull(ss_sold_date_sk#5), dynamicpruningexpression(ss_sold_date_sk#5 IN dynamicpruning#6)] +PushedFilters: [IsNotNull(ss_item_sk), IsNotNull(ss_store_sk)] +ReadSchema: struct<ss_item_sk:int,ss_store_sk:int,ss_ext_sales_price:decimal(7,2),ss_net_profit:decimal(7,2)> + +(2) CometFilter +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)) + +(3) ColumnarToRow [codegen id : 4] +Input [5]: [ss_item_sk#1, ss_store_sk#2, ss_ext_sales_price#3, ss_net_profit#4, ss_sold_date_sk#5] + +(4) ReusedExchange [Reuses operator id: 33] +Output [1]: [d_date_sk#7] + +(5) BroadcastHashJoin [codegen id : 4] +Left keys [1]: [ss_sold_date_sk#5] +Right keys [1]: [d_date_sk#7] +Join type: Inner +Join condition: None + +(6) Project [codegen id : 4] +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] + +(7) Scan parquet spark_catalog.default.item +Output [3]: [i_item_sk#8, i_class#9, i_category#10] +Batched: true +Location [not included in comparison]/{warehouse_dir}/item] +PushedFilters: [IsNotNull(i_item_sk)] +ReadSchema: struct<i_item_sk:int,i_class:string,i_category:string> + +(8) CometFilter +Input [3]: [i_item_sk#8, i_class#9, i_category#10] +Condition : isnotnull(i_item_sk#8) + +(9) ColumnarToRow [codegen id : 2] +Input [3]: [i_item_sk#8, i_class#9, i_category#10] + +(10) BroadcastExchange +Input [3]: [i_item_sk#8, i_class#9, i_category#10] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=1] + +(11) BroadcastHashJoin [codegen id : 4] +Left keys [1]: [ss_item_sk#1] +Right keys [1]: [i_item_sk#8] +Join type: Inner +Join condition: None + +(12) Project [codegen id : 4] +Output [5]: [ss_store_sk#2, ss_ext_sales_price#3, ss_net_profit#4, i_class#9, i_category#10] +Input [7]: [ss_item_sk#1, ss_store_sk#2, ss_ext_sales_price#3, ss_net_profit#4, i_item_sk#8, i_class#9, i_category#10] + +(13) Scan parquet spark_catalog.default.store +Output [2]: [s_store_sk#11, s_state#12] +Batched: true +Location [not included in comparison]/{warehouse_dir}/store] +PushedFilters: [IsNotNull(s_state), EqualTo(s_state,TN), IsNotNull(s_store_sk)] +ReadSchema: struct<s_store_sk:int,s_state:string> + +(14) CometFilter +Input [2]: [s_store_sk#11, s_state#12] +Condition : ((isnotnull(s_state#12) AND (s_state#12 = TN)) AND isnotnull(s_store_sk#11)) + +(15) CometProject +Input [2]: [s_store_sk#11, s_state#12] +Arguments: [s_store_sk#11], [s_store_sk#11] + +(16) ColumnarToRow [codegen id : 3] +Input [1]: [s_store_sk#11] + +(17) BroadcastExchange +Input [1]: [s_store_sk#11] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] + +(18) BroadcastHashJoin [codegen id : 4] +Left keys [1]: [ss_store_sk#2] +Right keys [1]: [s_store_sk#11] +Join type: Inner +Join condition: None + +(19) Project [codegen id : 4] +Output [4]: [ss_ext_sales_price#3, ss_net_profit#4, i_category#10, i_class#9] +Input [6]: [ss_store_sk#2, ss_ext_sales_price#3, ss_net_profit#4, i_class#9, i_category#10, s_store_sk#11] + +(20) Expand [codegen id : 4] +Input [4]: [ss_ext_sales_price#3, ss_net_profit#4, i_category#10, i_class#9] +Arguments: [[ss_ext_sales_price#3, ss_net_profit#4, i_category#10, i_class#9, 0], [ss_ext_sales_price#3, ss_net_profit#4, i_category#10, null, 1], [ss_ext_sales_price#3, ss_net_profit#4, null, null, 3]], [ss_ext_sales_price#3, ss_net_profit#4, i_category#13, i_class#14, spark_grouping_id#15] + +(21) HashAggregate [codegen id : 4] +Input [5]: [ss_ext_sales_price#3, ss_net_profit#4, i_category#13, i_class#14, spark_grouping_id#15] +Keys [3]: [i_category#13, i_class#14, spark_grouping_id#15] +Functions [2]: [partial_sum(UnscaledValue(ss_net_profit#4)), partial_sum(UnscaledValue(ss_ext_sales_price#3))] +Aggregate Attributes [2]: [sum#16, sum#17] +Results [5]: [i_category#13, i_class#14, spark_grouping_id#15, sum#18, sum#19] + +(22) Exchange +Input [5]: [i_category#13, i_class#14, spark_grouping_id#15, sum#18, sum#19] +Arguments: hashpartitioning(i_category#13, i_class#14, spark_grouping_id#15, 5), ENSURE_REQUIREMENTS, [plan_id=3] + +(23) HashAggregate [codegen id : 5] +Input [5]: [i_category#13, i_class#14, spark_grouping_id#15, sum#18, sum#19] +Keys [3]: [i_category#13, i_class#14, spark_grouping_id#15] +Functions [2]: [sum(UnscaledValue(ss_net_profit#4)), sum(UnscaledValue(ss_ext_sales_price#3))] +Aggregate Attributes [2]: [sum(UnscaledValue(ss_net_profit#4))#20, sum(UnscaledValue(ss_ext_sales_price#3))#21] +Results [7]: [(MakeDecimal(sum(UnscaledValue(ss_net_profit#4))#20,17,2) / MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#3))#21,17,2)) AS gross_margin#22, i_category#13, i_class#14, (cast((shiftright(spark_grouping_id#15, 1) & 1) as tinyint) + cast((shiftright(spark_grouping_id#15, 0) & 1) as tinyint)) AS lochierarchy#23, (MakeDecimal(sum(UnscaledValue(ss_net_profit#4))#20,17,2) / MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#3))#21,17,2)) AS _w0#24, (cast((shiftright(spark_grouping_id#15, 1) & 1) as tinyint) + cast((shiftright(spark_grouping_id#15, 0) & 1) as tinyint)) AS _w1#25, CASE WHEN (cast((shiftright(spark_grouping_id#15, 0) & 1) as tinyint) = 0) THEN i_category#13 END AS _w2#26] + +(24) Exchange +Input [7]: [gross_margin#22, i_category#13, i_class#14, lochierarchy#23, _w0#24, _w1#25, _w2#26] +Arguments: hashpartitioning(_w1#25, _w2#26, 5), ENSURE_REQUIREMENTS, [plan_id=4] + +(25) Sort [codegen id : 6] +Input [7]: [gross_margin#22, i_category#13, i_class#14, lochierarchy#23, _w0#24, _w1#25, _w2#26] +Arguments: [_w1#25 ASC NULLS FIRST, _w2#26 ASC NULLS FIRST, _w0#24 ASC NULLS FIRST], false, 0 + +(26) Window +Input [7]: [gross_margin#22, i_category#13, i_class#14, lochierarchy#23, _w0#24, _w1#25, _w2#26] +Arguments: [rank(_w0#24) windowspecdefinition(_w1#25, _w2#26, _w0#24 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rank_within_parent#27], [_w1#25, _w2#26], [_w0#24 ASC NULLS FIRST] + +(27) Project [codegen id : 7] +Output [5]: [gross_margin#22, i_category#13, i_class#14, lochierarchy#23, rank_within_parent#27] +Input [8]: [gross_margin#22, i_category#13, i_class#14, lochierarchy#23, _w0#24, _w1#25, _w2#26, rank_within_parent#27] + +(28) TakeOrderedAndProject +Input [5]: [gross_margin#22, i_category#13, i_class#14, lochierarchy#23, rank_within_parent#27] +Arguments: 100, [lochierarchy#23 DESC NULLS LAST, CASE WHEN (lochierarchy#23 = 0) THEN i_category#13 END ASC NULLS FIRST, rank_within_parent#27 ASC NULLS FIRST], [gross_margin#22, i_category#13, i_class#14, lochierarchy#23, rank_within_parent#27] + +===== Subqueries ===== + +Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#5 IN dynamicpruning#6 +BroadcastExchange (33) ++- * ColumnarToRow (32) + +- CometProject (31) + +- CometFilter (30) + +- CometScan parquet spark_catalog.default.date_dim (29) + + +(29) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#7, d_year#28] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), IsNotNull(d_date_sk)] +ReadSchema: struct<d_date_sk:int,d_year:int> + +(30) CometFilter +Input [2]: [d_date_sk#7, d_year#28] +Condition : ((isnotnull(d_year#28) AND (d_year#28 = 2001)) AND isnotnull(d_date_sk#7)) + +(31) CometProject +Input [2]: [d_date_sk#7, d_year#28] +Arguments: [d_date_sk#7], [d_date_sk#7] + +(32) ColumnarToRow [codegen id : 1] +Input [1]: [d_date_sk#7] + +(33) BroadcastExchange +Input [1]: [d_date_sk#7] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] + + diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q36/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q36/simplified.txt new file mode 100644 index 000000000..7eeb607c3 --- /dev/null +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q36/simplified.txt @@ -0,0 +1,51 @@ +TakeOrderedAndProject [lochierarchy,i_category,rank_within_parent,gross_margin,i_class] + WholeStageCodegen (7) + Project [gross_margin,i_category,i_class,lochierarchy,rank_within_parent] + InputAdapter + Window [_w0,_w1,_w2] + WholeStageCodegen (6) + Sort [_w1,_w2,_w0] + InputAdapter + Exchange [_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] + InputAdapter + Exchange [i_category,i_class,spark_grouping_id] #2 + WholeStageCodegen (4) + HashAggregate [i_category,i_class,spark_grouping_id,ss_net_profit,ss_ext_sales_price] [sum,sum,sum,sum] + Expand [ss_ext_sales_price,ss_net_profit,i_category,i_class] + Project [ss_ext_sales_price,ss_net_profit,i_category,i_class] + BroadcastHashJoin [ss_store_sk,s_store_sk] + Project [ss_store_sk,ss_ext_sales_price,ss_net_profit,i_class,i_category] + BroadcastHashJoin [ss_item_sk,i_item_sk] + Project [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + ColumnarToRow + InputAdapter + CometFilter [ss_item_sk,ss_store_sk] + CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_year,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + InputAdapter + ReusedExchange [d_date_sk] #3 + InputAdapter + BroadcastExchange #4 + WholeStageCodegen (2) + ColumnarToRow + InputAdapter + CometFilter [i_item_sk] + CometScan parquet spark_catalog.default.item [i_item_sk,i_class,i_category] + InputAdapter + BroadcastExchange #5 + WholeStageCodegen (3) + ColumnarToRow + InputAdapter + CometProject [s_store_sk] + CometFilter [s_state,s_store_sk] + CometScan parquet spark_catalog.default.store [s_store_sk,s_state] 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 new file mode 100644 index 000000000..23c4ae742 --- /dev/null +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q37/explain.txt @@ -0,0 +1,179 @@ +== Physical Plan == +TakeOrderedAndProject (25) ++- * HashAggregate (24) + +- Exchange (23) + +- * HashAggregate (22) + +- * Project (21) + +- * BroadcastHashJoin Inner BuildLeft (20) + :- BroadcastExchange (15) + : +- * Project (14) + : +- * BroadcastHashJoin Inner BuildRight (13) + : :- * Project (11) + : : +- * BroadcastHashJoin Inner BuildRight (10) + : : :- * ColumnarToRow (4) + : : : +- CometProject (3) + : : : +- CometFilter (2) + : : : +- CometScan parquet spark_catalog.default.item (1) + : : +- BroadcastExchange (9) + : : +- * ColumnarToRow (8) + : : +- CometProject (7) + : : +- CometFilter (6) + : : +- CometScan parquet spark_catalog.default.inventory (5) + : +- ReusedExchange (12) + +- * ColumnarToRow (19) + +- CometProject (18) + +- CometFilter (17) + +- CometScan parquet spark_catalog.default.catalog_sales (16) + + +(1) Scan parquet spark_catalog.default.item +Output [5]: [i_item_sk#1, i_item_id#2, i_item_desc#3, i_current_price#4, i_manufact_id#5] +Batched: true +Location [not included in comparison]/{warehouse_dir}/item] +PushedFilters: [IsNotNull(i_current_price), GreaterThanOrEqual(i_current_price,68.00), LessThanOrEqual(i_current_price,98.00), In(i_manufact_id, [677,694,808,940]), IsNotNull(i_item_sk)] +ReadSchema: struct<i_item_sk:int,i_item_id:string,i_item_desc:string,i_current_price:decimal(7,2),i_manufact_id:int> + +(2) CometFilter +Input [5]: [i_item_sk#1, i_item_id#2, i_item_desc#3, i_current_price#4, i_manufact_id#5] +Condition : ((((isnotnull(i_current_price#4) AND (i_current_price#4 >= 68.00)) AND (i_current_price#4 <= 98.00)) AND i_manufact_id#5 IN (677,940,694,808)) AND isnotnull(i_item_sk#1)) + +(3) CometProject +Input [5]: [i_item_sk#1, i_item_id#2, i_item_desc#3, i_current_price#4, i_manufact_id#5] +Arguments: [i_item_sk#1, i_item_id#2, i_item_desc#3, i_current_price#4], [i_item_sk#1, i_item_id#2, i_item_desc#3, i_current_price#4] + +(4) ColumnarToRow [codegen id : 3] +Input [4]: [i_item_sk#1, i_item_id#2, i_item_desc#3, i_current_price#4] + +(5) Scan parquet spark_catalog.default.inventory +Output [3]: [inv_item_sk#6, inv_quantity_on_hand#7, inv_date_sk#8] +Batched: true +Location: InMemoryFileIndex [] +PartitionFilters: [isnotnull(inv_date_sk#8), dynamicpruningexpression(inv_date_sk#8 IN dynamicpruning#9)] +PushedFilters: [IsNotNull(inv_quantity_on_hand), GreaterThanOrEqual(inv_quantity_on_hand,100), LessThanOrEqual(inv_quantity_on_hand,500), IsNotNull(inv_item_sk)] +ReadSchema: struct<inv_item_sk:int,inv_quantity_on_hand:int> + +(6) CometFilter +Input [3]: [inv_item_sk#6, inv_quantity_on_hand#7, inv_date_sk#8] +Condition : (((isnotnull(inv_quantity_on_hand#7) AND (inv_quantity_on_hand#7 >= 100)) AND (inv_quantity_on_hand#7 <= 500)) AND isnotnull(inv_item_sk#6)) + +(7) CometProject +Input [3]: [inv_item_sk#6, inv_quantity_on_hand#7, inv_date_sk#8] +Arguments: [inv_item_sk#6, inv_date_sk#8], [inv_item_sk#6, inv_date_sk#8] + +(8) ColumnarToRow [codegen id : 1] +Input [2]: [inv_item_sk#6, inv_date_sk#8] + +(9) BroadcastExchange +Input [2]: [inv_item_sk#6, inv_date_sk#8] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] + +(10) BroadcastHashJoin [codegen id : 3] +Left keys [1]: [i_item_sk#1] +Right keys [1]: [inv_item_sk#6] +Join type: Inner +Join condition: None + +(11) Project [codegen id : 3] +Output [5]: [i_item_sk#1, i_item_id#2, i_item_desc#3, i_current_price#4, inv_date_sk#8] +Input [6]: [i_item_sk#1, i_item_id#2, i_item_desc#3, i_current_price#4, inv_item_sk#6, inv_date_sk#8] + +(12) ReusedExchange [Reuses operator id: 30] +Output [1]: [d_date_sk#10] + +(13) BroadcastHashJoin [codegen id : 3] +Left keys [1]: [inv_date_sk#8] +Right keys [1]: [d_date_sk#10] +Join type: Inner +Join condition: None + +(14) Project [codegen id : 3] +Output [4]: [i_item_sk#1, i_item_id#2, i_item_desc#3, i_current_price#4] +Input [6]: [i_item_sk#1, i_item_id#2, i_item_desc#3, i_current_price#4, inv_date_sk#8, d_date_sk#10] + +(15) BroadcastExchange +Input [4]: [i_item_sk#1, i_item_id#2, i_item_desc#3, i_current_price#4] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] + +(16) Scan parquet spark_catalog.default.catalog_sales +Output [2]: [cs_item_sk#11, cs_sold_date_sk#12] +Batched: true +Location [not included in comparison]/{warehouse_dir}/catalog_sales] +PushedFilters: [IsNotNull(cs_item_sk)] +ReadSchema: struct<cs_item_sk:int> + +(17) CometFilter +Input [2]: [cs_item_sk#11, cs_sold_date_sk#12] +Condition : isnotnull(cs_item_sk#11) + +(18) CometProject +Input [2]: [cs_item_sk#11, cs_sold_date_sk#12] +Arguments: [cs_item_sk#11], [cs_item_sk#11] + +(19) ColumnarToRow +Input [1]: [cs_item_sk#11] + +(20) BroadcastHashJoin [codegen id : 4] +Left keys [1]: [i_item_sk#1] +Right keys [1]: [cs_item_sk#11] +Join type: Inner +Join condition: None + +(21) Project [codegen id : 4] +Output [3]: [i_item_id#2, i_item_desc#3, i_current_price#4] +Input [5]: [i_item_sk#1, i_item_id#2, i_item_desc#3, i_current_price#4, cs_item_sk#11] + +(22) HashAggregate [codegen id : 4] +Input [3]: [i_item_id#2, i_item_desc#3, i_current_price#4] +Keys [3]: [i_item_id#2, i_item_desc#3, i_current_price#4] +Functions: [] +Aggregate Attributes: [] +Results [3]: [i_item_id#2, i_item_desc#3, i_current_price#4] + +(23) Exchange +Input [3]: [i_item_id#2, i_item_desc#3, i_current_price#4] +Arguments: hashpartitioning(i_item_id#2, i_item_desc#3, i_current_price#4, 5), ENSURE_REQUIREMENTS, [plan_id=3] + +(24) HashAggregate [codegen id : 5] +Input [3]: [i_item_id#2, i_item_desc#3, i_current_price#4] +Keys [3]: [i_item_id#2, i_item_desc#3, i_current_price#4] +Functions: [] +Aggregate Attributes: [] +Results [3]: [i_item_id#2, i_item_desc#3, i_current_price#4] + +(25) TakeOrderedAndProject +Input [3]: [i_item_id#2, i_item_desc#3, i_current_price#4] +Arguments: 100, [i_item_id#2 ASC NULLS FIRST], [i_item_id#2, i_item_desc#3, i_current_price#4] + +===== Subqueries ===== + +Subquery:1 Hosting operator id = 5 Hosting Expression = inv_date_sk#8 IN dynamicpruning#9 +BroadcastExchange (30) ++- * ColumnarToRow (29) + +- CometProject (28) + +- CometFilter (27) + +- CometScan parquet spark_catalog.default.date_dim (26) + + +(26) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#10, d_date#13] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,2000-02-01), LessThanOrEqual(d_date,2000-04-01), IsNotNull(d_date_sk)] +ReadSchema: struct<d_date_sk:int,d_date:date> + +(27) CometFilter +Input [2]: [d_date_sk#10, d_date#13] +Condition : (((isnotnull(d_date#13) AND (d_date#13 >= 2000-02-01)) AND (d_date#13 <= 2000-04-01)) AND isnotnull(d_date_sk#10)) + +(28) CometProject +Input [2]: [d_date_sk#10, d_date#13] +Arguments: [d_date_sk#10], [d_date_sk#10] + +(29) ColumnarToRow [codegen id : 1] +Input [1]: [d_date_sk#10] + +(30) BroadcastExchange +Input [1]: [d_date_sk#10] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] + + diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q37/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q37/simplified.txt new file mode 100644 index 000000000..65bb06348 --- /dev/null +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q37/simplified.txt @@ -0,0 +1,44 @@ +TakeOrderedAndProject [i_item_id,i_item_desc,i_current_price] + WholeStageCodegen (5) + HashAggregate [i_item_id,i_item_desc,i_current_price] + InputAdapter + Exchange [i_item_id,i_item_desc,i_current_price] #1 + WholeStageCodegen (4) + HashAggregate [i_item_id,i_item_desc,i_current_price] + Project [i_item_id,i_item_desc,i_current_price] + BroadcastHashJoin [i_item_sk,cs_item_sk] + InputAdapter + BroadcastExchange #2 + WholeStageCodegen (3) + Project [i_item_sk,i_item_id,i_item_desc,i_current_price] + BroadcastHashJoin [inv_date_sk,d_date_sk] + Project [i_item_sk,i_item_id,i_item_desc,i_current_price,inv_date_sk] + BroadcastHashJoin [i_item_sk,inv_item_sk] + ColumnarToRow + InputAdapter + CometProject [i_item_sk,i_item_id,i_item_desc,i_current_price] + CometFilter [i_current_price,i_manufact_id,i_item_sk] + CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_manufact_id] + InputAdapter + BroadcastExchange #3 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [inv_item_sk,inv_date_sk] + CometFilter [inv_quantity_on_hand,inv_item_sk] + CometScan parquet spark_catalog.default.inventory [inv_item_sk,inv_quantity_on_hand,inv_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #4 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] + InputAdapter + ReusedExchange [d_date_sk] #4 + ColumnarToRow + InputAdapter + CometProject [cs_item_sk] + CometFilter [cs_item_sk] + CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_sold_date_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q38/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q38/explain.txt new file mode 100644 index 000000000..5503439ee --- /dev/null +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q38/explain.txt @@ -0,0 +1,321 @@ +== Physical Plan == +* HashAggregate (47) ++- Exchange (46) + +- * HashAggregate (45) + +- * Project (44) + +- * BroadcastHashJoin LeftSemi BuildRight (43) + :- * BroadcastHashJoin LeftSemi BuildRight (29) + : :- * HashAggregate (15) + : : +- Exchange (14) + : : +- * HashAggregate (13) + : : +- * Project (12) + : : +- * BroadcastHashJoin Inner BuildRight (11) + : : :- * Project (6) + : : : +- * BroadcastHashJoin Inner BuildRight (5) + : : : :- * ColumnarToRow (3) + : : : : +- CometFilter (2) + : : : : +- CometScan parquet spark_catalog.default.store_sales (1) + : : : +- ReusedExchange (4) + : : +- BroadcastExchange (10) + : : +- * ColumnarToRow (9) + : : +- CometFilter (8) + : : +- CometScan parquet spark_catalog.default.customer (7) + : +- BroadcastExchange (28) + : +- * HashAggregate (27) + : +- Exchange (26) + : +- * HashAggregate (25) + : +- * Project (24) + : +- * BroadcastHashJoin Inner BuildRight (23) + : :- * Project (21) + : : +- * BroadcastHashJoin Inner BuildRight (20) + : : :- * ColumnarToRow (18) + : : : +- CometFilter (17) + : : : +- CometScan parquet spark_catalog.default.catalog_sales (16) + : : +- ReusedExchange (19) + : +- ReusedExchange (22) + +- BroadcastExchange (42) + +- * HashAggregate (41) + +- Exchange (40) + +- * HashAggregate (39) + +- * Project (38) + +- * BroadcastHashJoin Inner BuildRight (37) + :- * Project (35) + : +- * BroadcastHashJoin Inner BuildRight (34) + : :- * ColumnarToRow (32) + : : +- CometFilter (31) + : : +- CometScan parquet spark_catalog.default.web_sales (30) + : +- ReusedExchange (33) + +- ReusedExchange (36) + + +(1) Scan parquet spark_catalog.default.store_sales +Output [2]: [ss_customer_sk#1, ss_sold_date_sk#2] +Batched: true +Location: InMemoryFileIndex [] +PartitionFilters: [isnotnull(ss_sold_date_sk#2), dynamicpruningexpression(ss_sold_date_sk#2 IN dynamicpruning#3)] +PushedFilters: [IsNotNull(ss_customer_sk)] +ReadSchema: struct<ss_customer_sk:int> + +(2) CometFilter +Input [2]: [ss_customer_sk#1, ss_sold_date_sk#2] +Condition : isnotnull(ss_customer_sk#1) + +(3) ColumnarToRow [codegen id : 3] +Input [2]: [ss_customer_sk#1, ss_sold_date_sk#2] + +(4) ReusedExchange [Reuses operator id: 52] +Output [2]: [d_date_sk#4, d_date#5] + +(5) BroadcastHashJoin [codegen id : 3] +Left keys [1]: [ss_sold_date_sk#2] +Right keys [1]: [d_date_sk#4] +Join type: Inner +Join condition: None + +(6) Project [codegen id : 3] +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] + +(7) Scan parquet spark_catalog.default.customer +Output [3]: [c_customer_sk#6, c_first_name#7, c_last_name#8] +Batched: true +Location [not included in comparison]/{warehouse_dir}/customer] +PushedFilters: [IsNotNull(c_customer_sk)] +ReadSchema: struct<c_customer_sk:int,c_first_name:string,c_last_name:string> + +(8) CometFilter +Input [3]: [c_customer_sk#6, c_first_name#7, c_last_name#8] +Condition : isnotnull(c_customer_sk#6) + +(9) ColumnarToRow [codegen id : 2] +Input [3]: [c_customer_sk#6, c_first_name#7, c_last_name#8] + +(10) BroadcastExchange +Input [3]: [c_customer_sk#6, c_first_name#7, c_last_name#8] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=1] + +(11) BroadcastHashJoin [codegen id : 3] +Left keys [1]: [ss_customer_sk#1] +Right keys [1]: [c_customer_sk#6] +Join type: Inner +Join condition: None + +(12) Project [codegen id : 3] +Output [3]: [c_last_name#8, c_first_name#7, d_date#5] +Input [5]: [ss_customer_sk#1, d_date#5, c_customer_sk#6, c_first_name#7, c_last_name#8] + +(13) HashAggregate [codegen id : 3] +Input [3]: [c_last_name#8, c_first_name#7, d_date#5] +Keys [3]: [c_last_name#8, c_first_name#7, d_date#5] +Functions: [] +Aggregate Attributes: [] +Results [3]: [c_last_name#8, c_first_name#7, d_date#5] + +(14) Exchange +Input [3]: [c_last_name#8, c_first_name#7, d_date#5] +Arguments: hashpartitioning(c_last_name#8, c_first_name#7, d_date#5, 5), ENSURE_REQUIREMENTS, [plan_id=2] + +(15) HashAggregate [codegen id : 12] +Input [3]: [c_last_name#8, c_first_name#7, d_date#5] +Keys [3]: [c_last_name#8, c_first_name#7, d_date#5] +Functions: [] +Aggregate Attributes: [] +Results [3]: [c_last_name#8, c_first_name#7, d_date#5] + +(16) Scan parquet spark_catalog.default.catalog_sales +Output [2]: [cs_bill_customer_sk#9, cs_sold_date_sk#10] +Batched: true +Location: InMemoryFileIndex [] +PartitionFilters: [isnotnull(cs_sold_date_sk#10), dynamicpruningexpression(cs_sold_date_sk#10 IN dynamicpruning#11)] +PushedFilters: [IsNotNull(cs_bill_customer_sk)] +ReadSchema: struct<cs_bill_customer_sk:int> + +(17) CometFilter +Input [2]: [cs_bill_customer_sk#9, cs_sold_date_sk#10] +Condition : isnotnull(cs_bill_customer_sk#9) + +(18) ColumnarToRow [codegen id : 6] +Input [2]: [cs_bill_customer_sk#9, cs_sold_date_sk#10] + +(19) ReusedExchange [Reuses operator id: 52] +Output [2]: [d_date_sk#12, d_date#13] + +(20) BroadcastHashJoin [codegen id : 6] +Left keys [1]: [cs_sold_date_sk#10] +Right keys [1]: [d_date_sk#12] +Join type: Inner +Join condition: None + +(21) Project [codegen id : 6] +Output [2]: [cs_bill_customer_sk#9, d_date#13] +Input [4]: [cs_bill_customer_sk#9, cs_sold_date_sk#10, d_date_sk#12, d_date#13] + +(22) ReusedExchange [Reuses operator id: 10] +Output [3]: [c_customer_sk#14, c_first_name#15, c_last_name#16] + +(23) BroadcastHashJoin [codegen id : 6] +Left keys [1]: [cs_bill_customer_sk#9] +Right keys [1]: [c_customer_sk#14] +Join type: Inner +Join condition: None + +(24) Project [codegen id : 6] +Output [3]: [c_last_name#16, c_first_name#15, d_date#13] +Input [5]: [cs_bill_customer_sk#9, d_date#13, c_customer_sk#14, c_first_name#15, c_last_name#16] + +(25) HashAggregate [codegen id : 6] +Input [3]: [c_last_name#16, c_first_name#15, d_date#13] +Keys [3]: [c_last_name#16, c_first_name#15, d_date#13] +Functions: [] +Aggregate Attributes: [] +Results [3]: [c_last_name#16, c_first_name#15, d_date#13] + +(26) Exchange +Input [3]: [c_last_name#16, c_first_name#15, d_date#13] +Arguments: hashpartitioning(c_last_name#16, c_first_name#15, d_date#13, 5), ENSURE_REQUIREMENTS, [plan_id=3] + +(27) HashAggregate [codegen id : 7] +Input [3]: [c_last_name#16, c_first_name#15, d_date#13] +Keys [3]: [c_last_name#16, c_first_name#15, d_date#13] +Functions: [] +Aggregate Attributes: [] +Results [3]: [c_last_name#16, c_first_name#15, d_date#13] + +(28) BroadcastExchange +Input [3]: [c_last_name#16, c_first_name#15, d_date#13] +Arguments: HashedRelationBroadcastMode(List(coalesce(input[0, string, true], ), isnull(input[0, string, true]), coalesce(input[1, string, true], ), isnull(input[1, string, true]), coalesce(input[2, date, true], 1970-01-01), isnull(input[2, date, true])),false), [plan_id=4] + +(29) BroadcastHashJoin [codegen id : 12] +Left keys [6]: [coalesce(c_last_name#8, ), isnull(c_last_name#8), coalesce(c_first_name#7, ), isnull(c_first_name#7), coalesce(d_date#5, 1970-01-01), isnull(d_date#5)] +Right keys [6]: [coalesce(c_last_name#16, ), isnull(c_last_name#16), coalesce(c_first_name#15, ), isnull(c_first_name#15), coalesce(d_date#13, 1970-01-01), isnull(d_date#13)] +Join type: LeftSemi +Join condition: None + +(30) Scan parquet spark_catalog.default.web_sales +Output [2]: [ws_bill_customer_sk#17, ws_sold_date_sk#18] +Batched: true +Location: InMemoryFileIndex [] +PartitionFilters: [isnotnull(ws_sold_date_sk#18), dynamicpruningexpression(ws_sold_date_sk#18 IN dynamicpruning#19)] +PushedFilters: [IsNotNull(ws_bill_customer_sk)] +ReadSchema: struct<ws_bill_customer_sk:int> + +(31) CometFilter +Input [2]: [ws_bill_customer_sk#17, ws_sold_date_sk#18] +Condition : isnotnull(ws_bill_customer_sk#17) + +(32) ColumnarToRow [codegen id : 10] +Input [2]: [ws_bill_customer_sk#17, ws_sold_date_sk#18] + +(33) ReusedExchange [Reuses operator id: 52] +Output [2]: [d_date_sk#20, d_date#21] + +(34) BroadcastHashJoin [codegen id : 10] +Left keys [1]: [ws_sold_date_sk#18] +Right keys [1]: [d_date_sk#20] +Join type: Inner +Join condition: None + +(35) Project [codegen id : 10] +Output [2]: [ws_bill_customer_sk#17, d_date#21] +Input [4]: [ws_bill_customer_sk#17, ws_sold_date_sk#18, d_date_sk#20, d_date#21] + +(36) ReusedExchange [Reuses operator id: 10] +Output [3]: [c_customer_sk#22, c_first_name#23, c_last_name#24] + +(37) BroadcastHashJoin [codegen id : 10] +Left keys [1]: [ws_bill_customer_sk#17] +Right keys [1]: [c_customer_sk#22] +Join type: Inner +Join condition: None + +(38) Project [codegen id : 10] +Output [3]: [c_last_name#24, c_first_name#23, d_date#21] +Input [5]: [ws_bill_customer_sk#17, d_date#21, c_customer_sk#22, c_first_name#23, c_last_name#24] + +(39) HashAggregate [codegen id : 10] +Input [3]: [c_last_name#24, c_first_name#23, d_date#21] +Keys [3]: [c_last_name#24, c_first_name#23, d_date#21] +Functions: [] +Aggregate Attributes: [] +Results [3]: [c_last_name#24, c_first_name#23, d_date#21] + +(40) Exchange +Input [3]: [c_last_name#24, c_first_name#23, d_date#21] +Arguments: hashpartitioning(c_last_name#24, c_first_name#23, d_date#21, 5), ENSURE_REQUIREMENTS, [plan_id=5] + +(41) HashAggregate [codegen id : 11] +Input [3]: [c_last_name#24, c_first_name#23, d_date#21] +Keys [3]: [c_last_name#24, c_first_name#23, d_date#21] +Functions: [] +Aggregate Attributes: [] +Results [3]: [c_last_name#24, c_first_name#23, d_date#21] + +(42) BroadcastExchange +Input [3]: [c_last_name#24, c_first_name#23, d_date#21] +Arguments: HashedRelationBroadcastMode(List(coalesce(input[0, string, true], ), isnull(input[0, string, true]), coalesce(input[1, string, true], ), isnull(input[1, string, true]), coalesce(input[2, date, true], 1970-01-01), isnull(input[2, date, true])),false), [plan_id=6] + +(43) BroadcastHashJoin [codegen id : 12] +Left keys [6]: [coalesce(c_last_name#8, ), isnull(c_last_name#8), coalesce(c_first_name#7, ), isnull(c_first_name#7), coalesce(d_date#5, 1970-01-01), isnull(d_date#5)] +Right keys [6]: [coalesce(c_last_name#24, ), isnull(c_last_name#24), coalesce(c_first_name#23, ), isnull(c_first_name#23), coalesce(d_date#21, 1970-01-01), isnull(d_date#21)] +Join type: LeftSemi +Join condition: None + +(44) Project [codegen id : 12] +Output: [] +Input [3]: [c_last_name#8, c_first_name#7, d_date#5] + +(45) HashAggregate [codegen id : 12] +Input: [] +Keys: [] +Functions [1]: [partial_count(1)] +Aggregate Attributes [1]: [count#25] +Results [1]: [count#26] + +(46) Exchange +Input [1]: [count#26] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=7] + +(47) HashAggregate [codegen id : 13] +Input [1]: [count#26] +Keys: [] +Functions [1]: [count(1)] +Aggregate Attributes [1]: [count(1)#27] +Results [1]: [count(1)#27 AS count(1)#28] + +===== Subqueries ===== + +Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#2 IN dynamicpruning#3 +BroadcastExchange (52) ++- * ColumnarToRow (51) + +- CometProject (50) + +- CometFilter (49) + +- CometScan parquet spark_catalog.default.date_dim (48) + + +(48) Scan parquet spark_catalog.default.date_dim +Output [3]: [d_date_sk#4, d_date#5, d_month_seq#29] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1200), LessThanOrEqual(d_month_seq,1211), IsNotNull(d_date_sk)] +ReadSchema: struct<d_date_sk:int,d_date:date,d_month_seq:int> + +(49) CometFilter +Input [3]: [d_date_sk#4, d_date#5, d_month_seq#29] +Condition : (((isnotnull(d_month_seq#29) AND (d_month_seq#29 >= 1200)) AND (d_month_seq#29 <= 1211)) AND isnotnull(d_date_sk#4)) + +(50) CometProject +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] + +(51) ColumnarToRow [codegen id : 1] +Input [2]: [d_date_sk#4, d_date#5] + +(52) BroadcastExchange +Input [2]: [d_date_sk#4, d_date#5] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=8] + +Subquery:2 Hosting operator id = 16 Hosting Expression = cs_sold_date_sk#10 IN dynamicpruning#3 + +Subquery:3 Hosting operator id = 30 Hosting Expression = ws_sold_date_sk#18 IN dynamicpruning#3 + + diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q38/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q38/simplified.txt new file mode 100644 index 000000000..315afe660 --- /dev/null +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q38/simplified.txt @@ -0,0 +1,81 @@ +WholeStageCodegen (13) + HashAggregate [count] [count(1),count(1),count] + InputAdapter + Exchange #1 + WholeStageCodegen (12) + HashAggregate [count,count] + Project + BroadcastHashJoin [c_last_name,c_first_name,d_date,c_last_name,c_first_name,d_date] + BroadcastHashJoin [c_last_name,c_first_name,d_date,c_last_name,c_first_name,d_date] + HashAggregate [c_last_name,c_first_name,d_date] + InputAdapter + Exchange [c_last_name,c_first_name,d_date] #2 + WholeStageCodegen (3) + HashAggregate [c_last_name,c_first_name,d_date] + Project [c_last_name,c_first_name,d_date] + BroadcastHashJoin [ss_customer_sk,c_customer_sk] + Project [ss_customer_sk,d_date] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + ColumnarToRow + InputAdapter + CometFilter [ss_customer_sk] + CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk,d_date] + CometFilter [d_month_seq,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_month_seq] + InputAdapter + ReusedExchange [d_date_sk,d_date] #3 + InputAdapter + BroadcastExchange #4 + WholeStageCodegen (2) + ColumnarToRow + InputAdapter + CometFilter [c_customer_sk] + CometScan parquet spark_catalog.default.customer [c_customer_sk,c_first_name,c_last_name] + InputAdapter + BroadcastExchange #5 + WholeStageCodegen (7) + HashAggregate [c_last_name,c_first_name,d_date] + InputAdapter + Exchange [c_last_name,c_first_name,d_date] #6 + WholeStageCodegen (6) + HashAggregate [c_last_name,c_first_name,d_date] + Project [c_last_name,c_first_name,d_date] + BroadcastHashJoin [cs_bill_customer_sk,c_customer_sk] + Project [cs_bill_customer_sk,d_date] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + ColumnarToRow + InputAdapter + CometFilter [cs_bill_customer_sk] + CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + InputAdapter + ReusedExchange [d_date_sk,d_date] #3 + InputAdapter + ReusedExchange [c_customer_sk,c_first_name,c_last_name] #4 + InputAdapter + BroadcastExchange #7 + WholeStageCodegen (11) + HashAggregate [c_last_name,c_first_name,d_date] + InputAdapter + Exchange [c_last_name,c_first_name,d_date] #8 + WholeStageCodegen (10) + HashAggregate [c_last_name,c_first_name,d_date] + Project [c_last_name,c_first_name,d_date] + BroadcastHashJoin [ws_bill_customer_sk,c_customer_sk] + Project [ws_bill_customer_sk,d_date] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + ColumnarToRow + InputAdapter + CometFilter [ws_bill_customer_sk] + CometScan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + InputAdapter + ReusedExchange [d_date_sk,d_date] #3 + InputAdapter + ReusedExchange [c_customer_sk,c_first_name,c_last_name] #4 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 new file mode 100644 index 000000000..d851f61bf --- /dev/null +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q39a/explain.txt @@ -0,0 +1,318 @@ +== Physical Plan == +* Sort (44) ++- Exchange (43) + +- * BroadcastHashJoin Inner BuildRight (42) + :- * Project (23) + : +- * Filter (22) + : +- * HashAggregate (21) + : +- Exchange (20) + : +- * HashAggregate (19) + : +- * Project (18) + : +- * BroadcastHashJoin Inner BuildRight (17) + : :- * Project (15) + : : +- * BroadcastHashJoin Inner BuildRight (14) + : : :- * Project (9) + : : : +- * BroadcastHashJoin Inner BuildRight (8) + : : : :- * ColumnarToRow (3) + : : : : +- CometFilter (2) + : : : : +- CometScan parquet spark_catalog.default.inventory (1) + : : : +- BroadcastExchange (7) + : : : +- * ColumnarToRow (6) + : : : +- CometFilter (5) + : : : +- CometScan parquet spark_catalog.default.item (4) + : : +- BroadcastExchange (13) + : : +- * ColumnarToRow (12) + : : +- CometFilter (11) + : : +- CometScan parquet spark_catalog.default.warehouse (10) + : +- ReusedExchange (16) + +- BroadcastExchange (41) + +- * Project (40) + +- * Filter (39) + +- * HashAggregate (38) + +- Exchange (37) + +- * HashAggregate (36) + +- * Project (35) + +- * BroadcastHashJoin Inner BuildRight (34) + :- * Project (32) + : +- * BroadcastHashJoin Inner BuildRight (31) + : :- * Project (29) + : : +- * BroadcastHashJoin Inner BuildRight (28) + : : :- * ColumnarToRow (26) + : : : +- CometFilter (25) + : : : +- CometScan parquet spark_catalog.default.inventory (24) + : : +- ReusedExchange (27) + : +- ReusedExchange (30) + +- ReusedExchange (33) + + +(1) Scan parquet spark_catalog.default.inventory +Output [4]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4] +Batched: true +Location: InMemoryFileIndex [] +PartitionFilters: [isnotnull(inv_date_sk#4), dynamicpruningexpression(inv_date_sk#4 IN dynamicpruning#5)] +PushedFilters: [IsNotNull(inv_item_sk), IsNotNull(inv_warehouse_sk)] +ReadSchema: struct<inv_item_sk:int,inv_warehouse_sk:int,inv_quantity_on_hand:int> + +(2) CometFilter +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)) + +(3) ColumnarToRow [codegen id : 4] +Input [4]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4] + +(4) Scan parquet spark_catalog.default.item +Output [1]: [i_item_sk#6] +Batched: true +Location [not included in comparison]/{warehouse_dir}/item] +PushedFilters: [IsNotNull(i_item_sk)] +ReadSchema: struct<i_item_sk:int> + +(5) CometFilter +Input [1]: [i_item_sk#6] +Condition : isnotnull(i_item_sk#6) + +(6) ColumnarToRow [codegen id : 1] +Input [1]: [i_item_sk#6] + +(7) BroadcastExchange +Input [1]: [i_item_sk#6] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=1] + +(8) BroadcastHashJoin [codegen id : 4] +Left keys [1]: [inv_item_sk#1] +Right keys [1]: [i_item_sk#6] +Join type: Inner +Join condition: None + +(9) Project [codegen id : 4] +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] + +(10) Scan parquet spark_catalog.default.warehouse +Output [2]: [w_warehouse_sk#7, w_warehouse_name#8] +Batched: true +Location [not included in comparison]/{warehouse_dir}/warehouse] +PushedFilters: [IsNotNull(w_warehouse_sk)] +ReadSchema: struct<w_warehouse_sk:int,w_warehouse_name:string> + +(11) CometFilter +Input [2]: [w_warehouse_sk#7, w_warehouse_name#8] +Condition : isnotnull(w_warehouse_sk#7) + +(12) ColumnarToRow [codegen id : 2] +Input [2]: [w_warehouse_sk#7, w_warehouse_name#8] + +(13) BroadcastExchange +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] +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] +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: 49] +Output [2]: [d_date_sk#9, d_moy#10] + +(17) BroadcastHashJoin [codegen id : 4] +Left keys [1]: [inv_date_sk#4] +Right keys [1]: [d_date_sk#9] +Join type: Inner +Join condition: None + +(18) Project [codegen id : 4] +Output [5]: [inv_quantity_on_hand#3, i_item_sk#6, w_warehouse_sk#7, w_warehouse_name#8, d_moy#10] +Input [7]: [inv_quantity_on_hand#3, inv_date_sk#4, i_item_sk#6, w_warehouse_sk#7, w_warehouse_name#8, d_date_sk#9, d_moy#10] + +(19) HashAggregate [codegen id : 4] +Input [5]: [inv_quantity_on_hand#3, i_item_sk#6, w_warehouse_sk#7, w_warehouse_name#8, d_moy#10] +Keys [4]: [w_warehouse_name#8, w_warehouse_sk#7, i_item_sk#6, d_moy#10] +Functions [2]: [partial_stddev_samp(cast(inv_quantity_on_hand#3 as double)), partial_avg(inv_quantity_on_hand#3)] +Aggregate Attributes [5]: [n#11, avg#12, m2#13, sum#14, count#15] +Results [9]: [w_warehouse_name#8, w_warehouse_sk#7, i_item_sk#6, d_moy#10, n#16, avg#17, m2#18, sum#19, count#20] + +(20) Exchange +Input [9]: [w_warehouse_name#8, w_warehouse_sk#7, i_item_sk#6, d_moy#10, n#16, avg#17, m2#18, sum#19, count#20] +Arguments: hashpartitioning(w_warehouse_name#8, w_warehouse_sk#7, i_item_sk#6, d_moy#10, 5), ENSURE_REQUIREMENTS, [plan_id=3] + +(21) HashAggregate [codegen id : 10] +Input [9]: [w_warehouse_name#8, w_warehouse_sk#7, i_item_sk#6, d_moy#10, n#16, avg#17, m2#18, sum#19, count#20] +Keys [4]: [w_warehouse_name#8, w_warehouse_sk#7, i_item_sk#6, d_moy#10] +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] + +(22) Filter [codegen id : 10] +Input [5]: [w_warehouse_sk#7, i_item_sk#6, d_moy#10, stdev#23, mean#24] +Condition : CASE WHEN (mean#24 = 0.0) THEN false ELSE ((stdev#23 / mean#24) > 1.0) END + +(23) Project [codegen id : 10] +Output [5]: [w_warehouse_sk#7, i_item_sk#6, d_moy#10, mean#24, CASE WHEN (mean#24 = 0.0) THEN null ELSE (stdev#23 / mean#24) END AS cov#25] +Input [5]: [w_warehouse_sk#7, i_item_sk#6, d_moy#10, stdev#23, mean#24] + +(24) Scan parquet spark_catalog.default.inventory +Output [4]: [inv_item_sk#26, inv_warehouse_sk#27, inv_quantity_on_hand#28, inv_date_sk#29] +Batched: true +Location: InMemoryFileIndex [] +PartitionFilters: [isnotnull(inv_date_sk#29), dynamicpruningexpression(inv_date_sk#29 IN dynamicpruning#30)] +PushedFilters: [IsNotNull(inv_item_sk), IsNotNull(inv_warehouse_sk)] +ReadSchema: struct<inv_item_sk:int,inv_warehouse_sk:int,inv_quantity_on_hand:int> + +(25) CometFilter +Input [4]: [inv_item_sk#26, inv_warehouse_sk#27, inv_quantity_on_hand#28, inv_date_sk#29] +Condition : (isnotnull(inv_item_sk#26) AND isnotnull(inv_warehouse_sk#27)) + +(26) ColumnarToRow [codegen id : 8] +Input [4]: [inv_item_sk#26, inv_warehouse_sk#27, inv_quantity_on_hand#28, inv_date_sk#29] + +(27) ReusedExchange [Reuses operator id: 7] +Output [1]: [i_item_sk#31] + +(28) BroadcastHashJoin [codegen id : 8] +Left keys [1]: [inv_item_sk#26] +Right keys [1]: [i_item_sk#31] +Join type: Inner +Join condition: None + +(29) Project [codegen id : 8] +Output [4]: [inv_warehouse_sk#27, inv_quantity_on_hand#28, inv_date_sk#29, i_item_sk#31] +Input [5]: [inv_item_sk#26, inv_warehouse_sk#27, inv_quantity_on_hand#28, inv_date_sk#29, i_item_sk#31] + +(30) ReusedExchange [Reuses operator id: 13] +Output [2]: [w_warehouse_sk#32, w_warehouse_name#33] + +(31) BroadcastHashJoin [codegen id : 8] +Left keys [1]: [inv_warehouse_sk#27] +Right keys [1]: [w_warehouse_sk#32] +Join type: Inner +Join condition: None + +(32) Project [codegen id : 8] +Output [5]: [inv_quantity_on_hand#28, inv_date_sk#29, i_item_sk#31, w_warehouse_sk#32, w_warehouse_name#33] +Input [6]: [inv_warehouse_sk#27, inv_quantity_on_hand#28, inv_date_sk#29, i_item_sk#31, w_warehouse_sk#32, w_warehouse_name#33] + +(33) ReusedExchange [Reuses operator id: 54] +Output [2]: [d_date_sk#34, d_moy#35] + +(34) BroadcastHashJoin [codegen id : 8] +Left keys [1]: [inv_date_sk#29] +Right keys [1]: [d_date_sk#34] +Join type: Inner +Join condition: None + +(35) Project [codegen id : 8] +Output [5]: [inv_quantity_on_hand#28, i_item_sk#31, w_warehouse_sk#32, w_warehouse_name#33, d_moy#35] +Input [7]: [inv_quantity_on_hand#28, inv_date_sk#29, i_item_sk#31, w_warehouse_sk#32, w_warehouse_name#33, d_date_sk#34, d_moy#35] + +(36) HashAggregate [codegen id : 8] +Input [5]: [inv_quantity_on_hand#28, i_item_sk#31, w_warehouse_sk#32, w_warehouse_name#33, d_moy#35] +Keys [4]: [w_warehouse_name#33, w_warehouse_sk#32, i_item_sk#31, d_moy#35] +Functions [2]: [partial_stddev_samp(cast(inv_quantity_on_hand#28 as double)), partial_avg(inv_quantity_on_hand#28)] +Aggregate Attributes [5]: [n#36, avg#37, m2#38, sum#39, count#40] +Results [9]: [w_warehouse_name#33, w_warehouse_sk#32, i_item_sk#31, d_moy#35, n#41, avg#42, m2#43, sum#44, count#45] + +(37) Exchange +Input [9]: [w_warehouse_name#33, w_warehouse_sk#32, i_item_sk#31, d_moy#35, n#41, avg#42, m2#43, sum#44, count#45] +Arguments: hashpartitioning(w_warehouse_name#33, w_warehouse_sk#32, i_item_sk#31, d_moy#35, 5), ENSURE_REQUIREMENTS, [plan_id=4] + +(38) HashAggregate [codegen id : 9] +Input [9]: [w_warehouse_name#33, w_warehouse_sk#32, i_item_sk#31, d_moy#35, n#41, avg#42, m2#43, sum#44, count#45] +Keys [4]: [w_warehouse_name#33, w_warehouse_sk#32, i_item_sk#31, d_moy#35] +Functions [2]: [stddev_samp(cast(inv_quantity_on_hand#28 as double)), avg(inv_quantity_on_hand#28)] +Aggregate Attributes [2]: [stddev_samp(cast(inv_quantity_on_hand#28 as double))#21, avg(inv_quantity_on_hand#28)#22] +Results [5]: [w_warehouse_sk#32, i_item_sk#31, d_moy#35, stddev_samp(cast(inv_quantity_on_hand#28 as double))#21 AS stdev#46, avg(inv_quantity_on_hand#28)#22 AS mean#47] + +(39) Filter [codegen id : 9] +Input [5]: [w_warehouse_sk#32, i_item_sk#31, d_moy#35, stdev#46, mean#47] +Condition : CASE WHEN (mean#47 = 0.0) THEN false ELSE ((stdev#46 / mean#47) > 1.0) END + +(40) Project [codegen id : 9] +Output [5]: [w_warehouse_sk#32, i_item_sk#31, d_moy#35, mean#47, CASE WHEN (mean#47 = 0.0) THEN null ELSE (stdev#46 / mean#47) END AS cov#48] +Input [5]: [w_warehouse_sk#32, i_item_sk#31, d_moy#35, stdev#46, mean#47] + +(41) BroadcastExchange +Input [5]: [w_warehouse_sk#32, i_item_sk#31, d_moy#35, mean#47, cov#48] +Arguments: HashedRelationBroadcastMode(List((shiftleft(cast(input[1, int, true] as bigint), 32) | (cast(input[0, int, true] as bigint) & 4294967295))),false), [plan_id=5] + +(42) BroadcastHashJoin [codegen id : 10] +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 +Join condition: None + +(43) Exchange +Input [10]: [w_warehouse_sk#7, i_item_sk#6, d_moy#10, mean#24, cov#25, w_warehouse_sk#32, i_item_sk#31, d_moy#35, mean#47, cov#48] +Arguments: rangepartitioning(w_warehouse_sk#7 ASC NULLS FIRST, i_item_sk#6 ASC NULLS FIRST, d_moy#10 ASC NULLS FIRST, mean#24 ASC NULLS FIRST, cov#25 ASC NULLS FIRST, d_moy#35 ASC NULLS FIRST, mean#47 ASC NULLS FIRST, cov#48 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, [plan_id=6] + +(44) Sort [codegen id : 11] +Input [10]: [w_warehouse_sk#7, i_item_sk#6, d_moy#10, mean#24, cov#25, w_warehouse_sk#32, i_item_sk#31, d_moy#35, mean#47, cov#48] +Arguments: [w_warehouse_sk#7 ASC NULLS FIRST, i_item_sk#6 ASC NULLS FIRST, d_moy#10 ASC NULLS FIRST, mean#24 ASC NULLS FIRST, cov#25 ASC NULLS FIRST, d_moy#35 ASC NULLS FIRST, mean#47 ASC NULLS FIRST, cov#48 ASC NULLS FIRST], true, 0 + +===== Subqueries ===== + +Subquery:1 Hosting operator id = 1 Hosting Expression = inv_date_sk#4 IN dynamicpruning#5 +BroadcastExchange (49) ++- * ColumnarToRow (48) + +- CometProject (47) + +- CometFilter (46) + +- CometScan parquet spark_catalog.default.date_dim (45) + + +(45) Scan parquet spark_catalog.default.date_dim +Output [3]: [d_date_sk#9, d_year#49, d_moy#10] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2001), EqualTo(d_moy,1), IsNotNull(d_date_sk)] +ReadSchema: struct<d_date_sk:int,d_year:int,d_moy:int> + +(46) CometFilter +Input [3]: [d_date_sk#9, d_year#49, d_moy#10] +Condition : ((((isnotnull(d_year#49) AND isnotnull(d_moy#10)) AND (d_year#49 = 2001)) AND (d_moy#10 = 1)) AND isnotnull(d_date_sk#9)) + +(47) CometProject +Input [3]: [d_date_sk#9, d_year#49, d_moy#10] +Arguments: [d_date_sk#9, d_moy#10], [d_date_sk#9, d_moy#10] + +(48) ColumnarToRow [codegen id : 1] +Input [2]: [d_date_sk#9, d_moy#10] + +(49) BroadcastExchange +Input [2]: [d_date_sk#9, d_moy#10] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=7] + +Subquery:2 Hosting operator id = 24 Hosting Expression = inv_date_sk#29 IN dynamicpruning#30 +BroadcastExchange (54) ++- * ColumnarToRow (53) + +- CometProject (52) + +- CometFilter (51) + +- CometScan parquet spark_catalog.default.date_dim (50) + + +(50) Scan parquet spark_catalog.default.date_dim +Output [3]: [d_date_sk#34, d_year#50, d_moy#35] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2001), EqualTo(d_moy,2), IsNotNull(d_date_sk)] +ReadSchema: struct<d_date_sk:int,d_year:int,d_moy:int> + +(51) CometFilter +Input [3]: [d_date_sk#34, d_year#50, d_moy#35] +Condition : ((((isnotnull(d_year#50) AND isnotnull(d_moy#35)) AND (d_year#50 = 2001)) AND (d_moy#35 = 2)) AND isnotnull(d_date_sk#34)) + +(52) CometProject +Input [3]: [d_date_sk#34, d_year#50, d_moy#35] +Arguments: [d_date_sk#34, d_moy#35], [d_date_sk#34, d_moy#35] + +(53) ColumnarToRow [codegen id : 1] +Input [2]: [d_date_sk#34, d_moy#35] + +(54) BroadcastExchange +Input [2]: [d_date_sk#34, d_moy#35] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=8] + + 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 new file mode 100644 index 000000000..002266e76 --- /dev/null +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q39a/simplified.txt @@ -0,0 +1,81 @@ +WholeStageCodegen (11) + Sort [w_warehouse_sk,i_item_sk,d_moy,mean,cov,d_moy,mean,cov] + InputAdapter + Exchange [w_warehouse_sk,i_item_sk,d_moy,mean,cov,d_moy,mean,cov] #1 + WholeStageCodegen (10) + BroadcastHashJoin [i_item_sk,w_warehouse_sk,i_item_sk,w_warehouse_sk] + Project [w_warehouse_sk,i_item_sk,d_moy,mean,stdev] + Filter [mean,stdev] + HashAggregate [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy,n,avg,m2,sum,count] [stddev_samp(cast(inv_quantity_on_hand as double)),avg(inv_quantity_on_hand),stdev,mean,n,avg,m2,sum,count] + InputAdapter + Exchange [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy] #2 + WholeStageCodegen (4) + HashAggregate [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy,inv_quantity_on_hand] [n,avg,m2,sum,count,n,avg,m2,sum,count] + Project [inv_quantity_on_hand,i_item_sk,w_warehouse_sk,w_warehouse_name,d_moy] + BroadcastHashJoin [inv_date_sk,d_date_sk] + Project [inv_quantity_on_hand,inv_date_sk,i_item_sk,w_warehouse_sk,w_warehouse_name] + BroadcastHashJoin [inv_warehouse_sk,w_warehouse_sk] + Project [inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk,i_item_sk] + BroadcastHashJoin [inv_item_sk,i_item_sk] + ColumnarToRow + InputAdapter + CometFilter [inv_item_sk,inv_warehouse_sk] + CometScan parquet spark_catalog.default.inventory [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk,d_moy] + CometFilter [d_year,d_moy,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + InputAdapter + BroadcastExchange #4 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometFilter [i_item_sk] + CometScan parquet spark_catalog.default.item [i_item_sk] + InputAdapter + BroadcastExchange #5 + WholeStageCodegen (2) + ColumnarToRow + InputAdapter + CometFilter [w_warehouse_sk] + CometScan parquet spark_catalog.default.warehouse [w_warehouse_sk,w_warehouse_name] + InputAdapter + ReusedExchange [d_date_sk,d_moy] #3 + 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] + InputAdapter + Exchange [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy] #7 + WholeStageCodegen (8) + HashAggregate [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy,inv_quantity_on_hand] [n,avg,m2,sum,count,n,avg,m2,sum,count] + Project [inv_quantity_on_hand,i_item_sk,w_warehouse_sk,w_warehouse_name,d_moy] + BroadcastHashJoin [inv_date_sk,d_date_sk] + Project [inv_quantity_on_hand,inv_date_sk,i_item_sk,w_warehouse_sk,w_warehouse_name] + BroadcastHashJoin [inv_warehouse_sk,w_warehouse_sk] + Project [inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk,i_item_sk] + BroadcastHashJoin [inv_item_sk,i_item_sk] + ColumnarToRow + InputAdapter + CometFilter [inv_item_sk,inv_warehouse_sk] + CometScan parquet spark_catalog.default.inventory [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] + SubqueryBroadcast [d_date_sk] #2 + BroadcastExchange #8 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk,d_moy] + CometFilter [d_year,d_moy,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + InputAdapter + ReusedExchange [i_item_sk] #4 + InputAdapter + ReusedExchange [w_warehouse_sk,w_warehouse_name] #5 + InputAdapter + ReusedExchange [d_date_sk,d_moy] #8 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 new file mode 100644 index 000000000..a4184150e --- /dev/null +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q39b/explain.txt @@ -0,0 +1,318 @@ +== Physical Plan == +* Sort (44) ++- Exchange (43) + +- * BroadcastHashJoin Inner BuildRight (42) + :- * Project (23) + : +- * Filter (22) + : +- * HashAggregate (21) + : +- Exchange (20) + : +- * HashAggregate (19) + : +- * Project (18) + : +- * BroadcastHashJoin Inner BuildRight (17) + : :- * Project (15) + : : +- * BroadcastHashJoin Inner BuildRight (14) + : : :- * Project (9) + : : : +- * BroadcastHashJoin Inner BuildRight (8) + : : : :- * ColumnarToRow (3) + : : : : +- CometFilter (2) + : : : : +- CometScan parquet spark_catalog.default.inventory (1) + : : : +- BroadcastExchange (7) + : : : +- * ColumnarToRow (6) + : : : +- CometFilter (5) + : : : +- CometScan parquet spark_catalog.default.item (4) + : : +- BroadcastExchange (13) + : : +- * ColumnarToRow (12) + : : +- CometFilter (11) + : : +- CometScan parquet spark_catalog.default.warehouse (10) + : +- ReusedExchange (16) + +- BroadcastExchange (41) + +- * Project (40) + +- * Filter (39) + +- * HashAggregate (38) + +- Exchange (37) + +- * HashAggregate (36) + +- * Project (35) + +- * BroadcastHashJoin Inner BuildRight (34) + :- * Project (32) + : +- * BroadcastHashJoin Inner BuildRight (31) + : :- * Project (29) + : : +- * BroadcastHashJoin Inner BuildRight (28) + : : :- * ColumnarToRow (26) + : : : +- CometFilter (25) + : : : +- CometScan parquet spark_catalog.default.inventory (24) + : : +- ReusedExchange (27) + : +- ReusedExchange (30) + +- ReusedExchange (33) + + +(1) Scan parquet spark_catalog.default.inventory +Output [4]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4] +Batched: true +Location: InMemoryFileIndex [] +PartitionFilters: [isnotnull(inv_date_sk#4), dynamicpruningexpression(inv_date_sk#4 IN dynamicpruning#5)] +PushedFilters: [IsNotNull(inv_item_sk), IsNotNull(inv_warehouse_sk)] +ReadSchema: struct<inv_item_sk:int,inv_warehouse_sk:int,inv_quantity_on_hand:int> + +(2) CometFilter +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)) + +(3) ColumnarToRow [codegen id : 4] +Input [4]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4] + +(4) Scan parquet spark_catalog.default.item +Output [1]: [i_item_sk#6] +Batched: true +Location [not included in comparison]/{warehouse_dir}/item] +PushedFilters: [IsNotNull(i_item_sk)] +ReadSchema: struct<i_item_sk:int> + +(5) CometFilter +Input [1]: [i_item_sk#6] +Condition : isnotnull(i_item_sk#6) + +(6) ColumnarToRow [codegen id : 1] +Input [1]: [i_item_sk#6] + +(7) BroadcastExchange +Input [1]: [i_item_sk#6] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=1] + +(8) BroadcastHashJoin [codegen id : 4] +Left keys [1]: [inv_item_sk#1] +Right keys [1]: [i_item_sk#6] +Join type: Inner +Join condition: None + +(9) Project [codegen id : 4] +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] + +(10) Scan parquet spark_catalog.default.warehouse +Output [2]: [w_warehouse_sk#7, w_warehouse_name#8] +Batched: true +Location [not included in comparison]/{warehouse_dir}/warehouse] +PushedFilters: [IsNotNull(w_warehouse_sk)] +ReadSchema: struct<w_warehouse_sk:int,w_warehouse_name:string> + +(11) CometFilter +Input [2]: [w_warehouse_sk#7, w_warehouse_name#8] +Condition : isnotnull(w_warehouse_sk#7) + +(12) ColumnarToRow [codegen id : 2] +Input [2]: [w_warehouse_sk#7, w_warehouse_name#8] + +(13) BroadcastExchange +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] +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] +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: 49] +Output [2]: [d_date_sk#9, d_moy#10] + +(17) BroadcastHashJoin [codegen id : 4] +Left keys [1]: [inv_date_sk#4] +Right keys [1]: [d_date_sk#9] +Join type: Inner +Join condition: None + +(18) Project [codegen id : 4] +Output [5]: [inv_quantity_on_hand#3, i_item_sk#6, w_warehouse_sk#7, w_warehouse_name#8, d_moy#10] +Input [7]: [inv_quantity_on_hand#3, inv_date_sk#4, i_item_sk#6, w_warehouse_sk#7, w_warehouse_name#8, d_date_sk#9, d_moy#10] + +(19) HashAggregate [codegen id : 4] +Input [5]: [inv_quantity_on_hand#3, i_item_sk#6, w_warehouse_sk#7, w_warehouse_name#8, d_moy#10] +Keys [4]: [w_warehouse_name#8, w_warehouse_sk#7, i_item_sk#6, d_moy#10] +Functions [2]: [partial_stddev_samp(cast(inv_quantity_on_hand#3 as double)), partial_avg(inv_quantity_on_hand#3)] +Aggregate Attributes [5]: [n#11, avg#12, m2#13, sum#14, count#15] +Results [9]: [w_warehouse_name#8, w_warehouse_sk#7, i_item_sk#6, d_moy#10, n#16, avg#17, m2#18, sum#19, count#20] + +(20) Exchange +Input [9]: [w_warehouse_name#8, w_warehouse_sk#7, i_item_sk#6, d_moy#10, n#16, avg#17, m2#18, sum#19, count#20] +Arguments: hashpartitioning(w_warehouse_name#8, w_warehouse_sk#7, i_item_sk#6, d_moy#10, 5), ENSURE_REQUIREMENTS, [plan_id=3] + +(21) HashAggregate [codegen id : 10] +Input [9]: [w_warehouse_name#8, w_warehouse_sk#7, i_item_sk#6, d_moy#10, n#16, avg#17, m2#18, sum#19, count#20] +Keys [4]: [w_warehouse_name#8, w_warehouse_sk#7, i_item_sk#6, d_moy#10] +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] + +(22) Filter [codegen id : 10] +Input [5]: [w_warehouse_sk#7, i_item_sk#6, d_moy#10, stdev#23, mean#24] +Condition : (CASE WHEN (mean#24 = 0.0) THEN false ELSE ((stdev#23 / mean#24) > 1.0) END AND CASE WHEN (mean#24 = 0.0) THEN false ELSE ((stdev#23 / mean#24) > 1.5) END) + +(23) Project [codegen id : 10] +Output [5]: [w_warehouse_sk#7, i_item_sk#6, d_moy#10, mean#24, CASE WHEN (mean#24 = 0.0) THEN null ELSE (stdev#23 / mean#24) END AS cov#25] +Input [5]: [w_warehouse_sk#7, i_item_sk#6, d_moy#10, stdev#23, mean#24] + +(24) Scan parquet spark_catalog.default.inventory +Output [4]: [inv_item_sk#26, inv_warehouse_sk#27, inv_quantity_on_hand#28, inv_date_sk#29] +Batched: true +Location: InMemoryFileIndex [] +PartitionFilters: [isnotnull(inv_date_sk#29), dynamicpruningexpression(inv_date_sk#29 IN dynamicpruning#30)] +PushedFilters: [IsNotNull(inv_item_sk), IsNotNull(inv_warehouse_sk)] +ReadSchema: struct<inv_item_sk:int,inv_warehouse_sk:int,inv_quantity_on_hand:int> + +(25) CometFilter +Input [4]: [inv_item_sk#26, inv_warehouse_sk#27, inv_quantity_on_hand#28, inv_date_sk#29] +Condition : (isnotnull(inv_item_sk#26) AND isnotnull(inv_warehouse_sk#27)) + +(26) ColumnarToRow [codegen id : 8] +Input [4]: [inv_item_sk#26, inv_warehouse_sk#27, inv_quantity_on_hand#28, inv_date_sk#29] + +(27) ReusedExchange [Reuses operator id: 7] +Output [1]: [i_item_sk#31] + +(28) BroadcastHashJoin [codegen id : 8] +Left keys [1]: [inv_item_sk#26] +Right keys [1]: [i_item_sk#31] +Join type: Inner +Join condition: None + +(29) Project [codegen id : 8] +Output [4]: [inv_warehouse_sk#27, inv_quantity_on_hand#28, inv_date_sk#29, i_item_sk#31] +Input [5]: [inv_item_sk#26, inv_warehouse_sk#27, inv_quantity_on_hand#28, inv_date_sk#29, i_item_sk#31] + +(30) ReusedExchange [Reuses operator id: 13] +Output [2]: [w_warehouse_sk#32, w_warehouse_name#33] + +(31) BroadcastHashJoin [codegen id : 8] +Left keys [1]: [inv_warehouse_sk#27] +Right keys [1]: [w_warehouse_sk#32] +Join type: Inner +Join condition: None + +(32) Project [codegen id : 8] +Output [5]: [inv_quantity_on_hand#28, inv_date_sk#29, i_item_sk#31, w_warehouse_sk#32, w_warehouse_name#33] +Input [6]: [inv_warehouse_sk#27, inv_quantity_on_hand#28, inv_date_sk#29, i_item_sk#31, w_warehouse_sk#32, w_warehouse_name#33] + +(33) ReusedExchange [Reuses operator id: 54] +Output [2]: [d_date_sk#34, d_moy#35] + +(34) BroadcastHashJoin [codegen id : 8] +Left keys [1]: [inv_date_sk#29] +Right keys [1]: [d_date_sk#34] +Join type: Inner +Join condition: None + +(35) Project [codegen id : 8] +Output [5]: [inv_quantity_on_hand#28, i_item_sk#31, w_warehouse_sk#32, w_warehouse_name#33, d_moy#35] +Input [7]: [inv_quantity_on_hand#28, inv_date_sk#29, i_item_sk#31, w_warehouse_sk#32, w_warehouse_name#33, d_date_sk#34, d_moy#35] + +(36) HashAggregate [codegen id : 8] +Input [5]: [inv_quantity_on_hand#28, i_item_sk#31, w_warehouse_sk#32, w_warehouse_name#33, d_moy#35] +Keys [4]: [w_warehouse_name#33, w_warehouse_sk#32, i_item_sk#31, d_moy#35] +Functions [2]: [partial_stddev_samp(cast(inv_quantity_on_hand#28 as double)), partial_avg(inv_quantity_on_hand#28)] +Aggregate Attributes [5]: [n#36, avg#37, m2#38, sum#39, count#40] +Results [9]: [w_warehouse_name#33, w_warehouse_sk#32, i_item_sk#31, d_moy#35, n#41, avg#42, m2#43, sum#44, count#45] + +(37) Exchange +Input [9]: [w_warehouse_name#33, w_warehouse_sk#32, i_item_sk#31, d_moy#35, n#41, avg#42, m2#43, sum#44, count#45] +Arguments: hashpartitioning(w_warehouse_name#33, w_warehouse_sk#32, i_item_sk#31, d_moy#35, 5), ENSURE_REQUIREMENTS, [plan_id=4] + +(38) HashAggregate [codegen id : 9] +Input [9]: [w_warehouse_name#33, w_warehouse_sk#32, i_item_sk#31, d_moy#35, n#41, avg#42, m2#43, sum#44, count#45] +Keys [4]: [w_warehouse_name#33, w_warehouse_sk#32, i_item_sk#31, d_moy#35] +Functions [2]: [stddev_samp(cast(inv_quantity_on_hand#28 as double)), avg(inv_quantity_on_hand#28)] +Aggregate Attributes [2]: [stddev_samp(cast(inv_quantity_on_hand#28 as double))#21, avg(inv_quantity_on_hand#28)#22] +Results [5]: [w_warehouse_sk#32, i_item_sk#31, d_moy#35, stddev_samp(cast(inv_quantity_on_hand#28 as double))#21 AS stdev#46, avg(inv_quantity_on_hand#28)#22 AS mean#47] + +(39) Filter [codegen id : 9] +Input [5]: [w_warehouse_sk#32, i_item_sk#31, d_moy#35, stdev#46, mean#47] +Condition : CASE WHEN (mean#47 = 0.0) THEN false ELSE ((stdev#46 / mean#47) > 1.0) END + +(40) Project [codegen id : 9] +Output [5]: [w_warehouse_sk#32, i_item_sk#31, d_moy#35, mean#47, CASE WHEN (mean#47 = 0.0) THEN null ELSE (stdev#46 / mean#47) END AS cov#48] +Input [5]: [w_warehouse_sk#32, i_item_sk#31, d_moy#35, stdev#46, mean#47] + +(41) BroadcastExchange +Input [5]: [w_warehouse_sk#32, i_item_sk#31, d_moy#35, mean#47, cov#48] +Arguments: HashedRelationBroadcastMode(List((shiftleft(cast(input[1, int, true] as bigint), 32) | (cast(input[0, int, true] as bigint) & 4294967295))),false), [plan_id=5] + +(42) BroadcastHashJoin [codegen id : 10] +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 +Join condition: None + +(43) Exchange +Input [10]: [w_warehouse_sk#7, i_item_sk#6, d_moy#10, mean#24, cov#25, w_warehouse_sk#32, i_item_sk#31, d_moy#35, mean#47, cov#48] +Arguments: rangepartitioning(w_warehouse_sk#7 ASC NULLS FIRST, i_item_sk#6 ASC NULLS FIRST, d_moy#10 ASC NULLS FIRST, mean#24 ASC NULLS FIRST, cov#25 ASC NULLS FIRST, d_moy#35 ASC NULLS FIRST, mean#47 ASC NULLS FIRST, cov#48 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, [plan_id=6] + +(44) Sort [codegen id : 11] +Input [10]: [w_warehouse_sk#7, i_item_sk#6, d_moy#10, mean#24, cov#25, w_warehouse_sk#32, i_item_sk#31, d_moy#35, mean#47, cov#48] +Arguments: [w_warehouse_sk#7 ASC NULLS FIRST, i_item_sk#6 ASC NULLS FIRST, d_moy#10 ASC NULLS FIRST, mean#24 ASC NULLS FIRST, cov#25 ASC NULLS FIRST, d_moy#35 ASC NULLS FIRST, mean#47 ASC NULLS FIRST, cov#48 ASC NULLS FIRST], true, 0 + +===== Subqueries ===== + +Subquery:1 Hosting operator id = 1 Hosting Expression = inv_date_sk#4 IN dynamicpruning#5 +BroadcastExchange (49) ++- * ColumnarToRow (48) + +- CometProject (47) + +- CometFilter (46) + +- CometScan parquet spark_catalog.default.date_dim (45) + + +(45) Scan parquet spark_catalog.default.date_dim +Output [3]: [d_date_sk#9, d_year#49, d_moy#10] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2001), EqualTo(d_moy,1), IsNotNull(d_date_sk)] +ReadSchema: struct<d_date_sk:int,d_year:int,d_moy:int> + +(46) CometFilter +Input [3]: [d_date_sk#9, d_year#49, d_moy#10] +Condition : ((((isnotnull(d_year#49) AND isnotnull(d_moy#10)) AND (d_year#49 = 2001)) AND (d_moy#10 = 1)) AND isnotnull(d_date_sk#9)) + +(47) CometProject +Input [3]: [d_date_sk#9, d_year#49, d_moy#10] +Arguments: [d_date_sk#9, d_moy#10], [d_date_sk#9, d_moy#10] + +(48) ColumnarToRow [codegen id : 1] +Input [2]: [d_date_sk#9, d_moy#10] + +(49) BroadcastExchange +Input [2]: [d_date_sk#9, d_moy#10] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=7] + +Subquery:2 Hosting operator id = 24 Hosting Expression = inv_date_sk#29 IN dynamicpruning#30 +BroadcastExchange (54) ++- * ColumnarToRow (53) + +- CometProject (52) + +- CometFilter (51) + +- CometScan parquet spark_catalog.default.date_dim (50) + + +(50) Scan parquet spark_catalog.default.date_dim +Output [3]: [d_date_sk#34, d_year#50, d_moy#35] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2001), EqualTo(d_moy,2), IsNotNull(d_date_sk)] +ReadSchema: struct<d_date_sk:int,d_year:int,d_moy:int> + +(51) CometFilter +Input [3]: [d_date_sk#34, d_year#50, d_moy#35] +Condition : ((((isnotnull(d_year#50) AND isnotnull(d_moy#35)) AND (d_year#50 = 2001)) AND (d_moy#35 = 2)) AND isnotnull(d_date_sk#34)) + +(52) CometProject +Input [3]: [d_date_sk#34, d_year#50, d_moy#35] +Arguments: [d_date_sk#34, d_moy#35], [d_date_sk#34, d_moy#35] + +(53) ColumnarToRow [codegen id : 1] +Input [2]: [d_date_sk#34, d_moy#35] + +(54) BroadcastExchange +Input [2]: [d_date_sk#34, d_moy#35] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=8] + + 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 new file mode 100644 index 000000000..002266e76 --- /dev/null +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q39b/simplified.txt @@ -0,0 +1,81 @@ +WholeStageCodegen (11) + Sort [w_warehouse_sk,i_item_sk,d_moy,mean,cov,d_moy,mean,cov] + InputAdapter + Exchange [w_warehouse_sk,i_item_sk,d_moy,mean,cov,d_moy,mean,cov] #1 + WholeStageCodegen (10) + BroadcastHashJoin [i_item_sk,w_warehouse_sk,i_item_sk,w_warehouse_sk] + Project [w_warehouse_sk,i_item_sk,d_moy,mean,stdev] + Filter [mean,stdev] + HashAggregate [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy,n,avg,m2,sum,count] [stddev_samp(cast(inv_quantity_on_hand as double)),avg(inv_quantity_on_hand),stdev,mean,n,avg,m2,sum,count] + InputAdapter + Exchange [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy] #2 + WholeStageCodegen (4) + HashAggregate [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy,inv_quantity_on_hand] [n,avg,m2,sum,count,n,avg,m2,sum,count] + Project [inv_quantity_on_hand,i_item_sk,w_warehouse_sk,w_warehouse_name,d_moy] + BroadcastHashJoin [inv_date_sk,d_date_sk] + Project [inv_quantity_on_hand,inv_date_sk,i_item_sk,w_warehouse_sk,w_warehouse_name] + BroadcastHashJoin [inv_warehouse_sk,w_warehouse_sk] + Project [inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk,i_item_sk] + BroadcastHashJoin [inv_item_sk,i_item_sk] + ColumnarToRow + InputAdapter + CometFilter [inv_item_sk,inv_warehouse_sk] + CometScan parquet spark_catalog.default.inventory [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk,d_moy] + CometFilter [d_year,d_moy,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + InputAdapter + BroadcastExchange #4 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometFilter [i_item_sk] + CometScan parquet spark_catalog.default.item [i_item_sk] + InputAdapter + BroadcastExchange #5 + WholeStageCodegen (2) + ColumnarToRow + InputAdapter + CometFilter [w_warehouse_sk] + CometScan parquet spark_catalog.default.warehouse [w_warehouse_sk,w_warehouse_name] + InputAdapter + ReusedExchange [d_date_sk,d_moy] #3 + 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] + InputAdapter + Exchange [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy] #7 + WholeStageCodegen (8) + HashAggregate [w_warehouse_name,w_warehouse_sk,i_item_sk,d_moy,inv_quantity_on_hand] [n,avg,m2,sum,count,n,avg,m2,sum,count] + Project [inv_quantity_on_hand,i_item_sk,w_warehouse_sk,w_warehouse_name,d_moy] + BroadcastHashJoin [inv_date_sk,d_date_sk] + Project [inv_quantity_on_hand,inv_date_sk,i_item_sk,w_warehouse_sk,w_warehouse_name] + BroadcastHashJoin [inv_warehouse_sk,w_warehouse_sk] + Project [inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk,i_item_sk] + BroadcastHashJoin [inv_item_sk,i_item_sk] + ColumnarToRow + InputAdapter + CometFilter [inv_item_sk,inv_warehouse_sk] + CometScan parquet spark_catalog.default.inventory [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] + SubqueryBroadcast [d_date_sk] #2 + BroadcastExchange #8 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk,d_moy] + CometFilter [d_year,d_moy,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + InputAdapter + ReusedExchange [i_item_sk] #4 + InputAdapter + ReusedExchange [w_warehouse_sk,w_warehouse_name] #5 + InputAdapter + ReusedExchange [d_date_sk,d_moy] #8 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 new file mode 100644 index 000000000..a854a1041 --- /dev/null +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q4/explain.txt @@ -0,0 +1,698 @@ +== Physical Plan == +TakeOrderedAndProject (108) ++- * Project (107) + +- * BroadcastHashJoin Inner BuildRight (106) + :- * Project (89) + : +- * BroadcastHashJoin Inner BuildRight (88) + : :- * Project (70) + : : +- * BroadcastHashJoin Inner BuildRight (69) + : : :- * Project (52) + : : : +- * BroadcastHashJoin Inner BuildRight (51) + : : : :- * BroadcastHashJoin Inner BuildRight (33) + : : : : :- * Filter (16) + : : : : : +- * HashAggregate (15) + : : : : : +- Exchange (14) + : : : : : +- * HashAggregate (13) + : : : : : +- * Project (12) + : : : : : +- * BroadcastHashJoin Inner BuildRight (11) + : : : : : :- * Project (9) + : : : : : : +- * BroadcastHashJoin Inner BuildRight (8) + : : : : : : :- * ColumnarToRow (3) + : : : : : : : +- CometFilter (2) + : : : : : : : +- CometScan parquet spark_catalog.default.customer (1) + : : : : : : +- BroadcastExchange (7) + : : : : : : +- * ColumnarToRow (6) + : : : : : : +- CometFilter (5) + : : : : : : +- CometScan parquet spark_catalog.default.store_sales (4) + : : : : : +- ReusedExchange (10) + : : : : +- BroadcastExchange (32) + : : : : +- * HashAggregate (31) + : : : : +- Exchange (30) + : : : : +- * HashAggregate (29) + : : : : +- * Project (28) + : : : : +- * BroadcastHashJoin Inner BuildRight (27) + : : : : :- * Project (25) + : : : : : +- * BroadcastHashJoin Inner BuildRight (24) + : : : : : :- * ColumnarToRow (19) + : : : : : : +- CometFilter (18) + : : : : : : +- CometScan parquet spark_catalog.default.customer (17) + : : : : : +- BroadcastExchange (23) + : : : : : +- * ColumnarToRow (22) + : : : : : +- CometFilter (21) + : : : : : +- CometScan parquet spark_catalog.default.store_sales (20) + : : : : +- ReusedExchange (26) + : : : +- BroadcastExchange (50) + : : : +- * Filter (49) + : : : +- * HashAggregate (48) + : : : +- Exchange (47) + : : : +- * HashAggregate (46) + : : : +- * Project (45) + : : : +- * BroadcastHashJoin Inner BuildRight (44) + : : : :- * Project (42) + : : : : +- * BroadcastHashJoin Inner BuildRight (41) + : : : : :- * ColumnarToRow (36) + : : : : : +- CometFilter (35) + : : : : : +- CometScan parquet spark_catalog.default.customer (34) + : : : : +- BroadcastExchange (40) + : : : : +- * ColumnarToRow (39) + : : : : +- CometFilter (38) + : : : : +- CometScan parquet spark_catalog.default.catalog_sales (37) + : : : +- ReusedExchange (43) + : : +- BroadcastExchange (68) + : : +- * HashAggregate (67) + : : +- Exchange (66) + : : +- * HashAggregate (65) + : : +- * Project (64) + : : +- * BroadcastHashJoin Inner BuildRight (63) + : : :- * Project (61) + : : : +- * BroadcastHashJoin Inner BuildRight (60) + : : : :- * ColumnarToRow (55) + : : : : +- CometFilter (54) + : : : : +- CometScan parquet spark_catalog.default.customer (53) + : : : +- BroadcastExchange (59) + : : : +- * ColumnarToRow (58) + : : : +- CometFilter (57) + : : : +- CometScan parquet spark_catalog.default.catalog_sales (56) + : : +- ReusedExchange (62) + : +- BroadcastExchange (87) + : +- * Filter (86) + : +- * HashAggregate (85) + : +- Exchange (84) + : +- * HashAggregate (83) + : +- * Project (82) + : +- * BroadcastHashJoin Inner BuildRight (81) + : :- * Project (79) + : : +- * BroadcastHashJoin Inner BuildRight (78) + : : :- * ColumnarToRow (73) + : : : +- CometFilter (72) + : : : +- CometScan parquet spark_catalog.default.customer (71) + : : +- BroadcastExchange (77) + : : +- * ColumnarToRow (76) + : : +- CometFilter (75) + : : +- CometScan parquet spark_catalog.default.web_sales (74) + : +- ReusedExchange (80) + +- BroadcastExchange (105) + +- * HashAggregate (104) + +- Exchange (103) + +- * HashAggregate (102) + +- * Project (101) + +- * BroadcastHashJoin Inner BuildRight (100) + :- * Project (98) + : +- * BroadcastHashJoin Inner BuildRight (97) + : :- * ColumnarToRow (92) + : : +- CometFilter (91) + : : +- CometScan parquet spark_catalog.default.customer (90) + : +- BroadcastExchange (96) + : +- * ColumnarToRow (95) + : +- CometFilter (94) + : +- CometScan parquet spark_catalog.default.web_sales (93) + +- ReusedExchange (99) + + +(1) Scan parquet spark_catalog.default.customer +Output [8]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8] +Batched: true +Location [not included in comparison]/{warehouse_dir}/customer] +PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_customer_id)] +ReadSchema: struct<c_customer_sk:int,c_customer_id:string,c_first_name:string,c_last_name:string,c_preferred_cust_flag:string,c_birth_country:string,c_login:string,c_email_address:string> + +(2) CometFilter +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] +Condition : (isnotnull(c_customer_sk#1) AND isnotnull(c_customer_id#2)) + +(3) ColumnarToRow [codegen id : 3] +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] + +(4) Scan parquet spark_catalog.default.store_sales +Output [6]: [ss_customer_sk#9, ss_ext_discount_amt#10, ss_ext_sales_price#11, ss_ext_wholesale_cost#12, ss_ext_list_price#13, ss_sold_date_sk#14] +Batched: true +Location: InMemoryFileIndex [] +PartitionFilters: [isnotnull(ss_sold_date_sk#14), dynamicpruningexpression(ss_sold_date_sk#14 IN dynamicpruning#15)] +PushedFilters: [IsNotNull(ss_customer_sk)] +ReadSchema: struct<ss_customer_sk:int,ss_ext_discount_amt:decimal(7,2),ss_ext_sales_price:decimal(7,2),ss_ext_wholesale_cost:decimal(7,2),ss_ext_list_price:decimal(7,2)> + +(5) CometFilter +Input [6]: [ss_customer_sk#9, ss_ext_discount_amt#10, ss_ext_sales_price#11, ss_ext_wholesale_cost#12, ss_ext_list_price#13, ss_sold_date_sk#14] +Condition : isnotnull(ss_customer_sk#9) + +(6) ColumnarToRow [codegen id : 1] +Input [6]: [ss_customer_sk#9, ss_ext_discount_amt#10, ss_ext_sales_price#11, ss_ext_wholesale_cost#12, ss_ext_list_price#13, ss_sold_date_sk#14] + +(7) BroadcastExchange +Input [6]: [ss_customer_sk#9, ss_ext_discount_amt#10, ss_ext_sales_price#11, ss_ext_wholesale_cost#12, ss_ext_list_price#13, ss_sold_date_sk#14] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=1] + +(8) BroadcastHashJoin [codegen id : 3] +Left keys [1]: [c_customer_sk#1] +Right keys [1]: [ss_customer_sk#9] +Join type: Inner +Join condition: None + +(9) Project [codegen id : 3] +Output [12]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, ss_ext_discount_amt#10, ss_ext_sales_price#11, ss_ext_wholesale_cost#12, ss_ext_list_price#13, ss_sold_date_sk#14] +Input [14]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, ss_customer_sk#9, ss_ext_discount_amt#10, ss_ext_sales_price#11, ss_ext_wholesale_cost#12, ss_ext_list_price#13, ss_sold_date_sk#14] + +(10) ReusedExchange [Reuses operator id: 112] +Output [2]: [d_date_sk#16, d_year#17] + +(11) BroadcastHashJoin [codegen id : 3] +Left keys [1]: [ss_sold_date_sk#14] +Right keys [1]: [d_date_sk#16] +Join type: Inner +Join condition: None + +(12) Project [codegen id : 3] +Output [12]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, ss_ext_discount_amt#10, ss_ext_sales_price#11, ss_ext_wholesale_cost#12, ss_ext_list_price#13, d_year#17] +Input [14]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, ss_ext_discount_amt#10, ss_ext_sales_price#11, ss_ext_wholesale_cost#12, ss_ext_list_price#13, ss_sold_date_sk#14, d_date_sk#16, d_year#17] + +(13) HashAggregate [codegen id : 3] +Input [12]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, ss_ext_discount_amt#10, ss_ext_sales_price#11, ss_ext_wholesale_cost#12, ss_ext_list_price#13, d_year#17] +Keys [8]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, d_year#17] +Functions [1]: [partial_sum(((((ss_ext_list_price#13 - ss_ext_wholesale_cost#12) - ss_ext_discount_amt#10) + ss_ext_sales_price#11) / 2))] +Aggregate Attributes [2]: [sum#18, isEmpty#19] +Results [10]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, d_year#17, sum#20, isEmpty#21] + +(14) Exchange +Input [10]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, d_year#17, sum#20, isEmpty#21] +Arguments: hashpartitioning(c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, d_year#17, 5), ENSURE_REQUIREMENTS, [plan_id=2] + +(15) HashAggregate [codegen id : 24] +Input [10]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, d_year#17, sum#20, isEmpty#21] +Keys [8]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, d_year#17] +Functions [1]: [sum(((((ss_ext_list_price#13 - ss_ext_wholesale_cost#12) - ss_ext_discount_amt#10) + ss_ext_sales_price#11) / 2))] +Aggregate Attributes [1]: [sum(((((ss_ext_list_price#13 - ss_ext_wholesale_cost#12) - ss_ext_discount_amt#10) + ss_ext_sales_price#11) / 2))#22] +Results [2]: [c_customer_id#2 AS customer_id#23, sum(((((ss_ext_list_price#13 - ss_ext_wholesale_cost#12) - ss_ext_discount_amt#10) + ss_ext_sales_price#11) / 2))#22 AS year_total#24] + +(16) Filter [codegen id : 24] +Input [2]: [customer_id#23, year_total#24] +Condition : (isnotnull(year_total#24) AND (year_total#24 > 0.000000)) + +(17) Scan parquet spark_catalog.default.customer +Output [8]: [c_customer_sk#25, c_customer_id#26, c_first_name#27, c_last_name#28, c_preferred_cust_flag#29, c_birth_country#30, c_login#31, c_email_address#32] +Batched: true +Location [not included in comparison]/{warehouse_dir}/customer] +PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_customer_id)] +ReadSchema: struct<c_customer_sk:int,c_customer_id:string,c_first_name:string,c_last_name:string,c_preferred_cust_flag:string,c_birth_country:string,c_login:string,c_email_address:string> + +(18) CometFilter +Input [8]: [c_customer_sk#25, c_customer_id#26, c_first_name#27, c_last_name#28, c_preferred_cust_flag#29, c_birth_country#30, c_login#31, c_email_address#32] +Condition : (isnotnull(c_customer_sk#25) AND isnotnull(c_customer_id#26)) + +(19) ColumnarToRow [codegen id : 6] +Input [8]: [c_customer_sk#25, c_customer_id#26, c_first_name#27, c_last_name#28, c_preferred_cust_flag#29, c_birth_country#30, c_login#31, c_email_address#32] + +(20) Scan parquet spark_catalog.default.store_sales +Output [6]: [ss_customer_sk#33, ss_ext_discount_amt#34, ss_ext_sales_price#35, ss_ext_wholesale_cost#36, ss_ext_list_price#37, ss_sold_date_sk#38] +Batched: true +Location: InMemoryFileIndex [] +PartitionFilters: [isnotnull(ss_sold_date_sk#38), dynamicpruningexpression(ss_sold_date_sk#38 IN dynamicpruning#39)] +PushedFilters: [IsNotNull(ss_customer_sk)] +ReadSchema: struct<ss_customer_sk:int,ss_ext_discount_amt:decimal(7,2),ss_ext_sales_price:decimal(7,2),ss_ext_wholesale_cost:decimal(7,2),ss_ext_list_price:decimal(7,2)> + +(21) CometFilter +Input [6]: [ss_customer_sk#33, ss_ext_discount_amt#34, ss_ext_sales_price#35, ss_ext_wholesale_cost#36, ss_ext_list_price#37, ss_sold_date_sk#38] +Condition : isnotnull(ss_customer_sk#33) + +(22) ColumnarToRow [codegen id : 4] +Input [6]: [ss_customer_sk#33, ss_ext_discount_amt#34, ss_ext_sales_price#35, ss_ext_wholesale_cost#36, ss_ext_list_price#37, ss_sold_date_sk#38] + +(23) BroadcastExchange +Input [6]: [ss_customer_sk#33, ss_ext_discount_amt#34, ss_ext_sales_price#35, ss_ext_wholesale_cost#36, ss_ext_list_price#37, ss_sold_date_sk#38] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=3] + +(24) BroadcastHashJoin [codegen id : 6] +Left keys [1]: [c_customer_sk#25] +Right keys [1]: [ss_customer_sk#33] +Join type: Inner +Join condition: None + +(25) Project [codegen id : 6] +Output [12]: [c_customer_id#26, c_first_name#27, c_last_name#28, c_preferred_cust_flag#29, c_birth_country#30, c_login#31, c_email_address#32, ss_ext_discount_amt#34, ss_ext_sales_price#35, ss_ext_wholesale_cost#36, ss_ext_list_price#37, ss_sold_date_sk#38] +Input [14]: [c_customer_sk#25, c_customer_id#26, c_first_name#27, c_last_name#28, c_preferred_cust_flag#29, c_birth_country#30, c_login#31, c_email_address#32, ss_customer_sk#33, ss_ext_discount_amt#34, ss_ext_sales_price#35, ss_ext_wholesale_cost#36, ss_ext_list_price#37, ss_sold_date_sk#38] + +(26) ReusedExchange [Reuses operator id: 116] +Output [2]: [d_date_sk#40, d_year#41] + +(27) BroadcastHashJoin [codegen id : 6] +Left keys [1]: [ss_sold_date_sk#38] +Right keys [1]: [d_date_sk#40] +Join type: Inner +Join condition: None + +(28) Project [codegen id : 6] +Output [12]: [c_customer_id#26, c_first_name#27, c_last_name#28, c_preferred_cust_flag#29, c_birth_country#30, c_login#31, c_email_address#32, ss_ext_discount_amt#34, ss_ext_sales_price#35, ss_ext_wholesale_cost#36, ss_ext_list_price#37, d_year#41] +Input [14]: [c_customer_id#26, c_first_name#27, c_last_name#28, c_preferred_cust_flag#29, c_birth_country#30, c_login#31, c_email_address#32, ss_ext_discount_amt#34, ss_ext_sales_price#35, ss_ext_wholesale_cost#36, ss_ext_list_price#37, ss_sold_date_sk#38, d_date_sk#40, d_year#41] + +(29) HashAggregate [codegen id : 6] +Input [12]: [c_customer_id#26, c_first_name#27, c_last_name#28, c_preferred_cust_flag#29, c_birth_country#30, c_login#31, c_email_address#32, ss_ext_discount_amt#34, ss_ext_sales_price#35, ss_ext_wholesale_cost#36, ss_ext_list_price#37, d_year#41] +Keys [8]: [c_customer_id#26, c_first_name#27, c_last_name#28, c_preferred_cust_flag#29, c_birth_country#30, c_login#31, c_email_address#32, d_year#41] +Functions [1]: [partial_sum(((((ss_ext_list_price#37 - ss_ext_wholesale_cost#36) - ss_ext_discount_amt#34) + ss_ext_sales_price#35) / 2))] +Aggregate Attributes [2]: [sum#42, isEmpty#43] +Results [10]: [c_customer_id#26, c_first_name#27, c_last_name#28, c_preferred_cust_flag#29, c_birth_country#30, c_login#31, c_email_address#32, d_year#41, sum#44, isEmpty#45] + +(30) Exchange +Input [10]: [c_customer_id#26, c_first_name#27, c_last_name#28, c_preferred_cust_flag#29, c_birth_country#30, c_login#31, c_email_address#32, d_year#41, sum#44, isEmpty#45] +Arguments: hashpartitioning(c_customer_id#26, c_first_name#27, c_last_name#28, c_preferred_cust_flag#29, c_birth_country#30, c_login#31, c_email_address#32, d_year#41, 5), ENSURE_REQUIREMENTS, [plan_id=4] + +(31) HashAggregate [codegen id : 7] +Input [10]: [c_customer_id#26, c_first_name#27, c_last_name#28, c_preferred_cust_flag#29, c_birth_country#30, c_login#31, c_email_address#32, d_year#41, sum#44, isEmpty#45] +Keys [8]: [c_customer_id#26, c_first_name#27, c_last_name#28, c_preferred_cust_flag#29, c_birth_country#30, c_login#31, c_email_address#32, d_year#41] +Functions [1]: [sum(((((ss_ext_list_price#37 - ss_ext_wholesale_cost#36) - ss_ext_discount_amt#34) + ss_ext_sales_price#35) / 2))] +Aggregate Attributes [1]: [sum(((((ss_ext_list_price#37 - ss_ext_wholesale_cost#36) - ss_ext_discount_amt#34) + ss_ext_sales_price#35) / 2))#22] +Results [8]: [c_customer_id#26 AS customer_id#46, c_first_name#27 AS customer_first_name#47, c_last_name#28 AS customer_last_name#48, c_preferred_cust_flag#29 AS customer_preferred_cust_flag#49, c_birth_country#30 AS customer_birth_country#50, c_login#31 AS customer_login#51, c_email_address#32 AS customer_email_address#52, sum(((((ss_ext_list_price#37 - ss_ext_wholesale_cost#36) - ss_ext_discount_amt#34) + ss_ext_sales_price#35) / 2))#22 AS year_total#53] + +(32) BroadcastExchange +Input [8]: [customer_id#46, customer_first_name#47, customer_last_name#48, customer_preferred_cust_flag#49, customer_birth_country#50, customer_login#51, customer_email_address#52, year_total#53] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=5] + +(33) BroadcastHashJoin [codegen id : 24] +Left keys [1]: [customer_id#23] +Right keys [1]: [customer_id#46] +Join type: Inner +Join condition: None + +(34) Scan parquet spark_catalog.default.customer +Output [8]: [c_customer_sk#54, c_customer_id#55, c_first_name#56, c_last_name#57, c_preferred_cust_flag#58, c_birth_country#59, c_login#60, c_email_address#61] +Batched: true +Location [not included in comparison]/{warehouse_dir}/customer] +PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_customer_id)] +ReadSchema: struct<c_customer_sk:int,c_customer_id:string,c_first_name:string,c_last_name:string,c_preferred_cust_flag:string,c_birth_country:string,c_login:string,c_email_address:string> + +(35) CometFilter +Input [8]: [c_customer_sk#54, c_customer_id#55, c_first_name#56, c_last_name#57, c_preferred_cust_flag#58, c_birth_country#59, c_login#60, c_email_address#61] +Condition : (isnotnull(c_customer_sk#54) AND isnotnull(c_customer_id#55)) + +(36) ColumnarToRow [codegen id : 10] +Input [8]: [c_customer_sk#54, c_customer_id#55, c_first_name#56, c_last_name#57, c_preferred_cust_flag#58, c_birth_country#59, c_login#60, c_email_address#61] + +(37) Scan parquet spark_catalog.default.catalog_sales +Output [6]: [cs_bill_customer_sk#62, cs_ext_discount_amt#63, cs_ext_sales_price#64, cs_ext_wholesale_cost#65, cs_ext_list_price#66, cs_sold_date_sk#67] +Batched: true +Location: InMemoryFileIndex [] +PartitionFilters: [isnotnull(cs_sold_date_sk#67), dynamicpruningexpression(cs_sold_date_sk#67 IN dynamicpruning#68)] +PushedFilters: [IsNotNull(cs_bill_customer_sk)] +ReadSchema: struct<cs_bill_customer_sk:int,cs_ext_discount_amt:decimal(7,2),cs_ext_sales_price:decimal(7,2),cs_ext_wholesale_cost:decimal(7,2),cs_ext_list_price:decimal(7,2)> + +(38) CometFilter +Input [6]: [cs_bill_customer_sk#62, cs_ext_discount_amt#63, cs_ext_sales_price#64, cs_ext_wholesale_cost#65, cs_ext_list_price#66, cs_sold_date_sk#67] +Condition : isnotnull(cs_bill_customer_sk#62) + +(39) ColumnarToRow [codegen id : 8] +Input [6]: [cs_bill_customer_sk#62, cs_ext_discount_amt#63, cs_ext_sales_price#64, cs_ext_wholesale_cost#65, cs_ext_list_price#66, cs_sold_date_sk#67] + +(40) BroadcastExchange +Input [6]: [cs_bill_customer_sk#62, cs_ext_discount_amt#63, cs_ext_sales_price#64, cs_ext_wholesale_cost#65, cs_ext_list_price#66, cs_sold_date_sk#67] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=6] + +(41) BroadcastHashJoin [codegen id : 10] +Left keys [1]: [c_customer_sk#54] +Right keys [1]: [cs_bill_customer_sk#62] +Join type: Inner +Join condition: None + +(42) Project [codegen id : 10] +Output [12]: [c_customer_id#55, c_first_name#56, c_last_name#57, c_preferred_cust_flag#58, c_birth_country#59, c_login#60, c_email_address#61, cs_ext_discount_amt#63, cs_ext_sales_price#64, cs_ext_wholesale_cost#65, cs_ext_list_price#66, cs_sold_date_sk#67] +Input [14]: [c_customer_sk#54, c_customer_id#55, c_first_name#56, c_last_name#57, c_preferred_cust_flag#58, c_birth_country#59, c_login#60, c_email_address#61, cs_bill_customer_sk#62, cs_ext_discount_amt#63, cs_ext_sales_price#64, cs_ext_wholesale_cost#65, cs_ext_list_price#66, cs_sold_date_sk#67] + +(43) ReusedExchange [Reuses operator id: 112] +Output [2]: [d_date_sk#69, d_year#70] + +(44) BroadcastHashJoin [codegen id : 10] +Left keys [1]: [cs_sold_date_sk#67] +Right keys [1]: [d_date_sk#69] +Join type: Inner +Join condition: None + +(45) Project [codegen id : 10] +Output [12]: [c_customer_id#55, c_first_name#56, c_last_name#57, c_preferred_cust_flag#58, c_birth_country#59, c_login#60, c_email_address#61, cs_ext_discount_amt#63, cs_ext_sales_price#64, cs_ext_wholesale_cost#65, cs_ext_list_price#66, d_year#70] +Input [14]: [c_customer_id#55, c_first_name#56, c_last_name#57, c_preferred_cust_flag#58, c_birth_country#59, c_login#60, c_email_address#61, cs_ext_discount_amt#63, cs_ext_sales_price#64, cs_ext_wholesale_cost#65, cs_ext_list_price#66, cs_sold_date_sk#67, d_date_sk#69, d_year#70] + +(46) HashAggregate [codegen id : 10] +Input [12]: [c_customer_id#55, c_first_name#56, c_last_name#57, c_preferred_cust_flag#58, c_birth_country#59, c_login#60, c_email_address#61, cs_ext_discount_amt#63, cs_ext_sales_price#64, cs_ext_wholesale_cost#65, cs_ext_list_price#66, d_year#70] +Keys [8]: [c_customer_id#55, c_first_name#56, c_last_name#57, c_preferred_cust_flag#58, c_birth_country#59, c_login#60, c_email_address#61, d_year#70] +Functions [1]: [partial_sum(((((cs_ext_list_price#66 - cs_ext_wholesale_cost#65) - cs_ext_discount_amt#63) + cs_ext_sales_price#64) / 2))] +Aggregate Attributes [2]: [sum#71, isEmpty#72] +Results [10]: [c_customer_id#55, c_first_name#56, c_last_name#57, c_preferred_cust_flag#58, c_birth_country#59, c_login#60, c_email_address#61, d_year#70, sum#73, isEmpty#74] + +(47) Exchange +Input [10]: [c_customer_id#55, c_first_name#56, c_last_name#57, c_preferred_cust_flag#58, c_birth_country#59, c_login#60, c_email_address#61, d_year#70, sum#73, isEmpty#74] +Arguments: hashpartitioning(c_customer_id#55, c_first_name#56, c_last_name#57, c_preferred_cust_flag#58, c_birth_country#59, c_login#60, c_email_address#61, d_year#70, 5), ENSURE_REQUIREMENTS, [plan_id=7] + +(48) HashAggregate [codegen id : 11] +Input [10]: [c_customer_id#55, c_first_name#56, c_last_name#57, c_preferred_cust_flag#58, c_birth_country#59, c_login#60, c_email_address#61, d_year#70, sum#73, isEmpty#74] +Keys [8]: [c_customer_id#55, c_first_name#56, c_last_name#57, c_preferred_cust_flag#58, c_birth_country#59, c_login#60, c_email_address#61, d_year#70] +Functions [1]: [sum(((((cs_ext_list_price#66 - cs_ext_wholesale_cost#65) - cs_ext_discount_amt#63) + cs_ext_sales_price#64) / 2))] +Aggregate Attributes [1]: [sum(((((cs_ext_list_price#66 - cs_ext_wholesale_cost#65) - cs_ext_discount_amt#63) + cs_ext_sales_price#64) / 2))#75] +Results [2]: [c_customer_id#55 AS customer_id#76, sum(((((cs_ext_list_price#66 - cs_ext_wholesale_cost#65) - cs_ext_discount_amt#63) + cs_ext_sales_price#64) / 2))#75 AS year_total#77] + +(49) Filter [codegen id : 11] +Input [2]: [customer_id#76, year_total#77] +Condition : (isnotnull(year_total#77) AND (year_total#77 > 0.000000)) + +(50) BroadcastExchange +Input [2]: [customer_id#76, year_total#77] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=8] + +(51) BroadcastHashJoin [codegen id : 24] +Left keys [1]: [customer_id#23] +Right keys [1]: [customer_id#76] +Join type: Inner +Join condition: None + +(52) Project [codegen id : 24] +Output [11]: [customer_id#23, year_total#24, customer_id#46, customer_first_name#47, customer_last_name#48, customer_preferred_cust_flag#49, customer_birth_country#50, customer_login#51, customer_email_address#52, year_total#53, year_total#77] +Input [12]: [customer_id#23, year_total#24, customer_id#46, customer_first_name#47, customer_last_name#48, customer_preferred_cust_flag#49, customer_birth_country#50, customer_login#51, customer_email_address#52, year_total#53, customer_id#76, year_total#77] + +(53) Scan parquet spark_catalog.default.customer +Output [8]: [c_customer_sk#78, c_customer_id#79, c_first_name#80, c_last_name#81, c_preferred_cust_flag#82, c_birth_country#83, c_login#84, c_email_address#85] +Batched: true +Location [not included in comparison]/{warehouse_dir}/customer] +PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_customer_id)] +ReadSchema: struct<c_customer_sk:int,c_customer_id:string,c_first_name:string,c_last_name:string,c_preferred_cust_flag:string,c_birth_country:string,c_login:string,c_email_address:string> + +(54) CometFilter +Input [8]: [c_customer_sk#78, c_customer_id#79, c_first_name#80, c_last_name#81, c_preferred_cust_flag#82, c_birth_country#83, c_login#84, c_email_address#85] +Condition : (isnotnull(c_customer_sk#78) AND isnotnull(c_customer_id#79)) + +(55) ColumnarToRow [codegen id : 14] +Input [8]: [c_customer_sk#78, c_customer_id#79, c_first_name#80, c_last_name#81, c_preferred_cust_flag#82, c_birth_country#83, c_login#84, c_email_address#85] + +(56) Scan parquet spark_catalog.default.catalog_sales +Output [6]: [cs_bill_customer_sk#86, cs_ext_discount_amt#87, cs_ext_sales_price#88, cs_ext_wholesale_cost#89, cs_ext_list_price#90, cs_sold_date_sk#91] +Batched: true +Location: InMemoryFileIndex [] +PartitionFilters: [isnotnull(cs_sold_date_sk#91), dynamicpruningexpression(cs_sold_date_sk#91 IN dynamicpruning#92)] +PushedFilters: [IsNotNull(cs_bill_customer_sk)] +ReadSchema: struct<cs_bill_customer_sk:int,cs_ext_discount_amt:decimal(7,2),cs_ext_sales_price:decimal(7,2),cs_ext_wholesale_cost:decimal(7,2),cs_ext_list_price:decimal(7,2)> + +(57) CometFilter +Input [6]: [cs_bill_customer_sk#86, cs_ext_discount_amt#87, cs_ext_sales_price#88, cs_ext_wholesale_cost#89, cs_ext_list_price#90, cs_sold_date_sk#91] +Condition : isnotnull(cs_bill_customer_sk#86) + +(58) ColumnarToRow [codegen id : 12] +Input [6]: [cs_bill_customer_sk#86, cs_ext_discount_amt#87, cs_ext_sales_price#88, cs_ext_wholesale_cost#89, cs_ext_list_price#90, cs_sold_date_sk#91] + +(59) BroadcastExchange +Input [6]: [cs_bill_customer_sk#86, cs_ext_discount_amt#87, cs_ext_sales_price#88, cs_ext_wholesale_cost#89, cs_ext_list_price#90, cs_sold_date_sk#91] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=9] + +(60) BroadcastHashJoin [codegen id : 14] +Left keys [1]: [c_customer_sk#78] +Right keys [1]: [cs_bill_customer_sk#86] +Join type: Inner +Join condition: None + +(61) Project [codegen id : 14] +Output [12]: [c_customer_id#79, c_first_name#80, c_last_name#81, c_preferred_cust_flag#82, c_birth_country#83, c_login#84, c_email_address#85, cs_ext_discount_amt#87, cs_ext_sales_price#88, cs_ext_wholesale_cost#89, cs_ext_list_price#90, cs_sold_date_sk#91] +Input [14]: [c_customer_sk#78, c_customer_id#79, c_first_name#80, c_last_name#81, c_preferred_cust_flag#82, c_birth_country#83, c_login#84, c_email_address#85, cs_bill_customer_sk#86, cs_ext_discount_amt#87, cs_ext_sales_price#88, cs_ext_wholesale_cost#89, cs_ext_list_price#90, cs_sold_date_sk#91] + +(62) ReusedExchange [Reuses operator id: 116] +Output [2]: [d_date_sk#93, d_year#94] + +(63) BroadcastHashJoin [codegen id : 14] +Left keys [1]: [cs_sold_date_sk#91] +Right keys [1]: [d_date_sk#93] +Join type: Inner +Join condition: None + +(64) Project [codegen id : 14] +Output [12]: [c_customer_id#79, c_first_name#80, c_last_name#81, c_preferred_cust_flag#82, c_birth_country#83, c_login#84, c_email_address#85, cs_ext_discount_amt#87, cs_ext_sales_price#88, cs_ext_wholesale_cost#89, cs_ext_list_price#90, d_year#94] +Input [14]: [c_customer_id#79, c_first_name#80, c_last_name#81, c_preferred_cust_flag#82, c_birth_country#83, c_login#84, c_email_address#85, cs_ext_discount_amt#87, cs_ext_sales_price#88, cs_ext_wholesale_cost#89, cs_ext_list_price#90, cs_sold_date_sk#91, d_date_sk#93, d_year#94] + +(65) HashAggregate [codegen id : 14] +Input [12]: [c_customer_id#79, c_first_name#80, c_last_name#81, c_preferred_cust_flag#82, c_birth_country#83, c_login#84, c_email_address#85, cs_ext_discount_amt#87, cs_ext_sales_price#88, cs_ext_wholesale_cost#89, cs_ext_list_price#90, d_year#94] +Keys [8]: [c_customer_id#79, c_first_name#80, c_last_name#81, c_preferred_cust_flag#82, c_birth_country#83, c_login#84, c_email_address#85, d_year#94] +Functions [1]: [partial_sum(((((cs_ext_list_price#90 - cs_ext_wholesale_cost#89) - cs_ext_discount_amt#87) + cs_ext_sales_price#88) / 2))] +Aggregate Attributes [2]: [sum#95, isEmpty#96] +Results [10]: [c_customer_id#79, c_first_name#80, c_last_name#81, c_preferred_cust_flag#82, c_birth_country#83, c_login#84, c_email_address#85, d_year#94, sum#97, isEmpty#98] + +(66) Exchange +Input [10]: [c_customer_id#79, c_first_name#80, c_last_name#81, c_preferred_cust_flag#82, c_birth_country#83, c_login#84, c_email_address#85, d_year#94, sum#97, isEmpty#98] +Arguments: hashpartitioning(c_customer_id#79, c_first_name#80, c_last_name#81, c_preferred_cust_flag#82, c_birth_country#83, c_login#84, c_email_address#85, d_year#94, 5), ENSURE_REQUIREMENTS, [plan_id=10] + +(67) HashAggregate [codegen id : 15] +Input [10]: [c_customer_id#79, c_first_name#80, c_last_name#81, c_preferred_cust_flag#82, c_birth_country#83, c_login#84, c_email_address#85, d_year#94, sum#97, isEmpty#98] +Keys [8]: [c_customer_id#79, c_first_name#80, c_last_name#81, c_preferred_cust_flag#82, c_birth_country#83, c_login#84, c_email_address#85, d_year#94] +Functions [1]: [sum(((((cs_ext_list_price#90 - cs_ext_wholesale_cost#89) - cs_ext_discount_amt#87) + cs_ext_sales_price#88) / 2))] +Aggregate Attributes [1]: [sum(((((cs_ext_list_price#90 - cs_ext_wholesale_cost#89) - cs_ext_discount_amt#87) + cs_ext_sales_price#88) / 2))#75] +Results [2]: [c_customer_id#79 AS customer_id#99, sum(((((cs_ext_list_price#90 - cs_ext_wholesale_cost#89) - cs_ext_discount_amt#87) + cs_ext_sales_price#88) / 2))#75 AS year_total#100] + +(68) BroadcastExchange +Input [2]: [customer_id#99, year_total#100] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=11] + +(69) BroadcastHashJoin [codegen id : 24] +Left keys [1]: [customer_id#23] +Right keys [1]: [customer_id#99] +Join type: Inner +Join condition: (CASE WHEN (year_total#77 > 0.000000) THEN (year_total#100 / year_total#77) END > CASE WHEN (year_total#24 > 0.000000) THEN (year_total#53 / year_total#24) END) + +(70) Project [codegen id : 24] +Output [10]: [customer_id#23, customer_id#46, customer_first_name#47, customer_last_name#48, customer_preferred_cust_flag#49, customer_birth_country#50, customer_login#51, customer_email_address#52, year_total#77, year_total#100] +Input [13]: [customer_id#23, year_total#24, customer_id#46, customer_first_name#47, customer_last_name#48, customer_preferred_cust_flag#49, customer_birth_country#50, customer_login#51, customer_email_address#52, year_total#53, year_total#77, customer_id#99, year_total#100] + +(71) Scan parquet spark_catalog.default.customer +Output [8]: [c_customer_sk#101, c_customer_id#102, c_first_name#103, c_last_name#104, c_preferred_cust_flag#105, c_birth_country#106, c_login#107, c_email_address#108] +Batched: true +Location [not included in comparison]/{warehouse_dir}/customer] +PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_customer_id)] +ReadSchema: struct<c_customer_sk:int,c_customer_id:string,c_first_name:string,c_last_name:string,c_preferred_cust_flag:string,c_birth_country:string,c_login:string,c_email_address:string> + +(72) CometFilter +Input [8]: [c_customer_sk#101, c_customer_id#102, c_first_name#103, c_last_name#104, c_preferred_cust_flag#105, c_birth_country#106, c_login#107, c_email_address#108] +Condition : (isnotnull(c_customer_sk#101) AND isnotnull(c_customer_id#102)) + +(73) ColumnarToRow [codegen id : 18] +Input [8]: [c_customer_sk#101, c_customer_id#102, c_first_name#103, c_last_name#104, c_preferred_cust_flag#105, c_birth_country#106, c_login#107, c_email_address#108] + +(74) Scan parquet spark_catalog.default.web_sales +Output [6]: [ws_bill_customer_sk#109, ws_ext_discount_amt#110, ws_ext_sales_price#111, ws_ext_wholesale_cost#112, ws_ext_list_price#113, ws_sold_date_sk#114] +Batched: true +Location: InMemoryFileIndex [] +PartitionFilters: [isnotnull(ws_sold_date_sk#114), dynamicpruningexpression(ws_sold_date_sk#114 IN dynamicpruning#115)] +PushedFilters: [IsNotNull(ws_bill_customer_sk)] +ReadSchema: struct<ws_bill_customer_sk:int,ws_ext_discount_amt:decimal(7,2),ws_ext_sales_price:decimal(7,2),ws_ext_wholesale_cost:decimal(7,2),ws_ext_list_price:decimal(7,2)> + +(75) CometFilter +Input [6]: [ws_bill_customer_sk#109, ws_ext_discount_amt#110, ws_ext_sales_price#111, ws_ext_wholesale_cost#112, ws_ext_list_price#113, ws_sold_date_sk#114] +Condition : isnotnull(ws_bill_customer_sk#109) + +(76) ColumnarToRow [codegen id : 16] +Input [6]: [ws_bill_customer_sk#109, ws_ext_discount_amt#110, ws_ext_sales_price#111, ws_ext_wholesale_cost#112, ws_ext_list_price#113, ws_sold_date_sk#114] + +(77) BroadcastExchange +Input [6]: [ws_bill_customer_sk#109, ws_ext_discount_amt#110, ws_ext_sales_price#111, ws_ext_wholesale_cost#112, ws_ext_list_price#113, ws_sold_date_sk#114] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=12] + +(78) BroadcastHashJoin [codegen id : 18] +Left keys [1]: [c_customer_sk#101] +Right keys [1]: [ws_bill_customer_sk#109] +Join type: Inner +Join condition: None + +(79) Project [codegen id : 18] +Output [12]: [c_customer_id#102, c_first_name#103, c_last_name#104, c_preferred_cust_flag#105, c_birth_country#106, c_login#107, c_email_address#108, ws_ext_discount_amt#110, ws_ext_sales_price#111, ws_ext_wholesale_cost#112, ws_ext_list_price#113, ws_sold_date_sk#114] +Input [14]: [c_customer_sk#101, c_customer_id#102, c_first_name#103, c_last_name#104, c_preferred_cust_flag#105, c_birth_country#106, c_login#107, c_email_address#108, ws_bill_customer_sk#109, ws_ext_discount_amt#110, ws_ext_sales_price#111, ws_ext_wholesale_cost#112, ws_ext_list_price#113, ws_sold_date_sk#114] + +(80) ReusedExchange [Reuses operator id: 112] +Output [2]: [d_date_sk#116, d_year#117] + +(81) BroadcastHashJoin [codegen id : 18] +Left keys [1]: [ws_sold_date_sk#114] +Right keys [1]: [d_date_sk#116] +Join type: Inner +Join condition: None + +(82) Project [codegen id : 18] +Output [12]: [c_customer_id#102, c_first_name#103, c_last_name#104, c_preferred_cust_flag#105, c_birth_country#106, c_login#107, c_email_address#108, ws_ext_discount_amt#110, ws_ext_sales_price#111, ws_ext_wholesale_cost#112, ws_ext_list_price#113, d_year#117] +Input [14]: [c_customer_id#102, c_first_name#103, c_last_name#104, c_preferred_cust_flag#105, c_birth_country#106, c_login#107, c_email_address#108, ws_ext_discount_amt#110, ws_ext_sales_price#111, ws_ext_wholesale_cost#112, ws_ext_list_price#113, ws_sold_date_sk#114, d_date_sk#116, d_year#117] + +(83) HashAggregate [codegen id : 18] +Input [12]: [c_customer_id#102, c_first_name#103, c_last_name#104, c_preferred_cust_flag#105, c_birth_country#106, c_login#107, c_email_address#108, ws_ext_discount_amt#110, ws_ext_sales_price#111, ws_ext_wholesale_cost#112, ws_ext_list_price#113, d_year#117] +Keys [8]: [c_customer_id#102, c_first_name#103, c_last_name#104, c_preferred_cust_flag#105, c_birth_country#106, c_login#107, c_email_address#108, d_year#117] +Functions [1]: [partial_sum(((((ws_ext_list_price#113 - ws_ext_wholesale_cost#112) - ws_ext_discount_amt#110) + ws_ext_sales_price#111) / 2))] +Aggregate Attributes [2]: [sum#118, isEmpty#119] +Results [10]: [c_customer_id#102, c_first_name#103, c_last_name#104, c_preferred_cust_flag#105, c_birth_country#106, c_login#107, c_email_address#108, d_year#117, sum#120, isEmpty#121] + +(84) Exchange +Input [10]: [c_customer_id#102, c_first_name#103, c_last_name#104, c_preferred_cust_flag#105, c_birth_country#106, c_login#107, c_email_address#108, d_year#117, sum#120, isEmpty#121] +Arguments: hashpartitioning(c_customer_id#102, c_first_name#103, c_last_name#104, c_preferred_cust_flag#105, c_birth_country#106, c_login#107, c_email_address#108, d_year#117, 5), ENSURE_REQUIREMENTS, [plan_id=13] + +(85) HashAggregate [codegen id : 19] +Input [10]: [c_customer_id#102, c_first_name#103, c_last_name#104, c_preferred_cust_flag#105, c_birth_country#106, c_login#107, c_email_address#108, d_year#117, sum#120, isEmpty#121] +Keys [8]: [c_customer_id#102, c_first_name#103, c_last_name#104, c_preferred_cust_flag#105, c_birth_country#106, c_login#107, c_email_address#108, d_year#117] +Functions [1]: [sum(((((ws_ext_list_price#113 - ws_ext_wholesale_cost#112) - ws_ext_discount_amt#110) + ws_ext_sales_price#111) / 2))] +Aggregate Attributes [1]: [sum(((((ws_ext_list_price#113 - ws_ext_wholesale_cost#112) - ws_ext_discount_amt#110) + ws_ext_sales_price#111) / 2))#122] +Results [2]: [c_customer_id#102 AS customer_id#123, sum(((((ws_ext_list_price#113 - ws_ext_wholesale_cost#112) - ws_ext_discount_amt#110) + ws_ext_sales_price#111) / 2))#122 AS year_total#124] + +(86) Filter [codegen id : 19] +Input [2]: [customer_id#123, year_total#124] +Condition : (isnotnull(year_total#124) AND (year_total#124 > 0.000000)) + +(87) BroadcastExchange +Input [2]: [customer_id#123, year_total#124] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=14] + +(88) BroadcastHashJoin [codegen id : 24] +Left keys [1]: [customer_id#23] +Right keys [1]: [customer_id#123] +Join type: Inner +Join condition: None + +(89) Project [codegen id : 24] +Output [11]: [customer_id#23, customer_id#46, customer_first_name#47, customer_last_name#48, customer_preferred_cust_flag#49, customer_birth_country#50, customer_login#51, customer_email_address#52, year_total#77, year_total#100, year_total#124] +Input [12]: [customer_id#23, customer_id#46, customer_first_name#47, customer_last_name#48, customer_preferred_cust_flag#49, customer_birth_country#50, customer_login#51, customer_email_address#52, year_total#77, year_total#100, customer_id#123, year_total#124] + +(90) Scan parquet spark_catalog.default.customer +Output [8]: [c_customer_sk#125, c_customer_id#126, c_first_name#127, c_last_name#128, c_preferred_cust_flag#129, c_birth_country#130, c_login#131, c_email_address#132] +Batched: true +Location [not included in comparison]/{warehouse_dir}/customer] +PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_customer_id)] +ReadSchema: struct<c_customer_sk:int,c_customer_id:string,c_first_name:string,c_last_name:string,c_preferred_cust_flag:string,c_birth_country:string,c_login:string,c_email_address:string> + +(91) CometFilter +Input [8]: [c_customer_sk#125, c_customer_id#126, c_first_name#127, c_last_name#128, c_preferred_cust_flag#129, c_birth_country#130, c_login#131, c_email_address#132] +Condition : (isnotnull(c_customer_sk#125) AND isnotnull(c_customer_id#126)) + +(92) ColumnarToRow [codegen id : 22] +Input [8]: [c_customer_sk#125, c_customer_id#126, c_first_name#127, c_last_name#128, c_preferred_cust_flag#129, c_birth_country#130, c_login#131, c_email_address#132] + +(93) Scan parquet spark_catalog.default.web_sales +Output [6]: [ws_bill_customer_sk#133, ws_ext_discount_amt#134, ws_ext_sales_price#135, ws_ext_wholesale_cost#136, ws_ext_list_price#137, ws_sold_date_sk#138] +Batched: true +Location: InMemoryFileIndex [] +PartitionFilters: [isnotnull(ws_sold_date_sk#138), dynamicpruningexpression(ws_sold_date_sk#138 IN dynamicpruning#139)] +PushedFilters: [IsNotNull(ws_bill_customer_sk)] +ReadSchema: struct<ws_bill_customer_sk:int,ws_ext_discount_amt:decimal(7,2),ws_ext_sales_price:decimal(7,2),ws_ext_wholesale_cost:decimal(7,2),ws_ext_list_price:decimal(7,2)> + +(94) CometFilter +Input [6]: [ws_bill_customer_sk#133, ws_ext_discount_amt#134, ws_ext_sales_price#135, ws_ext_wholesale_cost#136, ws_ext_list_price#137, ws_sold_date_sk#138] +Condition : isnotnull(ws_bill_customer_sk#133) + +(95) ColumnarToRow [codegen id : 20] +Input [6]: [ws_bill_customer_sk#133, ws_ext_discount_amt#134, ws_ext_sales_price#135, ws_ext_wholesale_cost#136, ws_ext_list_price#137, ws_sold_date_sk#138] + +(96) BroadcastExchange +Input [6]: [ws_bill_customer_sk#133, ws_ext_discount_amt#134, ws_ext_sales_price#135, ws_ext_wholesale_cost#136, ws_ext_list_price#137, ws_sold_date_sk#138] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=15] + +(97) BroadcastHashJoin [codegen id : 22] +Left keys [1]: [c_customer_sk#125] +Right keys [1]: [ws_bill_customer_sk#133] +Join type: Inner +Join condition: None + +(98) Project [codegen id : 22] +Output [12]: [c_customer_id#126, c_first_name#127, c_last_name#128, c_preferred_cust_flag#129, c_birth_country#130, c_login#131, c_email_address#132, ws_ext_discount_amt#134, ws_ext_sales_price#135, ws_ext_wholesale_cost#136, ws_ext_list_price#137, ws_sold_date_sk#138] +Input [14]: [c_customer_sk#125, c_customer_id#126, c_first_name#127, c_last_name#128, c_preferred_cust_flag#129, c_birth_country#130, c_login#131, c_email_address#132, ws_bill_customer_sk#133, ws_ext_discount_amt#134, ws_ext_sales_price#135, ws_ext_wholesale_cost#136, ws_ext_list_price#137, ws_sold_date_sk#138] + +(99) ReusedExchange [Reuses operator id: 116] +Output [2]: [d_date_sk#140, d_year#141] + +(100) BroadcastHashJoin [codegen id : 22] +Left keys [1]: [ws_sold_date_sk#138] +Right keys [1]: [d_date_sk#140] +Join type: Inner +Join condition: None + +(101) Project [codegen id : 22] +Output [12]: [c_customer_id#126, c_first_name#127, c_last_name#128, c_preferred_cust_flag#129, c_birth_country#130, c_login#131, c_email_address#132, ws_ext_discount_amt#134, ws_ext_sales_price#135, ws_ext_wholesale_cost#136, ws_ext_list_price#137, d_year#141] +Input [14]: [c_customer_id#126, c_first_name#127, c_last_name#128, c_preferred_cust_flag#129, c_birth_country#130, c_login#131, c_email_address#132, ws_ext_discount_amt#134, ws_ext_sales_price#135, ws_ext_wholesale_cost#136, ws_ext_list_price#137, ws_sold_date_sk#138, d_date_sk#140, d_year#141] + +(102) HashAggregate [codegen id : 22] +Input [12]: [c_customer_id#126, c_first_name#127, c_last_name#128, c_preferred_cust_flag#129, c_birth_country#130, c_login#131, c_email_address#132, ws_ext_discount_amt#134, ws_ext_sales_price#135, ws_ext_wholesale_cost#136, ws_ext_list_price#137, d_year#141] +Keys [8]: [c_customer_id#126, c_first_name#127, c_last_name#128, c_preferred_cust_flag#129, c_birth_country#130, c_login#131, c_email_address#132, d_year#141] +Functions [1]: [partial_sum(((((ws_ext_list_price#137 - ws_ext_wholesale_cost#136) - ws_ext_discount_amt#134) + ws_ext_sales_price#135) / 2))] +Aggregate Attributes [2]: [sum#142, isEmpty#143] +Results [10]: [c_customer_id#126, c_first_name#127, c_last_name#128, c_preferred_cust_flag#129, c_birth_country#130, c_login#131, c_email_address#132, d_year#141, sum#144, isEmpty#145] + +(103) Exchange +Input [10]: [c_customer_id#126, c_first_name#127, c_last_name#128, c_preferred_cust_flag#129, c_birth_country#130, c_login#131, c_email_address#132, d_year#141, sum#144, isEmpty#145] +Arguments: hashpartitioning(c_customer_id#126, c_first_name#127, c_last_name#128, c_preferred_cust_flag#129, c_birth_country#130, c_login#131, c_email_address#132, d_year#141, 5), ENSURE_REQUIREMENTS, [plan_id=16] + +(104) HashAggregate [codegen id : 23] +Input [10]: [c_customer_id#126, c_first_name#127, c_last_name#128, c_preferred_cust_flag#129, c_birth_country#130, c_login#131, c_email_address#132, d_year#141, sum#144, isEmpty#145] +Keys [8]: [c_customer_id#126, c_first_name#127, c_last_name#128, c_preferred_cust_flag#129, c_birth_country#130, c_login#131, c_email_address#132, d_year#141] +Functions [1]: [sum(((((ws_ext_list_price#137 - ws_ext_wholesale_cost#136) - ws_ext_discount_amt#134) + ws_ext_sales_price#135) / 2))] +Aggregate Attributes [1]: [sum(((((ws_ext_list_price#137 - ws_ext_wholesale_cost#136) - ws_ext_discount_amt#134) + ws_ext_sales_price#135) / 2))#122] +Results [2]: [c_customer_id#126 AS customer_id#146, sum(((((ws_ext_list_price#137 - ws_ext_wholesale_cost#136) - ws_ext_discount_amt#134) + ws_ext_sales_price#135) / 2))#122 AS year_total#147] + +(105) BroadcastExchange +Input [2]: [customer_id#146, year_total#147] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=17] + +(106) BroadcastHashJoin [codegen id : 24] +Left keys [1]: [customer_id#23] +Right keys [1]: [customer_id#146] +Join type: Inner +Join condition: (CASE WHEN (year_total#77 > 0.000000) THEN (year_total#100 / year_total#77) END > CASE WHEN (year_total#124 > 0.000000) THEN (year_total#147 / year_total#124) END) + +(107) Project [codegen id : 24] +Output [7]: [customer_id#46, customer_first_name#47, customer_last_name#48, customer_preferred_cust_flag#49, customer_birth_country#50, customer_login#51, customer_email_address#52] +Input [13]: [customer_id#23, customer_id#46, customer_first_name#47, customer_last_name#48, customer_preferred_cust_flag#49, customer_birth_country#50, customer_login#51, customer_email_address#52, year_total#77, year_total#100, year_total#124, customer_id#146, year_total#147] + +(108) TakeOrderedAndProject +Input [7]: [customer_id#46, customer_first_name#47, customer_last_name#48, customer_preferred_cust_flag#49, customer_birth_country#50, customer_login#51, customer_email_address#52] +Arguments: 100, [customer_id#46 ASC NULLS FIRST, customer_first_name#47 ASC NULLS FIRST, customer_last_name#48 ASC NULLS FIRST, customer_preferred_cust_flag#49 ASC NULLS FIRST, customer_birth_country#50 ASC NULLS FIRST, customer_login#51 ASC NULLS FIRST, customer_email_address#52 ASC NULLS FIRST], [customer_id#46, customer_first_name#47, customer_last_name#48, customer_preferred_cust_flag#49, customer_birth_country#50, customer_login#51, customer_email_address#52] + +===== Subqueries ===== + +Subquery:1 Hosting operator id = 4 Hosting Expression = ss_sold_date_sk#14 IN dynamicpruning#15 +BroadcastExchange (112) ++- * ColumnarToRow (111) + +- CometFilter (110) + +- CometScan parquet spark_catalog.default.date_dim (109) + + +(109) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#16, d_year#17] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), IsNotNull(d_date_sk)] +ReadSchema: struct<d_date_sk:int,d_year:int> + +(110) CometFilter +Input [2]: [d_date_sk#16, d_year#17] +Condition : ((isnotnull(d_year#17) AND (d_year#17 = 2001)) AND isnotnull(d_date_sk#16)) + +(111) ColumnarToRow [codegen id : 1] +Input [2]: [d_date_sk#16, d_year#17] + +(112) BroadcastExchange +Input [2]: [d_date_sk#16, d_year#17] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=18] + +Subquery:2 Hosting operator id = 20 Hosting Expression = ss_sold_date_sk#38 IN dynamicpruning#39 +BroadcastExchange (116) ++- * ColumnarToRow (115) + +- CometFilter (114) + +- CometScan parquet spark_catalog.default.date_dim (113) + + +(113) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#40, d_year#41] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2002), IsNotNull(d_date_sk)] +ReadSchema: struct<d_date_sk:int,d_year:int> + +(114) CometFilter +Input [2]: [d_date_sk#40, d_year#41] +Condition : ((isnotnull(d_year#41) AND (d_year#41 = 2002)) AND isnotnull(d_date_sk#40)) + +(115) ColumnarToRow [codegen id : 1] +Input [2]: [d_date_sk#40, d_year#41] + +(116) BroadcastExchange +Input [2]: [d_date_sk#40, d_year#41] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=19] + +Subquery:3 Hosting operator id = 37 Hosting Expression = cs_sold_date_sk#67 IN dynamicpruning#15 + +Subquery:4 Hosting operator id = 56 Hosting Expression = cs_sold_date_sk#91 IN dynamicpruning#39 + +Subquery:5 Hosting operator id = 74 Hosting Expression = ws_sold_date_sk#114 IN dynamicpruning#15 + +Subquery:6 Hosting operator id = 93 Hosting Expression = ws_sold_date_sk#138 IN dynamicpruning#39 + + diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q4/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q4/simplified.txt new file mode 100644 index 000000000..99e255a0e --- /dev/null +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q4/simplified.txt @@ -0,0 +1,179 @@ +TakeOrderedAndProject [customer_id,customer_first_name,customer_last_name,customer_preferred_cust_flag,customer_birth_country,customer_login,customer_email_address] + WholeStageCodegen (24) + Project [customer_id,customer_first_name,customer_last_name,customer_preferred_cust_flag,customer_birth_country,customer_login,customer_email_address] + BroadcastHashJoin [customer_id,customer_id,year_total,year_total,year_total,year_total] + Project [customer_id,customer_id,customer_first_name,customer_last_name,customer_preferred_cust_flag,customer_birth_country,customer_login,customer_email_address,year_total,year_total,year_total] + BroadcastHashJoin [customer_id,customer_id] + Project [customer_id,customer_id,customer_first_name,customer_last_name,customer_preferred_cust_flag,customer_birth_country,customer_login,customer_email_address,year_total,year_total] + BroadcastHashJoin [customer_id,customer_id,year_total,year_total,year_total,year_total] + Project [customer_id,year_total,customer_id,customer_first_name,customer_last_name,customer_preferred_cust_flag,customer_birth_country,customer_login,customer_email_address,year_total,year_total] + BroadcastHashJoin [customer_id,customer_id] + BroadcastHashJoin [customer_id,customer_id] + Filter [year_total] + HashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum,isEmpty] [sum(((((ss_ext_list_price - ss_ext_wholesale_cost) - ss_ext_discount_amt) + ss_ext_sales_price) / 2)),customer_id,year_total,sum,isEmpty] + InputAdapter + Exchange [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year] #1 + WholeStageCodegen (3) + HashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,ss_ext_list_price,ss_ext_wholesale_cost,ss_ext_discount_amt,ss_ext_sales_price] [sum,isEmpty,sum,isEmpty] + Project [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_discount_amt,ss_ext_sales_price,ss_ext_wholesale_cost,ss_ext_list_price,d_year] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_discount_amt,ss_ext_sales_price,ss_ext_wholesale_cost,ss_ext_list_price,ss_sold_date_sk] + BroadcastHashJoin [c_customer_sk,ss_customer_sk] + ColumnarToRow + InputAdapter + CometFilter [c_customer_sk,c_customer_id] + CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] + InputAdapter + BroadcastExchange #2 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometFilter [ss_customer_sk] + CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_ext_discount_amt,ss_ext_sales_price,ss_ext_wholesale_cost,ss_ext_list_price,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometFilter [d_year,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + InputAdapter + ReusedExchange [d_date_sk,d_year] #3 + InputAdapter + BroadcastExchange #4 + WholeStageCodegen (7) + HashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum,isEmpty] [sum(((((ss_ext_list_price - ss_ext_wholesale_cost) - ss_ext_discount_amt) + ss_ext_sales_price) / 2)),customer_id,customer_first_name,customer_last_name,customer_preferred_cust_flag,customer_birth_country,customer_login,customer_email_address,year_total,sum,isEmpty] + InputAdapter + Exchange [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year] #5 + WholeStageCodegen (6) + HashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,ss_ext_list_price,ss_ext_wholesale_cost,ss_ext_discount_amt,ss_ext_sales_price] [sum,isEmpty,sum,isEmpty] + Project [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_discount_amt,ss_ext_sales_price,ss_ext_wholesale_cost,ss_ext_list_price,d_year] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_discount_amt,ss_ext_sales_price,ss_ext_wholesale_cost,ss_ext_list_price,ss_sold_date_sk] + BroadcastHashJoin [c_customer_sk,ss_customer_sk] + ColumnarToRow + InputAdapter + CometFilter [c_customer_sk,c_customer_id] + CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] + InputAdapter + BroadcastExchange #6 + WholeStageCodegen (4) + ColumnarToRow + InputAdapter + CometFilter [ss_customer_sk] + CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_ext_discount_amt,ss_ext_sales_price,ss_ext_wholesale_cost,ss_ext_list_price,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #2 + BroadcastExchange #7 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometFilter [d_year,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + InputAdapter + ReusedExchange [d_date_sk,d_year] #7 + InputAdapter + BroadcastExchange #8 + WholeStageCodegen (11) + Filter [year_total] + HashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum,isEmpty] [sum(((((cs_ext_list_price - cs_ext_wholesale_cost) - cs_ext_discount_amt) + cs_ext_sales_price) / 2)),customer_id,year_total,sum,isEmpty] + InputAdapter + Exchange [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year] #9 + WholeStageCodegen (10) + HashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,cs_ext_list_price,cs_ext_wholesale_cost,cs_ext_discount_amt,cs_ext_sales_price] [sum,isEmpty,sum,isEmpty] + Project [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,cs_ext_discount_amt,cs_ext_sales_price,cs_ext_wholesale_cost,cs_ext_list_price,d_year] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + Project [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,cs_ext_discount_amt,cs_ext_sales_price,cs_ext_wholesale_cost,cs_ext_list_price,cs_sold_date_sk] + BroadcastHashJoin [c_customer_sk,cs_bill_customer_sk] + ColumnarToRow + InputAdapter + CometFilter [c_customer_sk,c_customer_id] + CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] + InputAdapter + BroadcastExchange #10 + WholeStageCodegen (8) + ColumnarToRow + InputAdapter + CometFilter [cs_bill_customer_sk] + CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_ext_discount_amt,cs_ext_sales_price,cs_ext_wholesale_cost,cs_ext_list_price,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + InputAdapter + ReusedExchange [d_date_sk,d_year] #3 + InputAdapter + BroadcastExchange #11 + WholeStageCodegen (15) + HashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum,isEmpty] [sum(((((cs_ext_list_price - cs_ext_wholesale_cost) - cs_ext_discount_amt) + cs_ext_sales_price) / 2)),customer_id,year_total,sum,isEmpty] + InputAdapter + Exchange [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year] #12 + WholeStageCodegen (14) + HashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,cs_ext_list_price,cs_ext_wholesale_cost,cs_ext_discount_amt,cs_ext_sales_price] [sum,isEmpty,sum,isEmpty] + Project [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,cs_ext_discount_amt,cs_ext_sales_price,cs_ext_wholesale_cost,cs_ext_list_price,d_year] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + Project [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,cs_ext_discount_amt,cs_ext_sales_price,cs_ext_wholesale_cost,cs_ext_list_price,cs_sold_date_sk] + BroadcastHashJoin [c_customer_sk,cs_bill_customer_sk] + ColumnarToRow + InputAdapter + CometFilter [c_customer_sk,c_customer_id] + CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] + InputAdapter + BroadcastExchange #13 + WholeStageCodegen (12) + ColumnarToRow + InputAdapter + CometFilter [cs_bill_customer_sk] + CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_ext_discount_amt,cs_ext_sales_price,cs_ext_wholesale_cost,cs_ext_list_price,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #2 + InputAdapter + ReusedExchange [d_date_sk,d_year] #7 + InputAdapter + BroadcastExchange #14 + WholeStageCodegen (19) + Filter [year_total] + HashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum,isEmpty] [sum(((((ws_ext_list_price - ws_ext_wholesale_cost) - ws_ext_discount_amt) + ws_ext_sales_price) / 2)),customer_id,year_total,sum,isEmpty] + InputAdapter + Exchange [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year] #15 + WholeStageCodegen (18) + HashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,ws_ext_list_price,ws_ext_wholesale_cost,ws_ext_discount_amt,ws_ext_sales_price] [sum,isEmpty,sum,isEmpty] + Project [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_sales_price,ws_ext_wholesale_cost,ws_ext_list_price,d_year] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + Project [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_sales_price,ws_ext_wholesale_cost,ws_ext_list_price,ws_sold_date_sk] + BroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] + ColumnarToRow + InputAdapter + CometFilter [c_customer_sk,c_customer_id] + CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] + InputAdapter + BroadcastExchange #16 + WholeStageCodegen (16) + ColumnarToRow + InputAdapter + CometFilter [ws_bill_customer_sk] + CometScan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_sales_price,ws_ext_wholesale_cost,ws_ext_list_price,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + InputAdapter + ReusedExchange [d_date_sk,d_year] #3 + InputAdapter + BroadcastExchange #17 + WholeStageCodegen (23) + HashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum,isEmpty] [sum(((((ws_ext_list_price - ws_ext_wholesale_cost) - ws_ext_discount_amt) + ws_ext_sales_price) / 2)),customer_id,year_total,sum,isEmpty] + InputAdapter + Exchange [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year] #18 + WholeStageCodegen (22) + HashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,ws_ext_list_price,ws_ext_wholesale_cost,ws_ext_discount_amt,ws_ext_sales_price] [sum,isEmpty,sum,isEmpty] + Project [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_sales_price,ws_ext_wholesale_cost,ws_ext_list_price,d_year] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + Project [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_sales_price,ws_ext_wholesale_cost,ws_ext_list_price,ws_sold_date_sk] + BroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] + ColumnarToRow + InputAdapter + CometFilter [c_customer_sk,c_customer_id] + CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] + InputAdapter + BroadcastExchange #19 + WholeStageCodegen (20) + ColumnarToRow + InputAdapter + CometFilter [ws_bill_customer_sk] + CometScan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_sales_price,ws_ext_wholesale_cost,ws_ext_list_price,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #2 + InputAdapter + ReusedExchange [d_date_sk,d_year] #7 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 new file mode 100644 index 000000000..f63b94658 --- /dev/null +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q40/explain.txt @@ -0,0 +1,218 @@ +== Physical Plan == +TakeOrderedAndProject (33) ++- * HashAggregate (32) + +- Exchange (31) + +- * HashAggregate (30) + +- * Project (29) + +- * BroadcastHashJoin Inner BuildRight (28) + :- * Project (26) + : +- * BroadcastHashJoin Inner BuildRight (25) + : :- * Project (19) + : : +- * BroadcastHashJoin Inner BuildRight (18) + : : :- * Project (13) + : : : +- * SortMergeJoin LeftOuter (12) + : : : :- * Sort (5) + : : : : +- Exchange (4) + : : : : +- * ColumnarToRow (3) + : : : : +- CometFilter (2) + : : : : +- CometScan parquet spark_catalog.default.catalog_sales (1) + : : : +- * Sort (11) + : : : +- Exchange (10) + : : : +- * ColumnarToRow (9) + : : : +- CometProject (8) + : : : +- CometFilter (7) + : : : +- CometScan parquet spark_catalog.default.catalog_returns (6) + : : +- BroadcastExchange (17) + : : +- * ColumnarToRow (16) + : : +- CometFilter (15) + : : +- CometScan parquet spark_catalog.default.warehouse (14) + : +- BroadcastExchange (24) + : +- * ColumnarToRow (23) + : +- CometProject (22) + : +- CometFilter (21) + : +- CometScan parquet spark_catalog.default.item (20) + +- ReusedExchange (27) + + +(1) Scan parquet spark_catalog.default.catalog_sales +Output [5]: [cs_warehouse_sk#1, cs_item_sk#2, cs_order_number#3, cs_sales_price#4, cs_sold_date_sk#5] +Batched: true +Location: InMemoryFileIndex [] +PartitionFilters: [isnotnull(cs_sold_date_sk#5), dynamicpruningexpression(cs_sold_date_sk#5 IN dynamicpruning#6)] +PushedFilters: [IsNotNull(cs_warehouse_sk), IsNotNull(cs_item_sk)] +ReadSchema: struct<cs_warehouse_sk:int,cs_item_sk:int,cs_order_number:int,cs_sales_price:decimal(7,2)> + +(2) CometFilter +Input [5]: [cs_warehouse_sk#1, cs_item_sk#2, cs_order_number#3, cs_sales_price#4, cs_sold_date_sk#5] +Condition : (isnotnull(cs_warehouse_sk#1) AND isnotnull(cs_item_sk#2)) + +(3) ColumnarToRow [codegen id : 1] +Input [5]: [cs_warehouse_sk#1, cs_item_sk#2, cs_order_number#3, cs_sales_price#4, cs_sold_date_sk#5] + +(4) Exchange +Input [5]: [cs_warehouse_sk#1, cs_item_sk#2, cs_order_number#3, cs_sales_price#4, cs_sold_date_sk#5] +Arguments: hashpartitioning(cs_order_number#3, cs_item_sk#2, 5), ENSURE_REQUIREMENTS, [plan_id=1] + +(5) Sort [codegen id : 2] +Input [5]: [cs_warehouse_sk#1, cs_item_sk#2, cs_order_number#3, cs_sales_price#4, cs_sold_date_sk#5] +Arguments: [cs_order_number#3 ASC NULLS FIRST, cs_item_sk#2 ASC NULLS FIRST], false, 0 + +(6) Scan parquet spark_catalog.default.catalog_returns +Output [4]: [cr_item_sk#7, cr_order_number#8, cr_refunded_cash#9, cr_returned_date_sk#10] +Batched: true +Location [not included in comparison]/{warehouse_dir}/catalog_returns] +PushedFilters: [IsNotNull(cr_order_number), IsNotNull(cr_item_sk)] +ReadSchema: struct<cr_item_sk:int,cr_order_number:int,cr_refunded_cash:decimal(7,2)> + +(7) CometFilter +Input [4]: [cr_item_sk#7, cr_order_number#8, cr_refunded_cash#9, cr_returned_date_sk#10] +Condition : (isnotnull(cr_order_number#8) AND isnotnull(cr_item_sk#7)) + +(8) CometProject +Input [4]: [cr_item_sk#7, cr_order_number#8, cr_refunded_cash#9, cr_returned_date_sk#10] +Arguments: [cr_item_sk#7, cr_order_number#8, cr_refunded_cash#9], [cr_item_sk#7, cr_order_number#8, cr_refunded_cash#9] + +(9) ColumnarToRow [codegen id : 3] +Input [3]: [cr_item_sk#7, cr_order_number#8, cr_refunded_cash#9] + +(10) Exchange +Input [3]: [cr_item_sk#7, cr_order_number#8, cr_refunded_cash#9] +Arguments: hashpartitioning(cr_order_number#8, cr_item_sk#7, 5), ENSURE_REQUIREMENTS, [plan_id=2] + +(11) Sort [codegen id : 4] +Input [3]: [cr_item_sk#7, cr_order_number#8, cr_refunded_cash#9] +Arguments: [cr_order_number#8 ASC NULLS FIRST, cr_item_sk#7 ASC NULLS FIRST], false, 0 + +(12) SortMergeJoin [codegen id : 8] +Left keys [2]: [cs_order_number#3, cs_item_sk#2] +Right keys [2]: [cr_order_number#8, cr_item_sk#7] +Join type: LeftOuter +Join condition: None + +(13) Project [codegen id : 8] +Output [5]: [cs_warehouse_sk#1, cs_item_sk#2, cs_sales_price#4, cs_sold_date_sk#5, cr_refunded_cash#9] +Input [8]: [cs_warehouse_sk#1, cs_item_sk#2, cs_order_number#3, cs_sales_price#4, cs_sold_date_sk#5, cr_item_sk#7, cr_order_number#8, cr_refunded_cash#9] + +(14) Scan parquet spark_catalog.default.warehouse +Output [2]: [w_warehouse_sk#11, w_state#12] +Batched: true +Location [not included in comparison]/{warehouse_dir}/warehouse] +PushedFilters: [IsNotNull(w_warehouse_sk)] +ReadSchema: struct<w_warehouse_sk:int,w_state:string> + +(15) CometFilter +Input [2]: [w_warehouse_sk#11, w_state#12] +Condition : isnotnull(w_warehouse_sk#11) + +(16) ColumnarToRow [codegen id : 5] +Input [2]: [w_warehouse_sk#11, w_state#12] + +(17) BroadcastExchange +Input [2]: [w_warehouse_sk#11, w_state#12] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=3] + +(18) BroadcastHashJoin [codegen id : 8] +Left keys [1]: [cs_warehouse_sk#1] +Right keys [1]: [w_warehouse_sk#11] +Join type: Inner +Join condition: None + +(19) Project [codegen id : 8] +Output [5]: [cs_item_sk#2, cs_sales_price#4, cs_sold_date_sk#5, cr_refunded_cash#9, w_state#12] +Input [7]: [cs_warehouse_sk#1, cs_item_sk#2, cs_sales_price#4, cs_sold_date_sk#5, cr_refunded_cash#9, w_warehouse_sk#11, w_state#12] + +(20) Scan parquet spark_catalog.default.item +Output [3]: [i_item_sk#13, i_item_id#14, i_current_price#15] +Batched: true +Location [not included in comparison]/{warehouse_dir}/item] +PushedFilters: [IsNotNull(i_current_price), GreaterThanOrEqual(i_current_price,0.99), LessThanOrEqual(i_current_price,1.49), IsNotNull(i_item_sk)] +ReadSchema: struct<i_item_sk:int,i_item_id:string,i_current_price:decimal(7,2)> + +(21) CometFilter +Input [3]: [i_item_sk#13, i_item_id#14, i_current_price#15] +Condition : (((isnotnull(i_current_price#15) AND (i_current_price#15 >= 0.99)) AND (i_current_price#15 <= 1.49)) AND isnotnull(i_item_sk#13)) + +(22) CometProject +Input [3]: [i_item_sk#13, i_item_id#14, i_current_price#15] +Arguments: [i_item_sk#13, i_item_id#14], [i_item_sk#13, i_item_id#14] + +(23) ColumnarToRow [codegen id : 6] +Input [2]: [i_item_sk#13, i_item_id#14] + +(24) BroadcastExchange +Input [2]: [i_item_sk#13, i_item_id#14] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] + +(25) BroadcastHashJoin [codegen id : 8] +Left keys [1]: [cs_item_sk#2] +Right keys [1]: [i_item_sk#13] +Join type: Inner +Join condition: None + +(26) Project [codegen id : 8] +Output [5]: [cs_sales_price#4, cs_sold_date_sk#5, cr_refunded_cash#9, w_state#12, i_item_id#14] +Input [7]: [cs_item_sk#2, cs_sales_price#4, cs_sold_date_sk#5, cr_refunded_cash#9, w_state#12, i_item_sk#13, i_item_id#14] + +(27) ReusedExchange [Reuses operator id: 37] +Output [2]: [d_date_sk#16, d_date#17] + +(28) BroadcastHashJoin [codegen id : 8] +Left keys [1]: [cs_sold_date_sk#5] +Right keys [1]: [d_date_sk#16] +Join type: Inner +Join condition: None + +(29) Project [codegen id : 8] +Output [5]: [cs_sales_price#4, cr_refunded_cash#9, w_state#12, i_item_id#14, d_date#17] +Input [7]: [cs_sales_price#4, cs_sold_date_sk#5, cr_refunded_cash#9, w_state#12, i_item_id#14, d_date_sk#16, d_date#17] + +(30) HashAggregate [codegen id : 8] +Input [5]: [cs_sales_price#4, cr_refunded_cash#9, w_state#12, i_item_id#14, d_date#17] +Keys [2]: [w_state#12, i_item_id#14] +Functions [2]: [partial_sum(CASE WHEN (d_date#17 < 2000-03-11) THEN (cs_sales_price#4 - coalesce(cast(cr_refunded_cash#9 as decimal(12,2)), 0.00)) ELSE 0.00 END), partial_sum(CASE WHEN (d_date#17 >= 2000-03-11) THEN (cs_sales_price#4 - coalesce(cast(cr_refunded_cash#9 as decimal(12,2)), 0.00)) ELSE 0.00 END)] +Aggregate Attributes [4]: [sum#18, isEmpty#19, sum#20, isEmpty#21] +Results [6]: [w_state#12, i_item_id#14, sum#22, isEmpty#23, sum#24, isEmpty#25] + +(31) Exchange +Input [6]: [w_state#12, i_item_id#14, sum#22, isEmpty#23, sum#24, isEmpty#25] +Arguments: hashpartitioning(w_state#12, i_item_id#14, 5), ENSURE_REQUIREMENTS, [plan_id=5] + +(32) HashAggregate [codegen id : 9] +Input [6]: [w_state#12, i_item_id#14, sum#22, isEmpty#23, sum#24, isEmpty#25] +Keys [2]: [w_state#12, i_item_id#14] +Functions [2]: [sum(CASE WHEN (d_date#17 < 2000-03-11) THEN (cs_sales_price#4 - coalesce(cast(cr_refunded_cash#9 as decimal(12,2)), 0.00)) ELSE 0.00 END), sum(CASE WHEN (d_date#17 >= 2000-03-11) THEN (cs_sales_price#4 - coalesce(cast(cr_refunded_cash#9 as decimal(12,2)), 0.00)) ELSE 0.00 END)] +Aggregate Attributes [2]: [sum(CASE WHEN (d_date#17 < 2000-03-11) THEN (cs_sales_price#4 - coalesce(cast(cr_refunded_cash#9 as decimal(12,2)), 0.00)) ELSE 0.00 END)#26, sum(CASE WHEN (d_date#17 >= 2000-03-11) THEN (cs_sales_price#4 - coalesce(cast(cr_refunded_cash#9 as decimal(12,2)), 0.00)) ELSE 0.00 END)#27] +Results [4]: [w_state#12, i_item_id#14, sum(CASE WHEN (d_date#17 < 2000-03-11) THEN (cs_sales_price#4 - coalesce(cast(cr_refunded_cash#9 as decimal(12,2)), 0.00)) ELSE 0.00 END)#26 AS sales_before#28, sum(CASE WHEN (d_date#17 >= 2000-03-11) THEN (cs_sales_price#4 - coalesce(cast(cr_refunded_cash#9 as decimal(12,2)), 0.00)) ELSE 0.00 END)#27 AS sales_after#29] + +(33) TakeOrderedAndProject +Input [4]: [w_state#12, i_item_id#14, sales_before#28, sales_after#29] +Arguments: 100, [w_state#12 ASC NULLS FIRST, i_item_id#14 ASC NULLS FIRST], [w_state#12, i_item_id#14, sales_before#28, sales_after#29] + +===== Subqueries ===== + +Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#5 IN dynamicpruning#6 +BroadcastExchange (37) ++- * ColumnarToRow (36) + +- CometFilter (35) + +- CometScan parquet spark_catalog.default.date_dim (34) + + +(34) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#16, d_date#17] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,2000-02-10), LessThanOrEqual(d_date,2000-04-10), IsNotNull(d_date_sk)] +ReadSchema: struct<d_date_sk:int,d_date:date> + +(35) CometFilter +Input [2]: [d_date_sk#16, d_date#17] +Condition : (((isnotnull(d_date#17) AND (d_date#17 >= 2000-02-10)) AND (d_date#17 <= 2000-04-10)) AND isnotnull(d_date_sk#16)) + +(36) ColumnarToRow [codegen id : 1] +Input [2]: [d_date_sk#16, d_date#17] + +(37) BroadcastExchange +Input [2]: [d_date_sk#16, d_date#17] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=6] + + diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q40/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q40/simplified.txt new file mode 100644 index 000000000..10e0735b4 --- /dev/null +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q40/simplified.txt @@ -0,0 +1,60 @@ +TakeOrderedAndProject [w_state,i_item_id,sales_before,sales_after] + WholeStageCodegen (9) + HashAggregate [w_state,i_item_id,sum,isEmpty,sum,isEmpty] [sum(CASE WHEN (d_date < 2000-03-11) THEN (cs_sales_price - coalesce(cast(cr_refunded_cash as decimal(12,2)), 0.00)) ELSE 0.00 END),sum(CASE WHEN (d_date >= 2000-03-11) THEN (cs_sales_price - coalesce(cast(cr_refunded_cash as decimal(12,2)), 0.00)) ELSE 0.00 END),sales_before,sales_after,sum,isEmpty,sum,isEmpty] + InputAdapter + Exchange [w_state,i_item_id] #1 + WholeStageCodegen (8) + HashAggregate [w_state,i_item_id,d_date,cs_sales_price,cr_refunded_cash] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] + Project [cs_sales_price,cr_refunded_cash,w_state,i_item_id,d_date] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + Project [cs_sales_price,cs_sold_date_sk,cr_refunded_cash,w_state,i_item_id] + BroadcastHashJoin [cs_item_sk,i_item_sk] + Project [cs_item_sk,cs_sales_price,cs_sold_date_sk,cr_refunded_cash,w_state] + BroadcastHashJoin [cs_warehouse_sk,w_warehouse_sk] + Project [cs_warehouse_sk,cs_item_sk,cs_sales_price,cs_sold_date_sk,cr_refunded_cash] + SortMergeJoin [cs_order_number,cs_item_sk,cr_order_number,cr_item_sk] + InputAdapter + WholeStageCodegen (2) + Sort [cs_order_number,cs_item_sk] + InputAdapter + Exchange [cs_order_number,cs_item_sk] #2 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometFilter [cs_warehouse_sk,cs_item_sk] + CometScan parquet spark_catalog.default.catalog_sales [cs_warehouse_sk,cs_item_sk,cs_order_number,cs_sales_price,cs_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometFilter [d_date,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] + InputAdapter + WholeStageCodegen (4) + Sort [cr_order_number,cr_item_sk] + InputAdapter + Exchange [cr_order_number,cr_item_sk] #4 + WholeStageCodegen (3) + ColumnarToRow + InputAdapter + CometProject [cr_item_sk,cr_order_number,cr_refunded_cash] + CometFilter [cr_order_number,cr_item_sk] + CometScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_refunded_cash,cr_returned_date_sk] + InputAdapter + BroadcastExchange #5 + WholeStageCodegen (5) + ColumnarToRow + InputAdapter + CometFilter [w_warehouse_sk] + CometScan parquet spark_catalog.default.warehouse [w_warehouse_sk,w_state] + InputAdapter + BroadcastExchange #6 + WholeStageCodegen (6) + ColumnarToRow + InputAdapter + CometProject [i_item_sk,i_item_id] + CometFilter [i_current_price,i_item_sk] + CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_current_price] + InputAdapter + ReusedExchange [d_date_sk,d_date] #3 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 new file mode 100644 index 000000000..07196ba8c --- /dev/null +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q41/explain.txt @@ -0,0 +1,119 @@ +== Physical Plan == +TakeOrderedAndProject (20) ++- * HashAggregate (19) + +- Exchange (18) + +- * HashAggregate (17) + +- * Project (16) + +- * BroadcastHashJoin Inner BuildRight (15) + :- * ColumnarToRow (4) + : +- CometProject (3) + : +- CometFilter (2) + : +- CometScan parquet spark_catalog.default.item (1) + +- BroadcastExchange (14) + +- * Project (13) + +- * Filter (12) + +- * HashAggregate (11) + +- Exchange (10) + +- * ColumnarToRow (9) + +- CometHashAggregate (8) + +- CometProject (7) + +- CometFilter (6) + +- CometScan parquet spark_catalog.default.item (5) + + +(1) Scan parquet spark_catalog.default.item +Output [3]: [i_manufact_id#1, i_manufact#2, i_product_name#3] +Batched: true +Location [not included in comparison]/{warehouse_dir}/item] +PushedFilters: [IsNotNull(i_manufact_id), GreaterThanOrEqual(i_manufact_id,738), LessThanOrEqual(i_manufact_id,778), IsNotNull(i_manufact)] +ReadSchema: struct<i_manufact_id:int,i_manufact:string,i_product_name:string> + +(2) CometFilter +Input [3]: [i_manufact_id#1, i_manufact#2, i_product_name#3] +Condition : (((isnotnull(i_manufact_id#1) AND (i_manufact_id#1 >= 738)) AND (i_manufact_id#1 <= 778)) AND isnotnull(i_manufact#2)) + +(3) CometProject +Input [3]: [i_manufact_id#1, i_manufact#2, i_product_name#3] +Arguments: [i_manufact#2, i_product_name#3], [i_manufact#2, i_product_name#3] + +(4) ColumnarToRow [codegen id : 3] +Input [2]: [i_manufact#2, i_product_name#3] + +(5) Scan parquet spark_catalog.default.item +Output [5]: [i_category#4, i_manufact#5, i_size#6, i_color#7, i_units#8] +Batched: true +Location [not included in comparison]/{warehouse_dir}/item] +PushedFilters: [Or(Or(And(EqualTo(i_category,Women ),Or(And(And(Or(EqualTo(i_color,powder ),EqualTo(i_color,khaki )),Or(EqualTo(i_units,Ounce ),EqualTo(i_units,Oz ))),Or(EqualTo(i_size,medium ),EqualTo(i_size,extra large ))),And(And(Or(EqualTo(i_color,brown ),EqualTo(i_color,honeydew )),Or(EqualTo(i_units,Bunch ),EqualTo(i_units,Ton ))),Or(EqualTo(i_size,N/A ),EqualTo(i_size,small ))))),And(EqualTo(i_category,Men ),Or(And(And(Or(EqualTo(i_color,floral ),EqualTo(i_color,deep )),Or(EqualTo(i_units,N/A ),EqualTo(i_units,Dozen ))),Or(EqualTo(i_size,petite ),EqualTo(i_size,large ))),And(And(Or(EqualTo(i_color,light ),EqualTo(i_color,cornflower )),Or(EqualTo(i_units,Box ),EqualTo(i_units,Pound ))),Or(EqualTo(i_size,medium ),EqualTo(i_size,extra large )))))),Or(And(EqualTo(i_category,Women ),Or(And(And(Or(EqualTo(i_color,midnight ),EqualTo(i_color,snow )),Or(EqualTo(i_units,Pallet ),EqualTo(i_units,Gross ))),Or(EqualTo(i_size,medium ),EqualTo(i_size,extra large ))),And(And(Or(EqualTo(i_color,cyan ),EqualTo(i_color,papaya )),Or(EqualTo(i_units,Cup ),EqualTo(i_units,Dram ))),Or(EqualTo(i_size,N/A ),EqualTo(i_size,small ))))),And(EqualTo(i_category,Men ),Or(And(And(Or(EqualTo(i_color,orange ),EqualTo(i_color,frosted )),Or(EqualTo(i_units,Each ),EqualTo(i_units,Tbl ))),Or(EqualTo(i_size,petite ),EqualTo(i_size,large ))),And(And(Or(EqualTo(i_color,forest ),EqualTo(i_color,ghost )),Or(EqualTo(i_units,Lb ),EqualTo(i_units,Bundle ))),Or(EqualTo(i_size,medium ),EqualTo(i_size,extra large ))))))), IsNotNull(i_manufact)] +ReadSchema: struct<i_category:string,i_manufact:string,i_size:string,i_color:string,i_units:string> + +(6) CometFilter +Input [5]: [i_category#4, i_manufact#5, i_size#6, i_color#7, i_units#8] +Condition : (((((i_category#4 = Women ) AND (((((i_color#7 = powder ) OR (i_color#7 = khaki )) AND ((i_units#8 = Ounce ) OR (i_units#8 = Oz ))) AND ((i_size#6 = medium ) OR (i_size#6 = extra large ))) OR ((((i_color#7 = brown ) OR (i_color#7 = honeydew )) AND ((i_units#8 = Bunch ) OR (i_units#8 = Ton ))) AND ((i_size#6 = N/A ) OR (i_size#6 = small ))))) OR ((i_category#4 = Men ) AND (((((i_color#7 = floral ) OR (i_color#7 = deep )) AND ((i_units#8 = N/A ) OR (i_units#8 = Dozen ))) AND ((i_size#6 = petite ) OR (i_size#6 = large ))) OR ((((i_color#7 = light ) OR (i_color#7 = cornflower )) AND ((i_units#8 = Box ) OR (i_units#8 = Pound ))) AND ((i_size#6 = medium ) OR (i_size#6 = extra large )))))) OR (((i_category#4 = Women ) AND (((((i_color#7 = midnight ) OR (i_color#7 = snow )) AND ((i_units#8 = Pallet ) OR (i_units#8 = Gross ))) AND ((i_size#6 = medium ) OR (i_size#6 = extra large ))) OR ((((i_color#7 = cyan ) OR (i_color#7 = papaya )) AND ((i_units#8 = Cup ) OR (i_units#8 = Dram ))) AND ((i_size#6 = N/A ) OR (i_size#6 = small ))))) OR ((i_category#4 = Men ) AND (((((i_color#7 = orange ) OR (i_color#7 = frosted )) AND ((i_units#8 = Each ) OR (i_units#8 = Tbl ))) AND ((i_size#6 = petite ) OR (i_size#6 = large ))) OR ((((i_color#7 = forest ) OR (i_color#7 = ghost )) AND ((i_units#8 = Lb ) OR (i_units#8 = Bundle ))) AND ((i_size#6 = medium ) OR (i_size#6 = extra large ))))))) AND isnotnull(i_manufact#5)) + +(7) CometProject +Input [5]: [i_category#4, i_manufact#5, i_size#6, i_color#7, i_units#8] +Arguments: [i_manufact#5], [i_manufact#5] + +(8) CometHashAggregate +Input [1]: [i_manufact#5] +Keys [1]: [i_manufact#5] +Functions [1]: [partial_count(1)] + +(9) ColumnarToRow [codegen id : 1] +Input [2]: [i_manufact#5, count#9] + +(10) Exchange +Input [2]: [i_manufact#5, count#9] +Arguments: hashpartitioning(i_manufact#5, 5), ENSURE_REQUIREMENTS, [plan_id=1] + +(11) HashAggregate [codegen id : 2] +Input [2]: [i_manufact#5, count#9] +Keys [1]: [i_manufact#5] +Functions [1]: [count(1)] +Aggregate Attributes [1]: [count(1)#10] +Results [2]: [count(1)#10 AS item_cnt#11, i_manufact#5] + +(12) Filter [codegen id : 2] +Input [2]: [item_cnt#11, i_manufact#5] +Condition : (item_cnt#11 > 0) + +(13) Project [codegen id : 2] +Output [1]: [i_manufact#5] +Input [2]: [item_cnt#11, i_manufact#5] + +(14) BroadcastExchange +Input [1]: [i_manufact#5] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=2] + +(15) BroadcastHashJoin [codegen id : 3] +Left keys [1]: [i_manufact#2] +Right keys [1]: [i_manufact#5] +Join type: Inner +Join condition: None + +(16) Project [codegen id : 3] +Output [1]: [i_product_name#3] +Input [3]: [i_manufact#2, i_product_name#3, i_manufact#5] + +(17) HashAggregate [codegen id : 3] +Input [1]: [i_product_name#3] +Keys [1]: [i_product_name#3] +Functions: [] +Aggregate Attributes: [] +Results [1]: [i_product_name#3] + +(18) Exchange +Input [1]: [i_product_name#3] +Arguments: hashpartitioning(i_product_name#3, 5), ENSURE_REQUIREMENTS, [plan_id=3] + +(19) HashAggregate [codegen id : 4] +Input [1]: [i_product_name#3] +Keys [1]: [i_product_name#3] +Functions: [] +Aggregate Attributes: [] +Results [1]: [i_product_name#3] + +(20) TakeOrderedAndProject +Input [1]: [i_product_name#3] +Arguments: 100, [i_product_name#3 ASC NULLS FIRST], [i_product_name#3] + diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q41/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q41/simplified.txt new file mode 100644 index 000000000..e31217066 --- /dev/null +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q41/simplified.txt @@ -0,0 +1,29 @@ +TakeOrderedAndProject [i_product_name] + WholeStageCodegen (4) + HashAggregate [i_product_name] + InputAdapter + Exchange [i_product_name] #1 + WholeStageCodegen (3) + HashAggregate [i_product_name] + Project [i_product_name] + BroadcastHashJoin [i_manufact,i_manufact] + ColumnarToRow + InputAdapter + CometProject [i_manufact,i_product_name] + CometFilter [i_manufact_id,i_manufact] + CometScan parquet spark_catalog.default.item [i_manufact_id,i_manufact,i_product_name] + InputAdapter + BroadcastExchange #2 + WholeStageCodegen (2) + Project [i_manufact] + Filter [item_cnt] + HashAggregate [i_manufact,count] [count(1),item_cnt,count] + InputAdapter + Exchange [i_manufact] #3 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometHashAggregate [i_manufact] + CometProject [i_manufact] + CometFilter [i_category,i_color,i_units,i_size,i_manufact] + CometScan parquet spark_catalog.default.item [i_category,i_manufact,i_size,i_color,i_units] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q42/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q42/explain.txt new file mode 100644 index 000000000..d51d63d8a --- /dev/null +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q42/explain.txt @@ -0,0 +1,125 @@ +== Physical Plan == +TakeOrderedAndProject (21) ++- * HashAggregate (20) + +- Exchange (19) + +- * HashAggregate (18) + +- * Project (17) + +- * BroadcastHashJoin Inner BuildRight (16) + :- * Project (10) + : +- * BroadcastHashJoin Inner BuildRight (9) + : :- * ColumnarToRow (4) + : : +- CometProject (3) + : : +- CometFilter (2) + : : +- CometScan parquet spark_catalog.default.date_dim (1) + : +- BroadcastExchange (8) + : +- * ColumnarToRow (7) + : +- CometFilter (6) + : +- CometScan parquet spark_catalog.default.store_sales (5) + +- BroadcastExchange (15) + +- * ColumnarToRow (14) + +- CometProject (13) + +- CometFilter (12) + +- CometScan parquet spark_catalog.default.item (11) + + +(1) Scan parquet spark_catalog.default.date_dim +Output [3]: [d_date_sk#1, d_year#2, d_moy#3] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_moy), IsNotNull(d_year), EqualTo(d_moy,11), EqualTo(d_year,2000), IsNotNull(d_date_sk)] +ReadSchema: struct<d_date_sk:int,d_year:int,d_moy:int> + +(2) CometFilter +Input [3]: [d_date_sk#1, d_year#2, d_moy#3] +Condition : ((((isnotnull(d_moy#3) AND isnotnull(d_year#2)) AND (d_moy#3 = 11)) AND (d_year#2 = 2000)) AND isnotnull(d_date_sk#1)) + +(3) CometProject +Input [3]: [d_date_sk#1, d_year#2, d_moy#3] +Arguments: [d_date_sk#1, d_year#2], [d_date_sk#1, d_year#2] + +(4) ColumnarToRow [codegen id : 3] +Input [2]: [d_date_sk#1, d_year#2] + +(5) Scan parquet spark_catalog.default.store_sales +Output [3]: [ss_item_sk#4, ss_ext_sales_price#5, ss_sold_date_sk#6] +Batched: true +Location: InMemoryFileIndex [] +PartitionFilters: [isnotnull(ss_sold_date_sk#6), dynamicpruningexpression(true)] +PushedFilters: [IsNotNull(ss_item_sk)] +ReadSchema: struct<ss_item_sk:int,ss_ext_sales_price:decimal(7,2)> + +(6) CometFilter +Input [3]: [ss_item_sk#4, ss_ext_sales_price#5, ss_sold_date_sk#6] +Condition : isnotnull(ss_item_sk#4) + +(7) ColumnarToRow [codegen id : 1] +Input [3]: [ss_item_sk#4, ss_ext_sales_price#5, ss_sold_date_sk#6] + +(8) BroadcastExchange +Input [3]: [ss_item_sk#4, ss_ext_sales_price#5, ss_sold_date_sk#6] +Arguments: HashedRelationBroadcastMode(List(cast(input[2, int, true] as bigint)),false), [plan_id=1] + +(9) BroadcastHashJoin [codegen id : 3] +Left keys [1]: [d_date_sk#1] +Right keys [1]: [ss_sold_date_sk#6] +Join type: Inner +Join condition: None + +(10) Project [codegen id : 3] +Output [3]: [d_year#2, ss_item_sk#4, ss_ext_sales_price#5] +Input [5]: [d_date_sk#1, d_year#2, ss_item_sk#4, ss_ext_sales_price#5, ss_sold_date_sk#6] + +(11) Scan parquet spark_catalog.default.item +Output [4]: [i_item_sk#7, i_category_id#8, i_category#9, i_manager_id#10] +Batched: true +Location [not included in comparison]/{warehouse_dir}/item] +PushedFilters: [IsNotNull(i_manager_id), EqualTo(i_manager_id,1), IsNotNull(i_item_sk)] +ReadSchema: struct<i_item_sk:int,i_category_id:int,i_category:string,i_manager_id:int> + +(12) CometFilter +Input [4]: [i_item_sk#7, i_category_id#8, i_category#9, i_manager_id#10] +Condition : ((isnotnull(i_manager_id#10) AND (i_manager_id#10 = 1)) AND isnotnull(i_item_sk#7)) + +(13) CometProject +Input [4]: [i_item_sk#7, i_category_id#8, i_category#9, i_manager_id#10] +Arguments: [i_item_sk#7, i_category_id#8, i_category#9], [i_item_sk#7, i_category_id#8, i_category#9] + +(14) ColumnarToRow [codegen id : 2] +Input [3]: [i_item_sk#7, i_category_id#8, i_category#9] + +(15) BroadcastExchange +Input [3]: [i_item_sk#7, i_category_id#8, i_category#9] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] + +(16) BroadcastHashJoin [codegen id : 3] +Left keys [1]: [ss_item_sk#4] +Right keys [1]: [i_item_sk#7] +Join type: Inner +Join condition: None + +(17) Project [codegen id : 3] +Output [4]: [d_year#2, ss_ext_sales_price#5, i_category_id#8, i_category#9] +Input [6]: [d_year#2, ss_item_sk#4, ss_ext_sales_price#5, i_item_sk#7, i_category_id#8, i_category#9] + +(18) HashAggregate [codegen id : 3] +Input [4]: [d_year#2, ss_ext_sales_price#5, i_category_id#8, i_category#9] +Keys [3]: [d_year#2, i_category_id#8, i_category#9] +Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#5))] +Aggregate Attributes [1]: [sum#11] +Results [4]: [d_year#2, i_category_id#8, i_category#9, sum#12] + +(19) Exchange +Input [4]: [d_year#2, i_category_id#8, i_category#9, sum#12] +Arguments: hashpartitioning(d_year#2, i_category_id#8, i_category#9, 5), ENSURE_REQUIREMENTS, [plan_id=3] + +(20) HashAggregate [codegen id : 4] +Input [4]: [d_year#2, i_category_id#8, i_category#9, sum#12] +Keys [3]: [d_year#2, i_category_id#8, i_category#9] +Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#5))] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#5))#13] +Results [4]: [d_year#2, i_category_id#8, i_category#9, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#5))#13,17,2) AS sum(ss_ext_sales_price)#14] + +(21) TakeOrderedAndProject +Input [4]: [d_year#2, i_category_id#8, i_category#9, sum(ss_ext_sales_price)#14] +Arguments: 100, [sum(ss_ext_sales_price)#14 DESC NULLS LAST, d_year#2 ASC NULLS FIRST, i_category_id#8 ASC NULLS FIRST, i_category#9 ASC NULLS FIRST], [d_year#2, i_category_id#8, i_category#9, sum(ss_ext_sales_price)#14] + diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q42/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q42/simplified.txt new file mode 100644 index 000000000..67906b8c7 --- /dev/null +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q42/simplified.txt @@ -0,0 +1,31 @@ +TakeOrderedAndProject [sum(ss_ext_sales_price),d_year,i_category_id,i_category] + WholeStageCodegen (4) + HashAggregate [d_year,i_category_id,i_category,sum] [sum(UnscaledValue(ss_ext_sales_price)),sum(ss_ext_sales_price),sum] + InputAdapter + Exchange [d_year,i_category_id,i_category] #1 + WholeStageCodegen (3) + HashAggregate [d_year,i_category_id,i_category,ss_ext_sales_price] [sum,sum] + Project [d_year,ss_ext_sales_price,i_category_id,i_category] + BroadcastHashJoin [ss_item_sk,i_item_sk] + Project [d_year,ss_item_sk,ss_ext_sales_price] + BroadcastHashJoin [d_date_sk,ss_sold_date_sk] + ColumnarToRow + InputAdapter + CometProject [d_date_sk,d_year] + CometFilter [d_moy,d_year,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + InputAdapter + BroadcastExchange #2 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometFilter [ss_item_sk] + CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] + InputAdapter + BroadcastExchange #3 + WholeStageCodegen (2) + ColumnarToRow + InputAdapter + CometProject [i_item_sk,i_category_id,i_category] + CometFilter [i_manager_id,i_item_sk] + CometScan parquet spark_catalog.default.item [i_item_sk,i_category_id,i_category,i_manager_id] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q43/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q43/explain.txt new file mode 100644 index 000000000..e892aa469 --- /dev/null +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q43/explain.txt @@ -0,0 +1,125 @@ +== Physical Plan == +TakeOrderedAndProject (21) ++- * HashAggregate (20) + +- Exchange (19) + +- * HashAggregate (18) + +- * Project (17) + +- * BroadcastHashJoin Inner BuildRight (16) + :- * Project (10) + : +- * BroadcastHashJoin Inner BuildRight (9) + : :- * ColumnarToRow (4) + : : +- CometProject (3) + : : +- CometFilter (2) + : : +- CometScan parquet spark_catalog.default.date_dim (1) + : +- BroadcastExchange (8) + : +- * ColumnarToRow (7) + : +- CometFilter (6) + : +- CometScan parquet spark_catalog.default.store_sales (5) + +- BroadcastExchange (15) + +- * ColumnarToRow (14) + +- CometProject (13) + +- CometFilter (12) + +- CometScan parquet spark_catalog.default.store (11) + + +(1) Scan parquet spark_catalog.default.date_dim +Output [3]: [d_date_sk#1, d_year#2, d_day_name#3] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2000), IsNotNull(d_date_sk)] +ReadSchema: struct<d_date_sk:int,d_year:int,d_day_name:string> + +(2) CometFilter +Input [3]: [d_date_sk#1, d_year#2, d_day_name#3] +Condition : ((isnotnull(d_year#2) AND (d_year#2 = 2000)) AND isnotnull(d_date_sk#1)) + +(3) CometProject +Input [3]: [d_date_sk#1, d_year#2, d_day_name#3] +Arguments: [d_date_sk#1, d_day_name#3], [d_date_sk#1, d_day_name#3] + +(4) ColumnarToRow [codegen id : 3] +Input [2]: [d_date_sk#1, d_day_name#3] + +(5) Scan parquet spark_catalog.default.store_sales +Output [3]: [ss_store_sk#4, ss_sales_price#5, ss_sold_date_sk#6] +Batched: true +Location: InMemoryFileIndex [] +PartitionFilters: [isnotnull(ss_sold_date_sk#6), dynamicpruningexpression(true)] +PushedFilters: [IsNotNull(ss_store_sk)] +ReadSchema: struct<ss_store_sk:int,ss_sales_price:decimal(7,2)> + +(6) CometFilter +Input [3]: [ss_store_sk#4, ss_sales_price#5, ss_sold_date_sk#6] +Condition : isnotnull(ss_store_sk#4) + +(7) ColumnarToRow [codegen id : 1] +Input [3]: [ss_store_sk#4, ss_sales_price#5, ss_sold_date_sk#6] + +(8) BroadcastExchange +Input [3]: [ss_store_sk#4, ss_sales_price#5, ss_sold_date_sk#6] +Arguments: HashedRelationBroadcastMode(List(cast(input[2, int, true] as bigint)),false), [plan_id=1] + +(9) BroadcastHashJoin [codegen id : 3] +Left keys [1]: [d_date_sk#1] +Right keys [1]: [ss_sold_date_sk#6] +Join type: Inner +Join condition: None + +(10) Project [codegen id : 3] +Output [3]: [d_day_name#3, ss_store_sk#4, ss_sales_price#5] +Input [5]: [d_date_sk#1, d_day_name#3, ss_store_sk#4, ss_sales_price#5, ss_sold_date_sk#6] + +(11) Scan parquet spark_catalog.default.store +Output [4]: [s_store_sk#7, s_store_id#8, s_store_name#9, s_gmt_offset#10] +Batched: true +Location [not included in comparison]/{warehouse_dir}/store] +PushedFilters: [IsNotNull(s_gmt_offset), EqualTo(s_gmt_offset,-5.00), IsNotNull(s_store_sk)] +ReadSchema: struct<s_store_sk:int,s_store_id:string,s_store_name:string,s_gmt_offset:decimal(5,2)> + +(12) CometFilter +Input [4]: [s_store_sk#7, s_store_id#8, s_store_name#9, s_gmt_offset#10] +Condition : ((isnotnull(s_gmt_offset#10) AND (s_gmt_offset#10 = -5.00)) AND isnotnull(s_store_sk#7)) + +(13) CometProject +Input [4]: [s_store_sk#7, s_store_id#8, s_store_name#9, s_gmt_offset#10] +Arguments: [s_store_sk#7, s_store_id#8, s_store_name#9], [s_store_sk#7, s_store_id#8, s_store_name#9] + +(14) ColumnarToRow [codegen id : 2] +Input [3]: [s_store_sk#7, s_store_id#8, s_store_name#9] + +(15) BroadcastExchange +Input [3]: [s_store_sk#7, s_store_id#8, s_store_name#9] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] + +(16) BroadcastHashJoin [codegen id : 3] +Left keys [1]: [ss_store_sk#4] +Right keys [1]: [s_store_sk#7] +Join type: Inner +Join condition: None + +(17) Project [codegen id : 3] +Output [4]: [d_day_name#3, ss_sales_price#5, s_store_id#8, s_store_name#9] +Input [6]: [d_day_name#3, ss_store_sk#4, ss_sales_price#5, s_store_sk#7, s_store_id#8, s_store_name#9] + +(18) HashAggregate [codegen id : 3] +Input [4]: [d_day_name#3, ss_sales_price#5, s_store_id#8, s_store_name#9] +Keys [2]: [s_store_name#9, s_store_id#8] +Functions [7]: [partial_sum(UnscaledValue(CASE WHEN (d_day_name#3 = Sunday ) THEN ss_sales_price#5 END)), partial_sum(UnscaledValue(CASE WHEN (d_day_name#3 = Monday ) THEN ss_sales_price#5 END)), partial_sum(UnscaledValue(CASE WHEN (d_day_name#3 = Tuesday ) THEN ss_sales_price#5 END)), partial_sum(UnscaledValue(CASE WHEN (d_day_name#3 = Wednesday) THEN ss_sales_price#5 END)), partial_sum(UnscaledValue(CASE WHEN (d_day_name#3 = Thursday ) THEN ss_sales_price#5 END)), partial_sum(UnscaledValue(CASE WHEN (d_day_name#3 = Friday ) THEN ss_sales_price#5 END)), partial_sum(UnscaledValue(CASE WHEN (d_day_name#3 = Saturday ) THEN ss_sales_price#5 END))] +Aggregate Attributes [7]: [sum#11, sum#12, sum#13, sum#14, sum#15, sum#16, sum#17] +Results [9]: [s_store_name#9, s_store_id#8, sum#18, sum#19, sum#20, sum#21, sum#22, sum#23, sum#24] + +(19) Exchange +Input [9]: [s_store_name#9, s_store_id#8, sum#18, sum#19, sum#20, sum#21, sum#22, sum#23, sum#24] +Arguments: hashpartitioning(s_store_name#9, s_store_id#8, 5), ENSURE_REQUIREMENTS, [plan_id=3] + +(20) HashAggregate [codegen id : 4] +Input [9]: [s_store_name#9, s_store_id#8, sum#18, sum#19, sum#20, sum#21, sum#22, sum#23, sum#24] +Keys [2]: [s_store_name#9, s_store_id#8] +Functions [7]: [sum(UnscaledValue(CASE WHEN (d_day_name#3 = Sunday ) THEN ss_sales_price#5 END)), sum(UnscaledValue(CASE WHEN (d_day_name#3 = Monday ) THEN ss_sales_price#5 END)), sum(UnscaledValue(CASE WHEN (d_day_name#3 = Tuesday ) THEN ss_sales_price#5 END)), sum(UnscaledValue(CASE WHEN (d_day_name#3 = Wednesday) THEN ss_sales_price#5 END)), sum(UnscaledValue(CASE WHEN (d_day_name#3 = Thursday ) THEN ss_sales_price#5 END)), sum(UnscaledValue(CASE WHEN (d_day_name#3 = Friday ) THEN ss_sales_price#5 END)), sum(UnscaledValue(CASE WHEN (d_day_name#3 = Saturday ) THEN ss_sales_price#5 END))] +Aggregate Attributes [7]: [sum(UnscaledValue(CASE WHEN (d_day_name#3 = Sunday ) THEN ss_sales_price#5 END))#25, sum(UnscaledValue(CASE WHEN (d_day_name#3 = Monday ) THEN ss_sales_price#5 END))#26, sum(UnscaledValue(CASE WHEN (d_day_name#3 = Tuesday ) THEN ss_sales_price#5 END))#27, sum(UnscaledValue(CASE WHEN (d_day_name#3 = Wednesday) THEN ss_sales_price#5 END))#28, sum(UnscaledValue(CASE WHEN (d_day_name#3 = Thursday ) THEN ss_sales_price#5 END))#29, sum(UnscaledValue(CASE WHEN (d_day_name#3 = Friday ) THEN ss_sales_price#5 END))#30, sum(UnscaledValue(CASE WHEN (d_day_name#3 = Saturday ) THEN ss_sales_price#5 END))#31] +Results [9]: [s_store_name#9, s_store_id#8, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#3 = Sunday ) THEN ss_sales_price#5 END))#25,17,2) AS sun_sales#32, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#3 = Monday ) THEN ss_sales_price#5 END))#26,17,2) AS mon_sales#33, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#3 = Tuesday ) THEN ss_sales_price#5 END))#27,17,2) AS tue_sales#34, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#3 = Wednesday) THEN ss_sales_price#5 END))#28,17,2) AS wed_sales#35, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#3 = Thursday ) THEN ss_sales_price#5 END))#29,17,2) AS thu_sales#36, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#3 = Friday ) THEN ss_sales_price#5 END))#30,17,2) AS fri_sales#37, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#3 = Saturday ) THEN ss_sales_price#5 END))#31,17,2) AS sat_sales#38] + +(21) TakeOrderedAndProject +Input [9]: [s_store_name#9, s_store_id#8, sun_sales#32, mon_sales#33, tue_sales#34, wed_sales#35, thu_sales#36, fri_sales#37, sat_sales#38] +Arguments: 100, [s_store_name#9 ASC NULLS FIRST, s_store_id#8 ASC NULLS FIRST, sun_sales#32 ASC NULLS FIRST, mon_sales#33 ASC NULLS FIRST, tue_sales#34 ASC NULLS FIRST, wed_sales#35 ASC NULLS FIRST, thu_sales#36 ASC NULLS FIRST, fri_sales#37 ASC NULLS FIRST, sat_sales#38 ASC NULLS FIRST], [s_store_name#9, s_store_id#8, sun_sales#32, mon_sales#33, tue_sales#34, wed_sales#35, thu_sales#36, fri_sales#37, sat_sales#38] + diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q43/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q43/simplified.txt new file mode 100644 index 000000000..ef2043096 --- /dev/null +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q43/simplified.txt @@ -0,0 +1,31 @@ +TakeOrderedAndProject [s_store_name,s_store_id,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales] + WholeStageCodegen (4) + HashAggregate [s_store_name,s_store_id,sum,sum,sum,sum,sum,sum,sum] [sum(UnscaledValue(CASE WHEN (d_day_name = Sunday ) THEN ss_sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Monday ) THEN ss_sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Tuesday ) THEN ss_sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Wednesday) THEN ss_sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Thursday ) THEN ss_sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Friday ) THEN ss_sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Saturday ) THEN ss_sales_price END)),sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales,sum,sum,sum,sum,sum,sum,sum] + InputAdapter + Exchange [s_store_name,s_store_id] #1 + WholeStageCodegen (3) + HashAggregate [s_store_name,s_store_id,d_day_name,ss_sales_price] [sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum] + Project [d_day_name,ss_sales_price,s_store_id,s_store_name] + BroadcastHashJoin [ss_store_sk,s_store_sk] + Project [d_day_name,ss_store_sk,ss_sales_price] + BroadcastHashJoin [d_date_sk,ss_sold_date_sk] + ColumnarToRow + InputAdapter + CometProject [d_date_sk,d_day_name] + CometFilter [d_year,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_day_name] + InputAdapter + BroadcastExchange #2 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometFilter [ss_store_sk] + CometScan parquet spark_catalog.default.store_sales [ss_store_sk,ss_sales_price,ss_sold_date_sk] + InputAdapter + BroadcastExchange #3 + WholeStageCodegen (2) + ColumnarToRow + InputAdapter + CometProject [s_store_sk,s_store_id,s_store_name] + CometFilter [s_gmt_offset,s_store_sk] + CometScan parquet spark_catalog.default.store [s_store_sk,s_store_id,s_store_name,s_gmt_offset] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q44/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q44/explain.txt new file mode 100644 index 000000000..812f9f391 --- /dev/null +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q44/explain.txt @@ -0,0 +1,286 @@ +== Physical Plan == +TakeOrderedAndProject (43) ++- * Project (42) + +- * BroadcastHashJoin Inner BuildRight (41) + :- * Project (39) + : +- * BroadcastHashJoin Inner BuildRight (38) + : :- * Project (33) + : : +- * SortMergeJoin Inner (32) + : : :- * Sort (18) + : : : +- Exchange (17) + : : : +- * Project (16) + : : : +- * Filter (15) + : : : +- Window (14) + : : : +- WindowGroupLimit (13) + : : : +- * Sort (12) + : : : +- Exchange (11) + : : : +- WindowGroupLimit (10) + : : : +- * Sort (9) + : : : +- * Filter (8) + : : : +- * HashAggregate (7) + : : : +- Exchange (6) + : : : +- * HashAggregate (5) + : : : +- * ColumnarToRow (4) + : : : +- CometProject (3) + : : : +- CometFilter (2) + : : : +- CometScan parquet spark_catalog.default.store_sales (1) + : : +- * Sort (31) + : : +- Exchange (30) + : : +- * Project (29) + : : +- * Filter (28) + : : +- Window (27) + : : +- WindowGroupLimit (26) + : : +- * Sort (25) + : : +- Exchange (24) + : : +- WindowGroupLimit (23) + : : +- * Sort (22) + : : +- * Filter (21) + : : +- * HashAggregate (20) + : : +- ReusedExchange (19) + : +- BroadcastExchange (37) + : +- * ColumnarToRow (36) + : +- CometFilter (35) + : +- CometScan parquet spark_catalog.default.item (34) + +- ReusedExchange (40) + + +(1) Scan parquet spark_catalog.default.store_sales +Output [4]: [ss_item_sk#1, ss_store_sk#2, ss_net_profit#3, ss_sold_date_sk#4] +Batched: true +Location [not included in comparison]/{warehouse_dir}/store_sales] +PushedFilters: [IsNotNull(ss_store_sk), EqualTo(ss_store_sk,4)] +ReadSchema: struct<ss_item_sk:int,ss_store_sk:int,ss_net_profit:decimal(7,2)> + +(2) CometFilter +Input [4]: [ss_item_sk#1, ss_store_sk#2, ss_net_profit#3, ss_sold_date_sk#4] +Condition : (isnotnull(ss_store_sk#2) AND (ss_store_sk#2 = 4)) + +(3) CometProject +Input [4]: [ss_item_sk#1, ss_store_sk#2, ss_net_profit#3, ss_sold_date_sk#4] +Arguments: [ss_item_sk#1, ss_net_profit#3], [ss_item_sk#1, ss_net_profit#3] + +(4) ColumnarToRow [codegen id : 1] +Input [2]: [ss_item_sk#1, ss_net_profit#3] + +(5) HashAggregate [codegen id : 1] +Input [2]: [ss_item_sk#1, ss_net_profit#3] +Keys [1]: [ss_item_sk#1] +Functions [1]: [partial_avg(UnscaledValue(ss_net_profit#3))] +Aggregate Attributes [2]: [sum#5, count#6] +Results [3]: [ss_item_sk#1, sum#7, count#8] + +(6) Exchange +Input [3]: [ss_item_sk#1, sum#7, count#8] +Arguments: hashpartitioning(ss_item_sk#1, 5), ENSURE_REQUIREMENTS, [plan_id=1] + +(7) HashAggregate [codegen id : 2] +Input [3]: [ss_item_sk#1, sum#7, count#8] +Keys [1]: [ss_item_sk#1] +Functions [1]: [avg(UnscaledValue(ss_net_profit#3))] +Aggregate Attributes [1]: [avg(UnscaledValue(ss_net_profit#3))#9] +Results [2]: [ss_item_sk#1 AS item_sk#10, cast((avg(UnscaledValue(ss_net_profit#3))#9 / 100.0) as decimal(11,6)) AS rank_col#11] + +(8) Filter [codegen id : 2] +Input [2]: [item_sk#10, rank_col#11] +Condition : (isnotnull(rank_col#11) AND (cast(rank_col#11 as decimal(13,7)) > (0.9 * Subquery scalar-subquery#12, [id=#13]))) + +(9) Sort [codegen id : 2] +Input [2]: [item_sk#10, rank_col#11] +Arguments: [rank_col#11 ASC NULLS FIRST], false, 0 + +(10) WindowGroupLimit +Input [2]: [item_sk#10, rank_col#11] +Arguments: [rank_col#11 ASC NULLS FIRST], rank(rank_col#11), 10, Partial + +(11) Exchange +Input [2]: [item_sk#10, rank_col#11] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=2] + +(12) Sort [codegen id : 3] +Input [2]: [item_sk#10, rank_col#11] +Arguments: [rank_col#11 ASC NULLS FIRST], false, 0 + +(13) WindowGroupLimit +Input [2]: [item_sk#10, rank_col#11] +Arguments: [rank_col#11 ASC NULLS FIRST], rank(rank_col#11), 10, Final + +(14) Window +Input [2]: [item_sk#10, rank_col#11] +Arguments: [rank(rank_col#11) windowspecdefinition(rank_col#11 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rnk#14], [rank_col#11 ASC NULLS FIRST] + +(15) Filter [codegen id : 4] +Input [3]: [item_sk#10, rank_col#11, rnk#14] +Condition : ((rnk#14 < 11) AND isnotnull(item_sk#10)) + +(16) Project [codegen id : 4] +Output [2]: [item_sk#10, rnk#14] +Input [3]: [item_sk#10, rank_col#11, rnk#14] + +(17) Exchange +Input [2]: [item_sk#10, rnk#14] +Arguments: hashpartitioning(rnk#14, 5), ENSURE_REQUIREMENTS, [plan_id=3] + +(18) Sort [codegen id : 5] +Input [2]: [item_sk#10, rnk#14] +Arguments: [rnk#14 ASC NULLS FIRST], false, 0 + +(19) ReusedExchange [Reuses operator id: 6] +Output [3]: [ss_item_sk#15, sum#16, count#17] + +(20) HashAggregate [codegen id : 7] +Input [3]: [ss_item_sk#15, sum#16, count#17] +Keys [1]: [ss_item_sk#15] +Functions [1]: [avg(UnscaledValue(ss_net_profit#18))] +Aggregate Attributes [1]: [avg(UnscaledValue(ss_net_profit#18))#19] +Results [2]: [ss_item_sk#15 AS item_sk#20, cast((avg(UnscaledValue(ss_net_profit#18))#19 / 100.0) as decimal(11,6)) AS rank_col#21] + +(21) Filter [codegen id : 7] +Input [2]: [item_sk#20, rank_col#21] +Condition : (isnotnull(rank_col#21) AND (cast(rank_col#21 as decimal(13,7)) > (0.9 * ReusedSubquery Subquery scalar-subquery#12, [id=#13]))) + +(22) Sort [codegen id : 7] +Input [2]: [item_sk#20, rank_col#21] +Arguments: [rank_col#21 DESC NULLS LAST], false, 0 + +(23) WindowGroupLimit +Input [2]: [item_sk#20, rank_col#21] +Arguments: [rank_col#21 DESC NULLS LAST], rank(rank_col#21), 10, Partial + +(24) Exchange +Input [2]: [item_sk#20, rank_col#21] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=4] + +(25) Sort [codegen id : 8] +Input [2]: [item_sk#20, rank_col#21] +Arguments: [rank_col#21 DESC NULLS LAST], false, 0 + +(26) WindowGroupLimit +Input [2]: [item_sk#20, rank_col#21] +Arguments: [rank_col#21 DESC NULLS LAST], rank(rank_col#21), 10, Final + +(27) Window +Input [2]: [item_sk#20, rank_col#21] +Arguments: [rank(rank_col#21) windowspecdefinition(rank_col#21 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rnk#22], [rank_col#21 DESC NULLS LAST] + +(28) Filter [codegen id : 9] +Input [3]: [item_sk#20, rank_col#21, rnk#22] +Condition : ((rnk#22 < 11) AND isnotnull(item_sk#20)) + +(29) Project [codegen id : 9] +Output [2]: [item_sk#20, rnk#22] +Input [3]: [item_sk#20, rank_col#21, rnk#22] + +(30) Exchange +Input [2]: [item_sk#20, rnk#22] +Arguments: hashpartitioning(rnk#22, 5), ENSURE_REQUIREMENTS, [plan_id=5] + +(31) Sort [codegen id : 10] +Input [2]: [item_sk#20, rnk#22] +Arguments: [rnk#22 ASC NULLS FIRST], false, 0 + +(32) SortMergeJoin [codegen id : 13] +Left keys [1]: [rnk#14] +Right keys [1]: [rnk#22] +Join type: Inner +Join condition: None + +(33) Project [codegen id : 13] +Output [3]: [item_sk#10, rnk#14, item_sk#20] +Input [4]: [item_sk#10, rnk#14, item_sk#20, rnk#22] + +(34) Scan parquet spark_catalog.default.item +Output [2]: [i_item_sk#23, i_product_name#24] +Batched: true +Location [not included in comparison]/{warehouse_dir}/item] +PushedFilters: [IsNotNull(i_item_sk)] +ReadSchema: struct<i_item_sk:int,i_product_name:string> + +(35) CometFilter +Input [2]: [i_item_sk#23, i_product_name#24] +Condition : isnotnull(i_item_sk#23) + +(36) ColumnarToRow [codegen id : 11] +Input [2]: [i_item_sk#23, i_product_name#24] + +(37) BroadcastExchange +Input [2]: [i_item_sk#23, i_product_name#24] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=6] + +(38) BroadcastHashJoin [codegen id : 13] +Left keys [1]: [item_sk#10] +Right keys [1]: [i_item_sk#23] +Join type: Inner +Join condition: None + +(39) Project [codegen id : 13] +Output [3]: [rnk#14, item_sk#20, i_product_name#24] +Input [5]: [item_sk#10, rnk#14, item_sk#20, i_item_sk#23, i_product_name#24] + +(40) ReusedExchange [Reuses operator id: 37] +Output [2]: [i_item_sk#25, i_product_name#26] + +(41) BroadcastHashJoin [codegen id : 13] +Left keys [1]: [item_sk#20] +Right keys [1]: [i_item_sk#25] +Join type: Inner +Join condition: None + +(42) Project [codegen id : 13] +Output [3]: [rnk#14, i_product_name#24 AS best_performing#27, i_product_name#26 AS worst_performing#28] +Input [5]: [rnk#14, item_sk#20, i_product_name#24, i_item_sk#25, i_product_name#26] + +(43) TakeOrderedAndProject +Input [3]: [rnk#14, best_performing#27, worst_performing#28] +Arguments: 100, [rnk#14 ASC NULLS FIRST], [rnk#14, best_performing#27, worst_performing#28] + +===== Subqueries ===== + +Subquery:1 Hosting operator id = 8 Hosting Expression = Subquery scalar-subquery#12, [id=#13] +* HashAggregate (50) ++- Exchange (49) + +- * HashAggregate (48) + +- * ColumnarToRow (47) + +- CometProject (46) + +- CometFilter (45) + +- CometScan parquet spark_catalog.default.store_sales (44) + + +(44) Scan parquet spark_catalog.default.store_sales +Output [4]: [ss_addr_sk#29, ss_store_sk#30, ss_net_profit#31, ss_sold_date_sk#32] +Batched: true +Location [not included in comparison]/{warehouse_dir}/store_sales] +PushedFilters: [IsNotNull(ss_store_sk), EqualTo(ss_store_sk,4), IsNull(ss_addr_sk)] +ReadSchema: struct<ss_addr_sk:int,ss_store_sk:int,ss_net_profit:decimal(7,2)> + +(45) CometFilter +Input [4]: [ss_addr_sk#29, ss_store_sk#30, ss_net_profit#31, ss_sold_date_sk#32] +Condition : ((isnotnull(ss_store_sk#30) AND (ss_store_sk#30 = 4)) AND isnull(ss_addr_sk#29)) + +(46) CometProject +Input [4]: [ss_addr_sk#29, ss_store_sk#30, ss_net_profit#31, ss_sold_date_sk#32] +Arguments: [ss_store_sk#30, ss_net_profit#31], [ss_store_sk#30, ss_net_profit#31] + +(47) ColumnarToRow [codegen id : 1] +Input [2]: [ss_store_sk#30, ss_net_profit#31] + +(48) HashAggregate [codegen id : 1] +Input [2]: [ss_store_sk#30, ss_net_profit#31] +Keys [1]: [ss_store_sk#30] +Functions [1]: [partial_avg(UnscaledValue(ss_net_profit#31))] +Aggregate Attributes [2]: [sum#33, count#34] +Results [3]: [ss_store_sk#30, sum#35, count#36] + +(49) Exchange +Input [3]: [ss_store_sk#30, sum#35, count#36] +Arguments: hashpartitioning(ss_store_sk#30, 5), ENSURE_REQUIREMENTS, [plan_id=7] + +(50) HashAggregate [codegen id : 2] +Input [3]: [ss_store_sk#30, sum#35, count#36] +Keys [1]: [ss_store_sk#30] +Functions [1]: [avg(UnscaledValue(ss_net_profit#31))] +Aggregate Attributes [1]: [avg(UnscaledValue(ss_net_profit#31))#37] +Results [1]: [cast((avg(UnscaledValue(ss_net_profit#31))#37 / 100.0) as decimal(11,6)) AS rank_col#38] + +Subquery:2 Hosting operator id = 21 Hosting Expression = ReusedSubquery Subquery scalar-subquery#12, [id=#13] + + diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q44/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q44/simplified.txt new file mode 100644 index 000000000..35a3e9efa --- /dev/null +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q44/simplified.txt @@ -0,0 +1,81 @@ +TakeOrderedAndProject [rnk,best_performing,worst_performing] + WholeStageCodegen (13) + Project [rnk,i_product_name,i_product_name] + BroadcastHashJoin [item_sk,i_item_sk] + Project [rnk,item_sk,i_product_name] + BroadcastHashJoin [item_sk,i_item_sk] + Project [item_sk,rnk,item_sk] + SortMergeJoin [rnk,rnk] + InputAdapter + WholeStageCodegen (5) + Sort [rnk] + InputAdapter + Exchange [rnk] #1 + WholeStageCodegen (4) + Project [item_sk,rnk] + Filter [rnk,item_sk] + InputAdapter + Window [rank_col] + WindowGroupLimit [rank_col] + WholeStageCodegen (3) + Sort [rank_col] + InputAdapter + Exchange #2 + WindowGroupLimit [rank_col] + WholeStageCodegen (2) + Sort [rank_col] + Filter [rank_col] + Subquery #1 + WholeStageCodegen (2) + HashAggregate [ss_store_sk,sum,count] [avg(UnscaledValue(ss_net_profit)),rank_col,sum,count] + InputAdapter + Exchange [ss_store_sk] #4 + WholeStageCodegen (1) + HashAggregate [ss_store_sk,ss_net_profit] [sum,count,sum,count] + ColumnarToRow + InputAdapter + CometProject [ss_store_sk,ss_net_profit] + CometFilter [ss_store_sk,ss_addr_sk] + CometScan parquet spark_catalog.default.store_sales [ss_addr_sk,ss_store_sk,ss_net_profit,ss_sold_date_sk] + HashAggregate [ss_item_sk,sum,count] [avg(UnscaledValue(ss_net_profit)),item_sk,rank_col,sum,count] + InputAdapter + Exchange [ss_item_sk] #3 + WholeStageCodegen (1) + HashAggregate [ss_item_sk,ss_net_profit] [sum,count,sum,count] + ColumnarToRow + InputAdapter + CometProject [ss_item_sk,ss_net_profit] + CometFilter [ss_store_sk] + CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_net_profit,ss_sold_date_sk] + InputAdapter + WholeStageCodegen (10) + Sort [rnk] + InputAdapter + Exchange [rnk] #5 + WholeStageCodegen (9) + Project [item_sk,rnk] + Filter [rnk,item_sk] + InputAdapter + Window [rank_col] + WindowGroupLimit [rank_col] + WholeStageCodegen (8) + Sort [rank_col] + InputAdapter + Exchange #6 + WindowGroupLimit [rank_col] + WholeStageCodegen (7) + Sort [rank_col] + Filter [rank_col] + ReusedSubquery [rank_col] #1 + HashAggregate [ss_item_sk,sum,count] [avg(UnscaledValue(ss_net_profit)),item_sk,rank_col,sum,count] + InputAdapter + ReusedExchange [ss_item_sk,sum,count] #3 + InputAdapter + BroadcastExchange #7 + WholeStageCodegen (11) + ColumnarToRow + InputAdapter + CometFilter [i_item_sk] + CometScan parquet spark_catalog.default.item [i_item_sk,i_product_name] + InputAdapter + ReusedExchange [i_item_sk,i_product_name] #7 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q45/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q45/explain.txt new file mode 100644 index 000000000..d0d74569b --- /dev/null +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q45/explain.txt @@ -0,0 +1,242 @@ +== Physical Plan == +TakeOrderedAndProject (36) ++- * HashAggregate (35) + +- Exchange (34) + +- * HashAggregate (33) + +- * Project (32) + +- * Filter (31) + +- * BroadcastHashJoin ExistenceJoin(exists#1) BuildRight (30) + :- * Project (24) + : +- * BroadcastHashJoin Inner BuildRight (23) + : :- * Project (18) + : : +- * BroadcastHashJoin Inner BuildRight (17) + : : :- * Project (15) + : : : +- * BroadcastHashJoin Inner BuildRight (14) + : : : :- * Project (9) + : : : : +- * BroadcastHashJoin Inner BuildRight (8) + : : : : :- * ColumnarToRow (3) + : : : : : +- CometFilter (2) + : : : : : +- CometScan parquet spark_catalog.default.web_sales (1) + : : : : +- BroadcastExchange (7) + : : : : +- * ColumnarToRow (6) + : : : : +- CometFilter (5) + : : : : +- CometScan parquet spark_catalog.default.customer (4) + : : : +- BroadcastExchange (13) + : : : +- * ColumnarToRow (12) + : : : +- CometFilter (11) + : : : +- CometScan parquet spark_catalog.default.customer_address (10) + : : +- ReusedExchange (16) + : +- BroadcastExchange (22) + : +- * ColumnarToRow (21) + : +- CometFilter (20) + : +- CometScan parquet spark_catalog.default.item (19) + +- BroadcastExchange (29) + +- * ColumnarToRow (28) + +- CometProject (27) + +- CometFilter (26) + +- CometScan parquet spark_catalog.default.item (25) + + +(1) Scan parquet spark_catalog.default.web_sales +Output [4]: [ws_item_sk#2, ws_bill_customer_sk#3, ws_sales_price#4, ws_sold_date_sk#5] +Batched: true +Location: InMemoryFileIndex [] +PartitionFilters: [isnotnull(ws_sold_date_sk#5), dynamicpruningexpression(ws_sold_date_sk#5 IN dynamicpruning#6)] +PushedFilters: [IsNotNull(ws_bill_customer_sk), IsNotNull(ws_item_sk)] +ReadSchema: struct<ws_item_sk:int,ws_bill_customer_sk:int,ws_sales_price:decimal(7,2)> + +(2) CometFilter +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)) + +(3) ColumnarToRow [codegen id : 6] +Input [4]: [ws_item_sk#2, ws_bill_customer_sk#3, ws_sales_price#4, ws_sold_date_sk#5] + +(4) Scan parquet spark_catalog.default.customer +Output [2]: [c_customer_sk#7, c_current_addr_sk#8] +Batched: true +Location [not included in comparison]/{warehouse_dir}/customer] +PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_current_addr_sk)] +ReadSchema: struct<c_customer_sk:int,c_current_addr_sk:int> + +(5) CometFilter +Input [2]: [c_customer_sk#7, c_current_addr_sk#8] +Condition : (isnotnull(c_customer_sk#7) AND isnotnull(c_current_addr_sk#8)) + +(6) ColumnarToRow [codegen id : 1] +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] +Left keys [1]: [ws_bill_customer_sk#3] +Right keys [1]: [c_customer_sk#7] +Join type: Inner +Join condition: None + +(9) Project [codegen id : 6] +Output [4]: [ws_item_sk#2, ws_sales_price#4, ws_sold_date_sk#5, c_current_addr_sk#8] +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] + +(10) Scan parquet spark_catalog.default.customer_address +Output [3]: [ca_address_sk#9, ca_city#10, ca_zip#11] +Batched: true +Location [not included in comparison]/{warehouse_dir}/customer_address] +PushedFilters: [IsNotNull(ca_address_sk)] +ReadSchema: struct<ca_address_sk:int,ca_city:string,ca_zip:string> + +(11) CometFilter +Input [3]: [ca_address_sk#9, ca_city#10, ca_zip#11] +Condition : isnotnull(ca_address_sk#9) + +(12) ColumnarToRow [codegen id : 2] +Input [3]: [ca_address_sk#9, ca_city#10, ca_zip#11] + +(13) BroadcastExchange +Input [3]: [ca_address_sk#9, ca_city#10, ca_zip#11] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=2] + +(14) BroadcastHashJoin [codegen id : 6] +Left keys [1]: [c_current_addr_sk#8] +Right keys [1]: [ca_address_sk#9] +Join type: Inner +Join condition: None + +(15) Project [codegen id : 6] +Output [5]: [ws_item_sk#2, ws_sales_price#4, ws_sold_date_sk#5, ca_city#10, ca_zip#11] +Input [7]: [ws_item_sk#2, ws_sales_price#4, ws_sold_date_sk#5, c_current_addr_sk#8, ca_address_sk#9, ca_city#10, ca_zip#11] + +(16) ReusedExchange [Reuses operator id: 41] +Output [1]: [d_date_sk#12] + +(17) BroadcastHashJoin [codegen id : 6] +Left keys [1]: [ws_sold_date_sk#5] +Right keys [1]: [d_date_sk#12] +Join type: Inner +Join condition: None + +(18) Project [codegen id : 6] +Output [4]: [ws_item_sk#2, ws_sales_price#4, ca_city#10, ca_zip#11] +Input [6]: [ws_item_sk#2, ws_sales_price#4, ws_sold_date_sk#5, ca_city#10, ca_zip#11, d_date_sk#12] + +(19) Scan parquet spark_catalog.default.item +Output [2]: [i_item_sk#13, i_item_id#14] +Batched: true +Location [not included in comparison]/{warehouse_dir}/item] +PushedFilters: [IsNotNull(i_item_sk)] +ReadSchema: struct<i_item_sk:int,i_item_id:string> + +(20) CometFilter +Input [2]: [i_item_sk#13, i_item_id#14] +Condition : isnotnull(i_item_sk#13) + +(21) ColumnarToRow [codegen id : 4] +Input [2]: [i_item_sk#13, i_item_id#14] + +(22) BroadcastExchange +Input [2]: [i_item_sk#13, i_item_id#14] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=3] + +(23) BroadcastHashJoin [codegen id : 6] +Left keys [1]: [ws_item_sk#2] +Right keys [1]: [i_item_sk#13] +Join type: Inner +Join condition: None + +(24) Project [codegen id : 6] +Output [4]: [ws_sales_price#4, ca_city#10, ca_zip#11, i_item_id#14] +Input [6]: [ws_item_sk#2, ws_sales_price#4, ca_city#10, ca_zip#11, i_item_sk#13, i_item_id#14] + +(25) Scan parquet spark_catalog.default.item +Output [2]: [i_item_sk#15, i_item_id#16] +Batched: true +Location [not included in comparison]/{warehouse_dir}/item] +PushedFilters: [In(i_item_sk, [11,13,17,19,2,23,29,3,5,7])] +ReadSchema: struct<i_item_sk:int,i_item_id:string> + +(26) CometFilter +Input [2]: [i_item_sk#15, i_item_id#16] +Condition : i_item_sk#15 IN (2,3,5,7,11,13,17,19,23,29) + +(27) CometProject +Input [2]: [i_item_sk#15, i_item_id#16] +Arguments: [i_item_id#16], [i_item_id#16] + +(28) ColumnarToRow [codegen id : 5] +Input [1]: [i_item_id#16] + +(29) BroadcastExchange +Input [1]: [i_item_id#16] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=4] + +(30) BroadcastHashJoin [codegen id : 6] +Left keys [1]: [i_item_id#14] +Right keys [1]: [i_item_id#16] +Join type: ExistenceJoin(exists#1) +Join condition: None + +(31) Filter [codegen id : 6] +Input [5]: [ws_sales_price#4, ca_city#10, ca_zip#11, i_item_id#14, exists#1] +Condition : (substr(ca_zip#11, 1, 5) IN (85669,86197,88274,83405,86475,85392,85460,80348,81792) OR exists#1) + +(32) Project [codegen id : 6] +Output [3]: [ws_sales_price#4, ca_city#10, ca_zip#11] +Input [5]: [ws_sales_price#4, ca_city#10, ca_zip#11, i_item_id#14, exists#1] + +(33) HashAggregate [codegen id : 6] +Input [3]: [ws_sales_price#4, ca_city#10, ca_zip#11] +Keys [2]: [ca_zip#11, ca_city#10] +Functions [1]: [partial_sum(UnscaledValue(ws_sales_price#4))] +Aggregate Attributes [1]: [sum#17] +Results [3]: [ca_zip#11, ca_city#10, sum#18] + +(34) Exchange +Input [3]: [ca_zip#11, ca_city#10, sum#18] +Arguments: hashpartitioning(ca_zip#11, ca_city#10, 5), ENSURE_REQUIREMENTS, [plan_id=5] + +(35) HashAggregate [codegen id : 7] +Input [3]: [ca_zip#11, ca_city#10, sum#18] +Keys [2]: [ca_zip#11, ca_city#10] +Functions [1]: [sum(UnscaledValue(ws_sales_price#4))] +Aggregate Attributes [1]: [sum(UnscaledValue(ws_sales_price#4))#19] +Results [3]: [ca_zip#11, ca_city#10, MakeDecimal(sum(UnscaledValue(ws_sales_price#4))#19,17,2) AS sum(ws_sales_price)#20] + +(36) TakeOrderedAndProject +Input [3]: [ca_zip#11, ca_city#10, sum(ws_sales_price)#20] +Arguments: 100, [ca_zip#11 ASC NULLS FIRST, ca_city#10 ASC NULLS FIRST], [ca_zip#11, ca_city#10, sum(ws_sales_price)#20] + +===== Subqueries ===== + +Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#5 IN dynamicpruning#6 +BroadcastExchange (41) ++- * ColumnarToRow (40) + +- CometProject (39) + +- CometFilter (38) + +- CometScan parquet spark_catalog.default.date_dim (37) + + +(37) Scan parquet spark_catalog.default.date_dim +Output [3]: [d_date_sk#12, d_year#21, d_qoy#22] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_qoy), IsNotNull(d_year), EqualTo(d_qoy,2), EqualTo(d_year,2001), IsNotNull(d_date_sk)] +ReadSchema: struct<d_date_sk:int,d_year:int,d_qoy:int> + +(38) CometFilter +Input [3]: [d_date_sk#12, d_year#21, d_qoy#22] +Condition : ((((isnotnull(d_qoy#22) AND isnotnull(d_year#21)) AND (d_qoy#22 = 2)) AND (d_year#21 = 2001)) AND isnotnull(d_date_sk#12)) + +(39) CometProject +Input [3]: [d_date_sk#12, d_year#21, d_qoy#22] +Arguments: [d_date_sk#12], [d_date_sk#12] + +(40) ColumnarToRow [codegen id : 1] +Input [1]: [d_date_sk#12] + +(41) BroadcastExchange +Input [1]: [d_date_sk#12] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=6] + + 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 new file mode 100644 index 000000000..383cbb7e3 --- /dev/null +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q45/simplified.txt @@ -0,0 +1,61 @@ +TakeOrderedAndProject [ca_zip,ca_city,sum(ws_sales_price)] + WholeStageCodegen (7) + HashAggregate [ca_zip,ca_city,sum] [sum(UnscaledValue(ws_sales_price)),sum(ws_sales_price),sum] + InputAdapter + Exchange [ca_zip,ca_city] #1 + WholeStageCodegen (6) + HashAggregate [ca_zip,ca_city,ws_sales_price] [sum,sum] + Project [ws_sales_price,ca_city,ca_zip] + Filter [ca_zip,exists] + BroadcastHashJoin [i_item_id,i_item_id] + Project [ws_sales_price,ca_city,ca_zip,i_item_id] + BroadcastHashJoin [ws_item_sk,i_item_sk] + Project [ws_item_sk,ws_sales_price,ca_city,ca_zip] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + Project [ws_item_sk,ws_sales_price,ws_sold_date_sk,ca_city,ca_zip] + BroadcastHashJoin [c_current_addr_sk,ca_address_sk] + Project [ws_item_sk,ws_sales_price,ws_sold_date_sk,c_current_addr_sk] + BroadcastHashJoin [ws_bill_customer_sk,c_customer_sk] + ColumnarToRow + InputAdapter + CometFilter [ws_bill_customer_sk,ws_item_sk] + CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_bill_customer_sk,ws_sales_price,ws_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #2 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_qoy,d_year,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] + InputAdapter + BroadcastExchange #3 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometFilter [c_customer_sk,c_current_addr_sk] + CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk] + InputAdapter + BroadcastExchange #4 + WholeStageCodegen (2) + ColumnarToRow + InputAdapter + CometFilter [ca_address_sk] + CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_city,ca_zip] + InputAdapter + ReusedExchange [d_date_sk] #2 + InputAdapter + BroadcastExchange #5 + WholeStageCodegen (4) + ColumnarToRow + InputAdapter + CometFilter [i_item_sk] + CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id] + InputAdapter + BroadcastExchange #6 + WholeStageCodegen (5) + ColumnarToRow + InputAdapter + CometProject [i_item_id] + CometFilter [i_item_sk] + CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q46/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q46/explain.txt new file mode 100644 index 000000000..e07e2ab24 --- /dev/null +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q46/explain.txt @@ -0,0 +1,258 @@ +== Physical Plan == +TakeOrderedAndProject (39) ++- * Project (38) + +- * BroadcastHashJoin Inner BuildRight (37) + :- * Project (35) + : +- * BroadcastHashJoin Inner BuildRight (34) + : :- * HashAggregate (29) + : : +- Exchange (28) + : : +- * HashAggregate (27) + : : +- * Project (26) + : : +- * BroadcastHashJoin Inner BuildRight (25) + : : :- * Project (20) + : : : +- * BroadcastHashJoin Inner BuildRight (19) + : : : :- * Project (13) + : : : : +- * BroadcastHashJoin Inner BuildRight (12) + : : : : :- * Project (6) + : : : : : +- * BroadcastHashJoin Inner BuildRight (5) + : : : : : :- * ColumnarToRow (3) + : : : : : : +- CometFilter (2) + : : : : : : +- CometScan parquet spark_catalog.default.store_sales (1) + : : : : : +- ReusedExchange (4) + : : : : +- BroadcastExchange (11) + : : : : +- * ColumnarToRow (10) + : : : : +- CometProject (9) + : : : : +- CometFilter (8) + : : : : +- CometScan parquet spark_catalog.default.store (7) + : : : +- BroadcastExchange (18) + : : : +- * ColumnarToRow (17) + : : : +- CometProject (16) + : : : +- CometFilter (15) + : : : +- CometScan parquet spark_catalog.default.household_demographics (14) + : : +- BroadcastExchange (24) + : : +- * ColumnarToRow (23) + : : +- CometFilter (22) + : : +- CometScan parquet spark_catalog.default.customer_address (21) + : +- BroadcastExchange (33) + : +- * ColumnarToRow (32) + : +- CometFilter (31) + : +- CometScan parquet spark_catalog.default.customer (30) + +- ReusedExchange (36) + + +(1) Scan parquet spark_catalog.default.store_sales +Output [8]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_store_sk#4, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7, ss_sold_date_sk#8] +Batched: true +Location: InMemoryFileIndex [] +PartitionFilters: [isnotnull(ss_sold_date_sk#8), dynamicpruningexpression(ss_sold_date_sk#8 IN dynamicpruning#9)] +PushedFilters: [IsNotNull(ss_store_sk), IsNotNull(ss_hdemo_sk), IsNotNull(ss_addr_sk), IsNotNull(ss_customer_sk)] +ReadSchema: struct<ss_customer_sk:int,ss_hdemo_sk:int,ss_addr_sk:int,ss_store_sk:int,ss_ticket_number:int,ss_coupon_amt:decimal(7,2),ss_net_profit:decimal(7,2)> + +(2) CometFilter +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)) + +(3) ColumnarToRow [codegen id : 5] +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] + +(4) ReusedExchange [Reuses operator id: 44] +Output [1]: [d_date_sk#10] + +(5) BroadcastHashJoin [codegen id : 5] +Left keys [1]: [ss_sold_date_sk#8] +Right keys [1]: [d_date_sk#10] +Join type: Inner +Join condition: None + +(6) Project [codegen id : 5] +Output [7]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_store_sk#4, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7] +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] + +(7) Scan parquet spark_catalog.default.store +Output [2]: [s_store_sk#11, s_city#12] +Batched: true +Location [not included in comparison]/{warehouse_dir}/store] +PushedFilters: [In(s_city, [Fairview,Midway]), IsNotNull(s_store_sk)] +ReadSchema: struct<s_store_sk:int,s_city:string> + +(8) CometFilter +Input [2]: [s_store_sk#11, s_city#12] +Condition : (s_city#12 IN (Fairview,Midway) AND isnotnull(s_store_sk#11)) + +(9) CometProject +Input [2]: [s_store_sk#11, s_city#12] +Arguments: [s_store_sk#11], [s_store_sk#11] + +(10) ColumnarToRow [codegen id : 2] +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] +Left keys [1]: [ss_store_sk#4] +Right keys [1]: [s_store_sk#11] +Join type: Inner +Join condition: None + +(13) Project [codegen id : 5] +Output [6]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7] +Input [8]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_store_sk#4, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7, s_store_sk#11] + +(14) Scan parquet spark_catalog.default.household_demographics +Output [3]: [hd_demo_sk#13, hd_dep_count#14, hd_vehicle_count#15] +Batched: true +Location [not included in comparison]/{warehouse_dir}/household_demographics] +PushedFilters: [Or(EqualTo(hd_dep_count,4),EqualTo(hd_vehicle_count,3)), IsNotNull(hd_demo_sk)] +ReadSchema: struct<hd_demo_sk:int,hd_dep_count:int,hd_vehicle_count:int> + +(15) CometFilter +Input [3]: [hd_demo_sk#13, hd_dep_count#14, hd_vehicle_count#15] +Condition : (((hd_dep_count#14 = 4) OR (hd_vehicle_count#15 = 3)) AND isnotnull(hd_demo_sk#13)) + +(16) CometProject +Input [3]: [hd_demo_sk#13, hd_dep_count#14, hd_vehicle_count#15] +Arguments: [hd_demo_sk#13], [hd_demo_sk#13] + +(17) ColumnarToRow [codegen id : 3] +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] +Left keys [1]: [ss_hdemo_sk#2] +Right keys [1]: [hd_demo_sk#13] +Join type: Inner +Join condition: None + +(20) Project [codegen id : 5] +Output [5]: [ss_customer_sk#1, ss_addr_sk#3, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7] +Input [7]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7, hd_demo_sk#13] + +(21) Scan parquet spark_catalog.default.customer_address +Output [2]: [ca_address_sk#16, ca_city#17] +Batched: true +Location [not included in comparison]/{warehouse_dir}/customer_address] +PushedFilters: [IsNotNull(ca_address_sk), IsNotNull(ca_city)] +ReadSchema: struct<ca_address_sk:int,ca_city:string> + +(22) CometFilter +Input [2]: [ca_address_sk#16, ca_city#17] +Condition : (isnotnull(ca_address_sk#16) AND isnotnull(ca_city#17)) + +(23) ColumnarToRow [codegen id : 4] +Input [2]: [ca_address_sk#16, ca_city#17] + +(24) BroadcastExchange +Input [2]: [ca_address_sk#16, ca_city#17] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=3] + +(25) BroadcastHashJoin [codegen id : 5] +Left keys [1]: [ss_addr_sk#3] +Right keys [1]: [ca_address_sk#16] +Join type: Inner +Join condition: None + +(26) Project [codegen id : 5] +Output [6]: [ss_customer_sk#1, ss_addr_sk#3, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7, ca_city#17] +Input [7]: [ss_customer_sk#1, ss_addr_sk#3, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7, ca_address_sk#16, ca_city#17] + +(27) HashAggregate [codegen id : 5] +Input [6]: [ss_customer_sk#1, ss_addr_sk#3, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7, ca_city#17] +Keys [4]: [ss_ticket_number#5, ss_customer_sk#1, ss_addr_sk#3, ca_city#17] +Functions [2]: [partial_sum(UnscaledValue(ss_coupon_amt#6)), partial_sum(UnscaledValue(ss_net_profit#7))] +Aggregate Attributes [2]: [sum#18, sum#19] +Results [6]: [ss_ticket_number#5, ss_customer_sk#1, ss_addr_sk#3, ca_city#17, sum#20, sum#21] + +(28) Exchange +Input [6]: [ss_ticket_number#5, ss_customer_sk#1, ss_addr_sk#3, ca_city#17, sum#20, sum#21] +Arguments: hashpartitioning(ss_ticket_number#5, ss_customer_sk#1, ss_addr_sk#3, ca_city#17, 5), ENSURE_REQUIREMENTS, [plan_id=4] + +(29) HashAggregate [codegen id : 8] +Input [6]: [ss_ticket_number#5, ss_customer_sk#1, ss_addr_sk#3, ca_city#17, sum#20, sum#21] +Keys [4]: [ss_ticket_number#5, ss_customer_sk#1, ss_addr_sk#3, ca_city#17] +Functions [2]: [sum(UnscaledValue(ss_coupon_amt#6)), sum(UnscaledValue(ss_net_profit#7))] +Aggregate Attributes [2]: [sum(UnscaledValue(ss_coupon_amt#6))#22, sum(UnscaledValue(ss_net_profit#7))#23] +Results [5]: [ss_ticket_number#5, ss_customer_sk#1, ca_city#17 AS bought_city#24, MakeDecimal(sum(UnscaledValue(ss_coupon_amt#6))#22,17,2) AS amt#25, MakeDecimal(sum(UnscaledValue(ss_net_profit#7))#23,17,2) AS profit#26] + +(30) Scan parquet spark_catalog.default.customer +Output [4]: [c_customer_sk#27, c_current_addr_sk#28, c_first_name#29, c_last_name#30] +Batched: true +Location [not included in comparison]/{warehouse_dir}/customer] +PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_current_addr_sk)] +ReadSchema: struct<c_customer_sk:int,c_current_addr_sk:int,c_first_name:string,c_last_name:string> + +(31) CometFilter +Input [4]: [c_customer_sk#27, c_current_addr_sk#28, c_first_name#29, c_last_name#30] +Condition : (isnotnull(c_customer_sk#27) AND isnotnull(c_current_addr_sk#28)) + +(32) ColumnarToRow [codegen id : 6] +Input [4]: [c_customer_sk#27, c_current_addr_sk#28, c_first_name#29, c_last_name#30] + +(33) BroadcastExchange +Input [4]: [c_customer_sk#27, c_current_addr_sk#28, c_first_name#29, c_last_name#30] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=5] + +(34) BroadcastHashJoin [codegen id : 8] +Left keys [1]: [ss_customer_sk#1] +Right keys [1]: [c_customer_sk#27] +Join type: Inner +Join condition: None + +(35) Project [codegen id : 8] +Output [7]: [ss_ticket_number#5, bought_city#24, amt#25, profit#26, c_current_addr_sk#28, c_first_name#29, c_last_name#30] +Input [9]: [ss_ticket_number#5, ss_customer_sk#1, bought_city#24, amt#25, profit#26, c_customer_sk#27, c_current_addr_sk#28, c_first_name#29, c_last_name#30] + +(36) ReusedExchange [Reuses operator id: 24] +Output [2]: [ca_address_sk#31, ca_city#32] + +(37) BroadcastHashJoin [codegen id : 8] +Left keys [1]: [c_current_addr_sk#28] +Right keys [1]: [ca_address_sk#31] +Join type: Inner +Join condition: NOT (ca_city#32 = bought_city#24) + +(38) Project [codegen id : 8] +Output [7]: [c_last_name#30, c_first_name#29, ca_city#32, bought_city#24, ss_ticket_number#5, amt#25, profit#26] +Input [9]: [ss_ticket_number#5, bought_city#24, amt#25, profit#26, c_current_addr_sk#28, c_first_name#29, c_last_name#30, ca_address_sk#31, ca_city#32] + +(39) TakeOrderedAndProject +Input [7]: [c_last_name#30, c_first_name#29, ca_city#32, bought_city#24, ss_ticket_number#5, amt#25, profit#26] +Arguments: 100, [c_last_name#30 ASC NULLS FIRST, c_first_name#29 ASC NULLS FIRST, ca_city#32 ASC NULLS FIRST, bought_city#24 ASC NULLS FIRST, ss_ticket_number#5 ASC NULLS FIRST], [c_last_name#30, c_first_name#29, ca_city#32, bought_city#24, ss_ticket_number#5, amt#25, profit#26] + +===== Subqueries ===== + +Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#8 IN dynamicpruning#9 +BroadcastExchange (44) ++- * ColumnarToRow (43) + +- CometProject (42) + +- CometFilter (41) + +- CometScan parquet spark_catalog.default.date_dim (40) + + +(40) Scan parquet spark_catalog.default.date_dim +Output [3]: [d_date_sk#10, d_year#33, d_dow#34] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [In(d_dow, [0,6]), In(d_year, [1999,2000,2001]), IsNotNull(d_date_sk)] +ReadSchema: struct<d_date_sk:int,d_year:int,d_dow:int> + +(41) CometFilter +Input [3]: [d_date_sk#10, d_year#33, d_dow#34] +Condition : ((d_dow#34 IN (6,0) AND d_year#33 IN (1999,2000,2001)) AND isnotnull(d_date_sk#10)) + +(42) CometProject +Input [3]: [d_date_sk#10, d_year#33, d_dow#34] +Arguments: [d_date_sk#10], [d_date_sk#10] + +(43) ColumnarToRow [codegen id : 1] +Input [1]: [d_date_sk#10] + +(44) BroadcastExchange +Input [1]: [d_date_sk#10] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=6] + + diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q46/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q46/simplified.txt new file mode 100644 index 000000000..04c59a2d3 --- /dev/null +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q46/simplified.txt @@ -0,0 +1,65 @@ +TakeOrderedAndProject [c_last_name,c_first_name,ca_city,bought_city,ss_ticket_number,amt,profit] + WholeStageCodegen (8) + Project [c_last_name,c_first_name,ca_city,bought_city,ss_ticket_number,amt,profit] + BroadcastHashJoin [c_current_addr_sk,ca_address_sk,ca_city,bought_city] + Project [ss_ticket_number,bought_city,amt,profit,c_current_addr_sk,c_first_name,c_last_name] + BroadcastHashJoin [ss_customer_sk,c_customer_sk] + HashAggregate [ss_ticket_number,ss_customer_sk,ss_addr_sk,ca_city,sum,sum] [sum(UnscaledValue(ss_coupon_amt)),sum(UnscaledValue(ss_net_profit)),bought_city,amt,profit,sum,sum] + InputAdapter + Exchange [ss_ticket_number,ss_customer_sk,ss_addr_sk,ca_city] #1 + WholeStageCodegen (5) + HashAggregate [ss_ticket_number,ss_customer_sk,ss_addr_sk,ca_city,ss_coupon_amt,ss_net_profit] [sum,sum,sum,sum] + Project [ss_customer_sk,ss_addr_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit,ca_city] + BroadcastHashJoin [ss_addr_sk,ca_address_sk] + Project [ss_customer_sk,ss_addr_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit] + BroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] + Project [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit] + BroadcastHashJoin [ss_store_sk,s_store_sk] + Project [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + ColumnarToRow + InputAdapter + CometFilter [ss_store_sk,ss_hdemo_sk,ss_addr_sk,ss_customer_sk] + CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #2 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_dow,d_year,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dow] + InputAdapter + ReusedExchange [d_date_sk] #2 + InputAdapter + BroadcastExchange #3 + WholeStageCodegen (2) + ColumnarToRow + InputAdapter + CometProject [s_store_sk] + CometFilter [s_city,s_store_sk] + CometScan parquet spark_catalog.default.store [s_store_sk,s_city] + InputAdapter + BroadcastExchange #4 + WholeStageCodegen (3) + ColumnarToRow + InputAdapter + CometProject [hd_demo_sk] + CometFilter [hd_dep_count,hd_vehicle_count,hd_demo_sk] + CometScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_dep_count,hd_vehicle_count] + InputAdapter + BroadcastExchange #5 + WholeStageCodegen (4) + ColumnarToRow + InputAdapter + CometFilter [ca_address_sk,ca_city] + CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_city] + InputAdapter + BroadcastExchange #6 + WholeStageCodegen (6) + ColumnarToRow + InputAdapter + CometFilter [c_customer_sk,c_current_addr_sk] + CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name] + InputAdapter + ReusedExchange [ca_address_sk,ca_city] #5 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 new file mode 100644 index 000000000..fca7362ce --- /dev/null +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q47/explain.txt @@ -0,0 +1,279 @@ +== Physical Plan == +TakeOrderedAndProject (45) ++- * Project (44) + +- * BroadcastHashJoin Inner BuildRight (43) + :- * Project (37) + : +- * BroadcastHashJoin Inner BuildRight (36) + : :- * Project (28) + : : +- * Filter (27) + : : +- Window (26) + : : +- * Filter (25) + : : +- Window (24) + : : +- * Sort (23) + : : +- Exchange (22) + : : +- * HashAggregate (21) + : : +- Exchange (20) + : : +- * HashAggregate (19) + : : +- * Project (18) + : : +- * BroadcastHashJoin Inner BuildRight (17) + : : :- * Project (12) + : : : +- * BroadcastHashJoin Inner BuildRight (11) + : : : :- * Project (9) + : : : : +- * BroadcastHashJoin Inner BuildRight (8) + : : : : :- * ColumnarToRow (3) + : : : : : +- CometFilter (2) + : : : : : +- CometScan parquet spark_catalog.default.item (1) + : : : : +- BroadcastExchange (7) + : : : : +- * ColumnarToRow (6) + : : : : +- CometFilter (5) + : : : : +- CometScan parquet spark_catalog.default.store_sales (4) + : : : +- ReusedExchange (10) + : : +- BroadcastExchange (16) + : : +- * ColumnarToRow (15) + : : +- CometFilter (14) + : : +- CometScan parquet spark_catalog.default.store (13) + : +- BroadcastExchange (35) + : +- * Project (34) + : +- Window (33) + : +- * Sort (32) + : +- Exchange (31) + : +- * HashAggregate (30) + : +- ReusedExchange (29) + +- BroadcastExchange (42) + +- * Project (41) + +- Window (40) + +- * Sort (39) + +- ReusedExchange (38) + + +(1) Scan parquet spark_catalog.default.item +Output [3]: [i_item_sk#1, i_brand#2, i_category#3] +Batched: true +Location [not included in comparison]/{warehouse_dir}/item] +PushedFilters: [IsNotNull(i_item_sk), IsNotNull(i_category), IsNotNull(i_brand)] +ReadSchema: struct<i_item_sk:int,i_brand:string,i_category:string> + +(2) CometFilter +Input [3]: [i_item_sk#1, i_brand#2, i_category#3] +Condition : ((isnotnull(i_item_sk#1) AND isnotnull(i_category#3)) AND isnotnull(i_brand#2)) + +(3) ColumnarToRow [codegen id : 4] +Input [3]: [i_item_sk#1, i_brand#2, i_category#3] + +(4) Scan parquet spark_catalog.default.store_sales +Output [4]: [ss_item_sk#4, ss_store_sk#5, ss_sales_price#6, ss_sold_date_sk#7] +Batched: true +Location: InMemoryFileIndex [] +PartitionFilters: [isnotnull(ss_sold_date_sk#7), dynamicpruningexpression(ss_sold_date_sk#7 IN dynamicpruning#8)] +PushedFilters: [IsNotNull(ss_item_sk), IsNotNull(ss_store_sk)] +ReadSchema: struct<ss_item_sk:int,ss_store_sk:int,ss_sales_price:decimal(7,2)> + +(5) CometFilter +Input [4]: [ss_item_sk#4, ss_store_sk#5, ss_sales_price#6, ss_sold_date_sk#7] +Condition : (isnotnull(ss_item_sk#4) AND isnotnull(ss_store_sk#5)) + +(6) ColumnarToRow [codegen id : 1] +Input [4]: [ss_item_sk#4, ss_store_sk#5, ss_sales_price#6, ss_sold_date_sk#7] + +(7) BroadcastExchange +Input [4]: [ss_item_sk#4, ss_store_sk#5, ss_sales_price#6, ss_sold_date_sk#7] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=1] + +(8) BroadcastHashJoin [codegen id : 4] +Left keys [1]: [i_item_sk#1] +Right keys [1]: [ss_item_sk#4] +Join type: Inner +Join condition: None + +(9) Project [codegen id : 4] +Output [5]: [i_brand#2, i_category#3, ss_store_sk#5, ss_sales_price#6, ss_sold_date_sk#7] +Input [7]: [i_item_sk#1, i_brand#2, i_category#3, ss_item_sk#4, ss_store_sk#5, ss_sales_price#6, ss_sold_date_sk#7] + +(10) ReusedExchange [Reuses operator id: 49] +Output [3]: [d_date_sk#9, d_year#10, d_moy#11] + +(11) BroadcastHashJoin [codegen id : 4] +Left keys [1]: [ss_sold_date_sk#7] +Right keys [1]: [d_date_sk#9] +Join type: Inner +Join condition: None + +(12) Project [codegen id : 4] +Output [6]: [i_brand#2, i_category#3, ss_store_sk#5, ss_sales_price#6, d_year#10, d_moy#11] +Input [8]: [i_brand#2, i_category#3, ss_store_sk#5, ss_sales_price#6, ss_sold_date_sk#7, d_date_sk#9, d_year#10, d_moy#11] + +(13) Scan parquet spark_catalog.default.store +Output [3]: [s_store_sk#12, s_store_name#13, s_company_name#14] +Batched: true +Location [not included in comparison]/{warehouse_dir}/store] +PushedFilters: [IsNotNull(s_store_sk), IsNotNull(s_store_name), IsNotNull(s_company_name)] +ReadSchema: struct<s_store_sk:int,s_store_name:string,s_company_name:string> + +(14) CometFilter +Input [3]: [s_store_sk#12, s_store_name#13, s_company_name#14] +Condition : ((isnotnull(s_store_sk#12) AND isnotnull(s_store_name#13)) AND isnotnull(s_company_name#14)) + +(15) ColumnarToRow [codegen id : 3] +Input [3]: [s_store_sk#12, s_store_name#13, s_company_name#14] + +(16) BroadcastExchange +Input [3]: [s_store_sk#12, s_store_name#13, s_company_name#14] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=2] + +(17) BroadcastHashJoin [codegen id : 4] +Left keys [1]: [ss_store_sk#5] +Right keys [1]: [s_store_sk#12] +Join type: Inner +Join condition: None + +(18) Project [codegen id : 4] +Output [7]: [i_brand#2, i_category#3, ss_sales_price#6, d_year#10, d_moy#11, s_store_name#13, s_company_name#14] +Input [9]: [i_brand#2, i_category#3, ss_store_sk#5, ss_sales_price#6, d_year#10, d_moy#11, s_store_sk#12, s_store_name#13, s_company_name#14] + +(19) HashAggregate [codegen id : 4] +Input [7]: [i_brand#2, i_category#3, ss_sales_price#6, d_year#10, d_moy#11, s_store_name#13, s_company_name#14] +Keys [6]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11] +Functions [1]: [partial_sum(UnscaledValue(ss_sales_price#6))] +Aggregate Attributes [1]: [sum#15] +Results [7]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, sum#16] + +(20) Exchange +Input [7]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, sum#16] +Arguments: hashpartitioning(i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, 5), ENSURE_REQUIREMENTS, [plan_id=3] + +(21) HashAggregate [codegen id : 5] +Input [7]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, sum#16] +Keys [6]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11] +Functions [1]: [sum(UnscaledValue(ss_sales_price#6))] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_sales_price#6))#17] +Results [8]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, MakeDecimal(sum(UnscaledValue(ss_sales_price#6))#17,17,2) AS sum_sales#18, MakeDecimal(sum(UnscaledValue(ss_sales_price#6))#17,17,2) AS _w0#19] + +(22) Exchange +Input [8]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, sum_sales#18, _w0#19] +Arguments: hashpartitioning(i_category#3, i_brand#2, s_store_name#13, s_company_name#14, 5), ENSURE_REQUIREMENTS, [plan_id=4] + +(23) Sort [codegen id : 6] +Input [8]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, sum_sales#18, _w0#19] +Arguments: [i_category#3 ASC NULLS FIRST, i_brand#2 ASC NULLS FIRST, s_store_name#13 ASC NULLS FIRST, s_company_name#14 ASC NULLS FIRST, d_year#10 ASC NULLS FIRST, d_moy#11 ASC NULLS FIRST], false, 0 + +(24) Window +Input [8]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, sum_sales#18, _w0#19] +Arguments: [rank(d_year#10, d_moy#11) windowspecdefinition(i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10 ASC NULLS FIRST, d_moy#11 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#20], [i_category#3, i_brand#2, s_store_name#13, s_company_name#14], [d_year#10 ASC NULLS FIRST, d_moy#11 ASC NULLS FIRST] + +(25) Filter [codegen id : 7] +Input [9]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, sum_sales#18, _w0#19, rn#20] +Condition : (isnotnull(d_year#10) AND (d_year#10 = 1999)) + +(26) Window +Input [9]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, sum_sales#18, _w0#19, rn#20] +Arguments: [avg(_w0#19) windowspecdefinition(i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS avg_monthly_sales#21], [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10] + +(27) Filter [codegen id : 22] +Input [10]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, sum_sales#18, _w0#19, rn#20, avg_monthly_sales#21] +Condition : ((isnotnull(avg_monthly_sales#21) AND (avg_monthly_sales#21 > 0.000000)) AND CASE WHEN (avg_monthly_sales#21 > 0.000000) THEN ((abs((sum_sales#18 - avg_monthly_sales#21)) / avg_monthly_sales#21) > 0.1000000000000000) END) + +(28) Project [codegen id : 22] +Output [9]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, sum_sales#18, avg_monthly_sales#21, rn#20] +Input [10]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, sum_sales#18, _w0#19, rn#20, avg_monthly_sales#21] + +(29) ReusedExchange [Reuses operator id: 20] +Output [7]: [i_category#22, i_brand#23, s_store_name#24, s_company_name#25, d_year#26, d_moy#27, sum#28] + +(30) HashAggregate [codegen id : 12] +Input [7]: [i_category#22, i_brand#23, s_store_name#24, s_company_name#25, d_year#26, d_moy#27, sum#28] +Keys [6]: [i_category#22, i_brand#23, s_store_name#24, s_company_name#25, d_year#26, d_moy#27] +Functions [1]: [sum(UnscaledValue(ss_sales_price#29))] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_sales_price#29))#17] +Results [7]: [i_category#22, i_brand#23, s_store_name#24, s_company_name#25, d_year#26, d_moy#27, MakeDecimal(sum(UnscaledValue(ss_sales_price#29))#17,17,2) AS sum_sales#30] + +(31) Exchange +Input [7]: [i_category#22, i_brand#23, s_store_name#24, s_company_name#25, d_year#26, d_moy#27, sum_sales#30] +Arguments: hashpartitioning(i_category#22, i_brand#23, s_store_name#24, s_company_name#25, 5), ENSURE_REQUIREMENTS, [plan_id=5] + +(32) Sort [codegen id : 13] +Input [7]: [i_category#22, i_brand#23, s_store_name#24, s_company_name#25, d_year#26, d_moy#27, sum_sales#30] +Arguments: [i_category#22 ASC NULLS FIRST, i_brand#23 ASC NULLS FIRST, s_store_name#24 ASC NULLS FIRST, s_company_name#25 ASC NULLS FIRST, d_year#26 ASC NULLS FIRST, d_moy#27 ASC NULLS FIRST], false, 0 + +(33) Window +Input [7]: [i_category#22, i_brand#23, s_store_name#24, s_company_name#25, d_year#26, d_moy#27, sum_sales#30] +Arguments: [rank(d_year#26, d_moy#27) windowspecdefinition(i_category#22, i_brand#23, s_store_name#24, s_company_name#25, d_year#26 ASC NULLS FIRST, d_moy#27 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#31], [i_category#22, i_brand#23, s_store_name#24, s_company_name#25], [d_year#26 ASC NULLS FIRST, d_moy#27 ASC NULLS FIRST] + +(34) Project [codegen id : 14] +Output [6]: [i_category#22, i_brand#23, s_store_name#24, s_company_name#25, sum_sales#30, rn#31] +Input [8]: [i_category#22, i_brand#23, s_store_name#24, s_company_name#25, d_year#26, d_moy#27, sum_sales#30, rn#31] + +(35) BroadcastExchange +Input [6]: [i_category#22, i_brand#23, s_store_name#24, s_company_name#25, sum_sales#30, rn#31] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], input[3, string, true], (input[5, int, false] + 1)),false), [plan_id=6] + +(36) BroadcastHashJoin [codegen id : 22] +Left keys [5]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, rn#20] +Right keys [5]: [i_category#22, i_brand#23, s_store_name#24, s_company_name#25, (rn#31 + 1)] +Join type: Inner +Join condition: None + +(37) Project [codegen id : 22] +Output [10]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, sum_sales#18, avg_monthly_sales#21, rn#20, sum_sales#30] +Input [15]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, sum_sales#18, avg_monthly_sales#21, rn#20, i_category#22, i_brand#23, s_store_name#24, s_company_name#25, sum_sales#30, rn#31] + +(38) ReusedExchange [Reuses operator id: 31] +Output [7]: [i_category#32, i_brand#33, s_store_name#34, s_company_name#35, d_year#36, d_moy#37, sum_sales#38] + +(39) Sort [codegen id : 20] +Input [7]: [i_category#32, i_brand#33, s_store_name#34, s_company_name#35, d_year#36, d_moy#37, sum_sales#38] +Arguments: [i_category#32 ASC NULLS FIRST, i_brand#33 ASC NULLS FIRST, s_store_name#34 ASC NULLS FIRST, s_company_name#35 ASC NULLS FIRST, d_year#36 ASC NULLS FIRST, d_moy#37 ASC NULLS FIRST], false, 0 + +(40) Window +Input [7]: [i_category#32, i_brand#33, s_store_name#34, s_company_name#35, d_year#36, d_moy#37, sum_sales#38] +Arguments: [rank(d_year#36, d_moy#37) windowspecdefinition(i_category#32, i_brand#33, s_store_name#34, s_company_name#35, d_year#36 ASC NULLS FIRST, d_moy#37 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#39], [i_category#32, i_brand#33, s_store_name#34, s_company_name#35], [d_year#36 ASC NULLS FIRST, d_moy#37 ASC NULLS FIRST] + +(41) Project [codegen id : 21] +Output [6]: [i_category#32, i_brand#33, s_store_name#34, s_company_name#35, sum_sales#38, rn#39] +Input [8]: [i_category#32, i_brand#33, s_store_name#34, s_company_name#35, d_year#36, d_moy#37, sum_sales#38, rn#39] + +(42) BroadcastExchange +Input [6]: [i_category#32, i_brand#33, s_store_name#34, s_company_name#35, sum_sales#38, rn#39] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], input[3, string, true], (input[5, int, false] - 1)),false), [plan_id=7] + +(43) BroadcastHashJoin [codegen id : 22] +Left keys [5]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, rn#20] +Right keys [5]: [i_category#32, i_brand#33, s_store_name#34, s_company_name#35, (rn#39 - 1)] +Join type: Inner +Join condition: None + +(44) Project [codegen id : 22] +Output [10]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, avg_monthly_sales#21, sum_sales#18, sum_sales#30 AS psum#40, sum_sales#38 AS nsum#41] +Input [16]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, sum_sales#18, avg_monthly_sales#21, rn#20, sum_sales#30, i_category#32, i_brand#33, s_store_name#34, s_company_name#35, sum_sales#38, rn#39] + +(45) TakeOrderedAndProject +Input [10]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, avg_monthly_sales#21, sum_sales#18, psum#40, nsum#41] +Arguments: 100, [(sum_sales#18 - avg_monthly_sales#21) ASC NULLS FIRST, s_store_name#13 ASC NULLS FIRST], [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, avg_monthly_sales#21, sum_sales#18, psum#40, nsum#41] + +===== Subqueries ===== + +Subquery:1 Hosting operator id = 4 Hosting Expression = ss_sold_date_sk#7 IN dynamicpruning#8 +BroadcastExchange (49) ++- * ColumnarToRow (48) + +- CometFilter (47) + +- CometScan parquet spark_catalog.default.date_dim (46) + + +(46) Scan parquet spark_catalog.default.date_dim +Output [3]: [d_date_sk#9, d_year#10, d_moy#11] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [Or(Or(EqualTo(d_year,1999),And(EqualTo(d_year,1998),EqualTo(d_moy,12))),And(EqualTo(d_year,2000),EqualTo(d_moy,1))), IsNotNull(d_date_sk)] +ReadSchema: struct<d_date_sk:int,d_year:int,d_moy:int> + +(47) CometFilter +Input [3]: [d_date_sk#9, d_year#10, d_moy#11] +Condition : ((((d_year#10 = 1999) OR ((d_year#10 = 1998) AND (d_moy#11 = 12))) OR ((d_year#10 = 2000) AND (d_moy#11 = 1))) AND isnotnull(d_date_sk#9)) + +(48) ColumnarToRow [codegen id : 1] +Input [3]: [d_date_sk#9, d_year#10, d_moy#11] + +(49) BroadcastExchange +Input [3]: [d_date_sk#9, d_year#10, d_moy#11] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=8] + + 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 new file mode 100644 index 000000000..80b8da7b1 --- /dev/null +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q47/simplified.txt @@ -0,0 +1,81 @@ +TakeOrderedAndProject [sum_sales,avg_monthly_sales,s_store_name,i_category,i_brand,s_company_name,d_year,d_moy,psum,nsum] + WholeStageCodegen (22) + 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] + 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] + Filter [avg_monthly_sales,sum_sales] + InputAdapter + Window [_w0,i_category,i_brand,s_store_name,s_company_name,d_year] + WholeStageCodegen (7) + Filter [d_year] + InputAdapter + Window [d_year,d_moy,i_category,i_brand,s_store_name,s_company_name] + WholeStageCodegen (6) + Sort [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy] + InputAdapter + Exchange [i_category,i_brand,s_store_name,s_company_name] #1 + WholeStageCodegen (5) + HashAggregate [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum] [sum(UnscaledValue(ss_sales_price)),sum_sales,_w0,sum] + InputAdapter + Exchange [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy] #2 + WholeStageCodegen (4) + HashAggregate [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,ss_sales_price] [sum,sum] + Project [i_brand,i_category,ss_sales_price,d_year,d_moy,s_store_name,s_company_name] + BroadcastHashJoin [ss_store_sk,s_store_sk] + Project [i_brand,i_category,ss_store_sk,ss_sales_price,d_year,d_moy] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [i_brand,i_category,ss_store_sk,ss_sales_price,ss_sold_date_sk] + BroadcastHashJoin [i_item_sk,ss_item_sk] + ColumnarToRow + InputAdapter + CometFilter [i_item_sk,i_category,i_brand] + CometScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_category] + InputAdapter + BroadcastExchange #3 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometFilter [ss_item_sk,ss_store_sk] + CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #4 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometFilter [d_year,d_moy,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + InputAdapter + ReusedExchange [d_date_sk,d_year,d_moy] #4 + InputAdapter + BroadcastExchange #5 + WholeStageCodegen (3) + ColumnarToRow + InputAdapter + CometFilter [s_store_sk,s_store_name,s_company_name] + CometScan parquet spark_catalog.default.store [s_store_sk,s_store_name,s_company_name] + InputAdapter + BroadcastExchange #6 + WholeStageCodegen (14) + 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) + Sort [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy] + InputAdapter + Exchange [i_category,i_brand,s_store_name,s_company_name] #7 + WholeStageCodegen (12) + HashAggregate [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum] [sum(UnscaledValue(ss_sales_price)),sum_sales,sum] + InputAdapter + ReusedExchange [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum] #2 + InputAdapter + BroadcastExchange #8 + WholeStageCodegen (21) + 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) + Sort [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy] + InputAdapter + ReusedExchange [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales] #7 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 new file mode 100644 index 000000000..718f3fb31 --- /dev/null +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q48/explain.txt @@ -0,0 +1,198 @@ +== Physical Plan == +* HashAggregate (28) ++- Exchange (27) + +- * HashAggregate (26) + +- * Project (25) + +- * BroadcastHashJoin Inner BuildRight (24) + :- * Project (22) + : +- * BroadcastHashJoin Inner BuildRight (21) + : :- * Project (15) + : : +- * BroadcastHashJoin Inner BuildRight (14) + : : :- * Project (9) + : : : +- * BroadcastHashJoin Inner BuildRight (8) + : : : :- * ColumnarToRow (3) + : : : : +- CometFilter (2) + : : : : +- CometScan parquet spark_catalog.default.store_sales (1) + : : : +- BroadcastExchange (7) + : : : +- * ColumnarToRow (6) + : : : +- CometFilter (5) + : : : +- CometScan parquet spark_catalog.default.store (4) + : : +- BroadcastExchange (13) + : : +- * ColumnarToRow (12) + : : +- CometFilter (11) + : : +- CometScan parquet spark_catalog.default.customer_demographics (10) + : +- BroadcastExchange (20) + : +- * ColumnarToRow (19) + : +- CometProject (18) + : +- CometFilter (17) + : +- CometScan parquet spark_catalog.default.customer_address (16) + +- ReusedExchange (23) + + +(1) Scan parquet spark_catalog.default.store_sales +Output [7]: [ss_cdemo_sk#1, ss_addr_sk#2, ss_store_sk#3, ss_quantity#4, ss_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7] +Batched: true +Location: InMemoryFileIndex [] +PartitionFilters: [isnotnull(ss_sold_date_sk#7), dynamicpruningexpression(ss_sold_date_sk#7 IN dynamicpruning#8)] +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<ss_cdemo_sk:int,ss_addr_sk:int,ss_store_sk:int,ss_quantity:int,ss_sales_price:decimal(7,2),ss_net_profit:decimal(7,2)> + +(2) CometFilter +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)))) + +(3) ColumnarToRow [codegen id : 5] +Input [7]: [ss_cdemo_sk#1, ss_addr_sk#2, ss_store_sk#3, ss_quantity#4, ss_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7] + +(4) Scan parquet spark_catalog.default.store +Output [1]: [s_store_sk#9] +Batched: true +Location [not included in comparison]/{warehouse_dir}/store] +PushedFilters: [IsNotNull(s_store_sk)] +ReadSchema: struct<s_store_sk:int> + +(5) CometFilter +Input [1]: [s_store_sk#9] +Condition : isnotnull(s_store_sk#9) + +(6) ColumnarToRow [codegen id : 1] +Input [1]: [s_store_sk#9] + +(7) BroadcastExchange +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] +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] +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] + +(10) Scan parquet spark_catalog.default.customer_demographics +Output [3]: [cd_demo_sk#10, cd_marital_status#11, cd_education_status#12] +Batched: true +Location [not included in comparison]/{warehouse_dir}/customer_demographics] +PushedFilters: [IsNotNull(cd_demo_sk), Or(Or(And(EqualTo(cd_marital_status,M),EqualTo(cd_education_status,4 yr Degree )),And(EqualTo(cd_marital_status,D),EqualTo(cd_education_status,2 yr Degree ))),And(EqualTo(cd_marital_status,S),EqualTo(cd_education_status,College )))] +ReadSchema: struct<cd_demo_sk:int,cd_marital_status:string,cd_education_status:string> + +(11) CometFilter +Input [3]: [cd_demo_sk#10, cd_marital_status#11, cd_education_status#12] +Condition : (isnotnull(cd_demo_sk#10) AND ((((cd_marital_status#11 = M) AND (cd_education_status#12 = 4 yr Degree )) OR ((cd_marital_status#11 = D) AND (cd_education_status#12 = 2 yr Degree ))) OR ((cd_marital_status#11 = S) AND (cd_education_status#12 = College )))) + +(12) ColumnarToRow [codegen id : 2] +Input [3]: [cd_demo_sk#10, cd_marital_status#11, cd_education_status#12] + +(13) BroadcastExchange +Input [3]: [cd_demo_sk#10, cd_marital_status#11, cd_education_status#12] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=2] + +(14) BroadcastHashJoin [codegen id : 5] +Left keys [1]: [ss_cdemo_sk#1] +Right keys [1]: [cd_demo_sk#10] +Join type: Inner +Join condition: ((((((cd_marital_status#11 = M) AND (cd_education_status#12 = 4 yr Degree )) AND (ss_sales_price#5 >= 100.00)) AND (ss_sales_price#5 <= 150.00)) OR ((((cd_marital_status#11 = D) AND (cd_education_status#12 = 2 yr Degree )) AND (ss_sales_price#5 >= 50.00)) AND (ss_sales_price#5 <= 100.00))) OR ((((cd_marital_status#11 = S) AND (cd_education_status#12 = College )) AND (ss_sales_price#5 >= 150.00)) AND (ss_sales_price#5 <= 200.00))) + +(15) Project [codegen id : 5] +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#11, cd_education_status#12] + +(16) Scan parquet spark_catalog.default.customer_address +Output [3]: [ca_address_sk#13, ca_state#14, ca_country#15] +Batched: true +Location [not included in comparison]/{warehouse_dir}/customer_address] +PushedFilters: [IsNotNull(ca_country), EqualTo(ca_country,United States), IsNotNull(ca_address_sk), Or(Or(In(ca_state, [CO,OH,TX]),In(ca_state, [KY,MN,OR])),In(ca_state, [CA,MS,VA]))] +ReadSchema: struct<ca_address_sk:int,ca_state:string,ca_country:string> + +(17) CometFilter +Input [3]: [ca_address_sk#13, ca_state#14, ca_country#15] +Condition : (((isnotnull(ca_country#15) AND (ca_country#15 = United States)) AND isnotnull(ca_address_sk#13)) AND ((ca_state#14 IN (CO,OH,TX) OR ca_state#14 IN (OR,MN,KY)) OR ca_state#14 IN (VA,CA,MS))) + +(18) CometProject +Input [3]: [ca_address_sk#13, ca_state#14, ca_country#15] +Arguments: [ca_address_sk#13, ca_state#14], [ca_address_sk#13, ca_state#14] + +(19) ColumnarToRow [codegen id : 3] +Input [2]: [ca_address_sk#13, ca_state#14] + +(20) BroadcastExchange +Input [2]: [ca_address_sk#13, ca_state#14] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] + +(21) BroadcastHashJoin [codegen id : 5] +Left keys [1]: [ss_addr_sk#2] +Right keys [1]: [ca_address_sk#13] +Join type: Inner +Join condition: ((((ca_state#14 IN (CO,OH,TX) AND (ss_net_profit#6 >= 0.00)) AND (ss_net_profit#6 <= 2000.00)) OR ((ca_state#14 IN (OR,MN,KY) AND (ss_net_profit#6 >= 150.00)) AND (ss_net_profit#6 <= 3000.00))) OR ((ca_state#14 IN (VA,CA,MS) AND (ss_net_profit#6 >= 50.00)) AND (ss_net_profit#6 <= 25000.00))) + +(22) Project [codegen id : 5] +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#13, ca_state#14] + +(23) ReusedExchange [Reuses operator id: 33] +Output [1]: [d_date_sk#16] + +(24) BroadcastHashJoin [codegen id : 5] +Left keys [1]: [ss_sold_date_sk#7] +Right keys [1]: [d_date_sk#16] +Join type: Inner +Join condition: None + +(25) Project [codegen id : 5] +Output [1]: [ss_quantity#4] +Input [3]: [ss_quantity#4, ss_sold_date_sk#7, d_date_sk#16] + +(26) HashAggregate [codegen id : 5] +Input [1]: [ss_quantity#4] +Keys: [] +Functions [1]: [partial_sum(ss_quantity#4)] +Aggregate Attributes [1]: [sum#17] +Results [1]: [sum#18] + +(27) Exchange +Input [1]: [sum#18] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=4] + +(28) HashAggregate [codegen id : 6] +Input [1]: [sum#18] +Keys: [] +Functions [1]: [sum(ss_quantity#4)] +Aggregate Attributes [1]: [sum(ss_quantity#4)#19] +Results [1]: [sum(ss_quantity#4)#19 AS sum(ss_quantity)#20] + +===== Subqueries ===== + +Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#7 IN dynamicpruning#8 +BroadcastExchange (33) ++- * ColumnarToRow (32) + +- CometProject (31) + +- CometFilter (30) + +- CometScan parquet spark_catalog.default.date_dim (29) + + +(29) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#16, d_year#21] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), IsNotNull(d_date_sk)] +ReadSchema: struct<d_date_sk:int,d_year:int> + +(30) CometFilter +Input [2]: [d_date_sk#16, d_year#21] +Condition : ((isnotnull(d_year#21) AND (d_year#21 = 2001)) AND isnotnull(d_date_sk#16)) + +(31) CometProject +Input [2]: [d_date_sk#16, d_year#21] +Arguments: [d_date_sk#16], [d_date_sk#16] + +(32) ColumnarToRow [codegen id : 1] +Input [1]: [d_date_sk#16] + +(33) BroadcastExchange +Input [1]: [d_date_sk#16] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] + + 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 new file mode 100644 index 000000000..4022da74f --- /dev/null +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q48/simplified.txt @@ -0,0 +1,50 @@ +WholeStageCodegen (6) + HashAggregate [sum] [sum(ss_quantity),sum(ss_quantity),sum] + InputAdapter + Exchange #1 + WholeStageCodegen (5) + HashAggregate [ss_quantity] [sum,sum] + Project [ss_quantity] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [ss_quantity,ss_sold_date_sk] + BroadcastHashJoin [ss_addr_sk,ca_address_sk,ca_state,ss_net_profit] + Project [ss_addr_sk,ss_quantity,ss_net_profit,ss_sold_date_sk] + BroadcastHashJoin [ss_cdemo_sk,cd_demo_sk,cd_marital_status,cd_education_status,ss_sales_price] + Project [ss_cdemo_sk,ss_addr_sk,ss_quantity,ss_sales_price,ss_net_profit,ss_sold_date_sk] + BroadcastHashJoin [ss_store_sk,s_store_sk] + ColumnarToRow + InputAdapter + CometFilter [ss_store_sk,ss_cdemo_sk,ss_addr_sk,ss_sales_price,ss_net_profit] + CometScan parquet spark_catalog.default.store_sales [ss_cdemo_sk,ss_addr_sk,ss_store_sk,ss_quantity,ss_sales_price,ss_net_profit,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #2 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_year,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + InputAdapter + BroadcastExchange #3 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometFilter [s_store_sk] + CometScan parquet spark_catalog.default.store [s_store_sk] + InputAdapter + BroadcastExchange #4 + WholeStageCodegen (2) + ColumnarToRow + InputAdapter + CometFilter [cd_demo_sk,cd_marital_status,cd_education_status] + CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_marital_status,cd_education_status] + InputAdapter + BroadcastExchange #5 + WholeStageCodegen (3) + ColumnarToRow + InputAdapter + CometProject [ca_address_sk,ca_state] + CometFilter [ca_country,ca_address_sk,ca_state] + CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state,ca_country] + InputAdapter + ReusedExchange [d_date_sk] #2 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 new file mode 100644 index 000000000..bbb550e05 --- /dev/null +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q49/explain.txt @@ -0,0 +1,456 @@ +== Physical Plan == +TakeOrderedAndProject (74) ++- * HashAggregate (73) + +- Exchange (72) + +- * HashAggregate (71) + +- Union (70) + :- * Project (23) + : +- * Filter (22) + : +- Window (21) + : +- * Sort (20) + : +- Window (19) + : +- * Sort (18) + : +- Exchange (17) + : +- * HashAggregate (16) + : +- Exchange (15) + : +- * HashAggregate (14) + : +- * Project (13) + : +- * BroadcastHashJoin Inner BuildRight (12) + : :- * ColumnarToRow (10) + : : +- CometProject (9) + : : +- CometBroadcastHashJoin (8) + : : :- CometBroadcastExchange (4) + : : : +- CometProject (3) + : : : +- CometFilter (2) + : : : +- CometScan parquet spark_catalog.default.web_sales (1) + : : +- CometProject (7) + : : +- CometFilter (6) + : : +- CometScan parquet spark_catalog.default.web_returns (5) + : +- ReusedExchange (11) + :- * Project (46) + : +- * Filter (45) + : +- Window (44) + : +- * Sort (43) + : +- Window (42) + : +- * Sort (41) + : +- Exchange (40) + : +- * HashAggregate (39) + : +- Exchange (38) + : +- * HashAggregate (37) + : +- * Project (36) + : +- * BroadcastHashJoin Inner BuildRight (35) + : :- * ColumnarToRow (33) + : : +- CometProject (32) + : : +- CometBroadcastHashJoin (31) + : : :- CometBroadcastExchange (27) + : : : +- CometProject (26) + : : : +- CometFilter (25) + : : : +- CometScan parquet spark_catalog.default.catalog_sales (24) + : : +- CometProject (30) + : : +- CometFilter (29) + : : +- CometScan parquet spark_catalog.default.catalog_returns (28) + : +- ReusedExchange (34) + +- * Project (69) + +- * Filter (68) + +- Window (67) + +- * Sort (66) + +- Window (65) + +- * Sort (64) + +- Exchange (63) + +- * HashAggregate (62) + +- Exchange (61) + +- * HashAggregate (60) + +- * Project (59) + +- * BroadcastHashJoin Inner BuildRight (58) + :- * ColumnarToRow (56) + : +- CometProject (55) + : +- CometBroadcastHashJoin (54) + : :- CometBroadcastExchange (50) + : : +- CometProject (49) + : : +- CometFilter (48) + : : +- CometScan parquet spark_catalog.default.store_sales (47) + : +- CometProject (53) + : +- CometFilter (52) + : +- CometScan parquet spark_catalog.default.store_returns (51) + +- ReusedExchange (57) + + +(1) Scan parquet spark_catalog.default.web_sales +Output [6]: [ws_item_sk#1, ws_order_number#2, ws_quantity#3, ws_net_paid#4, ws_net_profit#5, ws_sold_date_sk#6] +Batched: true +Location: InMemoryFileIndex [] +PartitionFilters: [isnotnull(ws_sold_date_sk#6), dynamicpruningexpression(ws_sold_date_sk#6 IN dynamicpruning#7)] +PushedFilters: [IsNotNull(ws_net_profit), IsNotNull(ws_net_paid), IsNotNull(ws_quantity), GreaterThan(ws_net_profit,1.00), GreaterThan(ws_net_paid,0.00), GreaterThan(ws_quantity,0), IsNotNull(ws_order_number), IsNotNull(ws_item_sk)] +ReadSchema: struct<ws_item_sk:int,ws_order_number:int,ws_quantity:int,ws_net_paid:decimal(7,2),ws_net_profit:decimal(7,2)> + +(2) CometFilter +Input [6]: [ws_item_sk#1, ws_order_number#2, ws_quantity#3, ws_net_paid#4, ws_net_profit#5, ws_sold_date_sk#6] +Condition : (((((((isnotnull(ws_net_profit#5) AND isnotnull(ws_net_paid#4)) AND isnotnull(ws_quantity#3)) AND (ws_net_profit#5 > 1.00)) AND (ws_net_paid#4 > 0.00)) AND (ws_quantity#3 > 0)) AND isnotnull(ws_order_number#2)) AND isnotnull(ws_item_sk#1)) + +(3) CometProject +Input [6]: [ws_item_sk#1, ws_order_number#2, ws_quantity#3, ws_net_paid#4, ws_net_profit#5, ws_sold_date_sk#6] +Arguments: [ws_item_sk#1, ws_order_number#2, ws_quantity#3, ws_net_paid#4, ws_sold_date_sk#6], [ws_item_sk#1, ws_order_number#2, ws_quantity#3, ws_net_paid#4, ws_sold_date_sk#6] + +(4) CometBroadcastExchange +Input [5]: [ws_item_sk#1, ws_order_number#2, ws_quantity#3, ws_net_paid#4, ws_sold_date_sk#6] +Arguments: [ws_item_sk#1, ws_order_number#2, ws_quantity#3, ws_net_paid#4, ws_sold_date_sk#6] + +(5) Scan parquet spark_catalog.default.web_returns +Output [5]: [wr_item_sk#8, wr_order_number#9, wr_return_quantity#10, wr_return_amt#11, wr_returned_date_sk#12] +Batched: true +Location [not included in comparison]/{warehouse_dir}/web_returns] +PushedFilters: [IsNotNull(wr_return_amt), GreaterThan(wr_return_amt,10000.00), IsNotNull(wr_order_number), IsNotNull(wr_item_sk)] +ReadSchema: struct<wr_item_sk:int,wr_order_number:int,wr_return_quantity:int,wr_return_amt:decimal(7,2)> + +(6) CometFilter +Input [5]: [wr_item_sk#8, wr_order_number#9, wr_return_quantity#10, wr_return_amt#11, wr_returned_date_sk#12] +Condition : (((isnotnull(wr_return_amt#11) AND (wr_return_amt#11 > 10000.00)) AND isnotnull(wr_order_number#9)) AND isnotnull(wr_item_sk#8)) + +(7) CometProject +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] + +(8) CometBroadcastHashJoin +Left output [5]: [ws_item_sk#1, ws_order_number#2, ws_quantity#3, ws_net_paid#4, ws_sold_date_sk#6] +Right output [4]: [wr_item_sk#8, wr_order_number#9, wr_return_quantity#10, wr_return_amt#11] +Arguments: [ws_order_number#2, ws_item_sk#1], [wr_order_number#9, wr_item_sk#8], Inner + +(9) CometProject +Input [9]: [ws_item_sk#1, ws_order_number#2, ws_quantity#3, ws_net_paid#4, ws_sold_date_sk#6, wr_item_sk#8, wr_order_number#9, wr_return_quantity#10, wr_return_amt#11] +Arguments: [ws_item_sk#1, ws_quantity#3, ws_net_paid#4, ws_sold_date_sk#6, wr_return_quantity#10, wr_return_amt#11], [ws_item_sk#1, ws_quantity#3, ws_net_paid#4, ws_sold_date_sk#6, wr_return_quantity#10, wr_return_amt#11] + +(10) ColumnarToRow [codegen id : 2] +Input [6]: [ws_item_sk#1, ws_quantity#3, ws_net_paid#4, ws_sold_date_sk#6, wr_return_quantity#10, wr_return_amt#11] + +(11) ReusedExchange [Reuses operator id: 79] +Output [1]: [d_date_sk#13] + +(12) BroadcastHashJoin [codegen id : 2] +Left keys [1]: [ws_sold_date_sk#6] +Right keys [1]: [d_date_sk#13] +Join type: Inner +Join condition: None + +(13) Project [codegen id : 2] +Output [5]: [ws_item_sk#1, ws_quantity#3, ws_net_paid#4, wr_return_quantity#10, wr_return_amt#11] +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] + +(14) 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))] +Aggregate Attributes [6]: [sum#14, sum#15, sum#16, isEmpty#17, sum#18, isEmpty#19] +Results [7]: [ws_item_sk#1, sum#20, sum#21, sum#22, isEmpty#23, sum#24, isEmpty#25] + +(15) Exchange +Input [7]: [ws_item_sk#1, sum#20, sum#21, sum#22, isEmpty#23, sum#24, isEmpty#25] +Arguments: hashpartitioning(ws_item_sk#1, 5), ENSURE_REQUIREMENTS, [plan_id=1] + +(16) HashAggregate [codegen id : 3] +Input [7]: [ws_item_sk#1, sum#20, sum#21, sum#22, isEmpty#23, sum#24, isEmpty#25] +Keys [1]: [ws_item_sk#1] +Functions [4]: [sum(coalesce(wr_return_quantity#10, 0)), sum(coalesce(ws_quantity#3, 0)), sum(coalesce(cast(wr_return_amt#11 as decimal(12,2)), 0.00)), sum(coalesce(cast(ws_net_paid#4 as decimal(12,2)), 0.00))] +Aggregate Attributes [4]: [sum(coalesce(wr_return_quantity#10, 0))#26, sum(coalesce(ws_quantity#3, 0))#27, sum(coalesce(cast(wr_return_amt#11 as decimal(12,2)), 0.00))#28, sum(coalesce(cast(ws_net_paid#4 as decimal(12,2)), 0.00))#29] +Results [3]: [ws_item_sk#1 AS item#30, (cast(sum(coalesce(wr_return_quantity#10, 0))#26 as decimal(15,4)) / cast(sum(coalesce(ws_quantity#3, 0))#27 as decimal(15,4))) AS return_ratio#31, (cast(sum(coalesce(cast(wr_return_amt#11 as decimal(12,2)), 0.00))#28 as decimal(15,4)) / cast(sum(coalesce(cast(ws_net_paid#4 as decimal(12,2)), 0.00))#29 as decimal(15,4))) AS currency_ratio#32] + +(17) Exchange +Input [3]: [item#30, return_ratio#31, currency_ratio#32] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=2] + +(18) Sort [codegen id : 4] +Input [3]: [item#30, return_ratio#31, currency_ratio#32] +Arguments: [return_ratio#31 ASC NULLS FIRST], false, 0 + +(19) Window +Input [3]: [item#30, return_ratio#31, currency_ratio#32] +Arguments: [rank(return_ratio#31) windowspecdefinition(return_ratio#31 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS return_rank#33], [return_ratio#31 ASC NULLS FIRST] + +(20) Sort [codegen id : 5] +Input [4]: [item#30, return_ratio#31, currency_ratio#32, return_rank#33] +Arguments: [currency_ratio#32 ASC NULLS FIRST], false, 0 + +(21) Window +Input [4]: [item#30, return_ratio#31, currency_ratio#32, return_rank#33] +Arguments: [rank(currency_ratio#32) windowspecdefinition(currency_ratio#32 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS currency_rank#34], [currency_ratio#32 ASC NULLS FIRST] + +(22) Filter [codegen id : 6] +Input [5]: [item#30, return_ratio#31, currency_ratio#32, return_rank#33, currency_rank#34] +Condition : ((return_rank#33 <= 10) OR (currency_rank#34 <= 10)) + +(23) Project [codegen id : 6] +Output [5]: [web AS channel#35, item#30, return_ratio#31, return_rank#33, currency_rank#34] +Input [5]: [item#30, return_ratio#31, currency_ratio#32, return_rank#33, currency_rank#34] + +(24) Scan parquet spark_catalog.default.catalog_sales +Output [6]: [cs_item_sk#36, cs_order_number#37, cs_quantity#38, cs_net_paid#39, cs_net_profit#40, cs_sold_date_sk#41] +Batched: true +Location: InMemoryFileIndex [] +PartitionFilters: [isnotnull(cs_sold_date_sk#41), dynamicpruningexpression(cs_sold_date_sk#41 IN dynamicpruning#42)] +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<cs_item_sk:int,cs_order_number:int,cs_quantity:int,cs_net_paid:decimal(7,2),cs_net_profit:decimal(7,2)> + +(25) CometFilter +Input [6]: [cs_item_sk#36, cs_order_number#37, cs_quantity#38, cs_net_paid#39, cs_net_profit#40, cs_sold_date_sk#41] +Condition : (((((((isnotnull(cs_net_profit#40) AND isnotnull(cs_net_paid#39)) AND isnotnull(cs_quantity#38)) AND (cs_net_profit#40 > 1.00)) AND (cs_net_paid#39 > 0.00)) AND (cs_quantity#38 > 0)) AND isnotnull(cs_order_number#37)) AND isnotnull(cs_item_sk#36)) + +(26) CometProject +Input [6]: [cs_item_sk#36, cs_order_number#37, cs_quantity#38, cs_net_paid#39, cs_net_profit#40, cs_sold_date_sk#41] +Arguments: [cs_item_sk#36, cs_order_number#37, cs_quantity#38, cs_net_paid#39, cs_sold_date_sk#41], [cs_item_sk#36, cs_order_number#37, cs_quantity#38, cs_net_paid#39, cs_sold_date_sk#41] + +(27) CometBroadcastExchange +Input [5]: [cs_item_sk#36, cs_order_number#37, cs_quantity#38, cs_net_paid#39, cs_sold_date_sk#41] +Arguments: [cs_item_sk#36, cs_order_number#37, cs_quantity#38, cs_net_paid#39, cs_sold_date_sk#41] + +(28) Scan parquet spark_catalog.default.catalog_returns +Output [5]: [cr_item_sk#43, cr_order_number#44, cr_return_quantity#45, cr_return_amount#46, cr_returned_date_sk#47] +Batched: true +Location [not included in comparison]/{warehouse_dir}/catalog_returns] +PushedFilters: [IsNotNull(cr_return_amount), GreaterThan(cr_return_amount,10000.00), IsNotNull(cr_order_number), IsNotNull(cr_item_sk)] +ReadSchema: struct<cr_item_sk:int,cr_order_number:int,cr_return_quantity:int,cr_return_amount:decimal(7,2)> + +(29) CometFilter +Input [5]: [cr_item_sk#43, cr_order_number#44, cr_return_quantity#45, cr_return_amount#46, cr_returned_date_sk#47] +Condition : (((isnotnull(cr_return_amount#46) AND (cr_return_amount#46 > 10000.00)) AND isnotnull(cr_order_number#44)) AND isnotnull(cr_item_sk#43)) + +(30) CometProject +Input [5]: [cr_item_sk#43, cr_order_number#44, cr_return_quantity#45, cr_return_amount#46, cr_returned_date_sk#47] +Arguments: [cr_item_sk#43, cr_order_number#44, cr_return_quantity#45, cr_return_amount#46], [cr_item_sk#43, cr_order_number#44, cr_return_quantity#45, cr_return_amount#46] + +(31) CometBroadcastHashJoin +Left output [5]: [cs_item_sk#36, cs_order_number#37, cs_quantity#38, cs_net_paid#39, cs_sold_date_sk#41] +Right output [4]: [cr_item_sk#43, cr_order_number#44, cr_return_quantity#45, cr_return_amount#46] +Arguments: [cs_order_number#37, cs_item_sk#36], [cr_order_number#44, cr_item_sk#43], Inner + +(32) CometProject +Input [9]: [cs_item_sk#36, cs_order_number#37, cs_quantity#38, cs_net_paid#39, cs_sold_date_sk#41, cr_item_sk#43, cr_order_number#44, cr_return_quantity#45, cr_return_amount#46] +Arguments: [cs_item_sk#36, cs_quantity#38, cs_net_paid#39, cs_sold_date_sk#41, cr_return_quantity#45, cr_return_amount#46], [cs_item_sk#36, cs_quantity#38, cs_net_paid#39, cs_sold_date_sk#41, cr_return_quantity#45, cr_return_amount#46] + +(33) ColumnarToRow [codegen id : 8] +Input [6]: [cs_item_sk#36, cs_quantity#38, cs_net_paid#39, cs_sold_date_sk#41, cr_return_quantity#45, cr_return_amount#46] + +(34) ReusedExchange [Reuses operator id: 79] +Output [1]: [d_date_sk#48] + +(35) BroadcastHashJoin [codegen id : 8] +Left keys [1]: [cs_sold_date_sk#41] +Right keys [1]: [d_date_sk#48] +Join type: Inner +Join condition: None + +(36) Project [codegen id : 8] +Output [5]: [cs_item_sk#36, cs_quantity#38, cs_net_paid#39, cr_return_quantity#45, cr_return_amount#46] +Input [7]: [cs_item_sk#36, cs_quantity#38, cs_net_paid#39, cs_sold_date_sk#41, cr_return_quantity#45, cr_return_amount#46, d_date_sk#48] + +(37) HashAggregate [codegen id : 8] +Input [5]: [cs_item_sk#36, cs_quantity#38, cs_net_paid#39, cr_return_quantity#45, cr_return_amount#46] +Keys [1]: [cs_item_sk#36] +Functions [4]: [partial_sum(coalesce(cr_return_quantity#45, 0)), partial_sum(coalesce(cs_quantity#38, 0)), partial_sum(coalesce(cast(cr_return_amount#46 as decimal(12,2)), 0.00)), partial_sum(coalesce(cast(cs_net_paid#39 as decimal(12,2)), 0.00))] +Aggregate Attributes [6]: [sum#49, sum#50, sum#51, isEmpty#52, sum#53, isEmpty#54] +Results [7]: [cs_item_sk#36, sum#55, sum#56, sum#57, isEmpty#58, sum#59, isEmpty#60] + +(38) Exchange +Input [7]: [cs_item_sk#36, sum#55, sum#56, sum#57, isEmpty#58, sum#59, isEmpty#60] +Arguments: hashpartitioning(cs_item_sk#36, 5), ENSURE_REQUIREMENTS, [plan_id=3] + +(39) HashAggregate [codegen id : 9] +Input [7]: [cs_item_sk#36, sum#55, sum#56, sum#57, isEmpty#58, sum#59, isEmpty#60] +Keys [1]: [cs_item_sk#36] +Functions [4]: [sum(coalesce(cr_return_quantity#45, 0)), sum(coalesce(cs_quantity#38, 0)), sum(coalesce(cast(cr_return_amount#46 as decimal(12,2)), 0.00)), sum(coalesce(cast(cs_net_paid#39 as decimal(12,2)), 0.00))] +Aggregate Attributes [4]: [sum(coalesce(cr_return_quantity#45, 0))#61, sum(coalesce(cs_quantity#38, 0))#62, sum(coalesce(cast(cr_return_amount#46 as decimal(12,2)), 0.00))#63, sum(coalesce(cast(cs_net_paid#39 as decimal(12,2)), 0.00))#64] +Results [3]: [cs_item_sk#36 AS item#65, (cast(sum(coalesce(cr_return_quantity#45, 0))#61 as decimal(15,4)) / cast(sum(coalesce(cs_quantity#38, 0))#62 as decimal(15,4))) AS return_ratio#66, (cast(sum(coalesce(cast(cr_return_amount#46 as decimal(12,2)), 0.00))#63 as decimal(15,4)) / cast(sum(coalesce(cast(cs_net_paid#39 as decimal(12,2)), 0.00))#64 as decimal(15,4))) AS currency_ratio#67] + +(40) Exchange +Input [3]: [item#65, return_ratio#66, currency_ratio#67] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=4] + +(41) Sort [codegen id : 10] +Input [3]: [item#65, return_ratio#66, currency_ratio#67] +Arguments: [return_ratio#66 ASC NULLS FIRST], false, 0 + +(42) Window +Input [3]: [item#65, return_ratio#66, currency_ratio#67] +Arguments: [rank(return_ratio#66) windowspecdefinition(return_ratio#66 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS return_rank#68], [return_ratio#66 ASC NULLS FIRST] + +(43) Sort [codegen id : 11] +Input [4]: [item#65, return_ratio#66, currency_ratio#67, return_rank#68] +Arguments: [currency_ratio#67 ASC NULLS FIRST], false, 0 + +(44) Window +Input [4]: [item#65, return_ratio#66, currency_ratio#67, return_rank#68] +Arguments: [rank(currency_ratio#67) windowspecdefinition(currency_ratio#67 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS currency_rank#69], [currency_ratio#67 ASC NULLS FIRST] + +(45) Filter [codegen id : 12] +Input [5]: [item#65, return_ratio#66, currency_ratio#67, return_rank#68, currency_rank#69] +Condition : ((return_rank#68 <= 10) OR (currency_rank#69 <= 10)) + +(46) Project [codegen id : 12] +Output [5]: [catalog AS channel#70, item#65, return_ratio#66, return_rank#68, currency_rank#69] +Input [5]: [item#65, return_ratio#66, currency_ratio#67, return_rank#68, currency_rank#69] + +(47) Scan parquet spark_catalog.default.store_sales +Output [6]: [ss_item_sk#71, ss_ticket_number#72, ss_quantity#73, ss_net_paid#74, ss_net_profit#75, ss_sold_date_sk#76] +Batched: true +Location: InMemoryFileIndex [] +PartitionFilters: [isnotnull(ss_sold_date_sk#76), dynamicpruningexpression(ss_sold_date_sk#76 IN dynamicpruning#77)] +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<ss_item_sk:int,ss_ticket_number:int,ss_quantity:int,ss_net_paid:decimal(7,2),ss_net_profit:decimal(7,2)> + +(48) CometFilter +Input [6]: [ss_item_sk#71, ss_ticket_number#72, ss_quantity#73, ss_net_paid#74, ss_net_profit#75, ss_sold_date_sk#76] +Condition : (((((((isnotnull(ss_net_profit#75) AND isnotnull(ss_net_paid#74)) AND isnotnull(ss_quantity#73)) AND (ss_net_profit#75 > 1.00)) AND (ss_net_paid#74 > 0.00)) AND (ss_quantity#73 > 0)) AND isnotnull(ss_ticket_number#72)) AND isnotnull(ss_item_sk#71)) + +(49) CometProject +Input [6]: [ss_item_sk#71, ss_ticket_number#72, ss_quantity#73, ss_net_paid#74, ss_net_profit#75, ss_sold_date_sk#76] +Arguments: [ss_item_sk#71, ss_ticket_number#72, ss_quantity#73, ss_net_paid#74, ss_sold_date_sk#76], [ss_item_sk#71, ss_ticket_number#72, ss_quantity#73, ss_net_paid#74, ss_sold_date_sk#76] + +(50) CometBroadcastExchange +Input [5]: [ss_item_sk#71, ss_ticket_number#72, ss_quantity#73, ss_net_paid#74, ss_sold_date_sk#76] +Arguments: [ss_item_sk#71, ss_ticket_number#72, ss_quantity#73, ss_net_paid#74, ss_sold_date_sk#76] + +(51) Scan parquet spark_catalog.default.store_returns +Output [5]: [sr_item_sk#78, sr_ticket_number#79, sr_return_quantity#80, sr_return_amt#81, sr_returned_date_sk#82] +Batched: true +Location [not included in comparison]/{warehouse_dir}/store_returns] +PushedFilters: [IsNotNull(sr_return_amt), GreaterThan(sr_return_amt,10000.00), IsNotNull(sr_ticket_number), IsNotNull(sr_item_sk)] +ReadSchema: struct<sr_item_sk:int,sr_ticket_number:int,sr_return_quantity:int,sr_return_amt:decimal(7,2)> + +(52) CometFilter +Input [5]: [sr_item_sk#78, sr_ticket_number#79, sr_return_quantity#80, sr_return_amt#81, sr_returned_date_sk#82] +Condition : (((isnotnull(sr_return_amt#81) AND (sr_return_amt#81 > 10000.00)) AND isnotnull(sr_ticket_number#79)) AND isnotnull(sr_item_sk#78)) + +(53) CometProject +Input [5]: [sr_item_sk#78, sr_ticket_number#79, sr_return_quantity#80, sr_return_amt#81, sr_returned_date_sk#82] +Arguments: [sr_item_sk#78, sr_ticket_number#79, sr_return_quantity#80, sr_return_amt#81], [sr_item_sk#78, sr_ticket_number#79, sr_return_quantity#80, sr_return_amt#81] + +(54) CometBroadcastHashJoin +Left output [5]: [ss_item_sk#71, ss_ticket_number#72, ss_quantity#73, ss_net_paid#74, ss_sold_date_sk#76] +Right output [4]: [sr_item_sk#78, sr_ticket_number#79, sr_return_quantity#80, sr_return_amt#81] +Arguments: [ss_ticket_number#72, ss_item_sk#71], [sr_ticket_number#79, sr_item_sk#78], Inner + +(55) CometProject +Input [9]: [ss_item_sk#71, ss_ticket_number#72, ss_quantity#73, ss_net_paid#74, ss_sold_date_sk#76, sr_item_sk#78, sr_ticket_number#79, sr_return_quantity#80, sr_return_amt#81] +Arguments: [ss_item_sk#71, ss_quantity#73, ss_net_paid#74, ss_sold_date_sk#76, sr_return_quantity#80, sr_return_amt#81], [ss_item_sk#71, ss_quantity#73, ss_net_paid#74, ss_sold_date_sk#76, sr_return_quantity#80, sr_return_amt#81] + +(56) ColumnarToRow [codegen id : 14] +Input [6]: [ss_item_sk#71, ss_quantity#73, ss_net_paid#74, ss_sold_date_sk#76, sr_return_quantity#80, sr_return_amt#81] + +(57) ReusedExchange [Reuses operator id: 79] +Output [1]: [d_date_sk#83] + +(58) BroadcastHashJoin [codegen id : 14] +Left keys [1]: [ss_sold_date_sk#76] +Right keys [1]: [d_date_sk#83] +Join type: Inner +Join condition: None + +(59) Project [codegen id : 14] +Output [5]: [ss_item_sk#71, ss_quantity#73, ss_net_paid#74, sr_return_quantity#80, sr_return_amt#81] +Input [7]: [ss_item_sk#71, ss_quantity#73, ss_net_paid#74, ss_sold_date_sk#76, sr_return_quantity#80, sr_return_amt#81, d_date_sk#83] + +(60) HashAggregate [codegen id : 14] +Input [5]: [ss_item_sk#71, ss_quantity#73, ss_net_paid#74, sr_return_quantity#80, sr_return_amt#81] +Keys [1]: [ss_item_sk#71] +Functions [4]: [partial_sum(coalesce(sr_return_quantity#80, 0)), partial_sum(coalesce(ss_quantity#73, 0)), partial_sum(coalesce(cast(sr_return_amt#81 as decimal(12,2)), 0.00)), partial_sum(coalesce(cast(ss_net_paid#74 as decimal(12,2)), 0.00))] +Aggregate Attributes [6]: [sum#84, sum#85, sum#86, isEmpty#87, sum#88, isEmpty#89] +Results [7]: [ss_item_sk#71, sum#90, sum#91, sum#92, isEmpty#93, sum#94, isEmpty#95] + +(61) Exchange +Input [7]: [ss_item_sk#71, sum#90, sum#91, sum#92, isEmpty#93, sum#94, isEmpty#95] +Arguments: hashpartitioning(ss_item_sk#71, 5), ENSURE_REQUIREMENTS, [plan_id=5] + +(62) HashAggregate [codegen id : 15] +Input [7]: [ss_item_sk#71, sum#90, sum#91, sum#92, isEmpty#93, sum#94, isEmpty#95] +Keys [1]: [ss_item_sk#71] +Functions [4]: [sum(coalesce(sr_return_quantity#80, 0)), sum(coalesce(ss_quantity#73, 0)), sum(coalesce(cast(sr_return_amt#81 as decimal(12,2)), 0.00)), sum(coalesce(cast(ss_net_paid#74 as decimal(12,2)), 0.00))] +Aggregate Attributes [4]: [sum(coalesce(sr_return_quantity#80, 0))#96, sum(coalesce(ss_quantity#73, 0))#97, sum(coalesce(cast(sr_return_amt#81 as decimal(12,2)), 0.00))#98, sum(coalesce(cast(ss_net_paid#74 as decimal(12,2)), 0.00))#99] +Results [3]: [ss_item_sk#71 AS item#100, (cast(sum(coalesce(sr_return_quantity#80, 0))#96 as decimal(15,4)) / cast(sum(coalesce(ss_quantity#73, 0))#97 as decimal(15,4))) AS return_ratio#101, (cast(sum(coalesce(cast(sr_return_amt#81 as decimal(12,2)), 0.00))#98 as decimal(15,4)) / cast(sum(coalesce(cast(ss_net_paid#74 as decimal(12,2)), 0.00))#99 as decimal(15,4))) AS currency_ratio#102] + +(63) Exchange +Input [3]: [item#100, return_ratio#101, currency_ratio#102] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=6] + +(64) Sort [codegen id : 16] +Input [3]: [item#100, return_ratio#101, currency_ratio#102] +Arguments: [return_ratio#101 ASC NULLS FIRST], false, 0 + +(65) Window +Input [3]: [item#100, return_ratio#101, currency_ratio#102] +Arguments: [rank(return_ratio#101) windowspecdefinition(return_ratio#101 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS return_rank#103], [return_ratio#101 ASC NULLS FIRST] + +(66) Sort [codegen id : 17] +Input [4]: [item#100, return_ratio#101, currency_ratio#102, return_rank#103] +Arguments: [currency_ratio#102 ASC NULLS FIRST], false, 0 + +(67) Window +Input [4]: [item#100, return_ratio#101, currency_ratio#102, return_rank#103] +Arguments: [rank(currency_ratio#102) windowspecdefinition(currency_ratio#102 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS currency_rank#104], [currency_ratio#102 ASC NULLS FIRST] + +(68) Filter [codegen id : 18] +Input [5]: [item#100, return_ratio#101, currency_ratio#102, return_rank#103, currency_rank#104] +Condition : ((return_rank#103 <= 10) OR (currency_rank#104 <= 10)) + +(69) Project [codegen id : 18] +Output [5]: [store AS channel#105, item#100, return_ratio#101, return_rank#103, currency_rank#104] +Input [5]: [item#100, return_ratio#101, currency_ratio#102, return_rank#103, currency_rank#104] + +(70) Union + +(71) HashAggregate [codegen id : 19] +Input [5]: [channel#35, item#30, return_ratio#31, return_rank#33, currency_rank#34] +Keys [5]: [channel#35, item#30, return_ratio#31, return_rank#33, currency_rank#34] +Functions: [] +Aggregate Attributes: [] +Results [5]: [channel#35, item#30, return_ratio#31, return_rank#33, currency_rank#34] + +(72) Exchange +Input [5]: [channel#35, item#30, return_ratio#31, return_rank#33, currency_rank#34] +Arguments: hashpartitioning(channel#35, item#30, return_ratio#31, return_rank#33, currency_rank#34, 5), ENSURE_REQUIREMENTS, [plan_id=7] + +(73) HashAggregate [codegen id : 20] +Input [5]: [channel#35, item#30, return_ratio#31, return_rank#33, currency_rank#34] +Keys [5]: [channel#35, item#30, return_ratio#31, return_rank#33, currency_rank#34] +Functions: [] +Aggregate Attributes: [] +Results [5]: [channel#35, item#30, return_ratio#31, return_rank#33, currency_rank#34] + +(74) TakeOrderedAndProject +Input [5]: [channel#35, item#30, return_ratio#31, return_rank#33, currency_rank#34] +Arguments: 100, [channel#35 ASC NULLS FIRST, return_rank#33 ASC NULLS FIRST, currency_rank#34 ASC NULLS FIRST], [channel#35, item#30, return_ratio#31, return_rank#33, currency_rank#34] + +===== Subqueries ===== + +Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#6 IN dynamicpruning#7 +BroadcastExchange (79) ++- * ColumnarToRow (78) + +- CometProject (77) + +- CometFilter (76) + +- CometScan parquet spark_catalog.default.date_dim (75) + + +(75) Scan parquet spark_catalog.default.date_dim +Output [3]: [d_date_sk#13, d_year#106, d_moy#107] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2001), EqualTo(d_moy,12), IsNotNull(d_date_sk)] +ReadSchema: struct<d_date_sk:int,d_year:int,d_moy:int> + +(76) CometFilter +Input [3]: [d_date_sk#13, d_year#106, d_moy#107] +Condition : ((((isnotnull(d_year#106) AND isnotnull(d_moy#107)) AND (d_year#106 = 2001)) AND (d_moy#107 = 12)) AND isnotnull(d_date_sk#13)) + +(77) CometProject +Input [3]: [d_date_sk#13, d_year#106, d_moy#107] +Arguments: [d_date_sk#13], [d_date_sk#13] + +(78) ColumnarToRow [codegen id : 1] +Input [1]: [d_date_sk#13] + +(79) BroadcastExchange +Input [1]: [d_date_sk#13] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=8] + +Subquery:2 Hosting operator id = 24 Hosting Expression = cs_sold_date_sk#41 IN dynamicpruning#7 + +Subquery:3 Hosting operator id = 47 Hosting Expression = ss_sold_date_sk#76 IN dynamicpruning#7 + + diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q49/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q49/simplified.txt new file mode 100644 index 000000000..43ebf34cc --- /dev/null +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q49/simplified.txt @@ -0,0 +1,121 @@ +TakeOrderedAndProject [channel,return_rank,currency_rank,item,return_ratio] + WholeStageCodegen (20) + HashAggregate [channel,item,return_ratio,return_rank,currency_rank] + InputAdapter + Exchange [channel,item,return_ratio,return_rank,currency_rank] #1 + WholeStageCodegen (19) + HashAggregate [channel,item,return_ratio,return_rank,currency_rank] + InputAdapter + Union + WholeStageCodegen (6) + Project [item,return_ratio,return_rank,currency_rank] + Filter [return_rank,currency_rank] + InputAdapter + Window [currency_ratio] + WholeStageCodegen (5) + Sort [currency_ratio] + InputAdapter + Window [return_ratio] + WholeStageCodegen (4) + Sort [return_ratio] + InputAdapter + Exchange #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 + Exchange [ws_item_sk] #3 + WholeStageCodegen (2) + HashAggregate [ws_item_sk,wr_return_quantity,ws_quantity,wr_return_amt,ws_net_paid] [sum,sum,sum,isEmpty,sum,isEmpty,sum,sum,sum,isEmpty,sum,isEmpty] + Project [ws_item_sk,ws_quantity,ws_net_paid,wr_return_quantity,wr_return_amt] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + ColumnarToRow + InputAdapter + CometProject [ws_item_sk,ws_quantity,ws_net_paid,ws_sold_date_sk,wr_return_quantity,wr_return_amt] + CometBroadcastHashJoin [ws_order_number,ws_item_sk,wr_order_number,wr_item_sk] + CometBroadcastExchange #4 + CometProject [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_sold_date_sk] + CometFilter [ws_net_profit,ws_net_paid,ws_quantity,ws_order_number,ws_item_sk] + CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_net_profit,ws_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #5 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_year,d_moy,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometProject [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] + CometFilter [wr_return_amt,wr_order_number,wr_item_sk] + CometScan parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk] + InputAdapter + ReusedExchange [d_date_sk] #5 + WholeStageCodegen (12) + Project [item,return_ratio,return_rank,currency_rank] + Filter [return_rank,currency_rank] + InputAdapter + Window [currency_ratio] + WholeStageCodegen (11) + Sort [currency_ratio] + InputAdapter + Window [return_ratio] + WholeStageCodegen (10) + Sort [return_ratio] + InputAdapter + Exchange #6 + WholeStageCodegen (9) + HashAggregate [cs_item_sk,sum,sum,sum,isEmpty,sum,isEmpty] [sum(coalesce(cr_return_quantity, 0)),sum(coalesce(cs_quantity, 0)),sum(coalesce(cast(cr_return_amount as decimal(12,2)), 0.00)),sum(coalesce(cast(cs_net_paid as decimal(12,2)), 0.00)),item,return_ratio,currency_ratio,sum,sum,sum,isEmpty,sum,isEmpty] + InputAdapter + Exchange [cs_item_sk] #7 + WholeStageCodegen (8) + HashAggregate [cs_item_sk,cr_return_quantity,cs_quantity,cr_return_amount,cs_net_paid] [sum,sum,sum,isEmpty,sum,isEmpty,sum,sum,sum,isEmpty,sum,isEmpty] + Project [cs_item_sk,cs_quantity,cs_net_paid,cr_return_quantity,cr_return_amount] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + ColumnarToRow + InputAdapter + CometProject [cs_item_sk,cs_quantity,cs_net_paid,cs_sold_date_sk,cr_return_quantity,cr_return_amount] + CometBroadcastHashJoin [cs_order_number,cs_item_sk,cr_order_number,cr_item_sk] + CometBroadcastExchange #8 + CometProject [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_sold_date_sk] + CometFilter [cs_net_profit,cs_net_paid,cs_quantity,cs_order_number,cs_item_sk] + CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_net_profit,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + CometProject [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] + CometFilter [cr_return_amount,cr_order_number,cr_item_sk] + CometScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount,cr_returned_date_sk] + InputAdapter + ReusedExchange [d_date_sk] #5 + WholeStageCodegen (18) + Project [item,return_ratio,return_rank,currency_rank] + Filter [return_rank,currency_rank] + InputAdapter + Window [currency_ratio] + WholeStageCodegen (17) + Sort [currency_ratio] + InputAdapter + Window [return_ratio] + WholeStageCodegen (16) + Sort [return_ratio] + InputAdapter + Exchange #9 + WholeStageCodegen (15) + HashAggregate [ss_item_sk,sum,sum,sum,isEmpty,sum,isEmpty] [sum(coalesce(sr_return_quantity, 0)),sum(coalesce(ss_quantity, 0)),sum(coalesce(cast(sr_return_amt as decimal(12,2)), 0.00)),sum(coalesce(cast(ss_net_paid as decimal(12,2)), 0.00)),item,return_ratio,currency_ratio,sum,sum,sum,isEmpty,sum,isEmpty] + InputAdapter + Exchange [ss_item_sk] #10 + WholeStageCodegen (14) + HashAggregate [ss_item_sk,sr_return_quantity,ss_quantity,sr_return_amt,ss_net_paid] [sum,sum,sum,isEmpty,sum,isEmpty,sum,sum,sum,isEmpty,sum,isEmpty] + Project [ss_item_sk,ss_quantity,ss_net_paid,sr_return_quantity,sr_return_amt] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + ColumnarToRow + InputAdapter + CometProject [ss_item_sk,ss_quantity,ss_net_paid,ss_sold_date_sk,sr_return_quantity,sr_return_amt] + CometBroadcastHashJoin [ss_ticket_number,ss_item_sk,sr_ticket_number,sr_item_sk] + CometBroadcastExchange #11 + CometProject [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_sold_date_sk] + CometFilter [ss_net_profit,ss_net_paid,ss_quantity,ss_ticket_number,ss_item_sk] + CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_net_profit,ss_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + CometProject [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] + CometFilter [sr_return_amt,sr_ticket_number,sr_item_sk] + CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt,sr_returned_date_sk] + InputAdapter + ReusedExchange [d_date_sk] #5 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 new file mode 100644 index 000000000..e3f7538d1 --- /dev/null +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q5/explain.txt @@ -0,0 +1,457 @@ +== Physical Plan == +TakeOrderedAndProject (70) ++- * HashAggregate (69) + +- Exchange (68) + +- * HashAggregate (67) + +- * Expand (66) + +- Union (65) + :- * HashAggregate (20) + : +- Exchange (19) + : +- * HashAggregate (18) + : +- * Project (17) + : +- * BroadcastHashJoin Inner BuildRight (16) + : :- * Project (11) + : : +- * BroadcastHashJoin Inner BuildRight (10) + : : :- * ColumnarToRow (8) + : : : +- CometUnion (7) + : : : :- CometProject (3) + : : : : +- CometFilter (2) + : : : : +- CometScan parquet spark_catalog.default.store_sales (1) + : : : +- CometProject (6) + : : : +- CometFilter (5) + : : : +- CometScan parquet spark_catalog.default.store_returns (4) + : : +- ReusedExchange (9) + : +- BroadcastExchange (15) + : +- * ColumnarToRow (14) + : +- CometFilter (13) + : +- CometScan parquet spark_catalog.default.store (12) + :- * HashAggregate (40) + : +- Exchange (39) + : +- * HashAggregate (38) + : +- * Project (37) + : +- * BroadcastHashJoin Inner BuildRight (36) + : :- * Project (31) + : : +- * BroadcastHashJoin Inner BuildRight (30) + : : :- * ColumnarToRow (28) + : : : +- CometUnion (27) + : : : :- CometProject (23) + : : : : +- CometFilter (22) + : : : : +- CometScan parquet spark_catalog.default.catalog_sales (21) + : : : +- CometProject (26) + : : : +- CometFilter (25) + : : : +- CometScan parquet spark_catalog.default.catalog_returns (24) + : : +- ReusedExchange (29) + : +- BroadcastExchange (35) + : +- * ColumnarToRow (34) + : +- CometFilter (33) + : +- CometScan parquet spark_catalog.default.catalog_page (32) + +- * HashAggregate (64) + +- Exchange (63) + +- * HashAggregate (62) + +- * Project (61) + +- * BroadcastHashJoin Inner BuildRight (60) + :- * Project (55) + : +- * BroadcastHashJoin Inner BuildRight (54) + : :- * ColumnarToRow (52) + : : +- CometUnion (51) + : : :- CometProject (43) + : : : +- CometFilter (42) + : : : +- CometScan parquet spark_catalog.default.web_sales (41) + : : +- CometProject (50) + : : +- CometBroadcastHashJoin (49) + : : :- CometBroadcastExchange (45) + : : : +- CometScan parquet spark_catalog.default.web_returns (44) + : : +- CometProject (48) + : : +- CometFilter (47) + : : +- CometScan parquet spark_catalog.default.web_sales (46) + : +- ReusedExchange (53) + +- BroadcastExchange (59) + +- * ColumnarToRow (58) + +- CometFilter (57) + +- CometScan parquet spark_catalog.default.web_site (56) + + +(1) Scan parquet spark_catalog.default.store_sales +Output [4]: [ss_store_sk#1, ss_ext_sales_price#2, ss_net_profit#3, ss_sold_date_sk#4] +Batched: true +Location: InMemoryFileIndex [] +PartitionFilters: [isnotnull(ss_sold_date_sk#4), dynamicpruningexpression(ss_sold_date_sk#4 IN dynamicpruning#5)] +PushedFilters: [IsNotNull(ss_store_sk)] +ReadSchema: struct<ss_store_sk:int,ss_ext_sales_price:decimal(7,2),ss_net_profit:decimal(7,2)> + +(2) CometFilter +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) + +(3) CometProject +Input [4]: [ss_store_sk#1, ss_ext_sales_price#2, ss_net_profit#3, ss_sold_date_sk#4] +Arguments: [store_sk#6, date_sk#7, sales_price#8, profit#9, return_amt#10, net_loss#11], [ss_store_sk#1 AS store_sk#6, ss_sold_date_sk#4 AS date_sk#7, ss_ext_sales_price#2 AS sales_price#8, ss_net_profit#3 AS profit#9, 0.00 AS return_amt#10, 0.00 AS net_loss#11] + +(4) Scan parquet spark_catalog.default.store_returns +Output [4]: [sr_store_sk#12, sr_return_amt#13, sr_net_loss#14, sr_returned_date_sk#15] +Batched: true +Location: InMemoryFileIndex [] +PartitionFilters: [isnotnull(sr_returned_date_sk#15), dynamicpruningexpression(sr_returned_date_sk#15 IN dynamicpruning#5)] +PushedFilters: [IsNotNull(sr_store_sk)] +ReadSchema: struct<sr_store_sk:int,sr_return_amt:decimal(7,2),sr_net_loss:decimal(7,2)> + +(5) CometFilter +Input [4]: [sr_store_sk#12, sr_return_amt#13, sr_net_loss#14, sr_returned_date_sk#15] +Condition : isnotnull(sr_store_sk#12) + +(6) CometProject +Input [4]: [sr_store_sk#12, sr_return_amt#13, sr_net_loss#14, sr_returned_date_sk#15] +Arguments: [store_sk#16, date_sk#17, sales_price#18, profit#19, return_amt#20, net_loss#21], [sr_store_sk#12 AS store_sk#16, sr_returned_date_sk#15 AS date_sk#17, 0.00 AS sales_price#18, 0.00 AS profit#19, sr_return_amt#13 AS return_amt#20, sr_net_loss#14 AS net_loss#21] + +(7) CometUnion +Child 0 Input [6]: [store_sk#6, date_sk#7, sales_price#8, profit#9, return_amt#10, net_loss#11] +Child 1 Input [6]: [store_sk#16, date_sk#17, sales_price#18, profit#19, return_amt#20, net_loss#21] + +(8) ColumnarToRow [codegen id : 3] +Input [6]: [store_sk#6, date_sk#7, sales_price#8, profit#9, return_amt#10, net_loss#11] + +(9) ReusedExchange [Reuses operator id: 75] +Output [1]: [d_date_sk#22] + +(10) BroadcastHashJoin [codegen id : 3] +Left keys [1]: [date_sk#7] +Right keys [1]: [d_date_sk#22] +Join type: Inner +Join condition: None + +(11) 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] + +(12) Scan parquet spark_catalog.default.store +Output [2]: [s_store_sk#23, s_store_id#24] +Batched: true +Location [not included in comparison]/{warehouse_dir}/store] +PushedFilters: [IsNotNull(s_store_sk)] +ReadSchema: struct<s_store_sk:int,s_store_id:string> + +(13) CometFilter +Input [2]: [s_store_sk#23, s_store_id#24] +Condition : isnotnull(s_store_sk#23) + +(14) ColumnarToRow [codegen id : 2] +Input [2]: [s_store_sk#23, s_store_id#24] + +(15) BroadcastExchange +Input [2]: [s_store_sk#23, s_store_id#24] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=1] + +(16) BroadcastHashJoin [codegen id : 3] +Left keys [1]: [store_sk#6] +Right keys [1]: [s_store_sk#23] +Join type: Inner +Join condition: None + +(17) Project [codegen id : 3] +Output [5]: [sales_price#8, profit#9, return_amt#10, net_loss#11, s_store_id#24] +Input [7]: [store_sk#6, sales_price#8, profit#9, return_amt#10, net_loss#11, s_store_sk#23, s_store_id#24] + +(18) HashAggregate [codegen id : 3] +Input [5]: [sales_price#8, profit#9, return_amt#10, net_loss#11, s_store_id#24] +Keys [1]: [s_store_id#24] +Functions [4]: [partial_sum(UnscaledValue(sales_price#8)), partial_sum(UnscaledValue(return_amt#10)), partial_sum(UnscaledValue(profit#9)), partial_sum(UnscaledValue(net_loss#11))] +Aggregate Attributes [4]: [sum#25, sum#26, sum#27, sum#28] +Results [5]: [s_store_id#24, sum#29, sum#30, sum#31, sum#32] + +(19) Exchange +Input [5]: [s_store_id#24, sum#29, sum#30, sum#31, sum#32] +Arguments: hashpartitioning(s_store_id#24, 5), ENSURE_REQUIREMENTS, [plan_id=2] + +(20) HashAggregate [codegen id : 4] +Input [5]: [s_store_id#24, sum#29, sum#30, sum#31, sum#32] +Keys [1]: [s_store_id#24] +Functions [4]: [sum(UnscaledValue(sales_price#8)), sum(UnscaledValue(return_amt#10)), sum(UnscaledValue(profit#9)), sum(UnscaledValue(net_loss#11))] +Aggregate Attributes [4]: [sum(UnscaledValue(sales_price#8))#33, sum(UnscaledValue(return_amt#10))#34, sum(UnscaledValue(profit#9))#35, sum(UnscaledValue(net_loss#11))#36] +Results [5]: [MakeDecimal(sum(UnscaledValue(sales_price#8))#33,17,2) AS sales#37, MakeDecimal(sum(UnscaledValue(return_amt#10))#34,17,2) AS returns#38, (MakeDecimal(sum(UnscaledValue(profit#9))#35,17,2) - MakeDecimal(sum(UnscaledValue(net_loss#11))#36,17,2)) AS profit#39, store channel AS channel#40, concat(store, s_store_id#24) AS id#41] + +(21) Scan parquet spark_catalog.default.catalog_sales +Output [4]: [cs_catalog_page_sk#42, cs_ext_sales_price#43, cs_net_profit#44, cs_sold_date_sk#45] +Batched: true +Location: InMemoryFileIndex [] +PartitionFilters: [isnotnull(cs_sold_date_sk#45), dynamicpruningexpression(cs_sold_date_sk#45 IN dynamicpruning#46)] +PushedFilters: [IsNotNull(cs_catalog_page_sk)] +ReadSchema: struct<cs_catalog_page_sk:int,cs_ext_sales_price:decimal(7,2),cs_net_profit:decimal(7,2)> + +(22) CometFilter +Input [4]: [cs_catalog_page_sk#42, cs_ext_sales_price#43, cs_net_profit#44, cs_sold_date_sk#45] +Condition : isnotnull(cs_catalog_page_sk#42) + +(23) CometProject +Input [4]: [cs_catalog_page_sk#42, cs_ext_sales_price#43, cs_net_profit#44, cs_sold_date_sk#45] +Arguments: [page_sk#47, date_sk#48, sales_price#49, profit#50, return_amt#51, net_loss#52], [cs_catalog_page_sk#42 AS page_sk#47, cs_sold_date_sk#45 AS date_sk#48, cs_ext_sales_price#43 AS sales_price#49, cs_net_profit#44 AS profit#50, 0.00 AS return_amt#51, 0.00 AS net_loss#52] + +(24) Scan parquet spark_catalog.default.catalog_returns +Output [4]: [cr_catalog_page_sk#53, cr_return_amount#54, cr_net_loss#55, cr_returned_date_sk#56] +Batched: true +Location: InMemoryFileIndex [] +PartitionFilters: [isnotnull(cr_returned_date_sk#56), dynamicpruningexpression(cr_returned_date_sk#56 IN dynamicpruning#46)] +PushedFilters: [IsNotNull(cr_catalog_page_sk)] +ReadSchema: struct<cr_catalog_page_sk:int,cr_return_amount:decimal(7,2),cr_net_loss:decimal(7,2)> + +(25) CometFilter +Input [4]: [cr_catalog_page_sk#53, cr_return_amount#54, cr_net_loss#55, cr_returned_date_sk#56] +Condition : isnotnull(cr_catalog_page_sk#53) + +(26) CometProject +Input [4]: [cr_catalog_page_sk#53, cr_return_amount#54, cr_net_loss#55, cr_returned_date_sk#56] +Arguments: [page_sk#57, date_sk#58, sales_price#59, profit#60, return_amt#61, net_loss#62], [cr_catalog_page_sk#53 AS page_sk#57, cr_returned_date_sk#56 AS date_sk#58, 0.00 AS sales_price#59, 0.00 AS profit#60, cr_return_amount#54 AS return_amt#61, cr_net_loss#55 AS net_loss#62] + +(27) CometUnion +Child 0 Input [6]: [page_sk#47, date_sk#48, sales_price#49, profit#50, return_amt#51, net_loss#52] +Child 1 Input [6]: [page_sk#57, date_sk#58, sales_price#59, profit#60, return_amt#61, net_loss#62] + +(28) ColumnarToRow [codegen id : 7] +Input [6]: [page_sk#47, date_sk#48, sales_price#49, profit#50, return_amt#51, net_loss#52] + +(29) ReusedExchange [Reuses operator id: 75] +Output [1]: [d_date_sk#63] + +(30) BroadcastHashJoin [codegen id : 7] +Left keys [1]: [date_sk#48] +Right keys [1]: [d_date_sk#63] +Join type: Inner +Join condition: None + +(31) Project [codegen id : 7] +Output [5]: [page_sk#47, sales_price#49, profit#50, return_amt#51, net_loss#52] +Input [7]: [page_sk#47, date_sk#48, sales_price#49, profit#50, return_amt#51, net_loss#52, d_date_sk#63] + +(32) Scan parquet spark_catalog.default.catalog_page +Output [2]: [cp_catalog_page_sk#64, cp_catalog_page_id#65] +Batched: true +Location [not included in comparison]/{warehouse_dir}/catalog_page] +PushedFilters: [IsNotNull(cp_catalog_page_sk)] +ReadSchema: struct<cp_catalog_page_sk:int,cp_catalog_page_id:string> + +(33) CometFilter +Input [2]: [cp_catalog_page_sk#64, cp_catalog_page_id#65] +Condition : isnotnull(cp_catalog_page_sk#64) + +(34) ColumnarToRow [codegen id : 6] +Input [2]: [cp_catalog_page_sk#64, cp_catalog_page_id#65] + +(35) BroadcastExchange +Input [2]: [cp_catalog_page_sk#64, cp_catalog_page_id#65] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=3] + +(36) BroadcastHashJoin [codegen id : 7] +Left keys [1]: [page_sk#47] +Right keys [1]: [cp_catalog_page_sk#64] +Join type: Inner +Join condition: None + +(37) Project [codegen id : 7] +Output [5]: [sales_price#49, profit#50, return_amt#51, net_loss#52, cp_catalog_page_id#65] +Input [7]: [page_sk#47, sales_price#49, profit#50, return_amt#51, net_loss#52, cp_catalog_page_sk#64, cp_catalog_page_id#65] + +(38) HashAggregate [codegen id : 7] +Input [5]: [sales_price#49, profit#50, return_amt#51, net_loss#52, cp_catalog_page_id#65] +Keys [1]: [cp_catalog_page_id#65] +Functions [4]: [partial_sum(UnscaledValue(sales_price#49)), partial_sum(UnscaledValue(return_amt#51)), partial_sum(UnscaledValue(profit#50)), partial_sum(UnscaledValue(net_loss#52))] +Aggregate Attributes [4]: [sum#66, sum#67, sum#68, sum#69] +Results [5]: [cp_catalog_page_id#65, sum#70, sum#71, sum#72, sum#73] + +(39) Exchange +Input [5]: [cp_catalog_page_id#65, sum#70, sum#71, sum#72, sum#73] +Arguments: hashpartitioning(cp_catalog_page_id#65, 5), ENSURE_REQUIREMENTS, [plan_id=4] + +(40) HashAggregate [codegen id : 8] +Input [5]: [cp_catalog_page_id#65, sum#70, sum#71, sum#72, sum#73] +Keys [1]: [cp_catalog_page_id#65] +Functions [4]: [sum(UnscaledValue(sales_price#49)), sum(UnscaledValue(return_amt#51)), sum(UnscaledValue(profit#50)), sum(UnscaledValue(net_loss#52))] +Aggregate Attributes [4]: [sum(UnscaledValue(sales_price#49))#74, sum(UnscaledValue(return_amt#51))#75, sum(UnscaledValue(profit#50))#76, sum(UnscaledValue(net_loss#52))#77] +Results [5]: [MakeDecimal(sum(UnscaledValue(sales_price#49))#74,17,2) AS sales#78, MakeDecimal(sum(UnscaledValue(return_amt#51))#75,17,2) AS returns#79, (MakeDecimal(sum(UnscaledValue(profit#50))#76,17,2) - MakeDecimal(sum(UnscaledValue(net_loss#52))#77,17,2)) AS profit#80, catalog channel AS channel#81, concat(catalog_page, cp_catalog_page_id#65) AS id#82] + +(41) Scan parquet spark_catalog.default.web_sales +Output [4]: [ws_web_site_sk#83, ws_ext_sales_price#84, ws_net_profit#85, ws_sold_date_sk#86] +Batched: true +Location: InMemoryFileIndex [] +PartitionFilters: [isnotnull(ws_sold_date_sk#86), dynamicpruningexpression(ws_sold_date_sk#86 IN dynamicpruning#87)] +PushedFilters: [IsNotNull(ws_web_site_sk)] +ReadSchema: struct<ws_web_site_sk:int,ws_ext_sales_price:decimal(7,2),ws_net_profit:decimal(7,2)> + +(42) CometFilter +Input [4]: [ws_web_site_sk#83, ws_ext_sales_price#84, ws_net_profit#85, ws_sold_date_sk#86] +Condition : isnotnull(ws_web_site_sk#83) + +(43) CometProject +Input [4]: [ws_web_site_sk#83, ws_ext_sales_price#84, ws_net_profit#85, ws_sold_date_sk#86] +Arguments: [wsr_web_site_sk#88, date_sk#89, sales_price#90, profit#91, return_amt#92, net_loss#93], [ws_web_site_sk#83 AS wsr_web_site_sk#88, ws_sold_date_sk#86 AS date_sk#89, ws_ext_sales_price#84 AS sales_price#90, ws_net_profit#85 AS profit#91, 0.00 AS return_amt#92, 0.00 AS net_loss#93] + +(44) Scan parquet spark_catalog.default.web_returns +Output [5]: [wr_item_sk#94, wr_order_number#95, wr_return_amt#96, wr_net_loss#97, wr_returned_date_sk#98] +Batched: true +Location: InMemoryFileIndex [] +PartitionFilters: [isnotnull(wr_returned_date_sk#98), dynamicpruningexpression(wr_returned_date_sk#98 IN dynamicpruning#87)] +ReadSchema: struct<wr_item_sk:int,wr_order_number:int,wr_return_amt:decimal(7,2),wr_net_loss:decimal(7,2)> + +(45) CometBroadcastExchange +Input [5]: [wr_item_sk#94, wr_order_number#95, wr_return_amt#96, wr_net_loss#97, wr_returned_date_sk#98] +Arguments: [wr_item_sk#94, wr_order_number#95, wr_return_amt#96, wr_net_loss#97, wr_returned_date_sk#98] + +(46) Scan parquet spark_catalog.default.web_sales +Output [4]: [ws_item_sk#99, ws_web_site_sk#100, ws_order_number#101, ws_sold_date_sk#102] +Batched: true +Location [not included in comparison]/{warehouse_dir}/web_sales] +PushedFilters: [IsNotNull(ws_item_sk), IsNotNull(ws_order_number), IsNotNull(ws_web_site_sk)] +ReadSchema: struct<ws_item_sk:int,ws_web_site_sk:int,ws_order_number:int> + +(47) CometFilter +Input [4]: [ws_item_sk#99, ws_web_site_sk#100, ws_order_number#101, ws_sold_date_sk#102] +Condition : ((isnotnull(ws_item_sk#99) AND isnotnull(ws_order_number#101)) AND isnotnull(ws_web_site_sk#100)) + +(48) CometProject +Input [4]: [ws_item_sk#99, ws_web_site_sk#100, ws_order_number#101, ws_sold_date_sk#102] +Arguments: [ws_item_sk#99, ws_web_site_sk#100, ws_order_number#101], [ws_item_sk#99, ws_web_site_sk#100, ws_order_number#101] + +(49) CometBroadcastHashJoin +Left output [5]: [wr_item_sk#94, wr_order_number#95, wr_return_amt#96, wr_net_loss#97, wr_returned_date_sk#98] +Right output [3]: [ws_item_sk#99, ws_web_site_sk#100, ws_order_number#101] +Arguments: [wr_item_sk#94, wr_order_number#95], [ws_item_sk#99, ws_order_number#101], Inner + +(50) CometProject +Input [8]: [wr_item_sk#94, wr_order_number#95, wr_return_amt#96, wr_net_loss#97, wr_returned_date_sk#98, ws_item_sk#99, ws_web_site_sk#100, ws_order_number#101] +Arguments: [wsr_web_site_sk#103, date_sk#104, sales_price#105, profit#106, return_amt#107, net_loss#108], [ws_web_site_sk#100 AS wsr_web_site_sk#103, wr_returned_date_sk#98 AS date_sk#104, 0.00 AS sales_price#105, 0.00 AS profit#106, wr_return_amt#96 AS return_amt#107, wr_net_loss#97 AS net_loss#108] + +(51) CometUnion +Child 0 Input [6]: [wsr_web_site_sk#88, date_sk#89, sales_price#90, profit#91, return_amt#92, net_loss#93] +Child 1 Input [6]: [wsr_web_site_sk#103, date_sk#104, sales_price#105, profit#106, return_amt#107, net_loss#108] + +(52) ColumnarToRow [codegen id : 11] +Input [6]: [wsr_web_site_sk#88, date_sk#89, sales_price#90, profit#91, return_amt#92, net_loss#93] + +(53) ReusedExchange [Reuses operator id: 75] +Output [1]: [d_date_sk#109] + +(54) BroadcastHashJoin [codegen id : 11] +Left keys [1]: [date_sk#89] +Right keys [1]: [d_date_sk#109] +Join type: Inner +Join condition: None + +(55) Project [codegen id : 11] +Output [5]: [wsr_web_site_sk#88, sales_price#90, profit#91, return_amt#92, net_loss#93] +Input [7]: [wsr_web_site_sk#88, date_sk#89, sales_price#90, profit#91, return_amt#92, net_loss#93, d_date_sk#109] + +(56) Scan parquet spark_catalog.default.web_site +Output [2]: [web_site_sk#110, web_site_id#111] +Batched: true +Location [not included in comparison]/{warehouse_dir}/web_site] +PushedFilters: [IsNotNull(web_site_sk)] +ReadSchema: struct<web_site_sk:int,web_site_id:string> + +(57) CometFilter +Input [2]: [web_site_sk#110, web_site_id#111] +Condition : isnotnull(web_site_sk#110) + +(58) ColumnarToRow [codegen id : 10] +Input [2]: [web_site_sk#110, web_site_id#111] + +(59) BroadcastExchange +Input [2]: [web_site_sk#110, web_site_id#111] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=5] + +(60) BroadcastHashJoin [codegen id : 11] +Left keys [1]: [wsr_web_site_sk#88] +Right keys [1]: [web_site_sk#110] +Join type: Inner +Join condition: None + +(61) Project [codegen id : 11] +Output [5]: [sales_price#90, profit#91, return_amt#92, net_loss#93, web_site_id#111] +Input [7]: [wsr_web_site_sk#88, sales_price#90, profit#91, return_amt#92, net_loss#93, web_site_sk#110, web_site_id#111] + +(62) HashAggregate [codegen id : 11] +Input [5]: [sales_price#90, profit#91, return_amt#92, net_loss#93, web_site_id#111] +Keys [1]: [web_site_id#111] +Functions [4]: [partial_sum(UnscaledValue(sales_price#90)), partial_sum(UnscaledValue(return_amt#92)), partial_sum(UnscaledValue(profit#91)), partial_sum(UnscaledValue(net_loss#93))] +Aggregate Attributes [4]: [sum#112, sum#113, sum#114, sum#115] +Results [5]: [web_site_id#111, sum#116, sum#117, sum#118, sum#119] + +(63) Exchange +Input [5]: [web_site_id#111, sum#116, sum#117, sum#118, sum#119] +Arguments: hashpartitioning(web_site_id#111, 5), ENSURE_REQUIREMENTS, [plan_id=6] + +(64) HashAggregate [codegen id : 12] +Input [5]: [web_site_id#111, sum#116, sum#117, sum#118, sum#119] +Keys [1]: [web_site_id#111] +Functions [4]: [sum(UnscaledValue(sales_price#90)), sum(UnscaledValue(return_amt#92)), sum(UnscaledValue(profit#91)), sum(UnscaledValue(net_loss#93))] +Aggregate Attributes [4]: [sum(UnscaledValue(sales_price#90))#120, sum(UnscaledValue(return_amt#92))#121, sum(UnscaledValue(profit#91))#122, sum(UnscaledValue(net_loss#93))#123] +Results [5]: [MakeDecimal(sum(UnscaledValue(sales_price#90))#120,17,2) AS sales#124, MakeDecimal(sum(UnscaledValue(return_amt#92))#121,17,2) AS returns#125, (MakeDecimal(sum(UnscaledValue(profit#91))#122,17,2) - MakeDecimal(sum(UnscaledValue(net_loss#93))#123,17,2)) AS profit#126, web channel AS channel#127, concat(web_site, web_site_id#111) AS id#128] + +(65) Union + +(66) Expand [codegen id : 13] +Input [5]: [sales#37, returns#38, profit#39, channel#40, id#41] +Arguments: [[sales#37, returns#38, profit#39, channel#40, id#41, 0], [sales#37, returns#38, profit#39, channel#40, null, 1], [sales#37, returns#38, profit#39, null, null, 3]], [sales#37, returns#38, profit#39, channel#129, id#130, spark_grouping_id#131] + +(67) HashAggregate [codegen id : 13] +Input [6]: [sales#37, returns#38, profit#39, channel#129, id#130, spark_grouping_id#131] +Keys [3]: [channel#129, id#130, spark_grouping_id#131] +Functions [3]: [partial_sum(sales#37), partial_sum(returns#38), partial_sum(profit#39)] +Aggregate Attributes [6]: [sum#132, isEmpty#133, sum#134, isEmpty#135, sum#136, isEmpty#137] +Results [9]: [channel#129, id#130, spark_grouping_id#131, sum#138, isEmpty#139, sum#140, isEmpty#141, sum#142, isEmpty#143] + +(68) Exchange +Input [9]: [channel#129, id#130, spark_grouping_id#131, sum#138, isEmpty#139, sum#140, isEmpty#141, sum#142, isEmpty#143] +Arguments: hashpartitioning(channel#129, id#130, spark_grouping_id#131, 5), ENSURE_REQUIREMENTS, [plan_id=7] + +(69) HashAggregate [codegen id : 14] +Input [9]: [channel#129, id#130, spark_grouping_id#131, sum#138, isEmpty#139, sum#140, isEmpty#141, sum#142, isEmpty#143] +Keys [3]: [channel#129, id#130, spark_grouping_id#131] +Functions [3]: [sum(sales#37), sum(returns#38), sum(profit#39)] +Aggregate Attributes [3]: [sum(sales#37)#144, sum(returns#38)#145, sum(profit#39)#146] +Results [5]: [channel#129, id#130, sum(sales#37)#144 AS sales#147, sum(returns#38)#145 AS returns#148, sum(profit#39)#146 AS profit#149] + +(70) TakeOrderedAndProject +Input [5]: [channel#129, id#130, sales#147, returns#148, profit#149] +Arguments: 100, [channel#129 ASC NULLS FIRST, id#130 ASC NULLS FIRST], [channel#129, id#130, sales#147, returns#148, profit#149] + +===== Subqueries ===== + +Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 +BroadcastExchange (75) ++- * ColumnarToRow (74) + +- CometProject (73) + +- CometFilter (72) + +- CometScan parquet spark_catalog.default.date_dim (71) + + +(71) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#22, d_date#150] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,2000-08-23), LessThanOrEqual(d_date,2000-09-06), IsNotNull(d_date_sk)] +ReadSchema: struct<d_date_sk:int,d_date:date> + +(72) CometFilter +Input [2]: [d_date_sk#22, d_date#150] +Condition : (((isnotnull(d_date#150) AND (d_date#150 >= 2000-08-23)) AND (d_date#150 <= 2000-09-06)) AND isnotnull(d_date_sk#22)) + +(73) CometProject +Input [2]: [d_date_sk#22, d_date#150] +Arguments: [d_date_sk#22], [d_date_sk#22] + +(74) ColumnarToRow [codegen id : 1] +Input [1]: [d_date_sk#22] + +(75) BroadcastExchange +Input [1]: [d_date_sk#22] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=8] + +Subquery:2 Hosting operator id = 4 Hosting Expression = sr_returned_date_sk#15 IN dynamicpruning#5 + +Subquery:3 Hosting operator id = 21 Hosting Expression = cs_sold_date_sk#45 IN dynamicpruning#5 + +Subquery:4 Hosting operator id = 24 Hosting Expression = cr_returned_date_sk#56 IN dynamicpruning#5 + +Subquery:5 Hosting operator id = 41 Hosting Expression = ws_sold_date_sk#86 IN dynamicpruning#5 + +Subquery:6 Hosting operator id = 44 Hosting Expression = wr_returned_date_sk#98 IN dynamicpruning#5 + + diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q5/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q5/simplified.txt new file mode 100644 index 000000000..3d539d591 --- /dev/null +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q5/simplified.txt @@ -0,0 +1,111 @@ +TakeOrderedAndProject [channel,id,sales,returns,profit] + WholeStageCodegen (14) + HashAggregate [channel,id,spark_grouping_id,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),sales,returns,profit,sum,isEmpty,sum,isEmpty,sum,isEmpty] + InputAdapter + Exchange [channel,id,spark_grouping_id] #1 + WholeStageCodegen (13) + HashAggregate [channel,id,spark_grouping_id,sales,returns,profit] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] + Expand [sales,returns,profit,channel,id] + InputAdapter + Union + WholeStageCodegen (4) + HashAggregate [s_store_id,sum,sum,sum,sum] [sum(UnscaledValue(sales_price)),sum(UnscaledValue(return_amt)),sum(UnscaledValue(profit)),sum(UnscaledValue(net_loss)),sales,returns,profit,channel,id,sum,sum,sum,sum] + InputAdapter + Exchange [s_store_id] #2 + WholeStageCodegen (3) + HashAggregate [s_store_id,sales_price,return_amt,profit,net_loss] [sum,sum,sum,sum,sum,sum,sum,sum] + Project [sales_price,profit,return_amt,net_loss,s_store_id] + BroadcastHashJoin [store_sk,s_store_sk] + Project [store_sk,sales_price,profit,return_amt,net_loss] + BroadcastHashJoin [date_sk,d_date_sk] + ColumnarToRow + InputAdapter + CometUnion + CometProject [ss_store_sk,ss_sold_date_sk,ss_ext_sales_price,ss_net_profit] [store_sk,date_sk,sales_price,profit,return_amt,net_loss] + CometFilter [ss_store_sk] + CometScan parquet spark_catalog.default.store_sales [ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometProject [sr_store_sk,sr_returned_date_sk,sr_return_amt,sr_net_loss] [store_sk,date_sk,sales_price,profit,return_amt,net_loss] + CometFilter [sr_store_sk] + CometScan parquet spark_catalog.default.store_returns [sr_store_sk,sr_return_amt,sr_net_loss,sr_returned_date_sk] + ReusedSubquery [d_date_sk] #1 + InputAdapter + ReusedExchange [d_date_sk] #3 + InputAdapter + BroadcastExchange #4 + WholeStageCodegen (2) + ColumnarToRow + InputAdapter + CometFilter [s_store_sk] + CometScan parquet spark_catalog.default.store [s_store_sk,s_store_id] + WholeStageCodegen (8) + HashAggregate [cp_catalog_page_id,sum,sum,sum,sum] [sum(UnscaledValue(sales_price)),sum(UnscaledValue(return_amt)),sum(UnscaledValue(profit)),sum(UnscaledValue(net_loss)),sales,returns,profit,channel,id,sum,sum,sum,sum] + InputAdapter + Exchange [cp_catalog_page_id] #5 + WholeStageCodegen (7) + HashAggregate [cp_catalog_page_id,sales_price,return_amt,profit,net_loss] [sum,sum,sum,sum,sum,sum,sum,sum] + Project [sales_price,profit,return_amt,net_loss,cp_catalog_page_id] + BroadcastHashJoin [page_sk,cp_catalog_page_sk] + Project [page_sk,sales_price,profit,return_amt,net_loss] + BroadcastHashJoin [date_sk,d_date_sk] + ColumnarToRow + InputAdapter + CometUnion + CometProject [cs_catalog_page_sk,cs_sold_date_sk,cs_ext_sales_price,cs_net_profit] [page_sk,date_sk,sales_price,profit,return_amt,net_loss] + CometFilter [cs_catalog_page_sk] + CometScan parquet spark_catalog.default.catalog_sales [cs_catalog_page_sk,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + CometProject [cr_catalog_page_sk,cr_returned_date_sk,cr_return_amount,cr_net_loss] [page_sk,date_sk,sales_price,profit,return_amt,net_loss] + CometFilter [cr_catalog_page_sk] + CometScan parquet spark_catalog.default.catalog_returns [cr_catalog_page_sk,cr_return_amount,cr_net_loss,cr_returned_date_sk] + ReusedSubquery [d_date_sk] #1 + InputAdapter + ReusedExchange [d_date_sk] #3 + InputAdapter + BroadcastExchange #6 + WholeStageCodegen (6) + ColumnarToRow + InputAdapter + CometFilter [cp_catalog_page_sk] + CometScan parquet spark_catalog.default.catalog_page [cp_catalog_page_sk,cp_catalog_page_id] + WholeStageCodegen (12) + HashAggregate [web_site_id,sum,sum,sum,sum] [sum(UnscaledValue(sales_price)),sum(UnscaledValue(return_amt)),sum(UnscaledValue(profit)),sum(UnscaledValue(net_loss)),sales,returns,profit,channel,id,sum,sum,sum,sum] + InputAdapter + Exchange [web_site_id] #7 + WholeStageCodegen (11) + HashAggregate [web_site_id,sales_price,return_amt,profit,net_loss] [sum,sum,sum,sum,sum,sum,sum,sum] + Project [sales_price,profit,return_amt,net_loss,web_site_id] + BroadcastHashJoin [wsr_web_site_sk,web_site_sk] + Project [wsr_web_site_sk,sales_price,profit,return_amt,net_loss] + BroadcastHashJoin [date_sk,d_date_sk] + ColumnarToRow + InputAdapter + CometUnion + CometProject [ws_web_site_sk,ws_sold_date_sk,ws_ext_sales_price,ws_net_profit] [wsr_web_site_sk,date_sk,sales_price,profit,return_amt,net_loss] + CometFilter [ws_web_site_sk] + CometScan parquet spark_catalog.default.web_sales [ws_web_site_sk,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + CometProject [ws_web_site_sk,wr_returned_date_sk,wr_return_amt,wr_net_loss] [wsr_web_site_sk,date_sk,sales_price,profit,return_amt,net_loss] + CometBroadcastHashJoin [wr_item_sk,wr_order_number,ws_item_sk,ws_order_number] + CometBroadcastExchange #8 + CometScan parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk] + ReusedSubquery [d_date_sk] #1 + CometProject [ws_item_sk,ws_web_site_sk,ws_order_number] + CometFilter [ws_item_sk,ws_order_number,ws_web_site_sk] + CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_web_site_sk,ws_order_number,ws_sold_date_sk] + InputAdapter + ReusedExchange [d_date_sk] #3 + InputAdapter + BroadcastExchange #9 + WholeStageCodegen (10) + ColumnarToRow + InputAdapter + CometFilter [web_site_sk] + CometScan parquet spark_catalog.default.web_site [web_site_sk,web_site_id] 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 new file mode 100644 index 000000000..0182e0ac1 --- /dev/null +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q50/explain.txt @@ -0,0 +1,199 @@ +== Physical Plan == +TakeOrderedAndProject (28) ++- * HashAggregate (27) + +- Exchange (26) + +- * HashAggregate (25) + +- * Project (24) + +- * BroadcastHashJoin Inner BuildRight (23) + :- * Project (21) + : +- * BroadcastHashJoin Inner BuildRight (20) + : :- * Project (15) + : : +- * BroadcastHashJoin Inner BuildRight (14) + : : :- * Project (9) + : : : +- * BroadcastHashJoin Inner BuildRight (8) + : : : :- * ColumnarToRow (3) + : : : : +- CometFilter (2) + : : : : +- CometScan parquet spark_catalog.default.store_sales (1) + : : : +- BroadcastExchange (7) + : : : +- * ColumnarToRow (6) + : : : +- CometFilter (5) + : : : +- CometScan parquet spark_catalog.default.store_returns (4) + : : +- BroadcastExchange (13) + : : +- * ColumnarToRow (12) + : : +- CometFilter (11) + : : +- CometScan parquet spark_catalog.default.store (10) + : +- BroadcastExchange (19) + : +- * ColumnarToRow (18) + : +- CometFilter (17) + : +- CometScan parquet spark_catalog.default.date_dim (16) + +- ReusedExchange (22) + + +(1) Scan parquet spark_catalog.default.store_sales +Output [5]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_sold_date_sk#5] +Batched: true +Location: InMemoryFileIndex [] +PartitionFilters: [isnotnull(ss_sold_date_sk#5)] +PushedFilters: [IsNotNull(ss_ticket_number), IsNotNull(ss_item_sk), IsNotNull(ss_customer_sk), IsNotNull(ss_store_sk)] +ReadSchema: struct<ss_item_sk:int,ss_customer_sk:int,ss_store_sk:int,ss_ticket_number:int> + +(2) CometFilter +Input [5]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_sold_date_sk#5] +Condition : (((isnotnull(ss_ticket_number#4) AND isnotnull(ss_item_sk#1)) AND isnotnull(ss_customer_sk#2)) AND isnotnull(ss_store_sk#3)) + +(3) ColumnarToRow [codegen id : 5] +Input [5]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_sold_date_sk#5] + +(4) Scan parquet spark_catalog.default.store_returns +Output [4]: [sr_item_sk#6, sr_customer_sk#7, sr_ticket_number#8, sr_returned_date_sk#9] +Batched: true +Location: InMemoryFileIndex [] +PartitionFilters: [isnotnull(sr_returned_date_sk#9), dynamicpruningexpression(sr_returned_date_sk#9 IN dynamicpruning#10)] +PushedFilters: [IsNotNull(sr_ticket_number), IsNotNull(sr_item_sk), IsNotNull(sr_customer_sk)] +ReadSchema: struct<sr_item_sk:int,sr_customer_sk:int,sr_ticket_number:int> + +(5) CometFilter +Input [4]: [sr_item_sk#6, sr_customer_sk#7, sr_ticket_number#8, sr_returned_date_sk#9] +Condition : ((isnotnull(sr_ticket_number#8) AND isnotnull(sr_item_sk#6)) AND isnotnull(sr_customer_sk#7)) + +(6) ColumnarToRow [codegen id : 1] +Input [4]: [sr_item_sk#6, sr_customer_sk#7, sr_ticket_number#8, sr_returned_date_sk#9] + +(7) BroadcastExchange +Input [4]: [sr_item_sk#6, sr_customer_sk#7, sr_ticket_number#8, sr_returned_date_sk#9] +Arguments: HashedRelationBroadcastMode(List(input[2, int, false], input[0, int, false], input[1, int, false]),false), [plan_id=1] + +(8) BroadcastHashJoin [codegen id : 5] +Left keys [3]: [ss_ticket_number#4, ss_item_sk#1, ss_customer_sk#2] +Right keys [3]: [sr_ticket_number#8, sr_item_sk#6, sr_customer_sk#7] +Join type: Inner +Join condition: None + +(9) Project [codegen id : 5] +Output [3]: [ss_store_sk#3, ss_sold_date_sk#5, sr_returned_date_sk#9] +Input [9]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_sold_date_sk#5, sr_item_sk#6, sr_customer_sk#7, sr_ticket_number#8, sr_returned_date_sk#9] + +(10) Scan parquet spark_catalog.default.store +Output [11]: [s_store_sk#11, s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#16, s_suite_number#17, s_city#18, s_county#19, s_state#20, s_zip#21] +Batched: true +Location [not included in comparison]/{warehouse_dir}/store] +PushedFilters: [IsNotNull(s_store_sk)] +ReadSchema: struct<s_store_sk:int,s_store_name:string,s_company_id:int,s_street_number:string,s_street_name:string,s_street_type:string,s_suite_number:string,s_city:string,s_county:string,s_state:string,s_zip:string> + +(11) CometFilter +Input [11]: [s_store_sk#11, s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#16, s_suite_number#17, s_city#18, s_county#19, s_state#20, s_zip#21] +Condition : isnotnull(s_store_sk#11) + +(12) ColumnarToRow [codegen id : 2] +Input [11]: [s_store_sk#11, s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#16, s_suite_number#17, s_city#18, s_county#19, s_state#20, s_zip#21] + +(13) BroadcastExchange +Input [11]: [s_store_sk#11, s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#16, s_suite_number#17, s_city#18, s_county#19, s_state#20, s_zip#21] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=2] + +(14) BroadcastHashJoin [codegen id : 5] +Left keys [1]: [ss_store_sk#3] +Right keys [1]: [s_store_sk#11] +Join type: Inner +Join condition: None + +(15) Project [codegen id : 5] +Output [12]: [ss_sold_date_sk#5, sr_returned_date_sk#9, s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#16, s_suite_number#17, s_city#18, s_county#19, s_state#20, s_zip#21] +Input [14]: [ss_store_sk#3, ss_sold_date_sk#5, sr_returned_date_sk#9, s_store_sk#11, s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#16, s_suite_number#17, s_city#18, s_county#19, s_state#20, s_zip#21] + +(16) Scan parquet spark_catalog.default.date_dim +Output [1]: [d_date_sk#22] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_date_sk)] +ReadSchema: struct<d_date_sk:int> + +(17) CometFilter +Input [1]: [d_date_sk#22] +Condition : isnotnull(d_date_sk#22) + +(18) ColumnarToRow [codegen id : 3] +Input [1]: [d_date_sk#22] + +(19) BroadcastExchange +Input [1]: [d_date_sk#22] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=3] + +(20) BroadcastHashJoin [codegen id : 5] +Left keys [1]: [ss_sold_date_sk#5] +Right keys [1]: [d_date_sk#22] +Join type: Inner +Join condition: None + +(21) Project [codegen id : 5] +Output [12]: [ss_sold_date_sk#5, sr_returned_date_sk#9, s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#16, s_suite_number#17, s_city#18, s_county#19, s_state#20, s_zip#21] +Input [13]: [ss_sold_date_sk#5, sr_returned_date_sk#9, s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#16, s_suite_number#17, s_city#18, s_county#19, s_state#20, s_zip#21, d_date_sk#22] + +(22) ReusedExchange [Reuses operator id: 33] +Output [1]: [d_date_sk#23] + +(23) BroadcastHashJoin [codegen id : 5] +Left keys [1]: [sr_returned_date_sk#9] +Right keys [1]: [d_date_sk#23] +Join type: Inner +Join condition: None + +(24) Project [codegen id : 5] +Output [12]: [ss_sold_date_sk#5, sr_returned_date_sk#9, s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#16, s_suite_number#17, s_city#18, s_county#19, s_state#20, s_zip#21] +Input [13]: [ss_sold_date_sk#5, sr_returned_date_sk#9, s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#16, s_suite_number#17, s_city#18, s_county#19, s_state#20, s_zip#21, d_date_sk#23] + +(25) HashAggregate [codegen id : 5] +Input [12]: [ss_sold_date_sk#5, sr_returned_date_sk#9, s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#16, s_suite_number#17, s_city#18, s_county#19, s_state#20, s_zip#21] +Keys [10]: [s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#16, s_suite_number#17, s_city#18, s_county#19, s_state#20, s_zip#21] +Functions [5]: [partial_sum(CASE WHEN ((sr_returned_date_sk#9 - ss_sold_date_sk#5) <= 30) THEN 1 ELSE 0 END), partial_sum(CASE WHEN (((sr_returned_date_sk#9 - ss_sold_date_sk#5) > 30) AND ((sr_returned_date_sk#9 - ss_sold_date_sk#5) <= 60)) THEN 1 ELSE 0 END), partial_sum(CASE WHEN (((sr_returned_date_sk#9 - ss_sold_date_sk#5) > 60) AND ((sr_returned_date_sk#9 - ss_sold_date_sk#5) <= 90)) THEN 1 ELSE 0 END), partial_sum(CASE WHEN (((sr_returned_date_sk#9 - ss_sold_date_sk#5) > 90) AND ((sr_returned_date_sk#9 - ss_sold_date_sk#5) <= 120)) THEN 1 ELSE 0 END), partial_sum(CASE WHEN ((sr_returned_date_sk#9 - ss_sold_date_sk#5) > 120) THEN 1 ELSE 0 END)] +Aggregate Attributes [5]: [sum#24, sum#25, sum#26, sum#27, sum#28] +Results [15]: [s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#16, s_suite_number#17, s_city#18, s_county#19, s_state#20, s_zip#21, sum#29, sum#30, sum#31, sum#32, sum#33] + +(26) Exchange +Input [15]: [s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#16, s_suite_number#17, s_city#18, s_county#19, s_state#20, s_zip#21, sum#29, sum#30, sum#31, sum#32, sum#33] +Arguments: hashpartitioning(s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#16, s_suite_number#17, s_city#18, s_county#19, s_state#20, s_zip#21, 5), ENSURE_REQUIREMENTS, [plan_id=4] + +(27) HashAggregate [codegen id : 6] +Input [15]: [s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#16, s_suite_number#17, s_city#18, s_county#19, s_state#20, s_zip#21, sum#29, sum#30, sum#31, sum#32, sum#33] +Keys [10]: [s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#16, s_suite_number#17, s_city#18, s_county#19, s_state#20, s_zip#21] +Functions [5]: [sum(CASE WHEN ((sr_returned_date_sk#9 - ss_sold_date_sk#5) <= 30) THEN 1 ELSE 0 END), sum(CASE WHEN (((sr_returned_date_sk#9 - ss_sold_date_sk#5) > 30) AND ((sr_returned_date_sk#9 - ss_sold_date_sk#5) <= 60)) THEN 1 ELSE 0 END), sum(CASE WHEN (((sr_returned_date_sk#9 - ss_sold_date_sk#5) > 60) AND ((sr_returned_date_sk#9 - ss_sold_date_sk#5) <= 90)) THEN 1 ELSE 0 END), sum(CASE WHEN (((sr_returned_date_sk#9 - ss_sold_date_sk#5) > 90) AND ((sr_returned_date_sk#9 - ss_sold_date_sk#5) <= 120)) THEN 1 ELSE 0 END), sum(CASE WHEN ((sr_returned_date_sk#9 - ss_sold_date_sk#5) > 120) THEN 1 ELSE 0 END)] +Aggregate Attributes [5]: [sum(CASE WHEN ((sr_returned_date_sk#9 - ss_sold_date_sk#5) <= 30) THEN 1 ELSE 0 END)#34, sum(CASE WHEN (((sr_returned_date_sk#9 - ss_sold_date_sk#5) > 30) AND ((sr_returned_date_sk#9 - ss_sold_date_sk#5) <= 60)) THEN 1 ELSE 0 END)#35, sum(CASE WHEN (((sr_returned_date_sk#9 - ss_sold_date_sk#5) > 60) AND ((sr_returned_date_sk#9 - ss_sold_date_sk#5) <= 90)) THEN 1 ELSE 0 END)#36, sum(CASE WHEN (((sr_returned_date_sk#9 - ss_sold_date_sk#5) > 90) AND ((sr_returned_date_sk#9 - ss_sold_date_sk#5) <= 120)) THEN 1 ELSE 0 END)#37, sum(CASE WHEN ((sr_returned_date_sk#9 - ss_sold_date_sk#5) > 120) THEN 1 ELSE 0 END)#38] +Results [15]: [s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#16, s_suite_number#17, s_city#18, s_county#19, s_state#20, s_zip#21, sum(CASE WHEN ((sr_returned_date_sk#9 - ss_sold_date_sk#5) <= 30) THEN 1 ELSE 0 END)#34 AS 30 days #39, sum(CASE WHEN (((sr_returned_date_sk#9 - ss_sold_date_sk#5) > 30) AND ((sr_returned_date_sk#9 - ss_sold_date_sk#5) <= 60)) THEN 1 ELSE 0 END)#35 AS 31 - 60 days #40, sum(CASE WHEN (((sr_returned_date_sk#9 - ss_sold_date_sk#5) > 60) AND ((sr_returned_date_sk#9 - ss_sold_date_sk#5) <= 90)) THEN 1 ELSE 0 END)#36 AS 61 - 90 days #41, sum(CASE WHEN (((sr_returned_date_sk#9 - ss_sold_date_sk#5) > 90) AND ((sr_returned_date_sk#9 - ss_sold_date_sk#5) <= 120)) THEN 1 ELSE 0 END)#37 AS 91 - 120 days #42, sum(CASE WHEN ((sr_returned_date_sk#9 - ss_sold_date_sk#5) > 120) THEN 1 ELSE 0 END)#38 AS >120 days #43] + +(28) TakeOrderedAndProject +Input [15]: [s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#16, s_suite_number#17, s_city#18, s_county#19, s_state#20, s_zip#21, 30 days #39, 31 - 60 days #40, 61 - 90 days #41, 91 - 120 days #42, >120 days #43] +Arguments: 100, [s_store_name#12 ASC NULLS FIRST, s_company_id#13 ASC NULLS FIRST, s_street_number#14 ASC NULLS FIRST, s_street_name#15 ASC NULLS FIRST, s_street_type#16 ASC NULLS FIRST, s_suite_number#17 ASC NULLS FIRST, s_city#18 ASC NULLS FIRST, s_county#19 ASC NULLS FIRST, s_state#20 ASC NULLS FIRST, s_zip#21 ASC NULLS FIRST], [s_store_name#12, s_company_id#13, s_street_number#14, s_street_name#15, s_street_type#16, s_suite_number#17, s_city#18, s_county#19, s_state#20, s_zip#21, 30 days #39, 31 - 60 days #40, 61 - 90 days #41, 91 - 120 days #42, >120 days #43] + +===== Subqueries ===== + +Subquery:1 Hosting operator id = 4 Hosting Expression = sr_returned_date_sk#9 IN dynamicpruning#10 +BroadcastExchange (33) ++- * ColumnarToRow (32) + +- CometProject (31) + +- CometFilter (30) + +- CometScan parquet spark_catalog.default.date_dim (29) + + +(29) Scan parquet spark_catalog.default.date_dim +Output [3]: [d_date_sk#23, d_year#44, d_moy#45] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2001), EqualTo(d_moy,8), IsNotNull(d_date_sk)] +ReadSchema: struct<d_date_sk:int,d_year:int,d_moy:int> + +(30) CometFilter +Input [3]: [d_date_sk#23, d_year#44, d_moy#45] +Condition : ((((isnotnull(d_year#44) AND isnotnull(d_moy#45)) AND (d_year#44 = 2001)) AND (d_moy#45 = 8)) AND isnotnull(d_date_sk#23)) + +(31) CometProject +Input [3]: [d_date_sk#23, d_year#44, d_moy#45] +Arguments: [d_date_sk#23], [d_date_sk#23] + +(32) ColumnarToRow [codegen id : 1] +Input [1]: [d_date_sk#23] + +(33) BroadcastExchange +Input [1]: [d_date_sk#23] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] + + 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 new file mode 100644 index 000000000..dfdcaf497 --- /dev/null +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q50/simplified.txt @@ -0,0 +1,50 @@ +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) + HashAggregate [s_store_name,s_company_id,s_street_number,s_street_name,s_street_type,s_suite_number,s_city,s_county,s_state,s_zip,sum,sum,sum,sum,sum] [sum(CASE WHEN ((sr_returned_date_sk - ss_sold_date_sk) <= 30) THEN 1 ELSE 0 END),sum(CASE WHEN (((sr_returned_date_sk - ss_sold_date_sk) > 30) AND ((sr_returned_date_sk - ss_sold_date_sk) <= 60)) THEN 1 ELSE 0 END),sum(CASE WHEN (((sr_returned_date_sk - ss_sold_date_sk) > 60) AND ((sr_returned_date_sk - ss_sold_date_sk) <= 90)) THEN 1 ELSE 0 END),sum(CASE WHEN (((sr_returned_date_sk - ss_sold_date_sk) > 90) AND ((sr_returned_date_sk - ss_sold_date_sk) <= 120)) THEN 1 ELSE 0 END),sum(CASE WHEN ((sr_returned_date_sk - ss_sold_date_sk) > 120) THEN 1 ELSE 0 END),30 days ,31 - 60 days ,61 - 90 days ,91 - 120 days ,>120 days ,sum,sum,sum,sum,sum] + InputAdapter + Exchange [s_store_name,s_company_id,s_street_number,s_street_name,s_street_type,s_suite_number,s_city,s_county,s_state,s_zip] #1 + WholeStageCodegen (5) + HashAggregate [s_store_name,s_company_id,s_street_number,s_street_name,s_street_type,s_suite_number,s_city,s_county,s_state,s_zip,sr_returned_date_sk,ss_sold_date_sk] [sum,sum,sum,sum,sum,sum,sum,sum,sum,sum] + Project [ss_sold_date_sk,sr_returned_date_sk,s_store_name,s_company_id,s_street_number,s_street_name,s_street_type,s_suite_number,s_city,s_county,s_state,s_zip] + BroadcastHashJoin [sr_returned_date_sk,d_date_sk] + Project [ss_sold_date_sk,sr_returned_date_sk,s_store_name,s_company_id,s_street_number,s_street_name,s_street_type,s_suite_number,s_city,s_county,s_state,s_zip] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [ss_sold_date_sk,sr_returned_date_sk,s_store_name,s_company_id,s_street_number,s_street_name,s_street_type,s_suite_number,s_city,s_county,s_state,s_zip] + BroadcastHashJoin [ss_store_sk,s_store_sk] + Project [ss_store_sk,ss_sold_date_sk,sr_returned_date_sk] + BroadcastHashJoin [ss_ticket_number,ss_item_sk,ss_customer_sk,sr_ticket_number,sr_item_sk,sr_customer_sk] + ColumnarToRow + InputAdapter + CometFilter [ss_ticket_number,ss_item_sk,ss_customer_sk,ss_store_sk] + CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_sold_date_sk] + InputAdapter + BroadcastExchange #2 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometFilter [sr_ticket_number,sr_item_sk,sr_customer_sk] + CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_customer_sk,sr_ticket_number,sr_returned_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_year,d_moy,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + InputAdapter + BroadcastExchange #4 + WholeStageCodegen (2) + ColumnarToRow + InputAdapter + CometFilter [s_store_sk] + CometScan parquet spark_catalog.default.store [s_store_sk,s_store_name,s_company_id,s_street_number,s_street_name,s_street_type,s_suite_number,s_city,s_county,s_state,s_zip] + InputAdapter + BroadcastExchange #5 + WholeStageCodegen (3) + ColumnarToRow + InputAdapter + CometFilter [d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk] + InputAdapter + ReusedExchange [d_date_sk] #3 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 new file mode 100644 index 000000000..2613551f0 --- /dev/null +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q51/explain.txt @@ -0,0 +1,245 @@ +== Physical Plan == +TakeOrderedAndProject (37) ++- * Filter (36) + +- Window (35) + +- * Sort (34) + +- Exchange (33) + +- * Project (32) + +- * SortMergeJoin FullOuter (31) + :- * Sort (15) + : +- Exchange (14) + : +- * Project (13) + : +- Window (12) + : +- * Sort (11) + : +- Exchange (10) + : +- * HashAggregate (9) + : +- Exchange (8) + : +- * HashAggregate (7) + : +- * Project (6) + : +- * BroadcastHashJoin Inner BuildRight (5) + : :- * ColumnarToRow (3) + : : +- CometFilter (2) + : : +- CometScan parquet spark_catalog.default.web_sales (1) + : +- ReusedExchange (4) + +- * Sort (30) + +- Exchange (29) + +- * Project (28) + +- Window (27) + +- * Sort (26) + +- Exchange (25) + +- * HashAggregate (24) + +- Exchange (23) + +- * HashAggregate (22) + +- * Project (21) + +- * BroadcastHashJoin Inner BuildRight (20) + :- * ColumnarToRow (18) + : +- CometFilter (17) + : +- CometScan parquet spark_catalog.default.store_sales (16) + +- ReusedExchange (19) + + +(1) Scan parquet spark_catalog.default.web_sales +Output [3]: [ws_item_sk#1, ws_sales_price#2, ws_sold_date_sk#3] +Batched: true +Location: InMemoryFileIndex [] +PartitionFilters: [isnotnull(ws_sold_date_sk#3), dynamicpruningexpression(ws_sold_date_sk#3 IN dynamicpruning#4)] +PushedFilters: [IsNotNull(ws_item_sk)] +ReadSchema: struct<ws_item_sk:int,ws_sales_price:decimal(7,2)> + +(2) CometFilter +Input [3]: [ws_item_sk#1, ws_sales_price#2, ws_sold_date_sk#3] +Condition : isnotnull(ws_item_sk#1) + +(3) ColumnarToRow [codegen id : 2] +Input [3]: [ws_item_sk#1, ws_sales_price#2, ws_sold_date_sk#3] + +(4) ReusedExchange [Reuses operator id: 42] +Output [2]: [d_date_sk#5, d_date#6] + +(5) BroadcastHashJoin [codegen id : 2] +Left keys [1]: [ws_sold_date_sk#3] +Right keys [1]: [d_date_sk#5] +Join type: Inner +Join condition: None + +(6) Project [codegen id : 2] +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] +Input [3]: [ws_item_sk#1, ws_sales_price#2, d_date#6] +Keys [2]: [ws_item_sk#1, d_date#6] +Functions [1]: [partial_sum(UnscaledValue(ws_sales_price#2))] +Aggregate Attributes [1]: [sum#7] +Results [3]: [ws_item_sk#1, d_date#6, sum#8] + +(8) Exchange +Input [3]: [ws_item_sk#1, d_date#6, sum#8] +Arguments: hashpartitioning(ws_item_sk#1, d_date#6, 5), ENSURE_REQUIREMENTS, [plan_id=1] + +(9) HashAggregate [codegen id : 3] +Input [3]: [ws_item_sk#1, d_date#6, sum#8] +Keys [2]: [ws_item_sk#1, d_date#6] +Functions [1]: [sum(UnscaledValue(ws_sales_price#2))] +Aggregate Attributes [1]: [sum(UnscaledValue(ws_sales_price#2))#9] +Results [4]: [ws_item_sk#1 AS item_sk#10, d_date#6, MakeDecimal(sum(UnscaledValue(ws_sales_price#2))#9,17,2) AS _w0#11, ws_item_sk#1] + +(10) Exchange +Input [4]: [item_sk#10, d_date#6, _w0#11, ws_item_sk#1] +Arguments: hashpartitioning(ws_item_sk#1, 5), ENSURE_REQUIREMENTS, [plan_id=2] + +(11) Sort [codegen id : 4] +Input [4]: [item_sk#10, d_date#6, _w0#11, ws_item_sk#1] +Arguments: [ws_item_sk#1 ASC NULLS FIRST, d_date#6 ASC NULLS FIRST], false, 0 + +(12) Window +Input [4]: [item_sk#10, d_date#6, _w0#11, ws_item_sk#1] +Arguments: [sum(_w0#11) windowspecdefinition(ws_item_sk#1, d_date#6 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS cume_sales#12], [ws_item_sk#1], [d_date#6 ASC NULLS FIRST] + +(13) Project [codegen id : 5] +Output [3]: [item_sk#10, d_date#6, cume_sales#12] +Input [5]: [item_sk#10, d_date#6, _w0#11, ws_item_sk#1, cume_sales#12] + +(14) Exchange +Input [3]: [item_sk#10, d_date#6, cume_sales#12] +Arguments: hashpartitioning(item_sk#10, d_date#6, 5), ENSURE_REQUIREMENTS, [plan_id=3] + +(15) Sort [codegen id : 6] +Input [3]: [item_sk#10, d_date#6, cume_sales#12] +Arguments: [item_sk#10 ASC NULLS FIRST, d_date#6 ASC NULLS FIRST], false, 0 + +(16) Scan parquet spark_catalog.default.store_sales +Output [3]: [ss_item_sk#13, ss_sales_price#14, ss_sold_date_sk#15] +Batched: true +Location: InMemoryFileIndex [] +PartitionFilters: [isnotnull(ss_sold_date_sk#15), dynamicpruningexpression(ss_sold_date_sk#15 IN dynamicpruning#16)] +PushedFilters: [IsNotNull(ss_item_sk)] +ReadSchema: struct<ss_item_sk:int,ss_sales_price:decimal(7,2)> + +(17) CometFilter +Input [3]: [ss_item_sk#13, ss_sales_price#14, ss_sold_date_sk#15] +Condition : isnotnull(ss_item_sk#13) + +(18) ColumnarToRow [codegen id : 8] +Input [3]: [ss_item_sk#13, ss_sales_price#14, ss_sold_date_sk#15] + +(19) ReusedExchange [Reuses operator id: 42] +Output [2]: [d_date_sk#17, d_date#18] + +(20) BroadcastHashJoin [codegen id : 8] +Left keys [1]: [ss_sold_date_sk#15] +Right keys [1]: [d_date_sk#17] +Join type: Inner +Join condition: None + +(21) Project [codegen id : 8] +Output [3]: [ss_item_sk#13, ss_sales_price#14, d_date#18] +Input [5]: [ss_item_sk#13, ss_sales_price#14, ss_sold_date_sk#15, d_date_sk#17, d_date#18] + +(22) HashAggregate [codegen id : 8] +Input [3]: [ss_item_sk#13, ss_sales_price#14, d_date#18] +Keys [2]: [ss_item_sk#13, d_date#18] +Functions [1]: [partial_sum(UnscaledValue(ss_sales_price#14))] +Aggregate Attributes [1]: [sum#19] +Results [3]: [ss_item_sk#13, d_date#18, sum#20] + +(23) Exchange +Input [3]: [ss_item_sk#13, d_date#18, sum#20] +Arguments: hashpartitioning(ss_item_sk#13, d_date#18, 5), ENSURE_REQUIREMENTS, [plan_id=4] + +(24) HashAggregate [codegen id : 9] +Input [3]: [ss_item_sk#13, d_date#18, sum#20] +Keys [2]: [ss_item_sk#13, d_date#18] +Functions [1]: [sum(UnscaledValue(ss_sales_price#14))] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_sales_price#14))#21] +Results [4]: [ss_item_sk#13 AS item_sk#22, d_date#18, MakeDecimal(sum(UnscaledValue(ss_sales_price#14))#21,17,2) AS _w0#23, ss_item_sk#13] + +(25) Exchange +Input [4]: [item_sk#22, d_date#18, _w0#23, ss_item_sk#13] +Arguments: hashpartitioning(ss_item_sk#13, 5), ENSURE_REQUIREMENTS, [plan_id=5] + +(26) Sort [codegen id : 10] +Input [4]: [item_sk#22, d_date#18, _w0#23, ss_item_sk#13] +Arguments: [ss_item_sk#13 ASC NULLS FIRST, d_date#18 ASC NULLS FIRST], false, 0 + +(27) Window +Input [4]: [item_sk#22, d_date#18, _w0#23, ss_item_sk#13] +Arguments: [sum(_w0#23) windowspecdefinition(ss_item_sk#13, d_date#18 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS cume_sales#24], [ss_item_sk#13], [d_date#18 ASC NULLS FIRST] + +(28) Project [codegen id : 11] +Output [3]: [item_sk#22, d_date#18, cume_sales#24] +Input [5]: [item_sk#22, d_date#18, _w0#23, ss_item_sk#13, cume_sales#24] + +(29) Exchange +Input [3]: [item_sk#22, d_date#18, cume_sales#24] +Arguments: hashpartitioning(item_sk#22, d_date#18, 5), ENSURE_REQUIREMENTS, [plan_id=6] + +(30) Sort [codegen id : 12] +Input [3]: [item_sk#22, d_date#18, cume_sales#24] +Arguments: [item_sk#22 ASC NULLS FIRST, d_date#18 ASC NULLS FIRST], false, 0 + +(31) SortMergeJoin [codegen id : 13] +Left keys [2]: [item_sk#10, d_date#6] +Right keys [2]: [item_sk#22, d_date#18] +Join type: FullOuter +Join condition: None + +(32) Project [codegen id : 13] +Output [4]: [CASE WHEN isnotnull(item_sk#10) THEN item_sk#10 ELSE item_sk#22 END AS item_sk#25, CASE WHEN isnotnull(d_date#6) THEN d_date#6 ELSE d_date#18 END AS d_date#26, cume_sales#12 AS web_sales#27, cume_sales#24 AS store_sales#28] +Input [6]: [item_sk#10, d_date#6, cume_sales#12, item_sk#22, d_date#18, cume_sales#24] + +(33) Exchange +Input [4]: [item_sk#25, d_date#26, web_sales#27, store_sales#28] +Arguments: hashpartitioning(item_sk#25, 5), ENSURE_REQUIREMENTS, [plan_id=7] + +(34) Sort [codegen id : 14] +Input [4]: [item_sk#25, d_date#26, web_sales#27, store_sales#28] +Arguments: [item_sk#25 ASC NULLS FIRST, d_date#26 ASC NULLS FIRST], false, 0 + +(35) Window +Input [4]: [item_sk#25, d_date#26, web_sales#27, store_sales#28] +Arguments: [max(web_sales#27) windowspecdefinition(item_sk#25, d_date#26 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS web_cumulative#29, max(store_sales#28) windowspecdefinition(item_sk#25, d_date#26 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS store_cumulative#30], [item_sk#25], [d_date#26 ASC NULLS FIRST] + +(36) Filter [codegen id : 15] +Input [6]: [item_sk#25, d_date#26, web_sales#27, store_sales#28, web_cumulative#29, store_cumulative#30] +Condition : ((isnotnull(web_cumulative#29) AND isnotnull(store_cumulative#30)) AND (web_cumulative#29 > store_cumulative#30)) + +(37) TakeOrderedAndProject +Input [6]: [item_sk#25, d_date#26, web_sales#27, store_sales#28, web_cumulative#29, store_cumulative#30] +Arguments: 100, [item_sk#25 ASC NULLS FIRST, d_date#26 ASC NULLS FIRST], [item_sk#25, d_date#26, web_sales#27, store_sales#28, web_cumulative#29, store_cumulative#30] + +===== Subqueries ===== + +Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#3 IN dynamicpruning#4 +BroadcastExchange (42) ++- * ColumnarToRow (41) + +- CometProject (40) + +- CometFilter (39) + +- CometScan parquet spark_catalog.default.date_dim (38) + + +(38) Scan parquet spark_catalog.default.date_dim +Output [3]: [d_date_sk#5, d_date#6, d_month_seq#31] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1200), LessThanOrEqual(d_month_seq,1211), IsNotNull(d_date_sk)] +ReadSchema: struct<d_date_sk:int,d_date:date,d_month_seq:int> + +(39) CometFilter +Input [3]: [d_date_sk#5, d_date#6, d_month_seq#31] +Condition : (((isnotnull(d_month_seq#31) AND (d_month_seq#31 >= 1200)) AND (d_month_seq#31 <= 1211)) AND isnotnull(d_date_sk#5)) + +(40) CometProject +Input [3]: [d_date_sk#5, d_date#6, d_month_seq#31] +Arguments: [d_date_sk#5, d_date#6], [d_date_sk#5, d_date#6] + +(41) ColumnarToRow [codegen id : 1] +Input [2]: [d_date_sk#5, d_date#6] + +(42) BroadcastExchange +Input [2]: [d_date_sk#5, d_date#6] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=8] + +Subquery:2 Hosting operator id = 16 Hosting Expression = ss_sold_date_sk#15 IN dynamicpruning#4 + + 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 new file mode 100644 index 000000000..181cd1b98 --- /dev/null +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q51/simplified.txt @@ -0,0 +1,75 @@ +TakeOrderedAndProject [item_sk,d_date,web_sales,store_sales,web_cumulative,store_cumulative] + WholeStageCodegen (15) + Filter [web_cumulative,store_cumulative] + InputAdapter + Window [web_sales,item_sk,d_date,store_sales] + WholeStageCodegen (14) + Sort [item_sk,d_date] + InputAdapter + Exchange [item_sk] #1 + WholeStageCodegen (13) + Project [item_sk,item_sk,d_date,d_date,cume_sales,cume_sales] + SortMergeJoin [item_sk,d_date,item_sk,d_date] + InputAdapter + WholeStageCodegen (6) + Sort [item_sk,d_date] + InputAdapter + Exchange [item_sk,d_date] #2 + WholeStageCodegen (5) + Project [item_sk,d_date,cume_sales] + InputAdapter + Window [_w0,ws_item_sk,d_date] + WholeStageCodegen (4) + Sort [ws_item_sk,d_date] + InputAdapter + Exchange [ws_item_sk] #3 + WholeStageCodegen (3) + HashAggregate [ws_item_sk,d_date,sum] [sum(UnscaledValue(ws_sales_price)),item_sk,_w0,sum] + InputAdapter + Exchange [ws_item_sk,d_date] #4 + WholeStageCodegen (2) + HashAggregate [ws_item_sk,d_date,ws_sales_price] [sum,sum] + Project [ws_item_sk,ws_sales_price,d_date] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + ColumnarToRow + InputAdapter + CometFilter [ws_item_sk] + CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_sales_price,ws_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #5 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk,d_date] + CometFilter [d_month_seq,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_month_seq] + InputAdapter + ReusedExchange [d_date_sk,d_date] #5 + InputAdapter + WholeStageCodegen (12) + Sort [item_sk,d_date] + InputAdapter + Exchange [item_sk,d_date] #6 + WholeStageCodegen (11) + Project [item_sk,d_date,cume_sales] + InputAdapter + Window [_w0,ss_item_sk,d_date] + WholeStageCodegen (10) + Sort [ss_item_sk,d_date] + InputAdapter + Exchange [ss_item_sk] #7 + WholeStageCodegen (9) + HashAggregate [ss_item_sk,d_date,sum] [sum(UnscaledValue(ss_sales_price)),item_sk,_w0,sum] + InputAdapter + Exchange [ss_item_sk,d_date] #8 + WholeStageCodegen (8) + HashAggregate [ss_item_sk,d_date,ss_sales_price] [sum,sum] + Project [ss_item_sk,ss_sales_price,d_date] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + ColumnarToRow + InputAdapter + CometFilter [ss_item_sk] + CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_sales_price,ss_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + InputAdapter + ReusedExchange [d_date_sk,d_date] #5 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q52/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q52/explain.txt new file mode 100644 index 000000000..3d5317eb0 --- /dev/null +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q52/explain.txt @@ -0,0 +1,125 @@ +== Physical Plan == +TakeOrderedAndProject (21) ++- * HashAggregate (20) + +- Exchange (19) + +- * HashAggregate (18) + +- * Project (17) + +- * BroadcastHashJoin Inner BuildRight (16) + :- * Project (10) + : +- * BroadcastHashJoin Inner BuildRight (9) + : :- * ColumnarToRow (4) + : : +- CometProject (3) + : : +- CometFilter (2) + : : +- CometScan parquet spark_catalog.default.date_dim (1) + : +- BroadcastExchange (8) + : +- * ColumnarToRow (7) + : +- CometFilter (6) + : +- CometScan parquet spark_catalog.default.store_sales (5) + +- BroadcastExchange (15) + +- * ColumnarToRow (14) + +- CometProject (13) + +- CometFilter (12) + +- CometScan parquet spark_catalog.default.item (11) + + +(1) Scan parquet spark_catalog.default.date_dim +Output [3]: [d_date_sk#1, d_year#2, d_moy#3] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_moy), IsNotNull(d_year), EqualTo(d_moy,11), EqualTo(d_year,2000), IsNotNull(d_date_sk)] +ReadSchema: struct<d_date_sk:int,d_year:int,d_moy:int> + +(2) CometFilter +Input [3]: [d_date_sk#1, d_year#2, d_moy#3] +Condition : ((((isnotnull(d_moy#3) AND isnotnull(d_year#2)) AND (d_moy#3 = 11)) AND (d_year#2 = 2000)) AND isnotnull(d_date_sk#1)) + +(3) CometProject +Input [3]: [d_date_sk#1, d_year#2, d_moy#3] +Arguments: [d_date_sk#1, d_year#2], [d_date_sk#1, d_year#2] + +(4) ColumnarToRow [codegen id : 3] +Input [2]: [d_date_sk#1, d_year#2] + +(5) Scan parquet spark_catalog.default.store_sales +Output [3]: [ss_item_sk#4, ss_ext_sales_price#5, ss_sold_date_sk#6] +Batched: true +Location: InMemoryFileIndex [] +PartitionFilters: [isnotnull(ss_sold_date_sk#6), dynamicpruningexpression(true)] +PushedFilters: [IsNotNull(ss_item_sk)] +ReadSchema: struct<ss_item_sk:int,ss_ext_sales_price:decimal(7,2)> + +(6) CometFilter +Input [3]: [ss_item_sk#4, ss_ext_sales_price#5, ss_sold_date_sk#6] +Condition : isnotnull(ss_item_sk#4) + +(7) ColumnarToRow [codegen id : 1] +Input [3]: [ss_item_sk#4, ss_ext_sales_price#5, ss_sold_date_sk#6] + +(8) BroadcastExchange +Input [3]: [ss_item_sk#4, ss_ext_sales_price#5, ss_sold_date_sk#6] +Arguments: HashedRelationBroadcastMode(List(cast(input[2, int, true] as bigint)),false), [plan_id=1] + +(9) BroadcastHashJoin [codegen id : 3] +Left keys [1]: [d_date_sk#1] +Right keys [1]: [ss_sold_date_sk#6] +Join type: Inner +Join condition: None + +(10) Project [codegen id : 3] +Output [3]: [d_year#2, ss_item_sk#4, ss_ext_sales_price#5] +Input [5]: [d_date_sk#1, d_year#2, ss_item_sk#4, ss_ext_sales_price#5, ss_sold_date_sk#6] + +(11) Scan parquet spark_catalog.default.item +Output [4]: [i_item_sk#7, i_brand_id#8, i_brand#9, i_manager_id#10] +Batched: true +Location [not included in comparison]/{warehouse_dir}/item] +PushedFilters: [IsNotNull(i_manager_id), EqualTo(i_manager_id,1), IsNotNull(i_item_sk)] +ReadSchema: struct<i_item_sk:int,i_brand_id:int,i_brand:string,i_manager_id:int> + +(12) CometFilter +Input [4]: [i_item_sk#7, i_brand_id#8, i_brand#9, i_manager_id#10] +Condition : ((isnotnull(i_manager_id#10) AND (i_manager_id#10 = 1)) AND isnotnull(i_item_sk#7)) + +(13) CometProject +Input [4]: [i_item_sk#7, i_brand_id#8, i_brand#9, i_manager_id#10] +Arguments: [i_item_sk#7, i_brand_id#8, i_brand#9], [i_item_sk#7, i_brand_id#8, i_brand#9] + +(14) ColumnarToRow [codegen id : 2] +Input [3]: [i_item_sk#7, i_brand_id#8, i_brand#9] + +(15) BroadcastExchange +Input [3]: [i_item_sk#7, i_brand_id#8, i_brand#9] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] + +(16) BroadcastHashJoin [codegen id : 3] +Left keys [1]: [ss_item_sk#4] +Right keys [1]: [i_item_sk#7] +Join type: Inner +Join condition: None + +(17) Project [codegen id : 3] +Output [4]: [d_year#2, ss_ext_sales_price#5, i_brand_id#8, i_brand#9] +Input [6]: [d_year#2, ss_item_sk#4, ss_ext_sales_price#5, i_item_sk#7, i_brand_id#8, i_brand#9] + +(18) HashAggregate [codegen id : 3] +Input [4]: [d_year#2, ss_ext_sales_price#5, i_brand_id#8, i_brand#9] +Keys [3]: [d_year#2, i_brand#9, i_brand_id#8] +Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#5))] +Aggregate Attributes [1]: [sum#11] +Results [4]: [d_year#2, i_brand#9, i_brand_id#8, sum#12] + +(19) Exchange +Input [4]: [d_year#2, i_brand#9, i_brand_id#8, sum#12] +Arguments: hashpartitioning(d_year#2, i_brand#9, i_brand_id#8, 5), ENSURE_REQUIREMENTS, [plan_id=3] + +(20) HashAggregate [codegen id : 4] +Input [4]: [d_year#2, i_brand#9, i_brand_id#8, sum#12] +Keys [3]: [d_year#2, i_brand#9, i_brand_id#8] +Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#5))] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#5))#13] +Results [4]: [d_year#2, i_brand_id#8 AS brand_id#14, i_brand#9 AS brand#15, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#5))#13,17,2) AS ext_price#16] + +(21) TakeOrderedAndProject +Input [4]: [d_year#2, brand_id#14, brand#15, ext_price#16] +Arguments: 100, [d_year#2 ASC NULLS FIRST, ext_price#16 DESC NULLS LAST, brand_id#14 ASC NULLS FIRST], [d_year#2, brand_id#14, brand#15, ext_price#16] + diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q52/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q52/simplified.txt new file mode 100644 index 000000000..91fdc2f17 --- /dev/null +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q52/simplified.txt @@ -0,0 +1,31 @@ +TakeOrderedAndProject [d_year,ext_price,brand_id,brand] + WholeStageCodegen (4) + HashAggregate [d_year,i_brand,i_brand_id,sum] [sum(UnscaledValue(ss_ext_sales_price)),brand_id,brand,ext_price,sum] + InputAdapter + Exchange [d_year,i_brand,i_brand_id] #1 + WholeStageCodegen (3) + HashAggregate [d_year,i_brand,i_brand_id,ss_ext_sales_price] [sum,sum] + Project [d_year,ss_ext_sales_price,i_brand_id,i_brand] + BroadcastHashJoin [ss_item_sk,i_item_sk] + Project [d_year,ss_item_sk,ss_ext_sales_price] + BroadcastHashJoin [d_date_sk,ss_sold_date_sk] + ColumnarToRow + InputAdapter + CometProject [d_date_sk,d_year] + CometFilter [d_moy,d_year,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + InputAdapter + BroadcastExchange #2 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometFilter [ss_item_sk] + CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] + InputAdapter + BroadcastExchange #3 + WholeStageCodegen (2) + ColumnarToRow + InputAdapter + CometProject [i_item_sk,i_brand_id,i_brand] + CometFilter [i_manager_id,i_item_sk] + CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_brand,i_manager_id] 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 new file mode 100644 index 000000000..335dc7fa2 --- /dev/null +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q53/explain.txt @@ -0,0 +1,194 @@ +== Physical Plan == +TakeOrderedAndProject (28) ++- * Project (27) + +- * Filter (26) + +- Window (25) + +- * Sort (24) + +- Exchange (23) + +- * HashAggregate (22) + +- Exchange (21) + +- * HashAggregate (20) + +- * Project (19) + +- * BroadcastHashJoin Inner BuildRight (18) + :- * Project (13) + : +- * BroadcastHashJoin Inner BuildRight (12) + : :- * Project (10) + : : +- * BroadcastHashJoin Inner BuildRight (9) + : : :- * ColumnarToRow (4) + : : : +- CometProject (3) + : : : +- CometFilter (2) + : : : +- CometScan parquet spark_catalog.default.item (1) + : : +- BroadcastExchange (8) + : : +- * ColumnarToRow (7) + : : +- CometFilter (6) + : : +- CometScan parquet spark_catalog.default.store_sales (5) + : +- ReusedExchange (11) + +- BroadcastExchange (17) + +- * ColumnarToRow (16) + +- CometFilter (15) + +- CometScan parquet spark_catalog.default.store (14) + + +(1) Scan parquet spark_catalog.default.item +Output [5]: [i_item_sk#1, i_brand#2, i_class#3, i_category#4, i_manufact_id#5] +Batched: true +Location [not included in comparison]/{warehouse_dir}/item] +PushedFilters: [Or(And(And(In(i_category, [Books ,Children ,Electronics ]),In(i_class, [personal ,portable ,reference ,self-help ])),In(i_brand, [exportiunivamalg #6 ,scholaramalgamalg #7 ,scholaramalgamalg #8 ,scholaramalgamalg #6 ])),And(And(In(i_category, [Men ,Music ,Women ]),In(i_class, [accessories ,classical ,fragrances ,pants ])),In(i_brand, [amalgimporto #9 ,edu packscholar #9 ,exportiimporto #9 ,importoamalg #9 ]))), IsNotNull(i_item_sk)] +ReadSchema: struct<i_item_sk:int,i_brand:string,i_class:string,i_category:string,i_manufact_id:int> + +(2) CometFilter +Input [5]: [i_item_sk#1, i_brand#2, i_class#3, i_category#4, i_manufact_id#5] +Condition : ((((i_category#4 IN (Books ,Children ,Electronics ) AND i_class#3 IN (personal ,portable ,reference ,self-help )) AND i_brand#2 IN (scholaramalgamalg #7 ,scholaramalgamalg #8 ,exportiunivamalg #6 ,scholaramalgamalg #6 )) OR ((i_category#4 IN (Women ,Music ,Men ) AND i_class#3 IN (accessories ,classical ,fragrances ,pants )) AND i_brand#2 IN (amalgimporto #9 ,edu packscholar #9 ,exportiimporto #9 ,importoamalg #9 ))) AND isnotnull(i_item_sk#1)) + +(3) CometProject +Input [5]: [i_item_sk#1, i_brand#2, i_class#3, i_category#4, i_manufact_id#5] +Arguments: [i_item_sk#1, i_manufact_id#5], [i_item_sk#1, i_manufact_id#5] + +(4) ColumnarToRow [codegen id : 4] +Input [2]: [i_item_sk#1, i_manufact_id#5] + +(5) Scan parquet spark_catalog.default.store_sales +Output [4]: [ss_item_sk#10, ss_store_sk#11, ss_sales_price#12, ss_sold_date_sk#13] +Batched: true +Location: InMemoryFileIndex [] +PartitionFilters: [isnotnull(ss_sold_date_sk#13), dynamicpruningexpression(ss_sold_date_sk#13 IN dynamicpruning#14)] +PushedFilters: [IsNotNull(ss_item_sk), IsNotNull(ss_store_sk)] +ReadSchema: struct<ss_item_sk:int,ss_store_sk:int,ss_sales_price:decimal(7,2)> + +(6) CometFilter +Input [4]: [ss_item_sk#10, ss_store_sk#11, ss_sales_price#12, ss_sold_date_sk#13] +Condition : (isnotnull(ss_item_sk#10) AND isnotnull(ss_store_sk#11)) + +(7) ColumnarToRow [codegen id : 1] +Input [4]: [ss_item_sk#10, ss_store_sk#11, ss_sales_price#12, ss_sold_date_sk#13] + +(8) BroadcastExchange +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] +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] +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: 33] +Output [2]: [d_date_sk#15, d_qoy#16] + +(12) BroadcastHashJoin [codegen id : 4] +Left keys [1]: [ss_sold_date_sk#13] +Right keys [1]: [d_date_sk#15] +Join type: Inner +Join condition: None + +(13) Project [codegen id : 4] +Output [4]: [i_manufact_id#5, ss_store_sk#11, ss_sales_price#12, d_qoy#16] +Input [6]: [i_manufact_id#5, ss_store_sk#11, ss_sales_price#12, ss_sold_date_sk#13, d_date_sk#15, d_qoy#16] + +(14) Scan parquet spark_catalog.default.store +Output [1]: [s_store_sk#17] +Batched: true +Location [not included in comparison]/{warehouse_dir}/store] +PushedFilters: [IsNotNull(s_store_sk)] +ReadSchema: struct<s_store_sk:int> + +(15) CometFilter +Input [1]: [s_store_sk#17] +Condition : isnotnull(s_store_sk#17) + +(16) ColumnarToRow [codegen id : 3] +Input [1]: [s_store_sk#17] + +(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] +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] +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] +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))] +Aggregate Attributes [1]: [sum#18] +Results [3]: [i_manufact_id#5, d_qoy#16, sum#19] + +(21) Exchange +Input [3]: [i_manufact_id#5, d_qoy#16, sum#19] +Arguments: hashpartitioning(i_manufact_id#5, d_qoy#16, 5), ENSURE_REQUIREMENTS, [plan_id=3] + +(22) HashAggregate [codegen id : 5] +Input [3]: [i_manufact_id#5, d_qoy#16, sum#19] +Keys [2]: [i_manufact_id#5, d_qoy#16] +Functions [1]: [sum(UnscaledValue(ss_sales_price#12))] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_sales_price#12))#20] +Results [3]: [i_manufact_id#5, MakeDecimal(sum(UnscaledValue(ss_sales_price#12))#20,17,2) AS sum_sales#21, MakeDecimal(sum(UnscaledValue(ss_sales_price#12))#20,17,2) AS _w0#22] + +(23) Exchange +Input [3]: [i_manufact_id#5, sum_sales#21, _w0#22] +Arguments: hashpartitioning(i_manufact_id#5, 5), ENSURE_REQUIREMENTS, [plan_id=4] + +(24) Sort [codegen id : 6] +Input [3]: [i_manufact_id#5, sum_sales#21, _w0#22] +Arguments: [i_manufact_id#5 ASC NULLS FIRST], false, 0 + +(25) Window +Input [3]: [i_manufact_id#5, sum_sales#21, _w0#22] +Arguments: [avg(_w0#22) windowspecdefinition(i_manufact_id#5, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS avg_quarterly_sales#23], [i_manufact_id#5] + +(26) Filter [codegen id : 7] +Input [4]: [i_manufact_id#5, sum_sales#21, _w0#22, avg_quarterly_sales#23] +Condition : CASE WHEN (avg_quarterly_sales#23 > 0.000000) THEN ((abs((sum_sales#21 - avg_quarterly_sales#23)) / avg_quarterly_sales#23) > 0.1000000000000000) ELSE false END + +(27) Project [codegen id : 7] +Output [3]: [i_manufact_id#5, sum_sales#21, avg_quarterly_sales#23] +Input [4]: [i_manufact_id#5, sum_sales#21, _w0#22, avg_quarterly_sales#23] + +(28) TakeOrderedAndProject +Input [3]: [i_manufact_id#5, sum_sales#21, avg_quarterly_sales#23] +Arguments: 100, [avg_quarterly_sales#23 ASC NULLS FIRST, sum_sales#21 ASC NULLS FIRST, i_manufact_id#5 ASC NULLS FIRST], [i_manufact_id#5, sum_sales#21, avg_quarterly_sales#23] + +===== Subqueries ===== + +Subquery:1 Hosting operator id = 5 Hosting Expression = ss_sold_date_sk#13 IN dynamicpruning#14 +BroadcastExchange (33) ++- * ColumnarToRow (32) + +- CometProject (31) + +- CometFilter (30) + +- CometScan parquet spark_catalog.default.date_dim (29) + + +(29) Scan parquet spark_catalog.default.date_dim +Output [3]: [d_date_sk#15, d_month_seq#24, d_qoy#16] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [In(d_month_seq, [1200,1201,1202,1203,1204,1205,1206,1207,1208,1209,1210,1211]), IsNotNull(d_date_sk)] +ReadSchema: struct<d_date_sk:int,d_month_seq:int,d_qoy:int> + +(30) CometFilter +Input [3]: [d_date_sk#15, d_month_seq#24, d_qoy#16] +Condition : (d_month_seq#24 INSET 1200, 1201, 1202, 1203, 1204, 1205, 1206, 1207, 1208, 1209, 1210, 1211 AND isnotnull(d_date_sk#15)) + +(31) CometProject +Input [3]: [d_date_sk#15, d_month_seq#24, d_qoy#16] +Arguments: [d_date_sk#15, d_qoy#16], [d_date_sk#15, d_qoy#16] + +(32) ColumnarToRow [codegen id : 1] +Input [2]: [d_date_sk#15, d_qoy#16] + +(33) BroadcastExchange +Input [2]: [d_date_sk#15, d_qoy#16] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] + + 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 new file mode 100644 index 000000000..adda5c34f --- /dev/null +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q53/simplified.txt @@ -0,0 +1,51 @@ +TakeOrderedAndProject [avg_quarterly_sales,sum_sales,i_manufact_id] + WholeStageCodegen (7) + Project [i_manufact_id,sum_sales,avg_quarterly_sales] + Filter [avg_quarterly_sales,sum_sales] + InputAdapter + Window [_w0,i_manufact_id] + WholeStageCodegen (6) + Sort [i_manufact_id] + InputAdapter + Exchange [i_manufact_id] #1 + WholeStageCodegen (5) + HashAggregate [i_manufact_id,d_qoy,sum] [sum(UnscaledValue(ss_sales_price)),sum_sales,_w0,sum] + InputAdapter + Exchange [i_manufact_id,d_qoy] #2 + WholeStageCodegen (4) + HashAggregate [i_manufact_id,d_qoy,ss_sales_price] [sum,sum] + Project [i_manufact_id,ss_sales_price,d_qoy] + BroadcastHashJoin [ss_store_sk,s_store_sk] + Project [i_manufact_id,ss_store_sk,ss_sales_price,d_qoy] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [i_manufact_id,ss_store_sk,ss_sales_price,ss_sold_date_sk] + BroadcastHashJoin [i_item_sk,ss_item_sk] + ColumnarToRow + InputAdapter + CometProject [i_item_sk,i_manufact_id] + CometFilter [i_category,i_class,i_brand,i_item_sk] + CometScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category,i_manufact_id] + InputAdapter + BroadcastExchange #3 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometFilter [ss_item_sk,ss_store_sk] + CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #4 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk,d_qoy] + CometFilter [d_month_seq,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_qoy] + InputAdapter + ReusedExchange [d_date_sk,d_qoy] #4 + InputAdapter + BroadcastExchange #5 + WholeStageCodegen (3) + ColumnarToRow + InputAdapter + CometFilter [s_store_sk] + CometScan parquet spark_catalog.default.store [s_store_sk] 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 new file mode 100644 index 000000000..ca308b19a --- /dev/null +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q54/explain.txt @@ -0,0 +1,483 @@ +== Physical Plan == +TakeOrderedAndProject (55) ++- * HashAggregate (54) + +- Exchange (53) + +- * HashAggregate (52) + +- * HashAggregate (51) + +- Exchange (50) + +- * HashAggregate (49) + +- * Project (48) + +- * BroadcastHashJoin Inner BuildRight (47) + :- * Project (45) + : +- * BroadcastHashJoin Inner BuildRight (44) + : :- * Project (39) + : : +- * BroadcastHashJoin Inner BuildRight (38) + : : :- * Project (33) + : : : +- * BroadcastHashJoin Inner BuildRight (32) + : : : :- * HashAggregate (27) + : : : : +- Exchange (26) + : : : : +- * HashAggregate (25) + : : : : +- * Project (24) + : : : : +- * BroadcastHashJoin Inner BuildRight (23) + : : : : :- * Project (18) + : : : : : +- * BroadcastHashJoin Inner BuildRight (17) + : : : : : :- * Project (15) + : : : : : : +- * BroadcastHashJoin Inner BuildRight (14) + : : : : : : :- * ColumnarToRow (8) + : : : : : : : +- CometUnion (7) + : : : : : : : :- CometProject (3) + : : : : : : : : +- CometFilter (2) + : : : : : : : : +- CometScan parquet spark_catalog.default.catalog_sales (1) + : : : : : : : +- CometProject (6) + : : : : : : : +- CometFilter (5) + : : : : : : : +- CometScan parquet spark_catalog.default.web_sales (4) + : : : : : : +- BroadcastExchange (13) + : : : : : : +- * ColumnarToRow (12) + : : : : : : +- CometProject (11) + : : : : : : +- CometFilter (10) + : : : : : : +- CometScan parquet spark_catalog.default.item (9) + : : : : : +- ReusedExchange (16) + : : : : +- BroadcastExchange (22) + : : : : +- * ColumnarToRow (21) + : : : : +- CometFilter (20) + : : : : +- CometScan parquet spark_catalog.default.customer (19) + : : : +- BroadcastExchange (31) + : : : +- * ColumnarToRow (30) + : : : +- CometFilter (29) + : : : +- CometScan parquet spark_catalog.default.store_sales (28) + : : +- BroadcastExchange (37) + : : +- * ColumnarToRow (36) + : : +- CometFilter (35) + : : +- CometScan parquet spark_catalog.default.customer_address (34) + : +- BroadcastExchange (43) + : +- * ColumnarToRow (42) + : +- CometFilter (41) + : +- CometScan parquet spark_catalog.default.store (40) + +- ReusedExchange (46) + + +(1) Scan parquet spark_catalog.default.catalog_sales +Output [3]: [cs_bill_customer_sk#1, cs_item_sk#2, cs_sold_date_sk#3] +Batched: true +Location: InMemoryFileIndex [] +PartitionFilters: [isnotnull(cs_sold_date_sk#3), dynamicpruningexpression(cs_sold_date_sk#3 IN dynamicpruning#4)] +PushedFilters: [IsNotNull(cs_item_sk), IsNotNull(cs_bill_customer_sk)] +ReadSchema: struct<cs_bill_customer_sk:int,cs_item_sk:int> + +(2) CometFilter +Input [3]: [cs_bill_customer_sk#1, cs_item_sk#2, cs_sold_date_sk#3] +Condition : (isnotnull(cs_item_sk#2) AND isnotnull(cs_bill_customer_sk#1)) + +(3) CometProject +Input [3]: [cs_bill_customer_sk#1, cs_item_sk#2, cs_sold_date_sk#3] +Arguments: [sold_date_sk#5, customer_sk#6, item_sk#7], [cs_sold_date_sk#3 AS sold_date_sk#5, cs_bill_customer_sk#1 AS customer_sk#6, cs_item_sk#2 AS item_sk#7] + +(4) Scan parquet spark_catalog.default.web_sales +Output [3]: [ws_item_sk#8, ws_bill_customer_sk#9, ws_sold_date_sk#10] +Batched: true +Location: InMemoryFileIndex [] +PartitionFilters: [isnotnull(ws_sold_date_sk#10), dynamicpruningexpression(ws_sold_date_sk#10 IN dynamicpruning#4)] +PushedFilters: [IsNotNull(ws_item_sk), IsNotNull(ws_bill_customer_sk)] +ReadSchema: struct<ws_item_sk:int,ws_bill_customer_sk:int> + +(5) CometFilter +Input [3]: [ws_item_sk#8, ws_bill_customer_sk#9, ws_sold_date_sk#10] +Condition : (isnotnull(ws_item_sk#8) AND isnotnull(ws_bill_customer_sk#9)) + +(6) CometProject +Input [3]: [ws_item_sk#8, ws_bill_customer_sk#9, ws_sold_date_sk#10] +Arguments: [sold_date_sk#11, customer_sk#12, item_sk#13], [ws_sold_date_sk#10 AS sold_date_sk#11, ws_bill_customer_sk#9 AS customer_sk#12, ws_item_sk#8 AS item_sk#13] + +(7) CometUnion +Child 0 Input [3]: [sold_date_sk#5, customer_sk#6, item_sk#7] +Child 1 Input [3]: [sold_date_sk#11, customer_sk#12, item_sk#13] + +(8) ColumnarToRow [codegen id : 4] +Input [3]: [sold_date_sk#5, customer_sk#6, item_sk#7] + +(9) Scan parquet spark_catalog.default.item +Output [3]: [i_item_sk#14, i_class#15, i_category#16] +Batched: true +Location [not included in comparison]/{warehouse_dir}/item] +PushedFilters: [IsNotNull(i_category), IsNotNull(i_class), EqualTo(i_category,Women ), EqualTo(i_class,maternity ), IsNotNull(i_item_sk)] +ReadSchema: struct<i_item_sk:int,i_class:string,i_category:string> + +(10) CometFilter +Input [3]: [i_item_sk#14, i_class#15, i_category#16] +Condition : ((((isnotnull(i_category#16) AND isnotnull(i_class#15)) AND (i_category#16 = Women )) AND (i_class#15 = maternity )) AND isnotnull(i_item_sk#14)) + +(11) CometProject +Input [3]: [i_item_sk#14, i_class#15, i_category#16] +Arguments: [i_item_sk#14], [i_item_sk#14] + +(12) ColumnarToRow [codegen id : 1] +Input [1]: [i_item_sk#14] + +(13) BroadcastExchange +Input [1]: [i_item_sk#14] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] + +(14) BroadcastHashJoin [codegen id : 4] +Left keys [1]: [item_sk#7] +Right keys [1]: [i_item_sk#14] +Join type: Inner +Join condition: None + +(15) Project [codegen id : 4] +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] + +(16) ReusedExchange [Reuses operator id: 60] +Output [1]: [d_date_sk#17] + +(17) BroadcastHashJoin [codegen id : 4] +Left keys [1]: [sold_date_sk#5] +Right keys [1]: [d_date_sk#17] +Join type: Inner +Join condition: None + +(18) Project [codegen id : 4] +Output [1]: [customer_sk#6] +Input [3]: [sold_date_sk#5, customer_sk#6, d_date_sk#17] + +(19) Scan parquet spark_catalog.default.customer +Output [2]: [c_customer_sk#18, c_current_addr_sk#19] +Batched: true +Location [not included in comparison]/{warehouse_dir}/customer] +PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_current_addr_sk)] +ReadSchema: struct<c_customer_sk:int,c_current_addr_sk:int> + +(20) CometFilter +Input [2]: [c_customer_sk#18, c_current_addr_sk#19] +Condition : (isnotnull(c_customer_sk#18) AND isnotnull(c_current_addr_sk#19)) + +(21) ColumnarToRow [codegen id : 3] +Input [2]: [c_customer_sk#18, c_current_addr_sk#19] + +(22) BroadcastExchange +Input [2]: [c_customer_sk#18, c_current_addr_sk#19] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=2] + +(23) BroadcastHashJoin [codegen id : 4] +Left keys [1]: [customer_sk#6] +Right keys [1]: [c_customer_sk#18] +Join type: Inner +Join condition: None + +(24) Project [codegen id : 4] +Output [2]: [c_customer_sk#18, c_current_addr_sk#19] +Input [3]: [customer_sk#6, c_customer_sk#18, c_current_addr_sk#19] + +(25) HashAggregate [codegen id : 4] +Input [2]: [c_customer_sk#18, c_current_addr_sk#19] +Keys [2]: [c_customer_sk#18, c_current_addr_sk#19] +Functions: [] +Aggregate Attributes: [] +Results [2]: [c_customer_sk#18, c_current_addr_sk#19] + +(26) Exchange +Input [2]: [c_customer_sk#18, c_current_addr_sk#19] +Arguments: hashpartitioning(c_customer_sk#18, c_current_addr_sk#19, 5), ENSURE_REQUIREMENTS, [plan_id=3] + +(27) HashAggregate [codegen id : 9] +Input [2]: [c_customer_sk#18, c_current_addr_sk#19] +Keys [2]: [c_customer_sk#18, c_current_addr_sk#19] +Functions: [] +Aggregate Attributes: [] +Results [2]: [c_customer_sk#18, c_current_addr_sk#19] + +(28) Scan parquet spark_catalog.default.store_sales +Output [3]: [ss_customer_sk#20, ss_ext_sales_price#21, ss_sold_date_sk#22] +Batched: true +Location: InMemoryFileIndex [] +PartitionFilters: [isnotnull(ss_sold_date_sk#22), dynamicpruningexpression(ss_sold_date_sk#22 IN dynamicpruning#23)] +PushedFilters: [IsNotNull(ss_customer_sk)] +ReadSchema: struct<ss_customer_sk:int,ss_ext_sales_price:decimal(7,2)> + +(29) CometFilter +Input [3]: [ss_customer_sk#20, ss_ext_sales_price#21, ss_sold_date_sk#22] +Condition : isnotnull(ss_customer_sk#20) + +(30) ColumnarToRow [codegen id : 5] +Input [3]: [ss_customer_sk#20, ss_ext_sales_price#21, ss_sold_date_sk#22] + +(31) BroadcastExchange +Input [3]: [ss_customer_sk#20, ss_ext_sales_price#21, ss_sold_date_sk#22] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=4] + +(32) BroadcastHashJoin [codegen id : 9] +Left keys [1]: [c_customer_sk#18] +Right keys [1]: [ss_customer_sk#20] +Join type: Inner +Join condition: None + +(33) Project [codegen id : 9] +Output [4]: [c_customer_sk#18, c_current_addr_sk#19, ss_ext_sales_price#21, ss_sold_date_sk#22] +Input [5]: [c_customer_sk#18, c_current_addr_sk#19, ss_customer_sk#20, ss_ext_sales_price#21, ss_sold_date_sk#22] + +(34) Scan parquet spark_catalog.default.customer_address +Output [3]: [ca_address_sk#24, ca_county#25, ca_state#26] +Batched: true +Location [not included in comparison]/{warehouse_dir}/customer_address] +PushedFilters: [IsNotNull(ca_address_sk), IsNotNull(ca_county), IsNotNull(ca_state)] +ReadSchema: struct<ca_address_sk:int,ca_county:string,ca_state:string> + +(35) CometFilter +Input [3]: [ca_address_sk#24, ca_county#25, ca_state#26] +Condition : ((isnotnull(ca_address_sk#24) AND isnotnull(ca_county#25)) AND isnotnull(ca_state#26)) + +(36) ColumnarToRow [codegen id : 6] +Input [3]: [ca_address_sk#24, ca_county#25, ca_state#26] + +(37) BroadcastExchange +Input [3]: [ca_address_sk#24, ca_county#25, ca_state#26] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=5] + +(38) BroadcastHashJoin [codegen id : 9] +Left keys [1]: [c_current_addr_sk#19] +Right keys [1]: [ca_address_sk#24] +Join type: Inner +Join condition: None + +(39) Project [codegen id : 9] +Output [5]: [c_customer_sk#18, ss_ext_sales_price#21, ss_sold_date_sk#22, ca_county#25, ca_state#26] +Input [7]: [c_customer_sk#18, c_current_addr_sk#19, ss_ext_sales_price#21, ss_sold_date_sk#22, ca_address_sk#24, ca_county#25, ca_state#26] + +(40) Scan parquet spark_catalog.default.store +Output [2]: [s_county#27, s_state#28] +Batched: true +Location [not included in comparison]/{warehouse_dir}/store] +PushedFilters: [IsNotNull(s_county), IsNotNull(s_state)] +ReadSchema: struct<s_county:string,s_state:string> + +(41) CometFilter +Input [2]: [s_county#27, s_state#28] +Condition : (isnotnull(s_county#27) AND isnotnull(s_state#28)) + +(42) ColumnarToRow [codegen id : 7] +Input [2]: [s_county#27, s_state#28] + +(43) BroadcastExchange +Input [2]: [s_county#27, s_state#28] +Arguments: HashedRelationBroadcastMode(List(input[0, string, false], input[1, string, false]),false), [plan_id=6] + +(44) BroadcastHashJoin [codegen id : 9] +Left keys [2]: [ca_county#25, ca_state#26] +Right keys [2]: [s_county#27, s_state#28] +Join type: Inner +Join condition: None + +(45) Project [codegen id : 9] +Output [3]: [c_customer_sk#18, ss_ext_sales_price#21, ss_sold_date_sk#22] +Input [7]: [c_customer_sk#18, ss_ext_sales_price#21, ss_sold_date_sk#22, ca_county#25, ca_state#26, s_county#27, s_state#28] + +(46) ReusedExchange [Reuses operator id: 65] +Output [1]: [d_date_sk#29] + +(47) BroadcastHashJoin [codegen id : 9] +Left keys [1]: [ss_sold_date_sk#22] +Right keys [1]: [d_date_sk#29] +Join type: Inner +Join condition: None + +(48) Project [codegen id : 9] +Output [2]: [c_customer_sk#18, ss_ext_sales_price#21] +Input [4]: [c_customer_sk#18, ss_ext_sales_price#21, ss_sold_date_sk#22, d_date_sk#29] + +(49) HashAggregate [codegen id : 9] +Input [2]: [c_customer_sk#18, ss_ext_sales_price#21] +Keys [1]: [c_customer_sk#18] +Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#21))] +Aggregate Attributes [1]: [sum#30] +Results [2]: [c_customer_sk#18, sum#31] + +(50) Exchange +Input [2]: [c_customer_sk#18, sum#31] +Arguments: hashpartitioning(c_customer_sk#18, 5), ENSURE_REQUIREMENTS, [plan_id=7] + +(51) HashAggregate [codegen id : 10] +Input [2]: [c_customer_sk#18, sum#31] +Keys [1]: [c_customer_sk#18] +Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#21))] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#21))#32] +Results [1]: [cast((MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#21))#32,17,2) / 50) as int) AS segment#33] + +(52) HashAggregate [codegen id : 10] +Input [1]: [segment#33] +Keys [1]: [segment#33] +Functions [1]: [partial_count(1)] +Aggregate Attributes [1]: [count#34] +Results [2]: [segment#33, count#35] + +(53) Exchange +Input [2]: [segment#33, count#35] +Arguments: hashpartitioning(segment#33, 5), ENSURE_REQUIREMENTS, [plan_id=8] + +(54) HashAggregate [codegen id : 11] +Input [2]: [segment#33, count#35] +Keys [1]: [segment#33] +Functions [1]: [count(1)] +Aggregate Attributes [1]: [count(1)#36] +Results [3]: [segment#33, count(1)#36 AS num_customers#37, (segment#33 * 50) AS segment_base#38] + +(55) TakeOrderedAndProject +Input [3]: [segment#33, num_customers#37, segment_base#38] +Arguments: 100, [segment#33 ASC NULLS FIRST, num_customers#37 ASC NULLS FIRST], [segment#33, num_customers#37, segment_base#38] + +===== Subqueries ===== + +Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#3 IN dynamicpruning#4 +BroadcastExchange (60) ++- * ColumnarToRow (59) + +- CometProject (58) + +- CometFilter (57) + +- CometScan parquet spark_catalog.default.date_dim (56) + + +(56) Scan parquet spark_catalog.default.date_dim +Output [3]: [d_date_sk#17, d_year#39, d_moy#40] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_moy), IsNotNull(d_year), EqualTo(d_moy,12), EqualTo(d_year,1998), IsNotNull(d_date_sk)] +ReadSchema: struct<d_date_sk:int,d_year:int,d_moy:int> + +(57) CometFilter +Input [3]: [d_date_sk#17, d_year#39, d_moy#40] +Condition : ((((isnotnull(d_moy#40) AND isnotnull(d_year#39)) AND (d_moy#40 = 12)) AND (d_year#39 = 1998)) AND isnotnull(d_date_sk#17)) + +(58) CometProject +Input [3]: [d_date_sk#17, d_year#39, d_moy#40] +Arguments: [d_date_sk#17], [d_date_sk#17] + +(59) ColumnarToRow [codegen id : 1] +Input [1]: [d_date_sk#17] + +(60) BroadcastExchange +Input [1]: [d_date_sk#17] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=9] + +Subquery:2 Hosting operator id = 4 Hosting Expression = ws_sold_date_sk#10 IN dynamicpruning#4 + +Subquery:3 Hosting operator id = 28 Hosting Expression = ss_sold_date_sk#22 IN dynamicpruning#23 +BroadcastExchange (65) ++- * ColumnarToRow (64) + +- CometProject (63) + +- CometFilter (62) + +- CometScan parquet spark_catalog.default.date_dim (61) + + +(61) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#29, d_month_seq#41] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,ScalarSubquery#42), LessThanOrEqual(d_month_seq,ScalarSubquery#43), IsNotNull(d_date_sk)] +ReadSchema: struct<d_date_sk:int,d_month_seq:int> + +(62) CometFilter +Input [2]: [d_date_sk#29, d_month_seq#41] +Condition : (((isnotnull(d_month_seq#41) AND (d_month_seq#41 >= ReusedSubquery Subquery scalar-subquery#42, [id=#44])) AND (d_month_seq#41 <= ReusedSubquery Subquery scalar-subquery#43, [id=#45])) AND isnotnull(d_date_sk#29)) + +(63) CometProject +Input [2]: [d_date_sk#29, d_month_seq#41] +Arguments: [d_date_sk#29], [d_date_sk#29] + +(64) ColumnarToRow [codegen id : 1] +Input [1]: [d_date_sk#29] + +(65) BroadcastExchange +Input [1]: [d_date_sk#29] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=10] + +Subquery:4 Hosting operator id = 62 Hosting Expression = ReusedSubquery Subquery scalar-subquery#42, [id=#44] + +Subquery:5 Hosting operator id = 62 Hosting Expression = ReusedSubquery Subquery scalar-subquery#43, [id=#45] + +Subquery:6 Hosting operator id = 61 Hosting Expression = Subquery scalar-subquery#42, [id=#44] +* HashAggregate (72) ++- Exchange (71) + +- * ColumnarToRow (70) + +- CometHashAggregate (69) + +- CometProject (68) + +- CometFilter (67) + +- CometScan parquet spark_catalog.default.date_dim (66) + + +(66) Scan parquet spark_catalog.default.date_dim +Output [3]: [d_month_seq#46, d_year#47, d_moy#48] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,1998), EqualTo(d_moy,12)] +ReadSchema: struct<d_month_seq:int,d_year:int,d_moy:int> + +(67) CometFilter +Input [3]: [d_month_seq#46, d_year#47, d_moy#48] +Condition : (((isnotnull(d_year#47) AND isnotnull(d_moy#48)) AND (d_year#47 = 1998)) AND (d_moy#48 = 12)) + +(68) CometProject +Input [3]: [d_month_seq#46, d_year#47, d_moy#48] +Arguments: [(d_month_seq + 1)#49], [(d_month_seq#46 + 1) AS (d_month_seq + 1)#49] + +(69) CometHashAggregate +Input [1]: [(d_month_seq + 1)#49] +Keys [1]: [(d_month_seq + 1)#49] +Functions: [] + +(70) ColumnarToRow [codegen id : 1] +Input [1]: [(d_month_seq + 1)#49] + +(71) Exchange +Input [1]: [(d_month_seq + 1)#49] +Arguments: hashpartitioning((d_month_seq + 1)#49, 5), ENSURE_REQUIREMENTS, [plan_id=11] + +(72) HashAggregate [codegen id : 2] +Input [1]: [(d_month_seq + 1)#49] +Keys [1]: [(d_month_seq + 1)#49] +Functions: [] +Aggregate Attributes: [] +Results [1]: [(d_month_seq + 1)#49] + +Subquery:7 Hosting operator id = 61 Hosting Expression = Subquery scalar-subquery#43, [id=#45] +* HashAggregate (79) ++- Exchange (78) + +- * ColumnarToRow (77) + +- CometHashAggregate (76) + +- CometProject (75) + +- CometFilter (74) + +- CometScan parquet spark_catalog.default.date_dim (73) + + +(73) Scan parquet spark_catalog.default.date_dim +Output [3]: [d_month_seq#50, d_year#51, d_moy#52] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,1998), EqualTo(d_moy,12)] +ReadSchema: struct<d_month_seq:int,d_year:int,d_moy:int> + +(74) CometFilter +Input [3]: [d_month_seq#50, d_year#51, d_moy#52] +Condition : (((isnotnull(d_year#51) AND isnotnull(d_moy#52)) AND (d_year#51 = 1998)) AND (d_moy#52 = 12)) + +(75) CometProject +Input [3]: [d_month_seq#50, d_year#51, d_moy#52] +Arguments: [(d_month_seq + 3)#53], [(d_month_seq#50 + 3) AS (d_month_seq + 3)#53] + +(76) CometHashAggregate +Input [1]: [(d_month_seq + 3)#53] +Keys [1]: [(d_month_seq + 3)#53] +Functions: [] + +(77) ColumnarToRow [codegen id : 1] +Input [1]: [(d_month_seq + 3)#53] + +(78) Exchange +Input [1]: [(d_month_seq + 3)#53] +Arguments: hashpartitioning((d_month_seq + 3)#53, 5), ENSURE_REQUIREMENTS, [plan_id=12] + +(79) HashAggregate [codegen id : 2] +Input [1]: [(d_month_seq + 3)#53] +Keys [1]: [(d_month_seq + 3)#53] +Functions: [] +Aggregate Attributes: [] +Results [1]: [(d_month_seq + 3)#53] + + 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 new file mode 100644 index 000000000..30ba4b743 --- /dev/null +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q54/simplified.txt @@ -0,0 +1,123 @@ +TakeOrderedAndProject [segment,num_customers,segment_base] + WholeStageCodegen (11) + HashAggregate [segment,count] [count(1),num_customers,segment_base,count] + InputAdapter + Exchange [segment] #1 + WholeStageCodegen (10) + HashAggregate [segment] [count,count] + HashAggregate [c_customer_sk,sum] [sum(UnscaledValue(ss_ext_sales_price)),segment,sum] + InputAdapter + Exchange [c_customer_sk] #2 + WholeStageCodegen (9) + HashAggregate [c_customer_sk,ss_ext_sales_price] [sum,sum] + Project [c_customer_sk,ss_ext_sales_price] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [c_customer_sk,ss_ext_sales_price,ss_sold_date_sk] + BroadcastHashJoin [ca_county,ca_state,s_county,s_state] + Project [c_customer_sk,ss_ext_sales_price,ss_sold_date_sk,ca_county,ca_state] + BroadcastHashJoin [c_current_addr_sk,ca_address_sk] + Project [c_customer_sk,c_current_addr_sk,ss_ext_sales_price,ss_sold_date_sk] + BroadcastHashJoin [c_customer_sk,ss_customer_sk] + HashAggregate [c_customer_sk,c_current_addr_sk] + InputAdapter + Exchange [c_customer_sk,c_current_addr_sk] #3 + WholeStageCodegen (4) + HashAggregate [c_customer_sk,c_current_addr_sk] + Project [c_customer_sk,c_current_addr_sk] + BroadcastHashJoin [customer_sk,c_customer_sk] + Project [customer_sk] + BroadcastHashJoin [sold_date_sk,d_date_sk] + Project [sold_date_sk,customer_sk] + BroadcastHashJoin [item_sk,i_item_sk] + ColumnarToRow + InputAdapter + CometUnion + CometProject [cs_sold_date_sk,cs_bill_customer_sk,cs_item_sk] [sold_date_sk,customer_sk,item_sk] + CometFilter [cs_item_sk,cs_bill_customer_sk] + CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_item_sk,cs_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #4 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_moy,d_year,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometProject [ws_sold_date_sk,ws_bill_customer_sk,ws_item_sk] [sold_date_sk,customer_sk,item_sk] + CometFilter [ws_item_sk,ws_bill_customer_sk] + CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_bill_customer_sk,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + InputAdapter + BroadcastExchange #5 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [i_item_sk] + CometFilter [i_category,i_class,i_item_sk] + CometScan parquet spark_catalog.default.item [i_item_sk,i_class,i_category] + InputAdapter + ReusedExchange [d_date_sk] #4 + InputAdapter + BroadcastExchange #6 + WholeStageCodegen (3) + ColumnarToRow + InputAdapter + CometFilter [c_customer_sk,c_current_addr_sk] + CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk] + InputAdapter + BroadcastExchange #7 + WholeStageCodegen (5) + ColumnarToRow + InputAdapter + CometFilter [ss_customer_sk] + CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_ext_sales_price,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #2 + BroadcastExchange #8 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_month_seq,d_date_sk] + ReusedSubquery [(d_month_seq + 1)] #3 + ReusedSubquery [(d_month_seq + 3)] #4 + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] + Subquery #3 + WholeStageCodegen (2) + HashAggregate [(d_month_seq + 1)] + InputAdapter + Exchange [(d_month_seq + 1)] #9 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometHashAggregate [(d_month_seq + 1)] + CometProject [d_month_seq] [(d_month_seq + 1)] + CometFilter [d_year,d_moy] + CometScan parquet spark_catalog.default.date_dim [d_month_seq,d_year,d_moy] + Subquery #4 + WholeStageCodegen (2) + HashAggregate [(d_month_seq + 3)] + InputAdapter + Exchange [(d_month_seq + 3)] #10 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometHashAggregate [(d_month_seq + 3)] + CometProject [d_month_seq] [(d_month_seq + 3)] + CometFilter [d_year,d_moy] + CometScan parquet spark_catalog.default.date_dim [d_month_seq,d_year,d_moy] + InputAdapter + BroadcastExchange #11 + WholeStageCodegen (6) + ColumnarToRow + InputAdapter + CometFilter [ca_address_sk,ca_county,ca_state] + CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_county,ca_state] + InputAdapter + BroadcastExchange #12 + WholeStageCodegen (7) + ColumnarToRow + InputAdapter + CometFilter [s_county,s_state] + CometScan 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/q55/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q55/explain.txt new file mode 100644 index 000000000..3d1d689bc --- /dev/null +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q55/explain.txt @@ -0,0 +1,125 @@ +== Physical Plan == +TakeOrderedAndProject (21) ++- * HashAggregate (20) + +- Exchange (19) + +- * HashAggregate (18) + +- * Project (17) + +- * BroadcastHashJoin Inner BuildRight (16) + :- * Project (10) + : +- * BroadcastHashJoin Inner BuildRight (9) + : :- * ColumnarToRow (4) + : : +- CometProject (3) + : : +- CometFilter (2) + : : +- CometScan parquet spark_catalog.default.date_dim (1) + : +- BroadcastExchange (8) + : +- * ColumnarToRow (7) + : +- CometFilter (6) + : +- CometScan parquet spark_catalog.default.store_sales (5) + +- BroadcastExchange (15) + +- * ColumnarToRow (14) + +- CometProject (13) + +- CometFilter (12) + +- CometScan parquet spark_catalog.default.item (11) + + +(1) Scan parquet spark_catalog.default.date_dim +Output [3]: [d_date_sk#1, d_year#2, d_moy#3] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_moy), IsNotNull(d_year), EqualTo(d_moy,11), EqualTo(d_year,1999), IsNotNull(d_date_sk)] +ReadSchema: struct<d_date_sk:int,d_year:int,d_moy:int> + +(2) CometFilter +Input [3]: [d_date_sk#1, d_year#2, d_moy#3] +Condition : ((((isnotnull(d_moy#3) AND isnotnull(d_year#2)) AND (d_moy#3 = 11)) AND (d_year#2 = 1999)) AND isnotnull(d_date_sk#1)) + +(3) CometProject +Input [3]: [d_date_sk#1, d_year#2, d_moy#3] +Arguments: [d_date_sk#1], [d_date_sk#1] + +(4) ColumnarToRow [codegen id : 3] +Input [1]: [d_date_sk#1] + +(5) Scan parquet spark_catalog.default.store_sales +Output [3]: [ss_item_sk#4, ss_ext_sales_price#5, ss_sold_date_sk#6] +Batched: true +Location: InMemoryFileIndex [] +PartitionFilters: [isnotnull(ss_sold_date_sk#6), dynamicpruningexpression(true)] +PushedFilters: [IsNotNull(ss_item_sk)] +ReadSchema: struct<ss_item_sk:int,ss_ext_sales_price:decimal(7,2)> + +(6) CometFilter +Input [3]: [ss_item_sk#4, ss_ext_sales_price#5, ss_sold_date_sk#6] +Condition : isnotnull(ss_item_sk#4) + +(7) ColumnarToRow [codegen id : 1] +Input [3]: [ss_item_sk#4, ss_ext_sales_price#5, ss_sold_date_sk#6] + +(8) BroadcastExchange +Input [3]: [ss_item_sk#4, ss_ext_sales_price#5, ss_sold_date_sk#6] +Arguments: HashedRelationBroadcastMode(List(cast(input[2, int, true] as bigint)),false), [plan_id=1] + +(9) BroadcastHashJoin [codegen id : 3] +Left keys [1]: [d_date_sk#1] +Right keys [1]: [ss_sold_date_sk#6] +Join type: Inner +Join condition: None + +(10) Project [codegen id : 3] +Output [2]: [ss_item_sk#4, ss_ext_sales_price#5] +Input [4]: [d_date_sk#1, ss_item_sk#4, ss_ext_sales_price#5, ss_sold_date_sk#6] + +(11) Scan parquet spark_catalog.default.item +Output [4]: [i_item_sk#7, i_brand_id#8, i_brand#9, i_manager_id#10] +Batched: true +Location [not included in comparison]/{warehouse_dir}/item] +PushedFilters: [IsNotNull(i_manager_id), EqualTo(i_manager_id,28), IsNotNull(i_item_sk)] +ReadSchema: struct<i_item_sk:int,i_brand_id:int,i_brand:string,i_manager_id:int> + +(12) CometFilter +Input [4]: [i_item_sk#7, i_brand_id#8, i_brand#9, i_manager_id#10] +Condition : ((isnotnull(i_manager_id#10) AND (i_manager_id#10 = 28)) AND isnotnull(i_item_sk#7)) + +(13) CometProject +Input [4]: [i_item_sk#7, i_brand_id#8, i_brand#9, i_manager_id#10] +Arguments: [i_item_sk#7, i_brand_id#8, i_brand#9], [i_item_sk#7, i_brand_id#8, i_brand#9] + +(14) ColumnarToRow [codegen id : 2] +Input [3]: [i_item_sk#7, i_brand_id#8, i_brand#9] + +(15) BroadcastExchange +Input [3]: [i_item_sk#7, i_brand_id#8, i_brand#9] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] + +(16) BroadcastHashJoin [codegen id : 3] +Left keys [1]: [ss_item_sk#4] +Right keys [1]: [i_item_sk#7] +Join type: Inner +Join condition: None + +(17) Project [codegen id : 3] +Output [3]: [ss_ext_sales_price#5, i_brand_id#8, i_brand#9] +Input [5]: [ss_item_sk#4, ss_ext_sales_price#5, i_item_sk#7, i_brand_id#8, i_brand#9] + +(18) HashAggregate [codegen id : 3] +Input [3]: [ss_ext_sales_price#5, i_brand_id#8, i_brand#9] +Keys [2]: [i_brand#9, i_brand_id#8] +Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#5))] +Aggregate Attributes [1]: [sum#11] +Results [3]: [i_brand#9, i_brand_id#8, sum#12] + +(19) Exchange +Input [3]: [i_brand#9, i_brand_id#8, sum#12] +Arguments: hashpartitioning(i_brand#9, i_brand_id#8, 5), ENSURE_REQUIREMENTS, [plan_id=3] + +(20) HashAggregate [codegen id : 4] +Input [3]: [i_brand#9, i_brand_id#8, sum#12] +Keys [2]: [i_brand#9, i_brand_id#8] +Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#5))] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#5))#13] +Results [3]: [i_brand_id#8 AS brand_id#14, i_brand#9 AS brand#15, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#5))#13,17,2) AS ext_price#16] + +(21) TakeOrderedAndProject +Input [3]: [brand_id#14, brand#15, ext_price#16] +Arguments: 100, [ext_price#16 DESC NULLS LAST, brand_id#14 ASC NULLS FIRST], [brand_id#14, brand#15, ext_price#16] + diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q55/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q55/simplified.txt new file mode 100644 index 000000000..7a0fe8863 --- /dev/null +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q55/simplified.txt @@ -0,0 +1,31 @@ +TakeOrderedAndProject [ext_price,brand_id,brand] + WholeStageCodegen (4) + HashAggregate [i_brand,i_brand_id,sum] [sum(UnscaledValue(ss_ext_sales_price)),brand_id,brand,ext_price,sum] + InputAdapter + Exchange [i_brand,i_brand_id] #1 + WholeStageCodegen (3) + HashAggregate [i_brand,i_brand_id,ss_ext_sales_price] [sum,sum] + Project [ss_ext_sales_price,i_brand_id,i_brand] + BroadcastHashJoin [ss_item_sk,i_item_sk] + Project [ss_item_sk,ss_ext_sales_price] + BroadcastHashJoin [d_date_sk,ss_sold_date_sk] + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_moy,d_year,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + InputAdapter + BroadcastExchange #2 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometFilter [ss_item_sk] + CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] + InputAdapter + BroadcastExchange #3 + WholeStageCodegen (2) + ColumnarToRow + InputAdapter + CometProject [i_item_sk,i_brand_id,i_brand] + CometFilter [i_manager_id,i_item_sk] + CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_brand,i_manager_id] 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 new file mode 100644 index 000000000..bbed7eea6 --- /dev/null +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q56/explain.txt @@ -0,0 +1,405 @@ +== Physical Plan == +TakeOrderedAndProject (63) ++- * HashAggregate (62) + +- Exchange (61) + +- * HashAggregate (60) + +- Union (59) + :- * HashAggregate (28) + : +- Exchange (27) + : +- * HashAggregate (26) + : +- * Project (25) + : +- * BroadcastHashJoin Inner BuildRight (24) + : :- * Project (13) + : : +- * BroadcastHashJoin Inner BuildRight (12) + : : :- * Project (6) + : : : +- * BroadcastHashJoin Inner BuildRight (5) + : : : :- * ColumnarToRow (3) + : : : : +- CometFilter (2) + : : : : +- CometScan parquet spark_catalog.default.store_sales (1) + : : : +- ReusedExchange (4) + : : +- BroadcastExchange (11) + : : +- * ColumnarToRow (10) + : : +- CometProject (9) + : : +- CometFilter (8) + : : +- CometScan parquet spark_catalog.default.customer_address (7) + : +- BroadcastExchange (23) + : +- * BroadcastHashJoin LeftSemi BuildRight (22) + : :- * ColumnarToRow (16) + : : +- CometFilter (15) + : : +- CometScan parquet spark_catalog.default.item (14) + : +- BroadcastExchange (21) + : +- * ColumnarToRow (20) + : +- CometProject (19) + : +- CometFilter (18) + : +- CometScan parquet spark_catalog.default.item (17) + :- * HashAggregate (43) + : +- Exchange (42) + : +- * HashAggregate (41) + : +- * Project (40) + : +- * BroadcastHashJoin Inner BuildRight (39) + : :- * Project (37) + : : +- * BroadcastHashJoin Inner BuildRight (36) + : : :- * Project (34) + : : : +- * BroadcastHashJoin Inner BuildRight (33) + : : : :- * ColumnarToRow (31) + : : : : +- CometFilter (30) + : : : : +- CometScan parquet spark_catalog.default.catalog_sales (29) + : : : +- ReusedExchange (32) + : : +- ReusedExchange (35) + : +- ReusedExchange (38) + +- * HashAggregate (58) + +- Exchange (57) + +- * HashAggregate (56) + +- * Project (55) + +- * BroadcastHashJoin Inner BuildRight (54) + :- * Project (52) + : +- * BroadcastHashJoin Inner BuildRight (51) + : :- * Project (49) + : : +- * BroadcastHashJoin Inner BuildRight (48) + : : :- * ColumnarToRow (46) + : : : +- CometFilter (45) + : : : +- CometScan parquet spark_catalog.default.web_sales (44) + : : +- ReusedExchange (47) + : +- ReusedExchange (50) + +- ReusedExchange (53) + + +(1) Scan parquet spark_catalog.default.store_sales +Output [4]: [ss_item_sk#1, ss_addr_sk#2, ss_ext_sales_price#3, ss_sold_date_sk#4] +Batched: true +Location: InMemoryFileIndex [] +PartitionFilters: [isnotnull(ss_sold_date_sk#4), dynamicpruningexpression(ss_sold_date_sk#4 IN dynamicpruning#5)] +PushedFilters: [IsNotNull(ss_addr_sk), IsNotNull(ss_item_sk)] +ReadSchema: struct<ss_item_sk:int,ss_addr_sk:int,ss_ext_sales_price:decimal(7,2)> + +(2) CometFilter +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)) + +(3) ColumnarToRow [codegen id : 5] +Input [4]: [ss_item_sk#1, ss_addr_sk#2, ss_ext_sales_price#3, ss_sold_date_sk#4] + +(4) ReusedExchange [Reuses operator id: 68] +Output [1]: [d_date_sk#6] + +(5) BroadcastHashJoin [codegen id : 5] +Left keys [1]: [ss_sold_date_sk#4] +Right keys [1]: [d_date_sk#6] +Join type: Inner +Join condition: None + +(6) Project [codegen id : 5] +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] + +(7) Scan parquet spark_catalog.default.customer_address +Output [2]: [ca_address_sk#7, ca_gmt_offset#8] +Batched: true +Location [not included in comparison]/{warehouse_dir}/customer_address] +PushedFilters: [IsNotNull(ca_gmt_offset), EqualTo(ca_gmt_offset,-5.00), IsNotNull(ca_address_sk)] +ReadSchema: struct<ca_address_sk:int,ca_gmt_offset:decimal(5,2)> + +(8) CometFilter +Input [2]: [ca_address_sk#7, ca_gmt_offset#8] +Condition : ((isnotnull(ca_gmt_offset#8) AND (ca_gmt_offset#8 = -5.00)) AND isnotnull(ca_address_sk#7)) + +(9) CometProject +Input [2]: [ca_address_sk#7, ca_gmt_offset#8] +Arguments: [ca_address_sk#7], [ca_address_sk#7] + +(10) ColumnarToRow [codegen id : 2] +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 : 5] +Left keys [1]: [ss_addr_sk#2] +Right keys [1]: [ca_address_sk#7] +Join type: Inner +Join condition: None + +(13) Project [codegen id : 5] +Output [2]: [ss_item_sk#1, ss_ext_sales_price#3] +Input [4]: [ss_item_sk#1, ss_addr_sk#2, ss_ext_sales_price#3, ca_address_sk#7] + +(14) Scan parquet spark_catalog.default.item +Output [2]: [i_item_sk#9, i_item_id#10] +Batched: true +Location [not included in comparison]/{warehouse_dir}/item] +PushedFilters: [IsNotNull(i_item_sk)] +ReadSchema: struct<i_item_sk:int,i_item_id:string> + +(15) CometFilter +Input [2]: [i_item_sk#9, i_item_id#10] +Condition : isnotnull(i_item_sk#9) + +(16) ColumnarToRow [codegen id : 4] +Input [2]: [i_item_sk#9, i_item_id#10] + +(17) Scan parquet spark_catalog.default.item +Output [2]: [i_item_id#11, i_color#12] +Batched: true +Location [not included in comparison]/{warehouse_dir}/item] +PushedFilters: [In(i_color, [blanched ,burnished ,slate ])] +ReadSchema: struct<i_item_id:string,i_color:string> + +(18) CometFilter +Input [2]: [i_item_id#11, i_color#12] +Condition : i_color#12 IN (slate ,blanched ,burnished ) + +(19) CometProject +Input [2]: [i_item_id#11, i_color#12] +Arguments: [i_item_id#11], [i_item_id#11] + +(20) ColumnarToRow [codegen id : 3] +Input [1]: [i_item_id#11] + +(21) BroadcastExchange +Input [1]: [i_item_id#11] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=2] + +(22) BroadcastHashJoin [codegen id : 4] +Left keys [1]: [i_item_id#10] +Right keys [1]: [i_item_id#11] +Join type: LeftSemi +Join condition: None + +(23) BroadcastExchange +Input [2]: [i_item_sk#9, i_item_id#10] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=3] + +(24) BroadcastHashJoin [codegen id : 5] +Left keys [1]: [ss_item_sk#1] +Right keys [1]: [i_item_sk#9] +Join type: Inner +Join condition: None + +(25) Project [codegen id : 5] +Output [2]: [ss_ext_sales_price#3, i_item_id#10] +Input [4]: [ss_item_sk#1, ss_ext_sales_price#3, i_item_sk#9, i_item_id#10] + +(26) HashAggregate [codegen id : 5] +Input [2]: [ss_ext_sales_price#3, i_item_id#10] +Keys [1]: [i_item_id#10] +Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#3))] +Aggregate Attributes [1]: [sum#13] +Results [2]: [i_item_id#10, sum#14] + +(27) Exchange +Input [2]: [i_item_id#10, sum#14] +Arguments: hashpartitioning(i_item_id#10, 5), ENSURE_REQUIREMENTS, [plan_id=4] + +(28) HashAggregate [codegen id : 6] +Input [2]: [i_item_id#10, sum#14] +Keys [1]: [i_item_id#10] +Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#3))] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#3))#15] +Results [2]: [i_item_id#10, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#3))#15,17,2) AS total_sales#16] + +(29) Scan parquet spark_catalog.default.catalog_sales +Output [4]: [cs_bill_addr_sk#17, cs_item_sk#18, cs_ext_sales_price#19, cs_sold_date_sk#20] +Batched: true +Location: InMemoryFileIndex [] +PartitionFilters: [isnotnull(cs_sold_date_sk#20), dynamicpruningexpression(cs_sold_date_sk#20 IN dynamicpruning#21)] +PushedFilters: [IsNotNull(cs_bill_addr_sk), IsNotNull(cs_item_sk)] +ReadSchema: struct<cs_bill_addr_sk:int,cs_item_sk:int,cs_ext_sales_price:decimal(7,2)> + +(30) CometFilter +Input [4]: [cs_bill_addr_sk#17, cs_item_sk#18, cs_ext_sales_price#19, cs_sold_date_sk#20] +Condition : (isnotnull(cs_bill_addr_sk#17) AND isnotnull(cs_item_sk#18)) + +(31) ColumnarToRow [codegen id : 11] +Input [4]: [cs_bill_addr_sk#17, cs_item_sk#18, cs_ext_sales_price#19, cs_sold_date_sk#20] + +(32) ReusedExchange [Reuses operator id: 68] +Output [1]: [d_date_sk#22] + +(33) BroadcastHashJoin [codegen id : 11] +Left keys [1]: [cs_sold_date_sk#20] +Right keys [1]: [d_date_sk#22] +Join type: Inner +Join condition: None + +(34) Project [codegen id : 11] +Output [3]: [cs_bill_addr_sk#17, cs_item_sk#18, cs_ext_sales_price#19] +Input [5]: [cs_bill_addr_sk#17, cs_item_sk#18, cs_ext_sales_price#19, cs_sold_date_sk#20, d_date_sk#22] + +(35) ReusedExchange [Reuses operator id: 11] +Output [1]: [ca_address_sk#23] + +(36) BroadcastHashJoin [codegen id : 11] +Left keys [1]: [cs_bill_addr_sk#17] +Right keys [1]: [ca_address_sk#23] +Join type: Inner +Join condition: None + +(37) Project [codegen id : 11] +Output [2]: [cs_item_sk#18, cs_ext_sales_price#19] +Input [4]: [cs_bill_addr_sk#17, cs_item_sk#18, cs_ext_sales_price#19, ca_address_sk#23] + +(38) ReusedExchange [Reuses operator id: 23] +Output [2]: [i_item_sk#24, i_item_id#25] + +(39) BroadcastHashJoin [codegen id : 11] +Left keys [1]: [cs_item_sk#18] +Right keys [1]: [i_item_sk#24] +Join type: Inner +Join condition: None + +(40) Project [codegen id : 11] +Output [2]: [cs_ext_sales_price#19, i_item_id#25] +Input [4]: [cs_item_sk#18, cs_ext_sales_price#19, i_item_sk#24, i_item_id#25] + +(41) HashAggregate [codegen id : 11] +Input [2]: [cs_ext_sales_price#19, i_item_id#25] +Keys [1]: [i_item_id#25] +Functions [1]: [partial_sum(UnscaledValue(cs_ext_sales_price#19))] +Aggregate Attributes [1]: [sum#26] +Results [2]: [i_item_id#25, sum#27] + +(42) Exchange +Input [2]: [i_item_id#25, sum#27] +Arguments: hashpartitioning(i_item_id#25, 5), ENSURE_REQUIREMENTS, [plan_id=5] + +(43) HashAggregate [codegen id : 12] +Input [2]: [i_item_id#25, sum#27] +Keys [1]: [i_item_id#25] +Functions [1]: [sum(UnscaledValue(cs_ext_sales_price#19))] +Aggregate Attributes [1]: [sum(UnscaledValue(cs_ext_sales_price#19))#28] +Results [2]: [i_item_id#25, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#19))#28,17,2) AS total_sales#29] + +(44) Scan parquet spark_catalog.default.web_sales +Output [4]: [ws_item_sk#30, ws_bill_addr_sk#31, ws_ext_sales_price#32, ws_sold_date_sk#33] +Batched: true +Location: InMemoryFileIndex [] +PartitionFilters: [isnotnull(ws_sold_date_sk#33), dynamicpruningexpression(ws_sold_date_sk#33 IN dynamicpruning#34)] +PushedFilters: [IsNotNull(ws_bill_addr_sk), IsNotNull(ws_item_sk)] +ReadSchema: struct<ws_item_sk:int,ws_bill_addr_sk:int,ws_ext_sales_price:decimal(7,2)> + +(45) CometFilter +Input [4]: [ws_item_sk#30, ws_bill_addr_sk#31, ws_ext_sales_price#32, ws_sold_date_sk#33] +Condition : (isnotnull(ws_bill_addr_sk#31) AND isnotnull(ws_item_sk#30)) + +(46) ColumnarToRow [codegen id : 17] +Input [4]: [ws_item_sk#30, ws_bill_addr_sk#31, ws_ext_sales_price#32, ws_sold_date_sk#33] + +(47) ReusedExchange [Reuses operator id: 68] +Output [1]: [d_date_sk#35] + +(48) BroadcastHashJoin [codegen id : 17] +Left keys [1]: [ws_sold_date_sk#33] +Right keys [1]: [d_date_sk#35] +Join type: Inner +Join condition: None + +(49) Project [codegen id : 17] +Output [3]: [ws_item_sk#30, ws_bill_addr_sk#31, ws_ext_sales_price#32] +Input [5]: [ws_item_sk#30, ws_bill_addr_sk#31, ws_ext_sales_price#32, ws_sold_date_sk#33, d_date_sk#35] + +(50) ReusedExchange [Reuses operator id: 11] +Output [1]: [ca_address_sk#36] + +(51) BroadcastHashJoin [codegen id : 17] +Left keys [1]: [ws_bill_addr_sk#31] +Right keys [1]: [ca_address_sk#36] +Join type: Inner +Join condition: None + +(52) Project [codegen id : 17] +Output [2]: [ws_item_sk#30, ws_ext_sales_price#32] +Input [4]: [ws_item_sk#30, ws_bill_addr_sk#31, ws_ext_sales_price#32, ca_address_sk#36] + +(53) ReusedExchange [Reuses operator id: 23] +Output [2]: [i_item_sk#37, i_item_id#38] + +(54) BroadcastHashJoin [codegen id : 17] +Left keys [1]: [ws_item_sk#30] +Right keys [1]: [i_item_sk#37] +Join type: Inner +Join condition: None + +(55) Project [codegen id : 17] +Output [2]: [ws_ext_sales_price#32, i_item_id#38] +Input [4]: [ws_item_sk#30, ws_ext_sales_price#32, i_item_sk#37, i_item_id#38] + +(56) HashAggregate [codegen id : 17] +Input [2]: [ws_ext_sales_price#32, i_item_id#38] +Keys [1]: [i_item_id#38] +Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#32))] +Aggregate Attributes [1]: [sum#39] +Results [2]: [i_item_id#38, sum#40] + +(57) Exchange +Input [2]: [i_item_id#38, sum#40] +Arguments: hashpartitioning(i_item_id#38, 5), ENSURE_REQUIREMENTS, [plan_id=6] + +(58) HashAggregate [codegen id : 18] +Input [2]: [i_item_id#38, sum#40] +Keys [1]: [i_item_id#38] +Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#32))] +Aggregate Attributes [1]: [sum(UnscaledValue(ws_ext_sales_price#32))#41] +Results [2]: [i_item_id#38, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#32))#41,17,2) AS total_sales#42] + +(59) Union + +(60) HashAggregate [codegen id : 19] +Input [2]: [i_item_id#10, total_sales#16] +Keys [1]: [i_item_id#10] +Functions [1]: [partial_sum(total_sales#16)] +Aggregate Attributes [2]: [sum#43, isEmpty#44] +Results [3]: [i_item_id#10, sum#45, isEmpty#46] + +(61) Exchange +Input [3]: [i_item_id#10, sum#45, isEmpty#46] +Arguments: hashpartitioning(i_item_id#10, 5), ENSURE_REQUIREMENTS, [plan_id=7] + +(62) HashAggregate [codegen id : 20] +Input [3]: [i_item_id#10, sum#45, isEmpty#46] +Keys [1]: [i_item_id#10] +Functions [1]: [sum(total_sales#16)] +Aggregate Attributes [1]: [sum(total_sales#16)#47] +Results [2]: [i_item_id#10, sum(total_sales#16)#47 AS total_sales#48] + +(63) TakeOrderedAndProject +Input [2]: [i_item_id#10, total_sales#48] +Arguments: 100, [total_sales#48 ASC NULLS FIRST], [i_item_id#10, total_sales#48] + +===== Subqueries ===== + +Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 +BroadcastExchange (68) ++- * ColumnarToRow (67) + +- CometProject (66) + +- CometFilter (65) + +- CometScan parquet spark_catalog.default.date_dim (64) + + +(64) Scan parquet spark_catalog.default.date_dim +Output [3]: [d_date_sk#6, d_year#49, d_moy#50] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2001), EqualTo(d_moy,2), IsNotNull(d_date_sk)] +ReadSchema: struct<d_date_sk:int,d_year:int,d_moy:int> + +(65) CometFilter +Input [3]: [d_date_sk#6, d_year#49, d_moy#50] +Condition : ((((isnotnull(d_year#49) AND isnotnull(d_moy#50)) AND (d_year#49 = 2001)) AND (d_moy#50 = 2)) AND isnotnull(d_date_sk#6)) + +(66) CometProject +Input [3]: [d_date_sk#6, d_year#49, d_moy#50] +Arguments: [d_date_sk#6], [d_date_sk#6] + +(67) ColumnarToRow [codegen id : 1] +Input [1]: [d_date_sk#6] + +(68) BroadcastExchange +Input [1]: [d_date_sk#6] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=8] + +Subquery:2 Hosting operator id = 29 Hosting Expression = cs_sold_date_sk#20 IN dynamicpruning#5 + +Subquery:3 Hosting operator id = 44 Hosting Expression = ws_sold_date_sk#33 IN dynamicpruning#5 + + diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q56/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q56/simplified.txt new file mode 100644 index 000000000..f781ed1f7 --- /dev/null +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q56/simplified.txt @@ -0,0 +1,105 @@ +TakeOrderedAndProject [total_sales,i_item_id] + WholeStageCodegen (20) + HashAggregate [i_item_id,sum,isEmpty] [sum(total_sales),total_sales,sum,isEmpty] + InputAdapter + Exchange [i_item_id] #1 + WholeStageCodegen (19) + HashAggregate [i_item_id,total_sales] [sum,isEmpty,sum,isEmpty] + InputAdapter + Union + WholeStageCodegen (6) + HashAggregate [i_item_id,sum] [sum(UnscaledValue(ss_ext_sales_price)),total_sales,sum] + InputAdapter + Exchange [i_item_id] #2 + WholeStageCodegen (5) + HashAggregate [i_item_id,ss_ext_sales_price] [sum,sum] + Project [ss_ext_sales_price,i_item_id] + BroadcastHashJoin [ss_item_sk,i_item_sk] + Project [ss_item_sk,ss_ext_sales_price] + BroadcastHashJoin [ss_addr_sk,ca_address_sk] + Project [ss_item_sk,ss_addr_sk,ss_ext_sales_price] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + ColumnarToRow + InputAdapter + CometFilter [ss_addr_sk,ss_item_sk] + CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_year,d_moy,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + InputAdapter + ReusedExchange [d_date_sk] #3 + InputAdapter + BroadcastExchange #4 + WholeStageCodegen (2) + ColumnarToRow + InputAdapter + CometProject [ca_address_sk] + CometFilter [ca_gmt_offset,ca_address_sk] + CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_gmt_offset] + InputAdapter + BroadcastExchange #5 + WholeStageCodegen (4) + BroadcastHashJoin [i_item_id,i_item_id] + ColumnarToRow + InputAdapter + CometFilter [i_item_sk] + CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id] + InputAdapter + BroadcastExchange #6 + WholeStageCodegen (3) + ColumnarToRow + InputAdapter + CometProject [i_item_id] + CometFilter [i_color] + CometScan parquet spark_catalog.default.item [i_item_id,i_color] + WholeStageCodegen (12) + HashAggregate [i_item_id,sum] [sum(UnscaledValue(cs_ext_sales_price)),total_sales,sum] + InputAdapter + Exchange [i_item_id] #7 + WholeStageCodegen (11) + HashAggregate [i_item_id,cs_ext_sales_price] [sum,sum] + Project [cs_ext_sales_price,i_item_id] + BroadcastHashJoin [cs_item_sk,i_item_sk] + Project [cs_item_sk,cs_ext_sales_price] + BroadcastHashJoin [cs_bill_addr_sk,ca_address_sk] + Project [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + ColumnarToRow + InputAdapter + CometFilter [cs_bill_addr_sk,cs_item_sk] + CometScan parquet spark_catalog.default.catalog_sales [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + InputAdapter + ReusedExchange [d_date_sk] #3 + InputAdapter + ReusedExchange [ca_address_sk] #4 + InputAdapter + ReusedExchange [i_item_sk,i_item_id] #5 + WholeStageCodegen (18) + HashAggregate [i_item_id,sum] [sum(UnscaledValue(ws_ext_sales_price)),total_sales,sum] + InputAdapter + Exchange [i_item_id] #8 + WholeStageCodegen (17) + HashAggregate [i_item_id,ws_ext_sales_price] [sum,sum] + Project [ws_ext_sales_price,i_item_id] + BroadcastHashJoin [ws_item_sk,i_item_sk] + Project [ws_item_sk,ws_ext_sales_price] + BroadcastHashJoin [ws_bill_addr_sk,ca_address_sk] + Project [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + ColumnarToRow + InputAdapter + CometFilter [ws_bill_addr_sk,ws_item_sk] + CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + InputAdapter + ReusedExchange [d_date_sk] #3 + InputAdapter + ReusedExchange [ca_address_sk] #4 + InputAdapter + ReusedExchange [i_item_sk,i_item_id] #5 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 new file mode 100644 index 000000000..8746c36d6 --- /dev/null +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q57/explain.txt @@ -0,0 +1,279 @@ +== Physical Plan == +TakeOrderedAndProject (45) ++- * Project (44) + +- * BroadcastHashJoin Inner BuildRight (43) + :- * Project (37) + : +- * BroadcastHashJoin Inner BuildRight (36) + : :- * Project (28) + : : +- * Filter (27) + : : +- Window (26) + : : +- * Filter (25) + : : +- Window (24) + : : +- * Sort (23) + : : +- Exchange (22) + : : +- * HashAggregate (21) + : : +- Exchange (20) + : : +- * HashAggregate (19) + : : +- * Project (18) + : : +- * BroadcastHashJoin Inner BuildRight (17) + : : :- * Project (12) + : : : +- * BroadcastHashJoin Inner BuildRight (11) + : : : :- * Project (9) + : : : : +- * BroadcastHashJoin Inner BuildRight (8) + : : : : :- * ColumnarToRow (3) + : : : : : +- CometFilter (2) + : : : : : +- CometScan parquet spark_catalog.default.item (1) + : : : : +- BroadcastExchange (7) + : : : : +- * ColumnarToRow (6) + : : : : +- CometFilter (5) + : : : : +- CometScan parquet spark_catalog.default.catalog_sales (4) + : : : +- ReusedExchange (10) + : : +- BroadcastExchange (16) + : : +- * ColumnarToRow (15) + : : +- CometFilter (14) + : : +- CometScan parquet spark_catalog.default.call_center (13) + : +- BroadcastExchange (35) + : +- * Project (34) + : +- Window (33) + : +- * Sort (32) + : +- Exchange (31) + : +- * HashAggregate (30) + : +- ReusedExchange (29) + +- BroadcastExchange (42) + +- * Project (41) + +- Window (40) + +- * Sort (39) + +- ReusedExchange (38) + + +(1) Scan parquet spark_catalog.default.item +Output [3]: [i_item_sk#1, i_brand#2, i_category#3] +Batched: true +Location [not included in comparison]/{warehouse_dir}/item] +PushedFilters: [IsNotNull(i_item_sk), IsNotNull(i_category), IsNotNull(i_brand)] +ReadSchema: struct<i_item_sk:int,i_brand:string,i_category:string> + +(2) CometFilter +Input [3]: [i_item_sk#1, i_brand#2, i_category#3] +Condition : ((isnotnull(i_item_sk#1) AND isnotnull(i_category#3)) AND isnotnull(i_brand#2)) + +(3) ColumnarToRow [codegen id : 4] +Input [3]: [i_item_sk#1, i_brand#2, i_category#3] + +(4) Scan parquet spark_catalog.default.catalog_sales +Output [4]: [cs_call_center_sk#4, cs_item_sk#5, cs_sales_price#6, cs_sold_date_sk#7] +Batched: true +Location: InMemoryFileIndex [] +PartitionFilters: [isnotnull(cs_sold_date_sk#7), dynamicpruningexpression(cs_sold_date_sk#7 IN dynamicpruning#8)] +PushedFilters: [IsNotNull(cs_item_sk), IsNotNull(cs_call_center_sk)] +ReadSchema: struct<cs_call_center_sk:int,cs_item_sk:int,cs_sales_price:decimal(7,2)> + +(5) CometFilter +Input [4]: [cs_call_center_sk#4, cs_item_sk#5, cs_sales_price#6, cs_sold_date_sk#7] +Condition : (isnotnull(cs_item_sk#5) AND isnotnull(cs_call_center_sk#4)) + +(6) ColumnarToRow [codegen id : 1] +Input [4]: [cs_call_center_sk#4, cs_item_sk#5, cs_sales_price#6, cs_sold_date_sk#7] + +(7) BroadcastExchange +Input [4]: [cs_call_center_sk#4, cs_item_sk#5, cs_sales_price#6, cs_sold_date_sk#7] +Arguments: HashedRelationBroadcastMode(List(cast(input[1, int, false] as bigint)),false), [plan_id=1] + +(8) BroadcastHashJoin [codegen id : 4] +Left keys [1]: [i_item_sk#1] +Right keys [1]: [cs_item_sk#5] +Join type: Inner +Join condition: None + +(9) Project [codegen id : 4] +Output [5]: [i_brand#2, i_category#3, cs_call_center_sk#4, cs_sales_price#6, cs_sold_date_sk#7] +Input [7]: [i_item_sk#1, i_brand#2, i_category#3, cs_call_center_sk#4, cs_item_sk#5, cs_sales_price#6, cs_sold_date_sk#7] + +(10) ReusedExchange [Reuses operator id: 49] +Output [3]: [d_date_sk#9, d_year#10, d_moy#11] + +(11) BroadcastHashJoin [codegen id : 4] +Left keys [1]: [cs_sold_date_sk#7] +Right keys [1]: [d_date_sk#9] +Join type: Inner +Join condition: None + +(12) Project [codegen id : 4] +Output [6]: [i_brand#2, i_category#3, cs_call_center_sk#4, cs_sales_price#6, d_year#10, d_moy#11] +Input [8]: [i_brand#2, i_category#3, cs_call_center_sk#4, cs_sales_price#6, cs_sold_date_sk#7, d_date_sk#9, d_year#10, d_moy#11] + +(13) Scan parquet spark_catalog.default.call_center +Output [2]: [cc_call_center_sk#12, cc_name#13] +Batched: true +Location [not included in comparison]/{warehouse_dir}/call_center] +PushedFilters: [IsNotNull(cc_call_center_sk), IsNotNull(cc_name)] +ReadSchema: struct<cc_call_center_sk:int,cc_name:string> + +(14) CometFilter +Input [2]: [cc_call_center_sk#12, cc_name#13] +Condition : (isnotnull(cc_call_center_sk#12) AND isnotnull(cc_name#13)) + +(15) ColumnarToRow [codegen id : 3] +Input [2]: [cc_call_center_sk#12, cc_name#13] + +(16) BroadcastExchange +Input [2]: [cc_call_center_sk#12, cc_name#13] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=2] + +(17) BroadcastHashJoin [codegen id : 4] +Left keys [1]: [cs_call_center_sk#4] +Right keys [1]: [cc_call_center_sk#12] +Join type: Inner +Join condition: None + +(18) Project [codegen id : 4] +Output [6]: [i_brand#2, i_category#3, cs_sales_price#6, d_year#10, d_moy#11, cc_name#13] +Input [8]: [i_brand#2, i_category#3, cs_call_center_sk#4, cs_sales_price#6, d_year#10, d_moy#11, cc_call_center_sk#12, cc_name#13] + +(19) HashAggregate [codegen id : 4] +Input [6]: [i_brand#2, i_category#3, cs_sales_price#6, d_year#10, d_moy#11, cc_name#13] +Keys [5]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11] +Functions [1]: [partial_sum(UnscaledValue(cs_sales_price#6))] +Aggregate Attributes [1]: [sum#14] +Results [6]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, sum#15] + +(20) Exchange +Input [6]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, sum#15] +Arguments: hashpartitioning(i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, 5), ENSURE_REQUIREMENTS, [plan_id=3] + +(21) HashAggregate [codegen id : 5] +Input [6]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, sum#15] +Keys [5]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11] +Functions [1]: [sum(UnscaledValue(cs_sales_price#6))] +Aggregate Attributes [1]: [sum(UnscaledValue(cs_sales_price#6))#16] +Results [7]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, MakeDecimal(sum(UnscaledValue(cs_sales_price#6))#16,17,2) AS sum_sales#17, MakeDecimal(sum(UnscaledValue(cs_sales_price#6))#16,17,2) AS _w0#18] + +(22) Exchange +Input [7]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, sum_sales#17, _w0#18] +Arguments: hashpartitioning(i_category#3, i_brand#2, cc_name#13, 5), ENSURE_REQUIREMENTS, [plan_id=4] + +(23) Sort [codegen id : 6] +Input [7]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, sum_sales#17, _w0#18] +Arguments: [i_category#3 ASC NULLS FIRST, i_brand#2 ASC NULLS FIRST, cc_name#13 ASC NULLS FIRST, d_year#10 ASC NULLS FIRST, d_moy#11 ASC NULLS FIRST], false, 0 + +(24) Window +Input [7]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, sum_sales#17, _w0#18] +Arguments: [rank(d_year#10, d_moy#11) windowspecdefinition(i_category#3, i_brand#2, cc_name#13, d_year#10 ASC NULLS FIRST, d_moy#11 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#19], [i_category#3, i_brand#2, cc_name#13], [d_year#10 ASC NULLS FIRST, d_moy#11 ASC NULLS FIRST] + +(25) Filter [codegen id : 7] +Input [8]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, sum_sales#17, _w0#18, rn#19] +Condition : (isnotnull(d_year#10) AND (d_year#10 = 1999)) + +(26) Window +Input [8]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, sum_sales#17, _w0#18, rn#19] +Arguments: [avg(_w0#18) windowspecdefinition(i_category#3, i_brand#2, cc_name#13, d_year#10, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS avg_monthly_sales#20], [i_category#3, i_brand#2, cc_name#13, d_year#10] + +(27) Filter [codegen id : 22] +Input [9]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, sum_sales#17, _w0#18, rn#19, avg_monthly_sales#20] +Condition : ((isnotnull(avg_monthly_sales#20) AND (avg_monthly_sales#20 > 0.000000)) AND CASE WHEN (avg_monthly_sales#20 > 0.000000) THEN ((abs((sum_sales#17 - avg_monthly_sales#20)) / avg_monthly_sales#20) > 0.1000000000000000) END) + +(28) Project [codegen id : 22] +Output [8]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, sum_sales#17, avg_monthly_sales#20, rn#19] +Input [9]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, sum_sales#17, _w0#18, rn#19, avg_monthly_sales#20] + +(29) ReusedExchange [Reuses operator id: 20] +Output [6]: [i_category#21, i_brand#22, cc_name#23, d_year#24, d_moy#25, sum#26] + +(30) HashAggregate [codegen id : 12] +Input [6]: [i_category#21, i_brand#22, cc_name#23, d_year#24, d_moy#25, sum#26] +Keys [5]: [i_category#21, i_brand#22, cc_name#23, d_year#24, d_moy#25] +Functions [1]: [sum(UnscaledValue(cs_sales_price#27))] +Aggregate Attributes [1]: [sum(UnscaledValue(cs_sales_price#27))#16] +Results [6]: [i_category#21, i_brand#22, cc_name#23, d_year#24, d_moy#25, MakeDecimal(sum(UnscaledValue(cs_sales_price#27))#16,17,2) AS sum_sales#28] + +(31) Exchange +Input [6]: [i_category#21, i_brand#22, cc_name#23, d_year#24, d_moy#25, sum_sales#28] +Arguments: hashpartitioning(i_category#21, i_brand#22, cc_name#23, 5), ENSURE_REQUIREMENTS, [plan_id=5] + +(32) Sort [codegen id : 13] +Input [6]: [i_category#21, i_brand#22, cc_name#23, d_year#24, d_moy#25, sum_sales#28] +Arguments: [i_category#21 ASC NULLS FIRST, i_brand#22 ASC NULLS FIRST, cc_name#23 ASC NULLS FIRST, d_year#24 ASC NULLS FIRST, d_moy#25 ASC NULLS FIRST], false, 0 + +(33) Window +Input [6]: [i_category#21, i_brand#22, cc_name#23, d_year#24, d_moy#25, sum_sales#28] +Arguments: [rank(d_year#24, d_moy#25) windowspecdefinition(i_category#21, i_brand#22, cc_name#23, d_year#24 ASC NULLS FIRST, d_moy#25 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#29], [i_category#21, i_brand#22, cc_name#23], [d_year#24 ASC NULLS FIRST, d_moy#25 ASC NULLS FIRST] + +(34) Project [codegen id : 14] +Output [5]: [i_category#21, i_brand#22, cc_name#23, sum_sales#28, rn#29] +Input [7]: [i_category#21, i_brand#22, cc_name#23, d_year#24, d_moy#25, sum_sales#28, rn#29] + +(35) BroadcastExchange +Input [5]: [i_category#21, i_brand#22, cc_name#23, sum_sales#28, rn#29] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], (input[4, int, false] + 1)),false), [plan_id=6] + +(36) BroadcastHashJoin [codegen id : 22] +Left keys [4]: [i_category#3, i_brand#2, cc_name#13, rn#19] +Right keys [4]: [i_category#21, i_brand#22, cc_name#23, (rn#29 + 1)] +Join type: Inner +Join condition: None + +(37) Project [codegen id : 22] +Output [9]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, sum_sales#17, avg_monthly_sales#20, rn#19, sum_sales#28] +Input [13]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, sum_sales#17, avg_monthly_sales#20, rn#19, i_category#21, i_brand#22, cc_name#23, sum_sales#28, rn#29] + +(38) ReusedExchange [Reuses operator id: 31] +Output [6]: [i_category#30, i_brand#31, cc_name#32, d_year#33, d_moy#34, sum_sales#35] + +(39) Sort [codegen id : 20] +Input [6]: [i_category#30, i_brand#31, cc_name#32, d_year#33, d_moy#34, sum_sales#35] +Arguments: [i_category#30 ASC NULLS FIRST, i_brand#31 ASC NULLS FIRST, cc_name#32 ASC NULLS FIRST, d_year#33 ASC NULLS FIRST, d_moy#34 ASC NULLS FIRST], false, 0 + +(40) Window +Input [6]: [i_category#30, i_brand#31, cc_name#32, d_year#33, d_moy#34, sum_sales#35] +Arguments: [rank(d_year#33, d_moy#34) windowspecdefinition(i_category#30, i_brand#31, cc_name#32, d_year#33 ASC NULLS FIRST, d_moy#34 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#36], [i_category#30, i_brand#31, cc_name#32], [d_year#33 ASC NULLS FIRST, d_moy#34 ASC NULLS FIRST] + +(41) Project [codegen id : 21] +Output [5]: [i_category#30, i_brand#31, cc_name#32, sum_sales#35, rn#36] +Input [7]: [i_category#30, i_brand#31, cc_name#32, d_year#33, d_moy#34, sum_sales#35, rn#36] + +(42) BroadcastExchange +Input [5]: [i_category#30, i_brand#31, cc_name#32, sum_sales#35, rn#36] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], (input[4, int, false] - 1)),false), [plan_id=7] + +(43) BroadcastHashJoin [codegen id : 22] +Left keys [4]: [i_category#3, i_brand#2, cc_name#13, rn#19] +Right keys [4]: [i_category#30, i_brand#31, cc_name#32, (rn#36 - 1)] +Join type: Inner +Join condition: None + +(44) Project [codegen id : 22] +Output [9]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, avg_monthly_sales#20, sum_sales#17, sum_sales#28 AS psum#37, sum_sales#35 AS nsum#38] +Input [14]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, sum_sales#17, avg_monthly_sales#20, rn#19, sum_sales#28, i_category#30, i_brand#31, cc_name#32, sum_sales#35, rn#36] + +(45) TakeOrderedAndProject +Input [9]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, avg_monthly_sales#20, sum_sales#17, psum#37, nsum#38] +Arguments: 100, [(sum_sales#17 - avg_monthly_sales#20) ASC NULLS FIRST, cc_name#13 ASC NULLS FIRST], [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, avg_monthly_sales#20, sum_sales#17, psum#37, nsum#38] + +===== Subqueries ===== + +Subquery:1 Hosting operator id = 4 Hosting Expression = cs_sold_date_sk#7 IN dynamicpruning#8 +BroadcastExchange (49) ++- * ColumnarToRow (48) + +- CometFilter (47) + +- CometScan parquet spark_catalog.default.date_dim (46) + + +(46) Scan parquet spark_catalog.default.date_dim +Output [3]: [d_date_sk#9, d_year#10, d_moy#11] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [Or(Or(EqualTo(d_year,1999),And(EqualTo(d_year,1998),EqualTo(d_moy,12))),And(EqualTo(d_year,2000),EqualTo(d_moy,1))), IsNotNull(d_date_sk)] +ReadSchema: struct<d_date_sk:int,d_year:int,d_moy:int> + +(47) CometFilter +Input [3]: [d_date_sk#9, d_year#10, d_moy#11] +Condition : ((((d_year#10 = 1999) OR ((d_year#10 = 1998) AND (d_moy#11 = 12))) OR ((d_year#10 = 2000) AND (d_moy#11 = 1))) AND isnotnull(d_date_sk#9)) + +(48) ColumnarToRow [codegen id : 1] +Input [3]: [d_date_sk#9, d_year#10, d_moy#11] + +(49) BroadcastExchange +Input [3]: [d_date_sk#9, d_year#10, d_moy#11] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=8] + + 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 new file mode 100644 index 000000000..3bc01343a --- /dev/null +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q57/simplified.txt @@ -0,0 +1,81 @@ +TakeOrderedAndProject [sum_sales,avg_monthly_sales,cc_name,i_category,i_brand,d_year,d_moy,psum,nsum] + WholeStageCodegen (22) + 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] + 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] + Filter [avg_monthly_sales,sum_sales] + InputAdapter + Window [_w0,i_category,i_brand,cc_name,d_year] + WholeStageCodegen (7) + Filter [d_year] + InputAdapter + Window [d_year,d_moy,i_category,i_brand,cc_name] + WholeStageCodegen (6) + Sort [i_category,i_brand,cc_name,d_year,d_moy] + InputAdapter + Exchange [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] + InputAdapter + Exchange [i_category,i_brand,cc_name,d_year,d_moy] #2 + WholeStageCodegen (4) + HashAggregate [i_category,i_brand,cc_name,d_year,d_moy,cs_sales_price] [sum,sum] + Project [i_brand,i_category,cs_sales_price,d_year,d_moy,cc_name] + BroadcastHashJoin [cs_call_center_sk,cc_call_center_sk] + Project [i_brand,i_category,cs_call_center_sk,cs_sales_price,d_year,d_moy] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + Project [i_brand,i_category,cs_call_center_sk,cs_sales_price,cs_sold_date_sk] + BroadcastHashJoin [i_item_sk,cs_item_sk] + ColumnarToRow + InputAdapter + CometFilter [i_item_sk,i_category,i_brand] + CometScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_category] + InputAdapter + BroadcastExchange #3 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometFilter [cs_item_sk,cs_call_center_sk] + CometScan parquet spark_catalog.default.catalog_sales [cs_call_center_sk,cs_item_sk,cs_sales_price,cs_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #4 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometFilter [d_year,d_moy,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + InputAdapter + ReusedExchange [d_date_sk,d_year,d_moy] #4 + InputAdapter + BroadcastExchange #5 + WholeStageCodegen (3) + ColumnarToRow + InputAdapter + CometFilter [cc_call_center_sk,cc_name] + CometScan parquet spark_catalog.default.call_center [cc_call_center_sk,cc_name] + InputAdapter + BroadcastExchange #6 + WholeStageCodegen (14) + Project [i_category,i_brand,cc_name,sum_sales,rn] + InputAdapter + Window [d_year,d_moy,i_category,i_brand,cc_name] + WholeStageCodegen (13) + Sort [i_category,i_brand,cc_name,d_year,d_moy] + InputAdapter + Exchange [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] + InputAdapter + ReusedExchange [i_category,i_brand,cc_name,d_year,d_moy,sum] #2 + InputAdapter + BroadcastExchange #8 + WholeStageCodegen (21) + Project [i_category,i_brand,cc_name,sum_sales,rn] + InputAdapter + Window [d_year,d_moy,i_category,i_brand,cc_name] + WholeStageCodegen (20) + Sort [i_category,i_brand,cc_name,d_year,d_moy] + InputAdapter + ReusedExchange [i_category,i_brand,cc_name,d_year,d_moy,sum_sales] #7 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 new file mode 100644 index 000000000..35fedb6d3 --- /dev/null +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q58/explain.txt @@ -0,0 +1,386 @@ +== Physical Plan == +TakeOrderedAndProject (49) ++- * Project (48) + +- * BroadcastHashJoin Inner BuildRight (47) + :- * Project (32) + : +- * BroadcastHashJoin Inner BuildRight (31) + : :- * Filter (16) + : : +- * HashAggregate (15) + : : +- Exchange (14) + : : +- * HashAggregate (13) + : : +- * Project (12) + : : +- * BroadcastHashJoin Inner BuildRight (11) + : : :- * Project (9) + : : : +- * BroadcastHashJoin Inner BuildRight (8) + : : : :- * ColumnarToRow (3) + : : : : +- CometFilter (2) + : : : : +- CometScan parquet spark_catalog.default.store_sales (1) + : : : +- BroadcastExchange (7) + : : : +- * ColumnarToRow (6) + : : : +- CometFilter (5) + : : : +- CometScan parquet spark_catalog.default.item (4) + : : +- ReusedExchange (10) + : +- BroadcastExchange (30) + : +- * Filter (29) + : +- * HashAggregate (28) + : +- Exchange (27) + : +- * HashAggregate (26) + : +- * Project (25) + : +- * BroadcastHashJoin Inner BuildRight (24) + : :- * Project (22) + : : +- * BroadcastHashJoin Inner BuildRight (21) + : : :- * ColumnarToRow (19) + : : : +- CometFilter (18) + : : : +- CometScan parquet spark_catalog.default.catalog_sales (17) + : : +- ReusedExchange (20) + : +- ReusedExchange (23) + +- BroadcastExchange (46) + +- * Filter (45) + +- * HashAggregate (44) + +- Exchange (43) + +- * HashAggregate (42) + +- * Project (41) + +- * BroadcastHashJoin Inner BuildRight (40) + :- * Project (38) + : +- * BroadcastHashJoin Inner BuildRight (37) + : :- * ColumnarToRow (35) + : : +- CometFilter (34) + : : +- CometScan parquet spark_catalog.default.web_sales (33) + : +- ReusedExchange (36) + +- ReusedExchange (39) + + +(1) Scan parquet spark_catalog.default.store_sales +Output [3]: [ss_item_sk#1, ss_ext_sales_price#2, ss_sold_date_sk#3] +Batched: true +Location: InMemoryFileIndex [] +PartitionFilters: [isnotnull(ss_sold_date_sk#3), dynamicpruningexpression(ss_sold_date_sk#3 IN dynamicpruning#4)] +PushedFilters: [IsNotNull(ss_item_sk)] +ReadSchema: struct<ss_item_sk:int,ss_ext_sales_price:decimal(7,2)> + +(2) CometFilter +Input [3]: [ss_item_sk#1, ss_ext_sales_price#2, ss_sold_date_sk#3] +Condition : isnotnull(ss_item_sk#1) + +(3) ColumnarToRow [codegen id : 4] +Input [3]: [ss_item_sk#1, ss_ext_sales_price#2, ss_sold_date_sk#3] + +(4) Scan parquet spark_catalog.default.item +Output [2]: [i_item_sk#5, i_item_id#6] +Batched: true +Location [not included in comparison]/{warehouse_dir}/item] +PushedFilters: [IsNotNull(i_item_sk), IsNotNull(i_item_id)] +ReadSchema: struct<i_item_sk:int,i_item_id:string> + +(5) CometFilter +Input [2]: [i_item_sk#5, i_item_id#6] +Condition : (isnotnull(i_item_sk#5) AND isnotnull(i_item_id#6)) + +(6) ColumnarToRow [codegen id : 1] +Input [2]: [i_item_sk#5, i_item_id#6] + +(7) BroadcastExchange +Input [2]: [i_item_sk#5, i_item_id#6] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=1] + +(8) BroadcastHashJoin [codegen id : 4] +Left keys [1]: [ss_item_sk#1] +Right keys [1]: [i_item_sk#5] +Join type: Inner +Join condition: None + +(9) Project [codegen id : 4] +Output [3]: [ss_ext_sales_price#2, ss_sold_date_sk#3, i_item_id#6] +Input [5]: [ss_item_sk#1, ss_ext_sales_price#2, ss_sold_date_sk#3, i_item_sk#5, i_item_id#6] + +(10) ReusedExchange [Reuses operator id: 60] +Output [1]: [d_date_sk#7] + +(11) BroadcastHashJoin [codegen id : 4] +Left keys [1]: [ss_sold_date_sk#3] +Right keys [1]: [d_date_sk#7] +Join type: Inner +Join condition: None + +(12) Project [codegen id : 4] +Output [2]: [ss_ext_sales_price#2, i_item_id#6] +Input [4]: [ss_ext_sales_price#2, ss_sold_date_sk#3, i_item_id#6, d_date_sk#7] + +(13) HashAggregate [codegen id : 4] +Input [2]: [ss_ext_sales_price#2, i_item_id#6] +Keys [1]: [i_item_id#6] +Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#2))] +Aggregate Attributes [1]: [sum#8] +Results [2]: [i_item_id#6, sum#9] + +(14) Exchange +Input [2]: [i_item_id#6, sum#9] +Arguments: hashpartitioning(i_item_id#6, 5), ENSURE_REQUIREMENTS, [plan_id=2] + +(15) HashAggregate [codegen id : 15] +Input [2]: [i_item_id#6, sum#9] +Keys [1]: [i_item_id#6] +Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#2))] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#2))#10] +Results [2]: [i_item_id#6 AS item_id#11, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#2))#10,17,2) AS ss_item_rev#12] + +(16) Filter [codegen id : 15] +Input [2]: [item_id#11, ss_item_rev#12] +Condition : isnotnull(ss_item_rev#12) + +(17) Scan parquet spark_catalog.default.catalog_sales +Output [3]: [cs_item_sk#13, cs_ext_sales_price#14, cs_sold_date_sk#15] +Batched: true +Location: InMemoryFileIndex [] +PartitionFilters: [isnotnull(cs_sold_date_sk#15), dynamicpruningexpression(cs_sold_date_sk#15 IN dynamicpruning#16)] +PushedFilters: [IsNotNull(cs_item_sk)] +ReadSchema: struct<cs_item_sk:int,cs_ext_sales_price:decimal(7,2)> + +(18) CometFilter +Input [3]: [cs_item_sk#13, cs_ext_sales_price#14, cs_sold_date_sk#15] +Condition : isnotnull(cs_item_sk#13) + +(19) ColumnarToRow [codegen id : 8] +Input [3]: [cs_item_sk#13, cs_ext_sales_price#14, cs_sold_date_sk#15] + +(20) ReusedExchange [Reuses operator id: 7] +Output [2]: [i_item_sk#17, i_item_id#18] + +(21) BroadcastHashJoin [codegen id : 8] +Left keys [1]: [cs_item_sk#13] +Right keys [1]: [i_item_sk#17] +Join type: Inner +Join condition: None + +(22) Project [codegen id : 8] +Output [3]: [cs_ext_sales_price#14, cs_sold_date_sk#15, i_item_id#18] +Input [5]: [cs_item_sk#13, cs_ext_sales_price#14, cs_sold_date_sk#15, i_item_sk#17, i_item_id#18] + +(23) ReusedExchange [Reuses operator id: 60] +Output [1]: [d_date_sk#19] + +(24) BroadcastHashJoin [codegen id : 8] +Left keys [1]: [cs_sold_date_sk#15] +Right keys [1]: [d_date_sk#19] +Join type: Inner +Join condition: None + +(25) Project [codegen id : 8] +Output [2]: [cs_ext_sales_price#14, i_item_id#18] +Input [4]: [cs_ext_sales_price#14, cs_sold_date_sk#15, i_item_id#18, d_date_sk#19] + +(26) HashAggregate [codegen id : 8] +Input [2]: [cs_ext_sales_price#14, i_item_id#18] +Keys [1]: [i_item_id#18] +Functions [1]: [partial_sum(UnscaledValue(cs_ext_sales_price#14))] +Aggregate Attributes [1]: [sum#20] +Results [2]: [i_item_id#18, sum#21] + +(27) Exchange +Input [2]: [i_item_id#18, sum#21] +Arguments: hashpartitioning(i_item_id#18, 5), ENSURE_REQUIREMENTS, [plan_id=3] + +(28) HashAggregate [codegen id : 9] +Input [2]: [i_item_id#18, sum#21] +Keys [1]: [i_item_id#18] +Functions [1]: [sum(UnscaledValue(cs_ext_sales_price#14))] +Aggregate Attributes [1]: [sum(UnscaledValue(cs_ext_sales_price#14))#22] +Results [2]: [i_item_id#18 AS item_id#23, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#14))#22,17,2) AS cs_item_rev#24] + +(29) Filter [codegen id : 9] +Input [2]: [item_id#23, cs_item_rev#24] +Condition : isnotnull(cs_item_rev#24) + +(30) BroadcastExchange +Input [2]: [item_id#23, cs_item_rev#24] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=4] + +(31) BroadcastHashJoin [codegen id : 15] +Left keys [1]: [item_id#11] +Right keys [1]: [item_id#23] +Join type: Inner +Join condition: ((((cast(ss_item_rev#12 as decimal(19,3)) >= (0.9 * cs_item_rev#24)) AND (cast(ss_item_rev#12 as decimal(20,3)) <= (1.1 * cs_item_rev#24))) AND (cast(cs_item_rev#24 as decimal(19,3)) >= (0.9 * ss_item_rev#12))) AND (cast(cs_item_rev#24 as decimal(20,3)) <= (1.1 * ss_item_rev#12))) + +(32) Project [codegen id : 15] +Output [3]: [item_id#11, ss_item_rev#12, cs_item_rev#24] +Input [4]: [item_id#11, ss_item_rev#12, item_id#23, cs_item_rev#24] + +(33) Scan parquet spark_catalog.default.web_sales +Output [3]: [ws_item_sk#25, ws_ext_sales_price#26, ws_sold_date_sk#27] +Batched: true +Location: InMemoryFileIndex [] +PartitionFilters: [isnotnull(ws_sold_date_sk#27), dynamicpruningexpression(ws_sold_date_sk#27 IN dynamicpruning#28)] +PushedFilters: [IsNotNull(ws_item_sk)] +ReadSchema: struct<ws_item_sk:int,ws_ext_sales_price:decimal(7,2)> + +(34) CometFilter +Input [3]: [ws_item_sk#25, ws_ext_sales_price#26, ws_sold_date_sk#27] +Condition : isnotnull(ws_item_sk#25) + +(35) ColumnarToRow [codegen id : 13] +Input [3]: [ws_item_sk#25, ws_ext_sales_price#26, ws_sold_date_sk#27] + +(36) ReusedExchange [Reuses operator id: 7] +Output [2]: [i_item_sk#29, i_item_id#30] + +(37) BroadcastHashJoin [codegen id : 13] +Left keys [1]: [ws_item_sk#25] +Right keys [1]: [i_item_sk#29] +Join type: Inner +Join condition: None + +(38) Project [codegen id : 13] +Output [3]: [ws_ext_sales_price#26, ws_sold_date_sk#27, i_item_id#30] +Input [5]: [ws_item_sk#25, ws_ext_sales_price#26, ws_sold_date_sk#27, i_item_sk#29, i_item_id#30] + +(39) ReusedExchange [Reuses operator id: 60] +Output [1]: [d_date_sk#31] + +(40) BroadcastHashJoin [codegen id : 13] +Left keys [1]: [ws_sold_date_sk#27] +Right keys [1]: [d_date_sk#31] +Join type: Inner +Join condition: None + +(41) Project [codegen id : 13] +Output [2]: [ws_ext_sales_price#26, i_item_id#30] +Input [4]: [ws_ext_sales_price#26, ws_sold_date_sk#27, i_item_id#30, d_date_sk#31] + +(42) HashAggregate [codegen id : 13] +Input [2]: [ws_ext_sales_price#26, i_item_id#30] +Keys [1]: [i_item_id#30] +Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#26))] +Aggregate Attributes [1]: [sum#32] +Results [2]: [i_item_id#30, sum#33] + +(43) Exchange +Input [2]: [i_item_id#30, sum#33] +Arguments: hashpartitioning(i_item_id#30, 5), ENSURE_REQUIREMENTS, [plan_id=5] + +(44) HashAggregate [codegen id : 14] +Input [2]: [i_item_id#30, sum#33] +Keys [1]: [i_item_id#30] +Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#26))] +Aggregate Attributes [1]: [sum(UnscaledValue(ws_ext_sales_price#26))#34] +Results [2]: [i_item_id#30 AS item_id#35, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#26))#34,17,2) AS ws_item_rev#36] + +(45) Filter [codegen id : 14] +Input [2]: [item_id#35, ws_item_rev#36] +Condition : isnotnull(ws_item_rev#36) + +(46) BroadcastExchange +Input [2]: [item_id#35, ws_item_rev#36] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=6] + +(47) BroadcastHashJoin [codegen id : 15] +Left keys [1]: [item_id#11] +Right keys [1]: [item_id#35] +Join type: Inner +Join condition: ((((((((cast(ss_item_rev#12 as decimal(19,3)) >= (0.9 * ws_item_rev#36)) AND (cast(ss_item_rev#12 as decimal(20,3)) <= (1.1 * ws_item_rev#36))) AND (cast(cs_item_rev#24 as decimal(19,3)) >= (0.9 * ws_item_rev#36))) AND (cast(cs_item_rev#24 as decimal(20,3)) <= (1.1 * ws_item_rev#36))) AND (cast(ws_item_rev#36 as decimal(19,3)) >= (0.9 * ss_item_rev#12))) AND (cast(ws_item_rev#36 as decimal(20,3)) <= (1.1 * ss_item_rev#12))) AND (cast(ws_item_rev#36 as decimal(19,3)) >= (0.9 * cs_item_rev#24))) AND (cast(ws_item_rev#36 as decimal(20,3)) <= (1.1 * cs_item_rev#24))) + +(48) Project [codegen id : 15] +Output [8]: [item_id#11, ss_item_rev#12, (((ss_item_rev#12 / ((ss_item_rev#12 + cs_item_rev#24) + ws_item_rev#36)) / 3) * 100) AS ss_dev#37, cs_item_rev#24, (((cs_item_rev#24 / ((ss_item_rev#12 + cs_item_rev#24) + ws_item_rev#36)) / 3) * 100) AS cs_dev#38, ws_item_rev#36, (((ws_item_rev#36 / ((ss_item_rev#12 + cs_item_rev#24) + ws_item_rev#36)) / 3) * 100) AS ws_dev#39, (((ss_item_rev#12 + cs_item_rev#24) + ws_item_rev#36) / 3) AS average#40] +Input [5]: [item_id#11, ss_item_rev#12, cs_item_rev#24, item_id#35, ws_item_rev#36] + +(49) TakeOrderedAndProject +Input [8]: [item_id#11, ss_item_rev#12, ss_dev#37, cs_item_rev#24, cs_dev#38, ws_item_rev#36, ws_dev#39, average#40] +Arguments: 100, [item_id#11 ASC NULLS FIRST, ss_item_rev#12 ASC NULLS FIRST], [item_id#11, ss_item_rev#12, ss_dev#37, cs_item_rev#24, cs_dev#38, ws_item_rev#36, ws_dev#39, average#40] + +===== Subqueries ===== + +Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#3 IN dynamicpruning#4 +BroadcastExchange (60) ++- * Project (59) + +- * BroadcastHashJoin LeftSemi BuildRight (58) + :- * ColumnarToRow (52) + : +- CometFilter (51) + : +- CometScan parquet spark_catalog.default.date_dim (50) + +- BroadcastExchange (57) + +- * ColumnarToRow (56) + +- CometProject (55) + +- CometFilter (54) + +- CometScan parquet spark_catalog.default.date_dim (53) + + +(50) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#7, d_date#41] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_date_sk)] +ReadSchema: struct<d_date_sk:int,d_date:date> + +(51) CometFilter +Input [2]: [d_date_sk#7, d_date#41] +Condition : isnotnull(d_date_sk#7) + +(52) ColumnarToRow [codegen id : 2] +Input [2]: [d_date_sk#7, d_date#41] + +(53) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date#42, d_week_seq#43] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_week_seq), EqualTo(d_week_seq,ScalarSubquery#44)] +ReadSchema: struct<d_date:date,d_week_seq:int> + +(54) CometFilter +Input [2]: [d_date#42, d_week_seq#43] +Condition : (isnotnull(d_week_seq#43) AND (d_week_seq#43 = ReusedSubquery Subquery scalar-subquery#44, [id=#45])) + +(55) CometProject +Input [2]: [d_date#42, d_week_seq#43] +Arguments: [d_date#42], [d_date#42] + +(56) ColumnarToRow [codegen id : 1] +Input [1]: [d_date#42] + +(57) BroadcastExchange +Input [1]: [d_date#42] +Arguments: HashedRelationBroadcastMode(List(input[0, date, true]),false), [plan_id=7] + +(58) BroadcastHashJoin [codegen id : 2] +Left keys [1]: [d_date#41] +Right keys [1]: [d_date#42] +Join type: LeftSemi +Join condition: None + +(59) Project [codegen id : 2] +Output [1]: [d_date_sk#7] +Input [2]: [d_date_sk#7, d_date#41] + +(60) BroadcastExchange +Input [1]: [d_date_sk#7] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=8] + +Subquery:2 Hosting operator id = 54 Hosting Expression = ReusedSubquery Subquery scalar-subquery#44, [id=#45] + +Subquery:3 Hosting operator id = 53 Hosting Expression = Subquery scalar-subquery#44, [id=#45] +* ColumnarToRow (64) ++- CometProject (63) + +- CometFilter (62) + +- CometScan parquet spark_catalog.default.date_dim (61) + + +(61) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date#46, d_week_seq#47] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_date), EqualTo(d_date,2000-01-03)] +ReadSchema: struct<d_date:date,d_week_seq:int> + +(62) CometFilter +Input [2]: [d_date#46, d_week_seq#47] +Condition : (isnotnull(d_date#46) AND (d_date#46 = 2000-01-03)) + +(63) CometProject +Input [2]: [d_date#46, d_week_seq#47] +Arguments: [d_week_seq#47], [d_week_seq#47] + +(64) ColumnarToRow [codegen id : 1] +Input [1]: [d_week_seq#47] + +Subquery:4 Hosting operator id = 17 Hosting Expression = cs_sold_date_sk#15 IN dynamicpruning#4 + +Subquery:5 Hosting operator id = 33 Hosting Expression = ws_sold_date_sk#27 IN dynamicpruning#4 + + diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q58/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q58/simplified.txt new file mode 100644 index 000000000..c7ed479ad --- /dev/null +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q58/simplified.txt @@ -0,0 +1,98 @@ +TakeOrderedAndProject [item_id,ss_item_rev,ss_dev,cs_item_rev,cs_dev,ws_item_rev,ws_dev,average] + WholeStageCodegen (15) + Project [item_id,ss_item_rev,cs_item_rev,ws_item_rev] + BroadcastHashJoin [item_id,item_id,ss_item_rev,ws_item_rev,cs_item_rev] + Project [item_id,ss_item_rev,cs_item_rev] + BroadcastHashJoin [item_id,item_id,ss_item_rev,cs_item_rev] + Filter [ss_item_rev] + HashAggregate [i_item_id,sum] [sum(UnscaledValue(ss_ext_sales_price)),item_id,ss_item_rev,sum] + InputAdapter + Exchange [i_item_id] #1 + WholeStageCodegen (4) + HashAggregate [i_item_id,ss_ext_sales_price] [sum,sum] + Project [ss_ext_sales_price,i_item_id] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [ss_ext_sales_price,ss_sold_date_sk,i_item_id] + BroadcastHashJoin [ss_item_sk,i_item_sk] + ColumnarToRow + InputAdapter + CometFilter [ss_item_sk] + CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #2 + WholeStageCodegen (2) + Project [d_date_sk] + BroadcastHashJoin [d_date,d_date] + ColumnarToRow + InputAdapter + CometFilter [d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] + InputAdapter + BroadcastExchange #3 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date] + CometFilter [d_week_seq] + ReusedSubquery [d_week_seq] #2 + CometScan parquet spark_catalog.default.date_dim [d_date,d_week_seq] + Subquery #2 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_week_seq] + CometFilter [d_date] + CometScan parquet spark_catalog.default.date_dim [d_date,d_week_seq] + InputAdapter + BroadcastExchange #4 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometFilter [i_item_sk,i_item_id] + CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id] + InputAdapter + ReusedExchange [d_date_sk] #2 + InputAdapter + BroadcastExchange #5 + WholeStageCodegen (9) + Filter [cs_item_rev] + HashAggregate [i_item_id,sum] [sum(UnscaledValue(cs_ext_sales_price)),item_id,cs_item_rev,sum] + InputAdapter + Exchange [i_item_id] #6 + WholeStageCodegen (8) + HashAggregate [i_item_id,cs_ext_sales_price] [sum,sum] + Project [cs_ext_sales_price,i_item_id] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + Project [cs_ext_sales_price,cs_sold_date_sk,i_item_id] + BroadcastHashJoin [cs_item_sk,i_item_sk] + ColumnarToRow + InputAdapter + CometFilter [cs_item_sk] + CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + InputAdapter + ReusedExchange [i_item_sk,i_item_id] #4 + InputAdapter + ReusedExchange [d_date_sk] #2 + InputAdapter + BroadcastExchange #7 + WholeStageCodegen (14) + Filter [ws_item_rev] + HashAggregate [i_item_id,sum] [sum(UnscaledValue(ws_ext_sales_price)),item_id,ws_item_rev,sum] + InputAdapter + Exchange [i_item_id] #8 + WholeStageCodegen (13) + HashAggregate [i_item_id,ws_ext_sales_price] [sum,sum] + Project [ws_ext_sales_price,i_item_id] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + Project [ws_ext_sales_price,ws_sold_date_sk,i_item_id] + BroadcastHashJoin [ws_item_sk,i_item_sk] + ColumnarToRow + InputAdapter + CometFilter [ws_item_sk] + CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_ext_sales_price,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + InputAdapter + ReusedExchange [i_item_sk,i_item_id] #4 + InputAdapter + ReusedExchange [d_date_sk] #2 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q59/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q59/explain.txt new file mode 100644 index 000000000..023c54bdd --- /dev/null +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q59/explain.txt @@ -0,0 +1,256 @@ +== Physical Plan == +TakeOrderedAndProject (44) ++- * Project (43) + +- * BroadcastHashJoin Inner BuildRight (42) + :- * Project (25) + : +- * BroadcastHashJoin Inner BuildRight (24) + : :- * Project (18) + : : +- * BroadcastHashJoin Inner BuildRight (17) + : : :- * HashAggregate (12) + : : : +- Exchange (11) + : : : +- * HashAggregate (10) + : : : +- * Project (9) + : : : +- * BroadcastHashJoin Inner BuildRight (8) + : : : :- * ColumnarToRow (3) + : : : : +- CometFilter (2) + : : : : +- CometScan parquet spark_catalog.default.store_sales (1) + : : : +- BroadcastExchange (7) + : : : +- * ColumnarToRow (6) + : : : +- CometFilter (5) + : : : +- CometScan parquet spark_catalog.default.date_dim (4) + : : +- BroadcastExchange (16) + : : +- * ColumnarToRow (15) + : : +- CometFilter (14) + : : +- CometScan parquet spark_catalog.default.store (13) + : +- BroadcastExchange (23) + : +- * ColumnarToRow (22) + : +- CometProject (21) + : +- CometFilter (20) + : +- CometScan parquet spark_catalog.default.date_dim (19) + +- BroadcastExchange (41) + +- * Project (40) + +- * BroadcastHashJoin Inner BuildRight (39) + :- * Project (33) + : +- * BroadcastHashJoin Inner BuildRight (32) + : :- * HashAggregate (27) + : : +- ReusedExchange (26) + : +- BroadcastExchange (31) + : +- * ColumnarToRow (30) + : +- CometFilter (29) + : +- CometScan parquet spark_catalog.default.store (28) + +- BroadcastExchange (38) + +- * ColumnarToRow (37) + +- CometProject (36) + +- CometFilter (35) + +- CometScan parquet spark_catalog.default.date_dim (34) + + +(1) Scan parquet spark_catalog.default.store_sales +Output [3]: [ss_store_sk#1, ss_sales_price#2, ss_sold_date_sk#3] +Batched: true +Location: InMemoryFileIndex [] +PartitionFilters: [isnotnull(ss_sold_date_sk#3)] +PushedFilters: [IsNotNull(ss_store_sk)] +ReadSchema: struct<ss_store_sk:int,ss_sales_price:decimal(7,2)> + +(2) CometFilter +Input [3]: [ss_store_sk#1, ss_sales_price#2, ss_sold_date_sk#3] +Condition : isnotnull(ss_store_sk#1) + +(3) ColumnarToRow [codegen id : 2] +Input [3]: [ss_store_sk#1, ss_sales_price#2, ss_sold_date_sk#3] + +(4) Scan parquet spark_catalog.default.date_dim +Output [3]: [d_date_sk#4, d_week_seq#5, d_day_name#6] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_date_sk), IsNotNull(d_week_seq)] +ReadSchema: struct<d_date_sk:int,d_week_seq:int,d_day_name:string> + +(5) CometFilter +Input [3]: [d_date_sk#4, d_week_seq#5, d_day_name#6] +Condition : (isnotnull(d_date_sk#4) AND isnotnull(d_week_seq#5)) + +(6) ColumnarToRow [codegen id : 1] +Input [3]: [d_date_sk#4, d_week_seq#5, d_day_name#6] + +(7) BroadcastExchange +Input [3]: [d_date_sk#4, d_week_seq#5, d_day_name#6] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=1] + +(8) BroadcastHashJoin [codegen id : 2] +Left keys [1]: [ss_sold_date_sk#3] +Right keys [1]: [d_date_sk#4] +Join type: Inner +Join condition: None + +(9) Project [codegen id : 2] +Output [4]: [ss_store_sk#1, ss_sales_price#2, d_week_seq#5, d_day_name#6] +Input [6]: [ss_store_sk#1, ss_sales_price#2, ss_sold_date_sk#3, d_date_sk#4, d_week_seq#5, d_day_name#6] + +(10) HashAggregate [codegen id : 2] +Input [4]: [ss_store_sk#1, ss_sales_price#2, d_week_seq#5, d_day_name#6] +Keys [2]: [d_week_seq#5, ss_store_sk#1] +Functions [7]: [partial_sum(UnscaledValue(CASE WHEN (d_day_name#6 = Sunday ) THEN ss_sales_price#2 END)), partial_sum(UnscaledValue(CASE WHEN (d_day_name#6 = Monday ) THEN ss_sales_price#2 END)), partial_sum(UnscaledValue(CASE WHEN (d_day_name#6 = Tuesday ) THEN ss_sales_price#2 END)), partial_sum(UnscaledValue(CASE WHEN (d_day_name#6 = Wednesday) THEN ss_sales_price#2 END)), partial_sum(UnscaledValue(CASE WHEN (d_day_name#6 = Thursday ) THEN ss_sales_price#2 END)), partial_sum(UnscaledValue(CASE WHEN (d_day_name#6 = Friday ) THEN ss_sales_price#2 END)), partial_sum(UnscaledValue(CASE WHEN (d_day_name#6 = Saturday ) THEN ss_sales_price#2 END))] +Aggregate Attributes [7]: [sum#7, sum#8, sum#9, sum#10, sum#11, sum#12, sum#13] +Results [9]: [d_week_seq#5, ss_store_sk#1, sum#14, sum#15, sum#16, sum#17, sum#18, sum#19, sum#20] + +(11) Exchange +Input [9]: [d_week_seq#5, ss_store_sk#1, sum#14, sum#15, sum#16, sum#17, sum#18, sum#19, sum#20] +Arguments: hashpartitioning(d_week_seq#5, ss_store_sk#1, 5), ENSURE_REQUIREMENTS, [plan_id=2] + +(12) HashAggregate [codegen id : 10] +Input [9]: [d_week_seq#5, ss_store_sk#1, sum#14, sum#15, sum#16, sum#17, sum#18, sum#19, sum#20] +Keys [2]: [d_week_seq#5, ss_store_sk#1] +Functions [7]: [sum(UnscaledValue(CASE WHEN (d_day_name#6 = Sunday ) THEN ss_sales_price#2 END)), sum(UnscaledValue(CASE WHEN (d_day_name#6 = Monday ) THEN ss_sales_price#2 END)), sum(UnscaledValue(CASE WHEN (d_day_name#6 = Tuesday ) THEN ss_sales_price#2 END)), sum(UnscaledValue(CASE WHEN (d_day_name#6 = Wednesday) THEN ss_sales_price#2 END)), sum(UnscaledValue(CASE WHEN (d_day_name#6 = Thursday ) THEN ss_sales_price#2 END)), sum(UnscaledValue(CASE WHEN (d_day_name#6 = Friday ) THEN ss_sales_price#2 END)), sum(UnscaledValue(CASE WHEN (d_day_name#6 = Saturday ) THEN ss_sales_price#2 END))] +Aggregate Attributes [7]: [sum(UnscaledValue(CASE WHEN (d_day_name#6 = Sunday ) THEN ss_sales_price#2 END))#21, sum(UnscaledValue(CASE WHEN (d_day_name#6 = Monday ) THEN ss_sales_price#2 END))#22, sum(UnscaledValue(CASE WHEN (d_day_name#6 = Tuesday ) THEN ss_sales_price#2 END))#23, sum(UnscaledValue(CASE WHEN (d_day_name#6 = Wednesday) THEN ss_sales_price#2 END))#24, sum(UnscaledValue(CASE WHEN (d_day_name#6 = Thursday ) THEN ss_sales_price#2 END))#25, sum(UnscaledValue(CASE WHEN (d_day_name#6 = Friday ) THEN ss_sales_price#2 END))#26, sum(UnscaledValue(CASE WHEN (d_day_name#6 = Saturday ) THEN ss_sales_price#2 END))#27] +Results [9]: [d_week_seq#5, ss_store_sk#1, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#6 = Sunday ) THEN ss_sales_price#2 END))#21,17,2) AS sun_sales#28, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#6 = Monday ) THEN ss_sales_price#2 END))#22,17,2) AS mon_sales#29, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#6 = Tuesday ) THEN ss_sales_price#2 END))#23,17,2) AS tue_sales#30, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#6 = Wednesday) THEN ss_sales_price#2 END))#24,17,2) AS wed_sales#31, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#6 = Thursday ) THEN ss_sales_price#2 END))#25,17,2) AS thu_sales#32, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#6 = Friday ) THEN ss_sales_price#2 END))#26,17,2) AS fri_sales#33, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#6 = Saturday ) THEN ss_sales_price#2 END))#27,17,2) AS sat_sales#34] + +(13) Scan parquet spark_catalog.default.store +Output [3]: [s_store_sk#35, s_store_id#36, s_store_name#37] +Batched: true +Location [not included in comparison]/{warehouse_dir}/store] +PushedFilters: [IsNotNull(s_store_sk), IsNotNull(s_store_id)] +ReadSchema: struct<s_store_sk:int,s_store_id:string,s_store_name:string> + +(14) CometFilter +Input [3]: [s_store_sk#35, s_store_id#36, s_store_name#37] +Condition : (isnotnull(s_store_sk#35) AND isnotnull(s_store_id#36)) + +(15) ColumnarToRow [codegen id : 3] +Input [3]: [s_store_sk#35, s_store_id#36, s_store_name#37] + +(16) BroadcastExchange +Input [3]: [s_store_sk#35, s_store_id#36, s_store_name#37] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=3] + +(17) BroadcastHashJoin [codegen id : 10] +Left keys [1]: [ss_store_sk#1] +Right keys [1]: [s_store_sk#35] +Join type: Inner +Join condition: None + +(18) Project [codegen id : 10] +Output [10]: [d_week_seq#5, sun_sales#28, mon_sales#29, tue_sales#30, wed_sales#31, thu_sales#32, fri_sales#33, sat_sales#34, s_store_id#36, s_store_name#37] +Input [12]: [d_week_seq#5, ss_store_sk#1, sun_sales#28, mon_sales#29, tue_sales#30, wed_sales#31, thu_sales#32, fri_sales#33, sat_sales#34, s_store_sk#35, s_store_id#36, s_store_name#37] + +(19) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_month_seq#38, d_week_seq#39] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1212), LessThanOrEqual(d_month_seq,1223), IsNotNull(d_week_seq)] +ReadSchema: struct<d_month_seq:int,d_week_seq:int> + +(20) CometFilter +Input [2]: [d_month_seq#38, d_week_seq#39] +Condition : (((isnotnull(d_month_seq#38) AND (d_month_seq#38 >= 1212)) AND (d_month_seq#38 <= 1223)) AND isnotnull(d_week_seq#39)) + +(21) CometProject +Input [2]: [d_month_seq#38, d_week_seq#39] +Arguments: [d_week_seq#39], [d_week_seq#39] + +(22) ColumnarToRow [codegen id : 4] +Input [1]: [d_week_seq#39] + +(23) BroadcastExchange +Input [1]: [d_week_seq#39] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] + +(24) BroadcastHashJoin [codegen id : 10] +Left keys [1]: [d_week_seq#5] +Right keys [1]: [d_week_seq#39] +Join type: Inner +Join condition: None + +(25) Project [codegen id : 10] +Output [10]: [s_store_name#37 AS s_store_name1#40, d_week_seq#5 AS d_week_seq1#41, s_store_id#36 AS s_store_id1#42, sun_sales#28 AS sun_sales1#43, mon_sales#29 AS mon_sales1#44, tue_sales#30 AS tue_sales1#45, wed_sales#31 AS wed_sales1#46, thu_sales#32 AS thu_sales1#47, fri_sales#33 AS fri_sales1#48, sat_sales#34 AS sat_sales1#49] +Input [11]: [d_week_seq#5, sun_sales#28, mon_sales#29, tue_sales#30, wed_sales#31, thu_sales#32, fri_sales#33, sat_sales#34, s_store_id#36, s_store_name#37, d_week_seq#39] + +(26) ReusedExchange [Reuses operator id: 11] +Output [9]: [d_week_seq#50, ss_store_sk#51, sum#52, sum#53, sum#54, sum#55, sum#56, sum#57, sum#58] + +(27) HashAggregate [codegen id : 9] +Input [9]: [d_week_seq#50, ss_store_sk#51, sum#52, sum#53, sum#54, sum#55, sum#56, sum#57, sum#58] +Keys [2]: [d_week_seq#50, ss_store_sk#51] +Functions [7]: [sum(UnscaledValue(CASE WHEN (d_day_name#59 = Sunday ) THEN ss_sales_price#60 END)), sum(UnscaledValue(CASE WHEN (d_day_name#59 = Monday ) THEN ss_sales_price#60 END)), sum(UnscaledValue(CASE WHEN (d_day_name#59 = Tuesday ) THEN ss_sales_price#60 END)), sum(UnscaledValue(CASE WHEN (d_day_name#59 = Wednesday) THEN ss_sales_price#60 END)), sum(UnscaledValue(CASE WHEN (d_day_name#59 = Thursday ) THEN ss_sales_price#60 END)), sum(UnscaledValue(CASE WHEN (d_day_name#59 = Friday ) THEN ss_sales_price#60 END)), sum(UnscaledValue(CASE WHEN (d_day_name#59 = Saturday ) THEN ss_sales_price#60 END))] +Aggregate Attributes [7]: [sum(UnscaledValue(CASE WHEN (d_day_name#59 = Sunday ) THEN ss_sales_price#60 END))#21, sum(UnscaledValue(CASE WHEN (d_day_name#59 = Monday ) THEN ss_sales_price#60 END))#22, sum(UnscaledValue(CASE WHEN (d_day_name#59 = Tuesday ) THEN ss_sales_price#60 END))#23, sum(UnscaledValue(CASE WHEN (d_day_name#59 = Wednesday) THEN ss_sales_price#60 END))#24, sum(UnscaledValue(CASE WHEN (d_day_name#59 = Thursday ) THEN ss_sales_price#60 END))#25, sum(UnscaledValue(CASE WHEN (d_day_name#59 = Friday ) THEN ss_sales_price#60 END))#26, sum(UnscaledValue(CASE WHEN (d_day_name#59 = Saturday ) THEN ss_sales_price#60 END))#27] +Results [9]: [d_week_seq#50, ss_store_sk#51, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#59 = Sunday ) THEN ss_sales_price#60 END))#21,17,2) AS sun_sales#61, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#59 = Monday ) THEN ss_sales_price#60 END))#22,17,2) AS mon_sales#62, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#59 = Tuesday ) THEN ss_sales_price#60 END))#23,17,2) AS tue_sales#63, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#59 = Wednesday) THEN ss_sales_price#60 END))#24,17,2) AS wed_sales#64, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#59 = Thursday ) THEN ss_sales_price#60 END))#25,17,2) AS thu_sales#65, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#59 = Friday ) THEN ss_sales_price#60 END))#26,17,2) AS fri_sales#66, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#59 = Saturday ) THEN ss_sales_price#60 END))#27,17,2) AS sat_sales#67] + +(28) Scan parquet spark_catalog.default.store +Output [2]: [s_store_sk#68, s_store_id#69] +Batched: true +Location [not included in comparison]/{warehouse_dir}/store] +PushedFilters: [IsNotNull(s_store_sk), IsNotNull(s_store_id)] +ReadSchema: struct<s_store_sk:int,s_store_id:string> + +(29) CometFilter +Input [2]: [s_store_sk#68, s_store_id#69] +Condition : (isnotnull(s_store_sk#68) AND isnotnull(s_store_id#69)) + +(30) ColumnarToRow [codegen id : 7] +Input [2]: [s_store_sk#68, s_store_id#69] + +(31) BroadcastExchange +Input [2]: [s_store_sk#68, s_store_id#69] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=5] + +(32) BroadcastHashJoin [codegen id : 9] +Left keys [1]: [ss_store_sk#51] +Right keys [1]: [s_store_sk#68] +Join type: Inner +Join condition: None + +(33) Project [codegen id : 9] +Output [9]: [d_week_seq#50, sun_sales#61, mon_sales#62, tue_sales#63, wed_sales#64, thu_sales#65, fri_sales#66, sat_sales#67, s_store_id#69] +Input [11]: [d_week_seq#50, ss_store_sk#51, sun_sales#61, mon_sales#62, tue_sales#63, wed_sales#64, thu_sales#65, fri_sales#66, sat_sales#67, s_store_sk#68, s_store_id#69] + +(34) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_month_seq#70, d_week_seq#71] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1224), LessThanOrEqual(d_month_seq,1235), IsNotNull(d_week_seq)] +ReadSchema: struct<d_month_seq:int,d_week_seq:int> + +(35) CometFilter +Input [2]: [d_month_seq#70, d_week_seq#71] +Condition : (((isnotnull(d_month_seq#70) AND (d_month_seq#70 >= 1224)) AND (d_month_seq#70 <= 1235)) AND isnotnull(d_week_seq#71)) + +(36) CometProject +Input [2]: [d_month_seq#70, d_week_seq#71] +Arguments: [d_week_seq#71], [d_week_seq#71] + +(37) ColumnarToRow [codegen id : 8] +Input [1]: [d_week_seq#71] + +(38) BroadcastExchange +Input [1]: [d_week_seq#71] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=6] + +(39) BroadcastHashJoin [codegen id : 9] +Left keys [1]: [d_week_seq#50] +Right keys [1]: [d_week_seq#71] +Join type: Inner +Join condition: None + +(40) Project [codegen id : 9] +Output [9]: [d_week_seq#50 AS d_week_seq2#72, s_store_id#69 AS s_store_id2#73, sun_sales#61 AS sun_sales2#74, mon_sales#62 AS mon_sales2#75, tue_sales#63 AS tue_sales2#76, wed_sales#64 AS wed_sales2#77, thu_sales#65 AS thu_sales2#78, fri_sales#66 AS fri_sales2#79, sat_sales#67 AS sat_sales2#80] +Input [10]: [d_week_seq#50, sun_sales#61, mon_sales#62, tue_sales#63, wed_sales#64, thu_sales#65, fri_sales#66, sat_sales#67, s_store_id#69, d_week_seq#71] + +(41) BroadcastExchange +Input [9]: [d_week_seq2#72, s_store_id2#73, sun_sales2#74, mon_sales2#75, tue_sales2#76, wed_sales2#77, thu_sales2#78, fri_sales2#79, sat_sales2#80] +Arguments: HashedRelationBroadcastMode(List(input[1, string, true], (input[0, int, true] - 52)),false), [plan_id=7] + +(42) BroadcastHashJoin [codegen id : 10] +Left keys [2]: [s_store_id1#42, d_week_seq1#41] +Right keys [2]: [s_store_id2#73, (d_week_seq2#72 - 52)] +Join type: Inner +Join condition: None + +(43) Project [codegen id : 10] +Output [10]: [s_store_name1#40, s_store_id1#42, d_week_seq1#41, (sun_sales1#43 / sun_sales2#74) AS (sun_sales1 / sun_sales2)#81, (mon_sales1#44 / mon_sales2#75) AS (mon_sales1 / mon_sales2)#82, (tue_sales1#45 / tue_sales2#76) AS (tue_sales1 / tue_sales2)#83, (wed_sales1#46 / wed_sales2#77) AS (wed_sales1 / wed_sales2)#84, (thu_sales1#47 / thu_sales2#78) AS (thu_sales1 / thu_sales2)#85, (fri_sales1#48 / fri_sales2#79) AS (fri_sales1 / fri_sales2)#86, (sat_sales1#49 / sat_sales2#80) AS (sat_sales1 / sat_sales2)#87] +Input [19]: [s_store_name1#40, d_week_seq1#41, s_store_id1#42, sun_sales1#43, mon_sales1#44, tue_sales1#45, wed_sales1#46, thu_sales1#47, fri_sales1#48, sat_sales1#49, d_week_seq2#72, s_store_id2#73, sun_sales2#74, mon_sales2#75, tue_sales2#76, wed_sales2#77, thu_sales2#78, fri_sales2#79, sat_sales2#80] + +(44) TakeOrderedAndProject +Input [10]: [s_store_name1#40, s_store_id1#42, d_week_seq1#41, (sun_sales1 / sun_sales2)#81, (mon_sales1 / mon_sales2)#82, (tue_sales1 / tue_sales2)#83, (wed_sales1 / wed_sales2)#84, (thu_sales1 / thu_sales2)#85, (fri_sales1 / fri_sales2)#86, (sat_sales1 / sat_sales2)#87] +Arguments: 100, [s_store_name1#40 ASC NULLS FIRST, s_store_id1#42 ASC NULLS FIRST, d_week_seq1#41 ASC NULLS FIRST], [s_store_name1#40, s_store_id1#42, d_week_seq1#41, (sun_sales1 / sun_sales2)#81, (mon_sales1 / mon_sales2)#82, (tue_sales1 / tue_sales2)#83, (wed_sales1 / wed_sales2)#84, (thu_sales1 / thu_sales2)#85, (fri_sales1 / fri_sales2)#86, (sat_sales1 / sat_sales2)#87] + diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q59/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q59/simplified.txt new file mode 100644 index 000000000..9ad61e946 --- /dev/null +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q59/simplified.txt @@ -0,0 +1,66 @@ +TakeOrderedAndProject [s_store_name1,s_store_id1,d_week_seq1,(sun_sales1 / sun_sales2),(mon_sales1 / mon_sales2),(tue_sales1 / tue_sales2),(wed_sales1 / wed_sales2),(thu_sales1 / thu_sales2),(fri_sales1 / fri_sales2),(sat_sales1 / sat_sales2)] + WholeStageCodegen (10) + Project [s_store_name1,s_store_id1,d_week_seq1,sun_sales1,sun_sales2,mon_sales1,mon_sales2,tue_sales1,tue_sales2,wed_sales1,wed_sales2,thu_sales1,thu_sales2,fri_sales1,fri_sales2,sat_sales1,sat_sales2] + BroadcastHashJoin [s_store_id1,d_week_seq1,s_store_id2,d_week_seq2] + Project [s_store_name,d_week_seq,s_store_id,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales] + BroadcastHashJoin [d_week_seq,d_week_seq] + Project [d_week_seq,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales,s_store_id,s_store_name] + BroadcastHashJoin [ss_store_sk,s_store_sk] + HashAggregate [d_week_seq,ss_store_sk,sum,sum,sum,sum,sum,sum,sum] [sum(UnscaledValue(CASE WHEN (d_day_name = Sunday ) THEN ss_sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Monday ) THEN ss_sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Tuesday ) THEN ss_sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Wednesday) THEN ss_sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Thursday ) THEN ss_sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Friday ) THEN ss_sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Saturday ) THEN ss_sales_price END)),sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales,sum,sum,sum,sum,sum,sum,sum] + InputAdapter + Exchange [d_week_seq,ss_store_sk] #1 + WholeStageCodegen (2) + HashAggregate [d_week_seq,ss_store_sk,d_day_name,ss_sales_price] [sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum] + Project [ss_store_sk,ss_sales_price,d_week_seq,d_day_name] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + ColumnarToRow + InputAdapter + CometFilter [ss_store_sk] + CometScan parquet spark_catalog.default.store_sales [ss_store_sk,ss_sales_price,ss_sold_date_sk] + InputAdapter + BroadcastExchange #2 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometFilter [d_date_sk,d_week_seq] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq,d_day_name] + InputAdapter + BroadcastExchange #3 + WholeStageCodegen (3) + ColumnarToRow + InputAdapter + CometFilter [s_store_sk,s_store_id] + CometScan parquet spark_catalog.default.store [s_store_sk,s_store_id,s_store_name] + InputAdapter + BroadcastExchange #4 + WholeStageCodegen (4) + ColumnarToRow + InputAdapter + CometProject [d_week_seq] + CometFilter [d_month_seq,d_week_seq] + CometScan parquet spark_catalog.default.date_dim [d_month_seq,d_week_seq] + InputAdapter + BroadcastExchange #5 + WholeStageCodegen (9) + Project [d_week_seq,s_store_id,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales] + BroadcastHashJoin [d_week_seq,d_week_seq] + Project [d_week_seq,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales,s_store_id] + BroadcastHashJoin [ss_store_sk,s_store_sk] + HashAggregate [d_week_seq,ss_store_sk,sum,sum,sum,sum,sum,sum,sum] [sum(UnscaledValue(CASE WHEN (d_day_name = Sunday ) THEN ss_sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Monday ) THEN ss_sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Tuesday ) THEN ss_sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Wednesday) THEN ss_sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Thursday ) THEN ss_sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Friday ) THEN ss_sales_price END)),sum(UnscaledValue(CASE WHEN (d_day_name = Saturday ) THEN ss_sales_price END)),sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales,sum,sum,sum,sum,sum,sum,sum] + InputAdapter + ReusedExchange [d_week_seq,ss_store_sk,sum,sum,sum,sum,sum,sum,sum] #1 + InputAdapter + BroadcastExchange #6 + WholeStageCodegen (7) + ColumnarToRow + InputAdapter + CometFilter [s_store_sk,s_store_id] + CometScan parquet spark_catalog.default.store [s_store_sk,s_store_id] + InputAdapter + BroadcastExchange #7 + WholeStageCodegen (8) + ColumnarToRow + InputAdapter + CometProject [d_week_seq] + CometFilter [d_month_seq,d_week_seq] + CometScan parquet spark_catalog.default.date_dim [d_month_seq,d_week_seq] 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 new file mode 100644 index 000000000..f8ae10ebe --- /dev/null +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q6/explain.txt @@ -0,0 +1,309 @@ +== Physical Plan == +TakeOrderedAndProject (39) ++- * Filter (38) + +- * HashAggregate (37) + +- Exchange (36) + +- * HashAggregate (35) + +- * Project (34) + +- * BroadcastHashJoin Inner BuildRight (33) + :- * Project (18) + : +- * BroadcastHashJoin Inner BuildRight (17) + : :- * Project (15) + : : +- * BroadcastHashJoin Inner BuildRight (14) + : : :- * Project (9) + : : : +- * BroadcastHashJoin Inner BuildRight (8) + : : : :- * ColumnarToRow (3) + : : : : +- CometFilter (2) + : : : : +- CometScan parquet spark_catalog.default.customer_address (1) + : : : +- BroadcastExchange (7) + : : : +- * ColumnarToRow (6) + : : : +- CometFilter (5) + : : : +- CometScan parquet spark_catalog.default.customer (4) + : : +- BroadcastExchange (13) + : : +- * ColumnarToRow (12) + : : +- CometFilter (11) + : : +- CometScan parquet spark_catalog.default.store_sales (10) + : +- ReusedExchange (16) + +- BroadcastExchange (32) + +- * Project (31) + +- * BroadcastHashJoin Inner BuildRight (30) + :- * ColumnarToRow (21) + : +- CometFilter (20) + : +- CometScan parquet spark_catalog.default.item (19) + +- BroadcastExchange (29) + +- * Filter (28) + +- * HashAggregate (27) + +- Exchange (26) + +- * HashAggregate (25) + +- * ColumnarToRow (24) + +- CometFilter (23) + +- CometScan parquet spark_catalog.default.item (22) + + +(1) Scan parquet spark_catalog.default.customer_address +Output [2]: [ca_address_sk#1, ca_state#2] +Batched: true +Location [not included in comparison]/{warehouse_dir}/customer_address] +PushedFilters: [IsNotNull(ca_address_sk)] +ReadSchema: struct<ca_address_sk:int,ca_state:string> + +(2) CometFilter +Input [2]: [ca_address_sk#1, ca_state#2] +Condition : isnotnull(ca_address_sk#1) + +(3) ColumnarToRow [codegen id : 7] +Input [2]: [ca_address_sk#1, ca_state#2] + +(4) Scan parquet spark_catalog.default.customer +Output [2]: [c_customer_sk#3, c_current_addr_sk#4] +Batched: true +Location [not included in comparison]/{warehouse_dir}/customer] +PushedFilters: [IsNotNull(c_current_addr_sk), IsNotNull(c_customer_sk)] +ReadSchema: struct<c_customer_sk:int,c_current_addr_sk:int> + +(5) CometFilter +Input [2]: [c_customer_sk#3, c_current_addr_sk#4] +Condition : (isnotnull(c_current_addr_sk#4) AND isnotnull(c_customer_sk#3)) + +(6) ColumnarToRow [codegen id : 1] +Input [2]: [c_customer_sk#3, c_current_addr_sk#4] + +(7) BroadcastExchange +Input [2]: [c_customer_sk#3, c_current_addr_sk#4] +Arguments: HashedRelationBroadcastMode(List(cast(input[1, int, false] as bigint)),false), [plan_id=1] + +(8) BroadcastHashJoin [codegen id : 7] +Left keys [1]: [ca_address_sk#1] +Right keys [1]: [c_current_addr_sk#4] +Join type: Inner +Join condition: None + +(9) Project [codegen id : 7] +Output [2]: [ca_state#2, c_customer_sk#3] +Input [4]: [ca_address_sk#1, ca_state#2, c_customer_sk#3, c_current_addr_sk#4] + +(10) Scan parquet spark_catalog.default.store_sales +Output [3]: [ss_item_sk#5, ss_customer_sk#6, ss_sold_date_sk#7] +Batched: true +Location: InMemoryFileIndex [] +PartitionFilters: [isnotnull(ss_sold_date_sk#7), dynamicpruningexpression(ss_sold_date_sk#7 IN dynamicpruning#8)] +PushedFilters: [IsNotNull(ss_customer_sk), IsNotNull(ss_item_sk)] +ReadSchema: struct<ss_item_sk:int,ss_customer_sk:int> + +(11) CometFilter +Input [3]: [ss_item_sk#5, ss_customer_sk#6, ss_sold_date_sk#7] +Condition : (isnotnull(ss_customer_sk#6) AND isnotnull(ss_item_sk#5)) + +(12) ColumnarToRow [codegen id : 2] +Input [3]: [ss_item_sk#5, ss_customer_sk#6, ss_sold_date_sk#7] + +(13) BroadcastExchange +Input [3]: [ss_item_sk#5, ss_customer_sk#6, ss_sold_date_sk#7] +Arguments: HashedRelationBroadcastMode(List(cast(input[1, int, false] as bigint)),false), [plan_id=2] + +(14) BroadcastHashJoin [codegen id : 7] +Left keys [1]: [c_customer_sk#3] +Right keys [1]: [ss_customer_sk#6] +Join type: Inner +Join condition: None + +(15) Project [codegen id : 7] +Output [3]: [ca_state#2, ss_item_sk#5, ss_sold_date_sk#7] +Input [5]: [ca_state#2, c_customer_sk#3, ss_item_sk#5, ss_customer_sk#6, ss_sold_date_sk#7] + +(16) ReusedExchange [Reuses operator id: 44] +Output [1]: [d_date_sk#9] + +(17) BroadcastHashJoin [codegen id : 7] +Left keys [1]: [ss_sold_date_sk#7] +Right keys [1]: [d_date_sk#9] +Join type: Inner +Join condition: None + +(18) Project [codegen id : 7] +Output [2]: [ca_state#2, ss_item_sk#5] +Input [4]: [ca_state#2, ss_item_sk#5, ss_sold_date_sk#7, d_date_sk#9] + +(19) Scan parquet spark_catalog.default.item +Output [3]: [i_item_sk#10, i_current_price#11, i_category#12] +Batched: true +Location [not included in comparison]/{warehouse_dir}/item] +PushedFilters: [IsNotNull(i_current_price), IsNotNull(i_category), IsNotNull(i_item_sk)] +ReadSchema: struct<i_item_sk:int,i_current_price:decimal(7,2),i_category:string> + +(20) CometFilter +Input [3]: [i_item_sk#10, i_current_price#11, i_category#12] +Condition : ((isnotnull(i_current_price#11) AND isnotnull(i_category#12)) AND isnotnull(i_item_sk#10)) + +(21) ColumnarToRow [codegen id : 6] +Input [3]: [i_item_sk#10, i_current_price#11, i_category#12] + +(22) Scan parquet spark_catalog.default.item +Output [2]: [i_current_price#13, i_category#14] +Batched: true +Location [not included in comparison]/{warehouse_dir}/item] +PushedFilters: [IsNotNull(i_category)] +ReadSchema: struct<i_current_price:decimal(7,2),i_category:string> + +(23) CometFilter +Input [2]: [i_current_price#13, i_category#14] +Condition : isnotnull(i_category#14) + +(24) ColumnarToRow [codegen id : 4] +Input [2]: [i_current_price#13, i_category#14] + +(25) HashAggregate [codegen id : 4] +Input [2]: [i_current_price#13, i_category#14] +Keys [1]: [i_category#14] +Functions [1]: [partial_avg(UnscaledValue(i_current_price#13))] +Aggregate Attributes [2]: [sum#15, count#16] +Results [3]: [i_category#14, sum#17, count#18] + +(26) Exchange +Input [3]: [i_category#14, sum#17, count#18] +Arguments: hashpartitioning(i_category#14, 5), ENSURE_REQUIREMENTS, [plan_id=3] + +(27) HashAggregate [codegen id : 5] +Input [3]: [i_category#14, sum#17, count#18] +Keys [1]: [i_category#14] +Functions [1]: [avg(UnscaledValue(i_current_price#13))] +Aggregate Attributes [1]: [avg(UnscaledValue(i_current_price#13))#19] +Results [2]: [cast((avg(UnscaledValue(i_current_price#13))#19 / 100.0) as decimal(11,6)) AS avg(i_current_price)#20, i_category#14] + +(28) Filter [codegen id : 5] +Input [2]: [avg(i_current_price)#20, i_category#14] +Condition : isnotnull(avg(i_current_price)#20) + +(29) BroadcastExchange +Input [2]: [avg(i_current_price)#20, i_category#14] +Arguments: HashedRelationBroadcastMode(List(input[1, string, true]),false), [plan_id=4] + +(30) BroadcastHashJoin [codegen id : 6] +Left keys [1]: [i_category#12] +Right keys [1]: [i_category#14] +Join type: Inner +Join condition: (cast(i_current_price#11 as decimal(14,7)) > (1.2 * avg(i_current_price)#20)) + +(31) Project [codegen id : 6] +Output [1]: [i_item_sk#10] +Input [5]: [i_item_sk#10, i_current_price#11, i_category#12, avg(i_current_price)#20, i_category#14] + +(32) BroadcastExchange +Input [1]: [i_item_sk#10] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] + +(33) BroadcastHashJoin [codegen id : 7] +Left keys [1]: [ss_item_sk#5] +Right keys [1]: [i_item_sk#10] +Join type: Inner +Join condition: None + +(34) Project [codegen id : 7] +Output [1]: [ca_state#2] +Input [3]: [ca_state#2, ss_item_sk#5, i_item_sk#10] + +(35) HashAggregate [codegen id : 7] +Input [1]: [ca_state#2] +Keys [1]: [ca_state#2] +Functions [1]: [partial_count(1)] +Aggregate Attributes [1]: [count#21] +Results [2]: [ca_state#2, count#22] + +(36) Exchange +Input [2]: [ca_state#2, count#22] +Arguments: hashpartitioning(ca_state#2, 5), ENSURE_REQUIREMENTS, [plan_id=6] + +(37) HashAggregate [codegen id : 8] +Input [2]: [ca_state#2, count#22] +Keys [1]: [ca_state#2] +Functions [1]: [count(1)] +Aggregate Attributes [1]: [count(1)#23] +Results [2]: [ca_state#2 AS state#24, count(1)#23 AS cnt#25] + +(38) Filter [codegen id : 8] +Input [2]: [state#24, cnt#25] +Condition : (cnt#25 >= 10) + +(39) TakeOrderedAndProject +Input [2]: [state#24, cnt#25] +Arguments: 100, [cnt#25 ASC NULLS FIRST], [state#24, cnt#25] + +===== Subqueries ===== + +Subquery:1 Hosting operator id = 10 Hosting Expression = ss_sold_date_sk#7 IN dynamicpruning#8 +BroadcastExchange (44) ++- * ColumnarToRow (43) + +- CometProject (42) + +- CometFilter (41) + +- CometScan parquet spark_catalog.default.date_dim (40) + + +(40) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#9, d_month_seq#26] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_month_seq), EqualTo(d_month_seq,ScalarSubquery#27), IsNotNull(d_date_sk)] +ReadSchema: struct<d_date_sk:int,d_month_seq:int> + +(41) CometFilter +Input [2]: [d_date_sk#9, d_month_seq#26] +Condition : ((isnotnull(d_month_seq#26) AND (d_month_seq#26 = ReusedSubquery Subquery scalar-subquery#27, [id=#28])) AND isnotnull(d_date_sk#9)) + +(42) CometProject +Input [2]: [d_date_sk#9, d_month_seq#26] +Arguments: [d_date_sk#9], [d_date_sk#9] + +(43) ColumnarToRow [codegen id : 1] +Input [1]: [d_date_sk#9] + +(44) BroadcastExchange +Input [1]: [d_date_sk#9] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=7] + +Subquery:2 Hosting operator id = 41 Hosting Expression = ReusedSubquery Subquery scalar-subquery#27, [id=#28] + +Subquery:3 Hosting operator id = 40 Hosting Expression = Subquery scalar-subquery#27, [id=#28] +* HashAggregate (51) ++- Exchange (50) + +- * ColumnarToRow (49) + +- CometHashAggregate (48) + +- CometProject (47) + +- CometFilter (46) + +- CometScan parquet spark_catalog.default.date_dim (45) + + +(45) Scan parquet spark_catalog.default.date_dim +Output [3]: [d_month_seq#29, d_year#30, d_moy#31] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2000), EqualTo(d_moy,1)] +ReadSchema: struct<d_month_seq:int,d_year:int,d_moy:int> + +(46) CometFilter +Input [3]: [d_month_seq#29, d_year#30, d_moy#31] +Condition : (((isnotnull(d_year#30) AND isnotnull(d_moy#31)) AND (d_year#30 = 2000)) AND (d_moy#31 = 1)) + +(47) CometProject +Input [3]: [d_month_seq#29, d_year#30, d_moy#31] +Arguments: [d_month_seq#29], [d_month_seq#29] + +(48) CometHashAggregate +Input [1]: [d_month_seq#29] +Keys [1]: [d_month_seq#29] +Functions: [] + +(49) ColumnarToRow [codegen id : 1] +Input [1]: [d_month_seq#29] + +(50) Exchange +Input [1]: [d_month_seq#29] +Arguments: hashpartitioning(d_month_seq#29, 5), ENSURE_REQUIREMENTS, [plan_id=8] + +(51) HashAggregate [codegen id : 2] +Input [1]: [d_month_seq#29] +Keys [1]: [d_month_seq#29] +Functions: [] +Aggregate Attributes: [] +Results [1]: [d_month_seq#29] + + 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 new file mode 100644 index 000000000..c2d5a6ce8 --- /dev/null +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q6/simplified.txt @@ -0,0 +1,79 @@ +TakeOrderedAndProject [cnt,state] + WholeStageCodegen (8) + Filter [cnt] + HashAggregate [ca_state,count] [count(1),state,cnt,count] + InputAdapter + Exchange [ca_state] #1 + WholeStageCodegen (7) + HashAggregate [ca_state] [count,count] + Project [ca_state] + BroadcastHashJoin [ss_item_sk,i_item_sk] + Project [ca_state,ss_item_sk] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [ca_state,ss_item_sk,ss_sold_date_sk] + BroadcastHashJoin [c_customer_sk,ss_customer_sk] + Project [ca_state,c_customer_sk] + BroadcastHashJoin [ca_address_sk,c_current_addr_sk] + ColumnarToRow + InputAdapter + CometFilter [ca_address_sk] + CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] + InputAdapter + BroadcastExchange #2 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometFilter [c_current_addr_sk,c_customer_sk] + CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk] + InputAdapter + BroadcastExchange #3 + WholeStageCodegen (2) + ColumnarToRow + InputAdapter + CometFilter [ss_customer_sk,ss_item_sk] + CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #4 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_month_seq,d_date_sk] + ReusedSubquery [d_month_seq] #2 + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] + Subquery #2 + WholeStageCodegen (2) + HashAggregate [d_month_seq] + InputAdapter + Exchange [d_month_seq] #5 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometHashAggregate [d_month_seq] + CometProject [d_month_seq] + CometFilter [d_year,d_moy] + CometScan parquet spark_catalog.default.date_dim [d_month_seq,d_year,d_moy] + InputAdapter + ReusedExchange [d_date_sk] #4 + InputAdapter + BroadcastExchange #6 + WholeStageCodegen (6) + Project [i_item_sk] + BroadcastHashJoin [i_category,i_category,i_current_price,avg(i_current_price)] + ColumnarToRow + InputAdapter + CometFilter [i_current_price,i_category,i_item_sk] + CometScan parquet spark_catalog.default.item [i_item_sk,i_current_price,i_category] + InputAdapter + BroadcastExchange #7 + WholeStageCodegen (5) + Filter [avg(i_current_price)] + HashAggregate [i_category,sum,count] [avg(UnscaledValue(i_current_price)),avg(i_current_price),sum,count] + InputAdapter + Exchange [i_category] #8 + WholeStageCodegen (4) + HashAggregate [i_category,i_current_price] [sum,count,sum,count] + ColumnarToRow + InputAdapter + CometFilter [i_category] + CometScan parquet spark_catalog.default.item [i_current_price,i_category] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q60/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q60/explain.txt new file mode 100644 index 000000000..78f4b27ac --- /dev/null +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q60/explain.txt @@ -0,0 +1,405 @@ +== Physical Plan == +TakeOrderedAndProject (63) ++- * HashAggregate (62) + +- Exchange (61) + +- * HashAggregate (60) + +- Union (59) + :- * HashAggregate (28) + : +- Exchange (27) + : +- * HashAggregate (26) + : +- * Project (25) + : +- * BroadcastHashJoin Inner BuildRight (24) + : :- * Project (13) + : : +- * BroadcastHashJoin Inner BuildRight (12) + : : :- * Project (6) + : : : +- * BroadcastHashJoin Inner BuildRight (5) + : : : :- * ColumnarToRow (3) + : : : : +- CometFilter (2) + : : : : +- CometScan parquet spark_catalog.default.store_sales (1) + : : : +- ReusedExchange (4) + : : +- BroadcastExchange (11) + : : +- * ColumnarToRow (10) + : : +- CometProject (9) + : : +- CometFilter (8) + : : +- CometScan parquet spark_catalog.default.customer_address (7) + : +- BroadcastExchange (23) + : +- * BroadcastHashJoin LeftSemi BuildRight (22) + : :- * ColumnarToRow (16) + : : +- CometFilter (15) + : : +- CometScan parquet spark_catalog.default.item (14) + : +- BroadcastExchange (21) + : +- * ColumnarToRow (20) + : +- CometProject (19) + : +- CometFilter (18) + : +- CometScan parquet spark_catalog.default.item (17) + :- * HashAggregate (43) + : +- Exchange (42) + : +- * HashAggregate (41) + : +- * Project (40) + : +- * BroadcastHashJoin Inner BuildRight (39) + : :- * Project (37) + : : +- * BroadcastHashJoin Inner BuildRight (36) + : : :- * Project (34) + : : : +- * BroadcastHashJoin Inner BuildRight (33) + : : : :- * ColumnarToRow (31) + : : : : +- CometFilter (30) + : : : : +- CometScan parquet spark_catalog.default.catalog_sales (29) + : : : +- ReusedExchange (32) + : : +- ReusedExchange (35) + : +- ReusedExchange (38) + +- * HashAggregate (58) + +- Exchange (57) + +- * HashAggregate (56) + +- * Project (55) + +- * BroadcastHashJoin Inner BuildRight (54) + :- * Project (52) + : +- * BroadcastHashJoin Inner BuildRight (51) + : :- * Project (49) + : : +- * BroadcastHashJoin Inner BuildRight (48) + : : :- * ColumnarToRow (46) + : : : +- CometFilter (45) + : : : +- CometScan parquet spark_catalog.default.web_sales (44) + : : +- ReusedExchange (47) + : +- ReusedExchange (50) + +- ReusedExchange (53) + + +(1) Scan parquet spark_catalog.default.store_sales +Output [4]: [ss_item_sk#1, ss_addr_sk#2, ss_ext_sales_price#3, ss_sold_date_sk#4] +Batched: true +Location: InMemoryFileIndex [] +PartitionFilters: [isnotnull(ss_sold_date_sk#4), dynamicpruningexpression(ss_sold_date_sk#4 IN dynamicpruning#5)] +PushedFilters: [IsNotNull(ss_addr_sk), IsNotNull(ss_item_sk)] +ReadSchema: struct<ss_item_sk:int,ss_addr_sk:int,ss_ext_sales_price:decimal(7,2)> + +(2) CometFilter +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)) + +(3) ColumnarToRow [codegen id : 5] +Input [4]: [ss_item_sk#1, ss_addr_sk#2, ss_ext_sales_price#3, ss_sold_date_sk#4] + +(4) ReusedExchange [Reuses operator id: 68] +Output [1]: [d_date_sk#6] + +(5) BroadcastHashJoin [codegen id : 5] +Left keys [1]: [ss_sold_date_sk#4] +Right keys [1]: [d_date_sk#6] +Join type: Inner +Join condition: None + +(6) Project [codegen id : 5] +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] + +(7) Scan parquet spark_catalog.default.customer_address +Output [2]: [ca_address_sk#7, ca_gmt_offset#8] +Batched: true +Location [not included in comparison]/{warehouse_dir}/customer_address] +PushedFilters: [IsNotNull(ca_gmt_offset), EqualTo(ca_gmt_offset,-5.00), IsNotNull(ca_address_sk)] +ReadSchema: struct<ca_address_sk:int,ca_gmt_offset:decimal(5,2)> + +(8) CometFilter +Input [2]: [ca_address_sk#7, ca_gmt_offset#8] +Condition : ((isnotnull(ca_gmt_offset#8) AND (ca_gmt_offset#8 = -5.00)) AND isnotnull(ca_address_sk#7)) + +(9) CometProject +Input [2]: [ca_address_sk#7, ca_gmt_offset#8] +Arguments: [ca_address_sk#7], [ca_address_sk#7] + +(10) ColumnarToRow [codegen id : 2] +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 : 5] +Left keys [1]: [ss_addr_sk#2] +Right keys [1]: [ca_address_sk#7] +Join type: Inner +Join condition: None + +(13) Project [codegen id : 5] +Output [2]: [ss_item_sk#1, ss_ext_sales_price#3] +Input [4]: [ss_item_sk#1, ss_addr_sk#2, ss_ext_sales_price#3, ca_address_sk#7] + +(14) Scan parquet spark_catalog.default.item +Output [2]: [i_item_sk#9, i_item_id#10] +Batched: true +Location [not included in comparison]/{warehouse_dir}/item] +PushedFilters: [IsNotNull(i_item_sk)] +ReadSchema: struct<i_item_sk:int,i_item_id:string> + +(15) CometFilter +Input [2]: [i_item_sk#9, i_item_id#10] +Condition : isnotnull(i_item_sk#9) + +(16) ColumnarToRow [codegen id : 4] +Input [2]: [i_item_sk#9, i_item_id#10] + +(17) Scan parquet spark_catalog.default.item +Output [2]: [i_item_id#11, i_category#12] +Batched: true +Location [not included in comparison]/{warehouse_dir}/item] +PushedFilters: [IsNotNull(i_category), EqualTo(i_category,Music )] +ReadSchema: struct<i_item_id:string,i_category:string> + +(18) CometFilter +Input [2]: [i_item_id#11, i_category#12] +Condition : (isnotnull(i_category#12) AND (i_category#12 = Music )) + +(19) CometProject +Input [2]: [i_item_id#11, i_category#12] +Arguments: [i_item_id#11], [i_item_id#11] + +(20) ColumnarToRow [codegen id : 3] +Input [1]: [i_item_id#11] + +(21) BroadcastExchange +Input [1]: [i_item_id#11] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=2] + +(22) BroadcastHashJoin [codegen id : 4] +Left keys [1]: [i_item_id#10] +Right keys [1]: [i_item_id#11] +Join type: LeftSemi +Join condition: None + +(23) BroadcastExchange +Input [2]: [i_item_sk#9, i_item_id#10] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=3] + +(24) BroadcastHashJoin [codegen id : 5] +Left keys [1]: [ss_item_sk#1] +Right keys [1]: [i_item_sk#9] +Join type: Inner +Join condition: None + +(25) Project [codegen id : 5] +Output [2]: [ss_ext_sales_price#3, i_item_id#10] +Input [4]: [ss_item_sk#1, ss_ext_sales_price#3, i_item_sk#9, i_item_id#10] + +(26) HashAggregate [codegen id : 5] +Input [2]: [ss_ext_sales_price#3, i_item_id#10] +Keys [1]: [i_item_id#10] +Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#3))] +Aggregate Attributes [1]: [sum#13] +Results [2]: [i_item_id#10, sum#14] + +(27) Exchange +Input [2]: [i_item_id#10, sum#14] +Arguments: hashpartitioning(i_item_id#10, 5), ENSURE_REQUIREMENTS, [plan_id=4] + +(28) HashAggregate [codegen id : 6] +Input [2]: [i_item_id#10, sum#14] +Keys [1]: [i_item_id#10] +Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#3))] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#3))#15] +Results [2]: [i_item_id#10, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#3))#15,17,2) AS total_sales#16] + +(29) Scan parquet spark_catalog.default.catalog_sales +Output [4]: [cs_bill_addr_sk#17, cs_item_sk#18, cs_ext_sales_price#19, cs_sold_date_sk#20] +Batched: true +Location: InMemoryFileIndex [] +PartitionFilters: [isnotnull(cs_sold_date_sk#20), dynamicpruningexpression(cs_sold_date_sk#20 IN dynamicpruning#21)] +PushedFilters: [IsNotNull(cs_bill_addr_sk), IsNotNull(cs_item_sk)] +ReadSchema: struct<cs_bill_addr_sk:int,cs_item_sk:int,cs_ext_sales_price:decimal(7,2)> + +(30) CometFilter +Input [4]: [cs_bill_addr_sk#17, cs_item_sk#18, cs_ext_sales_price#19, cs_sold_date_sk#20] +Condition : (isnotnull(cs_bill_addr_sk#17) AND isnotnull(cs_item_sk#18)) + +(31) ColumnarToRow [codegen id : 11] +Input [4]: [cs_bill_addr_sk#17, cs_item_sk#18, cs_ext_sales_price#19, cs_sold_date_sk#20] + +(32) ReusedExchange [Reuses operator id: 68] +Output [1]: [d_date_sk#22] + +(33) BroadcastHashJoin [codegen id : 11] +Left keys [1]: [cs_sold_date_sk#20] +Right keys [1]: [d_date_sk#22] +Join type: Inner +Join condition: None + +(34) Project [codegen id : 11] +Output [3]: [cs_bill_addr_sk#17, cs_item_sk#18, cs_ext_sales_price#19] +Input [5]: [cs_bill_addr_sk#17, cs_item_sk#18, cs_ext_sales_price#19, cs_sold_date_sk#20, d_date_sk#22] + +(35) ReusedExchange [Reuses operator id: 11] +Output [1]: [ca_address_sk#23] + +(36) BroadcastHashJoin [codegen id : 11] +Left keys [1]: [cs_bill_addr_sk#17] +Right keys [1]: [ca_address_sk#23] +Join type: Inner +Join condition: None + +(37) Project [codegen id : 11] +Output [2]: [cs_item_sk#18, cs_ext_sales_price#19] +Input [4]: [cs_bill_addr_sk#17, cs_item_sk#18, cs_ext_sales_price#19, ca_address_sk#23] + +(38) ReusedExchange [Reuses operator id: 23] +Output [2]: [i_item_sk#24, i_item_id#25] + +(39) BroadcastHashJoin [codegen id : 11] +Left keys [1]: [cs_item_sk#18] +Right keys [1]: [i_item_sk#24] +Join type: Inner +Join condition: None + +(40) Project [codegen id : 11] +Output [2]: [cs_ext_sales_price#19, i_item_id#25] +Input [4]: [cs_item_sk#18, cs_ext_sales_price#19, i_item_sk#24, i_item_id#25] + +(41) HashAggregate [codegen id : 11] +Input [2]: [cs_ext_sales_price#19, i_item_id#25] +Keys [1]: [i_item_id#25] +Functions [1]: [partial_sum(UnscaledValue(cs_ext_sales_price#19))] +Aggregate Attributes [1]: [sum#26] +Results [2]: [i_item_id#25, sum#27] + +(42) Exchange +Input [2]: [i_item_id#25, sum#27] +Arguments: hashpartitioning(i_item_id#25, 5), ENSURE_REQUIREMENTS, [plan_id=5] + +(43) HashAggregate [codegen id : 12] +Input [2]: [i_item_id#25, sum#27] +Keys [1]: [i_item_id#25] +Functions [1]: [sum(UnscaledValue(cs_ext_sales_price#19))] +Aggregate Attributes [1]: [sum(UnscaledValue(cs_ext_sales_price#19))#28] +Results [2]: [i_item_id#25, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#19))#28,17,2) AS total_sales#29] + +(44) Scan parquet spark_catalog.default.web_sales +Output [4]: [ws_item_sk#30, ws_bill_addr_sk#31, ws_ext_sales_price#32, ws_sold_date_sk#33] +Batched: true +Location: InMemoryFileIndex [] +PartitionFilters: [isnotnull(ws_sold_date_sk#33), dynamicpruningexpression(ws_sold_date_sk#33 IN dynamicpruning#34)] +PushedFilters: [IsNotNull(ws_bill_addr_sk), IsNotNull(ws_item_sk)] +ReadSchema: struct<ws_item_sk:int,ws_bill_addr_sk:int,ws_ext_sales_price:decimal(7,2)> + +(45) CometFilter +Input [4]: [ws_item_sk#30, ws_bill_addr_sk#31, ws_ext_sales_price#32, ws_sold_date_sk#33] +Condition : (isnotnull(ws_bill_addr_sk#31) AND isnotnull(ws_item_sk#30)) + +(46) ColumnarToRow [codegen id : 17] +Input [4]: [ws_item_sk#30, ws_bill_addr_sk#31, ws_ext_sales_price#32, ws_sold_date_sk#33] + +(47) ReusedExchange [Reuses operator id: 68] +Output [1]: [d_date_sk#35] + +(48) BroadcastHashJoin [codegen id : 17] +Left keys [1]: [ws_sold_date_sk#33] +Right keys [1]: [d_date_sk#35] +Join type: Inner +Join condition: None + +(49) Project [codegen id : 17] +Output [3]: [ws_item_sk#30, ws_bill_addr_sk#31, ws_ext_sales_price#32] +Input [5]: [ws_item_sk#30, ws_bill_addr_sk#31, ws_ext_sales_price#32, ws_sold_date_sk#33, d_date_sk#35] + +(50) ReusedExchange [Reuses operator id: 11] +Output [1]: [ca_address_sk#36] + +(51) BroadcastHashJoin [codegen id : 17] +Left keys [1]: [ws_bill_addr_sk#31] +Right keys [1]: [ca_address_sk#36] +Join type: Inner +Join condition: None + +(52) Project [codegen id : 17] +Output [2]: [ws_item_sk#30, ws_ext_sales_price#32] +Input [4]: [ws_item_sk#30, ws_bill_addr_sk#31, ws_ext_sales_price#32, ca_address_sk#36] + +(53) ReusedExchange [Reuses operator id: 23] +Output [2]: [i_item_sk#37, i_item_id#38] + +(54) BroadcastHashJoin [codegen id : 17] +Left keys [1]: [ws_item_sk#30] +Right keys [1]: [i_item_sk#37] +Join type: Inner +Join condition: None + +(55) Project [codegen id : 17] +Output [2]: [ws_ext_sales_price#32, i_item_id#38] +Input [4]: [ws_item_sk#30, ws_ext_sales_price#32, i_item_sk#37, i_item_id#38] + +(56) HashAggregate [codegen id : 17] +Input [2]: [ws_ext_sales_price#32, i_item_id#38] +Keys [1]: [i_item_id#38] +Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#32))] +Aggregate Attributes [1]: [sum#39] +Results [2]: [i_item_id#38, sum#40] + +(57) Exchange +Input [2]: [i_item_id#38, sum#40] +Arguments: hashpartitioning(i_item_id#38, 5), ENSURE_REQUIREMENTS, [plan_id=6] + +(58) HashAggregate [codegen id : 18] +Input [2]: [i_item_id#38, sum#40] +Keys [1]: [i_item_id#38] +Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#32))] +Aggregate Attributes [1]: [sum(UnscaledValue(ws_ext_sales_price#32))#41] +Results [2]: [i_item_id#38, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#32))#41,17,2) AS total_sales#42] + +(59) Union + +(60) HashAggregate [codegen id : 19] +Input [2]: [i_item_id#10, total_sales#16] +Keys [1]: [i_item_id#10] +Functions [1]: [partial_sum(total_sales#16)] +Aggregate Attributes [2]: [sum#43, isEmpty#44] +Results [3]: [i_item_id#10, sum#45, isEmpty#46] + +(61) Exchange +Input [3]: [i_item_id#10, sum#45, isEmpty#46] +Arguments: hashpartitioning(i_item_id#10, 5), ENSURE_REQUIREMENTS, [plan_id=7] + +(62) HashAggregate [codegen id : 20] +Input [3]: [i_item_id#10, sum#45, isEmpty#46] +Keys [1]: [i_item_id#10] +Functions [1]: [sum(total_sales#16)] +Aggregate Attributes [1]: [sum(total_sales#16)#47] +Results [2]: [i_item_id#10, sum(total_sales#16)#47 AS total_sales#48] + +(63) TakeOrderedAndProject +Input [2]: [i_item_id#10, total_sales#48] +Arguments: 100, [i_item_id#10 ASC NULLS FIRST, total_sales#48 ASC NULLS FIRST], [i_item_id#10, total_sales#48] + +===== Subqueries ===== + +Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 +BroadcastExchange (68) ++- * ColumnarToRow (67) + +- CometProject (66) + +- CometFilter (65) + +- CometScan parquet spark_catalog.default.date_dim (64) + + +(64) Scan parquet spark_catalog.default.date_dim +Output [3]: [d_date_sk#6, d_year#49, d_moy#50] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,1998), EqualTo(d_moy,9), IsNotNull(d_date_sk)] +ReadSchema: struct<d_date_sk:int,d_year:int,d_moy:int> + +(65) CometFilter +Input [3]: [d_date_sk#6, d_year#49, d_moy#50] +Condition : ((((isnotnull(d_year#49) AND isnotnull(d_moy#50)) AND (d_year#49 = 1998)) AND (d_moy#50 = 9)) AND isnotnull(d_date_sk#6)) + +(66) CometProject +Input [3]: [d_date_sk#6, d_year#49, d_moy#50] +Arguments: [d_date_sk#6], [d_date_sk#6] + +(67) ColumnarToRow [codegen id : 1] +Input [1]: [d_date_sk#6] + +(68) BroadcastExchange +Input [1]: [d_date_sk#6] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=8] + +Subquery:2 Hosting operator id = 29 Hosting Expression = cs_sold_date_sk#20 IN dynamicpruning#5 + +Subquery:3 Hosting operator id = 44 Hosting Expression = ws_sold_date_sk#33 IN dynamicpruning#5 + + diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q60/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q60/simplified.txt new file mode 100644 index 000000000..b010414a8 --- /dev/null +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q60/simplified.txt @@ -0,0 +1,105 @@ +TakeOrderedAndProject [i_item_id,total_sales] + WholeStageCodegen (20) + HashAggregate [i_item_id,sum,isEmpty] [sum(total_sales),total_sales,sum,isEmpty] + InputAdapter + Exchange [i_item_id] #1 + WholeStageCodegen (19) + HashAggregate [i_item_id,total_sales] [sum,isEmpty,sum,isEmpty] + InputAdapter + Union + WholeStageCodegen (6) + HashAggregate [i_item_id,sum] [sum(UnscaledValue(ss_ext_sales_price)),total_sales,sum] + InputAdapter + Exchange [i_item_id] #2 + WholeStageCodegen (5) + HashAggregate [i_item_id,ss_ext_sales_price] [sum,sum] + Project [ss_ext_sales_price,i_item_id] + BroadcastHashJoin [ss_item_sk,i_item_sk] + Project [ss_item_sk,ss_ext_sales_price] + BroadcastHashJoin [ss_addr_sk,ca_address_sk] + Project [ss_item_sk,ss_addr_sk,ss_ext_sales_price] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + ColumnarToRow + InputAdapter + CometFilter [ss_addr_sk,ss_item_sk] + CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_addr_sk,ss_ext_sales_price,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_year,d_moy,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + InputAdapter + ReusedExchange [d_date_sk] #3 + InputAdapter + BroadcastExchange #4 + WholeStageCodegen (2) + ColumnarToRow + InputAdapter + CometProject [ca_address_sk] + CometFilter [ca_gmt_offset,ca_address_sk] + CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_gmt_offset] + InputAdapter + BroadcastExchange #5 + WholeStageCodegen (4) + BroadcastHashJoin [i_item_id,i_item_id] + ColumnarToRow + InputAdapter + CometFilter [i_item_sk] + CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id] + InputAdapter + BroadcastExchange #6 + WholeStageCodegen (3) + ColumnarToRow + InputAdapter + CometProject [i_item_id] + CometFilter [i_category] + CometScan parquet spark_catalog.default.item [i_item_id,i_category] + WholeStageCodegen (12) + HashAggregate [i_item_id,sum] [sum(UnscaledValue(cs_ext_sales_price)),total_sales,sum] + InputAdapter + Exchange [i_item_id] #7 + WholeStageCodegen (11) + HashAggregate [i_item_id,cs_ext_sales_price] [sum,sum] + Project [cs_ext_sales_price,i_item_id] + BroadcastHashJoin [cs_item_sk,i_item_sk] + Project [cs_item_sk,cs_ext_sales_price] + BroadcastHashJoin [cs_bill_addr_sk,ca_address_sk] + Project [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + ColumnarToRow + InputAdapter + CometFilter [cs_bill_addr_sk,cs_item_sk] + CometScan parquet spark_catalog.default.catalog_sales [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + InputAdapter + ReusedExchange [d_date_sk] #3 + InputAdapter + ReusedExchange [ca_address_sk] #4 + InputAdapter + ReusedExchange [i_item_sk,i_item_id] #5 + WholeStageCodegen (18) + HashAggregate [i_item_id,sum] [sum(UnscaledValue(ws_ext_sales_price)),total_sales,sum] + InputAdapter + Exchange [i_item_id] #8 + WholeStageCodegen (17) + HashAggregate [i_item_id,ws_ext_sales_price] [sum,sum] + Project [ws_ext_sales_price,i_item_id] + BroadcastHashJoin [ws_item_sk,i_item_sk] + Project [ws_item_sk,ws_ext_sales_price] + BroadcastHashJoin [ws_bill_addr_sk,ca_address_sk] + Project [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + ColumnarToRow + InputAdapter + CometFilter [ws_bill_addr_sk,ws_item_sk] + CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + InputAdapter + ReusedExchange [d_date_sk] #3 + InputAdapter + ReusedExchange [ca_address_sk] #4 + InputAdapter + ReusedExchange [i_item_sk,i_item_id] #5 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 new file mode 100644 index 000000000..766362167 --- /dev/null +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q61/explain.txt @@ -0,0 +1,417 @@ +== Physical Plan == +* Project (67) ++- * BroadcastNestedLoopJoin Inner BuildRight (66) + :- * HashAggregate (43) + : +- Exchange (42) + : +- * HashAggregate (41) + : +- * Project (40) + : +- * BroadcastHashJoin Inner BuildRight (39) + : :- * Project (33) + : : +- * BroadcastHashJoin Inner BuildRight (32) + : : :- * Project (26) + : : : +- * BroadcastHashJoin Inner BuildRight (25) + : : : :- * Project (20) + : : : : +- * BroadcastHashJoin Inner BuildRight (19) + : : : : :- * Project (17) + : : : : : +- * BroadcastHashJoin Inner BuildRight (16) + : : : : : :- * Project (10) + : : : : : : +- * BroadcastHashJoin Inner BuildRight (9) + : : : : : : :- * ColumnarToRow (3) + : : : : : : : +- CometFilter (2) + : : : : : : : +- CometScan parquet spark_catalog.default.store_sales (1) + : : : : : : +- BroadcastExchange (8) + : : : : : : +- * ColumnarToRow (7) + : : : : : : +- CometProject (6) + : : : : : : +- CometFilter (5) + : : : : : : +- CometScan parquet spark_catalog.default.store (4) + : : : : : +- BroadcastExchange (15) + : : : : : +- * ColumnarToRow (14) + : : : : : +- CometProject (13) + : : : : : +- CometFilter (12) + : : : : : +- CometScan parquet spark_catalog.default.promotion (11) + : : : : +- ReusedExchange (18) + : : : +- BroadcastExchange (24) + : : : +- * ColumnarToRow (23) + : : : +- CometFilter (22) + : : : +- CometScan parquet spark_catalog.default.customer (21) + : : +- BroadcastExchange (31) + : : +- * ColumnarToRow (30) + : : +- CometProject (29) + : : +- CometFilter (28) + : : +- CometScan parquet spark_catalog.default.customer_address (27) + : +- BroadcastExchange (38) + : +- * ColumnarToRow (37) + : +- CometProject (36) + : +- CometFilter (35) + : +- CometScan parquet spark_catalog.default.item (34) + +- BroadcastExchange (65) + +- * HashAggregate (64) + +- Exchange (63) + +- * HashAggregate (62) + +- * Project (61) + +- * BroadcastHashJoin Inner BuildRight (60) + :- * Project (58) + : +- * BroadcastHashJoin Inner BuildRight (57) + : :- * Project (55) + : : +- * BroadcastHashJoin Inner BuildRight (54) + : : :- * Project (52) + : : : +- * BroadcastHashJoin Inner BuildRight (51) + : : : :- * Project (49) + : : : : +- * BroadcastHashJoin Inner BuildRight (48) + : : : : :- * ColumnarToRow (46) + : : : : : +- CometFilter (45) + : : : : : +- CometScan parquet spark_catalog.default.store_sales (44) + : : : : +- ReusedExchange (47) + : : : +- ReusedExchange (50) + : : +- ReusedExchange (53) + : +- ReusedExchange (56) + +- ReusedExchange (59) + + +(1) Scan parquet spark_catalog.default.store_sales +Output [6]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_promo_sk#4, ss_ext_sales_price#5, ss_sold_date_sk#6] +Batched: true +Location: InMemoryFileIndex [] +PartitionFilters: [isnotnull(ss_sold_date_sk#6), dynamicpruningexpression(ss_sold_date_sk#6 IN dynamicpruning#7)] +PushedFilters: [IsNotNull(ss_store_sk), IsNotNull(ss_promo_sk), IsNotNull(ss_customer_sk), IsNotNull(ss_item_sk)] +ReadSchema: struct<ss_item_sk:int,ss_customer_sk:int,ss_store_sk:int,ss_promo_sk:int,ss_ext_sales_price:decimal(7,2)> + +(2) CometFilter +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)) + +(3) ColumnarToRow [codegen id : 7] +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] + +(4) Scan parquet spark_catalog.default.store +Output [2]: [s_store_sk#8, s_gmt_offset#9] +Batched: true +Location [not included in comparison]/{warehouse_dir}/store] +PushedFilters: [IsNotNull(s_gmt_offset), EqualTo(s_gmt_offset,-5.00), IsNotNull(s_store_sk)] +ReadSchema: struct<s_store_sk:int,s_gmt_offset:decimal(5,2)> + +(5) CometFilter +Input [2]: [s_store_sk#8, s_gmt_offset#9] +Condition : ((isnotnull(s_gmt_offset#9) AND (s_gmt_offset#9 = -5.00)) AND isnotnull(s_store_sk#8)) + +(6) CometProject +Input [2]: [s_store_sk#8, s_gmt_offset#9] +Arguments: [s_store_sk#8], [s_store_sk#8] + +(7) ColumnarToRow [codegen id : 1] +Input [1]: [s_store_sk#8] + +(8) BroadcastExchange +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] +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] +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] + +(11) Scan parquet spark_catalog.default.promotion +Output [4]: [p_promo_sk#10, p_channel_dmail#11, p_channel_email#12, p_channel_tv#13] +Batched: true +Location [not included in comparison]/{warehouse_dir}/promotion] +PushedFilters: [Or(Or(EqualTo(p_channel_dmail,Y),EqualTo(p_channel_email,Y)),EqualTo(p_channel_tv,Y)), IsNotNull(p_promo_sk)] +ReadSchema: struct<p_promo_sk:int,p_channel_dmail:string,p_channel_email:string,p_channel_tv:string> + +(12) CometFilter +Input [4]: [p_promo_sk#10, p_channel_dmail#11, p_channel_email#12, p_channel_tv#13] +Condition : ((((p_channel_dmail#11 = Y) OR (p_channel_email#12 = Y)) OR (p_channel_tv#13 = Y)) AND isnotnull(p_promo_sk#10)) + +(13) CometProject +Input [4]: [p_promo_sk#10, p_channel_dmail#11, p_channel_email#12, p_channel_tv#13] +Arguments: [p_promo_sk#10], [p_promo_sk#10] + +(14) ColumnarToRow [codegen id : 2] +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] +Left keys [1]: [ss_promo_sk#4] +Right keys [1]: [p_promo_sk#10] +Join type: Inner +Join condition: None + +(17) Project [codegen id : 7] +Output [4]: [ss_item_sk#1, ss_customer_sk#2, ss_ext_sales_price#5, ss_sold_date_sk#6] +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: 72] +Output [1]: [d_date_sk#14] + +(19) BroadcastHashJoin [codegen id : 7] +Left keys [1]: [ss_sold_date_sk#6] +Right keys [1]: [d_date_sk#14] +Join type: Inner +Join condition: None + +(20) Project [codegen id : 7] +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] + +(21) Scan parquet spark_catalog.default.customer +Output [2]: [c_customer_sk#15, c_current_addr_sk#16] +Batched: true +Location [not included in comparison]/{warehouse_dir}/customer] +PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_current_addr_sk)] +ReadSchema: struct<c_customer_sk:int,c_current_addr_sk:int> + +(22) CometFilter +Input [2]: [c_customer_sk#15, c_current_addr_sk#16] +Condition : (isnotnull(c_customer_sk#15) AND isnotnull(c_current_addr_sk#16)) + +(23) ColumnarToRow [codegen id : 4] +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] +Left keys [1]: [ss_customer_sk#2] +Right keys [1]: [c_customer_sk#15] +Join type: Inner +Join condition: None + +(26) Project [codegen id : 7] +Output [3]: [ss_item_sk#1, ss_ext_sales_price#5, c_current_addr_sk#16] +Input [5]: [ss_item_sk#1, ss_customer_sk#2, ss_ext_sales_price#5, c_customer_sk#15, c_current_addr_sk#16] + +(27) Scan parquet spark_catalog.default.customer_address +Output [2]: [ca_address_sk#17, ca_gmt_offset#18] +Batched: true +Location [not included in comparison]/{warehouse_dir}/customer_address] +PushedFilters: [IsNotNull(ca_gmt_offset), EqualTo(ca_gmt_offset,-5.00), IsNotNull(ca_address_sk)] +ReadSchema: struct<ca_address_sk:int,ca_gmt_offset:decimal(5,2)> + +(28) CometFilter +Input [2]: [ca_address_sk#17, ca_gmt_offset#18] +Condition : ((isnotnull(ca_gmt_offset#18) AND (ca_gmt_offset#18 = -5.00)) AND isnotnull(ca_address_sk#17)) + +(29) CometProject +Input [2]: [ca_address_sk#17, ca_gmt_offset#18] +Arguments: [ca_address_sk#17], [ca_address_sk#17] + +(30) ColumnarToRow [codegen id : 5] +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] +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] +Output [2]: [ss_item_sk#1, ss_ext_sales_price#5] +Input [4]: [ss_item_sk#1, ss_ext_sales_price#5, c_current_addr_sk#16, ca_address_sk#17] + +(34) Scan parquet spark_catalog.default.item +Output [2]: [i_item_sk#19, i_category#20] +Batched: true +Location [not included in comparison]/{warehouse_dir}/item] +PushedFilters: [IsNotNull(i_category), EqualTo(i_category,Jewelry ), IsNotNull(i_item_sk)] +ReadSchema: struct<i_item_sk:int,i_category:string> + +(35) CometFilter +Input [2]: [i_item_sk#19, i_category#20] +Condition : ((isnotnull(i_category#20) AND (i_category#20 = Jewelry )) AND isnotnull(i_item_sk#19)) + +(36) CometProject +Input [2]: [i_item_sk#19, i_category#20] +Arguments: [i_item_sk#19], [i_item_sk#19] + +(37) ColumnarToRow [codegen id : 6] +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] +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] +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] +Input [1]: [ss_ext_sales_price#5] +Keys: [] +Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#5))] +Aggregate Attributes [1]: [sum#21] +Results [1]: [sum#22] + +(42) Exchange +Input [1]: [sum#22] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=6] + +(43) HashAggregate [codegen id : 15] +Input [1]: [sum#22] +Keys: [] +Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#5))] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#5))#23] +Results [1]: [MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#5))#23,17,2) AS promotions#24] + +(44) Scan parquet spark_catalog.default.store_sales +Output [5]: [ss_item_sk#25, ss_customer_sk#26, ss_store_sk#27, ss_ext_sales_price#28, ss_sold_date_sk#29] +Batched: true +Location: InMemoryFileIndex [] +PartitionFilters: [isnotnull(ss_sold_date_sk#29), dynamicpruningexpression(ss_sold_date_sk#29 IN dynamicpruning#30)] +PushedFilters: [IsNotNull(ss_store_sk), IsNotNull(ss_customer_sk), IsNotNull(ss_item_sk)] +ReadSchema: struct<ss_item_sk:int,ss_customer_sk:int,ss_store_sk:int,ss_ext_sales_price:decimal(7,2)> + +(45) CometFilter +Input [5]: [ss_item_sk#25, ss_customer_sk#26, ss_store_sk#27, ss_ext_sales_price#28, ss_sold_date_sk#29] +Condition : ((isnotnull(ss_store_sk#27) AND isnotnull(ss_customer_sk#26)) AND isnotnull(ss_item_sk#25)) + +(46) ColumnarToRow [codegen id : 13] +Input [5]: [ss_item_sk#25, ss_customer_sk#26, ss_store_sk#27, ss_ext_sales_price#28, ss_sold_date_sk#29] + +(47) ReusedExchange [Reuses operator id: 8] +Output [1]: [s_store_sk#31] + +(48) BroadcastHashJoin [codegen id : 13] +Left keys [1]: [ss_store_sk#27] +Right keys [1]: [s_store_sk#31] +Join type: Inner +Join condition: None + +(49) Project [codegen id : 13] +Output [4]: [ss_item_sk#25, ss_customer_sk#26, ss_ext_sales_price#28, ss_sold_date_sk#29] +Input [6]: [ss_item_sk#25, ss_customer_sk#26, ss_store_sk#27, ss_ext_sales_price#28, ss_sold_date_sk#29, s_store_sk#31] + +(50) ReusedExchange [Reuses operator id: 72] +Output [1]: [d_date_sk#32] + +(51) BroadcastHashJoin [codegen id : 13] +Left keys [1]: [ss_sold_date_sk#29] +Right keys [1]: [d_date_sk#32] +Join type: Inner +Join condition: None + +(52) Project [codegen id : 13] +Output [3]: [ss_item_sk#25, ss_customer_sk#26, ss_ext_sales_price#28] +Input [5]: [ss_item_sk#25, ss_customer_sk#26, ss_ext_sales_price#28, ss_sold_date_sk#29, d_date_sk#32] + +(53) ReusedExchange [Reuses operator id: 24] +Output [2]: [c_customer_sk#33, c_current_addr_sk#34] + +(54) BroadcastHashJoin [codegen id : 13] +Left keys [1]: [ss_customer_sk#26] +Right keys [1]: [c_customer_sk#33] +Join type: Inner +Join condition: None + +(55) Project [codegen id : 13] +Output [3]: [ss_item_sk#25, ss_ext_sales_price#28, c_current_addr_sk#34] +Input [5]: [ss_item_sk#25, ss_customer_sk#26, ss_ext_sales_price#28, c_customer_sk#33, c_current_addr_sk#34] + +(56) ReusedExchange [Reuses operator id: 31] +Output [1]: [ca_address_sk#35] + +(57) BroadcastHashJoin [codegen id : 13] +Left keys [1]: [c_current_addr_sk#34] +Right keys [1]: [ca_address_sk#35] +Join type: Inner +Join condition: None + +(58) Project [codegen id : 13] +Output [2]: [ss_item_sk#25, ss_ext_sales_price#28] +Input [4]: [ss_item_sk#25, ss_ext_sales_price#28, c_current_addr_sk#34, ca_address_sk#35] + +(59) ReusedExchange [Reuses operator id: 38] +Output [1]: [i_item_sk#36] + +(60) BroadcastHashJoin [codegen id : 13] +Left keys [1]: [ss_item_sk#25] +Right keys [1]: [i_item_sk#36] +Join type: Inner +Join condition: None + +(61) Project [codegen id : 13] +Output [1]: [ss_ext_sales_price#28] +Input [3]: [ss_item_sk#25, ss_ext_sales_price#28, i_item_sk#36] + +(62) HashAggregate [codegen id : 13] +Input [1]: [ss_ext_sales_price#28] +Keys: [] +Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#28))] +Aggregate Attributes [1]: [sum#37] +Results [1]: [sum#38] + +(63) Exchange +Input [1]: [sum#38] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=7] + +(64) HashAggregate [codegen id : 14] +Input [1]: [sum#38] +Keys: [] +Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#28))] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#28))#39] +Results [1]: [MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#28))#39,17,2) AS total#40] + +(65) BroadcastExchange +Input [1]: [total#40] +Arguments: IdentityBroadcastMode, [plan_id=8] + +(66) BroadcastNestedLoopJoin [codegen id : 15] +Join type: Inner +Join condition: None + +(67) Project [codegen id : 15] +Output [3]: [promotions#24, total#40, ((cast(promotions#24 as decimal(15,4)) / cast(total#40 as decimal(15,4))) * 100) AS ((CAST(promotions AS DECIMAL(15,4)) / CAST(total AS DECIMAL(15,4))) * 100)#41] +Input [2]: [promotions#24, total#40] + +===== Subqueries ===== + +Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#6 IN dynamicpruning#7 +BroadcastExchange (72) ++- * ColumnarToRow (71) + +- CometProject (70) + +- CometFilter (69) + +- CometScan parquet spark_catalog.default.date_dim (68) + + +(68) Scan parquet spark_catalog.default.date_dim +Output [3]: [d_date_sk#14, d_year#42, d_moy#43] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,1998), EqualTo(d_moy,11), IsNotNull(d_date_sk)] +ReadSchema: struct<d_date_sk:int,d_year:int,d_moy:int> + +(69) CometFilter +Input [3]: [d_date_sk#14, d_year#42, d_moy#43] +Condition : ((((isnotnull(d_year#42) AND isnotnull(d_moy#43)) AND (d_year#42 = 1998)) AND (d_moy#43 = 11)) AND isnotnull(d_date_sk#14)) + +(70) CometProject +Input [3]: [d_date_sk#14, d_year#42, d_moy#43] +Arguments: [d_date_sk#14], [d_date_sk#14] + +(71) ColumnarToRow [codegen id : 1] +Input [1]: [d_date_sk#14] + +(72) BroadcastExchange +Input [1]: [d_date_sk#14] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=9] + +Subquery:2 Hosting operator id = 44 Hosting Expression = ss_sold_date_sk#29 IN dynamicpruning#7 + + diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q61/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q61/simplified.txt new file mode 100644 index 000000000..2c3d07ac6 --- /dev/null +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q61/simplified.txt @@ -0,0 +1,106 @@ +WholeStageCodegen (15) + Project [promotions,total] + BroadcastNestedLoopJoin + HashAggregate [sum] [sum(UnscaledValue(ss_ext_sales_price)),promotions,sum] + InputAdapter + Exchange #1 + WholeStageCodegen (7) + HashAggregate [ss_ext_sales_price] [sum,sum] + Project [ss_ext_sales_price] + BroadcastHashJoin [ss_item_sk,i_item_sk] + Project [ss_item_sk,ss_ext_sales_price] + BroadcastHashJoin [c_current_addr_sk,ca_address_sk] + Project [ss_item_sk,ss_ext_sales_price,c_current_addr_sk] + BroadcastHashJoin [ss_customer_sk,c_customer_sk] + Project [ss_item_sk,ss_customer_sk,ss_ext_sales_price] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [ss_item_sk,ss_customer_sk,ss_ext_sales_price,ss_sold_date_sk] + BroadcastHashJoin [ss_promo_sk,p_promo_sk] + Project [ss_item_sk,ss_customer_sk,ss_promo_sk,ss_ext_sales_price,ss_sold_date_sk] + BroadcastHashJoin [ss_store_sk,s_store_sk] + ColumnarToRow + InputAdapter + CometFilter [ss_store_sk,ss_promo_sk,ss_customer_sk,ss_item_sk] + CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_store_sk,ss_promo_sk,ss_ext_sales_price,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #2 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_year,d_moy,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + InputAdapter + BroadcastExchange #3 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [s_store_sk] + CometFilter [s_gmt_offset,s_store_sk] + CometScan parquet spark_catalog.default.store [s_store_sk,s_gmt_offset] + InputAdapter + BroadcastExchange #4 + WholeStageCodegen (2) + ColumnarToRow + InputAdapter + CometProject [p_promo_sk] + CometFilter [p_channel_dmail,p_channel_email,p_channel_tv,p_promo_sk] + CometScan parquet spark_catalog.default.promotion [p_promo_sk,p_channel_dmail,p_channel_email,p_channel_tv] + InputAdapter + ReusedExchange [d_date_sk] #2 + InputAdapter + BroadcastExchange #5 + WholeStageCodegen (4) + ColumnarToRow + InputAdapter + CometFilter [c_customer_sk,c_current_addr_sk] + CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk] + InputAdapter + BroadcastExchange #6 + WholeStageCodegen (5) + ColumnarToRow + InputAdapter + CometProject [ca_address_sk] + CometFilter [ca_gmt_offset,ca_address_sk] + CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_gmt_offset] + InputAdapter + BroadcastExchange #7 + WholeStageCodegen (6) + ColumnarToRow + InputAdapter + CometProject [i_item_sk] + CometFilter [i_category,i_item_sk] + CometScan parquet spark_catalog.default.item [i_item_sk,i_category] + InputAdapter + BroadcastExchange #8 + WholeStageCodegen (14) + HashAggregate [sum] [sum(UnscaledValue(ss_ext_sales_price)),total,sum] + InputAdapter + Exchange #9 + WholeStageCodegen (13) + HashAggregate [ss_ext_sales_price] [sum,sum] + Project [ss_ext_sales_price] + BroadcastHashJoin [ss_item_sk,i_item_sk] + Project [ss_item_sk,ss_ext_sales_price] + BroadcastHashJoin [c_current_addr_sk,ca_address_sk] + Project [ss_item_sk,ss_ext_sales_price,c_current_addr_sk] + BroadcastHashJoin [ss_customer_sk,c_customer_sk] + Project [ss_item_sk,ss_customer_sk,ss_ext_sales_price] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [ss_item_sk,ss_customer_sk,ss_ext_sales_price,ss_sold_date_sk] + BroadcastHashJoin [ss_store_sk,s_store_sk] + ColumnarToRow + InputAdapter + CometFilter [ss_store_sk,ss_customer_sk,ss_item_sk] + CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ext_sales_price,ss_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + InputAdapter + ReusedExchange [s_store_sk] #3 + InputAdapter + ReusedExchange [d_date_sk] #2 + InputAdapter + ReusedExchange [c_customer_sk,c_current_addr_sk] #5 + InputAdapter + ReusedExchange [ca_address_sk] #6 + InputAdapter + ReusedExchange [i_item_sk] #7 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q62/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q62/explain.txt new file mode 100644 index 000000000..0607d8077 --- /dev/null +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q62/explain.txt @@ -0,0 +1,187 @@ +== Physical Plan == +TakeOrderedAndProject (32) ++- * HashAggregate (31) + +- Exchange (30) + +- * HashAggregate (29) + +- * Project (28) + +- * BroadcastHashJoin Inner BuildRight (27) + :- * Project (21) + : +- * BroadcastHashJoin Inner BuildRight (20) + : :- * Project (15) + : : +- * BroadcastHashJoin Inner BuildRight (14) + : : :- * Project (9) + : : : +- * BroadcastHashJoin Inner BuildRight (8) + : : : :- * ColumnarToRow (3) + : : : : +- CometFilter (2) + : : : : +- CometScan parquet spark_catalog.default.web_sales (1) + : : : +- BroadcastExchange (7) + : : : +- * ColumnarToRow (6) + : : : +- CometFilter (5) + : : : +- CometScan parquet spark_catalog.default.warehouse (4) + : : +- BroadcastExchange (13) + : : +- * ColumnarToRow (12) + : : +- CometFilter (11) + : : +- CometScan parquet spark_catalog.default.ship_mode (10) + : +- BroadcastExchange (19) + : +- * ColumnarToRow (18) + : +- CometFilter (17) + : +- CometScan parquet spark_catalog.default.web_site (16) + +- BroadcastExchange (26) + +- * ColumnarToRow (25) + +- CometProject (24) + +- CometFilter (23) + +- CometScan parquet spark_catalog.default.date_dim (22) + + +(1) Scan parquet spark_catalog.default.web_sales +Output [5]: [ws_ship_date_sk#1, ws_web_site_sk#2, ws_ship_mode_sk#3, ws_warehouse_sk#4, ws_sold_date_sk#5] +Batched: true +Location [not included in comparison]/{warehouse_dir}/web_sales] +PushedFilters: [IsNotNull(ws_warehouse_sk), IsNotNull(ws_ship_mode_sk), IsNotNull(ws_web_site_sk), IsNotNull(ws_ship_date_sk)] +ReadSchema: struct<ws_ship_date_sk:int,ws_web_site_sk:int,ws_ship_mode_sk:int,ws_warehouse_sk:int> + +(2) CometFilter +Input [5]: [ws_ship_date_sk#1, ws_web_site_sk#2, ws_ship_mode_sk#3, ws_warehouse_sk#4, ws_sold_date_sk#5] +Condition : (((isnotnull(ws_warehouse_sk#4) AND isnotnull(ws_ship_mode_sk#3)) AND isnotnull(ws_web_site_sk#2)) AND isnotnull(ws_ship_date_sk#1)) + +(3) ColumnarToRow [codegen id : 5] +Input [5]: [ws_ship_date_sk#1, ws_web_site_sk#2, ws_ship_mode_sk#3, ws_warehouse_sk#4, ws_sold_date_sk#5] + +(4) Scan parquet spark_catalog.default.warehouse +Output [2]: [w_warehouse_sk#6, w_warehouse_name#7] +Batched: true +Location [not included in comparison]/{warehouse_dir}/warehouse] +PushedFilters: [IsNotNull(w_warehouse_sk)] +ReadSchema: struct<w_warehouse_sk:int,w_warehouse_name:string> + +(5) CometFilter +Input [2]: [w_warehouse_sk#6, w_warehouse_name#7] +Condition : isnotnull(w_warehouse_sk#6) + +(6) ColumnarToRow [codegen id : 1] +Input [2]: [w_warehouse_sk#6, w_warehouse_name#7] + +(7) BroadcastExchange +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 : 5] +Left keys [1]: [ws_warehouse_sk#4] +Right keys [1]: [w_warehouse_sk#6] +Join type: Inner +Join condition: None + +(9) Project [codegen id : 5] +Output [5]: [ws_ship_date_sk#1, ws_web_site_sk#2, ws_ship_mode_sk#3, ws_sold_date_sk#5, w_warehouse_name#7] +Input [7]: [ws_ship_date_sk#1, ws_web_site_sk#2, ws_ship_mode_sk#3, ws_warehouse_sk#4, ws_sold_date_sk#5, w_warehouse_sk#6, w_warehouse_name#7] + +(10) Scan parquet spark_catalog.default.ship_mode +Output [2]: [sm_ship_mode_sk#8, sm_type#9] +Batched: true +Location [not included in comparison]/{warehouse_dir}/ship_mode] +PushedFilters: [IsNotNull(sm_ship_mode_sk)] +ReadSchema: struct<sm_ship_mode_sk:int,sm_type:string> + +(11) CometFilter +Input [2]: [sm_ship_mode_sk#8, sm_type#9] +Condition : isnotnull(sm_ship_mode_sk#8) + +(12) ColumnarToRow [codegen id : 2] +Input [2]: [sm_ship_mode_sk#8, sm_type#9] + +(13) BroadcastExchange +Input [2]: [sm_ship_mode_sk#8, sm_type#9] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=2] + +(14) BroadcastHashJoin [codegen id : 5] +Left keys [1]: [ws_ship_mode_sk#3] +Right keys [1]: [sm_ship_mode_sk#8] +Join type: Inner +Join condition: None + +(15) Project [codegen id : 5] +Output [5]: [ws_ship_date_sk#1, ws_web_site_sk#2, ws_sold_date_sk#5, w_warehouse_name#7, sm_type#9] +Input [7]: [ws_ship_date_sk#1, ws_web_site_sk#2, ws_ship_mode_sk#3, ws_sold_date_sk#5, w_warehouse_name#7, sm_ship_mode_sk#8, sm_type#9] + +(16) Scan parquet spark_catalog.default.web_site +Output [2]: [web_site_sk#10, web_name#11] +Batched: true +Location [not included in comparison]/{warehouse_dir}/web_site] +PushedFilters: [IsNotNull(web_site_sk)] +ReadSchema: struct<web_site_sk:int,web_name:string> + +(17) CometFilter +Input [2]: [web_site_sk#10, web_name#11] +Condition : isnotnull(web_site_sk#10) + +(18) ColumnarToRow [codegen id : 3] +Input [2]: [web_site_sk#10, web_name#11] + +(19) BroadcastExchange +Input [2]: [web_site_sk#10, web_name#11] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=3] + +(20) BroadcastHashJoin [codegen id : 5] +Left keys [1]: [ws_web_site_sk#2] +Right keys [1]: [web_site_sk#10] +Join type: Inner +Join condition: None + +(21) Project [codegen id : 5] +Output [5]: [ws_ship_date_sk#1, ws_sold_date_sk#5, w_warehouse_name#7, sm_type#9, web_name#11] +Input [7]: [ws_ship_date_sk#1, ws_web_site_sk#2, ws_sold_date_sk#5, w_warehouse_name#7, sm_type#9, web_site_sk#10, web_name#11] + +(22) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#12, d_month_seq#13] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1200), LessThanOrEqual(d_month_seq,1211), IsNotNull(d_date_sk)] +ReadSchema: struct<d_date_sk:int,d_month_seq:int> + +(23) CometFilter +Input [2]: [d_date_sk#12, d_month_seq#13] +Condition : (((isnotnull(d_month_seq#13) AND (d_month_seq#13 >= 1200)) AND (d_month_seq#13 <= 1211)) AND isnotnull(d_date_sk#12)) + +(24) CometProject +Input [2]: [d_date_sk#12, d_month_seq#13] +Arguments: [d_date_sk#12], [d_date_sk#12] + +(25) ColumnarToRow [codegen id : 4] +Input [1]: [d_date_sk#12] + +(26) BroadcastExchange +Input [1]: [d_date_sk#12] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] + +(27) BroadcastHashJoin [codegen id : 5] +Left keys [1]: [ws_ship_date_sk#1] +Right keys [1]: [d_date_sk#12] +Join type: Inner +Join condition: None + +(28) Project [codegen id : 5] +Output [5]: [ws_ship_date_sk#1, ws_sold_date_sk#5, sm_type#9, web_name#11, substr(w_warehouse_name#7, 1, 20) AS _groupingexpression#14] +Input [6]: [ws_ship_date_sk#1, ws_sold_date_sk#5, w_warehouse_name#7, sm_type#9, web_name#11, d_date_sk#12] + +(29) HashAggregate [codegen id : 5] +Input [5]: [ws_ship_date_sk#1, ws_sold_date_sk#5, sm_type#9, web_name#11, _groupingexpression#14] +Keys [3]: [_groupingexpression#14, sm_type#9, web_name#11] +Functions [5]: [partial_sum(CASE WHEN ((ws_ship_date_sk#1 - ws_sold_date_sk#5) <= 30) THEN 1 ELSE 0 END), partial_sum(CASE WHEN (((ws_ship_date_sk#1 - ws_sold_date_sk#5) > 30) AND ((ws_ship_date_sk#1 - ws_sold_date_sk#5) <= 60)) THEN 1 ELSE 0 END), partial_sum(CASE WHEN (((ws_ship_date_sk#1 - ws_sold_date_sk#5) > 60) AND ((ws_ship_date_sk#1 - ws_sold_date_sk#5) <= 90)) THEN 1 ELSE 0 END), partial_sum(CASE WHEN (((ws_ship_date_sk#1 - ws_sold_date_sk#5) > 90) AND ((ws_ship_date_sk#1 - ws_sold_date_sk#5) <= 120)) THEN 1 ELSE 0 END), partial_sum(CASE WHEN ((ws_ship_date_sk#1 - ws_sold_date_sk#5) > 120) THEN 1 ELSE 0 END)] +Aggregate Attributes [5]: [sum#15, sum#16, sum#17, sum#18, sum#19] +Results [8]: [_groupingexpression#14, sm_type#9, web_name#11, sum#20, sum#21, sum#22, sum#23, sum#24] + +(30) Exchange +Input [8]: [_groupingexpression#14, sm_type#9, web_name#11, sum#20, sum#21, sum#22, sum#23, sum#24] +Arguments: hashpartitioning(_groupingexpression#14, sm_type#9, web_name#11, 5), ENSURE_REQUIREMENTS, [plan_id=5] + +(31) HashAggregate [codegen id : 6] +Input [8]: [_groupingexpression#14, sm_type#9, web_name#11, sum#20, sum#21, sum#22, sum#23, sum#24] +Keys [3]: [_groupingexpression#14, sm_type#9, web_name#11] +Functions [5]: [sum(CASE WHEN ((ws_ship_date_sk#1 - ws_sold_date_sk#5) <= 30) THEN 1 ELSE 0 END), sum(CASE WHEN (((ws_ship_date_sk#1 - ws_sold_date_sk#5) > 30) AND ((ws_ship_date_sk#1 - ws_sold_date_sk#5) <= 60)) THEN 1 ELSE 0 END), sum(CASE WHEN (((ws_ship_date_sk#1 - ws_sold_date_sk#5) > 60) AND ((ws_ship_date_sk#1 - ws_sold_date_sk#5) <= 90)) THEN 1 ELSE 0 END), sum(CASE WHEN (((ws_ship_date_sk#1 - ws_sold_date_sk#5) > 90) AND ((ws_ship_date_sk#1 - ws_sold_date_sk#5) <= 120)) THEN 1 ELSE 0 END), sum(CASE WHEN ((ws_ship_date_sk#1 - ws_sold_date_sk#5) > 120) THEN 1 ELSE 0 END)] +Aggregate Attributes [5]: [sum(CASE WHEN ((ws_ship_date_sk#1 - ws_sold_date_sk#5) <= 30) THEN 1 ELSE 0 END)#25, sum(CASE WHEN (((ws_ship_date_sk#1 - ws_sold_date_sk#5) > 30) AND ((ws_ship_date_sk#1 - ws_sold_date_sk#5) <= 60)) THEN 1 ELSE 0 END)#26, sum(CASE WHEN (((ws_ship_date_sk#1 - ws_sold_date_sk#5) > 60) AND ((ws_ship_date_sk#1 - ws_sold_date_sk#5) <= 90)) THEN 1 ELSE 0 END)#27, sum(CASE WHEN (((ws_ship_date_sk#1 - ws_sold_date_sk#5) > 90) AND ((ws_ship_date_sk#1 - ws_sold_date_sk#5) <= 120)) THEN 1 ELSE 0 END)#28, sum(CASE WHEN ((ws_ship_date_sk#1 - ws_sold_date_sk#5) > 120) THEN 1 ELSE 0 END)#29] +Results [8]: [_groupingexpression#14 AS substr(w_warehouse_name, 1, 20)#30, sm_type#9, web_name#11, sum(CASE WHEN ((ws_ship_date_sk#1 - ws_sold_date_sk#5) <= 30) THEN 1 ELSE 0 END)#25 AS 30 days #31, sum(CASE WHEN (((ws_ship_date_sk#1 - ws_sold_date_sk#5) > 30) AND ((ws_ship_date_sk#1 - ws_sold_date_sk#5) <= 60)) THEN 1 ELSE 0 END)#26 AS 31 - 60 days #32, sum(CASE WHEN (((ws_ship_date_sk#1 - ws_sold_date_sk#5) > 60) AND ((ws_ship_date_sk#1 - ws_sold_date_sk#5) <= 90)) THEN 1 ELSE 0 END)#27 AS 61 - 90 days #33, sum(CASE WHEN (((ws_ship_date_sk#1 - ws_sold_date_sk#5) > 90) AND ((ws_ship_date_sk#1 - ws_sold_date_sk#5) <= 120)) THEN 1 ELSE 0 END)#28 AS 91 - 120 days #34, sum(CASE WHEN ((ws_ship_date_sk#1 - ws_sold_date_sk#5) > 120) THEN 1 ELSE 0 END)#29 AS >120 days #35] + +(32) TakeOrderedAndProject +Input [8]: [substr(w_warehouse_name, 1, 20)#30, sm_type#9, web_name#11, 30 days #31, 31 - 60 days #32, 61 - 90 days #33, 91 - 120 days #34, >120 days #35] +Arguments: 100, [substr(w_warehouse_name, 1, 20)#30 ASC NULLS FIRST, sm_type#9 ASC NULLS FIRST, web_name#11 ASC NULLS FIRST], [substr(w_warehouse_name, 1, 20)#30, sm_type#9, web_name#11, 30 days #31, 31 - 60 days #32, 61 - 90 days #33, 91 - 120 days #34, >120 days #35] + diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q62/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q62/simplified.txt new file mode 100644 index 000000000..5ae522ce1 --- /dev/null +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q62/simplified.txt @@ -0,0 +1,48 @@ +TakeOrderedAndProject [substr(w_warehouse_name, 1, 20),sm_type,web_name,30 days ,31 - 60 days ,61 - 90 days ,91 - 120 days ,>120 days ] + WholeStageCodegen (6) + HashAggregate [_groupingexpression,sm_type,web_name,sum,sum,sum,sum,sum] [sum(CASE WHEN ((ws_ship_date_sk - ws_sold_date_sk) <= 30) THEN 1 ELSE 0 END),sum(CASE WHEN (((ws_ship_date_sk - ws_sold_date_sk) > 30) AND ((ws_ship_date_sk - ws_sold_date_sk) <= 60)) THEN 1 ELSE 0 END),sum(CASE WHEN (((ws_ship_date_sk - ws_sold_date_sk) > 60) AND ((ws_ship_date_sk - ws_sold_date_sk) <= 90)) THEN 1 ELSE 0 END),sum(CASE WHEN (((ws_ship_date_sk - ws_sold_date_sk) > 90) AND ((ws_ship_date_sk - ws_sold_date_sk) <= 120)) THEN 1 ELSE 0 END),sum(CASE WHEN ((ws_ship_date_sk - ws_sold_date_sk) > 120) THEN 1 ELSE 0 END),substr(w_warehouse_name, 1, 20),30 days ,31 - 60 days ,61 - 90 days ,91 - 120 days ,>120 days ,sum,sum,sum,sum,sum] + InputAdapter + Exchange [_groupingexpression,sm_type,web_name] #1 + WholeStageCodegen (5) + HashAggregate [_groupingexpression,sm_type,web_name,ws_ship_date_sk,ws_sold_date_sk] [sum,sum,sum,sum,sum,sum,sum,sum,sum,sum] + Project [ws_ship_date_sk,ws_sold_date_sk,sm_type,web_name,w_warehouse_name] + BroadcastHashJoin [ws_ship_date_sk,d_date_sk] + Project [ws_ship_date_sk,ws_sold_date_sk,w_warehouse_name,sm_type,web_name] + BroadcastHashJoin [ws_web_site_sk,web_site_sk] + Project [ws_ship_date_sk,ws_web_site_sk,ws_sold_date_sk,w_warehouse_name,sm_type] + BroadcastHashJoin [ws_ship_mode_sk,sm_ship_mode_sk] + Project [ws_ship_date_sk,ws_web_site_sk,ws_ship_mode_sk,ws_sold_date_sk,w_warehouse_name] + BroadcastHashJoin [ws_warehouse_sk,w_warehouse_sk] + ColumnarToRow + InputAdapter + CometFilter [ws_warehouse_sk,ws_ship_mode_sk,ws_web_site_sk,ws_ship_date_sk] + CometScan parquet spark_catalog.default.web_sales [ws_ship_date_sk,ws_web_site_sk,ws_ship_mode_sk,ws_warehouse_sk,ws_sold_date_sk] + InputAdapter + BroadcastExchange #2 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometFilter [w_warehouse_sk] + CometScan parquet spark_catalog.default.warehouse [w_warehouse_sk,w_warehouse_name] + InputAdapter + BroadcastExchange #3 + WholeStageCodegen (2) + ColumnarToRow + InputAdapter + CometFilter [sm_ship_mode_sk] + CometScan parquet spark_catalog.default.ship_mode [sm_ship_mode_sk,sm_type] + InputAdapter + BroadcastExchange #4 + WholeStageCodegen (3) + ColumnarToRow + InputAdapter + CometFilter [web_site_sk] + CometScan parquet spark_catalog.default.web_site [web_site_sk,web_name] + InputAdapter + BroadcastExchange #5 + WholeStageCodegen (4) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_month_seq,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] 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 new file mode 100644 index 000000000..990bc3195 --- /dev/null +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q63/explain.txt @@ -0,0 +1,194 @@ +== Physical Plan == +TakeOrderedAndProject (28) ++- * Project (27) + +- * Filter (26) + +- Window (25) + +- * Sort (24) + +- Exchange (23) + +- * HashAggregate (22) + +- Exchange (21) + +- * HashAggregate (20) + +- * Project (19) + +- * BroadcastHashJoin Inner BuildRight (18) + :- * Project (13) + : +- * BroadcastHashJoin Inner BuildRight (12) + : :- * Project (10) + : : +- * BroadcastHashJoin Inner BuildRight (9) + : : :- * ColumnarToRow (4) + : : : +- CometProject (3) + : : : +- CometFilter (2) + : : : +- CometScan parquet spark_catalog.default.item (1) + : : +- BroadcastExchange (8) + : : +- * ColumnarToRow (7) + : : +- CometFilter (6) + : : +- CometScan parquet spark_catalog.default.store_sales (5) + : +- ReusedExchange (11) + +- BroadcastExchange (17) + +- * ColumnarToRow (16) + +- CometFilter (15) + +- CometScan parquet spark_catalog.default.store (14) + + +(1) Scan parquet spark_catalog.default.item +Output [5]: [i_item_sk#1, i_brand#2, i_class#3, i_category#4, i_manager_id#5] +Batched: true +Location [not included in comparison]/{warehouse_dir}/item] +PushedFilters: [Or(And(And(In(i_category, [Books ,Children ,Electronics ]),In(i_class, [personal ,portable ,refernece ,self-help ])),In(i_brand, [exportiunivamalg #6 ,scholaramalgamalg #7 ,scholaramalgamalg #8 ,scholaramalgamalg #6 ])),And(And(In(i_category, [Men ,Music ,Women ]),In(i_class, [accessories ,classical ,fragrances ,pants ])),In(i_brand, [amalgimporto #9 ,edu packscholar #9 ,exportiimporto #9 ,importoamalg #9 ]))), IsNotNull(i_item_sk)] +ReadSchema: struct<i_item_sk:int,i_brand:string,i_class:string,i_category:string,i_manager_id:int> + +(2) CometFilter +Input [5]: [i_item_sk#1, i_brand#2, i_class#3, i_category#4, i_manager_id#5] +Condition : ((((i_category#4 IN (Books ,Children ,Electronics ) AND i_class#3 IN (personal ,portable ,refernece ,self-help )) AND i_brand#2 IN (scholaramalgamalg #7 ,scholaramalgamalg #8 ,exportiunivamalg #6 ,scholaramalgamalg #6 )) OR ((i_category#4 IN (Women ,Music ,Men ) AND i_class#3 IN (accessories ,classical ,fragrances ,pants )) AND i_brand#2 IN (amalgimporto #9 ,edu packscholar #9 ,exportiimporto #9 ,importoamalg #9 ))) AND isnotnull(i_item_sk#1)) + +(3) CometProject +Input [5]: [i_item_sk#1, i_brand#2, i_class#3, i_category#4, i_manager_id#5] +Arguments: [i_item_sk#1, i_manager_id#5], [i_item_sk#1, i_manager_id#5] + +(4) ColumnarToRow [codegen id : 4] +Input [2]: [i_item_sk#1, i_manager_id#5] + +(5) Scan parquet spark_catalog.default.store_sales +Output [4]: [ss_item_sk#10, ss_store_sk#11, ss_sales_price#12, ss_sold_date_sk#13] +Batched: true +Location: InMemoryFileIndex [] +PartitionFilters: [isnotnull(ss_sold_date_sk#13), dynamicpruningexpression(ss_sold_date_sk#13 IN dynamicpruning#14)] +PushedFilters: [IsNotNull(ss_item_sk), IsNotNull(ss_store_sk)] +ReadSchema: struct<ss_item_sk:int,ss_store_sk:int,ss_sales_price:decimal(7,2)> + +(6) CometFilter +Input [4]: [ss_item_sk#10, ss_store_sk#11, ss_sales_price#12, ss_sold_date_sk#13] +Condition : (isnotnull(ss_item_sk#10) AND isnotnull(ss_store_sk#11)) + +(7) ColumnarToRow [codegen id : 1] +Input [4]: [ss_item_sk#10, ss_store_sk#11, ss_sales_price#12, ss_sold_date_sk#13] + +(8) BroadcastExchange +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] +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] +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: 33] +Output [2]: [d_date_sk#15, d_moy#16] + +(12) BroadcastHashJoin [codegen id : 4] +Left keys [1]: [ss_sold_date_sk#13] +Right keys [1]: [d_date_sk#15] +Join type: Inner +Join condition: None + +(13) Project [codegen id : 4] +Output [4]: [i_manager_id#5, ss_store_sk#11, ss_sales_price#12, d_moy#16] +Input [6]: [i_manager_id#5, ss_store_sk#11, ss_sales_price#12, ss_sold_date_sk#13, d_date_sk#15, d_moy#16] + +(14) Scan parquet spark_catalog.default.store +Output [1]: [s_store_sk#17] +Batched: true +Location [not included in comparison]/{warehouse_dir}/store] +PushedFilters: [IsNotNull(s_store_sk)] +ReadSchema: struct<s_store_sk:int> + +(15) CometFilter +Input [1]: [s_store_sk#17] +Condition : isnotnull(s_store_sk#17) + +(16) ColumnarToRow [codegen id : 3] +Input [1]: [s_store_sk#17] + +(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] +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] +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] +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))] +Aggregate Attributes [1]: [sum#18] +Results [3]: [i_manager_id#5, d_moy#16, sum#19] + +(21) Exchange +Input [3]: [i_manager_id#5, d_moy#16, sum#19] +Arguments: hashpartitioning(i_manager_id#5, d_moy#16, 5), ENSURE_REQUIREMENTS, [plan_id=3] + +(22) HashAggregate [codegen id : 5] +Input [3]: [i_manager_id#5, d_moy#16, sum#19] +Keys [2]: [i_manager_id#5, d_moy#16] +Functions [1]: [sum(UnscaledValue(ss_sales_price#12))] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_sales_price#12))#20] +Results [3]: [i_manager_id#5, MakeDecimal(sum(UnscaledValue(ss_sales_price#12))#20,17,2) AS sum_sales#21, MakeDecimal(sum(UnscaledValue(ss_sales_price#12))#20,17,2) AS _w0#22] + +(23) Exchange +Input [3]: [i_manager_id#5, sum_sales#21, _w0#22] +Arguments: hashpartitioning(i_manager_id#5, 5), ENSURE_REQUIREMENTS, [plan_id=4] + +(24) Sort [codegen id : 6] +Input [3]: [i_manager_id#5, sum_sales#21, _w0#22] +Arguments: [i_manager_id#5 ASC NULLS FIRST], false, 0 + +(25) Window +Input [3]: [i_manager_id#5, sum_sales#21, _w0#22] +Arguments: [avg(_w0#22) windowspecdefinition(i_manager_id#5, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS avg_monthly_sales#23], [i_manager_id#5] + +(26) Filter [codegen id : 7] +Input [4]: [i_manager_id#5, sum_sales#21, _w0#22, avg_monthly_sales#23] +Condition : CASE WHEN (avg_monthly_sales#23 > 0.000000) THEN ((abs((sum_sales#21 - avg_monthly_sales#23)) / avg_monthly_sales#23) > 0.1000000000000000) ELSE false END + +(27) Project [codegen id : 7] +Output [3]: [i_manager_id#5, sum_sales#21, avg_monthly_sales#23] +Input [4]: [i_manager_id#5, sum_sales#21, _w0#22, avg_monthly_sales#23] + +(28) TakeOrderedAndProject +Input [3]: [i_manager_id#5, sum_sales#21, avg_monthly_sales#23] +Arguments: 100, [i_manager_id#5 ASC NULLS FIRST, avg_monthly_sales#23 ASC NULLS FIRST, sum_sales#21 ASC NULLS FIRST], [i_manager_id#5, sum_sales#21, avg_monthly_sales#23] + +===== Subqueries ===== + +Subquery:1 Hosting operator id = 5 Hosting Expression = ss_sold_date_sk#13 IN dynamicpruning#14 +BroadcastExchange (33) ++- * ColumnarToRow (32) + +- CometProject (31) + +- CometFilter (30) + +- CometScan parquet spark_catalog.default.date_dim (29) + + +(29) Scan parquet spark_catalog.default.date_dim +Output [3]: [d_date_sk#15, d_month_seq#24, d_moy#16] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [In(d_month_seq, [1200,1201,1202,1203,1204,1205,1206,1207,1208,1209,1210,1211]), IsNotNull(d_date_sk)] +ReadSchema: struct<d_date_sk:int,d_month_seq:int,d_moy:int> + +(30) CometFilter +Input [3]: [d_date_sk#15, d_month_seq#24, d_moy#16] +Condition : (d_month_seq#24 INSET 1200, 1201, 1202, 1203, 1204, 1205, 1206, 1207, 1208, 1209, 1210, 1211 AND isnotnull(d_date_sk#15)) + +(31) CometProject +Input [3]: [d_date_sk#15, d_month_seq#24, d_moy#16] +Arguments: [d_date_sk#15, d_moy#16], [d_date_sk#15, d_moy#16] + +(32) ColumnarToRow [codegen id : 1] +Input [2]: [d_date_sk#15, d_moy#16] + +(33) BroadcastExchange +Input [2]: [d_date_sk#15, d_moy#16] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] + + 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 new file mode 100644 index 000000000..7f6f8c137 --- /dev/null +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q63/simplified.txt @@ -0,0 +1,51 @@ +TakeOrderedAndProject [i_manager_id,avg_monthly_sales,sum_sales] + WholeStageCodegen (7) + Project [i_manager_id,sum_sales,avg_monthly_sales] + Filter [avg_monthly_sales,sum_sales] + InputAdapter + Window [_w0,i_manager_id] + WholeStageCodegen (6) + Sort [i_manager_id] + InputAdapter + Exchange [i_manager_id] #1 + WholeStageCodegen (5) + HashAggregate [i_manager_id,d_moy,sum] [sum(UnscaledValue(ss_sales_price)),sum_sales,_w0,sum] + InputAdapter + Exchange [i_manager_id,d_moy] #2 + WholeStageCodegen (4) + HashAggregate [i_manager_id,d_moy,ss_sales_price] [sum,sum] + Project [i_manager_id,ss_sales_price,d_moy] + BroadcastHashJoin [ss_store_sk,s_store_sk] + Project [i_manager_id,ss_store_sk,ss_sales_price,d_moy] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [i_manager_id,ss_store_sk,ss_sales_price,ss_sold_date_sk] + BroadcastHashJoin [i_item_sk,ss_item_sk] + ColumnarToRow + InputAdapter + CometProject [i_item_sk,i_manager_id] + CometFilter [i_category,i_class,i_brand,i_item_sk] + CometScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category,i_manager_id] + InputAdapter + BroadcastExchange #3 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometFilter [ss_item_sk,ss_store_sk] + CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #4 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk,d_moy] + CometFilter [d_month_seq,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_moy] + InputAdapter + ReusedExchange [d_date_sk,d_moy] #4 + InputAdapter + BroadcastExchange #5 + WholeStageCodegen (3) + ColumnarToRow + InputAdapter + CometFilter [s_store_sk] + CometScan parquet spark_catalog.default.store [s_store_sk] 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 new file mode 100644 index 000000000..667362aa3 --- /dev/null +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q64/explain.txt @@ -0,0 +1,1064 @@ +== Physical Plan == +* Sort (181) ++- Exchange (180) + +- * Project (179) + +- * SortMergeJoin Inner (178) + :- * Sort (110) + : +- Exchange (109) + : +- * HashAggregate (108) + : +- * HashAggregate (107) + : +- * Project (106) + : +- * BroadcastHashJoin Inner BuildRight (105) + : :- * Project (99) + : : +- * BroadcastHashJoin Inner BuildRight (98) + : : :- * Project (96) + : : : +- * BroadcastHashJoin Inner BuildRight (95) + : : : :- * Project (90) + : : : : +- * BroadcastHashJoin Inner BuildRight (89) + : : : : :- * Project (87) + : : : : : +- * BroadcastHashJoin Inner BuildRight (86) + : : : : : :- * Project (81) + : : : : : : +- * BroadcastHashJoin Inner BuildRight (80) + : : : : : : :- * Project (78) + : : : : : : : +- * BroadcastHashJoin Inner BuildRight (77) + : : : : : : : :- * Project (72) + : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (71) + : : : : : : : : :- * Project (66) + : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (65) + : : : : : : : : : :- * Project (63) + : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (62) + : : : : : : : : : : :- * Project (57) + : : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (56) + : : : : : : : : : : : :- * Project (54) + : : : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (53) + : : : : : : : : : : : : :- * Project (48) + : : : : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (47) + : : : : : : : : : : : : : :- * Project (42) + : : : : : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (41) + : : : : : : : : : : : : : : :- * Project (36) + : : : : : : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (35) + : : : : : : : : : : : : : : : :- * Project (33) + : : : : : : : : : : : : : : : : +- * SortMergeJoin Inner (32) + : : : : : : : : : : : : : : : : :- * Sort (11) + : : : : : : : : : : : : : : : : : +- Exchange (10) + : : : : : : : : : : : : : : : : : +- * ColumnarToRow (9) + : : : : : : : : : : : : : : : : : +- CometProject (8) + : : : : : : : : : : : : : : : : : +- CometBroadcastHashJoin (7) + : : : : : : : : : : : : : : : : : :- CometBroadcastExchange (3) + : : : : : : : : : : : : : : : : : : +- CometFilter (2) + : : : : : : : : : : : : : : : : : : +- CometScan parquet spark_catalog.default.store_sales (1) + : : : : : : : : : : : : : : : : : +- CometProject (6) + : : : : : : : : : : : : : : : : : +- CometFilter (5) + : : : : : : : : : : : : : : : : : +- CometScan parquet spark_catalog.default.store_returns (4) + : : : : : : : : : : : : : : : : +- * Sort (31) + : : : : : : : : : : : : : : : : +- * Project (30) + : : : : : : : : : : : : : : : : +- * Filter (29) + : : : : : : : : : : : : : : : : +- * HashAggregate (28) + : : : : : : : : : : : : : : : : +- Exchange (27) + : : : : : : : : : : : : : : : : +- * HashAggregate (26) + : : : : : : : : : : : : : : : : +- * Project (25) + : : : : : : : : : : : : : : : : +- * SortMergeJoin Inner (24) + : : : : : : : : : : : : : : : : :- * Sort (17) + : : : : : : : : : : : : : : : : : +- Exchange (16) + : : : : : : : : : : : : : : : : : +- * ColumnarToRow (15) + : : : : : : : : : : : : : : : : : +- CometProject (14) + : : : : : : : : : : : : : : : : : +- CometFilter (13) + : : : : : : : : : : : : : : : : : +- CometScan parquet spark_catalog.default.catalog_sales (12) + : : : : : : : : : : : : : : : : +- * Sort (23) + : : : : : : : : : : : : : : : : +- Exchange (22) + : : : : : : : : : : : : : : : : +- * ColumnarToRow (21) + : : : : : : : : : : : : : : : : +- CometProject (20) + : : : : : : : : : : : : : : : : +- CometFilter (19) + : : : : : : : : : : : : : : : : +- CometScan parquet spark_catalog.default.catalog_returns (18) + : : : : : : : : : : : : : : : +- ReusedExchange (34) + : : : : : : : : : : : : : : +- BroadcastExchange (40) + : : : : : : : : : : : : : : +- * ColumnarToRow (39) + : : : : : : : : : : : : : : +- CometFilter (38) + : : : : : : : : : : : : : : +- CometScan parquet spark_catalog.default.store (37) + : : : : : : : : : : : : : +- BroadcastExchange (46) + : : : : : : : : : : : : : +- * ColumnarToRow (45) + : : : : : : : : : : : : : +- CometFilter (44) + : : : : : : : : : : : : : +- CometScan parquet spark_catalog.default.customer (43) + : : : : : : : : : : : : +- BroadcastExchange (52) + : : : : : : : : : : : : +- * ColumnarToRow (51) + : : : : : : : : : : : : +- CometFilter (50) + : : : : : : : : : : : : +- CometScan parquet spark_catalog.default.date_dim (49) + : : : : : : : : : : : +- ReusedExchange (55) + : : : : : : : : : : +- BroadcastExchange (61) + : : : : : : : : : : +- * ColumnarToRow (60) + : : : : : : : : : : +- CometFilter (59) + : : : : : : : : : : +- CometScan parquet spark_catalog.default.customer_demographics (58) + : : : : : : : : : +- ReusedExchange (64) + : : : : : : : : +- BroadcastExchange (70) + : : : : : : : : +- * ColumnarToRow (69) + : : : : : : : : +- CometFilter (68) + : : : : : : : : +- CometScan parquet spark_catalog.default.promotion (67) + : : : : : : : +- BroadcastExchange (76) + : : : : : : : +- * ColumnarToRow (75) + : : : : : : : +- CometFilter (74) + : : : : : : : +- CometScan parquet spark_catalog.default.household_demographics (73) + : : : : : : +- ReusedExchange (79) + : : : : : +- BroadcastExchange (85) + : : : : : +- * ColumnarToRow (84) + : : : : : +- CometFilter (83) + : : : : : +- CometScan parquet spark_catalog.default.customer_address (82) + : : : : +- ReusedExchange (88) + : : : +- BroadcastExchange (94) + : : : +- * ColumnarToRow (93) + : : : +- CometFilter (92) + : : : +- CometScan parquet spark_catalog.default.income_band (91) + : : +- ReusedExchange (97) + : +- BroadcastExchange (104) + : +- * ColumnarToRow (103) + : +- CometProject (102) + : +- CometFilter (101) + : +- CometScan parquet spark_catalog.default.item (100) + +- * Sort (177) + +- Exchange (176) + +- * HashAggregate (175) + +- * HashAggregate (174) + +- * Project (173) + +- * BroadcastHashJoin Inner BuildRight (172) + :- * Project (170) + : +- * BroadcastHashJoin Inner BuildRight (169) + : :- * Project (167) + : : +- * BroadcastHashJoin Inner BuildRight (166) + : : :- * Project (164) + : : : +- * BroadcastHashJoin Inner BuildRight (163) + : : : :- * Project (161) + : : : : +- * BroadcastHashJoin Inner BuildRight (160) + : : : : :- * Project (158) + : : : : : +- * BroadcastHashJoin Inner BuildRight (157) + : : : : : :- * Project (155) + : : : : : : +- * BroadcastHashJoin Inner BuildRight (154) + : : : : : : :- * Project (152) + : : : : : : : +- * BroadcastHashJoin Inner BuildRight (151) + : : : : : : : :- * Project (149) + : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (148) + : : : : : : : : :- * Project (146) + : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (145) + : : : : : : : : : :- * Project (143) + : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (142) + : : : : : : : : : : :- * Project (140) + : : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (139) + : : : : : : : : : : : :- * Project (137) + : : : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (136) + : : : : : : : : : : : : :- * Project (134) + : : : : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (133) + : : : : : : : : : : : : : :- * Project (131) + : : : : : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (130) + : : : : : : : : : : : : : : :- * Project (128) + : : : : : : : : : : : : : : : +- * SortMergeJoin Inner (127) + : : : : : : : : : : : : : : : :- * Sort (121) + : : : : : : : : : : : : : : : : +- Exchange (120) + : : : : : : : : : : : : : : : : +- * ColumnarToRow (119) + : : : : : : : : : : : : : : : : +- CometProject (118) + : : : : : : : : : : : : : : : : +- CometBroadcastHashJoin (117) + : : : : : : : : : : : : : : : : :- CometBroadcastExchange (113) + : : : : : : : : : : : : : : : : : +- CometFilter (112) + : : : : : : : : : : : : : : : : : +- CometScan parquet spark_catalog.default.store_sales (111) + : : : : : : : : : : : : : : : : +- CometProject (116) + : : : : : : : : : : : : : : : : +- CometFilter (115) + : : : : : : : : : : : : : : : : +- CometScan parquet spark_catalog.default.store_returns (114) + : : : : : : : : : : : : : : : +- * Sort (126) + : : : : : : : : : : : : : : : +- * Project (125) + : : : : : : : : : : : : : : : +- * Filter (124) + : : : : : : : : : : : : : : : +- * HashAggregate (123) + : : : : : : : : : : : : : : : +- ReusedExchange (122) + : : : : : : : : : : : : : : +- ReusedExchange (129) + : : : : : : : : : : : : : +- ReusedExchange (132) + : : : : : : : : : : : : +- ReusedExchange (135) + : : : : : : : : : : : +- ReusedExchange (138) + : : : : : : : : : : +- ReusedExchange (141) + : : : : : : : : : +- ReusedExchange (144) + : : : : : : : : +- ReusedExchange (147) + : : : : : : : +- ReusedExchange (150) + : : : : : : +- ReusedExchange (153) + : : : : : +- ReusedExchange (156) + : : : : +- ReusedExchange (159) + : : : +- ReusedExchange (162) + : : +- ReusedExchange (165) + : +- ReusedExchange (168) + +- ReusedExchange (171) + + +(1) Scan parquet spark_catalog.default.store_sales +Output [12]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_ticket_number#8, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] +Batched: true +Location: InMemoryFileIndex [] +PartitionFilters: [isnotnull(ss_sold_date_sk#12), dynamicpruningexpression(ss_sold_date_sk#12 IN dynamicpruning#13)] +PushedFilters: [IsNotNull(ss_item_sk), IsNotNull(ss_ticket_number), IsNotNull(ss_store_sk), IsNotNull(ss_customer_sk), IsNotNull(ss_cdemo_sk), IsNotNull(ss_promo_sk), IsNotNull(ss_hdemo_sk), IsNotNull(ss_addr_sk)] +ReadSchema: struct<ss_item_sk:int,ss_customer_sk:int,ss_cdemo_sk:int,ss_hdemo_sk:int,ss_addr_sk:int,ss_store_sk:int,ss_promo_sk:int,ss_ticket_number:int,ss_wholesale_cost:decimal(7,2),ss_list_price:decimal(7,2),ss_coupon_amt:decimal(7,2)> + +(2) CometFilter +Input [12]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_ticket_number#8, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] +Condition : (((((((isnotnull(ss_item_sk#1) AND isnotnull(ss_ticket_number#8)) AND isnotnull(ss_store_sk#6)) AND isnotnull(ss_customer_sk#2)) AND isnotnull(ss_cdemo_sk#3)) AND isnotnull(ss_promo_sk#7)) AND isnotnull(ss_hdemo_sk#4)) AND isnotnull(ss_addr_sk#5)) + +(3) CometBroadcastExchange +Input [12]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_ticket_number#8, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] +Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_ticket_number#8, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] + +(4) Scan parquet spark_catalog.default.store_returns +Output [3]: [sr_item_sk#14, sr_ticket_number#15, sr_returned_date_sk#16] +Batched: true +Location [not included in comparison]/{warehouse_dir}/store_returns] +PushedFilters: [IsNotNull(sr_item_sk), IsNotNull(sr_ticket_number)] +ReadSchema: struct<sr_item_sk:int,sr_ticket_number:int> + +(5) CometFilter +Input [3]: [sr_item_sk#14, sr_ticket_number#15, sr_returned_date_sk#16] +Condition : (isnotnull(sr_item_sk#14) AND isnotnull(sr_ticket_number#15)) + +(6) CometProject +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] + +(7) CometBroadcastHashJoin +Left output [12]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_ticket_number#8, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] +Right output [2]: [sr_item_sk#14, sr_ticket_number#15] +Arguments: [ss_item_sk#1, ss_ticket_number#8], [sr_item_sk#14, sr_ticket_number#15], Inner + +(8) CometProject +Input [14]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_ticket_number#8, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12, sr_item_sk#14, sr_ticket_number#15] +Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12], [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] + +(9) ColumnarToRow [codegen id : 1] +Input [11]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] + +(10) Exchange +Input [11]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] +Arguments: hashpartitioning(ss_item_sk#1, 5), ENSURE_REQUIREMENTS, [plan_id=1] + +(11) Sort [codegen id : 2] +Input [11]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] +Arguments: [ss_item_sk#1 ASC NULLS FIRST], false, 0 + +(12) Scan parquet spark_catalog.default.catalog_sales +Output [4]: [cs_item_sk#17, cs_order_number#18, cs_ext_list_price#19, cs_sold_date_sk#20] +Batched: true +Location [not included in comparison]/{warehouse_dir}/catalog_sales] +PushedFilters: [IsNotNull(cs_item_sk), IsNotNull(cs_order_number)] +ReadSchema: struct<cs_item_sk:int,cs_order_number:int,cs_ext_list_price:decimal(7,2)> + +(13) CometFilter +Input [4]: [cs_item_sk#17, cs_order_number#18, cs_ext_list_price#19, cs_sold_date_sk#20] +Condition : (isnotnull(cs_item_sk#17) AND isnotnull(cs_order_number#18)) + +(14) CometProject +Input [4]: [cs_item_sk#17, cs_order_number#18, cs_ext_list_price#19, cs_sold_date_sk#20] +Arguments: [cs_item_sk#17, cs_order_number#18, cs_ext_list_price#19], [cs_item_sk#17, cs_order_number#18, cs_ext_list_price#19] + +(15) ColumnarToRow [codegen id : 3] +Input [3]: [cs_item_sk#17, cs_order_number#18, cs_ext_list_price#19] + +(16) Exchange +Input [3]: [cs_item_sk#17, cs_order_number#18, cs_ext_list_price#19] +Arguments: hashpartitioning(cs_item_sk#17, cs_order_number#18, 5), ENSURE_REQUIREMENTS, [plan_id=2] + +(17) Sort [codegen id : 4] +Input [3]: [cs_item_sk#17, cs_order_number#18, cs_ext_list_price#19] +Arguments: [cs_item_sk#17 ASC NULLS FIRST, cs_order_number#18 ASC NULLS FIRST], false, 0 + +(18) Scan parquet spark_catalog.default.catalog_returns +Output [6]: [cr_item_sk#21, cr_order_number#22, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25, cr_returned_date_sk#26] +Batched: true +Location [not included in comparison]/{warehouse_dir}/catalog_returns] +PushedFilters: [IsNotNull(cr_item_sk), IsNotNull(cr_order_number)] +ReadSchema: struct<cr_item_sk:int,cr_order_number:int,cr_refunded_cash:decimal(7,2),cr_reversed_charge:decimal(7,2),cr_store_credit:decimal(7,2)> + +(19) CometFilter +Input [6]: [cr_item_sk#21, cr_order_number#22, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25, cr_returned_date_sk#26] +Condition : (isnotnull(cr_item_sk#21) AND isnotnull(cr_order_number#22)) + +(20) CometProject +Input [6]: [cr_item_sk#21, cr_order_number#22, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25, cr_returned_date_sk#26] +Arguments: [cr_item_sk#21, cr_order_number#22, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25], [cr_item_sk#21, cr_order_number#22, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25] + +(21) ColumnarToRow [codegen id : 5] +Input [5]: [cr_item_sk#21, cr_order_number#22, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25] + +(22) Exchange +Input [5]: [cr_item_sk#21, cr_order_number#22, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25] +Arguments: hashpartitioning(cr_item_sk#21, cr_order_number#22, 5), ENSURE_REQUIREMENTS, [plan_id=3] + +(23) Sort [codegen id : 6] +Input [5]: [cr_item_sk#21, cr_order_number#22, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25] +Arguments: [cr_item_sk#21 ASC NULLS FIRST, cr_order_number#22 ASC NULLS FIRST], false, 0 + +(24) SortMergeJoin [codegen id : 7] +Left keys [2]: [cs_item_sk#17, cs_order_number#18] +Right keys [2]: [cr_item_sk#21, cr_order_number#22] +Join type: Inner +Join condition: None + +(25) Project [codegen id : 7] +Output [5]: [cs_item_sk#17, cs_ext_list_price#19, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25] +Input [8]: [cs_item_sk#17, cs_order_number#18, cs_ext_list_price#19, cr_item_sk#21, cr_order_number#22, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25] + +(26) HashAggregate [codegen id : 7] +Input [5]: [cs_item_sk#17, cs_ext_list_price#19, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25] +Keys [1]: [cs_item_sk#17] +Functions [2]: [partial_sum(UnscaledValue(cs_ext_list_price#19)), partial_sum(((cr_refunded_cash#23 + cr_reversed_charge#24) + cr_store_credit#25))] +Aggregate Attributes [3]: [sum#27, sum#28, isEmpty#29] +Results [4]: [cs_item_sk#17, sum#30, sum#31, isEmpty#32] + +(27) Exchange +Input [4]: [cs_item_sk#17, sum#30, sum#31, isEmpty#32] +Arguments: hashpartitioning(cs_item_sk#17, 5), ENSURE_REQUIREMENTS, [plan_id=4] + +(28) HashAggregate [codegen id : 8] +Input [4]: [cs_item_sk#17, sum#30, sum#31, isEmpty#32] +Keys [1]: [cs_item_sk#17] +Functions [2]: [sum(UnscaledValue(cs_ext_list_price#19)), sum(((cr_refunded_cash#23 + cr_reversed_charge#24) + cr_store_credit#25))] +Aggregate Attributes [2]: [sum(UnscaledValue(cs_ext_list_price#19))#33, sum(((cr_refunded_cash#23 + cr_reversed_charge#24) + cr_store_credit#25))#34] +Results [3]: [cs_item_sk#17, MakeDecimal(sum(UnscaledValue(cs_ext_list_price#19))#33,17,2) AS sale#35, sum(((cr_refunded_cash#23 + cr_reversed_charge#24) + cr_store_credit#25))#34 AS refund#36] + +(29) Filter [codegen id : 8] +Input [3]: [cs_item_sk#17, sale#35, refund#36] +Condition : ((isnotnull(sale#35) AND isnotnull(refund#36)) AND (cast(sale#35 as decimal(21,2)) > (2 * refund#36))) + +(30) Project [codegen id : 8] +Output [1]: [cs_item_sk#17] +Input [3]: [cs_item_sk#17, sale#35, refund#36] + +(31) Sort [codegen id : 8] +Input [1]: [cs_item_sk#17] +Arguments: [cs_item_sk#17 ASC NULLS FIRST], false, 0 + +(32) SortMergeJoin [codegen id : 24] +Left keys [1]: [ss_item_sk#1] +Right keys [1]: [cs_item_sk#17] +Join type: Inner +Join condition: None + +(33) Project [codegen id : 24] +Output [11]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] +Input [12]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12, cs_item_sk#17] + +(34) ReusedExchange [Reuses operator id: 185] +Output [2]: [d_date_sk#37, d_year#38] + +(35) BroadcastHashJoin [codegen id : 24] +Left keys [1]: [ss_sold_date_sk#12] +Right keys [1]: [d_date_sk#37] +Join type: Inner +Join condition: None + +(36) Project [codegen id : 24] +Output [11]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#38] +Input [13]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12, d_date_sk#37, d_year#38] + +(37) Scan parquet spark_catalog.default.store +Output [3]: [s_store_sk#39, s_store_name#40, s_zip#41] +Batched: true +Location [not included in comparison]/{warehouse_dir}/store] +PushedFilters: [IsNotNull(s_store_sk), IsNotNull(s_store_name), IsNotNull(s_zip)] +ReadSchema: struct<s_store_sk:int,s_store_name:string,s_zip:string> + +(38) CometFilter +Input [3]: [s_store_sk#39, s_store_name#40, s_zip#41] +Condition : ((isnotnull(s_store_sk#39) AND isnotnull(s_store_name#40)) AND isnotnull(s_zip#41)) + +(39) ColumnarToRow [codegen id : 10] +Input [3]: [s_store_sk#39, s_store_name#40, s_zip#41] + +(40) BroadcastExchange +Input [3]: [s_store_sk#39, s_store_name#40, s_zip#41] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=5] + +(41) BroadcastHashJoin [codegen id : 24] +Left keys [1]: [ss_store_sk#6] +Right keys [1]: [s_store_sk#39] +Join type: Inner +Join condition: None + +(42) Project [codegen id : 24] +Output [12]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#38, s_store_name#40, s_zip#41] +Input [14]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#38, s_store_sk#39, s_store_name#40, s_zip#41] + +(43) Scan parquet spark_catalog.default.customer +Output [6]: [c_customer_sk#42, c_current_cdemo_sk#43, c_current_hdemo_sk#44, c_current_addr_sk#45, c_first_shipto_date_sk#46, c_first_sales_date_sk#47] +Batched: true +Location [not included in comparison]/{warehouse_dir}/customer] +PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_first_sales_date_sk), IsNotNull(c_first_shipto_date_sk), IsNotNull(c_current_cdemo_sk), IsNotNull(c_current_hdemo_sk), IsNotNull(c_current_addr_sk)] +ReadSchema: struct<c_customer_sk:int,c_current_cdemo_sk:int,c_current_hdemo_sk:int,c_current_addr_sk:int,c_first_shipto_date_sk:int,c_first_sales_date_sk:int> + +(44) CometFilter +Input [6]: [c_customer_sk#42, c_current_cdemo_sk#43, c_current_hdemo_sk#44, c_current_addr_sk#45, c_first_shipto_date_sk#46, c_first_sales_date_sk#47] +Condition : (((((isnotnull(c_customer_sk#42) AND isnotnull(c_first_sales_date_sk#47)) AND isnotnull(c_first_shipto_date_sk#46)) AND isnotnull(c_current_cdemo_sk#43)) AND isnotnull(c_current_hdemo_sk#44)) AND isnotnull(c_current_addr_sk#45)) + +(45) ColumnarToRow [codegen id : 11] +Input [6]: [c_customer_sk#42, c_current_cdemo_sk#43, c_current_hdemo_sk#44, c_current_addr_sk#45, c_first_shipto_date_sk#46, c_first_sales_date_sk#47] + +(46) BroadcastExchange +Input [6]: [c_customer_sk#42, c_current_cdemo_sk#43, c_current_hdemo_sk#44, c_current_addr_sk#45, c_first_shipto_date_sk#46, c_first_sales_date_sk#47] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=6] + +(47) BroadcastHashJoin [codegen id : 24] +Left keys [1]: [ss_customer_sk#2] +Right keys [1]: [c_customer_sk#42] +Join type: Inner +Join condition: None + +(48) Project [codegen id : 24] +Output [16]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#38, s_store_name#40, s_zip#41, c_current_cdemo_sk#43, c_current_hdemo_sk#44, c_current_addr_sk#45, c_first_shipto_date_sk#46, c_first_sales_date_sk#47] +Input [18]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#38, s_store_name#40, s_zip#41, c_customer_sk#42, c_current_cdemo_sk#43, c_current_hdemo_sk#44, c_current_addr_sk#45, c_first_shipto_date_sk#46, c_first_sales_date_sk#47] + +(49) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#48, d_year#49] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_date_sk)] +ReadSchema: struct<d_date_sk:int,d_year:int> + +(50) CometFilter +Input [2]: [d_date_sk#48, d_year#49] +Condition : isnotnull(d_date_sk#48) + +(51) ColumnarToRow [codegen id : 12] +Input [2]: [d_date_sk#48, d_year#49] + +(52) BroadcastExchange +Input [2]: [d_date_sk#48, d_year#49] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=7] + +(53) BroadcastHashJoin [codegen id : 24] +Left keys [1]: [c_first_sales_date_sk#47] +Right keys [1]: [d_date_sk#48] +Join type: Inner +Join condition: None + +(54) Project [codegen id : 24] +Output [16]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#38, s_store_name#40, s_zip#41, c_current_cdemo_sk#43, c_current_hdemo_sk#44, c_current_addr_sk#45, c_first_shipto_date_sk#46, d_year#49] +Input [18]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#38, s_store_name#40, s_zip#41, c_current_cdemo_sk#43, c_current_hdemo_sk#44, c_current_addr_sk#45, c_first_shipto_date_sk#46, c_first_sales_date_sk#47, d_date_sk#48, d_year#49] + +(55) ReusedExchange [Reuses operator id: 52] +Output [2]: [d_date_sk#50, d_year#51] + +(56) BroadcastHashJoin [codegen id : 24] +Left keys [1]: [c_first_shipto_date_sk#46] +Right keys [1]: [d_date_sk#50] +Join type: Inner +Join condition: None + +(57) Project [codegen id : 24] +Output [16]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#38, s_store_name#40, s_zip#41, c_current_cdemo_sk#43, c_current_hdemo_sk#44, c_current_addr_sk#45, d_year#49, d_year#51] +Input [18]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#38, s_store_name#40, s_zip#41, c_current_cdemo_sk#43, c_current_hdemo_sk#44, c_current_addr_sk#45, c_first_shipto_date_sk#46, d_year#49, d_date_sk#50, d_year#51] + +(58) Scan parquet spark_catalog.default.customer_demographics +Output [2]: [cd_demo_sk#52, cd_marital_status#53] +Batched: true +Location [not included in comparison]/{warehouse_dir}/customer_demographics] +PushedFilters: [IsNotNull(cd_demo_sk), IsNotNull(cd_marital_status)] +ReadSchema: struct<cd_demo_sk:int,cd_marital_status:string> + +(59) CometFilter +Input [2]: [cd_demo_sk#52, cd_marital_status#53] +Condition : (isnotnull(cd_demo_sk#52) AND isnotnull(cd_marital_status#53)) + +(60) ColumnarToRow [codegen id : 14] +Input [2]: [cd_demo_sk#52, cd_marital_status#53] + +(61) BroadcastExchange +Input [2]: [cd_demo_sk#52, cd_marital_status#53] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=8] + +(62) BroadcastHashJoin [codegen id : 24] +Left keys [1]: [ss_cdemo_sk#3] +Right keys [1]: [cd_demo_sk#52] +Join type: Inner +Join condition: None + +(63) Project [codegen id : 24] +Output [16]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#38, s_store_name#40, s_zip#41, c_current_cdemo_sk#43, c_current_hdemo_sk#44, c_current_addr_sk#45, d_year#49, d_year#51, cd_marital_status#53] +Input [18]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#38, s_store_name#40, s_zip#41, c_current_cdemo_sk#43, c_current_hdemo_sk#44, c_current_addr_sk#45, d_year#49, d_year#51, cd_demo_sk#52, cd_marital_status#53] + +(64) ReusedExchange [Reuses operator id: 61] +Output [2]: [cd_demo_sk#54, cd_marital_status#55] + +(65) BroadcastHashJoin [codegen id : 24] +Left keys [1]: [c_current_cdemo_sk#43] +Right keys [1]: [cd_demo_sk#54] +Join type: Inner +Join condition: NOT (cd_marital_status#53 = cd_marital_status#55) + +(66) Project [codegen id : 24] +Output [14]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#38, s_store_name#40, s_zip#41, c_current_hdemo_sk#44, c_current_addr_sk#45, d_year#49, d_year#51] +Input [18]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#38, s_store_name#40, s_zip#41, c_current_cdemo_sk#43, c_current_hdemo_sk#44, c_current_addr_sk#45, d_year#49, d_year#51, cd_marital_status#53, cd_demo_sk#54, cd_marital_status#55] + +(67) Scan parquet spark_catalog.default.promotion +Output [1]: [p_promo_sk#56] +Batched: true +Location [not included in comparison]/{warehouse_dir}/promotion] +PushedFilters: [IsNotNull(p_promo_sk)] +ReadSchema: struct<p_promo_sk:int> + +(68) CometFilter +Input [1]: [p_promo_sk#56] +Condition : isnotnull(p_promo_sk#56) + +(69) ColumnarToRow [codegen id : 16] +Input [1]: [p_promo_sk#56] + +(70) BroadcastExchange +Input [1]: [p_promo_sk#56] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=9] + +(71) BroadcastHashJoin [codegen id : 24] +Left keys [1]: [ss_promo_sk#7] +Right keys [1]: [p_promo_sk#56] +Join type: Inner +Join condition: None + +(72) Project [codegen id : 24] +Output [13]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#38, s_store_name#40, s_zip#41, c_current_hdemo_sk#44, c_current_addr_sk#45, d_year#49, d_year#51] +Input [15]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#38, s_store_name#40, s_zip#41, c_current_hdemo_sk#44, c_current_addr_sk#45, d_year#49, d_year#51, p_promo_sk#56] + +(73) Scan parquet spark_catalog.default.household_demographics +Output [2]: [hd_demo_sk#57, hd_income_band_sk#58] +Batched: true +Location [not included in comparison]/{warehouse_dir}/household_demographics] +PushedFilters: [IsNotNull(hd_demo_sk), IsNotNull(hd_income_band_sk)] +ReadSchema: struct<hd_demo_sk:int,hd_income_band_sk:int> + +(74) CometFilter +Input [2]: [hd_demo_sk#57, hd_income_band_sk#58] +Condition : (isnotnull(hd_demo_sk#57) AND isnotnull(hd_income_band_sk#58)) + +(75) ColumnarToRow [codegen id : 17] +Input [2]: [hd_demo_sk#57, hd_income_band_sk#58] + +(76) BroadcastExchange +Input [2]: [hd_demo_sk#57, hd_income_band_sk#58] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=10] + +(77) BroadcastHashJoin [codegen id : 24] +Left keys [1]: [ss_hdemo_sk#4] +Right keys [1]: [hd_demo_sk#57] +Join type: Inner +Join condition: None + +(78) Project [codegen id : 24] +Output [13]: [ss_item_sk#1, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#38, s_store_name#40, s_zip#41, c_current_hdemo_sk#44, c_current_addr_sk#45, d_year#49, d_year#51, hd_income_band_sk#58] +Input [15]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#38, s_store_name#40, s_zip#41, c_current_hdemo_sk#44, c_current_addr_sk#45, d_year#49, d_year#51, hd_demo_sk#57, hd_income_band_sk#58] + +(79) ReusedExchange [Reuses operator id: 76] +Output [2]: [hd_demo_sk#59, hd_income_band_sk#60] + +(80) BroadcastHashJoin [codegen id : 24] +Left keys [1]: [c_current_hdemo_sk#44] +Right keys [1]: [hd_demo_sk#59] +Join type: Inner +Join condition: None + +(81) Project [codegen id : 24] +Output [13]: [ss_item_sk#1, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#38, s_store_name#40, s_zip#41, c_current_addr_sk#45, d_year#49, d_year#51, hd_income_band_sk#58, hd_income_band_sk#60] +Input [15]: [ss_item_sk#1, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#38, s_store_name#40, s_zip#41, c_current_hdemo_sk#44, c_current_addr_sk#45, d_year#49, d_year#51, hd_income_band_sk#58, hd_demo_sk#59, hd_income_band_sk#60] + +(82) Scan parquet spark_catalog.default.customer_address +Output [5]: [ca_address_sk#61, ca_street_number#62, ca_street_name#63, ca_city#64, ca_zip#65] +Batched: true +Location [not included in comparison]/{warehouse_dir}/customer_address] +PushedFilters: [IsNotNull(ca_address_sk)] +ReadSchema: struct<ca_address_sk:int,ca_street_number:string,ca_street_name:string,ca_city:string,ca_zip:string> + +(83) CometFilter +Input [5]: [ca_address_sk#61, ca_street_number#62, ca_street_name#63, ca_city#64, ca_zip#65] +Condition : isnotnull(ca_address_sk#61) + +(84) ColumnarToRow [codegen id : 19] +Input [5]: [ca_address_sk#61, ca_street_number#62, ca_street_name#63, ca_city#64, ca_zip#65] + +(85) BroadcastExchange +Input [5]: [ca_address_sk#61, ca_street_number#62, ca_street_name#63, ca_city#64, ca_zip#65] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=11] + +(86) BroadcastHashJoin [codegen id : 24] +Left keys [1]: [ss_addr_sk#5] +Right keys [1]: [ca_address_sk#61] +Join type: Inner +Join condition: None + +(87) Project [codegen id : 24] +Output [16]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#38, s_store_name#40, s_zip#41, c_current_addr_sk#45, d_year#49, d_year#51, hd_income_band_sk#58, hd_income_band_sk#60, ca_street_number#62, ca_street_name#63, ca_city#64, ca_zip#65] +Input [18]: [ss_item_sk#1, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#38, s_store_name#40, s_zip#41, c_current_addr_sk#45, d_year#49, d_year#51, hd_income_band_sk#58, hd_income_band_sk#60, ca_address_sk#61, ca_street_number#62, ca_street_name#63, ca_city#64, ca_zip#65] + +(88) ReusedExchange [Reuses operator id: 85] +Output [5]: [ca_address_sk#66, ca_street_number#67, ca_street_name#68, ca_city#69, ca_zip#70] + +(89) BroadcastHashJoin [codegen id : 24] +Left keys [1]: [c_current_addr_sk#45] +Right keys [1]: [ca_address_sk#66] +Join type: Inner +Join condition: None + +(90) Project [codegen id : 24] +Output [19]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#38, s_store_name#40, s_zip#41, d_year#49, d_year#51, hd_income_band_sk#58, hd_income_band_sk#60, ca_street_number#62, ca_street_name#63, ca_city#64, ca_zip#65, ca_street_number#67, ca_street_name#68, ca_city#69, ca_zip#70] +Input [21]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#38, s_store_name#40, s_zip#41, c_current_addr_sk#45, d_year#49, d_year#51, hd_income_band_sk#58, hd_income_band_sk#60, ca_street_number#62, ca_street_name#63, ca_city#64, ca_zip#65, ca_address_sk#66, ca_street_number#67, ca_street_name#68, ca_city#69, ca_zip#70] + +(91) Scan parquet spark_catalog.default.income_band +Output [1]: [ib_income_band_sk#71] +Batched: true +Location [not included in comparison]/{warehouse_dir}/income_band] +PushedFilters: [IsNotNull(ib_income_band_sk)] +ReadSchema: struct<ib_income_band_sk:int> + +(92) CometFilter +Input [1]: [ib_income_band_sk#71] +Condition : isnotnull(ib_income_band_sk#71) + +(93) ColumnarToRow [codegen id : 21] +Input [1]: [ib_income_band_sk#71] + +(94) BroadcastExchange +Input [1]: [ib_income_band_sk#71] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=12] + +(95) BroadcastHashJoin [codegen id : 24] +Left keys [1]: [hd_income_band_sk#58] +Right keys [1]: [ib_income_band_sk#71] +Join type: Inner +Join condition: None + +(96) Project [codegen id : 24] +Output [18]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#38, s_store_name#40, s_zip#41, d_year#49, d_year#51, hd_income_band_sk#60, ca_street_number#62, ca_street_name#63, ca_city#64, ca_zip#65, ca_street_number#67, ca_street_name#68, ca_city#69, ca_zip#70] +Input [20]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#38, s_store_name#40, s_zip#41, d_year#49, d_year#51, hd_income_band_sk#58, hd_income_band_sk#60, ca_street_number#62, ca_street_name#63, ca_city#64, ca_zip#65, ca_street_number#67, ca_street_name#68, ca_city#69, ca_zip#70, ib_income_band_sk#71] + +(97) ReusedExchange [Reuses operator id: 94] +Output [1]: [ib_income_band_sk#72] + +(98) BroadcastHashJoin [codegen id : 24] +Left keys [1]: [hd_income_band_sk#60] +Right keys [1]: [ib_income_band_sk#72] +Join type: Inner +Join condition: None + +(99) Project [codegen id : 24] +Output [17]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#38, s_store_name#40, s_zip#41, d_year#49, d_year#51, ca_street_number#62, ca_street_name#63, ca_city#64, ca_zip#65, ca_street_number#67, ca_street_name#68, ca_city#69, ca_zip#70] +Input [19]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#38, s_store_name#40, s_zip#41, d_year#49, d_year#51, hd_income_band_sk#60, ca_street_number#62, ca_street_name#63, ca_city#64, ca_zip#65, ca_street_number#67, ca_street_name#68, ca_city#69, ca_zip#70, ib_income_band_sk#72] + +(100) Scan parquet spark_catalog.default.item +Output [4]: [i_item_sk#73, i_current_price#74, i_color#75, i_product_name#76] +Batched: true +Location [not included in comparison]/{warehouse_dir}/item] +PushedFilters: [IsNotNull(i_current_price), In(i_color, [burlywood ,floral ,indian ,medium ,purple ,spring ]), GreaterThanOrEqual(i_current_price,64.00), LessThanOrEqual(i_current_price,74.00), GreaterThanOrEqual(i_current_price,65.00), LessThanOrEqual(i_current_price,79.00), IsNotNull(i_item_sk)] +ReadSchema: struct<i_item_sk:int,i_current_price:decimal(7,2),i_color:string,i_product_name:string> + +(101) CometFilter +Input [4]: [i_item_sk#73, i_current_price#74, i_color#75, i_product_name#76] +Condition : ((((((isnotnull(i_current_price#74) AND i_color#75 IN (purple ,burlywood ,indian ,spring ,floral ,medium )) AND (i_current_price#74 >= 64.00)) AND (i_current_price#74 <= 74.00)) AND (i_current_price#74 >= 65.00)) AND (i_current_price#74 <= 79.00)) AND isnotnull(i_item_sk#73)) + +(102) CometProject +Input [4]: [i_item_sk#73, i_current_price#74, i_color#75, i_product_name#76] +Arguments: [i_item_sk#73, i_product_name#76], [i_item_sk#73, i_product_name#76] + +(103) ColumnarToRow [codegen id : 23] +Input [2]: [i_item_sk#73, i_product_name#76] + +(104) BroadcastExchange +Input [2]: [i_item_sk#73, i_product_name#76] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=13] + +(105) BroadcastHashJoin [codegen id : 24] +Left keys [1]: [ss_item_sk#1] +Right keys [1]: [i_item_sk#73] +Join type: Inner +Join condition: None + +(106) Project [codegen id : 24] +Output [18]: [ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#38, d_year#49, d_year#51, s_store_name#40, s_zip#41, ca_street_number#62, ca_street_name#63, ca_city#64, ca_zip#65, ca_street_number#67, ca_street_name#68, ca_city#69, ca_zip#70, i_item_sk#73, i_product_name#76] +Input [19]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#38, s_store_name#40, s_zip#41, d_year#49, d_year#51, ca_street_number#62, ca_street_name#63, ca_city#64, ca_zip#65, ca_street_number#67, ca_street_name#68, ca_city#69, ca_zip#70, i_item_sk#73, i_product_name#76] + +(107) HashAggregate [codegen id : 24] +Input [18]: [ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#38, d_year#49, d_year#51, s_store_name#40, s_zip#41, ca_street_number#62, ca_street_name#63, ca_city#64, ca_zip#65, ca_street_number#67, ca_street_name#68, ca_city#69, ca_zip#70, i_item_sk#73, i_product_name#76] +Keys [15]: [i_product_name#76, i_item_sk#73, s_store_name#40, s_zip#41, ca_street_number#62, ca_street_name#63, ca_city#64, ca_zip#65, ca_street_number#67, ca_street_name#68, ca_city#69, ca_zip#70, d_year#38, d_year#49, d_year#51] +Functions [4]: [partial_count(1), partial_sum(UnscaledValue(ss_wholesale_cost#9)), partial_sum(UnscaledValue(ss_list_price#10)), partial_sum(UnscaledValue(ss_coupon_amt#11))] +Aggregate Attributes [4]: [count#77, sum#78, sum#79, sum#80] +Results [19]: [i_product_name#76, i_item_sk#73, s_store_name#40, s_zip#41, ca_street_number#62, ca_street_name#63, ca_city#64, ca_zip#65, ca_street_number#67, ca_street_name#68, ca_city#69, ca_zip#70, d_year#38, d_year#49, d_year#51, count#81, sum#82, sum#83, sum#84] + +(108) HashAggregate [codegen id : 24] +Input [19]: [i_product_name#76, i_item_sk#73, s_store_name#40, s_zip#41, ca_street_number#62, ca_street_name#63, ca_city#64, ca_zip#65, ca_street_number#67, ca_street_name#68, ca_city#69, ca_zip#70, d_year#38, d_year#49, d_year#51, count#81, sum#82, sum#83, sum#84] +Keys [15]: [i_product_name#76, i_item_sk#73, s_store_name#40, s_zip#41, ca_street_number#62, ca_street_name#63, ca_city#64, ca_zip#65, ca_street_number#67, ca_street_name#68, ca_city#69, ca_zip#70, d_year#38, d_year#49, d_year#51] +Functions [4]: [count(1), sum(UnscaledValue(ss_wholesale_cost#9)), sum(UnscaledValue(ss_list_price#10)), sum(UnscaledValue(ss_coupon_amt#11))] +Aggregate Attributes [4]: [count(1)#85, sum(UnscaledValue(ss_wholesale_cost#9))#86, sum(UnscaledValue(ss_list_price#10))#87, sum(UnscaledValue(ss_coupon_amt#11))#88] +Results [17]: [i_product_name#76 AS product_name#89, i_item_sk#73 AS item_sk#90, s_store_name#40 AS store_name#91, s_zip#41 AS store_zip#92, ca_street_number#62 AS b_street_number#93, ca_street_name#63 AS b_streen_name#94, ca_city#64 AS b_city#95, ca_zip#65 AS b_zip#96, ca_street_number#67 AS c_street_number#97, ca_street_name#68 AS c_street_name#98, ca_city#69 AS c_city#99, ca_zip#70 AS c_zip#100, d_year#38 AS syear#101, count(1)#85 AS cnt#102, MakeDecimal(sum(UnscaledValue(ss_wholesale_cost#9))#86,17,2) AS s1#103, MakeDecimal(sum(UnscaledValue(ss_list_price#10))#87,17,2) AS s2#104, MakeDecimal(sum(UnscaledValue(ss_coupon_amt#11))#88,17,2) AS s3#105] + +(109) Exchange +Input [17]: [product_name#89, item_sk#90, store_name#91, store_zip#92, b_street_number#93, b_streen_name#94, b_city#95, b_zip#96, c_street_number#97, c_street_name#98, c_city#99, c_zip#100, syear#101, cnt#102, s1#103, s2#104, s3#105] +Arguments: hashpartitioning(item_sk#90, store_name#91, store_zip#92, 5), ENSURE_REQUIREMENTS, [plan_id=14] + +(110) Sort [codegen id : 25] +Input [17]: [product_name#89, item_sk#90, store_name#91, store_zip#92, b_street_number#93, b_streen_name#94, b_city#95, b_zip#96, c_street_number#97, c_street_name#98, c_city#99, c_zip#100, syear#101, cnt#102, s1#103, s2#104, s3#105] +Arguments: [item_sk#90 ASC NULLS FIRST, store_name#91 ASC NULLS FIRST, store_zip#92 ASC NULLS FIRST], false, 0 + +(111) Scan parquet spark_catalog.default.store_sales +Output [12]: [ss_item_sk#106, ss_customer_sk#107, ss_cdemo_sk#108, ss_hdemo_sk#109, ss_addr_sk#110, ss_store_sk#111, ss_promo_sk#112, ss_ticket_number#113, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, ss_sold_date_sk#117] +Batched: true +Location: InMemoryFileIndex [] +PartitionFilters: [isnotnull(ss_sold_date_sk#117), dynamicpruningexpression(ss_sold_date_sk#117 IN dynamicpruning#118)] +PushedFilters: [IsNotNull(ss_item_sk), IsNotNull(ss_ticket_number), IsNotNull(ss_store_sk), IsNotNull(ss_customer_sk), IsNotNull(ss_cdemo_sk), IsNotNull(ss_promo_sk), IsNotNull(ss_hdemo_sk), IsNotNull(ss_addr_sk)] +ReadSchema: struct<ss_item_sk:int,ss_customer_sk:int,ss_cdemo_sk:int,ss_hdemo_sk:int,ss_addr_sk:int,ss_store_sk:int,ss_promo_sk:int,ss_ticket_number:int,ss_wholesale_cost:decimal(7,2),ss_list_price:decimal(7,2),ss_coupon_amt:decimal(7,2)> + +(112) CometFilter +Input [12]: [ss_item_sk#106, ss_customer_sk#107, ss_cdemo_sk#108, ss_hdemo_sk#109, ss_addr_sk#110, ss_store_sk#111, ss_promo_sk#112, ss_ticket_number#113, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, ss_sold_date_sk#117] +Condition : (((((((isnotnull(ss_item_sk#106) AND isnotnull(ss_ticket_number#113)) AND isnotnull(ss_store_sk#111)) AND isnotnull(ss_customer_sk#107)) AND isnotnull(ss_cdemo_sk#108)) AND isnotnull(ss_promo_sk#112)) AND isnotnull(ss_hdemo_sk#109)) AND isnotnull(ss_addr_sk#110)) + +(113) CometBroadcastExchange +Input [12]: [ss_item_sk#106, ss_customer_sk#107, ss_cdemo_sk#108, ss_hdemo_sk#109, ss_addr_sk#110, ss_store_sk#111, ss_promo_sk#112, ss_ticket_number#113, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, ss_sold_date_sk#117] +Arguments: [ss_item_sk#106, ss_customer_sk#107, ss_cdemo_sk#108, ss_hdemo_sk#109, ss_addr_sk#110, ss_store_sk#111, ss_promo_sk#112, ss_ticket_number#113, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, ss_sold_date_sk#117] + +(114) Scan parquet spark_catalog.default.store_returns +Output [3]: [sr_item_sk#119, sr_ticket_number#120, sr_returned_date_sk#121] +Batched: true +Location [not included in comparison]/{warehouse_dir}/store_returns] +PushedFilters: [IsNotNull(sr_item_sk), IsNotNull(sr_ticket_number)] +ReadSchema: struct<sr_item_sk:int,sr_ticket_number:int> + +(115) CometFilter +Input [3]: [sr_item_sk#119, sr_ticket_number#120, sr_returned_date_sk#121] +Condition : (isnotnull(sr_item_sk#119) AND isnotnull(sr_ticket_number#120)) + +(116) CometProject +Input [3]: [sr_item_sk#119, sr_ticket_number#120, sr_returned_date_sk#121] +Arguments: [sr_item_sk#119, sr_ticket_number#120], [sr_item_sk#119, sr_ticket_number#120] + +(117) CometBroadcastHashJoin +Left output [12]: [ss_item_sk#106, ss_customer_sk#107, ss_cdemo_sk#108, ss_hdemo_sk#109, ss_addr_sk#110, ss_store_sk#111, ss_promo_sk#112, ss_ticket_number#113, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, ss_sold_date_sk#117] +Right output [2]: [sr_item_sk#119, sr_ticket_number#120] +Arguments: [ss_item_sk#106, ss_ticket_number#113], [sr_item_sk#119, sr_ticket_number#120], Inner + +(118) CometProject +Input [14]: [ss_item_sk#106, ss_customer_sk#107, ss_cdemo_sk#108, ss_hdemo_sk#109, ss_addr_sk#110, ss_store_sk#111, ss_promo_sk#112, ss_ticket_number#113, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, ss_sold_date_sk#117, sr_item_sk#119, sr_ticket_number#120] +Arguments: [ss_item_sk#106, ss_customer_sk#107, ss_cdemo_sk#108, ss_hdemo_sk#109, ss_addr_sk#110, ss_store_sk#111, ss_promo_sk#112, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, ss_sold_date_sk#117], [ss_item_sk#106, ss_customer_sk#107, ss_cdemo_sk#108, ss_hdemo_sk#109, ss_addr_sk#110, ss_store_sk#111, ss_promo_sk#112, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, ss_sold_date_sk#117] + +(119) ColumnarToRow [codegen id : 26] +Input [11]: [ss_item_sk#106, ss_customer_sk#107, ss_cdemo_sk#108, ss_hdemo_sk#109, ss_addr_sk#110, ss_store_sk#111, ss_promo_sk#112, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, ss_sold_date_sk#117] + +(120) Exchange +Input [11]: [ss_item_sk#106, ss_customer_sk#107, ss_cdemo_sk#108, ss_hdemo_sk#109, ss_addr_sk#110, ss_store_sk#111, ss_promo_sk#112, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, ss_sold_date_sk#117] +Arguments: hashpartitioning(ss_item_sk#106, 5), ENSURE_REQUIREMENTS, [plan_id=15] + +(121) Sort [codegen id : 27] +Input [11]: [ss_item_sk#106, ss_customer_sk#107, ss_cdemo_sk#108, ss_hdemo_sk#109, ss_addr_sk#110, ss_store_sk#111, ss_promo_sk#112, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, ss_sold_date_sk#117] +Arguments: [ss_item_sk#106 ASC NULLS FIRST], false, 0 + +(122) ReusedExchange [Reuses operator id: 27] +Output [4]: [cs_item_sk#122, sum#123, sum#124, isEmpty#125] + +(123) HashAggregate [codegen id : 33] +Input [4]: [cs_item_sk#122, sum#123, sum#124, isEmpty#125] +Keys [1]: [cs_item_sk#122] +Functions [2]: [sum(UnscaledValue(cs_ext_list_price#126)), sum(((cr_refunded_cash#127 + cr_reversed_charge#128) + cr_store_credit#129))] +Aggregate Attributes [2]: [sum(UnscaledValue(cs_ext_list_price#126))#33, sum(((cr_refunded_cash#127 + cr_reversed_charge#128) + cr_store_credit#129))#34] +Results [3]: [cs_item_sk#122, MakeDecimal(sum(UnscaledValue(cs_ext_list_price#126))#33,17,2) AS sale#130, sum(((cr_refunded_cash#127 + cr_reversed_charge#128) + cr_store_credit#129))#34 AS refund#131] + +(124) Filter [codegen id : 33] +Input [3]: [cs_item_sk#122, sale#130, refund#131] +Condition : ((isnotnull(sale#130) AND isnotnull(refund#131)) AND (cast(sale#130 as decimal(21,2)) > (2 * refund#131))) + +(125) Project [codegen id : 33] +Output [1]: [cs_item_sk#122] +Input [3]: [cs_item_sk#122, sale#130, refund#131] + +(126) Sort [codegen id : 33] +Input [1]: [cs_item_sk#122] +Arguments: [cs_item_sk#122 ASC NULLS FIRST], false, 0 + +(127) SortMergeJoin [codegen id : 49] +Left keys [1]: [ss_item_sk#106] +Right keys [1]: [cs_item_sk#122] +Join type: Inner +Join condition: None + +(128) Project [codegen id : 49] +Output [11]: [ss_item_sk#106, ss_customer_sk#107, ss_cdemo_sk#108, ss_hdemo_sk#109, ss_addr_sk#110, ss_store_sk#111, ss_promo_sk#112, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, ss_sold_date_sk#117] +Input [12]: [ss_item_sk#106, ss_customer_sk#107, ss_cdemo_sk#108, ss_hdemo_sk#109, ss_addr_sk#110, ss_store_sk#111, ss_promo_sk#112, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, ss_sold_date_sk#117, cs_item_sk#122] + +(129) ReusedExchange [Reuses operator id: 189] +Output [2]: [d_date_sk#132, d_year#133] + +(130) BroadcastHashJoin [codegen id : 49] +Left keys [1]: [ss_sold_date_sk#117] +Right keys [1]: [d_date_sk#132] +Join type: Inner +Join condition: None + +(131) Project [codegen id : 49] +Output [11]: [ss_item_sk#106, ss_customer_sk#107, ss_cdemo_sk#108, ss_hdemo_sk#109, ss_addr_sk#110, ss_store_sk#111, ss_promo_sk#112, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#133] +Input [13]: [ss_item_sk#106, ss_customer_sk#107, ss_cdemo_sk#108, ss_hdemo_sk#109, ss_addr_sk#110, ss_store_sk#111, ss_promo_sk#112, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, ss_sold_date_sk#117, d_date_sk#132, d_year#133] + +(132) ReusedExchange [Reuses operator id: 40] +Output [3]: [s_store_sk#134, s_store_name#135, s_zip#136] + +(133) BroadcastHashJoin [codegen id : 49] +Left keys [1]: [ss_store_sk#111] +Right keys [1]: [s_store_sk#134] +Join type: Inner +Join condition: None + +(134) Project [codegen id : 49] +Output [12]: [ss_item_sk#106, ss_customer_sk#107, ss_cdemo_sk#108, ss_hdemo_sk#109, ss_addr_sk#110, ss_promo_sk#112, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#133, s_store_name#135, s_zip#136] +Input [14]: [ss_item_sk#106, ss_customer_sk#107, ss_cdemo_sk#108, ss_hdemo_sk#109, ss_addr_sk#110, ss_store_sk#111, ss_promo_sk#112, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#133, s_store_sk#134, s_store_name#135, s_zip#136] + +(135) ReusedExchange [Reuses operator id: 46] +Output [6]: [c_customer_sk#137, c_current_cdemo_sk#138, c_current_hdemo_sk#139, c_current_addr_sk#140, c_first_shipto_date_sk#141, c_first_sales_date_sk#142] + +(136) BroadcastHashJoin [codegen id : 49] +Left keys [1]: [ss_customer_sk#107] +Right keys [1]: [c_customer_sk#137] +Join type: Inner +Join condition: None + +(137) Project [codegen id : 49] +Output [16]: [ss_item_sk#106, ss_cdemo_sk#108, ss_hdemo_sk#109, ss_addr_sk#110, ss_promo_sk#112, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#133, s_store_name#135, s_zip#136, c_current_cdemo_sk#138, c_current_hdemo_sk#139, c_current_addr_sk#140, c_first_shipto_date_sk#141, c_first_sales_date_sk#142] +Input [18]: [ss_item_sk#106, ss_customer_sk#107, ss_cdemo_sk#108, ss_hdemo_sk#109, ss_addr_sk#110, ss_promo_sk#112, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#133, s_store_name#135, s_zip#136, c_customer_sk#137, c_current_cdemo_sk#138, c_current_hdemo_sk#139, c_current_addr_sk#140, c_first_shipto_date_sk#141, c_first_sales_date_sk#142] + +(138) ReusedExchange [Reuses operator id: 52] +Output [2]: [d_date_sk#143, d_year#144] + +(139) BroadcastHashJoin [codegen id : 49] +Left keys [1]: [c_first_sales_date_sk#142] +Right keys [1]: [d_date_sk#143] +Join type: Inner +Join condition: None + +(140) Project [codegen id : 49] +Output [16]: [ss_item_sk#106, ss_cdemo_sk#108, ss_hdemo_sk#109, ss_addr_sk#110, ss_promo_sk#112, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#133, s_store_name#135, s_zip#136, c_current_cdemo_sk#138, c_current_hdemo_sk#139, c_current_addr_sk#140, c_first_shipto_date_sk#141, d_year#144] +Input [18]: [ss_item_sk#106, ss_cdemo_sk#108, ss_hdemo_sk#109, ss_addr_sk#110, ss_promo_sk#112, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#133, s_store_name#135, s_zip#136, c_current_cdemo_sk#138, c_current_hdemo_sk#139, c_current_addr_sk#140, c_first_shipto_date_sk#141, c_first_sales_date_sk#142, d_date_sk#143, d_year#144] + +(141) ReusedExchange [Reuses operator id: 52] +Output [2]: [d_date_sk#145, d_year#146] + +(142) BroadcastHashJoin [codegen id : 49] +Left keys [1]: [c_first_shipto_date_sk#141] +Right keys [1]: [d_date_sk#145] +Join type: Inner +Join condition: None + +(143) Project [codegen id : 49] +Output [16]: [ss_item_sk#106, ss_cdemo_sk#108, ss_hdemo_sk#109, ss_addr_sk#110, ss_promo_sk#112, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#133, s_store_name#135, s_zip#136, c_current_cdemo_sk#138, c_current_hdemo_sk#139, c_current_addr_sk#140, d_year#144, d_year#146] +Input [18]: [ss_item_sk#106, ss_cdemo_sk#108, ss_hdemo_sk#109, ss_addr_sk#110, ss_promo_sk#112, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#133, s_store_name#135, s_zip#136, c_current_cdemo_sk#138, c_current_hdemo_sk#139, c_current_addr_sk#140, c_first_shipto_date_sk#141, d_year#144, d_date_sk#145, d_year#146] + +(144) ReusedExchange [Reuses operator id: 61] +Output [2]: [cd_demo_sk#147, cd_marital_status#148] + +(145) BroadcastHashJoin [codegen id : 49] +Left keys [1]: [ss_cdemo_sk#108] +Right keys [1]: [cd_demo_sk#147] +Join type: Inner +Join condition: None + +(146) Project [codegen id : 49] +Output [16]: [ss_item_sk#106, ss_hdemo_sk#109, ss_addr_sk#110, ss_promo_sk#112, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#133, s_store_name#135, s_zip#136, c_current_cdemo_sk#138, c_current_hdemo_sk#139, c_current_addr_sk#140, d_year#144, d_year#146, cd_marital_status#148] +Input [18]: [ss_item_sk#106, ss_cdemo_sk#108, ss_hdemo_sk#109, ss_addr_sk#110, ss_promo_sk#112, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#133, s_store_name#135, s_zip#136, c_current_cdemo_sk#138, c_current_hdemo_sk#139, c_current_addr_sk#140, d_year#144, d_year#146, cd_demo_sk#147, cd_marital_status#148] + +(147) ReusedExchange [Reuses operator id: 61] +Output [2]: [cd_demo_sk#149, cd_marital_status#150] + +(148) BroadcastHashJoin [codegen id : 49] +Left keys [1]: [c_current_cdemo_sk#138] +Right keys [1]: [cd_demo_sk#149] +Join type: Inner +Join condition: NOT (cd_marital_status#148 = cd_marital_status#150) + +(149) Project [codegen id : 49] +Output [14]: [ss_item_sk#106, ss_hdemo_sk#109, ss_addr_sk#110, ss_promo_sk#112, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#133, s_store_name#135, s_zip#136, c_current_hdemo_sk#139, c_current_addr_sk#140, d_year#144, d_year#146] +Input [18]: [ss_item_sk#106, ss_hdemo_sk#109, ss_addr_sk#110, ss_promo_sk#112, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#133, s_store_name#135, s_zip#136, c_current_cdemo_sk#138, c_current_hdemo_sk#139, c_current_addr_sk#140, d_year#144, d_year#146, cd_marital_status#148, cd_demo_sk#149, cd_marital_status#150] + +(150) ReusedExchange [Reuses operator id: 70] +Output [1]: [p_promo_sk#151] + +(151) BroadcastHashJoin [codegen id : 49] +Left keys [1]: [ss_promo_sk#112] +Right keys [1]: [p_promo_sk#151] +Join type: Inner +Join condition: None + +(152) Project [codegen id : 49] +Output [13]: [ss_item_sk#106, ss_hdemo_sk#109, ss_addr_sk#110, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#133, s_store_name#135, s_zip#136, c_current_hdemo_sk#139, c_current_addr_sk#140, d_year#144, d_year#146] +Input [15]: [ss_item_sk#106, ss_hdemo_sk#109, ss_addr_sk#110, ss_promo_sk#112, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#133, s_store_name#135, s_zip#136, c_current_hdemo_sk#139, c_current_addr_sk#140, d_year#144, d_year#146, p_promo_sk#151] + +(153) ReusedExchange [Reuses operator id: 76] +Output [2]: [hd_demo_sk#152, hd_income_band_sk#153] + +(154) BroadcastHashJoin [codegen id : 49] +Left keys [1]: [ss_hdemo_sk#109] +Right keys [1]: [hd_demo_sk#152] +Join type: Inner +Join condition: None + +(155) Project [codegen id : 49] +Output [13]: [ss_item_sk#106, ss_addr_sk#110, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#133, s_store_name#135, s_zip#136, c_current_hdemo_sk#139, c_current_addr_sk#140, d_year#144, d_year#146, hd_income_band_sk#153] +Input [15]: [ss_item_sk#106, ss_hdemo_sk#109, ss_addr_sk#110, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#133, s_store_name#135, s_zip#136, c_current_hdemo_sk#139, c_current_addr_sk#140, d_year#144, d_year#146, hd_demo_sk#152, hd_income_band_sk#153] + +(156) ReusedExchange [Reuses operator id: 76] +Output [2]: [hd_demo_sk#154, hd_income_band_sk#155] + +(157) BroadcastHashJoin [codegen id : 49] +Left keys [1]: [c_current_hdemo_sk#139] +Right keys [1]: [hd_demo_sk#154] +Join type: Inner +Join condition: None + +(158) Project [codegen id : 49] +Output [13]: [ss_item_sk#106, ss_addr_sk#110, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#133, s_store_name#135, s_zip#136, c_current_addr_sk#140, d_year#144, d_year#146, hd_income_band_sk#153, hd_income_band_sk#155] +Input [15]: [ss_item_sk#106, ss_addr_sk#110, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#133, s_store_name#135, s_zip#136, c_current_hdemo_sk#139, c_current_addr_sk#140, d_year#144, d_year#146, hd_income_band_sk#153, hd_demo_sk#154, hd_income_band_sk#155] + +(159) ReusedExchange [Reuses operator id: 85] +Output [5]: [ca_address_sk#156, ca_street_number#157, ca_street_name#158, ca_city#159, ca_zip#160] + +(160) BroadcastHashJoin [codegen id : 49] +Left keys [1]: [ss_addr_sk#110] +Right keys [1]: [ca_address_sk#156] +Join type: Inner +Join condition: None + +(161) Project [codegen id : 49] +Output [16]: [ss_item_sk#106, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#133, s_store_name#135, s_zip#136, c_current_addr_sk#140, d_year#144, d_year#146, hd_income_band_sk#153, hd_income_band_sk#155, ca_street_number#157, ca_street_name#158, ca_city#159, ca_zip#160] +Input [18]: [ss_item_sk#106, ss_addr_sk#110, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#133, s_store_name#135, s_zip#136, c_current_addr_sk#140, d_year#144, d_year#146, hd_income_band_sk#153, hd_income_band_sk#155, ca_address_sk#156, ca_street_number#157, ca_street_name#158, ca_city#159, ca_zip#160] + +(162) ReusedExchange [Reuses operator id: 85] +Output [5]: [ca_address_sk#161, ca_street_number#162, ca_street_name#163, ca_city#164, ca_zip#165] + +(163) BroadcastHashJoin [codegen id : 49] +Left keys [1]: [c_current_addr_sk#140] +Right keys [1]: [ca_address_sk#161] +Join type: Inner +Join condition: None + +(164) Project [codegen id : 49] +Output [19]: [ss_item_sk#106, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#133, s_store_name#135, s_zip#136, d_year#144, d_year#146, hd_income_band_sk#153, hd_income_band_sk#155, ca_street_number#157, ca_street_name#158, ca_city#159, ca_zip#160, ca_street_number#162, ca_street_name#163, ca_city#164, ca_zip#165] +Input [21]: [ss_item_sk#106, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#133, s_store_name#135, s_zip#136, c_current_addr_sk#140, d_year#144, d_year#146, hd_income_band_sk#153, hd_income_band_sk#155, ca_street_number#157, ca_street_name#158, ca_city#159, ca_zip#160, ca_address_sk#161, ca_street_number#162, ca_street_name#163, ca_city#164, ca_zip#165] + +(165) ReusedExchange [Reuses operator id: 94] +Output [1]: [ib_income_band_sk#166] + +(166) BroadcastHashJoin [codegen id : 49] +Left keys [1]: [hd_income_band_sk#153] +Right keys [1]: [ib_income_band_sk#166] +Join type: Inner +Join condition: None + +(167) Project [codegen id : 49] +Output [18]: [ss_item_sk#106, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#133, s_store_name#135, s_zip#136, d_year#144, d_year#146, hd_income_band_sk#155, ca_street_number#157, ca_street_name#158, ca_city#159, ca_zip#160, ca_street_number#162, ca_street_name#163, ca_city#164, ca_zip#165] +Input [20]: [ss_item_sk#106, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#133, s_store_name#135, s_zip#136, d_year#144, d_year#146, hd_income_band_sk#153, hd_income_band_sk#155, ca_street_number#157, ca_street_name#158, ca_city#159, ca_zip#160, ca_street_number#162, ca_street_name#163, ca_city#164, ca_zip#165, ib_income_band_sk#166] + +(168) ReusedExchange [Reuses operator id: 94] +Output [1]: [ib_income_band_sk#167] + +(169) BroadcastHashJoin [codegen id : 49] +Left keys [1]: [hd_income_band_sk#155] +Right keys [1]: [ib_income_band_sk#167] +Join type: Inner +Join condition: None + +(170) Project [codegen id : 49] +Output [17]: [ss_item_sk#106, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#133, s_store_name#135, s_zip#136, d_year#144, d_year#146, ca_street_number#157, ca_street_name#158, ca_city#159, ca_zip#160, ca_street_number#162, ca_street_name#163, ca_city#164, ca_zip#165] +Input [19]: [ss_item_sk#106, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#133, s_store_name#135, s_zip#136, d_year#144, d_year#146, hd_income_band_sk#155, ca_street_number#157, ca_street_name#158, ca_city#159, ca_zip#160, ca_street_number#162, ca_street_name#163, ca_city#164, ca_zip#165, ib_income_band_sk#167] + +(171) ReusedExchange [Reuses operator id: 104] +Output [2]: [i_item_sk#168, i_product_name#169] + +(172) BroadcastHashJoin [codegen id : 49] +Left keys [1]: [ss_item_sk#106] +Right keys [1]: [i_item_sk#168] +Join type: Inner +Join condition: None + +(173) Project [codegen id : 49] +Output [18]: [ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#133, d_year#144, d_year#146, s_store_name#135, s_zip#136, ca_street_number#157, ca_street_name#158, ca_city#159, ca_zip#160, ca_street_number#162, ca_street_name#163, ca_city#164, ca_zip#165, i_item_sk#168, i_product_name#169] +Input [19]: [ss_item_sk#106, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#133, s_store_name#135, s_zip#136, d_year#144, d_year#146, ca_street_number#157, ca_street_name#158, ca_city#159, ca_zip#160, ca_street_number#162, ca_street_name#163, ca_city#164, ca_zip#165, i_item_sk#168, i_product_name#169] + +(174) HashAggregate [codegen id : 49] +Input [18]: [ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#133, d_year#144, d_year#146, s_store_name#135, s_zip#136, ca_street_number#157, ca_street_name#158, ca_city#159, ca_zip#160, ca_street_number#162, ca_street_name#163, ca_city#164, ca_zip#165, i_item_sk#168, i_product_name#169] +Keys [15]: [i_product_name#169, i_item_sk#168, s_store_name#135, s_zip#136, ca_street_number#157, ca_street_name#158, ca_city#159, ca_zip#160, ca_street_number#162, ca_street_name#163, ca_city#164, ca_zip#165, d_year#133, d_year#144, d_year#146] +Functions [4]: [partial_count(1), partial_sum(UnscaledValue(ss_wholesale_cost#114)), partial_sum(UnscaledValue(ss_list_price#115)), partial_sum(UnscaledValue(ss_coupon_amt#116))] +Aggregate Attributes [4]: [count#77, sum#170, sum#171, sum#172] +Results [19]: [i_product_name#169, i_item_sk#168, s_store_name#135, s_zip#136, ca_street_number#157, ca_street_name#158, ca_city#159, ca_zip#160, ca_street_number#162, ca_street_name#163, ca_city#164, ca_zip#165, d_year#133, d_year#144, d_year#146, count#81, sum#173, sum#174, sum#175] + +(175) HashAggregate [codegen id : 49] +Input [19]: [i_product_name#169, i_item_sk#168, s_store_name#135, s_zip#136, ca_street_number#157, ca_street_name#158, ca_city#159, ca_zip#160, ca_street_number#162, ca_street_name#163, ca_city#164, ca_zip#165, d_year#133, d_year#144, d_year#146, count#81, sum#173, sum#174, sum#175] +Keys [15]: [i_product_name#169, i_item_sk#168, s_store_name#135, s_zip#136, ca_street_number#157, ca_street_name#158, ca_city#159, ca_zip#160, ca_street_number#162, ca_street_name#163, ca_city#164, ca_zip#165, d_year#133, d_year#144, d_year#146] +Functions [4]: [count(1), sum(UnscaledValue(ss_wholesale_cost#114)), sum(UnscaledValue(ss_list_price#115)), sum(UnscaledValue(ss_coupon_amt#116))] +Aggregate Attributes [4]: [count(1)#85, sum(UnscaledValue(ss_wholesale_cost#114))#86, sum(UnscaledValue(ss_list_price#115))#87, sum(UnscaledValue(ss_coupon_amt#116))#88] +Results [8]: [i_item_sk#168 AS item_sk#176, s_store_name#135 AS store_name#177, s_zip#136 AS store_zip#178, d_year#133 AS syear#179, count(1)#85 AS cnt#180, MakeDecimal(sum(UnscaledValue(ss_wholesale_cost#114))#86,17,2) AS s1#181, MakeDecimal(sum(UnscaledValue(ss_list_price#115))#87,17,2) AS s2#182, MakeDecimal(sum(UnscaledValue(ss_coupon_amt#116))#88,17,2) AS s3#183] + +(176) Exchange +Input [8]: [item_sk#176, store_name#177, store_zip#178, syear#179, cnt#180, s1#181, s2#182, s3#183] +Arguments: hashpartitioning(item_sk#176, store_name#177, store_zip#178, 5), ENSURE_REQUIREMENTS, [plan_id=16] + +(177) Sort [codegen id : 50] +Input [8]: [item_sk#176, store_name#177, store_zip#178, syear#179, cnt#180, s1#181, s2#182, s3#183] +Arguments: [item_sk#176 ASC NULLS FIRST, store_name#177 ASC NULLS FIRST, store_zip#178 ASC NULLS FIRST], false, 0 + +(178) SortMergeJoin [codegen id : 51] +Left keys [3]: [item_sk#90, store_name#91, store_zip#92] +Right keys [3]: [item_sk#176, store_name#177, store_zip#178] +Join type: Inner +Join condition: (cnt#180 <= cnt#102) + +(179) Project [codegen id : 51] +Output [21]: [product_name#89, store_name#91, store_zip#92, b_street_number#93, b_streen_name#94, b_city#95, b_zip#96, c_street_number#97, c_street_name#98, c_city#99, c_zip#100, syear#101, cnt#102, s1#103, s2#104, s3#105, s1#181, s2#182, s3#183, syear#179, cnt#180] +Input [25]: [product_name#89, item_sk#90, store_name#91, store_zip#92, b_street_number#93, b_streen_name#94, b_city#95, b_zip#96, c_street_number#97, c_street_name#98, c_city#99, c_zip#100, syear#101, cnt#102, s1#103, s2#104, s3#105, item_sk#176, store_name#177, store_zip#178, syear#179, cnt#180, s1#181, s2#182, s3#183] + +(180) Exchange +Input [21]: [product_name#89, store_name#91, store_zip#92, b_street_number#93, b_streen_name#94, b_city#95, b_zip#96, c_street_number#97, c_street_name#98, c_city#99, c_zip#100, syear#101, cnt#102, s1#103, s2#104, s3#105, s1#181, s2#182, s3#183, syear#179, cnt#180] +Arguments: rangepartitioning(product_name#89 ASC NULLS FIRST, store_name#91 ASC NULLS FIRST, cnt#180 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, [plan_id=17] + +(181) Sort [codegen id : 52] +Input [21]: [product_name#89, store_name#91, store_zip#92, b_street_number#93, b_streen_name#94, b_city#95, b_zip#96, c_street_number#97, c_street_name#98, c_city#99, c_zip#100, syear#101, cnt#102, s1#103, s2#104, s3#105, s1#181, s2#182, s3#183, syear#179, cnt#180] +Arguments: [product_name#89 ASC NULLS FIRST, store_name#91 ASC NULLS FIRST, cnt#180 ASC NULLS FIRST], true, 0 + +===== Subqueries ===== + +Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#12 IN dynamicpruning#13 +BroadcastExchange (185) ++- * ColumnarToRow (184) + +- CometFilter (183) + +- CometScan parquet spark_catalog.default.date_dim (182) + + +(182) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#37, d_year#38] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_year), EqualTo(d_year,1999), IsNotNull(d_date_sk)] +ReadSchema: struct<d_date_sk:int,d_year:int> + +(183) CometFilter +Input [2]: [d_date_sk#37, d_year#38] +Condition : ((isnotnull(d_year#38) AND (d_year#38 = 1999)) AND isnotnull(d_date_sk#37)) + +(184) ColumnarToRow [codegen id : 1] +Input [2]: [d_date_sk#37, d_year#38] + +(185) BroadcastExchange +Input [2]: [d_date_sk#37, d_year#38] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=18] + +Subquery:2 Hosting operator id = 111 Hosting Expression = ss_sold_date_sk#117 IN dynamicpruning#118 +BroadcastExchange (189) ++- * ColumnarToRow (188) + +- CometFilter (187) + +- CometScan parquet spark_catalog.default.date_dim (186) + + +(186) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#132, d_year#133] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2000), IsNotNull(d_date_sk)] +ReadSchema: struct<d_date_sk:int,d_year:int> + +(187) CometFilter +Input [2]: [d_date_sk#132, d_year#133] +Condition : ((isnotnull(d_year#133) AND (d_year#133 = 2000)) AND isnotnull(d_date_sk#132)) + +(188) ColumnarToRow [codegen id : 1] +Input [2]: [d_date_sk#132, d_year#133] + +(189) BroadcastExchange +Input [2]: [d_date_sk#132, d_year#133] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=19] + + diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q64/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q64/simplified.txt new file mode 100644 index 000000000..d972e0082 --- /dev/null +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q64/simplified.txt @@ -0,0 +1,281 @@ +WholeStageCodegen (52) + Sort [product_name,store_name,cnt] + InputAdapter + Exchange [product_name,store_name,cnt] #1 + WholeStageCodegen (51) + Project [product_name,store_name,store_zip,b_street_number,b_streen_name,b_city,b_zip,c_street_number,c_street_name,c_city,c_zip,syear,cnt,s1,s2,s3,s1,s2,s3,syear,cnt] + SortMergeJoin [item_sk,store_name,store_zip,item_sk,store_name,store_zip,cnt,cnt] + InputAdapter + WholeStageCodegen (25) + Sort [item_sk,store_name,store_zip] + InputAdapter + Exchange [item_sk,store_name,store_zip] #2 + WholeStageCodegen (24) + HashAggregate [i_product_name,i_item_sk,s_store_name,s_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,d_year,d_year,d_year,count,sum,sum,sum] [count(1),sum(UnscaledValue(ss_wholesale_cost)),sum(UnscaledValue(ss_list_price)),sum(UnscaledValue(ss_coupon_amt)),product_name,item_sk,store_name,store_zip,b_street_number,b_streen_name,b_city,b_zip,c_street_number,c_street_name,c_city,c_zip,syear,cnt,s1,s2,s3,count,sum,sum,sum] + HashAggregate [i_product_name,i_item_sk,s_store_name,s_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,d_year,d_year,d_year,ss_wholesale_cost,ss_list_price,ss_coupon_amt] [count,sum,sum,sum,count,sum,sum,sum] + Project [ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,d_year,d_year,s_store_name,s_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,i_item_sk,i_product_name] + BroadcastHashJoin [ss_item_sk,i_item_sk] + Project [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip] + BroadcastHashJoin [hd_income_band_sk,ib_income_band_sk] + Project [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip] + BroadcastHashJoin [hd_income_band_sk,ib_income_band_sk] + Project [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip] + BroadcastHashJoin [c_current_addr_sk,ca_address_sk] + Project [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip] + BroadcastHashJoin [ss_addr_sk,ca_address_sk] + Project [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk] + BroadcastHashJoin [c_current_hdemo_sk,hd_demo_sk] + Project [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,hd_income_band_sk] + BroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] + Project [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year] + BroadcastHashJoin [ss_promo_sk,p_promo_sk] + Project [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year] + BroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk,cd_marital_status,cd_marital_status] + Project [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,cd_marital_status] + BroadcastHashJoin [ss_cdemo_sk,cd_demo_sk] + Project [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year] + BroadcastHashJoin [c_first_shipto_date_sk,d_date_sk] + Project [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,d_year] + BroadcastHashJoin [c_first_sales_date_sk,d_date_sk] + Project [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk] + BroadcastHashJoin [ss_customer_sk,c_customer_sk] + Project [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip] + BroadcastHashJoin [ss_store_sk,s_store_sk] + Project [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] + SortMergeJoin [ss_item_sk,cs_item_sk] + InputAdapter + WholeStageCodegen (2) + Sort [ss_item_sk] + InputAdapter + Exchange [ss_item_sk] #3 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] + CometBroadcastHashJoin [ss_item_sk,ss_ticket_number,sr_item_sk,sr_ticket_number] + CometBroadcastExchange #4 + CometFilter [ss_item_sk,ss_ticket_number,ss_store_sk,ss_customer_sk,ss_cdemo_sk,ss_promo_sk,ss_hdemo_sk,ss_addr_sk] + CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #5 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometFilter [d_year,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometProject [sr_item_sk,sr_ticket_number] + CometFilter [sr_item_sk,sr_ticket_number] + CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk] + InputAdapter + WholeStageCodegen (8) + Sort [cs_item_sk] + Project [cs_item_sk] + Filter [sale,refund] + HashAggregate [cs_item_sk,sum,sum,isEmpty] [sum(UnscaledValue(cs_ext_list_price)),sum(((cr_refunded_cash + cr_reversed_charge) + cr_store_credit)),sale,refund,sum,sum,isEmpty] + InputAdapter + Exchange [cs_item_sk] #6 + WholeStageCodegen (7) + HashAggregate [cs_item_sk,cs_ext_list_price,cr_refunded_cash,cr_reversed_charge,cr_store_credit] [sum,sum,isEmpty,sum,sum,isEmpty] + Project [cs_item_sk,cs_ext_list_price,cr_refunded_cash,cr_reversed_charge,cr_store_credit] + SortMergeJoin [cs_item_sk,cs_order_number,cr_item_sk,cr_order_number] + InputAdapter + WholeStageCodegen (4) + Sort [cs_item_sk,cs_order_number] + InputAdapter + Exchange [cs_item_sk,cs_order_number] #7 + WholeStageCodegen (3) + ColumnarToRow + InputAdapter + CometProject [cs_item_sk,cs_order_number,cs_ext_list_price] + CometFilter [cs_item_sk,cs_order_number] + CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_order_number,cs_ext_list_price,cs_sold_date_sk] + InputAdapter + WholeStageCodegen (6) + Sort [cr_item_sk,cr_order_number] + InputAdapter + Exchange [cr_item_sk,cr_order_number] #8 + WholeStageCodegen (5) + ColumnarToRow + InputAdapter + CometProject [cr_item_sk,cr_order_number,cr_refunded_cash,cr_reversed_charge,cr_store_credit] + CometFilter [cr_item_sk,cr_order_number] + CometScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_refunded_cash,cr_reversed_charge,cr_store_credit,cr_returned_date_sk] + InputAdapter + ReusedExchange [d_date_sk,d_year] #5 + InputAdapter + BroadcastExchange #9 + WholeStageCodegen (10) + ColumnarToRow + InputAdapter + CometFilter [s_store_sk,s_store_name,s_zip] + CometScan parquet spark_catalog.default.store [s_store_sk,s_store_name,s_zip] + InputAdapter + BroadcastExchange #10 + WholeStageCodegen (11) + ColumnarToRow + InputAdapter + CometFilter [c_customer_sk,c_first_sales_date_sk,c_first_shipto_date_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk] + CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk] + InputAdapter + BroadcastExchange #11 + WholeStageCodegen (12) + ColumnarToRow + InputAdapter + CometFilter [d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + InputAdapter + ReusedExchange [d_date_sk,d_year] #11 + InputAdapter + BroadcastExchange #12 + WholeStageCodegen (14) + ColumnarToRow + InputAdapter + CometFilter [cd_demo_sk,cd_marital_status] + CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_marital_status] + InputAdapter + ReusedExchange [cd_demo_sk,cd_marital_status] #12 + InputAdapter + BroadcastExchange #13 + WholeStageCodegen (16) + ColumnarToRow + InputAdapter + CometFilter [p_promo_sk] + CometScan parquet spark_catalog.default.promotion [p_promo_sk] + InputAdapter + BroadcastExchange #14 + WholeStageCodegen (17) + ColumnarToRow + InputAdapter + CometFilter [hd_demo_sk,hd_income_band_sk] + CometScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_income_band_sk] + InputAdapter + ReusedExchange [hd_demo_sk,hd_income_band_sk] #14 + InputAdapter + BroadcastExchange #15 + WholeStageCodegen (19) + ColumnarToRow + InputAdapter + CometFilter [ca_address_sk] + CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] + InputAdapter + ReusedExchange [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] #15 + InputAdapter + BroadcastExchange #16 + WholeStageCodegen (21) + ColumnarToRow + InputAdapter + CometFilter [ib_income_band_sk] + CometScan parquet spark_catalog.default.income_band [ib_income_band_sk] + InputAdapter + ReusedExchange [ib_income_band_sk] #16 + InputAdapter + BroadcastExchange #17 + WholeStageCodegen (23) + ColumnarToRow + InputAdapter + CometProject [i_item_sk,i_product_name] + CometFilter [i_current_price,i_color,i_item_sk] + CometScan parquet spark_catalog.default.item [i_item_sk,i_current_price,i_color,i_product_name] + InputAdapter + WholeStageCodegen (50) + Sort [item_sk,store_name,store_zip] + InputAdapter + Exchange [item_sk,store_name,store_zip] #18 + WholeStageCodegen (49) + HashAggregate [i_product_name,i_item_sk,s_store_name,s_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,d_year,d_year,d_year,count,sum,sum,sum] [count(1),sum(UnscaledValue(ss_wholesale_cost)),sum(UnscaledValue(ss_list_price)),sum(UnscaledValue(ss_coupon_amt)),item_sk,store_name,store_zip,syear,cnt,s1,s2,s3,count,sum,sum,sum] + HashAggregate [i_product_name,i_item_sk,s_store_name,s_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,d_year,d_year,d_year,ss_wholesale_cost,ss_list_price,ss_coupon_amt] [count,sum,sum,sum,count,sum,sum,sum] + Project [ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,d_year,d_year,s_store_name,s_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,i_item_sk,i_product_name] + BroadcastHashJoin [ss_item_sk,i_item_sk] + Project [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip] + BroadcastHashJoin [hd_income_band_sk,ib_income_band_sk] + Project [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip] + BroadcastHashJoin [hd_income_band_sk,ib_income_band_sk] + Project [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip] + BroadcastHashJoin [c_current_addr_sk,ca_address_sk] + Project [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip] + BroadcastHashJoin [ss_addr_sk,ca_address_sk] + Project [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk] + BroadcastHashJoin [c_current_hdemo_sk,hd_demo_sk] + Project [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,hd_income_band_sk] + BroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] + Project [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year] + BroadcastHashJoin [ss_promo_sk,p_promo_sk] + Project [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year] + BroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk,cd_marital_status,cd_marital_status] + Project [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,cd_marital_status] + BroadcastHashJoin [ss_cdemo_sk,cd_demo_sk] + Project [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year] + BroadcastHashJoin [c_first_shipto_date_sk,d_date_sk] + Project [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,d_year] + BroadcastHashJoin [c_first_sales_date_sk,d_date_sk] + Project [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk] + BroadcastHashJoin [ss_customer_sk,c_customer_sk] + Project [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip] + BroadcastHashJoin [ss_store_sk,s_store_sk] + Project [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] + SortMergeJoin [ss_item_sk,cs_item_sk] + InputAdapter + WholeStageCodegen (27) + Sort [ss_item_sk] + InputAdapter + Exchange [ss_item_sk] #19 + WholeStageCodegen (26) + ColumnarToRow + InputAdapter + CometProject [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] + CometBroadcastHashJoin [ss_item_sk,ss_ticket_number,sr_item_sk,sr_ticket_number] + CometBroadcastExchange #20 + CometFilter [ss_item_sk,ss_ticket_number,ss_store_sk,ss_customer_sk,ss_cdemo_sk,ss_promo_sk,ss_hdemo_sk,ss_addr_sk] + CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #2 + BroadcastExchange #21 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometFilter [d_year,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometProject [sr_item_sk,sr_ticket_number] + CometFilter [sr_item_sk,sr_ticket_number] + CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk] + InputAdapter + WholeStageCodegen (33) + Sort [cs_item_sk] + Project [cs_item_sk] + Filter [sale,refund] + HashAggregate [cs_item_sk,sum,sum,isEmpty] [sum(UnscaledValue(cs_ext_list_price)),sum(((cr_refunded_cash + cr_reversed_charge) + cr_store_credit)),sale,refund,sum,sum,isEmpty] + InputAdapter + ReusedExchange [cs_item_sk,sum,sum,isEmpty] #6 + InputAdapter + ReusedExchange [d_date_sk,d_year] #21 + InputAdapter + ReusedExchange [s_store_sk,s_store_name,s_zip] #9 + InputAdapter + ReusedExchange [c_customer_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk] #10 + InputAdapter + ReusedExchange [d_date_sk,d_year] #11 + InputAdapter + ReusedExchange [d_date_sk,d_year] #11 + InputAdapter + ReusedExchange [cd_demo_sk,cd_marital_status] #12 + InputAdapter + ReusedExchange [cd_demo_sk,cd_marital_status] #12 + InputAdapter + ReusedExchange [p_promo_sk] #13 + InputAdapter + ReusedExchange [hd_demo_sk,hd_income_band_sk] #14 + InputAdapter + ReusedExchange [hd_demo_sk,hd_income_band_sk] #14 + InputAdapter + ReusedExchange [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] #15 + InputAdapter + ReusedExchange [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] #15 + InputAdapter + ReusedExchange [ib_income_band_sk] #16 + InputAdapter + ReusedExchange [ib_income_band_sk] #16 + InputAdapter + ReusedExchange [i_item_sk,i_product_name] #17 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 new file mode 100644 index 000000000..eda84bb52 --- /dev/null +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q65/explain.txt @@ -0,0 +1,269 @@ +== Physical Plan == +TakeOrderedAndProject (39) ++- * Project (38) + +- * BroadcastHashJoin Inner BuildRight (37) + :- * Project (22) + : +- * BroadcastHashJoin Inner BuildRight (21) + : :- * Project (16) + : : +- * BroadcastHashJoin Inner BuildRight (15) + : : :- * ColumnarToRow (3) + : : : +- CometFilter (2) + : : : +- CometScan parquet spark_catalog.default.store (1) + : : +- BroadcastExchange (14) + : : +- * Filter (13) + : : +- * HashAggregate (12) + : : +- Exchange (11) + : : +- * HashAggregate (10) + : : +- * Project (9) + : : +- * BroadcastHashJoin Inner BuildRight (8) + : : :- * ColumnarToRow (6) + : : : +- CometFilter (5) + : : : +- CometScan parquet spark_catalog.default.store_sales (4) + : : +- ReusedExchange (7) + : +- BroadcastExchange (20) + : +- * ColumnarToRow (19) + : +- CometFilter (18) + : +- CometScan parquet spark_catalog.default.item (17) + +- BroadcastExchange (36) + +- * Filter (35) + +- * HashAggregate (34) + +- Exchange (33) + +- * HashAggregate (32) + +- * HashAggregate (31) + +- Exchange (30) + +- * HashAggregate (29) + +- * Project (28) + +- * BroadcastHashJoin Inner BuildRight (27) + :- * ColumnarToRow (25) + : +- CometFilter (24) + : +- CometScan parquet spark_catalog.default.store_sales (23) + +- ReusedExchange (26) + + +(1) Scan parquet spark_catalog.default.store +Output [2]: [s_store_sk#1, s_store_name#2] +Batched: true +Location [not included in comparison]/{warehouse_dir}/store] +PushedFilters: [IsNotNull(s_store_sk)] +ReadSchema: struct<s_store_sk:int,s_store_name:string> + +(2) CometFilter +Input [2]: [s_store_sk#1, s_store_name#2] +Condition : isnotnull(s_store_sk#1) + +(3) ColumnarToRow [codegen id : 9] +Input [2]: [s_store_sk#1, s_store_name#2] + +(4) Scan parquet spark_catalog.default.store_sales +Output [4]: [ss_item_sk#3, ss_store_sk#4, ss_sales_price#5, ss_sold_date_sk#6] +Batched: true +Location: InMemoryFileIndex [] +PartitionFilters: [isnotnull(ss_sold_date_sk#6), dynamicpruningexpression(ss_sold_date_sk#6 IN dynamicpruning#7)] +PushedFilters: [IsNotNull(ss_store_sk), IsNotNull(ss_item_sk)] +ReadSchema: struct<ss_item_sk:int,ss_store_sk:int,ss_sales_price:decimal(7,2)> + +(5) CometFilter +Input [4]: [ss_item_sk#3, ss_store_sk#4, ss_sales_price#5, ss_sold_date_sk#6] +Condition : (isnotnull(ss_store_sk#4) AND isnotnull(ss_item_sk#3)) + +(6) ColumnarToRow [codegen id : 2] +Input [4]: [ss_item_sk#3, ss_store_sk#4, ss_sales_price#5, ss_sold_date_sk#6] + +(7) ReusedExchange [Reuses operator id: 44] +Output [1]: [d_date_sk#8] + +(8) BroadcastHashJoin [codegen id : 2] +Left keys [1]: [ss_sold_date_sk#6] +Right keys [1]: [d_date_sk#8] +Join type: Inner +Join condition: None + +(9) Project [codegen id : 2] +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] +Input [3]: [ss_item_sk#3, ss_store_sk#4, ss_sales_price#5] +Keys [2]: [ss_store_sk#4, ss_item_sk#3] +Functions [1]: [partial_sum(UnscaledValue(ss_sales_price#5))] +Aggregate Attributes [1]: [sum#9] +Results [3]: [ss_store_sk#4, ss_item_sk#3, sum#10] + +(11) Exchange +Input [3]: [ss_store_sk#4, ss_item_sk#3, sum#10] +Arguments: hashpartitioning(ss_store_sk#4, ss_item_sk#3, 5), ENSURE_REQUIREMENTS, [plan_id=1] + +(12) HashAggregate [codegen id : 3] +Input [3]: [ss_store_sk#4, ss_item_sk#3, sum#10] +Keys [2]: [ss_store_sk#4, ss_item_sk#3] +Functions [1]: [sum(UnscaledValue(ss_sales_price#5))] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_sales_price#5))#11] +Results [3]: [ss_store_sk#4, ss_item_sk#3, MakeDecimal(sum(UnscaledValue(ss_sales_price#5))#11,17,2) AS revenue#12] + +(13) Filter [codegen id : 3] +Input [3]: [ss_store_sk#4, ss_item_sk#3, revenue#12] +Condition : isnotnull(revenue#12) + +(14) BroadcastExchange +Input [3]: [ss_store_sk#4, ss_item_sk#3, revenue#12] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] + +(15) BroadcastHashJoin [codegen id : 9] +Left keys [1]: [s_store_sk#1] +Right keys [1]: [ss_store_sk#4] +Join type: Inner +Join condition: None + +(16) Project [codegen id : 9] +Output [4]: [s_store_name#2, ss_store_sk#4, ss_item_sk#3, revenue#12] +Input [5]: [s_store_sk#1, s_store_name#2, ss_store_sk#4, ss_item_sk#3, revenue#12] + +(17) Scan parquet spark_catalog.default.item +Output [5]: [i_item_sk#13, i_item_desc#14, i_current_price#15, i_wholesale_cost#16, i_brand#17] +Batched: true +Location [not included in comparison]/{warehouse_dir}/item] +PushedFilters: [IsNotNull(i_item_sk)] +ReadSchema: struct<i_item_sk:int,i_item_desc:string,i_current_price:decimal(7,2),i_wholesale_cost:decimal(7,2),i_brand:string> + +(18) CometFilter +Input [5]: [i_item_sk#13, i_item_desc#14, i_current_price#15, i_wholesale_cost#16, i_brand#17] +Condition : isnotnull(i_item_sk#13) + +(19) ColumnarToRow [codegen id : 4] +Input [5]: [i_item_sk#13, i_item_desc#14, i_current_price#15, i_wholesale_cost#16, i_brand#17] + +(20) BroadcastExchange +Input [5]: [i_item_sk#13, i_item_desc#14, i_current_price#15, i_wholesale_cost#16, i_brand#17] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=3] + +(21) BroadcastHashJoin [codegen id : 9] +Left keys [1]: [ss_item_sk#3] +Right keys [1]: [i_item_sk#13] +Join type: Inner +Join condition: None + +(22) Project [codegen id : 9] +Output [7]: [s_store_name#2, ss_store_sk#4, revenue#12, i_item_desc#14, i_current_price#15, i_wholesale_cost#16, i_brand#17] +Input [9]: [s_store_name#2, ss_store_sk#4, ss_item_sk#3, revenue#12, i_item_sk#13, i_item_desc#14, i_current_price#15, i_wholesale_cost#16, i_brand#17] + +(23) Scan parquet spark_catalog.default.store_sales +Output [4]: [ss_item_sk#18, ss_store_sk#19, ss_sales_price#20, ss_sold_date_sk#21] +Batched: true +Location: InMemoryFileIndex [] +PartitionFilters: [isnotnull(ss_sold_date_sk#21), dynamicpruningexpression(ss_sold_date_sk#21 IN dynamicpruning#22)] +PushedFilters: [IsNotNull(ss_store_sk)] +ReadSchema: struct<ss_item_sk:int,ss_store_sk:int,ss_sales_price:decimal(7,2)> + +(24) CometFilter +Input [4]: [ss_item_sk#18, ss_store_sk#19, ss_sales_price#20, ss_sold_date_sk#21] +Condition : isnotnull(ss_store_sk#19) + +(25) ColumnarToRow [codegen id : 6] +Input [4]: [ss_item_sk#18, ss_store_sk#19, ss_sales_price#20, ss_sold_date_sk#21] + +(26) ReusedExchange [Reuses operator id: 44] +Output [1]: [d_date_sk#23] + +(27) BroadcastHashJoin [codegen id : 6] +Left keys [1]: [ss_sold_date_sk#21] +Right keys [1]: [d_date_sk#23] +Join type: Inner +Join condition: None + +(28) Project [codegen id : 6] +Output [3]: [ss_item_sk#18, ss_store_sk#19, ss_sales_price#20] +Input [5]: [ss_item_sk#18, ss_store_sk#19, ss_sales_price#20, ss_sold_date_sk#21, d_date_sk#23] + +(29) HashAggregate [codegen id : 6] +Input [3]: [ss_item_sk#18, ss_store_sk#19, ss_sales_price#20] +Keys [2]: [ss_store_sk#19, ss_item_sk#18] +Functions [1]: [partial_sum(UnscaledValue(ss_sales_price#20))] +Aggregate Attributes [1]: [sum#24] +Results [3]: [ss_store_sk#19, ss_item_sk#18, sum#25] + +(30) Exchange +Input [3]: [ss_store_sk#19, ss_item_sk#18, sum#25] +Arguments: hashpartitioning(ss_store_sk#19, ss_item_sk#18, 5), ENSURE_REQUIREMENTS, [plan_id=4] + +(31) HashAggregate [codegen id : 7] +Input [3]: [ss_store_sk#19, ss_item_sk#18, sum#25] +Keys [2]: [ss_store_sk#19, ss_item_sk#18] +Functions [1]: [sum(UnscaledValue(ss_sales_price#20))] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_sales_price#20))#26] +Results [2]: [ss_store_sk#19, MakeDecimal(sum(UnscaledValue(ss_sales_price#20))#26,17,2) AS revenue#27] + +(32) HashAggregate [codegen id : 7] +Input [2]: [ss_store_sk#19, revenue#27] +Keys [1]: [ss_store_sk#19] +Functions [1]: [partial_avg(revenue#27)] +Aggregate Attributes [2]: [sum#28, count#29] +Results [3]: [ss_store_sk#19, sum#30, count#31] + +(33) Exchange +Input [3]: [ss_store_sk#19, sum#30, count#31] +Arguments: hashpartitioning(ss_store_sk#19, 5), ENSURE_REQUIREMENTS, [plan_id=5] + +(34) HashAggregate [codegen id : 8] +Input [3]: [ss_store_sk#19, sum#30, count#31] +Keys [1]: [ss_store_sk#19] +Functions [1]: [avg(revenue#27)] +Aggregate Attributes [1]: [avg(revenue#27)#32] +Results [2]: [ss_store_sk#19, avg(revenue#27)#32 AS ave#33] + +(35) Filter [codegen id : 8] +Input [2]: [ss_store_sk#19, ave#33] +Condition : isnotnull(ave#33) + +(36) BroadcastExchange +Input [2]: [ss_store_sk#19, ave#33] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=6] + +(37) BroadcastHashJoin [codegen id : 9] +Left keys [1]: [ss_store_sk#4] +Right keys [1]: [ss_store_sk#19] +Join type: Inner +Join condition: (cast(revenue#12 as decimal(23,7)) <= (0.1 * ave#33)) + +(38) Project [codegen id : 9] +Output [6]: [s_store_name#2, i_item_desc#14, revenue#12, i_current_price#15, i_wholesale_cost#16, i_brand#17] +Input [9]: [s_store_name#2, ss_store_sk#4, revenue#12, i_item_desc#14, i_current_price#15, i_wholesale_cost#16, i_brand#17, ss_store_sk#19, ave#33] + +(39) TakeOrderedAndProject +Input [6]: [s_store_name#2, i_item_desc#14, revenue#12, i_current_price#15, i_wholesale_cost#16, i_brand#17] +Arguments: 100, [s_store_name#2 ASC NULLS FIRST, i_item_desc#14 ASC NULLS FIRST], [s_store_name#2, i_item_desc#14, revenue#12, i_current_price#15, i_wholesale_cost#16, i_brand#17] + +===== Subqueries ===== + +Subquery:1 Hosting operator id = 4 Hosting Expression = ss_sold_date_sk#6 IN dynamicpruning#7 +BroadcastExchange (44) ++- * ColumnarToRow (43) + +- CometProject (42) + +- CometFilter (41) + +- CometScan parquet spark_catalog.default.date_dim (40) + + +(40) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#8, d_month_seq#34] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1176), LessThanOrEqual(d_month_seq,1187), IsNotNull(d_date_sk)] +ReadSchema: struct<d_date_sk:int,d_month_seq:int> + +(41) CometFilter +Input [2]: [d_date_sk#8, d_month_seq#34] +Condition : (((isnotnull(d_month_seq#34) AND (d_month_seq#34 >= 1176)) AND (d_month_seq#34 <= 1187)) AND isnotnull(d_date_sk#8)) + +(42) CometProject +Input [2]: [d_date_sk#8, d_month_seq#34] +Arguments: [d_date_sk#8], [d_date_sk#8] + +(43) ColumnarToRow [codegen id : 1] +Input [1]: [d_date_sk#8] + +(44) BroadcastExchange +Input [1]: [d_date_sk#8] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=7] + +Subquery:2 Hosting operator id = 23 Hosting Expression = ss_sold_date_sk#21 IN dynamicpruning#7 + + 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 new file mode 100644 index 000000000..33b695e81 --- /dev/null +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q65/simplified.txt @@ -0,0 +1,67 @@ +TakeOrderedAndProject [s_store_name,i_item_desc,revenue,i_current_price,i_wholesale_cost,i_brand] + WholeStageCodegen (9) + Project [s_store_name,i_item_desc,revenue,i_current_price,i_wholesale_cost,i_brand] + BroadcastHashJoin [ss_store_sk,ss_store_sk,revenue,ave] + Project [s_store_name,ss_store_sk,revenue,i_item_desc,i_current_price,i_wholesale_cost,i_brand] + BroadcastHashJoin [ss_item_sk,i_item_sk] + Project [s_store_name,ss_store_sk,ss_item_sk,revenue] + BroadcastHashJoin [s_store_sk,ss_store_sk] + ColumnarToRow + InputAdapter + CometFilter [s_store_sk] + CometScan parquet spark_catalog.default.store [s_store_sk,s_store_name] + InputAdapter + BroadcastExchange #1 + WholeStageCodegen (3) + Filter [revenue] + HashAggregate [ss_store_sk,ss_item_sk,sum] [sum(UnscaledValue(ss_sales_price)),revenue,sum] + InputAdapter + Exchange [ss_store_sk,ss_item_sk] #2 + WholeStageCodegen (2) + HashAggregate [ss_store_sk,ss_item_sk,ss_sales_price] [sum,sum] + Project [ss_item_sk,ss_store_sk,ss_sales_price] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + ColumnarToRow + InputAdapter + CometFilter [ss_store_sk,ss_item_sk] + CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_month_seq,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] + InputAdapter + ReusedExchange [d_date_sk] #3 + InputAdapter + BroadcastExchange #4 + WholeStageCodegen (4) + ColumnarToRow + InputAdapter + CometFilter [i_item_sk] + CometScan parquet spark_catalog.default.item [i_item_sk,i_item_desc,i_current_price,i_wholesale_cost,i_brand] + InputAdapter + BroadcastExchange #5 + WholeStageCodegen (8) + Filter [ave] + HashAggregate [ss_store_sk,sum,count] [avg(revenue),ave,sum,count] + InputAdapter + Exchange [ss_store_sk] #6 + WholeStageCodegen (7) + HashAggregate [ss_store_sk,revenue] [sum,count,sum,count] + HashAggregate [ss_store_sk,ss_item_sk,sum] [sum(UnscaledValue(ss_sales_price)),revenue,sum] + InputAdapter + Exchange [ss_store_sk,ss_item_sk] #7 + WholeStageCodegen (6) + HashAggregate [ss_store_sk,ss_item_sk,ss_sales_price] [sum,sum] + Project [ss_item_sk,ss_store_sk,ss_sales_price] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + ColumnarToRow + InputAdapter + CometFilter [ss_store_sk] + CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + InputAdapter + ReusedExchange [d_date_sk] #3 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 new file mode 100644 index 000000000..a26c457aa --- /dev/null +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q66/explain.txt @@ -0,0 +1,332 @@ +== Physical Plan == +TakeOrderedAndProject (52) ++- * HashAggregate (51) + +- Exchange (50) + +- * HashAggregate (49) + +- Union (48) + :- * HashAggregate (29) + : +- Exchange (28) + : +- * HashAggregate (27) + : +- * Project (26) + : +- * BroadcastHashJoin Inner BuildRight (25) + : :- * Project (19) + : : +- * BroadcastHashJoin Inner BuildRight (18) + : : :- * Project (12) + : : : +- * BroadcastHashJoin Inner BuildRight (11) + : : : :- * Project (9) + : : : : +- * BroadcastHashJoin Inner BuildRight (8) + : : : : :- * ColumnarToRow (3) + : : : : : +- CometFilter (2) + : : : : : +- CometScan parquet spark_catalog.default.web_sales (1) + : : : : +- BroadcastExchange (7) + : : : : +- * ColumnarToRow (6) + : : : : +- CometFilter (5) + : : : : +- CometScan parquet spark_catalog.default.warehouse (4) + : : : +- ReusedExchange (10) + : : +- BroadcastExchange (17) + : : +- * ColumnarToRow (16) + : : +- CometProject (15) + : : +- CometFilter (14) + : : +- CometScan parquet spark_catalog.default.time_dim (13) + : +- BroadcastExchange (24) + : +- * ColumnarToRow (23) + : +- CometProject (22) + : +- CometFilter (21) + : +- CometScan parquet spark_catalog.default.ship_mode (20) + +- * HashAggregate (47) + +- Exchange (46) + +- * HashAggregate (45) + +- * Project (44) + +- * BroadcastHashJoin Inner BuildRight (43) + :- * Project (41) + : +- * BroadcastHashJoin Inner BuildRight (40) + : :- * Project (38) + : : +- * BroadcastHashJoin Inner BuildRight (37) + : : :- * Project (35) + : : : +- * BroadcastHashJoin Inner BuildRight (34) + : : : :- * ColumnarToRow (32) + : : : : +- CometFilter (31) + : : : : +- CometScan parquet spark_catalog.default.catalog_sales (30) + : : : +- ReusedExchange (33) + : : +- ReusedExchange (36) + : +- ReusedExchange (39) + +- ReusedExchange (42) + + +(1) Scan parquet spark_catalog.default.web_sales +Output [7]: [ws_sold_time_sk#1, ws_ship_mode_sk#2, ws_warehouse_sk#3, ws_quantity#4, ws_ext_sales_price#5, ws_net_paid#6, ws_sold_date_sk#7] +Batched: true +Location: InMemoryFileIndex [] +PartitionFilters: [isnotnull(ws_sold_date_sk#7), dynamicpruningexpression(ws_sold_date_sk#7 IN dynamicpruning#8)] +PushedFilters: [IsNotNull(ws_warehouse_sk), IsNotNull(ws_sold_time_sk), IsNotNull(ws_ship_mode_sk)] +ReadSchema: struct<ws_sold_time_sk:int,ws_ship_mode_sk:int,ws_warehouse_sk:int,ws_quantity:int,ws_ext_sales_price:decimal(7,2),ws_net_paid:decimal(7,2)> + +(2) CometFilter +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)) + +(3) ColumnarToRow [codegen id : 5] +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] + +(4) Scan parquet spark_catalog.default.warehouse +Output [7]: [w_warehouse_sk#9, w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15] +Batched: true +Location [not included in comparison]/{warehouse_dir}/warehouse] +PushedFilters: [IsNotNull(w_warehouse_sk)] +ReadSchema: struct<w_warehouse_sk:int,w_warehouse_name:string,w_warehouse_sq_ft:int,w_city:string,w_county:string,w_state:string,w_country:string> + +(5) CometFilter +Input [7]: [w_warehouse_sk#9, w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15] +Condition : isnotnull(w_warehouse_sk#9) + +(6) ColumnarToRow [codegen id : 1] +Input [7]: [w_warehouse_sk#9, w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15] + +(7) BroadcastExchange +Input [7]: [w_warehouse_sk#9, w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=1] + +(8) BroadcastHashJoin [codegen id : 5] +Left keys [1]: [ws_warehouse_sk#3] +Right keys [1]: [w_warehouse_sk#9] +Join type: Inner +Join condition: None + +(9) Project [codegen id : 5] +Output [12]: [ws_sold_time_sk#1, ws_ship_mode_sk#2, ws_quantity#4, ws_ext_sales_price#5, ws_net_paid#6, ws_sold_date_sk#7, w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15] +Input [14]: [ws_sold_time_sk#1, ws_ship_mode_sk#2, ws_warehouse_sk#3, ws_quantity#4, ws_ext_sales_price#5, ws_net_paid#6, ws_sold_date_sk#7, w_warehouse_sk#9, w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15] + +(10) ReusedExchange [Reuses operator id: 56] +Output [3]: [d_date_sk#16, d_year#17, d_moy#18] + +(11) BroadcastHashJoin [codegen id : 5] +Left keys [1]: [ws_sold_date_sk#7] +Right keys [1]: [d_date_sk#16] +Join type: Inner +Join condition: None + +(12) Project [codegen id : 5] +Output [13]: [ws_sold_time_sk#1, ws_ship_mode_sk#2, ws_quantity#4, ws_ext_sales_price#5, ws_net_paid#6, w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15, d_year#17, d_moy#18] +Input [15]: [ws_sold_time_sk#1, ws_ship_mode_sk#2, ws_quantity#4, ws_ext_sales_price#5, ws_net_paid#6, ws_sold_date_sk#7, w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15, d_date_sk#16, d_year#17, d_moy#18] + +(13) Scan parquet spark_catalog.default.time_dim +Output [2]: [t_time_sk#19, t_time#20] +Batched: true +Location [not included in comparison]/{warehouse_dir}/time_dim] +PushedFilters: [IsNotNull(t_time), GreaterThanOrEqual(t_time,30838), LessThanOrEqual(t_time,59638), IsNotNull(t_time_sk)] +ReadSchema: struct<t_time_sk:int,t_time:int> + +(14) CometFilter +Input [2]: [t_time_sk#19, t_time#20] +Condition : (((isnotnull(t_time#20) AND (t_time#20 >= 30838)) AND (t_time#20 <= 59638)) AND isnotnull(t_time_sk#19)) + +(15) CometProject +Input [2]: [t_time_sk#19, t_time#20] +Arguments: [t_time_sk#19], [t_time_sk#19] + +(16) ColumnarToRow [codegen id : 3] +Input [1]: [t_time_sk#19] + +(17) BroadcastExchange +Input [1]: [t_time_sk#19] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] + +(18) BroadcastHashJoin [codegen id : 5] +Left keys [1]: [ws_sold_time_sk#1] +Right keys [1]: [t_time_sk#19] +Join type: Inner +Join condition: None + +(19) Project [codegen id : 5] +Output [12]: [ws_ship_mode_sk#2, ws_quantity#4, ws_ext_sales_price#5, ws_net_paid#6, w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15, d_year#17, d_moy#18] +Input [14]: [ws_sold_time_sk#1, ws_ship_mode_sk#2, ws_quantity#4, ws_ext_sales_price#5, ws_net_paid#6, w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15, d_year#17, d_moy#18, t_time_sk#19] + +(20) Scan parquet spark_catalog.default.ship_mode +Output [2]: [sm_ship_mode_sk#21, sm_carrier#22] +Batched: true +Location [not included in comparison]/{warehouse_dir}/ship_mode] +PushedFilters: [In(sm_carrier, [BARIAN ,DHL ]), IsNotNull(sm_ship_mode_sk)] +ReadSchema: struct<sm_ship_mode_sk:int,sm_carrier:string> + +(21) CometFilter +Input [2]: [sm_ship_mode_sk#21, sm_carrier#22] +Condition : (sm_carrier#22 IN (DHL ,BARIAN ) AND isnotnull(sm_ship_mode_sk#21)) + +(22) CometProject +Input [2]: [sm_ship_mode_sk#21, sm_carrier#22] +Arguments: [sm_ship_mode_sk#21], [sm_ship_mode_sk#21] + +(23) ColumnarToRow [codegen id : 4] +Input [1]: [sm_ship_mode_sk#21] + +(24) BroadcastExchange +Input [1]: [sm_ship_mode_sk#21] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] + +(25) BroadcastHashJoin [codegen id : 5] +Left keys [1]: [ws_ship_mode_sk#2] +Right keys [1]: [sm_ship_mode_sk#21] +Join type: Inner +Join condition: None + +(26) Project [codegen id : 5] +Output [11]: [ws_quantity#4, ws_ext_sales_price#5, ws_net_paid#6, w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15, d_year#17, d_moy#18] +Input [13]: [ws_ship_mode_sk#2, ws_quantity#4, ws_ext_sales_price#5, ws_net_paid#6, w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15, d_year#17, d_moy#18, sm_ship_mode_sk#21] + +(27) HashAggregate [codegen id : 5] +Input [11]: [ws_quantity#4, ws_ext_sales_price#5, ws_net_paid#6, w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15, d_year#17, d_moy#18] +Keys [7]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15, d_year#17] +Functions [24]: [partial_sum(CASE WHEN (d_moy#18 = 1) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#18 = 2) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#18 = 3) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#18 = 4) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#18 = 5) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#18 = 6) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#18 = 7) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#18 = 8) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#18 = 9) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#18 = 10) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#18 = 11) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#18 = 12) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#18 = 1) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#18 = 2) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#18 = 3) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#18 = 4) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#18 = 5) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#18 = 6) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#18 = 7) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#18 = 8) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#18 = 9) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#18 = 10) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#18 = 11) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#18 = 12) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)] +Aggregate Attributes [48]: [sum#23, isEmpty#24, sum#25, isEmpty#26, sum#27, isEmpty#28, sum#29, isEmpty#30, sum#31, isEmpty#32, sum#33, isEmpty#34, sum#35, isEmpty#36, sum#37, isEmpty#38, sum#39, isEmpty#40, sum#41, isEmpty#42, sum#43, isEmpty#44, sum#45, isEmpty#46, sum#47, isEmpty#48, sum#49, isEmpty#50, sum#51, isEmpty#52, sum#53, isEmpty#54, sum#55, isEmpty#56, sum#57, isEmpty#58, sum#59, isEmpty#60, sum#61, isEmpty#62, sum#63, isEmpty#64, sum#65, isEmpty#66, sum#67, isEmpty#68, sum#69, isEmpty#70] +Results [55]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15, d_year#17, sum#71, isEmpty#72, sum#73, isEmpty#74, sum#75, isEmpty#76, sum#77, isEmpty#78, sum#79, isEmpty#80, sum#81, isEmpty#82, sum#83, isEmpty#84, sum#85, isEmpty#86, sum#87, isEmpty#88, sum#89, isEmpty#90, sum#91, isEmpty#92, sum#93, isEmpty#94, sum#95, isEmpty#96, sum#97, isEmpty#98, sum#99, isEmpty#100, sum#101, isEmpty#102, sum#103, isEmpty#104, sum#105, isEmpty#106, sum#107, isEmpty#108, sum#109, isEmpty#110, sum#111, isEmpty#112, sum#113, isEmpty#114, sum#115, isEmpty#116, sum#117, isEmpty#118] + +(28) Exchange +Input [55]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15, d_year#17, sum#71, isEmpty#72, sum#73, isEmpty#74, sum#75, isEmpty#76, sum#77, isEmpty#78, sum#79, isEmpty#80, sum#81, isEmpty#82, sum#83, isEmpty#84, sum#85, isEmpty#86, sum#87, isEmpty#88, sum#89, isEmpty#90, sum#91, isEmpty#92, sum#93, isEmpty#94, sum#95, isEmpty#96, sum#97, isEmpty#98, sum#99, isEmpty#100, sum#101, isEmpty#102, sum#103, isEmpty#104, sum#105, isEmpty#106, sum#107, isEmpty#108, sum#109, isEmpty#110, sum#111, isEmpty#112, sum#113, isEmpty#114, sum#115, isEmpty#116, sum#117, isEmpty#118] +Arguments: hashpartitioning(w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15, d_year#17, 5), ENSURE_REQUIREMENTS, [plan_id=4] + +(29) HashAggregate [codegen id : 6] +Input [55]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15, d_year#17, sum#71, isEmpty#72, sum#73, isEmpty#74, sum#75, isEmpty#76, sum#77, isEmpty#78, sum#79, isEmpty#80, sum#81, isEmpty#82, sum#83, isEmpty#84, sum#85, isEmpty#86, sum#87, isEmpty#88, sum#89, isEmpty#90, sum#91, isEmpty#92, sum#93, isEmpty#94, sum#95, isEmpty#96, sum#97, isEmpty#98, sum#99, isEmpty#100, sum#101, isEmpty#102, sum#103, isEmpty#104, sum#105, isEmpty#106, sum#107, isEmpty#108, sum#109, isEmpty#110, sum#111, isEmpty#112, sum#113, isEmpty#114, sum#115, isEmpty#116, sum#117, isEmpty#118] +Keys [7]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15, d_year#17] +Functions [24]: [sum(CASE WHEN (d_moy#18 = 1) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#18 = 2) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#18 = 3) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#18 = 4) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#18 = 5) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#18 = 6) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#18 = 7) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#18 = 8) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#18 = 9) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#18 = 10) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#18 = 11) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#18 = 12) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#18 = 1) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#18 = 2) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#18 = 3) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#18 = 4) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#18 = 5) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#18 = 6) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#18 = 7) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#18 = 8) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#18 = 9) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#18 = 10) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#18 = 11) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#18 = 12) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)] +Aggregate Attributes [24]: [sum(CASE WHEN (d_moy#18 = 1) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#119, sum(CASE WHEN (d_moy#18 = 2) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#120, sum(CASE WHEN (d_moy#18 = 3) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#121, sum(CASE WHEN (d_moy#18 = 4) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#122, sum(CASE WHEN (d_moy#18 = 5) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#123, sum(CASE WHEN (d_moy#18 = 6) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#124, sum(CASE WHEN (d_moy#18 = 7) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#125, sum(CASE WHEN (d_moy#18 = 8) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#126, sum(CASE WHEN (d_moy#18 = 9) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#127, sum(CASE WHEN (d_moy#18 = 10) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#128, sum(CASE WHEN (d_moy#18 = 11) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#129, sum(CASE WHEN (d_moy#18 = 12) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#130, sum(CASE WHEN (d_moy#18 = 1) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#131, sum(CASE WHEN (d_moy#18 = 2) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#132, sum(CASE WHEN (d_moy#18 = 3) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#133, sum(CASE WHEN (d_moy#18 = 4) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#134, sum(CASE WHEN (d_moy#18 = 5) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#135, sum(CASE WHEN (d_moy#18 = 6) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#136, sum(CASE WHEN (d_moy#18 = 7) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#137, sum(CASE WHEN (d_moy#18 = 8) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#138, sum(CASE WHEN (d_moy#18 = 9) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#139, sum(CASE WHEN (d_moy#18 = 10) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#140, sum(CASE WHEN (d_moy#18 = 11) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#141, sum(CASE WHEN (d_moy#18 = 12) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#142] +Results [32]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15, DHL,BARIAN AS ship_carriers#143, d_year#17 AS year#144, sum(CASE WHEN (d_moy#18 = 1) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#119 AS jan_sales#145, sum(CASE WHEN (d_moy#18 = 2) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#120 AS feb_sales#146, sum(CASE WHEN (d_moy#18 = 3) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#121 AS mar_sales#147, sum(CASE WHEN (d_moy#18 = 4) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#122 AS apr_sales#148, sum(CASE WHEN (d_moy#18 = 5) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#123 AS may_sales#149, sum(CASE WHEN (d_moy#18 = 6) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#124 AS jun_sales#150, sum(CASE WHEN (d_moy#18 = 7) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#125 AS jul_sales#151, sum(CASE WHEN (d_moy#18 = 8) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#126 AS aug_sales#152, sum(CASE WHEN (d_moy#18 = 9) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#127 AS sep_sales#153, sum(CASE WHEN (d_moy#18 = 10) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#128 AS oct_sales#154, sum(CASE WHEN (d_moy#18 = 11) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#129 AS nov_sales#155, sum(CASE WHEN (d_moy#18 = 12) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#130 AS dec_sales#156, sum(CASE WHEN (d_moy#18 = 1) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#131 AS jan_net#157, sum(CASE WHEN (d_moy#18 = 2) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#132 AS feb_net#158, sum(CASE WHEN (d_moy#18 = 3) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#133 AS mar_net#159, sum(CASE WHEN (d_moy#18 = 4) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#134 AS apr_net#160, sum(CASE WHEN (d_moy#18 = 5) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#135 AS may_net#161, sum(CASE WHEN (d_moy#18 = 6) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#136 AS jun_net#162, sum(CASE WHEN (d_moy#18 = 7) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#137 AS jul_net#163, sum(CASE WHEN (d_moy#18 = 8) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#138 AS aug_net#164, sum(CASE WHEN (d_moy#18 = 9) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#139 AS sep_net#165, sum(CASE WHEN (d_moy#18 = 10) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#140 AS oct_net#166, sum(CASE WHEN (d_moy#18 = 11) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#141 AS nov_net#167, sum(CASE WHEN (d_moy#18 = 12) THEN (ws_net_paid#6 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#142 AS dec_net#168] + +(30) Scan parquet spark_catalog.default.catalog_sales +Output [7]: [cs_sold_time_sk#169, cs_ship_mode_sk#170, cs_warehouse_sk#171, cs_quantity#172, cs_sales_price#173, cs_net_paid_inc_tax#174, cs_sold_date_sk#175] +Batched: true +Location: InMemoryFileIndex [] +PartitionFilters: [isnotnull(cs_sold_date_sk#175), dynamicpruningexpression(cs_sold_date_sk#175 IN dynamicpruning#176)] +PushedFilters: [IsNotNull(cs_warehouse_sk), IsNotNull(cs_sold_time_sk), IsNotNull(cs_ship_mode_sk)] +ReadSchema: struct<cs_sold_time_sk:int,cs_ship_mode_sk:int,cs_warehouse_sk:int,cs_quantity:int,cs_sales_price:decimal(7,2),cs_net_paid_inc_tax:decimal(7,2)> + +(31) CometFilter +Input [7]: [cs_sold_time_sk#169, cs_ship_mode_sk#170, cs_warehouse_sk#171, cs_quantity#172, cs_sales_price#173, cs_net_paid_inc_tax#174, cs_sold_date_sk#175] +Condition : ((isnotnull(cs_warehouse_sk#171) AND isnotnull(cs_sold_time_sk#169)) AND isnotnull(cs_ship_mode_sk#170)) + +(32) ColumnarToRow [codegen id : 11] +Input [7]: [cs_sold_time_sk#169, cs_ship_mode_sk#170, cs_warehouse_sk#171, cs_quantity#172, cs_sales_price#173, cs_net_paid_inc_tax#174, cs_sold_date_sk#175] + +(33) ReusedExchange [Reuses operator id: 7] +Output [7]: [w_warehouse_sk#177, w_warehouse_name#178, w_warehouse_sq_ft#179, w_city#180, w_county#181, w_state#182, w_country#183] + +(34) BroadcastHashJoin [codegen id : 11] +Left keys [1]: [cs_warehouse_sk#171] +Right keys [1]: [w_warehouse_sk#177] +Join type: Inner +Join condition: None + +(35) Project [codegen id : 11] +Output [12]: [cs_sold_time_sk#169, cs_ship_mode_sk#170, cs_quantity#172, cs_sales_price#173, cs_net_paid_inc_tax#174, cs_sold_date_sk#175, w_warehouse_name#178, w_warehouse_sq_ft#179, w_city#180, w_county#181, w_state#182, w_country#183] +Input [14]: [cs_sold_time_sk#169, cs_ship_mode_sk#170, cs_warehouse_sk#171, cs_quantity#172, cs_sales_price#173, cs_net_paid_inc_tax#174, cs_sold_date_sk#175, w_warehouse_sk#177, w_warehouse_name#178, w_warehouse_sq_ft#179, w_city#180, w_county#181, w_state#182, w_country#183] + +(36) ReusedExchange [Reuses operator id: 56] +Output [3]: [d_date_sk#184, d_year#185, d_moy#186] + +(37) BroadcastHashJoin [codegen id : 11] +Left keys [1]: [cs_sold_date_sk#175] +Right keys [1]: [d_date_sk#184] +Join type: Inner +Join condition: None + +(38) Project [codegen id : 11] +Output [13]: [cs_sold_time_sk#169, cs_ship_mode_sk#170, cs_quantity#172, cs_sales_price#173, cs_net_paid_inc_tax#174, w_warehouse_name#178, w_warehouse_sq_ft#179, w_city#180, w_county#181, w_state#182, w_country#183, d_year#185, d_moy#186] +Input [15]: [cs_sold_time_sk#169, cs_ship_mode_sk#170, cs_quantity#172, cs_sales_price#173, cs_net_paid_inc_tax#174, cs_sold_date_sk#175, w_warehouse_name#178, w_warehouse_sq_ft#179, w_city#180, w_county#181, w_state#182, w_country#183, d_date_sk#184, d_year#185, d_moy#186] + +(39) ReusedExchange [Reuses operator id: 17] +Output [1]: [t_time_sk#187] + +(40) BroadcastHashJoin [codegen id : 11] +Left keys [1]: [cs_sold_time_sk#169] +Right keys [1]: [t_time_sk#187] +Join type: Inner +Join condition: None + +(41) Project [codegen id : 11] +Output [12]: [cs_ship_mode_sk#170, cs_quantity#172, cs_sales_price#173, cs_net_paid_inc_tax#174, w_warehouse_name#178, w_warehouse_sq_ft#179, w_city#180, w_county#181, w_state#182, w_country#183, d_year#185, d_moy#186] +Input [14]: [cs_sold_time_sk#169, cs_ship_mode_sk#170, cs_quantity#172, cs_sales_price#173, cs_net_paid_inc_tax#174, w_warehouse_name#178, w_warehouse_sq_ft#179, w_city#180, w_county#181, w_state#182, w_country#183, d_year#185, d_moy#186, t_time_sk#187] + +(42) ReusedExchange [Reuses operator id: 24] +Output [1]: [sm_ship_mode_sk#188] + +(43) BroadcastHashJoin [codegen id : 11] +Left keys [1]: [cs_ship_mode_sk#170] +Right keys [1]: [sm_ship_mode_sk#188] +Join type: Inner +Join condition: None + +(44) Project [codegen id : 11] +Output [11]: [cs_quantity#172, cs_sales_price#173, cs_net_paid_inc_tax#174, w_warehouse_name#178, w_warehouse_sq_ft#179, w_city#180, w_county#181, w_state#182, w_country#183, d_year#185, d_moy#186] +Input [13]: [cs_ship_mode_sk#170, cs_quantity#172, cs_sales_price#173, cs_net_paid_inc_tax#174, w_warehouse_name#178, w_warehouse_sq_ft#179, w_city#180, w_county#181, w_state#182, w_country#183, d_year#185, d_moy#186, sm_ship_mode_sk#188] + +(45) HashAggregate [codegen id : 11] +Input [11]: [cs_quantity#172, cs_sales_price#173, cs_net_paid_inc_tax#174, w_warehouse_name#178, w_warehouse_sq_ft#179, w_city#180, w_county#181, w_state#182, w_country#183, d_year#185, d_moy#186] +Keys [7]: [w_warehouse_name#178, w_warehouse_sq_ft#179, w_city#180, w_county#181, w_state#182, w_country#183, d_year#185] +Functions [24]: [partial_sum(CASE WHEN (d_moy#186 = 1) THEN (cs_sales_price#173 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#186 = 2) THEN (cs_sales_price#173 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#186 = 3) THEN (cs_sales_price#173 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#186 = 4) THEN (cs_sales_price#173 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#186 = 5) THEN (cs_sales_price#173 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#186 = 6) THEN (cs_sales_price#173 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#186 = 7) THEN (cs_sales_price#173 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#186 = 8) THEN (cs_sales_price#173 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#186 = 9) THEN (cs_sales_price#173 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#186 = 10) THEN (cs_sales_price#173 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#186 = 11) THEN (cs_sales_price#173 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#186 = 12) THEN (cs_sales_price#173 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#186 = 1) THEN (cs_net_paid_inc_tax#174 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#186 = 2) THEN (cs_net_paid_inc_tax#174 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#186 = 3) THEN (cs_net_paid_inc_tax#174 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#186 = 4) THEN (cs_net_paid_inc_tax#174 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#186 = 5) THEN (cs_net_paid_inc_tax#174 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#186 = 6) THEN (cs_net_paid_inc_tax#174 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#186 = 7) THEN (cs_net_paid_inc_tax#174 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#186 = 8) THEN (cs_net_paid_inc_tax#174 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#186 = 9) THEN (cs_net_paid_inc_tax#174 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#186 = 10) THEN (cs_net_paid_inc_tax#174 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#186 = 11) THEN (cs_net_paid_inc_tax#174 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#186 = 12) THEN (cs_net_paid_inc_tax#174 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END)] +Aggregate Attributes [48]: [sum#189, isEmpty#190, sum#191, isEmpty#192, sum#193, isEmpty#194, sum#195, isEmpty#196, sum#197, isEmpty#198, sum#199, isEmpty#200, sum#201, isEmpty#202, sum#203, isEmpty#204, sum#205, isEmpty#206, sum#207, isEmpty#208, sum#209, isEmpty#210, sum#211, isEmpty#212, sum#213, isEmpty#214, sum#215, isEmpty#216, sum#217, isEmpty#218, sum#219, isEmpty#220, sum#221, isEmpty#222, sum#223, isEmpty#224, sum#225, isEmpty#226, sum#227, isEmpty#228, sum#229, isEmpty#230, sum#231, isEmpty#232, sum#233, isEmpty#234, sum#235, isEmpty#236] +Results [55]: [w_warehouse_name#178, w_warehouse_sq_ft#179, w_city#180, w_county#181, w_state#182, w_country#183, d_year#185, sum#237, isEmpty#238, sum#239, isEmpty#240, sum#241, isEmpty#242, sum#243, isEmpty#244, sum#245, isEmpty#246, sum#247, isEmpty#248, sum#249, isEmpty#250, sum#251, isEmpty#252, sum#253, isEmpty#254, sum#255, isEmpty#256, sum#257, isEmpty#258, sum#259, isEmpty#260, sum#261, isEmpty#262, sum#263, isEmpty#264, sum#265, isEmpty#266, sum#267, isEmpty#268, sum#269, isEmpty#270, sum#271, isEmpty#272, sum#273, isEmpty#274, sum#275, isEmpty#276, sum#277, isEmpty#278, sum#279, isEmpty#280, sum#281, isEmpty#282, sum#283, isEmpty#284] + +(46) Exchange +Input [55]: [w_warehouse_name#178, w_warehouse_sq_ft#179, w_city#180, w_county#181, w_state#182, w_country#183, d_year#185, sum#237, isEmpty#238, sum#239, isEmpty#240, sum#241, isEmpty#242, sum#243, isEmpty#244, sum#245, isEmpty#246, sum#247, isEmpty#248, sum#249, isEmpty#250, sum#251, isEmpty#252, sum#253, isEmpty#254, sum#255, isEmpty#256, sum#257, isEmpty#258, sum#259, isEmpty#260, sum#261, isEmpty#262, sum#263, isEmpty#264, sum#265, isEmpty#266, sum#267, isEmpty#268, sum#269, isEmpty#270, sum#271, isEmpty#272, sum#273, isEmpty#274, sum#275, isEmpty#276, sum#277, isEmpty#278, sum#279, isEmpty#280, sum#281, isEmpty#282, sum#283, isEmpty#284] +Arguments: hashpartitioning(w_warehouse_name#178, w_warehouse_sq_ft#179, w_city#180, w_county#181, w_state#182, w_country#183, d_year#185, 5), ENSURE_REQUIREMENTS, [plan_id=5] + +(47) HashAggregate [codegen id : 12] +Input [55]: [w_warehouse_name#178, w_warehouse_sq_ft#179, w_city#180, w_county#181, w_state#182, w_country#183, d_year#185, sum#237, isEmpty#238, sum#239, isEmpty#240, sum#241, isEmpty#242, sum#243, isEmpty#244, sum#245, isEmpty#246, sum#247, isEmpty#248, sum#249, isEmpty#250, sum#251, isEmpty#252, sum#253, isEmpty#254, sum#255, isEmpty#256, sum#257, isEmpty#258, sum#259, isEmpty#260, sum#261, isEmpty#262, sum#263, isEmpty#264, sum#265, isEmpty#266, sum#267, isEmpty#268, sum#269, isEmpty#270, sum#271, isEmpty#272, sum#273, isEmpty#274, sum#275, isEmpty#276, sum#277, isEmpty#278, sum#279, isEmpty#280, sum#281, isEmpty#282, sum#283, isEmpty#284] +Keys [7]: [w_warehouse_name#178, w_warehouse_sq_ft#179, w_city#180, w_county#181, w_state#182, w_country#183, d_year#185] +Functions [24]: [sum(CASE WHEN (d_moy#186 = 1) THEN (cs_sales_price#173 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#186 = 2) THEN (cs_sales_price#173 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#186 = 3) THEN (cs_sales_price#173 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#186 = 4) THEN (cs_sales_price#173 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#186 = 5) THEN (cs_sales_price#173 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#186 = 6) THEN (cs_sales_price#173 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#186 = 7) THEN (cs_sales_price#173 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#186 = 8) THEN (cs_sales_price#173 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#186 = 9) THEN (cs_sales_price#173 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#186 = 10) THEN (cs_sales_price#173 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#186 = 11) THEN (cs_sales_price#173 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#186 = 12) THEN (cs_sales_price#173 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#186 = 1) THEN (cs_net_paid_inc_tax#174 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#186 = 2) THEN (cs_net_paid_inc_tax#174 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#186 = 3) THEN (cs_net_paid_inc_tax#174 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#186 = 4) THEN (cs_net_paid_inc_tax#174 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#186 = 5) THEN (cs_net_paid_inc_tax#174 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#186 = 6) THEN (cs_net_paid_inc_tax#174 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#186 = 7) THEN (cs_net_paid_inc_tax#174 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#186 = 8) THEN (cs_net_paid_inc_tax#174 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#186 = 9) THEN (cs_net_paid_inc_tax#174 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#186 = 10) THEN (cs_net_paid_inc_tax#174 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#186 = 11) THEN (cs_net_paid_inc_tax#174 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#186 = 12) THEN (cs_net_paid_inc_tax#174 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END)] +Aggregate Attributes [24]: [sum(CASE WHEN (d_moy#186 = 1) THEN (cs_sales_price#173 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END)#285, sum(CASE WHEN (d_moy#186 = 2) THEN (cs_sales_price#173 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END)#286, sum(CASE WHEN (d_moy#186 = 3) THEN (cs_sales_price#173 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END)#287, sum(CASE WHEN (d_moy#186 = 4) THEN (cs_sales_price#173 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END)#288, sum(CASE WHEN (d_moy#186 = 5) THEN (cs_sales_price#173 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END)#289, sum(CASE WHEN (d_moy#186 = 6) THEN (cs_sales_price#173 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END)#290, sum(CASE WHEN (d_moy#186 = 7) THEN (cs_sales_price#173 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END)#291, sum(CASE WHEN (d_moy#186 = 8) THEN (cs_sales_price#173 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END)#292, sum(CASE WHEN (d_moy#186 = 9) THEN (cs_sales_price#173 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END)#293, sum(CASE WHEN (d_moy#186 = 10) THEN (cs_sales_price#173 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END)#294, sum(CASE WHEN (d_moy#186 = 11) THEN (cs_sales_price#173 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END)#295, sum(CASE WHEN (d_moy#186 = 12) THEN (cs_sales_price#173 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END)#296, sum(CASE WHEN (d_moy#186 = 1) THEN (cs_net_paid_inc_tax#174 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END)#297, sum(CASE WHEN (d_moy#186 = 2) THEN (cs_net_paid_inc_tax#174 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END)#298, sum(CASE WHEN (d_moy#186 = 3) THEN (cs_net_paid_inc_tax#174 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END)#299, sum(CASE WHEN (d_moy#186 = 4) THEN (cs_net_paid_inc_tax#174 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END)#300, sum(CASE WHEN (d_moy#186 = 5) THEN (cs_net_paid_inc_tax#174 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END)#301, sum(CASE WHEN (d_moy#186 = 6) THEN (cs_net_paid_inc_tax#174 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END)#302, sum(CASE WHEN (d_moy#186 = 7) THEN (cs_net_paid_inc_tax#174 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END)#303, sum(CASE WHEN (d_moy#186 = 8) THEN (cs_net_paid_inc_tax#174 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END)#304, sum(CASE WHEN (d_moy#186 = 9) THEN (cs_net_paid_inc_tax#174 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END)#305, sum(CASE WHEN (d_moy#186 = 10) THEN (cs_net_paid_inc_tax#174 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END)#306, sum(CASE WHEN (d_moy#186 = 11) THEN (cs_net_paid_inc_tax#174 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END)#307, sum(CASE WHEN (d_moy#186 = 12) THEN (cs_net_paid_inc_tax#174 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END)#308] +Results [32]: [w_warehouse_name#178, w_warehouse_sq_ft#179, w_city#180, w_county#181, w_state#182, w_country#183, DHL,BARIAN AS ship_carriers#309, d_year#185 AS year#310, sum(CASE WHEN (d_moy#186 = 1) THEN (cs_sales_price#173 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END)#285 AS jan_sales#311, sum(CASE WHEN (d_moy#186 = 2) THEN (cs_sales_price#173 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END)#286 AS feb_sales#312, sum(CASE WHEN (d_moy#186 = 3) THEN (cs_sales_price#173 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END)#287 AS mar_sales#313, sum(CASE WHEN (d_moy#186 = 4) THEN (cs_sales_price#173 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END)#288 AS apr_sales#314, sum(CASE WHEN (d_moy#186 = 5) THEN (cs_sales_price#173 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END)#289 AS may_sales#315, sum(CASE WHEN (d_moy#186 = 6) THEN (cs_sales_price#173 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END)#290 AS jun_sales#316, sum(CASE WHEN (d_moy#186 = 7) THEN (cs_sales_price#173 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END)#291 AS jul_sales#317, sum(CASE WHEN (d_moy#186 = 8) THEN (cs_sales_price#173 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END)#292 AS aug_sales#318, sum(CASE WHEN (d_moy#186 = 9) THEN (cs_sales_price#173 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END)#293 AS sep_sales#319, sum(CASE WHEN (d_moy#186 = 10) THEN (cs_sales_price#173 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END)#294 AS oct_sales#320, sum(CASE WHEN (d_moy#186 = 11) THEN (cs_sales_price#173 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END)#295 AS nov_sales#321, sum(CASE WHEN (d_moy#186 = 12) THEN (cs_sales_price#173 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END)#296 AS dec_sales#322, sum(CASE WHEN (d_moy#186 = 1) THEN (cs_net_paid_inc_tax#174 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END)#297 AS jan_net#323, sum(CASE WHEN (d_moy#186 = 2) THEN (cs_net_paid_inc_tax#174 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END)#298 AS feb_net#324, sum(CASE WHEN (d_moy#186 = 3) THEN (cs_net_paid_inc_tax#174 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END)#299 AS mar_net#325, sum(CASE WHEN (d_moy#186 = 4) THEN (cs_net_paid_inc_tax#174 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END)#300 AS apr_net#326, sum(CASE WHEN (d_moy#186 = 5) THEN (cs_net_paid_inc_tax#174 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END)#301 AS may_net#327, sum(CASE WHEN (d_moy#186 = 6) THEN (cs_net_paid_inc_tax#174 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END)#302 AS jun_net#328, sum(CASE WHEN (d_moy#186 = 7) THEN (cs_net_paid_inc_tax#174 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END)#303 AS jul_net#329, sum(CASE WHEN (d_moy#186 = 8) THEN (cs_net_paid_inc_tax#174 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END)#304 AS aug_net#330, sum(CASE WHEN (d_moy#186 = 9) THEN (cs_net_paid_inc_tax#174 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END)#305 AS sep_net#331, sum(CASE WHEN (d_moy#186 = 10) THEN (cs_net_paid_inc_tax#174 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END)#306 AS oct_net#332, sum(CASE WHEN (d_moy#186 = 11) THEN (cs_net_paid_inc_tax#174 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END)#307 AS nov_net#333, sum(CASE WHEN (d_moy#186 = 12) THEN (cs_net_paid_inc_tax#174 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END)#308 AS dec_net#334] + +(48) Union + +(49) HashAggregate [codegen id : 13] +Input [32]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15, ship_carriers#143, year#144, jan_sales#145, feb_sales#146, mar_sales#147, apr_sales#148, may_sales#149, jun_sales#150, jul_sales#151, aug_sales#152, sep_sales#153, oct_sales#154, nov_sales#155, dec_sales#156, jan_net#157, feb_net#158, mar_net#159, apr_net#160, may_net#161, jun_net#162, jul_net#163, aug_net#164, sep_net#165, oct_net#166, nov_net#167, dec_net#168] +Keys [8]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15, ship_carriers#143, year#144] +Functions [36]: [partial_sum(jan_sales#145), partial_sum(feb_sales#146), partial_sum(mar_sales#147), partial_sum(apr_sales#148), partial_sum(may_sales#149), partial_sum(jun_sales#150), partial_sum(jul_sales#151), partial_sum(aug_sales#152), partial_sum(sep_sales#153), partial_sum(oct_sales#154), partial_sum(nov_sales#155), partial_sum(dec_sales#156), partial_sum((jan_sales#145 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), partial_sum((feb_sales#146 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), partial_sum((mar_sales#147 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), partial_sum((apr_sales#148 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), partial_sum((may_sales#149 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), partial_sum((jun_sales#150 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), partial_sum((jul_sales#151 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), partial_sum((aug_sales#152 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), partial_sum((sep_sales#153 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), partial_sum((oct_sales#154 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), partial_sum((nov_sales#155 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), partial_sum((dec_sales#156 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), partial_sum(jan_net#157), partial_sum(feb_net#158), partial_sum(mar_net#159), partial_sum(apr_net#160), partial_sum(may_net#161), partial_sum(jun_net#162), partial_sum(jul_net#163), partial_sum(aug_net#164), partial_sum(sep_net#165), partial_sum(oct_net#166), partial_sum(nov_net#167), partial_sum(dec_net#168)] +Aggregate Attributes [72]: [sum#335, isEmpty#336, sum#337, isEmpty#338, sum#339, isEmpty#340, sum#341, isEmpty#342, sum#343, isEmpty#344, sum#345, isEmpty#346, sum#347, isEmpty#348, sum#349, isEmpty#350, sum#351, isEmpty#352, sum#353, isEmpty#354, sum#355, isEmpty#356, sum#357, isEmpty#358, sum#359, isEmpty#360, sum#361, isEmpty#362, sum#363, isEmpty#364, sum#365, isEmpty#366, sum#367, isEmpty#368, sum#369, isEmpty#370, sum#371, isEmpty#372, sum#373, isEmpty#374, sum#375, isEmpty#376, sum#377, isEmpty#378, sum#379, isEmpty#380, sum#381, isEmpty#382, sum#383, isEmpty#384, sum#385, isEmpty#386, sum#387, isEmpty#388, sum#389, isEmpty#390, sum#391, isEmpty#392, sum#393, isEmpty#394, sum#395, isEmpty#396, sum#397, isEmpty#398, sum#399, isEmpty#400, sum#401, isEmpty#402, sum#403, isEmpty#404, sum#405, isEmpty#406] +Results [80]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15, ship_carriers#143, year#144, sum#407, isEmpty#408, sum#409, isEmpty#410, sum#411, isEmpty#412, sum#413, isEmpty#414, sum#415, isEmpty#416, sum#417, isEmpty#418, sum#419, isEmpty#420, sum#421, isEmpty#422, sum#423, isEmpty#424, sum#425, isEmpty#426, sum#427, isEmpty#428, sum#429, isEmpty#430, sum#431, isEmpty#432, sum#433, isEmpty#434, sum#435, isEmpty#436, sum#437, isEmpty#438, sum#439, isEmpty#440, sum#441, isEmpty#442, sum#443, isEmpty#444, sum#445, isEmpty#446, sum#447, isEmpty#448, sum#449, isEmpty#450, sum#451, isEmpty#452, sum#453, isEmpty#454, sum#455, isEmpty#456, sum#457, isEmpty#458, sum#459, isEmpty#460, sum#461, isEmpty#462, sum#463, isEmpty#464, sum#465, isEmpty#466, sum#467, isEmpty#468, sum#469, isEmpty#470, sum#471, isEmpty#472, sum#473, isEmpty#474, sum#475, isEmpty#476, sum#477, isEmpty#478] + +(50) Exchange +Input [80]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15, ship_carriers#143, year#144, sum#407, isEmpty#408, sum#409, isEmpty#410, sum#411, isEmpty#412, sum#413, isEmpty#414, sum#415, isEmpty#416, sum#417, isEmpty#418, sum#419, isEmpty#420, sum#421, isEmpty#422, sum#423, isEmpty#424, sum#425, isEmpty#426, sum#427, isEmpty#428, sum#429, isEmpty#430, sum#431, isEmpty#432, sum#433, isEmpty#434, sum#435, isEmpty#436, sum#437, isEmpty#438, sum#439, isEmpty#440, sum#441, isEmpty#442, sum#443, isEmpty#444, sum#445, isEmpty#446, sum#447, isEmpty#448, sum#449, isEmpty#450, sum#451, isEmpty#452, sum#453, isEmpty#454, sum#455, isEmpty#456, sum#457, isEmpty#458, sum#459, isEmpty#460, sum#461, isEmpty#462, sum#463, isEmpty#464, sum#465, isEmpty#466, sum#467, isEmpty#468, sum#469, isEmpty#470, sum#471, isEmpty#472, sum#473, isEmpty#474, sum#475, isEmpty#476, sum#477, isEmpty#478] +Arguments: hashpartitioning(w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15, ship_carriers#143, year#144, 5), ENSURE_REQUIREMENTS, [plan_id=6] + +(51) HashAggregate [codegen id : 14] +Input [80]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15, ship_carriers#143, year#144, sum#407, isEmpty#408, sum#409, isEmpty#410, sum#411, isEmpty#412, sum#413, isEmpty#414, sum#415, isEmpty#416, sum#417, isEmpty#418, sum#419, isEmpty#420, sum#421, isEmpty#422, sum#423, isEmpty#424, sum#425, isEmpty#426, sum#427, isEmpty#428, sum#429, isEmpty#430, sum#431, isEmpty#432, sum#433, isEmpty#434, sum#435, isEmpty#436, sum#437, isEmpty#438, sum#439, isEmpty#440, sum#441, isEmpty#442, sum#443, isEmpty#444, sum#445, isEmpty#446, sum#447, isEmpty#448, sum#449, isEmpty#450, sum#451, isEmpty#452, sum#453, isEmpty#454, sum#455, isEmpty#456, sum#457, isEmpty#458, sum#459, isEmpty#460, sum#461, isEmpty#462, sum#463, isEmpty#464, sum#465, isEmpty#466, sum#467, isEmpty#468, sum#469, isEmpty#470, sum#471, isEmpty#472, sum#473, isEmpty#474, sum#475, isEmpty#476, sum#477, isEmpty#478] +Keys [8]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15, ship_carriers#143, year#144] +Functions [36]: [sum(jan_sales#145), sum(feb_sales#146), sum(mar_sales#147), sum(apr_sales#148), sum(may_sales#149), sum(jun_sales#150), sum(jul_sales#151), sum(aug_sales#152), sum(sep_sales#153), sum(oct_sales#154), sum(nov_sales#155), sum(dec_sales#156), sum((jan_sales#145 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), sum((feb_sales#146 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), sum((mar_sales#147 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), sum((apr_sales#148 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), sum((may_sales#149 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), sum((jun_sales#150 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), sum((jul_sales#151 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), sum((aug_sales#152 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), sum((sep_sales#153 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), sum((oct_sales#154 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), sum((nov_sales#155 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), sum((dec_sales#156 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), sum(jan_net#157), sum(feb_net#158), sum(mar_net#159), sum(apr_net#160), sum(may_net#161), sum(jun_net#162), sum(jul_net#163), sum(aug_net#164), sum(sep_net#165), sum(oct_net#166), sum(nov_net#167), sum(dec_net#168)] +Aggregate Attributes [36]: [sum(jan_sales#145)#479, sum(feb_sales#146)#480, sum(mar_sales#147)#481, sum(apr_sales#148)#482, sum(may_sales#149)#483, sum(jun_sales#150)#484, sum(jul_sales#151)#485, sum(aug_sales#152)#486, sum(sep_sales#153)#487, sum(oct_sales#154)#488, sum(nov_sales#155)#489, sum(dec_sales#156)#490, sum((jan_sales#145 / cast(w_warehouse_sq_ft#11 as decimal(10,0))))#491, sum((feb_sales#146 / cast(w_warehouse_sq_ft#11 as decimal(10,0))))#492, sum((mar_sales#147 / cast(w_warehouse_sq_ft#11 as decimal(10,0))))#493, sum((apr_sales#148 / cast(w_warehouse_sq_ft#11 as decimal(10,0))))#494, sum((may_sales#149 / cast(w_warehouse_sq_ft#11 as decimal(10,0))))#495, sum((jun_sales#150 / cast(w_warehouse_sq_ft#11 as decimal(10,0))))#496, sum((jul_sales#151 / cast(w_warehouse_sq_ft#11 as decimal(10,0))))#497, sum((aug_sales#152 / cast(w_warehouse_sq_ft#11 as decimal(10,0))))#498, sum((sep_sales#153 / cast(w_warehouse_sq_ft#11 as decimal(10,0))))#499, sum((oct_sales#154 / cast(w_warehouse_sq_ft#11 as decimal(10,0))))#500, sum((nov_sales#155 / cast(w_warehouse_sq_ft#11 as decimal(10,0))))#501, sum((dec_sales#156 / cast(w_warehouse_sq_ft#11 as decimal(10,0))))#502, sum(jan_net#157)#503, sum(feb_net#158)#504, sum(mar_net#159)#505, sum(apr_net#160)#506, sum(may_net#161)#507, sum(jun_net#162)#508, sum(jul_net#163)#509, sum(aug_net#164)#510, sum(sep_net#165)#511, sum(oct_net#166)#512, sum(nov_net#167)#513, sum(dec_net#168)#514] +Results [44]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15, ship_carriers#143, year#144, sum(jan_sales#145)#479 AS jan_sales#515, sum(feb_sales#146)#480 AS feb_sales#516, sum(mar_sales#147)#481 AS mar_sales#517, sum(apr_sales#148)#482 AS apr_sales#518, sum(may_sales#149)#483 AS may_sales#519, sum(jun_sales#150)#484 AS jun_sales#520, sum(jul_sales#151)#485 AS jul_sales#521, sum(aug_sales#152)#486 AS aug_sales#522, sum(sep_sales#153)#487 AS sep_sales#523, sum(oct_sales#154)#488 AS oct_sales#524, sum(nov_sales#155)#489 AS nov_sales#525, sum(dec_sales#156)#490 AS dec_sales#526, sum((jan_sales#145 / cast(w_warehouse_sq_ft#11 as decimal(10,0))))#491 AS jan_sales_per_sq_foot#527, sum((feb_sales#146 / cast(w_warehouse_sq_ft#11 as decimal(10,0))))#492 AS feb_sales_per_sq_foot#528, sum((mar_sales#147 / cast(w_warehouse_sq_ft#11 as decimal(10,0))))#493 AS mar_sales_per_sq_foot#529, sum((apr_sales#148 / cast(w_warehouse_sq_ft#11 as decimal(10,0))))#494 AS apr_sales_per_sq_foot#530, sum((may_sales#149 / cast(w_warehouse_sq_ft#11 as decimal(10,0))))#495 AS may_sales_per_sq_foot#531, sum((jun_sales#150 / cast(w_warehouse_sq_ft#11 as decimal(10,0))))#496 AS jun_sales_per_sq_foot#532, sum((jul_sales#151 / cast(w_warehouse_sq_ft#11 as decimal(10,0))))#497 AS jul_sales_per_sq_foot#533, sum((aug_sales#152 / cast(w_warehouse_sq_ft#11 as decimal(10,0))))#498 AS aug_sales_per_sq_foot#534, sum((sep_sales#153 / cast(w_warehouse_sq_ft#11 as decimal(10,0))))#499 AS sep_sales_per_sq_foot#535, sum((oct_sales#154 / cast(w_warehouse_sq_ft#11 as decimal(10,0))))#500 AS oct_sales_per_sq_foot#536, sum((nov_sales#155 / cast(w_warehouse_sq_ft#11 as decimal(10,0))))#501 AS nov_sales_per_sq_foot#537, sum((dec_sales#156 / cast(w_warehouse_sq_ft#11 as decimal(10,0))))#502 AS dec_sales_per_sq_foot#538, sum(jan_net#157)#503 AS jan_net#539, sum(feb_net#158)#504 AS feb_net#540, sum(mar_net#159)#505 AS mar_net#541, sum(apr_net#160)#506 AS apr_net#542, sum(may_net#161)#507 AS may_net#543, sum(jun_net#162)#508 AS jun_net#544, sum(jul_net#163)#509 AS jul_net#545, sum(aug_net#164)#510 AS aug_net#546, sum(sep_net#165)#511 AS sep_net#547, sum(oct_net#166)#512 AS oct_net#548, sum(nov_net#167)#513 AS nov_net#549, sum(dec_net#168)#514 AS dec_net#550] + +(52) TakeOrderedAndProject +Input [44]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15, ship_carriers#143, year#144, jan_sales#515, feb_sales#516, mar_sales#517, apr_sales#518, may_sales#519, jun_sales#520, jul_sales#521, aug_sales#522, sep_sales#523, oct_sales#524, nov_sales#525, dec_sales#526, jan_sales_per_sq_foot#527, feb_sales_per_sq_foot#528, mar_sales_per_sq_foot#529, apr_sales_per_sq_foot#530, may_sales_per_sq_foot#531, jun_sales_per_sq_foot#532, jul_sales_per_sq_foot#533, aug_sales_per_sq_foot#534, sep_sales_per_sq_foot#535, oct_sales_per_sq_foot#536, nov_sales_per_sq_foot#537, dec_sales_per_sq_foot#538, jan_net#539, feb_net#540, mar_net#541, apr_net#542, may_net#543, jun_net#544, jul_net#545, aug_net#546, sep_net#547, oct_net#548, nov_net#549, dec_net#550] +Arguments: 100, [w_warehouse_name#10 ASC NULLS FIRST], [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15, ship_carriers#143, year#144, jan_sales#515, feb_sales#516, mar_sales#517, apr_sales#518, may_sales#519, jun_sales#520, jul_sales#521, aug_sales#522, sep_sales#523, oct_sales#524, nov_sales#525, dec_sales#526, jan_sales_per_sq_foot#527, feb_sales_per_sq_foot#528, mar_sales_per_sq_foot#529, apr_sales_per_sq_foot#530, may_sales_per_sq_foot#531, jun_sales_per_sq_foot#532, jul_sales_per_sq_foot#533, aug_sales_per_sq_foot#534, sep_sales_per_sq_foot#535, oct_sales_per_sq_foot#536, nov_sales_per_sq_foot#537, dec_sales_per_sq_foot#538, jan_net#539, feb_net#540, mar_net#541, apr_net#542, may_net#543, jun_net#544, jul_net#545, aug_net#546, sep_net#547, oct_net#548, nov_net#549, dec_net#550] + +===== Subqueries ===== + +Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#7 IN dynamicpruning#8 +BroadcastExchange (56) ++- * ColumnarToRow (55) + +- CometFilter (54) + +- CometScan parquet spark_catalog.default.date_dim (53) + + +(53) Scan parquet spark_catalog.default.date_dim +Output [3]: [d_date_sk#16, d_year#17, d_moy#18] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), IsNotNull(d_date_sk)] +ReadSchema: struct<d_date_sk:int,d_year:int,d_moy:int> + +(54) CometFilter +Input [3]: [d_date_sk#16, d_year#17, d_moy#18] +Condition : ((isnotnull(d_year#17) AND (d_year#17 = 2001)) AND isnotnull(d_date_sk#16)) + +(55) ColumnarToRow [codegen id : 1] +Input [3]: [d_date_sk#16, d_year#17, d_moy#18] + +(56) BroadcastExchange +Input [3]: [d_date_sk#16, d_year#17, d_moy#18] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=7] + +Subquery:2 Hosting operator id = 30 Hosting Expression = cs_sold_date_sk#175 IN dynamicpruning#8 + + diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q66/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q66/simplified.txt new file mode 100644 index 000000000..8ed74582f --- /dev/null +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q66/simplified.txt @@ -0,0 +1,86 @@ +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) + HashAggregate [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,ship_carriers,year,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(jan_sales),sum(feb_sales),sum(mar_sales),sum(apr_sales),sum(may_sales),sum(jun_sales),sum(jul_sales),sum(aug_sales),sum(sep_sales),sum(oct_sales),sum(nov_sales),sum(dec_sales),sum((jan_sales / cast(w_warehouse_sq_ft as decimal(10,0)))),sum((feb_sales / cast(w_warehouse_sq_ft as decimal(10,0)))),sum((mar_sales / cast(w_warehouse_sq_ft as decimal(10,0)))),sum((apr_sales / cast(w_warehouse_sq_ft as decimal(10,0)))),sum((may_sales / cast(w_warehouse_sq_ft as decimal(10,0)))),sum((jun_sales / cast(w_warehouse_sq_ft as decimal(10,0)))),sum((jul_sales / cast(w_warehouse_sq_ft as decimal(10,0)))),sum((aug_sales / cast(w_warehouse_sq_ft as decimal(10,0)))),sum((sep_sales / cast(w_warehouse_sq_ft as decimal(10,0)))),sum((oct_sales / cast(w_warehouse_sq_ft as decimal(10,0)))),sum((nov_sales / cast(w_warehouse_sq_ft as decimal(10,0)))),sum((dec_sales / cast(w_warehouse_sq_ft as decimal(10,0)))),sum(jan_net),sum(feb_net),sum(mar_net),sum(apr_net),sum(may_net),sum(jun_net),sum(jul_net),sum(aug_net),sum(sep_net),sum(oct_net),sum(nov_net),sum(dec_net),jan_sales,feb_sales,mar_sales,apr_sales,may_sales,jun_sales,jul_sales,aug_sales,sep_sales,oct_sales,nov_sales,dec_sales,jan_sales_per_sq_foot,feb_sales_per_sq_foot,mar_sales_per_sq_foot,apr_sales_per_sq_foot,may_sales_per_sq_foot,jun_sales_per_sq_foot,jul_sales_per_sq_foot,aug_sales_per_sq_foot,sep_sales_per_sq_foot,oct_sales_per_sq_foot,nov_sales_per_sq_foot,dec_sales_per_sq_foot,jan_net,feb_net,mar_net,apr_net,may_net,jun_net,jul_net,aug_net,sep_net,oct_net,nov_net,dec_net,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] + InputAdapter + Exchange [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,ship_carriers,year] #1 + WholeStageCodegen (13) + 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) + HashAggregate [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(CASE WHEN (d_moy = 1) THEN (ws_ext_sales_price * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 2) THEN (ws_ext_sales_price * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 3) THEN (ws_ext_sales_price * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 4) THEN (ws_ext_sales_price * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 5) THEN (ws_ext_sales_price * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 6) THEN (ws_ext_sales_price * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 7) THEN (ws_ext_sales_price * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 8) THEN (ws_ext_sales_price * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 9) THEN (ws_ext_sales_price * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 10) THEN (ws_ext_sales_price * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 11) THEN (ws_ext_sales_price * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 12) THEN (ws_ext_sales_price * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 1) THEN (ws_net_paid * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 2) THEN (ws_net_paid * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 3) THEN (ws_net_paid * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 4) THEN (ws_net_paid * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 5) THEN (ws_net_paid * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 6) THEN (ws_net_paid * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 7) THEN (ws_net_paid * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 8) THEN (ws_net_paid * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 9) THEN (ws_net_paid * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 10) THEN (ws_net_paid * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 11) THEN (ws_net_paid * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 12) THEN (ws_net_paid * cast(ws_quantity as decimal(10,0))) ELSE 0.00 END),ship_carriers,year,jan_sales,feb_sales,mar_sales,apr_sales,may_sales,jun_sales,jul_sales,aug_sales,sep_sales,oct_sales,nov_sales,dec_sales,jan_net,feb_net,mar_net,apr_net,may_net,jun_net,jul_net,aug_net,sep_net,oct_net,nov_net,dec_net,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] + InputAdapter + Exchange [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year] #2 + WholeStageCodegen (5) + HashAggregate [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year,d_moy,ws_ext_sales_price,ws_quantity,ws_net_paid] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] + Project [ws_quantity,ws_ext_sales_price,ws_net_paid,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year,d_moy] + BroadcastHashJoin [ws_ship_mode_sk,sm_ship_mode_sk] + Project [ws_ship_mode_sk,ws_quantity,ws_ext_sales_price,ws_net_paid,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year,d_moy] + BroadcastHashJoin [ws_sold_time_sk,t_time_sk] + Project [ws_sold_time_sk,ws_ship_mode_sk,ws_quantity,ws_ext_sales_price,ws_net_paid,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year,d_moy] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + Project [ws_sold_time_sk,ws_ship_mode_sk,ws_quantity,ws_ext_sales_price,ws_net_paid,ws_sold_date_sk,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country] + BroadcastHashJoin [ws_warehouse_sk,w_warehouse_sk] + ColumnarToRow + InputAdapter + CometFilter [ws_warehouse_sk,ws_sold_time_sk,ws_ship_mode_sk] + CometScan parquet spark_catalog.default.web_sales [ws_sold_time_sk,ws_ship_mode_sk,ws_warehouse_sk,ws_quantity,ws_ext_sales_price,ws_net_paid,ws_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometFilter [d_year,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + InputAdapter + BroadcastExchange #4 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometFilter [w_warehouse_sk] + CometScan parquet spark_catalog.default.warehouse [w_warehouse_sk,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country] + InputAdapter + ReusedExchange [d_date_sk,d_year,d_moy] #3 + InputAdapter + BroadcastExchange #5 + WholeStageCodegen (3) + ColumnarToRow + InputAdapter + CometProject [t_time_sk] + CometFilter [t_time,t_time_sk] + CometScan parquet spark_catalog.default.time_dim [t_time_sk,t_time] + InputAdapter + BroadcastExchange #6 + WholeStageCodegen (4) + ColumnarToRow + InputAdapter + CometProject [sm_ship_mode_sk] + CometFilter [sm_carrier,sm_ship_mode_sk] + CometScan parquet spark_catalog.default.ship_mode [sm_ship_mode_sk,sm_carrier] + WholeStageCodegen (12) + HashAggregate [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(CASE WHEN (d_moy = 1) THEN (cs_sales_price * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 2) THEN (cs_sales_price * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 3) THEN (cs_sales_price * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 4) THEN (cs_sales_price * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 5) THEN (cs_sales_price * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 6) THEN (cs_sales_price * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 7) THEN (cs_sales_price * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 8) THEN (cs_sales_price * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 9) THEN (cs_sales_price * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 10) THEN (cs_sales_price * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 11) THEN (cs_sales_price * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 12) THEN (cs_sales_price * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 1) THEN (cs_net_paid_inc_tax * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 2) THEN (cs_net_paid_inc_tax * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 3) THEN (cs_net_paid_inc_tax * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 4) THEN (cs_net_paid_inc_tax * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 5) THEN (cs_net_paid_inc_tax * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 6) THEN (cs_net_paid_inc_tax * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 7) THEN (cs_net_paid_inc_tax * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 8) THEN (cs_net_paid_inc_tax * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 9) THEN (cs_net_paid_inc_tax * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 10) THEN (cs_net_paid_inc_tax * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 11) THEN (cs_net_paid_inc_tax * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),sum(CASE WHEN (d_moy = 12) THEN (cs_net_paid_inc_tax * cast(cs_quantity as decimal(10,0))) ELSE 0.00 END),ship_carriers,year,jan_sales,feb_sales,mar_sales,apr_sales,may_sales,jun_sales,jul_sales,aug_sales,sep_sales,oct_sales,nov_sales,dec_sales,jan_net,feb_net,mar_net,apr_net,may_net,jun_net,jul_net,aug_net,sep_net,oct_net,nov_net,dec_net,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] + InputAdapter + Exchange [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year] #7 + WholeStageCodegen (11) + HashAggregate [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year,d_moy,cs_sales_price,cs_quantity,cs_net_paid_inc_tax] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] + Project [cs_quantity,cs_sales_price,cs_net_paid_inc_tax,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year,d_moy] + BroadcastHashJoin [cs_ship_mode_sk,sm_ship_mode_sk] + Project [cs_ship_mode_sk,cs_quantity,cs_sales_price,cs_net_paid_inc_tax,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year,d_moy] + BroadcastHashJoin [cs_sold_time_sk,t_time_sk] + Project [cs_sold_time_sk,cs_ship_mode_sk,cs_quantity,cs_sales_price,cs_net_paid_inc_tax,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year,d_moy] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + Project [cs_sold_time_sk,cs_ship_mode_sk,cs_quantity,cs_sales_price,cs_net_paid_inc_tax,cs_sold_date_sk,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country] + BroadcastHashJoin [cs_warehouse_sk,w_warehouse_sk] + ColumnarToRow + InputAdapter + CometFilter [cs_warehouse_sk,cs_sold_time_sk,cs_ship_mode_sk] + CometScan parquet spark_catalog.default.catalog_sales [cs_sold_time_sk,cs_ship_mode_sk,cs_warehouse_sk,cs_quantity,cs_sales_price,cs_net_paid_inc_tax,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + InputAdapter + ReusedExchange [w_warehouse_sk,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country] #4 + InputAdapter + ReusedExchange [d_date_sk,d_year,d_moy] #3 + InputAdapter + ReusedExchange [t_time_sk] #5 + InputAdapter + ReusedExchange [sm_ship_mode_sk] #6 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 new file mode 100644 index 000000000..d1a6a4f47 --- /dev/null +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q67/explain.txt @@ -0,0 +1,204 @@ +== Physical Plan == +TakeOrderedAndProject (30) ++- * Filter (29) + +- Window (28) + +- WindowGroupLimit (27) + +- * Sort (26) + +- Exchange (25) + +- WindowGroupLimit (24) + +- * Sort (23) + +- * HashAggregate (22) + +- Exchange (21) + +- * HashAggregate (20) + +- * Expand (19) + +- * Project (18) + +- * BroadcastHashJoin Inner BuildRight (17) + :- * Project (12) + : +- * BroadcastHashJoin Inner BuildRight (11) + : :- * Project (6) + : : +- * BroadcastHashJoin Inner BuildRight (5) + : : :- * ColumnarToRow (3) + : : : +- CometFilter (2) + : : : +- CometScan parquet spark_catalog.default.store_sales (1) + : : +- ReusedExchange (4) + : +- BroadcastExchange (10) + : +- * ColumnarToRow (9) + : +- CometFilter (8) + : +- CometScan parquet spark_catalog.default.store (7) + +- BroadcastExchange (16) + +- * ColumnarToRow (15) + +- CometFilter (14) + +- CometScan parquet spark_catalog.default.item (13) + + +(1) Scan parquet spark_catalog.default.store_sales +Output [5]: [ss_item_sk#1, ss_store_sk#2, ss_quantity#3, ss_sales_price#4, ss_sold_date_sk#5] +Batched: true +Location: InMemoryFileIndex [] +PartitionFilters: [isnotnull(ss_sold_date_sk#5), dynamicpruningexpression(ss_sold_date_sk#5 IN dynamicpruning#6)] +PushedFilters: [IsNotNull(ss_store_sk), IsNotNull(ss_item_sk)] +ReadSchema: struct<ss_item_sk:int,ss_store_sk:int,ss_quantity:int,ss_sales_price:decimal(7,2)> + +(2) CometFilter +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)) + +(3) ColumnarToRow [codegen id : 4] +Input [5]: [ss_item_sk#1, ss_store_sk#2, ss_quantity#3, ss_sales_price#4, ss_sold_date_sk#5] + +(4) ReusedExchange [Reuses operator id: 35] +Output [4]: [d_date_sk#7, d_year#8, d_moy#9, d_qoy#10] + +(5) BroadcastHashJoin [codegen id : 4] +Left keys [1]: [ss_sold_date_sk#5] +Right keys [1]: [d_date_sk#7] +Join type: Inner +Join condition: None + +(6) Project [codegen id : 4] +Output [7]: [ss_item_sk#1, ss_store_sk#2, ss_quantity#3, ss_sales_price#4, d_year#8, d_moy#9, d_qoy#10] +Input [9]: [ss_item_sk#1, ss_store_sk#2, ss_quantity#3, ss_sales_price#4, ss_sold_date_sk#5, d_date_sk#7, d_year#8, d_moy#9, d_qoy#10] + +(7) Scan parquet spark_catalog.default.store +Output [2]: [s_store_sk#11, s_store_id#12] +Batched: true +Location [not included in comparison]/{warehouse_dir}/store] +PushedFilters: [IsNotNull(s_store_sk)] +ReadSchema: struct<s_store_sk:int,s_store_id:string> + +(8) CometFilter +Input [2]: [s_store_sk#11, s_store_id#12] +Condition : isnotnull(s_store_sk#11) + +(9) ColumnarToRow [codegen id : 2] +Input [2]: [s_store_sk#11, s_store_id#12] + +(10) BroadcastExchange +Input [2]: [s_store_sk#11, s_store_id#12] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=1] + +(11) BroadcastHashJoin [codegen id : 4] +Left keys [1]: [ss_store_sk#2] +Right keys [1]: [s_store_sk#11] +Join type: Inner +Join condition: None + +(12) Project [codegen id : 4] +Output [7]: [ss_item_sk#1, ss_quantity#3, ss_sales_price#4, d_year#8, d_moy#9, d_qoy#10, s_store_id#12] +Input [9]: [ss_item_sk#1, ss_store_sk#2, ss_quantity#3, ss_sales_price#4, d_year#8, d_moy#9, d_qoy#10, s_store_sk#11, s_store_id#12] + +(13) Scan parquet spark_catalog.default.item +Output [5]: [i_item_sk#13, i_brand#14, i_class#15, i_category#16, i_product_name#17] +Batched: true +Location [not included in comparison]/{warehouse_dir}/item] +PushedFilters: [IsNotNull(i_item_sk)] +ReadSchema: struct<i_item_sk:int,i_brand:string,i_class:string,i_category:string,i_product_name:string> + +(14) CometFilter +Input [5]: [i_item_sk#13, i_brand#14, i_class#15, i_category#16, i_product_name#17] +Condition : isnotnull(i_item_sk#13) + +(15) ColumnarToRow [codegen id : 3] +Input [5]: [i_item_sk#13, i_brand#14, i_class#15, i_category#16, i_product_name#17] + +(16) BroadcastExchange +Input [5]: [i_item_sk#13, i_brand#14, i_class#15, i_category#16, i_product_name#17] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=2] + +(17) BroadcastHashJoin [codegen id : 4] +Left keys [1]: [ss_item_sk#1] +Right keys [1]: [i_item_sk#13] +Join type: Inner +Join condition: None + +(18) Project [codegen id : 4] +Output [10]: [ss_quantity#3, ss_sales_price#4, i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, s_store_id#12] +Input [12]: [ss_item_sk#1, ss_quantity#3, ss_sales_price#4, d_year#8, d_moy#9, d_qoy#10, s_store_id#12, i_item_sk#13, i_brand#14, i_class#15, i_category#16, i_product_name#17] + +(19) Expand [codegen id : 4] +Input [10]: [ss_quantity#3, ss_sales_price#4, i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, s_store_id#12] +Arguments: [[ss_quantity#3, ss_sales_price#4, i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, s_store_id#12, 0], [ss_quantity#3, ss_sales_price#4, i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, null, 1], [ss_quantity#3, ss_sales_price#4, i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, null, null, 3], [ss_quantity#3, ss_sales_price#4, i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, null, null, null, 7], [ss_quantity#3, ss_sales_price#4, i_category#16, i_class#15, i_brand#14, i_product_name#17, null, null, null, null, 15], [ss_quantity#3, ss_sales_price#4, i_category#16, i_class#15, i_brand#14, null, null, null, null, null, 31], [ss_quantity#3, ss_sales_price#4, i_category#16, i_class#15, null, null, null, null, null, null, 63], [ss_quantity#3, ss_sales_price#4, i_category#16, null, null, null, null, null, null, null, 127], [ss_quantity#3, ss_sales_price#4, null, null, null, null, null, null, null, null, 255]], [ss_quantity#3, ss_sales_price#4, i_category#18, i_class#19, i_brand#20, i_product_name#21, d_year#22, d_qoy#23, d_moy#24, s_store_id#25, spark_grouping_id#26] + +(20) HashAggregate [codegen id : 4] +Input [11]: [ss_quantity#3, ss_sales_price#4, i_category#18, i_class#19, i_brand#20, i_product_name#21, d_year#22, d_qoy#23, d_moy#24, s_store_id#25, spark_grouping_id#26] +Keys [9]: [i_category#18, i_class#19, i_brand#20, i_product_name#21, d_year#22, d_qoy#23, d_moy#24, s_store_id#25, spark_grouping_id#26] +Functions [1]: [partial_sum(coalesce((ss_sales_price#4 * cast(ss_quantity#3 as decimal(10,0))), 0.00))] +Aggregate Attributes [2]: [sum#27, isEmpty#28] +Results [11]: [i_category#18, i_class#19, i_brand#20, i_product_name#21, d_year#22, d_qoy#23, d_moy#24, s_store_id#25, spark_grouping_id#26, sum#29, isEmpty#30] + +(21) Exchange +Input [11]: [i_category#18, i_class#19, i_brand#20, i_product_name#21, d_year#22, d_qoy#23, d_moy#24, s_store_id#25, spark_grouping_id#26, sum#29, isEmpty#30] +Arguments: hashpartitioning(i_category#18, i_class#19, i_brand#20, i_product_name#21, d_year#22, d_qoy#23, d_moy#24, s_store_id#25, spark_grouping_id#26, 5), ENSURE_REQUIREMENTS, [plan_id=3] + +(22) HashAggregate [codegen id : 5] +Input [11]: [i_category#18, i_class#19, i_brand#20, i_product_name#21, d_year#22, d_qoy#23, d_moy#24, s_store_id#25, spark_grouping_id#26, sum#29, isEmpty#30] +Keys [9]: [i_category#18, i_class#19, i_brand#20, i_product_name#21, d_year#22, d_qoy#23, d_moy#24, s_store_id#25, spark_grouping_id#26] +Functions [1]: [sum(coalesce((ss_sales_price#4 * cast(ss_quantity#3 as decimal(10,0))), 0.00))] +Aggregate Attributes [1]: [sum(coalesce((ss_sales_price#4 * cast(ss_quantity#3 as decimal(10,0))), 0.00))#31] +Results [9]: [i_category#18, i_class#19, i_brand#20, i_product_name#21, d_year#22, d_qoy#23, d_moy#24, s_store_id#25, sum(coalesce((ss_sales_price#4 * cast(ss_quantity#3 as decimal(10,0))), 0.00))#31 AS sumsales#32] + +(23) Sort [codegen id : 5] +Input [9]: [i_category#18, i_class#19, i_brand#20, i_product_name#21, d_year#22, d_qoy#23, d_moy#24, s_store_id#25, sumsales#32] +Arguments: [i_category#18 ASC NULLS FIRST, sumsales#32 DESC NULLS LAST], false, 0 + +(24) WindowGroupLimit +Input [9]: [i_category#18, i_class#19, i_brand#20, i_product_name#21, d_year#22, d_qoy#23, d_moy#24, s_store_id#25, sumsales#32] +Arguments: [i_category#18], [sumsales#32 DESC NULLS LAST], rank(sumsales#32), 100, Partial + +(25) Exchange +Input [9]: [i_category#18, i_class#19, i_brand#20, i_product_name#21, d_year#22, d_qoy#23, d_moy#24, s_store_id#25, sumsales#32] +Arguments: hashpartitioning(i_category#18, 5), ENSURE_REQUIREMENTS, [plan_id=4] + +(26) Sort [codegen id : 6] +Input [9]: [i_category#18, i_class#19, i_brand#20, i_product_name#21, d_year#22, d_qoy#23, d_moy#24, s_store_id#25, sumsales#32] +Arguments: [i_category#18 ASC NULLS FIRST, sumsales#32 DESC NULLS LAST], false, 0 + +(27) WindowGroupLimit +Input [9]: [i_category#18, i_class#19, i_brand#20, i_product_name#21, d_year#22, d_qoy#23, d_moy#24, s_store_id#25, sumsales#32] +Arguments: [i_category#18], [sumsales#32 DESC NULLS LAST], rank(sumsales#32), 100, Final + +(28) Window +Input [9]: [i_category#18, i_class#19, i_brand#20, i_product_name#21, d_year#22, d_qoy#23, d_moy#24, s_store_id#25, sumsales#32] +Arguments: [rank(sumsales#32) windowspecdefinition(i_category#18, sumsales#32 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#33], [i_category#18], [sumsales#32 DESC NULLS LAST] + +(29) Filter [codegen id : 7] +Input [10]: [i_category#18, i_class#19, i_brand#20, i_product_name#21, d_year#22, d_qoy#23, d_moy#24, s_store_id#25, sumsales#32, rk#33] +Condition : (rk#33 <= 100) + +(30) TakeOrderedAndProject +Input [10]: [i_category#18, i_class#19, i_brand#20, i_product_name#21, d_year#22, d_qoy#23, d_moy#24, s_store_id#25, sumsales#32, rk#33] +Arguments: 100, [i_category#18 ASC NULLS FIRST, i_class#19 ASC NULLS FIRST, i_brand#20 ASC NULLS FIRST, i_product_name#21 ASC NULLS FIRST, d_year#22 ASC NULLS FIRST, d_qoy#23 ASC NULLS FIRST, d_moy#24 ASC NULLS FIRST, s_store_id#25 ASC NULLS FIRST, sumsales#32 ASC NULLS FIRST, rk#33 ASC NULLS FIRST], [i_category#18, i_class#19, i_brand#20, i_product_name#21, d_year#22, d_qoy#23, d_moy#24, s_store_id#25, sumsales#32, rk#33] + +===== Subqueries ===== + +Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#5 IN dynamicpruning#6 +BroadcastExchange (35) ++- * ColumnarToRow (34) + +- CometProject (33) + +- CometFilter (32) + +- CometScan parquet spark_catalog.default.date_dim (31) + + +(31) Scan parquet spark_catalog.default.date_dim +Output [5]: [d_date_sk#7, d_month_seq#34, d_year#8, d_moy#9, d_qoy#10] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1200), LessThanOrEqual(d_month_seq,1211), IsNotNull(d_date_sk)] +ReadSchema: struct<d_date_sk:int,d_month_seq:int,d_year:int,d_moy:int,d_qoy:int> + +(32) CometFilter +Input [5]: [d_date_sk#7, d_month_seq#34, d_year#8, d_moy#9, d_qoy#10] +Condition : (((isnotnull(d_month_seq#34) AND (d_month_seq#34 >= 1200)) AND (d_month_seq#34 <= 1211)) AND isnotnull(d_date_sk#7)) + +(33) CometProject +Input [5]: [d_date_sk#7, d_month_seq#34, d_year#8, d_moy#9, d_qoy#10] +Arguments: [d_date_sk#7, d_year#8, d_moy#9, d_qoy#10], [d_date_sk#7, d_year#8, d_moy#9, d_qoy#10] + +(34) ColumnarToRow [codegen id : 1] +Input [4]: [d_date_sk#7, d_year#8, d_moy#9, d_qoy#10] + +(35) BroadcastExchange +Input [4]: [d_date_sk#7, d_year#8, d_moy#9, d_qoy#10] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] + + 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 new file mode 100644 index 000000000..e10def397 --- /dev/null +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q67/simplified.txt @@ -0,0 +1,53 @@ +TakeOrderedAndProject [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,rk] + WholeStageCodegen (7) + Filter [rk] + InputAdapter + Window [sumsales,i_category] + WindowGroupLimit [i_category,sumsales] + WholeStageCodegen (6) + Sort [i_category,sumsales] + InputAdapter + Exchange [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] + InputAdapter + Exchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,spark_grouping_id] #2 + WholeStageCodegen (4) + HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,spark_grouping_id,ss_sales_price,ss_quantity] [sum,isEmpty,sum,isEmpty] + Expand [ss_quantity,ss_sales_price,i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id] + Project [ss_quantity,ss_sales_price,i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id] + BroadcastHashJoin [ss_item_sk,i_item_sk] + Project [ss_item_sk,ss_quantity,ss_sales_price,d_year,d_moy,d_qoy,s_store_id] + BroadcastHashJoin [ss_store_sk,s_store_sk] + Project [ss_item_sk,ss_store_sk,ss_quantity,ss_sales_price,d_year,d_moy,d_qoy] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + ColumnarToRow + InputAdapter + CometFilter [ss_store_sk,ss_item_sk] + CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_quantity,ss_sales_price,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk,d_year,d_moy,d_qoy] + CometFilter [d_month_seq,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_year,d_moy,d_qoy] + InputAdapter + ReusedExchange [d_date_sk,d_year,d_moy,d_qoy] #3 + InputAdapter + BroadcastExchange #4 + WholeStageCodegen (2) + ColumnarToRow + InputAdapter + CometFilter [s_store_sk] + CometScan parquet spark_catalog.default.store [s_store_sk,s_store_id] + InputAdapter + BroadcastExchange #5 + WholeStageCodegen (3) + ColumnarToRow + InputAdapter + CometFilter [i_item_sk] + CometScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category,i_product_name] 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 new file mode 100644 index 000000000..734b6c11c --- /dev/null +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q68/explain.txt @@ -0,0 +1,258 @@ +== Physical Plan == +TakeOrderedAndProject (39) ++- * Project (38) + +- * BroadcastHashJoin Inner BuildRight (37) + :- * Project (35) + : +- * BroadcastHashJoin Inner BuildRight (34) + : :- * HashAggregate (29) + : : +- Exchange (28) + : : +- * HashAggregate (27) + : : +- * Project (26) + : : +- * BroadcastHashJoin Inner BuildRight (25) + : : :- * Project (20) + : : : +- * BroadcastHashJoin Inner BuildRight (19) + : : : :- * Project (13) + : : : : +- * BroadcastHashJoin Inner BuildRight (12) + : : : : :- * Project (6) + : : : : : +- * BroadcastHashJoin Inner BuildRight (5) + : : : : : :- * ColumnarToRow (3) + : : : : : : +- CometFilter (2) + : : : : : : +- CometScan parquet spark_catalog.default.store_sales (1) + : : : : : +- ReusedExchange (4) + : : : : +- BroadcastExchange (11) + : : : : +- * ColumnarToRow (10) + : : : : +- CometProject (9) + : : : : +- CometFilter (8) + : : : : +- CometScan parquet spark_catalog.default.store (7) + : : : +- BroadcastExchange (18) + : : : +- * ColumnarToRow (17) + : : : +- CometProject (16) + : : : +- CometFilter (15) + : : : +- CometScan parquet spark_catalog.default.household_demographics (14) + : : +- BroadcastExchange (24) + : : +- * ColumnarToRow (23) + : : +- CometFilter (22) + : : +- CometScan parquet spark_catalog.default.customer_address (21) + : +- BroadcastExchange (33) + : +- * ColumnarToRow (32) + : +- CometFilter (31) + : +- CometScan parquet spark_catalog.default.customer (30) + +- ReusedExchange (36) + + +(1) Scan parquet spark_catalog.default.store_sales +Output [9]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_store_sk#4, ss_ticket_number#5, ss_ext_sales_price#6, ss_ext_list_price#7, ss_ext_tax#8, ss_sold_date_sk#9] +Batched: true +Location: InMemoryFileIndex [] +PartitionFilters: [isnotnull(ss_sold_date_sk#9), dynamicpruningexpression(ss_sold_date_sk#9 IN dynamicpruning#10)] +PushedFilters: [IsNotNull(ss_store_sk), IsNotNull(ss_hdemo_sk), IsNotNull(ss_addr_sk), IsNotNull(ss_customer_sk)] +ReadSchema: struct<ss_customer_sk:int,ss_hdemo_sk:int,ss_addr_sk:int,ss_store_sk:int,ss_ticket_number:int,ss_ext_sales_price:decimal(7,2),ss_ext_list_price:decimal(7,2),ss_ext_tax:decimal(7,2)> + +(2) CometFilter +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)) + +(3) ColumnarToRow [codegen id : 5] +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] + +(4) ReusedExchange [Reuses operator id: 44] +Output [1]: [d_date_sk#11] + +(5) BroadcastHashJoin [codegen id : 5] +Left keys [1]: [ss_sold_date_sk#9] +Right keys [1]: [d_date_sk#11] +Join type: Inner +Join condition: None + +(6) Project [codegen id : 5] +Output [8]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_store_sk#4, ss_ticket_number#5, ss_ext_sales_price#6, ss_ext_list_price#7, ss_ext_tax#8] +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] + +(7) Scan parquet spark_catalog.default.store +Output [2]: [s_store_sk#12, s_city#13] +Batched: true +Location [not included in comparison]/{warehouse_dir}/store] +PushedFilters: [In(s_city, [Fairview,Midway]), IsNotNull(s_store_sk)] +ReadSchema: struct<s_store_sk:int,s_city:string> + +(8) CometFilter +Input [2]: [s_store_sk#12, s_city#13] +Condition : (s_city#13 IN (Midway,Fairview) AND isnotnull(s_store_sk#12)) + +(9) CometProject +Input [2]: [s_store_sk#12, s_city#13] +Arguments: [s_store_sk#12], [s_store_sk#12] + +(10) ColumnarToRow [codegen id : 2] +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] +Left keys [1]: [ss_store_sk#4] +Right keys [1]: [s_store_sk#12] +Join type: Inner +Join condition: None + +(13) Project [codegen id : 5] +Output [7]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_ticket_number#5, ss_ext_sales_price#6, ss_ext_list_price#7, ss_ext_tax#8] +Input [9]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_store_sk#4, ss_ticket_number#5, ss_ext_sales_price#6, ss_ext_list_price#7, ss_ext_tax#8, s_store_sk#12] + +(14) Scan parquet spark_catalog.default.household_demographics +Output [3]: [hd_demo_sk#14, hd_dep_count#15, hd_vehicle_count#16] +Batched: true +Location [not included in comparison]/{warehouse_dir}/household_demographics] +PushedFilters: [Or(EqualTo(hd_dep_count,4),EqualTo(hd_vehicle_count,3)), IsNotNull(hd_demo_sk)] +ReadSchema: struct<hd_demo_sk:int,hd_dep_count:int,hd_vehicle_count:int> + +(15) CometFilter +Input [3]: [hd_demo_sk#14, hd_dep_count#15, hd_vehicle_count#16] +Condition : (((hd_dep_count#15 = 4) OR (hd_vehicle_count#16 = 3)) AND isnotnull(hd_demo_sk#14)) + +(16) CometProject +Input [3]: [hd_demo_sk#14, hd_dep_count#15, hd_vehicle_count#16] +Arguments: [hd_demo_sk#14], [hd_demo_sk#14] + +(17) ColumnarToRow [codegen id : 3] +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] +Left keys [1]: [ss_hdemo_sk#2] +Right keys [1]: [hd_demo_sk#14] +Join type: Inner +Join condition: None + +(20) Project [codegen id : 5] +Output [6]: [ss_customer_sk#1, ss_addr_sk#3, ss_ticket_number#5, ss_ext_sales_price#6, ss_ext_list_price#7, ss_ext_tax#8] +Input [8]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_ticket_number#5, ss_ext_sales_price#6, ss_ext_list_price#7, ss_ext_tax#8, hd_demo_sk#14] + +(21) Scan parquet spark_catalog.default.customer_address +Output [2]: [ca_address_sk#17, ca_city#18] +Batched: true +Location [not included in comparison]/{warehouse_dir}/customer_address] +PushedFilters: [IsNotNull(ca_address_sk), IsNotNull(ca_city)] +ReadSchema: struct<ca_address_sk:int,ca_city:string> + +(22) CometFilter +Input [2]: [ca_address_sk#17, ca_city#18] +Condition : (isnotnull(ca_address_sk#17) AND isnotnull(ca_city#18)) + +(23) ColumnarToRow [codegen id : 4] +Input [2]: [ca_address_sk#17, ca_city#18] + +(24) BroadcastExchange +Input [2]: [ca_address_sk#17, ca_city#18] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=3] + +(25) BroadcastHashJoin [codegen id : 5] +Left keys [1]: [ss_addr_sk#3] +Right keys [1]: [ca_address_sk#17] +Join type: Inner +Join condition: None + +(26) Project [codegen id : 5] +Output [7]: [ss_customer_sk#1, ss_addr_sk#3, ss_ticket_number#5, ss_ext_sales_price#6, ss_ext_list_price#7, ss_ext_tax#8, ca_city#18] +Input [8]: [ss_customer_sk#1, ss_addr_sk#3, ss_ticket_number#5, ss_ext_sales_price#6, ss_ext_list_price#7, ss_ext_tax#8, ca_address_sk#17, ca_city#18] + +(27) HashAggregate [codegen id : 5] +Input [7]: [ss_customer_sk#1, ss_addr_sk#3, ss_ticket_number#5, ss_ext_sales_price#6, ss_ext_list_price#7, ss_ext_tax#8, ca_city#18] +Keys [4]: [ss_ticket_number#5, ss_customer_sk#1, ss_addr_sk#3, ca_city#18] +Functions [3]: [partial_sum(UnscaledValue(ss_ext_sales_price#6)), partial_sum(UnscaledValue(ss_ext_list_price#7)), partial_sum(UnscaledValue(ss_ext_tax#8))] +Aggregate Attributes [3]: [sum#19, sum#20, sum#21] +Results [7]: [ss_ticket_number#5, ss_customer_sk#1, ss_addr_sk#3, ca_city#18, sum#22, sum#23, sum#24] + +(28) Exchange +Input [7]: [ss_ticket_number#5, ss_customer_sk#1, ss_addr_sk#3, ca_city#18, sum#22, sum#23, sum#24] +Arguments: hashpartitioning(ss_ticket_number#5, ss_customer_sk#1, ss_addr_sk#3, ca_city#18, 5), ENSURE_REQUIREMENTS, [plan_id=4] + +(29) HashAggregate [codegen id : 8] +Input [7]: [ss_ticket_number#5, ss_customer_sk#1, ss_addr_sk#3, ca_city#18, sum#22, sum#23, sum#24] +Keys [4]: [ss_ticket_number#5, ss_customer_sk#1, ss_addr_sk#3, ca_city#18] +Functions [3]: [sum(UnscaledValue(ss_ext_sales_price#6)), sum(UnscaledValue(ss_ext_list_price#7)), sum(UnscaledValue(ss_ext_tax#8))] +Aggregate Attributes [3]: [sum(UnscaledValue(ss_ext_sales_price#6))#25, sum(UnscaledValue(ss_ext_list_price#7))#26, sum(UnscaledValue(ss_ext_tax#8))#27] +Results [6]: [ss_ticket_number#5, ss_customer_sk#1, ca_city#18 AS bought_city#28, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#6))#25,17,2) AS extended_price#29, MakeDecimal(sum(UnscaledValue(ss_ext_list_price#7))#26,17,2) AS list_price#30, MakeDecimal(sum(UnscaledValue(ss_ext_tax#8))#27,17,2) AS extended_tax#31] + +(30) Scan parquet spark_catalog.default.customer +Output [4]: [c_customer_sk#32, c_current_addr_sk#33, c_first_name#34, c_last_name#35] +Batched: true +Location [not included in comparison]/{warehouse_dir}/customer] +PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_current_addr_sk)] +ReadSchema: struct<c_customer_sk:int,c_current_addr_sk:int,c_first_name:string,c_last_name:string> + +(31) CometFilter +Input [4]: [c_customer_sk#32, c_current_addr_sk#33, c_first_name#34, c_last_name#35] +Condition : (isnotnull(c_customer_sk#32) AND isnotnull(c_current_addr_sk#33)) + +(32) ColumnarToRow [codegen id : 6] +Input [4]: [c_customer_sk#32, c_current_addr_sk#33, c_first_name#34, c_last_name#35] + +(33) BroadcastExchange +Input [4]: [c_customer_sk#32, c_current_addr_sk#33, c_first_name#34, c_last_name#35] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=5] + +(34) BroadcastHashJoin [codegen id : 8] +Left keys [1]: [ss_customer_sk#1] +Right keys [1]: [c_customer_sk#32] +Join type: Inner +Join condition: None + +(35) Project [codegen id : 8] +Output [8]: [ss_ticket_number#5, bought_city#28, extended_price#29, list_price#30, extended_tax#31, c_current_addr_sk#33, c_first_name#34, c_last_name#35] +Input [10]: [ss_ticket_number#5, ss_customer_sk#1, bought_city#28, extended_price#29, list_price#30, extended_tax#31, c_customer_sk#32, c_current_addr_sk#33, c_first_name#34, c_last_name#35] + +(36) ReusedExchange [Reuses operator id: 24] +Output [2]: [ca_address_sk#36, ca_city#37] + +(37) BroadcastHashJoin [codegen id : 8] +Left keys [1]: [c_current_addr_sk#33] +Right keys [1]: [ca_address_sk#36] +Join type: Inner +Join condition: NOT (ca_city#37 = bought_city#28) + +(38) Project [codegen id : 8] +Output [8]: [c_last_name#35, c_first_name#34, ca_city#37, bought_city#28, ss_ticket_number#5, extended_price#29, extended_tax#31, list_price#30] +Input [10]: [ss_ticket_number#5, bought_city#28, extended_price#29, list_price#30, extended_tax#31, c_current_addr_sk#33, c_first_name#34, c_last_name#35, ca_address_sk#36, ca_city#37] + +(39) TakeOrderedAndProject +Input [8]: [c_last_name#35, c_first_name#34, ca_city#37, bought_city#28, ss_ticket_number#5, extended_price#29, extended_tax#31, list_price#30] +Arguments: 100, [c_last_name#35 ASC NULLS FIRST, ss_ticket_number#5 ASC NULLS FIRST], [c_last_name#35, c_first_name#34, ca_city#37, bought_city#28, ss_ticket_number#5, extended_price#29, extended_tax#31, list_price#30] + +===== Subqueries ===== + +Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#9 IN dynamicpruning#10 +BroadcastExchange (44) ++- * ColumnarToRow (43) + +- CometProject (42) + +- CometFilter (41) + +- CometScan parquet spark_catalog.default.date_dim (40) + + +(40) Scan parquet spark_catalog.default.date_dim +Output [3]: [d_date_sk#11, d_year#38, d_dom#39] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_dom), GreaterThanOrEqual(d_dom,1), LessThanOrEqual(d_dom,2), In(d_year, [1999,2000,2001]), IsNotNull(d_date_sk)] +ReadSchema: struct<d_date_sk:int,d_year:int,d_dom:int> + +(41) CometFilter +Input [3]: [d_date_sk#11, d_year#38, d_dom#39] +Condition : ((((isnotnull(d_dom#39) AND (d_dom#39 >= 1)) AND (d_dom#39 <= 2)) AND d_year#38 IN (1999,2000,2001)) AND isnotnull(d_date_sk#11)) + +(42) CometProject +Input [3]: [d_date_sk#11, d_year#38, d_dom#39] +Arguments: [d_date_sk#11], [d_date_sk#11] + +(43) ColumnarToRow [codegen id : 1] +Input [1]: [d_date_sk#11] + +(44) BroadcastExchange +Input [1]: [d_date_sk#11] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=6] + + diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q68/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q68/simplified.txt new file mode 100644 index 000000000..f2680bebb --- /dev/null +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q68/simplified.txt @@ -0,0 +1,65 @@ +TakeOrderedAndProject [c_last_name,ss_ticket_number,c_first_name,ca_city,bought_city,extended_price,extended_tax,list_price] + WholeStageCodegen (8) + Project [c_last_name,c_first_name,ca_city,bought_city,ss_ticket_number,extended_price,extended_tax,list_price] + BroadcastHashJoin [c_current_addr_sk,ca_address_sk,ca_city,bought_city] + Project [ss_ticket_number,bought_city,extended_price,list_price,extended_tax,c_current_addr_sk,c_first_name,c_last_name] + BroadcastHashJoin [ss_customer_sk,c_customer_sk] + HashAggregate [ss_ticket_number,ss_customer_sk,ss_addr_sk,ca_city,sum,sum,sum] [sum(UnscaledValue(ss_ext_sales_price)),sum(UnscaledValue(ss_ext_list_price)),sum(UnscaledValue(ss_ext_tax)),bought_city,extended_price,list_price,extended_tax,sum,sum,sum] + InputAdapter + Exchange [ss_ticket_number,ss_customer_sk,ss_addr_sk,ca_city] #1 + WholeStageCodegen (5) + HashAggregate [ss_ticket_number,ss_customer_sk,ss_addr_sk,ca_city,ss_ext_sales_price,ss_ext_list_price,ss_ext_tax] [sum,sum,sum,sum,sum,sum] + Project [ss_customer_sk,ss_addr_sk,ss_ticket_number,ss_ext_sales_price,ss_ext_list_price,ss_ext_tax,ca_city] + BroadcastHashJoin [ss_addr_sk,ca_address_sk] + Project [ss_customer_sk,ss_addr_sk,ss_ticket_number,ss_ext_sales_price,ss_ext_list_price,ss_ext_tax] + BroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] + Project [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_ticket_number,ss_ext_sales_price,ss_ext_list_price,ss_ext_tax] + BroadcastHashJoin [ss_store_sk,s_store_sk] + Project [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_ticket_number,ss_ext_sales_price,ss_ext_list_price,ss_ext_tax] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + ColumnarToRow + InputAdapter + CometFilter [ss_store_sk,ss_hdemo_sk,ss_addr_sk,ss_customer_sk] + CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_ticket_number,ss_ext_sales_price,ss_ext_list_price,ss_ext_tax,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #2 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_dom,d_year,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dom] + InputAdapter + ReusedExchange [d_date_sk] #2 + InputAdapter + BroadcastExchange #3 + WholeStageCodegen (2) + ColumnarToRow + InputAdapter + CometProject [s_store_sk] + CometFilter [s_city,s_store_sk] + CometScan parquet spark_catalog.default.store [s_store_sk,s_city] + InputAdapter + BroadcastExchange #4 + WholeStageCodegen (3) + ColumnarToRow + InputAdapter + CometProject [hd_demo_sk] + CometFilter [hd_dep_count,hd_vehicle_count,hd_demo_sk] + CometScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_dep_count,hd_vehicle_count] + InputAdapter + BroadcastExchange #5 + WholeStageCodegen (4) + ColumnarToRow + InputAdapter + CometFilter [ca_address_sk,ca_city] + CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_city] + InputAdapter + BroadcastExchange #6 + WholeStageCodegen (6) + ColumnarToRow + InputAdapter + CometFilter [c_customer_sk,c_current_addr_sk] + CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name] + InputAdapter + ReusedExchange [ca_address_sk,ca_city] #5 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 new file mode 100644 index 000000000..d56b01d73 --- /dev/null +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q69/explain.txt @@ -0,0 +1,281 @@ +== Physical Plan == +TakeOrderedAndProject (42) ++- * HashAggregate (41) + +- Exchange (40) + +- * HashAggregate (39) + +- * Project (38) + +- * BroadcastHashJoin Inner BuildRight (37) + :- * Project (32) + : +- * BroadcastHashJoin Inner BuildRight (31) + : :- * Project (25) + : : +- * BroadcastHashJoin LeftAnti BuildRight (24) + : : :- * BroadcastHashJoin LeftAnti BuildRight (17) + : : : :- * BroadcastHashJoin LeftSemi BuildRight (10) + : : : : :- * ColumnarToRow (3) + : : : : : +- CometFilter (2) + : : : : : +- CometScan parquet spark_catalog.default.customer (1) + : : : : +- BroadcastExchange (9) + : : : : +- * Project (8) + : : : : +- * BroadcastHashJoin Inner BuildRight (7) + : : : : :- * ColumnarToRow (5) + : : : : : +- CometScan parquet spark_catalog.default.store_sales (4) + : : : : +- ReusedExchange (6) + : : : +- BroadcastExchange (16) + : : : +- * Project (15) + : : : +- * BroadcastHashJoin Inner BuildRight (14) + : : : :- * ColumnarToRow (12) + : : : : +- CometScan parquet spark_catalog.default.web_sales (11) + : : : +- ReusedExchange (13) + : : +- BroadcastExchange (23) + : : +- * Project (22) + : : +- * BroadcastHashJoin Inner BuildRight (21) + : : :- * ColumnarToRow (19) + : : : +- CometScan parquet spark_catalog.default.catalog_sales (18) + : : +- ReusedExchange (20) + : +- BroadcastExchange (30) + : +- * ColumnarToRow (29) + : +- CometProject (28) + : +- CometFilter (27) + : +- CometScan parquet spark_catalog.default.customer_address (26) + +- BroadcastExchange (36) + +- * ColumnarToRow (35) + +- CometFilter (34) + +- CometScan parquet spark_catalog.default.customer_demographics (33) + + +(1) Scan parquet spark_catalog.default.customer +Output [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] +Batched: true +Location [not included in comparison]/{warehouse_dir}/customer] +PushedFilters: [IsNotNull(c_current_addr_sk), IsNotNull(c_current_cdemo_sk)] +ReadSchema: struct<c_customer_sk:int,c_current_cdemo_sk:int,c_current_addr_sk:int> + +(2) CometFilter +Input [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] +Condition : (isnotnull(c_current_addr_sk#3) AND isnotnull(c_current_cdemo_sk#2)) + +(3) ColumnarToRow [codegen id : 9] +Input [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] + +(4) Scan parquet spark_catalog.default.store_sales +Output [2]: [ss_customer_sk#4, ss_sold_date_sk#5] +Batched: true +Location: InMemoryFileIndex [] +PartitionFilters: [isnotnull(ss_sold_date_sk#5), dynamicpruningexpression(ss_sold_date_sk#5 IN dynamicpruning#6)] +ReadSchema: struct<ss_customer_sk:int> + +(5) ColumnarToRow [codegen id : 2] +Input [2]: [ss_customer_sk#4, ss_sold_date_sk#5] + +(6) ReusedExchange [Reuses operator id: 47] +Output [1]: [d_date_sk#7] + +(7) BroadcastHashJoin [codegen id : 2] +Left keys [1]: [ss_sold_date_sk#5] +Right keys [1]: [d_date_sk#7] +Join type: Inner +Join condition: None + +(8) Project [codegen id : 2] +Output [1]: [ss_customer_sk#4] +Input [3]: [ss_customer_sk#4, ss_sold_date_sk#5, d_date_sk#7] + +(9) BroadcastExchange +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] +Left keys [1]: [c_customer_sk#1] +Right keys [1]: [ss_customer_sk#4] +Join type: LeftSemi +Join condition: None + +(11) Scan parquet spark_catalog.default.web_sales +Output [2]: [ws_bill_customer_sk#8, ws_sold_date_sk#9] +Batched: true +Location: InMemoryFileIndex [] +PartitionFilters: [isnotnull(ws_sold_date_sk#9), dynamicpruningexpression(ws_sold_date_sk#9 IN dynamicpruning#10)] +ReadSchema: struct<ws_bill_customer_sk:int> + +(12) ColumnarToRow [codegen id : 4] +Input [2]: [ws_bill_customer_sk#8, ws_sold_date_sk#9] + +(13) ReusedExchange [Reuses operator id: 47] +Output [1]: [d_date_sk#11] + +(14) BroadcastHashJoin [codegen id : 4] +Left keys [1]: [ws_sold_date_sk#9] +Right keys [1]: [d_date_sk#11] +Join type: Inner +Join condition: None + +(15) Project [codegen id : 4] +Output [1]: [ws_bill_customer_sk#8] +Input [3]: [ws_bill_customer_sk#8, ws_sold_date_sk#9, d_date_sk#11] + +(16) BroadcastExchange +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] +Left keys [1]: [c_customer_sk#1] +Right keys [1]: [ws_bill_customer_sk#8] +Join type: LeftAnti +Join condition: None + +(18) Scan parquet spark_catalog.default.catalog_sales +Output [2]: [cs_ship_customer_sk#12, cs_sold_date_sk#13] +Batched: true +Location: InMemoryFileIndex [] +PartitionFilters: [isnotnull(cs_sold_date_sk#13), dynamicpruningexpression(cs_sold_date_sk#13 IN dynamicpruning#14)] +ReadSchema: struct<cs_ship_customer_sk:int> + +(19) ColumnarToRow [codegen id : 6] +Input [2]: [cs_ship_customer_sk#12, cs_sold_date_sk#13] + +(20) ReusedExchange [Reuses operator id: 47] +Output [1]: [d_date_sk#15] + +(21) BroadcastHashJoin [codegen id : 6] +Left keys [1]: [cs_sold_date_sk#13] +Right keys [1]: [d_date_sk#15] +Join type: Inner +Join condition: None + +(22) Project [codegen id : 6] +Output [1]: [cs_ship_customer_sk#12] +Input [3]: [cs_ship_customer_sk#12, cs_sold_date_sk#13, d_date_sk#15] + +(23) BroadcastExchange +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] +Left keys [1]: [c_customer_sk#1] +Right keys [1]: [cs_ship_customer_sk#12] +Join type: LeftAnti +Join condition: None + +(25) Project [codegen id : 9] +Output [2]: [c_current_cdemo_sk#2, c_current_addr_sk#3] +Input [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] + +(26) Scan parquet spark_catalog.default.customer_address +Output [2]: [ca_address_sk#16, ca_state#17] +Batched: true +Location [not included in comparison]/{warehouse_dir}/customer_address] +PushedFilters: [In(ca_state, [GA,KY,NM]), IsNotNull(ca_address_sk)] +ReadSchema: struct<ca_address_sk:int,ca_state:string> + +(27) CometFilter +Input [2]: [ca_address_sk#16, ca_state#17] +Condition : (ca_state#17 IN (KY,GA,NM) AND isnotnull(ca_address_sk#16)) + +(28) CometProject +Input [2]: [ca_address_sk#16, ca_state#17] +Arguments: [ca_address_sk#16], [ca_address_sk#16] + +(29) ColumnarToRow [codegen id : 7] +Input [1]: [ca_address_sk#16] + +(30) BroadcastExchange +Input [1]: [ca_address_sk#16] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] + +(31) BroadcastHashJoin [codegen id : 9] +Left keys [1]: [c_current_addr_sk#3] +Right keys [1]: [ca_address_sk#16] +Join type: Inner +Join condition: None + +(32) Project [codegen id : 9] +Output [1]: [c_current_cdemo_sk#2] +Input [3]: [c_current_cdemo_sk#2, c_current_addr_sk#3, ca_address_sk#16] + +(33) Scan parquet spark_catalog.default.customer_demographics +Output [6]: [cd_demo_sk#18, cd_gender#19, cd_marital_status#20, cd_education_status#21, cd_purchase_estimate#22, cd_credit_rating#23] +Batched: true +Location [not included in comparison]/{warehouse_dir}/customer_demographics] +PushedFilters: [IsNotNull(cd_demo_sk)] +ReadSchema: struct<cd_demo_sk:int,cd_gender:string,cd_marital_status:string,cd_education_status:string,cd_purchase_estimate:int,cd_credit_rating:string> + +(34) CometFilter +Input [6]: [cd_demo_sk#18, cd_gender#19, cd_marital_status#20, cd_education_status#21, cd_purchase_estimate#22, cd_credit_rating#23] +Condition : isnotnull(cd_demo_sk#18) + +(35) ColumnarToRow [codegen id : 8] +Input [6]: [cd_demo_sk#18, cd_gender#19, cd_marital_status#20, cd_education_status#21, cd_purchase_estimate#22, cd_credit_rating#23] + +(36) BroadcastExchange +Input [6]: [cd_demo_sk#18, cd_gender#19, cd_marital_status#20, cd_education_status#21, cd_purchase_estimate#22, cd_credit_rating#23] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=5] + +(37) BroadcastHashJoin [codegen id : 9] +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] +Output [5]: [cd_gender#19, cd_marital_status#20, cd_education_status#21, cd_purchase_estimate#22, cd_credit_rating#23] +Input [7]: [c_current_cdemo_sk#2, cd_demo_sk#18, cd_gender#19, cd_marital_status#20, cd_education_status#21, cd_purchase_estimate#22, cd_credit_rating#23] + +(39) HashAggregate [codegen id : 9] +Input [5]: [cd_gender#19, cd_marital_status#20, cd_education_status#21, cd_purchase_estimate#22, cd_credit_rating#23] +Keys [5]: [cd_gender#19, cd_marital_status#20, cd_education_status#21, cd_purchase_estimate#22, cd_credit_rating#23] +Functions [1]: [partial_count(1)] +Aggregate Attributes [1]: [count#24] +Results [6]: [cd_gender#19, cd_marital_status#20, cd_education_status#21, cd_purchase_estimate#22, cd_credit_rating#23, count#25] + +(40) Exchange +Input [6]: [cd_gender#19, cd_marital_status#20, cd_education_status#21, cd_purchase_estimate#22, cd_credit_rating#23, count#25] +Arguments: hashpartitioning(cd_gender#19, cd_marital_status#20, cd_education_status#21, cd_purchase_estimate#22, cd_credit_rating#23, 5), ENSURE_REQUIREMENTS, [plan_id=6] + +(41) HashAggregate [codegen id : 10] +Input [6]: [cd_gender#19, cd_marital_status#20, cd_education_status#21, cd_purchase_estimate#22, cd_credit_rating#23, count#25] +Keys [5]: [cd_gender#19, cd_marital_status#20, cd_education_status#21, cd_purchase_estimate#22, cd_credit_rating#23] +Functions [1]: [count(1)] +Aggregate Attributes [1]: [count(1)#26] +Results [8]: [cd_gender#19, cd_marital_status#20, cd_education_status#21, count(1)#26 AS cnt1#27, cd_purchase_estimate#22, count(1)#26 AS cnt2#28, cd_credit_rating#23, count(1)#26 AS cnt3#29] + +(42) TakeOrderedAndProject +Input [8]: [cd_gender#19, cd_marital_status#20, cd_education_status#21, cnt1#27, cd_purchase_estimate#22, cnt2#28, cd_credit_rating#23, cnt3#29] +Arguments: 100, [cd_gender#19 ASC NULLS FIRST, cd_marital_status#20 ASC NULLS FIRST, cd_education_status#21 ASC NULLS FIRST, cd_purchase_estimate#22 ASC NULLS FIRST, cd_credit_rating#23 ASC NULLS FIRST], [cd_gender#19, cd_marital_status#20, cd_education_status#21, cnt1#27, cd_purchase_estimate#22, cnt2#28, cd_credit_rating#23, cnt3#29] + +===== Subqueries ===== + +Subquery:1 Hosting operator id = 4 Hosting Expression = ss_sold_date_sk#5 IN dynamicpruning#6 +BroadcastExchange (47) ++- * ColumnarToRow (46) + +- CometProject (45) + +- CometFilter (44) + +- CometScan parquet spark_catalog.default.date_dim (43) + + +(43) Scan parquet spark_catalog.default.date_dim +Output [3]: [d_date_sk#7, d_year#30, d_moy#31] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2001), GreaterThanOrEqual(d_moy,4), LessThanOrEqual(d_moy,6), IsNotNull(d_date_sk)] +ReadSchema: struct<d_date_sk:int,d_year:int,d_moy:int> + +(44) CometFilter +Input [3]: [d_date_sk#7, d_year#30, d_moy#31] +Condition : (((((isnotnull(d_year#30) AND isnotnull(d_moy#31)) AND (d_year#30 = 2001)) AND (d_moy#31 >= 4)) AND (d_moy#31 <= 6)) AND isnotnull(d_date_sk#7)) + +(45) CometProject +Input [3]: [d_date_sk#7, d_year#30, d_moy#31] +Arguments: [d_date_sk#7], [d_date_sk#7] + +(46) ColumnarToRow [codegen id : 1] +Input [1]: [d_date_sk#7] + +(47) BroadcastExchange +Input [1]: [d_date_sk#7] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=7] + +Subquery:2 Hosting operator id = 11 Hosting Expression = ws_sold_date_sk#9 IN dynamicpruning#6 + +Subquery:3 Hosting operator id = 18 Hosting Expression = cs_sold_date_sk#13 IN dynamicpruning#6 + + diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q69/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q69/simplified.txt new file mode 100644 index 000000000..f5b4eccfb --- /dev/null +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q69/simplified.txt @@ -0,0 +1,74 @@ +TakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cnt1,cnt2,cnt3] + WholeStageCodegen (10) + HashAggregate [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,count] [count(1),cnt1,cnt2,cnt3,count] + InputAdapter + Exchange [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating] #1 + WholeStageCodegen (9) + HashAggregate [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating] [count,count] + Project [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating] + BroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] + Project [c_current_cdemo_sk] + BroadcastHashJoin [c_current_addr_sk,ca_address_sk] + Project [c_current_cdemo_sk,c_current_addr_sk] + BroadcastHashJoin [c_customer_sk,cs_ship_customer_sk] + BroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] + BroadcastHashJoin [c_customer_sk,ss_customer_sk] + ColumnarToRow + InputAdapter + CometFilter [c_current_addr_sk,c_current_cdemo_sk] + CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] + InputAdapter + BroadcastExchange #2 + WholeStageCodegen (2) + Project [ss_customer_sk] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + ColumnarToRow + InputAdapter + CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_year,d_moy,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + InputAdapter + ReusedExchange [d_date_sk] #3 + InputAdapter + BroadcastExchange #4 + WholeStageCodegen (4) + Project [ws_bill_customer_sk] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + ColumnarToRow + InputAdapter + CometScan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + InputAdapter + ReusedExchange [d_date_sk] #3 + InputAdapter + BroadcastExchange #5 + WholeStageCodegen (6) + Project [cs_ship_customer_sk] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + ColumnarToRow + InputAdapter + CometScan parquet spark_catalog.default.catalog_sales [cs_ship_customer_sk,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + InputAdapter + ReusedExchange [d_date_sk] #3 + InputAdapter + BroadcastExchange #6 + WholeStageCodegen (7) + ColumnarToRow + InputAdapter + CometProject [ca_address_sk] + CometFilter [ca_state,ca_address_sk] + CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] + InputAdapter + BroadcastExchange #7 + WholeStageCodegen (8) + ColumnarToRow + InputAdapter + CometFilter [cd_demo_sk] + CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating] 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 new file mode 100644 index 000000000..790a917e3 --- /dev/null +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q7/explain.txt @@ -0,0 +1,208 @@ +== Physical Plan == +TakeOrderedAndProject (30) ++- * HashAggregate (29) + +- Exchange (28) + +- * HashAggregate (27) + +- * Project (26) + +- * BroadcastHashJoin Inner BuildRight (25) + :- * Project (19) + : +- * BroadcastHashJoin Inner BuildRight (18) + : :- * Project (13) + : : +- * BroadcastHashJoin Inner BuildRight (12) + : : :- * Project (10) + : : : +- * BroadcastHashJoin Inner BuildRight (9) + : : : :- * ColumnarToRow (3) + : : : : +- CometFilter (2) + : : : : +- CometScan parquet spark_catalog.default.store_sales (1) + : : : +- BroadcastExchange (8) + : : : +- * ColumnarToRow (7) + : : : +- CometProject (6) + : : : +- CometFilter (5) + : : : +- CometScan parquet spark_catalog.default.customer_demographics (4) + : : +- ReusedExchange (11) + : +- BroadcastExchange (17) + : +- * ColumnarToRow (16) + : +- CometFilter (15) + : +- CometScan parquet spark_catalog.default.item (14) + +- BroadcastExchange (24) + +- * ColumnarToRow (23) + +- CometProject (22) + +- CometFilter (21) + +- CometScan parquet spark_catalog.default.promotion (20) + + +(1) Scan parquet spark_catalog.default.store_sales +Output [8]: [ss_item_sk#1, ss_cdemo_sk#2, ss_promo_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8] +Batched: true +Location: InMemoryFileIndex [] +PartitionFilters: [isnotnull(ss_sold_date_sk#8), dynamicpruningexpression(ss_sold_date_sk#8 IN dynamicpruning#9)] +PushedFilters: [IsNotNull(ss_cdemo_sk), IsNotNull(ss_item_sk), IsNotNull(ss_promo_sk)] +ReadSchema: struct<ss_item_sk:int,ss_cdemo_sk:int,ss_promo_sk:int,ss_quantity:int,ss_list_price:decimal(7,2),ss_sales_price:decimal(7,2),ss_coupon_amt:decimal(7,2)> + +(2) CometFilter +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)) + +(3) ColumnarToRow [codegen id : 5] +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] + +(4) Scan parquet spark_catalog.default.customer_demographics +Output [4]: [cd_demo_sk#10, cd_gender#11, cd_marital_status#12, cd_education_status#13] +Batched: true +Location [not included in comparison]/{warehouse_dir}/customer_demographics] +PushedFilters: [IsNotNull(cd_gender), IsNotNull(cd_marital_status), IsNotNull(cd_education_status), EqualTo(cd_gender,M), EqualTo(cd_marital_status,S), EqualTo(cd_education_status,College ), IsNotNull(cd_demo_sk)] +ReadSchema: struct<cd_demo_sk:int,cd_gender:string,cd_marital_status:string,cd_education_status:string> + +(5) CometFilter +Input [4]: [cd_demo_sk#10, cd_gender#11, cd_marital_status#12, cd_education_status#13] +Condition : ((((((isnotnull(cd_gender#11) AND isnotnull(cd_marital_status#12)) AND isnotnull(cd_education_status#13)) AND (cd_gender#11 = M)) AND (cd_marital_status#12 = S)) AND (cd_education_status#13 = College )) AND isnotnull(cd_demo_sk#10)) + +(6) CometProject +Input [4]: [cd_demo_sk#10, cd_gender#11, cd_marital_status#12, cd_education_status#13] +Arguments: [cd_demo_sk#10], [cd_demo_sk#10] + +(7) ColumnarToRow [codegen id : 1] +Input [1]: [cd_demo_sk#10] + +(8) BroadcastExchange +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] +Left keys [1]: [ss_cdemo_sk#2] +Right keys [1]: [cd_demo_sk#10] +Join type: Inner +Join condition: None + +(10) Project [codegen id : 5] +Output [7]: [ss_item_sk#1, ss_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: 35] +Output [1]: [d_date_sk#14] + +(12) BroadcastHashJoin [codegen id : 5] +Left keys [1]: [ss_sold_date_sk#8] +Right keys [1]: [d_date_sk#14] +Join type: Inner +Join condition: None + +(13) Project [codegen id : 5] +Output [6]: [ss_item_sk#1, ss_promo_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7] +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] + +(14) Scan parquet spark_catalog.default.item +Output [2]: [i_item_sk#15, i_item_id#16] +Batched: true +Location [not included in comparison]/{warehouse_dir}/item] +PushedFilters: [IsNotNull(i_item_sk)] +ReadSchema: struct<i_item_sk:int,i_item_id:string> + +(15) CometFilter +Input [2]: [i_item_sk#15, i_item_id#16] +Condition : isnotnull(i_item_sk#15) + +(16) ColumnarToRow [codegen id : 3] +Input [2]: [i_item_sk#15, i_item_id#16] + +(17) BroadcastExchange +Input [2]: [i_item_sk#15, i_item_id#16] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=2] + +(18) BroadcastHashJoin [codegen id : 5] +Left keys [1]: [ss_item_sk#1] +Right keys [1]: [i_item_sk#15] +Join type: Inner +Join condition: None + +(19) Project [codegen id : 5] +Output [6]: [ss_promo_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, i_item_id#16] +Input [8]: [ss_item_sk#1, ss_promo_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, i_item_sk#15, i_item_id#16] + +(20) Scan parquet spark_catalog.default.promotion +Output [3]: [p_promo_sk#17, p_channel_email#18, p_channel_event#19] +Batched: true +Location [not included in comparison]/{warehouse_dir}/promotion] +PushedFilters: [Or(EqualTo(p_channel_email,N),EqualTo(p_channel_event,N)), IsNotNull(p_promo_sk)] +ReadSchema: struct<p_promo_sk:int,p_channel_email:string,p_channel_event:string> + +(21) CometFilter +Input [3]: [p_promo_sk#17, p_channel_email#18, p_channel_event#19] +Condition : (((p_channel_email#18 = N) OR (p_channel_event#19 = N)) AND isnotnull(p_promo_sk#17)) + +(22) CometProject +Input [3]: [p_promo_sk#17, p_channel_email#18, p_channel_event#19] +Arguments: [p_promo_sk#17], [p_promo_sk#17] + +(23) ColumnarToRow [codegen id : 4] +Input [1]: [p_promo_sk#17] + +(24) BroadcastExchange +Input [1]: [p_promo_sk#17] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] + +(25) BroadcastHashJoin [codegen id : 5] +Left keys [1]: [ss_promo_sk#3] +Right keys [1]: [p_promo_sk#17] +Join type: Inner +Join condition: None + +(26) Project [codegen id : 5] +Output [5]: [ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, i_item_id#16] +Input [7]: [ss_promo_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, i_item_id#16, p_promo_sk#17] + +(27) HashAggregate [codegen id : 5] +Input [5]: [ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, i_item_id#16] +Keys [1]: [i_item_id#16] +Functions [4]: [partial_avg(ss_quantity#4), partial_avg(UnscaledValue(ss_list_price#5)), partial_avg(UnscaledValue(ss_coupon_amt#7)), partial_avg(UnscaledValue(ss_sales_price#6))] +Aggregate Attributes [8]: [sum#20, count#21, sum#22, count#23, sum#24, count#25, sum#26, count#27] +Results [9]: [i_item_id#16, sum#28, count#29, sum#30, count#31, sum#32, count#33, sum#34, count#35] + +(28) Exchange +Input [9]: [i_item_id#16, sum#28, count#29, sum#30, count#31, sum#32, count#33, sum#34, count#35] +Arguments: hashpartitioning(i_item_id#16, 5), ENSURE_REQUIREMENTS, [plan_id=4] + +(29) HashAggregate [codegen id : 6] +Input [9]: [i_item_id#16, sum#28, count#29, sum#30, count#31, sum#32, count#33, sum#34, count#35] +Keys [1]: [i_item_id#16] +Functions [4]: [avg(ss_quantity#4), avg(UnscaledValue(ss_list_price#5)), avg(UnscaledValue(ss_coupon_amt#7)), avg(UnscaledValue(ss_sales_price#6))] +Aggregate Attributes [4]: [avg(ss_quantity#4)#36, avg(UnscaledValue(ss_list_price#5))#37, avg(UnscaledValue(ss_coupon_amt#7))#38, avg(UnscaledValue(ss_sales_price#6))#39] +Results [5]: [i_item_id#16, avg(ss_quantity#4)#36 AS agg1#40, cast((avg(UnscaledValue(ss_list_price#5))#37 / 100.0) as decimal(11,6)) AS agg2#41, cast((avg(UnscaledValue(ss_coupon_amt#7))#38 / 100.0) as decimal(11,6)) AS agg3#42, cast((avg(UnscaledValue(ss_sales_price#6))#39 / 100.0) as decimal(11,6)) AS agg4#43] + +(30) TakeOrderedAndProject +Input [5]: [i_item_id#16, agg1#40, agg2#41, agg3#42, agg4#43] +Arguments: 100, [i_item_id#16 ASC NULLS FIRST], [i_item_id#16, agg1#40, agg2#41, agg3#42, agg4#43] + +===== Subqueries ===== + +Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#8 IN dynamicpruning#9 +BroadcastExchange (35) ++- * ColumnarToRow (34) + +- CometProject (33) + +- CometFilter (32) + +- CometScan parquet spark_catalog.default.date_dim (31) + + +(31) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#14, d_year#44] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2000), IsNotNull(d_date_sk)] +ReadSchema: struct<d_date_sk:int,d_year:int> + +(32) CometFilter +Input [2]: [d_date_sk#14, d_year#44] +Condition : ((isnotnull(d_year#44) AND (d_year#44 = 2000)) AND isnotnull(d_date_sk#14)) + +(33) CometProject +Input [2]: [d_date_sk#14, d_year#44] +Arguments: [d_date_sk#14], [d_date_sk#14] + +(34) ColumnarToRow [codegen id : 1] +Input [1]: [d_date_sk#14] + +(35) BroadcastExchange +Input [1]: [d_date_sk#14] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] + + 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 new file mode 100644 index 000000000..2471de20a --- /dev/null +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q7/simplified.txt @@ -0,0 +1,52 @@ +TakeOrderedAndProject [i_item_id,agg1,agg2,agg3,agg4] + WholeStageCodegen (6) + HashAggregate [i_item_id,sum,count,sum,count,sum,count,sum,count] [avg(ss_quantity),avg(UnscaledValue(ss_list_price)),avg(UnscaledValue(ss_coupon_amt)),avg(UnscaledValue(ss_sales_price)),agg1,agg2,agg3,agg4,sum,count,sum,count,sum,count,sum,count] + InputAdapter + Exchange [i_item_id] #1 + WholeStageCodegen (5) + HashAggregate [i_item_id,ss_quantity,ss_list_price,ss_coupon_amt,ss_sales_price] [sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] + Project [ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,i_item_id] + BroadcastHashJoin [ss_promo_sk,p_promo_sk] + Project [ss_promo_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,i_item_id] + BroadcastHashJoin [ss_item_sk,i_item_sk] + Project [ss_item_sk,ss_promo_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [ss_item_sk,ss_promo_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk] + BroadcastHashJoin [ss_cdemo_sk,cd_demo_sk] + ColumnarToRow + InputAdapter + CometFilter [ss_cdemo_sk,ss_item_sk,ss_promo_sk] + CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_cdemo_sk,ss_promo_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #2 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_year,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + InputAdapter + BroadcastExchange #3 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [cd_demo_sk] + CometFilter [cd_gender,cd_marital_status,cd_education_status,cd_demo_sk] + CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status] + InputAdapter + ReusedExchange [d_date_sk] #2 + InputAdapter + BroadcastExchange #4 + WholeStageCodegen (3) + ColumnarToRow + InputAdapter + CometFilter [i_item_sk] + CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id] + InputAdapter + BroadcastExchange #5 + WholeStageCodegen (4) + ColumnarToRow + InputAdapter + CometProject [p_promo_sk] + CometFilter [p_channel_email,p_channel_event,p_promo_sk] + CometScan parquet spark_catalog.default.promotion [p_promo_sk,p_channel_email,p_channel_event] 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 new file mode 100644 index 000000000..c3acf0a8d --- /dev/null +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q70/explain.txt @@ -0,0 +1,283 @@ +== Physical Plan == +TakeOrderedAndProject (43) ++- * Project (42) + +- Window (41) + +- * Sort (40) + +- Exchange (39) + +- * HashAggregate (38) + +- Exchange (37) + +- * HashAggregate (36) + +- * Expand (35) + +- * Project (34) + +- * BroadcastHashJoin Inner BuildRight (33) + :- * Project (6) + : +- * BroadcastHashJoin Inner BuildRight (5) + : :- * ColumnarToRow (3) + : : +- CometFilter (2) + : : +- CometScan parquet spark_catalog.default.store_sales (1) + : +- ReusedExchange (4) + +- BroadcastExchange (32) + +- * BroadcastHashJoin LeftSemi BuildRight (31) + :- * ColumnarToRow (9) + : +- CometFilter (8) + : +- CometScan parquet spark_catalog.default.store (7) + +- BroadcastExchange (30) + +- * Project (29) + +- * Filter (28) + +- Window (27) + +- WindowGroupLimit (26) + +- * Sort (25) + +- * HashAggregate (24) + +- Exchange (23) + +- * HashAggregate (22) + +- * Project (21) + +- * BroadcastHashJoin Inner BuildRight (20) + :- * Project (18) + : +- * BroadcastHashJoin Inner BuildRight (17) + : :- * ColumnarToRow (12) + : : +- CometFilter (11) + : : +- CometScan parquet spark_catalog.default.store_sales (10) + : +- BroadcastExchange (16) + : +- * ColumnarToRow (15) + : +- CometFilter (14) + : +- CometScan parquet spark_catalog.default.store (13) + +- ReusedExchange (19) + + +(1) Scan parquet spark_catalog.default.store_sales +Output [3]: [ss_store_sk#1, ss_net_profit#2, ss_sold_date_sk#3] +Batched: true +Location: InMemoryFileIndex [] +PartitionFilters: [isnotnull(ss_sold_date_sk#3), dynamicpruningexpression(ss_sold_date_sk#3 IN dynamicpruning#4)] +PushedFilters: [IsNotNull(ss_store_sk)] +ReadSchema: struct<ss_store_sk:int,ss_net_profit:decimal(7,2)> + +(2) CometFilter +Input [3]: [ss_store_sk#1, ss_net_profit#2, ss_sold_date_sk#3] +Condition : isnotnull(ss_store_sk#1) + +(3) ColumnarToRow [codegen id : 8] +Input [3]: [ss_store_sk#1, ss_net_profit#2, ss_sold_date_sk#3] + +(4) ReusedExchange [Reuses operator id: 48] +Output [1]: [d_date_sk#5] + +(5) BroadcastHashJoin [codegen id : 8] +Left keys [1]: [ss_sold_date_sk#3] +Right keys [1]: [d_date_sk#5] +Join type: Inner +Join condition: None + +(6) Project [codegen id : 8] +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] + +(7) Scan parquet spark_catalog.default.store +Output [3]: [s_store_sk#6, s_county#7, s_state#8] +Batched: true +Location [not included in comparison]/{warehouse_dir}/store] +PushedFilters: [IsNotNull(s_store_sk)] +ReadSchema: struct<s_store_sk:int,s_county:string,s_state:string> + +(8) CometFilter +Input [3]: [s_store_sk#6, s_county#7, s_state#8] +Condition : isnotnull(s_store_sk#6) + +(9) ColumnarToRow [codegen id : 7] +Input [3]: [s_store_sk#6, s_county#7, s_state#8] + +(10) Scan parquet spark_catalog.default.store_sales +Output [3]: [ss_store_sk#9, ss_net_profit#10, ss_sold_date_sk#11] +Batched: true +Location: InMemoryFileIndex [] +PartitionFilters: [isnotnull(ss_sold_date_sk#11), dynamicpruningexpression(ss_sold_date_sk#11 IN dynamicpruning#12)] +PushedFilters: [IsNotNull(ss_store_sk)] +ReadSchema: struct<ss_store_sk:int,ss_net_profit:decimal(7,2)> + +(11) CometFilter +Input [3]: [ss_store_sk#9, ss_net_profit#10, ss_sold_date_sk#11] +Condition : isnotnull(ss_store_sk#9) + +(12) ColumnarToRow [codegen id : 4] +Input [3]: [ss_store_sk#9, ss_net_profit#10, ss_sold_date_sk#11] + +(13) Scan parquet spark_catalog.default.store +Output [2]: [s_store_sk#13, s_state#14] +Batched: true +Location [not included in comparison]/{warehouse_dir}/store] +PushedFilters: [IsNotNull(s_store_sk)] +ReadSchema: struct<s_store_sk:int,s_state:string> + +(14) CometFilter +Input [2]: [s_store_sk#13, s_state#14] +Condition : isnotnull(s_store_sk#13) + +(15) ColumnarToRow [codegen id : 2] +Input [2]: [s_store_sk#13, s_state#14] + +(16) BroadcastExchange +Input [2]: [s_store_sk#13, s_state#14] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=1] + +(17) BroadcastHashJoin [codegen id : 4] +Left keys [1]: [ss_store_sk#9] +Right keys [1]: [s_store_sk#13] +Join type: Inner +Join condition: None + +(18) Project [codegen id : 4] +Output [3]: [ss_net_profit#10, ss_sold_date_sk#11, s_state#14] +Input [5]: [ss_store_sk#9, ss_net_profit#10, ss_sold_date_sk#11, s_store_sk#13, s_state#14] + +(19) ReusedExchange [Reuses operator id: 48] +Output [1]: [d_date_sk#15] + +(20) BroadcastHashJoin [codegen id : 4] +Left keys [1]: [ss_sold_date_sk#11] +Right keys [1]: [d_date_sk#15] +Join type: Inner +Join condition: None + +(21) Project [codegen id : 4] +Output [2]: [ss_net_profit#10, s_state#14] +Input [4]: [ss_net_profit#10, ss_sold_date_sk#11, s_state#14, d_date_sk#15] + +(22) HashAggregate [codegen id : 4] +Input [2]: [ss_net_profit#10, s_state#14] +Keys [1]: [s_state#14] +Functions [1]: [partial_sum(UnscaledValue(ss_net_profit#10))] +Aggregate Attributes [1]: [sum#16] +Results [2]: [s_state#14, sum#17] + +(23) Exchange +Input [2]: [s_state#14, sum#17] +Arguments: hashpartitioning(s_state#14, 5), ENSURE_REQUIREMENTS, [plan_id=2] + +(24) HashAggregate [codegen id : 5] +Input [2]: [s_state#14, sum#17] +Keys [1]: [s_state#14] +Functions [1]: [sum(UnscaledValue(ss_net_profit#10))] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_profit#10))#18] +Results [3]: [s_state#14, MakeDecimal(sum(UnscaledValue(ss_net_profit#10))#18,17,2) AS _w0#19, s_state#14] + +(25) Sort [codegen id : 5] +Input [3]: [s_state#14, _w0#19, s_state#14] +Arguments: [s_state#14 ASC NULLS FIRST, _w0#19 DESC NULLS LAST], false, 0 + +(26) WindowGroupLimit +Input [3]: [s_state#14, _w0#19, s_state#14] +Arguments: [s_state#14], [_w0#19 DESC NULLS LAST], rank(_w0#19), 5, Final + +(27) Window +Input [3]: [s_state#14, _w0#19, s_state#14] +Arguments: [rank(_w0#19) windowspecdefinition(s_state#14, _w0#19 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS ranking#20], [s_state#14], [_w0#19 DESC NULLS LAST] + +(28) Filter [codegen id : 6] +Input [4]: [s_state#14, _w0#19, s_state#14, ranking#20] +Condition : (ranking#20 <= 5) + +(29) Project [codegen id : 6] +Output [1]: [s_state#14] +Input [4]: [s_state#14, _w0#19, s_state#14, ranking#20] + +(30) BroadcastExchange +Input [1]: [s_state#14] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=3] + +(31) BroadcastHashJoin [codegen id : 7] +Left keys [1]: [s_state#8] +Right keys [1]: [s_state#14] +Join type: LeftSemi +Join condition: None + +(32) BroadcastExchange +Input [3]: [s_store_sk#6, s_county#7, s_state#8] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=4] + +(33) BroadcastHashJoin [codegen id : 8] +Left keys [1]: [ss_store_sk#1] +Right keys [1]: [s_store_sk#6] +Join type: Inner +Join condition: None + +(34) Project [codegen id : 8] +Output [3]: [ss_net_profit#2, s_state#8, s_county#7] +Input [5]: [ss_store_sk#1, ss_net_profit#2, s_store_sk#6, s_county#7, s_state#8] + +(35) Expand [codegen id : 8] +Input [3]: [ss_net_profit#2, s_state#8, s_county#7] +Arguments: [[ss_net_profit#2, s_state#8, s_county#7, 0], [ss_net_profit#2, s_state#8, null, 1], [ss_net_profit#2, null, null, 3]], [ss_net_profit#2, s_state#21, s_county#22, spark_grouping_id#23] + +(36) HashAggregate [codegen id : 8] +Input [4]: [ss_net_profit#2, s_state#21, s_county#22, spark_grouping_id#23] +Keys [3]: [s_state#21, s_county#22, spark_grouping_id#23] +Functions [1]: [partial_sum(UnscaledValue(ss_net_profit#2))] +Aggregate Attributes [1]: [sum#24] +Results [4]: [s_state#21, s_county#22, spark_grouping_id#23, sum#25] + +(37) Exchange +Input [4]: [s_state#21, s_county#22, spark_grouping_id#23, sum#25] +Arguments: hashpartitioning(s_state#21, s_county#22, spark_grouping_id#23, 5), ENSURE_REQUIREMENTS, [plan_id=5] + +(38) HashAggregate [codegen id : 9] +Input [4]: [s_state#21, s_county#22, spark_grouping_id#23, sum#25] +Keys [3]: [s_state#21, s_county#22, spark_grouping_id#23] +Functions [1]: [sum(UnscaledValue(ss_net_profit#2))] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_profit#2))#26] +Results [7]: [MakeDecimal(sum(UnscaledValue(ss_net_profit#2))#26,17,2) AS total_sum#27, s_state#21, s_county#22, (cast((shiftright(spark_grouping_id#23, 1) & 1) as tinyint) + cast((shiftright(spark_grouping_id#23, 0) & 1) as tinyint)) AS lochierarchy#28, MakeDecimal(sum(UnscaledValue(ss_net_profit#2))#26,17,2) AS _w0#29, (cast((shiftright(spark_grouping_id#23, 1) & 1) as tinyint) + cast((shiftright(spark_grouping_id#23, 0) & 1) as tinyint)) AS _w1#30, CASE WHEN (cast((shiftright(spark_grouping_id#23, 0) & 1) as tinyint) = 0) THEN s_state#21 END AS _w2#31] + +(39) Exchange +Input [7]: [total_sum#27, s_state#21, s_county#22, lochierarchy#28, _w0#29, _w1#30, _w2#31] +Arguments: hashpartitioning(_w1#30, _w2#31, 5), ENSURE_REQUIREMENTS, [plan_id=6] + +(40) Sort [codegen id : 10] +Input [7]: [total_sum#27, s_state#21, s_county#22, lochierarchy#28, _w0#29, _w1#30, _w2#31] +Arguments: [_w1#30 ASC NULLS FIRST, _w2#31 ASC NULLS FIRST, _w0#29 DESC NULLS LAST], false, 0 + +(41) Window +Input [7]: [total_sum#27, s_state#21, s_county#22, lochierarchy#28, _w0#29, _w1#30, _w2#31] +Arguments: [rank(_w0#29) windowspecdefinition(_w1#30, _w2#31, _w0#29 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rank_within_parent#32], [_w1#30, _w2#31], [_w0#29 DESC NULLS LAST] + +(42) Project [codegen id : 11] +Output [5]: [total_sum#27, s_state#21, s_county#22, lochierarchy#28, rank_within_parent#32] +Input [8]: [total_sum#27, s_state#21, s_county#22, lochierarchy#28, _w0#29, _w1#30, _w2#31, rank_within_parent#32] + +(43) TakeOrderedAndProject +Input [5]: [total_sum#27, s_state#21, s_county#22, lochierarchy#28, rank_within_parent#32] +Arguments: 100, [lochierarchy#28 DESC NULLS LAST, CASE WHEN (lochierarchy#28 = 0) THEN s_state#21 END ASC NULLS FIRST, rank_within_parent#32 ASC NULLS FIRST], [total_sum#27, s_state#21, s_county#22, lochierarchy#28, rank_within_parent#32] + +===== Subqueries ===== + +Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#3 IN dynamicpruning#4 +BroadcastExchange (48) ++- * ColumnarToRow (47) + +- CometProject (46) + +- CometFilter (45) + +- CometScan parquet spark_catalog.default.date_dim (44) + + +(44) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#5, d_month_seq#33] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1200), LessThanOrEqual(d_month_seq,1211), IsNotNull(d_date_sk)] +ReadSchema: struct<d_date_sk:int,d_month_seq:int> + +(45) CometFilter +Input [2]: [d_date_sk#5, d_month_seq#33] +Condition : (((isnotnull(d_month_seq#33) AND (d_month_seq#33 >= 1200)) AND (d_month_seq#33 <= 1211)) AND isnotnull(d_date_sk#5)) + +(46) CometProject +Input [2]: [d_date_sk#5, d_month_seq#33] +Arguments: [d_date_sk#5], [d_date_sk#5] + +(47) ColumnarToRow [codegen id : 1] +Input [1]: [d_date_sk#5] + +(48) BroadcastExchange +Input [1]: [d_date_sk#5] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=7] + +Subquery:2 Hosting operator id = 10 Hosting Expression = ss_sold_date_sk#11 IN dynamicpruning#4 + + diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q70/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q70/simplified.txt new file mode 100644 index 000000000..268af2e2f --- /dev/null +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q70/simplified.txt @@ -0,0 +1,75 @@ +TakeOrderedAndProject [lochierarchy,s_state,rank_within_parent,total_sum,s_county] + WholeStageCodegen (11) + Project [total_sum,s_state,s_county,lochierarchy,rank_within_parent] + InputAdapter + Window [_w0,_w1,_w2] + WholeStageCodegen (10) + Sort [_w1,_w2,_w0] + InputAdapter + Exchange [_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] + InputAdapter + Exchange [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] + ColumnarToRow + InputAdapter + CometFilter [ss_store_sk] + CometScan parquet spark_catalog.default.store_sales [ss_store_sk,ss_net_profit,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_month_seq,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] + InputAdapter + ReusedExchange [d_date_sk] #3 + InputAdapter + BroadcastExchange #4 + WholeStageCodegen (7) + BroadcastHashJoin [s_state,s_state] + ColumnarToRow + InputAdapter + CometFilter [s_store_sk] + CometScan parquet spark_catalog.default.store [s_store_sk,s_county,s_state] + InputAdapter + BroadcastExchange #5 + WholeStageCodegen (6) + 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] + InputAdapter + Exchange [s_state] #6 + WholeStageCodegen (4) + HashAggregate [s_state,ss_net_profit] [sum,sum] + Project [ss_net_profit,s_state] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [ss_net_profit,ss_sold_date_sk,s_state] + BroadcastHashJoin [ss_store_sk,s_store_sk] + ColumnarToRow + InputAdapter + CometFilter [ss_store_sk] + CometScan parquet spark_catalog.default.store_sales [ss_store_sk,ss_net_profit,ss_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + InputAdapter + BroadcastExchange #7 + WholeStageCodegen (2) + ColumnarToRow + InputAdapter + CometFilter [s_store_sk] + CometScan parquet spark_catalog.default.store [s_store_sk,s_state] + InputAdapter + ReusedExchange [d_date_sk] #3 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 new file mode 100644 index 000000000..624103b66 --- /dev/null +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q71/explain.txt @@ -0,0 +1,254 @@ +== Physical Plan == +* Sort (38) ++- Exchange (37) + +- * HashAggregate (36) + +- Exchange (35) + +- * HashAggregate (34) + +- * Project (33) + +- * BroadcastHashJoin Inner BuildRight (32) + :- * Project (26) + : +- * BroadcastHashJoin Inner BuildLeft (25) + : :- BroadcastExchange (5) + : : +- * ColumnarToRow (4) + : : +- CometProject (3) + : : +- CometFilter (2) + : : +- CometScan parquet spark_catalog.default.item (1) + : +- Union (24) + : :- * Project (11) + : : +- * BroadcastHashJoin Inner BuildRight (10) + : : :- * ColumnarToRow (8) + : : : +- CometFilter (7) + : : : +- CometScan parquet spark_catalog.default.web_sales (6) + : : +- ReusedExchange (9) + : :- * Project (17) + : : +- * BroadcastHashJoin Inner BuildRight (16) + : : :- * ColumnarToRow (14) + : : : +- CometFilter (13) + : : : +- CometScan parquet spark_catalog.default.catalog_sales (12) + : : +- ReusedExchange (15) + : +- * Project (23) + : +- * BroadcastHashJoin Inner BuildRight (22) + : :- * ColumnarToRow (20) + : : +- CometFilter (19) + : : +- CometScan parquet spark_catalog.default.store_sales (18) + : +- ReusedExchange (21) + +- BroadcastExchange (31) + +- * ColumnarToRow (30) + +- CometProject (29) + +- CometFilter (28) + +- CometScan parquet spark_catalog.default.time_dim (27) + + +(1) Scan parquet spark_catalog.default.item +Output [4]: [i_item_sk#1, i_brand_id#2, i_brand#3, i_manager_id#4] +Batched: true +Location [not included in comparison]/{warehouse_dir}/item] +PushedFilters: [IsNotNull(i_manager_id), EqualTo(i_manager_id,1), IsNotNull(i_item_sk)] +ReadSchema: struct<i_item_sk:int,i_brand_id:int,i_brand:string,i_manager_id:int> + +(2) CometFilter +Input [4]: [i_item_sk#1, i_brand_id#2, i_brand#3, i_manager_id#4] +Condition : ((isnotnull(i_manager_id#4) AND (i_manager_id#4 = 1)) AND isnotnull(i_item_sk#1)) + +(3) CometProject +Input [4]: [i_item_sk#1, i_brand_id#2, i_brand#3, i_manager_id#4] +Arguments: [i_item_sk#1, i_brand_id#2, i_brand#3], [i_item_sk#1, i_brand_id#2, i_brand#3] + +(4) ColumnarToRow [codegen id : 1] +Input [3]: [i_item_sk#1, i_brand_id#2, i_brand#3] + +(5) BroadcastExchange +Input [3]: [i_item_sk#1, i_brand_id#2, i_brand#3] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] + +(6) Scan parquet spark_catalog.default.web_sales +Output [4]: [ws_sold_time_sk#5, ws_item_sk#6, ws_ext_sales_price#7, ws_sold_date_sk#8] +Batched: true +Location: InMemoryFileIndex [] +PartitionFilters: [isnotnull(ws_sold_date_sk#8), dynamicpruningexpression(ws_sold_date_sk#8 IN dynamicpruning#9)] +PushedFilters: [IsNotNull(ws_item_sk), IsNotNull(ws_sold_time_sk)] +ReadSchema: struct<ws_sold_time_sk:int,ws_item_sk:int,ws_ext_sales_price:decimal(7,2)> + +(7) CometFilter +Input [4]: [ws_sold_time_sk#5, ws_item_sk#6, ws_ext_sales_price#7, ws_sold_date_sk#8] +Condition : (isnotnull(ws_item_sk#6) AND isnotnull(ws_sold_time_sk#5)) + +(8) ColumnarToRow [codegen id : 3] +Input [4]: [ws_sold_time_sk#5, ws_item_sk#6, ws_ext_sales_price#7, ws_sold_date_sk#8] + +(9) ReusedExchange [Reuses operator id: 43] +Output [1]: [d_date_sk#10] + +(10) BroadcastHashJoin [codegen id : 3] +Left keys [1]: [ws_sold_date_sk#8] +Right keys [1]: [d_date_sk#10] +Join type: Inner +Join condition: None + +(11) Project [codegen id : 3] +Output [3]: [ws_ext_sales_price#7 AS ext_price#11, ws_item_sk#6 AS sold_item_sk#12, ws_sold_time_sk#5 AS time_sk#13] +Input [5]: [ws_sold_time_sk#5, ws_item_sk#6, ws_ext_sales_price#7, ws_sold_date_sk#8, d_date_sk#10] + +(12) Scan parquet spark_catalog.default.catalog_sales +Output [4]: [cs_sold_time_sk#14, cs_item_sk#15, cs_ext_sales_price#16, cs_sold_date_sk#17] +Batched: true +Location: InMemoryFileIndex [] +PartitionFilters: [isnotnull(cs_sold_date_sk#17), dynamicpruningexpression(cs_sold_date_sk#17 IN dynamicpruning#18)] +PushedFilters: [IsNotNull(cs_item_sk), IsNotNull(cs_sold_time_sk)] +ReadSchema: struct<cs_sold_time_sk:int,cs_item_sk:int,cs_ext_sales_price:decimal(7,2)> + +(13) CometFilter +Input [4]: [cs_sold_time_sk#14, cs_item_sk#15, cs_ext_sales_price#16, cs_sold_date_sk#17] +Condition : (isnotnull(cs_item_sk#15) AND isnotnull(cs_sold_time_sk#14)) + +(14) ColumnarToRow [codegen id : 5] +Input [4]: [cs_sold_time_sk#14, cs_item_sk#15, cs_ext_sales_price#16, cs_sold_date_sk#17] + +(15) ReusedExchange [Reuses operator id: 43] +Output [1]: [d_date_sk#19] + +(16) BroadcastHashJoin [codegen id : 5] +Left keys [1]: [cs_sold_date_sk#17] +Right keys [1]: [d_date_sk#19] +Join type: Inner +Join condition: None + +(17) Project [codegen id : 5] +Output [3]: [cs_ext_sales_price#16 AS ext_price#20, cs_item_sk#15 AS sold_item_sk#21, cs_sold_time_sk#14 AS time_sk#22] +Input [5]: [cs_sold_time_sk#14, cs_item_sk#15, cs_ext_sales_price#16, cs_sold_date_sk#17, d_date_sk#19] + +(18) Scan parquet spark_catalog.default.store_sales +Output [4]: [ss_sold_time_sk#23, ss_item_sk#24, ss_ext_sales_price#25, ss_sold_date_sk#26] +Batched: true +Location: InMemoryFileIndex [] +PartitionFilters: [isnotnull(ss_sold_date_sk#26), dynamicpruningexpression(ss_sold_date_sk#26 IN dynamicpruning#27)] +PushedFilters: [IsNotNull(ss_item_sk), IsNotNull(ss_sold_time_sk)] +ReadSchema: struct<ss_sold_time_sk:int,ss_item_sk:int,ss_ext_sales_price:decimal(7,2)> + +(19) CometFilter +Input [4]: [ss_sold_time_sk#23, ss_item_sk#24, ss_ext_sales_price#25, ss_sold_date_sk#26] +Condition : (isnotnull(ss_item_sk#24) AND isnotnull(ss_sold_time_sk#23)) + +(20) ColumnarToRow [codegen id : 7] +Input [4]: [ss_sold_time_sk#23, ss_item_sk#24, ss_ext_sales_price#25, ss_sold_date_sk#26] + +(21) ReusedExchange [Reuses operator id: 43] +Output [1]: [d_date_sk#28] + +(22) BroadcastHashJoin [codegen id : 7] +Left keys [1]: [ss_sold_date_sk#26] +Right keys [1]: [d_date_sk#28] +Join type: Inner +Join condition: None + +(23) Project [codegen id : 7] +Output [3]: [ss_ext_sales_price#25 AS ext_price#29, ss_item_sk#24 AS sold_item_sk#30, ss_sold_time_sk#23 AS time_sk#31] +Input [5]: [ss_sold_time_sk#23, ss_item_sk#24, ss_ext_sales_price#25, ss_sold_date_sk#26, d_date_sk#28] + +(24) Union + +(25) BroadcastHashJoin [codegen id : 9] +Left keys [1]: [i_item_sk#1] +Right keys [1]: [sold_item_sk#12] +Join type: Inner +Join condition: None + +(26) Project [codegen id : 9] +Output [4]: [i_brand_id#2, i_brand#3, ext_price#11, time_sk#13] +Input [6]: [i_item_sk#1, i_brand_id#2, i_brand#3, ext_price#11, sold_item_sk#12, time_sk#13] + +(27) Scan parquet spark_catalog.default.time_dim +Output [4]: [t_time_sk#32, t_hour#33, t_minute#34, t_meal_time#35] +Batched: true +Location [not included in comparison]/{warehouse_dir}/time_dim] +PushedFilters: [Or(EqualTo(t_meal_time,breakfast ),EqualTo(t_meal_time,dinner )), IsNotNull(t_time_sk)] +ReadSchema: struct<t_time_sk:int,t_hour:int,t_minute:int,t_meal_time:string> + +(28) CometFilter +Input [4]: [t_time_sk#32, t_hour#33, t_minute#34, t_meal_time#35] +Condition : (((t_meal_time#35 = breakfast ) OR (t_meal_time#35 = dinner )) AND isnotnull(t_time_sk#32)) + +(29) CometProject +Input [4]: [t_time_sk#32, t_hour#33, t_minute#34, t_meal_time#35] +Arguments: [t_time_sk#32, t_hour#33, t_minute#34], [t_time_sk#32, t_hour#33, t_minute#34] + +(30) ColumnarToRow [codegen id : 8] +Input [3]: [t_time_sk#32, t_hour#33, t_minute#34] + +(31) BroadcastExchange +Input [3]: [t_time_sk#32, t_hour#33, t_minute#34] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] + +(32) BroadcastHashJoin [codegen id : 9] +Left keys [1]: [time_sk#13] +Right keys [1]: [t_time_sk#32] +Join type: Inner +Join condition: None + +(33) Project [codegen id : 9] +Output [5]: [i_brand_id#2, i_brand#3, ext_price#11, t_hour#33, t_minute#34] +Input [7]: [i_brand_id#2, i_brand#3, ext_price#11, time_sk#13, t_time_sk#32, t_hour#33, t_minute#34] + +(34) HashAggregate [codegen id : 9] +Input [5]: [i_brand_id#2, i_brand#3, ext_price#11, t_hour#33, t_minute#34] +Keys [4]: [i_brand#3, i_brand_id#2, t_hour#33, t_minute#34] +Functions [1]: [partial_sum(UnscaledValue(ext_price#11))] +Aggregate Attributes [1]: [sum#36] +Results [5]: [i_brand#3, i_brand_id#2, t_hour#33, t_minute#34, sum#37] + +(35) Exchange +Input [5]: [i_brand#3, i_brand_id#2, t_hour#33, t_minute#34, sum#37] +Arguments: hashpartitioning(i_brand#3, i_brand_id#2, t_hour#33, t_minute#34, 5), ENSURE_REQUIREMENTS, [plan_id=3] + +(36) HashAggregate [codegen id : 10] +Input [5]: [i_brand#3, i_brand_id#2, t_hour#33, t_minute#34, sum#37] +Keys [4]: [i_brand#3, i_brand_id#2, t_hour#33, t_minute#34] +Functions [1]: [sum(UnscaledValue(ext_price#11))] +Aggregate Attributes [1]: [sum(UnscaledValue(ext_price#11))#38] +Results [5]: [i_brand_id#2 AS brand_id#39, i_brand#3 AS brand#40, t_hour#33, t_minute#34, MakeDecimal(sum(UnscaledValue(ext_price#11))#38,17,2) AS ext_price#41] + +(37) Exchange +Input [5]: [brand_id#39, brand#40, t_hour#33, t_minute#34, ext_price#41] +Arguments: rangepartitioning(ext_price#41 DESC NULLS LAST, brand_id#39 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, [plan_id=4] + +(38) Sort [codegen id : 11] +Input [5]: [brand_id#39, brand#40, t_hour#33, t_minute#34, ext_price#41] +Arguments: [ext_price#41 DESC NULLS LAST, brand_id#39 ASC NULLS FIRST], true, 0 + +===== Subqueries ===== + +Subquery:1 Hosting operator id = 6 Hosting Expression = ws_sold_date_sk#8 IN dynamicpruning#9 +BroadcastExchange (43) ++- * ColumnarToRow (42) + +- CometProject (41) + +- CometFilter (40) + +- CometScan parquet spark_catalog.default.date_dim (39) + + +(39) Scan parquet spark_catalog.default.date_dim +Output [3]: [d_date_sk#10, d_year#42, d_moy#43] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_moy), IsNotNull(d_year), EqualTo(d_moy,11), EqualTo(d_year,1999), IsNotNull(d_date_sk)] +ReadSchema: struct<d_date_sk:int,d_year:int,d_moy:int> + +(40) CometFilter +Input [3]: [d_date_sk#10, d_year#42, d_moy#43] +Condition : ((((isnotnull(d_moy#43) AND isnotnull(d_year#42)) AND (d_moy#43 = 11)) AND (d_year#42 = 1999)) AND isnotnull(d_date_sk#10)) + +(41) CometProject +Input [3]: [d_date_sk#10, d_year#42, d_moy#43] +Arguments: [d_date_sk#10], [d_date_sk#10] + +(42) ColumnarToRow [codegen id : 1] +Input [1]: [d_date_sk#10] + +(43) BroadcastExchange +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 = 12 Hosting Expression = cs_sold_date_sk#17 IN dynamicpruning#9 + +Subquery:3 Hosting operator id = 18 Hosting Expression = ss_sold_date_sk#26 IN dynamicpruning#9 + + diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q71/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q71/simplified.txt new file mode 100644 index 000000000..bea5376a0 --- /dev/null +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q71/simplified.txt @@ -0,0 +1,69 @@ +WholeStageCodegen (11) + Sort [ext_price,brand_id] + InputAdapter + Exchange [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] + InputAdapter + Exchange [i_brand,i_brand_id,t_hour,t_minute] #2 + WholeStageCodegen (9) + HashAggregate [i_brand,i_brand_id,t_hour,t_minute,ext_price] [sum,sum] + Project [i_brand_id,i_brand,ext_price,t_hour,t_minute] + BroadcastHashJoin [time_sk,t_time_sk] + Project [i_brand_id,i_brand,ext_price,time_sk] + BroadcastHashJoin [i_item_sk,sold_item_sk] + InputAdapter + BroadcastExchange #3 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [i_item_sk,i_brand_id,i_brand] + CometFilter [i_manager_id,i_item_sk] + CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_brand,i_manager_id] + InputAdapter + Union + WholeStageCodegen (3) + Project [ws_ext_sales_price,ws_item_sk,ws_sold_time_sk] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + ColumnarToRow + InputAdapter + CometFilter [ws_item_sk,ws_sold_time_sk] + CometScan parquet spark_catalog.default.web_sales [ws_sold_time_sk,ws_item_sk,ws_ext_sales_price,ws_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #4 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_moy,d_year,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + InputAdapter + ReusedExchange [d_date_sk] #4 + WholeStageCodegen (5) + Project [cs_ext_sales_price,cs_item_sk,cs_sold_time_sk] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + ColumnarToRow + InputAdapter + CometFilter [cs_item_sk,cs_sold_time_sk] + CometScan parquet spark_catalog.default.catalog_sales [cs_sold_time_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + InputAdapter + ReusedExchange [d_date_sk] #4 + WholeStageCodegen (7) + Project [ss_ext_sales_price,ss_item_sk,ss_sold_time_sk] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + ColumnarToRow + InputAdapter + CometFilter [ss_item_sk,ss_sold_time_sk] + CometScan parquet spark_catalog.default.store_sales [ss_sold_time_sk,ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + InputAdapter + ReusedExchange [d_date_sk] #4 + InputAdapter + BroadcastExchange #5 + WholeStageCodegen (8) + ColumnarToRow + InputAdapter + CometProject [t_time_sk,t_hour,t_minute] + CometFilter [t_meal_time,t_time_sk] + CometScan parquet spark_catalog.default.time_dim [t_time_sk,t_hour,t_minute,t_meal_time] 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 new file mode 100644 index 000000000..4e5d9e9f6 --- /dev/null +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q72/explain.txt @@ -0,0 +1,433 @@ +== Physical Plan == +TakeOrderedAndProject (70) ++- * HashAggregate (69) + +- Exchange (68) + +- * HashAggregate (67) + +- * Project (66) + +- * SortMergeJoin LeftOuter (65) + :- * Sort (58) + : +- Exchange (57) + : +- * Project (56) + : +- * BroadcastHashJoin LeftOuter BuildRight (55) + : :- * Project (50) + : : +- * BroadcastHashJoin Inner BuildRight (49) + : : :- * Project (44) + : : : +- * BroadcastHashJoin Inner BuildRight (43) + : : : :- * Project (38) + : : : : +- * BroadcastHashJoin Inner BuildRight (37) + : : : : :- * Project (35) + : : : : : +- * BroadcastHashJoin Inner BuildRight (34) + : : : : : :- * Project (28) + : : : : : : +- * BroadcastHashJoin Inner BuildRight (27) + : : : : : : :- * Project (21) + : : : : : : : +- * BroadcastHashJoin Inner BuildRight (20) + : : : : : : : :- * Project (15) + : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (14) + : : : : : : : : :- * Project (9) + : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (8) + : : : : : : : : : :- * ColumnarToRow (3) + : : : : : : : : : : +- CometFilter (2) + : : : : : : : : : : +- CometScan parquet spark_catalog.default.catalog_sales (1) + : : : : : : : : : +- BroadcastExchange (7) + : : : : : : : : : +- * ColumnarToRow (6) + : : : : : : : : : +- CometFilter (5) + : : : : : : : : : +- CometScan parquet spark_catalog.default.inventory (4) + : : : : : : : : +- BroadcastExchange (13) + : : : : : : : : +- * ColumnarToRow (12) + : : : : : : : : +- CometFilter (11) + : : : : : : : : +- CometScan parquet spark_catalog.default.warehouse (10) + : : : : : : : +- BroadcastExchange (19) + : : : : : : : +- * ColumnarToRow (18) + : : : : : : : +- CometFilter (17) + : : : : : : : +- CometScan parquet spark_catalog.default.item (16) + : : : : : : +- BroadcastExchange (26) + : : : : : : +- * ColumnarToRow (25) + : : : : : : +- CometProject (24) + : : : : : : +- CometFilter (23) + : : : : : : +- CometScan parquet spark_catalog.default.customer_demographics (22) + : : : : : +- BroadcastExchange (33) + : : : : : +- * ColumnarToRow (32) + : : : : : +- CometProject (31) + : : : : : +- CometFilter (30) + : : : : : +- CometScan parquet spark_catalog.default.household_demographics (29) + : : : : +- ReusedExchange (36) + : : : +- BroadcastExchange (42) + : : : +- * ColumnarToRow (41) + : : : +- CometFilter (40) + : : : +- CometScan parquet spark_catalog.default.date_dim (39) + : : +- BroadcastExchange (48) + : : +- * ColumnarToRow (47) + : : +- CometFilter (46) + : : +- CometScan parquet spark_catalog.default.date_dim (45) + : +- BroadcastExchange (54) + : +- * ColumnarToRow (53) + : +- CometFilter (52) + : +- CometScan parquet spark_catalog.default.promotion (51) + +- * Sort (64) + +- Exchange (63) + +- * ColumnarToRow (62) + +- CometProject (61) + +- CometFilter (60) + +- CometScan parquet spark_catalog.default.catalog_returns (59) + + +(1) Scan parquet spark_catalog.default.catalog_sales +Output [8]: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_quantity#7, cs_sold_date_sk#8] +Batched: true +Location: InMemoryFileIndex [] +PartitionFilters: [isnotnull(cs_sold_date_sk#8), dynamicpruningexpression(cs_sold_date_sk#8 IN dynamicpruning#9)] +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<cs_ship_date_sk:int,cs_bill_cdemo_sk:int,cs_bill_hdemo_sk:int,cs_item_sk:int,cs_promo_sk:int,cs_order_number:int,cs_quantity:int> + +(2) CometFilter +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)) + +(3) ColumnarToRow [codegen id : 10] +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] + +(4) Scan parquet spark_catalog.default.inventory +Output [4]: [inv_item_sk#10, inv_warehouse_sk#11, inv_quantity_on_hand#12, inv_date_sk#13] +Batched: true +Location: InMemoryFileIndex [] +PartitionFilters: [isnotnull(inv_date_sk#13)] +PushedFilters: [IsNotNull(inv_quantity_on_hand), IsNotNull(inv_item_sk), IsNotNull(inv_warehouse_sk)] +ReadSchema: struct<inv_item_sk:int,inv_warehouse_sk:int,inv_quantity_on_hand:int> + +(5) CometFilter +Input [4]: [inv_item_sk#10, inv_warehouse_sk#11, inv_quantity_on_hand#12, inv_date_sk#13] +Condition : ((isnotnull(inv_quantity_on_hand#12) AND isnotnull(inv_item_sk#10)) AND isnotnull(inv_warehouse_sk#11)) + +(6) ColumnarToRow [codegen id : 1] +Input [4]: [inv_item_sk#10, inv_warehouse_sk#11, inv_quantity_on_hand#12, inv_date_sk#13] + +(7) BroadcastExchange +Input [4]: [inv_item_sk#10, inv_warehouse_sk#11, inv_quantity_on_hand#12, inv_date_sk#13] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=1] + +(8) BroadcastHashJoin [codegen id : 10] +Left keys [1]: [cs_item_sk#4] +Right keys [1]: [inv_item_sk#10] +Join type: Inner +Join condition: (inv_quantity_on_hand#12 < cs_quantity#7) + +(9) Project [codegen id : 10] +Output [9]: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_warehouse_sk#11, inv_date_sk#13] +Input [12]: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_quantity#7, cs_sold_date_sk#8, inv_item_sk#10, inv_warehouse_sk#11, inv_quantity_on_hand#12, inv_date_sk#13] + +(10) Scan parquet spark_catalog.default.warehouse +Output [2]: [w_warehouse_sk#14, w_warehouse_name#15] +Batched: true +Location [not included in comparison]/{warehouse_dir}/warehouse] +PushedFilters: [IsNotNull(w_warehouse_sk)] +ReadSchema: struct<w_warehouse_sk:int,w_warehouse_name:string> + +(11) CometFilter +Input [2]: [w_warehouse_sk#14, w_warehouse_name#15] +Condition : isnotnull(w_warehouse_sk#14) + +(12) ColumnarToRow [codegen id : 2] +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] +Left keys [1]: [inv_warehouse_sk#11] +Right keys [1]: [w_warehouse_sk#14] +Join type: Inner +Join condition: None + +(15) Project [codegen id : 10] +Output [9]: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15] +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] + +(16) Scan parquet spark_catalog.default.item +Output [2]: [i_item_sk#16, i_item_desc#17] +Batched: true +Location [not included in comparison]/{warehouse_dir}/item] +PushedFilters: [IsNotNull(i_item_sk)] +ReadSchema: struct<i_item_sk:int,i_item_desc:string> + +(17) CometFilter +Input [2]: [i_item_sk#16, i_item_desc#17] +Condition : isnotnull(i_item_sk#16) + +(18) ColumnarToRow [codegen id : 3] +Input [2]: [i_item_sk#16, i_item_desc#17] + +(19) BroadcastExchange +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] +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] +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] + +(22) Scan parquet spark_catalog.default.customer_demographics +Output [2]: [cd_demo_sk#18, cd_marital_status#19] +Batched: true +Location [not included in comparison]/{warehouse_dir}/customer_demographics] +PushedFilters: [IsNotNull(cd_marital_status), EqualTo(cd_marital_status,D), IsNotNull(cd_demo_sk)] +ReadSchema: struct<cd_demo_sk:int,cd_marital_status:string> + +(23) CometFilter +Input [2]: [cd_demo_sk#18, cd_marital_status#19] +Condition : ((isnotnull(cd_marital_status#19) AND (cd_marital_status#19 = D)) AND isnotnull(cd_demo_sk#18)) + +(24) CometProject +Input [2]: [cd_demo_sk#18, cd_marital_status#19] +Arguments: [cd_demo_sk#18], [cd_demo_sk#18] + +(25) ColumnarToRow [codegen id : 4] +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] +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] +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] + +(29) Scan parquet spark_catalog.default.household_demographics +Output [2]: [hd_demo_sk#20, hd_buy_potential#21] +Batched: true +Location [not included in comparison]/{warehouse_dir}/household_demographics] +PushedFilters: [IsNotNull(hd_buy_potential), EqualTo(hd_buy_potential,>10000 ), IsNotNull(hd_demo_sk)] +ReadSchema: struct<hd_demo_sk:int,hd_buy_potential:string> + +(30) CometFilter +Input [2]: [hd_demo_sk#20, hd_buy_potential#21] +Condition : ((isnotnull(hd_buy_potential#21) AND (hd_buy_potential#21 = >10000 )) AND isnotnull(hd_demo_sk#20)) + +(31) CometProject +Input [2]: [hd_demo_sk#20, hd_buy_potential#21] +Arguments: [hd_demo_sk#20], [hd_demo_sk#20] + +(32) ColumnarToRow [codegen id : 5] +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] +Left keys [1]: [cs_bill_hdemo_sk#3] +Right keys [1]: [hd_demo_sk#20] +Join type: Inner +Join condition: None + +(35) Project [codegen id : 10] +Output [8]: [cs_ship_date_sk#1, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17] +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] +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] +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] + +(39) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#25, d_week_seq#26] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_week_seq), IsNotNull(d_date_sk)] +ReadSchema: struct<d_date_sk:int,d_week_seq:int> + +(40) CometFilter +Input [2]: [d_date_sk#25, d_week_seq#26] +Condition : (isnotnull(d_week_seq#26) AND isnotnull(d_date_sk#25)) + +(41) ColumnarToRow [codegen id : 7] +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] +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] +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] + +(45) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#27, d_date#28] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_date), IsNotNull(d_date_sk)] +ReadSchema: struct<d_date_sk:int,d_date:date> + +(46) CometFilter +Input [2]: [d_date_sk#27, d_date#28] +Condition : (isnotnull(d_date#28) AND isnotnull(d_date_sk#27)) + +(47) ColumnarToRow [codegen id : 8] +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] +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] +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] + +(51) Scan parquet spark_catalog.default.promotion +Output [1]: [p_promo_sk#29] +Batched: true +Location [not included in comparison]/{warehouse_dir}/promotion] +PushedFilters: [IsNotNull(p_promo_sk)] +ReadSchema: struct<p_promo_sk:int> + +(52) CometFilter +Input [1]: [p_promo_sk#29] +Condition : isnotnull(p_promo_sk#29) + +(53) ColumnarToRow [codegen id : 9] +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] +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] +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] + +(57) Exchange +Input [5]: [cs_item_sk#4, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_week_seq#24] +Arguments: hashpartitioning(cs_item_sk#4, cs_order_number#6, 5), ENSURE_REQUIREMENTS, [plan_id=9] + +(58) Sort [codegen id : 11] +Input [5]: [cs_item_sk#4, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_week_seq#24] +Arguments: [cs_item_sk#4 ASC NULLS FIRST, cs_order_number#6 ASC NULLS FIRST], false, 0 + +(59) Scan parquet spark_catalog.default.catalog_returns +Output [3]: [cr_item_sk#30, cr_order_number#31, cr_returned_date_sk#32] +Batched: true +Location [not included in comparison]/{warehouse_dir}/catalog_returns] +PushedFilters: [IsNotNull(cr_item_sk), IsNotNull(cr_order_number)] +ReadSchema: struct<cr_item_sk:int,cr_order_number:int> + +(60) CometFilter +Input [3]: [cr_item_sk#30, cr_order_number#31, cr_returned_date_sk#32] +Condition : (isnotnull(cr_item_sk#30) AND isnotnull(cr_order_number#31)) + +(61) CometProject +Input [3]: [cr_item_sk#30, cr_order_number#31, cr_returned_date_sk#32] +Arguments: [cr_item_sk#30, cr_order_number#31], [cr_item_sk#30, cr_order_number#31] + +(62) ColumnarToRow [codegen id : 12] +Input [2]: [cr_item_sk#30, cr_order_number#31] + +(63) Exchange +Input [2]: [cr_item_sk#30, cr_order_number#31] +Arguments: hashpartitioning(cr_item_sk#30, cr_order_number#31, 5), ENSURE_REQUIREMENTS, [plan_id=10] + +(64) Sort [codegen id : 13] +Input [2]: [cr_item_sk#30, cr_order_number#31] +Arguments: [cr_item_sk#30 ASC NULLS FIRST, cr_order_number#31 ASC NULLS FIRST], false, 0 + +(65) SortMergeJoin [codegen id : 14] +Left keys [2]: [cs_item_sk#4, cs_order_number#6] +Right keys [2]: [cr_item_sk#30, cr_order_number#31] +Join type: LeftOuter +Join condition: None + +(66) Project [codegen id : 14] +Output [3]: [w_warehouse_name#15, i_item_desc#17, d_week_seq#24] +Input [7]: [cs_item_sk#4, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_week_seq#24, cr_item_sk#30, cr_order_number#31] + +(67) HashAggregate [codegen id : 14] +Input [3]: [w_warehouse_name#15, i_item_desc#17, d_week_seq#24] +Keys [3]: [i_item_desc#17, w_warehouse_name#15, d_week_seq#24] +Functions [1]: [partial_count(1)] +Aggregate Attributes [1]: [count#33] +Results [4]: [i_item_desc#17, w_warehouse_name#15, d_week_seq#24, count#34] + +(68) Exchange +Input [4]: [i_item_desc#17, w_warehouse_name#15, d_week_seq#24, count#34] +Arguments: hashpartitioning(i_item_desc#17, w_warehouse_name#15, d_week_seq#24, 5), ENSURE_REQUIREMENTS, [plan_id=11] + +(69) HashAggregate [codegen id : 15] +Input [4]: [i_item_desc#17, w_warehouse_name#15, d_week_seq#24, count#34] +Keys [3]: [i_item_desc#17, w_warehouse_name#15, d_week_seq#24] +Functions [1]: [count(1)] +Aggregate Attributes [1]: [count(1)#35] +Results [6]: [i_item_desc#17, w_warehouse_name#15, d_week_seq#24, count(1)#35 AS no_promo#36, count(1)#35 AS promo#37, count(1)#35 AS total_cnt#38] + +(70) TakeOrderedAndProject +Input [6]: [i_item_desc#17, w_warehouse_name#15, d_week_seq#24, no_promo#36, promo#37, total_cnt#38] +Arguments: 100, [total_cnt#38 DESC NULLS LAST, i_item_desc#17 ASC NULLS FIRST, w_warehouse_name#15 ASC NULLS FIRST, d_week_seq#24 ASC NULLS FIRST], [i_item_desc#17, w_warehouse_name#15, d_week_seq#24, no_promo#36, promo#37, total_cnt#38] + +===== Subqueries ===== + +Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#8 IN dynamicpruning#9 +BroadcastExchange (75) ++- * ColumnarToRow (74) + +- CometProject (73) + +- CometFilter (72) + +- CometScan parquet spark_catalog.default.date_dim (71) + + +(71) Scan parquet spark_catalog.default.date_dim +Output [4]: [d_date_sk#22, d_date#23, d_week_seq#24, d_year#39] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_year), EqualTo(d_year,1999), IsNotNull(d_date_sk), IsNotNull(d_week_seq), IsNotNull(d_date)] +ReadSchema: struct<d_date_sk:int,d_date:date,d_week_seq:int,d_year:int> + +(72) CometFilter +Input [4]: [d_date_sk#22, d_date#23, d_week_seq#24, d_year#39] +Condition : ((((isnotnull(d_year#39) AND (d_year#39 = 1999)) AND isnotnull(d_date_sk#22)) AND isnotnull(d_week_seq#24)) AND isnotnull(d_date#23)) + +(73) CometProject +Input [4]: [d_date_sk#22, d_date#23, d_week_seq#24, d_year#39] +Arguments: [d_date_sk#22, d_date#23, d_week_seq#24], [d_date_sk#22, d_date#23, d_week_seq#24] + +(74) ColumnarToRow [codegen id : 1] +Input [3]: [d_date_sk#22, d_date#23, d_week_seq#24] + +(75) BroadcastExchange +Input [3]: [d_date_sk#22, d_date#23, d_week_seq#24] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=12] + + 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 new file mode 100644 index 000000000..17fc9dee7 --- /dev/null +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q72/simplified.txt @@ -0,0 +1,116 @@ +TakeOrderedAndProject [total_cnt,i_item_desc,w_warehouse_name,d_week_seq,no_promo,promo] + WholeStageCodegen (15) + HashAggregate [i_item_desc,w_warehouse_name,d_week_seq,count] [count(1),no_promo,promo,total_cnt,count] + InputAdapter + Exchange [i_item_desc,w_warehouse_name,d_week_seq] #1 + WholeStageCodegen (14) + HashAggregate [i_item_desc,w_warehouse_name,d_week_seq] [count,count] + Project [w_warehouse_name,i_item_desc,d_week_seq] + SortMergeJoin [cs_item_sk,cs_order_number,cr_item_sk,cr_order_number] + InputAdapter + WholeStageCodegen (11) + Sort [cs_item_sk,cs_order_number] + InputAdapter + Exchange [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] + ColumnarToRow + InputAdapter + CometFilter [cs_quantity,cs_item_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_ship_date_sk] + CometScan parquet spark_catalog.default.catalog_sales [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_quantity,cs_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk,d_date,d_week_seq] + CometFilter [d_year,d_date_sk,d_week_seq,d_date] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_week_seq,d_year] + InputAdapter + BroadcastExchange #4 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometFilter [inv_quantity_on_hand,inv_item_sk,inv_warehouse_sk] + CometScan parquet spark_catalog.default.inventory [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] + InputAdapter + BroadcastExchange #5 + WholeStageCodegen (2) + ColumnarToRow + InputAdapter + CometFilter [w_warehouse_sk] + CometScan parquet spark_catalog.default.warehouse [w_warehouse_sk,w_warehouse_name] + InputAdapter + BroadcastExchange #6 + WholeStageCodegen (3) + ColumnarToRow + InputAdapter + CometFilter [i_item_sk] + CometScan parquet spark_catalog.default.item [i_item_sk,i_item_desc] + InputAdapter + BroadcastExchange #7 + WholeStageCodegen (4) + ColumnarToRow + InputAdapter + CometProject [cd_demo_sk] + CometFilter [cd_marital_status,cd_demo_sk] + CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_marital_status] + InputAdapter + BroadcastExchange #8 + WholeStageCodegen (5) + ColumnarToRow + InputAdapter + CometProject [hd_demo_sk] + CometFilter [hd_buy_potential,hd_demo_sk] + CometScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_buy_potential] + InputAdapter + ReusedExchange [d_date_sk,d_date,d_week_seq] #3 + InputAdapter + BroadcastExchange #9 + WholeStageCodegen (7) + ColumnarToRow + InputAdapter + CometFilter [d_week_seq,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq] + InputAdapter + BroadcastExchange #10 + WholeStageCodegen (8) + ColumnarToRow + InputAdapter + CometFilter [d_date,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] + InputAdapter + BroadcastExchange #11 + WholeStageCodegen (9) + ColumnarToRow + InputAdapter + CometFilter [p_promo_sk] + CometScan parquet spark_catalog.default.promotion [p_promo_sk] + InputAdapter + WholeStageCodegen (13) + Sort [cr_item_sk,cr_order_number] + InputAdapter + Exchange [cr_item_sk,cr_order_number] #12 + WholeStageCodegen (12) + ColumnarToRow + InputAdapter + CometProject [cr_item_sk,cr_order_number] + CometFilter [cr_item_sk,cr_order_number] + CometScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_returned_date_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q73/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q73/explain.txt new file mode 100644 index 000000000..ef2c38aea --- /dev/null +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q73/explain.txt @@ -0,0 +1,218 @@ +== Physical Plan == +* Sort (32) ++- Exchange (31) + +- * Project (30) + +- * BroadcastHashJoin Inner BuildRight (29) + :- * Filter (24) + : +- * HashAggregate (23) + : +- Exchange (22) + : +- * HashAggregate (21) + : +- * Project (20) + : +- * BroadcastHashJoin Inner BuildRight (19) + : :- * Project (13) + : : +- * BroadcastHashJoin Inner BuildRight (12) + : : :- * Project (6) + : : : +- * BroadcastHashJoin Inner BuildRight (5) + : : : :- * ColumnarToRow (3) + : : : : +- CometFilter (2) + : : : : +- CometScan parquet spark_catalog.default.store_sales (1) + : : : +- ReusedExchange (4) + : : +- BroadcastExchange (11) + : : +- * ColumnarToRow (10) + : : +- CometProject (9) + : : +- CometFilter (8) + : : +- CometScan parquet spark_catalog.default.store (7) + : +- BroadcastExchange (18) + : +- * ColumnarToRow (17) + : +- CometProject (16) + : +- CometFilter (15) + : +- CometScan parquet spark_catalog.default.household_demographics (14) + +- BroadcastExchange (28) + +- * ColumnarToRow (27) + +- CometFilter (26) + +- CometScan parquet spark_catalog.default.customer (25) + + +(1) Scan parquet spark_catalog.default.store_sales +Output [5]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_sold_date_sk#5] +Batched: true +Location: InMemoryFileIndex [] +PartitionFilters: [isnotnull(ss_sold_date_sk#5), dynamicpruningexpression(ss_sold_date_sk#5 IN dynamicpruning#6)] +PushedFilters: [IsNotNull(ss_store_sk), IsNotNull(ss_hdemo_sk), IsNotNull(ss_customer_sk)] +ReadSchema: struct<ss_customer_sk:int,ss_hdemo_sk:int,ss_store_sk:int,ss_ticket_number:int> + +(2) CometFilter +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)) + +(3) ColumnarToRow [codegen id : 4] +Input [5]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_sold_date_sk#5] + +(4) ReusedExchange [Reuses operator id: 37] +Output [1]: [d_date_sk#7] + +(5) BroadcastHashJoin [codegen id : 4] +Left keys [1]: [ss_sold_date_sk#5] +Right keys [1]: [d_date_sk#7] +Join type: Inner +Join condition: None + +(6) Project [codegen id : 4] +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] + +(7) Scan parquet spark_catalog.default.store +Output [2]: [s_store_sk#8, s_county#9] +Batched: true +Location [not included in comparison]/{warehouse_dir}/store] +PushedFilters: [In(s_county, [Bronx County,Franklin Parish,Orange County,Williamson County]), IsNotNull(s_store_sk)] +ReadSchema: struct<s_store_sk:int,s_county:string> + +(8) CometFilter +Input [2]: [s_store_sk#8, s_county#9] +Condition : (s_county#9 IN (Williamson County,Franklin Parish,Bronx County,Orange County) AND isnotnull(s_store_sk#8)) + +(9) CometProject +Input [2]: [s_store_sk#8, s_county#9] +Arguments: [s_store_sk#8], [s_store_sk#8] + +(10) ColumnarToRow [codegen id : 2] +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] +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] +Output [3]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_ticket_number#4] +Input [5]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4, s_store_sk#8] + +(14) Scan parquet spark_catalog.default.household_demographics +Output [4]: [hd_demo_sk#10, hd_buy_potential#11, hd_dep_count#12, hd_vehicle_count#13] +Batched: true +Location [not included in comparison]/{warehouse_dir}/household_demographics] +PushedFilters: [IsNotNull(hd_vehicle_count), Or(EqualTo(hd_buy_potential,>10000 ),EqualTo(hd_buy_potential,unknown )), GreaterThan(hd_vehicle_count,0), IsNotNull(hd_demo_sk)] +ReadSchema: struct<hd_demo_sk:int,hd_buy_potential:string,hd_dep_count:int,hd_vehicle_count:int> + +(15) CometFilter +Input [4]: [hd_demo_sk#10, hd_buy_potential#11, hd_dep_count#12, hd_vehicle_count#13] +Condition : ((((isnotnull(hd_vehicle_count#13) AND ((hd_buy_potential#11 = >10000 ) OR (hd_buy_potential#11 = unknown ))) AND (hd_vehicle_count#13 > 0)) AND CASE WHEN (hd_vehicle_count#13 > 0) THEN ((cast(hd_dep_count#12 as double) / cast(hd_vehicle_count#13 as double)) > 1.0) END) AND isnotnull(hd_demo_sk#10)) + +(16) CometProject +Input [4]: [hd_demo_sk#10, hd_buy_potential#11, hd_dep_count#12, hd_vehicle_count#13] +Arguments: [hd_demo_sk#10], [hd_demo_sk#10] + +(17) ColumnarToRow [codegen id : 3] +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] +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] +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] +Input [2]: [ss_customer_sk#1, ss_ticket_number#4] +Keys [2]: [ss_ticket_number#4, ss_customer_sk#1] +Functions [1]: [partial_count(1)] +Aggregate Attributes [1]: [count#14] +Results [3]: [ss_ticket_number#4, ss_customer_sk#1, count#15] + +(22) Exchange +Input [3]: [ss_ticket_number#4, ss_customer_sk#1, count#15] +Arguments: hashpartitioning(ss_ticket_number#4, ss_customer_sk#1, 5), ENSURE_REQUIREMENTS, [plan_id=3] + +(23) HashAggregate [codegen id : 6] +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] + +(24) Filter [codegen id : 6] +Input [3]: [ss_ticket_number#4, ss_customer_sk#1, cnt#17] +Condition : ((cnt#17 >= 1) AND (cnt#17 <= 5)) + +(25) Scan parquet spark_catalog.default.customer +Output [5]: [c_customer_sk#18, c_salutation#19, c_first_name#20, c_last_name#21, c_preferred_cust_flag#22] +Batched: true +Location [not included in comparison]/{warehouse_dir}/customer] +PushedFilters: [IsNotNull(c_customer_sk)] +ReadSchema: struct<c_customer_sk:int,c_salutation:string,c_first_name:string,c_last_name:string,c_preferred_cust_flag:string> + +(26) CometFilter +Input [5]: [c_customer_sk#18, c_salutation#19, c_first_name#20, c_last_name#21, c_preferred_cust_flag#22] +Condition : isnotnull(c_customer_sk#18) + +(27) ColumnarToRow [codegen id : 5] +Input [5]: [c_customer_sk#18, c_salutation#19, c_first_name#20, c_last_name#21, c_preferred_cust_flag#22] + +(28) BroadcastExchange +Input [5]: [c_customer_sk#18, c_salutation#19, c_first_name#20, c_last_name#21, c_preferred_cust_flag#22] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=4] + +(29) BroadcastHashJoin [codegen id : 6] +Left keys [1]: [ss_customer_sk#1] +Right keys [1]: [c_customer_sk#18] +Join type: Inner +Join condition: None + +(30) Project [codegen id : 6] +Output [6]: [c_last_name#21, c_first_name#20, c_salutation#19, c_preferred_cust_flag#22, ss_ticket_number#4, cnt#17] +Input [8]: [ss_ticket_number#4, ss_customer_sk#1, cnt#17, c_customer_sk#18, c_salutation#19, c_first_name#20, c_last_name#21, c_preferred_cust_flag#22] + +(31) Exchange +Input [6]: [c_last_name#21, c_first_name#20, c_salutation#19, c_preferred_cust_flag#22, ss_ticket_number#4, cnt#17] +Arguments: rangepartitioning(cnt#17 DESC NULLS LAST, 5), ENSURE_REQUIREMENTS, [plan_id=5] + +(32) Sort [codegen id : 7] +Input [6]: [c_last_name#21, c_first_name#20, c_salutation#19, c_preferred_cust_flag#22, ss_ticket_number#4, cnt#17] +Arguments: [cnt#17 DESC NULLS LAST], true, 0 + +===== Subqueries ===== + +Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#5 IN dynamicpruning#6 +BroadcastExchange (37) ++- * ColumnarToRow (36) + +- CometProject (35) + +- CometFilter (34) + +- CometScan parquet spark_catalog.default.date_dim (33) + + +(33) Scan parquet spark_catalog.default.date_dim +Output [3]: [d_date_sk#7, d_year#23, d_dom#24] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_dom), GreaterThanOrEqual(d_dom,1), LessThanOrEqual(d_dom,2), In(d_year, [1999,2000,2001]), IsNotNull(d_date_sk)] +ReadSchema: struct<d_date_sk:int,d_year:int,d_dom:int> + +(34) CometFilter +Input [3]: [d_date_sk#7, d_year#23, d_dom#24] +Condition : ((((isnotnull(d_dom#24) AND (d_dom#24 >= 1)) AND (d_dom#24 <= 2)) AND d_year#23 IN (1999,2000,2001)) AND isnotnull(d_date_sk#7)) + +(35) CometProject +Input [3]: [d_date_sk#7, d_year#23, d_dom#24] +Arguments: [d_date_sk#7], [d_date_sk#7] + +(36) ColumnarToRow [codegen id : 1] +Input [1]: [d_date_sk#7] + +(37) BroadcastExchange +Input [1]: [d_date_sk#7] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=6] + + diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q73/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q73/simplified.txt new file mode 100644 index 000000000..7c5ee1ef5 --- /dev/null +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q73/simplified.txt @@ -0,0 +1,56 @@ +WholeStageCodegen (7) + Sort [cnt] + InputAdapter + Exchange [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] + InputAdapter + Exchange [ss_ticket_number,ss_customer_sk] #2 + WholeStageCodegen (4) + HashAggregate [ss_ticket_number,ss_customer_sk] [count,count] + Project [ss_customer_sk,ss_ticket_number] + BroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] + Project [ss_customer_sk,ss_hdemo_sk,ss_ticket_number] + BroadcastHashJoin [ss_store_sk,s_store_sk] + Project [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + ColumnarToRow + InputAdapter + CometFilter [ss_store_sk,ss_hdemo_sk,ss_customer_sk] + CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_dom,d_year,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dom] + InputAdapter + ReusedExchange [d_date_sk] #3 + InputAdapter + BroadcastExchange #4 + WholeStageCodegen (2) + ColumnarToRow + InputAdapter + CometProject [s_store_sk] + CometFilter [s_county,s_store_sk] + CometScan parquet spark_catalog.default.store [s_store_sk,s_county] + InputAdapter + BroadcastExchange #5 + WholeStageCodegen (3) + ColumnarToRow + InputAdapter + CometProject [hd_demo_sk] + CometFilter [hd_vehicle_count,hd_buy_potential,hd_dep_count,hd_demo_sk] + CometScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_buy_potential,hd_dep_count,hd_vehicle_count] + InputAdapter + BroadcastExchange #6 + WholeStageCodegen (5) + ColumnarToRow + InputAdapter + CometFilter [c_customer_sk] + CometScan parquet spark_catalog.default.customer [c_customer_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q74/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q74/explain.txt new file mode 100644 index 000000000..dad94eb2c --- /dev/null +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q74/explain.txt @@ -0,0 +1,477 @@ +== Physical Plan == +TakeOrderedAndProject (71) ++- * Project (70) + +- * BroadcastHashJoin Inner BuildRight (69) + :- * Project (52) + : +- * BroadcastHashJoin Inner BuildRight (51) + : :- * BroadcastHashJoin Inner BuildRight (33) + : : :- * Filter (16) + : : : +- * HashAggregate (15) + : : : +- Exchange (14) + : : : +- * HashAggregate (13) + : : : +- * Project (12) + : : : +- * BroadcastHashJoin Inner BuildRight (11) + : : : :- * Project (9) + : : : : +- * BroadcastHashJoin Inner BuildRight (8) + : : : : :- * ColumnarToRow (3) + : : : : : +- CometFilter (2) + : : : : : +- CometScan parquet spark_catalog.default.customer (1) + : : : : +- BroadcastExchange (7) + : : : : +- * ColumnarToRow (6) + : : : : +- CometFilter (5) + : : : : +- CometScan parquet spark_catalog.default.store_sales (4) + : : : +- ReusedExchange (10) + : : +- BroadcastExchange (32) + : : +- * HashAggregate (31) + : : +- Exchange (30) + : : +- * HashAggregate (29) + : : +- * Project (28) + : : +- * BroadcastHashJoin Inner BuildRight (27) + : : :- * Project (25) + : : : +- * BroadcastHashJoin Inner BuildRight (24) + : : : :- * ColumnarToRow (19) + : : : : +- CometFilter (18) + : : : : +- CometScan parquet spark_catalog.default.customer (17) + : : : +- BroadcastExchange (23) + : : : +- * ColumnarToRow (22) + : : : +- CometFilter (21) + : : : +- CometScan parquet spark_catalog.default.store_sales (20) + : : +- ReusedExchange (26) + : +- BroadcastExchange (50) + : +- * Filter (49) + : +- * HashAggregate (48) + : +- Exchange (47) + : +- * HashAggregate (46) + : +- * Project (45) + : +- * BroadcastHashJoin Inner BuildRight (44) + : :- * Project (42) + : : +- * BroadcastHashJoin Inner BuildRight (41) + : : :- * ColumnarToRow (36) + : : : +- CometFilter (35) + : : : +- CometScan parquet spark_catalog.default.customer (34) + : : +- BroadcastExchange (40) + : : +- * ColumnarToRow (39) + : : +- CometFilter (38) + : : +- CometScan parquet spark_catalog.default.web_sales (37) + : +- ReusedExchange (43) + +- BroadcastExchange (68) + +- * HashAggregate (67) + +- Exchange (66) + +- * HashAggregate (65) + +- * Project (64) + +- * BroadcastHashJoin Inner BuildRight (63) + :- * Project (61) + : +- * BroadcastHashJoin Inner BuildRight (60) + : :- * ColumnarToRow (55) + : : +- CometFilter (54) + : : +- CometScan parquet spark_catalog.default.customer (53) + : +- BroadcastExchange (59) + : +- * ColumnarToRow (58) + : +- CometFilter (57) + : +- CometScan parquet spark_catalog.default.web_sales (56) + +- ReusedExchange (62) + + +(1) Scan parquet spark_catalog.default.customer +Output [4]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4] +Batched: true +Location [not included in comparison]/{warehouse_dir}/customer] +PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_customer_id)] +ReadSchema: struct<c_customer_sk:int,c_customer_id:string,c_first_name:string,c_last_name:string> + +(2) CometFilter +Input [4]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4] +Condition : (isnotnull(c_customer_sk#1) AND isnotnull(c_customer_id#2)) + +(3) ColumnarToRow [codegen id : 3] +Input [4]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4] + +(4) Scan parquet spark_catalog.default.store_sales +Output [3]: [ss_customer_sk#5, ss_net_paid#6, ss_sold_date_sk#7] +Batched: true +Location: InMemoryFileIndex [] +PartitionFilters: [isnotnull(ss_sold_date_sk#7), dynamicpruningexpression(ss_sold_date_sk#7 IN dynamicpruning#8)] +PushedFilters: [IsNotNull(ss_customer_sk)] +ReadSchema: struct<ss_customer_sk:int,ss_net_paid:decimal(7,2)> + +(5) CometFilter +Input [3]: [ss_customer_sk#5, ss_net_paid#6, ss_sold_date_sk#7] +Condition : isnotnull(ss_customer_sk#5) + +(6) ColumnarToRow [codegen id : 1] +Input [3]: [ss_customer_sk#5, ss_net_paid#6, ss_sold_date_sk#7] + +(7) BroadcastExchange +Input [3]: [ss_customer_sk#5, ss_net_paid#6, ss_sold_date_sk#7] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=1] + +(8) BroadcastHashJoin [codegen id : 3] +Left keys [1]: [c_customer_sk#1] +Right keys [1]: [ss_customer_sk#5] +Join type: Inner +Join condition: None + +(9) Project [codegen id : 3] +Output [5]: [c_customer_id#2, c_first_name#3, c_last_name#4, ss_net_paid#6, ss_sold_date_sk#7] +Input [7]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, ss_customer_sk#5, ss_net_paid#6, ss_sold_date_sk#7] + +(10) ReusedExchange [Reuses operator id: 75] +Output [2]: [d_date_sk#9, d_year#10] + +(11) BroadcastHashJoin [codegen id : 3] +Left keys [1]: [ss_sold_date_sk#7] +Right keys [1]: [d_date_sk#9] +Join type: Inner +Join condition: None + +(12) Project [codegen id : 3] +Output [5]: [c_customer_id#2, c_first_name#3, c_last_name#4, ss_net_paid#6, d_year#10] +Input [7]: [c_customer_id#2, c_first_name#3, c_last_name#4, ss_net_paid#6, ss_sold_date_sk#7, d_date_sk#9, d_year#10] + +(13) HashAggregate [codegen id : 3] +Input [5]: [c_customer_id#2, c_first_name#3, c_last_name#4, ss_net_paid#6, d_year#10] +Keys [4]: [c_customer_id#2, c_first_name#3, c_last_name#4, d_year#10] +Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#6))] +Aggregate Attributes [1]: [sum#11] +Results [5]: [c_customer_id#2, c_first_name#3, c_last_name#4, d_year#10, sum#12] + +(14) Exchange +Input [5]: [c_customer_id#2, c_first_name#3, c_last_name#4, d_year#10, sum#12] +Arguments: hashpartitioning(c_customer_id#2, c_first_name#3, c_last_name#4, d_year#10, 5), ENSURE_REQUIREMENTS, [plan_id=2] + +(15) HashAggregate [codegen id : 16] +Input [5]: [c_customer_id#2, c_first_name#3, c_last_name#4, d_year#10, sum#12] +Keys [4]: [c_customer_id#2, c_first_name#3, c_last_name#4, d_year#10] +Functions [1]: [sum(UnscaledValue(ss_net_paid#6))] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_paid#6))#13] +Results [2]: [c_customer_id#2 AS customer_id#14, MakeDecimal(sum(UnscaledValue(ss_net_paid#6))#13,17,2) AS year_total#15] + +(16) Filter [codegen id : 16] +Input [2]: [customer_id#14, year_total#15] +Condition : (isnotnull(year_total#15) AND (year_total#15 > 0.00)) + +(17) Scan parquet spark_catalog.default.customer +Output [4]: [c_customer_sk#16, c_customer_id#17, c_first_name#18, c_last_name#19] +Batched: true +Location [not included in comparison]/{warehouse_dir}/customer] +PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_customer_id)] +ReadSchema: struct<c_customer_sk:int,c_customer_id:string,c_first_name:string,c_last_name:string> + +(18) CometFilter +Input [4]: [c_customer_sk#16, c_customer_id#17, c_first_name#18, c_last_name#19] +Condition : (isnotnull(c_customer_sk#16) AND isnotnull(c_customer_id#17)) + +(19) ColumnarToRow [codegen id : 6] +Input [4]: [c_customer_sk#16, c_customer_id#17, c_first_name#18, c_last_name#19] + +(20) Scan parquet spark_catalog.default.store_sales +Output [3]: [ss_customer_sk#20, ss_net_paid#21, ss_sold_date_sk#22] +Batched: true +Location: InMemoryFileIndex [] +PartitionFilters: [isnotnull(ss_sold_date_sk#22), dynamicpruningexpression(ss_sold_date_sk#22 IN dynamicpruning#23)] +PushedFilters: [IsNotNull(ss_customer_sk)] +ReadSchema: struct<ss_customer_sk:int,ss_net_paid:decimal(7,2)> + +(21) CometFilter +Input [3]: [ss_customer_sk#20, ss_net_paid#21, ss_sold_date_sk#22] +Condition : isnotnull(ss_customer_sk#20) + +(22) ColumnarToRow [codegen id : 4] +Input [3]: [ss_customer_sk#20, ss_net_paid#21, ss_sold_date_sk#22] + +(23) BroadcastExchange +Input [3]: [ss_customer_sk#20, ss_net_paid#21, ss_sold_date_sk#22] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=3] + +(24) BroadcastHashJoin [codegen id : 6] +Left keys [1]: [c_customer_sk#16] +Right keys [1]: [ss_customer_sk#20] +Join type: Inner +Join condition: None + +(25) Project [codegen id : 6] +Output [5]: [c_customer_id#17, c_first_name#18, c_last_name#19, ss_net_paid#21, ss_sold_date_sk#22] +Input [7]: [c_customer_sk#16, c_customer_id#17, c_first_name#18, c_last_name#19, ss_customer_sk#20, ss_net_paid#21, ss_sold_date_sk#22] + +(26) ReusedExchange [Reuses operator id: 79] +Output [2]: [d_date_sk#24, d_year#25] + +(27) BroadcastHashJoin [codegen id : 6] +Left keys [1]: [ss_sold_date_sk#22] +Right keys [1]: [d_date_sk#24] +Join type: Inner +Join condition: None + +(28) Project [codegen id : 6] +Output [5]: [c_customer_id#17, c_first_name#18, c_last_name#19, ss_net_paid#21, d_year#25] +Input [7]: [c_customer_id#17, c_first_name#18, c_last_name#19, ss_net_paid#21, ss_sold_date_sk#22, d_date_sk#24, d_year#25] + +(29) HashAggregate [codegen id : 6] +Input [5]: [c_customer_id#17, c_first_name#18, c_last_name#19, ss_net_paid#21, d_year#25] +Keys [4]: [c_customer_id#17, c_first_name#18, c_last_name#19, d_year#25] +Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#21))] +Aggregate Attributes [1]: [sum#26] +Results [5]: [c_customer_id#17, c_first_name#18, c_last_name#19, d_year#25, sum#27] + +(30) Exchange +Input [5]: [c_customer_id#17, c_first_name#18, c_last_name#19, d_year#25, sum#27] +Arguments: hashpartitioning(c_customer_id#17, c_first_name#18, c_last_name#19, d_year#25, 5), ENSURE_REQUIREMENTS, [plan_id=4] + +(31) HashAggregate [codegen id : 7] +Input [5]: [c_customer_id#17, c_first_name#18, c_last_name#19, d_year#25, sum#27] +Keys [4]: [c_customer_id#17, c_first_name#18, c_last_name#19, d_year#25] +Functions [1]: [sum(UnscaledValue(ss_net_paid#21))] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_paid#21))#13] +Results [4]: [c_customer_id#17 AS customer_id#28, c_first_name#18 AS customer_first_name#29, c_last_name#19 AS customer_last_name#30, MakeDecimal(sum(UnscaledValue(ss_net_paid#21))#13,17,2) AS year_total#31] + +(32) BroadcastExchange +Input [4]: [customer_id#28, customer_first_name#29, customer_last_name#30, year_total#31] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=5] + +(33) BroadcastHashJoin [codegen id : 16] +Left keys [1]: [customer_id#14] +Right keys [1]: [customer_id#28] +Join type: Inner +Join condition: None + +(34) Scan parquet spark_catalog.default.customer +Output [4]: [c_customer_sk#32, c_customer_id#33, c_first_name#34, c_last_name#35] +Batched: true +Location [not included in comparison]/{warehouse_dir}/customer] +PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_customer_id)] +ReadSchema: struct<c_customer_sk:int,c_customer_id:string,c_first_name:string,c_last_name:string> + +(35) CometFilter +Input [4]: [c_customer_sk#32, c_customer_id#33, c_first_name#34, c_last_name#35] +Condition : (isnotnull(c_customer_sk#32) AND isnotnull(c_customer_id#33)) + +(36) ColumnarToRow [codegen id : 10] +Input [4]: [c_customer_sk#32, c_customer_id#33, c_first_name#34, c_last_name#35] + +(37) Scan parquet spark_catalog.default.web_sales +Output [3]: [ws_bill_customer_sk#36, ws_net_paid#37, ws_sold_date_sk#38] +Batched: true +Location: InMemoryFileIndex [] +PartitionFilters: [isnotnull(ws_sold_date_sk#38), dynamicpruningexpression(ws_sold_date_sk#38 IN dynamicpruning#39)] +PushedFilters: [IsNotNull(ws_bill_customer_sk)] +ReadSchema: struct<ws_bill_customer_sk:int,ws_net_paid:decimal(7,2)> + +(38) CometFilter +Input [3]: [ws_bill_customer_sk#36, ws_net_paid#37, ws_sold_date_sk#38] +Condition : isnotnull(ws_bill_customer_sk#36) + +(39) ColumnarToRow [codegen id : 8] +Input [3]: [ws_bill_customer_sk#36, ws_net_paid#37, ws_sold_date_sk#38] + +(40) BroadcastExchange +Input [3]: [ws_bill_customer_sk#36, ws_net_paid#37, ws_sold_date_sk#38] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=6] + +(41) BroadcastHashJoin [codegen id : 10] +Left keys [1]: [c_customer_sk#32] +Right keys [1]: [ws_bill_customer_sk#36] +Join type: Inner +Join condition: None + +(42) Project [codegen id : 10] +Output [5]: [c_customer_id#33, c_first_name#34, c_last_name#35, ws_net_paid#37, ws_sold_date_sk#38] +Input [7]: [c_customer_sk#32, c_customer_id#33, c_first_name#34, c_last_name#35, ws_bill_customer_sk#36, ws_net_paid#37, ws_sold_date_sk#38] + +(43) ReusedExchange [Reuses operator id: 75] +Output [2]: [d_date_sk#40, d_year#41] + +(44) BroadcastHashJoin [codegen id : 10] +Left keys [1]: [ws_sold_date_sk#38] +Right keys [1]: [d_date_sk#40] +Join type: Inner +Join condition: None + +(45) Project [codegen id : 10] +Output [5]: [c_customer_id#33, c_first_name#34, c_last_name#35, ws_net_paid#37, d_year#41] +Input [7]: [c_customer_id#33, c_first_name#34, c_last_name#35, ws_net_paid#37, ws_sold_date_sk#38, d_date_sk#40, d_year#41] + +(46) HashAggregate [codegen id : 10] +Input [5]: [c_customer_id#33, c_first_name#34, c_last_name#35, ws_net_paid#37, d_year#41] +Keys [4]: [c_customer_id#33, c_first_name#34, c_last_name#35, d_year#41] +Functions [1]: [partial_sum(UnscaledValue(ws_net_paid#37))] +Aggregate Attributes [1]: [sum#42] +Results [5]: [c_customer_id#33, c_first_name#34, c_last_name#35, d_year#41, sum#43] + +(47) Exchange +Input [5]: [c_customer_id#33, c_first_name#34, c_last_name#35, d_year#41, sum#43] +Arguments: hashpartitioning(c_customer_id#33, c_first_name#34, c_last_name#35, d_year#41, 5), ENSURE_REQUIREMENTS, [plan_id=7] + +(48) HashAggregate [codegen id : 11] +Input [5]: [c_customer_id#33, c_first_name#34, c_last_name#35, d_year#41, sum#43] +Keys [4]: [c_customer_id#33, c_first_name#34, c_last_name#35, d_year#41] +Functions [1]: [sum(UnscaledValue(ws_net_paid#37))] +Aggregate Attributes [1]: [sum(UnscaledValue(ws_net_paid#37))#44] +Results [2]: [c_customer_id#33 AS customer_id#45, MakeDecimal(sum(UnscaledValue(ws_net_paid#37))#44,17,2) AS year_total#46] + +(49) Filter [codegen id : 11] +Input [2]: [customer_id#45, year_total#46] +Condition : (isnotnull(year_total#46) AND (year_total#46 > 0.00)) + +(50) BroadcastExchange +Input [2]: [customer_id#45, year_total#46] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=8] + +(51) BroadcastHashJoin [codegen id : 16] +Left keys [1]: [customer_id#14] +Right keys [1]: [customer_id#45] +Join type: Inner +Join condition: None + +(52) Project [codegen id : 16] +Output [7]: [customer_id#14, year_total#15, customer_id#28, customer_first_name#29, customer_last_name#30, year_total#31, year_total#46] +Input [8]: [customer_id#14, year_total#15, customer_id#28, customer_first_name#29, customer_last_name#30, year_total#31, customer_id#45, year_total#46] + +(53) Scan parquet spark_catalog.default.customer +Output [4]: [c_customer_sk#47, c_customer_id#48, c_first_name#49, c_last_name#50] +Batched: true +Location [not included in comparison]/{warehouse_dir}/customer] +PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_customer_id)] +ReadSchema: struct<c_customer_sk:int,c_customer_id:string,c_first_name:string,c_last_name:string> + +(54) CometFilter +Input [4]: [c_customer_sk#47, c_customer_id#48, c_first_name#49, c_last_name#50] +Condition : (isnotnull(c_customer_sk#47) AND isnotnull(c_customer_id#48)) + +(55) ColumnarToRow [codegen id : 14] +Input [4]: [c_customer_sk#47, c_customer_id#48, c_first_name#49, c_last_name#50] + +(56) Scan parquet spark_catalog.default.web_sales +Output [3]: [ws_bill_customer_sk#51, ws_net_paid#52, ws_sold_date_sk#53] +Batched: true +Location: InMemoryFileIndex [] +PartitionFilters: [isnotnull(ws_sold_date_sk#53), dynamicpruningexpression(ws_sold_date_sk#53 IN dynamicpruning#54)] +PushedFilters: [IsNotNull(ws_bill_customer_sk)] +ReadSchema: struct<ws_bill_customer_sk:int,ws_net_paid:decimal(7,2)> + +(57) CometFilter +Input [3]: [ws_bill_customer_sk#51, ws_net_paid#52, ws_sold_date_sk#53] +Condition : isnotnull(ws_bill_customer_sk#51) + +(58) ColumnarToRow [codegen id : 12] +Input [3]: [ws_bill_customer_sk#51, ws_net_paid#52, ws_sold_date_sk#53] + +(59) BroadcastExchange +Input [3]: [ws_bill_customer_sk#51, ws_net_paid#52, ws_sold_date_sk#53] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=9] + +(60) BroadcastHashJoin [codegen id : 14] +Left keys [1]: [c_customer_sk#47] +Right keys [1]: [ws_bill_customer_sk#51] +Join type: Inner +Join condition: None + +(61) Project [codegen id : 14] +Output [5]: [c_customer_id#48, c_first_name#49, c_last_name#50, ws_net_paid#52, ws_sold_date_sk#53] +Input [7]: [c_customer_sk#47, c_customer_id#48, c_first_name#49, c_last_name#50, ws_bill_customer_sk#51, ws_net_paid#52, ws_sold_date_sk#53] + +(62) ReusedExchange [Reuses operator id: 79] +Output [2]: [d_date_sk#55, d_year#56] + +(63) BroadcastHashJoin [codegen id : 14] +Left keys [1]: [ws_sold_date_sk#53] +Right keys [1]: [d_date_sk#55] +Join type: Inner +Join condition: None + +(64) Project [codegen id : 14] +Output [5]: [c_customer_id#48, c_first_name#49, c_last_name#50, ws_net_paid#52, d_year#56] +Input [7]: [c_customer_id#48, c_first_name#49, c_last_name#50, ws_net_paid#52, ws_sold_date_sk#53, d_date_sk#55, d_year#56] + +(65) HashAggregate [codegen id : 14] +Input [5]: [c_customer_id#48, c_first_name#49, c_last_name#50, ws_net_paid#52, d_year#56] +Keys [4]: [c_customer_id#48, c_first_name#49, c_last_name#50, d_year#56] +Functions [1]: [partial_sum(UnscaledValue(ws_net_paid#52))] +Aggregate Attributes [1]: [sum#57] +Results [5]: [c_customer_id#48, c_first_name#49, c_last_name#50, d_year#56, sum#58] + +(66) Exchange +Input [5]: [c_customer_id#48, c_first_name#49, c_last_name#50, d_year#56, sum#58] +Arguments: hashpartitioning(c_customer_id#48, c_first_name#49, c_last_name#50, d_year#56, 5), ENSURE_REQUIREMENTS, [plan_id=10] + +(67) HashAggregate [codegen id : 15] +Input [5]: [c_customer_id#48, c_first_name#49, c_last_name#50, d_year#56, sum#58] +Keys [4]: [c_customer_id#48, c_first_name#49, c_last_name#50, d_year#56] +Functions [1]: [sum(UnscaledValue(ws_net_paid#52))] +Aggregate Attributes [1]: [sum(UnscaledValue(ws_net_paid#52))#44] +Results [2]: [c_customer_id#48 AS customer_id#59, MakeDecimal(sum(UnscaledValue(ws_net_paid#52))#44,17,2) AS year_total#60] + +(68) BroadcastExchange +Input [2]: [customer_id#59, year_total#60] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=11] + +(69) BroadcastHashJoin [codegen id : 16] +Left keys [1]: [customer_id#14] +Right keys [1]: [customer_id#59] +Join type: Inner +Join condition: (CASE WHEN (year_total#46 > 0.00) THEN (year_total#60 / year_total#46) END > CASE WHEN (year_total#15 > 0.00) THEN (year_total#31 / year_total#15) END) + +(70) Project [codegen id : 16] +Output [3]: [customer_id#28, customer_first_name#29, customer_last_name#30] +Input [9]: [customer_id#14, year_total#15, customer_id#28, customer_first_name#29, customer_last_name#30, year_total#31, year_total#46, customer_id#59, year_total#60] + +(71) TakeOrderedAndProject +Input [3]: [customer_id#28, customer_first_name#29, customer_last_name#30] +Arguments: 100, [customer_id#28 ASC NULLS FIRST, customer_id#28 ASC NULLS FIRST, customer_id#28 ASC NULLS FIRST], [customer_id#28, customer_first_name#29, customer_last_name#30] + +===== Subqueries ===== + +Subquery:1 Hosting operator id = 4 Hosting Expression = ss_sold_date_sk#7 IN dynamicpruning#8 +BroadcastExchange (75) ++- * ColumnarToRow (74) + +- CometFilter (73) + +- CometScan parquet spark_catalog.default.date_dim (72) + + +(72) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#9, d_year#10] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), In(d_year, [2001,2002]), IsNotNull(d_date_sk)] +ReadSchema: struct<d_date_sk:int,d_year:int> + +(73) CometFilter +Input [2]: [d_date_sk#9, d_year#10] +Condition : (((isnotnull(d_year#10) AND (d_year#10 = 2001)) AND d_year#10 IN (2001,2002)) AND isnotnull(d_date_sk#9)) + +(74) ColumnarToRow [codegen id : 1] +Input [2]: [d_date_sk#9, d_year#10] + +(75) BroadcastExchange +Input [2]: [d_date_sk#9, d_year#10] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=12] + +Subquery:2 Hosting operator id = 20 Hosting Expression = ss_sold_date_sk#22 IN dynamicpruning#23 +BroadcastExchange (79) ++- * ColumnarToRow (78) + +- CometFilter (77) + +- CometScan parquet spark_catalog.default.date_dim (76) + + +(76) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#24, d_year#25] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2002), In(d_year, [2001,2002]), IsNotNull(d_date_sk)] +ReadSchema: struct<d_date_sk:int,d_year:int> + +(77) CometFilter +Input [2]: [d_date_sk#24, d_year#25] +Condition : (((isnotnull(d_year#25) AND (d_year#25 = 2002)) AND d_year#25 IN (2001,2002)) AND isnotnull(d_date_sk#24)) + +(78) ColumnarToRow [codegen id : 1] +Input [2]: [d_date_sk#24, d_year#25] + +(79) BroadcastExchange +Input [2]: [d_date_sk#24, d_year#25] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=13] + +Subquery:3 Hosting operator id = 37 Hosting Expression = ws_sold_date_sk#38 IN dynamicpruning#8 + +Subquery:4 Hosting operator id = 56 Hosting Expression = ws_sold_date_sk#53 IN dynamicpruning#23 + + diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q74/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q74/simplified.txt new file mode 100644 index 000000000..9d3ae8fbe --- /dev/null +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q74/simplified.txt @@ -0,0 +1,122 @@ +TakeOrderedAndProject [customer_id,customer_first_name,customer_last_name] + WholeStageCodegen (16) + Project [customer_id,customer_first_name,customer_last_name] + BroadcastHashJoin [customer_id,customer_id,year_total,year_total,year_total,year_total] + Project [customer_id,year_total,customer_id,customer_first_name,customer_last_name,year_total,year_total] + BroadcastHashJoin [customer_id,customer_id] + BroadcastHashJoin [customer_id,customer_id] + Filter [year_total] + HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,sum] [sum(UnscaledValue(ss_net_paid)),customer_id,year_total,sum] + InputAdapter + Exchange [c_customer_id,c_first_name,c_last_name,d_year] #1 + WholeStageCodegen (3) + HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,ss_net_paid] [sum,sum] + Project [c_customer_id,c_first_name,c_last_name,ss_net_paid,d_year] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [c_customer_id,c_first_name,c_last_name,ss_net_paid,ss_sold_date_sk] + BroadcastHashJoin [c_customer_sk,ss_customer_sk] + ColumnarToRow + InputAdapter + CometFilter [c_customer_sk,c_customer_id] + CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name] + InputAdapter + BroadcastExchange #2 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometFilter [ss_customer_sk] + CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_net_paid,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometFilter [d_year,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + InputAdapter + ReusedExchange [d_date_sk,d_year] #3 + InputAdapter + BroadcastExchange #4 + WholeStageCodegen (7) + HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,sum] [sum(UnscaledValue(ss_net_paid)),customer_id,customer_first_name,customer_last_name,year_total,sum] + InputAdapter + Exchange [c_customer_id,c_first_name,c_last_name,d_year] #5 + WholeStageCodegen (6) + HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,ss_net_paid] [sum,sum] + Project [c_customer_id,c_first_name,c_last_name,ss_net_paid,d_year] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [c_customer_id,c_first_name,c_last_name,ss_net_paid,ss_sold_date_sk] + BroadcastHashJoin [c_customer_sk,ss_customer_sk] + ColumnarToRow + InputAdapter + CometFilter [c_customer_sk,c_customer_id] + CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name] + InputAdapter + BroadcastExchange #6 + WholeStageCodegen (4) + ColumnarToRow + InputAdapter + CometFilter [ss_customer_sk] + CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_net_paid,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #2 + BroadcastExchange #7 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometFilter [d_year,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + InputAdapter + ReusedExchange [d_date_sk,d_year] #7 + InputAdapter + BroadcastExchange #8 + WholeStageCodegen (11) + Filter [year_total] + HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,sum] [sum(UnscaledValue(ws_net_paid)),customer_id,year_total,sum] + InputAdapter + Exchange [c_customer_id,c_first_name,c_last_name,d_year] #9 + WholeStageCodegen (10) + HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,ws_net_paid] [sum,sum] + Project [c_customer_id,c_first_name,c_last_name,ws_net_paid,d_year] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + Project [c_customer_id,c_first_name,c_last_name,ws_net_paid,ws_sold_date_sk] + BroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] + ColumnarToRow + InputAdapter + CometFilter [c_customer_sk,c_customer_id] + CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name] + InputAdapter + BroadcastExchange #10 + WholeStageCodegen (8) + ColumnarToRow + InputAdapter + CometFilter [ws_bill_customer_sk] + CometScan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + InputAdapter + ReusedExchange [d_date_sk,d_year] #3 + InputAdapter + BroadcastExchange #11 + WholeStageCodegen (15) + HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,sum] [sum(UnscaledValue(ws_net_paid)),customer_id,year_total,sum] + InputAdapter + Exchange [c_customer_id,c_first_name,c_last_name,d_year] #12 + WholeStageCodegen (14) + HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,ws_net_paid] [sum,sum] + Project [c_customer_id,c_first_name,c_last_name,ws_net_paid,d_year] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + Project [c_customer_id,c_first_name,c_last_name,ws_net_paid,ws_sold_date_sk] + BroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] + ColumnarToRow + InputAdapter + CometFilter [c_customer_sk,c_customer_id] + CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name] + InputAdapter + BroadcastExchange #13 + WholeStageCodegen (12) + ColumnarToRow + InputAdapter + CometFilter [ws_bill_customer_sk] + CometScan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #2 + InputAdapter + ReusedExchange [d_date_sk,d_year] #7 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 new file mode 100644 index 000000000..a3d12a1d4 --- /dev/null +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q75/explain.txt @@ -0,0 +1,779 @@ +== Physical Plan == +TakeOrderedAndProject (129) ++- * Project (128) + +- * SortMergeJoin Inner (127) + :- * Sort (71) + : +- Exchange (70) + : +- * Filter (69) + : +- * HashAggregate (68) + : +- Exchange (67) + : +- * HashAggregate (66) + : +- * HashAggregate (65) + : +- Exchange (64) + : +- * HashAggregate (63) + : +- Union (62) + : :- * Project (23) + : : +- * SortMergeJoin LeftOuter (22) + : : :- * Sort (15) + : : : +- Exchange (14) + : : : +- * Project (13) + : : : +- * BroadcastHashJoin Inner BuildRight (12) + : : : :- * Project (10) + : : : : +- * BroadcastHashJoin Inner BuildRight (9) + : : : : :- * ColumnarToRow (3) + : : : : : +- CometFilter (2) + : : : : : +- CometScan parquet spark_catalog.default.catalog_sales (1) + : : : : +- BroadcastExchange (8) + : : : : +- * ColumnarToRow (7) + : : : : +- CometProject (6) + : : : : +- CometFilter (5) + : : : : +- CometScan parquet spark_catalog.default.item (4) + : : : +- ReusedExchange (11) + : : +- * Sort (21) + : : +- Exchange (20) + : : +- * ColumnarToRow (19) + : : +- CometProject (18) + : : +- CometFilter (17) + : : +- CometScan parquet spark_catalog.default.catalog_returns (16) + : :- * Project (42) + : : +- * SortMergeJoin LeftOuter (41) + : : :- * Sort (34) + : : : +- Exchange (33) + : : : +- * Project (32) + : : : +- * BroadcastHashJoin Inner BuildRight (31) + : : : :- * Project (29) + : : : : +- * BroadcastHashJoin Inner BuildRight (28) + : : : : :- * ColumnarToRow (26) + : : : : : +- CometFilter (25) + : : : : : +- CometScan parquet spark_catalog.default.store_sales (24) + : : : : +- ReusedExchange (27) + : : : +- ReusedExchange (30) + : : +- * Sort (40) + : : +- Exchange (39) + : : +- * ColumnarToRow (38) + : : +- CometProject (37) + : : +- CometFilter (36) + : : +- CometScan parquet spark_catalog.default.store_returns (35) + : +- * Project (61) + : +- * SortMergeJoin LeftOuter (60) + : :- * Sort (53) + : : +- Exchange (52) + : : +- * Project (51) + : : +- * BroadcastHashJoin Inner BuildRight (50) + : : :- * Project (48) + : : : +- * BroadcastHashJoin Inner BuildRight (47) + : : : :- * ColumnarToRow (45) + : : : : +- CometFilter (44) + : : : : +- CometScan parquet spark_catalog.default.web_sales (43) + : : : +- ReusedExchange (46) + : : +- ReusedExchange (49) + : +- * Sort (59) + : +- Exchange (58) + : +- * ColumnarToRow (57) + : +- CometProject (56) + : +- CometFilter (55) + : +- CometScan parquet spark_catalog.default.web_returns (54) + +- * Sort (126) + +- Exchange (125) + +- * Filter (124) + +- * HashAggregate (123) + +- Exchange (122) + +- * HashAggregate (121) + +- * HashAggregate (120) + +- Exchange (119) + +- * HashAggregate (118) + +- Union (117) + :- * Project (86) + : +- * SortMergeJoin LeftOuter (85) + : :- * Sort (82) + : : +- Exchange (81) + : : +- * Project (80) + : : +- * BroadcastHashJoin Inner BuildRight (79) + : : :- * Project (77) + : : : +- * BroadcastHashJoin Inner BuildRight (76) + : : : :- * ColumnarToRow (74) + : : : : +- CometFilter (73) + : : : : +- CometScan parquet spark_catalog.default.catalog_sales (72) + : : : +- ReusedExchange (75) + : : +- ReusedExchange (78) + : +- * Sort (84) + : +- ReusedExchange (83) + :- * Project (101) + : +- * SortMergeJoin LeftOuter (100) + : :- * Sort (97) + : : +- Exchange (96) + : : +- * Project (95) + : : +- * BroadcastHashJoin Inner BuildRight (94) + : : :- * Project (92) + : : : +- * BroadcastHashJoin Inner BuildRight (91) + : : : :- * ColumnarToRow (89) + : : : : +- CometFilter (88) + : : : : +- CometScan parquet spark_catalog.default.store_sales (87) + : : : +- ReusedExchange (90) + : : +- ReusedExchange (93) + : +- * Sort (99) + : +- ReusedExchange (98) + +- * Project (116) + +- * SortMergeJoin LeftOuter (115) + :- * Sort (112) + : +- Exchange (111) + : +- * Project (110) + : +- * BroadcastHashJoin Inner BuildRight (109) + : :- * Project (107) + : : +- * BroadcastHashJoin Inner BuildRight (106) + : : :- * ColumnarToRow (104) + : : : +- CometFilter (103) + : : : +- CometScan parquet spark_catalog.default.web_sales (102) + : : +- ReusedExchange (105) + : +- ReusedExchange (108) + +- * Sort (114) + +- ReusedExchange (113) + + +(1) Scan parquet spark_catalog.default.catalog_sales +Output [5]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, cs_sold_date_sk#5] +Batched: true +Location: InMemoryFileIndex [] +PartitionFilters: [isnotnull(cs_sold_date_sk#5), dynamicpruningexpression(cs_sold_date_sk#5 IN dynamicpruning#6)] +PushedFilters: [IsNotNull(cs_item_sk)] +ReadSchema: struct<cs_item_sk:int,cs_order_number:int,cs_quantity:int,cs_ext_sales_price:decimal(7,2)> + +(2) CometFilter +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) + +(3) ColumnarToRow [codegen id : 3] +Input [5]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, cs_sold_date_sk#5] + +(4) Scan parquet spark_catalog.default.item +Output [6]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_category#11, i_manufact_id#12] +Batched: true +Location [not included in comparison]/{warehouse_dir}/item] +PushedFilters: [IsNotNull(i_category), EqualTo(i_category,Books ), IsNotNull(i_item_sk), IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id), IsNotNull(i_manufact_id)] +ReadSchema: struct<i_item_sk:int,i_brand_id:int,i_class_id:int,i_category_id:int,i_category:string,i_manufact_id:int> + +(5) CometFilter +Input [6]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_category#11, i_manufact_id#12] +Condition : ((((((isnotnull(i_category#11) AND (i_category#11 = Books )) AND isnotnull(i_item_sk#7)) AND isnotnull(i_brand_id#8)) AND isnotnull(i_class_id#9)) AND isnotnull(i_category_id#10)) AND isnotnull(i_manufact_id#12)) + +(6) CometProject +Input [6]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_category#11, i_manufact_id#12] +Arguments: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12], [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] + +(7) ColumnarToRow [codegen id : 1] +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] +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] +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: 133] +Output [2]: [d_date_sk#13, d_year#14] + +(12) BroadcastHashJoin [codegen id : 3] +Left keys [1]: [cs_sold_date_sk#5] +Right keys [1]: [d_date_sk#13] +Join type: Inner +Join condition: None + +(13) Project [codegen id : 3] +Output [9]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_year#14] +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] + +(14) Exchange +Input [9]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_year#14] +Arguments: hashpartitioning(cs_order_number#2, cs_item_sk#1, 5), ENSURE_REQUIREMENTS, [plan_id=2] + +(15) Sort [codegen id : 4] +Input [9]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_year#14] +Arguments: [cs_order_number#2 ASC NULLS FIRST, cs_item_sk#1 ASC NULLS FIRST], false, 0 + +(16) Scan parquet spark_catalog.default.catalog_returns +Output [5]: [cr_item_sk#15, cr_order_number#16, cr_return_quantity#17, cr_return_amount#18, cr_returned_date_sk#19] +Batched: true +Location [not included in comparison]/{warehouse_dir}/catalog_returns] +PushedFilters: [IsNotNull(cr_order_number), IsNotNull(cr_item_sk)] +ReadSchema: struct<cr_item_sk:int,cr_order_number:int,cr_return_quantity:int,cr_return_amount:decimal(7,2)> + +(17) CometFilter +Input [5]: [cr_item_sk#15, cr_order_number#16, cr_return_quantity#17, cr_return_amount#18, cr_returned_date_sk#19] +Condition : (isnotnull(cr_order_number#16) AND isnotnull(cr_item_sk#15)) + +(18) CometProject +Input [5]: [cr_item_sk#15, cr_order_number#16, cr_return_quantity#17, cr_return_amount#18, cr_returned_date_sk#19] +Arguments: [cr_item_sk#15, cr_order_number#16, cr_return_quantity#17, cr_return_amount#18], [cr_item_sk#15, cr_order_number#16, cr_return_quantity#17, cr_return_amount#18] + +(19) ColumnarToRow [codegen id : 5] +Input [4]: [cr_item_sk#15, cr_order_number#16, cr_return_quantity#17, cr_return_amount#18] + +(20) Exchange +Input [4]: [cr_item_sk#15, cr_order_number#16, cr_return_quantity#17, cr_return_amount#18] +Arguments: hashpartitioning(cr_order_number#16, cr_item_sk#15, 5), ENSURE_REQUIREMENTS, [plan_id=3] + +(21) Sort [codegen id : 6] +Input [4]: [cr_item_sk#15, cr_order_number#16, cr_return_quantity#17, cr_return_amount#18] +Arguments: [cr_order_number#16 ASC NULLS FIRST, cr_item_sk#15 ASC NULLS FIRST], false, 0 + +(22) SortMergeJoin [codegen id : 7] +Left keys [2]: [cs_order_number#2, cs_item_sk#1] +Right keys [2]: [cr_order_number#16, cr_item_sk#15] +Join type: LeftOuter +Join condition: None + +(23) Project [codegen id : 7] +Output [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, (cs_quantity#3 - coalesce(cr_return_quantity#17, 0)) AS sales_cnt#20, (cs_ext_sales_price#4 - coalesce(cr_return_amount#18, 0.00)) AS sales_amt#21] +Input [13]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_year#14, cr_item_sk#15, cr_order_number#16, cr_return_quantity#17, cr_return_amount#18] + +(24) Scan parquet spark_catalog.default.store_sales +Output [5]: [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, ss_sold_date_sk#26] +Batched: true +Location: InMemoryFileIndex [] +PartitionFilters: [isnotnull(ss_sold_date_sk#26), dynamicpruningexpression(ss_sold_date_sk#26 IN dynamicpruning#27)] +PushedFilters: [IsNotNull(ss_item_sk)] +ReadSchema: struct<ss_item_sk:int,ss_ticket_number:int,ss_quantity:int,ss_ext_sales_price:decimal(7,2)> + +(25) CometFilter +Input [5]: [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, ss_sold_date_sk#26] +Condition : isnotnull(ss_item_sk#22) + +(26) ColumnarToRow [codegen id : 10] +Input [5]: [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, ss_sold_date_sk#26] + +(27) ReusedExchange [Reuses operator id: 8] +Output [5]: [i_item_sk#28, i_brand_id#29, i_class_id#30, i_category_id#31, i_manufact_id#32] + +(28) BroadcastHashJoin [codegen id : 10] +Left keys [1]: [ss_item_sk#22] +Right keys [1]: [i_item_sk#28] +Join type: Inner +Join condition: None + +(29) Project [codegen id : 10] +Output [9]: [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, ss_sold_date_sk#26, i_brand_id#29, i_class_id#30, i_category_id#31, i_manufact_id#32] +Input [10]: [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, ss_sold_date_sk#26, i_item_sk#28, i_brand_id#29, i_class_id#30, i_category_id#31, i_manufact_id#32] + +(30) ReusedExchange [Reuses operator id: 133] +Output [2]: [d_date_sk#33, d_year#34] + +(31) BroadcastHashJoin [codegen id : 10] +Left keys [1]: [ss_sold_date_sk#26] +Right keys [1]: [d_date_sk#33] +Join type: Inner +Join condition: None + +(32) Project [codegen id : 10] +Output [9]: [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, i_brand_id#29, i_class_id#30, i_category_id#31, i_manufact_id#32, d_year#34] +Input [11]: [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, ss_sold_date_sk#26, i_brand_id#29, i_class_id#30, i_category_id#31, i_manufact_id#32, d_date_sk#33, d_year#34] + +(33) Exchange +Input [9]: [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, i_brand_id#29, i_class_id#30, i_category_id#31, i_manufact_id#32, d_year#34] +Arguments: hashpartitioning(ss_ticket_number#23, ss_item_sk#22, 5), ENSURE_REQUIREMENTS, [plan_id=4] + +(34) Sort [codegen id : 11] +Input [9]: [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, i_brand_id#29, i_class_id#30, i_category_id#31, i_manufact_id#32, d_year#34] +Arguments: [ss_ticket_number#23 ASC NULLS FIRST, ss_item_sk#22 ASC NULLS FIRST], false, 0 + +(35) Scan parquet spark_catalog.default.store_returns +Output [5]: [sr_item_sk#35, sr_ticket_number#36, sr_return_quantity#37, sr_return_amt#38, sr_returned_date_sk#39] +Batched: true +Location [not included in comparison]/{warehouse_dir}/store_returns] +PushedFilters: [IsNotNull(sr_ticket_number), IsNotNull(sr_item_sk)] +ReadSchema: struct<sr_item_sk:int,sr_ticket_number:int,sr_return_quantity:int,sr_return_amt:decimal(7,2)> + +(36) CometFilter +Input [5]: [sr_item_sk#35, sr_ticket_number#36, sr_return_quantity#37, sr_return_amt#38, sr_returned_date_sk#39] +Condition : (isnotnull(sr_ticket_number#36) AND isnotnull(sr_item_sk#35)) + +(37) CometProject +Input [5]: [sr_item_sk#35, sr_ticket_number#36, sr_return_quantity#37, sr_return_amt#38, sr_returned_date_sk#39] +Arguments: [sr_item_sk#35, sr_ticket_number#36, sr_return_quantity#37, sr_return_amt#38], [sr_item_sk#35, sr_ticket_number#36, sr_return_quantity#37, sr_return_amt#38] + +(38) ColumnarToRow [codegen id : 12] +Input [4]: [sr_item_sk#35, sr_ticket_number#36, sr_return_quantity#37, sr_return_amt#38] + +(39) Exchange +Input [4]: [sr_item_sk#35, sr_ticket_number#36, sr_return_quantity#37, sr_return_amt#38] +Arguments: hashpartitioning(sr_ticket_number#36, sr_item_sk#35, 5), ENSURE_REQUIREMENTS, [plan_id=5] + +(40) Sort [codegen id : 13] +Input [4]: [sr_item_sk#35, sr_ticket_number#36, sr_return_quantity#37, sr_return_amt#38] +Arguments: [sr_ticket_number#36 ASC NULLS FIRST, sr_item_sk#35 ASC NULLS FIRST], false, 0 + +(41) SortMergeJoin [codegen id : 14] +Left keys [2]: [ss_ticket_number#23, ss_item_sk#22] +Right keys [2]: [sr_ticket_number#36, sr_item_sk#35] +Join type: LeftOuter +Join condition: None + +(42) Project [codegen id : 14] +Output [7]: [d_year#34, i_brand_id#29, i_class_id#30, i_category_id#31, i_manufact_id#32, (ss_quantity#24 - coalesce(sr_return_quantity#37, 0)) AS sales_cnt#40, (ss_ext_sales_price#25 - coalesce(sr_return_amt#38, 0.00)) AS sales_amt#41] +Input [13]: [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, i_brand_id#29, i_class_id#30, i_category_id#31, i_manufact_id#32, d_year#34, sr_item_sk#35, sr_ticket_number#36, sr_return_quantity#37, sr_return_amt#38] + +(43) Scan parquet spark_catalog.default.web_sales +Output [5]: [ws_item_sk#42, ws_order_number#43, ws_quantity#44, ws_ext_sales_price#45, ws_sold_date_sk#46] +Batched: true +Location: InMemoryFileIndex [] +PartitionFilters: [isnotnull(ws_sold_date_sk#46), dynamicpruningexpression(ws_sold_date_sk#46 IN dynamicpruning#47)] +PushedFilters: [IsNotNull(ws_item_sk)] +ReadSchema: struct<ws_item_sk:int,ws_order_number:int,ws_quantity:int,ws_ext_sales_price:decimal(7,2)> + +(44) CometFilter +Input [5]: [ws_item_sk#42, ws_order_number#43, ws_quantity#44, ws_ext_sales_price#45, ws_sold_date_sk#46] +Condition : isnotnull(ws_item_sk#42) + +(45) ColumnarToRow [codegen id : 17] +Input [5]: [ws_item_sk#42, ws_order_number#43, ws_quantity#44, ws_ext_sales_price#45, ws_sold_date_sk#46] + +(46) ReusedExchange [Reuses operator id: 8] +Output [5]: [i_item_sk#48, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52] + +(47) BroadcastHashJoin [codegen id : 17] +Left keys [1]: [ws_item_sk#42] +Right keys [1]: [i_item_sk#48] +Join type: Inner +Join condition: None + +(48) Project [codegen id : 17] +Output [9]: [ws_item_sk#42, ws_order_number#43, ws_quantity#44, ws_ext_sales_price#45, ws_sold_date_sk#46, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52] +Input [10]: [ws_item_sk#42, ws_order_number#43, ws_quantity#44, ws_ext_sales_price#45, ws_sold_date_sk#46, i_item_sk#48, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52] + +(49) ReusedExchange [Reuses operator id: 133] +Output [2]: [d_date_sk#53, d_year#54] + +(50) BroadcastHashJoin [codegen id : 17] +Left keys [1]: [ws_sold_date_sk#46] +Right keys [1]: [d_date_sk#53] +Join type: Inner +Join condition: None + +(51) Project [codegen id : 17] +Output [9]: [ws_item_sk#42, ws_order_number#43, ws_quantity#44, ws_ext_sales_price#45, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52, d_year#54] +Input [11]: [ws_item_sk#42, ws_order_number#43, ws_quantity#44, ws_ext_sales_price#45, ws_sold_date_sk#46, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52, d_date_sk#53, d_year#54] + +(52) Exchange +Input [9]: [ws_item_sk#42, ws_order_number#43, ws_quantity#44, ws_ext_sales_price#45, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52, d_year#54] +Arguments: hashpartitioning(ws_order_number#43, ws_item_sk#42, 5), ENSURE_REQUIREMENTS, [plan_id=6] + +(53) Sort [codegen id : 18] +Input [9]: [ws_item_sk#42, ws_order_number#43, ws_quantity#44, ws_ext_sales_price#45, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52, d_year#54] +Arguments: [ws_order_number#43 ASC NULLS FIRST, ws_item_sk#42 ASC NULLS FIRST], false, 0 + +(54) Scan parquet spark_catalog.default.web_returns +Output [5]: [wr_item_sk#55, wr_order_number#56, wr_return_quantity#57, wr_return_amt#58, wr_returned_date_sk#59] +Batched: true +Location [not included in comparison]/{warehouse_dir}/web_returns] +PushedFilters: [IsNotNull(wr_order_number), IsNotNull(wr_item_sk)] +ReadSchema: struct<wr_item_sk:int,wr_order_number:int,wr_return_quantity:int,wr_return_amt:decimal(7,2)> + +(55) CometFilter +Input [5]: [wr_item_sk#55, wr_order_number#56, wr_return_quantity#57, wr_return_amt#58, wr_returned_date_sk#59] +Condition : (isnotnull(wr_order_number#56) AND isnotnull(wr_item_sk#55)) + +(56) CometProject +Input [5]: [wr_item_sk#55, wr_order_number#56, wr_return_quantity#57, wr_return_amt#58, wr_returned_date_sk#59] +Arguments: [wr_item_sk#55, wr_order_number#56, wr_return_quantity#57, wr_return_amt#58], [wr_item_sk#55, wr_order_number#56, wr_return_quantity#57, wr_return_amt#58] + +(57) ColumnarToRow [codegen id : 19] +Input [4]: [wr_item_sk#55, wr_order_number#56, wr_return_quantity#57, wr_return_amt#58] + +(58) Exchange +Input [4]: [wr_item_sk#55, wr_order_number#56, wr_return_quantity#57, wr_return_amt#58] +Arguments: hashpartitioning(wr_order_number#56, wr_item_sk#55, 5), ENSURE_REQUIREMENTS, [plan_id=7] + +(59) Sort [codegen id : 20] +Input [4]: [wr_item_sk#55, wr_order_number#56, wr_return_quantity#57, wr_return_amt#58] +Arguments: [wr_order_number#56 ASC NULLS FIRST, wr_item_sk#55 ASC NULLS FIRST], false, 0 + +(60) SortMergeJoin [codegen id : 21] +Left keys [2]: [ws_order_number#43, ws_item_sk#42] +Right keys [2]: [wr_order_number#56, wr_item_sk#55] +Join type: LeftOuter +Join condition: None + +(61) Project [codegen id : 21] +Output [7]: [d_year#54, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52, (ws_quantity#44 - coalesce(wr_return_quantity#57, 0)) AS sales_cnt#60, (ws_ext_sales_price#45 - coalesce(wr_return_amt#58, 0.00)) AS sales_amt#61] +Input [13]: [ws_item_sk#42, ws_order_number#43, ws_quantity#44, ws_ext_sales_price#45, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52, d_year#54, wr_item_sk#55, wr_order_number#56, wr_return_quantity#57, wr_return_amt#58] + +(62) Union + +(63) HashAggregate [codegen id : 22] +Input [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#20, sales_amt#21] +Keys [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#20, sales_amt#21] +Functions: [] +Aggregate Attributes: [] +Results [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#20, sales_amt#21] + +(64) Exchange +Input [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#20, sales_amt#21] +Arguments: hashpartitioning(d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#20, sales_amt#21, 5), ENSURE_REQUIREMENTS, [plan_id=8] + +(65) HashAggregate [codegen id : 23] +Input [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#20, sales_amt#21] +Keys [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#20, sales_amt#21] +Functions: [] +Aggregate Attributes: [] +Results [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#20, sales_amt#21] + +(66) HashAggregate [codegen id : 23] +Input [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#20, sales_amt#21] +Keys [5]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] +Functions [2]: [partial_sum(sales_cnt#20), partial_sum(UnscaledValue(sales_amt#21))] +Aggregate Attributes [2]: [sum#62, sum#63] +Results [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sum#64, sum#65] + +(67) Exchange +Input [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sum#64, sum#65] +Arguments: hashpartitioning(d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, 5), ENSURE_REQUIREMENTS, [plan_id=9] + +(68) HashAggregate [codegen id : 24] +Input [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sum#64, sum#65] +Keys [5]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] +Functions [2]: [sum(sales_cnt#20), sum(UnscaledValue(sales_amt#21))] +Aggregate Attributes [2]: [sum(sales_cnt#20)#66, sum(UnscaledValue(sales_amt#21))#67] +Results [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sum(sales_cnt#20)#66 AS sales_cnt#68, MakeDecimal(sum(UnscaledValue(sales_amt#21))#67,18,2) AS sales_amt#69] + +(69) Filter [codegen id : 24] +Input [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#68, sales_amt#69] +Condition : isnotnull(sales_cnt#68) + +(70) Exchange +Input [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#68, sales_amt#69] +Arguments: hashpartitioning(i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, 5), ENSURE_REQUIREMENTS, [plan_id=10] + +(71) Sort [codegen id : 25] +Input [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#68, sales_amt#69] +Arguments: [i_brand_id#8 ASC NULLS FIRST, i_class_id#9 ASC NULLS FIRST, i_category_id#10 ASC NULLS FIRST, i_manufact_id#12 ASC NULLS FIRST], false, 0 + +(72) Scan parquet spark_catalog.default.catalog_sales +Output [5]: [cs_item_sk#70, cs_order_number#71, cs_quantity#72, cs_ext_sales_price#73, cs_sold_date_sk#74] +Batched: true +Location: InMemoryFileIndex [] +PartitionFilters: [isnotnull(cs_sold_date_sk#74), dynamicpruningexpression(cs_sold_date_sk#74 IN dynamicpruning#75)] +PushedFilters: [IsNotNull(cs_item_sk)] +ReadSchema: struct<cs_item_sk:int,cs_order_number:int,cs_quantity:int,cs_ext_sales_price:decimal(7,2)> + +(73) CometFilter +Input [5]: [cs_item_sk#70, cs_order_number#71, cs_quantity#72, cs_ext_sales_price#73, cs_sold_date_sk#74] +Condition : isnotnull(cs_item_sk#70) + +(74) ColumnarToRow [codegen id : 28] +Input [5]: [cs_item_sk#70, cs_order_number#71, cs_quantity#72, cs_ext_sales_price#73, cs_sold_date_sk#74] + +(75) ReusedExchange [Reuses operator id: 8] +Output [5]: [i_item_sk#76, i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80] + +(76) BroadcastHashJoin [codegen id : 28] +Left keys [1]: [cs_item_sk#70] +Right keys [1]: [i_item_sk#76] +Join type: Inner +Join condition: None + +(77) Project [codegen id : 28] +Output [9]: [cs_item_sk#70, cs_order_number#71, cs_quantity#72, cs_ext_sales_price#73, cs_sold_date_sk#74, i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80] +Input [10]: [cs_item_sk#70, cs_order_number#71, cs_quantity#72, cs_ext_sales_price#73, cs_sold_date_sk#74, i_item_sk#76, i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80] + +(78) ReusedExchange [Reuses operator id: 137] +Output [2]: [d_date_sk#81, d_year#82] + +(79) BroadcastHashJoin [codegen id : 28] +Left keys [1]: [cs_sold_date_sk#74] +Right keys [1]: [d_date_sk#81] +Join type: Inner +Join condition: None + +(80) Project [codegen id : 28] +Output [9]: [cs_item_sk#70, cs_order_number#71, cs_quantity#72, cs_ext_sales_price#73, i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80, d_year#82] +Input [11]: [cs_item_sk#70, cs_order_number#71, cs_quantity#72, cs_ext_sales_price#73, cs_sold_date_sk#74, i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80, d_date_sk#81, d_year#82] + +(81) Exchange +Input [9]: [cs_item_sk#70, cs_order_number#71, cs_quantity#72, cs_ext_sales_price#73, i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80, d_year#82] +Arguments: hashpartitioning(cs_order_number#71, cs_item_sk#70, 5), ENSURE_REQUIREMENTS, [plan_id=11] + +(82) Sort [codegen id : 29] +Input [9]: [cs_item_sk#70, cs_order_number#71, cs_quantity#72, cs_ext_sales_price#73, i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80, d_year#82] +Arguments: [cs_order_number#71 ASC NULLS FIRST, cs_item_sk#70 ASC NULLS FIRST], false, 0 + +(83) ReusedExchange [Reuses operator id: 20] +Output [4]: [cr_item_sk#83, cr_order_number#84, cr_return_quantity#85, cr_return_amount#86] + +(84) Sort [codegen id : 31] +Input [4]: [cr_item_sk#83, cr_order_number#84, cr_return_quantity#85, cr_return_amount#86] +Arguments: [cr_order_number#84 ASC NULLS FIRST, cr_item_sk#83 ASC NULLS FIRST], false, 0 + +(85) SortMergeJoin [codegen id : 32] +Left keys [2]: [cs_order_number#71, cs_item_sk#70] +Right keys [2]: [cr_order_number#84, cr_item_sk#83] +Join type: LeftOuter +Join condition: None + +(86) Project [codegen id : 32] +Output [7]: [d_year#82, i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80, (cs_quantity#72 - coalesce(cr_return_quantity#85, 0)) AS sales_cnt#87, (cs_ext_sales_price#73 - coalesce(cr_return_amount#86, 0.00)) AS sales_amt#88] +Input [13]: [cs_item_sk#70, cs_order_number#71, cs_quantity#72, cs_ext_sales_price#73, i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80, d_year#82, cr_item_sk#83, cr_order_number#84, cr_return_quantity#85, cr_return_amount#86] + +(87) Scan parquet spark_catalog.default.store_sales +Output [5]: [ss_item_sk#89, ss_ticket_number#90, ss_quantity#91, ss_ext_sales_price#92, ss_sold_date_sk#93] +Batched: true +Location: InMemoryFileIndex [] +PartitionFilters: [isnotnull(ss_sold_date_sk#93), dynamicpruningexpression(ss_sold_date_sk#93 IN dynamicpruning#94)] +PushedFilters: [IsNotNull(ss_item_sk)] +ReadSchema: struct<ss_item_sk:int,ss_ticket_number:int,ss_quantity:int,ss_ext_sales_price:decimal(7,2)> + +(88) CometFilter +Input [5]: [ss_item_sk#89, ss_ticket_number#90, ss_quantity#91, ss_ext_sales_price#92, ss_sold_date_sk#93] +Condition : isnotnull(ss_item_sk#89) + +(89) ColumnarToRow [codegen id : 35] +Input [5]: [ss_item_sk#89, ss_ticket_number#90, ss_quantity#91, ss_ext_sales_price#92, ss_sold_date_sk#93] + +(90) ReusedExchange [Reuses operator id: 8] +Output [5]: [i_item_sk#95, i_brand_id#96, i_class_id#97, i_category_id#98, i_manufact_id#99] + +(91) BroadcastHashJoin [codegen id : 35] +Left keys [1]: [ss_item_sk#89] +Right keys [1]: [i_item_sk#95] +Join type: Inner +Join condition: None + +(92) Project [codegen id : 35] +Output [9]: [ss_item_sk#89, ss_ticket_number#90, ss_quantity#91, ss_ext_sales_price#92, ss_sold_date_sk#93, i_brand_id#96, i_class_id#97, i_category_id#98, i_manufact_id#99] +Input [10]: [ss_item_sk#89, ss_ticket_number#90, ss_quantity#91, ss_ext_sales_price#92, ss_sold_date_sk#93, i_item_sk#95, i_brand_id#96, i_class_id#97, i_category_id#98, i_manufact_id#99] + +(93) ReusedExchange [Reuses operator id: 137] +Output [2]: [d_date_sk#100, d_year#101] + +(94) BroadcastHashJoin [codegen id : 35] +Left keys [1]: [ss_sold_date_sk#93] +Right keys [1]: [d_date_sk#100] +Join type: Inner +Join condition: None + +(95) Project [codegen id : 35] +Output [9]: [ss_item_sk#89, ss_ticket_number#90, ss_quantity#91, ss_ext_sales_price#92, i_brand_id#96, i_class_id#97, i_category_id#98, i_manufact_id#99, d_year#101] +Input [11]: [ss_item_sk#89, ss_ticket_number#90, ss_quantity#91, ss_ext_sales_price#92, ss_sold_date_sk#93, i_brand_id#96, i_class_id#97, i_category_id#98, i_manufact_id#99, d_date_sk#100, d_year#101] + +(96) Exchange +Input [9]: [ss_item_sk#89, ss_ticket_number#90, ss_quantity#91, ss_ext_sales_price#92, i_brand_id#96, i_class_id#97, i_category_id#98, i_manufact_id#99, d_year#101] +Arguments: hashpartitioning(ss_ticket_number#90, ss_item_sk#89, 5), ENSURE_REQUIREMENTS, [plan_id=12] + +(97) Sort [codegen id : 36] +Input [9]: [ss_item_sk#89, ss_ticket_number#90, ss_quantity#91, ss_ext_sales_price#92, i_brand_id#96, i_class_id#97, i_category_id#98, i_manufact_id#99, d_year#101] +Arguments: [ss_ticket_number#90 ASC NULLS FIRST, ss_item_sk#89 ASC NULLS FIRST], false, 0 + +(98) ReusedExchange [Reuses operator id: 39] +Output [4]: [sr_item_sk#102, sr_ticket_number#103, sr_return_quantity#104, sr_return_amt#105] + +(99) Sort [codegen id : 38] +Input [4]: [sr_item_sk#102, sr_ticket_number#103, sr_return_quantity#104, sr_return_amt#105] +Arguments: [sr_ticket_number#103 ASC NULLS FIRST, sr_item_sk#102 ASC NULLS FIRST], false, 0 + +(100) SortMergeJoin [codegen id : 39] +Left keys [2]: [ss_ticket_number#90, ss_item_sk#89] +Right keys [2]: [sr_ticket_number#103, sr_item_sk#102] +Join type: LeftOuter +Join condition: None + +(101) Project [codegen id : 39] +Output [7]: [d_year#101, i_brand_id#96, i_class_id#97, i_category_id#98, i_manufact_id#99, (ss_quantity#91 - coalesce(sr_return_quantity#104, 0)) AS sales_cnt#106, (ss_ext_sales_price#92 - coalesce(sr_return_amt#105, 0.00)) AS sales_amt#107] +Input [13]: [ss_item_sk#89, ss_ticket_number#90, ss_quantity#91, ss_ext_sales_price#92, i_brand_id#96, i_class_id#97, i_category_id#98, i_manufact_id#99, d_year#101, sr_item_sk#102, sr_ticket_number#103, sr_return_quantity#104, sr_return_amt#105] + +(102) Scan parquet spark_catalog.default.web_sales +Output [5]: [ws_item_sk#108, ws_order_number#109, ws_quantity#110, ws_ext_sales_price#111, ws_sold_date_sk#112] +Batched: true +Location: InMemoryFileIndex [] +PartitionFilters: [isnotnull(ws_sold_date_sk#112), dynamicpruningexpression(ws_sold_date_sk#112 IN dynamicpruning#113)] +PushedFilters: [IsNotNull(ws_item_sk)] +ReadSchema: struct<ws_item_sk:int,ws_order_number:int,ws_quantity:int,ws_ext_sales_price:decimal(7,2)> + +(103) CometFilter +Input [5]: [ws_item_sk#108, ws_order_number#109, ws_quantity#110, ws_ext_sales_price#111, ws_sold_date_sk#112] +Condition : isnotnull(ws_item_sk#108) + +(104) ColumnarToRow [codegen id : 42] +Input [5]: [ws_item_sk#108, ws_order_number#109, ws_quantity#110, ws_ext_sales_price#111, ws_sold_date_sk#112] + +(105) ReusedExchange [Reuses operator id: 8] +Output [5]: [i_item_sk#114, i_brand_id#115, i_class_id#116, i_category_id#117, i_manufact_id#118] + +(106) BroadcastHashJoin [codegen id : 42] +Left keys [1]: [ws_item_sk#108] +Right keys [1]: [i_item_sk#114] +Join type: Inner +Join condition: None + +(107) Project [codegen id : 42] +Output [9]: [ws_item_sk#108, ws_order_number#109, ws_quantity#110, ws_ext_sales_price#111, ws_sold_date_sk#112, i_brand_id#115, i_class_id#116, i_category_id#117, i_manufact_id#118] +Input [10]: [ws_item_sk#108, ws_order_number#109, ws_quantity#110, ws_ext_sales_price#111, ws_sold_date_sk#112, i_item_sk#114, i_brand_id#115, i_class_id#116, i_category_id#117, i_manufact_id#118] + +(108) ReusedExchange [Reuses operator id: 137] +Output [2]: [d_date_sk#119, d_year#120] + +(109) BroadcastHashJoin [codegen id : 42] +Left keys [1]: [ws_sold_date_sk#112] +Right keys [1]: [d_date_sk#119] +Join type: Inner +Join condition: None + +(110) Project [codegen id : 42] +Output [9]: [ws_item_sk#108, ws_order_number#109, ws_quantity#110, ws_ext_sales_price#111, i_brand_id#115, i_class_id#116, i_category_id#117, i_manufact_id#118, d_year#120] +Input [11]: [ws_item_sk#108, ws_order_number#109, ws_quantity#110, ws_ext_sales_price#111, ws_sold_date_sk#112, i_brand_id#115, i_class_id#116, i_category_id#117, i_manufact_id#118, d_date_sk#119, d_year#120] + +(111) Exchange +Input [9]: [ws_item_sk#108, ws_order_number#109, ws_quantity#110, ws_ext_sales_price#111, i_brand_id#115, i_class_id#116, i_category_id#117, i_manufact_id#118, d_year#120] +Arguments: hashpartitioning(ws_order_number#109, ws_item_sk#108, 5), ENSURE_REQUIREMENTS, [plan_id=13] + +(112) Sort [codegen id : 43] +Input [9]: [ws_item_sk#108, ws_order_number#109, ws_quantity#110, ws_ext_sales_price#111, i_brand_id#115, i_class_id#116, i_category_id#117, i_manufact_id#118, d_year#120] +Arguments: [ws_order_number#109 ASC NULLS FIRST, ws_item_sk#108 ASC NULLS FIRST], false, 0 + +(113) ReusedExchange [Reuses operator id: 58] +Output [4]: [wr_item_sk#121, wr_order_number#122, wr_return_quantity#123, wr_return_amt#124] + +(114) Sort [codegen id : 45] +Input [4]: [wr_item_sk#121, wr_order_number#122, wr_return_quantity#123, wr_return_amt#124] +Arguments: [wr_order_number#122 ASC NULLS FIRST, wr_item_sk#121 ASC NULLS FIRST], false, 0 + +(115) SortMergeJoin [codegen id : 46] +Left keys [2]: [ws_order_number#109, ws_item_sk#108] +Right keys [2]: [wr_order_number#122, wr_item_sk#121] +Join type: LeftOuter +Join condition: None + +(116) Project [codegen id : 46] +Output [7]: [d_year#120, i_brand_id#115, i_class_id#116, i_category_id#117, i_manufact_id#118, (ws_quantity#110 - coalesce(wr_return_quantity#123, 0)) AS sales_cnt#125, (ws_ext_sales_price#111 - coalesce(wr_return_amt#124, 0.00)) AS sales_amt#126] +Input [13]: [ws_item_sk#108, ws_order_number#109, ws_quantity#110, ws_ext_sales_price#111, i_brand_id#115, i_class_id#116, i_category_id#117, i_manufact_id#118, d_year#120, wr_item_sk#121, wr_order_number#122, wr_return_quantity#123, wr_return_amt#124] + +(117) Union + +(118) HashAggregate [codegen id : 47] +Input [7]: [d_year#82, i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80, sales_cnt#87, sales_amt#88] +Keys [7]: [d_year#82, i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80, sales_cnt#87, sales_amt#88] +Functions: [] +Aggregate Attributes: [] +Results [7]: [d_year#82, i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80, sales_cnt#87, sales_amt#88] + +(119) Exchange +Input [7]: [d_year#82, i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80, sales_cnt#87, sales_amt#88] +Arguments: hashpartitioning(d_year#82, i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80, sales_cnt#87, sales_amt#88, 5), ENSURE_REQUIREMENTS, [plan_id=14] + +(120) HashAggregate [codegen id : 48] +Input [7]: [d_year#82, i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80, sales_cnt#87, sales_amt#88] +Keys [7]: [d_year#82, i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80, sales_cnt#87, sales_amt#88] +Functions: [] +Aggregate Attributes: [] +Results [7]: [d_year#82, i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80, sales_cnt#87, sales_amt#88] + +(121) HashAggregate [codegen id : 48] +Input [7]: [d_year#82, i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80, sales_cnt#87, sales_amt#88] +Keys [5]: [d_year#82, i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80] +Functions [2]: [partial_sum(sales_cnt#87), partial_sum(UnscaledValue(sales_amt#88))] +Aggregate Attributes [2]: [sum#127, sum#128] +Results [7]: [d_year#82, i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80, sum#129, sum#130] + +(122) Exchange +Input [7]: [d_year#82, i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80, sum#129, sum#130] +Arguments: hashpartitioning(d_year#82, i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80, 5), ENSURE_REQUIREMENTS, [plan_id=15] + +(123) HashAggregate [codegen id : 49] +Input [7]: [d_year#82, i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80, sum#129, sum#130] +Keys [5]: [d_year#82, i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80] +Functions [2]: [sum(sales_cnt#87), sum(UnscaledValue(sales_amt#88))] +Aggregate Attributes [2]: [sum(sales_cnt#87)#66, sum(UnscaledValue(sales_amt#88))#67] +Results [7]: [d_year#82, i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80, sum(sales_cnt#87)#66 AS sales_cnt#131, MakeDecimal(sum(UnscaledValue(sales_amt#88))#67,18,2) AS sales_amt#132] + +(124) Filter [codegen id : 49] +Input [7]: [d_year#82, i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80, sales_cnt#131, sales_amt#132] +Condition : isnotnull(sales_cnt#131) + +(125) Exchange +Input [7]: [d_year#82, i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80, sales_cnt#131, sales_amt#132] +Arguments: hashpartitioning(i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80, 5), ENSURE_REQUIREMENTS, [plan_id=16] + +(126) Sort [codegen id : 50] +Input [7]: [d_year#82, i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80, sales_cnt#131, sales_amt#132] +Arguments: [i_brand_id#77 ASC NULLS FIRST, i_class_id#78 ASC NULLS FIRST, i_category_id#79 ASC NULLS FIRST, i_manufact_id#80 ASC NULLS FIRST], false, 0 + +(127) SortMergeJoin [codegen id : 51] +Left keys [4]: [i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] +Right keys [4]: [i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80] +Join type: Inner +Join condition: ((cast(sales_cnt#68 as decimal(17,2)) / cast(sales_cnt#131 as decimal(17,2))) < 0.90000000000000000000) + +(128) Project [codegen id : 51] +Output [10]: [d_year#82 AS prev_year#133, d_year#14 AS year#134, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#131 AS prev_yr_cnt#135, sales_cnt#68 AS curr_yr_cnt#136, (sales_cnt#68 - sales_cnt#131) AS sales_cnt_diff#137, (sales_amt#69 - sales_amt#132) AS sales_amt_diff#138] +Input [14]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#68, sales_amt#69, d_year#82, i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80, sales_cnt#131, sales_amt#132] + +(129) TakeOrderedAndProject +Input [10]: [prev_year#133, year#134, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, prev_yr_cnt#135, curr_yr_cnt#136, sales_cnt_diff#137, sales_amt_diff#138] +Arguments: 100, [sales_cnt_diff#137 ASC NULLS FIRST], [prev_year#133, year#134, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, prev_yr_cnt#135, curr_yr_cnt#136, sales_cnt_diff#137, sales_amt_diff#138] + +===== Subqueries ===== + +Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#5 IN dynamicpruning#6 +BroadcastExchange (133) ++- * ColumnarToRow (132) + +- CometFilter (131) + +- CometScan parquet spark_catalog.default.date_dim (130) + + +(130) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#13, d_year#14] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2002), IsNotNull(d_date_sk)] +ReadSchema: struct<d_date_sk:int,d_year:int> + +(131) CometFilter +Input [2]: [d_date_sk#13, d_year#14] +Condition : ((isnotnull(d_year#14) AND (d_year#14 = 2002)) AND isnotnull(d_date_sk#13)) + +(132) ColumnarToRow [codegen id : 1] +Input [2]: [d_date_sk#13, d_year#14] + +(133) BroadcastExchange +Input [2]: [d_date_sk#13, d_year#14] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=17] + +Subquery:2 Hosting operator id = 24 Hosting Expression = ss_sold_date_sk#26 IN dynamicpruning#6 + +Subquery:3 Hosting operator id = 43 Hosting Expression = ws_sold_date_sk#46 IN dynamicpruning#6 + +Subquery:4 Hosting operator id = 72 Hosting Expression = cs_sold_date_sk#74 IN dynamicpruning#75 +BroadcastExchange (137) ++- * ColumnarToRow (136) + +- CometFilter (135) + +- CometScan parquet spark_catalog.default.date_dim (134) + + +(134) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#81, d_year#82] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), IsNotNull(d_date_sk)] +ReadSchema: struct<d_date_sk:int,d_year:int> + +(135) CometFilter +Input [2]: [d_date_sk#81, d_year#82] +Condition : ((isnotnull(d_year#82) AND (d_year#82 = 2001)) AND isnotnull(d_date_sk#81)) + +(136) ColumnarToRow [codegen id : 1] +Input [2]: [d_date_sk#81, d_year#82] + +(137) BroadcastExchange +Input [2]: [d_date_sk#81, d_year#82] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=18] + +Subquery:5 Hosting operator id = 87 Hosting Expression = ss_sold_date_sk#93 IN dynamicpruning#75 + +Subquery:6 Hosting operator id = 102 Hosting Expression = ws_sold_date_sk#112 IN dynamicpruning#75 + + 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 new file mode 100644 index 000000000..44bcabcdb --- /dev/null +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q75/simplified.txt @@ -0,0 +1,240 @@ +TakeOrderedAndProject [sales_cnt_diff,prev_year,year,i_brand_id,i_class_id,i_category_id,i_manufact_id,prev_yr_cnt,curr_yr_cnt,sales_amt_diff] + WholeStageCodegen (51) + Project [d_year,d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_cnt,sales_amt,sales_amt] + SortMergeJoin [i_brand_id,i_class_id,i_category_id,i_manufact_id,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_cnt] + InputAdapter + WholeStageCodegen (25) + Sort [i_brand_id,i_class_id,i_category_id,i_manufact_id] + InputAdapter + Exchange [i_brand_id,i_class_id,i_category_id,i_manufact_id] #1 + WholeStageCodegen (24) + Filter [sales_cnt] + HashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sum,sum] [sum(sales_cnt),sum(UnscaledValue(sales_amt)),sales_cnt,sales_amt,sum,sum] + InputAdapter + Exchange [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id] #2 + WholeStageCodegen (23) + HashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] [sum,sum,sum,sum] + HashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] + InputAdapter + Exchange [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] #3 + WholeStageCodegen (22) + HashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] + InputAdapter + Union + WholeStageCodegen (7) + Project [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,cs_quantity,cr_return_quantity,cs_ext_sales_price,cr_return_amount] + SortMergeJoin [cs_order_number,cs_item_sk,cr_order_number,cr_item_sk] + InputAdapter + WholeStageCodegen (4) + Sort [cs_order_number,cs_item_sk] + InputAdapter + Exchange [cs_order_number,cs_item_sk] #4 + WholeStageCodegen (3) + Project [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + Project [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] + BroadcastHashJoin [cs_item_sk,i_item_sk] + ColumnarToRow + InputAdapter + CometFilter [cs_item_sk] + CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #5 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometFilter [d_year,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + InputAdapter + BroadcastExchange #6 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] + CometFilter [i_category,i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] + CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id,i_category,i_manufact_id] + InputAdapter + ReusedExchange [d_date_sk,d_year] #5 + InputAdapter + WholeStageCodegen (6) + Sort [cr_order_number,cr_item_sk] + InputAdapter + Exchange [cr_order_number,cr_item_sk] #7 + WholeStageCodegen (5) + ColumnarToRow + InputAdapter + CometProject [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] + CometFilter [cr_order_number,cr_item_sk] + CometScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount,cr_returned_date_sk] + WholeStageCodegen (14) + Project [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,ss_quantity,sr_return_quantity,ss_ext_sales_price,sr_return_amt] + SortMergeJoin [ss_ticket_number,ss_item_sk,sr_ticket_number,sr_item_sk] + InputAdapter + WholeStageCodegen (11) + Sort [ss_ticket_number,ss_item_sk] + InputAdapter + Exchange [ss_ticket_number,ss_item_sk] #8 + WholeStageCodegen (10) + Project [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] + BroadcastHashJoin [ss_item_sk,i_item_sk] + ColumnarToRow + InputAdapter + CometFilter [ss_item_sk] + CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + InputAdapter + ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 + InputAdapter + ReusedExchange [d_date_sk,d_year] #5 + InputAdapter + WholeStageCodegen (13) + Sort [sr_ticket_number,sr_item_sk] + InputAdapter + Exchange [sr_ticket_number,sr_item_sk] #9 + WholeStageCodegen (12) + ColumnarToRow + InputAdapter + CometProject [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] + CometFilter [sr_ticket_number,sr_item_sk] + CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt,sr_returned_date_sk] + WholeStageCodegen (21) + Project [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,ws_quantity,wr_return_quantity,ws_ext_sales_price,wr_return_amt] + SortMergeJoin [ws_order_number,ws_item_sk,wr_order_number,wr_item_sk] + InputAdapter + WholeStageCodegen (18) + Sort [ws_order_number,ws_item_sk] + InputAdapter + Exchange [ws_order_number,ws_item_sk] #10 + WholeStageCodegen (17) + Project [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + Project [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] + BroadcastHashJoin [ws_item_sk,i_item_sk] + ColumnarToRow + InputAdapter + CometFilter [ws_item_sk] + CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + InputAdapter + ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 + InputAdapter + ReusedExchange [d_date_sk,d_year] #5 + InputAdapter + WholeStageCodegen (20) + Sort [wr_order_number,wr_item_sk] + InputAdapter + Exchange [wr_order_number,wr_item_sk] #11 + WholeStageCodegen (19) + ColumnarToRow + InputAdapter + CometProject [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] + CometFilter [wr_order_number,wr_item_sk] + CometScan parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk] + InputAdapter + WholeStageCodegen (50) + Sort [i_brand_id,i_class_id,i_category_id,i_manufact_id] + InputAdapter + Exchange [i_brand_id,i_class_id,i_category_id,i_manufact_id] #12 + WholeStageCodegen (49) + Filter [sales_cnt] + HashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sum,sum] [sum(sales_cnt),sum(UnscaledValue(sales_amt)),sales_cnt,sales_amt,sum,sum] + InputAdapter + Exchange [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id] #13 + WholeStageCodegen (48) + HashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] [sum,sum,sum,sum] + HashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] + InputAdapter + Exchange [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] #14 + WholeStageCodegen (47) + HashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] + InputAdapter + Union + WholeStageCodegen (32) + Project [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,cs_quantity,cr_return_quantity,cs_ext_sales_price,cr_return_amount] + SortMergeJoin [cs_order_number,cs_item_sk,cr_order_number,cr_item_sk] + InputAdapter + WholeStageCodegen (29) + Sort [cs_order_number,cs_item_sk] + InputAdapter + Exchange [cs_order_number,cs_item_sk] #15 + WholeStageCodegen (28) + Project [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + Project [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] + BroadcastHashJoin [cs_item_sk,i_item_sk] + ColumnarToRow + InputAdapter + CometFilter [cs_item_sk] + CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk] + SubqueryBroadcast [d_date_sk] #2 + BroadcastExchange #16 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometFilter [d_year,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + InputAdapter + ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 + InputAdapter + ReusedExchange [d_date_sk,d_year] #16 + InputAdapter + WholeStageCodegen (31) + Sort [cr_order_number,cr_item_sk] + InputAdapter + ReusedExchange [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] #7 + WholeStageCodegen (39) + Project [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,ss_quantity,sr_return_quantity,ss_ext_sales_price,sr_return_amt] + SortMergeJoin [ss_ticket_number,ss_item_sk,sr_ticket_number,sr_item_sk] + InputAdapter + WholeStageCodegen (36) + Sort [ss_ticket_number,ss_item_sk] + InputAdapter + Exchange [ss_ticket_number,ss_item_sk] #17 + WholeStageCodegen (35) + Project [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] + BroadcastHashJoin [ss_item_sk,i_item_sk] + ColumnarToRow + InputAdapter + CometFilter [ss_item_sk] + CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk] + ReusedSubquery [d_date_sk] #2 + InputAdapter + ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 + InputAdapter + ReusedExchange [d_date_sk,d_year] #16 + InputAdapter + WholeStageCodegen (38) + Sort [sr_ticket_number,sr_item_sk] + InputAdapter + ReusedExchange [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] #9 + WholeStageCodegen (46) + Project [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,ws_quantity,wr_return_quantity,ws_ext_sales_price,wr_return_amt] + SortMergeJoin [ws_order_number,ws_item_sk,wr_order_number,wr_item_sk] + InputAdapter + WholeStageCodegen (43) + Sort [ws_order_number,ws_item_sk] + InputAdapter + Exchange [ws_order_number,ws_item_sk] #18 + WholeStageCodegen (42) + Project [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + Project [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] + BroadcastHashJoin [ws_item_sk,i_item_sk] + ColumnarToRow + InputAdapter + CometFilter [ws_item_sk] + CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #2 + InputAdapter + ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 + InputAdapter + ReusedExchange [d_date_sk,d_year] #16 + InputAdapter + WholeStageCodegen (45) + Sort [wr_order_number,wr_item_sk] + InputAdapter + 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/q76/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q76/explain.txt new file mode 100644 index 000000000..f3b80d8bd --- /dev/null +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q76/explain.txt @@ -0,0 +1,218 @@ +== Physical Plan == +TakeOrderedAndProject (38) ++- * HashAggregate (37) + +- Exchange (36) + +- * HashAggregate (35) + +- Union (34) + :- * Project (15) + : +- * BroadcastHashJoin Inner BuildRight (14) + : :- * Project (9) + : : +- * BroadcastHashJoin Inner BuildRight (8) + : : :- * ColumnarToRow (3) + : : : +- CometFilter (2) + : : : +- CometScan parquet spark_catalog.default.store_sales (1) + : : +- BroadcastExchange (7) + : : +- * ColumnarToRow (6) + : : +- CometFilter (5) + : : +- CometScan parquet spark_catalog.default.item (4) + : +- BroadcastExchange (13) + : +- * ColumnarToRow (12) + : +- CometFilter (11) + : +- CometScan parquet spark_catalog.default.date_dim (10) + :- * Project (24) + : +- * BroadcastHashJoin Inner BuildRight (23) + : :- * Project (21) + : : +- * BroadcastHashJoin Inner BuildRight (20) + : : :- * ColumnarToRow (18) + : : : +- CometFilter (17) + : : : +- CometScan parquet spark_catalog.default.web_sales (16) + : : +- ReusedExchange (19) + : +- ReusedExchange (22) + +- * Project (33) + +- * BroadcastHashJoin Inner BuildRight (32) + :- * Project (30) + : +- * BroadcastHashJoin Inner BuildRight (29) + : :- * ColumnarToRow (27) + : : +- CometFilter (26) + : : +- CometScan parquet spark_catalog.default.catalog_sales (25) + : +- ReusedExchange (28) + +- ReusedExchange (31) + + +(1) Scan parquet spark_catalog.default.store_sales +Output [4]: [ss_item_sk#1, ss_store_sk#2, ss_ext_sales_price#3, ss_sold_date_sk#4] +Batched: true +Location: InMemoryFileIndex [] +PartitionFilters: [isnotnull(ss_sold_date_sk#4)] +PushedFilters: [IsNull(ss_store_sk), IsNotNull(ss_item_sk)] +ReadSchema: struct<ss_item_sk:int,ss_store_sk:int,ss_ext_sales_price:decimal(7,2)> + +(2) CometFilter +Input [4]: [ss_item_sk#1, ss_store_sk#2, ss_ext_sales_price#3, ss_sold_date_sk#4] +Condition : (isnull(ss_store_sk#2) AND isnotnull(ss_item_sk#1)) + +(3) ColumnarToRow [codegen id : 3] +Input [4]: [ss_item_sk#1, ss_store_sk#2, ss_ext_sales_price#3, ss_sold_date_sk#4] + +(4) Scan parquet spark_catalog.default.item +Output [2]: [i_item_sk#5, i_category#6] +Batched: true +Location [not included in comparison]/{warehouse_dir}/item] +PushedFilters: [IsNotNull(i_item_sk)] +ReadSchema: struct<i_item_sk:int,i_category:string> + +(5) CometFilter +Input [2]: [i_item_sk#5, i_category#6] +Condition : isnotnull(i_item_sk#5) + +(6) ColumnarToRow [codegen id : 1] +Input [2]: [i_item_sk#5, i_category#6] + +(7) BroadcastExchange +Input [2]: [i_item_sk#5, i_category#6] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=1] + +(8) BroadcastHashJoin [codegen id : 3] +Left keys [1]: [ss_item_sk#1] +Right keys [1]: [i_item_sk#5] +Join type: Inner +Join condition: None + +(9) Project [codegen id : 3] +Output [4]: [ss_store_sk#2, ss_ext_sales_price#3, ss_sold_date_sk#4, i_category#6] +Input [6]: [ss_item_sk#1, ss_store_sk#2, ss_ext_sales_price#3, ss_sold_date_sk#4, i_item_sk#5, i_category#6] + +(10) Scan parquet spark_catalog.default.date_dim +Output [3]: [d_date_sk#7, d_year#8, d_qoy#9] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_date_sk)] +ReadSchema: struct<d_date_sk:int,d_year:int,d_qoy:int> + +(11) CometFilter +Input [3]: [d_date_sk#7, d_year#8, d_qoy#9] +Condition : isnotnull(d_date_sk#7) + +(12) ColumnarToRow [codegen id : 2] +Input [3]: [d_date_sk#7, d_year#8, d_qoy#9] + +(13) BroadcastExchange +Input [3]: [d_date_sk#7, d_year#8, d_qoy#9] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=2] + +(14) BroadcastHashJoin [codegen id : 3] +Left keys [1]: [ss_sold_date_sk#4] +Right keys [1]: [d_date_sk#7] +Join type: Inner +Join condition: None + +(15) Project [codegen id : 3] +Output [6]: [store AS channel#10, ss_store_sk#2 AS col_name#11, d_year#8, d_qoy#9, i_category#6, ss_ext_sales_price#3 AS ext_sales_price#12] +Input [7]: [ss_store_sk#2, ss_ext_sales_price#3, ss_sold_date_sk#4, i_category#6, d_date_sk#7, d_year#8, d_qoy#9] + +(16) Scan parquet spark_catalog.default.web_sales +Output [4]: [ws_item_sk#13, ws_ship_customer_sk#14, ws_ext_sales_price#15, ws_sold_date_sk#16] +Batched: true +Location: InMemoryFileIndex [] +PartitionFilters: [isnotnull(ws_sold_date_sk#16)] +PushedFilters: [IsNull(ws_ship_customer_sk), IsNotNull(ws_item_sk)] +ReadSchema: struct<ws_item_sk:int,ws_ship_customer_sk:int,ws_ext_sales_price:decimal(7,2)> + +(17) CometFilter +Input [4]: [ws_item_sk#13, ws_ship_customer_sk#14, ws_ext_sales_price#15, ws_sold_date_sk#16] +Condition : (isnull(ws_ship_customer_sk#14) AND isnotnull(ws_item_sk#13)) + +(18) ColumnarToRow [codegen id : 6] +Input [4]: [ws_item_sk#13, ws_ship_customer_sk#14, ws_ext_sales_price#15, ws_sold_date_sk#16] + +(19) ReusedExchange [Reuses operator id: 7] +Output [2]: [i_item_sk#17, i_category#18] + +(20) BroadcastHashJoin [codegen id : 6] +Left keys [1]: [ws_item_sk#13] +Right keys [1]: [i_item_sk#17] +Join type: Inner +Join condition: None + +(21) Project [codegen id : 6] +Output [4]: [ws_ship_customer_sk#14, ws_ext_sales_price#15, ws_sold_date_sk#16, i_category#18] +Input [6]: [ws_item_sk#13, ws_ship_customer_sk#14, ws_ext_sales_price#15, ws_sold_date_sk#16, i_item_sk#17, i_category#18] + +(22) ReusedExchange [Reuses operator id: 13] +Output [3]: [d_date_sk#19, d_year#20, d_qoy#21] + +(23) BroadcastHashJoin [codegen id : 6] +Left keys [1]: [ws_sold_date_sk#16] +Right keys [1]: [d_date_sk#19] +Join type: Inner +Join condition: None + +(24) Project [codegen id : 6] +Output [6]: [web AS channel#22, ws_ship_customer_sk#14 AS col_name#23, d_year#20, d_qoy#21, i_category#18, ws_ext_sales_price#15 AS ext_sales_price#24] +Input [7]: [ws_ship_customer_sk#14, ws_ext_sales_price#15, ws_sold_date_sk#16, i_category#18, d_date_sk#19, d_year#20, d_qoy#21] + +(25) Scan parquet spark_catalog.default.catalog_sales +Output [4]: [cs_ship_addr_sk#25, cs_item_sk#26, cs_ext_sales_price#27, cs_sold_date_sk#28] +Batched: true +Location: InMemoryFileIndex [] +PartitionFilters: [isnotnull(cs_sold_date_sk#28)] +PushedFilters: [IsNull(cs_ship_addr_sk), IsNotNull(cs_item_sk)] +ReadSchema: struct<cs_ship_addr_sk:int,cs_item_sk:int,cs_ext_sales_price:decimal(7,2)> + +(26) CometFilter +Input [4]: [cs_ship_addr_sk#25, cs_item_sk#26, cs_ext_sales_price#27, cs_sold_date_sk#28] +Condition : (isnull(cs_ship_addr_sk#25) AND isnotnull(cs_item_sk#26)) + +(27) ColumnarToRow [codegen id : 9] +Input [4]: [cs_ship_addr_sk#25, cs_item_sk#26, cs_ext_sales_price#27, cs_sold_date_sk#28] + +(28) ReusedExchange [Reuses operator id: 7] +Output [2]: [i_item_sk#29, i_category#30] + +(29) BroadcastHashJoin [codegen id : 9] +Left keys [1]: [cs_item_sk#26] +Right keys [1]: [i_item_sk#29] +Join type: Inner +Join condition: None + +(30) Project [codegen id : 9] +Output [4]: [cs_ship_addr_sk#25, cs_ext_sales_price#27, cs_sold_date_sk#28, i_category#30] +Input [6]: [cs_ship_addr_sk#25, cs_item_sk#26, cs_ext_sales_price#27, cs_sold_date_sk#28, i_item_sk#29, i_category#30] + +(31) ReusedExchange [Reuses operator id: 13] +Output [3]: [d_date_sk#31, d_year#32, d_qoy#33] + +(32) BroadcastHashJoin [codegen id : 9] +Left keys [1]: [cs_sold_date_sk#28] +Right keys [1]: [d_date_sk#31] +Join type: Inner +Join condition: None + +(33) Project [codegen id : 9] +Output [6]: [catalog AS channel#34, cs_ship_addr_sk#25 AS col_name#35, d_year#32, d_qoy#33, i_category#30, cs_ext_sales_price#27 AS ext_sales_price#36] +Input [7]: [cs_ship_addr_sk#25, cs_ext_sales_price#27, cs_sold_date_sk#28, i_category#30, d_date_sk#31, d_year#32, d_qoy#33] + +(34) Union + +(35) HashAggregate [codegen id : 10] +Input [6]: [channel#10, col_name#11, d_year#8, d_qoy#9, i_category#6, ext_sales_price#12] +Keys [5]: [channel#10, col_name#11, d_year#8, d_qoy#9, i_category#6] +Functions [2]: [partial_count(1), partial_sum(UnscaledValue(ext_sales_price#12))] +Aggregate Attributes [2]: [count#37, sum#38] +Results [7]: [channel#10, col_name#11, d_year#8, d_qoy#9, i_category#6, count#39, sum#40] + +(36) Exchange +Input [7]: [channel#10, col_name#11, d_year#8, d_qoy#9, i_category#6, count#39, sum#40] +Arguments: hashpartitioning(channel#10, col_name#11, d_year#8, d_qoy#9, i_category#6, 5), ENSURE_REQUIREMENTS, [plan_id=3] + +(37) HashAggregate [codegen id : 11] +Input [7]: [channel#10, col_name#11, d_year#8, d_qoy#9, i_category#6, count#39, sum#40] +Keys [5]: [channel#10, col_name#11, d_year#8, d_qoy#9, i_category#6] +Functions [2]: [count(1), sum(UnscaledValue(ext_sales_price#12))] +Aggregate Attributes [2]: [count(1)#41, sum(UnscaledValue(ext_sales_price#12))#42] +Results [7]: [channel#10, col_name#11, d_year#8, d_qoy#9, i_category#6, count(1)#41 AS sales_cnt#43, MakeDecimal(sum(UnscaledValue(ext_sales_price#12))#42,17,2) AS sales_amt#44] + +(38) TakeOrderedAndProject +Input [7]: [channel#10, col_name#11, d_year#8, d_qoy#9, i_category#6, sales_cnt#43, sales_amt#44] +Arguments: 100, [channel#10 ASC NULLS FIRST, col_name#11 ASC NULLS FIRST, d_year#8 ASC NULLS FIRST, d_qoy#9 ASC NULLS FIRST, i_category#6 ASC NULLS FIRST], [channel#10, col_name#11, d_year#8, d_qoy#9, i_category#6, sales_cnt#43, sales_amt#44] + diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q76/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q76/simplified.txt new file mode 100644 index 000000000..73e6b09af --- /dev/null +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q76/simplified.txt @@ -0,0 +1,58 @@ +TakeOrderedAndProject [channel,col_name,d_year,d_qoy,i_category,sales_cnt,sales_amt] + WholeStageCodegen (11) + HashAggregate [channel,col_name,d_year,d_qoy,i_category,count,sum] [count(1),sum(UnscaledValue(ext_sales_price)),sales_cnt,sales_amt,count,sum] + InputAdapter + Exchange [channel,col_name,d_year,d_qoy,i_category] #1 + WholeStageCodegen (10) + HashAggregate [channel,col_name,d_year,d_qoy,i_category,ext_sales_price] [count,sum,count,sum] + InputAdapter + Union + WholeStageCodegen (3) + Project [ss_store_sk,d_year,d_qoy,i_category,ss_ext_sales_price] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [ss_store_sk,ss_ext_sales_price,ss_sold_date_sk,i_category] + BroadcastHashJoin [ss_item_sk,i_item_sk] + ColumnarToRow + InputAdapter + CometFilter [ss_store_sk,ss_item_sk] + CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_sold_date_sk] + InputAdapter + BroadcastExchange #2 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometFilter [i_item_sk] + CometScan parquet spark_catalog.default.item [i_item_sk,i_category] + InputAdapter + BroadcastExchange #3 + WholeStageCodegen (2) + ColumnarToRow + InputAdapter + CometFilter [d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] + WholeStageCodegen (6) + Project [ws_ship_customer_sk,d_year,d_qoy,i_category,ws_ext_sales_price] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + Project [ws_ship_customer_sk,ws_ext_sales_price,ws_sold_date_sk,i_category] + BroadcastHashJoin [ws_item_sk,i_item_sk] + ColumnarToRow + InputAdapter + CometFilter [ws_ship_customer_sk,ws_item_sk] + CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_ship_customer_sk,ws_ext_sales_price,ws_sold_date_sk] + InputAdapter + ReusedExchange [i_item_sk,i_category] #2 + InputAdapter + ReusedExchange [d_date_sk,d_year,d_qoy] #3 + WholeStageCodegen (9) + Project [cs_ship_addr_sk,d_year,d_qoy,i_category,cs_ext_sales_price] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + Project [cs_ship_addr_sk,cs_ext_sales_price,cs_sold_date_sk,i_category] + BroadcastHashJoin [cs_item_sk,i_item_sk] + ColumnarToRow + InputAdapter + CometFilter [cs_ship_addr_sk,cs_item_sk] + CometScan parquet spark_catalog.default.catalog_sales [cs_ship_addr_sk,cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] + InputAdapter + ReusedExchange [i_item_sk,i_category] #2 + InputAdapter + ReusedExchange [d_date_sk,d_year,d_qoy] #3 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 new file mode 100644 index 000000000..bbfa6a4c4 --- /dev/null +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q77/explain.txt @@ -0,0 +1,547 @@ +== Physical Plan == +TakeOrderedAndProject (85) ++- * HashAggregate (84) + +- Exchange (83) + +- * HashAggregate (82) + +- * Expand (81) + +- Union (80) + :- * Project (30) + : +- * BroadcastHashJoin LeftOuter BuildRight (29) + : :- * HashAggregate (15) + : : +- Exchange (14) + : : +- * HashAggregate (13) + : : +- * Project (12) + : : +- * BroadcastHashJoin Inner BuildRight (11) + : : :- * Project (6) + : : : +- * BroadcastHashJoin Inner BuildRight (5) + : : : :- * ColumnarToRow (3) + : : : : +- CometFilter (2) + : : : : +- CometScan parquet spark_catalog.default.store_sales (1) + : : : +- ReusedExchange (4) + : : +- BroadcastExchange (10) + : : +- * ColumnarToRow (9) + : : +- CometFilter (8) + : : +- CometScan parquet spark_catalog.default.store (7) + : +- BroadcastExchange (28) + : +- * HashAggregate (27) + : +- Exchange (26) + : +- * HashAggregate (25) + : +- * Project (24) + : +- * BroadcastHashJoin Inner BuildRight (23) + : :- * Project (21) + : : +- * BroadcastHashJoin Inner BuildRight (20) + : : :- * ColumnarToRow (18) + : : : +- CometFilter (17) + : : : +- CometScan parquet spark_catalog.default.store_returns (16) + : : +- ReusedExchange (19) + : +- ReusedExchange (22) + :- * Project (49) + : +- * BroadcastNestedLoopJoin Inner BuildLeft (48) + : :- BroadcastExchange (39) + : : +- * HashAggregate (38) + : : +- Exchange (37) + : : +- * HashAggregate (36) + : : +- * Project (35) + : : +- * BroadcastHashJoin Inner BuildRight (34) + : : :- * ColumnarToRow (32) + : : : +- CometScan parquet spark_catalog.default.catalog_sales (31) + : : +- ReusedExchange (33) + : +- * HashAggregate (47) + : +- Exchange (46) + : +- * HashAggregate (45) + : +- * Project (44) + : +- * BroadcastHashJoin Inner BuildRight (43) + : :- * ColumnarToRow (41) + : : +- CometScan parquet spark_catalog.default.catalog_returns (40) + : +- ReusedExchange (42) + +- * Project (79) + +- * BroadcastHashJoin LeftOuter BuildRight (78) + :- * HashAggregate (64) + : +- Exchange (63) + : +- * HashAggregate (62) + : +- * Project (61) + : +- * BroadcastHashJoin Inner BuildRight (60) + : :- * Project (55) + : : +- * BroadcastHashJoin Inner BuildRight (54) + : : :- * ColumnarToRow (52) + : : : +- CometFilter (51) + : : : +- CometScan parquet spark_catalog.default.web_sales (50) + : : +- ReusedExchange (53) + : +- BroadcastExchange (59) + : +- * ColumnarToRow (58) + : +- CometFilter (57) + : +- CometScan parquet spark_catalog.default.web_page (56) + +- BroadcastExchange (77) + +- * HashAggregate (76) + +- Exchange (75) + +- * HashAggregate (74) + +- * Project (73) + +- * BroadcastHashJoin Inner BuildRight (72) + :- * Project (70) + : +- * BroadcastHashJoin Inner BuildRight (69) + : :- * ColumnarToRow (67) + : : +- CometFilter (66) + : : +- CometScan parquet spark_catalog.default.web_returns (65) + : +- ReusedExchange (68) + +- ReusedExchange (71) + + +(1) Scan parquet spark_catalog.default.store_sales +Output [4]: [ss_store_sk#1, ss_ext_sales_price#2, ss_net_profit#3, ss_sold_date_sk#4] +Batched: true +Location: InMemoryFileIndex [] +PartitionFilters: [isnotnull(ss_sold_date_sk#4), dynamicpruningexpression(ss_sold_date_sk#4 IN dynamicpruning#5)] +PushedFilters: [IsNotNull(ss_store_sk)] +ReadSchema: struct<ss_store_sk:int,ss_ext_sales_price:decimal(7,2),ss_net_profit:decimal(7,2)> + +(2) CometFilter +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) + +(3) ColumnarToRow [codegen id : 3] +Input [4]: [ss_store_sk#1, ss_ext_sales_price#2, ss_net_profit#3, ss_sold_date_sk#4] + +(4) ReusedExchange [Reuses operator id: 90] +Output [1]: [d_date_sk#6] + +(5) BroadcastHashJoin [codegen id : 3] +Left keys [1]: [ss_sold_date_sk#4] +Right keys [1]: [d_date_sk#6] +Join type: Inner +Join condition: None + +(6) Project [codegen id : 3] +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] + +(7) Scan parquet spark_catalog.default.store +Output [1]: [s_store_sk#7] +Batched: true +Location [not included in comparison]/{warehouse_dir}/store] +PushedFilters: [IsNotNull(s_store_sk)] +ReadSchema: struct<s_store_sk:int> + +(8) CometFilter +Input [1]: [s_store_sk#7] +Condition : isnotnull(s_store_sk#7) + +(9) ColumnarToRow [codegen id : 2] +Input [1]: [s_store_sk#7] + +(10) BroadcastExchange +Input [1]: [s_store_sk#7] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=1] + +(11) BroadcastHashJoin [codegen id : 3] +Left keys [1]: [ss_store_sk#1] +Right keys [1]: [s_store_sk#7] +Join type: Inner +Join condition: None + +(12) Project [codegen id : 3] +Output [3]: [ss_ext_sales_price#2, ss_net_profit#3, s_store_sk#7] +Input [4]: [ss_store_sk#1, ss_ext_sales_price#2, ss_net_profit#3, s_store_sk#7] + +(13) HashAggregate [codegen id : 3] +Input [3]: [ss_ext_sales_price#2, ss_net_profit#3, s_store_sk#7] +Keys [1]: [s_store_sk#7] +Functions [2]: [partial_sum(UnscaledValue(ss_ext_sales_price#2)), partial_sum(UnscaledValue(ss_net_profit#3))] +Aggregate Attributes [2]: [sum#8, sum#9] +Results [3]: [s_store_sk#7, sum#10, sum#11] + +(14) Exchange +Input [3]: [s_store_sk#7, sum#10, sum#11] +Arguments: hashpartitioning(s_store_sk#7, 5), ENSURE_REQUIREMENTS, [plan_id=2] + +(15) HashAggregate [codegen id : 8] +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))] +Aggregate Attributes [2]: [sum(UnscaledValue(ss_ext_sales_price#2))#12, sum(UnscaledValue(ss_net_profit#3))#13] +Results [3]: [s_store_sk#7, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#2))#12,17,2) AS sales#14, MakeDecimal(sum(UnscaledValue(ss_net_profit#3))#13,17,2) AS profit#15] + +(16) Scan parquet spark_catalog.default.store_returns +Output [4]: [sr_store_sk#16, sr_return_amt#17, sr_net_loss#18, sr_returned_date_sk#19] +Batched: true +Location: InMemoryFileIndex [] +PartitionFilters: [isnotnull(sr_returned_date_sk#19), dynamicpruningexpression(sr_returned_date_sk#19 IN dynamicpruning#20)] +PushedFilters: [IsNotNull(sr_store_sk)] +ReadSchema: struct<sr_store_sk:int,sr_return_amt:decimal(7,2),sr_net_loss:decimal(7,2)> + +(17) CometFilter +Input [4]: [sr_store_sk#16, sr_return_amt#17, sr_net_loss#18, sr_returned_date_sk#19] +Condition : isnotnull(sr_store_sk#16) + +(18) ColumnarToRow [codegen id : 6] +Input [4]: [sr_store_sk#16, sr_return_amt#17, sr_net_loss#18, sr_returned_date_sk#19] + +(19) ReusedExchange [Reuses operator id: 90] +Output [1]: [d_date_sk#21] + +(20) BroadcastHashJoin [codegen id : 6] +Left keys [1]: [sr_returned_date_sk#19] +Right keys [1]: [d_date_sk#21] +Join type: Inner +Join condition: None + +(21) Project [codegen id : 6] +Output [3]: [sr_store_sk#16, sr_return_amt#17, sr_net_loss#18] +Input [5]: [sr_store_sk#16, sr_return_amt#17, sr_net_loss#18, sr_returned_date_sk#19, d_date_sk#21] + +(22) ReusedExchange [Reuses operator id: 10] +Output [1]: [s_store_sk#22] + +(23) BroadcastHashJoin [codegen id : 6] +Left keys [1]: [sr_store_sk#16] +Right keys [1]: [s_store_sk#22] +Join type: Inner +Join condition: None + +(24) Project [codegen id : 6] +Output [3]: [sr_return_amt#17, sr_net_loss#18, s_store_sk#22] +Input [4]: [sr_store_sk#16, sr_return_amt#17, sr_net_loss#18, s_store_sk#22] + +(25) HashAggregate [codegen id : 6] +Input [3]: [sr_return_amt#17, sr_net_loss#18, s_store_sk#22] +Keys [1]: [s_store_sk#22] +Functions [2]: [partial_sum(UnscaledValue(sr_return_amt#17)), partial_sum(UnscaledValue(sr_net_loss#18))] +Aggregate Attributes [2]: [sum#23, sum#24] +Results [3]: [s_store_sk#22, sum#25, sum#26] + +(26) Exchange +Input [3]: [s_store_sk#22, sum#25, sum#26] +Arguments: hashpartitioning(s_store_sk#22, 5), ENSURE_REQUIREMENTS, [plan_id=3] + +(27) HashAggregate [codegen id : 7] +Input [3]: [s_store_sk#22, sum#25, sum#26] +Keys [1]: [s_store_sk#22] +Functions [2]: [sum(UnscaledValue(sr_return_amt#17)), sum(UnscaledValue(sr_net_loss#18))] +Aggregate Attributes [2]: [sum(UnscaledValue(sr_return_amt#17))#27, sum(UnscaledValue(sr_net_loss#18))#28] +Results [3]: [s_store_sk#22, MakeDecimal(sum(UnscaledValue(sr_return_amt#17))#27,17,2) AS returns#29, MakeDecimal(sum(UnscaledValue(sr_net_loss#18))#28,17,2) AS profit_loss#30] + +(28) BroadcastExchange +Input [3]: [s_store_sk#22, returns#29, profit_loss#30] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] + +(29) BroadcastHashJoin [codegen id : 8] +Left keys [1]: [s_store_sk#7] +Right keys [1]: [s_store_sk#22] +Join type: LeftOuter +Join condition: None + +(30) Project [codegen id : 8] +Output [5]: [sales#14, coalesce(returns#29, 0.00) AS returns#31, (profit#15 - coalesce(profit_loss#30, 0.00)) AS profit#32, store channel AS channel#33, s_store_sk#7 AS id#34] +Input [6]: [s_store_sk#7, sales#14, profit#15, s_store_sk#22, returns#29, profit_loss#30] + +(31) Scan parquet spark_catalog.default.catalog_sales +Output [4]: [cs_call_center_sk#35, cs_ext_sales_price#36, cs_net_profit#37, cs_sold_date_sk#38] +Batched: true +Location: InMemoryFileIndex [] +PartitionFilters: [isnotnull(cs_sold_date_sk#38), dynamicpruningexpression(cs_sold_date_sk#38 IN dynamicpruning#39)] +ReadSchema: struct<cs_call_center_sk:int,cs_ext_sales_price:decimal(7,2),cs_net_profit:decimal(7,2)> + +(32) ColumnarToRow [codegen id : 10] +Input [4]: [cs_call_center_sk#35, cs_ext_sales_price#36, cs_net_profit#37, cs_sold_date_sk#38] + +(33) ReusedExchange [Reuses operator id: 90] +Output [1]: [d_date_sk#40] + +(34) BroadcastHashJoin [codegen id : 10] +Left keys [1]: [cs_sold_date_sk#38] +Right keys [1]: [d_date_sk#40] +Join type: Inner +Join condition: None + +(35) Project [codegen id : 10] +Output [3]: [cs_call_center_sk#35, cs_ext_sales_price#36, cs_net_profit#37] +Input [5]: [cs_call_center_sk#35, cs_ext_sales_price#36, cs_net_profit#37, cs_sold_date_sk#38, d_date_sk#40] + +(36) HashAggregate [codegen id : 10] +Input [3]: [cs_call_center_sk#35, cs_ext_sales_price#36, cs_net_profit#37] +Keys [1]: [cs_call_center_sk#35] +Functions [2]: [partial_sum(UnscaledValue(cs_ext_sales_price#36)), partial_sum(UnscaledValue(cs_net_profit#37))] +Aggregate Attributes [2]: [sum#41, sum#42] +Results [3]: [cs_call_center_sk#35, sum#43, sum#44] + +(37) Exchange +Input [3]: [cs_call_center_sk#35, sum#43, sum#44] +Arguments: hashpartitioning(cs_call_center_sk#35, 5), ENSURE_REQUIREMENTS, [plan_id=5] + +(38) HashAggregate [codegen id : 11] +Input [3]: [cs_call_center_sk#35, sum#43, sum#44] +Keys [1]: [cs_call_center_sk#35] +Functions [2]: [sum(UnscaledValue(cs_ext_sales_price#36)), sum(UnscaledValue(cs_net_profit#37))] +Aggregate Attributes [2]: [sum(UnscaledValue(cs_ext_sales_price#36))#45, sum(UnscaledValue(cs_net_profit#37))#46] +Results [3]: [cs_call_center_sk#35, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#36))#45,17,2) AS sales#47, MakeDecimal(sum(UnscaledValue(cs_net_profit#37))#46,17,2) AS profit#48] + +(39) BroadcastExchange +Input [3]: [cs_call_center_sk#35, sales#47, profit#48] +Arguments: IdentityBroadcastMode, [plan_id=6] + +(40) Scan parquet spark_catalog.default.catalog_returns +Output [3]: [cr_return_amount#49, cr_net_loss#50, cr_returned_date_sk#51] +Batched: true +Location: InMemoryFileIndex [] +PartitionFilters: [isnotnull(cr_returned_date_sk#51), dynamicpruningexpression(cr_returned_date_sk#51 IN dynamicpruning#52)] +ReadSchema: struct<cr_return_amount:decimal(7,2),cr_net_loss:decimal(7,2)> + +(41) ColumnarToRow [codegen id : 13] +Input [3]: [cr_return_amount#49, cr_net_loss#50, cr_returned_date_sk#51] + +(42) ReusedExchange [Reuses operator id: 90] +Output [1]: [d_date_sk#53] + +(43) BroadcastHashJoin [codegen id : 13] +Left keys [1]: [cr_returned_date_sk#51] +Right keys [1]: [d_date_sk#53] +Join type: Inner +Join condition: None + +(44) Project [codegen id : 13] +Output [2]: [cr_return_amount#49, cr_net_loss#50] +Input [4]: [cr_return_amount#49, cr_net_loss#50, cr_returned_date_sk#51, d_date_sk#53] + +(45) HashAggregate [codegen id : 13] +Input [2]: [cr_return_amount#49, cr_net_loss#50] +Keys: [] +Functions [2]: [partial_sum(UnscaledValue(cr_return_amount#49)), partial_sum(UnscaledValue(cr_net_loss#50))] +Aggregate Attributes [2]: [sum#54, sum#55] +Results [2]: [sum#56, sum#57] + +(46) Exchange +Input [2]: [sum#56, sum#57] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=7] + +(47) HashAggregate +Input [2]: [sum#56, sum#57] +Keys: [] +Functions [2]: [sum(UnscaledValue(cr_return_amount#49)), sum(UnscaledValue(cr_net_loss#50))] +Aggregate Attributes [2]: [sum(UnscaledValue(cr_return_amount#49))#58, sum(UnscaledValue(cr_net_loss#50))#59] +Results [2]: [MakeDecimal(sum(UnscaledValue(cr_return_amount#49))#58,17,2) AS returns#60, MakeDecimal(sum(UnscaledValue(cr_net_loss#50))#59,17,2) AS profit_loss#61] + +(48) BroadcastNestedLoopJoin [codegen id : 14] +Join type: Inner +Join condition: None + +(49) Project [codegen id : 14] +Output [5]: [sales#47, returns#60, (profit#48 - profit_loss#61) AS profit#62, catalog channel AS channel#63, cs_call_center_sk#35 AS id#64] +Input [5]: [cs_call_center_sk#35, sales#47, profit#48, returns#60, profit_loss#61] + +(50) Scan parquet spark_catalog.default.web_sales +Output [4]: [ws_web_page_sk#65, ws_ext_sales_price#66, ws_net_profit#67, ws_sold_date_sk#68] +Batched: true +Location: InMemoryFileIndex [] +PartitionFilters: [isnotnull(ws_sold_date_sk#68), dynamicpruningexpression(ws_sold_date_sk#68 IN dynamicpruning#69)] +PushedFilters: [IsNotNull(ws_web_page_sk)] +ReadSchema: struct<ws_web_page_sk:int,ws_ext_sales_price:decimal(7,2),ws_net_profit:decimal(7,2)> + +(51) CometFilter +Input [4]: [ws_web_page_sk#65, ws_ext_sales_price#66, ws_net_profit#67, ws_sold_date_sk#68] +Condition : isnotnull(ws_web_page_sk#65) + +(52) ColumnarToRow [codegen id : 17] +Input [4]: [ws_web_page_sk#65, ws_ext_sales_price#66, ws_net_profit#67, ws_sold_date_sk#68] + +(53) ReusedExchange [Reuses operator id: 90] +Output [1]: [d_date_sk#70] + +(54) BroadcastHashJoin [codegen id : 17] +Left keys [1]: [ws_sold_date_sk#68] +Right keys [1]: [d_date_sk#70] +Join type: Inner +Join condition: None + +(55) Project [codegen id : 17] +Output [3]: [ws_web_page_sk#65, ws_ext_sales_price#66, ws_net_profit#67] +Input [5]: [ws_web_page_sk#65, ws_ext_sales_price#66, ws_net_profit#67, ws_sold_date_sk#68, d_date_sk#70] + +(56) Scan parquet spark_catalog.default.web_page +Output [1]: [wp_web_page_sk#71] +Batched: true +Location [not included in comparison]/{warehouse_dir}/web_page] +PushedFilters: [IsNotNull(wp_web_page_sk)] +ReadSchema: struct<wp_web_page_sk:int> + +(57) CometFilter +Input [1]: [wp_web_page_sk#71] +Condition : isnotnull(wp_web_page_sk#71) + +(58) ColumnarToRow [codegen id : 16] +Input [1]: [wp_web_page_sk#71] + +(59) BroadcastExchange +Input [1]: [wp_web_page_sk#71] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=8] + +(60) BroadcastHashJoin [codegen id : 17] +Left keys [1]: [ws_web_page_sk#65] +Right keys [1]: [wp_web_page_sk#71] +Join type: Inner +Join condition: None + +(61) Project [codegen id : 17] +Output [3]: [ws_ext_sales_price#66, ws_net_profit#67, wp_web_page_sk#71] +Input [4]: [ws_web_page_sk#65, ws_ext_sales_price#66, ws_net_profit#67, wp_web_page_sk#71] + +(62) HashAggregate [codegen id : 17] +Input [3]: [ws_ext_sales_price#66, ws_net_profit#67, wp_web_page_sk#71] +Keys [1]: [wp_web_page_sk#71] +Functions [2]: [partial_sum(UnscaledValue(ws_ext_sales_price#66)), partial_sum(UnscaledValue(ws_net_profit#67))] +Aggregate Attributes [2]: [sum#72, sum#73] +Results [3]: [wp_web_page_sk#71, sum#74, sum#75] + +(63) Exchange +Input [3]: [wp_web_page_sk#71, sum#74, sum#75] +Arguments: hashpartitioning(wp_web_page_sk#71, 5), ENSURE_REQUIREMENTS, [plan_id=9] + +(64) HashAggregate [codegen id : 22] +Input [3]: [wp_web_page_sk#71, sum#74, sum#75] +Keys [1]: [wp_web_page_sk#71] +Functions [2]: [sum(UnscaledValue(ws_ext_sales_price#66)), sum(UnscaledValue(ws_net_profit#67))] +Aggregate Attributes [2]: [sum(UnscaledValue(ws_ext_sales_price#66))#76, sum(UnscaledValue(ws_net_profit#67))#77] +Results [3]: [wp_web_page_sk#71, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#66))#76,17,2) AS sales#78, MakeDecimal(sum(UnscaledValue(ws_net_profit#67))#77,17,2) AS profit#79] + +(65) Scan parquet spark_catalog.default.web_returns +Output [4]: [wr_web_page_sk#80, wr_return_amt#81, wr_net_loss#82, wr_returned_date_sk#83] +Batched: true +Location: InMemoryFileIndex [] +PartitionFilters: [isnotnull(wr_returned_date_sk#83), dynamicpruningexpression(wr_returned_date_sk#83 IN dynamicpruning#84)] +PushedFilters: [IsNotNull(wr_web_page_sk)] +ReadSchema: struct<wr_web_page_sk:int,wr_return_amt:decimal(7,2),wr_net_loss:decimal(7,2)> + +(66) CometFilter +Input [4]: [wr_web_page_sk#80, wr_return_amt#81, wr_net_loss#82, wr_returned_date_sk#83] +Condition : isnotnull(wr_web_page_sk#80) + +(67) ColumnarToRow [codegen id : 20] +Input [4]: [wr_web_page_sk#80, wr_return_amt#81, wr_net_loss#82, wr_returned_date_sk#83] + +(68) ReusedExchange [Reuses operator id: 90] +Output [1]: [d_date_sk#85] + +(69) BroadcastHashJoin [codegen id : 20] +Left keys [1]: [wr_returned_date_sk#83] +Right keys [1]: [d_date_sk#85] +Join type: Inner +Join condition: None + +(70) Project [codegen id : 20] +Output [3]: [wr_web_page_sk#80, wr_return_amt#81, wr_net_loss#82] +Input [5]: [wr_web_page_sk#80, wr_return_amt#81, wr_net_loss#82, wr_returned_date_sk#83, d_date_sk#85] + +(71) ReusedExchange [Reuses operator id: 59] +Output [1]: [wp_web_page_sk#86] + +(72) BroadcastHashJoin [codegen id : 20] +Left keys [1]: [wr_web_page_sk#80] +Right keys [1]: [wp_web_page_sk#86] +Join type: Inner +Join condition: None + +(73) Project [codegen id : 20] +Output [3]: [wr_return_amt#81, wr_net_loss#82, wp_web_page_sk#86] +Input [4]: [wr_web_page_sk#80, wr_return_amt#81, wr_net_loss#82, wp_web_page_sk#86] + +(74) HashAggregate [codegen id : 20] +Input [3]: [wr_return_amt#81, wr_net_loss#82, wp_web_page_sk#86] +Keys [1]: [wp_web_page_sk#86] +Functions [2]: [partial_sum(UnscaledValue(wr_return_amt#81)), partial_sum(UnscaledValue(wr_net_loss#82))] +Aggregate Attributes [2]: [sum#87, sum#88] +Results [3]: [wp_web_page_sk#86, sum#89, sum#90] + +(75) Exchange +Input [3]: [wp_web_page_sk#86, sum#89, sum#90] +Arguments: hashpartitioning(wp_web_page_sk#86, 5), ENSURE_REQUIREMENTS, [plan_id=10] + +(76) HashAggregate [codegen id : 21] +Input [3]: [wp_web_page_sk#86, sum#89, sum#90] +Keys [1]: [wp_web_page_sk#86] +Functions [2]: [sum(UnscaledValue(wr_return_amt#81)), sum(UnscaledValue(wr_net_loss#82))] +Aggregate Attributes [2]: [sum(UnscaledValue(wr_return_amt#81))#91, sum(UnscaledValue(wr_net_loss#82))#92] +Results [3]: [wp_web_page_sk#86, MakeDecimal(sum(UnscaledValue(wr_return_amt#81))#91,17,2) AS returns#93, MakeDecimal(sum(UnscaledValue(wr_net_loss#82))#92,17,2) AS profit_loss#94] + +(77) BroadcastExchange +Input [3]: [wp_web_page_sk#86, returns#93, profit_loss#94] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=11] + +(78) BroadcastHashJoin [codegen id : 22] +Left keys [1]: [wp_web_page_sk#71] +Right keys [1]: [wp_web_page_sk#86] +Join type: LeftOuter +Join condition: None + +(79) Project [codegen id : 22] +Output [5]: [sales#78, coalesce(returns#93, 0.00) AS returns#95, (profit#79 - coalesce(profit_loss#94, 0.00)) AS profit#96, web channel AS channel#97, wp_web_page_sk#71 AS id#98] +Input [6]: [wp_web_page_sk#71, sales#78, profit#79, wp_web_page_sk#86, returns#93, profit_loss#94] + +(80) Union + +(81) Expand [codegen id : 23] +Input [5]: [sales#14, returns#31, profit#32, channel#33, id#34] +Arguments: [[sales#14, returns#31, profit#32, channel#33, id#34, 0], [sales#14, returns#31, profit#32, channel#33, null, 1], [sales#14, returns#31, profit#32, null, null, 3]], [sales#14, returns#31, profit#32, channel#99, id#100, spark_grouping_id#101] + +(82) HashAggregate [codegen id : 23] +Input [6]: [sales#14, returns#31, profit#32, channel#99, id#100, spark_grouping_id#101] +Keys [3]: [channel#99, id#100, spark_grouping_id#101] +Functions [3]: [partial_sum(sales#14), partial_sum(returns#31), partial_sum(profit#32)] +Aggregate Attributes [6]: [sum#102, isEmpty#103, sum#104, isEmpty#105, sum#106, isEmpty#107] +Results [9]: [channel#99, id#100, spark_grouping_id#101, sum#108, isEmpty#109, sum#110, isEmpty#111, sum#112, isEmpty#113] + +(83) Exchange +Input [9]: [channel#99, id#100, spark_grouping_id#101, sum#108, isEmpty#109, sum#110, isEmpty#111, sum#112, isEmpty#113] +Arguments: hashpartitioning(channel#99, id#100, spark_grouping_id#101, 5), ENSURE_REQUIREMENTS, [plan_id=12] + +(84) HashAggregate [codegen id : 24] +Input [9]: [channel#99, id#100, spark_grouping_id#101, sum#108, isEmpty#109, sum#110, isEmpty#111, sum#112, isEmpty#113] +Keys [3]: [channel#99, id#100, spark_grouping_id#101] +Functions [3]: [sum(sales#14), sum(returns#31), sum(profit#32)] +Aggregate Attributes [3]: [sum(sales#14)#114, sum(returns#31)#115, sum(profit#32)#116] +Results [5]: [channel#99, id#100, sum(sales#14)#114 AS sales#117, sum(returns#31)#115 AS returns#118, sum(profit#32)#116 AS profit#119] + +(85) TakeOrderedAndProject +Input [5]: [channel#99, id#100, sales#117, returns#118, profit#119] +Arguments: 100, [channel#99 ASC NULLS FIRST, id#100 ASC NULLS FIRST], [channel#99, id#100, sales#117, returns#118, profit#119] + +===== Subqueries ===== + +Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 +BroadcastExchange (90) ++- * ColumnarToRow (89) + +- CometProject (88) + +- CometFilter (87) + +- CometScan parquet spark_catalog.default.date_dim (86) + + +(86) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#6, d_date#120] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,2000-08-03), LessThanOrEqual(d_date,2000-09-02), IsNotNull(d_date_sk)] +ReadSchema: struct<d_date_sk:int,d_date:date> + +(87) CometFilter +Input [2]: [d_date_sk#6, d_date#120] +Condition : (((isnotnull(d_date#120) AND (d_date#120 >= 2000-08-03)) AND (d_date#120 <= 2000-09-02)) AND isnotnull(d_date_sk#6)) + +(88) CometProject +Input [2]: [d_date_sk#6, d_date#120] +Arguments: [d_date_sk#6], [d_date_sk#6] + +(89) ColumnarToRow [codegen id : 1] +Input [1]: [d_date_sk#6] + +(90) BroadcastExchange +Input [1]: [d_date_sk#6] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=13] + +Subquery:2 Hosting operator id = 16 Hosting Expression = sr_returned_date_sk#19 IN dynamicpruning#5 + +Subquery:3 Hosting operator id = 31 Hosting Expression = cs_sold_date_sk#38 IN dynamicpruning#5 + +Subquery:4 Hosting operator id = 40 Hosting Expression = cr_returned_date_sk#51 IN dynamicpruning#5 + +Subquery:5 Hosting operator id = 50 Hosting Expression = ws_sold_date_sk#68 IN dynamicpruning#5 + +Subquery:6 Hosting operator id = 65 Hosting Expression = wr_returned_date_sk#83 IN dynamicpruning#5 + + diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q77/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q77/simplified.txt new file mode 100644 index 000000000..d6693067f --- /dev/null +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q77/simplified.txt @@ -0,0 +1,143 @@ +TakeOrderedAndProject [channel,id,sales,returns,profit] + WholeStageCodegen (24) + HashAggregate [channel,id,spark_grouping_id,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),sales,returns,profit,sum,isEmpty,sum,isEmpty,sum,isEmpty] + InputAdapter + Exchange [channel,id,spark_grouping_id] #1 + WholeStageCodegen (23) + 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) + Project [sales,returns,profit,profit_loss,s_store_sk] + BroadcastHashJoin [s_store_sk,s_store_sk] + HashAggregate [s_store_sk,sum,sum] [sum(UnscaledValue(ss_ext_sales_price)),sum(UnscaledValue(ss_net_profit)),sales,profit,sum,sum] + InputAdapter + Exchange [s_store_sk] #2 + WholeStageCodegen (3) + HashAggregate [s_store_sk,ss_ext_sales_price,ss_net_profit] [sum,sum,sum,sum] + Project [ss_ext_sales_price,ss_net_profit,s_store_sk] + BroadcastHashJoin [ss_store_sk,s_store_sk] + Project [ss_store_sk,ss_ext_sales_price,ss_net_profit] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + ColumnarToRow + InputAdapter + CometFilter [ss_store_sk] + CometScan parquet spark_catalog.default.store_sales [ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] + InputAdapter + ReusedExchange [d_date_sk] #3 + InputAdapter + BroadcastExchange #4 + WholeStageCodegen (2) + ColumnarToRow + InputAdapter + CometFilter [s_store_sk] + CometScan parquet spark_catalog.default.store [s_store_sk] + InputAdapter + BroadcastExchange #5 + WholeStageCodegen (7) + HashAggregate [s_store_sk,sum,sum] [sum(UnscaledValue(sr_return_amt)),sum(UnscaledValue(sr_net_loss)),returns,profit_loss,sum,sum] + InputAdapter + Exchange [s_store_sk] #6 + WholeStageCodegen (6) + HashAggregate [s_store_sk,sr_return_amt,sr_net_loss] [sum,sum,sum,sum] + Project [sr_return_amt,sr_net_loss,s_store_sk] + BroadcastHashJoin [sr_store_sk,s_store_sk] + Project [sr_store_sk,sr_return_amt,sr_net_loss] + BroadcastHashJoin [sr_returned_date_sk,d_date_sk] + ColumnarToRow + InputAdapter + CometFilter [sr_store_sk] + CometScan parquet spark_catalog.default.store_returns [sr_store_sk,sr_return_amt,sr_net_loss,sr_returned_date_sk] + ReusedSubquery [d_date_sk] #1 + InputAdapter + ReusedExchange [d_date_sk] #3 + InputAdapter + ReusedExchange [s_store_sk] #4 + WholeStageCodegen (14) + Project [sales,returns,profit,profit_loss,cs_call_center_sk] + BroadcastNestedLoopJoin + InputAdapter + BroadcastExchange #7 + WholeStageCodegen (11) + HashAggregate [cs_call_center_sk,sum,sum] [sum(UnscaledValue(cs_ext_sales_price)),sum(UnscaledValue(cs_net_profit)),sales,profit,sum,sum] + InputAdapter + Exchange [cs_call_center_sk] #8 + WholeStageCodegen (10) + HashAggregate [cs_call_center_sk,cs_ext_sales_price,cs_net_profit] [sum,sum,sum,sum] + Project [cs_call_center_sk,cs_ext_sales_price,cs_net_profit] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + ColumnarToRow + InputAdapter + CometScan parquet spark_catalog.default.catalog_sales [cs_call_center_sk,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + InputAdapter + ReusedExchange [d_date_sk] #3 + HashAggregate [sum,sum] [sum(UnscaledValue(cr_return_amount)),sum(UnscaledValue(cr_net_loss)),returns,profit_loss,sum,sum] + InputAdapter + Exchange #9 + WholeStageCodegen (13) + HashAggregate [cr_return_amount,cr_net_loss] [sum,sum,sum,sum] + Project [cr_return_amount,cr_net_loss] + BroadcastHashJoin [cr_returned_date_sk,d_date_sk] + ColumnarToRow + InputAdapter + CometScan parquet spark_catalog.default.catalog_returns [cr_return_amount,cr_net_loss,cr_returned_date_sk] + ReusedSubquery [d_date_sk] #1 + InputAdapter + ReusedExchange [d_date_sk] #3 + WholeStageCodegen (22) + Project [sales,returns,profit,profit_loss,wp_web_page_sk] + BroadcastHashJoin [wp_web_page_sk,wp_web_page_sk] + HashAggregate [wp_web_page_sk,sum,sum] [sum(UnscaledValue(ws_ext_sales_price)),sum(UnscaledValue(ws_net_profit)),sales,profit,sum,sum] + InputAdapter + Exchange [wp_web_page_sk] #10 + WholeStageCodegen (17) + HashAggregate [wp_web_page_sk,ws_ext_sales_price,ws_net_profit] [sum,sum,sum,sum] + Project [ws_ext_sales_price,ws_net_profit,wp_web_page_sk] + BroadcastHashJoin [ws_web_page_sk,wp_web_page_sk] + Project [ws_web_page_sk,ws_ext_sales_price,ws_net_profit] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + ColumnarToRow + InputAdapter + CometFilter [ws_web_page_sk] + CometScan parquet spark_catalog.default.web_sales [ws_web_page_sk,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + InputAdapter + ReusedExchange [d_date_sk] #3 + InputAdapter + BroadcastExchange #11 + WholeStageCodegen (16) + ColumnarToRow + InputAdapter + CometFilter [wp_web_page_sk] + CometScan parquet spark_catalog.default.web_page [wp_web_page_sk] + InputAdapter + BroadcastExchange #12 + WholeStageCodegen (21) + HashAggregate [wp_web_page_sk,sum,sum] [sum(UnscaledValue(wr_return_amt)),sum(UnscaledValue(wr_net_loss)),returns,profit_loss,sum,sum] + InputAdapter + Exchange [wp_web_page_sk] #13 + WholeStageCodegen (20) + HashAggregate [wp_web_page_sk,wr_return_amt,wr_net_loss] [sum,sum,sum,sum] + Project [wr_return_amt,wr_net_loss,wp_web_page_sk] + BroadcastHashJoin [wr_web_page_sk,wp_web_page_sk] + Project [wr_web_page_sk,wr_return_amt,wr_net_loss] + BroadcastHashJoin [wr_returned_date_sk,d_date_sk] + ColumnarToRow + InputAdapter + CometFilter [wr_web_page_sk] + CometScan parquet spark_catalog.default.web_returns [wr_web_page_sk,wr_return_amt,wr_net_loss,wr_returned_date_sk] + ReusedSubquery [d_date_sk] #1 + InputAdapter + ReusedExchange [d_date_sk] #3 + InputAdapter + ReusedExchange [wp_web_page_sk] #11 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 new file mode 100644 index 000000000..7f2688112 --- /dev/null +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q78/explain.txt @@ -0,0 +1,431 @@ +== Physical Plan == +TakeOrderedAndProject (70) ++- * Project (69) + +- * SortMergeJoin Inner (68) + :- * Project (45) + : +- * SortMergeJoin Inner (44) + : :- * Sort (21) + : : +- * HashAggregate (20) + : : +- Exchange (19) + : : +- * HashAggregate (18) + : : +- * Project (17) + : : +- * BroadcastHashJoin Inner BuildRight (16) + : : :- * Project (14) + : : : +- * Filter (13) + : : : +- * SortMergeJoin LeftOuter (12) + : : : :- * Sort (5) + : : : : +- Exchange (4) + : : : : +- * ColumnarToRow (3) + : : : : +- CometFilter (2) + : : : : +- CometScan parquet spark_catalog.default.store_sales (1) + : : : +- * Sort (11) + : : : +- Exchange (10) + : : : +- * ColumnarToRow (9) + : : : +- CometProject (8) + : : : +- CometFilter (7) + : : : +- CometScan parquet spark_catalog.default.store_returns (6) + : : +- ReusedExchange (15) + : +- * Sort (43) + : +- * Filter (42) + : +- * HashAggregate (41) + : +- Exchange (40) + : +- * HashAggregate (39) + : +- * Project (38) + : +- * BroadcastHashJoin Inner BuildRight (37) + : :- * Project (35) + : : +- * Filter (34) + : : +- * SortMergeJoin LeftOuter (33) + : : :- * Sort (26) + : : : +- Exchange (25) + : : : +- * ColumnarToRow (24) + : : : +- CometFilter (23) + : : : +- CometScan parquet spark_catalog.default.web_sales (22) + : : +- * Sort (32) + : : +- Exchange (31) + : : +- * ColumnarToRow (30) + : : +- CometProject (29) + : : +- CometFilter (28) + : : +- CometScan parquet spark_catalog.default.web_returns (27) + : +- ReusedExchange (36) + +- * Sort (67) + +- * Filter (66) + +- * HashAggregate (65) + +- Exchange (64) + +- * HashAggregate (63) + +- * Project (62) + +- * BroadcastHashJoin Inner BuildRight (61) + :- * Project (59) + : +- * Filter (58) + : +- * SortMergeJoin LeftOuter (57) + : :- * Sort (50) + : : +- Exchange (49) + : : +- * ColumnarToRow (48) + : : +- CometFilter (47) + : : +- CometScan parquet spark_catalog.default.catalog_sales (46) + : +- * Sort (56) + : +- Exchange (55) + : +- * ColumnarToRow (54) + : +- CometProject (53) + : +- CometFilter (52) + : +- CometScan parquet spark_catalog.default.catalog_returns (51) + +- ReusedExchange (60) + + +(1) Scan parquet spark_catalog.default.store_sales +Output [7]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#3, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, ss_sold_date_sk#7] +Batched: true +Location: InMemoryFileIndex [] +PartitionFilters: [isnotnull(ss_sold_date_sk#7), dynamicpruningexpression(ss_sold_date_sk#7 IN dynamicpruning#8)] +PushedFilters: [IsNotNull(ss_item_sk), IsNotNull(ss_customer_sk)] +ReadSchema: struct<ss_item_sk:int,ss_customer_sk:int,ss_ticket_number:int,ss_quantity:int,ss_wholesale_cost:decimal(7,2),ss_sales_price:decimal(7,2)> + +(2) CometFilter +Input [7]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#3, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, ss_sold_date_sk#7] +Condition : (isnotnull(ss_item_sk#1) AND isnotnull(ss_customer_sk#2)) + +(3) ColumnarToRow [codegen id : 1] +Input [7]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#3, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, ss_sold_date_sk#7] + +(4) Exchange +Input [7]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#3, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, ss_sold_date_sk#7] +Arguments: hashpartitioning(ss_ticket_number#3, ss_item_sk#1, 5), ENSURE_REQUIREMENTS, [plan_id=1] + +(5) Sort [codegen id : 2] +Input [7]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#3, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, ss_sold_date_sk#7] +Arguments: [ss_ticket_number#3 ASC NULLS FIRST, ss_item_sk#1 ASC NULLS FIRST], false, 0 + +(6) Scan parquet spark_catalog.default.store_returns +Output [3]: [sr_item_sk#9, sr_ticket_number#10, sr_returned_date_sk#11] +Batched: true +Location [not included in comparison]/{warehouse_dir}/store_returns] +PushedFilters: [IsNotNull(sr_ticket_number), IsNotNull(sr_item_sk)] +ReadSchema: struct<sr_item_sk:int,sr_ticket_number:int> + +(7) CometFilter +Input [3]: [sr_item_sk#9, sr_ticket_number#10, sr_returned_date_sk#11] +Condition : (isnotnull(sr_ticket_number#10) AND isnotnull(sr_item_sk#9)) + +(8) CometProject +Input [3]: [sr_item_sk#9, sr_ticket_number#10, sr_returned_date_sk#11] +Arguments: [sr_item_sk#9, sr_ticket_number#10], [sr_item_sk#9, sr_ticket_number#10] + +(9) ColumnarToRow [codegen id : 3] +Input [2]: [sr_item_sk#9, sr_ticket_number#10] + +(10) Exchange +Input [2]: [sr_item_sk#9, sr_ticket_number#10] +Arguments: hashpartitioning(sr_ticket_number#10, sr_item_sk#9, 5), ENSURE_REQUIREMENTS, [plan_id=2] + +(11) Sort [codegen id : 4] +Input [2]: [sr_item_sk#9, sr_ticket_number#10] +Arguments: [sr_ticket_number#10 ASC NULLS FIRST, sr_item_sk#9 ASC NULLS FIRST], false, 0 + +(12) SortMergeJoin [codegen id : 6] +Left keys [2]: [ss_ticket_number#3, ss_item_sk#1] +Right keys [2]: [sr_ticket_number#10, sr_item_sk#9] +Join type: LeftOuter +Join condition: None + +(13) Filter [codegen id : 6] +Input [9]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#3, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, ss_sold_date_sk#7, sr_item_sk#9, sr_ticket_number#10] +Condition : isnull(sr_ticket_number#10) + +(14) Project [codegen id : 6] +Output [6]: [ss_item_sk#1, ss_customer_sk#2, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, ss_sold_date_sk#7] +Input [9]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#3, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, ss_sold_date_sk#7, sr_item_sk#9, sr_ticket_number#10] + +(15) ReusedExchange [Reuses operator id: 74] +Output [2]: [d_date_sk#12, d_year#13] + +(16) BroadcastHashJoin [codegen id : 6] +Left keys [1]: [ss_sold_date_sk#7] +Right keys [1]: [d_date_sk#12] +Join type: Inner +Join condition: None + +(17) Project [codegen id : 6] +Output [6]: [ss_item_sk#1, ss_customer_sk#2, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, d_year#13] +Input [8]: [ss_item_sk#1, ss_customer_sk#2, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, ss_sold_date_sk#7, d_date_sk#12, d_year#13] + +(18) HashAggregate [codegen id : 6] +Input [6]: [ss_item_sk#1, ss_customer_sk#2, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, d_year#13] +Keys [3]: [d_year#13, ss_item_sk#1, ss_customer_sk#2] +Functions [3]: [partial_sum(ss_quantity#4), partial_sum(UnscaledValue(ss_wholesale_cost#5)), partial_sum(UnscaledValue(ss_sales_price#6))] +Aggregate Attributes [3]: [sum#14, sum#15, sum#16] +Results [6]: [d_year#13, ss_item_sk#1, ss_customer_sk#2, sum#17, sum#18, sum#19] + +(19) Exchange +Input [6]: [d_year#13, ss_item_sk#1, ss_customer_sk#2, sum#17, sum#18, sum#19] +Arguments: hashpartitioning(d_year#13, ss_item_sk#1, ss_customer_sk#2, 5), ENSURE_REQUIREMENTS, [plan_id=3] + +(20) HashAggregate [codegen id : 7] +Input [6]: [d_year#13, ss_item_sk#1, ss_customer_sk#2, sum#17, sum#18, sum#19] +Keys [3]: [d_year#13, ss_item_sk#1, ss_customer_sk#2] +Functions [3]: [sum(ss_quantity#4), sum(UnscaledValue(ss_wholesale_cost#5)), sum(UnscaledValue(ss_sales_price#6))] +Aggregate Attributes [3]: [sum(ss_quantity#4)#20, sum(UnscaledValue(ss_wholesale_cost#5))#21, sum(UnscaledValue(ss_sales_price#6))#22] +Results [6]: [d_year#13 AS ss_sold_year#23, ss_item_sk#1, ss_customer_sk#2, sum(ss_quantity#4)#20 AS ss_qty#24, MakeDecimal(sum(UnscaledValue(ss_wholesale_cost#5))#21,17,2) AS ss_wc#25, MakeDecimal(sum(UnscaledValue(ss_sales_price#6))#22,17,2) AS ss_sp#26] + +(21) Sort [codegen id : 7] +Input [6]: [ss_sold_year#23, ss_item_sk#1, ss_customer_sk#2, ss_qty#24, ss_wc#25, ss_sp#26] +Arguments: [ss_sold_year#23 ASC NULLS FIRST, ss_item_sk#1 ASC NULLS FIRST, ss_customer_sk#2 ASC NULLS FIRST], false, 0 + +(22) Scan parquet spark_catalog.default.web_sales +Output [7]: [ws_item_sk#27, ws_bill_customer_sk#28, ws_order_number#29, ws_quantity#30, ws_wholesale_cost#31, ws_sales_price#32, ws_sold_date_sk#33] +Batched: true +Location: InMemoryFileIndex [] +PartitionFilters: [isnotnull(ws_sold_date_sk#33), dynamicpruningexpression(ws_sold_date_sk#33 IN dynamicpruning#34)] +PushedFilters: [IsNotNull(ws_item_sk), IsNotNull(ws_bill_customer_sk)] +ReadSchema: struct<ws_item_sk:int,ws_bill_customer_sk:int,ws_order_number:int,ws_quantity:int,ws_wholesale_cost:decimal(7,2),ws_sales_price:decimal(7,2)> + +(23) CometFilter +Input [7]: [ws_item_sk#27, ws_bill_customer_sk#28, ws_order_number#29, ws_quantity#30, ws_wholesale_cost#31, ws_sales_price#32, ws_sold_date_sk#33] +Condition : (isnotnull(ws_item_sk#27) AND isnotnull(ws_bill_customer_sk#28)) + +(24) ColumnarToRow [codegen id : 8] +Input [7]: [ws_item_sk#27, ws_bill_customer_sk#28, ws_order_number#29, ws_quantity#30, ws_wholesale_cost#31, ws_sales_price#32, ws_sold_date_sk#33] + +(25) Exchange +Input [7]: [ws_item_sk#27, ws_bill_customer_sk#28, ws_order_number#29, ws_quantity#30, ws_wholesale_cost#31, ws_sales_price#32, ws_sold_date_sk#33] +Arguments: hashpartitioning(ws_order_number#29, ws_item_sk#27, 5), ENSURE_REQUIREMENTS, [plan_id=4] + +(26) Sort [codegen id : 9] +Input [7]: [ws_item_sk#27, ws_bill_customer_sk#28, ws_order_number#29, ws_quantity#30, ws_wholesale_cost#31, ws_sales_price#32, ws_sold_date_sk#33] +Arguments: [ws_order_number#29 ASC NULLS FIRST, ws_item_sk#27 ASC NULLS FIRST], false, 0 + +(27) Scan parquet spark_catalog.default.web_returns +Output [3]: [wr_item_sk#35, wr_order_number#36, wr_returned_date_sk#37] +Batched: true +Location [not included in comparison]/{warehouse_dir}/web_returns] +PushedFilters: [IsNotNull(wr_order_number), IsNotNull(wr_item_sk)] +ReadSchema: struct<wr_item_sk:int,wr_order_number:int> + +(28) CometFilter +Input [3]: [wr_item_sk#35, wr_order_number#36, wr_returned_date_sk#37] +Condition : (isnotnull(wr_order_number#36) AND isnotnull(wr_item_sk#35)) + +(29) CometProject +Input [3]: [wr_item_sk#35, wr_order_number#36, wr_returned_date_sk#37] +Arguments: [wr_item_sk#35, wr_order_number#36], [wr_item_sk#35, wr_order_number#36] + +(30) ColumnarToRow [codegen id : 10] +Input [2]: [wr_item_sk#35, wr_order_number#36] + +(31) Exchange +Input [2]: [wr_item_sk#35, wr_order_number#36] +Arguments: hashpartitioning(wr_order_number#36, wr_item_sk#35, 5), ENSURE_REQUIREMENTS, [plan_id=5] + +(32) Sort [codegen id : 11] +Input [2]: [wr_item_sk#35, wr_order_number#36] +Arguments: [wr_order_number#36 ASC NULLS FIRST, wr_item_sk#35 ASC NULLS FIRST], false, 0 + +(33) SortMergeJoin [codegen id : 13] +Left keys [2]: [ws_order_number#29, ws_item_sk#27] +Right keys [2]: [wr_order_number#36, wr_item_sk#35] +Join type: LeftOuter +Join condition: None + +(34) Filter [codegen id : 13] +Input [9]: [ws_item_sk#27, ws_bill_customer_sk#28, ws_order_number#29, ws_quantity#30, ws_wholesale_cost#31, ws_sales_price#32, ws_sold_date_sk#33, wr_item_sk#35, wr_order_number#36] +Condition : isnull(wr_order_number#36) + +(35) Project [codegen id : 13] +Output [6]: [ws_item_sk#27, ws_bill_customer_sk#28, ws_quantity#30, ws_wholesale_cost#31, ws_sales_price#32, ws_sold_date_sk#33] +Input [9]: [ws_item_sk#27, ws_bill_customer_sk#28, ws_order_number#29, ws_quantity#30, ws_wholesale_cost#31, ws_sales_price#32, ws_sold_date_sk#33, wr_item_sk#35, wr_order_number#36] + +(36) ReusedExchange [Reuses operator id: 74] +Output [2]: [d_date_sk#38, d_year#39] + +(37) BroadcastHashJoin [codegen id : 13] +Left keys [1]: [ws_sold_date_sk#33] +Right keys [1]: [d_date_sk#38] +Join type: Inner +Join condition: None + +(38) Project [codegen id : 13] +Output [6]: [ws_item_sk#27, ws_bill_customer_sk#28, ws_quantity#30, ws_wholesale_cost#31, ws_sales_price#32, d_year#39] +Input [8]: [ws_item_sk#27, ws_bill_customer_sk#28, ws_quantity#30, ws_wholesale_cost#31, ws_sales_price#32, ws_sold_date_sk#33, d_date_sk#38, d_year#39] + +(39) HashAggregate [codegen id : 13] +Input [6]: [ws_item_sk#27, ws_bill_customer_sk#28, ws_quantity#30, ws_wholesale_cost#31, ws_sales_price#32, d_year#39] +Keys [3]: [d_year#39, ws_item_sk#27, ws_bill_customer_sk#28] +Functions [3]: [partial_sum(ws_quantity#30), partial_sum(UnscaledValue(ws_wholesale_cost#31)), partial_sum(UnscaledValue(ws_sales_price#32))] +Aggregate Attributes [3]: [sum#40, sum#41, sum#42] +Results [6]: [d_year#39, ws_item_sk#27, ws_bill_customer_sk#28, sum#43, sum#44, sum#45] + +(40) Exchange +Input [6]: [d_year#39, ws_item_sk#27, ws_bill_customer_sk#28, sum#43, sum#44, sum#45] +Arguments: hashpartitioning(d_year#39, ws_item_sk#27, ws_bill_customer_sk#28, 5), ENSURE_REQUIREMENTS, [plan_id=6] + +(41) HashAggregate [codegen id : 14] +Input [6]: [d_year#39, ws_item_sk#27, ws_bill_customer_sk#28, sum#43, sum#44, sum#45] +Keys [3]: [d_year#39, ws_item_sk#27, ws_bill_customer_sk#28] +Functions [3]: [sum(ws_quantity#30), sum(UnscaledValue(ws_wholesale_cost#31)), sum(UnscaledValue(ws_sales_price#32))] +Aggregate Attributes [3]: [sum(ws_quantity#30)#46, sum(UnscaledValue(ws_wholesale_cost#31))#47, sum(UnscaledValue(ws_sales_price#32))#48] +Results [6]: [d_year#39 AS ws_sold_year#49, ws_item_sk#27, ws_bill_customer_sk#28 AS ws_customer_sk#50, sum(ws_quantity#30)#46 AS ws_qty#51, MakeDecimal(sum(UnscaledValue(ws_wholesale_cost#31))#47,17,2) AS ws_wc#52, MakeDecimal(sum(UnscaledValue(ws_sales_price#32))#48,17,2) AS ws_sp#53] + +(42) Filter [codegen id : 14] +Input [6]: [ws_sold_year#49, ws_item_sk#27, ws_customer_sk#50, ws_qty#51, ws_wc#52, ws_sp#53] +Condition : (coalesce(ws_qty#51, 0) > 0) + +(43) Sort [codegen id : 14] +Input [6]: [ws_sold_year#49, ws_item_sk#27, ws_customer_sk#50, ws_qty#51, ws_wc#52, ws_sp#53] +Arguments: [ws_sold_year#49 ASC NULLS FIRST, ws_item_sk#27 ASC NULLS FIRST, ws_customer_sk#50 ASC NULLS FIRST], false, 0 + +(44) SortMergeJoin [codegen id : 15] +Left keys [3]: [ss_sold_year#23, ss_item_sk#1, ss_customer_sk#2] +Right keys [3]: [ws_sold_year#49, ws_item_sk#27, ws_customer_sk#50] +Join type: Inner +Join condition: None + +(45) Project [codegen id : 15] +Output [9]: [ss_sold_year#23, ss_item_sk#1, ss_customer_sk#2, ss_qty#24, ss_wc#25, ss_sp#26, ws_qty#51, ws_wc#52, ws_sp#53] +Input [12]: [ss_sold_year#23, ss_item_sk#1, ss_customer_sk#2, ss_qty#24, ss_wc#25, ss_sp#26, ws_sold_year#49, ws_item_sk#27, ws_customer_sk#50, ws_qty#51, ws_wc#52, ws_sp#53] + +(46) Scan parquet spark_catalog.default.catalog_sales +Output [7]: [cs_bill_customer_sk#54, cs_item_sk#55, cs_order_number#56, cs_quantity#57, cs_wholesale_cost#58, cs_sales_price#59, cs_sold_date_sk#60] +Batched: true +Location: InMemoryFileIndex [] +PartitionFilters: [isnotnull(cs_sold_date_sk#60), dynamicpruningexpression(cs_sold_date_sk#60 IN dynamicpruning#61)] +PushedFilters: [IsNotNull(cs_item_sk), IsNotNull(cs_bill_customer_sk)] +ReadSchema: struct<cs_bill_customer_sk:int,cs_item_sk:int,cs_order_number:int,cs_quantity:int,cs_wholesale_cost:decimal(7,2),cs_sales_price:decimal(7,2)> + +(47) CometFilter +Input [7]: [cs_bill_customer_sk#54, cs_item_sk#55, cs_order_number#56, cs_quantity#57, cs_wholesale_cost#58, cs_sales_price#59, cs_sold_date_sk#60] +Condition : (isnotnull(cs_item_sk#55) AND isnotnull(cs_bill_customer_sk#54)) + +(48) ColumnarToRow [codegen id : 16] +Input [7]: [cs_bill_customer_sk#54, cs_item_sk#55, cs_order_number#56, cs_quantity#57, cs_wholesale_cost#58, cs_sales_price#59, cs_sold_date_sk#60] + +(49) Exchange +Input [7]: [cs_bill_customer_sk#54, cs_item_sk#55, cs_order_number#56, cs_quantity#57, cs_wholesale_cost#58, cs_sales_price#59, cs_sold_date_sk#60] +Arguments: hashpartitioning(cs_order_number#56, cs_item_sk#55, 5), ENSURE_REQUIREMENTS, [plan_id=7] + +(50) Sort [codegen id : 17] +Input [7]: [cs_bill_customer_sk#54, cs_item_sk#55, cs_order_number#56, cs_quantity#57, cs_wholesale_cost#58, cs_sales_price#59, cs_sold_date_sk#60] +Arguments: [cs_order_number#56 ASC NULLS FIRST, cs_item_sk#55 ASC NULLS FIRST], false, 0 + +(51) Scan parquet spark_catalog.default.catalog_returns +Output [3]: [cr_item_sk#62, cr_order_number#63, cr_returned_date_sk#64] +Batched: true +Location [not included in comparison]/{warehouse_dir}/catalog_returns] +PushedFilters: [IsNotNull(cr_order_number), IsNotNull(cr_item_sk)] +ReadSchema: struct<cr_item_sk:int,cr_order_number:int> + +(52) CometFilter +Input [3]: [cr_item_sk#62, cr_order_number#63, cr_returned_date_sk#64] +Condition : (isnotnull(cr_order_number#63) AND isnotnull(cr_item_sk#62)) + +(53) CometProject +Input [3]: [cr_item_sk#62, cr_order_number#63, cr_returned_date_sk#64] +Arguments: [cr_item_sk#62, cr_order_number#63], [cr_item_sk#62, cr_order_number#63] + +(54) ColumnarToRow [codegen id : 18] +Input [2]: [cr_item_sk#62, cr_order_number#63] + +(55) Exchange +Input [2]: [cr_item_sk#62, cr_order_number#63] +Arguments: hashpartitioning(cr_order_number#63, cr_item_sk#62, 5), ENSURE_REQUIREMENTS, [plan_id=8] + +(56) Sort [codegen id : 19] +Input [2]: [cr_item_sk#62, cr_order_number#63] +Arguments: [cr_order_number#63 ASC NULLS FIRST, cr_item_sk#62 ASC NULLS FIRST], false, 0 + +(57) SortMergeJoin [codegen id : 21] +Left keys [2]: [cs_order_number#56, cs_item_sk#55] +Right keys [2]: [cr_order_number#63, cr_item_sk#62] +Join type: LeftOuter +Join condition: None + +(58) Filter [codegen id : 21] +Input [9]: [cs_bill_customer_sk#54, cs_item_sk#55, cs_order_number#56, cs_quantity#57, cs_wholesale_cost#58, cs_sales_price#59, cs_sold_date_sk#60, cr_item_sk#62, cr_order_number#63] +Condition : isnull(cr_order_number#63) + +(59) Project [codegen id : 21] +Output [6]: [cs_bill_customer_sk#54, cs_item_sk#55, cs_quantity#57, cs_wholesale_cost#58, cs_sales_price#59, cs_sold_date_sk#60] +Input [9]: [cs_bill_customer_sk#54, cs_item_sk#55, cs_order_number#56, cs_quantity#57, cs_wholesale_cost#58, cs_sales_price#59, cs_sold_date_sk#60, cr_item_sk#62, cr_order_number#63] + +(60) ReusedExchange [Reuses operator id: 74] +Output [2]: [d_date_sk#65, d_year#66] + +(61) BroadcastHashJoin [codegen id : 21] +Left keys [1]: [cs_sold_date_sk#60] +Right keys [1]: [d_date_sk#65] +Join type: Inner +Join condition: None + +(62) Project [codegen id : 21] +Output [6]: [cs_bill_customer_sk#54, cs_item_sk#55, cs_quantity#57, cs_wholesale_cost#58, cs_sales_price#59, d_year#66] +Input [8]: [cs_bill_customer_sk#54, cs_item_sk#55, cs_quantity#57, cs_wholesale_cost#58, cs_sales_price#59, cs_sold_date_sk#60, d_date_sk#65, d_year#66] + +(63) HashAggregate [codegen id : 21] +Input [6]: [cs_bill_customer_sk#54, cs_item_sk#55, cs_quantity#57, cs_wholesale_cost#58, cs_sales_price#59, d_year#66] +Keys [3]: [d_year#66, cs_item_sk#55, cs_bill_customer_sk#54] +Functions [3]: [partial_sum(cs_quantity#57), partial_sum(UnscaledValue(cs_wholesale_cost#58)), partial_sum(UnscaledValue(cs_sales_price#59))] +Aggregate Attributes [3]: [sum#67, sum#68, sum#69] +Results [6]: [d_year#66, cs_item_sk#55, cs_bill_customer_sk#54, sum#70, sum#71, sum#72] + +(64) Exchange +Input [6]: [d_year#66, cs_item_sk#55, cs_bill_customer_sk#54, sum#70, sum#71, sum#72] +Arguments: hashpartitioning(d_year#66, cs_item_sk#55, cs_bill_customer_sk#54, 5), ENSURE_REQUIREMENTS, [plan_id=9] + +(65) HashAggregate [codegen id : 22] +Input [6]: [d_year#66, cs_item_sk#55, cs_bill_customer_sk#54, sum#70, sum#71, sum#72] +Keys [3]: [d_year#66, cs_item_sk#55, cs_bill_customer_sk#54] +Functions [3]: [sum(cs_quantity#57), sum(UnscaledValue(cs_wholesale_cost#58)), sum(UnscaledValue(cs_sales_price#59))] +Aggregate Attributes [3]: [sum(cs_quantity#57)#73, sum(UnscaledValue(cs_wholesale_cost#58))#74, sum(UnscaledValue(cs_sales_price#59))#75] +Results [6]: [d_year#66 AS cs_sold_year#76, cs_item_sk#55, cs_bill_customer_sk#54 AS cs_customer_sk#77, sum(cs_quantity#57)#73 AS cs_qty#78, MakeDecimal(sum(UnscaledValue(cs_wholesale_cost#58))#74,17,2) AS cs_wc#79, MakeDecimal(sum(UnscaledValue(cs_sales_price#59))#75,17,2) AS cs_sp#80] + +(66) Filter [codegen id : 22] +Input [6]: [cs_sold_year#76, cs_item_sk#55, cs_customer_sk#77, cs_qty#78, cs_wc#79, cs_sp#80] +Condition : (coalesce(cs_qty#78, 0) > 0) + +(67) Sort [codegen id : 22] +Input [6]: [cs_sold_year#76, cs_item_sk#55, cs_customer_sk#77, cs_qty#78, cs_wc#79, cs_sp#80] +Arguments: [cs_sold_year#76 ASC NULLS FIRST, cs_item_sk#55 ASC NULLS FIRST, cs_customer_sk#77 ASC NULLS FIRST], false, 0 + +(68) SortMergeJoin [codegen id : 23] +Left keys [3]: [ss_sold_year#23, ss_item_sk#1, ss_customer_sk#2] +Right keys [3]: [cs_sold_year#76, cs_item_sk#55, cs_customer_sk#77] +Join type: Inner +Join condition: None + +(69) Project [codegen id : 23] +Output [12]: [round((cast(ss_qty#24 as double) / cast(coalesce((ws_qty#51 + cs_qty#78), 1) as double)), 2) AS ratio#81, ss_qty#24 AS store_qty#82, ss_wc#25 AS store_wholesale_cost#83, ss_sp#26 AS store_sales_price#84, (coalesce(ws_qty#51, 0) + coalesce(cs_qty#78, 0)) AS other_chan_qty#85, (coalesce(ws_wc#52, 0.00) + coalesce(cs_wc#79, 0.00)) AS other_chan_wholesale_cost#86, (coalesce(ws_sp#53, 0.00) + coalesce(cs_sp#80, 0.00)) AS other_chan_sales_price#87, ss_qty#24, ss_wc#25, ss_sp#26, ws_qty#51, cs_qty#78] +Input [15]: [ss_sold_year#23, ss_item_sk#1, ss_customer_sk#2, ss_qty#24, ss_wc#25, ss_sp#26, ws_qty#51, ws_wc#52, ws_sp#53, cs_sold_year#76, cs_item_sk#55, cs_customer_sk#77, cs_qty#78, cs_wc#79, cs_sp#80] + +(70) TakeOrderedAndProject +Input [12]: [ratio#81, store_qty#82, store_wholesale_cost#83, store_sales_price#84, other_chan_qty#85, other_chan_wholesale_cost#86, other_chan_sales_price#87, ss_qty#24, ss_wc#25, ss_sp#26, ws_qty#51, cs_qty#78] +Arguments: 100, [ratio#81 ASC NULLS FIRST, ss_qty#24 DESC NULLS LAST, ss_wc#25 DESC NULLS LAST, ss_sp#26 DESC NULLS LAST, other_chan_qty#85 ASC NULLS FIRST, other_chan_wholesale_cost#86 ASC NULLS FIRST, other_chan_sales_price#87 ASC NULLS FIRST, round((cast(ss_qty#24 as double) / cast(coalesce((ws_qty#51 + cs_qty#78), 1) as double)), 2) ASC NULLS FIRST], [ratio#81, store_qty#82, store_wholesale_cost#83, store_sales_price#84, other_chan_qty#85, other_chan_wholesale_cost#86, other_chan_sales_price#87] + +===== Subqueries ===== + +Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#7 IN dynamicpruning#8 +BroadcastExchange (74) ++- * ColumnarToRow (73) + +- CometFilter (72) + +- CometScan parquet spark_catalog.default.date_dim (71) + + +(71) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#12, d_year#13] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2000), IsNotNull(d_date_sk)] +ReadSchema: struct<d_date_sk:int,d_year:int> + +(72) CometFilter +Input [2]: [d_date_sk#12, d_year#13] +Condition : ((isnotnull(d_year#13) AND (d_year#13 = 2000)) AND isnotnull(d_date_sk#12)) + +(73) ColumnarToRow [codegen id : 1] +Input [2]: [d_date_sk#12, d_year#13] + +(74) BroadcastExchange +Input [2]: [d_date_sk#12, d_year#13] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=10] + +Subquery:2 Hosting operator id = 22 Hosting Expression = ws_sold_date_sk#33 IN dynamicpruning#8 + +Subquery:3 Hosting operator id = 46 Hosting Expression = cs_sold_date_sk#60 IN dynamicpruning#8 + + diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q78/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q78/simplified.txt new file mode 100644 index 000000000..280687e30 --- /dev/null +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q78/simplified.txt @@ -0,0 +1,127 @@ +TakeOrderedAndProject [ratio,ss_qty,ss_wc,ss_sp,other_chan_qty,other_chan_wholesale_cost,other_chan_sales_price,ws_qty,cs_qty,store_qty,store_wholesale_cost,store_sales_price] + WholeStageCodegen (23) + Project [ss_qty,ws_qty,cs_qty,ss_wc,ss_sp,ws_wc,cs_wc,ws_sp,cs_sp] + SortMergeJoin [ss_sold_year,ss_item_sk,ss_customer_sk,cs_sold_year,cs_item_sk,cs_customer_sk] + InputAdapter + WholeStageCodegen (15) + Project [ss_sold_year,ss_item_sk,ss_customer_sk,ss_qty,ss_wc,ss_sp,ws_qty,ws_wc,ws_sp] + SortMergeJoin [ss_sold_year,ss_item_sk,ss_customer_sk,ws_sold_year,ws_item_sk,ws_customer_sk] + InputAdapter + WholeStageCodegen (7) + Sort [ss_sold_year,ss_item_sk,ss_customer_sk] + HashAggregate [d_year,ss_item_sk,ss_customer_sk,sum,sum,sum] [sum(ss_quantity),sum(UnscaledValue(ss_wholesale_cost)),sum(UnscaledValue(ss_sales_price)),ss_sold_year,ss_qty,ss_wc,ss_sp,sum,sum,sum] + InputAdapter + Exchange [d_year,ss_item_sk,ss_customer_sk] #1 + WholeStageCodegen (6) + HashAggregate [d_year,ss_item_sk,ss_customer_sk,ss_quantity,ss_wholesale_cost,ss_sales_price] [sum,sum,sum,sum,sum,sum] + Project [ss_item_sk,ss_customer_sk,ss_quantity,ss_wholesale_cost,ss_sales_price,d_year] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [ss_item_sk,ss_customer_sk,ss_quantity,ss_wholesale_cost,ss_sales_price,ss_sold_date_sk] + Filter [sr_ticket_number] + SortMergeJoin [ss_ticket_number,ss_item_sk,sr_ticket_number,sr_item_sk] + InputAdapter + WholeStageCodegen (2) + Sort [ss_ticket_number,ss_item_sk] + InputAdapter + Exchange [ss_ticket_number,ss_item_sk] #2 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometFilter [ss_item_sk,ss_customer_sk] + CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_quantity,ss_wholesale_cost,ss_sales_price,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometFilter [d_year,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + InputAdapter + WholeStageCodegen (4) + Sort [sr_ticket_number,sr_item_sk] + InputAdapter + Exchange [sr_ticket_number,sr_item_sk] #4 + WholeStageCodegen (3) + ColumnarToRow + InputAdapter + CometProject [sr_item_sk,sr_ticket_number] + CometFilter [sr_ticket_number,sr_item_sk] + CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk] + InputAdapter + ReusedExchange [d_date_sk,d_year] #3 + InputAdapter + WholeStageCodegen (14) + Sort [ws_sold_year,ws_item_sk,ws_customer_sk] + Filter [ws_qty] + HashAggregate [d_year,ws_item_sk,ws_bill_customer_sk,sum,sum,sum] [sum(ws_quantity),sum(UnscaledValue(ws_wholesale_cost)),sum(UnscaledValue(ws_sales_price)),ws_sold_year,ws_customer_sk,ws_qty,ws_wc,ws_sp,sum,sum,sum] + InputAdapter + Exchange [d_year,ws_item_sk,ws_bill_customer_sk] #5 + WholeStageCodegen (13) + HashAggregate [d_year,ws_item_sk,ws_bill_customer_sk,ws_quantity,ws_wholesale_cost,ws_sales_price] [sum,sum,sum,sum,sum,sum] + Project [ws_item_sk,ws_bill_customer_sk,ws_quantity,ws_wholesale_cost,ws_sales_price,d_year] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + Project [ws_item_sk,ws_bill_customer_sk,ws_quantity,ws_wholesale_cost,ws_sales_price,ws_sold_date_sk] + Filter [wr_order_number] + SortMergeJoin [ws_order_number,ws_item_sk,wr_order_number,wr_item_sk] + InputAdapter + WholeStageCodegen (9) + Sort [ws_order_number,ws_item_sk] + InputAdapter + Exchange [ws_order_number,ws_item_sk] #6 + WholeStageCodegen (8) + ColumnarToRow + InputAdapter + CometFilter [ws_item_sk,ws_bill_customer_sk] + CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_bill_customer_sk,ws_order_number,ws_quantity,ws_wholesale_cost,ws_sales_price,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + InputAdapter + WholeStageCodegen (11) + Sort [wr_order_number,wr_item_sk] + InputAdapter + Exchange [wr_order_number,wr_item_sk] #7 + WholeStageCodegen (10) + ColumnarToRow + InputAdapter + CometProject [wr_item_sk,wr_order_number] + CometFilter [wr_order_number,wr_item_sk] + CometScan parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_returned_date_sk] + InputAdapter + ReusedExchange [d_date_sk,d_year] #3 + InputAdapter + WholeStageCodegen (22) + Sort [cs_sold_year,cs_item_sk,cs_customer_sk] + Filter [cs_qty] + HashAggregate [d_year,cs_item_sk,cs_bill_customer_sk,sum,sum,sum] [sum(cs_quantity),sum(UnscaledValue(cs_wholesale_cost)),sum(UnscaledValue(cs_sales_price)),cs_sold_year,cs_customer_sk,cs_qty,cs_wc,cs_sp,sum,sum,sum] + InputAdapter + Exchange [d_year,cs_item_sk,cs_bill_customer_sk] #8 + WholeStageCodegen (21) + HashAggregate [d_year,cs_item_sk,cs_bill_customer_sk,cs_quantity,cs_wholesale_cost,cs_sales_price] [sum,sum,sum,sum,sum,sum] + Project [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_wholesale_cost,cs_sales_price,d_year] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + Project [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_wholesale_cost,cs_sales_price,cs_sold_date_sk] + Filter [cr_order_number] + SortMergeJoin [cs_order_number,cs_item_sk,cr_order_number,cr_item_sk] + InputAdapter + WholeStageCodegen (17) + Sort [cs_order_number,cs_item_sk] + InputAdapter + Exchange [cs_order_number,cs_item_sk] #9 + WholeStageCodegen (16) + ColumnarToRow + InputAdapter + CometFilter [cs_item_sk,cs_bill_customer_sk] + CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_item_sk,cs_order_number,cs_quantity,cs_wholesale_cost,cs_sales_price,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + InputAdapter + WholeStageCodegen (19) + Sort [cr_order_number,cr_item_sk] + InputAdapter + Exchange [cr_order_number,cr_item_sk] #10 + WholeStageCodegen (18) + ColumnarToRow + InputAdapter + CometProject [cr_item_sk,cr_order_number] + CometFilter [cr_order_number,cr_item_sk] + CometScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_returned_date_sk] + InputAdapter + ReusedExchange [d_date_sk,d_year] #3 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 new file mode 100644 index 000000000..c89bad220 --- /dev/null +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q79/explain.txt @@ -0,0 +1,208 @@ +== Physical Plan == +TakeOrderedAndProject (30) ++- * Project (29) + +- * BroadcastHashJoin Inner BuildRight (28) + :- * HashAggregate (23) + : +- Exchange (22) + : +- * HashAggregate (21) + : +- * Project (20) + : +- * BroadcastHashJoin Inner BuildRight (19) + : :- * Project (13) + : : +- * BroadcastHashJoin Inner BuildRight (12) + : : :- * Project (6) + : : : +- * BroadcastHashJoin Inner BuildRight (5) + : : : :- * ColumnarToRow (3) + : : : : +- CometFilter (2) + : : : : +- CometScan parquet spark_catalog.default.store_sales (1) + : : : +- ReusedExchange (4) + : : +- BroadcastExchange (11) + : : +- * ColumnarToRow (10) + : : +- CometProject (9) + : : +- CometFilter (8) + : : +- CometScan parquet spark_catalog.default.store (7) + : +- BroadcastExchange (18) + : +- * ColumnarToRow (17) + : +- CometProject (16) + : +- CometFilter (15) + : +- CometScan parquet spark_catalog.default.household_demographics (14) + +- BroadcastExchange (27) + +- * ColumnarToRow (26) + +- CometFilter (25) + +- CometScan parquet spark_catalog.default.customer (24) + + +(1) Scan parquet spark_catalog.default.store_sales +Output [8]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_store_sk#4, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7, ss_sold_date_sk#8] +Batched: true +Location: InMemoryFileIndex [] +PartitionFilters: [isnotnull(ss_sold_date_sk#8), dynamicpruningexpression(ss_sold_date_sk#8 IN dynamicpruning#9)] +PushedFilters: [IsNotNull(ss_store_sk), IsNotNull(ss_hdemo_sk), IsNotNull(ss_customer_sk)] +ReadSchema: struct<ss_customer_sk:int,ss_hdemo_sk:int,ss_addr_sk:int,ss_store_sk:int,ss_ticket_number:int,ss_coupon_amt:decimal(7,2),ss_net_profit:decimal(7,2)> + +(2) CometFilter +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)) + +(3) ColumnarToRow [codegen id : 4] +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] + +(4) ReusedExchange [Reuses operator id: 35] +Output [1]: [d_date_sk#10] + +(5) BroadcastHashJoin [codegen id : 4] +Left keys [1]: [ss_sold_date_sk#8] +Right keys [1]: [d_date_sk#10] +Join type: Inner +Join condition: None + +(6) Project [codegen id : 4] +Output [7]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_store_sk#4, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7] +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] + +(7) Scan parquet spark_catalog.default.store +Output [3]: [s_store_sk#11, s_number_employees#12, s_city#13] +Batched: true +Location [not included in comparison]/{warehouse_dir}/store] +PushedFilters: [IsNotNull(s_number_employees), GreaterThanOrEqual(s_number_employees,200), LessThanOrEqual(s_number_employees,295), IsNotNull(s_store_sk)] +ReadSchema: struct<s_store_sk:int,s_number_employees:int,s_city:string> + +(8) CometFilter +Input [3]: [s_store_sk#11, s_number_employees#12, s_city#13] +Condition : (((isnotnull(s_number_employees#12) AND (s_number_employees#12 >= 200)) AND (s_number_employees#12 <= 295)) AND isnotnull(s_store_sk#11)) + +(9) CometProject +Input [3]: [s_store_sk#11, s_number_employees#12, s_city#13] +Arguments: [s_store_sk#11, s_city#13], [s_store_sk#11, s_city#13] + +(10) ColumnarToRow [codegen id : 2] +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] +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] +Output [7]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7, s_city#13] +Input [9]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_store_sk#4, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7, s_store_sk#11, s_city#13] + +(14) Scan parquet spark_catalog.default.household_demographics +Output [3]: [hd_demo_sk#14, hd_dep_count#15, hd_vehicle_count#16] +Batched: true +Location [not included in comparison]/{warehouse_dir}/household_demographics] +PushedFilters: [Or(EqualTo(hd_dep_count,6),GreaterThan(hd_vehicle_count,2)), IsNotNull(hd_demo_sk)] +ReadSchema: struct<hd_demo_sk:int,hd_dep_count:int,hd_vehicle_count:int> + +(15) CometFilter +Input [3]: [hd_demo_sk#14, hd_dep_count#15, hd_vehicle_count#16] +Condition : (((hd_dep_count#15 = 6) OR (hd_vehicle_count#16 > 2)) AND isnotnull(hd_demo_sk#14)) + +(16) CometProject +Input [3]: [hd_demo_sk#14, hd_dep_count#15, hd_vehicle_count#16] +Arguments: [hd_demo_sk#14], [hd_demo_sk#14] + +(17) ColumnarToRow [codegen id : 3] +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] +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] +Output [6]: [ss_customer_sk#1, ss_addr_sk#3, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7, s_city#13] +Input [8]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_addr_sk#3, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7, s_city#13, hd_demo_sk#14] + +(21) HashAggregate [codegen id : 4] +Input [6]: [ss_customer_sk#1, ss_addr_sk#3, ss_ticket_number#5, ss_coupon_amt#6, ss_net_profit#7, s_city#13] +Keys [4]: [ss_ticket_number#5, ss_customer_sk#1, ss_addr_sk#3, s_city#13] +Functions [2]: [partial_sum(UnscaledValue(ss_coupon_amt#6)), partial_sum(UnscaledValue(ss_net_profit#7))] +Aggregate Attributes [2]: [sum#17, sum#18] +Results [6]: [ss_ticket_number#5, ss_customer_sk#1, ss_addr_sk#3, s_city#13, sum#19, sum#20] + +(22) Exchange +Input [6]: [ss_ticket_number#5, ss_customer_sk#1, ss_addr_sk#3, s_city#13, sum#19, sum#20] +Arguments: hashpartitioning(ss_ticket_number#5, ss_customer_sk#1, ss_addr_sk#3, s_city#13, 5), ENSURE_REQUIREMENTS, [plan_id=3] + +(23) HashAggregate [codegen id : 6] +Input [6]: [ss_ticket_number#5, ss_customer_sk#1, ss_addr_sk#3, s_city#13, sum#19, sum#20] +Keys [4]: [ss_ticket_number#5, ss_customer_sk#1, ss_addr_sk#3, s_city#13] +Functions [2]: [sum(UnscaledValue(ss_coupon_amt#6)), sum(UnscaledValue(ss_net_profit#7))] +Aggregate Attributes [2]: [sum(UnscaledValue(ss_coupon_amt#6))#21, sum(UnscaledValue(ss_net_profit#7))#22] +Results [5]: [ss_ticket_number#5, ss_customer_sk#1, s_city#13, MakeDecimal(sum(UnscaledValue(ss_coupon_amt#6))#21,17,2) AS amt#23, MakeDecimal(sum(UnscaledValue(ss_net_profit#7))#22,17,2) AS profit#24] + +(24) Scan parquet spark_catalog.default.customer +Output [3]: [c_customer_sk#25, c_first_name#26, c_last_name#27] +Batched: true +Location [not included in comparison]/{warehouse_dir}/customer] +PushedFilters: [IsNotNull(c_customer_sk)] +ReadSchema: struct<c_customer_sk:int,c_first_name:string,c_last_name:string> + +(25) CometFilter +Input [3]: [c_customer_sk#25, c_first_name#26, c_last_name#27] +Condition : isnotnull(c_customer_sk#25) + +(26) ColumnarToRow [codegen id : 5] +Input [3]: [c_customer_sk#25, c_first_name#26, c_last_name#27] + +(27) BroadcastExchange +Input [3]: [c_customer_sk#25, c_first_name#26, c_last_name#27] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=4] + +(28) BroadcastHashJoin [codegen id : 6] +Left keys [1]: [ss_customer_sk#1] +Right keys [1]: [c_customer_sk#25] +Join type: Inner +Join condition: None + +(29) Project [codegen id : 6] +Output [7]: [c_last_name#27, c_first_name#26, substr(s_city#13, 1, 30) AS substr(s_city, 1, 30)#28, ss_ticket_number#5, amt#23, profit#24, s_city#13] +Input [8]: [ss_ticket_number#5, ss_customer_sk#1, s_city#13, amt#23, profit#24, c_customer_sk#25, c_first_name#26, c_last_name#27] + +(30) TakeOrderedAndProject +Input [7]: [c_last_name#27, c_first_name#26, substr(s_city, 1, 30)#28, ss_ticket_number#5, amt#23, profit#24, s_city#13] +Arguments: 100, [c_last_name#27 ASC NULLS FIRST, c_first_name#26 ASC NULLS FIRST, substr(s_city#13, 1, 30) ASC NULLS FIRST, profit#24 ASC NULLS FIRST], [c_last_name#27, c_first_name#26, substr(s_city, 1, 30)#28, ss_ticket_number#5, amt#23, profit#24] + +===== Subqueries ===== + +Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#8 IN dynamicpruning#9 +BroadcastExchange (35) ++- * ColumnarToRow (34) + +- CometProject (33) + +- CometFilter (32) + +- CometScan parquet spark_catalog.default.date_dim (31) + + +(31) Scan parquet spark_catalog.default.date_dim +Output [3]: [d_date_sk#10, d_year#29, d_dow#30] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_dow), EqualTo(d_dow,1), In(d_year, [1999,2000,2001]), IsNotNull(d_date_sk)] +ReadSchema: struct<d_date_sk:int,d_year:int,d_dow:int> + +(32) CometFilter +Input [3]: [d_date_sk#10, d_year#29, d_dow#30] +Condition : (((isnotnull(d_dow#30) AND (d_dow#30 = 1)) AND d_year#29 IN (1999,2000,2001)) AND isnotnull(d_date_sk#10)) + +(33) CometProject +Input [3]: [d_date_sk#10, d_year#29, d_dow#30] +Arguments: [d_date_sk#10], [d_date_sk#10] + +(34) ColumnarToRow [codegen id : 1] +Input [1]: [d_date_sk#10] + +(35) BroadcastExchange +Input [1]: [d_date_sk#10] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] + + diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q79/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q79/simplified.txt new file mode 100644 index 000000000..4c05c449c --- /dev/null +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q79/simplified.txt @@ -0,0 +1,52 @@ +TakeOrderedAndProject [c_last_name,c_first_name,s_city,profit,substr(s_city, 1, 30),ss_ticket_number,amt] + WholeStageCodegen (6) + Project [c_last_name,c_first_name,s_city,ss_ticket_number,amt,profit] + BroadcastHashJoin [ss_customer_sk,c_customer_sk] + HashAggregate [ss_ticket_number,ss_customer_sk,ss_addr_sk,s_city,sum,sum] [sum(UnscaledValue(ss_coupon_amt)),sum(UnscaledValue(ss_net_profit)),amt,profit,sum,sum] + InputAdapter + Exchange [ss_ticket_number,ss_customer_sk,ss_addr_sk,s_city] #1 + WholeStageCodegen (4) + HashAggregate [ss_ticket_number,ss_customer_sk,ss_addr_sk,s_city,ss_coupon_amt,ss_net_profit] [sum,sum,sum,sum] + Project [ss_customer_sk,ss_addr_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit,s_city] + BroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] + Project [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit,s_city] + BroadcastHashJoin [ss_store_sk,s_store_sk] + Project [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + ColumnarToRow + InputAdapter + CometFilter [ss_store_sk,ss_hdemo_sk,ss_customer_sk] + CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_ticket_number,ss_coupon_amt,ss_net_profit,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #2 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_dow,d_year,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dow] + InputAdapter + ReusedExchange [d_date_sk] #2 + InputAdapter + BroadcastExchange #3 + WholeStageCodegen (2) + ColumnarToRow + InputAdapter + CometProject [s_store_sk,s_city] + CometFilter [s_number_employees,s_store_sk] + CometScan parquet spark_catalog.default.store [s_store_sk,s_number_employees,s_city] + InputAdapter + BroadcastExchange #4 + WholeStageCodegen (3) + ColumnarToRow + InputAdapter + CometProject [hd_demo_sk] + CometFilter [hd_dep_count,hd_vehicle_count,hd_demo_sk] + CometScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_dep_count,hd_vehicle_count] + InputAdapter + BroadcastExchange #5 + WholeStageCodegen (5) + ColumnarToRow + InputAdapter + CometFilter [c_customer_sk] + CometScan parquet spark_catalog.default.customer [c_customer_sk,c_first_name,c_last_name] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q8/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q8/explain.txt new file mode 100644 index 000000000..7d6d717c1 --- /dev/null +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q8/explain.txt @@ -0,0 +1,288 @@ +== Physical Plan == +TakeOrderedAndProject (43) ++- * HashAggregate (42) + +- Exchange (41) + +- * HashAggregate (40) + +- * Project (39) + +- * BroadcastHashJoin Inner BuildRight (38) + :- * Project (12) + : +- * BroadcastHashJoin Inner BuildRight (11) + : :- * Project (6) + : : +- * BroadcastHashJoin Inner BuildRight (5) + : : :- * ColumnarToRow (3) + : : : +- CometFilter (2) + : : : +- CometScan parquet spark_catalog.default.store_sales (1) + : : +- ReusedExchange (4) + : +- BroadcastExchange (10) + : +- * ColumnarToRow (9) + : +- CometFilter (8) + : +- CometScan parquet spark_catalog.default.store (7) + +- BroadcastExchange (37) + +- * HashAggregate (36) + +- Exchange (35) + +- * HashAggregate (34) + +- * BroadcastHashJoin LeftSemi BuildRight (33) + :- * ColumnarToRow (16) + : +- CometProject (15) + : +- CometFilter (14) + : +- CometScan parquet spark_catalog.default.customer_address (13) + +- BroadcastExchange (32) + +- * Project (31) + +- * Filter (30) + +- * HashAggregate (29) + +- Exchange (28) + +- * HashAggregate (27) + +- * Project (26) + +- * BroadcastHashJoin Inner BuildRight (25) + :- * ColumnarToRow (19) + : +- CometFilter (18) + : +- CometScan parquet spark_catalog.default.customer_address (17) + +- BroadcastExchange (24) + +- * ColumnarToRow (23) + +- CometProject (22) + +- CometFilter (21) + +- CometScan parquet spark_catalog.default.customer (20) + + +(1) Scan parquet spark_catalog.default.store_sales +Output [3]: [ss_store_sk#1, ss_net_profit#2, ss_sold_date_sk#3] +Batched: true +Location: InMemoryFileIndex [] +PartitionFilters: [isnotnull(ss_sold_date_sk#3), dynamicpruningexpression(ss_sold_date_sk#3 IN dynamicpruning#4)] +PushedFilters: [IsNotNull(ss_store_sk)] +ReadSchema: struct<ss_store_sk:int,ss_net_profit:decimal(7,2)> + +(2) CometFilter +Input [3]: [ss_store_sk#1, ss_net_profit#2, ss_sold_date_sk#3] +Condition : isnotnull(ss_store_sk#1) + +(3) ColumnarToRow [codegen id : 8] +Input [3]: [ss_store_sk#1, ss_net_profit#2, ss_sold_date_sk#3] + +(4) ReusedExchange [Reuses operator id: 48] +Output [1]: [d_date_sk#5] + +(5) BroadcastHashJoin [codegen id : 8] +Left keys [1]: [ss_sold_date_sk#3] +Right keys [1]: [d_date_sk#5] +Join type: Inner +Join condition: None + +(6) Project [codegen id : 8] +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] + +(7) Scan parquet spark_catalog.default.store +Output [3]: [s_store_sk#6, s_store_name#7, s_zip#8] +Batched: true +Location [not included in comparison]/{warehouse_dir}/store] +PushedFilters: [IsNotNull(s_store_sk), IsNotNull(s_zip)] +ReadSchema: struct<s_store_sk:int,s_store_name:string,s_zip:string> + +(8) CometFilter +Input [3]: [s_store_sk#6, s_store_name#7, s_zip#8] +Condition : (isnotnull(s_store_sk#6) AND isnotnull(s_zip#8)) + +(9) ColumnarToRow [codegen id : 2] +Input [3]: [s_store_sk#6, s_store_name#7, s_zip#8] + +(10) BroadcastExchange +Input [3]: [s_store_sk#6, s_store_name#7, s_zip#8] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=1] + +(11) BroadcastHashJoin [codegen id : 8] +Left keys [1]: [ss_store_sk#1] +Right keys [1]: [s_store_sk#6] +Join type: Inner +Join condition: None + +(12) Project [codegen id : 8] +Output [3]: [ss_net_profit#2, s_store_name#7, s_zip#8] +Input [5]: [ss_store_sk#1, ss_net_profit#2, s_store_sk#6, s_store_name#7, s_zip#8] + +(13) Scan parquet spark_catalog.default.customer_address +Output [1]: [ca_zip#9] +Batched: true +Location [not included in comparison]/{warehouse_dir}/customer_address] +ReadSchema: struct<ca_zip:string> + +(14) CometFilter +Input [1]: [ca_zip#9] +Condition : (substr(ca_zip#9, 1, 5) INSET 10144, 10336, 10390, 10445, 10516, 10567, 11101, 11356, 11376, 11489, 11634, 11928, 12305, 13354, 13375, 13376, 13394, 13595, 13695, 13955, 14060, 14089, 14171, 14328, 14663, 14867, 14922, 15126, 15146, 15371, 15455, 15559, 15723, 15734, 15765, 15798, 15882, 16021, 16725, 16807, 17043, 17183, 17871, 17879, 17920, 18119, 18270, 18376, 18383, 18426, 18652, 18767, 18799, 18840, 18842, 18845, 18906, 19430, 19505, 19512, 19515, 19736, 19769, 19849, 20004, 20260, 20548, 21076, 21195, 21286, 21309, 21337, 21756, 22152, 22245, 22246, 22351, 22437, 22461, 22685, 22744, 22752, 22927, 23006, 23470, 23932, 23968, 24128, 24206, 24317, 24610, 24671, 24676, 24996, 25003, 25103, 25280, 25486, 25631, 25733, 25782, 25858, 25989, 26065, 26105, 26231, 26233, 26653, 26689, 26859, 27068, 27156, 27385, 27700, 28286, 28488, 28545, 28577, 28587, 28709, 28810, 28898, 28915, 29178, 29741, 29839, 30010, 30122, 30431, 30450, 30469, 30625, 30903, 31016, 31029, 31387, 31671, 31880, 32213, 32754, 33123, 33282, 33515, 33786, 34102, 34322, 34425, 35258, 35458, 35474, 35576, 35850, 35942, 36233, 36420, 36446, 36495, 36634, 37125, 37126, 37930, 38122, 38193, 38415, 38607, 38935, 39127, 39192, 39371, 39516, 39736, 39861, 39972, 40081, 40162, 40558, 40604, 41248, 41367, 41368, 41766, 41918, 42029, 42666, 42961, 43285, 43848, 43933, 44165, 44438, 45200, 45266, 45375, 45549, 45692, 45721, 45748, 46081, 46136, 46820, 47305, 47537, 47770, 48033, 48425, 48583, 49130, 49156, 49448, 50016, 50298, 50308, 50412, 51061, 51103, 51200, 51211, 51622, 51649, 51650, 51798, 51949, 52867, 53179, 53268, 53535, 53672, 54364, 54601, 54917, 55253, 55307, 55565, 56240, 56458, 56529, 56571, 56575, 56616, 56691, 56910, 57047, 57647, 57665, 57834, 57855, 58048, 58058, 58078, 58263, 58470, 58943, 59166, 59402, 60099, 60279, 60576, 61265, 61547, 61810, 61860, 62377, 62496, 62878, 62971, 63089, 63193, 63435, 63792, 63837, 63981, 64034, 64147, 64457, 64528, 64544, 65084, 65164, 66162, 66708, 66864, 67030, 67301, 67467, 67473, 67853, 67875, 67897, 68014, 68100, 68101, 68309, 68341, 68621, 68786, 68806, 68880, 68893, 68908, 69035, 69399, 69913, 69952, 70372, 70466, 70738, 71256, 71286, 71791, 71954, 72013, 72151, 72175, 72305, 72325, 72425, 72550, 72823, 73134, 73171, 73241, 73273, 73520, 73650, 74351, 75691, 76107, 76231, 76232, 76614, 76638, 76698, 77191, 77556, 77610, 77721, 78451, 78567, 78668, 78890, 79077, 79777, 79994, 81019, 81096, 81312, 81426, 82136, 82276, 82636, 83041, 83144, 83444, 83849, 83921, 83926, 83933, 84093, 84935, 85816, 86057, 86198, 86284, 86379, 87343, 87501, 87816, 88086, 88190, 88424, 88885, 89091, 89360, 90225, 90257, 90578, 91068, 91110, 91137, 91393, 92712, 94167, 94627, 94898, 94945, 94983, 96451, 96576, 96765, 96888, 96976, 97189, 97789, 98025, 98235, 98294, 98359, 98569, 99076, 99543 AND isnotnull(substr(ca_zip#9, 1, 5))) + +(15) CometProject +Input [1]: [ca_zip#9] +Arguments: [ca_zip#10], [substr(ca_zip#9, 1, 5) AS ca_zip#10] + +(16) ColumnarToRow [codegen id : 6] +Input [1]: [ca_zip#10] + +(17) Scan parquet spark_catalog.default.customer_address +Output [2]: [ca_address_sk#11, ca_zip#12] +Batched: true +Location [not included in comparison]/{warehouse_dir}/customer_address] +PushedFilters: [IsNotNull(ca_address_sk)] +ReadSchema: struct<ca_address_sk:int,ca_zip:string> + +(18) CometFilter +Input [2]: [ca_address_sk#11, ca_zip#12] +Condition : isnotnull(ca_address_sk#11) + +(19) ColumnarToRow [codegen id : 4] +Input [2]: [ca_address_sk#11, ca_zip#12] + +(20) Scan parquet spark_catalog.default.customer +Output [2]: [c_current_addr_sk#13, c_preferred_cust_flag#14] +Batched: true +Location [not included in comparison]/{warehouse_dir}/customer] +PushedFilters: [IsNotNull(c_preferred_cust_flag), EqualTo(c_preferred_cust_flag,Y), IsNotNull(c_current_addr_sk)] +ReadSchema: struct<c_current_addr_sk:int,c_preferred_cust_flag:string> + +(21) CometFilter +Input [2]: [c_current_addr_sk#13, c_preferred_cust_flag#14] +Condition : ((isnotnull(c_preferred_cust_flag#14) AND (c_preferred_cust_flag#14 = Y)) AND isnotnull(c_current_addr_sk#13)) + +(22) CometProject +Input [2]: [c_current_addr_sk#13, c_preferred_cust_flag#14] +Arguments: [c_current_addr_sk#13], [c_current_addr_sk#13] + +(23) ColumnarToRow [codegen id : 3] +Input [1]: [c_current_addr_sk#13] + +(24) BroadcastExchange +Input [1]: [c_current_addr_sk#13] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] + +(25) BroadcastHashJoin [codegen id : 4] +Left keys [1]: [ca_address_sk#11] +Right keys [1]: [c_current_addr_sk#13] +Join type: Inner +Join condition: None + +(26) Project [codegen id : 4] +Output [1]: [ca_zip#12] +Input [3]: [ca_address_sk#11, ca_zip#12, c_current_addr_sk#13] + +(27) HashAggregate [codegen id : 4] +Input [1]: [ca_zip#12] +Keys [1]: [ca_zip#12] +Functions [1]: [partial_count(1)] +Aggregate Attributes [1]: [count#15] +Results [2]: [ca_zip#12, count#16] + +(28) Exchange +Input [2]: [ca_zip#12, count#16] +Arguments: hashpartitioning(ca_zip#12, 5), ENSURE_REQUIREMENTS, [plan_id=3] + +(29) HashAggregate [codegen id : 5] +Input [2]: [ca_zip#12, count#16] +Keys [1]: [ca_zip#12] +Functions [1]: [count(1)] +Aggregate Attributes [1]: [count(1)#17] +Results [2]: [substr(ca_zip#12, 1, 5) AS ca_zip#18, count(1)#17 AS cnt#19] + +(30) Filter [codegen id : 5] +Input [2]: [ca_zip#18, cnt#19] +Condition : (cnt#19 > 10) + +(31) Project [codegen id : 5] +Output [1]: [ca_zip#18] +Input [2]: [ca_zip#18, cnt#19] + +(32) BroadcastExchange +Input [1]: [ca_zip#18] +Arguments: HashedRelationBroadcastMode(List(coalesce(input[0, string, true], ), isnull(input[0, string, true])),false), [plan_id=4] + +(33) BroadcastHashJoin [codegen id : 6] +Left keys [2]: [coalesce(ca_zip#10, ), isnull(ca_zip#10)] +Right keys [2]: [coalesce(ca_zip#18, ), isnull(ca_zip#18)] +Join type: LeftSemi +Join condition: None + +(34) HashAggregate [codegen id : 6] +Input [1]: [ca_zip#10] +Keys [1]: [ca_zip#10] +Functions: [] +Aggregate Attributes: [] +Results [1]: [ca_zip#10] + +(35) Exchange +Input [1]: [ca_zip#10] +Arguments: hashpartitioning(ca_zip#10, 5), ENSURE_REQUIREMENTS, [plan_id=5] + +(36) HashAggregate [codegen id : 7] +Input [1]: [ca_zip#10] +Keys [1]: [ca_zip#10] +Functions: [] +Aggregate Attributes: [] +Results [1]: [ca_zip#10] + +(37) BroadcastExchange +Input [1]: [ca_zip#10] +Arguments: HashedRelationBroadcastMode(List(substr(input[0, string, true], 1, 2)),false), [plan_id=6] + +(38) BroadcastHashJoin [codegen id : 8] +Left keys [1]: [substr(s_zip#8, 1, 2)] +Right keys [1]: [substr(ca_zip#10, 1, 2)] +Join type: Inner +Join condition: None + +(39) Project [codegen id : 8] +Output [2]: [ss_net_profit#2, s_store_name#7] +Input [4]: [ss_net_profit#2, s_store_name#7, s_zip#8, ca_zip#10] + +(40) HashAggregate [codegen id : 8] +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))] +Aggregate Attributes [1]: [sum#20] +Results [2]: [s_store_name#7, sum#21] + +(41) Exchange +Input [2]: [s_store_name#7, sum#21] +Arguments: hashpartitioning(s_store_name#7, 5), ENSURE_REQUIREMENTS, [plan_id=7] + +(42) HashAggregate [codegen id : 9] +Input [2]: [s_store_name#7, sum#21] +Keys [1]: [s_store_name#7] +Functions [1]: [sum(UnscaledValue(ss_net_profit#2))] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_profit#2))#22] +Results [2]: [s_store_name#7, MakeDecimal(sum(UnscaledValue(ss_net_profit#2))#22,17,2) AS sum(ss_net_profit)#23] + +(43) TakeOrderedAndProject +Input [2]: [s_store_name#7, sum(ss_net_profit)#23] +Arguments: 100, [s_store_name#7 ASC NULLS FIRST], [s_store_name#7, sum(ss_net_profit)#23] + +===== Subqueries ===== + +Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#3 IN dynamicpruning#4 +BroadcastExchange (48) ++- * ColumnarToRow (47) + +- CometProject (46) + +- CometFilter (45) + +- CometScan parquet spark_catalog.default.date_dim (44) + + +(44) Scan parquet spark_catalog.default.date_dim +Output [3]: [d_date_sk#5, d_year#24, d_qoy#25] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_qoy), IsNotNull(d_year), EqualTo(d_qoy,2), EqualTo(d_year,1998), IsNotNull(d_date_sk)] +ReadSchema: struct<d_date_sk:int,d_year:int,d_qoy:int> + +(45) CometFilter +Input [3]: [d_date_sk#5, d_year#24, d_qoy#25] +Condition : ((((isnotnull(d_qoy#25) AND isnotnull(d_year#24)) AND (d_qoy#25 = 2)) AND (d_year#24 = 1998)) AND isnotnull(d_date_sk#5)) + +(46) CometProject +Input [3]: [d_date_sk#5, d_year#24, d_qoy#25] +Arguments: [d_date_sk#5], [d_date_sk#5] + +(47) ColumnarToRow [codegen id : 1] +Input [1]: [d_date_sk#5] + +(48) BroadcastExchange +Input [1]: [d_date_sk#5] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=8] + + diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q8/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q8/simplified.txt new file mode 100644 index 000000000..76fa27693 --- /dev/null +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q8/simplified.txt @@ -0,0 +1,72 @@ +TakeOrderedAndProject [s_store_name,sum(ss_net_profit)] + WholeStageCodegen (9) + HashAggregate [s_store_name,sum] [sum(UnscaledValue(ss_net_profit)),sum(ss_net_profit),sum] + InputAdapter + Exchange [s_store_name] #1 + WholeStageCodegen (8) + HashAggregate [s_store_name,ss_net_profit] [sum,sum] + Project [ss_net_profit,s_store_name] + BroadcastHashJoin [s_zip,ca_zip] + Project [ss_net_profit,s_store_name,s_zip] + BroadcastHashJoin [ss_store_sk,s_store_sk] + Project [ss_store_sk,ss_net_profit] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + ColumnarToRow + InputAdapter + CometFilter [ss_store_sk] + CometScan parquet spark_catalog.default.store_sales [ss_store_sk,ss_net_profit,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #2 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_qoy,d_year,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] + InputAdapter + ReusedExchange [d_date_sk] #2 + InputAdapter + BroadcastExchange #3 + WholeStageCodegen (2) + ColumnarToRow + InputAdapter + CometFilter [s_store_sk,s_zip] + CometScan parquet spark_catalog.default.store [s_store_sk,s_store_name,s_zip] + InputAdapter + BroadcastExchange #4 + WholeStageCodegen (7) + HashAggregate [ca_zip] + InputAdapter + Exchange [ca_zip] #5 + WholeStageCodegen (6) + HashAggregate [ca_zip] + BroadcastHashJoin [ca_zip,ca_zip] + ColumnarToRow + InputAdapter + CometProject [ca_zip] [ca_zip] + CometFilter [ca_zip] + CometScan parquet spark_catalog.default.customer_address [ca_zip] + InputAdapter + BroadcastExchange #6 + WholeStageCodegen (5) + Project [ca_zip] + Filter [cnt] + HashAggregate [ca_zip,count] [count(1),ca_zip,cnt,count] + InputAdapter + Exchange [ca_zip] #7 + WholeStageCodegen (4) + HashAggregate [ca_zip] [count,count] + Project [ca_zip] + BroadcastHashJoin [ca_address_sk,c_current_addr_sk] + ColumnarToRow + InputAdapter + CometFilter [ca_address_sk] + CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_zip] + InputAdapter + BroadcastExchange #8 + WholeStageCodegen (3) + ColumnarToRow + InputAdapter + CometProject [c_current_addr_sk] + CometFilter [c_preferred_cust_flag,c_current_addr_sk] + CometScan parquet spark_catalog.default.customer [c_current_addr_sk,c_preferred_cust_flag] 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 new file mode 100644 index 000000000..db2d015db --- /dev/null +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q80/explain.txt @@ -0,0 +1,645 @@ +== Physical Plan == +TakeOrderedAndProject (107) ++- * HashAggregate (106) + +- Exchange (105) + +- * HashAggregate (104) + +- * Expand (103) + +- Union (102) + :- * HashAggregate (39) + : +- Exchange (38) + : +- * HashAggregate (37) + : +- * Project (36) + : +- * BroadcastHashJoin Inner BuildRight (35) + : :- * Project (29) + : : +- * BroadcastHashJoin Inner BuildRight (28) + : : :- * Project (22) + : : : +- * BroadcastHashJoin Inner BuildRight (21) + : : : :- * Project (16) + : : : : +- * BroadcastHashJoin Inner BuildRight (15) + : : : : :- * Project (13) + : : : : : +- * SortMergeJoin LeftOuter (12) + : : : : : :- * Sort (5) + : : : : : : +- Exchange (4) + : : : : : : +- * ColumnarToRow (3) + : : : : : : +- CometFilter (2) + : : : : : : +- CometScan parquet spark_catalog.default.store_sales (1) + : : : : : +- * Sort (11) + : : : : : +- Exchange (10) + : : : : : +- * ColumnarToRow (9) + : : : : : +- CometProject (8) + : : : : : +- CometFilter (7) + : : : : : +- CometScan parquet spark_catalog.default.store_returns (6) + : : : : +- ReusedExchange (14) + : : : +- BroadcastExchange (20) + : : : +- * ColumnarToRow (19) + : : : +- CometFilter (18) + : : : +- CometScan parquet spark_catalog.default.store (17) + : : +- BroadcastExchange (27) + : : +- * ColumnarToRow (26) + : : +- CometProject (25) + : : +- CometFilter (24) + : : +- CometScan parquet spark_catalog.default.item (23) + : +- BroadcastExchange (34) + : +- * ColumnarToRow (33) + : +- CometProject (32) + : +- CometFilter (31) + : +- CometScan parquet spark_catalog.default.promotion (30) + :- * HashAggregate (70) + : +- Exchange (69) + : +- * HashAggregate (68) + : +- * Project (67) + : +- * BroadcastHashJoin Inner BuildRight (66) + : :- * Project (64) + : : +- * BroadcastHashJoin Inner BuildRight (63) + : : :- * Project (61) + : : : +- * BroadcastHashJoin Inner BuildRight (60) + : : : :- * Project (55) + : : : : +- * BroadcastHashJoin Inner BuildRight (54) + : : : : :- * Project (52) + : : : : : +- * SortMergeJoin LeftOuter (51) + : : : : : :- * Sort (44) + : : : : : : +- Exchange (43) + : : : : : : +- * ColumnarToRow (42) + : : : : : : +- CometFilter (41) + : : : : : : +- CometScan parquet spark_catalog.default.catalog_sales (40) + : : : : : +- * Sort (50) + : : : : : +- Exchange (49) + : : : : : +- * ColumnarToRow (48) + : : : : : +- CometProject (47) + : : : : : +- CometFilter (46) + : : : : : +- CometScan parquet spark_catalog.default.catalog_returns (45) + : : : : +- ReusedExchange (53) + : : : +- BroadcastExchange (59) + : : : +- * ColumnarToRow (58) + : : : +- CometFilter (57) + : : : +- CometScan parquet spark_catalog.default.catalog_page (56) + : : +- ReusedExchange (62) + : +- ReusedExchange (65) + +- * HashAggregate (101) + +- Exchange (100) + +- * HashAggregate (99) + +- * Project (98) + +- * BroadcastHashJoin Inner BuildRight (97) + :- * Project (95) + : +- * BroadcastHashJoin Inner BuildRight (94) + : :- * Project (92) + : : +- * BroadcastHashJoin Inner BuildRight (91) + : : :- * Project (86) + : : : +- * BroadcastHashJoin Inner BuildRight (85) + : : : :- * Project (83) + : : : : +- * SortMergeJoin LeftOuter (82) + : : : : :- * Sort (75) + : : : : : +- Exchange (74) + : : : : : +- * ColumnarToRow (73) + : : : : : +- CometFilter (72) + : : : : : +- CometScan parquet spark_catalog.default.web_sales (71) + : : : : +- * Sort (81) + : : : : +- Exchange (80) + : : : : +- * ColumnarToRow (79) + : : : : +- CometProject (78) + : : : : +- CometFilter (77) + : : : : +- CometScan parquet spark_catalog.default.web_returns (76) + : : : +- ReusedExchange (84) + : : +- BroadcastExchange (90) + : : +- * ColumnarToRow (89) + : : +- CometFilter (88) + : : +- CometScan parquet spark_catalog.default.web_site (87) + : +- ReusedExchange (93) + +- ReusedExchange (96) + + +(1) Scan parquet spark_catalog.default.store_sales +Output [7]: [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ticket_number#4, ss_ext_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7] +Batched: true +Location: InMemoryFileIndex [] +PartitionFilters: [isnotnull(ss_sold_date_sk#7), dynamicpruningexpression(ss_sold_date_sk#7 IN dynamicpruning#8)] +PushedFilters: [IsNotNull(ss_store_sk), IsNotNull(ss_item_sk), IsNotNull(ss_promo_sk)] +ReadSchema: struct<ss_item_sk:int,ss_store_sk:int,ss_promo_sk:int,ss_ticket_number:int,ss_ext_sales_price:decimal(7,2),ss_net_profit:decimal(7,2)> + +(2) CometFilter +Input [7]: [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ticket_number#4, ss_ext_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7] +Condition : ((isnotnull(ss_store_sk#2) AND isnotnull(ss_item_sk#1)) AND isnotnull(ss_promo_sk#3)) + +(3) ColumnarToRow [codegen id : 1] +Input [7]: [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ticket_number#4, ss_ext_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7] + +(4) Exchange +Input [7]: [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ticket_number#4, ss_ext_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7] +Arguments: hashpartitioning(ss_item_sk#1, ss_ticket_number#4, 5), ENSURE_REQUIREMENTS, [plan_id=1] + +(5) Sort [codegen id : 2] +Input [7]: [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ticket_number#4, ss_ext_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7] +Arguments: [ss_item_sk#1 ASC NULLS FIRST, ss_ticket_number#4 ASC NULLS FIRST], false, 0 + +(6) Scan parquet spark_catalog.default.store_returns +Output [5]: [sr_item_sk#9, sr_ticket_number#10, sr_return_amt#11, sr_net_loss#12, sr_returned_date_sk#13] +Batched: true +Location [not included in comparison]/{warehouse_dir}/store_returns] +PushedFilters: [IsNotNull(sr_item_sk), IsNotNull(sr_ticket_number)] +ReadSchema: struct<sr_item_sk:int,sr_ticket_number:int,sr_return_amt:decimal(7,2),sr_net_loss:decimal(7,2)> + +(7) CometFilter +Input [5]: [sr_item_sk#9, sr_ticket_number#10, sr_return_amt#11, sr_net_loss#12, sr_returned_date_sk#13] +Condition : (isnotnull(sr_item_sk#9) AND isnotnull(sr_ticket_number#10)) + +(8) CometProject +Input [5]: [sr_item_sk#9, sr_ticket_number#10, sr_return_amt#11, sr_net_loss#12, sr_returned_date_sk#13] +Arguments: [sr_item_sk#9, sr_ticket_number#10, sr_return_amt#11, sr_net_loss#12], [sr_item_sk#9, sr_ticket_number#10, sr_return_amt#11, sr_net_loss#12] + +(9) ColumnarToRow [codegen id : 3] +Input [4]: [sr_item_sk#9, sr_ticket_number#10, sr_return_amt#11, sr_net_loss#12] + +(10) Exchange +Input [4]: [sr_item_sk#9, sr_ticket_number#10, sr_return_amt#11, sr_net_loss#12] +Arguments: hashpartitioning(sr_item_sk#9, sr_ticket_number#10, 5), ENSURE_REQUIREMENTS, [plan_id=2] + +(11) Sort [codegen id : 4] +Input [4]: [sr_item_sk#9, sr_ticket_number#10, sr_return_amt#11, sr_net_loss#12] +Arguments: [sr_item_sk#9 ASC NULLS FIRST, sr_ticket_number#10 ASC NULLS FIRST], false, 0 + +(12) SortMergeJoin [codegen id : 9] +Left keys [2]: [ss_item_sk#1, ss_ticket_number#4] +Right keys [2]: [sr_item_sk#9, sr_ticket_number#10] +Join type: LeftOuter +Join condition: None + +(13) Project [codegen id : 9] +Output [8]: [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7, sr_return_amt#11, sr_net_loss#12] +Input [11]: [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ticket_number#4, ss_ext_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7, sr_item_sk#9, sr_ticket_number#10, sr_return_amt#11, sr_net_loss#12] + +(14) ReusedExchange [Reuses operator id: 112] +Output [1]: [d_date_sk#14] + +(15) BroadcastHashJoin [codegen id : 9] +Left keys [1]: [ss_sold_date_sk#7] +Right keys [1]: [d_date_sk#14] +Join type: Inner +Join condition: None + +(16) Project [codegen id : 9] +Output [7]: [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12] +Input [9]: [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7, sr_return_amt#11, sr_net_loss#12, d_date_sk#14] + +(17) Scan parquet spark_catalog.default.store +Output [2]: [s_store_sk#15, s_store_id#16] +Batched: true +Location [not included in comparison]/{warehouse_dir}/store] +PushedFilters: [IsNotNull(s_store_sk)] +ReadSchema: struct<s_store_sk:int,s_store_id:string> + +(18) CometFilter +Input [2]: [s_store_sk#15, s_store_id#16] +Condition : isnotnull(s_store_sk#15) + +(19) ColumnarToRow [codegen id : 6] +Input [2]: [s_store_sk#15, s_store_id#16] + +(20) BroadcastExchange +Input [2]: [s_store_sk#15, s_store_id#16] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=3] + +(21) BroadcastHashJoin [codegen id : 9] +Left keys [1]: [ss_store_sk#2] +Right keys [1]: [s_store_sk#15] +Join type: Inner +Join condition: None + +(22) Project [codegen id : 9] +Output [7]: [ss_item_sk#1, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#16] +Input [9]: [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_sk#15, s_store_id#16] + +(23) Scan parquet spark_catalog.default.item +Output [2]: [i_item_sk#17, i_current_price#18] +Batched: true +Location [not included in comparison]/{warehouse_dir}/item] +PushedFilters: [IsNotNull(i_current_price), GreaterThan(i_current_price,50.00), IsNotNull(i_item_sk)] +ReadSchema: struct<i_item_sk:int,i_current_price:decimal(7,2)> + +(24) CometFilter +Input [2]: [i_item_sk#17, i_current_price#18] +Condition : ((isnotnull(i_current_price#18) AND (i_current_price#18 > 50.00)) AND isnotnull(i_item_sk#17)) + +(25) CometProject +Input [2]: [i_item_sk#17, i_current_price#18] +Arguments: [i_item_sk#17], [i_item_sk#17] + +(26) ColumnarToRow [codegen id : 7] +Input [1]: [i_item_sk#17] + +(27) BroadcastExchange +Input [1]: [i_item_sk#17] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] + +(28) BroadcastHashJoin [codegen id : 9] +Left keys [1]: [ss_item_sk#1] +Right keys [1]: [i_item_sk#17] +Join type: Inner +Join condition: None + +(29) Project [codegen id : 9] +Output [6]: [ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#16] +Input [8]: [ss_item_sk#1, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#16, i_item_sk#17] + +(30) Scan parquet spark_catalog.default.promotion +Output [2]: [p_promo_sk#19, p_channel_tv#20] +Batched: true +Location [not included in comparison]/{warehouse_dir}/promotion] +PushedFilters: [IsNotNull(p_channel_tv), EqualTo(p_channel_tv,N), IsNotNull(p_promo_sk)] +ReadSchema: struct<p_promo_sk:int,p_channel_tv:string> + +(31) CometFilter +Input [2]: [p_promo_sk#19, p_channel_tv#20] +Condition : ((isnotnull(p_channel_tv#20) AND (p_channel_tv#20 = N)) AND isnotnull(p_promo_sk#19)) + +(32) CometProject +Input [2]: [p_promo_sk#19, p_channel_tv#20] +Arguments: [p_promo_sk#19], [p_promo_sk#19] + +(33) ColumnarToRow [codegen id : 8] +Input [1]: [p_promo_sk#19] + +(34) BroadcastExchange +Input [1]: [p_promo_sk#19] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] + +(35) BroadcastHashJoin [codegen id : 9] +Left keys [1]: [ss_promo_sk#3] +Right keys [1]: [p_promo_sk#19] +Join type: Inner +Join condition: None + +(36) Project [codegen id : 9] +Output [5]: [ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#16] +Input [7]: [ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#16, p_promo_sk#19] + +(37) HashAggregate [codegen id : 9] +Input [5]: [ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#16] +Keys [1]: [s_store_id#16] +Functions [3]: [partial_sum(UnscaledValue(ss_ext_sales_price#5)), partial_sum(coalesce(cast(sr_return_amt#11 as decimal(12,2)), 0.00)), partial_sum((ss_net_profit#6 - coalesce(cast(sr_net_loss#12 as decimal(12,2)), 0.00)))] +Aggregate Attributes [5]: [sum#21, sum#22, isEmpty#23, sum#24, isEmpty#25] +Results [6]: [s_store_id#16, sum#26, sum#27, isEmpty#28, sum#29, isEmpty#30] + +(38) Exchange +Input [6]: [s_store_id#16, sum#26, sum#27, isEmpty#28, sum#29, isEmpty#30] +Arguments: hashpartitioning(s_store_id#16, 5), ENSURE_REQUIREMENTS, [plan_id=6] + +(39) HashAggregate [codegen id : 10] +Input [6]: [s_store_id#16, sum#26, sum#27, isEmpty#28, sum#29, isEmpty#30] +Keys [1]: [s_store_id#16] +Functions [3]: [sum(UnscaledValue(ss_ext_sales_price#5)), sum(coalesce(cast(sr_return_amt#11 as decimal(12,2)), 0.00)), sum((ss_net_profit#6 - coalesce(cast(sr_net_loss#12 as decimal(12,2)), 0.00)))] +Aggregate Attributes [3]: [sum(UnscaledValue(ss_ext_sales_price#5))#31, sum(coalesce(cast(sr_return_amt#11 as decimal(12,2)), 0.00))#32, sum((ss_net_profit#6 - coalesce(cast(sr_net_loss#12 as decimal(12,2)), 0.00)))#33] +Results [5]: [MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#5))#31,17,2) AS sales#34, sum(coalesce(cast(sr_return_amt#11 as decimal(12,2)), 0.00))#32 AS returns#35, sum((ss_net_profit#6 - coalesce(cast(sr_net_loss#12 as decimal(12,2)), 0.00)))#33 AS profit#36, store channel AS channel#37, concat(store, s_store_id#16) AS id#38] + +(40) Scan parquet spark_catalog.default.catalog_sales +Output [7]: [cs_catalog_page_sk#39, cs_item_sk#40, cs_promo_sk#41, cs_order_number#42, cs_ext_sales_price#43, cs_net_profit#44, cs_sold_date_sk#45] +Batched: true +Location: InMemoryFileIndex [] +PartitionFilters: [isnotnull(cs_sold_date_sk#45), dynamicpruningexpression(cs_sold_date_sk#45 IN dynamicpruning#46)] +PushedFilters: [IsNotNull(cs_catalog_page_sk), IsNotNull(cs_item_sk), IsNotNull(cs_promo_sk)] +ReadSchema: struct<cs_catalog_page_sk:int,cs_item_sk:int,cs_promo_sk:int,cs_order_number:int,cs_ext_sales_price:decimal(7,2),cs_net_profit:decimal(7,2)> + +(41) CometFilter +Input [7]: [cs_catalog_page_sk#39, cs_item_sk#40, cs_promo_sk#41, cs_order_number#42, cs_ext_sales_price#43, cs_net_profit#44, cs_sold_date_sk#45] +Condition : ((isnotnull(cs_catalog_page_sk#39) AND isnotnull(cs_item_sk#40)) AND isnotnull(cs_promo_sk#41)) + +(42) ColumnarToRow [codegen id : 11] +Input [7]: [cs_catalog_page_sk#39, cs_item_sk#40, cs_promo_sk#41, cs_order_number#42, cs_ext_sales_price#43, cs_net_profit#44, cs_sold_date_sk#45] + +(43) Exchange +Input [7]: [cs_catalog_page_sk#39, cs_item_sk#40, cs_promo_sk#41, cs_order_number#42, cs_ext_sales_price#43, cs_net_profit#44, cs_sold_date_sk#45] +Arguments: hashpartitioning(cs_item_sk#40, cs_order_number#42, 5), ENSURE_REQUIREMENTS, [plan_id=7] + +(44) Sort [codegen id : 12] +Input [7]: [cs_catalog_page_sk#39, cs_item_sk#40, cs_promo_sk#41, cs_order_number#42, cs_ext_sales_price#43, cs_net_profit#44, cs_sold_date_sk#45] +Arguments: [cs_item_sk#40 ASC NULLS FIRST, cs_order_number#42 ASC NULLS FIRST], false, 0 + +(45) Scan parquet spark_catalog.default.catalog_returns +Output [5]: [cr_item_sk#47, cr_order_number#48, cr_return_amount#49, cr_net_loss#50, cr_returned_date_sk#51] +Batched: true +Location [not included in comparison]/{warehouse_dir}/catalog_returns] +PushedFilters: [IsNotNull(cr_item_sk), IsNotNull(cr_order_number)] +ReadSchema: struct<cr_item_sk:int,cr_order_number:int,cr_return_amount:decimal(7,2),cr_net_loss:decimal(7,2)> + +(46) CometFilter +Input [5]: [cr_item_sk#47, cr_order_number#48, cr_return_amount#49, cr_net_loss#50, cr_returned_date_sk#51] +Condition : (isnotnull(cr_item_sk#47) AND isnotnull(cr_order_number#48)) + +(47) CometProject +Input [5]: [cr_item_sk#47, cr_order_number#48, cr_return_amount#49, cr_net_loss#50, cr_returned_date_sk#51] +Arguments: [cr_item_sk#47, cr_order_number#48, cr_return_amount#49, cr_net_loss#50], [cr_item_sk#47, cr_order_number#48, cr_return_amount#49, cr_net_loss#50] + +(48) ColumnarToRow [codegen id : 13] +Input [4]: [cr_item_sk#47, cr_order_number#48, cr_return_amount#49, cr_net_loss#50] + +(49) Exchange +Input [4]: [cr_item_sk#47, cr_order_number#48, cr_return_amount#49, cr_net_loss#50] +Arguments: hashpartitioning(cr_item_sk#47, cr_order_number#48, 5), ENSURE_REQUIREMENTS, [plan_id=8] + +(50) Sort [codegen id : 14] +Input [4]: [cr_item_sk#47, cr_order_number#48, cr_return_amount#49, cr_net_loss#50] +Arguments: [cr_item_sk#47 ASC NULLS FIRST, cr_order_number#48 ASC NULLS FIRST], false, 0 + +(51) SortMergeJoin [codegen id : 19] +Left keys [2]: [cs_item_sk#40, cs_order_number#42] +Right keys [2]: [cr_item_sk#47, cr_order_number#48] +Join type: LeftOuter +Join condition: None + +(52) Project [codegen id : 19] +Output [8]: [cs_catalog_page_sk#39, cs_item_sk#40, cs_promo_sk#41, cs_ext_sales_price#43, cs_net_profit#44, cs_sold_date_sk#45, cr_return_amount#49, cr_net_loss#50] +Input [11]: [cs_catalog_page_sk#39, cs_item_sk#40, cs_promo_sk#41, cs_order_number#42, cs_ext_sales_price#43, cs_net_profit#44, cs_sold_date_sk#45, cr_item_sk#47, cr_order_number#48, cr_return_amount#49, cr_net_loss#50] + +(53) ReusedExchange [Reuses operator id: 112] +Output [1]: [d_date_sk#52] + +(54) BroadcastHashJoin [codegen id : 19] +Left keys [1]: [cs_sold_date_sk#45] +Right keys [1]: [d_date_sk#52] +Join type: Inner +Join condition: None + +(55) Project [codegen id : 19] +Output [7]: [cs_catalog_page_sk#39, cs_item_sk#40, cs_promo_sk#41, cs_ext_sales_price#43, cs_net_profit#44, cr_return_amount#49, cr_net_loss#50] +Input [9]: [cs_catalog_page_sk#39, cs_item_sk#40, cs_promo_sk#41, cs_ext_sales_price#43, cs_net_profit#44, cs_sold_date_sk#45, cr_return_amount#49, cr_net_loss#50, d_date_sk#52] + +(56) Scan parquet spark_catalog.default.catalog_page +Output [2]: [cp_catalog_page_sk#53, cp_catalog_page_id#54] +Batched: true +Location [not included in comparison]/{warehouse_dir}/catalog_page] +PushedFilters: [IsNotNull(cp_catalog_page_sk)] +ReadSchema: struct<cp_catalog_page_sk:int,cp_catalog_page_id:string> + +(57) CometFilter +Input [2]: [cp_catalog_page_sk#53, cp_catalog_page_id#54] +Condition : isnotnull(cp_catalog_page_sk#53) + +(58) ColumnarToRow [codegen id : 16] +Input [2]: [cp_catalog_page_sk#53, cp_catalog_page_id#54] + +(59) BroadcastExchange +Input [2]: [cp_catalog_page_sk#53, cp_catalog_page_id#54] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=9] + +(60) BroadcastHashJoin [codegen id : 19] +Left keys [1]: [cs_catalog_page_sk#39] +Right keys [1]: [cp_catalog_page_sk#53] +Join type: Inner +Join condition: None + +(61) Project [codegen id : 19] +Output [7]: [cs_item_sk#40, cs_promo_sk#41, cs_ext_sales_price#43, cs_net_profit#44, cr_return_amount#49, cr_net_loss#50, cp_catalog_page_id#54] +Input [9]: [cs_catalog_page_sk#39, cs_item_sk#40, cs_promo_sk#41, cs_ext_sales_price#43, cs_net_profit#44, cr_return_amount#49, cr_net_loss#50, cp_catalog_page_sk#53, cp_catalog_page_id#54] + +(62) ReusedExchange [Reuses operator id: 27] +Output [1]: [i_item_sk#55] + +(63) BroadcastHashJoin [codegen id : 19] +Left keys [1]: [cs_item_sk#40] +Right keys [1]: [i_item_sk#55] +Join type: Inner +Join condition: None + +(64) Project [codegen id : 19] +Output [6]: [cs_promo_sk#41, cs_ext_sales_price#43, cs_net_profit#44, cr_return_amount#49, cr_net_loss#50, cp_catalog_page_id#54] +Input [8]: [cs_item_sk#40, cs_promo_sk#41, cs_ext_sales_price#43, cs_net_profit#44, cr_return_amount#49, cr_net_loss#50, cp_catalog_page_id#54, i_item_sk#55] + +(65) ReusedExchange [Reuses operator id: 34] +Output [1]: [p_promo_sk#56] + +(66) BroadcastHashJoin [codegen id : 19] +Left keys [1]: [cs_promo_sk#41] +Right keys [1]: [p_promo_sk#56] +Join type: Inner +Join condition: None + +(67) Project [codegen id : 19] +Output [5]: [cs_ext_sales_price#43, cs_net_profit#44, cr_return_amount#49, cr_net_loss#50, cp_catalog_page_id#54] +Input [7]: [cs_promo_sk#41, cs_ext_sales_price#43, cs_net_profit#44, cr_return_amount#49, cr_net_loss#50, cp_catalog_page_id#54, p_promo_sk#56] + +(68) HashAggregate [codegen id : 19] +Input [5]: [cs_ext_sales_price#43, cs_net_profit#44, cr_return_amount#49, cr_net_loss#50, cp_catalog_page_id#54] +Keys [1]: [cp_catalog_page_id#54] +Functions [3]: [partial_sum(UnscaledValue(cs_ext_sales_price#43)), partial_sum(coalesce(cast(cr_return_amount#49 as decimal(12,2)), 0.00)), partial_sum((cs_net_profit#44 - coalesce(cast(cr_net_loss#50 as decimal(12,2)), 0.00)))] +Aggregate Attributes [5]: [sum#57, sum#58, isEmpty#59, sum#60, isEmpty#61] +Results [6]: [cp_catalog_page_id#54, sum#62, sum#63, isEmpty#64, sum#65, isEmpty#66] + +(69) Exchange +Input [6]: [cp_catalog_page_id#54, sum#62, sum#63, isEmpty#64, sum#65, isEmpty#66] +Arguments: hashpartitioning(cp_catalog_page_id#54, 5), ENSURE_REQUIREMENTS, [plan_id=10] + +(70) HashAggregate [codegen id : 20] +Input [6]: [cp_catalog_page_id#54, sum#62, sum#63, isEmpty#64, sum#65, isEmpty#66] +Keys [1]: [cp_catalog_page_id#54] +Functions [3]: [sum(UnscaledValue(cs_ext_sales_price#43)), sum(coalesce(cast(cr_return_amount#49 as decimal(12,2)), 0.00)), sum((cs_net_profit#44 - coalesce(cast(cr_net_loss#50 as decimal(12,2)), 0.00)))] +Aggregate Attributes [3]: [sum(UnscaledValue(cs_ext_sales_price#43))#67, sum(coalesce(cast(cr_return_amount#49 as decimal(12,2)), 0.00))#68, sum((cs_net_profit#44 - coalesce(cast(cr_net_loss#50 as decimal(12,2)), 0.00)))#69] +Results [5]: [MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#43))#67,17,2) AS sales#70, sum(coalesce(cast(cr_return_amount#49 as decimal(12,2)), 0.00))#68 AS returns#71, sum((cs_net_profit#44 - coalesce(cast(cr_net_loss#50 as decimal(12,2)), 0.00)))#69 AS profit#72, catalog channel AS channel#73, concat(catalog_page, cp_catalog_page_id#54) AS id#74] + +(71) Scan parquet spark_catalog.default.web_sales +Output [7]: [ws_item_sk#75, ws_web_site_sk#76, ws_promo_sk#77, ws_order_number#78, ws_ext_sales_price#79, ws_net_profit#80, ws_sold_date_sk#81] +Batched: true +Location: InMemoryFileIndex [] +PartitionFilters: [isnotnull(ws_sold_date_sk#81), dynamicpruningexpression(ws_sold_date_sk#81 IN dynamicpruning#82)] +PushedFilters: [IsNotNull(ws_web_site_sk), IsNotNull(ws_item_sk), IsNotNull(ws_promo_sk)] +ReadSchema: struct<ws_item_sk:int,ws_web_site_sk:int,ws_promo_sk:int,ws_order_number:int,ws_ext_sales_price:decimal(7,2),ws_net_profit:decimal(7,2)> + +(72) CometFilter +Input [7]: [ws_item_sk#75, ws_web_site_sk#76, ws_promo_sk#77, ws_order_number#78, ws_ext_sales_price#79, ws_net_profit#80, ws_sold_date_sk#81] +Condition : ((isnotnull(ws_web_site_sk#76) AND isnotnull(ws_item_sk#75)) AND isnotnull(ws_promo_sk#77)) + +(73) ColumnarToRow [codegen id : 21] +Input [7]: [ws_item_sk#75, ws_web_site_sk#76, ws_promo_sk#77, ws_order_number#78, ws_ext_sales_price#79, ws_net_profit#80, ws_sold_date_sk#81] + +(74) Exchange +Input [7]: [ws_item_sk#75, ws_web_site_sk#76, ws_promo_sk#77, ws_order_number#78, ws_ext_sales_price#79, ws_net_profit#80, ws_sold_date_sk#81] +Arguments: hashpartitioning(ws_item_sk#75, ws_order_number#78, 5), ENSURE_REQUIREMENTS, [plan_id=11] + +(75) Sort [codegen id : 22] +Input [7]: [ws_item_sk#75, ws_web_site_sk#76, ws_promo_sk#77, ws_order_number#78, ws_ext_sales_price#79, ws_net_profit#80, ws_sold_date_sk#81] +Arguments: [ws_item_sk#75 ASC NULLS FIRST, ws_order_number#78 ASC NULLS FIRST], false, 0 + +(76) Scan parquet spark_catalog.default.web_returns +Output [5]: [wr_item_sk#83, wr_order_number#84, wr_return_amt#85, wr_net_loss#86, wr_returned_date_sk#87] +Batched: true +Location [not included in comparison]/{warehouse_dir}/web_returns] +PushedFilters: [IsNotNull(wr_item_sk), IsNotNull(wr_order_number)] +ReadSchema: struct<wr_item_sk:int,wr_order_number:int,wr_return_amt:decimal(7,2),wr_net_loss:decimal(7,2)> + +(77) CometFilter +Input [5]: [wr_item_sk#83, wr_order_number#84, wr_return_amt#85, wr_net_loss#86, wr_returned_date_sk#87] +Condition : (isnotnull(wr_item_sk#83) AND isnotnull(wr_order_number#84)) + +(78) CometProject +Input [5]: [wr_item_sk#83, wr_order_number#84, wr_return_amt#85, wr_net_loss#86, wr_returned_date_sk#87] +Arguments: [wr_item_sk#83, wr_order_number#84, wr_return_amt#85, wr_net_loss#86], [wr_item_sk#83, wr_order_number#84, wr_return_amt#85, wr_net_loss#86] + +(79) ColumnarToRow [codegen id : 23] +Input [4]: [wr_item_sk#83, wr_order_number#84, wr_return_amt#85, wr_net_loss#86] + +(80) Exchange +Input [4]: [wr_item_sk#83, wr_order_number#84, wr_return_amt#85, wr_net_loss#86] +Arguments: hashpartitioning(wr_item_sk#83, wr_order_number#84, 5), ENSURE_REQUIREMENTS, [plan_id=12] + +(81) Sort [codegen id : 24] +Input [4]: [wr_item_sk#83, wr_order_number#84, wr_return_amt#85, wr_net_loss#86] +Arguments: [wr_item_sk#83 ASC NULLS FIRST, wr_order_number#84 ASC NULLS FIRST], false, 0 + +(82) SortMergeJoin [codegen id : 29] +Left keys [2]: [ws_item_sk#75, ws_order_number#78] +Right keys [2]: [wr_item_sk#83, wr_order_number#84] +Join type: LeftOuter +Join condition: None + +(83) Project [codegen id : 29] +Output [8]: [ws_item_sk#75, ws_web_site_sk#76, ws_promo_sk#77, ws_ext_sales_price#79, ws_net_profit#80, ws_sold_date_sk#81, wr_return_amt#85, wr_net_loss#86] +Input [11]: [ws_item_sk#75, ws_web_site_sk#76, ws_promo_sk#77, ws_order_number#78, ws_ext_sales_price#79, ws_net_profit#80, ws_sold_date_sk#81, wr_item_sk#83, wr_order_number#84, wr_return_amt#85, wr_net_loss#86] + +(84) ReusedExchange [Reuses operator id: 112] +Output [1]: [d_date_sk#88] + +(85) BroadcastHashJoin [codegen id : 29] +Left keys [1]: [ws_sold_date_sk#81] +Right keys [1]: [d_date_sk#88] +Join type: Inner +Join condition: None + +(86) Project [codegen id : 29] +Output [7]: [ws_item_sk#75, ws_web_site_sk#76, ws_promo_sk#77, ws_ext_sales_price#79, ws_net_profit#80, wr_return_amt#85, wr_net_loss#86] +Input [9]: [ws_item_sk#75, ws_web_site_sk#76, ws_promo_sk#77, ws_ext_sales_price#79, ws_net_profit#80, ws_sold_date_sk#81, wr_return_amt#85, wr_net_loss#86, d_date_sk#88] + +(87) Scan parquet spark_catalog.default.web_site +Output [2]: [web_site_sk#89, web_site_id#90] +Batched: true +Location [not included in comparison]/{warehouse_dir}/web_site] +PushedFilters: [IsNotNull(web_site_sk)] +ReadSchema: struct<web_site_sk:int,web_site_id:string> + +(88) CometFilter +Input [2]: [web_site_sk#89, web_site_id#90] +Condition : isnotnull(web_site_sk#89) + +(89) ColumnarToRow [codegen id : 26] +Input [2]: [web_site_sk#89, web_site_id#90] + +(90) BroadcastExchange +Input [2]: [web_site_sk#89, web_site_id#90] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=13] + +(91) BroadcastHashJoin [codegen id : 29] +Left keys [1]: [ws_web_site_sk#76] +Right keys [1]: [web_site_sk#89] +Join type: Inner +Join condition: None + +(92) Project [codegen id : 29] +Output [7]: [ws_item_sk#75, ws_promo_sk#77, ws_ext_sales_price#79, ws_net_profit#80, wr_return_amt#85, wr_net_loss#86, web_site_id#90] +Input [9]: [ws_item_sk#75, ws_web_site_sk#76, ws_promo_sk#77, ws_ext_sales_price#79, ws_net_profit#80, wr_return_amt#85, wr_net_loss#86, web_site_sk#89, web_site_id#90] + +(93) ReusedExchange [Reuses operator id: 27] +Output [1]: [i_item_sk#91] + +(94) BroadcastHashJoin [codegen id : 29] +Left keys [1]: [ws_item_sk#75] +Right keys [1]: [i_item_sk#91] +Join type: Inner +Join condition: None + +(95) Project [codegen id : 29] +Output [6]: [ws_promo_sk#77, ws_ext_sales_price#79, ws_net_profit#80, wr_return_amt#85, wr_net_loss#86, web_site_id#90] +Input [8]: [ws_item_sk#75, ws_promo_sk#77, ws_ext_sales_price#79, ws_net_profit#80, wr_return_amt#85, wr_net_loss#86, web_site_id#90, i_item_sk#91] + +(96) ReusedExchange [Reuses operator id: 34] +Output [1]: [p_promo_sk#92] + +(97) BroadcastHashJoin [codegen id : 29] +Left keys [1]: [ws_promo_sk#77] +Right keys [1]: [p_promo_sk#92] +Join type: Inner +Join condition: None + +(98) Project [codegen id : 29] +Output [5]: [ws_ext_sales_price#79, ws_net_profit#80, wr_return_amt#85, wr_net_loss#86, web_site_id#90] +Input [7]: [ws_promo_sk#77, ws_ext_sales_price#79, ws_net_profit#80, wr_return_amt#85, wr_net_loss#86, web_site_id#90, p_promo_sk#92] + +(99) HashAggregate [codegen id : 29] +Input [5]: [ws_ext_sales_price#79, ws_net_profit#80, wr_return_amt#85, wr_net_loss#86, web_site_id#90] +Keys [1]: [web_site_id#90] +Functions [3]: [partial_sum(UnscaledValue(ws_ext_sales_price#79)), partial_sum(coalesce(cast(wr_return_amt#85 as decimal(12,2)), 0.00)), partial_sum((ws_net_profit#80 - coalesce(cast(wr_net_loss#86 as decimal(12,2)), 0.00)))] +Aggregate Attributes [5]: [sum#93, sum#94, isEmpty#95, sum#96, isEmpty#97] +Results [6]: [web_site_id#90, sum#98, sum#99, isEmpty#100, sum#101, isEmpty#102] + +(100) Exchange +Input [6]: [web_site_id#90, sum#98, sum#99, isEmpty#100, sum#101, isEmpty#102] +Arguments: hashpartitioning(web_site_id#90, 5), ENSURE_REQUIREMENTS, [plan_id=14] + +(101) HashAggregate [codegen id : 30] +Input [6]: [web_site_id#90, sum#98, sum#99, isEmpty#100, sum#101, isEmpty#102] +Keys [1]: [web_site_id#90] +Functions [3]: [sum(UnscaledValue(ws_ext_sales_price#79)), sum(coalesce(cast(wr_return_amt#85 as decimal(12,2)), 0.00)), sum((ws_net_profit#80 - coalesce(cast(wr_net_loss#86 as decimal(12,2)), 0.00)))] +Aggregate Attributes [3]: [sum(UnscaledValue(ws_ext_sales_price#79))#103, sum(coalesce(cast(wr_return_amt#85 as decimal(12,2)), 0.00))#104, sum((ws_net_profit#80 - coalesce(cast(wr_net_loss#86 as decimal(12,2)), 0.00)))#105] +Results [5]: [MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#79))#103,17,2) AS sales#106, sum(coalesce(cast(wr_return_amt#85 as decimal(12,2)), 0.00))#104 AS returns#107, sum((ws_net_profit#80 - coalesce(cast(wr_net_loss#86 as decimal(12,2)), 0.00)))#105 AS profit#108, web channel AS channel#109, concat(web_site, web_site_id#90) AS id#110] + +(102) Union + +(103) Expand [codegen id : 31] +Input [5]: [sales#34, returns#35, profit#36, channel#37, id#38] +Arguments: [[sales#34, returns#35, profit#36, channel#37, id#38, 0], [sales#34, returns#35, profit#36, channel#37, null, 1], [sales#34, returns#35, profit#36, null, null, 3]], [sales#34, returns#35, profit#36, channel#111, id#112, spark_grouping_id#113] + +(104) HashAggregate [codegen id : 31] +Input [6]: [sales#34, returns#35, profit#36, channel#111, id#112, spark_grouping_id#113] +Keys [3]: [channel#111, id#112, spark_grouping_id#113] +Functions [3]: [partial_sum(sales#34), partial_sum(returns#35), partial_sum(profit#36)] +Aggregate Attributes [6]: [sum#114, isEmpty#115, sum#116, isEmpty#117, sum#118, isEmpty#119] +Results [9]: [channel#111, id#112, spark_grouping_id#113, sum#120, isEmpty#121, sum#122, isEmpty#123, sum#124, isEmpty#125] + +(105) Exchange +Input [9]: [channel#111, id#112, spark_grouping_id#113, sum#120, isEmpty#121, sum#122, isEmpty#123, sum#124, isEmpty#125] +Arguments: hashpartitioning(channel#111, id#112, spark_grouping_id#113, 5), ENSURE_REQUIREMENTS, [plan_id=15] + +(106) HashAggregate [codegen id : 32] +Input [9]: [channel#111, id#112, spark_grouping_id#113, sum#120, isEmpty#121, sum#122, isEmpty#123, sum#124, isEmpty#125] +Keys [3]: [channel#111, id#112, spark_grouping_id#113] +Functions [3]: [sum(sales#34), sum(returns#35), sum(profit#36)] +Aggregate Attributes [3]: [sum(sales#34)#126, sum(returns#35)#127, sum(profit#36)#128] +Results [5]: [channel#111, id#112, sum(sales#34)#126 AS sales#129, sum(returns#35)#127 AS returns#130, sum(profit#36)#128 AS profit#131] + +(107) TakeOrderedAndProject +Input [5]: [channel#111, id#112, sales#129, returns#130, profit#131] +Arguments: 100, [channel#111 ASC NULLS FIRST, id#112 ASC NULLS FIRST], [channel#111, id#112, sales#129, returns#130, profit#131] + +===== Subqueries ===== + +Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#7 IN dynamicpruning#8 +BroadcastExchange (112) ++- * ColumnarToRow (111) + +- CometProject (110) + +- CometFilter (109) + +- CometScan parquet spark_catalog.default.date_dim (108) + + +(108) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#14, d_date#132] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,2000-08-23), LessThanOrEqual(d_date,2000-09-22), IsNotNull(d_date_sk)] +ReadSchema: struct<d_date_sk:int,d_date:date> + +(109) CometFilter +Input [2]: [d_date_sk#14, d_date#132] +Condition : (((isnotnull(d_date#132) AND (d_date#132 >= 2000-08-23)) AND (d_date#132 <= 2000-09-22)) AND isnotnull(d_date_sk#14)) + +(110) CometProject +Input [2]: [d_date_sk#14, d_date#132] +Arguments: [d_date_sk#14], [d_date_sk#14] + +(111) ColumnarToRow [codegen id : 1] +Input [1]: [d_date_sk#14] + +(112) BroadcastExchange +Input [1]: [d_date_sk#14] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=16] + +Subquery:2 Hosting operator id = 40 Hosting Expression = cs_sold_date_sk#45 IN dynamicpruning#8 + +Subquery:3 Hosting operator id = 71 Hosting Expression = ws_sold_date_sk#81 IN dynamicpruning#8 + + diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q80/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q80/simplified.txt new file mode 100644 index 000000000..7e257bdc6 --- /dev/null +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q80/simplified.txt @@ -0,0 +1,182 @@ +TakeOrderedAndProject [channel,id,sales,returns,profit] + WholeStageCodegen (32) + HashAggregate [channel,id,spark_grouping_id,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),sales,returns,profit,sum,isEmpty,sum,isEmpty,sum,isEmpty] + InputAdapter + Exchange [channel,id,spark_grouping_id] #1 + WholeStageCodegen (31) + HashAggregate [channel,id,spark_grouping_id,sales,returns,profit] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] + Expand [sales,returns,profit,channel,id] + InputAdapter + Union + WholeStageCodegen (10) + HashAggregate [s_store_id,sum,sum,isEmpty,sum,isEmpty] [sum(UnscaledValue(ss_ext_sales_price)),sum(coalesce(cast(sr_return_amt as decimal(12,2)), 0.00)),sum((ss_net_profit - coalesce(cast(sr_net_loss as decimal(12,2)), 0.00))),sales,returns,profit,channel,id,sum,sum,isEmpty,sum,isEmpty] + InputAdapter + Exchange [s_store_id] #2 + WholeStageCodegen (9) + HashAggregate [s_store_id,ss_ext_sales_price,sr_return_amt,ss_net_profit,sr_net_loss] [sum,sum,isEmpty,sum,isEmpty,sum,sum,isEmpty,sum,isEmpty] + Project [ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss,s_store_id] + BroadcastHashJoin [ss_promo_sk,p_promo_sk] + Project [ss_promo_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss,s_store_id] + BroadcastHashJoin [ss_item_sk,i_item_sk] + Project [ss_item_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss,s_store_id] + BroadcastHashJoin [ss_store_sk,s_store_sk] + Project [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk,sr_return_amt,sr_net_loss] + SortMergeJoin [ss_item_sk,ss_ticket_number,sr_item_sk,sr_ticket_number] + InputAdapter + WholeStageCodegen (2) + Sort [ss_item_sk,ss_ticket_number] + InputAdapter + Exchange [ss_item_sk,ss_ticket_number] #3 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometFilter [ss_store_sk,ss_item_sk,ss_promo_sk] + CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #4 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] + InputAdapter + WholeStageCodegen (4) + Sort [sr_item_sk,sr_ticket_number] + InputAdapter + Exchange [sr_item_sk,sr_ticket_number] #5 + WholeStageCodegen (3) + ColumnarToRow + InputAdapter + CometProject [sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss] + CometFilter [sr_item_sk,sr_ticket_number] + CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss,sr_returned_date_sk] + InputAdapter + ReusedExchange [d_date_sk] #4 + InputAdapter + BroadcastExchange #6 + WholeStageCodegen (6) + ColumnarToRow + InputAdapter + CometFilter [s_store_sk] + CometScan parquet spark_catalog.default.store [s_store_sk,s_store_id] + InputAdapter + BroadcastExchange #7 + WholeStageCodegen (7) + ColumnarToRow + InputAdapter + CometProject [i_item_sk] + CometFilter [i_current_price,i_item_sk] + CometScan parquet spark_catalog.default.item [i_item_sk,i_current_price] + InputAdapter + BroadcastExchange #8 + WholeStageCodegen (8) + ColumnarToRow + InputAdapter + CometProject [p_promo_sk] + CometFilter [p_channel_tv,p_promo_sk] + CometScan parquet spark_catalog.default.promotion [p_promo_sk,p_channel_tv] + WholeStageCodegen (20) + HashAggregate [cp_catalog_page_id,sum,sum,isEmpty,sum,isEmpty] [sum(UnscaledValue(cs_ext_sales_price)),sum(coalesce(cast(cr_return_amount as decimal(12,2)), 0.00)),sum((cs_net_profit - coalesce(cast(cr_net_loss as decimal(12,2)), 0.00))),sales,returns,profit,channel,id,sum,sum,isEmpty,sum,isEmpty] + InputAdapter + Exchange [cp_catalog_page_id] #9 + WholeStageCodegen (19) + HashAggregate [cp_catalog_page_id,cs_ext_sales_price,cr_return_amount,cs_net_profit,cr_net_loss] [sum,sum,isEmpty,sum,isEmpty,sum,sum,isEmpty,sum,isEmpty] + Project [cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss,cp_catalog_page_id] + BroadcastHashJoin [cs_promo_sk,p_promo_sk] + Project [cs_promo_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss,cp_catalog_page_id] + BroadcastHashJoin [cs_item_sk,i_item_sk] + Project [cs_item_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss,cp_catalog_page_id] + BroadcastHashJoin [cs_catalog_page_sk,cp_catalog_page_sk] + Project [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + Project [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk,cr_return_amount,cr_net_loss] + SortMergeJoin [cs_item_sk,cs_order_number,cr_item_sk,cr_order_number] + InputAdapter + WholeStageCodegen (12) + Sort [cs_item_sk,cs_order_number] + InputAdapter + Exchange [cs_item_sk,cs_order_number] #10 + WholeStageCodegen (11) + ColumnarToRow + InputAdapter + CometFilter [cs_catalog_page_sk,cs_item_sk,cs_promo_sk] + CometScan parquet spark_catalog.default.catalog_sales [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + InputAdapter + WholeStageCodegen (14) + Sort [cr_item_sk,cr_order_number] + InputAdapter + Exchange [cr_item_sk,cr_order_number] #11 + WholeStageCodegen (13) + ColumnarToRow + InputAdapter + CometProject [cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss] + CometFilter [cr_item_sk,cr_order_number] + CometScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss,cr_returned_date_sk] + InputAdapter + ReusedExchange [d_date_sk] #4 + InputAdapter + BroadcastExchange #12 + WholeStageCodegen (16) + ColumnarToRow + InputAdapter + CometFilter [cp_catalog_page_sk] + CometScan parquet spark_catalog.default.catalog_page [cp_catalog_page_sk,cp_catalog_page_id] + InputAdapter + ReusedExchange [i_item_sk] #7 + InputAdapter + ReusedExchange [p_promo_sk] #8 + WholeStageCodegen (30) + HashAggregate [web_site_id,sum,sum,isEmpty,sum,isEmpty] [sum(UnscaledValue(ws_ext_sales_price)),sum(coalesce(cast(wr_return_amt as decimal(12,2)), 0.00)),sum((ws_net_profit - coalesce(cast(wr_net_loss as decimal(12,2)), 0.00))),sales,returns,profit,channel,id,sum,sum,isEmpty,sum,isEmpty] + InputAdapter + Exchange [web_site_id] #13 + WholeStageCodegen (29) + HashAggregate [web_site_id,ws_ext_sales_price,wr_return_amt,ws_net_profit,wr_net_loss] [sum,sum,isEmpty,sum,isEmpty,sum,sum,isEmpty,sum,isEmpty] + Project [ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss,web_site_id] + BroadcastHashJoin [ws_promo_sk,p_promo_sk] + Project [ws_promo_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss,web_site_id] + BroadcastHashJoin [ws_item_sk,i_item_sk] + Project [ws_item_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss,web_site_id] + BroadcastHashJoin [ws_web_site_sk,web_site_sk] + Project [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + Project [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk,wr_return_amt,wr_net_loss] + SortMergeJoin [ws_item_sk,ws_order_number,wr_item_sk,wr_order_number] + InputAdapter + WholeStageCodegen (22) + Sort [ws_item_sk,ws_order_number] + InputAdapter + Exchange [ws_item_sk,ws_order_number] #14 + WholeStageCodegen (21) + ColumnarToRow + InputAdapter + CometFilter [ws_web_site_sk,ws_item_sk,ws_promo_sk] + CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_order_number,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + InputAdapter + WholeStageCodegen (24) + Sort [wr_item_sk,wr_order_number] + InputAdapter + Exchange [wr_item_sk,wr_order_number] #15 + WholeStageCodegen (23) + ColumnarToRow + InputAdapter + CometProject [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss] + CometFilter [wr_item_sk,wr_order_number] + CometScan parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk] + InputAdapter + ReusedExchange [d_date_sk] #4 + InputAdapter + BroadcastExchange #16 + WholeStageCodegen (26) + ColumnarToRow + InputAdapter + CometFilter [web_site_sk] + CometScan parquet spark_catalog.default.web_site [web_site_sk,web_site_id] + InputAdapter + ReusedExchange [i_item_sk] #7 + InputAdapter + ReusedExchange [p_promo_sk] #8 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 new file mode 100644 index 000000000..431d1453d --- /dev/null +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q81/explain.txt @@ -0,0 +1,319 @@ +== Physical Plan == +TakeOrderedAndProject (48) ++- * Project (47) + +- * BroadcastHashJoin Inner BuildRight (46) + :- * Project (41) + : +- * BroadcastHashJoin Inner BuildRight (40) + : :- * Project (35) + : : +- * BroadcastHashJoin Inner BuildRight (34) + : : :- * Filter (16) + : : : +- * HashAggregate (15) + : : : +- Exchange (14) + : : : +- * HashAggregate (13) + : : : +- * Project (12) + : : : +- * BroadcastHashJoin Inner BuildRight (11) + : : : :- * Project (6) + : : : : +- * BroadcastHashJoin Inner BuildRight (5) + : : : : :- * ColumnarToRow (3) + : : : : : +- CometFilter (2) + : : : : : +- CometScan parquet spark_catalog.default.catalog_returns (1) + : : : : +- ReusedExchange (4) + : : : +- BroadcastExchange (10) + : : : +- * ColumnarToRow (9) + : : : +- CometFilter (8) + : : : +- CometScan parquet spark_catalog.default.customer_address (7) + : : +- BroadcastExchange (33) + : : +- * Filter (32) + : : +- * HashAggregate (31) + : : +- Exchange (30) + : : +- * HashAggregate (29) + : : +- * HashAggregate (28) + : : +- Exchange (27) + : : +- * HashAggregate (26) + : : +- * Project (25) + : : +- * BroadcastHashJoin Inner BuildRight (24) + : : :- * Project (22) + : : : +- * BroadcastHashJoin Inner BuildRight (21) + : : : :- * ColumnarToRow (19) + : : : : +- CometFilter (18) + : : : : +- CometScan parquet spark_catalog.default.catalog_returns (17) + : : : +- ReusedExchange (20) + : : +- ReusedExchange (23) + : +- BroadcastExchange (39) + : +- * ColumnarToRow (38) + : +- CometFilter (37) + : +- CometScan parquet spark_catalog.default.customer (36) + +- BroadcastExchange (45) + +- * ColumnarToRow (44) + +- CometFilter (43) + +- CometScan parquet spark_catalog.default.customer_address (42) + + +(1) Scan parquet spark_catalog.default.catalog_returns +Output [4]: [cr_returning_customer_sk#1, cr_returning_addr_sk#2, cr_return_amt_inc_tax#3, cr_returned_date_sk#4] +Batched: true +Location: InMemoryFileIndex [] +PartitionFilters: [isnotnull(cr_returned_date_sk#4), dynamicpruningexpression(cr_returned_date_sk#4 IN dynamicpruning#5)] +PushedFilters: [IsNotNull(cr_returning_addr_sk), IsNotNull(cr_returning_customer_sk)] +ReadSchema: struct<cr_returning_customer_sk:int,cr_returning_addr_sk:int,cr_return_amt_inc_tax:decimal(7,2)> + +(2) CometFilter +Input [4]: [cr_returning_customer_sk#1, cr_returning_addr_sk#2, cr_return_amt_inc_tax#3, cr_returned_date_sk#4] +Condition : (isnotnull(cr_returning_addr_sk#2) AND isnotnull(cr_returning_customer_sk#1)) + +(3) ColumnarToRow [codegen id : 3] +Input [4]: [cr_returning_customer_sk#1, cr_returning_addr_sk#2, cr_return_amt_inc_tax#3, cr_returned_date_sk#4] + +(4) ReusedExchange [Reuses operator id: 53] +Output [1]: [d_date_sk#6] + +(5) BroadcastHashJoin [codegen id : 3] +Left keys [1]: [cr_returned_date_sk#4] +Right keys [1]: [d_date_sk#6] +Join type: Inner +Join condition: None + +(6) Project [codegen id : 3] +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] + +(7) Scan parquet spark_catalog.default.customer_address +Output [2]: [ca_address_sk#7, ca_state#8] +Batched: true +Location [not included in comparison]/{warehouse_dir}/customer_address] +PushedFilters: [IsNotNull(ca_address_sk), IsNotNull(ca_state)] +ReadSchema: struct<ca_address_sk:int,ca_state:string> + +(8) CometFilter +Input [2]: [ca_address_sk#7, ca_state#8] +Condition : (isnotnull(ca_address_sk#7) AND isnotnull(ca_state#8)) + +(9) ColumnarToRow [codegen id : 2] +Input [2]: [ca_address_sk#7, ca_state#8] + +(10) BroadcastExchange +Input [2]: [ca_address_sk#7, ca_state#8] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=1] + +(11) BroadcastHashJoin [codegen id : 3] +Left keys [1]: [cr_returning_addr_sk#2] +Right keys [1]: [ca_address_sk#7] +Join type: Inner +Join condition: None + +(12) Project [codegen id : 3] +Output [3]: [cr_returning_customer_sk#1, cr_return_amt_inc_tax#3, ca_state#8] +Input [5]: [cr_returning_customer_sk#1, cr_returning_addr_sk#2, cr_return_amt_inc_tax#3, ca_address_sk#7, ca_state#8] + +(13) HashAggregate [codegen id : 3] +Input [3]: [cr_returning_customer_sk#1, cr_return_amt_inc_tax#3, ca_state#8] +Keys [2]: [cr_returning_customer_sk#1, ca_state#8] +Functions [1]: [partial_sum(UnscaledValue(cr_return_amt_inc_tax#3))] +Aggregate Attributes [1]: [sum#9] +Results [3]: [cr_returning_customer_sk#1, ca_state#8, sum#10] + +(14) Exchange +Input [3]: [cr_returning_customer_sk#1, ca_state#8, sum#10] +Arguments: hashpartitioning(cr_returning_customer_sk#1, ca_state#8, 5), ENSURE_REQUIREMENTS, [plan_id=2] + +(15) HashAggregate [codegen id : 11] +Input [3]: [cr_returning_customer_sk#1, ca_state#8, sum#10] +Keys [2]: [cr_returning_customer_sk#1, ca_state#8] +Functions [1]: [sum(UnscaledValue(cr_return_amt_inc_tax#3))] +Aggregate Attributes [1]: [sum(UnscaledValue(cr_return_amt_inc_tax#3))#11] +Results [3]: [cr_returning_customer_sk#1 AS ctr_customer_sk#12, ca_state#8 AS ctr_state#13, MakeDecimal(sum(UnscaledValue(cr_return_amt_inc_tax#3))#11,17,2) AS ctr_total_return#14] + +(16) Filter [codegen id : 11] +Input [3]: [ctr_customer_sk#12, ctr_state#13, ctr_total_return#14] +Condition : isnotnull(ctr_total_return#14) + +(17) Scan parquet spark_catalog.default.catalog_returns +Output [4]: [cr_returning_customer_sk#15, cr_returning_addr_sk#16, cr_return_amt_inc_tax#17, cr_returned_date_sk#18] +Batched: true +Location: InMemoryFileIndex [] +PartitionFilters: [isnotnull(cr_returned_date_sk#18), dynamicpruningexpression(cr_returned_date_sk#18 IN dynamicpruning#19)] +PushedFilters: [IsNotNull(cr_returning_addr_sk)] +ReadSchema: struct<cr_returning_customer_sk:int,cr_returning_addr_sk:int,cr_return_amt_inc_tax:decimal(7,2)> + +(18) CometFilter +Input [4]: [cr_returning_customer_sk#15, cr_returning_addr_sk#16, cr_return_amt_inc_tax#17, cr_returned_date_sk#18] +Condition : isnotnull(cr_returning_addr_sk#16) + +(19) ColumnarToRow [codegen id : 6] +Input [4]: [cr_returning_customer_sk#15, cr_returning_addr_sk#16, cr_return_amt_inc_tax#17, cr_returned_date_sk#18] + +(20) ReusedExchange [Reuses operator id: 53] +Output [1]: [d_date_sk#20] + +(21) BroadcastHashJoin [codegen id : 6] +Left keys [1]: [cr_returned_date_sk#18] +Right keys [1]: [d_date_sk#20] +Join type: Inner +Join condition: None + +(22) Project [codegen id : 6] +Output [3]: [cr_returning_customer_sk#15, cr_returning_addr_sk#16, cr_return_amt_inc_tax#17] +Input [5]: [cr_returning_customer_sk#15, cr_returning_addr_sk#16, cr_return_amt_inc_tax#17, cr_returned_date_sk#18, d_date_sk#20] + +(23) ReusedExchange [Reuses operator id: 10] +Output [2]: [ca_address_sk#21, ca_state#22] + +(24) BroadcastHashJoin [codegen id : 6] +Left keys [1]: [cr_returning_addr_sk#16] +Right keys [1]: [ca_address_sk#21] +Join type: Inner +Join condition: None + +(25) Project [codegen id : 6] +Output [3]: [cr_returning_customer_sk#15, cr_return_amt_inc_tax#17, ca_state#22] +Input [5]: [cr_returning_customer_sk#15, cr_returning_addr_sk#16, cr_return_amt_inc_tax#17, ca_address_sk#21, ca_state#22] + +(26) HashAggregate [codegen id : 6] +Input [3]: [cr_returning_customer_sk#15, cr_return_amt_inc_tax#17, ca_state#22] +Keys [2]: [cr_returning_customer_sk#15, ca_state#22] +Functions [1]: [partial_sum(UnscaledValue(cr_return_amt_inc_tax#17))] +Aggregate Attributes [1]: [sum#23] +Results [3]: [cr_returning_customer_sk#15, ca_state#22, sum#24] + +(27) Exchange +Input [3]: [cr_returning_customer_sk#15, ca_state#22, sum#24] +Arguments: hashpartitioning(cr_returning_customer_sk#15, ca_state#22, 5), ENSURE_REQUIREMENTS, [plan_id=3] + +(28) HashAggregate [codegen id : 7] +Input [3]: [cr_returning_customer_sk#15, ca_state#22, sum#24] +Keys [2]: [cr_returning_customer_sk#15, ca_state#22] +Functions [1]: [sum(UnscaledValue(cr_return_amt_inc_tax#17))] +Aggregate Attributes [1]: [sum(UnscaledValue(cr_return_amt_inc_tax#17))#11] +Results [2]: [ca_state#22 AS ctr_state#25, MakeDecimal(sum(UnscaledValue(cr_return_amt_inc_tax#17))#11,17,2) AS ctr_total_return#26] + +(29) HashAggregate [codegen id : 7] +Input [2]: [ctr_state#25, ctr_total_return#26] +Keys [1]: [ctr_state#25] +Functions [1]: [partial_avg(ctr_total_return#26)] +Aggregate Attributes [2]: [sum#27, count#28] +Results [3]: [ctr_state#25, sum#29, count#30] + +(30) Exchange +Input [3]: [ctr_state#25, sum#29, count#30] +Arguments: hashpartitioning(ctr_state#25, 5), ENSURE_REQUIREMENTS, [plan_id=4] + +(31) HashAggregate [codegen id : 8] +Input [3]: [ctr_state#25, sum#29, count#30] +Keys [1]: [ctr_state#25] +Functions [1]: [avg(ctr_total_return#26)] +Aggregate Attributes [1]: [avg(ctr_total_return#26)#31] +Results [2]: [(avg(ctr_total_return#26)#31 * 1.2) AS (avg(ctr_total_return) * 1.2)#32, ctr_state#25] + +(32) Filter [codegen id : 8] +Input [2]: [(avg(ctr_total_return) * 1.2)#32, ctr_state#25] +Condition : isnotnull((avg(ctr_total_return) * 1.2)#32) + +(33) BroadcastExchange +Input [2]: [(avg(ctr_total_return) * 1.2)#32, ctr_state#25] +Arguments: HashedRelationBroadcastMode(List(input[1, string, true]),false), [plan_id=5] + +(34) BroadcastHashJoin [codegen id : 11] +Left keys [1]: [ctr_state#13] +Right keys [1]: [ctr_state#25] +Join type: Inner +Join condition: (cast(ctr_total_return#14 as decimal(24,7)) > (avg(ctr_total_return) * 1.2)#32) + +(35) Project [codegen id : 11] +Output [2]: [ctr_customer_sk#12, ctr_total_return#14] +Input [5]: [ctr_customer_sk#12, ctr_state#13, ctr_total_return#14, (avg(ctr_total_return) * 1.2)#32, ctr_state#25] + +(36) Scan parquet spark_catalog.default.customer +Output [6]: [c_customer_sk#33, c_customer_id#34, c_current_addr_sk#35, c_salutation#36, c_first_name#37, c_last_name#38] +Batched: true +Location [not included in comparison]/{warehouse_dir}/customer] +PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_current_addr_sk)] +ReadSchema: struct<c_customer_sk:int,c_customer_id:string,c_current_addr_sk:int,c_salutation:string,c_first_name:string,c_last_name:string> + +(37) CometFilter +Input [6]: [c_customer_sk#33, c_customer_id#34, c_current_addr_sk#35, c_salutation#36, c_first_name#37, c_last_name#38] +Condition : (isnotnull(c_customer_sk#33) AND isnotnull(c_current_addr_sk#35)) + +(38) ColumnarToRow [codegen id : 9] +Input [6]: [c_customer_sk#33, c_customer_id#34, c_current_addr_sk#35, c_salutation#36, c_first_name#37, c_last_name#38] + +(39) BroadcastExchange +Input [6]: [c_customer_sk#33, c_customer_id#34, c_current_addr_sk#35, c_salutation#36, c_first_name#37, c_last_name#38] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=6] + +(40) BroadcastHashJoin [codegen id : 11] +Left keys [1]: [ctr_customer_sk#12] +Right keys [1]: [c_customer_sk#33] +Join type: Inner +Join condition: None + +(41) Project [codegen id : 11] +Output [6]: [ctr_total_return#14, c_customer_id#34, c_current_addr_sk#35, c_salutation#36, c_first_name#37, c_last_name#38] +Input [8]: [ctr_customer_sk#12, ctr_total_return#14, c_customer_sk#33, c_customer_id#34, c_current_addr_sk#35, c_salutation#36, c_first_name#37, c_last_name#38] + +(42) Scan parquet spark_catalog.default.customer_address +Output [12]: [ca_address_sk#39, ca_street_number#40, ca_street_name#41, ca_street_type#42, ca_suite_number#43, ca_city#44, ca_county#45, ca_state#46, ca_zip#47, ca_country#48, ca_gmt_offset#49, ca_location_type#50] +Batched: true +Location [not included in comparison]/{warehouse_dir}/customer_address] +PushedFilters: [IsNotNull(ca_state), EqualTo(ca_state,GA), IsNotNull(ca_address_sk)] +ReadSchema: struct<ca_address_sk:int,ca_street_number:string,ca_street_name:string,ca_street_type:string,ca_suite_number:string,ca_city:string,ca_county:string,ca_state:string,ca_zip:string,ca_country:string,ca_gmt_offset:decimal(5,2),ca_location_type:string> + +(43) CometFilter +Input [12]: [ca_address_sk#39, ca_street_number#40, ca_street_name#41, ca_street_type#42, ca_suite_number#43, ca_city#44, ca_county#45, ca_state#46, ca_zip#47, ca_country#48, ca_gmt_offset#49, ca_location_type#50] +Condition : ((isnotnull(ca_state#46) AND (ca_state#46 = GA)) AND isnotnull(ca_address_sk#39)) + +(44) ColumnarToRow [codegen id : 10] +Input [12]: [ca_address_sk#39, ca_street_number#40, ca_street_name#41, ca_street_type#42, ca_suite_number#43, ca_city#44, ca_county#45, ca_state#46, ca_zip#47, ca_country#48, ca_gmt_offset#49, ca_location_type#50] + +(45) BroadcastExchange +Input [12]: [ca_address_sk#39, ca_street_number#40, ca_street_name#41, ca_street_type#42, ca_suite_number#43, ca_city#44, ca_county#45, ca_state#46, ca_zip#47, ca_country#48, ca_gmt_offset#49, ca_location_type#50] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=7] + +(46) BroadcastHashJoin [codegen id : 11] +Left keys [1]: [c_current_addr_sk#35] +Right keys [1]: [ca_address_sk#39] +Join type: Inner +Join condition: None + +(47) Project [codegen id : 11] +Output [16]: [c_customer_id#34, c_salutation#36, c_first_name#37, c_last_name#38, ca_street_number#40, ca_street_name#41, ca_street_type#42, ca_suite_number#43, ca_city#44, ca_county#45, ca_state#46, ca_zip#47, ca_country#48, ca_gmt_offset#49, ca_location_type#50, ctr_total_return#14] +Input [18]: [ctr_total_return#14, c_customer_id#34, c_current_addr_sk#35, c_salutation#36, c_first_name#37, c_last_name#38, ca_address_sk#39, ca_street_number#40, ca_street_name#41, ca_street_type#42, ca_suite_number#43, ca_city#44, ca_county#45, ca_state#46, ca_zip#47, ca_country#48, ca_gmt_offset#49, ca_location_type#50] + +(48) TakeOrderedAndProject +Input [16]: [c_customer_id#34, c_salutation#36, c_first_name#37, c_last_name#38, ca_street_number#40, ca_street_name#41, ca_street_type#42, ca_suite_number#43, ca_city#44, ca_county#45, ca_state#46, ca_zip#47, ca_country#48, ca_gmt_offset#49, ca_location_type#50, ctr_total_return#14] +Arguments: 100, [c_customer_id#34 ASC NULLS FIRST, c_salutation#36 ASC NULLS FIRST, c_first_name#37 ASC NULLS FIRST, c_last_name#38 ASC NULLS FIRST, ca_street_number#40 ASC NULLS FIRST, ca_street_name#41 ASC NULLS FIRST, ca_street_type#42 ASC NULLS FIRST, ca_suite_number#43 ASC NULLS FIRST, ca_city#44 ASC NULLS FIRST, ca_county#45 ASC NULLS FIRST, ca_state#46 ASC NULLS FIRST, ca_zip#47 ASC NULLS FIRST, ca_country#48 ASC NULLS FIRST, ca_gmt_offset#49 ASC NULLS FIRST, ca_location_type#50 ASC NULLS FIRST, ctr_total_return#14 ASC NULLS FIRST], [c_customer_id#34, c_salutation#36, c_first_name#37, c_last_name#38, ca_street_number#40, ca_street_name#41, ca_street_type#42, ca_suite_number#43, ca_city#44, ca_county#45, ca_state#46, ca_zip#47, ca_country#48, ca_gmt_offset#49, ca_location_type#50, ctr_total_return#14] + +===== Subqueries ===== + +Subquery:1 Hosting operator id = 1 Hosting Expression = cr_returned_date_sk#4 IN dynamicpruning#5 +BroadcastExchange (53) ++- * ColumnarToRow (52) + +- CometProject (51) + +- CometFilter (50) + +- CometScan parquet spark_catalog.default.date_dim (49) + + +(49) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#6, d_year#51] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2000), IsNotNull(d_date_sk)] +ReadSchema: struct<d_date_sk:int,d_year:int> + +(50) CometFilter +Input [2]: [d_date_sk#6, d_year#51] +Condition : ((isnotnull(d_year#51) AND (d_year#51 = 2000)) AND isnotnull(d_date_sk#6)) + +(51) CometProject +Input [2]: [d_date_sk#6, d_year#51] +Arguments: [d_date_sk#6], [d_date_sk#6] + +(52) ColumnarToRow [codegen id : 1] +Input [1]: [d_date_sk#6] + +(53) BroadcastExchange +Input [1]: [d_date_sk#6] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=8] + +Subquery:2 Hosting operator id = 17 Hosting Expression = cr_returned_date_sk#18 IN dynamicpruning#5 + + diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q81/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q81/simplified.txt new file mode 100644 index 000000000..6f042847c --- /dev/null +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q81/simplified.txt @@ -0,0 +1,80 @@ +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) + Project [c_customer_id,c_salutation,c_first_name,c_last_name,ca_street_number,ca_street_name,ca_street_type,ca_suite_number,ca_city,ca_county,ca_state,ca_zip,ca_country,ca_gmt_offset,ca_location_type,ctr_total_return] + BroadcastHashJoin [c_current_addr_sk,ca_address_sk] + Project [ctr_total_return,c_customer_id,c_current_addr_sk,c_salutation,c_first_name,c_last_name] + BroadcastHashJoin [ctr_customer_sk,c_customer_sk] + Project [ctr_customer_sk,ctr_total_return] + BroadcastHashJoin [ctr_state,ctr_state,ctr_total_return,(avg(ctr_total_return) * 1.2)] + Filter [ctr_total_return] + HashAggregate [cr_returning_customer_sk,ca_state,sum] [sum(UnscaledValue(cr_return_amt_inc_tax)),ctr_customer_sk,ctr_state,ctr_total_return,sum] + InputAdapter + Exchange [cr_returning_customer_sk,ca_state] #1 + WholeStageCodegen (3) + HashAggregate [cr_returning_customer_sk,ca_state,cr_return_amt_inc_tax] [sum,sum] + Project [cr_returning_customer_sk,cr_return_amt_inc_tax,ca_state] + BroadcastHashJoin [cr_returning_addr_sk,ca_address_sk] + Project [cr_returning_customer_sk,cr_returning_addr_sk,cr_return_amt_inc_tax] + BroadcastHashJoin [cr_returned_date_sk,d_date_sk] + ColumnarToRow + InputAdapter + CometFilter [cr_returning_addr_sk,cr_returning_customer_sk] + CometScan parquet spark_catalog.default.catalog_returns [cr_returning_customer_sk,cr_returning_addr_sk,cr_return_amt_inc_tax,cr_returned_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #2 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_year,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + InputAdapter + ReusedExchange [d_date_sk] #2 + InputAdapter + BroadcastExchange #3 + WholeStageCodegen (2) + ColumnarToRow + InputAdapter + CometFilter [ca_address_sk,ca_state] + CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] + InputAdapter + BroadcastExchange #4 + WholeStageCodegen (8) + Filter [(avg(ctr_total_return) * 1.2)] + HashAggregate [ctr_state,sum,count] [avg(ctr_total_return),(avg(ctr_total_return) * 1.2),sum,count] + InputAdapter + Exchange [ctr_state] #5 + WholeStageCodegen (7) + HashAggregate [ctr_state,ctr_total_return] [sum,count,sum,count] + HashAggregate [cr_returning_customer_sk,ca_state,sum] [sum(UnscaledValue(cr_return_amt_inc_tax)),ctr_state,ctr_total_return,sum] + InputAdapter + Exchange [cr_returning_customer_sk,ca_state] #6 + WholeStageCodegen (6) + HashAggregate [cr_returning_customer_sk,ca_state,cr_return_amt_inc_tax] [sum,sum] + Project [cr_returning_customer_sk,cr_return_amt_inc_tax,ca_state] + BroadcastHashJoin [cr_returning_addr_sk,ca_address_sk] + Project [cr_returning_customer_sk,cr_returning_addr_sk,cr_return_amt_inc_tax] + BroadcastHashJoin [cr_returned_date_sk,d_date_sk] + ColumnarToRow + InputAdapter + CometFilter [cr_returning_addr_sk] + CometScan parquet spark_catalog.default.catalog_returns [cr_returning_customer_sk,cr_returning_addr_sk,cr_return_amt_inc_tax,cr_returned_date_sk] + ReusedSubquery [d_date_sk] #1 + InputAdapter + ReusedExchange [d_date_sk] #2 + InputAdapter + ReusedExchange [ca_address_sk,ca_state] #3 + InputAdapter + BroadcastExchange #7 + WholeStageCodegen (9) + ColumnarToRow + InputAdapter + CometFilter [c_customer_sk,c_current_addr_sk] + CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_current_addr_sk,c_salutation,c_first_name,c_last_name] + InputAdapter + BroadcastExchange #8 + WholeStageCodegen (10) + ColumnarToRow + InputAdapter + CometFilter [ca_state,ca_address_sk] + CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_street_number,ca_street_name,ca_street_type,ca_suite_number,ca_city,ca_county,ca_state,ca_zip,ca_country,ca_gmt_offset,ca_location_type] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q82/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q82/explain.txt new file mode 100644 index 000000000..7609fa520 --- /dev/null +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q82/explain.txt @@ -0,0 +1,179 @@ +== Physical Plan == +TakeOrderedAndProject (25) ++- * HashAggregate (24) + +- Exchange (23) + +- * HashAggregate (22) + +- * Project (21) + +- * BroadcastHashJoin Inner BuildLeft (20) + :- BroadcastExchange (15) + : +- * Project (14) + : +- * BroadcastHashJoin Inner BuildRight (13) + : :- * Project (11) + : : +- * BroadcastHashJoin Inner BuildRight (10) + : : :- * ColumnarToRow (4) + : : : +- CometProject (3) + : : : +- CometFilter (2) + : : : +- CometScan parquet spark_catalog.default.item (1) + : : +- BroadcastExchange (9) + : : +- * ColumnarToRow (8) + : : +- CometProject (7) + : : +- CometFilter (6) + : : +- CometScan parquet spark_catalog.default.inventory (5) + : +- ReusedExchange (12) + +- * ColumnarToRow (19) + +- CometProject (18) + +- CometFilter (17) + +- CometScan parquet spark_catalog.default.store_sales (16) + + +(1) Scan parquet spark_catalog.default.item +Output [5]: [i_item_sk#1, i_item_id#2, i_item_desc#3, i_current_price#4, i_manufact_id#5] +Batched: true +Location [not included in comparison]/{warehouse_dir}/item] +PushedFilters: [IsNotNull(i_current_price), GreaterThanOrEqual(i_current_price,62.00), LessThanOrEqual(i_current_price,92.00), In(i_manufact_id, [129,270,423,821]), IsNotNull(i_item_sk)] +ReadSchema: struct<i_item_sk:int,i_item_id:string,i_item_desc:string,i_current_price:decimal(7,2),i_manufact_id:int> + +(2) CometFilter +Input [5]: [i_item_sk#1, i_item_id#2, i_item_desc#3, i_current_price#4, i_manufact_id#5] +Condition : ((((isnotnull(i_current_price#4) AND (i_current_price#4 >= 62.00)) AND (i_current_price#4 <= 92.00)) AND i_manufact_id#5 IN (129,270,821,423)) AND isnotnull(i_item_sk#1)) + +(3) CometProject +Input [5]: [i_item_sk#1, i_item_id#2, i_item_desc#3, i_current_price#4, i_manufact_id#5] +Arguments: [i_item_sk#1, i_item_id#2, i_item_desc#3, i_current_price#4], [i_item_sk#1, i_item_id#2, i_item_desc#3, i_current_price#4] + +(4) ColumnarToRow [codegen id : 3] +Input [4]: [i_item_sk#1, i_item_id#2, i_item_desc#3, i_current_price#4] + +(5) Scan parquet spark_catalog.default.inventory +Output [3]: [inv_item_sk#6, inv_quantity_on_hand#7, inv_date_sk#8] +Batched: true +Location: InMemoryFileIndex [] +PartitionFilters: [isnotnull(inv_date_sk#8), dynamicpruningexpression(inv_date_sk#8 IN dynamicpruning#9)] +PushedFilters: [IsNotNull(inv_quantity_on_hand), GreaterThanOrEqual(inv_quantity_on_hand,100), LessThanOrEqual(inv_quantity_on_hand,500), IsNotNull(inv_item_sk)] +ReadSchema: struct<inv_item_sk:int,inv_quantity_on_hand:int> + +(6) CometFilter +Input [3]: [inv_item_sk#6, inv_quantity_on_hand#7, inv_date_sk#8] +Condition : (((isnotnull(inv_quantity_on_hand#7) AND (inv_quantity_on_hand#7 >= 100)) AND (inv_quantity_on_hand#7 <= 500)) AND isnotnull(inv_item_sk#6)) + +(7) CometProject +Input [3]: [inv_item_sk#6, inv_quantity_on_hand#7, inv_date_sk#8] +Arguments: [inv_item_sk#6, inv_date_sk#8], [inv_item_sk#6, inv_date_sk#8] + +(8) ColumnarToRow [codegen id : 1] +Input [2]: [inv_item_sk#6, inv_date_sk#8] + +(9) BroadcastExchange +Input [2]: [inv_item_sk#6, inv_date_sk#8] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] + +(10) BroadcastHashJoin [codegen id : 3] +Left keys [1]: [i_item_sk#1] +Right keys [1]: [inv_item_sk#6] +Join type: Inner +Join condition: None + +(11) Project [codegen id : 3] +Output [5]: [i_item_sk#1, i_item_id#2, i_item_desc#3, i_current_price#4, inv_date_sk#8] +Input [6]: [i_item_sk#1, i_item_id#2, i_item_desc#3, i_current_price#4, inv_item_sk#6, inv_date_sk#8] + +(12) ReusedExchange [Reuses operator id: 30] +Output [1]: [d_date_sk#10] + +(13) BroadcastHashJoin [codegen id : 3] +Left keys [1]: [inv_date_sk#8] +Right keys [1]: [d_date_sk#10] +Join type: Inner +Join condition: None + +(14) Project [codegen id : 3] +Output [4]: [i_item_sk#1, i_item_id#2, i_item_desc#3, i_current_price#4] +Input [6]: [i_item_sk#1, i_item_id#2, i_item_desc#3, i_current_price#4, inv_date_sk#8, d_date_sk#10] + +(15) BroadcastExchange +Input [4]: [i_item_sk#1, i_item_id#2, i_item_desc#3, i_current_price#4] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] + +(16) Scan parquet spark_catalog.default.store_sales +Output [2]: [ss_item_sk#11, ss_sold_date_sk#12] +Batched: true +Location [not included in comparison]/{warehouse_dir}/store_sales] +PushedFilters: [IsNotNull(ss_item_sk)] +ReadSchema: struct<ss_item_sk:int> + +(17) CometFilter +Input [2]: [ss_item_sk#11, ss_sold_date_sk#12] +Condition : isnotnull(ss_item_sk#11) + +(18) CometProject +Input [2]: [ss_item_sk#11, ss_sold_date_sk#12] +Arguments: [ss_item_sk#11], [ss_item_sk#11] + +(19) ColumnarToRow +Input [1]: [ss_item_sk#11] + +(20) BroadcastHashJoin [codegen id : 4] +Left keys [1]: [i_item_sk#1] +Right keys [1]: [ss_item_sk#11] +Join type: Inner +Join condition: None + +(21) Project [codegen id : 4] +Output [3]: [i_item_id#2, i_item_desc#3, i_current_price#4] +Input [5]: [i_item_sk#1, i_item_id#2, i_item_desc#3, i_current_price#4, ss_item_sk#11] + +(22) HashAggregate [codegen id : 4] +Input [3]: [i_item_id#2, i_item_desc#3, i_current_price#4] +Keys [3]: [i_item_id#2, i_item_desc#3, i_current_price#4] +Functions: [] +Aggregate Attributes: [] +Results [3]: [i_item_id#2, i_item_desc#3, i_current_price#4] + +(23) Exchange +Input [3]: [i_item_id#2, i_item_desc#3, i_current_price#4] +Arguments: hashpartitioning(i_item_id#2, i_item_desc#3, i_current_price#4, 5), ENSURE_REQUIREMENTS, [plan_id=3] + +(24) HashAggregate [codegen id : 5] +Input [3]: [i_item_id#2, i_item_desc#3, i_current_price#4] +Keys [3]: [i_item_id#2, i_item_desc#3, i_current_price#4] +Functions: [] +Aggregate Attributes: [] +Results [3]: [i_item_id#2, i_item_desc#3, i_current_price#4] + +(25) TakeOrderedAndProject +Input [3]: [i_item_id#2, i_item_desc#3, i_current_price#4] +Arguments: 100, [i_item_id#2 ASC NULLS FIRST], [i_item_id#2, i_item_desc#3, i_current_price#4] + +===== Subqueries ===== + +Subquery:1 Hosting operator id = 5 Hosting Expression = inv_date_sk#8 IN dynamicpruning#9 +BroadcastExchange (30) ++- * ColumnarToRow (29) + +- CometProject (28) + +- CometFilter (27) + +- CometScan parquet spark_catalog.default.date_dim (26) + + +(26) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#10, d_date#13] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,2000-05-25), LessThanOrEqual(d_date,2000-07-24), IsNotNull(d_date_sk)] +ReadSchema: struct<d_date_sk:int,d_date:date> + +(27) CometFilter +Input [2]: [d_date_sk#10, d_date#13] +Condition : (((isnotnull(d_date#13) AND (d_date#13 >= 2000-05-25)) AND (d_date#13 <= 2000-07-24)) AND isnotnull(d_date_sk#10)) + +(28) CometProject +Input [2]: [d_date_sk#10, d_date#13] +Arguments: [d_date_sk#10], [d_date_sk#10] + +(29) ColumnarToRow [codegen id : 1] +Input [1]: [d_date_sk#10] + +(30) BroadcastExchange +Input [1]: [d_date_sk#10] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] + + diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q82/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q82/simplified.txt new file mode 100644 index 000000000..0252eb575 --- /dev/null +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q82/simplified.txt @@ -0,0 +1,44 @@ +TakeOrderedAndProject [i_item_id,i_item_desc,i_current_price] + WholeStageCodegen (5) + HashAggregate [i_item_id,i_item_desc,i_current_price] + InputAdapter + Exchange [i_item_id,i_item_desc,i_current_price] #1 + WholeStageCodegen (4) + HashAggregate [i_item_id,i_item_desc,i_current_price] + Project [i_item_id,i_item_desc,i_current_price] + BroadcastHashJoin [i_item_sk,ss_item_sk] + InputAdapter + BroadcastExchange #2 + WholeStageCodegen (3) + Project [i_item_sk,i_item_id,i_item_desc,i_current_price] + BroadcastHashJoin [inv_date_sk,d_date_sk] + Project [i_item_sk,i_item_id,i_item_desc,i_current_price,inv_date_sk] + BroadcastHashJoin [i_item_sk,inv_item_sk] + ColumnarToRow + InputAdapter + CometProject [i_item_sk,i_item_id,i_item_desc,i_current_price] + CometFilter [i_current_price,i_manufact_id,i_item_sk] + CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_manufact_id] + InputAdapter + BroadcastExchange #3 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [inv_item_sk,inv_date_sk] + CometFilter [inv_quantity_on_hand,inv_item_sk] + CometScan parquet spark_catalog.default.inventory [inv_item_sk,inv_quantity_on_hand,inv_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #4 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] + InputAdapter + ReusedExchange [d_date_sk] #4 + ColumnarToRow + InputAdapter + CometProject [ss_item_sk] + CometFilter [ss_item_sk] + CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_sold_date_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q83.ansi/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q83.ansi/explain.txt new file mode 100644 index 000000000..5183e0275 --- /dev/null +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q83.ansi/explain.txt @@ -0,0 +1,372 @@ +== Physical Plan == +TakeOrderedAndProject (46) ++- * Project (45) + +- * BroadcastHashJoin Inner BuildRight (44) + :- * Project (30) + : +- * BroadcastHashJoin Inner BuildRight (29) + : :- * HashAggregate (15) + : : +- Exchange (14) + : : +- * HashAggregate (13) + : : +- * Project (12) + : : +- * BroadcastHashJoin Inner BuildRight (11) + : : :- * Project (9) + : : : +- * BroadcastHashJoin Inner BuildRight (8) + : : : :- * ColumnarToRow (3) + : : : : +- CometFilter (2) + : : : : +- CometScan parquet spark_catalog.default.store_returns (1) + : : : +- BroadcastExchange (7) + : : : +- * ColumnarToRow (6) + : : : +- CometFilter (5) + : : : +- CometScan parquet spark_catalog.default.item (4) + : : +- ReusedExchange (10) + : +- BroadcastExchange (28) + : +- * HashAggregate (27) + : +- Exchange (26) + : +- * HashAggregate (25) + : +- * Project (24) + : +- * BroadcastHashJoin Inner BuildRight (23) + : :- * Project (21) + : : +- * BroadcastHashJoin Inner BuildRight (20) + : : :- * ColumnarToRow (18) + : : : +- CometFilter (17) + : : : +- CometScan parquet spark_catalog.default.catalog_returns (16) + : : +- ReusedExchange (19) + : +- ReusedExchange (22) + +- BroadcastExchange (43) + +- * HashAggregate (42) + +- Exchange (41) + +- * HashAggregate (40) + +- * Project (39) + +- * BroadcastHashJoin Inner BuildRight (38) + :- * Project (36) + : +- * BroadcastHashJoin Inner BuildRight (35) + : :- * ColumnarToRow (33) + : : +- CometFilter (32) + : : +- CometScan parquet spark_catalog.default.web_returns (31) + : +- ReusedExchange (34) + +- ReusedExchange (37) + + +(1) Scan parquet spark_catalog.default.store_returns +Output [3]: [sr_item_sk#1, sr_return_quantity#2, sr_returned_date_sk#3] +Batched: true +Location: InMemoryFileIndex [] +PartitionFilters: [isnotnull(sr_returned_date_sk#3), dynamicpruningexpression(sr_returned_date_sk#3 IN dynamicpruning#4)] +PushedFilters: [IsNotNull(sr_item_sk)] +ReadSchema: struct<sr_item_sk:int,sr_return_quantity:int> + +(2) CometFilter +Input [3]: [sr_item_sk#1, sr_return_quantity#2, sr_returned_date_sk#3] +Condition : isnotnull(sr_item_sk#1) + +(3) ColumnarToRow [codegen id : 5] +Input [3]: [sr_item_sk#1, sr_return_quantity#2, sr_returned_date_sk#3] + +(4) Scan parquet spark_catalog.default.item +Output [2]: [i_item_sk#5, i_item_id#6] +Batched: true +Location [not included in comparison]/{warehouse_dir}/item] +PushedFilters: [IsNotNull(i_item_sk), IsNotNull(i_item_id)] +ReadSchema: struct<i_item_sk:int,i_item_id:string> + +(5) CometFilter +Input [2]: [i_item_sk#5, i_item_id#6] +Condition : (isnotnull(i_item_sk#5) AND isnotnull(i_item_id#6)) + +(6) ColumnarToRow [codegen id : 1] +Input [2]: [i_item_sk#5, i_item_id#6] + +(7) BroadcastExchange +Input [2]: [i_item_sk#5, i_item_id#6] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=1] + +(8) BroadcastHashJoin [codegen id : 5] +Left keys [1]: [sr_item_sk#1] +Right keys [1]: [i_item_sk#5] +Join type: Inner +Join condition: None + +(9) Project [codegen id : 5] +Output [3]: [sr_return_quantity#2, sr_returned_date_sk#3, i_item_id#6] +Input [5]: [sr_item_sk#1, sr_return_quantity#2, sr_returned_date_sk#3, i_item_sk#5, i_item_id#6] + +(10) ReusedExchange [Reuses operator id: 62] +Output [1]: [d_date_sk#7] + +(11) BroadcastHashJoin [codegen id : 5] +Left keys [1]: [sr_returned_date_sk#3] +Right keys [1]: [d_date_sk#7] +Join type: Inner +Join condition: None + +(12) Project [codegen id : 5] +Output [2]: [sr_return_quantity#2, i_item_id#6] +Input [4]: [sr_return_quantity#2, sr_returned_date_sk#3, i_item_id#6, d_date_sk#7] + +(13) HashAggregate [codegen id : 5] +Input [2]: [sr_return_quantity#2, i_item_id#6] +Keys [1]: [i_item_id#6] +Functions [1]: [partial_sum(sr_return_quantity#2)] +Aggregate Attributes [1]: [sum#8] +Results [2]: [i_item_id#6, sum#9] + +(14) Exchange +Input [2]: [i_item_id#6, sum#9] +Arguments: hashpartitioning(i_item_id#6, 5), ENSURE_REQUIREMENTS, [plan_id=2] + +(15) HashAggregate [codegen id : 18] +Input [2]: [i_item_id#6, sum#9] +Keys [1]: [i_item_id#6] +Functions [1]: [sum(sr_return_quantity#2)] +Aggregate Attributes [1]: [sum(sr_return_quantity#2)#10] +Results [2]: [i_item_id#6 AS item_id#11, sum(sr_return_quantity#2)#10 AS sr_item_qty#12] + +(16) Scan parquet spark_catalog.default.catalog_returns +Output [3]: [cr_item_sk#13, cr_return_quantity#14, cr_returned_date_sk#15] +Batched: true +Location: InMemoryFileIndex [] +PartitionFilters: [isnotnull(cr_returned_date_sk#15), dynamicpruningexpression(cr_returned_date_sk#15 IN dynamicpruning#16)] +PushedFilters: [IsNotNull(cr_item_sk)] +ReadSchema: struct<cr_item_sk:int,cr_return_quantity:int> + +(17) CometFilter +Input [3]: [cr_item_sk#13, cr_return_quantity#14, cr_returned_date_sk#15] +Condition : isnotnull(cr_item_sk#13) + +(18) ColumnarToRow [codegen id : 10] +Input [3]: [cr_item_sk#13, cr_return_quantity#14, cr_returned_date_sk#15] + +(19) ReusedExchange [Reuses operator id: 7] +Output [2]: [i_item_sk#17, i_item_id#18] + +(20) BroadcastHashJoin [codegen id : 10] +Left keys [1]: [cr_item_sk#13] +Right keys [1]: [i_item_sk#17] +Join type: Inner +Join condition: None + +(21) Project [codegen id : 10] +Output [3]: [cr_return_quantity#14, cr_returned_date_sk#15, i_item_id#18] +Input [5]: [cr_item_sk#13, cr_return_quantity#14, cr_returned_date_sk#15, i_item_sk#17, i_item_id#18] + +(22) ReusedExchange [Reuses operator id: 62] +Output [1]: [d_date_sk#19] + +(23) BroadcastHashJoin [codegen id : 10] +Left keys [1]: [cr_returned_date_sk#15] +Right keys [1]: [d_date_sk#19] +Join type: Inner +Join condition: None + +(24) Project [codegen id : 10] +Output [2]: [cr_return_quantity#14, i_item_id#18] +Input [4]: [cr_return_quantity#14, cr_returned_date_sk#15, i_item_id#18, d_date_sk#19] + +(25) HashAggregate [codegen id : 10] +Input [2]: [cr_return_quantity#14, i_item_id#18] +Keys [1]: [i_item_id#18] +Functions [1]: [partial_sum(cr_return_quantity#14)] +Aggregate Attributes [1]: [sum#20] +Results [2]: [i_item_id#18, sum#21] + +(26) Exchange +Input [2]: [i_item_id#18, sum#21] +Arguments: hashpartitioning(i_item_id#18, 5), ENSURE_REQUIREMENTS, [plan_id=3] + +(27) HashAggregate [codegen id : 11] +Input [2]: [i_item_id#18, sum#21] +Keys [1]: [i_item_id#18] +Functions [1]: [sum(cr_return_quantity#14)] +Aggregate Attributes [1]: [sum(cr_return_quantity#14)#22] +Results [2]: [i_item_id#18 AS item_id#23, sum(cr_return_quantity#14)#22 AS cr_item_qty#24] + +(28) BroadcastExchange +Input [2]: [item_id#23, cr_item_qty#24] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=4] + +(29) BroadcastHashJoin [codegen id : 18] +Left keys [1]: [item_id#11] +Right keys [1]: [item_id#23] +Join type: Inner +Join condition: None + +(30) Project [codegen id : 18] +Output [3]: [item_id#11, sr_item_qty#12, cr_item_qty#24] +Input [4]: [item_id#11, sr_item_qty#12, item_id#23, cr_item_qty#24] + +(31) Scan parquet spark_catalog.default.web_returns +Output [3]: [wr_item_sk#25, wr_return_quantity#26, wr_returned_date_sk#27] +Batched: true +Location: InMemoryFileIndex [] +PartitionFilters: [isnotnull(wr_returned_date_sk#27), dynamicpruningexpression(wr_returned_date_sk#27 IN dynamicpruning#28)] +PushedFilters: [IsNotNull(wr_item_sk)] +ReadSchema: struct<wr_item_sk:int,wr_return_quantity:int> + +(32) CometFilter +Input [3]: [wr_item_sk#25, wr_return_quantity#26, wr_returned_date_sk#27] +Condition : isnotnull(wr_item_sk#25) + +(33) ColumnarToRow [codegen id : 16] +Input [3]: [wr_item_sk#25, wr_return_quantity#26, wr_returned_date_sk#27] + +(34) ReusedExchange [Reuses operator id: 7] +Output [2]: [i_item_sk#29, i_item_id#30] + +(35) BroadcastHashJoin [codegen id : 16] +Left keys [1]: [wr_item_sk#25] +Right keys [1]: [i_item_sk#29] +Join type: Inner +Join condition: None + +(36) Project [codegen id : 16] +Output [3]: [wr_return_quantity#26, wr_returned_date_sk#27, i_item_id#30] +Input [5]: [wr_item_sk#25, wr_return_quantity#26, wr_returned_date_sk#27, i_item_sk#29, i_item_id#30] + +(37) ReusedExchange [Reuses operator id: 62] +Output [1]: [d_date_sk#31] + +(38) BroadcastHashJoin [codegen id : 16] +Left keys [1]: [wr_returned_date_sk#27] +Right keys [1]: [d_date_sk#31] +Join type: Inner +Join condition: None + +(39) Project [codegen id : 16] +Output [2]: [wr_return_quantity#26, i_item_id#30] +Input [4]: [wr_return_quantity#26, wr_returned_date_sk#27, i_item_id#30, d_date_sk#31] + +(40) HashAggregate [codegen id : 16] +Input [2]: [wr_return_quantity#26, i_item_id#30] +Keys [1]: [i_item_id#30] +Functions [1]: [partial_sum(wr_return_quantity#26)] +Aggregate Attributes [1]: [sum#32] +Results [2]: [i_item_id#30, sum#33] + +(41) Exchange +Input [2]: [i_item_id#30, sum#33] +Arguments: hashpartitioning(i_item_id#30, 5), ENSURE_REQUIREMENTS, [plan_id=5] + +(42) HashAggregate [codegen id : 17] +Input [2]: [i_item_id#30, sum#33] +Keys [1]: [i_item_id#30] +Functions [1]: [sum(wr_return_quantity#26)] +Aggregate Attributes [1]: [sum(wr_return_quantity#26)#34] +Results [2]: [i_item_id#30 AS item_id#35, sum(wr_return_quantity#26)#34 AS wr_item_qty#36] + +(43) BroadcastExchange +Input [2]: [item_id#35, wr_item_qty#36] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=6] + +(44) BroadcastHashJoin [codegen id : 18] +Left keys [1]: [item_id#11] +Right keys [1]: [item_id#35] +Join type: Inner +Join condition: None + +(45) Project [codegen id : 18] +Output [8]: [item_id#11, sr_item_qty#12, (((cast(sr_item_qty#12 as double) / cast(((sr_item_qty#12 + cr_item_qty#24) + wr_item_qty#36) as double)) / 3.0) * 100.0) AS sr_dev#37, cr_item_qty#24, (((cast(cr_item_qty#24 as double) / cast(((sr_item_qty#12 + cr_item_qty#24) + wr_item_qty#36) as double)) / 3.0) * 100.0) AS cr_dev#38, wr_item_qty#36, (((cast(wr_item_qty#36 as double) / cast(((sr_item_qty#12 + cr_item_qty#24) + wr_item_qty#36) as double)) / 3.0) * 100.0) AS wr_dev#39, (cast(((sr_item_qty#12 + cr_item_qty#24) + wr_item_qty#36) as decimal(20,0)) / 3.0) AS average#40] +Input [5]: [item_id#11, sr_item_qty#12, cr_item_qty#24, item_id#35, wr_item_qty#36] + +(46) TakeOrderedAndProject +Input [8]: [item_id#11, sr_item_qty#12, sr_dev#37, cr_item_qty#24, cr_dev#38, wr_item_qty#36, wr_dev#39, average#40] +Arguments: 100, [item_id#11 ASC NULLS FIRST, sr_item_qty#12 ASC NULLS FIRST], [item_id#11, sr_item_qty#12, sr_dev#37, cr_item_qty#24, cr_dev#38, wr_item_qty#36, wr_dev#39, average#40] + +===== Subqueries ===== + +Subquery:1 Hosting operator id = 1 Hosting Expression = sr_returned_date_sk#3 IN dynamicpruning#4 +BroadcastExchange (62) ++- * Project (61) + +- * BroadcastHashJoin LeftSemi BuildRight (60) + :- * ColumnarToRow (49) + : +- CometFilter (48) + : +- CometScan parquet spark_catalog.default.date_dim (47) + +- BroadcastExchange (59) + +- * Project (58) + +- * BroadcastHashJoin LeftSemi BuildRight (57) + :- * ColumnarToRow (51) + : +- CometScan parquet spark_catalog.default.date_dim (50) + +- BroadcastExchange (56) + +- * ColumnarToRow (55) + +- CometProject (54) + +- CometFilter (53) + +- CometScan parquet spark_catalog.default.date_dim (52) + + +(47) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#7, d_date#41] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_date_sk)] +ReadSchema: struct<d_date_sk:int,d_date:date> + +(48) CometFilter +Input [2]: [d_date_sk#7, d_date#41] +Condition : isnotnull(d_date_sk#7) + +(49) ColumnarToRow [codegen id : 3] +Input [2]: [d_date_sk#7, d_date#41] + +(50) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date#42, d_week_seq#43] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +ReadSchema: struct<d_date:date,d_week_seq:int> + +(51) ColumnarToRow [codegen id : 2] +Input [2]: [d_date#42, d_week_seq#43] + +(52) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date#44, d_week_seq#45] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [In(d_date, [2000-06-30,2000-09-27,2000-11-17])] +ReadSchema: struct<d_date:date,d_week_seq:int> + +(53) CometFilter +Input [2]: [d_date#44, d_week_seq#45] +Condition : d_date#44 IN (2000-06-30,2000-09-27,2000-11-17) + +(54) CometProject +Input [2]: [d_date#44, d_week_seq#45] +Arguments: [d_week_seq#45], [d_week_seq#45] + +(55) ColumnarToRow [codegen id : 1] +Input [1]: [d_week_seq#45] + +(56) BroadcastExchange +Input [1]: [d_week_seq#45] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=7] + +(57) BroadcastHashJoin [codegen id : 2] +Left keys [1]: [d_week_seq#43] +Right keys [1]: [d_week_seq#45] +Join type: LeftSemi +Join condition: None + +(58) Project [codegen id : 2] +Output [1]: [d_date#42] +Input [2]: [d_date#42, d_week_seq#43] + +(59) BroadcastExchange +Input [1]: [d_date#42] +Arguments: HashedRelationBroadcastMode(List(input[0, date, true]),false), [plan_id=8] + +(60) BroadcastHashJoin [codegen id : 3] +Left keys [1]: [d_date#41] +Right keys [1]: [d_date#42] +Join type: LeftSemi +Join condition: None + +(61) Project [codegen id : 3] +Output [1]: [d_date_sk#7] +Input [2]: [d_date_sk#7, d_date#41] + +(62) BroadcastExchange +Input [1]: [d_date_sk#7] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=9] + +Subquery:2 Hosting operator id = 16 Hosting Expression = cr_returned_date_sk#15 IN dynamicpruning#4 + +Subquery:3 Hosting operator id = 31 Hosting Expression = wr_returned_date_sk#27 IN dynamicpruning#4 + + diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q83.ansi/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q83.ansi/simplified.txt new file mode 100644 index 000000000..a8f1ba3f1 --- /dev/null +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q83.ansi/simplified.txt @@ -0,0 +1,95 @@ +TakeOrderedAndProject [item_id,sr_item_qty,sr_dev,cr_item_qty,cr_dev,wr_item_qty,wr_dev,average] + WholeStageCodegen (18) + Project [item_id,sr_item_qty,cr_item_qty,wr_item_qty] + BroadcastHashJoin [item_id,item_id] + Project [item_id,sr_item_qty,cr_item_qty] + BroadcastHashJoin [item_id,item_id] + HashAggregate [i_item_id,sum] [sum(sr_return_quantity),item_id,sr_item_qty,sum] + InputAdapter + Exchange [i_item_id] #1 + WholeStageCodegen (5) + HashAggregate [i_item_id,sr_return_quantity] [sum,sum] + Project [sr_return_quantity,i_item_id] + BroadcastHashJoin [sr_returned_date_sk,d_date_sk] + Project [sr_return_quantity,sr_returned_date_sk,i_item_id] + BroadcastHashJoin [sr_item_sk,i_item_sk] + ColumnarToRow + InputAdapter + CometFilter [sr_item_sk] + CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_return_quantity,sr_returned_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #2 + WholeStageCodegen (3) + Project [d_date_sk] + BroadcastHashJoin [d_date,d_date] + ColumnarToRow + InputAdapter + CometFilter [d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] + InputAdapter + BroadcastExchange #3 + WholeStageCodegen (2) + Project [d_date] + BroadcastHashJoin [d_week_seq,d_week_seq] + ColumnarToRow + InputAdapter + CometScan parquet spark_catalog.default.date_dim [d_date,d_week_seq] + InputAdapter + BroadcastExchange #4 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_week_seq] + CometFilter [d_date] + CometScan parquet spark_catalog.default.date_dim [d_date,d_week_seq] + InputAdapter + BroadcastExchange #5 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometFilter [i_item_sk,i_item_id] + CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id] + InputAdapter + ReusedExchange [d_date_sk] #2 + InputAdapter + BroadcastExchange #6 + WholeStageCodegen (11) + HashAggregate [i_item_id,sum] [sum(cr_return_quantity),item_id,cr_item_qty,sum] + InputAdapter + Exchange [i_item_id] #7 + WholeStageCodegen (10) + HashAggregate [i_item_id,cr_return_quantity] [sum,sum] + Project [cr_return_quantity,i_item_id] + BroadcastHashJoin [cr_returned_date_sk,d_date_sk] + Project [cr_return_quantity,cr_returned_date_sk,i_item_id] + BroadcastHashJoin [cr_item_sk,i_item_sk] + ColumnarToRow + InputAdapter + CometFilter [cr_item_sk] + CometScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_return_quantity,cr_returned_date_sk] + ReusedSubquery [d_date_sk] #1 + InputAdapter + ReusedExchange [i_item_sk,i_item_id] #5 + InputAdapter + ReusedExchange [d_date_sk] #2 + InputAdapter + BroadcastExchange #8 + WholeStageCodegen (17) + HashAggregate [i_item_id,sum] [sum(wr_return_quantity),item_id,wr_item_qty,sum] + InputAdapter + Exchange [i_item_id] #9 + WholeStageCodegen (16) + HashAggregate [i_item_id,wr_return_quantity] [sum,sum] + Project [wr_return_quantity,i_item_id] + BroadcastHashJoin [wr_returned_date_sk,d_date_sk] + Project [wr_return_quantity,wr_returned_date_sk,i_item_id] + BroadcastHashJoin [wr_item_sk,i_item_sk] + ColumnarToRow + InputAdapter + CometFilter [wr_item_sk] + CometScan parquet spark_catalog.default.web_returns [wr_item_sk,wr_return_quantity,wr_returned_date_sk] + ReusedSubquery [d_date_sk] #1 + InputAdapter + ReusedExchange [i_item_sk,i_item_id] #5 + InputAdapter + ReusedExchange [d_date_sk] #2 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q84/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q84/explain.txt new file mode 100644 index 000000000..8dc935d1d --- /dev/null +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q84/explain.txt @@ -0,0 +1,210 @@ +== Physical Plan == +TakeOrderedAndProject (37) ++- * Project (36) + +- * BroadcastHashJoin Inner BuildLeft (35) + :- BroadcastExchange (30) + : +- * Project (29) + : +- * BroadcastHashJoin Inner BuildRight (28) + : :- * Project (22) + : : +- * BroadcastHashJoin Inner BuildRight (21) + : : :- * Project (16) + : : : +- * BroadcastHashJoin Inner BuildRight (15) + : : : :- * Project (10) + : : : : +- * BroadcastHashJoin Inner BuildRight (9) + : : : : :- * ColumnarToRow (3) + : : : : : +- CometFilter (2) + : : : : : +- CometScan parquet spark_catalog.default.customer (1) + : : : : +- BroadcastExchange (8) + : : : : +- * ColumnarToRow (7) + : : : : +- CometProject (6) + : : : : +- CometFilter (5) + : : : : +- CometScan parquet spark_catalog.default.customer_address (4) + : : : +- BroadcastExchange (14) + : : : +- * ColumnarToRow (13) + : : : +- CometFilter (12) + : : : +- CometScan parquet spark_catalog.default.customer_demographics (11) + : : +- BroadcastExchange (20) + : : +- * ColumnarToRow (19) + : : +- CometFilter (18) + : : +- CometScan parquet spark_catalog.default.household_demographics (17) + : +- BroadcastExchange (27) + : +- * ColumnarToRow (26) + : +- CometProject (25) + : +- CometFilter (24) + : +- CometScan parquet spark_catalog.default.income_band (23) + +- * ColumnarToRow (34) + +- CometProject (33) + +- CometFilter (32) + +- CometScan parquet spark_catalog.default.store_returns (31) + + +(1) Scan parquet spark_catalog.default.customer +Output [6]: [c_customer_id#1, c_current_cdemo_sk#2, c_current_hdemo_sk#3, c_current_addr_sk#4, c_first_name#5, c_last_name#6] +Batched: true +Location [not included in comparison]/{warehouse_dir}/customer] +PushedFilters: [IsNotNull(c_current_addr_sk), IsNotNull(c_current_cdemo_sk), IsNotNull(c_current_hdemo_sk)] +ReadSchema: struct<c_customer_id:string,c_current_cdemo_sk:int,c_current_hdemo_sk:int,c_current_addr_sk:int,c_first_name:string,c_last_name:string> + +(2) CometFilter +Input [6]: [c_customer_id#1, c_current_cdemo_sk#2, c_current_hdemo_sk#3, c_current_addr_sk#4, c_first_name#5, c_last_name#6] +Condition : ((isnotnull(c_current_addr_sk#4) AND isnotnull(c_current_cdemo_sk#2)) AND isnotnull(c_current_hdemo_sk#3)) + +(3) ColumnarToRow [codegen id : 5] +Input [6]: [c_customer_id#1, c_current_cdemo_sk#2, c_current_hdemo_sk#3, c_current_addr_sk#4, c_first_name#5, c_last_name#6] + +(4) Scan parquet spark_catalog.default.customer_address +Output [2]: [ca_address_sk#7, ca_city#8] +Batched: true +Location [not included in comparison]/{warehouse_dir}/customer_address] +PushedFilters: [IsNotNull(ca_city), EqualTo(ca_city,Edgewood), IsNotNull(ca_address_sk)] +ReadSchema: struct<ca_address_sk:int,ca_city:string> + +(5) CometFilter +Input [2]: [ca_address_sk#7, ca_city#8] +Condition : ((isnotnull(ca_city#8) AND (ca_city#8 = Edgewood)) AND isnotnull(ca_address_sk#7)) + +(6) CometProject +Input [2]: [ca_address_sk#7, ca_city#8] +Arguments: [ca_address_sk#7], [ca_address_sk#7] + +(7) ColumnarToRow [codegen id : 1] +Input [1]: [ca_address_sk#7] + +(8) BroadcastExchange +Input [1]: [ca_address_sk#7] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] + +(9) BroadcastHashJoin [codegen id : 5] +Left keys [1]: [c_current_addr_sk#4] +Right keys [1]: [ca_address_sk#7] +Join type: Inner +Join condition: None + +(10) Project [codegen id : 5] +Output [5]: [c_customer_id#1, c_current_cdemo_sk#2, c_current_hdemo_sk#3, c_first_name#5, c_last_name#6] +Input [7]: [c_customer_id#1, c_current_cdemo_sk#2, c_current_hdemo_sk#3, c_current_addr_sk#4, c_first_name#5, c_last_name#6, ca_address_sk#7] + +(11) Scan parquet spark_catalog.default.customer_demographics +Output [1]: [cd_demo_sk#9] +Batched: true +Location [not included in comparison]/{warehouse_dir}/customer_demographics] +PushedFilters: [IsNotNull(cd_demo_sk)] +ReadSchema: struct<cd_demo_sk:int> + +(12) CometFilter +Input [1]: [cd_demo_sk#9] +Condition : isnotnull(cd_demo_sk#9) + +(13) ColumnarToRow [codegen id : 2] +Input [1]: [cd_demo_sk#9] + +(14) BroadcastExchange +Input [1]: [cd_demo_sk#9] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=2] + +(15) BroadcastHashJoin [codegen id : 5] +Left keys [1]: [c_current_cdemo_sk#2] +Right keys [1]: [cd_demo_sk#9] +Join type: Inner +Join condition: None + +(16) Project [codegen id : 5] +Output [5]: [c_customer_id#1, c_current_hdemo_sk#3, c_first_name#5, c_last_name#6, cd_demo_sk#9] +Input [6]: [c_customer_id#1, c_current_cdemo_sk#2, c_current_hdemo_sk#3, c_first_name#5, c_last_name#6, cd_demo_sk#9] + +(17) Scan parquet spark_catalog.default.household_demographics +Output [2]: [hd_demo_sk#10, hd_income_band_sk#11] +Batched: true +Location [not included in comparison]/{warehouse_dir}/household_demographics] +PushedFilters: [IsNotNull(hd_demo_sk), IsNotNull(hd_income_band_sk)] +ReadSchema: struct<hd_demo_sk:int,hd_income_band_sk:int> + +(18) CometFilter +Input [2]: [hd_demo_sk#10, hd_income_band_sk#11] +Condition : (isnotnull(hd_demo_sk#10) AND isnotnull(hd_income_band_sk#11)) + +(19) ColumnarToRow [codegen id : 3] +Input [2]: [hd_demo_sk#10, hd_income_band_sk#11] + +(20) BroadcastExchange +Input [2]: [hd_demo_sk#10, hd_income_band_sk#11] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=3] + +(21) BroadcastHashJoin [codegen id : 5] +Left keys [1]: [c_current_hdemo_sk#3] +Right keys [1]: [hd_demo_sk#10] +Join type: Inner +Join condition: None + +(22) Project [codegen id : 5] +Output [5]: [c_customer_id#1, c_first_name#5, c_last_name#6, cd_demo_sk#9, hd_income_band_sk#11] +Input [7]: [c_customer_id#1, c_current_hdemo_sk#3, c_first_name#5, c_last_name#6, cd_demo_sk#9, hd_demo_sk#10, hd_income_band_sk#11] + +(23) Scan parquet spark_catalog.default.income_band +Output [3]: [ib_income_band_sk#12, ib_lower_bound#13, ib_upper_bound#14] +Batched: true +Location [not included in comparison]/{warehouse_dir}/income_band] +PushedFilters: [IsNotNull(ib_lower_bound), IsNotNull(ib_upper_bound), GreaterThanOrEqual(ib_lower_bound,38128), LessThanOrEqual(ib_upper_bound,88128), IsNotNull(ib_income_band_sk)] +ReadSchema: struct<ib_income_band_sk:int,ib_lower_bound:int,ib_upper_bound:int> + +(24) CometFilter +Input [3]: [ib_income_band_sk#12, ib_lower_bound#13, ib_upper_bound#14] +Condition : ((((isnotnull(ib_lower_bound#13) AND isnotnull(ib_upper_bound#14)) AND (ib_lower_bound#13 >= 38128)) AND (ib_upper_bound#14 <= 88128)) AND isnotnull(ib_income_band_sk#12)) + +(25) CometProject +Input [3]: [ib_income_band_sk#12, ib_lower_bound#13, ib_upper_bound#14] +Arguments: [ib_income_band_sk#12], [ib_income_band_sk#12] + +(26) ColumnarToRow [codegen id : 4] +Input [1]: [ib_income_band_sk#12] + +(27) BroadcastExchange +Input [1]: [ib_income_band_sk#12] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] + +(28) BroadcastHashJoin [codegen id : 5] +Left keys [1]: [hd_income_band_sk#11] +Right keys [1]: [ib_income_band_sk#12] +Join type: Inner +Join condition: None + +(29) Project [codegen id : 5] +Output [4]: [c_customer_id#1, c_first_name#5, c_last_name#6, cd_demo_sk#9] +Input [6]: [c_customer_id#1, c_first_name#5, c_last_name#6, cd_demo_sk#9, hd_income_band_sk#11, ib_income_band_sk#12] + +(30) BroadcastExchange +Input [4]: [c_customer_id#1, c_first_name#5, c_last_name#6, cd_demo_sk#9] +Arguments: HashedRelationBroadcastMode(List(cast(input[3, int, true] as bigint)),false), [plan_id=5] + +(31) Scan parquet spark_catalog.default.store_returns +Output [2]: [sr_cdemo_sk#15, sr_returned_date_sk#16] +Batched: true +Location [not included in comparison]/{warehouse_dir}/store_returns] +PushedFilters: [IsNotNull(sr_cdemo_sk)] +ReadSchema: struct<sr_cdemo_sk:int> + +(32) CometFilter +Input [2]: [sr_cdemo_sk#15, sr_returned_date_sk#16] +Condition : isnotnull(sr_cdemo_sk#15) + +(33) CometProject +Input [2]: [sr_cdemo_sk#15, sr_returned_date_sk#16] +Arguments: [sr_cdemo_sk#15], [sr_cdemo_sk#15] + +(34) ColumnarToRow +Input [1]: [sr_cdemo_sk#15] + +(35) BroadcastHashJoin [codegen id : 6] +Left keys [1]: [cd_demo_sk#9] +Right keys [1]: [sr_cdemo_sk#15] +Join type: Inner +Join condition: None + +(36) Project [codegen id : 6] +Output [3]: [c_customer_id#1 AS customer_id#17, concat(c_last_name#6, , , c_first_name#5) AS customername#18, c_customer_id#1] +Input [5]: [c_customer_id#1, c_first_name#5, c_last_name#6, cd_demo_sk#9, sr_cdemo_sk#15] + +(37) TakeOrderedAndProject +Input [3]: [customer_id#17, customername#18, c_customer_id#1] +Arguments: 100, [c_customer_id#1 ASC NULLS FIRST], [customer_id#17, customername#18] + diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q84/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q84/simplified.txt new file mode 100644 index 000000000..be3451d29 --- /dev/null +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q84/simplified.txt @@ -0,0 +1,54 @@ +TakeOrderedAndProject [c_customer_id,customer_id,customername] + WholeStageCodegen (6) + Project [c_customer_id,c_last_name,c_first_name] + BroadcastHashJoin [cd_demo_sk,sr_cdemo_sk] + InputAdapter + BroadcastExchange #1 + WholeStageCodegen (5) + Project [c_customer_id,c_first_name,c_last_name,cd_demo_sk] + BroadcastHashJoin [hd_income_band_sk,ib_income_band_sk] + Project [c_customer_id,c_first_name,c_last_name,cd_demo_sk,hd_income_band_sk] + BroadcastHashJoin [c_current_hdemo_sk,hd_demo_sk] + Project [c_customer_id,c_current_hdemo_sk,c_first_name,c_last_name,cd_demo_sk] + BroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] + Project [c_customer_id,c_current_cdemo_sk,c_current_hdemo_sk,c_first_name,c_last_name] + BroadcastHashJoin [c_current_addr_sk,ca_address_sk] + ColumnarToRow + InputAdapter + CometFilter [c_current_addr_sk,c_current_cdemo_sk,c_current_hdemo_sk] + CometScan parquet spark_catalog.default.customer [c_customer_id,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_name,c_last_name] + InputAdapter + BroadcastExchange #2 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [ca_address_sk] + CometFilter [ca_city,ca_address_sk] + CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_city] + InputAdapter + BroadcastExchange #3 + WholeStageCodegen (2) + ColumnarToRow + InputAdapter + CometFilter [cd_demo_sk] + CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk] + InputAdapter + BroadcastExchange #4 + WholeStageCodegen (3) + ColumnarToRow + InputAdapter + CometFilter [hd_demo_sk,hd_income_band_sk] + CometScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_income_band_sk] + InputAdapter + BroadcastExchange #5 + WholeStageCodegen (4) + ColumnarToRow + InputAdapter + CometProject [ib_income_band_sk] + CometFilter [ib_lower_bound,ib_upper_bound,ib_income_band_sk] + CometScan parquet spark_catalog.default.income_band [ib_income_band_sk,ib_lower_bound,ib_upper_bound] + ColumnarToRow + InputAdapter + CometProject [sr_cdemo_sk] + CometFilter [sr_cdemo_sk] + CometScan parquet spark_catalog.default.store_returns [sr_cdemo_sk,sr_returned_date_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q85/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q85/explain.txt new file mode 100644 index 000000000..ce1cc2262 --- /dev/null +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q85/explain.txt @@ -0,0 +1,305 @@ +== Physical Plan == +TakeOrderedAndProject (47) ++- * HashAggregate (46) + +- Exchange (45) + +- * HashAggregate (44) + +- * Project (43) + +- * BroadcastHashJoin Inner BuildRight (42) + :- * Project (37) + : +- * BroadcastHashJoin Inner BuildRight (36) + : :- * Project (34) + : : +- * BroadcastHashJoin Inner BuildRight (33) + : : :- * Project (27) + : : : +- * BroadcastHashJoin Inner BuildRight (26) + : : : :- * Project (21) + : : : : +- * BroadcastHashJoin Inner BuildRight (20) + : : : : :- * Project (15) + : : : : : +- * BroadcastHashJoin Inner BuildRight (14) + : : : : : :- * ColumnarToRow (9) + : : : : : : +- CometProject (8) + : : : : : : +- CometBroadcastHashJoin (7) + : : : : : : :- CometBroadcastExchange (3) + : : : : : : : +- CometFilter (2) + : : : : : : : +- CometScan parquet spark_catalog.default.web_sales (1) + : : : : : : +- CometProject (6) + : : : : : : +- CometFilter (5) + : : : : : : +- CometScan parquet spark_catalog.default.web_returns (4) + : : : : : +- BroadcastExchange (13) + : : : : : +- * ColumnarToRow (12) + : : : : : +- CometFilter (11) + : : : : : +- CometScan parquet spark_catalog.default.web_page (10) + : : : : +- BroadcastExchange (19) + : : : : +- * ColumnarToRow (18) + : : : : +- CometFilter (17) + : : : : +- CometScan parquet spark_catalog.default.customer_demographics (16) + : : : +- BroadcastExchange (25) + : : : +- * ColumnarToRow (24) + : : : +- CometFilter (23) + : : : +- CometScan parquet spark_catalog.default.customer_demographics (22) + : : +- BroadcastExchange (32) + : : +- * ColumnarToRow (31) + : : +- CometProject (30) + : : +- CometFilter (29) + : : +- CometScan parquet spark_catalog.default.customer_address (28) + : +- ReusedExchange (35) + +- BroadcastExchange (41) + +- * ColumnarToRow (40) + +- CometFilter (39) + +- CometScan parquet spark_catalog.default.reason (38) + + +(1) Scan parquet spark_catalog.default.web_sales +Output [7]: [ws_item_sk#1, ws_web_page_sk#2, ws_order_number#3, ws_quantity#4, ws_sales_price#5, ws_net_profit#6, ws_sold_date_sk#7] +Batched: true +Location: InMemoryFileIndex [] +PartitionFilters: [isnotnull(ws_sold_date_sk#7), dynamicpruningexpression(ws_sold_date_sk#7 IN dynamicpruning#8)] +PushedFilters: [IsNotNull(ws_item_sk), IsNotNull(ws_order_number), IsNotNull(ws_web_page_sk), Or(Or(And(GreaterThanOrEqual(ws_sales_price,100.00),LessThanOrEqual(ws_sales_price,150.00)),And(GreaterThanOrEqual(ws_sales_price,50.00),LessThanOrEqual(ws_sales_price,100.00))),And(GreaterThanOrEqual(ws_sales_price,150.00),LessThanOrEqual(ws_sales_price,200.00))), Or(Or(And(GreaterThanOrEqual(ws_net_profit,100.00),LessThanOrEqual(ws_net_profit,200.00)),And(GreaterThanOrEqual(ws_net_profit,150.00),LessThanOrEqual(ws_net_profit,300.00))),And(GreaterThanOrEqual(ws_net_profit,50.00),LessThanOrEqual(ws_net_profit,250.00)))] +ReadSchema: struct<ws_item_sk:int,ws_web_page_sk:int,ws_order_number:int,ws_quantity:int,ws_sales_price:decimal(7,2),ws_net_profit:decimal(7,2)> + +(2) CometFilter +Input [7]: [ws_item_sk#1, ws_web_page_sk#2, ws_order_number#3, ws_quantity#4, ws_sales_price#5, ws_net_profit#6, ws_sold_date_sk#7] +Condition : ((((isnotnull(ws_item_sk#1) AND isnotnull(ws_order_number#3)) AND isnotnull(ws_web_page_sk#2)) AND ((((ws_sales_price#5 >= 100.00) AND (ws_sales_price#5 <= 150.00)) OR ((ws_sales_price#5 >= 50.00) AND (ws_sales_price#5 <= 100.00))) OR ((ws_sales_price#5 >= 150.00) AND (ws_sales_price#5 <= 200.00)))) AND ((((ws_net_profit#6 >= 100.00) AND (ws_net_profit#6 <= 200.00)) OR ((ws_net_profit#6 >= 150.00) AND (ws_net_profit#6 <= 300.00))) OR ((ws_net_profit#6 >= 50.00) AND (ws_net_profit#6 <= 250.00)))) + +(3) CometBroadcastExchange +Input [7]: [ws_item_sk#1, ws_web_page_sk#2, ws_order_number#3, ws_quantity#4, ws_sales_price#5, ws_net_profit#6, ws_sold_date_sk#7] +Arguments: [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] + +(4) Scan parquet spark_catalog.default.web_returns +Output [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] +Batched: true +Location [not included in comparison]/{warehouse_dir}/web_returns] +PushedFilters: [IsNotNull(wr_item_sk), IsNotNull(wr_order_number), IsNotNull(wr_refunded_cdemo_sk), IsNotNull(wr_returning_cdemo_sk), IsNotNull(wr_refunded_addr_sk), IsNotNull(wr_reason_sk)] +ReadSchema: struct<wr_item_sk:int,wr_refunded_cdemo_sk:int,wr_refunded_addr_sk:int,wr_returning_cdemo_sk:int,wr_reason_sk:int,wr_order_number:int,wr_fee:decimal(7,2),wr_refunded_cash:decimal(7,2)> + +(5) CometFilter +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] +Condition : (((((isnotnull(wr_item_sk#9) AND isnotnull(wr_order_number#14)) AND isnotnull(wr_refunded_cdemo_sk#10)) AND isnotnull(wr_returning_cdemo_sk#12)) AND isnotnull(wr_refunded_addr_sk#11)) AND isnotnull(wr_reason_sk#13)) + +(6) CometProject +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] + +(7) CometBroadcastHashJoin +Left output [7]: [ws_item_sk#1, ws_web_page_sk#2, ws_order_number#3, ws_quantity#4, ws_sales_price#5, ws_net_profit#6, ws_sold_date_sk#7] +Right output [8]: [wr_item_sk#9, wr_refunded_cdemo_sk#10, wr_refunded_addr_sk#11, wr_returning_cdemo_sk#12, wr_reason_sk#13, wr_order_number#14, wr_fee#15, wr_refunded_cash#16] +Arguments: [ws_item_sk#1, ws_order_number#3], [wr_item_sk#9, wr_order_number#14], Inner + +(8) CometProject +Input [15]: [ws_item_sk#1, ws_web_page_sk#2, ws_order_number#3, ws_quantity#4, ws_sales_price#5, ws_net_profit#6, ws_sold_date_sk#7, wr_item_sk#9, wr_refunded_cdemo_sk#10, wr_refunded_addr_sk#11, wr_returning_cdemo_sk#12, wr_reason_sk#13, wr_order_number#14, wr_fee#15, wr_refunded_cash#16] +Arguments: [ws_web_page_sk#2, ws_quantity#4, ws_sales_price#5, ws_net_profit#6, ws_sold_date_sk#7, wr_refunded_cdemo_sk#10, wr_refunded_addr_sk#11, wr_returning_cdemo_sk#12, wr_reason_sk#13, wr_fee#15, wr_refunded_cash#16], [ws_web_page_sk#2, ws_quantity#4, ws_sales_price#5, ws_net_profit#6, ws_sold_date_sk#7, wr_refunded_cdemo_sk#10, wr_refunded_addr_sk#11, wr_returning_cdemo_sk#12, wr_reason_sk#13, wr_fee#15, wr_refunded_cash#16] + +(9) ColumnarToRow [codegen id : 7] +Input [11]: [ws_web_page_sk#2, ws_quantity#4, ws_sales_price#5, ws_net_profit#6, ws_sold_date_sk#7, wr_refunded_cdemo_sk#10, wr_refunded_addr_sk#11, wr_returning_cdemo_sk#12, wr_reason_sk#13, wr_fee#15, wr_refunded_cash#16] + +(10) Scan parquet spark_catalog.default.web_page +Output [1]: [wp_web_page_sk#18] +Batched: true +Location [not included in comparison]/{warehouse_dir}/web_page] +PushedFilters: [IsNotNull(wp_web_page_sk)] +ReadSchema: struct<wp_web_page_sk:int> + +(11) CometFilter +Input [1]: [wp_web_page_sk#18] +Condition : isnotnull(wp_web_page_sk#18) + +(12) ColumnarToRow [codegen id : 1] +Input [1]: [wp_web_page_sk#18] + +(13) BroadcastExchange +Input [1]: [wp_web_page_sk#18] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=1] + +(14) BroadcastHashJoin [codegen id : 7] +Left keys [1]: [ws_web_page_sk#2] +Right keys [1]: [wp_web_page_sk#18] +Join type: Inner +Join condition: None + +(15) Project [codegen id : 7] +Output [10]: [ws_quantity#4, ws_sales_price#5, ws_net_profit#6, ws_sold_date_sk#7, wr_refunded_cdemo_sk#10, wr_refunded_addr_sk#11, wr_returning_cdemo_sk#12, wr_reason_sk#13, wr_fee#15, wr_refunded_cash#16] +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] + +(16) Scan parquet spark_catalog.default.customer_demographics +Output [3]: [cd_demo_sk#19, cd_marital_status#20, cd_education_status#21] +Batched: true +Location [not included in comparison]/{warehouse_dir}/customer_demographics] +PushedFilters: [IsNotNull(cd_demo_sk), IsNotNull(cd_marital_status), IsNotNull(cd_education_status), Or(Or(And(EqualTo(cd_marital_status,M),EqualTo(cd_education_status,Advanced Degree )),And(EqualTo(cd_marital_status,S),EqualTo(cd_education_status,College ))),And(EqualTo(cd_marital_status,W),EqualTo(cd_education_status,2 yr Degree )))] +ReadSchema: struct<cd_demo_sk:int,cd_marital_status:string,cd_education_status:string> + +(17) CometFilter +Input [3]: [cd_demo_sk#19, cd_marital_status#20, cd_education_status#21] +Condition : (((isnotnull(cd_demo_sk#19) AND isnotnull(cd_marital_status#20)) AND isnotnull(cd_education_status#21)) AND ((((cd_marital_status#20 = M) AND (cd_education_status#21 = Advanced Degree )) OR ((cd_marital_status#20 = S) AND (cd_education_status#21 = College ))) OR ((cd_marital_status#20 = W) AND (cd_education_status#21 = 2 yr Degree )))) + +(18) ColumnarToRow [codegen id : 2] +Input [3]: [cd_demo_sk#19, cd_marital_status#20, cd_education_status#21] + +(19) BroadcastExchange +Input [3]: [cd_demo_sk#19, cd_marital_status#20, cd_education_status#21] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=2] + +(20) BroadcastHashJoin [codegen id : 7] +Left keys [1]: [wr_refunded_cdemo_sk#10] +Right keys [1]: [cd_demo_sk#19] +Join type: Inner +Join condition: ((((((cd_marital_status#20 = M) AND (cd_education_status#21 = Advanced Degree )) AND (ws_sales_price#5 >= 100.00)) AND (ws_sales_price#5 <= 150.00)) OR ((((cd_marital_status#20 = S) AND (cd_education_status#21 = College )) AND (ws_sales_price#5 >= 50.00)) AND (ws_sales_price#5 <= 100.00))) OR ((((cd_marital_status#20 = W) AND (cd_education_status#21 = 2 yr Degree )) AND (ws_sales_price#5 >= 150.00)) AND (ws_sales_price#5 <= 200.00))) + +(21) Project [codegen id : 7] +Output [10]: [ws_quantity#4, ws_net_profit#6, ws_sold_date_sk#7, wr_refunded_addr_sk#11, wr_returning_cdemo_sk#12, wr_reason_sk#13, wr_fee#15, wr_refunded_cash#16, cd_marital_status#20, cd_education_status#21] +Input [13]: [ws_quantity#4, ws_sales_price#5, ws_net_profit#6, ws_sold_date_sk#7, wr_refunded_cdemo_sk#10, wr_refunded_addr_sk#11, wr_returning_cdemo_sk#12, wr_reason_sk#13, wr_fee#15, wr_refunded_cash#16, cd_demo_sk#19, cd_marital_status#20, cd_education_status#21] + +(22) Scan parquet spark_catalog.default.customer_demographics +Output [3]: [cd_demo_sk#22, cd_marital_status#23, cd_education_status#24] +Batched: true +Location [not included in comparison]/{warehouse_dir}/customer_demographics] +PushedFilters: [IsNotNull(cd_demo_sk), IsNotNull(cd_marital_status), IsNotNull(cd_education_status)] +ReadSchema: struct<cd_demo_sk:int,cd_marital_status:string,cd_education_status:string> + +(23) CometFilter +Input [3]: [cd_demo_sk#22, cd_marital_status#23, cd_education_status#24] +Condition : ((isnotnull(cd_demo_sk#22) AND isnotnull(cd_marital_status#23)) AND isnotnull(cd_education_status#24)) + +(24) ColumnarToRow [codegen id : 3] +Input [3]: [cd_demo_sk#22, cd_marital_status#23, cd_education_status#24] + +(25) BroadcastExchange +Input [3]: [cd_demo_sk#22, cd_marital_status#23, cd_education_status#24] +Arguments: HashedRelationBroadcastMode(List(input[0, int, false], input[1, string, false], input[2, string, false]),false), [plan_id=3] + +(26) BroadcastHashJoin [codegen id : 7] +Left keys [3]: [wr_returning_cdemo_sk#12, cd_marital_status#20, cd_education_status#21] +Right keys [3]: [cd_demo_sk#22, cd_marital_status#23, cd_education_status#24] +Join type: Inner +Join condition: None + +(27) Project [codegen id : 7] +Output [7]: [ws_quantity#4, ws_net_profit#6, ws_sold_date_sk#7, wr_refunded_addr_sk#11, wr_reason_sk#13, wr_fee#15, wr_refunded_cash#16] +Input [13]: [ws_quantity#4, ws_net_profit#6, ws_sold_date_sk#7, wr_refunded_addr_sk#11, wr_returning_cdemo_sk#12, wr_reason_sk#13, wr_fee#15, wr_refunded_cash#16, cd_marital_status#20, cd_education_status#21, cd_demo_sk#22, cd_marital_status#23, cd_education_status#24] + +(28) Scan parquet spark_catalog.default.customer_address +Output [3]: [ca_address_sk#25, ca_state#26, ca_country#27] +Batched: true +Location [not included in comparison]/{warehouse_dir}/customer_address] +PushedFilters: [IsNotNull(ca_country), EqualTo(ca_country,United States), IsNotNull(ca_address_sk), Or(Or(In(ca_state, [IN,NJ,OH]),In(ca_state, [CT,KY,WI])),In(ca_state, [AR,IA,LA]))] +ReadSchema: struct<ca_address_sk:int,ca_state:string,ca_country:string> + +(29) CometFilter +Input [3]: [ca_address_sk#25, ca_state#26, ca_country#27] +Condition : (((isnotnull(ca_country#27) AND (ca_country#27 = United States)) AND isnotnull(ca_address_sk#25)) AND ((ca_state#26 IN (IN,OH,NJ) OR ca_state#26 IN (WI,CT,KY)) OR ca_state#26 IN (LA,IA,AR))) + +(30) CometProject +Input [3]: [ca_address_sk#25, ca_state#26, ca_country#27] +Arguments: [ca_address_sk#25, ca_state#26], [ca_address_sk#25, ca_state#26] + +(31) ColumnarToRow [codegen id : 4] +Input [2]: [ca_address_sk#25, ca_state#26] + +(32) BroadcastExchange +Input [2]: [ca_address_sk#25, ca_state#26] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] + +(33) BroadcastHashJoin [codegen id : 7] +Left keys [1]: [wr_refunded_addr_sk#11] +Right keys [1]: [ca_address_sk#25] +Join type: Inner +Join condition: ((((ca_state#26 IN (IN,OH,NJ) AND (ws_net_profit#6 >= 100.00)) AND (ws_net_profit#6 <= 200.00)) OR ((ca_state#26 IN (WI,CT,KY) AND (ws_net_profit#6 >= 150.00)) AND (ws_net_profit#6 <= 300.00))) OR ((ca_state#26 IN (LA,IA,AR) AND (ws_net_profit#6 >= 50.00)) AND (ws_net_profit#6 <= 250.00))) + +(34) Project [codegen id : 7] +Output [5]: [ws_quantity#4, ws_sold_date_sk#7, wr_reason_sk#13, wr_fee#15, wr_refunded_cash#16] +Input [9]: [ws_quantity#4, ws_net_profit#6, ws_sold_date_sk#7, wr_refunded_addr_sk#11, wr_reason_sk#13, wr_fee#15, wr_refunded_cash#16, ca_address_sk#25, ca_state#26] + +(35) ReusedExchange [Reuses operator id: 52] +Output [1]: [d_date_sk#28] + +(36) BroadcastHashJoin [codegen id : 7] +Left keys [1]: [ws_sold_date_sk#7] +Right keys [1]: [d_date_sk#28] +Join type: Inner +Join condition: None + +(37) Project [codegen id : 7] +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#28] + +(38) Scan parquet spark_catalog.default.reason +Output [2]: [r_reason_sk#29, r_reason_desc#30] +Batched: true +Location [not included in comparison]/{warehouse_dir}/reason] +PushedFilters: [IsNotNull(r_reason_sk)] +ReadSchema: struct<r_reason_sk:int,r_reason_desc:string> + +(39) CometFilter +Input [2]: [r_reason_sk#29, r_reason_desc#30] +Condition : isnotnull(r_reason_sk#29) + +(40) ColumnarToRow [codegen id : 6] +Input [2]: [r_reason_sk#29, r_reason_desc#30] + +(41) BroadcastExchange +Input [2]: [r_reason_sk#29, r_reason_desc#30] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=5] + +(42) BroadcastHashJoin [codegen id : 7] +Left keys [1]: [wr_reason_sk#13] +Right keys [1]: [r_reason_sk#29] +Join type: Inner +Join condition: None + +(43) Project [codegen id : 7] +Output [4]: [ws_quantity#4, wr_fee#15, wr_refunded_cash#16, r_reason_desc#30] +Input [6]: [ws_quantity#4, wr_reason_sk#13, wr_fee#15, wr_refunded_cash#16, r_reason_sk#29, r_reason_desc#30] + +(44) HashAggregate [codegen id : 7] +Input [4]: [ws_quantity#4, wr_fee#15, wr_refunded_cash#16, r_reason_desc#30] +Keys [1]: [r_reason_desc#30] +Functions [3]: [partial_avg(ws_quantity#4), partial_avg(UnscaledValue(wr_refunded_cash#16)), partial_avg(UnscaledValue(wr_fee#15))] +Aggregate Attributes [6]: [sum#31, count#32, sum#33, count#34, sum#35, count#36] +Results [7]: [r_reason_desc#30, sum#37, count#38, sum#39, count#40, sum#41, count#42] + +(45) Exchange +Input [7]: [r_reason_desc#30, sum#37, count#38, sum#39, count#40, sum#41, count#42] +Arguments: hashpartitioning(r_reason_desc#30, 5), ENSURE_REQUIREMENTS, [plan_id=6] + +(46) HashAggregate [codegen id : 8] +Input [7]: [r_reason_desc#30, sum#37, count#38, sum#39, count#40, sum#41, count#42] +Keys [1]: [r_reason_desc#30] +Functions [3]: [avg(ws_quantity#4), avg(UnscaledValue(wr_refunded_cash#16)), avg(UnscaledValue(wr_fee#15))] +Aggregate Attributes [3]: [avg(ws_quantity#4)#43, avg(UnscaledValue(wr_refunded_cash#16))#44, avg(UnscaledValue(wr_fee#15))#45] +Results [4]: [substr(r_reason_desc#30, 1, 20) AS substr(r_reason_desc, 1, 20)#46, avg(ws_quantity#4)#43 AS avg(ws_quantity)#47, cast((avg(UnscaledValue(wr_refunded_cash#16))#44 / 100.0) as decimal(11,6)) AS avg(wr_refunded_cash)#48, cast((avg(UnscaledValue(wr_fee#15))#45 / 100.0) as decimal(11,6)) AS avg(wr_fee)#49] + +(47) TakeOrderedAndProject +Input [4]: [substr(r_reason_desc, 1, 20)#46, avg(ws_quantity)#47, avg(wr_refunded_cash)#48, avg(wr_fee)#49] +Arguments: 100, [substr(r_reason_desc, 1, 20)#46 ASC NULLS FIRST, avg(ws_quantity)#47 ASC NULLS FIRST, avg(wr_refunded_cash)#48 ASC NULLS FIRST, avg(wr_fee)#49 ASC NULLS FIRST], [substr(r_reason_desc, 1, 20)#46, avg(ws_quantity)#47, avg(wr_refunded_cash)#48, avg(wr_fee)#49] + +===== Subqueries ===== + +Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#7 IN dynamicpruning#8 +BroadcastExchange (52) ++- * ColumnarToRow (51) + +- CometProject (50) + +- CometFilter (49) + +- CometScan parquet spark_catalog.default.date_dim (48) + + +(48) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#28, d_year#50] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2000), IsNotNull(d_date_sk)] +ReadSchema: struct<d_date_sk:int,d_year:int> + +(49) CometFilter +Input [2]: [d_date_sk#28, d_year#50] +Condition : ((isnotnull(d_year#50) AND (d_year#50 = 2000)) AND isnotnull(d_date_sk#28)) + +(50) CometProject +Input [2]: [d_date_sk#28, d_year#50] +Arguments: [d_date_sk#28], [d_date_sk#28] + +(51) ColumnarToRow [codegen id : 1] +Input [1]: [d_date_sk#28] + +(52) BroadcastExchange +Input [1]: [d_date_sk#28] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=7] + + diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q85/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q85/simplified.txt new file mode 100644 index 000000000..7c5ee727b --- /dev/null +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q85/simplified.txt @@ -0,0 +1,75 @@ +TakeOrderedAndProject [substr(r_reason_desc, 1, 20),avg(ws_quantity),avg(wr_refunded_cash),avg(wr_fee)] + WholeStageCodegen (8) + HashAggregate [r_reason_desc,sum,count,sum,count,sum,count] [avg(ws_quantity),avg(UnscaledValue(wr_refunded_cash)),avg(UnscaledValue(wr_fee)),substr(r_reason_desc, 1, 20),avg(ws_quantity),avg(wr_refunded_cash),avg(wr_fee),sum,count,sum,count,sum,count] + InputAdapter + Exchange [r_reason_desc] #1 + WholeStageCodegen (7) + HashAggregate [r_reason_desc,ws_quantity,wr_refunded_cash,wr_fee] [sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] + Project [ws_quantity,wr_fee,wr_refunded_cash,r_reason_desc] + BroadcastHashJoin [wr_reason_sk,r_reason_sk] + Project [ws_quantity,wr_reason_sk,wr_fee,wr_refunded_cash] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + Project [ws_quantity,ws_sold_date_sk,wr_reason_sk,wr_fee,wr_refunded_cash] + BroadcastHashJoin [wr_refunded_addr_sk,ca_address_sk,ca_state,ws_net_profit] + Project [ws_quantity,ws_net_profit,ws_sold_date_sk,wr_refunded_addr_sk,wr_reason_sk,wr_fee,wr_refunded_cash] + BroadcastHashJoin [wr_returning_cdemo_sk,cd_marital_status,cd_education_status,cd_demo_sk,cd_marital_status,cd_education_status] + Project [ws_quantity,ws_net_profit,ws_sold_date_sk,wr_refunded_addr_sk,wr_returning_cdemo_sk,wr_reason_sk,wr_fee,wr_refunded_cash,cd_marital_status,cd_education_status] + BroadcastHashJoin [wr_refunded_cdemo_sk,cd_demo_sk,cd_marital_status,cd_education_status,ws_sales_price] + Project [ws_quantity,ws_sales_price,ws_net_profit,ws_sold_date_sk,wr_refunded_cdemo_sk,wr_refunded_addr_sk,wr_returning_cdemo_sk,wr_reason_sk,wr_fee,wr_refunded_cash] + BroadcastHashJoin [ws_web_page_sk,wp_web_page_sk] + ColumnarToRow + InputAdapter + CometProject [ws_web_page_sk,ws_quantity,ws_sales_price,ws_net_profit,ws_sold_date_sk,wr_refunded_cdemo_sk,wr_refunded_addr_sk,wr_returning_cdemo_sk,wr_reason_sk,wr_fee,wr_refunded_cash] + CometBroadcastHashJoin [ws_item_sk,ws_order_number,wr_item_sk,wr_order_number] + CometBroadcastExchange #2 + CometFilter [ws_item_sk,ws_order_number,ws_web_page_sk,ws_sales_price,ws_net_profit] + CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_web_page_sk,ws_order_number,ws_quantity,ws_sales_price,ws_net_profit,ws_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_year,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometProject [wr_item_sk,wr_refunded_cdemo_sk,wr_refunded_addr_sk,wr_returning_cdemo_sk,wr_reason_sk,wr_order_number,wr_fee,wr_refunded_cash] + CometFilter [wr_item_sk,wr_order_number,wr_refunded_cdemo_sk,wr_returning_cdemo_sk,wr_refunded_addr_sk,wr_reason_sk] + CometScan parquet spark_catalog.default.web_returns [wr_item_sk,wr_refunded_cdemo_sk,wr_refunded_addr_sk,wr_returning_cdemo_sk,wr_reason_sk,wr_order_number,wr_fee,wr_refunded_cash,wr_returned_date_sk] + InputAdapter + BroadcastExchange #4 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometFilter [wp_web_page_sk] + CometScan parquet spark_catalog.default.web_page [wp_web_page_sk] + InputAdapter + BroadcastExchange #5 + WholeStageCodegen (2) + ColumnarToRow + InputAdapter + CometFilter [cd_demo_sk,cd_marital_status,cd_education_status] + CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_marital_status,cd_education_status] + InputAdapter + BroadcastExchange #6 + WholeStageCodegen (3) + ColumnarToRow + InputAdapter + CometFilter [cd_demo_sk,cd_marital_status,cd_education_status] + CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_marital_status,cd_education_status] + InputAdapter + BroadcastExchange #7 + WholeStageCodegen (4) + ColumnarToRow + InputAdapter + CometProject [ca_address_sk,ca_state] + CometFilter [ca_country,ca_address_sk,ca_state] + CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state,ca_country] + InputAdapter + ReusedExchange [d_date_sk] #3 + InputAdapter + BroadcastExchange #8 + WholeStageCodegen (6) + ColumnarToRow + InputAdapter + CometFilter [r_reason_sk] + CometScan parquet spark_catalog.default.reason [r_reason_sk,r_reason_desc] 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 new file mode 100644 index 000000000..5bfd2925d --- /dev/null +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q86/explain.txt @@ -0,0 +1,155 @@ +== Physical Plan == +TakeOrderedAndProject (21) ++- * Project (20) + +- Window (19) + +- * Sort (18) + +- Exchange (17) + +- * HashAggregate (16) + +- Exchange (15) + +- * HashAggregate (14) + +- * Expand (13) + +- * Project (12) + +- * BroadcastHashJoin Inner BuildRight (11) + :- * Project (6) + : +- * BroadcastHashJoin Inner BuildRight (5) + : :- * ColumnarToRow (3) + : : +- CometFilter (2) + : : +- CometScan parquet spark_catalog.default.web_sales (1) + : +- ReusedExchange (4) + +- BroadcastExchange (10) + +- * ColumnarToRow (9) + +- CometFilter (8) + +- CometScan parquet spark_catalog.default.item (7) + + +(1) Scan parquet spark_catalog.default.web_sales +Output [3]: [ws_item_sk#1, ws_net_paid#2, ws_sold_date_sk#3] +Batched: true +Location: InMemoryFileIndex [] +PartitionFilters: [isnotnull(ws_sold_date_sk#3), dynamicpruningexpression(ws_sold_date_sk#3 IN dynamicpruning#4)] +PushedFilters: [IsNotNull(ws_item_sk)] +ReadSchema: struct<ws_item_sk:int,ws_net_paid:decimal(7,2)> + +(2) CometFilter +Input [3]: [ws_item_sk#1, ws_net_paid#2, ws_sold_date_sk#3] +Condition : isnotnull(ws_item_sk#1) + +(3) ColumnarToRow [codegen id : 3] +Input [3]: [ws_item_sk#1, ws_net_paid#2, ws_sold_date_sk#3] + +(4) ReusedExchange [Reuses operator id: 26] +Output [1]: [d_date_sk#5] + +(5) BroadcastHashJoin [codegen id : 3] +Left keys [1]: [ws_sold_date_sk#3] +Right keys [1]: [d_date_sk#5] +Join type: Inner +Join condition: None + +(6) Project [codegen id : 3] +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] + +(7) Scan parquet spark_catalog.default.item +Output [3]: [i_item_sk#6, i_class#7, i_category#8] +Batched: true +Location [not included in comparison]/{warehouse_dir}/item] +PushedFilters: [IsNotNull(i_item_sk)] +ReadSchema: struct<i_item_sk:int,i_class:string,i_category:string> + +(8) CometFilter +Input [3]: [i_item_sk#6, i_class#7, i_category#8] +Condition : isnotnull(i_item_sk#6) + +(9) ColumnarToRow [codegen id : 2] +Input [3]: [i_item_sk#6, i_class#7, i_category#8] + +(10) BroadcastExchange +Input [3]: [i_item_sk#6, i_class#7, i_category#8] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=1] + +(11) BroadcastHashJoin [codegen id : 3] +Left keys [1]: [ws_item_sk#1] +Right keys [1]: [i_item_sk#6] +Join type: Inner +Join condition: None + +(12) Project [codegen id : 3] +Output [3]: [ws_net_paid#2, i_category#8, i_class#7] +Input [5]: [ws_item_sk#1, ws_net_paid#2, i_item_sk#6, i_class#7, i_category#8] + +(13) Expand [codegen id : 3] +Input [3]: [ws_net_paid#2, i_category#8, i_class#7] +Arguments: [[ws_net_paid#2, i_category#8, i_class#7, 0], [ws_net_paid#2, i_category#8, null, 1], [ws_net_paid#2, null, null, 3]], [ws_net_paid#2, i_category#9, i_class#10, spark_grouping_id#11] + +(14) HashAggregate [codegen id : 3] +Input [4]: [ws_net_paid#2, i_category#9, i_class#10, spark_grouping_id#11] +Keys [3]: [i_category#9, i_class#10, spark_grouping_id#11] +Functions [1]: [partial_sum(UnscaledValue(ws_net_paid#2))] +Aggregate Attributes [1]: [sum#12] +Results [4]: [i_category#9, i_class#10, spark_grouping_id#11, sum#13] + +(15) Exchange +Input [4]: [i_category#9, i_class#10, spark_grouping_id#11, sum#13] +Arguments: hashpartitioning(i_category#9, i_class#10, spark_grouping_id#11, 5), ENSURE_REQUIREMENTS, [plan_id=2] + +(16) HashAggregate [codegen id : 4] +Input [4]: [i_category#9, i_class#10, spark_grouping_id#11, sum#13] +Keys [3]: [i_category#9, i_class#10, spark_grouping_id#11] +Functions [1]: [sum(UnscaledValue(ws_net_paid#2))] +Aggregate Attributes [1]: [sum(UnscaledValue(ws_net_paid#2))#14] +Results [7]: [MakeDecimal(sum(UnscaledValue(ws_net_paid#2))#14,17,2) AS total_sum#15, i_category#9, i_class#10, (cast((shiftright(spark_grouping_id#11, 1) & 1) as tinyint) + cast((shiftright(spark_grouping_id#11, 0) & 1) as tinyint)) AS lochierarchy#16, MakeDecimal(sum(UnscaledValue(ws_net_paid#2))#14,17,2) AS _w0#17, (cast((shiftright(spark_grouping_id#11, 1) & 1) as tinyint) + cast((shiftright(spark_grouping_id#11, 0) & 1) as tinyint)) AS _w1#18, CASE WHEN (cast((shiftright(spark_grouping_id#11, 0) & 1) as tinyint) = 0) THEN i_category#9 END AS _w2#19] + +(17) Exchange +Input [7]: [total_sum#15, i_category#9, i_class#10, lochierarchy#16, _w0#17, _w1#18, _w2#19] +Arguments: hashpartitioning(_w1#18, _w2#19, 5), ENSURE_REQUIREMENTS, [plan_id=3] + +(18) Sort [codegen id : 5] +Input [7]: [total_sum#15, i_category#9, i_class#10, lochierarchy#16, _w0#17, _w1#18, _w2#19] +Arguments: [_w1#18 ASC NULLS FIRST, _w2#19 ASC NULLS FIRST, _w0#17 DESC NULLS LAST], false, 0 + +(19) Window +Input [7]: [total_sum#15, i_category#9, i_class#10, lochierarchy#16, _w0#17, _w1#18, _w2#19] +Arguments: [rank(_w0#17) windowspecdefinition(_w1#18, _w2#19, _w0#17 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rank_within_parent#20], [_w1#18, _w2#19], [_w0#17 DESC NULLS LAST] + +(20) Project [codegen id : 6] +Output [5]: [total_sum#15, i_category#9, i_class#10, lochierarchy#16, rank_within_parent#20] +Input [8]: [total_sum#15, i_category#9, i_class#10, lochierarchy#16, _w0#17, _w1#18, _w2#19, rank_within_parent#20] + +(21) TakeOrderedAndProject +Input [5]: [total_sum#15, i_category#9, i_class#10, lochierarchy#16, rank_within_parent#20] +Arguments: 100, [lochierarchy#16 DESC NULLS LAST, CASE WHEN (lochierarchy#16 = 0) THEN i_category#9 END ASC NULLS FIRST, rank_within_parent#20 ASC NULLS FIRST], [total_sum#15, i_category#9, i_class#10, lochierarchy#16, rank_within_parent#20] + +===== Subqueries ===== + +Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#3 IN dynamicpruning#4 +BroadcastExchange (26) ++- * ColumnarToRow (25) + +- CometProject (24) + +- CometFilter (23) + +- CometScan parquet spark_catalog.default.date_dim (22) + + +(22) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#5, d_month_seq#21] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1200), LessThanOrEqual(d_month_seq,1211), IsNotNull(d_date_sk)] +ReadSchema: struct<d_date_sk:int,d_month_seq:int> + +(23) CometFilter +Input [2]: [d_date_sk#5, d_month_seq#21] +Condition : (((isnotnull(d_month_seq#21) AND (d_month_seq#21 >= 1200)) AND (d_month_seq#21 <= 1211)) AND isnotnull(d_date_sk#5)) + +(24) CometProject +Input [2]: [d_date_sk#5, d_month_seq#21] +Arguments: [d_date_sk#5], [d_date_sk#5] + +(25) ColumnarToRow [codegen id : 1] +Input [1]: [d_date_sk#5] + +(26) BroadcastExchange +Input [1]: [d_date_sk#5] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] + + diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q86/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q86/simplified.txt new file mode 100644 index 000000000..f9db2ce7a --- /dev/null +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q86/simplified.txt @@ -0,0 +1,41 @@ +TakeOrderedAndProject [lochierarchy,i_category,rank_within_parent,total_sum,i_class] + WholeStageCodegen (6) + Project [total_sum,i_category,i_class,lochierarchy,rank_within_parent] + InputAdapter + Window [_w0,_w1,_w2] + WholeStageCodegen (5) + Sort [_w1,_w2,_w0] + InputAdapter + Exchange [_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] + InputAdapter + Exchange [i_category,i_class,spark_grouping_id] #2 + WholeStageCodegen (3) + HashAggregate [i_category,i_class,spark_grouping_id,ws_net_paid] [sum,sum] + Expand [ws_net_paid,i_category,i_class] + Project [ws_net_paid,i_category,i_class] + BroadcastHashJoin [ws_item_sk,i_item_sk] + Project [ws_item_sk,ws_net_paid] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + ColumnarToRow + InputAdapter + CometFilter [ws_item_sk] + CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_net_paid,ws_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_month_seq,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] + InputAdapter + ReusedExchange [d_date_sk] #3 + InputAdapter + BroadcastExchange #4 + WholeStageCodegen (2) + ColumnarToRow + InputAdapter + CometFilter [i_item_sk] + CometScan parquet spark_catalog.default.item [i_item_sk,i_class,i_category] 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 new file mode 100644 index 000000000..a4c9f13ce --- /dev/null +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q87/explain.txt @@ -0,0 +1,321 @@ +== Physical Plan == +* HashAggregate (47) ++- Exchange (46) + +- * HashAggregate (45) + +- * Project (44) + +- * BroadcastHashJoin LeftAnti BuildRight (43) + :- * BroadcastHashJoin LeftAnti BuildRight (29) + : :- * HashAggregate (15) + : : +- Exchange (14) + : : +- * HashAggregate (13) + : : +- * Project (12) + : : +- * BroadcastHashJoin Inner BuildRight (11) + : : :- * Project (6) + : : : +- * BroadcastHashJoin Inner BuildRight (5) + : : : :- * ColumnarToRow (3) + : : : : +- CometFilter (2) + : : : : +- CometScan parquet spark_catalog.default.store_sales (1) + : : : +- ReusedExchange (4) + : : +- BroadcastExchange (10) + : : +- * ColumnarToRow (9) + : : +- CometFilter (8) + : : +- CometScan parquet spark_catalog.default.customer (7) + : +- BroadcastExchange (28) + : +- * HashAggregate (27) + : +- Exchange (26) + : +- * HashAggregate (25) + : +- * Project (24) + : +- * BroadcastHashJoin Inner BuildRight (23) + : :- * Project (21) + : : +- * BroadcastHashJoin Inner BuildRight (20) + : : :- * ColumnarToRow (18) + : : : +- CometFilter (17) + : : : +- CometScan parquet spark_catalog.default.catalog_sales (16) + : : +- ReusedExchange (19) + : +- ReusedExchange (22) + +- BroadcastExchange (42) + +- * HashAggregate (41) + +- Exchange (40) + +- * HashAggregate (39) + +- * Project (38) + +- * BroadcastHashJoin Inner BuildRight (37) + :- * Project (35) + : +- * BroadcastHashJoin Inner BuildRight (34) + : :- * ColumnarToRow (32) + : : +- CometFilter (31) + : : +- CometScan parquet spark_catalog.default.web_sales (30) + : +- ReusedExchange (33) + +- ReusedExchange (36) + + +(1) Scan parquet spark_catalog.default.store_sales +Output [2]: [ss_customer_sk#1, ss_sold_date_sk#2] +Batched: true +Location: InMemoryFileIndex [] +PartitionFilters: [isnotnull(ss_sold_date_sk#2), dynamicpruningexpression(ss_sold_date_sk#2 IN dynamicpruning#3)] +PushedFilters: [IsNotNull(ss_customer_sk)] +ReadSchema: struct<ss_customer_sk:int> + +(2) CometFilter +Input [2]: [ss_customer_sk#1, ss_sold_date_sk#2] +Condition : isnotnull(ss_customer_sk#1) + +(3) ColumnarToRow [codegen id : 3] +Input [2]: [ss_customer_sk#1, ss_sold_date_sk#2] + +(4) ReusedExchange [Reuses operator id: 52] +Output [2]: [d_date_sk#4, d_date#5] + +(5) BroadcastHashJoin [codegen id : 3] +Left keys [1]: [ss_sold_date_sk#2] +Right keys [1]: [d_date_sk#4] +Join type: Inner +Join condition: None + +(6) Project [codegen id : 3] +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] + +(7) Scan parquet spark_catalog.default.customer +Output [3]: [c_customer_sk#6, c_first_name#7, c_last_name#8] +Batched: true +Location [not included in comparison]/{warehouse_dir}/customer] +PushedFilters: [IsNotNull(c_customer_sk)] +ReadSchema: struct<c_customer_sk:int,c_first_name:string,c_last_name:string> + +(8) CometFilter +Input [3]: [c_customer_sk#6, c_first_name#7, c_last_name#8] +Condition : isnotnull(c_customer_sk#6) + +(9) ColumnarToRow [codegen id : 2] +Input [3]: [c_customer_sk#6, c_first_name#7, c_last_name#8] + +(10) BroadcastExchange +Input [3]: [c_customer_sk#6, c_first_name#7, c_last_name#8] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=1] + +(11) BroadcastHashJoin [codegen id : 3] +Left keys [1]: [ss_customer_sk#1] +Right keys [1]: [c_customer_sk#6] +Join type: Inner +Join condition: None + +(12) Project [codegen id : 3] +Output [3]: [c_last_name#8, c_first_name#7, d_date#5] +Input [5]: [ss_customer_sk#1, d_date#5, c_customer_sk#6, c_first_name#7, c_last_name#8] + +(13) HashAggregate [codegen id : 3] +Input [3]: [c_last_name#8, c_first_name#7, d_date#5] +Keys [3]: [c_last_name#8, c_first_name#7, d_date#5] +Functions: [] +Aggregate Attributes: [] +Results [3]: [c_last_name#8, c_first_name#7, d_date#5] + +(14) Exchange +Input [3]: [c_last_name#8, c_first_name#7, d_date#5] +Arguments: hashpartitioning(c_last_name#8, c_first_name#7, d_date#5, 5), ENSURE_REQUIREMENTS, [plan_id=2] + +(15) HashAggregate [codegen id : 12] +Input [3]: [c_last_name#8, c_first_name#7, d_date#5] +Keys [3]: [c_last_name#8, c_first_name#7, d_date#5] +Functions: [] +Aggregate Attributes: [] +Results [3]: [c_last_name#8, c_first_name#7, d_date#5] + +(16) Scan parquet spark_catalog.default.catalog_sales +Output [2]: [cs_bill_customer_sk#9, cs_sold_date_sk#10] +Batched: true +Location: InMemoryFileIndex [] +PartitionFilters: [isnotnull(cs_sold_date_sk#10), dynamicpruningexpression(cs_sold_date_sk#10 IN dynamicpruning#11)] +PushedFilters: [IsNotNull(cs_bill_customer_sk)] +ReadSchema: struct<cs_bill_customer_sk:int> + +(17) CometFilter +Input [2]: [cs_bill_customer_sk#9, cs_sold_date_sk#10] +Condition : isnotnull(cs_bill_customer_sk#9) + +(18) ColumnarToRow [codegen id : 6] +Input [2]: [cs_bill_customer_sk#9, cs_sold_date_sk#10] + +(19) ReusedExchange [Reuses operator id: 52] +Output [2]: [d_date_sk#12, d_date#13] + +(20) BroadcastHashJoin [codegen id : 6] +Left keys [1]: [cs_sold_date_sk#10] +Right keys [1]: [d_date_sk#12] +Join type: Inner +Join condition: None + +(21) Project [codegen id : 6] +Output [2]: [cs_bill_customer_sk#9, d_date#13] +Input [4]: [cs_bill_customer_sk#9, cs_sold_date_sk#10, d_date_sk#12, d_date#13] + +(22) ReusedExchange [Reuses operator id: 10] +Output [3]: [c_customer_sk#14, c_first_name#15, c_last_name#16] + +(23) BroadcastHashJoin [codegen id : 6] +Left keys [1]: [cs_bill_customer_sk#9] +Right keys [1]: [c_customer_sk#14] +Join type: Inner +Join condition: None + +(24) Project [codegen id : 6] +Output [3]: [c_last_name#16, c_first_name#15, d_date#13] +Input [5]: [cs_bill_customer_sk#9, d_date#13, c_customer_sk#14, c_first_name#15, c_last_name#16] + +(25) HashAggregate [codegen id : 6] +Input [3]: [c_last_name#16, c_first_name#15, d_date#13] +Keys [3]: [c_last_name#16, c_first_name#15, d_date#13] +Functions: [] +Aggregate Attributes: [] +Results [3]: [c_last_name#16, c_first_name#15, d_date#13] + +(26) Exchange +Input [3]: [c_last_name#16, c_first_name#15, d_date#13] +Arguments: hashpartitioning(c_last_name#16, c_first_name#15, d_date#13, 5), ENSURE_REQUIREMENTS, [plan_id=3] + +(27) HashAggregate [codegen id : 7] +Input [3]: [c_last_name#16, c_first_name#15, d_date#13] +Keys [3]: [c_last_name#16, c_first_name#15, d_date#13] +Functions: [] +Aggregate Attributes: [] +Results [3]: [c_last_name#16, c_first_name#15, d_date#13] + +(28) BroadcastExchange +Input [3]: [c_last_name#16, c_first_name#15, d_date#13] +Arguments: HashedRelationBroadcastMode(List(coalesce(input[0, string, true], ), isnull(input[0, string, true]), coalesce(input[1, string, true], ), isnull(input[1, string, true]), coalesce(input[2, date, true], 1970-01-01), isnull(input[2, date, true])),false), [plan_id=4] + +(29) BroadcastHashJoin [codegen id : 12] +Left keys [6]: [coalesce(c_last_name#8, ), isnull(c_last_name#8), coalesce(c_first_name#7, ), isnull(c_first_name#7), coalesce(d_date#5, 1970-01-01), isnull(d_date#5)] +Right keys [6]: [coalesce(c_last_name#16, ), isnull(c_last_name#16), coalesce(c_first_name#15, ), isnull(c_first_name#15), coalesce(d_date#13, 1970-01-01), isnull(d_date#13)] +Join type: LeftAnti +Join condition: None + +(30) Scan parquet spark_catalog.default.web_sales +Output [2]: [ws_bill_customer_sk#17, ws_sold_date_sk#18] +Batched: true +Location: InMemoryFileIndex [] +PartitionFilters: [isnotnull(ws_sold_date_sk#18), dynamicpruningexpression(ws_sold_date_sk#18 IN dynamicpruning#19)] +PushedFilters: [IsNotNull(ws_bill_customer_sk)] +ReadSchema: struct<ws_bill_customer_sk:int> + +(31) CometFilter +Input [2]: [ws_bill_customer_sk#17, ws_sold_date_sk#18] +Condition : isnotnull(ws_bill_customer_sk#17) + +(32) ColumnarToRow [codegen id : 10] +Input [2]: [ws_bill_customer_sk#17, ws_sold_date_sk#18] + +(33) ReusedExchange [Reuses operator id: 52] +Output [2]: [d_date_sk#20, d_date#21] + +(34) BroadcastHashJoin [codegen id : 10] +Left keys [1]: [ws_sold_date_sk#18] +Right keys [1]: [d_date_sk#20] +Join type: Inner +Join condition: None + +(35) Project [codegen id : 10] +Output [2]: [ws_bill_customer_sk#17, d_date#21] +Input [4]: [ws_bill_customer_sk#17, ws_sold_date_sk#18, d_date_sk#20, d_date#21] + +(36) ReusedExchange [Reuses operator id: 10] +Output [3]: [c_customer_sk#22, c_first_name#23, c_last_name#24] + +(37) BroadcastHashJoin [codegen id : 10] +Left keys [1]: [ws_bill_customer_sk#17] +Right keys [1]: [c_customer_sk#22] +Join type: Inner +Join condition: None + +(38) Project [codegen id : 10] +Output [3]: [c_last_name#24, c_first_name#23, d_date#21] +Input [5]: [ws_bill_customer_sk#17, d_date#21, c_customer_sk#22, c_first_name#23, c_last_name#24] + +(39) HashAggregate [codegen id : 10] +Input [3]: [c_last_name#24, c_first_name#23, d_date#21] +Keys [3]: [c_last_name#24, c_first_name#23, d_date#21] +Functions: [] +Aggregate Attributes: [] +Results [3]: [c_last_name#24, c_first_name#23, d_date#21] + +(40) Exchange +Input [3]: [c_last_name#24, c_first_name#23, d_date#21] +Arguments: hashpartitioning(c_last_name#24, c_first_name#23, d_date#21, 5), ENSURE_REQUIREMENTS, [plan_id=5] + +(41) HashAggregate [codegen id : 11] +Input [3]: [c_last_name#24, c_first_name#23, d_date#21] +Keys [3]: [c_last_name#24, c_first_name#23, d_date#21] +Functions: [] +Aggregate Attributes: [] +Results [3]: [c_last_name#24, c_first_name#23, d_date#21] + +(42) BroadcastExchange +Input [3]: [c_last_name#24, c_first_name#23, d_date#21] +Arguments: HashedRelationBroadcastMode(List(coalesce(input[0, string, true], ), isnull(input[0, string, true]), coalesce(input[1, string, true], ), isnull(input[1, string, true]), coalesce(input[2, date, true], 1970-01-01), isnull(input[2, date, true])),false), [plan_id=6] + +(43) BroadcastHashJoin [codegen id : 12] +Left keys [6]: [coalesce(c_last_name#8, ), isnull(c_last_name#8), coalesce(c_first_name#7, ), isnull(c_first_name#7), coalesce(d_date#5, 1970-01-01), isnull(d_date#5)] +Right keys [6]: [coalesce(c_last_name#24, ), isnull(c_last_name#24), coalesce(c_first_name#23, ), isnull(c_first_name#23), coalesce(d_date#21, 1970-01-01), isnull(d_date#21)] +Join type: LeftAnti +Join condition: None + +(44) Project [codegen id : 12] +Output: [] +Input [3]: [c_last_name#8, c_first_name#7, d_date#5] + +(45) HashAggregate [codegen id : 12] +Input: [] +Keys: [] +Functions [1]: [partial_count(1)] +Aggregate Attributes [1]: [count#25] +Results [1]: [count#26] + +(46) Exchange +Input [1]: [count#26] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=7] + +(47) HashAggregate [codegen id : 13] +Input [1]: [count#26] +Keys: [] +Functions [1]: [count(1)] +Aggregate Attributes [1]: [count(1)#27] +Results [1]: [count(1)#27 AS count(1)#28] + +===== Subqueries ===== + +Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#2 IN dynamicpruning#3 +BroadcastExchange (52) ++- * ColumnarToRow (51) + +- CometProject (50) + +- CometFilter (49) + +- CometScan parquet spark_catalog.default.date_dim (48) + + +(48) Scan parquet spark_catalog.default.date_dim +Output [3]: [d_date_sk#4, d_date#5, d_month_seq#29] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1200), LessThanOrEqual(d_month_seq,1211), IsNotNull(d_date_sk)] +ReadSchema: struct<d_date_sk:int,d_date:date,d_month_seq:int> + +(49) CometFilter +Input [3]: [d_date_sk#4, d_date#5, d_month_seq#29] +Condition : (((isnotnull(d_month_seq#29) AND (d_month_seq#29 >= 1200)) AND (d_month_seq#29 <= 1211)) AND isnotnull(d_date_sk#4)) + +(50) CometProject +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] + +(51) ColumnarToRow [codegen id : 1] +Input [2]: [d_date_sk#4, d_date#5] + +(52) BroadcastExchange +Input [2]: [d_date_sk#4, d_date#5] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=8] + +Subquery:2 Hosting operator id = 16 Hosting Expression = cs_sold_date_sk#10 IN dynamicpruning#3 + +Subquery:3 Hosting operator id = 30 Hosting Expression = ws_sold_date_sk#18 IN dynamicpruning#3 + + diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q87/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q87/simplified.txt new file mode 100644 index 000000000..315afe660 --- /dev/null +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q87/simplified.txt @@ -0,0 +1,81 @@ +WholeStageCodegen (13) + HashAggregate [count] [count(1),count(1),count] + InputAdapter + Exchange #1 + WholeStageCodegen (12) + HashAggregate [count,count] + Project + BroadcastHashJoin [c_last_name,c_first_name,d_date,c_last_name,c_first_name,d_date] + BroadcastHashJoin [c_last_name,c_first_name,d_date,c_last_name,c_first_name,d_date] + HashAggregate [c_last_name,c_first_name,d_date] + InputAdapter + Exchange [c_last_name,c_first_name,d_date] #2 + WholeStageCodegen (3) + HashAggregate [c_last_name,c_first_name,d_date] + Project [c_last_name,c_first_name,d_date] + BroadcastHashJoin [ss_customer_sk,c_customer_sk] + Project [ss_customer_sk,d_date] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + ColumnarToRow + InputAdapter + CometFilter [ss_customer_sk] + CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk,d_date] + CometFilter [d_month_seq,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_month_seq] + InputAdapter + ReusedExchange [d_date_sk,d_date] #3 + InputAdapter + BroadcastExchange #4 + WholeStageCodegen (2) + ColumnarToRow + InputAdapter + CometFilter [c_customer_sk] + CometScan parquet spark_catalog.default.customer [c_customer_sk,c_first_name,c_last_name] + InputAdapter + BroadcastExchange #5 + WholeStageCodegen (7) + HashAggregate [c_last_name,c_first_name,d_date] + InputAdapter + Exchange [c_last_name,c_first_name,d_date] #6 + WholeStageCodegen (6) + HashAggregate [c_last_name,c_first_name,d_date] + Project [c_last_name,c_first_name,d_date] + BroadcastHashJoin [cs_bill_customer_sk,c_customer_sk] + Project [cs_bill_customer_sk,d_date] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + ColumnarToRow + InputAdapter + CometFilter [cs_bill_customer_sk] + CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + InputAdapter + ReusedExchange [d_date_sk,d_date] #3 + InputAdapter + ReusedExchange [c_customer_sk,c_first_name,c_last_name] #4 + InputAdapter + BroadcastExchange #7 + WholeStageCodegen (11) + HashAggregate [c_last_name,c_first_name,d_date] + InputAdapter + Exchange [c_last_name,c_first_name,d_date] #8 + WholeStageCodegen (10) + HashAggregate [c_last_name,c_first_name,d_date] + Project [c_last_name,c_first_name,d_date] + BroadcastHashJoin [ws_bill_customer_sk,c_customer_sk] + Project [ws_bill_customer_sk,d_date] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + ColumnarToRow + InputAdapter + CometFilter [ws_bill_customer_sk] + CometScan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + InputAdapter + ReusedExchange [d_date_sk,d_date] #3 + InputAdapter + ReusedExchange [c_customer_sk,c_first_name,c_last_name] #4 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q88/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q88/explain.txt new file mode 100644 index 000000000..a5f68e564 --- /dev/null +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q88/explain.txt @@ -0,0 +1,1031 @@ +== Physical Plan == +* BroadcastNestedLoopJoin Inner BuildRight (182) +:- * BroadcastNestedLoopJoin Inner BuildRight (160) +: :- * BroadcastNestedLoopJoin Inner BuildRight (138) +: : :- * BroadcastNestedLoopJoin Inner BuildRight (116) +: : : :- * BroadcastNestedLoopJoin Inner BuildRight (94) +: : : : :- * BroadcastNestedLoopJoin Inner BuildRight (72) +: : : : : :- * BroadcastNestedLoopJoin Inner BuildRight (50) +: : : : : : :- * HashAggregate (28) +: : : : : : : +- Exchange (27) +: : : : : : : +- * HashAggregate (26) +: : : : : : : +- * Project (25) +: : : : : : : +- * BroadcastHashJoin Inner BuildRight (24) +: : : : : : : :- * Project (18) +: : : : : : : : +- * BroadcastHashJoin Inner BuildRight (17) +: : : : : : : : :- * Project (11) +: : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (10) +: : : : : : : : : :- * ColumnarToRow (4) +: : : : : : : : : : +- CometProject (3) +: : : : : : : : : : +- CometFilter (2) +: : : : : : : : : : +- CometScan parquet spark_catalog.default.store_sales (1) +: : : : : : : : : +- BroadcastExchange (9) +: : : : : : : : : +- * ColumnarToRow (8) +: : : : : : : : : +- CometProject (7) +: : : : : : : : : +- CometFilter (6) +: : : : : : : : : +- CometScan parquet spark_catalog.default.household_demographics (5) +: : : : : : : : +- BroadcastExchange (16) +: : : : : : : : +- * ColumnarToRow (15) +: : : : : : : : +- CometProject (14) +: : : : : : : : +- CometFilter (13) +: : : : : : : : +- CometScan parquet spark_catalog.default.time_dim (12) +: : : : : : : +- BroadcastExchange (23) +: : : : : : : +- * ColumnarToRow (22) +: : : : : : : +- CometProject (21) +: : : : : : : +- CometFilter (20) +: : : : : : : +- CometScan parquet spark_catalog.default.store (19) +: : : : : : +- BroadcastExchange (49) +: : : : : : +- * HashAggregate (48) +: : : : : : +- Exchange (47) +: : : : : : +- * HashAggregate (46) +: : : : : : +- * Project (45) +: : : : : : +- * BroadcastHashJoin Inner BuildRight (44) +: : : : : : :- * Project (42) +: : : : : : : +- * BroadcastHashJoin Inner BuildRight (41) +: : : : : : : :- * Project (35) +: : : : : : : : +- * BroadcastHashJoin Inner BuildRight (34) +: : : : : : : : :- * ColumnarToRow (32) +: : : : : : : : : +- CometProject (31) +: : : : : : : : : +- CometFilter (30) +: : : : : : : : : +- CometScan parquet spark_catalog.default.store_sales (29) +: : : : : : : : +- ReusedExchange (33) +: : : : : : : +- BroadcastExchange (40) +: : : : : : : +- * ColumnarToRow (39) +: : : : : : : +- CometProject (38) +: : : : : : : +- CometFilter (37) +: : : : : : : +- CometScan parquet spark_catalog.default.time_dim (36) +: : : : : : +- ReusedExchange (43) +: : : : : +- BroadcastExchange (71) +: : : : : +- * HashAggregate (70) +: : : : : +- Exchange (69) +: : : : : +- * HashAggregate (68) +: : : : : +- * Project (67) +: : : : : +- * BroadcastHashJoin Inner BuildRight (66) +: : : : : :- * Project (64) +: : : : : : +- * BroadcastHashJoin Inner BuildRight (63) +: : : : : : :- * Project (57) +: : : : : : : +- * BroadcastHashJoin Inner BuildRight (56) +: : : : : : : :- * ColumnarToRow (54) +: : : : : : : : +- CometProject (53) +: : : : : : : : +- CometFilter (52) +: : : : : : : : +- CometScan parquet spark_catalog.default.store_sales (51) +: : : : : : : +- ReusedExchange (55) +: : : : : : +- BroadcastExchange (62) +: : : : : : +- * ColumnarToRow (61) +: : : : : : +- CometProject (60) +: : : : : : +- CometFilter (59) +: : : : : : +- CometScan parquet spark_catalog.default.time_dim (58) +: : : : : +- ReusedExchange (65) +: : : : +- BroadcastExchange (93) +: : : : +- * HashAggregate (92) +: : : : +- Exchange (91) +: : : : +- * HashAggregate (90) +: : : : +- * Project (89) +: : : : +- * BroadcastHashJoin Inner BuildRight (88) +: : : : :- * Project (86) +: : : : : +- * BroadcastHashJoin Inner BuildRight (85) +: : : : : :- * Project (79) +: : : : : : +- * BroadcastHashJoin Inner BuildRight (78) +: : : : : : :- * ColumnarToRow (76) +: : : : : : : +- CometProject (75) +: : : : : : : +- CometFilter (74) +: : : : : : : +- CometScan parquet spark_catalog.default.store_sales (73) +: : : : : : +- ReusedExchange (77) +: : : : : +- BroadcastExchange (84) +: : : : : +- * ColumnarToRow (83) +: : : : : +- CometProject (82) +: : : : : +- CometFilter (81) +: : : : : +- CometScan parquet spark_catalog.default.time_dim (80) +: : : : +- ReusedExchange (87) +: : : +- BroadcastExchange (115) +: : : +- * HashAggregate (114) +: : : +- Exchange (113) +: : : +- * HashAggregate (112) +: : : +- * Project (111) +: : : +- * BroadcastHashJoin Inner BuildRight (110) +: : : :- * Project (108) +: : : : +- * BroadcastHashJoin Inner BuildRight (107) +: : : : :- * Project (101) +: : : : : +- * BroadcastHashJoin Inner BuildRight (100) +: : : : : :- * ColumnarToRow (98) +: : : : : : +- CometProject (97) +: : : : : : +- CometFilter (96) +: : : : : : +- CometScan parquet spark_catalog.default.store_sales (95) +: : : : : +- ReusedExchange (99) +: : : : +- BroadcastExchange (106) +: : : : +- * ColumnarToRow (105) +: : : : +- CometProject (104) +: : : : +- CometFilter (103) +: : : : +- CometScan parquet spark_catalog.default.time_dim (102) +: : : +- ReusedExchange (109) +: : +- BroadcastExchange (137) +: : +- * HashAggregate (136) +: : +- Exchange (135) +: : +- * HashAggregate (134) +: : +- * Project (133) +: : +- * BroadcastHashJoin Inner BuildRight (132) +: : :- * Project (130) +: : : +- * BroadcastHashJoin Inner BuildRight (129) +: : : :- * Project (123) +: : : : +- * BroadcastHashJoin Inner BuildRight (122) +: : : : :- * ColumnarToRow (120) +: : : : : +- CometProject (119) +: : : : : +- CometFilter (118) +: : : : : +- CometScan parquet spark_catalog.default.store_sales (117) +: : : : +- ReusedExchange (121) +: : : +- BroadcastExchange (128) +: : : +- * ColumnarToRow (127) +: : : +- CometProject (126) +: : : +- CometFilter (125) +: : : +- CometScan parquet spark_catalog.default.time_dim (124) +: : +- ReusedExchange (131) +: +- BroadcastExchange (159) +: +- * HashAggregate (158) +: +- Exchange (157) +: +- * HashAggregate (156) +: +- * Project (155) +: +- * BroadcastHashJoin Inner BuildRight (154) +: :- * Project (152) +: : +- * BroadcastHashJoin Inner BuildRight (151) +: : :- * Project (145) +: : : +- * BroadcastHashJoin Inner BuildRight (144) +: : : :- * ColumnarToRow (142) +: : : : +- CometProject (141) +: : : : +- CometFilter (140) +: : : : +- CometScan parquet spark_catalog.default.store_sales (139) +: : : +- ReusedExchange (143) +: : +- BroadcastExchange (150) +: : +- * ColumnarToRow (149) +: : +- CometProject (148) +: : +- CometFilter (147) +: : +- CometScan parquet spark_catalog.default.time_dim (146) +: +- ReusedExchange (153) ++- BroadcastExchange (181) + +- * HashAggregate (180) + +- Exchange (179) + +- * HashAggregate (178) + +- * Project (177) + +- * BroadcastHashJoin Inner BuildRight (176) + :- * Project (174) + : +- * BroadcastHashJoin Inner BuildRight (173) + : :- * Project (167) + : : +- * BroadcastHashJoin Inner BuildRight (166) + : : :- * ColumnarToRow (164) + : : : +- CometProject (163) + : : : +- CometFilter (162) + : : : +- CometScan parquet spark_catalog.default.store_sales (161) + : : +- ReusedExchange (165) + : +- BroadcastExchange (172) + : +- * ColumnarToRow (171) + : +- CometProject (170) + : +- CometFilter (169) + : +- CometScan parquet spark_catalog.default.time_dim (168) + +- ReusedExchange (175) + + +(1) Scan parquet spark_catalog.default.store_sales +Output [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_sold_date_sk#4] +Batched: true +Location [not included in comparison]/{warehouse_dir}/store_sales] +PushedFilters: [IsNotNull(ss_hdemo_sk), IsNotNull(ss_sold_time_sk), IsNotNull(ss_store_sk)] +ReadSchema: struct<ss_sold_time_sk:int,ss_hdemo_sk:int,ss_store_sk:int> + +(2) CometFilter +Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_sold_date_sk#4] +Condition : ((isnotnull(ss_hdemo_sk#2) AND isnotnull(ss_sold_time_sk#1)) AND isnotnull(ss_store_sk#3)) + +(3) CometProject +Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_sold_date_sk#4] +Arguments: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3], [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3] + +(4) ColumnarToRow [codegen id : 4] +Input [3]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3] + +(5) Scan parquet spark_catalog.default.household_demographics +Output [3]: [hd_demo_sk#5, hd_dep_count#6, hd_vehicle_count#7] +Batched: true +Location [not included in comparison]/{warehouse_dir}/household_demographics] +PushedFilters: [Or(Or(And(EqualTo(hd_dep_count,4),LessThanOrEqual(hd_vehicle_count,6)),And(EqualTo(hd_dep_count,2),LessThanOrEqual(hd_vehicle_count,4))),And(EqualTo(hd_dep_count,0),LessThanOrEqual(hd_vehicle_count,2))), IsNotNull(hd_demo_sk)] +ReadSchema: struct<hd_demo_sk:int,hd_dep_count:int,hd_vehicle_count:int> + +(6) CometFilter +Input [3]: [hd_demo_sk#5, hd_dep_count#6, hd_vehicle_count#7] +Condition : (((((hd_dep_count#6 = 4) AND (hd_vehicle_count#7 <= 6)) OR ((hd_dep_count#6 = 2) AND (hd_vehicle_count#7 <= 4))) OR ((hd_dep_count#6 = 0) AND (hd_vehicle_count#7 <= 2))) AND isnotnull(hd_demo_sk#5)) + +(7) CometProject +Input [3]: [hd_demo_sk#5, hd_dep_count#6, hd_vehicle_count#7] +Arguments: [hd_demo_sk#5], [hd_demo_sk#5] + +(8) ColumnarToRow [codegen id : 1] +Input [1]: [hd_demo_sk#5] + +(9) BroadcastExchange +Input [1]: [hd_demo_sk#5] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] + +(10) BroadcastHashJoin [codegen id : 4] +Left keys [1]: [ss_hdemo_sk#2] +Right keys [1]: [hd_demo_sk#5] +Join type: Inner +Join condition: None + +(11) Project [codegen id : 4] +Output [2]: [ss_sold_time_sk#1, ss_store_sk#3] +Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, hd_demo_sk#5] + +(12) Scan parquet spark_catalog.default.time_dim +Output [3]: [t_time_sk#8, t_hour#9, t_minute#10] +Batched: true +Location [not included in comparison]/{warehouse_dir}/time_dim] +PushedFilters: [IsNotNull(t_hour), IsNotNull(t_minute), EqualTo(t_hour,8), GreaterThanOrEqual(t_minute,30), IsNotNull(t_time_sk)] +ReadSchema: struct<t_time_sk:int,t_hour:int,t_minute:int> + +(13) CometFilter +Input [3]: [t_time_sk#8, t_hour#9, t_minute#10] +Condition : ((((isnotnull(t_hour#9) AND isnotnull(t_minute#10)) AND (t_hour#9 = 8)) AND (t_minute#10 >= 30)) AND isnotnull(t_time_sk#8)) + +(14) CometProject +Input [3]: [t_time_sk#8, t_hour#9, t_minute#10] +Arguments: [t_time_sk#8], [t_time_sk#8] + +(15) ColumnarToRow [codegen id : 2] +Input [1]: [t_time_sk#8] + +(16) BroadcastExchange +Input [1]: [t_time_sk#8] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] + +(17) BroadcastHashJoin [codegen id : 4] +Left keys [1]: [ss_sold_time_sk#1] +Right keys [1]: [t_time_sk#8] +Join type: Inner +Join condition: None + +(18) Project [codegen id : 4] +Output [1]: [ss_store_sk#3] +Input [3]: [ss_sold_time_sk#1, ss_store_sk#3, t_time_sk#8] + +(19) Scan parquet spark_catalog.default.store +Output [2]: [s_store_sk#11, s_store_name#12] +Batched: true +Location [not included in comparison]/{warehouse_dir}/store] +PushedFilters: [IsNotNull(s_store_name), EqualTo(s_store_name,ese), IsNotNull(s_store_sk)] +ReadSchema: struct<s_store_sk:int,s_store_name:string> + +(20) CometFilter +Input [2]: [s_store_sk#11, s_store_name#12] +Condition : ((isnotnull(s_store_name#12) AND (s_store_name#12 = ese)) AND isnotnull(s_store_sk#11)) + +(21) CometProject +Input [2]: [s_store_sk#11, s_store_name#12] +Arguments: [s_store_sk#11], [s_store_sk#11] + +(22) ColumnarToRow [codegen id : 3] +Input [1]: [s_store_sk#11] + +(23) BroadcastExchange +Input [1]: [s_store_sk#11] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] + +(24) BroadcastHashJoin [codegen id : 4] +Left keys [1]: [ss_store_sk#3] +Right keys [1]: [s_store_sk#11] +Join type: Inner +Join condition: None + +(25) Project [codegen id : 4] +Output: [] +Input [2]: [ss_store_sk#3, s_store_sk#11] + +(26) HashAggregate [codegen id : 4] +Input: [] +Keys: [] +Functions [1]: [partial_count(1)] +Aggregate Attributes [1]: [count#13] +Results [1]: [count#14] + +(27) Exchange +Input [1]: [count#14] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=4] + +(28) HashAggregate [codegen id : 40] +Input [1]: [count#14] +Keys: [] +Functions [1]: [count(1)] +Aggregate Attributes [1]: [count(1)#15] +Results [1]: [count(1)#15 AS h8_30_to_9#16] + +(29) Scan parquet spark_catalog.default.store_sales +Output [4]: [ss_sold_time_sk#17, ss_hdemo_sk#18, ss_store_sk#19, ss_sold_date_sk#20] +Batched: true +Location [not included in comparison]/{warehouse_dir}/store_sales] +PushedFilters: [IsNotNull(ss_hdemo_sk), IsNotNull(ss_sold_time_sk), IsNotNull(ss_store_sk)] +ReadSchema: struct<ss_sold_time_sk:int,ss_hdemo_sk:int,ss_store_sk:int> + +(30) CometFilter +Input [4]: [ss_sold_time_sk#17, ss_hdemo_sk#18, ss_store_sk#19, ss_sold_date_sk#20] +Condition : ((isnotnull(ss_hdemo_sk#18) AND isnotnull(ss_sold_time_sk#17)) AND isnotnull(ss_store_sk#19)) + +(31) CometProject +Input [4]: [ss_sold_time_sk#17, ss_hdemo_sk#18, ss_store_sk#19, ss_sold_date_sk#20] +Arguments: [ss_sold_time_sk#17, ss_hdemo_sk#18, ss_store_sk#19], [ss_sold_time_sk#17, ss_hdemo_sk#18, ss_store_sk#19] + +(32) ColumnarToRow [codegen id : 8] +Input [3]: [ss_sold_time_sk#17, ss_hdemo_sk#18, ss_store_sk#19] + +(33) ReusedExchange [Reuses operator id: 9] +Output [1]: [hd_demo_sk#21] + +(34) BroadcastHashJoin [codegen id : 8] +Left keys [1]: [ss_hdemo_sk#18] +Right keys [1]: [hd_demo_sk#21] +Join type: Inner +Join condition: None + +(35) Project [codegen id : 8] +Output [2]: [ss_sold_time_sk#17, ss_store_sk#19] +Input [4]: [ss_sold_time_sk#17, ss_hdemo_sk#18, ss_store_sk#19, hd_demo_sk#21] + +(36) Scan parquet spark_catalog.default.time_dim +Output [3]: [t_time_sk#22, t_hour#23, t_minute#24] +Batched: true +Location [not included in comparison]/{warehouse_dir}/time_dim] +PushedFilters: [IsNotNull(t_hour), IsNotNull(t_minute), EqualTo(t_hour,9), LessThan(t_minute,30), IsNotNull(t_time_sk)] +ReadSchema: struct<t_time_sk:int,t_hour:int,t_minute:int> + +(37) CometFilter +Input [3]: [t_time_sk#22, t_hour#23, t_minute#24] +Condition : ((((isnotnull(t_hour#23) AND isnotnull(t_minute#24)) AND (t_hour#23 = 9)) AND (t_minute#24 < 30)) AND isnotnull(t_time_sk#22)) + +(38) CometProject +Input [3]: [t_time_sk#22, t_hour#23, t_minute#24] +Arguments: [t_time_sk#22], [t_time_sk#22] + +(39) ColumnarToRow [codegen id : 6] +Input [1]: [t_time_sk#22] + +(40) BroadcastExchange +Input [1]: [t_time_sk#22] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] + +(41) BroadcastHashJoin [codegen id : 8] +Left keys [1]: [ss_sold_time_sk#17] +Right keys [1]: [t_time_sk#22] +Join type: Inner +Join condition: None + +(42) Project [codegen id : 8] +Output [1]: [ss_store_sk#19] +Input [3]: [ss_sold_time_sk#17, ss_store_sk#19, t_time_sk#22] + +(43) ReusedExchange [Reuses operator id: 23] +Output [1]: [s_store_sk#25] + +(44) BroadcastHashJoin [codegen id : 8] +Left keys [1]: [ss_store_sk#19] +Right keys [1]: [s_store_sk#25] +Join type: Inner +Join condition: None + +(45) Project [codegen id : 8] +Output: [] +Input [2]: [ss_store_sk#19, s_store_sk#25] + +(46) HashAggregate [codegen id : 8] +Input: [] +Keys: [] +Functions [1]: [partial_count(1)] +Aggregate Attributes [1]: [count#26] +Results [1]: [count#27] + +(47) Exchange +Input [1]: [count#27] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=6] + +(48) HashAggregate [codegen id : 9] +Input [1]: [count#27] +Keys: [] +Functions [1]: [count(1)] +Aggregate Attributes [1]: [count(1)#28] +Results [1]: [count(1)#28 AS h9_to_9_30#29] + +(49) BroadcastExchange +Input [1]: [h9_to_9_30#29] +Arguments: IdentityBroadcastMode, [plan_id=7] + +(50) BroadcastNestedLoopJoin [codegen id : 40] +Join type: Inner +Join condition: None + +(51) Scan parquet spark_catalog.default.store_sales +Output [4]: [ss_sold_time_sk#30, ss_hdemo_sk#31, ss_store_sk#32, ss_sold_date_sk#33] +Batched: true +Location [not included in comparison]/{warehouse_dir}/store_sales] +PushedFilters: [IsNotNull(ss_hdemo_sk), IsNotNull(ss_sold_time_sk), IsNotNull(ss_store_sk)] +ReadSchema: struct<ss_sold_time_sk:int,ss_hdemo_sk:int,ss_store_sk:int> + +(52) CometFilter +Input [4]: [ss_sold_time_sk#30, ss_hdemo_sk#31, ss_store_sk#32, ss_sold_date_sk#33] +Condition : ((isnotnull(ss_hdemo_sk#31) AND isnotnull(ss_sold_time_sk#30)) AND isnotnull(ss_store_sk#32)) + +(53) CometProject +Input [4]: [ss_sold_time_sk#30, ss_hdemo_sk#31, ss_store_sk#32, ss_sold_date_sk#33] +Arguments: [ss_sold_time_sk#30, ss_hdemo_sk#31, ss_store_sk#32], [ss_sold_time_sk#30, ss_hdemo_sk#31, ss_store_sk#32] + +(54) ColumnarToRow [codegen id : 13] +Input [3]: [ss_sold_time_sk#30, ss_hdemo_sk#31, ss_store_sk#32] + +(55) ReusedExchange [Reuses operator id: 9] +Output [1]: [hd_demo_sk#34] + +(56) BroadcastHashJoin [codegen id : 13] +Left keys [1]: [ss_hdemo_sk#31] +Right keys [1]: [hd_demo_sk#34] +Join type: Inner +Join condition: None + +(57) Project [codegen id : 13] +Output [2]: [ss_sold_time_sk#30, ss_store_sk#32] +Input [4]: [ss_sold_time_sk#30, ss_hdemo_sk#31, ss_store_sk#32, hd_demo_sk#34] + +(58) Scan parquet spark_catalog.default.time_dim +Output [3]: [t_time_sk#35, t_hour#36, t_minute#37] +Batched: true +Location [not included in comparison]/{warehouse_dir}/time_dim] +PushedFilters: [IsNotNull(t_hour), IsNotNull(t_minute), EqualTo(t_hour,9), GreaterThanOrEqual(t_minute,30), IsNotNull(t_time_sk)] +ReadSchema: struct<t_time_sk:int,t_hour:int,t_minute:int> + +(59) CometFilter +Input [3]: [t_time_sk#35, t_hour#36, t_minute#37] +Condition : ((((isnotnull(t_hour#36) AND isnotnull(t_minute#37)) AND (t_hour#36 = 9)) AND (t_minute#37 >= 30)) AND isnotnull(t_time_sk#35)) + +(60) CometProject +Input [3]: [t_time_sk#35, t_hour#36, t_minute#37] +Arguments: [t_time_sk#35], [t_time_sk#35] + +(61) ColumnarToRow [codegen id : 11] +Input [1]: [t_time_sk#35] + +(62) BroadcastExchange +Input [1]: [t_time_sk#35] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=8] + +(63) BroadcastHashJoin [codegen id : 13] +Left keys [1]: [ss_sold_time_sk#30] +Right keys [1]: [t_time_sk#35] +Join type: Inner +Join condition: None + +(64) Project [codegen id : 13] +Output [1]: [ss_store_sk#32] +Input [3]: [ss_sold_time_sk#30, ss_store_sk#32, t_time_sk#35] + +(65) ReusedExchange [Reuses operator id: 23] +Output [1]: [s_store_sk#38] + +(66) BroadcastHashJoin [codegen id : 13] +Left keys [1]: [ss_store_sk#32] +Right keys [1]: [s_store_sk#38] +Join type: Inner +Join condition: None + +(67) Project [codegen id : 13] +Output: [] +Input [2]: [ss_store_sk#32, s_store_sk#38] + +(68) HashAggregate [codegen id : 13] +Input: [] +Keys: [] +Functions [1]: [partial_count(1)] +Aggregate Attributes [1]: [count#39] +Results [1]: [count#40] + +(69) Exchange +Input [1]: [count#40] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=9] + +(70) HashAggregate [codegen id : 14] +Input [1]: [count#40] +Keys: [] +Functions [1]: [count(1)] +Aggregate Attributes [1]: [count(1)#41] +Results [1]: [count(1)#41 AS h9_30_to_10#42] + +(71) BroadcastExchange +Input [1]: [h9_30_to_10#42] +Arguments: IdentityBroadcastMode, [plan_id=10] + +(72) BroadcastNestedLoopJoin [codegen id : 40] +Join type: Inner +Join condition: None + +(73) Scan parquet spark_catalog.default.store_sales +Output [4]: [ss_sold_time_sk#43, ss_hdemo_sk#44, ss_store_sk#45, ss_sold_date_sk#46] +Batched: true +Location [not included in comparison]/{warehouse_dir}/store_sales] +PushedFilters: [IsNotNull(ss_hdemo_sk), IsNotNull(ss_sold_time_sk), IsNotNull(ss_store_sk)] +ReadSchema: struct<ss_sold_time_sk:int,ss_hdemo_sk:int,ss_store_sk:int> + +(74) CometFilter +Input [4]: [ss_sold_time_sk#43, ss_hdemo_sk#44, ss_store_sk#45, ss_sold_date_sk#46] +Condition : ((isnotnull(ss_hdemo_sk#44) AND isnotnull(ss_sold_time_sk#43)) AND isnotnull(ss_store_sk#45)) + +(75) CometProject +Input [4]: [ss_sold_time_sk#43, ss_hdemo_sk#44, ss_store_sk#45, ss_sold_date_sk#46] +Arguments: [ss_sold_time_sk#43, ss_hdemo_sk#44, ss_store_sk#45], [ss_sold_time_sk#43, ss_hdemo_sk#44, ss_store_sk#45] + +(76) ColumnarToRow [codegen id : 18] +Input [3]: [ss_sold_time_sk#43, ss_hdemo_sk#44, ss_store_sk#45] + +(77) ReusedExchange [Reuses operator id: 9] +Output [1]: [hd_demo_sk#47] + +(78) BroadcastHashJoin [codegen id : 18] +Left keys [1]: [ss_hdemo_sk#44] +Right keys [1]: [hd_demo_sk#47] +Join type: Inner +Join condition: None + +(79) Project [codegen id : 18] +Output [2]: [ss_sold_time_sk#43, ss_store_sk#45] +Input [4]: [ss_sold_time_sk#43, ss_hdemo_sk#44, ss_store_sk#45, hd_demo_sk#47] + +(80) Scan parquet spark_catalog.default.time_dim +Output [3]: [t_time_sk#48, t_hour#49, t_minute#50] +Batched: true +Location [not included in comparison]/{warehouse_dir}/time_dim] +PushedFilters: [IsNotNull(t_hour), IsNotNull(t_minute), EqualTo(t_hour,10), LessThan(t_minute,30), IsNotNull(t_time_sk)] +ReadSchema: struct<t_time_sk:int,t_hour:int,t_minute:int> + +(81) CometFilter +Input [3]: [t_time_sk#48, t_hour#49, t_minute#50] +Condition : ((((isnotnull(t_hour#49) AND isnotnull(t_minute#50)) AND (t_hour#49 = 10)) AND (t_minute#50 < 30)) AND isnotnull(t_time_sk#48)) + +(82) CometProject +Input [3]: [t_time_sk#48, t_hour#49, t_minute#50] +Arguments: [t_time_sk#48], [t_time_sk#48] + +(83) ColumnarToRow [codegen id : 16] +Input [1]: [t_time_sk#48] + +(84) BroadcastExchange +Input [1]: [t_time_sk#48] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=11] + +(85) BroadcastHashJoin [codegen id : 18] +Left keys [1]: [ss_sold_time_sk#43] +Right keys [1]: [t_time_sk#48] +Join type: Inner +Join condition: None + +(86) Project [codegen id : 18] +Output [1]: [ss_store_sk#45] +Input [3]: [ss_sold_time_sk#43, ss_store_sk#45, t_time_sk#48] + +(87) ReusedExchange [Reuses operator id: 23] +Output [1]: [s_store_sk#51] + +(88) BroadcastHashJoin [codegen id : 18] +Left keys [1]: [ss_store_sk#45] +Right keys [1]: [s_store_sk#51] +Join type: Inner +Join condition: None + +(89) Project [codegen id : 18] +Output: [] +Input [2]: [ss_store_sk#45, s_store_sk#51] + +(90) HashAggregate [codegen id : 18] +Input: [] +Keys: [] +Functions [1]: [partial_count(1)] +Aggregate Attributes [1]: [count#52] +Results [1]: [count#53] + +(91) Exchange +Input [1]: [count#53] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=12] + +(92) HashAggregate [codegen id : 19] +Input [1]: [count#53] +Keys: [] +Functions [1]: [count(1)] +Aggregate Attributes [1]: [count(1)#54] +Results [1]: [count(1)#54 AS h10_to_10_30#55] + +(93) BroadcastExchange +Input [1]: [h10_to_10_30#55] +Arguments: IdentityBroadcastMode, [plan_id=13] + +(94) BroadcastNestedLoopJoin [codegen id : 40] +Join type: Inner +Join condition: None + +(95) Scan parquet spark_catalog.default.store_sales +Output [4]: [ss_sold_time_sk#56, ss_hdemo_sk#57, ss_store_sk#58, ss_sold_date_sk#59] +Batched: true +Location [not included in comparison]/{warehouse_dir}/store_sales] +PushedFilters: [IsNotNull(ss_hdemo_sk), IsNotNull(ss_sold_time_sk), IsNotNull(ss_store_sk)] +ReadSchema: struct<ss_sold_time_sk:int,ss_hdemo_sk:int,ss_store_sk:int> + +(96) CometFilter +Input [4]: [ss_sold_time_sk#56, ss_hdemo_sk#57, ss_store_sk#58, ss_sold_date_sk#59] +Condition : ((isnotnull(ss_hdemo_sk#57) AND isnotnull(ss_sold_time_sk#56)) AND isnotnull(ss_store_sk#58)) + +(97) CometProject +Input [4]: [ss_sold_time_sk#56, ss_hdemo_sk#57, ss_store_sk#58, ss_sold_date_sk#59] +Arguments: [ss_sold_time_sk#56, ss_hdemo_sk#57, ss_store_sk#58], [ss_sold_time_sk#56, ss_hdemo_sk#57, ss_store_sk#58] + +(98) ColumnarToRow [codegen id : 23] +Input [3]: [ss_sold_time_sk#56, ss_hdemo_sk#57, ss_store_sk#58] + +(99) ReusedExchange [Reuses operator id: 9] +Output [1]: [hd_demo_sk#60] + +(100) BroadcastHashJoin [codegen id : 23] +Left keys [1]: [ss_hdemo_sk#57] +Right keys [1]: [hd_demo_sk#60] +Join type: Inner +Join condition: None + +(101) Project [codegen id : 23] +Output [2]: [ss_sold_time_sk#56, ss_store_sk#58] +Input [4]: [ss_sold_time_sk#56, ss_hdemo_sk#57, ss_store_sk#58, hd_demo_sk#60] + +(102) Scan parquet spark_catalog.default.time_dim +Output [3]: [t_time_sk#61, t_hour#62, t_minute#63] +Batched: true +Location [not included in comparison]/{warehouse_dir}/time_dim] +PushedFilters: [IsNotNull(t_hour), IsNotNull(t_minute), EqualTo(t_hour,10), GreaterThanOrEqual(t_minute,30), IsNotNull(t_time_sk)] +ReadSchema: struct<t_time_sk:int,t_hour:int,t_minute:int> + +(103) CometFilter +Input [3]: [t_time_sk#61, t_hour#62, t_minute#63] +Condition : ((((isnotnull(t_hour#62) AND isnotnull(t_minute#63)) AND (t_hour#62 = 10)) AND (t_minute#63 >= 30)) AND isnotnull(t_time_sk#61)) + +(104) CometProject +Input [3]: [t_time_sk#61, t_hour#62, t_minute#63] +Arguments: [t_time_sk#61], [t_time_sk#61] + +(105) ColumnarToRow [codegen id : 21] +Input [1]: [t_time_sk#61] + +(106) BroadcastExchange +Input [1]: [t_time_sk#61] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=14] + +(107) BroadcastHashJoin [codegen id : 23] +Left keys [1]: [ss_sold_time_sk#56] +Right keys [1]: [t_time_sk#61] +Join type: Inner +Join condition: None + +(108) Project [codegen id : 23] +Output [1]: [ss_store_sk#58] +Input [3]: [ss_sold_time_sk#56, ss_store_sk#58, t_time_sk#61] + +(109) ReusedExchange [Reuses operator id: 23] +Output [1]: [s_store_sk#64] + +(110) BroadcastHashJoin [codegen id : 23] +Left keys [1]: [ss_store_sk#58] +Right keys [1]: [s_store_sk#64] +Join type: Inner +Join condition: None + +(111) Project [codegen id : 23] +Output: [] +Input [2]: [ss_store_sk#58, s_store_sk#64] + +(112) HashAggregate [codegen id : 23] +Input: [] +Keys: [] +Functions [1]: [partial_count(1)] +Aggregate Attributes [1]: [count#65] +Results [1]: [count#66] + +(113) Exchange +Input [1]: [count#66] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=15] + +(114) HashAggregate [codegen id : 24] +Input [1]: [count#66] +Keys: [] +Functions [1]: [count(1)] +Aggregate Attributes [1]: [count(1)#67] +Results [1]: [count(1)#67 AS h10_30_to_11#68] + +(115) BroadcastExchange +Input [1]: [h10_30_to_11#68] +Arguments: IdentityBroadcastMode, [plan_id=16] + +(116) BroadcastNestedLoopJoin [codegen id : 40] +Join type: Inner +Join condition: None + +(117) Scan parquet spark_catalog.default.store_sales +Output [4]: [ss_sold_time_sk#69, ss_hdemo_sk#70, ss_store_sk#71, ss_sold_date_sk#72] +Batched: true +Location [not included in comparison]/{warehouse_dir}/store_sales] +PushedFilters: [IsNotNull(ss_hdemo_sk), IsNotNull(ss_sold_time_sk), IsNotNull(ss_store_sk)] +ReadSchema: struct<ss_sold_time_sk:int,ss_hdemo_sk:int,ss_store_sk:int> + +(118) CometFilter +Input [4]: [ss_sold_time_sk#69, ss_hdemo_sk#70, ss_store_sk#71, ss_sold_date_sk#72] +Condition : ((isnotnull(ss_hdemo_sk#70) AND isnotnull(ss_sold_time_sk#69)) AND isnotnull(ss_store_sk#71)) + +(119) CometProject +Input [4]: [ss_sold_time_sk#69, ss_hdemo_sk#70, ss_store_sk#71, ss_sold_date_sk#72] +Arguments: [ss_sold_time_sk#69, ss_hdemo_sk#70, ss_store_sk#71], [ss_sold_time_sk#69, ss_hdemo_sk#70, ss_store_sk#71] + +(120) ColumnarToRow [codegen id : 28] +Input [3]: [ss_sold_time_sk#69, ss_hdemo_sk#70, ss_store_sk#71] + +(121) ReusedExchange [Reuses operator id: 9] +Output [1]: [hd_demo_sk#73] + +(122) BroadcastHashJoin [codegen id : 28] +Left keys [1]: [ss_hdemo_sk#70] +Right keys [1]: [hd_demo_sk#73] +Join type: Inner +Join condition: None + +(123) Project [codegen id : 28] +Output [2]: [ss_sold_time_sk#69, ss_store_sk#71] +Input [4]: [ss_sold_time_sk#69, ss_hdemo_sk#70, ss_store_sk#71, hd_demo_sk#73] + +(124) Scan parquet spark_catalog.default.time_dim +Output [3]: [t_time_sk#74, t_hour#75, t_minute#76] +Batched: true +Location [not included in comparison]/{warehouse_dir}/time_dim] +PushedFilters: [IsNotNull(t_hour), IsNotNull(t_minute), EqualTo(t_hour,11), LessThan(t_minute,30), IsNotNull(t_time_sk)] +ReadSchema: struct<t_time_sk:int,t_hour:int,t_minute:int> + +(125) CometFilter +Input [3]: [t_time_sk#74, t_hour#75, t_minute#76] +Condition : ((((isnotnull(t_hour#75) AND isnotnull(t_minute#76)) AND (t_hour#75 = 11)) AND (t_minute#76 < 30)) AND isnotnull(t_time_sk#74)) + +(126) CometProject +Input [3]: [t_time_sk#74, t_hour#75, t_minute#76] +Arguments: [t_time_sk#74], [t_time_sk#74] + +(127) ColumnarToRow [codegen id : 26] +Input [1]: [t_time_sk#74] + +(128) BroadcastExchange +Input [1]: [t_time_sk#74] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=17] + +(129) BroadcastHashJoin [codegen id : 28] +Left keys [1]: [ss_sold_time_sk#69] +Right keys [1]: [t_time_sk#74] +Join type: Inner +Join condition: None + +(130) Project [codegen id : 28] +Output [1]: [ss_store_sk#71] +Input [3]: [ss_sold_time_sk#69, ss_store_sk#71, t_time_sk#74] + +(131) ReusedExchange [Reuses operator id: 23] +Output [1]: [s_store_sk#77] + +(132) BroadcastHashJoin [codegen id : 28] +Left keys [1]: [ss_store_sk#71] +Right keys [1]: [s_store_sk#77] +Join type: Inner +Join condition: None + +(133) Project [codegen id : 28] +Output: [] +Input [2]: [ss_store_sk#71, s_store_sk#77] + +(134) HashAggregate [codegen id : 28] +Input: [] +Keys: [] +Functions [1]: [partial_count(1)] +Aggregate Attributes [1]: [count#78] +Results [1]: [count#79] + +(135) Exchange +Input [1]: [count#79] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=18] + +(136) HashAggregate [codegen id : 29] +Input [1]: [count#79] +Keys: [] +Functions [1]: [count(1)] +Aggregate Attributes [1]: [count(1)#80] +Results [1]: [count(1)#80 AS h11_to_11_30#81] + +(137) BroadcastExchange +Input [1]: [h11_to_11_30#81] +Arguments: IdentityBroadcastMode, [plan_id=19] + +(138) BroadcastNestedLoopJoin [codegen id : 40] +Join type: Inner +Join condition: None + +(139) Scan parquet spark_catalog.default.store_sales +Output [4]: [ss_sold_time_sk#82, ss_hdemo_sk#83, ss_store_sk#84, ss_sold_date_sk#85] +Batched: true +Location [not included in comparison]/{warehouse_dir}/store_sales] +PushedFilters: [IsNotNull(ss_hdemo_sk), IsNotNull(ss_sold_time_sk), IsNotNull(ss_store_sk)] +ReadSchema: struct<ss_sold_time_sk:int,ss_hdemo_sk:int,ss_store_sk:int> + +(140) CometFilter +Input [4]: [ss_sold_time_sk#82, ss_hdemo_sk#83, ss_store_sk#84, ss_sold_date_sk#85] +Condition : ((isnotnull(ss_hdemo_sk#83) AND isnotnull(ss_sold_time_sk#82)) AND isnotnull(ss_store_sk#84)) + +(141) CometProject +Input [4]: [ss_sold_time_sk#82, ss_hdemo_sk#83, ss_store_sk#84, ss_sold_date_sk#85] +Arguments: [ss_sold_time_sk#82, ss_hdemo_sk#83, ss_store_sk#84], [ss_sold_time_sk#82, ss_hdemo_sk#83, ss_store_sk#84] + +(142) ColumnarToRow [codegen id : 33] +Input [3]: [ss_sold_time_sk#82, ss_hdemo_sk#83, ss_store_sk#84] + +(143) ReusedExchange [Reuses operator id: 9] +Output [1]: [hd_demo_sk#86] + +(144) BroadcastHashJoin [codegen id : 33] +Left keys [1]: [ss_hdemo_sk#83] +Right keys [1]: [hd_demo_sk#86] +Join type: Inner +Join condition: None + +(145) Project [codegen id : 33] +Output [2]: [ss_sold_time_sk#82, ss_store_sk#84] +Input [4]: [ss_sold_time_sk#82, ss_hdemo_sk#83, ss_store_sk#84, hd_demo_sk#86] + +(146) Scan parquet spark_catalog.default.time_dim +Output [3]: [t_time_sk#87, t_hour#88, t_minute#89] +Batched: true +Location [not included in comparison]/{warehouse_dir}/time_dim] +PushedFilters: [IsNotNull(t_hour), IsNotNull(t_minute), EqualTo(t_hour,11), GreaterThanOrEqual(t_minute,30), IsNotNull(t_time_sk)] +ReadSchema: struct<t_time_sk:int,t_hour:int,t_minute:int> + +(147) CometFilter +Input [3]: [t_time_sk#87, t_hour#88, t_minute#89] +Condition : ((((isnotnull(t_hour#88) AND isnotnull(t_minute#89)) AND (t_hour#88 = 11)) AND (t_minute#89 >= 30)) AND isnotnull(t_time_sk#87)) + +(148) CometProject +Input [3]: [t_time_sk#87, t_hour#88, t_minute#89] +Arguments: [t_time_sk#87], [t_time_sk#87] + +(149) ColumnarToRow [codegen id : 31] +Input [1]: [t_time_sk#87] + +(150) BroadcastExchange +Input [1]: [t_time_sk#87] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=20] + +(151) BroadcastHashJoin [codegen id : 33] +Left keys [1]: [ss_sold_time_sk#82] +Right keys [1]: [t_time_sk#87] +Join type: Inner +Join condition: None + +(152) Project [codegen id : 33] +Output [1]: [ss_store_sk#84] +Input [3]: [ss_sold_time_sk#82, ss_store_sk#84, t_time_sk#87] + +(153) ReusedExchange [Reuses operator id: 23] +Output [1]: [s_store_sk#90] + +(154) BroadcastHashJoin [codegen id : 33] +Left keys [1]: [ss_store_sk#84] +Right keys [1]: [s_store_sk#90] +Join type: Inner +Join condition: None + +(155) Project [codegen id : 33] +Output: [] +Input [2]: [ss_store_sk#84, s_store_sk#90] + +(156) HashAggregate [codegen id : 33] +Input: [] +Keys: [] +Functions [1]: [partial_count(1)] +Aggregate Attributes [1]: [count#91] +Results [1]: [count#92] + +(157) Exchange +Input [1]: [count#92] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=21] + +(158) HashAggregate [codegen id : 34] +Input [1]: [count#92] +Keys: [] +Functions [1]: [count(1)] +Aggregate Attributes [1]: [count(1)#93] +Results [1]: [count(1)#93 AS h11_30_to_12#94] + +(159) BroadcastExchange +Input [1]: [h11_30_to_12#94] +Arguments: IdentityBroadcastMode, [plan_id=22] + +(160) BroadcastNestedLoopJoin [codegen id : 40] +Join type: Inner +Join condition: None + +(161) Scan parquet spark_catalog.default.store_sales +Output [4]: [ss_sold_time_sk#95, ss_hdemo_sk#96, ss_store_sk#97, ss_sold_date_sk#98] +Batched: true +Location [not included in comparison]/{warehouse_dir}/store_sales] +PushedFilters: [IsNotNull(ss_hdemo_sk), IsNotNull(ss_sold_time_sk), IsNotNull(ss_store_sk)] +ReadSchema: struct<ss_sold_time_sk:int,ss_hdemo_sk:int,ss_store_sk:int> + +(162) CometFilter +Input [4]: [ss_sold_time_sk#95, ss_hdemo_sk#96, ss_store_sk#97, ss_sold_date_sk#98] +Condition : ((isnotnull(ss_hdemo_sk#96) AND isnotnull(ss_sold_time_sk#95)) AND isnotnull(ss_store_sk#97)) + +(163) CometProject +Input [4]: [ss_sold_time_sk#95, ss_hdemo_sk#96, ss_store_sk#97, ss_sold_date_sk#98] +Arguments: [ss_sold_time_sk#95, ss_hdemo_sk#96, ss_store_sk#97], [ss_sold_time_sk#95, ss_hdemo_sk#96, ss_store_sk#97] + +(164) ColumnarToRow [codegen id : 38] +Input [3]: [ss_sold_time_sk#95, ss_hdemo_sk#96, ss_store_sk#97] + +(165) ReusedExchange [Reuses operator id: 9] +Output [1]: [hd_demo_sk#99] + +(166) BroadcastHashJoin [codegen id : 38] +Left keys [1]: [ss_hdemo_sk#96] +Right keys [1]: [hd_demo_sk#99] +Join type: Inner +Join condition: None + +(167) Project [codegen id : 38] +Output [2]: [ss_sold_time_sk#95, ss_store_sk#97] +Input [4]: [ss_sold_time_sk#95, ss_hdemo_sk#96, ss_store_sk#97, hd_demo_sk#99] + +(168) Scan parquet spark_catalog.default.time_dim +Output [3]: [t_time_sk#100, t_hour#101, t_minute#102] +Batched: true +Location [not included in comparison]/{warehouse_dir}/time_dim] +PushedFilters: [IsNotNull(t_hour), IsNotNull(t_minute), EqualTo(t_hour,12), LessThan(t_minute,30), IsNotNull(t_time_sk)] +ReadSchema: struct<t_time_sk:int,t_hour:int,t_minute:int> + +(169) CometFilter +Input [3]: [t_time_sk#100, t_hour#101, t_minute#102] +Condition : ((((isnotnull(t_hour#101) AND isnotnull(t_minute#102)) AND (t_hour#101 = 12)) AND (t_minute#102 < 30)) AND isnotnull(t_time_sk#100)) + +(170) CometProject +Input [3]: [t_time_sk#100, t_hour#101, t_minute#102] +Arguments: [t_time_sk#100], [t_time_sk#100] + +(171) ColumnarToRow [codegen id : 36] +Input [1]: [t_time_sk#100] + +(172) BroadcastExchange +Input [1]: [t_time_sk#100] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=23] + +(173) BroadcastHashJoin [codegen id : 38] +Left keys [1]: [ss_sold_time_sk#95] +Right keys [1]: [t_time_sk#100] +Join type: Inner +Join condition: None + +(174) Project [codegen id : 38] +Output [1]: [ss_store_sk#97] +Input [3]: [ss_sold_time_sk#95, ss_store_sk#97, t_time_sk#100] + +(175) ReusedExchange [Reuses operator id: 23] +Output [1]: [s_store_sk#103] + +(176) BroadcastHashJoin [codegen id : 38] +Left keys [1]: [ss_store_sk#97] +Right keys [1]: [s_store_sk#103] +Join type: Inner +Join condition: None + +(177) Project [codegen id : 38] +Output: [] +Input [2]: [ss_store_sk#97, s_store_sk#103] + +(178) HashAggregate [codegen id : 38] +Input: [] +Keys: [] +Functions [1]: [partial_count(1)] +Aggregate Attributes [1]: [count#104] +Results [1]: [count#105] + +(179) Exchange +Input [1]: [count#105] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=24] + +(180) HashAggregate [codegen id : 39] +Input [1]: [count#105] +Keys: [] +Functions [1]: [count(1)] +Aggregate Attributes [1]: [count(1)#106] +Results [1]: [count(1)#106 AS h12_to_12_30#107] + +(181) BroadcastExchange +Input [1]: [h12_to_12_30#107] +Arguments: IdentityBroadcastMode, [plan_id=25] + +(182) BroadcastNestedLoopJoin [codegen id : 40] +Join type: Inner +Join condition: None + diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q88/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q88/simplified.txt new file mode 100644 index 000000000..b497e0bab --- /dev/null +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q88/simplified.txt @@ -0,0 +1,265 @@ +WholeStageCodegen (40) + BroadcastNestedLoopJoin + BroadcastNestedLoopJoin + BroadcastNestedLoopJoin + BroadcastNestedLoopJoin + BroadcastNestedLoopJoin + BroadcastNestedLoopJoin + BroadcastNestedLoopJoin + HashAggregate [count] [count(1),h8_30_to_9,count] + InputAdapter + Exchange #1 + WholeStageCodegen (4) + HashAggregate [count,count] + Project + BroadcastHashJoin [ss_store_sk,s_store_sk] + Project [ss_store_sk] + BroadcastHashJoin [ss_sold_time_sk,t_time_sk] + Project [ss_sold_time_sk,ss_store_sk] + BroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] + ColumnarToRow + InputAdapter + CometProject [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk] + CometFilter [ss_hdemo_sk,ss_sold_time_sk,ss_store_sk] + CometScan parquet spark_catalog.default.store_sales [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] + InputAdapter + BroadcastExchange #2 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [hd_demo_sk] + CometFilter [hd_dep_count,hd_vehicle_count,hd_demo_sk] + CometScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_dep_count,hd_vehicle_count] + InputAdapter + BroadcastExchange #3 + WholeStageCodegen (2) + ColumnarToRow + InputAdapter + CometProject [t_time_sk] + CometFilter [t_hour,t_minute,t_time_sk] + CometScan parquet spark_catalog.default.time_dim [t_time_sk,t_hour,t_minute] + InputAdapter + BroadcastExchange #4 + WholeStageCodegen (3) + ColumnarToRow + InputAdapter + CometProject [s_store_sk] + CometFilter [s_store_name,s_store_sk] + CometScan parquet spark_catalog.default.store [s_store_sk,s_store_name] + InputAdapter + BroadcastExchange #5 + WholeStageCodegen (9) + HashAggregate [count] [count(1),h9_to_9_30,count] + InputAdapter + Exchange #6 + WholeStageCodegen (8) + HashAggregate [count,count] + Project + BroadcastHashJoin [ss_store_sk,s_store_sk] + Project [ss_store_sk] + BroadcastHashJoin [ss_sold_time_sk,t_time_sk] + Project [ss_sold_time_sk,ss_store_sk] + BroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] + ColumnarToRow + InputAdapter + CometProject [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk] + CometFilter [ss_hdemo_sk,ss_sold_time_sk,ss_store_sk] + CometScan parquet spark_catalog.default.store_sales [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] + InputAdapter + ReusedExchange [hd_demo_sk] #2 + InputAdapter + BroadcastExchange #7 + WholeStageCodegen (6) + ColumnarToRow + InputAdapter + CometProject [t_time_sk] + CometFilter [t_hour,t_minute,t_time_sk] + CometScan parquet spark_catalog.default.time_dim [t_time_sk,t_hour,t_minute] + InputAdapter + ReusedExchange [s_store_sk] #4 + InputAdapter + BroadcastExchange #8 + WholeStageCodegen (14) + HashAggregate [count] [count(1),h9_30_to_10,count] + InputAdapter + Exchange #9 + WholeStageCodegen (13) + HashAggregate [count,count] + Project + BroadcastHashJoin [ss_store_sk,s_store_sk] + Project [ss_store_sk] + BroadcastHashJoin [ss_sold_time_sk,t_time_sk] + Project [ss_sold_time_sk,ss_store_sk] + BroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] + ColumnarToRow + InputAdapter + CometProject [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk] + CometFilter [ss_hdemo_sk,ss_sold_time_sk,ss_store_sk] + CometScan parquet spark_catalog.default.store_sales [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] + InputAdapter + ReusedExchange [hd_demo_sk] #2 + InputAdapter + BroadcastExchange #10 + WholeStageCodegen (11) + ColumnarToRow + InputAdapter + CometProject [t_time_sk] + CometFilter [t_hour,t_minute,t_time_sk] + CometScan parquet spark_catalog.default.time_dim [t_time_sk,t_hour,t_minute] + InputAdapter + ReusedExchange [s_store_sk] #4 + InputAdapter + BroadcastExchange #11 + WholeStageCodegen (19) + HashAggregate [count] [count(1),h10_to_10_30,count] + InputAdapter + Exchange #12 + WholeStageCodegen (18) + HashAggregate [count,count] + Project + BroadcastHashJoin [ss_store_sk,s_store_sk] + Project [ss_store_sk] + BroadcastHashJoin [ss_sold_time_sk,t_time_sk] + Project [ss_sold_time_sk,ss_store_sk] + BroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] + ColumnarToRow + InputAdapter + CometProject [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk] + CometFilter [ss_hdemo_sk,ss_sold_time_sk,ss_store_sk] + CometScan parquet spark_catalog.default.store_sales [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] + InputAdapter + ReusedExchange [hd_demo_sk] #2 + InputAdapter + BroadcastExchange #13 + WholeStageCodegen (16) + ColumnarToRow + InputAdapter + CometProject [t_time_sk] + CometFilter [t_hour,t_minute,t_time_sk] + CometScan parquet spark_catalog.default.time_dim [t_time_sk,t_hour,t_minute] + InputAdapter + ReusedExchange [s_store_sk] #4 + InputAdapter + BroadcastExchange #14 + WholeStageCodegen (24) + HashAggregate [count] [count(1),h10_30_to_11,count] + InputAdapter + Exchange #15 + WholeStageCodegen (23) + HashAggregate [count,count] + Project + BroadcastHashJoin [ss_store_sk,s_store_sk] + Project [ss_store_sk] + BroadcastHashJoin [ss_sold_time_sk,t_time_sk] + Project [ss_sold_time_sk,ss_store_sk] + BroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] + ColumnarToRow + InputAdapter + CometProject [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk] + CometFilter [ss_hdemo_sk,ss_sold_time_sk,ss_store_sk] + CometScan parquet spark_catalog.default.store_sales [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] + InputAdapter + ReusedExchange [hd_demo_sk] #2 + InputAdapter + BroadcastExchange #16 + WholeStageCodegen (21) + ColumnarToRow + InputAdapter + CometProject [t_time_sk] + CometFilter [t_hour,t_minute,t_time_sk] + CometScan parquet spark_catalog.default.time_dim [t_time_sk,t_hour,t_minute] + InputAdapter + ReusedExchange [s_store_sk] #4 + InputAdapter + BroadcastExchange #17 + WholeStageCodegen (29) + HashAggregate [count] [count(1),h11_to_11_30,count] + InputAdapter + Exchange #18 + WholeStageCodegen (28) + HashAggregate [count,count] + Project + BroadcastHashJoin [ss_store_sk,s_store_sk] + Project [ss_store_sk] + BroadcastHashJoin [ss_sold_time_sk,t_time_sk] + Project [ss_sold_time_sk,ss_store_sk] + BroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] + ColumnarToRow + InputAdapter + CometProject [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk] + CometFilter [ss_hdemo_sk,ss_sold_time_sk,ss_store_sk] + CometScan parquet spark_catalog.default.store_sales [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] + InputAdapter + ReusedExchange [hd_demo_sk] #2 + InputAdapter + BroadcastExchange #19 + WholeStageCodegen (26) + ColumnarToRow + InputAdapter + CometProject [t_time_sk] + CometFilter [t_hour,t_minute,t_time_sk] + CometScan parquet spark_catalog.default.time_dim [t_time_sk,t_hour,t_minute] + InputAdapter + ReusedExchange [s_store_sk] #4 + InputAdapter + BroadcastExchange #20 + WholeStageCodegen (34) + HashAggregate [count] [count(1),h11_30_to_12,count] + InputAdapter + Exchange #21 + WholeStageCodegen (33) + HashAggregate [count,count] + Project + BroadcastHashJoin [ss_store_sk,s_store_sk] + Project [ss_store_sk] + BroadcastHashJoin [ss_sold_time_sk,t_time_sk] + Project [ss_sold_time_sk,ss_store_sk] + BroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] + ColumnarToRow + InputAdapter + CometProject [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk] + CometFilter [ss_hdemo_sk,ss_sold_time_sk,ss_store_sk] + CometScan parquet spark_catalog.default.store_sales [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] + InputAdapter + ReusedExchange [hd_demo_sk] #2 + InputAdapter + BroadcastExchange #22 + WholeStageCodegen (31) + ColumnarToRow + InputAdapter + CometProject [t_time_sk] + CometFilter [t_hour,t_minute,t_time_sk] + CometScan parquet spark_catalog.default.time_dim [t_time_sk,t_hour,t_minute] + InputAdapter + ReusedExchange [s_store_sk] #4 + InputAdapter + BroadcastExchange #23 + WholeStageCodegen (39) + HashAggregate [count] [count(1),h12_to_12_30,count] + InputAdapter + Exchange #24 + WholeStageCodegen (38) + HashAggregate [count,count] + Project + BroadcastHashJoin [ss_store_sk,s_store_sk] + Project [ss_store_sk] + BroadcastHashJoin [ss_sold_time_sk,t_time_sk] + Project [ss_sold_time_sk,ss_store_sk] + BroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] + ColumnarToRow + InputAdapter + CometProject [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk] + CometFilter [ss_hdemo_sk,ss_sold_time_sk,ss_store_sk] + CometScan parquet spark_catalog.default.store_sales [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] + InputAdapter + ReusedExchange [hd_demo_sk] #2 + InputAdapter + BroadcastExchange #25 + WholeStageCodegen (36) + ColumnarToRow + InputAdapter + CometProject [t_time_sk] + CometFilter [t_hour,t_minute,t_time_sk] + CometScan parquet spark_catalog.default.time_dim [t_time_sk,t_hour,t_minute] + InputAdapter + ReusedExchange [s_store_sk] #4 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 new file mode 100644 index 000000000..3657266e2 --- /dev/null +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q89/explain.txt @@ -0,0 +1,189 @@ +== Physical Plan == +TakeOrderedAndProject (27) ++- * Project (26) + +- * Filter (25) + +- Window (24) + +- * Sort (23) + +- Exchange (22) + +- * HashAggregate (21) + +- Exchange (20) + +- * HashAggregate (19) + +- * Project (18) + +- * BroadcastHashJoin Inner BuildRight (17) + :- * Project (12) + : +- * BroadcastHashJoin Inner BuildRight (11) + : :- * Project (9) + : : +- * BroadcastHashJoin Inner BuildRight (8) + : : :- * ColumnarToRow (3) + : : : +- CometFilter (2) + : : : +- CometScan parquet spark_catalog.default.item (1) + : : +- BroadcastExchange (7) + : : +- * ColumnarToRow (6) + : : +- CometFilter (5) + : : +- CometScan parquet spark_catalog.default.store_sales (4) + : +- ReusedExchange (10) + +- BroadcastExchange (16) + +- * ColumnarToRow (15) + +- CometFilter (14) + +- CometScan parquet spark_catalog.default.store (13) + + +(1) Scan parquet spark_catalog.default.item +Output [4]: [i_item_sk#1, i_brand#2, i_class#3, i_category#4] +Batched: true +Location [not included in comparison]/{warehouse_dir}/item] +PushedFilters: [Or(And(In(i_category, [Books ,Electronics ,Sports ]),In(i_class, [computers ,football ,stereo ])),And(In(i_category, [Jewelry ,Men ,Women ]),In(i_class, [birdal ,dresses ,shirts ]))), IsNotNull(i_item_sk)] +ReadSchema: struct<i_item_sk:int,i_brand:string,i_class:string,i_category:string> + +(2) CometFilter +Input [4]: [i_item_sk#1, i_brand#2, i_class#3, i_category#4] +Condition : (((i_category#4 IN (Books ,Electronics ,Sports ) AND i_class#3 IN (computers ,stereo ,football )) OR (i_category#4 IN (Men ,Jewelry ,Women ) AND i_class#3 IN (shirts ,birdal ,dresses ))) AND isnotnull(i_item_sk#1)) + +(3) ColumnarToRow [codegen id : 4] +Input [4]: [i_item_sk#1, i_brand#2, i_class#3, i_category#4] + +(4) Scan parquet spark_catalog.default.store_sales +Output [4]: [ss_item_sk#5, ss_store_sk#6, ss_sales_price#7, ss_sold_date_sk#8] +Batched: true +Location: InMemoryFileIndex [] +PartitionFilters: [isnotnull(ss_sold_date_sk#8), dynamicpruningexpression(ss_sold_date_sk#8 IN dynamicpruning#9)] +PushedFilters: [IsNotNull(ss_item_sk), IsNotNull(ss_store_sk)] +ReadSchema: struct<ss_item_sk:int,ss_store_sk:int,ss_sales_price:decimal(7,2)> + +(5) CometFilter +Input [4]: [ss_item_sk#5, ss_store_sk#6, ss_sales_price#7, ss_sold_date_sk#8] +Condition : (isnotnull(ss_item_sk#5) AND isnotnull(ss_store_sk#6)) + +(6) ColumnarToRow [codegen id : 1] +Input [4]: [ss_item_sk#5, ss_store_sk#6, ss_sales_price#7, ss_sold_date_sk#8] + +(7) BroadcastExchange +Input [4]: [ss_item_sk#5, ss_store_sk#6, ss_sales_price#7, ss_sold_date_sk#8] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=1] + +(8) BroadcastHashJoin [codegen id : 4] +Left keys [1]: [i_item_sk#1] +Right keys [1]: [ss_item_sk#5] +Join type: Inner +Join condition: None + +(9) Project [codegen id : 4] +Output [6]: [i_brand#2, i_class#3, i_category#4, ss_store_sk#6, ss_sales_price#7, ss_sold_date_sk#8] +Input [8]: [i_item_sk#1, i_brand#2, i_class#3, i_category#4, ss_item_sk#5, ss_store_sk#6, ss_sales_price#7, ss_sold_date_sk#8] + +(10) ReusedExchange [Reuses operator id: 32] +Output [2]: [d_date_sk#10, d_moy#11] + +(11) BroadcastHashJoin [codegen id : 4] +Left keys [1]: [ss_sold_date_sk#8] +Right keys [1]: [d_date_sk#10] +Join type: Inner +Join condition: None + +(12) Project [codegen id : 4] +Output [6]: [i_brand#2, i_class#3, i_category#4, ss_store_sk#6, ss_sales_price#7, d_moy#11] +Input [8]: [i_brand#2, i_class#3, i_category#4, ss_store_sk#6, ss_sales_price#7, ss_sold_date_sk#8, d_date_sk#10, d_moy#11] + +(13) Scan parquet spark_catalog.default.store +Output [3]: [s_store_sk#12, s_store_name#13, s_company_name#14] +Batched: true +Location [not included in comparison]/{warehouse_dir}/store] +PushedFilters: [IsNotNull(s_store_sk)] +ReadSchema: struct<s_store_sk:int,s_store_name:string,s_company_name:string> + +(14) CometFilter +Input [3]: [s_store_sk#12, s_store_name#13, s_company_name#14] +Condition : isnotnull(s_store_sk#12) + +(15) ColumnarToRow [codegen id : 3] +Input [3]: [s_store_sk#12, s_store_name#13, s_company_name#14] + +(16) BroadcastExchange +Input [3]: [s_store_sk#12, s_store_name#13, s_company_name#14] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=2] + +(17) BroadcastHashJoin [codegen id : 4] +Left keys [1]: [ss_store_sk#6] +Right keys [1]: [s_store_sk#12] +Join type: Inner +Join condition: None + +(18) Project [codegen id : 4] +Output [7]: [i_brand#2, i_class#3, i_category#4, ss_sales_price#7, d_moy#11, s_store_name#13, s_company_name#14] +Input [9]: [i_brand#2, i_class#3, i_category#4, ss_store_sk#6, ss_sales_price#7, d_moy#11, s_store_sk#12, s_store_name#13, s_company_name#14] + +(19) HashAggregate [codegen id : 4] +Input [7]: [i_brand#2, i_class#3, i_category#4, ss_sales_price#7, d_moy#11, s_store_name#13, s_company_name#14] +Keys [6]: [i_category#4, i_class#3, i_brand#2, s_store_name#13, s_company_name#14, d_moy#11] +Functions [1]: [partial_sum(UnscaledValue(ss_sales_price#7))] +Aggregate Attributes [1]: [sum#15] +Results [7]: [i_category#4, i_class#3, i_brand#2, s_store_name#13, s_company_name#14, d_moy#11, sum#16] + +(20) Exchange +Input [7]: [i_category#4, i_class#3, i_brand#2, s_store_name#13, s_company_name#14, d_moy#11, sum#16] +Arguments: hashpartitioning(i_category#4, i_class#3, i_brand#2, s_store_name#13, s_company_name#14, d_moy#11, 5), ENSURE_REQUIREMENTS, [plan_id=3] + +(21) HashAggregate [codegen id : 5] +Input [7]: [i_category#4, i_class#3, i_brand#2, s_store_name#13, s_company_name#14, d_moy#11, sum#16] +Keys [6]: [i_category#4, i_class#3, i_brand#2, s_store_name#13, s_company_name#14, d_moy#11] +Functions [1]: [sum(UnscaledValue(ss_sales_price#7))] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_sales_price#7))#17] +Results [8]: [i_category#4, i_class#3, i_brand#2, s_store_name#13, s_company_name#14, d_moy#11, MakeDecimal(sum(UnscaledValue(ss_sales_price#7))#17,17,2) AS sum_sales#18, MakeDecimal(sum(UnscaledValue(ss_sales_price#7))#17,17,2) AS _w0#19] + +(22) Exchange +Input [8]: [i_category#4, i_class#3, i_brand#2, s_store_name#13, s_company_name#14, d_moy#11, sum_sales#18, _w0#19] +Arguments: hashpartitioning(i_category#4, i_brand#2, s_store_name#13, s_company_name#14, 5), ENSURE_REQUIREMENTS, [plan_id=4] + +(23) Sort [codegen id : 6] +Input [8]: [i_category#4, i_class#3, i_brand#2, s_store_name#13, s_company_name#14, d_moy#11, sum_sales#18, _w0#19] +Arguments: [i_category#4 ASC NULLS FIRST, i_brand#2 ASC NULLS FIRST, s_store_name#13 ASC NULLS FIRST, s_company_name#14 ASC NULLS FIRST], false, 0 + +(24) Window +Input [8]: [i_category#4, i_class#3, i_brand#2, s_store_name#13, s_company_name#14, d_moy#11, sum_sales#18, _w0#19] +Arguments: [avg(_w0#19) windowspecdefinition(i_category#4, i_brand#2, s_store_name#13, s_company_name#14, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS avg_monthly_sales#20], [i_category#4, i_brand#2, s_store_name#13, s_company_name#14] + +(25) Filter [codegen id : 7] +Input [9]: [i_category#4, i_class#3, i_brand#2, s_store_name#13, s_company_name#14, d_moy#11, sum_sales#18, _w0#19, avg_monthly_sales#20] +Condition : CASE WHEN NOT (avg_monthly_sales#20 = 0.000000) THEN ((abs((sum_sales#18 - avg_monthly_sales#20)) / avg_monthly_sales#20) > 0.1000000000000000) END + +(26) Project [codegen id : 7] +Output [8]: [i_category#4, i_class#3, i_brand#2, s_store_name#13, s_company_name#14, d_moy#11, sum_sales#18, avg_monthly_sales#20] +Input [9]: [i_category#4, i_class#3, i_brand#2, s_store_name#13, s_company_name#14, d_moy#11, sum_sales#18, _w0#19, avg_monthly_sales#20] + +(27) TakeOrderedAndProject +Input [8]: [i_category#4, i_class#3, i_brand#2, s_store_name#13, s_company_name#14, d_moy#11, sum_sales#18, avg_monthly_sales#20] +Arguments: 100, [(sum_sales#18 - avg_monthly_sales#20) ASC NULLS FIRST, s_store_name#13 ASC NULLS FIRST], [i_category#4, i_class#3, i_brand#2, s_store_name#13, s_company_name#14, d_moy#11, sum_sales#18, avg_monthly_sales#20] + +===== Subqueries ===== + +Subquery:1 Hosting operator id = 4 Hosting Expression = ss_sold_date_sk#8 IN dynamicpruning#9 +BroadcastExchange (32) ++- * ColumnarToRow (31) + +- CometProject (30) + +- CometFilter (29) + +- CometScan parquet spark_catalog.default.date_dim (28) + + +(28) Scan parquet spark_catalog.default.date_dim +Output [3]: [d_date_sk#10, d_year#21, d_moy#11] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_year), EqualTo(d_year,1999), IsNotNull(d_date_sk)] +ReadSchema: struct<d_date_sk:int,d_year:int,d_moy:int> + +(29) CometFilter +Input [3]: [d_date_sk#10, d_year#21, d_moy#11] +Condition : ((isnotnull(d_year#21) AND (d_year#21 = 1999)) AND isnotnull(d_date_sk#10)) + +(30) CometProject +Input [3]: [d_date_sk#10, d_year#21, d_moy#11] +Arguments: [d_date_sk#10, d_moy#11], [d_date_sk#10, d_moy#11] + +(31) ColumnarToRow [codegen id : 1] +Input [2]: [d_date_sk#10, d_moy#11] + +(32) BroadcastExchange +Input [2]: [d_date_sk#10, d_moy#11] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] + + 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 new file mode 100644 index 000000000..bb9e4e17e --- /dev/null +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q89/simplified.txt @@ -0,0 +1,50 @@ +TakeOrderedAndProject [sum_sales,avg_monthly_sales,s_store_name,i_category,i_class,i_brand,s_company_name,d_moy] + WholeStageCodegen (7) + 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) + Sort [i_category,i_brand,s_store_name,s_company_name] + InputAdapter + Exchange [i_category,i_brand,s_store_name,s_company_name] #1 + WholeStageCodegen (5) + HashAggregate [i_category,i_class,i_brand,s_store_name,s_company_name,d_moy,sum] [sum(UnscaledValue(ss_sales_price)),sum_sales,_w0,sum] + InputAdapter + Exchange [i_category,i_class,i_brand,s_store_name,s_company_name,d_moy] #2 + WholeStageCodegen (4) + HashAggregate [i_category,i_class,i_brand,s_store_name,s_company_name,d_moy,ss_sales_price] [sum,sum] + Project [i_brand,i_class,i_category,ss_sales_price,d_moy,s_store_name,s_company_name] + BroadcastHashJoin [ss_store_sk,s_store_sk] + Project [i_brand,i_class,i_category,ss_store_sk,ss_sales_price,d_moy] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [i_brand,i_class,i_category,ss_store_sk,ss_sales_price,ss_sold_date_sk] + BroadcastHashJoin [i_item_sk,ss_item_sk] + ColumnarToRow + InputAdapter + CometFilter [i_category,i_class,i_item_sk] + CometScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category] + InputAdapter + BroadcastExchange #3 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometFilter [ss_item_sk,ss_store_sk] + CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #4 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk,d_moy] + CometFilter [d_year,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + InputAdapter + ReusedExchange [d_date_sk,d_moy] #4 + InputAdapter + BroadcastExchange #5 + WholeStageCodegen (3) + ColumnarToRow + InputAdapter + CometFilter [s_store_sk] + CometScan parquet spark_catalog.default.store [s_store_sk,s_store_name,s_company_name] 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 new file mode 100644 index 000000000..69c23211b --- /dev/null +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q9/explain.txt @@ -0,0 +1,303 @@ +== Physical Plan == +* Project (4) ++- * ColumnarToRow (3) + +- CometFilter (2) + +- CometScan parquet spark_catalog.default.reason (1) + + +(1) Scan parquet spark_catalog.default.reason +Output [1]: [r_reason_sk#1] +Batched: true +Location [not included in comparison]/{warehouse_dir}/reason] +PushedFilters: [IsNotNull(r_reason_sk), EqualTo(r_reason_sk,1)] +ReadSchema: struct<r_reason_sk:int> + +(2) CometFilter +Input [1]: [r_reason_sk#1] +Condition : (isnotnull(r_reason_sk#1) AND (r_reason_sk#1 = 1)) + +(3) ColumnarToRow [codegen id : 1] +Input [1]: [r_reason_sk#1] + +(4) Project [codegen id : 1] +Output [5]: [CASE WHEN (Subquery scalar-subquery#2, [id=#3].count(1) > 62316685) THEN ReusedSubquery Subquery scalar-subquery#2, [id=#3].avg(ss_ext_discount_amt) ELSE ReusedSubquery Subquery scalar-subquery#2, [id=#3].avg(ss_net_paid) END AS bucket1#4, CASE WHEN (Subquery scalar-subquery#5, [id=#6].count(1) > 19045798) THEN ReusedSubquery Subquery scalar-subquery#5, [id=#6].avg(ss_ext_discount_amt) ELSE ReusedSubquery Subquery scalar-subquery#5, [id=#6].avg(ss_net_paid) END AS bucket2#7, CASE WHEN (Subquery scalar-subquery#8, [id=#9].count(1) > 365541424) THEN ReusedSubquery Subquery scalar-subquery#8, [id=#9].avg(ss_ext_discount_amt) ELSE ReusedSubquery Subquery scalar-subquery#8, [id=#9].avg(ss_net_paid) END AS bucket3#10, CASE WHEN (Subquery scalar-subquery#11, [id=#12].count(1) > 216357808) THEN ReusedSubquery Subquery scalar-subquery#11, [id=#12].avg(ss_ext_discount_amt) ELSE ReusedSubquery Subquery scalar-subquery#11, [id=#12].avg(ss_net_paid) END AS bucket4#13, CASE WHEN (Subquery scalar-subquery#14, [id=#15].count(1) > 184483884) THEN ReusedSubquery Subquery scalar-subquery#14, [id=#15].avg(ss_ext_discount_amt) ELSE ReusedSubquery Subquery scalar-subquery#14, [id=#15].avg(ss_net_paid) END AS bucket5#16] +Input [1]: [r_reason_sk#1] + +===== Subqueries ===== + +Subquery:1 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquery#2, [id=#3] +* Project (12) ++- * HashAggregate (11) + +- Exchange (10) + +- * HashAggregate (9) + +- * ColumnarToRow (8) + +- CometProject (7) + +- CometFilter (6) + +- CometScan parquet spark_catalog.default.store_sales (5) + + +(5) Scan parquet spark_catalog.default.store_sales +Output [4]: [ss_quantity#17, ss_ext_discount_amt#18, ss_net_paid#19, ss_sold_date_sk#20] +Batched: true +Location [not included in comparison]/{warehouse_dir}/store_sales] +PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,1), LessThanOrEqual(ss_quantity,20)] +ReadSchema: struct<ss_quantity:int,ss_ext_discount_amt:decimal(7,2),ss_net_paid:decimal(7,2)> + +(6) CometFilter +Input [4]: [ss_quantity#17, ss_ext_discount_amt#18, ss_net_paid#19, ss_sold_date_sk#20] +Condition : ((isnotnull(ss_quantity#17) AND (ss_quantity#17 >= 1)) AND (ss_quantity#17 <= 20)) + +(7) CometProject +Input [4]: [ss_quantity#17, ss_ext_discount_amt#18, ss_net_paid#19, ss_sold_date_sk#20] +Arguments: [ss_ext_discount_amt#18, ss_net_paid#19], [ss_ext_discount_amt#18, ss_net_paid#19] + +(8) ColumnarToRow [codegen id : 1] +Input [2]: [ss_ext_discount_amt#18, ss_net_paid#19] + +(9) HashAggregate [codegen id : 1] +Input [2]: [ss_ext_discount_amt#18, ss_net_paid#19] +Keys: [] +Functions [3]: [partial_count(1), partial_avg(UnscaledValue(ss_ext_discount_amt#18)), partial_avg(UnscaledValue(ss_net_paid#19))] +Aggregate Attributes [5]: [count#21, sum#22, count#23, sum#24, count#25] +Results [5]: [count#26, sum#27, count#28, sum#29, count#30] + +(10) Exchange +Input [5]: [count#26, sum#27, count#28, sum#29, count#30] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=1] + +(11) HashAggregate [codegen id : 2] +Input [5]: [count#26, sum#27, count#28, sum#29, count#30] +Keys: [] +Functions [3]: [count(1), avg(UnscaledValue(ss_ext_discount_amt#18)), avg(UnscaledValue(ss_net_paid#19))] +Aggregate Attributes [3]: [count(1)#31, avg(UnscaledValue(ss_ext_discount_amt#18))#32, avg(UnscaledValue(ss_net_paid#19))#33] +Results [3]: [count(1)#31 AS count(1)#34, cast((avg(UnscaledValue(ss_ext_discount_amt#18))#32 / 100.0) as decimal(11,6)) AS avg(ss_ext_discount_amt)#35, cast((avg(UnscaledValue(ss_net_paid#19))#33 / 100.0) as decimal(11,6)) AS avg(ss_net_paid)#36] + +(12) Project [codegen id : 2] +Output [1]: [named_struct(count(1), count(1)#34, avg(ss_ext_discount_amt), avg(ss_ext_discount_amt)#35, avg(ss_net_paid), avg(ss_net_paid)#36) AS mergedValue#37] +Input [3]: [count(1)#34, avg(ss_ext_discount_amt)#35, avg(ss_net_paid)#36] + +Subquery:2 Hosting operator id = 4 Hosting Expression = ReusedSubquery Subquery scalar-subquery#2, [id=#3] + +Subquery:3 Hosting operator id = 4 Hosting Expression = ReusedSubquery Subquery scalar-subquery#2, [id=#3] + +Subquery:4 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquery#5, [id=#6] +* Project (20) ++- * HashAggregate (19) + +- Exchange (18) + +- * HashAggregate (17) + +- * ColumnarToRow (16) + +- CometProject (15) + +- CometFilter (14) + +- CometScan parquet spark_catalog.default.store_sales (13) + + +(13) Scan parquet spark_catalog.default.store_sales +Output [4]: [ss_quantity#38, ss_ext_discount_amt#39, ss_net_paid#40, ss_sold_date_sk#41] +Batched: true +Location [not included in comparison]/{warehouse_dir}/store_sales] +PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,21), LessThanOrEqual(ss_quantity,40)] +ReadSchema: struct<ss_quantity:int,ss_ext_discount_amt:decimal(7,2),ss_net_paid:decimal(7,2)> + +(14) CometFilter +Input [4]: [ss_quantity#38, ss_ext_discount_amt#39, ss_net_paid#40, ss_sold_date_sk#41] +Condition : ((isnotnull(ss_quantity#38) AND (ss_quantity#38 >= 21)) AND (ss_quantity#38 <= 40)) + +(15) CometProject +Input [4]: [ss_quantity#38, ss_ext_discount_amt#39, ss_net_paid#40, ss_sold_date_sk#41] +Arguments: [ss_ext_discount_amt#39, ss_net_paid#40], [ss_ext_discount_amt#39, ss_net_paid#40] + +(16) ColumnarToRow [codegen id : 1] +Input [2]: [ss_ext_discount_amt#39, ss_net_paid#40] + +(17) HashAggregate [codegen id : 1] +Input [2]: [ss_ext_discount_amt#39, ss_net_paid#40] +Keys: [] +Functions [3]: [partial_count(1), partial_avg(UnscaledValue(ss_ext_discount_amt#39)), partial_avg(UnscaledValue(ss_net_paid#40))] +Aggregate Attributes [5]: [count#42, sum#43, count#44, sum#45, count#46] +Results [5]: [count#47, sum#48, count#49, sum#50, count#51] + +(18) Exchange +Input [5]: [count#47, sum#48, count#49, sum#50, count#51] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=2] + +(19) HashAggregate [codegen id : 2] +Input [5]: [count#47, sum#48, count#49, sum#50, count#51] +Keys: [] +Functions [3]: [count(1), avg(UnscaledValue(ss_ext_discount_amt#39)), avg(UnscaledValue(ss_net_paid#40))] +Aggregate Attributes [3]: [count(1)#52, avg(UnscaledValue(ss_ext_discount_amt#39))#53, avg(UnscaledValue(ss_net_paid#40))#54] +Results [3]: [count(1)#52 AS count(1)#55, cast((avg(UnscaledValue(ss_ext_discount_amt#39))#53 / 100.0) as decimal(11,6)) AS avg(ss_ext_discount_amt)#56, cast((avg(UnscaledValue(ss_net_paid#40))#54 / 100.0) as decimal(11,6)) AS avg(ss_net_paid)#57] + +(20) Project [codegen id : 2] +Output [1]: [named_struct(count(1), count(1)#55, avg(ss_ext_discount_amt), avg(ss_ext_discount_amt)#56, avg(ss_net_paid), avg(ss_net_paid)#57) AS mergedValue#58] +Input [3]: [count(1)#55, avg(ss_ext_discount_amt)#56, avg(ss_net_paid)#57] + +Subquery:5 Hosting operator id = 4 Hosting Expression = ReusedSubquery Subquery scalar-subquery#5, [id=#6] + +Subquery:6 Hosting operator id = 4 Hosting Expression = ReusedSubquery Subquery scalar-subquery#5, [id=#6] + +Subquery:7 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquery#8, [id=#9] +* Project (28) ++- * HashAggregate (27) + +- Exchange (26) + +- * HashAggregate (25) + +- * ColumnarToRow (24) + +- CometProject (23) + +- CometFilter (22) + +- CometScan parquet spark_catalog.default.store_sales (21) + + +(21) Scan parquet spark_catalog.default.store_sales +Output [4]: [ss_quantity#59, ss_ext_discount_amt#60, ss_net_paid#61, ss_sold_date_sk#62] +Batched: true +Location [not included in comparison]/{warehouse_dir}/store_sales] +PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,41), LessThanOrEqual(ss_quantity,60)] +ReadSchema: struct<ss_quantity:int,ss_ext_discount_amt:decimal(7,2),ss_net_paid:decimal(7,2)> + +(22) CometFilter +Input [4]: [ss_quantity#59, ss_ext_discount_amt#60, ss_net_paid#61, ss_sold_date_sk#62] +Condition : ((isnotnull(ss_quantity#59) AND (ss_quantity#59 >= 41)) AND (ss_quantity#59 <= 60)) + +(23) CometProject +Input [4]: [ss_quantity#59, ss_ext_discount_amt#60, ss_net_paid#61, ss_sold_date_sk#62] +Arguments: [ss_ext_discount_amt#60, ss_net_paid#61], [ss_ext_discount_amt#60, ss_net_paid#61] + +(24) ColumnarToRow [codegen id : 1] +Input [2]: [ss_ext_discount_amt#60, ss_net_paid#61] + +(25) HashAggregate [codegen id : 1] +Input [2]: [ss_ext_discount_amt#60, ss_net_paid#61] +Keys: [] +Functions [3]: [partial_count(1), partial_avg(UnscaledValue(ss_ext_discount_amt#60)), partial_avg(UnscaledValue(ss_net_paid#61))] +Aggregate Attributes [5]: [count#63, sum#64, count#65, sum#66, count#67] +Results [5]: [count#68, sum#69, count#70, sum#71, count#72] + +(26) Exchange +Input [5]: [count#68, sum#69, count#70, sum#71, count#72] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=3] + +(27) HashAggregate [codegen id : 2] +Input [5]: [count#68, sum#69, count#70, sum#71, count#72] +Keys: [] +Functions [3]: [count(1), avg(UnscaledValue(ss_ext_discount_amt#60)), avg(UnscaledValue(ss_net_paid#61))] +Aggregate Attributes [3]: [count(1)#73, avg(UnscaledValue(ss_ext_discount_amt#60))#74, avg(UnscaledValue(ss_net_paid#61))#75] +Results [3]: [count(1)#73 AS count(1)#76, cast((avg(UnscaledValue(ss_ext_discount_amt#60))#74 / 100.0) as decimal(11,6)) AS avg(ss_ext_discount_amt)#77, cast((avg(UnscaledValue(ss_net_paid#61))#75 / 100.0) as decimal(11,6)) AS avg(ss_net_paid)#78] + +(28) Project [codegen id : 2] +Output [1]: [named_struct(count(1), count(1)#76, avg(ss_ext_discount_amt), avg(ss_ext_discount_amt)#77, avg(ss_net_paid), avg(ss_net_paid)#78) AS mergedValue#79] +Input [3]: [count(1)#76, avg(ss_ext_discount_amt)#77, avg(ss_net_paid)#78] + +Subquery:8 Hosting operator id = 4 Hosting Expression = ReusedSubquery Subquery scalar-subquery#8, [id=#9] + +Subquery:9 Hosting operator id = 4 Hosting Expression = ReusedSubquery Subquery scalar-subquery#8, [id=#9] + +Subquery:10 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquery#11, [id=#12] +* Project (36) ++- * HashAggregate (35) + +- Exchange (34) + +- * HashAggregate (33) + +- * ColumnarToRow (32) + +- CometProject (31) + +- CometFilter (30) + +- CometScan parquet spark_catalog.default.store_sales (29) + + +(29) Scan parquet spark_catalog.default.store_sales +Output [4]: [ss_quantity#80, ss_ext_discount_amt#81, ss_net_paid#82, ss_sold_date_sk#83] +Batched: true +Location [not included in comparison]/{warehouse_dir}/store_sales] +PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,61), LessThanOrEqual(ss_quantity,80)] +ReadSchema: struct<ss_quantity:int,ss_ext_discount_amt:decimal(7,2),ss_net_paid:decimal(7,2)> + +(30) CometFilter +Input [4]: [ss_quantity#80, ss_ext_discount_amt#81, ss_net_paid#82, ss_sold_date_sk#83] +Condition : ((isnotnull(ss_quantity#80) AND (ss_quantity#80 >= 61)) AND (ss_quantity#80 <= 80)) + +(31) CometProject +Input [4]: [ss_quantity#80, ss_ext_discount_amt#81, ss_net_paid#82, ss_sold_date_sk#83] +Arguments: [ss_ext_discount_amt#81, ss_net_paid#82], [ss_ext_discount_amt#81, ss_net_paid#82] + +(32) ColumnarToRow [codegen id : 1] +Input [2]: [ss_ext_discount_amt#81, ss_net_paid#82] + +(33) HashAggregate [codegen id : 1] +Input [2]: [ss_ext_discount_amt#81, ss_net_paid#82] +Keys: [] +Functions [3]: [partial_count(1), partial_avg(UnscaledValue(ss_ext_discount_amt#81)), partial_avg(UnscaledValue(ss_net_paid#82))] +Aggregate Attributes [5]: [count#84, sum#85, count#86, sum#87, count#88] +Results [5]: [count#89, sum#90, count#91, sum#92, count#93] + +(34) Exchange +Input [5]: [count#89, sum#90, count#91, sum#92, count#93] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=4] + +(35) HashAggregate [codegen id : 2] +Input [5]: [count#89, sum#90, count#91, sum#92, count#93] +Keys: [] +Functions [3]: [count(1), avg(UnscaledValue(ss_ext_discount_amt#81)), avg(UnscaledValue(ss_net_paid#82))] +Aggregate Attributes [3]: [count(1)#94, avg(UnscaledValue(ss_ext_discount_amt#81))#95, avg(UnscaledValue(ss_net_paid#82))#96] +Results [3]: [count(1)#94 AS count(1)#97, cast((avg(UnscaledValue(ss_ext_discount_amt#81))#95 / 100.0) as decimal(11,6)) AS avg(ss_ext_discount_amt)#98, cast((avg(UnscaledValue(ss_net_paid#82))#96 / 100.0) as decimal(11,6)) AS avg(ss_net_paid)#99] + +(36) Project [codegen id : 2] +Output [1]: [named_struct(count(1), count(1)#97, avg(ss_ext_discount_amt), avg(ss_ext_discount_amt)#98, avg(ss_net_paid), avg(ss_net_paid)#99) AS mergedValue#100] +Input [3]: [count(1)#97, avg(ss_ext_discount_amt)#98, avg(ss_net_paid)#99] + +Subquery:11 Hosting operator id = 4 Hosting Expression = ReusedSubquery Subquery scalar-subquery#11, [id=#12] + +Subquery:12 Hosting operator id = 4 Hosting Expression = ReusedSubquery Subquery scalar-subquery#11, [id=#12] + +Subquery:13 Hosting operator id = 4 Hosting Expression = Subquery scalar-subquery#14, [id=#15] +* Project (44) ++- * HashAggregate (43) + +- Exchange (42) + +- * HashAggregate (41) + +- * ColumnarToRow (40) + +- CometProject (39) + +- CometFilter (38) + +- CometScan parquet spark_catalog.default.store_sales (37) + + +(37) Scan parquet spark_catalog.default.store_sales +Output [4]: [ss_quantity#101, ss_ext_discount_amt#102, ss_net_paid#103, ss_sold_date_sk#104] +Batched: true +Location [not included in comparison]/{warehouse_dir}/store_sales] +PushedFilters: [IsNotNull(ss_quantity), GreaterThanOrEqual(ss_quantity,81), LessThanOrEqual(ss_quantity,100)] +ReadSchema: struct<ss_quantity:int,ss_ext_discount_amt:decimal(7,2),ss_net_paid:decimal(7,2)> + +(38) CometFilter +Input [4]: [ss_quantity#101, ss_ext_discount_amt#102, ss_net_paid#103, ss_sold_date_sk#104] +Condition : ((isnotnull(ss_quantity#101) AND (ss_quantity#101 >= 81)) AND (ss_quantity#101 <= 100)) + +(39) CometProject +Input [4]: [ss_quantity#101, ss_ext_discount_amt#102, ss_net_paid#103, ss_sold_date_sk#104] +Arguments: [ss_ext_discount_amt#102, ss_net_paid#103], [ss_ext_discount_amt#102, ss_net_paid#103] + +(40) ColumnarToRow [codegen id : 1] +Input [2]: [ss_ext_discount_amt#102, ss_net_paid#103] + +(41) HashAggregate [codegen id : 1] +Input [2]: [ss_ext_discount_amt#102, ss_net_paid#103] +Keys: [] +Functions [3]: [partial_count(1), partial_avg(UnscaledValue(ss_ext_discount_amt#102)), partial_avg(UnscaledValue(ss_net_paid#103))] +Aggregate Attributes [5]: [count#105, sum#106, count#107, sum#108, count#109] +Results [5]: [count#110, sum#111, count#112, sum#113, count#114] + +(42) Exchange +Input [5]: [count#110, sum#111, count#112, sum#113, count#114] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=5] + +(43) HashAggregate [codegen id : 2] +Input [5]: [count#110, sum#111, count#112, sum#113, count#114] +Keys: [] +Functions [3]: [count(1), avg(UnscaledValue(ss_ext_discount_amt#102)), avg(UnscaledValue(ss_net_paid#103))] +Aggregate Attributes [3]: [count(1)#115, avg(UnscaledValue(ss_ext_discount_amt#102))#116, avg(UnscaledValue(ss_net_paid#103))#117] +Results [3]: [count(1)#115 AS count(1)#118, cast((avg(UnscaledValue(ss_ext_discount_amt#102))#116 / 100.0) as decimal(11,6)) AS avg(ss_ext_discount_amt)#119, cast((avg(UnscaledValue(ss_net_paid#103))#117 / 100.0) as decimal(11,6)) AS avg(ss_net_paid)#120] + +(44) Project [codegen id : 2] +Output [1]: [named_struct(count(1), count(1)#118, avg(ss_ext_discount_amt), avg(ss_ext_discount_amt)#119, avg(ss_net_paid), avg(ss_net_paid)#120) AS mergedValue#121] +Input [3]: [count(1)#118, avg(ss_ext_discount_amt)#119, avg(ss_net_paid)#120] + +Subquery:14 Hosting operator id = 4 Hosting Expression = ReusedSubquery Subquery scalar-subquery#14, [id=#15] + +Subquery:15 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/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q9/simplified.txt new file mode 100644 index 000000000..fdd3bd293 --- /dev/null +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q9/simplified.txt @@ -0,0 +1,81 @@ +WholeStageCodegen (1) + Project + Subquery #1 + WholeStageCodegen (2) + Project [count(1),avg(ss_ext_discount_amt),avg(ss_net_paid)] + HashAggregate [count,sum,count,sum,count] [count(1),avg(UnscaledValue(ss_ext_discount_amt)),avg(UnscaledValue(ss_net_paid)),count(1),avg(ss_ext_discount_amt),avg(ss_net_paid),count,sum,count,sum,count] + InputAdapter + Exchange #1 + WholeStageCodegen (1) + HashAggregate [ss_ext_discount_amt,ss_net_paid] [count,sum,count,sum,count,count,sum,count,sum,count] + ColumnarToRow + InputAdapter + CometProject [ss_ext_discount_amt,ss_net_paid] + CometFilter [ss_quantity] + CometScan parquet spark_catalog.default.store_sales [ss_quantity,ss_ext_discount_amt,ss_net_paid,ss_sold_date_sk] + ReusedSubquery [mergedValue] #1 + ReusedSubquery [mergedValue] #1 + Subquery #2 + WholeStageCodegen (2) + Project [count(1),avg(ss_ext_discount_amt),avg(ss_net_paid)] + HashAggregate [count,sum,count,sum,count] [count(1),avg(UnscaledValue(ss_ext_discount_amt)),avg(UnscaledValue(ss_net_paid)),count(1),avg(ss_ext_discount_amt),avg(ss_net_paid),count,sum,count,sum,count] + InputAdapter + Exchange #2 + WholeStageCodegen (1) + HashAggregate [ss_ext_discount_amt,ss_net_paid] [count,sum,count,sum,count,count,sum,count,sum,count] + ColumnarToRow + InputAdapter + CometProject [ss_ext_discount_amt,ss_net_paid] + CometFilter [ss_quantity] + CometScan parquet spark_catalog.default.store_sales [ss_quantity,ss_ext_discount_amt,ss_net_paid,ss_sold_date_sk] + ReusedSubquery [mergedValue] #2 + ReusedSubquery [mergedValue] #2 + Subquery #3 + WholeStageCodegen (2) + Project [count(1),avg(ss_ext_discount_amt),avg(ss_net_paid)] + HashAggregate [count,sum,count,sum,count] [count(1),avg(UnscaledValue(ss_ext_discount_amt)),avg(UnscaledValue(ss_net_paid)),count(1),avg(ss_ext_discount_amt),avg(ss_net_paid),count,sum,count,sum,count] + InputAdapter + Exchange #3 + WholeStageCodegen (1) + HashAggregate [ss_ext_discount_amt,ss_net_paid] [count,sum,count,sum,count,count,sum,count,sum,count] + ColumnarToRow + InputAdapter + CometProject [ss_ext_discount_amt,ss_net_paid] + CometFilter [ss_quantity] + CometScan parquet spark_catalog.default.store_sales [ss_quantity,ss_ext_discount_amt,ss_net_paid,ss_sold_date_sk] + ReusedSubquery [mergedValue] #3 + ReusedSubquery [mergedValue] #3 + Subquery #4 + WholeStageCodegen (2) + Project [count(1),avg(ss_ext_discount_amt),avg(ss_net_paid)] + HashAggregate [count,sum,count,sum,count] [count(1),avg(UnscaledValue(ss_ext_discount_amt)),avg(UnscaledValue(ss_net_paid)),count(1),avg(ss_ext_discount_amt),avg(ss_net_paid),count,sum,count,sum,count] + InputAdapter + Exchange #4 + WholeStageCodegen (1) + HashAggregate [ss_ext_discount_amt,ss_net_paid] [count,sum,count,sum,count,count,sum,count,sum,count] + ColumnarToRow + InputAdapter + CometProject [ss_ext_discount_amt,ss_net_paid] + CometFilter [ss_quantity] + CometScan parquet spark_catalog.default.store_sales [ss_quantity,ss_ext_discount_amt,ss_net_paid,ss_sold_date_sk] + ReusedSubquery [mergedValue] #4 + ReusedSubquery [mergedValue] #4 + Subquery #5 + WholeStageCodegen (2) + Project [count(1),avg(ss_ext_discount_amt),avg(ss_net_paid)] + HashAggregate [count,sum,count,sum,count] [count(1),avg(UnscaledValue(ss_ext_discount_amt)),avg(UnscaledValue(ss_net_paid)),count(1),avg(ss_ext_discount_amt),avg(ss_net_paid),count,sum,count,sum,count] + InputAdapter + Exchange #5 + WholeStageCodegen (1) + HashAggregate [ss_ext_discount_amt,ss_net_paid] [count,sum,count,sum,count,count,sum,count,sum,count] + ColumnarToRow + InputAdapter + CometProject [ss_ext_discount_amt,ss_net_paid] + CometFilter [ss_quantity] + CometScan parquet spark_catalog.default.store_sales [ss_quantity,ss_ext_discount_amt,ss_net_paid,ss_sold_date_sk] + ReusedSubquery [mergedValue] #5 + ReusedSubquery [mergedValue] #5 + ColumnarToRow + InputAdapter + CometFilter [r_reason_sk] + CometScan 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/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q90/explain.txt new file mode 100644 index 000000000..fcfbca847 --- /dev/null +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q90/explain.txt @@ -0,0 +1,292 @@ +== Physical Plan == +* Project (51) ++- * BroadcastNestedLoopJoin Inner BuildRight (50) + :- * HashAggregate (28) + : +- Exchange (27) + : +- * HashAggregate (26) + : +- * Project (25) + : +- * BroadcastHashJoin Inner BuildRight (24) + : :- * Project (18) + : : +- * BroadcastHashJoin Inner BuildRight (17) + : : :- * Project (11) + : : : +- * BroadcastHashJoin Inner BuildRight (10) + : : : :- * ColumnarToRow (4) + : : : : +- CometProject (3) + : : : : +- CometFilter (2) + : : : : +- CometScan parquet spark_catalog.default.web_sales (1) + : : : +- BroadcastExchange (9) + : : : +- * ColumnarToRow (8) + : : : +- CometProject (7) + : : : +- CometFilter (6) + : : : +- CometScan parquet spark_catalog.default.household_demographics (5) + : : +- BroadcastExchange (16) + : : +- * ColumnarToRow (15) + : : +- CometProject (14) + : : +- CometFilter (13) + : : +- CometScan parquet spark_catalog.default.time_dim (12) + : +- BroadcastExchange (23) + : +- * ColumnarToRow (22) + : +- CometProject (21) + : +- CometFilter (20) + : +- CometScan parquet spark_catalog.default.web_page (19) + +- BroadcastExchange (49) + +- * HashAggregate (48) + +- Exchange (47) + +- * HashAggregate (46) + +- * Project (45) + +- * BroadcastHashJoin Inner BuildRight (44) + :- * Project (42) + : +- * BroadcastHashJoin Inner BuildRight (41) + : :- * Project (35) + : : +- * BroadcastHashJoin Inner BuildRight (34) + : : :- * ColumnarToRow (32) + : : : +- CometProject (31) + : : : +- CometFilter (30) + : : : +- CometScan parquet spark_catalog.default.web_sales (29) + : : +- ReusedExchange (33) + : +- BroadcastExchange (40) + : +- * ColumnarToRow (39) + : +- CometProject (38) + : +- CometFilter (37) + : +- CometScan parquet spark_catalog.default.time_dim (36) + +- ReusedExchange (43) + + +(1) Scan parquet spark_catalog.default.web_sales +Output [4]: [ws_sold_time_sk#1, ws_ship_hdemo_sk#2, ws_web_page_sk#3, ws_sold_date_sk#4] +Batched: true +Location [not included in comparison]/{warehouse_dir}/web_sales] +PushedFilters: [IsNotNull(ws_ship_hdemo_sk), IsNotNull(ws_sold_time_sk), IsNotNull(ws_web_page_sk)] +ReadSchema: struct<ws_sold_time_sk:int,ws_ship_hdemo_sk:int,ws_web_page_sk:int> + +(2) CometFilter +Input [4]: [ws_sold_time_sk#1, ws_ship_hdemo_sk#2, ws_web_page_sk#3, ws_sold_date_sk#4] +Condition : ((isnotnull(ws_ship_hdemo_sk#2) AND isnotnull(ws_sold_time_sk#1)) AND isnotnull(ws_web_page_sk#3)) + +(3) CometProject +Input [4]: [ws_sold_time_sk#1, ws_ship_hdemo_sk#2, ws_web_page_sk#3, ws_sold_date_sk#4] +Arguments: [ws_sold_time_sk#1, ws_ship_hdemo_sk#2, ws_web_page_sk#3], [ws_sold_time_sk#1, ws_ship_hdemo_sk#2, ws_web_page_sk#3] + +(4) ColumnarToRow [codegen id : 4] +Input [3]: [ws_sold_time_sk#1, ws_ship_hdemo_sk#2, ws_web_page_sk#3] + +(5) Scan parquet spark_catalog.default.household_demographics +Output [2]: [hd_demo_sk#5, hd_dep_count#6] +Batched: true +Location [not included in comparison]/{warehouse_dir}/household_demographics] +PushedFilters: [IsNotNull(hd_dep_count), EqualTo(hd_dep_count,6), IsNotNull(hd_demo_sk)] +ReadSchema: struct<hd_demo_sk:int,hd_dep_count:int> + +(6) CometFilter +Input [2]: [hd_demo_sk#5, hd_dep_count#6] +Condition : ((isnotnull(hd_dep_count#6) AND (hd_dep_count#6 = 6)) AND isnotnull(hd_demo_sk#5)) + +(7) CometProject +Input [2]: [hd_demo_sk#5, hd_dep_count#6] +Arguments: [hd_demo_sk#5], [hd_demo_sk#5] + +(8) ColumnarToRow [codegen id : 1] +Input [1]: [hd_demo_sk#5] + +(9) BroadcastExchange +Input [1]: [hd_demo_sk#5] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] + +(10) BroadcastHashJoin [codegen id : 4] +Left keys [1]: [ws_ship_hdemo_sk#2] +Right keys [1]: [hd_demo_sk#5] +Join type: Inner +Join condition: None + +(11) Project [codegen id : 4] +Output [2]: [ws_sold_time_sk#1, ws_web_page_sk#3] +Input [4]: [ws_sold_time_sk#1, ws_ship_hdemo_sk#2, ws_web_page_sk#3, hd_demo_sk#5] + +(12) Scan parquet spark_catalog.default.time_dim +Output [2]: [t_time_sk#7, t_hour#8] +Batched: true +Location [not included in comparison]/{warehouse_dir}/time_dim] +PushedFilters: [IsNotNull(t_hour), GreaterThanOrEqual(t_hour,8), LessThanOrEqual(t_hour,9), IsNotNull(t_time_sk)] +ReadSchema: struct<t_time_sk:int,t_hour:int> + +(13) CometFilter +Input [2]: [t_time_sk#7, t_hour#8] +Condition : (((isnotnull(t_hour#8) AND (t_hour#8 >= 8)) AND (t_hour#8 <= 9)) AND isnotnull(t_time_sk#7)) + +(14) CometProject +Input [2]: [t_time_sk#7, t_hour#8] +Arguments: [t_time_sk#7], [t_time_sk#7] + +(15) ColumnarToRow [codegen id : 2] +Input [1]: [t_time_sk#7] + +(16) BroadcastExchange +Input [1]: [t_time_sk#7] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] + +(17) BroadcastHashJoin [codegen id : 4] +Left keys [1]: [ws_sold_time_sk#1] +Right keys [1]: [t_time_sk#7] +Join type: Inner +Join condition: None + +(18) Project [codegen id : 4] +Output [1]: [ws_web_page_sk#3] +Input [3]: [ws_sold_time_sk#1, ws_web_page_sk#3, t_time_sk#7] + +(19) Scan parquet spark_catalog.default.web_page +Output [2]: [wp_web_page_sk#9, wp_char_count#10] +Batched: true +Location [not included in comparison]/{warehouse_dir}/web_page] +PushedFilters: [IsNotNull(wp_char_count), GreaterThanOrEqual(wp_char_count,5000), LessThanOrEqual(wp_char_count,5200), IsNotNull(wp_web_page_sk)] +ReadSchema: struct<wp_web_page_sk:int,wp_char_count:int> + +(20) CometFilter +Input [2]: [wp_web_page_sk#9, wp_char_count#10] +Condition : (((isnotnull(wp_char_count#10) AND (wp_char_count#10 >= 5000)) AND (wp_char_count#10 <= 5200)) AND isnotnull(wp_web_page_sk#9)) + +(21) CometProject +Input [2]: [wp_web_page_sk#9, wp_char_count#10] +Arguments: [wp_web_page_sk#9], [wp_web_page_sk#9] + +(22) ColumnarToRow [codegen id : 3] +Input [1]: [wp_web_page_sk#9] + +(23) BroadcastExchange +Input [1]: [wp_web_page_sk#9] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] + +(24) BroadcastHashJoin [codegen id : 4] +Left keys [1]: [ws_web_page_sk#3] +Right keys [1]: [wp_web_page_sk#9] +Join type: Inner +Join condition: None + +(25) Project [codegen id : 4] +Output: [] +Input [2]: [ws_web_page_sk#3, wp_web_page_sk#9] + +(26) HashAggregate [codegen id : 4] +Input: [] +Keys: [] +Functions [1]: [partial_count(1)] +Aggregate Attributes [1]: [count#11] +Results [1]: [count#12] + +(27) Exchange +Input [1]: [count#12] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=4] + +(28) HashAggregate [codegen id : 10] +Input [1]: [count#12] +Keys: [] +Functions [1]: [count(1)] +Aggregate Attributes [1]: [count(1)#13] +Results [1]: [count(1)#13 AS amc#14] + +(29) Scan parquet spark_catalog.default.web_sales +Output [4]: [ws_sold_time_sk#15, ws_ship_hdemo_sk#16, ws_web_page_sk#17, ws_sold_date_sk#18] +Batched: true +Location [not included in comparison]/{warehouse_dir}/web_sales] +PushedFilters: [IsNotNull(ws_ship_hdemo_sk), IsNotNull(ws_sold_time_sk), IsNotNull(ws_web_page_sk)] +ReadSchema: struct<ws_sold_time_sk:int,ws_ship_hdemo_sk:int,ws_web_page_sk:int> + +(30) CometFilter +Input [4]: [ws_sold_time_sk#15, ws_ship_hdemo_sk#16, ws_web_page_sk#17, ws_sold_date_sk#18] +Condition : ((isnotnull(ws_ship_hdemo_sk#16) AND isnotnull(ws_sold_time_sk#15)) AND isnotnull(ws_web_page_sk#17)) + +(31) CometProject +Input [4]: [ws_sold_time_sk#15, ws_ship_hdemo_sk#16, ws_web_page_sk#17, ws_sold_date_sk#18] +Arguments: [ws_sold_time_sk#15, ws_ship_hdemo_sk#16, ws_web_page_sk#17], [ws_sold_time_sk#15, ws_ship_hdemo_sk#16, ws_web_page_sk#17] + +(32) ColumnarToRow [codegen id : 8] +Input [3]: [ws_sold_time_sk#15, ws_ship_hdemo_sk#16, ws_web_page_sk#17] + +(33) ReusedExchange [Reuses operator id: 9] +Output [1]: [hd_demo_sk#19] + +(34) BroadcastHashJoin [codegen id : 8] +Left keys [1]: [ws_ship_hdemo_sk#16] +Right keys [1]: [hd_demo_sk#19] +Join type: Inner +Join condition: None + +(35) Project [codegen id : 8] +Output [2]: [ws_sold_time_sk#15, ws_web_page_sk#17] +Input [4]: [ws_sold_time_sk#15, ws_ship_hdemo_sk#16, ws_web_page_sk#17, hd_demo_sk#19] + +(36) Scan parquet spark_catalog.default.time_dim +Output [2]: [t_time_sk#20, t_hour#21] +Batched: true +Location [not included in comparison]/{warehouse_dir}/time_dim] +PushedFilters: [IsNotNull(t_hour), GreaterThanOrEqual(t_hour,19), LessThanOrEqual(t_hour,20), IsNotNull(t_time_sk)] +ReadSchema: struct<t_time_sk:int,t_hour:int> + +(37) CometFilter +Input [2]: [t_time_sk#20, t_hour#21] +Condition : (((isnotnull(t_hour#21) AND (t_hour#21 >= 19)) AND (t_hour#21 <= 20)) AND isnotnull(t_time_sk#20)) + +(38) CometProject +Input [2]: [t_time_sk#20, t_hour#21] +Arguments: [t_time_sk#20], [t_time_sk#20] + +(39) ColumnarToRow [codegen id : 6] +Input [1]: [t_time_sk#20] + +(40) BroadcastExchange +Input [1]: [t_time_sk#20] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] + +(41) BroadcastHashJoin [codegen id : 8] +Left keys [1]: [ws_sold_time_sk#15] +Right keys [1]: [t_time_sk#20] +Join type: Inner +Join condition: None + +(42) Project [codegen id : 8] +Output [1]: [ws_web_page_sk#17] +Input [3]: [ws_sold_time_sk#15, ws_web_page_sk#17, t_time_sk#20] + +(43) ReusedExchange [Reuses operator id: 23] +Output [1]: [wp_web_page_sk#22] + +(44) BroadcastHashJoin [codegen id : 8] +Left keys [1]: [ws_web_page_sk#17] +Right keys [1]: [wp_web_page_sk#22] +Join type: Inner +Join condition: None + +(45) Project [codegen id : 8] +Output: [] +Input [2]: [ws_web_page_sk#17, wp_web_page_sk#22] + +(46) HashAggregate [codegen id : 8] +Input: [] +Keys: [] +Functions [1]: [partial_count(1)] +Aggregate Attributes [1]: [count#23] +Results [1]: [count#24] + +(47) Exchange +Input [1]: [count#24] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=6] + +(48) HashAggregate [codegen id : 9] +Input [1]: [count#24] +Keys: [] +Functions [1]: [count(1)] +Aggregate Attributes [1]: [count(1)#25] +Results [1]: [count(1)#25 AS pmc#26] + +(49) BroadcastExchange +Input [1]: [pmc#26] +Arguments: IdentityBroadcastMode, [plan_id=7] + +(50) BroadcastNestedLoopJoin [codegen id : 10] +Join type: Inner +Join condition: None + +(51) Project [codegen id : 10] +Output [1]: [(cast(amc#14 as decimal(15,4)) / cast(pmc#26 as decimal(15,4))) AS am_pm_ratio#27] +Input [2]: [amc#14, pmc#26] + diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q90/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q90/simplified.txt new file mode 100644 index 000000000..c4e04b06b --- /dev/null +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q90/simplified.txt @@ -0,0 +1,74 @@ +WholeStageCodegen (10) + Project [amc,pmc] + BroadcastNestedLoopJoin + HashAggregate [count] [count(1),amc,count] + InputAdapter + Exchange #1 + WholeStageCodegen (4) + HashAggregate [count,count] + Project + BroadcastHashJoin [ws_web_page_sk,wp_web_page_sk] + Project [ws_web_page_sk] + BroadcastHashJoin [ws_sold_time_sk,t_time_sk] + Project [ws_sold_time_sk,ws_web_page_sk] + BroadcastHashJoin [ws_ship_hdemo_sk,hd_demo_sk] + ColumnarToRow + InputAdapter + CometProject [ws_sold_time_sk,ws_ship_hdemo_sk,ws_web_page_sk] + CometFilter [ws_ship_hdemo_sk,ws_sold_time_sk,ws_web_page_sk] + CometScan parquet spark_catalog.default.web_sales [ws_sold_time_sk,ws_ship_hdemo_sk,ws_web_page_sk,ws_sold_date_sk] + InputAdapter + BroadcastExchange #2 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [hd_demo_sk] + CometFilter [hd_dep_count,hd_demo_sk] + CometScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_dep_count] + InputAdapter + BroadcastExchange #3 + WholeStageCodegen (2) + ColumnarToRow + InputAdapter + CometProject [t_time_sk] + CometFilter [t_hour,t_time_sk] + CometScan parquet spark_catalog.default.time_dim [t_time_sk,t_hour] + InputAdapter + BroadcastExchange #4 + WholeStageCodegen (3) + ColumnarToRow + InputAdapter + CometProject [wp_web_page_sk] + CometFilter [wp_char_count,wp_web_page_sk] + CometScan parquet spark_catalog.default.web_page [wp_web_page_sk,wp_char_count] + InputAdapter + BroadcastExchange #5 + WholeStageCodegen (9) + HashAggregate [count] [count(1),pmc,count] + InputAdapter + Exchange #6 + WholeStageCodegen (8) + HashAggregate [count,count] + Project + BroadcastHashJoin [ws_web_page_sk,wp_web_page_sk] + Project [ws_web_page_sk] + BroadcastHashJoin [ws_sold_time_sk,t_time_sk] + Project [ws_sold_time_sk,ws_web_page_sk] + BroadcastHashJoin [ws_ship_hdemo_sk,hd_demo_sk] + ColumnarToRow + InputAdapter + CometProject [ws_sold_time_sk,ws_ship_hdemo_sk,ws_web_page_sk] + CometFilter [ws_ship_hdemo_sk,ws_sold_time_sk,ws_web_page_sk] + CometScan parquet spark_catalog.default.web_sales [ws_sold_time_sk,ws_ship_hdemo_sk,ws_web_page_sk,ws_sold_date_sk] + InputAdapter + ReusedExchange [hd_demo_sk] #2 + InputAdapter + BroadcastExchange #7 + WholeStageCodegen (6) + ColumnarToRow + InputAdapter + CometProject [t_time_sk] + CometFilter [t_hour,t_time_sk] + CometScan parquet spark_catalog.default.time_dim [t_time_sk,t_hour] + InputAdapter + ReusedExchange [wp_web_page_sk] #4 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q91/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q91/explain.txt new file mode 100644 index 000000000..61f35489a --- /dev/null +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q91/explain.txt @@ -0,0 +1,281 @@ +== Physical Plan == +* Sort (43) ++- Exchange (42) + +- * HashAggregate (41) + +- Exchange (40) + +- * HashAggregate (39) + +- * Project (38) + +- * BroadcastHashJoin Inner BuildRight (37) + :- * Project (31) + : +- * BroadcastHashJoin Inner BuildRight (30) + : :- * Project (25) + : : +- * BroadcastHashJoin Inner BuildRight (24) + : : :- * Project (18) + : : : +- * BroadcastHashJoin Inner BuildRight (17) + : : : :- * Project (12) + : : : : +- * BroadcastHashJoin Inner BuildRight (11) + : : : : :- * Project (9) + : : : : : +- * BroadcastHashJoin Inner BuildRight (8) + : : : : : :- * ColumnarToRow (3) + : : : : : : +- CometFilter (2) + : : : : : : +- CometScan parquet spark_catalog.default.call_center (1) + : : : : : +- BroadcastExchange (7) + : : : : : +- * ColumnarToRow (6) + : : : : : +- CometFilter (5) + : : : : : +- CometScan parquet spark_catalog.default.catalog_returns (4) + : : : : +- ReusedExchange (10) + : : : +- BroadcastExchange (16) + : : : +- * ColumnarToRow (15) + : : : +- CometFilter (14) + : : : +- CometScan parquet spark_catalog.default.customer (13) + : : +- BroadcastExchange (23) + : : +- * ColumnarToRow (22) + : : +- CometProject (21) + : : +- CometFilter (20) + : : +- CometScan parquet spark_catalog.default.customer_address (19) + : +- BroadcastExchange (29) + : +- * ColumnarToRow (28) + : +- CometFilter (27) + : +- CometScan parquet spark_catalog.default.customer_demographics (26) + +- BroadcastExchange (36) + +- * ColumnarToRow (35) + +- CometProject (34) + +- CometFilter (33) + +- CometScan parquet spark_catalog.default.household_demographics (32) + + +(1) Scan parquet spark_catalog.default.call_center +Output [4]: [cc_call_center_sk#1, cc_call_center_id#2, cc_name#3, cc_manager#4] +Batched: true +Location [not included in comparison]/{warehouse_dir}/call_center] +PushedFilters: [IsNotNull(cc_call_center_sk)] +ReadSchema: struct<cc_call_center_sk:int,cc_call_center_id:string,cc_name:string,cc_manager:string> + +(2) CometFilter +Input [4]: [cc_call_center_sk#1, cc_call_center_id#2, cc_name#3, cc_manager#4] +Condition : isnotnull(cc_call_center_sk#1) + +(3) ColumnarToRow [codegen id : 7] +Input [4]: [cc_call_center_sk#1, cc_call_center_id#2, cc_name#3, cc_manager#4] + +(4) Scan parquet spark_catalog.default.catalog_returns +Output [4]: [cr_returning_customer_sk#5, cr_call_center_sk#6, cr_net_loss#7, cr_returned_date_sk#8] +Batched: true +Location: InMemoryFileIndex [] +PartitionFilters: [isnotnull(cr_returned_date_sk#8), dynamicpruningexpression(cr_returned_date_sk#8 IN dynamicpruning#9)] +PushedFilters: [IsNotNull(cr_call_center_sk), IsNotNull(cr_returning_customer_sk)] +ReadSchema: struct<cr_returning_customer_sk:int,cr_call_center_sk:int,cr_net_loss:decimal(7,2)> + +(5) CometFilter +Input [4]: [cr_returning_customer_sk#5, cr_call_center_sk#6, cr_net_loss#7, cr_returned_date_sk#8] +Condition : (isnotnull(cr_call_center_sk#6) AND isnotnull(cr_returning_customer_sk#5)) + +(6) ColumnarToRow [codegen id : 1] +Input [4]: [cr_returning_customer_sk#5, cr_call_center_sk#6, cr_net_loss#7, cr_returned_date_sk#8] + +(7) BroadcastExchange +Input [4]: [cr_returning_customer_sk#5, cr_call_center_sk#6, cr_net_loss#7, cr_returned_date_sk#8] +Arguments: HashedRelationBroadcastMode(List(cast(input[1, int, false] as bigint)),false), [plan_id=1] + +(8) BroadcastHashJoin [codegen id : 7] +Left keys [1]: [cc_call_center_sk#1] +Right keys [1]: [cr_call_center_sk#6] +Join type: Inner +Join condition: None + +(9) Project [codegen id : 7] +Output [6]: [cc_call_center_id#2, cc_name#3, cc_manager#4, cr_returning_customer_sk#5, cr_net_loss#7, cr_returned_date_sk#8] +Input [8]: [cc_call_center_sk#1, cc_call_center_id#2, cc_name#3, cc_manager#4, cr_returning_customer_sk#5, cr_call_center_sk#6, cr_net_loss#7, cr_returned_date_sk#8] + +(10) ReusedExchange [Reuses operator id: 48] +Output [1]: [d_date_sk#10] + +(11) BroadcastHashJoin [codegen id : 7] +Left keys [1]: [cr_returned_date_sk#8] +Right keys [1]: [d_date_sk#10] +Join type: Inner +Join condition: None + +(12) Project [codegen id : 7] +Output [5]: [cc_call_center_id#2, cc_name#3, cc_manager#4, cr_returning_customer_sk#5, cr_net_loss#7] +Input [7]: [cc_call_center_id#2, cc_name#3, cc_manager#4, cr_returning_customer_sk#5, cr_net_loss#7, cr_returned_date_sk#8, d_date_sk#10] + +(13) Scan parquet spark_catalog.default.customer +Output [4]: [c_customer_sk#11, c_current_cdemo_sk#12, c_current_hdemo_sk#13, c_current_addr_sk#14] +Batched: true +Location [not included in comparison]/{warehouse_dir}/customer] +PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_current_addr_sk), IsNotNull(c_current_cdemo_sk), IsNotNull(c_current_hdemo_sk)] +ReadSchema: struct<c_customer_sk:int,c_current_cdemo_sk:int,c_current_hdemo_sk:int,c_current_addr_sk:int> + +(14) CometFilter +Input [4]: [c_customer_sk#11, c_current_cdemo_sk#12, c_current_hdemo_sk#13, c_current_addr_sk#14] +Condition : (((isnotnull(c_customer_sk#11) AND isnotnull(c_current_addr_sk#14)) AND isnotnull(c_current_cdemo_sk#12)) AND isnotnull(c_current_hdemo_sk#13)) + +(15) ColumnarToRow [codegen id : 3] +Input [4]: [c_customer_sk#11, c_current_cdemo_sk#12, c_current_hdemo_sk#13, c_current_addr_sk#14] + +(16) BroadcastExchange +Input [4]: [c_customer_sk#11, c_current_cdemo_sk#12, c_current_hdemo_sk#13, c_current_addr_sk#14] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=2] + +(17) BroadcastHashJoin [codegen id : 7] +Left keys [1]: [cr_returning_customer_sk#5] +Right keys [1]: [c_customer_sk#11] +Join type: Inner +Join condition: None + +(18) Project [codegen id : 7] +Output [7]: [cc_call_center_id#2, cc_name#3, cc_manager#4, cr_net_loss#7, c_current_cdemo_sk#12, c_current_hdemo_sk#13, c_current_addr_sk#14] +Input [9]: [cc_call_center_id#2, cc_name#3, cc_manager#4, cr_returning_customer_sk#5, cr_net_loss#7, c_customer_sk#11, c_current_cdemo_sk#12, c_current_hdemo_sk#13, c_current_addr_sk#14] + +(19) Scan parquet spark_catalog.default.customer_address +Output [2]: [ca_address_sk#15, ca_gmt_offset#16] +Batched: true +Location [not included in comparison]/{warehouse_dir}/customer_address] +PushedFilters: [IsNotNull(ca_gmt_offset), EqualTo(ca_gmt_offset,-7.00), IsNotNull(ca_address_sk)] +ReadSchema: struct<ca_address_sk:int,ca_gmt_offset:decimal(5,2)> + +(20) CometFilter +Input [2]: [ca_address_sk#15, ca_gmt_offset#16] +Condition : ((isnotnull(ca_gmt_offset#16) AND (ca_gmt_offset#16 = -7.00)) AND isnotnull(ca_address_sk#15)) + +(21) CometProject +Input [2]: [ca_address_sk#15, ca_gmt_offset#16] +Arguments: [ca_address_sk#15], [ca_address_sk#15] + +(22) ColumnarToRow [codegen id : 4] +Input [1]: [ca_address_sk#15] + +(23) BroadcastExchange +Input [1]: [ca_address_sk#15] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] + +(24) BroadcastHashJoin [codegen id : 7] +Left keys [1]: [c_current_addr_sk#14] +Right keys [1]: [ca_address_sk#15] +Join type: Inner +Join condition: None + +(25) Project [codegen id : 7] +Output [6]: [cc_call_center_id#2, cc_name#3, cc_manager#4, cr_net_loss#7, c_current_cdemo_sk#12, c_current_hdemo_sk#13] +Input [8]: [cc_call_center_id#2, cc_name#3, cc_manager#4, cr_net_loss#7, c_current_cdemo_sk#12, c_current_hdemo_sk#13, c_current_addr_sk#14, ca_address_sk#15] + +(26) Scan parquet spark_catalog.default.customer_demographics +Output [3]: [cd_demo_sk#17, cd_marital_status#18, cd_education_status#19] +Batched: true +Location [not included in comparison]/{warehouse_dir}/customer_demographics] +PushedFilters: [Or(And(EqualTo(cd_marital_status,M),EqualTo(cd_education_status,Unknown )),And(EqualTo(cd_marital_status,W),EqualTo(cd_education_status,Advanced Degree ))), IsNotNull(cd_demo_sk)] +ReadSchema: struct<cd_demo_sk:int,cd_marital_status:string,cd_education_status:string> + +(27) CometFilter +Input [3]: [cd_demo_sk#17, cd_marital_status#18, cd_education_status#19] +Condition : ((((cd_marital_status#18 = M) AND (cd_education_status#19 = Unknown )) OR ((cd_marital_status#18 = W) AND (cd_education_status#19 = Advanced Degree ))) AND isnotnull(cd_demo_sk#17)) + +(28) ColumnarToRow [codegen id : 5] +Input [3]: [cd_demo_sk#17, cd_marital_status#18, cd_education_status#19] + +(29) BroadcastExchange +Input [3]: [cd_demo_sk#17, cd_marital_status#18, cd_education_status#19] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=4] + +(30) BroadcastHashJoin [codegen id : 7] +Left keys [1]: [c_current_cdemo_sk#12] +Right keys [1]: [cd_demo_sk#17] +Join type: Inner +Join condition: None + +(31) Project [codegen id : 7] +Output [7]: [cc_call_center_id#2, cc_name#3, cc_manager#4, cr_net_loss#7, c_current_hdemo_sk#13, cd_marital_status#18, cd_education_status#19] +Input [9]: [cc_call_center_id#2, cc_name#3, cc_manager#4, cr_net_loss#7, c_current_cdemo_sk#12, c_current_hdemo_sk#13, cd_demo_sk#17, cd_marital_status#18, cd_education_status#19] + +(32) Scan parquet spark_catalog.default.household_demographics +Output [2]: [hd_demo_sk#20, hd_buy_potential#21] +Batched: true +Location [not included in comparison]/{warehouse_dir}/household_demographics] +PushedFilters: [IsNotNull(hd_buy_potential), StringStartsWith(hd_buy_potential,Unknown), IsNotNull(hd_demo_sk)] +ReadSchema: struct<hd_demo_sk:int,hd_buy_potential:string> + +(33) CometFilter +Input [2]: [hd_demo_sk#20, hd_buy_potential#21] +Condition : ((isnotnull(hd_buy_potential#21) AND StartsWith(hd_buy_potential#21, Unknown)) AND isnotnull(hd_demo_sk#20)) + +(34) CometProject +Input [2]: [hd_demo_sk#20, hd_buy_potential#21] +Arguments: [hd_demo_sk#20], [hd_demo_sk#20] + +(35) ColumnarToRow [codegen id : 6] +Input [1]: [hd_demo_sk#20] + +(36) BroadcastExchange +Input [1]: [hd_demo_sk#20] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] + +(37) BroadcastHashJoin [codegen id : 7] +Left keys [1]: [c_current_hdemo_sk#13] +Right keys [1]: [hd_demo_sk#20] +Join type: Inner +Join condition: None + +(38) Project [codegen id : 7] +Output [6]: [cc_call_center_id#2, cc_name#3, cc_manager#4, cr_net_loss#7, cd_marital_status#18, cd_education_status#19] +Input [8]: [cc_call_center_id#2, cc_name#3, cc_manager#4, cr_net_loss#7, c_current_hdemo_sk#13, cd_marital_status#18, cd_education_status#19, hd_demo_sk#20] + +(39) HashAggregate [codegen id : 7] +Input [6]: [cc_call_center_id#2, cc_name#3, cc_manager#4, cr_net_loss#7, cd_marital_status#18, cd_education_status#19] +Keys [5]: [cc_call_center_id#2, cc_name#3, cc_manager#4, cd_marital_status#18, cd_education_status#19] +Functions [1]: [partial_sum(UnscaledValue(cr_net_loss#7))] +Aggregate Attributes [1]: [sum#22] +Results [6]: [cc_call_center_id#2, cc_name#3, cc_manager#4, cd_marital_status#18, cd_education_status#19, sum#23] + +(40) Exchange +Input [6]: [cc_call_center_id#2, cc_name#3, cc_manager#4, cd_marital_status#18, cd_education_status#19, sum#23] +Arguments: hashpartitioning(cc_call_center_id#2, cc_name#3, cc_manager#4, cd_marital_status#18, cd_education_status#19, 5), ENSURE_REQUIREMENTS, [plan_id=6] + +(41) HashAggregate [codegen id : 8] +Input [6]: [cc_call_center_id#2, cc_name#3, cc_manager#4, cd_marital_status#18, cd_education_status#19, sum#23] +Keys [5]: [cc_call_center_id#2, cc_name#3, cc_manager#4, cd_marital_status#18, cd_education_status#19] +Functions [1]: [sum(UnscaledValue(cr_net_loss#7))] +Aggregate Attributes [1]: [sum(UnscaledValue(cr_net_loss#7))#24] +Results [4]: [cc_call_center_id#2 AS Call_Center#25, cc_name#3 AS Call_Center_Name#26, cc_manager#4 AS Manager#27, MakeDecimal(sum(UnscaledValue(cr_net_loss#7))#24,17,2) AS Returns_Loss#28] + +(42) Exchange +Input [4]: [Call_Center#25, Call_Center_Name#26, Manager#27, Returns_Loss#28] +Arguments: rangepartitioning(Returns_Loss#28 DESC NULLS LAST, 5), ENSURE_REQUIREMENTS, [plan_id=7] + +(43) Sort [codegen id : 9] +Input [4]: [Call_Center#25, Call_Center_Name#26, Manager#27, Returns_Loss#28] +Arguments: [Returns_Loss#28 DESC NULLS LAST], true, 0 + +===== Subqueries ===== + +Subquery:1 Hosting operator id = 4 Hosting Expression = cr_returned_date_sk#8 IN dynamicpruning#9 +BroadcastExchange (48) ++- * ColumnarToRow (47) + +- CometProject (46) + +- CometFilter (45) + +- CometScan parquet spark_catalog.default.date_dim (44) + + +(44) Scan parquet spark_catalog.default.date_dim +Output [3]: [d_date_sk#10, d_year#29, d_moy#30] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,1998), EqualTo(d_moy,11), IsNotNull(d_date_sk)] +ReadSchema: struct<d_date_sk:int,d_year:int,d_moy:int> + +(45) CometFilter +Input [3]: [d_date_sk#10, d_year#29, d_moy#30] +Condition : ((((isnotnull(d_year#29) AND isnotnull(d_moy#30)) AND (d_year#29 = 1998)) AND (d_moy#30 = 11)) AND isnotnull(d_date_sk#10)) + +(46) CometProject +Input [3]: [d_date_sk#10, d_year#29, d_moy#30] +Arguments: [d_date_sk#10], [d_date_sk#10] + +(47) ColumnarToRow [codegen id : 1] +Input [1]: [d_date_sk#10] + +(48) BroadcastExchange +Input [1]: [d_date_sk#10] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=8] + + diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q91/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q91/simplified.txt new file mode 100644 index 000000000..e5d62e3c0 --- /dev/null +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q91/simplified.txt @@ -0,0 +1,73 @@ +WholeStageCodegen (9) + Sort [Returns_Loss] + InputAdapter + Exchange [Returns_Loss] #1 + WholeStageCodegen (8) + HashAggregate [cc_call_center_id,cc_name,cc_manager,cd_marital_status,cd_education_status,sum] [sum(UnscaledValue(cr_net_loss)),Call_Center,Call_Center_Name,Manager,Returns_Loss,sum] + InputAdapter + Exchange [cc_call_center_id,cc_name,cc_manager,cd_marital_status,cd_education_status] #2 + WholeStageCodegen (7) + HashAggregate [cc_call_center_id,cc_name,cc_manager,cd_marital_status,cd_education_status,cr_net_loss] [sum,sum] + Project [cc_call_center_id,cc_name,cc_manager,cr_net_loss,cd_marital_status,cd_education_status] + BroadcastHashJoin [c_current_hdemo_sk,hd_demo_sk] + Project [cc_call_center_id,cc_name,cc_manager,cr_net_loss,c_current_hdemo_sk,cd_marital_status,cd_education_status] + BroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] + Project [cc_call_center_id,cc_name,cc_manager,cr_net_loss,c_current_cdemo_sk,c_current_hdemo_sk] + BroadcastHashJoin [c_current_addr_sk,ca_address_sk] + Project [cc_call_center_id,cc_name,cc_manager,cr_net_loss,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk] + BroadcastHashJoin [cr_returning_customer_sk,c_customer_sk] + Project [cc_call_center_id,cc_name,cc_manager,cr_returning_customer_sk,cr_net_loss] + BroadcastHashJoin [cr_returned_date_sk,d_date_sk] + Project [cc_call_center_id,cc_name,cc_manager,cr_returning_customer_sk,cr_net_loss,cr_returned_date_sk] + BroadcastHashJoin [cc_call_center_sk,cr_call_center_sk] + ColumnarToRow + InputAdapter + CometFilter [cc_call_center_sk] + CometScan parquet spark_catalog.default.call_center [cc_call_center_sk,cc_call_center_id,cc_name,cc_manager] + InputAdapter + BroadcastExchange #3 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometFilter [cr_call_center_sk,cr_returning_customer_sk] + CometScan parquet spark_catalog.default.catalog_returns [cr_returning_customer_sk,cr_call_center_sk,cr_net_loss,cr_returned_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #4 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_year,d_moy,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + InputAdapter + ReusedExchange [d_date_sk] #4 + InputAdapter + BroadcastExchange #5 + WholeStageCodegen (3) + ColumnarToRow + InputAdapter + CometFilter [c_customer_sk,c_current_addr_sk,c_current_cdemo_sk,c_current_hdemo_sk] + CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk] + InputAdapter + BroadcastExchange #6 + WholeStageCodegen (4) + ColumnarToRow + InputAdapter + CometProject [ca_address_sk] + CometFilter [ca_gmt_offset,ca_address_sk] + CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_gmt_offset] + InputAdapter + BroadcastExchange #7 + WholeStageCodegen (5) + ColumnarToRow + InputAdapter + CometFilter [cd_marital_status,cd_education_status,cd_demo_sk] + CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_marital_status,cd_education_status] + InputAdapter + BroadcastExchange #8 + WholeStageCodegen (6) + ColumnarToRow + InputAdapter + CometProject [hd_demo_sk] + CometFilter [hd_buy_potential,hd_demo_sk] + CometScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_buy_potential] 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 new file mode 100644 index 000000000..5f1f96168 --- /dev/null +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q92/explain.txt @@ -0,0 +1,209 @@ +== Physical Plan == +* HashAggregate (29) ++- Exchange (28) + +- * HashAggregate (27) + +- * Project (26) + +- * BroadcastHashJoin Inner BuildRight (25) + :- * Project (23) + : +- * BroadcastHashJoin Inner BuildRight (22) + : :- * Project (10) + : : +- * BroadcastHashJoin Inner BuildRight (9) + : : :- * ColumnarToRow (3) + : : : +- CometFilter (2) + : : : +- CometScan parquet spark_catalog.default.web_sales (1) + : : +- BroadcastExchange (8) + : : +- * ColumnarToRow (7) + : : +- CometProject (6) + : : +- CometFilter (5) + : : +- CometScan parquet spark_catalog.default.item (4) + : +- BroadcastExchange (21) + : +- * Filter (20) + : +- * HashAggregate (19) + : +- Exchange (18) + : +- * HashAggregate (17) + : +- * Project (16) + : +- * BroadcastHashJoin Inner BuildRight (15) + : :- * ColumnarToRow (13) + : : +- CometFilter (12) + : : +- CometScan parquet spark_catalog.default.web_sales (11) + : +- ReusedExchange (14) + +- ReusedExchange (24) + + +(1) Scan parquet spark_catalog.default.web_sales +Output [3]: [ws_item_sk#1, ws_ext_discount_amt#2, ws_sold_date_sk#3] +Batched: true +Location: InMemoryFileIndex [] +PartitionFilters: [isnotnull(ws_sold_date_sk#3), dynamicpruningexpression(ws_sold_date_sk#3 IN dynamicpruning#4)] +PushedFilters: [IsNotNull(ws_item_sk), IsNotNull(ws_ext_discount_amt)] +ReadSchema: struct<ws_item_sk:int,ws_ext_discount_amt:decimal(7,2)> + +(2) CometFilter +Input [3]: [ws_item_sk#1, ws_ext_discount_amt#2, ws_sold_date_sk#3] +Condition : (isnotnull(ws_item_sk#1) AND isnotnull(ws_ext_discount_amt#2)) + +(3) ColumnarToRow [codegen id : 6] +Input [3]: [ws_item_sk#1, ws_ext_discount_amt#2, ws_sold_date_sk#3] + +(4) Scan parquet spark_catalog.default.item +Output [2]: [i_item_sk#5, i_manufact_id#6] +Batched: true +Location [not included in comparison]/{warehouse_dir}/item] +PushedFilters: [IsNotNull(i_manufact_id), EqualTo(i_manufact_id,350), IsNotNull(i_item_sk)] +ReadSchema: struct<i_item_sk:int,i_manufact_id:int> + +(5) CometFilter +Input [2]: [i_item_sk#5, i_manufact_id#6] +Condition : ((isnotnull(i_manufact_id#6) AND (i_manufact_id#6 = 350)) AND isnotnull(i_item_sk#5)) + +(6) CometProject +Input [2]: [i_item_sk#5, i_manufact_id#6] +Arguments: [i_item_sk#5], [i_item_sk#5] + +(7) ColumnarToRow [codegen id : 1] +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] +Left keys [1]: [ws_item_sk#1] +Right keys [1]: [i_item_sk#5] +Join type: Inner +Join condition: None + +(10) Project [codegen id : 6] +Output [3]: [ws_ext_discount_amt#2, ws_sold_date_sk#3, i_item_sk#5] +Input [4]: [ws_item_sk#1, ws_ext_discount_amt#2, ws_sold_date_sk#3, i_item_sk#5] + +(11) Scan parquet spark_catalog.default.web_sales +Output [3]: [ws_item_sk#7, ws_ext_discount_amt#8, ws_sold_date_sk#9] +Batched: true +Location: InMemoryFileIndex [] +PartitionFilters: [isnotnull(ws_sold_date_sk#9), dynamicpruningexpression(ws_sold_date_sk#9 IN dynamicpruning#10)] +PushedFilters: [IsNotNull(ws_item_sk)] +ReadSchema: struct<ws_item_sk:int,ws_ext_discount_amt:decimal(7,2)> + +(12) CometFilter +Input [3]: [ws_item_sk#7, ws_ext_discount_amt#8, ws_sold_date_sk#9] +Condition : isnotnull(ws_item_sk#7) + +(13) ColumnarToRow [codegen id : 3] +Input [3]: [ws_item_sk#7, ws_ext_discount_amt#8, ws_sold_date_sk#9] + +(14) ReusedExchange [Reuses operator id: 34] +Output [1]: [d_date_sk#11] + +(15) BroadcastHashJoin [codegen id : 3] +Left keys [1]: [ws_sold_date_sk#9] +Right keys [1]: [d_date_sk#11] +Join type: Inner +Join condition: None + +(16) Project [codegen id : 3] +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#11] + +(17) HashAggregate [codegen id : 3] +Input [2]: [ws_item_sk#7, ws_ext_discount_amt#8] +Keys [1]: [ws_item_sk#7] +Functions [1]: [partial_avg(UnscaledValue(ws_ext_discount_amt#8))] +Aggregate Attributes [2]: [sum#12, count#13] +Results [3]: [ws_item_sk#7, sum#14, count#15] + +(18) Exchange +Input [3]: [ws_item_sk#7, sum#14, count#15] +Arguments: hashpartitioning(ws_item_sk#7, 5), ENSURE_REQUIREMENTS, [plan_id=2] + +(19) HashAggregate [codegen id : 4] +Input [3]: [ws_item_sk#7, sum#14, count#15] +Keys [1]: [ws_item_sk#7] +Functions [1]: [avg(UnscaledValue(ws_ext_discount_amt#8))] +Aggregate Attributes [1]: [avg(UnscaledValue(ws_ext_discount_amt#8))#16] +Results [2]: [(1.3 * cast((avg(UnscaledValue(ws_ext_discount_amt#8))#16 / 100.0) as decimal(11,6))) AS (1.3 * avg(ws_ext_discount_amt))#17, ws_item_sk#7] + +(20) Filter [codegen id : 4] +Input [2]: [(1.3 * avg(ws_ext_discount_amt))#17, ws_item_sk#7] +Condition : isnotnull((1.3 * avg(ws_ext_discount_amt))#17) + +(21) BroadcastExchange +Input [2]: [(1.3 * avg(ws_ext_discount_amt))#17, ws_item_sk#7] +Arguments: HashedRelationBroadcastMode(List(cast(input[1, int, true] as bigint)),false), [plan_id=3] + +(22) BroadcastHashJoin [codegen id : 6] +Left keys [1]: [i_item_sk#5] +Right keys [1]: [ws_item_sk#7] +Join type: Inner +Join condition: (cast(ws_ext_discount_amt#2 as decimal(14,7)) > (1.3 * avg(ws_ext_discount_amt))#17) + +(23) Project [codegen id : 6] +Output [2]: [ws_ext_discount_amt#2, ws_sold_date_sk#3] +Input [5]: [ws_ext_discount_amt#2, ws_sold_date_sk#3, i_item_sk#5, (1.3 * avg(ws_ext_discount_amt))#17, ws_item_sk#7] + +(24) ReusedExchange [Reuses operator id: 34] +Output [1]: [d_date_sk#18] + +(25) BroadcastHashJoin [codegen id : 6] +Left keys [1]: [ws_sold_date_sk#3] +Right keys [1]: [d_date_sk#18] +Join type: Inner +Join condition: None + +(26) Project [codegen id : 6] +Output [1]: [ws_ext_discount_amt#2] +Input [3]: [ws_ext_discount_amt#2, ws_sold_date_sk#3, d_date_sk#18] + +(27) HashAggregate [codegen id : 6] +Input [1]: [ws_ext_discount_amt#2] +Keys: [] +Functions [1]: [partial_sum(UnscaledValue(ws_ext_discount_amt#2))] +Aggregate Attributes [1]: [sum#19] +Results [1]: [sum#20] + +(28) Exchange +Input [1]: [sum#20] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=4] + +(29) HashAggregate [codegen id : 7] +Input [1]: [sum#20] +Keys: [] +Functions [1]: [sum(UnscaledValue(ws_ext_discount_amt#2))] +Aggregate Attributes [1]: [sum(UnscaledValue(ws_ext_discount_amt#2))#21] +Results [1]: [MakeDecimal(sum(UnscaledValue(ws_ext_discount_amt#2))#21,17,2) AS Excess Discount Amount #22] + +===== Subqueries ===== + +Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#3 IN dynamicpruning#4 +BroadcastExchange (34) ++- * ColumnarToRow (33) + +- CometProject (32) + +- CometFilter (31) + +- CometScan parquet spark_catalog.default.date_dim (30) + + +(30) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#18, d_date#23] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,2000-01-27), LessThanOrEqual(d_date,2000-04-26), IsNotNull(d_date_sk)] +ReadSchema: struct<d_date_sk:int,d_date:date> + +(31) CometFilter +Input [2]: [d_date_sk#18, d_date#23] +Condition : (((isnotnull(d_date#23) AND (d_date#23 >= 2000-01-27)) AND (d_date#23 <= 2000-04-26)) AND isnotnull(d_date_sk#18)) + +(32) CometProject +Input [2]: [d_date_sk#18, d_date#23] +Arguments: [d_date_sk#18], [d_date_sk#18] + +(33) ColumnarToRow [codegen id : 1] +Input [1]: [d_date_sk#18] + +(34) BroadcastExchange +Input [1]: [d_date_sk#18] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] + +Subquery:2 Hosting operator id = 11 Hosting Expression = ws_sold_date_sk#9 IN dynamicpruning#4 + + diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q92/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q92/simplified.txt new file mode 100644 index 000000000..a5e724c1f --- /dev/null +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q92/simplified.txt @@ -0,0 +1,52 @@ +WholeStageCodegen (7) + HashAggregate [sum] [sum(UnscaledValue(ws_ext_discount_amt)),Excess Discount Amount ,sum] + InputAdapter + Exchange #1 + WholeStageCodegen (6) + HashAggregate [ws_ext_discount_amt] [sum,sum] + Project [ws_ext_discount_amt] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + Project [ws_ext_discount_amt,ws_sold_date_sk] + BroadcastHashJoin [i_item_sk,ws_item_sk,ws_ext_discount_amt,(1.3 * avg(ws_ext_discount_amt))] + Project [ws_ext_discount_amt,ws_sold_date_sk,i_item_sk] + BroadcastHashJoin [ws_item_sk,i_item_sk] + ColumnarToRow + InputAdapter + CometFilter [ws_item_sk,ws_ext_discount_amt] + CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_ext_discount_amt,ws_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #2 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] + InputAdapter + BroadcastExchange #3 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [i_item_sk] + CometFilter [i_manufact_id,i_item_sk] + CometScan parquet spark_catalog.default.item [i_item_sk,i_manufact_id] + InputAdapter + BroadcastExchange #4 + WholeStageCodegen (4) + Filter [(1.3 * avg(ws_ext_discount_amt))] + HashAggregate [ws_item_sk,sum,count] [avg(UnscaledValue(ws_ext_discount_amt)),(1.3 * avg(ws_ext_discount_amt)),sum,count] + InputAdapter + Exchange [ws_item_sk] #5 + WholeStageCodegen (3) + HashAggregate [ws_item_sk,ws_ext_discount_amt] [sum,count,sum,count] + Project [ws_item_sk,ws_ext_discount_amt] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + ColumnarToRow + InputAdapter + CometFilter [ws_item_sk] + CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_ext_discount_amt,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + InputAdapter + ReusedExchange [d_date_sk] #2 + InputAdapter + ReusedExchange [d_date_sk] #2 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q93/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q93/explain.txt new file mode 100644 index 000000000..00ed822f2 --- /dev/null +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q93/explain.txt @@ -0,0 +1,138 @@ +== Physical Plan == +TakeOrderedAndProject (24) ++- * HashAggregate (23) + +- Exchange (22) + +- * HashAggregate (21) + +- * Project (20) + +- * BroadcastHashJoin Inner BuildRight (19) + :- * Project (13) + : +- * SortMergeJoin Inner (12) + : :- * Sort (5) + : : +- Exchange (4) + : : +- * ColumnarToRow (3) + : : +- CometProject (2) + : : +- CometScan parquet spark_catalog.default.store_sales (1) + : +- * Sort (11) + : +- Exchange (10) + : +- * ColumnarToRow (9) + : +- CometProject (8) + : +- CometFilter (7) + : +- CometScan parquet spark_catalog.default.store_returns (6) + +- BroadcastExchange (18) + +- * ColumnarToRow (17) + +- CometProject (16) + +- CometFilter (15) + +- CometScan parquet spark_catalog.default.reason (14) + + +(1) Scan parquet spark_catalog.default.store_sales +Output [6]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#3, ss_quantity#4, ss_sales_price#5, ss_sold_date_sk#6] +Batched: true +Location [not included in comparison]/{warehouse_dir}/store_sales] +ReadSchema: struct<ss_item_sk:int,ss_customer_sk:int,ss_ticket_number:int,ss_quantity:int,ss_sales_price:decimal(7,2)> + +(2) CometProject +Input [6]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#3, ss_quantity#4, ss_sales_price#5, ss_sold_date_sk#6] +Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#3, ss_quantity#4, ss_sales_price#5], [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#3, ss_quantity#4, ss_sales_price#5] + +(3) ColumnarToRow [codegen id : 1] +Input [5]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#3, ss_quantity#4, ss_sales_price#5] + +(4) Exchange +Input [5]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#3, ss_quantity#4, ss_sales_price#5] +Arguments: hashpartitioning(ss_item_sk#1, ss_ticket_number#3, 5), ENSURE_REQUIREMENTS, [plan_id=1] + +(5) Sort [codegen id : 2] +Input [5]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#3, ss_quantity#4, ss_sales_price#5] +Arguments: [ss_item_sk#1 ASC NULLS FIRST, ss_ticket_number#3 ASC NULLS FIRST], false, 0 + +(6) Scan parquet spark_catalog.default.store_returns +Output [5]: [sr_item_sk#7, sr_reason_sk#8, sr_ticket_number#9, sr_return_quantity#10, sr_returned_date_sk#11] +Batched: true +Location [not included in comparison]/{warehouse_dir}/store_returns] +PushedFilters: [IsNotNull(sr_item_sk), IsNotNull(sr_ticket_number), IsNotNull(sr_reason_sk)] +ReadSchema: struct<sr_item_sk:int,sr_reason_sk:int,sr_ticket_number:int,sr_return_quantity:int> + +(7) CometFilter +Input [5]: [sr_item_sk#7, sr_reason_sk#8, sr_ticket_number#9, sr_return_quantity#10, sr_returned_date_sk#11] +Condition : ((isnotnull(sr_item_sk#7) AND isnotnull(sr_ticket_number#9)) AND isnotnull(sr_reason_sk#8)) + +(8) CometProject +Input [5]: [sr_item_sk#7, sr_reason_sk#8, sr_ticket_number#9, sr_return_quantity#10, sr_returned_date_sk#11] +Arguments: [sr_item_sk#7, sr_reason_sk#8, sr_ticket_number#9, sr_return_quantity#10], [sr_item_sk#7, sr_reason_sk#8, sr_ticket_number#9, sr_return_quantity#10] + +(9) ColumnarToRow [codegen id : 3] +Input [4]: [sr_item_sk#7, sr_reason_sk#8, sr_ticket_number#9, sr_return_quantity#10] + +(10) Exchange +Input [4]: [sr_item_sk#7, sr_reason_sk#8, sr_ticket_number#9, sr_return_quantity#10] +Arguments: hashpartitioning(sr_item_sk#7, sr_ticket_number#9, 5), ENSURE_REQUIREMENTS, [plan_id=2] + +(11) Sort [codegen id : 4] +Input [4]: [sr_item_sk#7, sr_reason_sk#8, sr_ticket_number#9, sr_return_quantity#10] +Arguments: [sr_item_sk#7 ASC NULLS FIRST, sr_ticket_number#9 ASC NULLS FIRST], false, 0 + +(12) SortMergeJoin [codegen id : 6] +Left keys [2]: [ss_item_sk#1, ss_ticket_number#3] +Right keys [2]: [sr_item_sk#7, sr_ticket_number#9] +Join type: Inner +Join condition: None + +(13) Project [codegen id : 6] +Output [5]: [ss_customer_sk#2, ss_quantity#4, ss_sales_price#5, sr_reason_sk#8, sr_return_quantity#10] +Input [9]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#3, ss_quantity#4, ss_sales_price#5, sr_item_sk#7, sr_reason_sk#8, sr_ticket_number#9, sr_return_quantity#10] + +(14) Scan parquet spark_catalog.default.reason +Output [2]: [r_reason_sk#12, r_reason_desc#13] +Batched: true +Location [not included in comparison]/{warehouse_dir}/reason] +PushedFilters: [IsNotNull(r_reason_desc), EqualTo(r_reason_desc,reason 28 ), IsNotNull(r_reason_sk)] +ReadSchema: struct<r_reason_sk:int,r_reason_desc:string> + +(15) CometFilter +Input [2]: [r_reason_sk#12, r_reason_desc#13] +Condition : ((isnotnull(r_reason_desc#13) AND (r_reason_desc#13 = reason 28 )) AND isnotnull(r_reason_sk#12)) + +(16) CometProject +Input [2]: [r_reason_sk#12, r_reason_desc#13] +Arguments: [r_reason_sk#12], [r_reason_sk#12] + +(17) ColumnarToRow [codegen id : 5] +Input [1]: [r_reason_sk#12] + +(18) BroadcastExchange +Input [1]: [r_reason_sk#12] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] + +(19) BroadcastHashJoin [codegen id : 6] +Left keys [1]: [sr_reason_sk#8] +Right keys [1]: [r_reason_sk#12] +Join type: Inner +Join condition: None + +(20) Project [codegen id : 6] +Output [2]: [ss_customer_sk#2, CASE WHEN isnotnull(sr_return_quantity#10) THEN (cast((ss_quantity#4 - sr_return_quantity#10) as decimal(10,0)) * ss_sales_price#5) ELSE (cast(ss_quantity#4 as decimal(10,0)) * ss_sales_price#5) END AS act_sales#14] +Input [6]: [ss_customer_sk#2, ss_quantity#4, ss_sales_price#5, sr_reason_sk#8, sr_return_quantity#10, r_reason_sk#12] + +(21) HashAggregate [codegen id : 6] +Input [2]: [ss_customer_sk#2, act_sales#14] +Keys [1]: [ss_customer_sk#2] +Functions [1]: [partial_sum(act_sales#14)] +Aggregate Attributes [2]: [sum#15, isEmpty#16] +Results [3]: [ss_customer_sk#2, sum#17, isEmpty#18] + +(22) Exchange +Input [3]: [ss_customer_sk#2, sum#17, isEmpty#18] +Arguments: hashpartitioning(ss_customer_sk#2, 5), ENSURE_REQUIREMENTS, [plan_id=4] + +(23) HashAggregate [codegen id : 7] +Input [3]: [ss_customer_sk#2, sum#17, isEmpty#18] +Keys [1]: [ss_customer_sk#2] +Functions [1]: [sum(act_sales#14)] +Aggregate Attributes [1]: [sum(act_sales#14)#19] +Results [2]: [ss_customer_sk#2, sum(act_sales#14)#19 AS sumsales#20] + +(24) TakeOrderedAndProject +Input [2]: [ss_customer_sk#2, sumsales#20] +Arguments: 100, [sumsales#20 ASC NULLS FIRST, ss_customer_sk#2 ASC NULLS FIRST], [ss_customer_sk#2, sumsales#20] + diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q93/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q93/simplified.txt new file mode 100644 index 000000000..3ec7ac7b6 --- /dev/null +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q93/simplified.txt @@ -0,0 +1,40 @@ +TakeOrderedAndProject [sumsales,ss_customer_sk] + WholeStageCodegen (7) + HashAggregate [ss_customer_sk,sum,isEmpty] [sum(act_sales),sumsales,sum,isEmpty] + InputAdapter + Exchange [ss_customer_sk] #1 + WholeStageCodegen (6) + HashAggregate [ss_customer_sk,act_sales] [sum,isEmpty,sum,isEmpty] + Project [ss_customer_sk,sr_return_quantity,ss_quantity,ss_sales_price] + BroadcastHashJoin [sr_reason_sk,r_reason_sk] + Project [ss_customer_sk,ss_quantity,ss_sales_price,sr_reason_sk,sr_return_quantity] + SortMergeJoin [ss_item_sk,ss_ticket_number,sr_item_sk,sr_ticket_number] + InputAdapter + WholeStageCodegen (2) + Sort [ss_item_sk,ss_ticket_number] + InputAdapter + Exchange [ss_item_sk,ss_ticket_number] #2 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_quantity,ss_sales_price] + CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_quantity,ss_sales_price,ss_sold_date_sk] + InputAdapter + WholeStageCodegen (4) + Sort [sr_item_sk,sr_ticket_number] + InputAdapter + Exchange [sr_item_sk,sr_ticket_number] #3 + WholeStageCodegen (3) + ColumnarToRow + InputAdapter + CometProject [sr_item_sk,sr_reason_sk,sr_ticket_number,sr_return_quantity] + CometFilter [sr_item_sk,sr_ticket_number,sr_reason_sk] + CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_reason_sk,sr_ticket_number,sr_return_quantity,sr_returned_date_sk] + InputAdapter + BroadcastExchange #4 + WholeStageCodegen (5) + ColumnarToRow + InputAdapter + CometProject [r_reason_sk] + CometFilter [r_reason_desc,r_reason_sk] + CometScan parquet spark_catalog.default.reason [r_reason_sk,r_reason_desc] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q94/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q94/explain.txt new file mode 100644 index 000000000..d71f96e15 --- /dev/null +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q94/explain.txt @@ -0,0 +1,260 @@ +== Physical Plan == +* HashAggregate (45) ++- Exchange (44) + +- * HashAggregate (43) + +- * HashAggregate (42) + +- * HashAggregate (41) + +- * Project (40) + +- * BroadcastHashJoin Inner BuildRight (39) + :- * Project (33) + : +- * BroadcastHashJoin Inner BuildRight (32) + : :- * Project (26) + : : +- * BroadcastHashJoin Inner BuildRight (25) + : : :- * SortMergeJoin LeftAnti (19) + : : : :- * Project (13) + : : : : +- * SortMergeJoin LeftSemi (12) + : : : : :- * Sort (6) + : : : : : +- Exchange (5) + : : : : : +- * ColumnarToRow (4) + : : : : : +- CometProject (3) + : : : : : +- CometFilter (2) + : : : : : +- CometScan parquet spark_catalog.default.web_sales (1) + : : : : +- * Sort (11) + : : : : +- Exchange (10) + : : : : +- * ColumnarToRow (9) + : : : : +- CometProject (8) + : : : : +- CometScan parquet spark_catalog.default.web_sales (7) + : : : +- * Sort (18) + : : : +- Exchange (17) + : : : +- * ColumnarToRow (16) + : : : +- CometProject (15) + : : : +- CometScan parquet spark_catalog.default.web_returns (14) + : : +- BroadcastExchange (24) + : : +- * ColumnarToRow (23) + : : +- CometProject (22) + : : +- CometFilter (21) + : : +- CometScan parquet spark_catalog.default.date_dim (20) + : +- BroadcastExchange (31) + : +- * ColumnarToRow (30) + : +- CometProject (29) + : +- CometFilter (28) + : +- CometScan parquet spark_catalog.default.customer_address (27) + +- BroadcastExchange (38) + +- * ColumnarToRow (37) + +- CometProject (36) + +- CometFilter (35) + +- CometScan parquet spark_catalog.default.web_site (34) + + +(1) Scan parquet spark_catalog.default.web_sales +Output [8]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_warehouse_sk#4, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7, ws_sold_date_sk#8] +Batched: true +Location [not included in comparison]/{warehouse_dir}/web_sales] +PushedFilters: [IsNotNull(ws_ship_date_sk), IsNotNull(ws_ship_addr_sk), IsNotNull(ws_web_site_sk)] +ReadSchema: struct<ws_ship_date_sk:int,ws_ship_addr_sk:int,ws_web_site_sk:int,ws_warehouse_sk:int,ws_order_number:int,ws_ext_ship_cost:decimal(7,2),ws_net_profit:decimal(7,2)> + +(2) CometFilter +Input [8]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_warehouse_sk#4, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7, ws_sold_date_sk#8] +Condition : ((isnotnull(ws_ship_date_sk#1) AND isnotnull(ws_ship_addr_sk#2)) AND isnotnull(ws_web_site_sk#3)) + +(3) CometProject +Input [8]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_warehouse_sk#4, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7, ws_sold_date_sk#8] +Arguments: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_warehouse_sk#4, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7], [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_warehouse_sk#4, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7] + +(4) ColumnarToRow [codegen id : 1] +Input [7]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_warehouse_sk#4, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7] + +(5) Exchange +Input [7]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_warehouse_sk#4, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7] +Arguments: hashpartitioning(ws_order_number#5, 5), ENSURE_REQUIREMENTS, [plan_id=1] + +(6) Sort [codegen id : 2] +Input [7]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_warehouse_sk#4, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7] +Arguments: [ws_order_number#5 ASC NULLS FIRST], false, 0 + +(7) Scan parquet spark_catalog.default.web_sales +Output [3]: [ws_warehouse_sk#9, ws_order_number#10, ws_sold_date_sk#11] +Batched: true +Location [not included in comparison]/{warehouse_dir}/web_sales] +ReadSchema: struct<ws_warehouse_sk:int,ws_order_number:int> + +(8) CometProject +Input [3]: [ws_warehouse_sk#9, ws_order_number#10, ws_sold_date_sk#11] +Arguments: [ws_warehouse_sk#9, ws_order_number#10], [ws_warehouse_sk#9, ws_order_number#10] + +(9) ColumnarToRow [codegen id : 3] +Input [2]: [ws_warehouse_sk#9, ws_order_number#10] + +(10) Exchange +Input [2]: [ws_warehouse_sk#9, ws_order_number#10] +Arguments: hashpartitioning(ws_order_number#10, 5), ENSURE_REQUIREMENTS, [plan_id=2] + +(11) Sort [codegen id : 4] +Input [2]: [ws_warehouse_sk#9, ws_order_number#10] +Arguments: [ws_order_number#10 ASC NULLS FIRST], false, 0 + +(12) SortMergeJoin [codegen id : 5] +Left keys [1]: [ws_order_number#5] +Right keys [1]: [ws_order_number#10] +Join type: LeftSemi +Join condition: NOT (ws_warehouse_sk#4 = ws_warehouse_sk#9) + +(13) Project [codegen id : 5] +Output [6]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7] +Input [7]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_warehouse_sk#4, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7] + +(14) Scan parquet spark_catalog.default.web_returns +Output [2]: [wr_order_number#12, wr_returned_date_sk#13] +Batched: true +Location [not included in comparison]/{warehouse_dir}/web_returns] +ReadSchema: struct<wr_order_number:int> + +(15) CometProject +Input [2]: [wr_order_number#12, wr_returned_date_sk#13] +Arguments: [wr_order_number#12], [wr_order_number#12] + +(16) ColumnarToRow [codegen id : 6] +Input [1]: [wr_order_number#12] + +(17) Exchange +Input [1]: [wr_order_number#12] +Arguments: hashpartitioning(wr_order_number#12, 5), ENSURE_REQUIREMENTS, [plan_id=3] + +(18) Sort [codegen id : 7] +Input [1]: [wr_order_number#12] +Arguments: [wr_order_number#12 ASC NULLS FIRST], false, 0 + +(19) SortMergeJoin [codegen id : 11] +Left keys [1]: [ws_order_number#5] +Right keys [1]: [wr_order_number#12] +Join type: LeftAnti +Join condition: None + +(20) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#14, d_date#15] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,1999-02-01), LessThanOrEqual(d_date,1999-04-02), IsNotNull(d_date_sk)] +ReadSchema: struct<d_date_sk:int,d_date:date> + +(21) CometFilter +Input [2]: [d_date_sk#14, d_date#15] +Condition : (((isnotnull(d_date#15) AND (d_date#15 >= 1999-02-01)) AND (d_date#15 <= 1999-04-02)) AND isnotnull(d_date_sk#14)) + +(22) CometProject +Input [2]: [d_date_sk#14, d_date#15] +Arguments: [d_date_sk#14], [d_date_sk#14] + +(23) ColumnarToRow [codegen id : 8] +Input [1]: [d_date_sk#14] + +(24) BroadcastExchange +Input [1]: [d_date_sk#14] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] + +(25) BroadcastHashJoin [codegen id : 11] +Left keys [1]: [ws_ship_date_sk#1] +Right keys [1]: [d_date_sk#14] +Join type: Inner +Join condition: None + +(26) Project [codegen id : 11] +Output [5]: [ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7] +Input [7]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7, d_date_sk#14] + +(27) Scan parquet spark_catalog.default.customer_address +Output [2]: [ca_address_sk#16, ca_state#17] +Batched: true +Location [not included in comparison]/{warehouse_dir}/customer_address] +PushedFilters: [IsNotNull(ca_state), EqualTo(ca_state,IL), IsNotNull(ca_address_sk)] +ReadSchema: struct<ca_address_sk:int,ca_state:string> + +(28) CometFilter +Input [2]: [ca_address_sk#16, ca_state#17] +Condition : ((isnotnull(ca_state#17) AND (ca_state#17 = IL)) AND isnotnull(ca_address_sk#16)) + +(29) CometProject +Input [2]: [ca_address_sk#16, ca_state#17] +Arguments: [ca_address_sk#16], [ca_address_sk#16] + +(30) ColumnarToRow [codegen id : 9] +Input [1]: [ca_address_sk#16] + +(31) BroadcastExchange +Input [1]: [ca_address_sk#16] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] + +(32) BroadcastHashJoin [codegen id : 11] +Left keys [1]: [ws_ship_addr_sk#2] +Right keys [1]: [ca_address_sk#16] +Join type: Inner +Join condition: None + +(33) Project [codegen id : 11] +Output [4]: [ws_web_site_sk#3, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7] +Input [6]: [ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7, ca_address_sk#16] + +(34) Scan parquet spark_catalog.default.web_site +Output [2]: [web_site_sk#18, web_company_name#19] +Batched: true +Location [not included in comparison]/{warehouse_dir}/web_site] +PushedFilters: [IsNotNull(web_company_name), EqualTo(web_company_name,pri ), IsNotNull(web_site_sk)] +ReadSchema: struct<web_site_sk:int,web_company_name:string> + +(35) CometFilter +Input [2]: [web_site_sk#18, web_company_name#19] +Condition : ((isnotnull(web_company_name#19) AND (web_company_name#19 = pri )) AND isnotnull(web_site_sk#18)) + +(36) CometProject +Input [2]: [web_site_sk#18, web_company_name#19] +Arguments: [web_site_sk#18], [web_site_sk#18] + +(37) ColumnarToRow [codegen id : 10] +Input [1]: [web_site_sk#18] + +(38) BroadcastExchange +Input [1]: [web_site_sk#18] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=6] + +(39) BroadcastHashJoin [codegen id : 11] +Left keys [1]: [ws_web_site_sk#3] +Right keys [1]: [web_site_sk#18] +Join type: Inner +Join condition: None + +(40) Project [codegen id : 11] +Output [3]: [ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7] +Input [5]: [ws_web_site_sk#3, ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7, web_site_sk#18] + +(41) HashAggregate [codegen id : 11] +Input [3]: [ws_order_number#5, ws_ext_ship_cost#6, ws_net_profit#7] +Keys [1]: [ws_order_number#5] +Functions [2]: [partial_sum(UnscaledValue(ws_ext_ship_cost#6)), partial_sum(UnscaledValue(ws_net_profit#7))] +Aggregate Attributes [2]: [sum(UnscaledValue(ws_ext_ship_cost#6))#20, sum(UnscaledValue(ws_net_profit#7))#21] +Results [3]: [ws_order_number#5, sum#22, sum#23] + +(42) HashAggregate [codegen id : 11] +Input [3]: [ws_order_number#5, sum#22, sum#23] +Keys [1]: [ws_order_number#5] +Functions [2]: [merge_sum(UnscaledValue(ws_ext_ship_cost#6)), merge_sum(UnscaledValue(ws_net_profit#7))] +Aggregate Attributes [2]: [sum(UnscaledValue(ws_ext_ship_cost#6))#20, sum(UnscaledValue(ws_net_profit#7))#21] +Results [3]: [ws_order_number#5, sum#22, sum#23] + +(43) HashAggregate [codegen id : 11] +Input [3]: [ws_order_number#5, sum#22, sum#23] +Keys: [] +Functions [3]: [merge_sum(UnscaledValue(ws_ext_ship_cost#6)), merge_sum(UnscaledValue(ws_net_profit#7)), partial_count(distinct ws_order_number#5)] +Aggregate Attributes [3]: [sum(UnscaledValue(ws_ext_ship_cost#6))#20, sum(UnscaledValue(ws_net_profit#7))#21, count(ws_order_number#5)#24] +Results [3]: [sum#22, sum#23, count#25] + +(44) Exchange +Input [3]: [sum#22, sum#23, count#25] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=7] + +(45) HashAggregate [codegen id : 12] +Input [3]: [sum#22, sum#23, count#25] +Keys: [] +Functions [3]: [sum(UnscaledValue(ws_ext_ship_cost#6)), sum(UnscaledValue(ws_net_profit#7)), count(distinct ws_order_number#5)] +Aggregate Attributes [3]: [sum(UnscaledValue(ws_ext_ship_cost#6))#20, sum(UnscaledValue(ws_net_profit#7))#21, count(ws_order_number#5)#24] +Results [3]: [count(ws_order_number#5)#24 AS order count #26, MakeDecimal(sum(UnscaledValue(ws_ext_ship_cost#6))#20,17,2) AS total shipping cost #27, MakeDecimal(sum(UnscaledValue(ws_net_profit#7))#21,17,2) AS total net profit #28] + diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q94/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q94/simplified.txt new file mode 100644 index 000000000..34ddde768 --- /dev/null +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q94/simplified.txt @@ -0,0 +1,74 @@ +WholeStageCodegen (12) + HashAggregate [sum,sum,count] [sum(UnscaledValue(ws_ext_ship_cost)),sum(UnscaledValue(ws_net_profit)),count(ws_order_number),order count ,total shipping cost ,total net profit ,sum,sum,count] + InputAdapter + Exchange #1 + WholeStageCodegen (11) + HashAggregate [ws_order_number] [sum(UnscaledValue(ws_ext_ship_cost)),sum(UnscaledValue(ws_net_profit)),count(ws_order_number),sum,sum,count,sum,sum,count] + HashAggregate [ws_order_number] [sum(UnscaledValue(ws_ext_ship_cost)),sum(UnscaledValue(ws_net_profit)),sum,sum,sum,sum] + HashAggregate [ws_order_number,ws_ext_ship_cost,ws_net_profit] [sum(UnscaledValue(ws_ext_ship_cost)),sum(UnscaledValue(ws_net_profit)),sum,sum,sum,sum] + Project [ws_order_number,ws_ext_ship_cost,ws_net_profit] + BroadcastHashJoin [ws_web_site_sk,web_site_sk] + Project [ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit] + BroadcastHashJoin [ws_ship_addr_sk,ca_address_sk] + Project [ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit] + BroadcastHashJoin [ws_ship_date_sk,d_date_sk] + SortMergeJoin [ws_order_number,wr_order_number] + InputAdapter + WholeStageCodegen (5) + Project [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit] + SortMergeJoin [ws_order_number,ws_order_number,ws_warehouse_sk,ws_warehouse_sk] + InputAdapter + WholeStageCodegen (2) + Sort [ws_order_number] + InputAdapter + Exchange [ws_order_number] #2 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + 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] + CometScan parquet spark_catalog.default.web_sales [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_warehouse_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit,ws_sold_date_sk] + InputAdapter + WholeStageCodegen (4) + Sort [ws_order_number] + InputAdapter + Exchange [ws_order_number] #3 + WholeStageCodegen (3) + ColumnarToRow + InputAdapter + CometProject [ws_warehouse_sk,ws_order_number] + CometScan parquet spark_catalog.default.web_sales [ws_warehouse_sk,ws_order_number,ws_sold_date_sk] + InputAdapter + WholeStageCodegen (7) + Sort [wr_order_number] + InputAdapter + Exchange [wr_order_number] #4 + WholeStageCodegen (6) + ColumnarToRow + InputAdapter + CometProject [wr_order_number] + CometScan parquet spark_catalog.default.web_returns [wr_order_number,wr_returned_date_sk] + InputAdapter + BroadcastExchange #5 + WholeStageCodegen (8) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] + InputAdapter + BroadcastExchange #6 + WholeStageCodegen (9) + ColumnarToRow + InputAdapter + CometProject [ca_address_sk] + CometFilter [ca_state,ca_address_sk] + CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] + InputAdapter + BroadcastExchange #7 + WholeStageCodegen (10) + ColumnarToRow + InputAdapter + CometProject [web_site_sk] + CometFilter [web_company_name,web_site_sk] + CometScan parquet spark_catalog.default.web_site [web_site_sk,web_company_name] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q95/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q95/explain.txt new file mode 100644 index 000000000..c8cdce055 --- /dev/null +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q95/explain.txt @@ -0,0 +1,330 @@ +== Physical Plan == +* HashAggregate (58) ++- Exchange (57) + +- * HashAggregate (56) + +- * HashAggregate (55) + +- * HashAggregate (54) + +- * Project (53) + +- * BroadcastHashJoin Inner BuildRight (52) + :- * Project (46) + : +- * BroadcastHashJoin Inner BuildRight (45) + : :- * Project (39) + : : +- * BroadcastHashJoin Inner BuildRight (38) + : : :- * SortMergeJoin LeftSemi (32) + : : : :- * SortMergeJoin LeftSemi (17) + : : : : :- * Sort (6) + : : : : : +- Exchange (5) + : : : : : +- * ColumnarToRow (4) + : : : : : +- CometProject (3) + : : : : : +- CometFilter (2) + : : : : : +- CometScan parquet spark_catalog.default.web_sales (1) + : : : : +- * Project (16) + : : : : +- * SortMergeJoin Inner (15) + : : : : :- * Sort (12) + : : : : : +- Exchange (11) + : : : : : +- * ColumnarToRow (10) + : : : : : +- CometProject (9) + : : : : : +- CometFilter (8) + : : : : : +- CometScan parquet spark_catalog.default.web_sales (7) + : : : : +- * Sort (14) + : : : : +- ReusedExchange (13) + : : : +- * Project (31) + : : : +- * SortMergeJoin Inner (30) + : : : :- * Sort (23) + : : : : +- Exchange (22) + : : : : +- * ColumnarToRow (21) + : : : : +- CometProject (20) + : : : : +- CometFilter (19) + : : : : +- CometScan parquet spark_catalog.default.web_returns (18) + : : : +- * Project (29) + : : : +- * SortMergeJoin Inner (28) + : : : :- * Sort (25) + : : : : +- ReusedExchange (24) + : : : +- * Sort (27) + : : : +- ReusedExchange (26) + : : +- BroadcastExchange (37) + : : +- * ColumnarToRow (36) + : : +- CometProject (35) + : : +- CometFilter (34) + : : +- CometScan parquet spark_catalog.default.date_dim (33) + : +- BroadcastExchange (44) + : +- * ColumnarToRow (43) + : +- CometProject (42) + : +- CometFilter (41) + : +- CometScan parquet spark_catalog.default.customer_address (40) + +- BroadcastExchange (51) + +- * ColumnarToRow (50) + +- CometProject (49) + +- CometFilter (48) + +- CometScan parquet spark_catalog.default.web_site (47) + + +(1) Scan parquet spark_catalog.default.web_sales +Output [7]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6, ws_sold_date_sk#7] +Batched: true +Location [not included in comparison]/{warehouse_dir}/web_sales] +PushedFilters: [IsNotNull(ws_ship_date_sk), IsNotNull(ws_ship_addr_sk), IsNotNull(ws_web_site_sk)] +ReadSchema: struct<ws_ship_date_sk:int,ws_ship_addr_sk:int,ws_web_site_sk:int,ws_order_number:int,ws_ext_ship_cost:decimal(7,2),ws_net_profit:decimal(7,2)> + +(2) CometFilter +Input [7]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6, ws_sold_date_sk#7] +Condition : ((isnotnull(ws_ship_date_sk#1) AND isnotnull(ws_ship_addr_sk#2)) AND isnotnull(ws_web_site_sk#3)) + +(3) CometProject +Input [7]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6, ws_sold_date_sk#7] +Arguments: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6], [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6] + +(4) ColumnarToRow [codegen id : 1] +Input [6]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6] + +(5) Exchange +Input [6]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6] +Arguments: hashpartitioning(ws_order_number#4, 5), ENSURE_REQUIREMENTS, [plan_id=1] + +(6) Sort [codegen id : 2] +Input [6]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6] +Arguments: [ws_order_number#4 ASC NULLS FIRST], false, 0 + +(7) Scan parquet spark_catalog.default.web_sales +Output [3]: [ws_warehouse_sk#8, ws_order_number#9, ws_sold_date_sk#10] +Batched: true +Location [not included in comparison]/{warehouse_dir}/web_sales] +PushedFilters: [IsNotNull(ws_order_number), IsNotNull(ws_warehouse_sk)] +ReadSchema: struct<ws_warehouse_sk:int,ws_order_number:int> + +(8) CometFilter +Input [3]: [ws_warehouse_sk#8, ws_order_number#9, ws_sold_date_sk#10] +Condition : (isnotnull(ws_order_number#9) AND isnotnull(ws_warehouse_sk#8)) + +(9) CometProject +Input [3]: [ws_warehouse_sk#8, ws_order_number#9, ws_sold_date_sk#10] +Arguments: [ws_warehouse_sk#8, ws_order_number#9], [ws_warehouse_sk#8, ws_order_number#9] + +(10) ColumnarToRow [codegen id : 3] +Input [2]: [ws_warehouse_sk#8, ws_order_number#9] + +(11) Exchange +Input [2]: [ws_warehouse_sk#8, ws_order_number#9] +Arguments: hashpartitioning(ws_order_number#9, 5), ENSURE_REQUIREMENTS, [plan_id=2] + +(12) Sort [codegen id : 4] +Input [2]: [ws_warehouse_sk#8, ws_order_number#9] +Arguments: [ws_order_number#9 ASC NULLS FIRST], false, 0 + +(13) ReusedExchange [Reuses operator id: 11] +Output [2]: [ws_warehouse_sk#11, ws_order_number#12] + +(14) Sort [codegen id : 6] +Input [2]: [ws_warehouse_sk#11, ws_order_number#12] +Arguments: [ws_order_number#12 ASC NULLS FIRST], false, 0 + +(15) SortMergeJoin [codegen id : 7] +Left keys [1]: [ws_order_number#9] +Right keys [1]: [ws_order_number#12] +Join type: Inner +Join condition: NOT (ws_warehouse_sk#8 = ws_warehouse_sk#11) + +(16) Project [codegen id : 7] +Output [1]: [ws_order_number#9] +Input [4]: [ws_warehouse_sk#8, ws_order_number#9, ws_warehouse_sk#11, ws_order_number#12] + +(17) SortMergeJoin [codegen id : 8] +Left keys [1]: [ws_order_number#4] +Right keys [1]: [ws_order_number#9] +Join type: LeftSemi +Join condition: None + +(18) Scan parquet spark_catalog.default.web_returns +Output [2]: [wr_order_number#13, wr_returned_date_sk#14] +Batched: true +Location [not included in comparison]/{warehouse_dir}/web_returns] +PushedFilters: [IsNotNull(wr_order_number)] +ReadSchema: struct<wr_order_number:int> + +(19) CometFilter +Input [2]: [wr_order_number#13, wr_returned_date_sk#14] +Condition : isnotnull(wr_order_number#13) + +(20) CometProject +Input [2]: [wr_order_number#13, wr_returned_date_sk#14] +Arguments: [wr_order_number#13], [wr_order_number#13] + +(21) ColumnarToRow [codegen id : 9] +Input [1]: [wr_order_number#13] + +(22) Exchange +Input [1]: [wr_order_number#13] +Arguments: hashpartitioning(wr_order_number#13, 5), ENSURE_REQUIREMENTS, [plan_id=3] + +(23) Sort [codegen id : 10] +Input [1]: [wr_order_number#13] +Arguments: [wr_order_number#13 ASC NULLS FIRST], false, 0 + +(24) ReusedExchange [Reuses operator id: 11] +Output [2]: [ws_warehouse_sk#15, ws_order_number#16] + +(25) Sort [codegen id : 12] +Input [2]: [ws_warehouse_sk#15, ws_order_number#16] +Arguments: [ws_order_number#16 ASC NULLS FIRST], false, 0 + +(26) ReusedExchange [Reuses operator id: 11] +Output [2]: [ws_warehouse_sk#17, ws_order_number#18] + +(27) Sort [codegen id : 14] +Input [2]: [ws_warehouse_sk#17, ws_order_number#18] +Arguments: [ws_order_number#18 ASC NULLS FIRST], false, 0 + +(28) SortMergeJoin [codegen id : 15] +Left keys [1]: [ws_order_number#16] +Right keys [1]: [ws_order_number#18] +Join type: Inner +Join condition: NOT (ws_warehouse_sk#15 = ws_warehouse_sk#17) + +(29) Project [codegen id : 15] +Output [1]: [ws_order_number#16] +Input [4]: [ws_warehouse_sk#15, ws_order_number#16, ws_warehouse_sk#17, ws_order_number#18] + +(30) SortMergeJoin [codegen id : 16] +Left keys [1]: [wr_order_number#13] +Right keys [1]: [ws_order_number#16] +Join type: Inner +Join condition: None + +(31) Project [codegen id : 16] +Output [1]: [wr_order_number#13] +Input [2]: [wr_order_number#13, ws_order_number#16] + +(32) SortMergeJoin [codegen id : 20] +Left keys [1]: [ws_order_number#4] +Right keys [1]: [wr_order_number#13] +Join type: LeftSemi +Join condition: None + +(33) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#19, d_date#20] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,1999-02-01), LessThanOrEqual(d_date,1999-04-02), IsNotNull(d_date_sk)] +ReadSchema: struct<d_date_sk:int,d_date:date> + +(34) CometFilter +Input [2]: [d_date_sk#19, d_date#20] +Condition : (((isnotnull(d_date#20) AND (d_date#20 >= 1999-02-01)) AND (d_date#20 <= 1999-04-02)) AND isnotnull(d_date_sk#19)) + +(35) CometProject +Input [2]: [d_date_sk#19, d_date#20] +Arguments: [d_date_sk#19], [d_date_sk#19] + +(36) ColumnarToRow [codegen id : 17] +Input [1]: [d_date_sk#19] + +(37) BroadcastExchange +Input [1]: [d_date_sk#19] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] + +(38) BroadcastHashJoin [codegen id : 20] +Left keys [1]: [ws_ship_date_sk#1] +Right keys [1]: [d_date_sk#19] +Join type: Inner +Join condition: None + +(39) Project [codegen id : 20] +Output [5]: [ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6] +Input [7]: [ws_ship_date_sk#1, ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6, d_date_sk#19] + +(40) Scan parquet spark_catalog.default.customer_address +Output [2]: [ca_address_sk#21, ca_state#22] +Batched: true +Location [not included in comparison]/{warehouse_dir}/customer_address] +PushedFilters: [IsNotNull(ca_state), EqualTo(ca_state,IL), IsNotNull(ca_address_sk)] +ReadSchema: struct<ca_address_sk:int,ca_state:string> + +(41) CometFilter +Input [2]: [ca_address_sk#21, ca_state#22] +Condition : ((isnotnull(ca_state#22) AND (ca_state#22 = IL)) AND isnotnull(ca_address_sk#21)) + +(42) CometProject +Input [2]: [ca_address_sk#21, ca_state#22] +Arguments: [ca_address_sk#21], [ca_address_sk#21] + +(43) ColumnarToRow [codegen id : 18] +Input [1]: [ca_address_sk#21] + +(44) BroadcastExchange +Input [1]: [ca_address_sk#21] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] + +(45) BroadcastHashJoin [codegen id : 20] +Left keys [1]: [ws_ship_addr_sk#2] +Right keys [1]: [ca_address_sk#21] +Join type: Inner +Join condition: None + +(46) Project [codegen id : 20] +Output [4]: [ws_web_site_sk#3, ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6] +Input [6]: [ws_ship_addr_sk#2, ws_web_site_sk#3, ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6, ca_address_sk#21] + +(47) Scan parquet spark_catalog.default.web_site +Output [2]: [web_site_sk#23, web_company_name#24] +Batched: true +Location [not included in comparison]/{warehouse_dir}/web_site] +PushedFilters: [IsNotNull(web_company_name), EqualTo(web_company_name,pri ), IsNotNull(web_site_sk)] +ReadSchema: struct<web_site_sk:int,web_company_name:string> + +(48) CometFilter +Input [2]: [web_site_sk#23, web_company_name#24] +Condition : ((isnotnull(web_company_name#24) AND (web_company_name#24 = pri )) AND isnotnull(web_site_sk#23)) + +(49) CometProject +Input [2]: [web_site_sk#23, web_company_name#24] +Arguments: [web_site_sk#23], [web_site_sk#23] + +(50) ColumnarToRow [codegen id : 19] +Input [1]: [web_site_sk#23] + +(51) BroadcastExchange +Input [1]: [web_site_sk#23] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=6] + +(52) BroadcastHashJoin [codegen id : 20] +Left keys [1]: [ws_web_site_sk#3] +Right keys [1]: [web_site_sk#23] +Join type: Inner +Join condition: None + +(53) Project [codegen id : 20] +Output [3]: [ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6] +Input [5]: [ws_web_site_sk#3, ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6, web_site_sk#23] + +(54) HashAggregate [codegen id : 20] +Input [3]: [ws_order_number#4, ws_ext_ship_cost#5, ws_net_profit#6] +Keys [1]: [ws_order_number#4] +Functions [2]: [partial_sum(UnscaledValue(ws_ext_ship_cost#5)), partial_sum(UnscaledValue(ws_net_profit#6))] +Aggregate Attributes [2]: [sum(UnscaledValue(ws_ext_ship_cost#5))#25, sum(UnscaledValue(ws_net_profit#6))#26] +Results [3]: [ws_order_number#4, sum#27, sum#28] + +(55) HashAggregate [codegen id : 20] +Input [3]: [ws_order_number#4, sum#27, sum#28] +Keys [1]: [ws_order_number#4] +Functions [2]: [merge_sum(UnscaledValue(ws_ext_ship_cost#5)), merge_sum(UnscaledValue(ws_net_profit#6))] +Aggregate Attributes [2]: [sum(UnscaledValue(ws_ext_ship_cost#5))#25, sum(UnscaledValue(ws_net_profit#6))#26] +Results [3]: [ws_order_number#4, sum#27, sum#28] + +(56) HashAggregate [codegen id : 20] +Input [3]: [ws_order_number#4, sum#27, sum#28] +Keys: [] +Functions [3]: [merge_sum(UnscaledValue(ws_ext_ship_cost#5)), merge_sum(UnscaledValue(ws_net_profit#6)), partial_count(distinct ws_order_number#4)] +Aggregate Attributes [3]: [sum(UnscaledValue(ws_ext_ship_cost#5))#25, sum(UnscaledValue(ws_net_profit#6))#26, count(ws_order_number#4)#29] +Results [3]: [sum#27, sum#28, count#30] + +(57) Exchange +Input [3]: [sum#27, sum#28, count#30] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=7] + +(58) HashAggregate [codegen id : 21] +Input [3]: [sum#27, sum#28, count#30] +Keys: [] +Functions [3]: [sum(UnscaledValue(ws_ext_ship_cost#5)), sum(UnscaledValue(ws_net_profit#6)), count(distinct ws_order_number#4)] +Aggregate Attributes [3]: [sum(UnscaledValue(ws_ext_ship_cost#5))#25, sum(UnscaledValue(ws_net_profit#6))#26, count(ws_order_number#4)#29] +Results [3]: [count(ws_order_number#4)#29 AS order count #31, MakeDecimal(sum(UnscaledValue(ws_ext_ship_cost#5))#25,17,2) AS total shipping cost #32, MakeDecimal(sum(UnscaledValue(ws_net_profit#6))#26,17,2) AS total net profit #33] + diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q95/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q95/simplified.txt new file mode 100644 index 000000000..5b699890c --- /dev/null +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q95/simplified.txt @@ -0,0 +1,102 @@ +WholeStageCodegen (21) + HashAggregate [sum,sum,count] [sum(UnscaledValue(ws_ext_ship_cost)),sum(UnscaledValue(ws_net_profit)),count(ws_order_number),order count ,total shipping cost ,total net profit ,sum,sum,count] + InputAdapter + Exchange #1 + WholeStageCodegen (20) + HashAggregate [ws_order_number] [sum(UnscaledValue(ws_ext_ship_cost)),sum(UnscaledValue(ws_net_profit)),count(ws_order_number),sum,sum,count,sum,sum,count] + HashAggregate [ws_order_number] [sum(UnscaledValue(ws_ext_ship_cost)),sum(UnscaledValue(ws_net_profit)),sum,sum,sum,sum] + HashAggregate [ws_order_number,ws_ext_ship_cost,ws_net_profit] [sum(UnscaledValue(ws_ext_ship_cost)),sum(UnscaledValue(ws_net_profit)),sum,sum,sum,sum] + Project [ws_order_number,ws_ext_ship_cost,ws_net_profit] + BroadcastHashJoin [ws_web_site_sk,web_site_sk] + Project [ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit] + BroadcastHashJoin [ws_ship_addr_sk,ca_address_sk] + Project [ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit] + BroadcastHashJoin [ws_ship_date_sk,d_date_sk] + SortMergeJoin [ws_order_number,wr_order_number] + InputAdapter + WholeStageCodegen (8) + SortMergeJoin [ws_order_number,ws_order_number] + InputAdapter + WholeStageCodegen (2) + Sort [ws_order_number] + InputAdapter + Exchange [ws_order_number] #2 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + 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] + CometScan parquet spark_catalog.default.web_sales [ws_ship_date_sk,ws_ship_addr_sk,ws_web_site_sk,ws_order_number,ws_ext_ship_cost,ws_net_profit,ws_sold_date_sk] + InputAdapter + WholeStageCodegen (7) + Project [ws_order_number] + SortMergeJoin [ws_order_number,ws_order_number,ws_warehouse_sk,ws_warehouse_sk] + InputAdapter + WholeStageCodegen (4) + Sort [ws_order_number] + InputAdapter + Exchange [ws_order_number] #3 + WholeStageCodegen (3) + ColumnarToRow + InputAdapter + CometProject [ws_warehouse_sk,ws_order_number] + CometFilter [ws_order_number,ws_warehouse_sk] + CometScan parquet spark_catalog.default.web_sales [ws_warehouse_sk,ws_order_number,ws_sold_date_sk] + InputAdapter + WholeStageCodegen (6) + Sort [ws_order_number] + InputAdapter + ReusedExchange [ws_warehouse_sk,ws_order_number] #3 + InputAdapter + WholeStageCodegen (16) + Project [wr_order_number] + SortMergeJoin [wr_order_number,ws_order_number] + InputAdapter + WholeStageCodegen (10) + Sort [wr_order_number] + InputAdapter + Exchange [wr_order_number] #4 + WholeStageCodegen (9) + ColumnarToRow + InputAdapter + CometProject [wr_order_number] + CometFilter [wr_order_number] + CometScan parquet spark_catalog.default.web_returns [wr_order_number,wr_returned_date_sk] + InputAdapter + WholeStageCodegen (15) + Project [ws_order_number] + SortMergeJoin [ws_order_number,ws_order_number,ws_warehouse_sk,ws_warehouse_sk] + InputAdapter + WholeStageCodegen (12) + Sort [ws_order_number] + InputAdapter + ReusedExchange [ws_warehouse_sk,ws_order_number] #3 + InputAdapter + WholeStageCodegen (14) + Sort [ws_order_number] + InputAdapter + ReusedExchange [ws_warehouse_sk,ws_order_number] #3 + InputAdapter + BroadcastExchange #5 + WholeStageCodegen (17) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] + InputAdapter + BroadcastExchange #6 + WholeStageCodegen (18) + ColumnarToRow + InputAdapter + CometProject [ca_address_sk] + CometFilter [ca_state,ca_address_sk] + CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] + InputAdapter + BroadcastExchange #7 + WholeStageCodegen (19) + ColumnarToRow + InputAdapter + CometProject [web_site_sk] + CometFilter [web_company_name,web_site_sk] + CometScan parquet spark_catalog.default.web_site [web_site_sk,web_company_name] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q96/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q96/explain.txt new file mode 100644 index 000000000..60c262e9c --- /dev/null +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q96/explain.txt @@ -0,0 +1,163 @@ +== Physical Plan == +* HashAggregate (28) ++- Exchange (27) + +- * HashAggregate (26) + +- * Project (25) + +- * BroadcastHashJoin Inner BuildRight (24) + :- * Project (18) + : +- * BroadcastHashJoin Inner BuildRight (17) + : :- * Project (11) + : : +- * BroadcastHashJoin Inner BuildRight (10) + : : :- * ColumnarToRow (4) + : : : +- CometProject (3) + : : : +- CometFilter (2) + : : : +- CometScan parquet spark_catalog.default.store_sales (1) + : : +- BroadcastExchange (9) + : : +- * ColumnarToRow (8) + : : +- CometProject (7) + : : +- CometFilter (6) + : : +- CometScan parquet spark_catalog.default.household_demographics (5) + : +- BroadcastExchange (16) + : +- * ColumnarToRow (15) + : +- CometProject (14) + : +- CometFilter (13) + : +- CometScan parquet spark_catalog.default.time_dim (12) + +- BroadcastExchange (23) + +- * ColumnarToRow (22) + +- CometProject (21) + +- CometFilter (20) + +- CometScan parquet spark_catalog.default.store (19) + + +(1) Scan parquet spark_catalog.default.store_sales +Output [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_sold_date_sk#4] +Batched: true +Location [not included in comparison]/{warehouse_dir}/store_sales] +PushedFilters: [IsNotNull(ss_hdemo_sk), IsNotNull(ss_sold_time_sk), IsNotNull(ss_store_sk)] +ReadSchema: struct<ss_sold_time_sk:int,ss_hdemo_sk:int,ss_store_sk:int> + +(2) CometFilter +Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_sold_date_sk#4] +Condition : ((isnotnull(ss_hdemo_sk#2) AND isnotnull(ss_sold_time_sk#1)) AND isnotnull(ss_store_sk#3)) + +(3) CometProject +Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_sold_date_sk#4] +Arguments: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3], [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3] + +(4) ColumnarToRow [codegen id : 4] +Input [3]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3] + +(5) Scan parquet spark_catalog.default.household_demographics +Output [2]: [hd_demo_sk#5, hd_dep_count#6] +Batched: true +Location [not included in comparison]/{warehouse_dir}/household_demographics] +PushedFilters: [IsNotNull(hd_dep_count), EqualTo(hd_dep_count,7), IsNotNull(hd_demo_sk)] +ReadSchema: struct<hd_demo_sk:int,hd_dep_count:int> + +(6) CometFilter +Input [2]: [hd_demo_sk#5, hd_dep_count#6] +Condition : ((isnotnull(hd_dep_count#6) AND (hd_dep_count#6 = 7)) AND isnotnull(hd_demo_sk#5)) + +(7) CometProject +Input [2]: [hd_demo_sk#5, hd_dep_count#6] +Arguments: [hd_demo_sk#5], [hd_demo_sk#5] + +(8) ColumnarToRow [codegen id : 1] +Input [1]: [hd_demo_sk#5] + +(9) BroadcastExchange +Input [1]: [hd_demo_sk#5] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=1] + +(10) BroadcastHashJoin [codegen id : 4] +Left keys [1]: [ss_hdemo_sk#2] +Right keys [1]: [hd_demo_sk#5] +Join type: Inner +Join condition: None + +(11) Project [codegen id : 4] +Output [2]: [ss_sold_time_sk#1, ss_store_sk#3] +Input [4]: [ss_sold_time_sk#1, ss_hdemo_sk#2, ss_store_sk#3, hd_demo_sk#5] + +(12) Scan parquet spark_catalog.default.time_dim +Output [3]: [t_time_sk#7, t_hour#8, t_minute#9] +Batched: true +Location [not included in comparison]/{warehouse_dir}/time_dim] +PushedFilters: [IsNotNull(t_hour), IsNotNull(t_minute), EqualTo(t_hour,20), GreaterThanOrEqual(t_minute,30), IsNotNull(t_time_sk)] +ReadSchema: struct<t_time_sk:int,t_hour:int,t_minute:int> + +(13) CometFilter +Input [3]: [t_time_sk#7, t_hour#8, t_minute#9] +Condition : ((((isnotnull(t_hour#8) AND isnotnull(t_minute#9)) AND (t_hour#8 = 20)) AND (t_minute#9 >= 30)) AND isnotnull(t_time_sk#7)) + +(14) CometProject +Input [3]: [t_time_sk#7, t_hour#8, t_minute#9] +Arguments: [t_time_sk#7], [t_time_sk#7] + +(15) ColumnarToRow [codegen id : 2] +Input [1]: [t_time_sk#7] + +(16) BroadcastExchange +Input [1]: [t_time_sk#7] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] + +(17) BroadcastHashJoin [codegen id : 4] +Left keys [1]: [ss_sold_time_sk#1] +Right keys [1]: [t_time_sk#7] +Join type: Inner +Join condition: None + +(18) Project [codegen id : 4] +Output [1]: [ss_store_sk#3] +Input [3]: [ss_sold_time_sk#1, ss_store_sk#3, t_time_sk#7] + +(19) Scan parquet spark_catalog.default.store +Output [2]: [s_store_sk#10, s_store_name#11] +Batched: true +Location [not included in comparison]/{warehouse_dir}/store] +PushedFilters: [IsNotNull(s_store_name), EqualTo(s_store_name,ese), IsNotNull(s_store_sk)] +ReadSchema: struct<s_store_sk:int,s_store_name:string> + +(20) CometFilter +Input [2]: [s_store_sk#10, s_store_name#11] +Condition : ((isnotnull(s_store_name#11) AND (s_store_name#11 = ese)) AND isnotnull(s_store_sk#10)) + +(21) CometProject +Input [2]: [s_store_sk#10, s_store_name#11] +Arguments: [s_store_sk#10], [s_store_sk#10] + +(22) ColumnarToRow [codegen id : 3] +Input [1]: [s_store_sk#10] + +(23) BroadcastExchange +Input [1]: [s_store_sk#10] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] + +(24) BroadcastHashJoin [codegen id : 4] +Left keys [1]: [ss_store_sk#3] +Right keys [1]: [s_store_sk#10] +Join type: Inner +Join condition: None + +(25) Project [codegen id : 4] +Output: [] +Input [2]: [ss_store_sk#3, s_store_sk#10] + +(26) HashAggregate [codegen id : 4] +Input: [] +Keys: [] +Functions [1]: [partial_count(1)] +Aggregate Attributes [1]: [count#12] +Results [1]: [count#13] + +(27) Exchange +Input [1]: [count#13] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=4] + +(28) HashAggregate [codegen id : 5] +Input [1]: [count#13] +Keys: [] +Functions [1]: [count(1)] +Aggregate Attributes [1]: [count(1)#14] +Results [1]: [count(1)#14 AS count(1)#15] + diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q96/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q96/simplified.txt new file mode 100644 index 000000000..d1438f48e --- /dev/null +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q96/simplified.txt @@ -0,0 +1,41 @@ +WholeStageCodegen (5) + HashAggregate [count] [count(1),count(1),count] + InputAdapter + Exchange #1 + WholeStageCodegen (4) + HashAggregate [count,count] + Project + BroadcastHashJoin [ss_store_sk,s_store_sk] + Project [ss_store_sk] + BroadcastHashJoin [ss_sold_time_sk,t_time_sk] + Project [ss_sold_time_sk,ss_store_sk] + BroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] + ColumnarToRow + InputAdapter + CometProject [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk] + CometFilter [ss_hdemo_sk,ss_sold_time_sk,ss_store_sk] + CometScan parquet spark_catalog.default.store_sales [ss_sold_time_sk,ss_hdemo_sk,ss_store_sk,ss_sold_date_sk] + InputAdapter + BroadcastExchange #2 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [hd_demo_sk] + CometFilter [hd_dep_count,hd_demo_sk] + CometScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_dep_count] + InputAdapter + BroadcastExchange #3 + WholeStageCodegen (2) + ColumnarToRow + InputAdapter + CometProject [t_time_sk] + CometFilter [t_hour,t_minute,t_time_sk] + CometScan parquet spark_catalog.default.time_dim [t_time_sk,t_hour,t_minute] + InputAdapter + BroadcastExchange #4 + WholeStageCodegen (3) + ColumnarToRow + InputAdapter + CometProject [s_store_sk] + CometFilter [s_store_name,s_store_sk] + CometScan parquet spark_catalog.default.store [s_store_sk,s_store_name] 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 new file mode 100644 index 000000000..66ccf4f22 --- /dev/null +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q97/explain.txt @@ -0,0 +1,179 @@ +== Physical Plan == +* HashAggregate (23) ++- Exchange (22) + +- * HashAggregate (21) + +- * Project (20) + +- * SortMergeJoin FullOuter (19) + :- * Sort (9) + : +- * HashAggregate (8) + : +- Exchange (7) + : +- * HashAggregate (6) + : +- * Project (5) + : +- * BroadcastHashJoin Inner BuildRight (4) + : :- * ColumnarToRow (2) + : : +- CometScan parquet spark_catalog.default.store_sales (1) + : +- ReusedExchange (3) + +- * Sort (18) + +- * HashAggregate (17) + +- Exchange (16) + +- * HashAggregate (15) + +- * Project (14) + +- * BroadcastHashJoin Inner BuildRight (13) + :- * ColumnarToRow (11) + : +- CometScan parquet spark_catalog.default.catalog_sales (10) + +- ReusedExchange (12) + + +(1) Scan parquet spark_catalog.default.store_sales +Output [3]: [ss_item_sk#1, ss_customer_sk#2, ss_sold_date_sk#3] +Batched: true +Location: InMemoryFileIndex [] +PartitionFilters: [isnotnull(ss_sold_date_sk#3), dynamicpruningexpression(ss_sold_date_sk#3 IN dynamicpruning#4)] +ReadSchema: struct<ss_item_sk:int,ss_customer_sk:int> + +(2) ColumnarToRow [codegen id : 2] +Input [3]: [ss_item_sk#1, ss_customer_sk#2, ss_sold_date_sk#3] + +(3) ReusedExchange [Reuses operator id: 28] +Output [1]: [d_date_sk#5] + +(4) BroadcastHashJoin [codegen id : 2] +Left keys [1]: [ss_sold_date_sk#3] +Right keys [1]: [d_date_sk#5] +Join type: Inner +Join condition: None + +(5) Project [codegen id : 2] +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] +Input [2]: [ss_item_sk#1, ss_customer_sk#2] +Keys [2]: [ss_customer_sk#2, ss_item_sk#1] +Functions: [] +Aggregate Attributes: [] +Results [2]: [ss_customer_sk#2, ss_item_sk#1] + +(7) Exchange +Input [2]: [ss_customer_sk#2, ss_item_sk#1] +Arguments: hashpartitioning(ss_customer_sk#2, ss_item_sk#1, 5), ENSURE_REQUIREMENTS, [plan_id=1] + +(8) HashAggregate [codegen id : 3] +Input [2]: [ss_customer_sk#2, ss_item_sk#1] +Keys [2]: [ss_customer_sk#2, ss_item_sk#1] +Functions: [] +Aggregate Attributes: [] +Results [2]: [ss_customer_sk#2 AS customer_sk#6, ss_item_sk#1 AS item_sk#7] + +(9) Sort [codegen id : 3] +Input [2]: [customer_sk#6, item_sk#7] +Arguments: [customer_sk#6 ASC NULLS FIRST, item_sk#7 ASC NULLS FIRST], false, 0 + +(10) Scan parquet spark_catalog.default.catalog_sales +Output [3]: [cs_bill_customer_sk#8, cs_item_sk#9, cs_sold_date_sk#10] +Batched: true +Location: InMemoryFileIndex [] +PartitionFilters: [isnotnull(cs_sold_date_sk#10), dynamicpruningexpression(cs_sold_date_sk#10 IN dynamicpruning#11)] +ReadSchema: struct<cs_bill_customer_sk:int,cs_item_sk:int> + +(11) ColumnarToRow [codegen id : 5] +Input [3]: [cs_bill_customer_sk#8, cs_item_sk#9, cs_sold_date_sk#10] + +(12) ReusedExchange [Reuses operator id: 28] +Output [1]: [d_date_sk#12] + +(13) BroadcastHashJoin [codegen id : 5] +Left keys [1]: [cs_sold_date_sk#10] +Right keys [1]: [d_date_sk#12] +Join type: Inner +Join condition: None + +(14) Project [codegen id : 5] +Output [2]: [cs_bill_customer_sk#8, cs_item_sk#9] +Input [4]: [cs_bill_customer_sk#8, cs_item_sk#9, cs_sold_date_sk#10, d_date_sk#12] + +(15) HashAggregate [codegen id : 5] +Input [2]: [cs_bill_customer_sk#8, cs_item_sk#9] +Keys [2]: [cs_bill_customer_sk#8, cs_item_sk#9] +Functions: [] +Aggregate Attributes: [] +Results [2]: [cs_bill_customer_sk#8, cs_item_sk#9] + +(16) Exchange +Input [2]: [cs_bill_customer_sk#8, cs_item_sk#9] +Arguments: hashpartitioning(cs_bill_customer_sk#8, cs_item_sk#9, 5), ENSURE_REQUIREMENTS, [plan_id=2] + +(17) HashAggregate [codegen id : 6] +Input [2]: [cs_bill_customer_sk#8, cs_item_sk#9] +Keys [2]: [cs_bill_customer_sk#8, cs_item_sk#9] +Functions: [] +Aggregate Attributes: [] +Results [2]: [cs_bill_customer_sk#8 AS customer_sk#13, cs_item_sk#9 AS item_sk#14] + +(18) Sort [codegen id : 6] +Input [2]: [customer_sk#13, item_sk#14] +Arguments: [customer_sk#13 ASC NULLS FIRST, item_sk#14 ASC NULLS FIRST], false, 0 + +(19) SortMergeJoin [codegen id : 7] +Left keys [2]: [customer_sk#6, item_sk#7] +Right keys [2]: [customer_sk#13, item_sk#14] +Join type: FullOuter +Join condition: None + +(20) Project [codegen id : 7] +Output [2]: [customer_sk#6, customer_sk#13] +Input [4]: [customer_sk#6, item_sk#7, customer_sk#13, item_sk#14] + +(21) HashAggregate [codegen id : 7] +Input [2]: [customer_sk#6, customer_sk#13] +Keys: [] +Functions [3]: [partial_sum(CASE WHEN (isnotnull(customer_sk#6) AND isnull(customer_sk#13)) THEN 1 ELSE 0 END), partial_sum(CASE WHEN (isnull(customer_sk#6) AND isnotnull(customer_sk#13)) THEN 1 ELSE 0 END), partial_sum(CASE WHEN (isnotnull(customer_sk#6) AND isnotnull(customer_sk#13)) THEN 1 ELSE 0 END)] +Aggregate Attributes [3]: [sum#15, sum#16, sum#17] +Results [3]: [sum#18, sum#19, sum#20] + +(22) Exchange +Input [3]: [sum#18, sum#19, sum#20] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=3] + +(23) HashAggregate [codegen id : 8] +Input [3]: [sum#18, sum#19, sum#20] +Keys: [] +Functions [3]: [sum(CASE WHEN (isnotnull(customer_sk#6) AND isnull(customer_sk#13)) THEN 1 ELSE 0 END), sum(CASE WHEN (isnull(customer_sk#6) AND isnotnull(customer_sk#13)) THEN 1 ELSE 0 END), sum(CASE WHEN (isnotnull(customer_sk#6) AND isnotnull(customer_sk#13)) THEN 1 ELSE 0 END)] +Aggregate Attributes [3]: [sum(CASE WHEN (isnotnull(customer_sk#6) AND isnull(customer_sk#13)) THEN 1 ELSE 0 END)#21, sum(CASE WHEN (isnull(customer_sk#6) AND isnotnull(customer_sk#13)) THEN 1 ELSE 0 END)#22, sum(CASE WHEN (isnotnull(customer_sk#6) AND isnotnull(customer_sk#13)) THEN 1 ELSE 0 END)#23] +Results [3]: [sum(CASE WHEN (isnotnull(customer_sk#6) AND isnull(customer_sk#13)) THEN 1 ELSE 0 END)#21 AS store_only#24, sum(CASE WHEN (isnull(customer_sk#6) AND isnotnull(customer_sk#13)) THEN 1 ELSE 0 END)#22 AS catalog_only#25, sum(CASE WHEN (isnotnull(customer_sk#6) AND isnotnull(customer_sk#13)) THEN 1 ELSE 0 END)#23 AS store_and_catalog#26] + +===== Subqueries ===== + +Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#3 IN dynamicpruning#4 +BroadcastExchange (28) ++- * ColumnarToRow (27) + +- CometProject (26) + +- CometFilter (25) + +- CometScan parquet spark_catalog.default.date_dim (24) + + +(24) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#5, d_month_seq#27] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1200), LessThanOrEqual(d_month_seq,1211), IsNotNull(d_date_sk)] +ReadSchema: struct<d_date_sk:int,d_month_seq:int> + +(25) CometFilter +Input [2]: [d_date_sk#5, d_month_seq#27] +Condition : (((isnotnull(d_month_seq#27) AND (d_month_seq#27 >= 1200)) AND (d_month_seq#27 <= 1211)) AND isnotnull(d_date_sk#5)) + +(26) CometProject +Input [2]: [d_date_sk#5, d_month_seq#27] +Arguments: [d_date_sk#5], [d_date_sk#5] + +(27) ColumnarToRow [codegen id : 1] +Input [1]: [d_date_sk#5] + +(28) BroadcastExchange +Input [1]: [d_date_sk#5] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] + +Subquery:2 Hosting operator id = 10 Hosting Expression = cs_sold_date_sk#10 IN dynamicpruning#4 + + 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 new file mode 100644 index 000000000..be9c20a56 --- /dev/null +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q97/simplified.txt @@ -0,0 +1,47 @@ +WholeStageCodegen (8) + HashAggregate [sum,sum,sum] [sum(CASE WHEN (isnotnull(customer_sk) AND isnull(customer_sk)) THEN 1 ELSE 0 END),sum(CASE WHEN (isnull(customer_sk) AND isnotnull(customer_sk)) THEN 1 ELSE 0 END),sum(CASE WHEN (isnotnull(customer_sk) AND isnotnull(customer_sk)) THEN 1 ELSE 0 END),store_only,catalog_only,store_and_catalog,sum,sum,sum] + InputAdapter + Exchange #1 + WholeStageCodegen (7) + HashAggregate [customer_sk,customer_sk] [sum,sum,sum,sum,sum,sum] + Project [customer_sk,customer_sk] + SortMergeJoin [customer_sk,item_sk,customer_sk,item_sk] + InputAdapter + WholeStageCodegen (3) + Sort [customer_sk,item_sk] + HashAggregate [ss_customer_sk,ss_item_sk] [customer_sk,item_sk] + InputAdapter + Exchange [ss_customer_sk,ss_item_sk] #2 + WholeStageCodegen (2) + HashAggregate [ss_customer_sk,ss_item_sk] + Project [ss_item_sk,ss_customer_sk] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + ColumnarToRow + InputAdapter + CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_month_seq,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] + InputAdapter + ReusedExchange [d_date_sk] #3 + InputAdapter + WholeStageCodegen (6) + Sort [customer_sk,item_sk] + HashAggregate [cs_bill_customer_sk,cs_item_sk] [customer_sk,item_sk] + InputAdapter + Exchange [cs_bill_customer_sk,cs_item_sk] #4 + WholeStageCodegen (5) + HashAggregate [cs_bill_customer_sk,cs_item_sk] + Project [cs_bill_customer_sk,cs_item_sk] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + ColumnarToRow + InputAdapter + CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_item_sk,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + InputAdapter + ReusedExchange [d_date_sk] #3 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 new file mode 100644 index 000000000..3d66a07d0 --- /dev/null +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q98/explain.txt @@ -0,0 +1,160 @@ +== Physical Plan == +* Project (22) ++- * Sort (21) + +- Exchange (20) + +- * Project (19) + +- Window (18) + +- * Sort (17) + +- Exchange (16) + +- * HashAggregate (15) + +- Exchange (14) + +- * HashAggregate (13) + +- * Project (12) + +- * BroadcastHashJoin Inner BuildRight (11) + :- * Project (9) + : +- * BroadcastHashJoin Inner BuildRight (8) + : :- * ColumnarToRow (3) + : : +- CometFilter (2) + : : +- CometScan parquet spark_catalog.default.store_sales (1) + : +- BroadcastExchange (7) + : +- * ColumnarToRow (6) + : +- CometFilter (5) + : +- CometScan parquet spark_catalog.default.item (4) + +- ReusedExchange (10) + + +(1) Scan parquet spark_catalog.default.store_sales +Output [3]: [ss_item_sk#1, ss_ext_sales_price#2, ss_sold_date_sk#3] +Batched: true +Location: InMemoryFileIndex [] +PartitionFilters: [isnotnull(ss_sold_date_sk#3), dynamicpruningexpression(ss_sold_date_sk#3 IN dynamicpruning#4)] +PushedFilters: [IsNotNull(ss_item_sk)] +ReadSchema: struct<ss_item_sk:int,ss_ext_sales_price:decimal(7,2)> + +(2) CometFilter +Input [3]: [ss_item_sk#1, ss_ext_sales_price#2, ss_sold_date_sk#3] +Condition : isnotnull(ss_item_sk#1) + +(3) ColumnarToRow [codegen id : 3] +Input [3]: [ss_item_sk#1, ss_ext_sales_price#2, ss_sold_date_sk#3] + +(4) Scan parquet spark_catalog.default.item +Output [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] +Batched: true +Location [not included in comparison]/{warehouse_dir}/item] +PushedFilters: [In(i_category, [Books ,Home ,Sports ]), IsNotNull(i_item_sk)] +ReadSchema: struct<i_item_sk:int,i_item_id:string,i_item_desc:string,i_current_price:decimal(7,2),i_class:string,i_category:string> + +(5) CometFilter +Input [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] +Condition : (i_category#10 IN (Sports ,Books ,Home ) AND isnotnull(i_item_sk#5)) + +(6) ColumnarToRow [codegen id : 1] +Input [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] + +(7) BroadcastExchange +Input [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=1] + +(8) BroadcastHashJoin [codegen id : 3] +Left keys [1]: [ss_item_sk#1] +Right keys [1]: [i_item_sk#5] +Join type: Inner +Join condition: None + +(9) Project [codegen id : 3] +Output [7]: [ss_ext_sales_price#2, ss_sold_date_sk#3, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] +Input [9]: [ss_item_sk#1, ss_ext_sales_price#2, ss_sold_date_sk#3, i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] + +(10) ReusedExchange [Reuses operator id: 27] +Output [1]: [d_date_sk#11] + +(11) BroadcastHashJoin [codegen id : 3] +Left keys [1]: [ss_sold_date_sk#3] +Right keys [1]: [d_date_sk#11] +Join type: Inner +Join condition: None + +(12) Project [codegen id : 3] +Output [6]: [ss_ext_sales_price#2, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] +Input [8]: [ss_ext_sales_price#2, ss_sold_date_sk#3, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10, d_date_sk#11] + +(13) HashAggregate [codegen id : 3] +Input [6]: [ss_ext_sales_price#2, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] +Keys [5]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8] +Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#2))] +Aggregate Attributes [1]: [sum#12] +Results [6]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, sum#13] + +(14) Exchange +Input [6]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, sum#13] +Arguments: hashpartitioning(i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, 5), ENSURE_REQUIREMENTS, [plan_id=2] + +(15) HashAggregate [codegen id : 4] +Input [6]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, sum#13] +Keys [5]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8] +Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#2))] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#2))#14] +Results [7]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#2))#14,17,2) AS itemrevenue#15, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#2))#14,17,2) AS _w0#16, i_item_id#6] + +(16) Exchange +Input [7]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#15, _w0#16, i_item_id#6] +Arguments: hashpartitioning(i_class#9, 5), ENSURE_REQUIREMENTS, [plan_id=3] + +(17) Sort [codegen id : 5] +Input [7]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#15, _w0#16, i_item_id#6] +Arguments: [i_class#9 ASC NULLS FIRST], false, 0 + +(18) Window +Input [7]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#15, _w0#16, i_item_id#6] +Arguments: [sum(_w0#16) windowspecdefinition(i_class#9, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS _we0#17], [i_class#9] + +(19) Project [codegen id : 6] +Output [7]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#15, ((_w0#16 * 100) / _we0#17) AS revenueratio#18, i_item_id#6] +Input [8]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#15, _w0#16, i_item_id#6, _we0#17] + +(20) Exchange +Input [7]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#15, revenueratio#18, i_item_id#6] +Arguments: rangepartitioning(i_category#10 ASC NULLS FIRST, i_class#9 ASC NULLS FIRST, i_item_id#6 ASC NULLS FIRST, i_item_desc#7 ASC NULLS FIRST, revenueratio#18 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, [plan_id=4] + +(21) Sort [codegen id : 7] +Input [7]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#15, revenueratio#18, i_item_id#6] +Arguments: [i_category#10 ASC NULLS FIRST, i_class#9 ASC NULLS FIRST, i_item_id#6 ASC NULLS FIRST, i_item_desc#7 ASC NULLS FIRST, revenueratio#18 ASC NULLS FIRST], true, 0 + +(22) Project [codegen id : 7] +Output [6]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#15, revenueratio#18] +Input [7]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#15, revenueratio#18, i_item_id#6] + +===== Subqueries ===== + +Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#3 IN dynamicpruning#4 +BroadcastExchange (27) ++- * ColumnarToRow (26) + +- CometProject (25) + +- CometFilter (24) + +- CometScan parquet spark_catalog.default.date_dim (23) + + +(23) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#11, d_date#19] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,1999-02-22), LessThanOrEqual(d_date,1999-03-24), IsNotNull(d_date_sk)] +ReadSchema: struct<d_date_sk:int,d_date:date> + +(24) CometFilter +Input [2]: [d_date_sk#11, d_date#19] +Condition : (((isnotnull(d_date#19) AND (d_date#19 >= 1999-02-22)) AND (d_date#19 <= 1999-03-24)) AND isnotnull(d_date_sk#11)) + +(25) CometProject +Input [2]: [d_date_sk#11, d_date#19] +Arguments: [d_date_sk#11], [d_date_sk#11] + +(26) ColumnarToRow [codegen id : 1] +Input [1]: [d_date_sk#11] + +(27) BroadcastExchange +Input [1]: [d_date_sk#11] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] + + diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q98/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q98/simplified.txt new file mode 100644 index 000000000..9eabb9977 --- /dev/null +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q98/simplified.txt @@ -0,0 +1,44 @@ +WholeStageCodegen (7) + Project [i_item_desc,i_category,i_class,i_current_price,itemrevenue,revenueratio] + Sort [i_category,i_class,i_item_id,i_item_desc,revenueratio] + InputAdapter + Exchange [i_category,i_class,i_item_id,i_item_desc,revenueratio] #1 + WholeStageCodegen (6) + Project [i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,_we0,i_item_id] + InputAdapter + Window [_w0,i_class] + WholeStageCodegen (5) + Sort [i_class] + InputAdapter + Exchange [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] + InputAdapter + Exchange [i_item_id,i_item_desc,i_category,i_class,i_current_price] #3 + WholeStageCodegen (3) + HashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,ss_ext_sales_price] [sum,sum] + Project [ss_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [ss_ext_sales_price,ss_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + BroadcastHashJoin [ss_item_sk,i_item_sk] + ColumnarToRow + InputAdapter + CometFilter [ss_item_sk] + CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #4 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] + InputAdapter + BroadcastExchange #5 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometFilter [i_category,i_item_sk] + CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + InputAdapter + ReusedExchange [d_date_sk] #4 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q99/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q99/explain.txt new file mode 100644 index 000000000..936555026 --- /dev/null +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q99/explain.txt @@ -0,0 +1,187 @@ +== Physical Plan == +TakeOrderedAndProject (32) ++- * HashAggregate (31) + +- Exchange (30) + +- * HashAggregate (29) + +- * Project (28) + +- * BroadcastHashJoin Inner BuildRight (27) + :- * Project (21) + : +- * BroadcastHashJoin Inner BuildRight (20) + : :- * Project (15) + : : +- * BroadcastHashJoin Inner BuildRight (14) + : : :- * Project (9) + : : : +- * BroadcastHashJoin Inner BuildRight (8) + : : : :- * ColumnarToRow (3) + : : : : +- CometFilter (2) + : : : : +- CometScan parquet spark_catalog.default.catalog_sales (1) + : : : +- BroadcastExchange (7) + : : : +- * ColumnarToRow (6) + : : : +- CometFilter (5) + : : : +- CometScan parquet spark_catalog.default.warehouse (4) + : : +- BroadcastExchange (13) + : : +- * ColumnarToRow (12) + : : +- CometFilter (11) + : : +- CometScan parquet spark_catalog.default.ship_mode (10) + : +- BroadcastExchange (19) + : +- * ColumnarToRow (18) + : +- CometFilter (17) + : +- CometScan parquet spark_catalog.default.call_center (16) + +- BroadcastExchange (26) + +- * ColumnarToRow (25) + +- CometProject (24) + +- CometFilter (23) + +- CometScan parquet spark_catalog.default.date_dim (22) + + +(1) Scan parquet spark_catalog.default.catalog_sales +Output [5]: [cs_ship_date_sk#1, cs_call_center_sk#2, cs_ship_mode_sk#3, cs_warehouse_sk#4, cs_sold_date_sk#5] +Batched: true +Location [not included in comparison]/{warehouse_dir}/catalog_sales] +PushedFilters: [IsNotNull(cs_warehouse_sk), IsNotNull(cs_ship_mode_sk), IsNotNull(cs_call_center_sk), IsNotNull(cs_ship_date_sk)] +ReadSchema: struct<cs_ship_date_sk:int,cs_call_center_sk:int,cs_ship_mode_sk:int,cs_warehouse_sk:int> + +(2) CometFilter +Input [5]: [cs_ship_date_sk#1, cs_call_center_sk#2, cs_ship_mode_sk#3, cs_warehouse_sk#4, cs_sold_date_sk#5] +Condition : (((isnotnull(cs_warehouse_sk#4) AND isnotnull(cs_ship_mode_sk#3)) AND isnotnull(cs_call_center_sk#2)) AND isnotnull(cs_ship_date_sk#1)) + +(3) ColumnarToRow [codegen id : 5] +Input [5]: [cs_ship_date_sk#1, cs_call_center_sk#2, cs_ship_mode_sk#3, cs_warehouse_sk#4, cs_sold_date_sk#5] + +(4) Scan parquet spark_catalog.default.warehouse +Output [2]: [w_warehouse_sk#6, w_warehouse_name#7] +Batched: true +Location [not included in comparison]/{warehouse_dir}/warehouse] +PushedFilters: [IsNotNull(w_warehouse_sk)] +ReadSchema: struct<w_warehouse_sk:int,w_warehouse_name:string> + +(5) CometFilter +Input [2]: [w_warehouse_sk#6, w_warehouse_name#7] +Condition : isnotnull(w_warehouse_sk#6) + +(6) ColumnarToRow [codegen id : 1] +Input [2]: [w_warehouse_sk#6, w_warehouse_name#7] + +(7) BroadcastExchange +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 : 5] +Left keys [1]: [cs_warehouse_sk#4] +Right keys [1]: [w_warehouse_sk#6] +Join type: Inner +Join condition: None + +(9) Project [codegen id : 5] +Output [5]: [cs_ship_date_sk#1, cs_call_center_sk#2, cs_ship_mode_sk#3, cs_sold_date_sk#5, w_warehouse_name#7] +Input [7]: [cs_ship_date_sk#1, cs_call_center_sk#2, cs_ship_mode_sk#3, cs_warehouse_sk#4, cs_sold_date_sk#5, w_warehouse_sk#6, w_warehouse_name#7] + +(10) Scan parquet spark_catalog.default.ship_mode +Output [2]: [sm_ship_mode_sk#8, sm_type#9] +Batched: true +Location [not included in comparison]/{warehouse_dir}/ship_mode] +PushedFilters: [IsNotNull(sm_ship_mode_sk)] +ReadSchema: struct<sm_ship_mode_sk:int,sm_type:string> + +(11) CometFilter +Input [2]: [sm_ship_mode_sk#8, sm_type#9] +Condition : isnotnull(sm_ship_mode_sk#8) + +(12) ColumnarToRow [codegen id : 2] +Input [2]: [sm_ship_mode_sk#8, sm_type#9] + +(13) BroadcastExchange +Input [2]: [sm_ship_mode_sk#8, sm_type#9] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=2] + +(14) BroadcastHashJoin [codegen id : 5] +Left keys [1]: [cs_ship_mode_sk#3] +Right keys [1]: [sm_ship_mode_sk#8] +Join type: Inner +Join condition: None + +(15) Project [codegen id : 5] +Output [5]: [cs_ship_date_sk#1, cs_call_center_sk#2, cs_sold_date_sk#5, w_warehouse_name#7, sm_type#9] +Input [7]: [cs_ship_date_sk#1, cs_call_center_sk#2, cs_ship_mode_sk#3, cs_sold_date_sk#5, w_warehouse_name#7, sm_ship_mode_sk#8, sm_type#9] + +(16) Scan parquet spark_catalog.default.call_center +Output [2]: [cc_call_center_sk#10, cc_name#11] +Batched: true +Location [not included in comparison]/{warehouse_dir}/call_center] +PushedFilters: [IsNotNull(cc_call_center_sk)] +ReadSchema: struct<cc_call_center_sk:int,cc_name:string> + +(17) CometFilter +Input [2]: [cc_call_center_sk#10, cc_name#11] +Condition : isnotnull(cc_call_center_sk#10) + +(18) ColumnarToRow [codegen id : 3] +Input [2]: [cc_call_center_sk#10, cc_name#11] + +(19) BroadcastExchange +Input [2]: [cc_call_center_sk#10, cc_name#11] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=3] + +(20) BroadcastHashJoin [codegen id : 5] +Left keys [1]: [cs_call_center_sk#2] +Right keys [1]: [cc_call_center_sk#10] +Join type: Inner +Join condition: None + +(21) Project [codegen id : 5] +Output [5]: [cs_ship_date_sk#1, cs_sold_date_sk#5, w_warehouse_name#7, sm_type#9, cc_name#11] +Input [7]: [cs_ship_date_sk#1, cs_call_center_sk#2, cs_sold_date_sk#5, w_warehouse_name#7, sm_type#9, cc_call_center_sk#10, cc_name#11] + +(22) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#12, d_month_seq#13] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1200), LessThanOrEqual(d_month_seq,1211), IsNotNull(d_date_sk)] +ReadSchema: struct<d_date_sk:int,d_month_seq:int> + +(23) CometFilter +Input [2]: [d_date_sk#12, d_month_seq#13] +Condition : (((isnotnull(d_month_seq#13) AND (d_month_seq#13 >= 1200)) AND (d_month_seq#13 <= 1211)) AND isnotnull(d_date_sk#12)) + +(24) CometProject +Input [2]: [d_date_sk#12, d_month_seq#13] +Arguments: [d_date_sk#12], [d_date_sk#12] + +(25) ColumnarToRow [codegen id : 4] +Input [1]: [d_date_sk#12] + +(26) BroadcastExchange +Input [1]: [d_date_sk#12] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] + +(27) BroadcastHashJoin [codegen id : 5] +Left keys [1]: [cs_ship_date_sk#1] +Right keys [1]: [d_date_sk#12] +Join type: Inner +Join condition: None + +(28) Project [codegen id : 5] +Output [5]: [cs_ship_date_sk#1, cs_sold_date_sk#5, sm_type#9, cc_name#11, substr(w_warehouse_name#7, 1, 20) AS _groupingexpression#14] +Input [6]: [cs_ship_date_sk#1, cs_sold_date_sk#5, w_warehouse_name#7, sm_type#9, cc_name#11, d_date_sk#12] + +(29) HashAggregate [codegen id : 5] +Input [5]: [cs_ship_date_sk#1, cs_sold_date_sk#5, sm_type#9, cc_name#11, _groupingexpression#14] +Keys [3]: [_groupingexpression#14, sm_type#9, cc_name#11] +Functions [5]: [partial_sum(CASE WHEN ((cs_ship_date_sk#1 - cs_sold_date_sk#5) <= 30) THEN 1 ELSE 0 END), partial_sum(CASE WHEN (((cs_ship_date_sk#1 - cs_sold_date_sk#5) > 30) AND ((cs_ship_date_sk#1 - cs_sold_date_sk#5) <= 60)) THEN 1 ELSE 0 END), partial_sum(CASE WHEN (((cs_ship_date_sk#1 - cs_sold_date_sk#5) > 60) AND ((cs_ship_date_sk#1 - cs_sold_date_sk#5) <= 90)) THEN 1 ELSE 0 END), partial_sum(CASE WHEN (((cs_ship_date_sk#1 - cs_sold_date_sk#5) > 90) AND ((cs_ship_date_sk#1 - cs_sold_date_sk#5) <= 120)) THEN 1 ELSE 0 END), partial_sum(CASE WHEN ((cs_ship_date_sk#1 - cs_sold_date_sk#5) > 120) THEN 1 ELSE 0 END)] +Aggregate Attributes [5]: [sum#15, sum#16, sum#17, sum#18, sum#19] +Results [8]: [_groupingexpression#14, sm_type#9, cc_name#11, sum#20, sum#21, sum#22, sum#23, sum#24] + +(30) Exchange +Input [8]: [_groupingexpression#14, sm_type#9, cc_name#11, sum#20, sum#21, sum#22, sum#23, sum#24] +Arguments: hashpartitioning(_groupingexpression#14, sm_type#9, cc_name#11, 5), ENSURE_REQUIREMENTS, [plan_id=5] + +(31) HashAggregate [codegen id : 6] +Input [8]: [_groupingexpression#14, sm_type#9, cc_name#11, sum#20, sum#21, sum#22, sum#23, sum#24] +Keys [3]: [_groupingexpression#14, sm_type#9, cc_name#11] +Functions [5]: [sum(CASE WHEN ((cs_ship_date_sk#1 - cs_sold_date_sk#5) <= 30) THEN 1 ELSE 0 END), sum(CASE WHEN (((cs_ship_date_sk#1 - cs_sold_date_sk#5) > 30) AND ((cs_ship_date_sk#1 - cs_sold_date_sk#5) <= 60)) THEN 1 ELSE 0 END), sum(CASE WHEN (((cs_ship_date_sk#1 - cs_sold_date_sk#5) > 60) AND ((cs_ship_date_sk#1 - cs_sold_date_sk#5) <= 90)) THEN 1 ELSE 0 END), sum(CASE WHEN (((cs_ship_date_sk#1 - cs_sold_date_sk#5) > 90) AND ((cs_ship_date_sk#1 - cs_sold_date_sk#5) <= 120)) THEN 1 ELSE 0 END), sum(CASE WHEN ((cs_ship_date_sk#1 - cs_sold_date_sk#5) > 120) THEN 1 ELSE 0 END)] +Aggregate Attributes [5]: [sum(CASE WHEN ((cs_ship_date_sk#1 - cs_sold_date_sk#5) <= 30) THEN 1 ELSE 0 END)#25, sum(CASE WHEN (((cs_ship_date_sk#1 - cs_sold_date_sk#5) > 30) AND ((cs_ship_date_sk#1 - cs_sold_date_sk#5) <= 60)) THEN 1 ELSE 0 END)#26, sum(CASE WHEN (((cs_ship_date_sk#1 - cs_sold_date_sk#5) > 60) AND ((cs_ship_date_sk#1 - cs_sold_date_sk#5) <= 90)) THEN 1 ELSE 0 END)#27, sum(CASE WHEN (((cs_ship_date_sk#1 - cs_sold_date_sk#5) > 90) AND ((cs_ship_date_sk#1 - cs_sold_date_sk#5) <= 120)) THEN 1 ELSE 0 END)#28, sum(CASE WHEN ((cs_ship_date_sk#1 - cs_sold_date_sk#5) > 120) THEN 1 ELSE 0 END)#29] +Results [8]: [_groupingexpression#14 AS substr(w_warehouse_name, 1, 20)#30, sm_type#9, cc_name#11, sum(CASE WHEN ((cs_ship_date_sk#1 - cs_sold_date_sk#5) <= 30) THEN 1 ELSE 0 END)#25 AS 30 days #31, sum(CASE WHEN (((cs_ship_date_sk#1 - cs_sold_date_sk#5) > 30) AND ((cs_ship_date_sk#1 - cs_sold_date_sk#5) <= 60)) THEN 1 ELSE 0 END)#26 AS 31 - 60 days #32, sum(CASE WHEN (((cs_ship_date_sk#1 - cs_sold_date_sk#5) > 60) AND ((cs_ship_date_sk#1 - cs_sold_date_sk#5) <= 90)) THEN 1 ELSE 0 END)#27 AS 61 - 90 days #33, sum(CASE WHEN (((cs_ship_date_sk#1 - cs_sold_date_sk#5) > 90) AND ((cs_ship_date_sk#1 - cs_sold_date_sk#5) <= 120)) THEN 1 ELSE 0 END)#28 AS 91 - 120 days #34, sum(CASE WHEN ((cs_ship_date_sk#1 - cs_sold_date_sk#5) > 120) THEN 1 ELSE 0 END)#29 AS >120 days #35] + +(32) TakeOrderedAndProject +Input [8]: [substr(w_warehouse_name, 1, 20)#30, sm_type#9, cc_name#11, 30 days #31, 31 - 60 days #32, 61 - 90 days #33, 91 - 120 days #34, >120 days #35] +Arguments: 100, [substr(w_warehouse_name, 1, 20)#30 ASC NULLS FIRST, sm_type#9 ASC NULLS FIRST, cc_name#11 ASC NULLS FIRST], [substr(w_warehouse_name, 1, 20)#30, sm_type#9, cc_name#11, 30 days #31, 31 - 60 days #32, 61 - 90 days #33, 91 - 120 days #34, >120 days #35] + diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q99/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q99/simplified.txt new file mode 100644 index 000000000..c5f25f079 --- /dev/null +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q99/simplified.txt @@ -0,0 +1,48 @@ +TakeOrderedAndProject [substr(w_warehouse_name, 1, 20),sm_type,cc_name,30 days ,31 - 60 days ,61 - 90 days ,91 - 120 days ,>120 days ] + WholeStageCodegen (6) + HashAggregate [_groupingexpression,sm_type,cc_name,sum,sum,sum,sum,sum] [sum(CASE WHEN ((cs_ship_date_sk - cs_sold_date_sk) <= 30) THEN 1 ELSE 0 END),sum(CASE WHEN (((cs_ship_date_sk - cs_sold_date_sk) > 30) AND ((cs_ship_date_sk - cs_sold_date_sk) <= 60)) THEN 1 ELSE 0 END),sum(CASE WHEN (((cs_ship_date_sk - cs_sold_date_sk) > 60) AND ((cs_ship_date_sk - cs_sold_date_sk) <= 90)) THEN 1 ELSE 0 END),sum(CASE WHEN (((cs_ship_date_sk - cs_sold_date_sk) > 90) AND ((cs_ship_date_sk - cs_sold_date_sk) <= 120)) THEN 1 ELSE 0 END),sum(CASE WHEN ((cs_ship_date_sk - cs_sold_date_sk) > 120) THEN 1 ELSE 0 END),substr(w_warehouse_name, 1, 20),30 days ,31 - 60 days ,61 - 90 days ,91 - 120 days ,>120 days ,sum,sum,sum,sum,sum] + InputAdapter + Exchange [_groupingexpression,sm_type,cc_name] #1 + WholeStageCodegen (5) + HashAggregate [_groupingexpression,sm_type,cc_name,cs_ship_date_sk,cs_sold_date_sk] [sum,sum,sum,sum,sum,sum,sum,sum,sum,sum] + Project [cs_ship_date_sk,cs_sold_date_sk,sm_type,cc_name,w_warehouse_name] + BroadcastHashJoin [cs_ship_date_sk,d_date_sk] + Project [cs_ship_date_sk,cs_sold_date_sk,w_warehouse_name,sm_type,cc_name] + BroadcastHashJoin [cs_call_center_sk,cc_call_center_sk] + Project [cs_ship_date_sk,cs_call_center_sk,cs_sold_date_sk,w_warehouse_name,sm_type] + BroadcastHashJoin [cs_ship_mode_sk,sm_ship_mode_sk] + Project [cs_ship_date_sk,cs_call_center_sk,cs_ship_mode_sk,cs_sold_date_sk,w_warehouse_name] + BroadcastHashJoin [cs_warehouse_sk,w_warehouse_sk] + ColumnarToRow + InputAdapter + CometFilter [cs_warehouse_sk,cs_ship_mode_sk,cs_call_center_sk,cs_ship_date_sk] + CometScan parquet spark_catalog.default.catalog_sales [cs_ship_date_sk,cs_call_center_sk,cs_ship_mode_sk,cs_warehouse_sk,cs_sold_date_sk] + InputAdapter + BroadcastExchange #2 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometFilter [w_warehouse_sk] + CometScan parquet spark_catalog.default.warehouse [w_warehouse_sk,w_warehouse_name] + InputAdapter + BroadcastExchange #3 + WholeStageCodegen (2) + ColumnarToRow + InputAdapter + CometFilter [sm_ship_mode_sk] + CometScan parquet spark_catalog.default.ship_mode [sm_ship_mode_sk,sm_type] + InputAdapter + BroadcastExchange #4 + WholeStageCodegen (3) + ColumnarToRow + InputAdapter + CometFilter [cc_call_center_sk] + CometScan parquet spark_catalog.default.call_center [cc_call_center_sk,cc_name] + InputAdapter + BroadcastExchange #5 + WholeStageCodegen (4) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_month_seq,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] 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 new file mode 100644 index 000000000..be0e98db2 --- /dev/null +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q10a/explain.txt @@ -0,0 +1,272 @@ +== Physical Plan == +TakeOrderedAndProject (41) ++- * HashAggregate (40) + +- Exchange (39) + +- * HashAggregate (38) + +- * Project (37) + +- * BroadcastHashJoin Inner BuildRight (36) + :- * Project (31) + : +- * BroadcastHashJoin Inner BuildRight (30) + : :- * Project (24) + : : +- * BroadcastHashJoin LeftSemi BuildRight (23) + : : :- * BroadcastHashJoin LeftSemi BuildRight (10) + : : : :- * ColumnarToRow (3) + : : : : +- CometFilter (2) + : : : : +- CometScan parquet spark_catalog.default.customer (1) + : : : +- BroadcastExchange (9) + : : : +- * Project (8) + : : : +- * BroadcastHashJoin Inner BuildRight (7) + : : : :- * ColumnarToRow (5) + : : : : +- CometScan parquet spark_catalog.default.store_sales (4) + : : : +- ReusedExchange (6) + : : +- BroadcastExchange (22) + : : +- Union (21) + : : :- * Project (15) + : : : +- * BroadcastHashJoin Inner BuildRight (14) + : : : :- * ColumnarToRow (12) + : : : : +- CometScan parquet spark_catalog.default.web_sales (11) + : : : +- ReusedExchange (13) + : : +- * Project (20) + : : +- * BroadcastHashJoin Inner BuildRight (19) + : : :- * ColumnarToRow (17) + : : : +- CometScan parquet spark_catalog.default.catalog_sales (16) + : : +- ReusedExchange (18) + : +- BroadcastExchange (29) + : +- * ColumnarToRow (28) + : +- CometProject (27) + : +- CometFilter (26) + : +- CometScan parquet spark_catalog.default.customer_address (25) + +- BroadcastExchange (35) + +- * ColumnarToRow (34) + +- CometFilter (33) + +- CometScan parquet spark_catalog.default.customer_demographics (32) + + +(1) Scan parquet spark_catalog.default.customer +Output [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] +Batched: true +Location [not included in comparison]/{warehouse_dir}/customer] +PushedFilters: [IsNotNull(c_current_addr_sk), IsNotNull(c_current_cdemo_sk)] +ReadSchema: struct<c_customer_sk:int,c_current_cdemo_sk:int,c_current_addr_sk:int> + +(2) CometFilter +Input [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] +Condition : (isnotnull(c_current_addr_sk#3) AND isnotnull(c_current_cdemo_sk#2)) + +(3) ColumnarToRow [codegen id : 9] +Input [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] + +(4) Scan parquet spark_catalog.default.store_sales +Output [2]: [ss_customer_sk#4, ss_sold_date_sk#5] +Batched: true +Location: InMemoryFileIndex [] +PartitionFilters: [isnotnull(ss_sold_date_sk#5), dynamicpruningexpression(ss_sold_date_sk#5 IN dynamicpruning#6)] +ReadSchema: struct<ss_customer_sk:int> + +(5) ColumnarToRow [codegen id : 2] +Input [2]: [ss_customer_sk#4, ss_sold_date_sk#5] + +(6) ReusedExchange [Reuses operator id: 46] +Output [1]: [d_date_sk#7] + +(7) BroadcastHashJoin [codegen id : 2] +Left keys [1]: [ss_sold_date_sk#5] +Right keys [1]: [d_date_sk#7] +Join type: Inner +Join condition: None + +(8) Project [codegen id : 2] +Output [1]: [ss_customer_sk#4] +Input [3]: [ss_customer_sk#4, ss_sold_date_sk#5, d_date_sk#7] + +(9) BroadcastExchange +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] +Left keys [1]: [c_customer_sk#1] +Right keys [1]: [ss_customer_sk#4] +Join type: LeftSemi +Join condition: None + +(11) Scan parquet spark_catalog.default.web_sales +Output [2]: [ws_bill_customer_sk#8, ws_sold_date_sk#9] +Batched: true +Location: InMemoryFileIndex [] +PartitionFilters: [isnotnull(ws_sold_date_sk#9), dynamicpruningexpression(ws_sold_date_sk#9 IN dynamicpruning#10)] +ReadSchema: struct<ws_bill_customer_sk:int> + +(12) ColumnarToRow [codegen id : 4] +Input [2]: [ws_bill_customer_sk#8, ws_sold_date_sk#9] + +(13) ReusedExchange [Reuses operator id: 46] +Output [1]: [d_date_sk#11] + +(14) BroadcastHashJoin [codegen id : 4] +Left keys [1]: [ws_sold_date_sk#9] +Right keys [1]: [d_date_sk#11] +Join type: Inner +Join condition: None + +(15) Project [codegen id : 4] +Output [1]: [ws_bill_customer_sk#8 AS customer_sk#12] +Input [3]: [ws_bill_customer_sk#8, ws_sold_date_sk#9, d_date_sk#11] + +(16) Scan parquet spark_catalog.default.catalog_sales +Output [2]: [cs_ship_customer_sk#13, cs_sold_date_sk#14] +Batched: true +Location: InMemoryFileIndex [] +PartitionFilters: [isnotnull(cs_sold_date_sk#14), dynamicpruningexpression(cs_sold_date_sk#14 IN dynamicpruning#15)] +ReadSchema: struct<cs_ship_customer_sk:int> + +(17) ColumnarToRow [codegen id : 6] +Input [2]: [cs_ship_customer_sk#13, cs_sold_date_sk#14] + +(18) ReusedExchange [Reuses operator id: 46] +Output [1]: [d_date_sk#16] + +(19) BroadcastHashJoin [codegen id : 6] +Left keys [1]: [cs_sold_date_sk#14] +Right keys [1]: [d_date_sk#16] +Join type: Inner +Join condition: None + +(20) Project [codegen id : 6] +Output [1]: [cs_ship_customer_sk#13 AS customer_sk#17] +Input [3]: [cs_ship_customer_sk#13, cs_sold_date_sk#14, d_date_sk#16] + +(21) Union + +(22) BroadcastExchange +Input [1]: [customer_sk#12] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] + +(23) BroadcastHashJoin [codegen id : 9] +Left keys [1]: [c_customer_sk#1] +Right keys [1]: [customer_sk#12] +Join type: LeftSemi +Join condition: None + +(24) Project [codegen id : 9] +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] + +(25) Scan parquet spark_catalog.default.customer_address +Output [2]: [ca_address_sk#18, ca_county#19] +Batched: true +Location [not included in comparison]/{warehouse_dir}/customer_address] +PushedFilters: [In(ca_county, [Dona Ana County,Douglas County,Gaines County,Richland County,Walker County]), IsNotNull(ca_address_sk)] +ReadSchema: struct<ca_address_sk:int,ca_county:string> + +(26) CometFilter +Input [2]: [ca_address_sk#18, ca_county#19] +Condition : (ca_county#19 IN (Walker County,Richland County,Gaines County,Douglas County,Dona Ana County) AND isnotnull(ca_address_sk#18)) + +(27) CometProject +Input [2]: [ca_address_sk#18, ca_county#19] +Arguments: [ca_address_sk#18], [ca_address_sk#18] + +(28) ColumnarToRow [codegen id : 7] +Input [1]: [ca_address_sk#18] + +(29) BroadcastExchange +Input [1]: [ca_address_sk#18] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] + +(30) BroadcastHashJoin [codegen id : 9] +Left keys [1]: [c_current_addr_sk#3] +Right keys [1]: [ca_address_sk#18] +Join type: Inner +Join condition: None + +(31) Project [codegen id : 9] +Output [1]: [c_current_cdemo_sk#2] +Input [3]: [c_current_cdemo_sk#2, c_current_addr_sk#3, ca_address_sk#18] + +(32) Scan parquet spark_catalog.default.customer_demographics +Output [9]: [cd_demo_sk#20, cd_gender#21, cd_marital_status#22, cd_education_status#23, cd_purchase_estimate#24, cd_credit_rating#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] +Batched: true +Location [not included in comparison]/{warehouse_dir}/customer_demographics] +PushedFilters: [IsNotNull(cd_demo_sk)] +ReadSchema: struct<cd_demo_sk:int,cd_gender:string,cd_marital_status:string,cd_education_status:string,cd_purchase_estimate:int,cd_credit_rating:string,cd_dep_count:int,cd_dep_employed_count:int,cd_dep_college_count:int> + +(33) CometFilter +Input [9]: [cd_demo_sk#20, cd_gender#21, cd_marital_status#22, cd_education_status#23, cd_purchase_estimate#24, cd_credit_rating#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] +Condition : isnotnull(cd_demo_sk#20) + +(34) ColumnarToRow [codegen id : 8] +Input [9]: [cd_demo_sk#20, cd_gender#21, cd_marital_status#22, cd_education_status#23, cd_purchase_estimate#24, cd_credit_rating#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] + +(35) BroadcastExchange +Input [9]: [cd_demo_sk#20, cd_gender#21, cd_marital_status#22, cd_education_status#23, cd_purchase_estimate#24, cd_credit_rating#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=4] + +(36) BroadcastHashJoin [codegen id : 9] +Left keys [1]: [c_current_cdemo_sk#2] +Right keys [1]: [cd_demo_sk#20] +Join type: Inner +Join condition: None + +(37) Project [codegen id : 9] +Output [8]: [cd_gender#21, cd_marital_status#22, cd_education_status#23, cd_purchase_estimate#24, cd_credit_rating#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] +Input [10]: [c_current_cdemo_sk#2, cd_demo_sk#20, cd_gender#21, cd_marital_status#22, cd_education_status#23, cd_purchase_estimate#24, cd_credit_rating#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] + +(38) HashAggregate [codegen id : 9] +Input [8]: [cd_gender#21, cd_marital_status#22, cd_education_status#23, cd_purchase_estimate#24, cd_credit_rating#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] +Keys [8]: [cd_gender#21, cd_marital_status#22, cd_education_status#23, cd_purchase_estimate#24, cd_credit_rating#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] +Functions [1]: [partial_count(1)] +Aggregate Attributes [1]: [count#29] +Results [9]: [cd_gender#21, cd_marital_status#22, cd_education_status#23, cd_purchase_estimate#24, cd_credit_rating#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28, count#30] + +(39) Exchange +Input [9]: [cd_gender#21, cd_marital_status#22, cd_education_status#23, cd_purchase_estimate#24, cd_credit_rating#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28, count#30] +Arguments: hashpartitioning(cd_gender#21, cd_marital_status#22, cd_education_status#23, cd_purchase_estimate#24, cd_credit_rating#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28, 5), ENSURE_REQUIREMENTS, [plan_id=5] + +(40) HashAggregate [codegen id : 10] +Input [9]: [cd_gender#21, cd_marital_status#22, cd_education_status#23, cd_purchase_estimate#24, cd_credit_rating#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28, count#30] +Keys [8]: [cd_gender#21, cd_marital_status#22, cd_education_status#23, cd_purchase_estimate#24, cd_credit_rating#25, cd_dep_count#26, cd_dep_employed_count#27, cd_dep_college_count#28] +Functions [1]: [count(1)] +Aggregate Attributes [1]: [count(1)#31] +Results [14]: [cd_gender#21, cd_marital_status#22, cd_education_status#23, count(1)#31 AS cnt1#32, cd_purchase_estimate#24, count(1)#31 AS cnt2#33, cd_credit_rating#25, count(1)#31 AS cnt3#34, cd_dep_count#26, count(1)#31 AS cnt4#35, cd_dep_employed_count#27, count(1)#31 AS cnt5#36, cd_dep_college_count#28, count(1)#31 AS cnt6#37] + +(41) TakeOrderedAndProject +Input [14]: [cd_gender#21, cd_marital_status#22, cd_education_status#23, cnt1#32, cd_purchase_estimate#24, cnt2#33, cd_credit_rating#25, cnt3#34, cd_dep_count#26, cnt4#35, cd_dep_employed_count#27, cnt5#36, cd_dep_college_count#28, cnt6#37] +Arguments: 100, [cd_gender#21 ASC NULLS FIRST, cd_marital_status#22 ASC NULLS FIRST, cd_education_status#23 ASC NULLS FIRST, cd_purchase_estimate#24 ASC NULLS FIRST, cd_credit_rating#25 ASC NULLS FIRST, cd_dep_count#26 ASC NULLS FIRST, cd_dep_employed_count#27 ASC NULLS FIRST, cd_dep_college_count#28 ASC NULLS FIRST], [cd_gender#21, cd_marital_status#22, cd_education_status#23, cnt1#32, cd_purchase_estimate#24, cnt2#33, cd_credit_rating#25, cnt3#34, cd_dep_count#26, cnt4#35, cd_dep_employed_count#27, cnt5#36, cd_dep_college_count#28, cnt6#37] + +===== Subqueries ===== + +Subquery:1 Hosting operator id = 4 Hosting Expression = ss_sold_date_sk#5 IN dynamicpruning#6 +BroadcastExchange (46) ++- * ColumnarToRow (45) + +- CometProject (44) + +- CometFilter (43) + +- CometScan parquet spark_catalog.default.date_dim (42) + + +(42) Scan parquet spark_catalog.default.date_dim +Output [3]: [d_date_sk#7, d_year#38, d_moy#39] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2002), GreaterThanOrEqual(d_moy,4), LessThanOrEqual(d_moy,7), IsNotNull(d_date_sk)] +ReadSchema: struct<d_date_sk:int,d_year:int,d_moy:int> + +(43) CometFilter +Input [3]: [d_date_sk#7, d_year#38, d_moy#39] +Condition : (((((isnotnull(d_year#38) AND isnotnull(d_moy#39)) AND (d_year#38 = 2002)) AND (d_moy#39 >= 4)) AND (d_moy#39 <= 7)) AND isnotnull(d_date_sk#7)) + +(44) CometProject +Input [3]: [d_date_sk#7, d_year#38, d_moy#39] +Arguments: [d_date_sk#7], [d_date_sk#7] + +(45) ColumnarToRow [codegen id : 1] +Input [1]: [d_date_sk#7] + +(46) BroadcastExchange +Input [1]: [d_date_sk#7] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=6] + +Subquery:2 Hosting operator id = 11 Hosting Expression = ws_sold_date_sk#9 IN dynamicpruning#6 + +Subquery:3 Hosting operator id = 16 Hosting Expression = cs_sold_date_sk#14 IN dynamicpruning#6 + + diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q10a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q10a/simplified.txt new file mode 100644 index 000000000..3eb2210a6 --- /dev/null +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q10a/simplified.txt @@ -0,0 +1,72 @@ +TakeOrderedAndProject [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count,cnt1,cnt2,cnt3,cnt4,cnt5,cnt6] + WholeStageCodegen (10) + HashAggregate [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count,count] [count(1),cnt1,cnt2,cnt3,cnt4,cnt5,cnt6,count] + InputAdapter + Exchange [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] #1 + WholeStageCodegen (9) + HashAggregate [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] [count,count] + Project [cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] + BroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] + Project [c_current_cdemo_sk] + BroadcastHashJoin [c_current_addr_sk,ca_address_sk] + Project [c_current_cdemo_sk,c_current_addr_sk] + BroadcastHashJoin [c_customer_sk,customer_sk] + BroadcastHashJoin [c_customer_sk,ss_customer_sk] + ColumnarToRow + InputAdapter + CometFilter [c_current_addr_sk,c_current_cdemo_sk] + CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] + InputAdapter + BroadcastExchange #2 + WholeStageCodegen (2) + Project [ss_customer_sk] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + ColumnarToRow + InputAdapter + CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_year,d_moy,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + InputAdapter + ReusedExchange [d_date_sk] #3 + InputAdapter + BroadcastExchange #4 + Union + WholeStageCodegen (4) + Project [ws_bill_customer_sk] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + ColumnarToRow + InputAdapter + CometScan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + InputAdapter + ReusedExchange [d_date_sk] #3 + WholeStageCodegen (6) + Project [cs_ship_customer_sk] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + ColumnarToRow + InputAdapter + CometScan parquet spark_catalog.default.catalog_sales [cs_ship_customer_sk,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + InputAdapter + ReusedExchange [d_date_sk] #3 + InputAdapter + BroadcastExchange #5 + WholeStageCodegen (7) + ColumnarToRow + InputAdapter + CometProject [ca_address_sk] + CometFilter [ca_county,ca_address_sk] + CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_county] + InputAdapter + BroadcastExchange #6 + WholeStageCodegen (8) + ColumnarToRow + InputAdapter + CometFilter [cd_demo_sk] + CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status,cd_purchase_estimate,cd_credit_rating,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] 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 new file mode 100644 index 000000000..daa1f5243 --- /dev/null +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q11/explain.txt @@ -0,0 +1,477 @@ +== Physical Plan == +TakeOrderedAndProject (71) ++- * Project (70) + +- * BroadcastHashJoin Inner BuildRight (69) + :- * Project (52) + : +- * BroadcastHashJoin Inner BuildRight (51) + : :- * BroadcastHashJoin Inner BuildRight (33) + : : :- * Filter (16) + : : : +- * HashAggregate (15) + : : : +- Exchange (14) + : : : +- * HashAggregate (13) + : : : +- * Project (12) + : : : +- * BroadcastHashJoin Inner BuildRight (11) + : : : :- * Project (9) + : : : : +- * BroadcastHashJoin Inner BuildRight (8) + : : : : :- * ColumnarToRow (3) + : : : : : +- CometFilter (2) + : : : : : +- CometScan parquet spark_catalog.default.customer (1) + : : : : +- BroadcastExchange (7) + : : : : +- * ColumnarToRow (6) + : : : : +- CometFilter (5) + : : : : +- CometScan parquet spark_catalog.default.store_sales (4) + : : : +- ReusedExchange (10) + : : +- BroadcastExchange (32) + : : +- * HashAggregate (31) + : : +- Exchange (30) + : : +- * HashAggregate (29) + : : +- * Project (28) + : : +- * BroadcastHashJoin Inner BuildRight (27) + : : :- * Project (25) + : : : +- * BroadcastHashJoin Inner BuildRight (24) + : : : :- * ColumnarToRow (19) + : : : : +- CometFilter (18) + : : : : +- CometScan parquet spark_catalog.default.customer (17) + : : : +- BroadcastExchange (23) + : : : +- * ColumnarToRow (22) + : : : +- CometFilter (21) + : : : +- CometScan parquet spark_catalog.default.store_sales (20) + : : +- ReusedExchange (26) + : +- BroadcastExchange (50) + : +- * Filter (49) + : +- * HashAggregate (48) + : +- Exchange (47) + : +- * HashAggregate (46) + : +- * Project (45) + : +- * BroadcastHashJoin Inner BuildRight (44) + : :- * Project (42) + : : +- * BroadcastHashJoin Inner BuildRight (41) + : : :- * ColumnarToRow (36) + : : : +- CometFilter (35) + : : : +- CometScan parquet spark_catalog.default.customer (34) + : : +- BroadcastExchange (40) + : : +- * ColumnarToRow (39) + : : +- CometFilter (38) + : : +- CometScan parquet spark_catalog.default.web_sales (37) + : +- ReusedExchange (43) + +- BroadcastExchange (68) + +- * HashAggregate (67) + +- Exchange (66) + +- * HashAggregate (65) + +- * Project (64) + +- * BroadcastHashJoin Inner BuildRight (63) + :- * Project (61) + : +- * BroadcastHashJoin Inner BuildRight (60) + : :- * ColumnarToRow (55) + : : +- CometFilter (54) + : : +- CometScan parquet spark_catalog.default.customer (53) + : +- BroadcastExchange (59) + : +- * ColumnarToRow (58) + : +- CometFilter (57) + : +- CometScan parquet spark_catalog.default.web_sales (56) + +- ReusedExchange (62) + + +(1) Scan parquet spark_catalog.default.customer +Output [8]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8] +Batched: true +Location [not included in comparison]/{warehouse_dir}/customer] +PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_customer_id)] +ReadSchema: struct<c_customer_sk:int,c_customer_id:string,c_first_name:string,c_last_name:string,c_preferred_cust_flag:string,c_birth_country:string,c_login:string,c_email_address:string> + +(2) CometFilter +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] +Condition : (isnotnull(c_customer_sk#1) AND isnotnull(c_customer_id#2)) + +(3) ColumnarToRow [codegen id : 3] +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] + +(4) Scan parquet spark_catalog.default.store_sales +Output [4]: [ss_customer_sk#9, ss_ext_discount_amt#10, ss_ext_list_price#11, ss_sold_date_sk#12] +Batched: true +Location: InMemoryFileIndex [] +PartitionFilters: [isnotnull(ss_sold_date_sk#12), dynamicpruningexpression(ss_sold_date_sk#12 IN dynamicpruning#13)] +PushedFilters: [IsNotNull(ss_customer_sk)] +ReadSchema: struct<ss_customer_sk:int,ss_ext_discount_amt:decimal(7,2),ss_ext_list_price:decimal(7,2)> + +(5) CometFilter +Input [4]: [ss_customer_sk#9, ss_ext_discount_amt#10, ss_ext_list_price#11, ss_sold_date_sk#12] +Condition : isnotnull(ss_customer_sk#9) + +(6) ColumnarToRow [codegen id : 1] +Input [4]: [ss_customer_sk#9, ss_ext_discount_amt#10, ss_ext_list_price#11, ss_sold_date_sk#12] + +(7) BroadcastExchange +Input [4]: [ss_customer_sk#9, ss_ext_discount_amt#10, ss_ext_list_price#11, ss_sold_date_sk#12] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=1] + +(8) BroadcastHashJoin [codegen id : 3] +Left keys [1]: [c_customer_sk#1] +Right keys [1]: [ss_customer_sk#9] +Join type: Inner +Join condition: None + +(9) Project [codegen id : 3] +Output [10]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, ss_ext_discount_amt#10, ss_ext_list_price#11, ss_sold_date_sk#12] +Input [12]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, ss_customer_sk#9, ss_ext_discount_amt#10, ss_ext_list_price#11, ss_sold_date_sk#12] + +(10) ReusedExchange [Reuses operator id: 75] +Output [2]: [d_date_sk#14, d_year#15] + +(11) BroadcastHashJoin [codegen id : 3] +Left keys [1]: [ss_sold_date_sk#12] +Right keys [1]: [d_date_sk#14] +Join type: Inner +Join condition: None + +(12) Project [codegen id : 3] +Output [10]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, ss_ext_discount_amt#10, ss_ext_list_price#11, d_year#15] +Input [12]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, ss_ext_discount_amt#10, ss_ext_list_price#11, ss_sold_date_sk#12, d_date_sk#14, d_year#15] + +(13) HashAggregate [codegen id : 3] +Input [10]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, ss_ext_discount_amt#10, ss_ext_list_price#11, d_year#15] +Keys [8]: [c_customer_id#2, c_first_name#3, c_last_name#4, d_year#15, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8] +Functions [1]: [partial_sum(UnscaledValue((ss_ext_list_price#11 - ss_ext_discount_amt#10)))] +Aggregate Attributes [1]: [sum#16] +Results [9]: [c_customer_id#2, c_first_name#3, c_last_name#4, d_year#15, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, sum#17] + +(14) Exchange +Input [9]: [c_customer_id#2, c_first_name#3, c_last_name#4, d_year#15, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, sum#17] +Arguments: hashpartitioning(c_customer_id#2, c_first_name#3, c_last_name#4, d_year#15, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, 5), ENSURE_REQUIREMENTS, [plan_id=2] + +(15) HashAggregate [codegen id : 16] +Input [9]: [c_customer_id#2, c_first_name#3, c_last_name#4, d_year#15, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, sum#17] +Keys [8]: [c_customer_id#2, c_first_name#3, c_last_name#4, d_year#15, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8] +Functions [1]: [sum(UnscaledValue((ss_ext_list_price#11 - ss_ext_discount_amt#10)))] +Aggregate Attributes [1]: [sum(UnscaledValue((ss_ext_list_price#11 - ss_ext_discount_amt#10)))#18] +Results [2]: [c_customer_id#2 AS customer_id#19, MakeDecimal(sum(UnscaledValue((ss_ext_list_price#11 - ss_ext_discount_amt#10)))#18,18,2) AS year_total#20] + +(16) Filter [codegen id : 16] +Input [2]: [customer_id#19, year_total#20] +Condition : (isnotnull(year_total#20) AND (year_total#20 > 0.00)) + +(17) Scan parquet spark_catalog.default.customer +Output [8]: [c_customer_sk#21, c_customer_id#22, c_first_name#23, c_last_name#24, c_preferred_cust_flag#25, c_birth_country#26, c_login#27, c_email_address#28] +Batched: true +Location [not included in comparison]/{warehouse_dir}/customer] +PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_customer_id)] +ReadSchema: struct<c_customer_sk:int,c_customer_id:string,c_first_name:string,c_last_name:string,c_preferred_cust_flag:string,c_birth_country:string,c_login:string,c_email_address:string> + +(18) CometFilter +Input [8]: [c_customer_sk#21, c_customer_id#22, c_first_name#23, c_last_name#24, c_preferred_cust_flag#25, c_birth_country#26, c_login#27, c_email_address#28] +Condition : (isnotnull(c_customer_sk#21) AND isnotnull(c_customer_id#22)) + +(19) ColumnarToRow [codegen id : 6] +Input [8]: [c_customer_sk#21, c_customer_id#22, c_first_name#23, c_last_name#24, c_preferred_cust_flag#25, c_birth_country#26, c_login#27, c_email_address#28] + +(20) Scan parquet spark_catalog.default.store_sales +Output [4]: [ss_customer_sk#29, ss_ext_discount_amt#30, ss_ext_list_price#31, ss_sold_date_sk#32] +Batched: true +Location: InMemoryFileIndex [] +PartitionFilters: [isnotnull(ss_sold_date_sk#32), dynamicpruningexpression(ss_sold_date_sk#32 IN dynamicpruning#33)] +PushedFilters: [IsNotNull(ss_customer_sk)] +ReadSchema: struct<ss_customer_sk:int,ss_ext_discount_amt:decimal(7,2),ss_ext_list_price:decimal(7,2)> + +(21) CometFilter +Input [4]: [ss_customer_sk#29, ss_ext_discount_amt#30, ss_ext_list_price#31, ss_sold_date_sk#32] +Condition : isnotnull(ss_customer_sk#29) + +(22) ColumnarToRow [codegen id : 4] +Input [4]: [ss_customer_sk#29, ss_ext_discount_amt#30, ss_ext_list_price#31, ss_sold_date_sk#32] + +(23) BroadcastExchange +Input [4]: [ss_customer_sk#29, ss_ext_discount_amt#30, ss_ext_list_price#31, ss_sold_date_sk#32] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=3] + +(24) BroadcastHashJoin [codegen id : 6] +Left keys [1]: [c_customer_sk#21] +Right keys [1]: [ss_customer_sk#29] +Join type: Inner +Join condition: None + +(25) Project [codegen id : 6] +Output [10]: [c_customer_id#22, c_first_name#23, c_last_name#24, c_preferred_cust_flag#25, c_birth_country#26, c_login#27, c_email_address#28, ss_ext_discount_amt#30, ss_ext_list_price#31, ss_sold_date_sk#32] +Input [12]: [c_customer_sk#21, c_customer_id#22, c_first_name#23, c_last_name#24, c_preferred_cust_flag#25, c_birth_country#26, c_login#27, c_email_address#28, ss_customer_sk#29, ss_ext_discount_amt#30, ss_ext_list_price#31, ss_sold_date_sk#32] + +(26) ReusedExchange [Reuses operator id: 79] +Output [2]: [d_date_sk#34, d_year#35] + +(27) BroadcastHashJoin [codegen id : 6] +Left keys [1]: [ss_sold_date_sk#32] +Right keys [1]: [d_date_sk#34] +Join type: Inner +Join condition: None + +(28) Project [codegen id : 6] +Output [10]: [c_customer_id#22, c_first_name#23, c_last_name#24, c_preferred_cust_flag#25, c_birth_country#26, c_login#27, c_email_address#28, ss_ext_discount_amt#30, ss_ext_list_price#31, d_year#35] +Input [12]: [c_customer_id#22, c_first_name#23, c_last_name#24, c_preferred_cust_flag#25, c_birth_country#26, c_login#27, c_email_address#28, ss_ext_discount_amt#30, ss_ext_list_price#31, ss_sold_date_sk#32, d_date_sk#34, d_year#35] + +(29) HashAggregate [codegen id : 6] +Input [10]: [c_customer_id#22, c_first_name#23, c_last_name#24, c_preferred_cust_flag#25, c_birth_country#26, c_login#27, c_email_address#28, ss_ext_discount_amt#30, ss_ext_list_price#31, d_year#35] +Keys [8]: [c_customer_id#22, c_first_name#23, c_last_name#24, d_year#35, c_preferred_cust_flag#25, c_birth_country#26, c_login#27, c_email_address#28] +Functions [1]: [partial_sum(UnscaledValue((ss_ext_list_price#31 - ss_ext_discount_amt#30)))] +Aggregate Attributes [1]: [sum#36] +Results [9]: [c_customer_id#22, c_first_name#23, c_last_name#24, d_year#35, c_preferred_cust_flag#25, c_birth_country#26, c_login#27, c_email_address#28, sum#37] + +(30) Exchange +Input [9]: [c_customer_id#22, c_first_name#23, c_last_name#24, d_year#35, c_preferred_cust_flag#25, c_birth_country#26, c_login#27, c_email_address#28, sum#37] +Arguments: hashpartitioning(c_customer_id#22, c_first_name#23, c_last_name#24, d_year#35, c_preferred_cust_flag#25, c_birth_country#26, c_login#27, c_email_address#28, 5), ENSURE_REQUIREMENTS, [plan_id=4] + +(31) HashAggregate [codegen id : 7] +Input [9]: [c_customer_id#22, c_first_name#23, c_last_name#24, d_year#35, c_preferred_cust_flag#25, c_birth_country#26, c_login#27, c_email_address#28, sum#37] +Keys [8]: [c_customer_id#22, c_first_name#23, c_last_name#24, d_year#35, c_preferred_cust_flag#25, c_birth_country#26, c_login#27, c_email_address#28] +Functions [1]: [sum(UnscaledValue((ss_ext_list_price#31 - ss_ext_discount_amt#30)))] +Aggregate Attributes [1]: [sum(UnscaledValue((ss_ext_list_price#31 - ss_ext_discount_amt#30)))#18] +Results [5]: [c_customer_id#22 AS customer_id#38, c_first_name#23 AS customer_first_name#39, c_last_name#24 AS customer_last_name#40, c_email_address#28 AS customer_email_address#41, MakeDecimal(sum(UnscaledValue((ss_ext_list_price#31 - ss_ext_discount_amt#30)))#18,18,2) AS year_total#42] + +(32) BroadcastExchange +Input [5]: [customer_id#38, customer_first_name#39, customer_last_name#40, customer_email_address#41, year_total#42] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=5] + +(33) BroadcastHashJoin [codegen id : 16] +Left keys [1]: [customer_id#19] +Right keys [1]: [customer_id#38] +Join type: Inner +Join condition: None + +(34) Scan parquet spark_catalog.default.customer +Output [8]: [c_customer_sk#43, c_customer_id#44, c_first_name#45, c_last_name#46, c_preferred_cust_flag#47, c_birth_country#48, c_login#49, c_email_address#50] +Batched: true +Location [not included in comparison]/{warehouse_dir}/customer] +PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_customer_id)] +ReadSchema: struct<c_customer_sk:int,c_customer_id:string,c_first_name:string,c_last_name:string,c_preferred_cust_flag:string,c_birth_country:string,c_login:string,c_email_address:string> + +(35) CometFilter +Input [8]: [c_customer_sk#43, c_customer_id#44, c_first_name#45, c_last_name#46, c_preferred_cust_flag#47, c_birth_country#48, c_login#49, c_email_address#50] +Condition : (isnotnull(c_customer_sk#43) AND isnotnull(c_customer_id#44)) + +(36) ColumnarToRow [codegen id : 10] +Input [8]: [c_customer_sk#43, c_customer_id#44, c_first_name#45, c_last_name#46, c_preferred_cust_flag#47, c_birth_country#48, c_login#49, c_email_address#50] + +(37) Scan parquet spark_catalog.default.web_sales +Output [4]: [ws_bill_customer_sk#51, ws_ext_discount_amt#52, ws_ext_list_price#53, ws_sold_date_sk#54] +Batched: true +Location: InMemoryFileIndex [] +PartitionFilters: [isnotnull(ws_sold_date_sk#54), dynamicpruningexpression(ws_sold_date_sk#54 IN dynamicpruning#55)] +PushedFilters: [IsNotNull(ws_bill_customer_sk)] +ReadSchema: struct<ws_bill_customer_sk:int,ws_ext_discount_amt:decimal(7,2),ws_ext_list_price:decimal(7,2)> + +(38) CometFilter +Input [4]: [ws_bill_customer_sk#51, ws_ext_discount_amt#52, ws_ext_list_price#53, ws_sold_date_sk#54] +Condition : isnotnull(ws_bill_customer_sk#51) + +(39) ColumnarToRow [codegen id : 8] +Input [4]: [ws_bill_customer_sk#51, ws_ext_discount_amt#52, ws_ext_list_price#53, ws_sold_date_sk#54] + +(40) BroadcastExchange +Input [4]: [ws_bill_customer_sk#51, ws_ext_discount_amt#52, ws_ext_list_price#53, ws_sold_date_sk#54] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=6] + +(41) BroadcastHashJoin [codegen id : 10] +Left keys [1]: [c_customer_sk#43] +Right keys [1]: [ws_bill_customer_sk#51] +Join type: Inner +Join condition: None + +(42) Project [codegen id : 10] +Output [10]: [c_customer_id#44, c_first_name#45, c_last_name#46, c_preferred_cust_flag#47, c_birth_country#48, c_login#49, c_email_address#50, ws_ext_discount_amt#52, ws_ext_list_price#53, ws_sold_date_sk#54] +Input [12]: [c_customer_sk#43, c_customer_id#44, c_first_name#45, c_last_name#46, c_preferred_cust_flag#47, c_birth_country#48, c_login#49, c_email_address#50, ws_bill_customer_sk#51, ws_ext_discount_amt#52, ws_ext_list_price#53, ws_sold_date_sk#54] + +(43) ReusedExchange [Reuses operator id: 75] +Output [2]: [d_date_sk#56, d_year#57] + +(44) BroadcastHashJoin [codegen id : 10] +Left keys [1]: [ws_sold_date_sk#54] +Right keys [1]: [d_date_sk#56] +Join type: Inner +Join condition: None + +(45) Project [codegen id : 10] +Output [10]: [c_customer_id#44, c_first_name#45, c_last_name#46, c_preferred_cust_flag#47, c_birth_country#48, c_login#49, c_email_address#50, ws_ext_discount_amt#52, ws_ext_list_price#53, d_year#57] +Input [12]: [c_customer_id#44, c_first_name#45, c_last_name#46, c_preferred_cust_flag#47, c_birth_country#48, c_login#49, c_email_address#50, ws_ext_discount_amt#52, ws_ext_list_price#53, ws_sold_date_sk#54, d_date_sk#56, d_year#57] + +(46) HashAggregate [codegen id : 10] +Input [10]: [c_customer_id#44, c_first_name#45, c_last_name#46, c_preferred_cust_flag#47, c_birth_country#48, c_login#49, c_email_address#50, ws_ext_discount_amt#52, ws_ext_list_price#53, d_year#57] +Keys [8]: [c_customer_id#44, c_first_name#45, c_last_name#46, c_preferred_cust_flag#47, c_birth_country#48, c_login#49, c_email_address#50, d_year#57] +Functions [1]: [partial_sum(UnscaledValue((ws_ext_list_price#53 - ws_ext_discount_amt#52)))] +Aggregate Attributes [1]: [sum#58] +Results [9]: [c_customer_id#44, c_first_name#45, c_last_name#46, c_preferred_cust_flag#47, c_birth_country#48, c_login#49, c_email_address#50, d_year#57, sum#59] + +(47) Exchange +Input [9]: [c_customer_id#44, c_first_name#45, c_last_name#46, c_preferred_cust_flag#47, c_birth_country#48, c_login#49, c_email_address#50, d_year#57, sum#59] +Arguments: hashpartitioning(c_customer_id#44, c_first_name#45, c_last_name#46, c_preferred_cust_flag#47, c_birth_country#48, c_login#49, c_email_address#50, d_year#57, 5), ENSURE_REQUIREMENTS, [plan_id=7] + +(48) HashAggregate [codegen id : 11] +Input [9]: [c_customer_id#44, c_first_name#45, c_last_name#46, c_preferred_cust_flag#47, c_birth_country#48, c_login#49, c_email_address#50, d_year#57, sum#59] +Keys [8]: [c_customer_id#44, c_first_name#45, c_last_name#46, c_preferred_cust_flag#47, c_birth_country#48, c_login#49, c_email_address#50, d_year#57] +Functions [1]: [sum(UnscaledValue((ws_ext_list_price#53 - ws_ext_discount_amt#52)))] +Aggregate Attributes [1]: [sum(UnscaledValue((ws_ext_list_price#53 - ws_ext_discount_amt#52)))#60] +Results [2]: [c_customer_id#44 AS customer_id#61, MakeDecimal(sum(UnscaledValue((ws_ext_list_price#53 - ws_ext_discount_amt#52)))#60,18,2) AS year_total#62] + +(49) Filter [codegen id : 11] +Input [2]: [customer_id#61, year_total#62] +Condition : (isnotnull(year_total#62) AND (year_total#62 > 0.00)) + +(50) BroadcastExchange +Input [2]: [customer_id#61, year_total#62] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=8] + +(51) BroadcastHashJoin [codegen id : 16] +Left keys [1]: [customer_id#19] +Right keys [1]: [customer_id#61] +Join type: Inner +Join condition: None + +(52) Project [codegen id : 16] +Output [8]: [customer_id#19, year_total#20, customer_id#38, customer_first_name#39, customer_last_name#40, customer_email_address#41, year_total#42, year_total#62] +Input [9]: [customer_id#19, year_total#20, customer_id#38, customer_first_name#39, customer_last_name#40, customer_email_address#41, year_total#42, customer_id#61, year_total#62] + +(53) Scan parquet spark_catalog.default.customer +Output [8]: [c_customer_sk#63, c_customer_id#64, c_first_name#65, c_last_name#66, c_preferred_cust_flag#67, c_birth_country#68, c_login#69, c_email_address#70] +Batched: true +Location [not included in comparison]/{warehouse_dir}/customer] +PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_customer_id)] +ReadSchema: struct<c_customer_sk:int,c_customer_id:string,c_first_name:string,c_last_name:string,c_preferred_cust_flag:string,c_birth_country:string,c_login:string,c_email_address:string> + +(54) CometFilter +Input [8]: [c_customer_sk#63, c_customer_id#64, c_first_name#65, c_last_name#66, c_preferred_cust_flag#67, c_birth_country#68, c_login#69, c_email_address#70] +Condition : (isnotnull(c_customer_sk#63) AND isnotnull(c_customer_id#64)) + +(55) ColumnarToRow [codegen id : 14] +Input [8]: [c_customer_sk#63, c_customer_id#64, c_first_name#65, c_last_name#66, c_preferred_cust_flag#67, c_birth_country#68, c_login#69, c_email_address#70] + +(56) Scan parquet spark_catalog.default.web_sales +Output [4]: [ws_bill_customer_sk#71, ws_ext_discount_amt#72, ws_ext_list_price#73, ws_sold_date_sk#74] +Batched: true +Location: InMemoryFileIndex [] +PartitionFilters: [isnotnull(ws_sold_date_sk#74), dynamicpruningexpression(ws_sold_date_sk#74 IN dynamicpruning#75)] +PushedFilters: [IsNotNull(ws_bill_customer_sk)] +ReadSchema: struct<ws_bill_customer_sk:int,ws_ext_discount_amt:decimal(7,2),ws_ext_list_price:decimal(7,2)> + +(57) CometFilter +Input [4]: [ws_bill_customer_sk#71, ws_ext_discount_amt#72, ws_ext_list_price#73, ws_sold_date_sk#74] +Condition : isnotnull(ws_bill_customer_sk#71) + +(58) ColumnarToRow [codegen id : 12] +Input [4]: [ws_bill_customer_sk#71, ws_ext_discount_amt#72, ws_ext_list_price#73, ws_sold_date_sk#74] + +(59) BroadcastExchange +Input [4]: [ws_bill_customer_sk#71, ws_ext_discount_amt#72, ws_ext_list_price#73, ws_sold_date_sk#74] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=9] + +(60) BroadcastHashJoin [codegen id : 14] +Left keys [1]: [c_customer_sk#63] +Right keys [1]: [ws_bill_customer_sk#71] +Join type: Inner +Join condition: None + +(61) Project [codegen id : 14] +Output [10]: [c_customer_id#64, c_first_name#65, c_last_name#66, c_preferred_cust_flag#67, c_birth_country#68, c_login#69, c_email_address#70, ws_ext_discount_amt#72, ws_ext_list_price#73, ws_sold_date_sk#74] +Input [12]: [c_customer_sk#63, c_customer_id#64, c_first_name#65, c_last_name#66, c_preferred_cust_flag#67, c_birth_country#68, c_login#69, c_email_address#70, ws_bill_customer_sk#71, ws_ext_discount_amt#72, ws_ext_list_price#73, ws_sold_date_sk#74] + +(62) ReusedExchange [Reuses operator id: 79] +Output [2]: [d_date_sk#76, d_year#77] + +(63) BroadcastHashJoin [codegen id : 14] +Left keys [1]: [ws_sold_date_sk#74] +Right keys [1]: [d_date_sk#76] +Join type: Inner +Join condition: None + +(64) Project [codegen id : 14] +Output [10]: [c_customer_id#64, c_first_name#65, c_last_name#66, c_preferred_cust_flag#67, c_birth_country#68, c_login#69, c_email_address#70, ws_ext_discount_amt#72, ws_ext_list_price#73, d_year#77] +Input [12]: [c_customer_id#64, c_first_name#65, c_last_name#66, c_preferred_cust_flag#67, c_birth_country#68, c_login#69, c_email_address#70, ws_ext_discount_amt#72, ws_ext_list_price#73, ws_sold_date_sk#74, d_date_sk#76, d_year#77] + +(65) HashAggregate [codegen id : 14] +Input [10]: [c_customer_id#64, c_first_name#65, c_last_name#66, c_preferred_cust_flag#67, c_birth_country#68, c_login#69, c_email_address#70, ws_ext_discount_amt#72, ws_ext_list_price#73, d_year#77] +Keys [8]: [c_customer_id#64, c_first_name#65, c_last_name#66, c_preferred_cust_flag#67, c_birth_country#68, c_login#69, c_email_address#70, d_year#77] +Functions [1]: [partial_sum(UnscaledValue((ws_ext_list_price#73 - ws_ext_discount_amt#72)))] +Aggregate Attributes [1]: [sum#78] +Results [9]: [c_customer_id#64, c_first_name#65, c_last_name#66, c_preferred_cust_flag#67, c_birth_country#68, c_login#69, c_email_address#70, d_year#77, sum#79] + +(66) Exchange +Input [9]: [c_customer_id#64, c_first_name#65, c_last_name#66, c_preferred_cust_flag#67, c_birth_country#68, c_login#69, c_email_address#70, d_year#77, sum#79] +Arguments: hashpartitioning(c_customer_id#64, c_first_name#65, c_last_name#66, c_preferred_cust_flag#67, c_birth_country#68, c_login#69, c_email_address#70, d_year#77, 5), ENSURE_REQUIREMENTS, [plan_id=10] + +(67) HashAggregate [codegen id : 15] +Input [9]: [c_customer_id#64, c_first_name#65, c_last_name#66, c_preferred_cust_flag#67, c_birth_country#68, c_login#69, c_email_address#70, d_year#77, sum#79] +Keys [8]: [c_customer_id#64, c_first_name#65, c_last_name#66, c_preferred_cust_flag#67, c_birth_country#68, c_login#69, c_email_address#70, d_year#77] +Functions [1]: [sum(UnscaledValue((ws_ext_list_price#73 - ws_ext_discount_amt#72)))] +Aggregate Attributes [1]: [sum(UnscaledValue((ws_ext_list_price#73 - ws_ext_discount_amt#72)))#60] +Results [2]: [c_customer_id#64 AS customer_id#80, MakeDecimal(sum(UnscaledValue((ws_ext_list_price#73 - ws_ext_discount_amt#72)))#60,18,2) AS year_total#81] + +(68) BroadcastExchange +Input [2]: [customer_id#80, year_total#81] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=11] + +(69) BroadcastHashJoin [codegen id : 16] +Left keys [1]: [customer_id#19] +Right keys [1]: [customer_id#80] +Join type: Inner +Join condition: (CASE WHEN (year_total#62 > 0.00) THEN (year_total#81 / year_total#62) ELSE 0E-20 END > CASE WHEN (year_total#20 > 0.00) THEN (year_total#42 / year_total#20) ELSE 0E-20 END) + +(70) Project [codegen id : 16] +Output [4]: [customer_id#38, customer_first_name#39, customer_last_name#40, customer_email_address#41] +Input [10]: [customer_id#19, year_total#20, customer_id#38, customer_first_name#39, customer_last_name#40, customer_email_address#41, year_total#42, year_total#62, customer_id#80, year_total#81] + +(71) TakeOrderedAndProject +Input [4]: [customer_id#38, customer_first_name#39, customer_last_name#40, customer_email_address#41] +Arguments: 100, [customer_id#38 ASC NULLS FIRST, customer_first_name#39 ASC NULLS FIRST, customer_last_name#40 ASC NULLS FIRST, customer_email_address#41 ASC NULLS FIRST], [customer_id#38, customer_first_name#39, customer_last_name#40, customer_email_address#41] + +===== Subqueries ===== + +Subquery:1 Hosting operator id = 4 Hosting Expression = ss_sold_date_sk#12 IN dynamicpruning#13 +BroadcastExchange (75) ++- * ColumnarToRow (74) + +- CometFilter (73) + +- CometScan parquet spark_catalog.default.date_dim (72) + + +(72) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#14, d_year#15] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), IsNotNull(d_date_sk)] +ReadSchema: struct<d_date_sk:int,d_year:int> + +(73) CometFilter +Input [2]: [d_date_sk#14, d_year#15] +Condition : ((isnotnull(d_year#15) AND (d_year#15 = 2001)) AND isnotnull(d_date_sk#14)) + +(74) ColumnarToRow [codegen id : 1] +Input [2]: [d_date_sk#14, d_year#15] + +(75) BroadcastExchange +Input [2]: [d_date_sk#14, d_year#15] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=12] + +Subquery:2 Hosting operator id = 20 Hosting Expression = ss_sold_date_sk#32 IN dynamicpruning#33 +BroadcastExchange (79) ++- * ColumnarToRow (78) + +- CometFilter (77) + +- CometScan parquet spark_catalog.default.date_dim (76) + + +(76) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#34, d_year#35] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2002), IsNotNull(d_date_sk)] +ReadSchema: struct<d_date_sk:int,d_year:int> + +(77) CometFilter +Input [2]: [d_date_sk#34, d_year#35] +Condition : ((isnotnull(d_year#35) AND (d_year#35 = 2002)) AND isnotnull(d_date_sk#34)) + +(78) ColumnarToRow [codegen id : 1] +Input [2]: [d_date_sk#34, d_year#35] + +(79) BroadcastExchange +Input [2]: [d_date_sk#34, d_year#35] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=13] + +Subquery:3 Hosting operator id = 37 Hosting Expression = ws_sold_date_sk#54 IN dynamicpruning#13 + +Subquery:4 Hosting operator id = 56 Hosting Expression = ws_sold_date_sk#74 IN dynamicpruning#33 + + diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q11/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q11/simplified.txt new file mode 100644 index 000000000..0a30aba05 --- /dev/null +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q11/simplified.txt @@ -0,0 +1,122 @@ +TakeOrderedAndProject [customer_id,customer_first_name,customer_last_name,customer_email_address] + WholeStageCodegen (16) + Project [customer_id,customer_first_name,customer_last_name,customer_email_address] + BroadcastHashJoin [customer_id,customer_id,year_total,year_total,year_total,year_total] + Project [customer_id,year_total,customer_id,customer_first_name,customer_last_name,customer_email_address,year_total,year_total] + BroadcastHashJoin [customer_id,customer_id] + BroadcastHashJoin [customer_id,customer_id] + Filter [year_total] + HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,sum] [sum(UnscaledValue((ss_ext_list_price - ss_ext_discount_amt))),customer_id,year_total,sum] + InputAdapter + Exchange [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] #1 + WholeStageCodegen (3) + HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_list_price,ss_ext_discount_amt] [sum,sum] + Project [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_discount_amt,ss_ext_list_price,d_year] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] + BroadcastHashJoin [c_customer_sk,ss_customer_sk] + ColumnarToRow + InputAdapter + CometFilter [c_customer_sk,c_customer_id] + CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] + InputAdapter + BroadcastExchange #2 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometFilter [ss_customer_sk] + CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometFilter [d_year,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + InputAdapter + ReusedExchange [d_date_sk,d_year] #3 + InputAdapter + BroadcastExchange #4 + WholeStageCodegen (7) + HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,sum] [sum(UnscaledValue((ss_ext_list_price - ss_ext_discount_amt))),customer_id,customer_first_name,customer_last_name,customer_email_address,year_total,sum] + InputAdapter + Exchange [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] #5 + WholeStageCodegen (6) + HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_list_price,ss_ext_discount_amt] [sum,sum] + Project [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_discount_amt,ss_ext_list_price,d_year] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] + BroadcastHashJoin [c_customer_sk,ss_customer_sk] + ColumnarToRow + InputAdapter + CometFilter [c_customer_sk,c_customer_id] + CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] + InputAdapter + BroadcastExchange #6 + WholeStageCodegen (4) + ColumnarToRow + InputAdapter + CometFilter [ss_customer_sk] + CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_ext_discount_amt,ss_ext_list_price,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #2 + BroadcastExchange #7 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometFilter [d_year,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + InputAdapter + ReusedExchange [d_date_sk,d_year] #7 + InputAdapter + BroadcastExchange #8 + WholeStageCodegen (11) + Filter [year_total] + HashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum] [sum(UnscaledValue((ws_ext_list_price - ws_ext_discount_amt))),customer_id,year_total,sum] + InputAdapter + Exchange [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year] #9 + WholeStageCodegen (10) + HashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,ws_ext_list_price,ws_ext_discount_amt] [sum,sum] + Project [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_list_price,d_year] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + Project [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] + BroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] + ColumnarToRow + InputAdapter + CometFilter [c_customer_sk,c_customer_id] + CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] + InputAdapter + BroadcastExchange #10 + WholeStageCodegen (8) + ColumnarToRow + InputAdapter + CometFilter [ws_bill_customer_sk] + CometScan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + InputAdapter + ReusedExchange [d_date_sk,d_year] #3 + InputAdapter + BroadcastExchange #11 + WholeStageCodegen (15) + HashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum] [sum(UnscaledValue((ws_ext_list_price - ws_ext_discount_amt))),customer_id,year_total,sum] + InputAdapter + Exchange [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year] #12 + WholeStageCodegen (14) + HashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,ws_ext_list_price,ws_ext_discount_amt] [sum,sum] + Project [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_list_price,d_year] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + Project [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] + BroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] + ColumnarToRow + InputAdapter + CometFilter [c_customer_sk,c_customer_id] + CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] + InputAdapter + BroadcastExchange #13 + WholeStageCodegen (12) + ColumnarToRow + InputAdapter + CometFilter [ws_bill_customer_sk] + CometScan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_ext_discount_amt,ws_ext_list_price,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #2 + InputAdapter + ReusedExchange [d_date_sk,d_year] #7 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 new file mode 100644 index 000000000..c39a71879 --- /dev/null +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q12/explain.txt @@ -0,0 +1,150 @@ +== Physical Plan == +TakeOrderedAndProject (20) ++- * Project (19) + +- Window (18) + +- * Sort (17) + +- Exchange (16) + +- * HashAggregate (15) + +- Exchange (14) + +- * HashAggregate (13) + +- * Project (12) + +- * BroadcastHashJoin Inner BuildRight (11) + :- * Project (9) + : +- * BroadcastHashJoin Inner BuildRight (8) + : :- * ColumnarToRow (3) + : : +- CometFilter (2) + : : +- CometScan parquet spark_catalog.default.web_sales (1) + : +- BroadcastExchange (7) + : +- * ColumnarToRow (6) + : +- CometFilter (5) + : +- CometScan parquet spark_catalog.default.item (4) + +- ReusedExchange (10) + + +(1) Scan parquet spark_catalog.default.web_sales +Output [3]: [ws_item_sk#1, ws_ext_sales_price#2, ws_sold_date_sk#3] +Batched: true +Location: InMemoryFileIndex [] +PartitionFilters: [isnotnull(ws_sold_date_sk#3), dynamicpruningexpression(ws_sold_date_sk#3 IN dynamicpruning#4)] +PushedFilters: [IsNotNull(ws_item_sk)] +ReadSchema: struct<ws_item_sk:int,ws_ext_sales_price:decimal(7,2)> + +(2) CometFilter +Input [3]: [ws_item_sk#1, ws_ext_sales_price#2, ws_sold_date_sk#3] +Condition : isnotnull(ws_item_sk#1) + +(3) ColumnarToRow [codegen id : 3] +Input [3]: [ws_item_sk#1, ws_ext_sales_price#2, ws_sold_date_sk#3] + +(4) Scan parquet spark_catalog.default.item +Output [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] +Batched: true +Location [not included in comparison]/{warehouse_dir}/item] +PushedFilters: [In(i_category, [Books ,Home ,Sports ]), IsNotNull(i_item_sk)] +ReadSchema: struct<i_item_sk:int,i_item_id:string,i_item_desc:string,i_current_price:decimal(7,2),i_class:string,i_category:string> + +(5) CometFilter +Input [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] +Condition : (i_category#10 IN (Sports ,Books ,Home ) AND isnotnull(i_item_sk#5)) + +(6) ColumnarToRow [codegen id : 1] +Input [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] + +(7) BroadcastExchange +Input [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=1] + +(8) BroadcastHashJoin [codegen id : 3] +Left keys [1]: [ws_item_sk#1] +Right keys [1]: [i_item_sk#5] +Join type: Inner +Join condition: None + +(9) Project [codegen id : 3] +Output [7]: [ws_ext_sales_price#2, ws_sold_date_sk#3, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] +Input [9]: [ws_item_sk#1, ws_ext_sales_price#2, ws_sold_date_sk#3, i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] + +(10) ReusedExchange [Reuses operator id: 25] +Output [1]: [d_date_sk#11] + +(11) BroadcastHashJoin [codegen id : 3] +Left keys [1]: [ws_sold_date_sk#3] +Right keys [1]: [d_date_sk#11] +Join type: Inner +Join condition: None + +(12) Project [codegen id : 3] +Output [6]: [ws_ext_sales_price#2, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] +Input [8]: [ws_ext_sales_price#2, ws_sold_date_sk#3, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10, d_date_sk#11] + +(13) HashAggregate [codegen id : 3] +Input [6]: [ws_ext_sales_price#2, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] +Keys [5]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8] +Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#2))] +Aggregate Attributes [1]: [sum#12] +Results [6]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, sum#13] + +(14) Exchange +Input [6]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, sum#13] +Arguments: hashpartitioning(i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, 5), ENSURE_REQUIREMENTS, [plan_id=2] + +(15) HashAggregate [codegen id : 4] +Input [6]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, sum#13] +Keys [5]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8] +Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#2))] +Aggregate Attributes [1]: [sum(UnscaledValue(ws_ext_sales_price#2))#14] +Results [7]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#2))#14,17,2) AS itemrevenue#15, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#2))#14,17,2) AS _w0#16] + +(16) Exchange +Input [7]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#15, _w0#16] +Arguments: hashpartitioning(i_class#9, 5), ENSURE_REQUIREMENTS, [plan_id=3] + +(17) Sort [codegen id : 5] +Input [7]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#15, _w0#16] +Arguments: [i_class#9 ASC NULLS FIRST], false, 0 + +(18) Window +Input [7]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#15, _w0#16] +Arguments: [sum(_w0#16) windowspecdefinition(i_class#9, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS _we0#17], [i_class#9] + +(19) Project [codegen id : 6] +Output [7]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#15, ((_w0#16 * 100) / _we0#17) AS revenueratio#18] +Input [8]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#15, _w0#16, _we0#17] + +(20) TakeOrderedAndProject +Input [7]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#15, revenueratio#18] +Arguments: 100, [i_category#10 ASC NULLS FIRST, i_class#9 ASC NULLS FIRST, i_item_id#6 ASC NULLS FIRST, i_item_desc#7 ASC NULLS FIRST, revenueratio#18 ASC NULLS FIRST], [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#15, revenueratio#18] + +===== Subqueries ===== + +Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#3 IN dynamicpruning#4 +BroadcastExchange (25) ++- * ColumnarToRow (24) + +- CometProject (23) + +- CometFilter (22) + +- CometScan parquet spark_catalog.default.date_dim (21) + + +(21) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#11, d_date#19] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,1999-02-22), LessThanOrEqual(d_date,1999-03-24), IsNotNull(d_date_sk)] +ReadSchema: struct<d_date_sk:int,d_date:date> + +(22) CometFilter +Input [2]: [d_date_sk#11, d_date#19] +Condition : (((isnotnull(d_date#19) AND (d_date#19 >= 1999-02-22)) AND (d_date#19 <= 1999-03-24)) AND isnotnull(d_date_sk#11)) + +(23) CometProject +Input [2]: [d_date_sk#11, d_date#19] +Arguments: [d_date_sk#11], [d_date_sk#11] + +(24) ColumnarToRow [codegen id : 1] +Input [1]: [d_date_sk#11] + +(25) BroadcastExchange +Input [1]: [d_date_sk#11] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] + + diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q12/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q12/simplified.txt new file mode 100644 index 000000000..1bc2538b4 --- /dev/null +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q12/simplified.txt @@ -0,0 +1,40 @@ +TakeOrderedAndProject [i_category,i_class,i_item_id,i_item_desc,revenueratio,i_current_price,itemrevenue] + 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) + Sort [i_class] + InputAdapter + Exchange [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] + InputAdapter + Exchange [i_item_id,i_item_desc,i_category,i_class,i_current_price] #2 + WholeStageCodegen (3) + HashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,ws_ext_sales_price] [sum,sum] + Project [ws_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + Project [ws_ext_sales_price,ws_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + BroadcastHashJoin [ws_item_sk,i_item_sk] + ColumnarToRow + InputAdapter + CometFilter [ws_item_sk] + CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_ext_sales_price,ws_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] + InputAdapter + BroadcastExchange #4 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometFilter [i_category,i_item_sk] + CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + InputAdapter + ReusedExchange [d_date_sk] #3 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 new file mode 100644 index 000000000..ec52cea9f --- /dev/null +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q14/explain.txt @@ -0,0 +1,759 @@ +== Physical Plan == +TakeOrderedAndProject (84) ++- * BroadcastHashJoin Inner BuildRight (83) + :- * Filter (66) + : +- * HashAggregate (65) + : +- Exchange (64) + : +- * HashAggregate (63) + : +- * Project (62) + : +- * BroadcastHashJoin Inner BuildRight (61) + : :- * Project (59) + : : +- * BroadcastHashJoin Inner BuildRight (58) + : : :- * BroadcastHashJoin LeftSemi BuildRight (51) + : : : :- * ColumnarToRow (3) + : : : : +- CometFilter (2) + : : : : +- CometScan parquet spark_catalog.default.store_sales (1) + : : : +- BroadcastExchange (50) + : : : +- * Project (49) + : : : +- * BroadcastHashJoin Inner BuildRight (48) + : : : :- * ColumnarToRow (6) + : : : : +- CometFilter (5) + : : : : +- CometScan parquet spark_catalog.default.item (4) + : : : +- BroadcastExchange (47) + : : : +- * BroadcastHashJoin LeftSemi BuildRight (46) + : : : :- * HashAggregate (35) + : : : : +- Exchange (34) + : : : : +- * HashAggregate (33) + : : : : +- * Project (32) + : : : : +- * BroadcastHashJoin Inner BuildRight (31) + : : : : :- * Project (29) + : : : : : +- * BroadcastHashJoin Inner BuildRight (28) + : : : : : :- * ColumnarToRow (9) + : : : : : : +- CometFilter (8) + : : : : : : +- CometScan parquet spark_catalog.default.store_sales (7) + : : : : : +- BroadcastExchange (27) + : : : : : +- * BroadcastHashJoin LeftSemi BuildRight (26) + : : : : : :- * ColumnarToRow (12) + : : : : : : +- CometFilter (11) + : : : : : : +- CometScan parquet spark_catalog.default.item (10) + : : : : : +- BroadcastExchange (25) + : : : : : +- * Project (24) + : : : : : +- * BroadcastHashJoin Inner BuildRight (23) + : : : : : :- * Project (21) + : : : : : : +- * BroadcastHashJoin Inner BuildRight (20) + : : : : : : :- * ColumnarToRow (15) + : : : : : : : +- CometFilter (14) + : : : : : : : +- CometScan parquet spark_catalog.default.catalog_sales (13) + : : : : : : +- BroadcastExchange (19) + : : : : : : +- * ColumnarToRow (18) + : : : : : : +- CometFilter (17) + : : : : : : +- CometScan parquet spark_catalog.default.item (16) + : : : : : +- ReusedExchange (22) + : : : : +- ReusedExchange (30) + : : : +- BroadcastExchange (45) + : : : +- * Project (44) + : : : +- * BroadcastHashJoin Inner BuildRight (43) + : : : :- * Project (41) + : : : : +- * BroadcastHashJoin Inner BuildRight (40) + : : : : :- * ColumnarToRow (38) + : : : : : +- CometFilter (37) + : : : : : +- CometScan parquet spark_catalog.default.web_sales (36) + : : : : +- ReusedExchange (39) + : : : +- ReusedExchange (42) + : : +- BroadcastExchange (57) + : : +- * BroadcastHashJoin LeftSemi BuildRight (56) + : : :- * ColumnarToRow (54) + : : : +- CometFilter (53) + : : : +- CometScan parquet spark_catalog.default.item (52) + : : +- ReusedExchange (55) + : +- ReusedExchange (60) + +- BroadcastExchange (82) + +- * Filter (81) + +- * HashAggregate (80) + +- Exchange (79) + +- * HashAggregate (78) + +- * Project (77) + +- * BroadcastHashJoin Inner BuildRight (76) + :- * Project (74) + : +- * BroadcastHashJoin Inner BuildRight (73) + : :- * BroadcastHashJoin LeftSemi BuildRight (71) + : : :- * ColumnarToRow (69) + : : : +- CometFilter (68) + : : : +- CometScan parquet spark_catalog.default.store_sales (67) + : : +- ReusedExchange (70) + : +- ReusedExchange (72) + +- ReusedExchange (75) + + +(1) Scan parquet spark_catalog.default.store_sales +Output [4]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] +Batched: true +Location: InMemoryFileIndex [] +PartitionFilters: [isnotnull(ss_sold_date_sk#4), dynamicpruningexpression(ss_sold_date_sk#4 IN dynamicpruning#5)] +PushedFilters: [IsNotNull(ss_item_sk)] +ReadSchema: struct<ss_item_sk:int,ss_quantity:int,ss_list_price:decimal(7,2)> + +(2) CometFilter +Input [4]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] +Condition : isnotnull(ss_item_sk#1) + +(3) ColumnarToRow [codegen id : 25] +Input [4]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] + +(4) Scan parquet spark_catalog.default.item +Output [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] +Batched: true +Location [not included in comparison]/{warehouse_dir}/item] +PushedFilters: [IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id)] +ReadSchema: struct<i_item_sk:int,i_brand_id:int,i_class_id:int,i_category_id:int> + +(5) CometFilter +Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] +Condition : ((isnotnull(i_brand_id#7) AND isnotnull(i_class_id#8)) AND isnotnull(i_category_id#9)) + +(6) ColumnarToRow [codegen id : 11] +Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] + +(7) Scan parquet spark_catalog.default.store_sales +Output [2]: [ss_item_sk#10, ss_sold_date_sk#11] +Batched: true +Location: InMemoryFileIndex [] +PartitionFilters: [isnotnull(ss_sold_date_sk#11), dynamicpruningexpression(ss_sold_date_sk#11 IN dynamicpruning#12)] +PushedFilters: [IsNotNull(ss_item_sk)] +ReadSchema: struct<ss_item_sk:int> + +(8) CometFilter +Input [2]: [ss_item_sk#10, ss_sold_date_sk#11] +Condition : isnotnull(ss_item_sk#10) + +(9) ColumnarToRow [codegen id : 6] +Input [2]: [ss_item_sk#10, ss_sold_date_sk#11] + +(10) Scan parquet spark_catalog.default.item +Output [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] +Batched: true +Location [not included in comparison]/{warehouse_dir}/item] +PushedFilters: [IsNotNull(i_item_sk), IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id)] +ReadSchema: struct<i_item_sk:int,i_brand_id:int,i_class_id:int,i_category_id:int> + +(11) CometFilter +Input [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] +Condition : (((isnotnull(i_item_sk#13) AND isnotnull(i_brand_id#14)) AND isnotnull(i_class_id#15)) AND isnotnull(i_category_id#16)) + +(12) ColumnarToRow [codegen id : 4] +Input [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] + +(13) Scan parquet spark_catalog.default.catalog_sales +Output [2]: [cs_item_sk#17, cs_sold_date_sk#18] +Batched: true +Location: InMemoryFileIndex [] +PartitionFilters: [isnotnull(cs_sold_date_sk#18), dynamicpruningexpression(cs_sold_date_sk#18 IN dynamicpruning#19)] +PushedFilters: [IsNotNull(cs_item_sk)] +ReadSchema: struct<cs_item_sk:int> + +(14) CometFilter +Input [2]: [cs_item_sk#17, cs_sold_date_sk#18] +Condition : isnotnull(cs_item_sk#17) + +(15) ColumnarToRow [codegen id : 3] +Input [2]: [cs_item_sk#17, cs_sold_date_sk#18] + +(16) Scan parquet spark_catalog.default.item +Output [4]: [i_item_sk#20, i_brand_id#21, i_class_id#22, i_category_id#23] +Batched: true +Location [not included in comparison]/{warehouse_dir}/item] +PushedFilters: [IsNotNull(i_item_sk)] +ReadSchema: struct<i_item_sk:int,i_brand_id:int,i_class_id:int,i_category_id:int> + +(17) CometFilter +Input [4]: [i_item_sk#20, i_brand_id#21, i_class_id#22, i_category_id#23] +Condition : isnotnull(i_item_sk#20) + +(18) ColumnarToRow [codegen id : 1] +Input [4]: [i_item_sk#20, i_brand_id#21, i_class_id#22, i_category_id#23] + +(19) BroadcastExchange +Input [4]: [i_item_sk#20, i_brand_id#21, i_class_id#22, i_category_id#23] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=1] + +(20) BroadcastHashJoin [codegen id : 3] +Left keys [1]: [cs_item_sk#17] +Right keys [1]: [i_item_sk#20] +Join type: Inner +Join condition: None + +(21) Project [codegen id : 3] +Output [4]: [cs_sold_date_sk#18, i_brand_id#21, i_class_id#22, i_category_id#23] +Input [6]: [cs_item_sk#17, cs_sold_date_sk#18, i_item_sk#20, i_brand_id#21, i_class_id#22, i_category_id#23] + +(22) ReusedExchange [Reuses operator id: 117] +Output [1]: [d_date_sk#24] + +(23) BroadcastHashJoin [codegen id : 3] +Left keys [1]: [cs_sold_date_sk#18] +Right keys [1]: [d_date_sk#24] +Join type: Inner +Join condition: None + +(24) Project [codegen id : 3] +Output [3]: [i_brand_id#21, i_class_id#22, i_category_id#23] +Input [5]: [cs_sold_date_sk#18, i_brand_id#21, i_class_id#22, i_category_id#23, d_date_sk#24] + +(25) BroadcastExchange +Input [3]: [i_brand_id#21, i_class_id#22, i_category_id#23] +Arguments: HashedRelationBroadcastMode(List(coalesce(input[0, int, true], 0), isnull(input[0, int, true]), coalesce(input[1, int, true], 0), isnull(input[1, int, true]), coalesce(input[2, int, true], 0), isnull(input[2, int, true])),false), [plan_id=2] + +(26) BroadcastHashJoin [codegen id : 4] +Left keys [6]: [coalesce(i_brand_id#14, 0), isnull(i_brand_id#14), coalesce(i_class_id#15, 0), isnull(i_class_id#15), coalesce(i_category_id#16, 0), isnull(i_category_id#16)] +Right keys [6]: [coalesce(i_brand_id#21, 0), isnull(i_brand_id#21), coalesce(i_class_id#22, 0), isnull(i_class_id#22), coalesce(i_category_id#23, 0), isnull(i_category_id#23)] +Join type: LeftSemi +Join condition: None + +(27) BroadcastExchange +Input [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=3] + +(28) BroadcastHashJoin [codegen id : 6] +Left keys [1]: [ss_item_sk#10] +Right keys [1]: [i_item_sk#13] +Join type: Inner +Join condition: None + +(29) Project [codegen id : 6] +Output [4]: [ss_sold_date_sk#11, i_brand_id#14, i_class_id#15, i_category_id#16] +Input [6]: [ss_item_sk#10, ss_sold_date_sk#11, i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] + +(30) ReusedExchange [Reuses operator id: 117] +Output [1]: [d_date_sk#25] + +(31) BroadcastHashJoin [codegen id : 6] +Left keys [1]: [ss_sold_date_sk#11] +Right keys [1]: [d_date_sk#25] +Join type: Inner +Join condition: None + +(32) Project [codegen id : 6] +Output [3]: [i_brand_id#14 AS brand_id#26, i_class_id#15 AS class_id#27, i_category_id#16 AS category_id#28] +Input [5]: [ss_sold_date_sk#11, i_brand_id#14, i_class_id#15, i_category_id#16, d_date_sk#25] + +(33) HashAggregate [codegen id : 6] +Input [3]: [brand_id#26, class_id#27, category_id#28] +Keys [3]: [brand_id#26, class_id#27, category_id#28] +Functions: [] +Aggregate Attributes: [] +Results [3]: [brand_id#26, class_id#27, category_id#28] + +(34) Exchange +Input [3]: [brand_id#26, class_id#27, category_id#28] +Arguments: hashpartitioning(brand_id#26, class_id#27, category_id#28, 5), ENSURE_REQUIREMENTS, [plan_id=4] + +(35) HashAggregate [codegen id : 10] +Input [3]: [brand_id#26, class_id#27, category_id#28] +Keys [3]: [brand_id#26, class_id#27, category_id#28] +Functions: [] +Aggregate Attributes: [] +Results [3]: [brand_id#26, class_id#27, category_id#28] + +(36) Scan parquet spark_catalog.default.web_sales +Output [2]: [ws_item_sk#29, ws_sold_date_sk#30] +Batched: true +Location: InMemoryFileIndex [] +PartitionFilters: [isnotnull(ws_sold_date_sk#30), dynamicpruningexpression(ws_sold_date_sk#30 IN dynamicpruning#31)] +PushedFilters: [IsNotNull(ws_item_sk)] +ReadSchema: struct<ws_item_sk:int> + +(37) CometFilter +Input [2]: [ws_item_sk#29, ws_sold_date_sk#30] +Condition : isnotnull(ws_item_sk#29) + +(38) ColumnarToRow [codegen id : 9] +Input [2]: [ws_item_sk#29, ws_sold_date_sk#30] + +(39) ReusedExchange [Reuses operator id: 19] +Output [4]: [i_item_sk#32, i_brand_id#33, i_class_id#34, i_category_id#35] + +(40) BroadcastHashJoin [codegen id : 9] +Left keys [1]: [ws_item_sk#29] +Right keys [1]: [i_item_sk#32] +Join type: Inner +Join condition: None + +(41) Project [codegen id : 9] +Output [4]: [ws_sold_date_sk#30, i_brand_id#33, i_class_id#34, i_category_id#35] +Input [6]: [ws_item_sk#29, ws_sold_date_sk#30, i_item_sk#32, i_brand_id#33, i_class_id#34, i_category_id#35] + +(42) ReusedExchange [Reuses operator id: 117] +Output [1]: [d_date_sk#36] + +(43) BroadcastHashJoin [codegen id : 9] +Left keys [1]: [ws_sold_date_sk#30] +Right keys [1]: [d_date_sk#36] +Join type: Inner +Join condition: None + +(44) Project [codegen id : 9] +Output [3]: [i_brand_id#33, i_class_id#34, i_category_id#35] +Input [5]: [ws_sold_date_sk#30, i_brand_id#33, i_class_id#34, i_category_id#35, d_date_sk#36] + +(45) BroadcastExchange +Input [3]: [i_brand_id#33, i_class_id#34, i_category_id#35] +Arguments: HashedRelationBroadcastMode(List(coalesce(input[0, int, true], 0), isnull(input[0, int, true]), coalesce(input[1, int, true], 0), isnull(input[1, int, true]), coalesce(input[2, int, true], 0), isnull(input[2, int, true])),false), [plan_id=5] + +(46) BroadcastHashJoin [codegen id : 10] +Left keys [6]: [coalesce(brand_id#26, 0), isnull(brand_id#26), coalesce(class_id#27, 0), isnull(class_id#27), coalesce(category_id#28, 0), isnull(category_id#28)] +Right keys [6]: [coalesce(i_brand_id#33, 0), isnull(i_brand_id#33), coalesce(i_class_id#34, 0), isnull(i_class_id#34), coalesce(i_category_id#35, 0), isnull(i_category_id#35)] +Join type: LeftSemi +Join condition: None + +(47) BroadcastExchange +Input [3]: [brand_id#26, class_id#27, category_id#28] +Arguments: HashedRelationBroadcastMode(List(input[0, int, true], input[1, int, true], input[2, int, true]),false), [plan_id=6] + +(48) BroadcastHashJoin [codegen id : 11] +Left keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] +Right keys [3]: [brand_id#26, class_id#27, category_id#28] +Join type: Inner +Join condition: None + +(49) Project [codegen id : 11] +Output [1]: [i_item_sk#6 AS ss_item_sk#37] +Input [7]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9, brand_id#26, class_id#27, category_id#28] + +(50) BroadcastExchange +Input [1]: [ss_item_sk#37] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=7] + +(51) BroadcastHashJoin [codegen id : 25] +Left keys [1]: [ss_item_sk#1] +Right keys [1]: [ss_item_sk#37] +Join type: LeftSemi +Join condition: None + +(52) Scan parquet spark_catalog.default.item +Output [4]: [i_item_sk#38, i_brand_id#39, i_class_id#40, i_category_id#41] +Batched: true +Location [not included in comparison]/{warehouse_dir}/item] +PushedFilters: [IsNotNull(i_item_sk), IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id)] +ReadSchema: struct<i_item_sk:int,i_brand_id:int,i_class_id:int,i_category_id:int> + +(53) CometFilter +Input [4]: [i_item_sk#38, i_brand_id#39, i_class_id#40, i_category_id#41] +Condition : (((isnotnull(i_item_sk#38) AND isnotnull(i_brand_id#39)) AND isnotnull(i_class_id#40)) AND isnotnull(i_category_id#41)) + +(54) ColumnarToRow [codegen id : 23] +Input [4]: [i_item_sk#38, i_brand_id#39, i_class_id#40, i_category_id#41] + +(55) ReusedExchange [Reuses operator id: 50] +Output [1]: [ss_item_sk#37] + +(56) BroadcastHashJoin [codegen id : 23] +Left keys [1]: [i_item_sk#38] +Right keys [1]: [ss_item_sk#37] +Join type: LeftSemi +Join condition: None + +(57) BroadcastExchange +Input [4]: [i_item_sk#38, i_brand_id#39, i_class_id#40, i_category_id#41] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=8] + +(58) BroadcastHashJoin [codegen id : 25] +Left keys [1]: [ss_item_sk#1] +Right keys [1]: [i_item_sk#38] +Join type: Inner +Join condition: None + +(59) Project [codegen id : 25] +Output [6]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#39, i_class_id#40, i_category_id#41] +Input [8]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_item_sk#38, i_brand_id#39, i_class_id#40, i_category_id#41] + +(60) ReusedExchange [Reuses operator id: 108] +Output [1]: [d_date_sk#42] + +(61) BroadcastHashJoin [codegen id : 25] +Left keys [1]: [ss_sold_date_sk#4] +Right keys [1]: [d_date_sk#42] +Join type: Inner +Join condition: None + +(62) Project [codegen id : 25] +Output [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#39, i_class_id#40, i_category_id#41] +Input [7]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#39, i_class_id#40, i_category_id#41, d_date_sk#42] + +(63) HashAggregate [codegen id : 25] +Input [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#39, i_class_id#40, i_category_id#41] +Keys [3]: [i_brand_id#39, i_class_id#40, i_category_id#41] +Functions [2]: [partial_sum((cast(ss_quantity#2 as decimal(10,0)) * ss_list_price#3)), partial_count(1)] +Aggregate Attributes [3]: [sum#43, isEmpty#44, count#45] +Results [6]: [i_brand_id#39, i_class_id#40, i_category_id#41, sum#46, isEmpty#47, count#48] + +(64) Exchange +Input [6]: [i_brand_id#39, i_class_id#40, i_category_id#41, sum#46, isEmpty#47, count#48] +Arguments: hashpartitioning(i_brand_id#39, i_class_id#40, i_category_id#41, 5), ENSURE_REQUIREMENTS, [plan_id=9] + +(65) HashAggregate [codegen id : 52] +Input [6]: [i_brand_id#39, i_class_id#40, i_category_id#41, sum#46, isEmpty#47, count#48] +Keys [3]: [i_brand_id#39, i_class_id#40, i_category_id#41] +Functions [2]: [sum((cast(ss_quantity#2 as decimal(10,0)) * ss_list_price#3)), count(1)] +Aggregate Attributes [2]: [sum((cast(ss_quantity#2 as decimal(10,0)) * ss_list_price#3))#49, count(1)#50] +Results [6]: [store AS channel#51, i_brand_id#39, i_class_id#40, i_category_id#41, sum((cast(ss_quantity#2 as decimal(10,0)) * ss_list_price#3))#49 AS sales#52, count(1)#50 AS number_sales#53] + +(66) Filter [codegen id : 52] +Input [6]: [channel#51, i_brand_id#39, i_class_id#40, i_category_id#41, sales#52, number_sales#53] +Condition : (isnotnull(sales#52) AND (cast(sales#52 as decimal(32,6)) > cast(Subquery scalar-subquery#54, [id=#55] as decimal(32,6)))) + +(67) Scan parquet spark_catalog.default.store_sales +Output [4]: [ss_item_sk#56, ss_quantity#57, ss_list_price#58, ss_sold_date_sk#59] +Batched: true +Location: InMemoryFileIndex [] +PartitionFilters: [isnotnull(ss_sold_date_sk#59), dynamicpruningexpression(ss_sold_date_sk#59 IN dynamicpruning#60)] +PushedFilters: [IsNotNull(ss_item_sk)] +ReadSchema: struct<ss_item_sk:int,ss_quantity:int,ss_list_price:decimal(7,2)> + +(68) CometFilter +Input [4]: [ss_item_sk#56, ss_quantity#57, ss_list_price#58, ss_sold_date_sk#59] +Condition : isnotnull(ss_item_sk#56) + +(69) ColumnarToRow [codegen id : 50] +Input [4]: [ss_item_sk#56, ss_quantity#57, ss_list_price#58, ss_sold_date_sk#59] + +(70) ReusedExchange [Reuses operator id: 50] +Output [1]: [ss_item_sk#61] + +(71) BroadcastHashJoin [codegen id : 50] +Left keys [1]: [ss_item_sk#56] +Right keys [1]: [ss_item_sk#61] +Join type: LeftSemi +Join condition: None + +(72) ReusedExchange [Reuses operator id: 57] +Output [4]: [i_item_sk#62, i_brand_id#63, i_class_id#64, i_category_id#65] + +(73) BroadcastHashJoin [codegen id : 50] +Left keys [1]: [ss_item_sk#56] +Right keys [1]: [i_item_sk#62] +Join type: Inner +Join condition: None + +(74) Project [codegen id : 50] +Output [6]: [ss_quantity#57, ss_list_price#58, ss_sold_date_sk#59, i_brand_id#63, i_class_id#64, i_category_id#65] +Input [8]: [ss_item_sk#56, ss_quantity#57, ss_list_price#58, ss_sold_date_sk#59, i_item_sk#62, i_brand_id#63, i_class_id#64, i_category_id#65] + +(75) ReusedExchange [Reuses operator id: 122] +Output [1]: [d_date_sk#66] + +(76) BroadcastHashJoin [codegen id : 50] +Left keys [1]: [ss_sold_date_sk#59] +Right keys [1]: [d_date_sk#66] +Join type: Inner +Join condition: None + +(77) Project [codegen id : 50] +Output [5]: [ss_quantity#57, ss_list_price#58, i_brand_id#63, i_class_id#64, i_category_id#65] +Input [7]: [ss_quantity#57, ss_list_price#58, ss_sold_date_sk#59, i_brand_id#63, i_class_id#64, i_category_id#65, d_date_sk#66] + +(78) HashAggregate [codegen id : 50] +Input [5]: [ss_quantity#57, ss_list_price#58, i_brand_id#63, i_class_id#64, i_category_id#65] +Keys [3]: [i_brand_id#63, i_class_id#64, i_category_id#65] +Functions [2]: [partial_sum((cast(ss_quantity#57 as decimal(10,0)) * ss_list_price#58)), partial_count(1)] +Aggregate Attributes [3]: [sum#67, isEmpty#68, count#69] +Results [6]: [i_brand_id#63, i_class_id#64, i_category_id#65, sum#70, isEmpty#71, count#72] + +(79) Exchange +Input [6]: [i_brand_id#63, i_class_id#64, i_category_id#65, sum#70, isEmpty#71, count#72] +Arguments: hashpartitioning(i_brand_id#63, i_class_id#64, i_category_id#65, 5), ENSURE_REQUIREMENTS, [plan_id=10] + +(80) HashAggregate [codegen id : 51] +Input [6]: [i_brand_id#63, i_class_id#64, i_category_id#65, sum#70, isEmpty#71, count#72] +Keys [3]: [i_brand_id#63, i_class_id#64, i_category_id#65] +Functions [2]: [sum((cast(ss_quantity#57 as decimal(10,0)) * ss_list_price#58)), count(1)] +Aggregate Attributes [2]: [sum((cast(ss_quantity#57 as decimal(10,0)) * ss_list_price#58))#73, count(1)#74] +Results [6]: [store AS channel#75, i_brand_id#63, i_class_id#64, i_category_id#65, sum((cast(ss_quantity#57 as decimal(10,0)) * ss_list_price#58))#73 AS sales#76, count(1)#74 AS number_sales#77] + +(81) Filter [codegen id : 51] +Input [6]: [channel#75, i_brand_id#63, i_class_id#64, i_category_id#65, sales#76, number_sales#77] +Condition : (isnotnull(sales#76) AND (cast(sales#76 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#54, [id=#55] as decimal(32,6)))) + +(82) BroadcastExchange +Input [6]: [channel#75, i_brand_id#63, i_class_id#64, i_category_id#65, sales#76, number_sales#77] +Arguments: HashedRelationBroadcastMode(List(input[1, int, true], input[2, int, true], input[3, int, true]),false), [plan_id=11] + +(83) BroadcastHashJoin [codegen id : 52] +Left keys [3]: [i_brand_id#39, i_class_id#40, i_category_id#41] +Right keys [3]: [i_brand_id#63, i_class_id#64, i_category_id#65] +Join type: Inner +Join condition: None + +(84) TakeOrderedAndProject +Input [12]: [channel#51, i_brand_id#39, i_class_id#40, i_category_id#41, sales#52, number_sales#53, channel#75, i_brand_id#63, i_class_id#64, i_category_id#65, sales#76, number_sales#77] +Arguments: 100, [i_brand_id#39 ASC NULLS FIRST, i_class_id#40 ASC NULLS FIRST, i_category_id#41 ASC NULLS FIRST], [channel#51, i_brand_id#39, i_class_id#40, i_category_id#41, sales#52, number_sales#53, channel#75, i_brand_id#63, i_class_id#64, i_category_id#65, sales#76, number_sales#77] + +===== Subqueries ===== + +Subquery:1 Hosting operator id = 66 Hosting Expression = Subquery scalar-subquery#54, [id=#55] +* HashAggregate (103) ++- Exchange (102) + +- * HashAggregate (101) + +- Union (100) + :- * Project (89) + : +- * BroadcastHashJoin Inner BuildRight (88) + : :- * ColumnarToRow (86) + : : +- CometScan parquet spark_catalog.default.store_sales (85) + : +- ReusedExchange (87) + :- * Project (94) + : +- * BroadcastHashJoin Inner BuildRight (93) + : :- * ColumnarToRow (91) + : : +- CometScan parquet spark_catalog.default.catalog_sales (90) + : +- ReusedExchange (92) + +- * Project (99) + +- * BroadcastHashJoin Inner BuildRight (98) + :- * ColumnarToRow (96) + : +- CometScan parquet spark_catalog.default.web_sales (95) + +- ReusedExchange (97) + + +(85) Scan parquet spark_catalog.default.store_sales +Output [3]: [ss_quantity#78, ss_list_price#79, ss_sold_date_sk#80] +Batched: true +Location: InMemoryFileIndex [] +PartitionFilters: [isnotnull(ss_sold_date_sk#80), dynamicpruningexpression(ss_sold_date_sk#80 IN dynamicpruning#81)] +ReadSchema: struct<ss_quantity:int,ss_list_price:decimal(7,2)> + +(86) ColumnarToRow [codegen id : 2] +Input [3]: [ss_quantity#78, ss_list_price#79, ss_sold_date_sk#80] + +(87) ReusedExchange [Reuses operator id: 117] +Output [1]: [d_date_sk#82] + +(88) BroadcastHashJoin [codegen id : 2] +Left keys [1]: [ss_sold_date_sk#80] +Right keys [1]: [d_date_sk#82] +Join type: Inner +Join condition: None + +(89) Project [codegen id : 2] +Output [2]: [ss_quantity#78 AS quantity#83, ss_list_price#79 AS list_price#84] +Input [4]: [ss_quantity#78, ss_list_price#79, ss_sold_date_sk#80, d_date_sk#82] + +(90) Scan parquet spark_catalog.default.catalog_sales +Output [3]: [cs_quantity#85, cs_list_price#86, cs_sold_date_sk#87] +Batched: true +Location: InMemoryFileIndex [] +PartitionFilters: [isnotnull(cs_sold_date_sk#87), dynamicpruningexpression(cs_sold_date_sk#87 IN dynamicpruning#88)] +ReadSchema: struct<cs_quantity:int,cs_list_price:decimal(7,2)> + +(91) ColumnarToRow [codegen id : 4] +Input [3]: [cs_quantity#85, cs_list_price#86, cs_sold_date_sk#87] + +(92) ReusedExchange [Reuses operator id: 117] +Output [1]: [d_date_sk#89] + +(93) BroadcastHashJoin [codegen id : 4] +Left keys [1]: [cs_sold_date_sk#87] +Right keys [1]: [d_date_sk#89] +Join type: Inner +Join condition: None + +(94) Project [codegen id : 4] +Output [2]: [cs_quantity#85 AS quantity#90, cs_list_price#86 AS list_price#91] +Input [4]: [cs_quantity#85, cs_list_price#86, cs_sold_date_sk#87, d_date_sk#89] + +(95) Scan parquet spark_catalog.default.web_sales +Output [3]: [ws_quantity#92, ws_list_price#93, ws_sold_date_sk#94] +Batched: true +Location: InMemoryFileIndex [] +PartitionFilters: [isnotnull(ws_sold_date_sk#94), dynamicpruningexpression(ws_sold_date_sk#94 IN dynamicpruning#95)] +ReadSchema: struct<ws_quantity:int,ws_list_price:decimal(7,2)> + +(96) ColumnarToRow [codegen id : 6] +Input [3]: [ws_quantity#92, ws_list_price#93, ws_sold_date_sk#94] + +(97) ReusedExchange [Reuses operator id: 117] +Output [1]: [d_date_sk#96] + +(98) BroadcastHashJoin [codegen id : 6] +Left keys [1]: [ws_sold_date_sk#94] +Right keys [1]: [d_date_sk#96] +Join type: Inner +Join condition: None + +(99) Project [codegen id : 6] +Output [2]: [ws_quantity#92 AS quantity#97, ws_list_price#93 AS list_price#98] +Input [4]: [ws_quantity#92, ws_list_price#93, ws_sold_date_sk#94, d_date_sk#96] + +(100) Union + +(101) HashAggregate [codegen id : 7] +Input [2]: [quantity#83, list_price#84] +Keys: [] +Functions [1]: [partial_avg((cast(quantity#83 as decimal(10,0)) * list_price#84))] +Aggregate Attributes [2]: [sum#99, count#100] +Results [2]: [sum#101, count#102] + +(102) Exchange +Input [2]: [sum#101, count#102] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=12] + +(103) HashAggregate [codegen id : 8] +Input [2]: [sum#101, count#102] +Keys: [] +Functions [1]: [avg((cast(quantity#83 as decimal(10,0)) * list_price#84))] +Aggregate Attributes [1]: [avg((cast(quantity#83 as decimal(10,0)) * list_price#84))#103] +Results [1]: [avg((cast(quantity#83 as decimal(10,0)) * list_price#84))#103 AS average_sales#104] + +Subquery:2 Hosting operator id = 85 Hosting Expression = ss_sold_date_sk#80 IN dynamicpruning#12 + +Subquery:3 Hosting operator id = 90 Hosting Expression = cs_sold_date_sk#87 IN dynamicpruning#12 + +Subquery:4 Hosting operator id = 95 Hosting Expression = ws_sold_date_sk#94 IN dynamicpruning#12 + +Subquery:5 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 +BroadcastExchange (108) ++- * ColumnarToRow (107) + +- CometProject (106) + +- CometFilter (105) + +- CometScan parquet spark_catalog.default.date_dim (104) + + +(104) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#42, d_week_seq#105] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_week_seq), EqualTo(d_week_seq,ScalarSubquery#106), IsNotNull(d_date_sk)] +ReadSchema: struct<d_date_sk:int,d_week_seq:int> + +(105) CometFilter +Input [2]: [d_date_sk#42, d_week_seq#105] +Condition : ((isnotnull(d_week_seq#105) AND (d_week_seq#105 = ReusedSubquery Subquery scalar-subquery#106, [id=#107])) AND isnotnull(d_date_sk#42)) + +(106) CometProject +Input [2]: [d_date_sk#42, d_week_seq#105] +Arguments: [d_date_sk#42], [d_date_sk#42] + +(107) ColumnarToRow [codegen id : 1] +Input [1]: [d_date_sk#42] + +(108) BroadcastExchange +Input [1]: [d_date_sk#42] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=13] + +Subquery:6 Hosting operator id = 105 Hosting Expression = ReusedSubquery Subquery scalar-subquery#106, [id=#107] + +Subquery:7 Hosting operator id = 104 Hosting Expression = Subquery scalar-subquery#106, [id=#107] +* ColumnarToRow (112) ++- CometProject (111) + +- CometFilter (110) + +- CometScan parquet spark_catalog.default.date_dim (109) + + +(109) Scan parquet spark_catalog.default.date_dim +Output [4]: [d_week_seq#108, d_year#109, d_moy#110, d_dom#111] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), IsNotNull(d_dom), EqualTo(d_year,1999), EqualTo(d_moy,12), EqualTo(d_dom,16)] +ReadSchema: struct<d_week_seq:int,d_year:int,d_moy:int,d_dom:int> + +(110) CometFilter +Input [4]: [d_week_seq#108, d_year#109, d_moy#110, d_dom#111] +Condition : (((((isnotnull(d_year#109) AND isnotnull(d_moy#110)) AND isnotnull(d_dom#111)) AND (d_year#109 = 1999)) AND (d_moy#110 = 12)) AND (d_dom#111 = 16)) + +(111) CometProject +Input [4]: [d_week_seq#108, d_year#109, d_moy#110, d_dom#111] +Arguments: [d_week_seq#108], [d_week_seq#108] + +(112) ColumnarToRow [codegen id : 1] +Input [1]: [d_week_seq#108] + +Subquery:8 Hosting operator id = 7 Hosting Expression = ss_sold_date_sk#11 IN dynamicpruning#12 +BroadcastExchange (117) ++- * ColumnarToRow (116) + +- CometProject (115) + +- CometFilter (114) + +- CometScan parquet spark_catalog.default.date_dim (113) + + +(113) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#25, d_year#112] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_year), GreaterThanOrEqual(d_year,1998), LessThanOrEqual(d_year,2000), IsNotNull(d_date_sk)] +ReadSchema: struct<d_date_sk:int,d_year:int> + +(114) CometFilter +Input [2]: [d_date_sk#25, d_year#112] +Condition : (((isnotnull(d_year#112) AND (d_year#112 >= 1998)) AND (d_year#112 <= 2000)) AND isnotnull(d_date_sk#25)) + +(115) CometProject +Input [2]: [d_date_sk#25, d_year#112] +Arguments: [d_date_sk#25], [d_date_sk#25] + +(116) ColumnarToRow [codegen id : 1] +Input [1]: [d_date_sk#25] + +(117) BroadcastExchange +Input [1]: [d_date_sk#25] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=14] + +Subquery:9 Hosting operator id = 13 Hosting Expression = cs_sold_date_sk#18 IN dynamicpruning#12 + +Subquery:10 Hosting operator id = 36 Hosting Expression = ws_sold_date_sk#30 IN dynamicpruning#12 + +Subquery:11 Hosting operator id = 81 Hosting Expression = ReusedSubquery Subquery scalar-subquery#54, [id=#55] + +Subquery:12 Hosting operator id = 67 Hosting Expression = ss_sold_date_sk#59 IN dynamicpruning#60 +BroadcastExchange (122) ++- * ColumnarToRow (121) + +- CometProject (120) + +- CometFilter (119) + +- CometScan parquet spark_catalog.default.date_dim (118) + + +(118) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#66, d_week_seq#113] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_week_seq), EqualTo(d_week_seq,ScalarSubquery#114), IsNotNull(d_date_sk)] +ReadSchema: struct<d_date_sk:int,d_week_seq:int> + +(119) CometFilter +Input [2]: [d_date_sk#66, d_week_seq#113] +Condition : ((isnotnull(d_week_seq#113) AND (d_week_seq#113 = ReusedSubquery Subquery scalar-subquery#114, [id=#115])) AND isnotnull(d_date_sk#66)) + +(120) CometProject +Input [2]: [d_date_sk#66, d_week_seq#113] +Arguments: [d_date_sk#66], [d_date_sk#66] + +(121) ColumnarToRow [codegen id : 1] +Input [1]: [d_date_sk#66] + +(122) BroadcastExchange +Input [1]: [d_date_sk#66] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=15] + +Subquery:13 Hosting operator id = 119 Hosting Expression = ReusedSubquery Subquery scalar-subquery#114, [id=#115] + +Subquery:14 Hosting operator id = 118 Hosting Expression = Subquery scalar-subquery#114, [id=#115] +* ColumnarToRow (126) ++- CometProject (125) + +- CometFilter (124) + +- CometScan parquet spark_catalog.default.date_dim (123) + + +(123) Scan parquet spark_catalog.default.date_dim +Output [4]: [d_week_seq#116, d_year#117, d_moy#118, d_dom#119] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), IsNotNull(d_dom), EqualTo(d_year,1998), EqualTo(d_moy,12), EqualTo(d_dom,16)] +ReadSchema: struct<d_week_seq:int,d_year:int,d_moy:int,d_dom:int> + +(124) CometFilter +Input [4]: [d_week_seq#116, d_year#117, d_moy#118, d_dom#119] +Condition : (((((isnotnull(d_year#117) AND isnotnull(d_moy#118)) AND isnotnull(d_dom#119)) AND (d_year#117 = 1998)) AND (d_moy#118 = 12)) AND (d_dom#119 = 16)) + +(125) CometProject +Input [4]: [d_week_seq#116, d_year#117, d_moy#118, d_dom#119] +Arguments: [d_week_seq#116], [d_week_seq#116] + +(126) ColumnarToRow [codegen id : 1] +Input [1]: [d_week_seq#116] + + 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 new file mode 100644 index 000000000..55aa823ab --- /dev/null +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q14/simplified.txt @@ -0,0 +1,204 @@ +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) + 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) + HashAggregate [sum,count] [avg((cast(quantity as decimal(10,0)) * list_price)),average_sales,sum,count] + InputAdapter + Exchange #12 + WholeStageCodegen (7) + HashAggregate [quantity,list_price] [sum,count,sum,count] + InputAdapter + Union + WholeStageCodegen (2) + Project [ss_quantity,ss_list_price] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + ColumnarToRow + InputAdapter + CometScan parquet spark_catalog.default.store_sales [ss_quantity,ss_list_price,ss_sold_date_sk] + ReusedSubquery [d_date_sk] #3 + InputAdapter + ReusedExchange [d_date_sk] #6 + WholeStageCodegen (4) + Project [cs_quantity,cs_list_price] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + ColumnarToRow + InputAdapter + CometScan parquet spark_catalog.default.catalog_sales [cs_quantity,cs_list_price,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #3 + InputAdapter + ReusedExchange [d_date_sk] #6 + WholeStageCodegen (6) + Project [ws_quantity,ws_list_price] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + ColumnarToRow + InputAdapter + CometScan parquet spark_catalog.default.web_sales [ws_quantity,ws_list_price,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #3 + InputAdapter + ReusedExchange [d_date_sk] #6 + HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum((cast(ss_quantity as decimal(10,0)) * ss_list_price)),count(1),channel,sales,number_sales,sum,isEmpty,count] + InputAdapter + Exchange [i_brand_id,i_class_id,i_category_id] #1 + WholeStageCodegen (25) + HashAggregate [i_brand_id,i_class_id,i_category_id,ss_quantity,ss_list_price] [sum,isEmpty,count,sum,isEmpty,count] + Project [ss_quantity,ss_list_price,i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [ss_quantity,ss_list_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [ss_item_sk,i_item_sk] + BroadcastHashJoin [ss_item_sk,ss_item_sk] + ColumnarToRow + InputAdapter + CometFilter [ss_item_sk] + CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #2 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_week_seq,d_date_sk] + ReusedSubquery [d_week_seq] #2 + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq] + Subquery #2 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_week_seq] + CometFilter [d_year,d_moy,d_dom] + CometScan parquet spark_catalog.default.date_dim [d_week_seq,d_year,d_moy,d_dom] + InputAdapter + BroadcastExchange #3 + WholeStageCodegen (11) + Project [i_item_sk] + BroadcastHashJoin [i_brand_id,i_class_id,i_category_id,brand_id,class_id,category_id] + ColumnarToRow + InputAdapter + CometFilter [i_brand_id,i_class_id,i_category_id] + CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] + InputAdapter + BroadcastExchange #4 + WholeStageCodegen (10) + BroadcastHashJoin [brand_id,class_id,category_id,i_brand_id,i_class_id,i_category_id] + HashAggregate [brand_id,class_id,category_id] + InputAdapter + Exchange [brand_id,class_id,category_id] #5 + WholeStageCodegen (6) + HashAggregate [brand_id,class_id,category_id] + Project [i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [ss_sold_date_sk,i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [ss_item_sk,i_item_sk] + ColumnarToRow + InputAdapter + CometFilter [ss_item_sk] + CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #3 + BroadcastExchange #6 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_year,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + InputAdapter + BroadcastExchange #7 + WholeStageCodegen (4) + BroadcastHashJoin [i_brand_id,i_class_id,i_category_id,i_brand_id,i_class_id,i_category_id] + ColumnarToRow + InputAdapter + CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] + CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] + InputAdapter + BroadcastExchange #8 + WholeStageCodegen (3) + Project [i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + Project [cs_sold_date_sk,i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [cs_item_sk,i_item_sk] + ColumnarToRow + InputAdapter + CometFilter [cs_item_sk] + CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #3 + InputAdapter + BroadcastExchange #9 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometFilter [i_item_sk] + CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] + InputAdapter + ReusedExchange [d_date_sk] #6 + InputAdapter + ReusedExchange [d_date_sk] #6 + InputAdapter + BroadcastExchange #10 + WholeStageCodegen (9) + Project [i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + Project [ws_sold_date_sk,i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [ws_item_sk,i_item_sk] + ColumnarToRow + InputAdapter + CometFilter [ws_item_sk] + CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #3 + InputAdapter + ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #9 + InputAdapter + ReusedExchange [d_date_sk] #6 + InputAdapter + BroadcastExchange #11 + WholeStageCodegen (23) + BroadcastHashJoin [i_item_sk,ss_item_sk] + ColumnarToRow + InputAdapter + CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] + CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] + InputAdapter + ReusedExchange [ss_item_sk] #3 + InputAdapter + ReusedExchange [d_date_sk] #2 + InputAdapter + BroadcastExchange #13 + WholeStageCodegen (51) + Filter [sales] + ReusedSubquery [average_sales] #4 + HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum((cast(ss_quantity as decimal(10,0)) * ss_list_price)),count(1),channel,sales,number_sales,sum,isEmpty,count] + InputAdapter + Exchange [i_brand_id,i_class_id,i_category_id] #14 + WholeStageCodegen (50) + HashAggregate [i_brand_id,i_class_id,i_category_id,ss_quantity,ss_list_price] [sum,isEmpty,count,sum,isEmpty,count] + Project [ss_quantity,ss_list_price,i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [ss_quantity,ss_list_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [ss_item_sk,i_item_sk] + BroadcastHashJoin [ss_item_sk,ss_item_sk] + ColumnarToRow + InputAdapter + CometFilter [ss_item_sk] + CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #5 + BroadcastExchange #15 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_week_seq,d_date_sk] + ReusedSubquery [d_week_seq] #6 + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq] + Subquery #6 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_week_seq] + CometFilter [d_year,d_moy,d_dom] + CometScan parquet spark_catalog.default.date_dim [d_week_seq,d_year,d_moy,d_dom] + InputAdapter + ReusedExchange [ss_item_sk] #3 + InputAdapter + ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #11 + InputAdapter + ReusedExchange [d_date_sk] #15 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 new file mode 100644 index 000000000..d76059e01 --- /dev/null +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q14a/explain.txt @@ -0,0 +1,964 @@ +== Physical Plan == +TakeOrderedAndProject (125) ++- * HashAggregate (124) + +- Exchange (123) + +- * HashAggregate (122) + +- Union (121) + :- * HashAggregate (100) + : +- Exchange (99) + : +- * HashAggregate (98) + : +- Union (97) + : :- * Filter (66) + : : +- * HashAggregate (65) + : : +- Exchange (64) + : : +- * HashAggregate (63) + : : +- * Project (62) + : : +- * BroadcastHashJoin Inner BuildRight (61) + : : :- * Project (59) + : : : +- * BroadcastHashJoin Inner BuildRight (58) + : : : :- * BroadcastHashJoin LeftSemi BuildRight (51) + : : : : :- * ColumnarToRow (3) + : : : : : +- CometFilter (2) + : : : : : +- CometScan parquet spark_catalog.default.store_sales (1) + : : : : +- BroadcastExchange (50) + : : : : +- * Project (49) + : : : : +- * BroadcastHashJoin Inner BuildRight (48) + : : : : :- * ColumnarToRow (6) + : : : : : +- CometFilter (5) + : : : : : +- CometScan parquet spark_catalog.default.item (4) + : : : : +- BroadcastExchange (47) + : : : : +- * BroadcastHashJoin LeftSemi BuildRight (46) + : : : : :- * HashAggregate (35) + : : : : : +- Exchange (34) + : : : : : +- * HashAggregate (33) + : : : : : +- * Project (32) + : : : : : +- * BroadcastHashJoin Inner BuildRight (31) + : : : : : :- * Project (29) + : : : : : : +- * BroadcastHashJoin Inner BuildRight (28) + : : : : : : :- * ColumnarToRow (9) + : : : : : : : +- CometFilter (8) + : : : : : : : +- CometScan parquet spark_catalog.default.store_sales (7) + : : : : : : +- BroadcastExchange (27) + : : : : : : +- * BroadcastHashJoin LeftSemi BuildRight (26) + : : : : : : :- * ColumnarToRow (12) + : : : : : : : +- CometFilter (11) + : : : : : : : +- CometScan parquet spark_catalog.default.item (10) + : : : : : : +- BroadcastExchange (25) + : : : : : : +- * Project (24) + : : : : : : +- * BroadcastHashJoin Inner BuildRight (23) + : : : : : : :- * Project (21) + : : : : : : : +- * BroadcastHashJoin Inner BuildRight (20) + : : : : : : : :- * ColumnarToRow (15) + : : : : : : : : +- CometFilter (14) + : : : : : : : : +- CometScan parquet spark_catalog.default.catalog_sales (13) + : : : : : : : +- BroadcastExchange (19) + : : : : : : : +- * ColumnarToRow (18) + : : : : : : : +- CometFilter (17) + : : : : : : : +- CometScan parquet spark_catalog.default.item (16) + : : : : : : +- ReusedExchange (22) + : : : : : +- ReusedExchange (30) + : : : : +- BroadcastExchange (45) + : : : : +- * Project (44) + : : : : +- * BroadcastHashJoin Inner BuildRight (43) + : : : : :- * Project (41) + : : : : : +- * BroadcastHashJoin Inner BuildRight (40) + : : : : : :- * ColumnarToRow (38) + : : : : : : +- CometFilter (37) + : : : : : : +- CometScan parquet spark_catalog.default.web_sales (36) + : : : : : +- ReusedExchange (39) + : : : : +- ReusedExchange (42) + : : : +- BroadcastExchange (57) + : : : +- * BroadcastHashJoin LeftSemi BuildRight (56) + : : : :- * ColumnarToRow (54) + : : : : +- CometFilter (53) + : : : : +- CometScan parquet spark_catalog.default.item (52) + : : : +- ReusedExchange (55) + : : +- ReusedExchange (60) + : :- * Filter (81) + : : +- * HashAggregate (80) + : : +- Exchange (79) + : : +- * HashAggregate (78) + : : +- * Project (77) + : : +- * BroadcastHashJoin Inner BuildRight (76) + : : :- * Project (74) + : : : +- * BroadcastHashJoin Inner BuildRight (73) + : : : :- * BroadcastHashJoin LeftSemi BuildRight (71) + : : : : :- * ColumnarToRow (69) + : : : : : +- CometFilter (68) + : : : : : +- CometScan parquet spark_catalog.default.catalog_sales (67) + : : : : +- ReusedExchange (70) + : : : +- ReusedExchange (72) + : : +- ReusedExchange (75) + : +- * Filter (96) + : +- * HashAggregate (95) + : +- Exchange (94) + : +- * HashAggregate (93) + : +- * Project (92) + : +- * BroadcastHashJoin Inner BuildRight (91) + : :- * Project (89) + : : +- * BroadcastHashJoin Inner BuildRight (88) + : : :- * BroadcastHashJoin LeftSemi BuildRight (86) + : : : :- * ColumnarToRow (84) + : : : : +- CometFilter (83) + : : : : +- CometScan parquet spark_catalog.default.web_sales (82) + : : : +- ReusedExchange (85) + : : +- ReusedExchange (87) + : +- ReusedExchange (90) + :- * HashAggregate (105) + : +- Exchange (104) + : +- * HashAggregate (103) + : +- * HashAggregate (102) + : +- ReusedExchange (101) + :- * HashAggregate (110) + : +- Exchange (109) + : +- * HashAggregate (108) + : +- * HashAggregate (107) + : +- ReusedExchange (106) + :- * HashAggregate (115) + : +- Exchange (114) + : +- * HashAggregate (113) + : +- * HashAggregate (112) + : +- ReusedExchange (111) + +- * HashAggregate (120) + +- Exchange (119) + +- * HashAggregate (118) + +- * HashAggregate (117) + +- ReusedExchange (116) + + +(1) Scan parquet spark_catalog.default.store_sales +Output [4]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] +Batched: true +Location: InMemoryFileIndex [] +PartitionFilters: [isnotnull(ss_sold_date_sk#4), dynamicpruningexpression(ss_sold_date_sk#4 IN dynamicpruning#5)] +PushedFilters: [IsNotNull(ss_item_sk)] +ReadSchema: struct<ss_item_sk:int,ss_quantity:int,ss_list_price:decimal(7,2)> + +(2) CometFilter +Input [4]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] +Condition : isnotnull(ss_item_sk#1) + +(3) ColumnarToRow [codegen id : 25] +Input [4]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4] + +(4) Scan parquet spark_catalog.default.item +Output [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] +Batched: true +Location [not included in comparison]/{warehouse_dir}/item] +PushedFilters: [IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id)] +ReadSchema: struct<i_item_sk:int,i_brand_id:int,i_class_id:int,i_category_id:int> + +(5) CometFilter +Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] +Condition : ((isnotnull(i_brand_id#7) AND isnotnull(i_class_id#8)) AND isnotnull(i_category_id#9)) + +(6) ColumnarToRow [codegen id : 11] +Input [4]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9] + +(7) Scan parquet spark_catalog.default.store_sales +Output [2]: [ss_item_sk#10, ss_sold_date_sk#11] +Batched: true +Location: InMemoryFileIndex [] +PartitionFilters: [isnotnull(ss_sold_date_sk#11), dynamicpruningexpression(ss_sold_date_sk#11 IN dynamicpruning#12)] +PushedFilters: [IsNotNull(ss_item_sk)] +ReadSchema: struct<ss_item_sk:int> + +(8) CometFilter +Input [2]: [ss_item_sk#10, ss_sold_date_sk#11] +Condition : isnotnull(ss_item_sk#10) + +(9) ColumnarToRow [codegen id : 6] +Input [2]: [ss_item_sk#10, ss_sold_date_sk#11] + +(10) Scan parquet spark_catalog.default.item +Output [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] +Batched: true +Location [not included in comparison]/{warehouse_dir}/item] +PushedFilters: [IsNotNull(i_item_sk), IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id)] +ReadSchema: struct<i_item_sk:int,i_brand_id:int,i_class_id:int,i_category_id:int> + +(11) CometFilter +Input [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] +Condition : (((isnotnull(i_item_sk#13) AND isnotnull(i_brand_id#14)) AND isnotnull(i_class_id#15)) AND isnotnull(i_category_id#16)) + +(12) ColumnarToRow [codegen id : 4] +Input [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] + +(13) Scan parquet spark_catalog.default.catalog_sales +Output [2]: [cs_item_sk#17, cs_sold_date_sk#18] +Batched: true +Location: InMemoryFileIndex [] +PartitionFilters: [isnotnull(cs_sold_date_sk#18), dynamicpruningexpression(cs_sold_date_sk#18 IN dynamicpruning#19)] +PushedFilters: [IsNotNull(cs_item_sk)] +ReadSchema: struct<cs_item_sk:int> + +(14) CometFilter +Input [2]: [cs_item_sk#17, cs_sold_date_sk#18] +Condition : isnotnull(cs_item_sk#17) + +(15) ColumnarToRow [codegen id : 3] +Input [2]: [cs_item_sk#17, cs_sold_date_sk#18] + +(16) Scan parquet spark_catalog.default.item +Output [4]: [i_item_sk#20, i_brand_id#21, i_class_id#22, i_category_id#23] +Batched: true +Location [not included in comparison]/{warehouse_dir}/item] +PushedFilters: [IsNotNull(i_item_sk)] +ReadSchema: struct<i_item_sk:int,i_brand_id:int,i_class_id:int,i_category_id:int> + +(17) CometFilter +Input [4]: [i_item_sk#20, i_brand_id#21, i_class_id#22, i_category_id#23] +Condition : isnotnull(i_item_sk#20) + +(18) ColumnarToRow [codegen id : 1] +Input [4]: [i_item_sk#20, i_brand_id#21, i_class_id#22, i_category_id#23] + +(19) BroadcastExchange +Input [4]: [i_item_sk#20, i_brand_id#21, i_class_id#22, i_category_id#23] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=1] + +(20) BroadcastHashJoin [codegen id : 3] +Left keys [1]: [cs_item_sk#17] +Right keys [1]: [i_item_sk#20] +Join type: Inner +Join condition: None + +(21) Project [codegen id : 3] +Output [4]: [cs_sold_date_sk#18, i_brand_id#21, i_class_id#22, i_category_id#23] +Input [6]: [cs_item_sk#17, cs_sold_date_sk#18, i_item_sk#20, i_brand_id#21, i_class_id#22, i_category_id#23] + +(22) ReusedExchange [Reuses operator id: 159] +Output [1]: [d_date_sk#24] + +(23) BroadcastHashJoin [codegen id : 3] +Left keys [1]: [cs_sold_date_sk#18] +Right keys [1]: [d_date_sk#24] +Join type: Inner +Join condition: None + +(24) Project [codegen id : 3] +Output [3]: [i_brand_id#21, i_class_id#22, i_category_id#23] +Input [5]: [cs_sold_date_sk#18, i_brand_id#21, i_class_id#22, i_category_id#23, d_date_sk#24] + +(25) BroadcastExchange +Input [3]: [i_brand_id#21, i_class_id#22, i_category_id#23] +Arguments: HashedRelationBroadcastMode(List(coalesce(input[0, int, true], 0), isnull(input[0, int, true]), coalesce(input[1, int, true], 0), isnull(input[1, int, true]), coalesce(input[2, int, true], 0), isnull(input[2, int, true])),false), [plan_id=2] + +(26) BroadcastHashJoin [codegen id : 4] +Left keys [6]: [coalesce(i_brand_id#14, 0), isnull(i_brand_id#14), coalesce(i_class_id#15, 0), isnull(i_class_id#15), coalesce(i_category_id#16, 0), isnull(i_category_id#16)] +Right keys [6]: [coalesce(i_brand_id#21, 0), isnull(i_brand_id#21), coalesce(i_class_id#22, 0), isnull(i_class_id#22), coalesce(i_category_id#23, 0), isnull(i_category_id#23)] +Join type: LeftSemi +Join condition: None + +(27) BroadcastExchange +Input [4]: [i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=3] + +(28) BroadcastHashJoin [codegen id : 6] +Left keys [1]: [ss_item_sk#10] +Right keys [1]: [i_item_sk#13] +Join type: Inner +Join condition: None + +(29) Project [codegen id : 6] +Output [4]: [ss_sold_date_sk#11, i_brand_id#14, i_class_id#15, i_category_id#16] +Input [6]: [ss_item_sk#10, ss_sold_date_sk#11, i_item_sk#13, i_brand_id#14, i_class_id#15, i_category_id#16] + +(30) ReusedExchange [Reuses operator id: 159] +Output [1]: [d_date_sk#25] + +(31) BroadcastHashJoin [codegen id : 6] +Left keys [1]: [ss_sold_date_sk#11] +Right keys [1]: [d_date_sk#25] +Join type: Inner +Join condition: None + +(32) Project [codegen id : 6] +Output [3]: [i_brand_id#14 AS brand_id#26, i_class_id#15 AS class_id#27, i_category_id#16 AS category_id#28] +Input [5]: [ss_sold_date_sk#11, i_brand_id#14, i_class_id#15, i_category_id#16, d_date_sk#25] + +(33) HashAggregate [codegen id : 6] +Input [3]: [brand_id#26, class_id#27, category_id#28] +Keys [3]: [brand_id#26, class_id#27, category_id#28] +Functions: [] +Aggregate Attributes: [] +Results [3]: [brand_id#26, class_id#27, category_id#28] + +(34) Exchange +Input [3]: [brand_id#26, class_id#27, category_id#28] +Arguments: hashpartitioning(brand_id#26, class_id#27, category_id#28, 5), ENSURE_REQUIREMENTS, [plan_id=4] + +(35) HashAggregate [codegen id : 10] +Input [3]: [brand_id#26, class_id#27, category_id#28] +Keys [3]: [brand_id#26, class_id#27, category_id#28] +Functions: [] +Aggregate Attributes: [] +Results [3]: [brand_id#26, class_id#27, category_id#28] + +(36) Scan parquet spark_catalog.default.web_sales +Output [2]: [ws_item_sk#29, ws_sold_date_sk#30] +Batched: true +Location: InMemoryFileIndex [] +PartitionFilters: [isnotnull(ws_sold_date_sk#30), dynamicpruningexpression(ws_sold_date_sk#30 IN dynamicpruning#31)] +PushedFilters: [IsNotNull(ws_item_sk)] +ReadSchema: struct<ws_item_sk:int> + +(37) CometFilter +Input [2]: [ws_item_sk#29, ws_sold_date_sk#30] +Condition : isnotnull(ws_item_sk#29) + +(38) ColumnarToRow [codegen id : 9] +Input [2]: [ws_item_sk#29, ws_sold_date_sk#30] + +(39) ReusedExchange [Reuses operator id: 19] +Output [4]: [i_item_sk#32, i_brand_id#33, i_class_id#34, i_category_id#35] + +(40) BroadcastHashJoin [codegen id : 9] +Left keys [1]: [ws_item_sk#29] +Right keys [1]: [i_item_sk#32] +Join type: Inner +Join condition: None + +(41) Project [codegen id : 9] +Output [4]: [ws_sold_date_sk#30, i_brand_id#33, i_class_id#34, i_category_id#35] +Input [6]: [ws_item_sk#29, ws_sold_date_sk#30, i_item_sk#32, i_brand_id#33, i_class_id#34, i_category_id#35] + +(42) ReusedExchange [Reuses operator id: 159] +Output [1]: [d_date_sk#36] + +(43) BroadcastHashJoin [codegen id : 9] +Left keys [1]: [ws_sold_date_sk#30] +Right keys [1]: [d_date_sk#36] +Join type: Inner +Join condition: None + +(44) Project [codegen id : 9] +Output [3]: [i_brand_id#33, i_class_id#34, i_category_id#35] +Input [5]: [ws_sold_date_sk#30, i_brand_id#33, i_class_id#34, i_category_id#35, d_date_sk#36] + +(45) BroadcastExchange +Input [3]: [i_brand_id#33, i_class_id#34, i_category_id#35] +Arguments: HashedRelationBroadcastMode(List(coalesce(input[0, int, true], 0), isnull(input[0, int, true]), coalesce(input[1, int, true], 0), isnull(input[1, int, true]), coalesce(input[2, int, true], 0), isnull(input[2, int, true])),false), [plan_id=5] + +(46) BroadcastHashJoin [codegen id : 10] +Left keys [6]: [coalesce(brand_id#26, 0), isnull(brand_id#26), coalesce(class_id#27, 0), isnull(class_id#27), coalesce(category_id#28, 0), isnull(category_id#28)] +Right keys [6]: [coalesce(i_brand_id#33, 0), isnull(i_brand_id#33), coalesce(i_class_id#34, 0), isnull(i_class_id#34), coalesce(i_category_id#35, 0), isnull(i_category_id#35)] +Join type: LeftSemi +Join condition: None + +(47) BroadcastExchange +Input [3]: [brand_id#26, class_id#27, category_id#28] +Arguments: HashedRelationBroadcastMode(List(input[0, int, true], input[1, int, true], input[2, int, true]),false), [plan_id=6] + +(48) BroadcastHashJoin [codegen id : 11] +Left keys [3]: [i_brand_id#7, i_class_id#8, i_category_id#9] +Right keys [3]: [brand_id#26, class_id#27, category_id#28] +Join type: Inner +Join condition: None + +(49) Project [codegen id : 11] +Output [1]: [i_item_sk#6 AS ss_item_sk#37] +Input [7]: [i_item_sk#6, i_brand_id#7, i_class_id#8, i_category_id#9, brand_id#26, class_id#27, category_id#28] + +(50) BroadcastExchange +Input [1]: [ss_item_sk#37] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=7] + +(51) BroadcastHashJoin [codegen id : 25] +Left keys [1]: [ss_item_sk#1] +Right keys [1]: [ss_item_sk#37] +Join type: LeftSemi +Join condition: None + +(52) Scan parquet spark_catalog.default.item +Output [4]: [i_item_sk#38, i_brand_id#39, i_class_id#40, i_category_id#41] +Batched: true +Location [not included in comparison]/{warehouse_dir}/item] +PushedFilters: [IsNotNull(i_item_sk)] +ReadSchema: struct<i_item_sk:int,i_brand_id:int,i_class_id:int,i_category_id:int> + +(53) CometFilter +Input [4]: [i_item_sk#38, i_brand_id#39, i_class_id#40, i_category_id#41] +Condition : isnotnull(i_item_sk#38) + +(54) ColumnarToRow [codegen id : 23] +Input [4]: [i_item_sk#38, i_brand_id#39, i_class_id#40, i_category_id#41] + +(55) ReusedExchange [Reuses operator id: 50] +Output [1]: [ss_item_sk#37] + +(56) BroadcastHashJoin [codegen id : 23] +Left keys [1]: [i_item_sk#38] +Right keys [1]: [ss_item_sk#37] +Join type: LeftSemi +Join condition: None + +(57) BroadcastExchange +Input [4]: [i_item_sk#38, i_brand_id#39, i_class_id#40, i_category_id#41] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=8] + +(58) BroadcastHashJoin [codegen id : 25] +Left keys [1]: [ss_item_sk#1] +Right keys [1]: [i_item_sk#38] +Join type: Inner +Join condition: None + +(59) Project [codegen id : 25] +Output [6]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#39, i_class_id#40, i_category_id#41] +Input [8]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_item_sk#38, i_brand_id#39, i_class_id#40, i_category_id#41] + +(60) ReusedExchange [Reuses operator id: 154] +Output [1]: [d_date_sk#42] + +(61) BroadcastHashJoin [codegen id : 25] +Left keys [1]: [ss_sold_date_sk#4] +Right keys [1]: [d_date_sk#42] +Join type: Inner +Join condition: None + +(62) Project [codegen id : 25] +Output [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#39, i_class_id#40, i_category_id#41] +Input [7]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#39, i_class_id#40, i_category_id#41, d_date_sk#42] + +(63) HashAggregate [codegen id : 25] +Input [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#39, i_class_id#40, i_category_id#41] +Keys [3]: [i_brand_id#39, i_class_id#40, i_category_id#41] +Functions [2]: [partial_sum((cast(ss_quantity#2 as decimal(10,0)) * ss_list_price#3)), partial_count(1)] +Aggregate Attributes [3]: [sum#43, isEmpty#44, count#45] +Results [6]: [i_brand_id#39, i_class_id#40, i_category_id#41, sum#46, isEmpty#47, count#48] + +(64) Exchange +Input [6]: [i_brand_id#39, i_class_id#40, i_category_id#41, sum#46, isEmpty#47, count#48] +Arguments: hashpartitioning(i_brand_id#39, i_class_id#40, i_category_id#41, 5), ENSURE_REQUIREMENTS, [plan_id=9] + +(65) HashAggregate [codegen id : 26] +Input [6]: [i_brand_id#39, i_class_id#40, i_category_id#41, sum#46, isEmpty#47, count#48] +Keys [3]: [i_brand_id#39, i_class_id#40, i_category_id#41] +Functions [2]: [sum((cast(ss_quantity#2 as decimal(10,0)) * ss_list_price#3)), count(1)] +Aggregate Attributes [2]: [sum((cast(ss_quantity#2 as decimal(10,0)) * ss_list_price#3))#49, count(1)#50] +Results [6]: [store AS channel#51, i_brand_id#39, i_class_id#40, i_category_id#41, sum((cast(ss_quantity#2 as decimal(10,0)) * ss_list_price#3))#49 AS sales#52, count(1)#50 AS number_sales#53] + +(66) Filter [codegen id : 26] +Input [6]: [channel#51, i_brand_id#39, i_class_id#40, i_category_id#41, sales#52, number_sales#53] +Condition : (isnotnull(sales#52) AND (cast(sales#52 as decimal(32,6)) > cast(Subquery scalar-subquery#54, [id=#55] as decimal(32,6)))) + +(67) Scan parquet spark_catalog.default.catalog_sales +Output [4]: [cs_item_sk#56, cs_quantity#57, cs_list_price#58, cs_sold_date_sk#59] +Batched: true +Location: InMemoryFileIndex [] +PartitionFilters: [isnotnull(cs_sold_date_sk#59), dynamicpruningexpression(cs_sold_date_sk#59 IN dynamicpruning#60)] +PushedFilters: [IsNotNull(cs_item_sk)] +ReadSchema: struct<cs_item_sk:int,cs_quantity:int,cs_list_price:decimal(7,2)> + +(68) CometFilter +Input [4]: [cs_item_sk#56, cs_quantity#57, cs_list_price#58, cs_sold_date_sk#59] +Condition : isnotnull(cs_item_sk#56) + +(69) ColumnarToRow [codegen id : 51] +Input [4]: [cs_item_sk#56, cs_quantity#57, cs_list_price#58, cs_sold_date_sk#59] + +(70) ReusedExchange [Reuses operator id: 50] +Output [1]: [ss_item_sk#61] + +(71) BroadcastHashJoin [codegen id : 51] +Left keys [1]: [cs_item_sk#56] +Right keys [1]: [ss_item_sk#61] +Join type: LeftSemi +Join condition: None + +(72) ReusedExchange [Reuses operator id: 57] +Output [4]: [i_item_sk#62, i_brand_id#63, i_class_id#64, i_category_id#65] + +(73) BroadcastHashJoin [codegen id : 51] +Left keys [1]: [cs_item_sk#56] +Right keys [1]: [i_item_sk#62] +Join type: Inner +Join condition: None + +(74) Project [codegen id : 51] +Output [6]: [cs_quantity#57, cs_list_price#58, cs_sold_date_sk#59, i_brand_id#63, i_class_id#64, i_category_id#65] +Input [8]: [cs_item_sk#56, cs_quantity#57, cs_list_price#58, cs_sold_date_sk#59, i_item_sk#62, i_brand_id#63, i_class_id#64, i_category_id#65] + +(75) ReusedExchange [Reuses operator id: 154] +Output [1]: [d_date_sk#66] + +(76) BroadcastHashJoin [codegen id : 51] +Left keys [1]: [cs_sold_date_sk#59] +Right keys [1]: [d_date_sk#66] +Join type: Inner +Join condition: None + +(77) Project [codegen id : 51] +Output [5]: [cs_quantity#57, cs_list_price#58, i_brand_id#63, i_class_id#64, i_category_id#65] +Input [7]: [cs_quantity#57, cs_list_price#58, cs_sold_date_sk#59, i_brand_id#63, i_class_id#64, i_category_id#65, d_date_sk#66] + +(78) HashAggregate [codegen id : 51] +Input [5]: [cs_quantity#57, cs_list_price#58, i_brand_id#63, i_class_id#64, i_category_id#65] +Keys [3]: [i_brand_id#63, i_class_id#64, i_category_id#65] +Functions [2]: [partial_sum((cast(cs_quantity#57 as decimal(10,0)) * cs_list_price#58)), partial_count(1)] +Aggregate Attributes [3]: [sum#67, isEmpty#68, count#69] +Results [6]: [i_brand_id#63, i_class_id#64, i_category_id#65, sum#70, isEmpty#71, count#72] + +(79) Exchange +Input [6]: [i_brand_id#63, i_class_id#64, i_category_id#65, sum#70, isEmpty#71, count#72] +Arguments: hashpartitioning(i_brand_id#63, i_class_id#64, i_category_id#65, 5), ENSURE_REQUIREMENTS, [plan_id=10] + +(80) HashAggregate [codegen id : 52] +Input [6]: [i_brand_id#63, i_class_id#64, i_category_id#65, sum#70, isEmpty#71, count#72] +Keys [3]: [i_brand_id#63, i_class_id#64, i_category_id#65] +Functions [2]: [sum((cast(cs_quantity#57 as decimal(10,0)) * cs_list_price#58)), count(1)] +Aggregate Attributes [2]: [sum((cast(cs_quantity#57 as decimal(10,0)) * cs_list_price#58))#73, count(1)#74] +Results [6]: [catalog AS channel#75, i_brand_id#63, i_class_id#64, i_category_id#65, sum((cast(cs_quantity#57 as decimal(10,0)) * cs_list_price#58))#73 AS sales#76, count(1)#74 AS number_sales#77] + +(81) Filter [codegen id : 52] +Input [6]: [channel#75, i_brand_id#63, i_class_id#64, i_category_id#65, sales#76, number_sales#77] +Condition : (isnotnull(sales#76) AND (cast(sales#76 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#54, [id=#55] as decimal(32,6)))) + +(82) Scan parquet spark_catalog.default.web_sales +Output [4]: [ws_item_sk#78, ws_quantity#79, ws_list_price#80, ws_sold_date_sk#81] +Batched: true +Location: InMemoryFileIndex [] +PartitionFilters: [isnotnull(ws_sold_date_sk#81), dynamicpruningexpression(ws_sold_date_sk#81 IN dynamicpruning#82)] +PushedFilters: [IsNotNull(ws_item_sk)] +ReadSchema: struct<ws_item_sk:int,ws_quantity:int,ws_list_price:decimal(7,2)> + +(83) CometFilter +Input [4]: [ws_item_sk#78, ws_quantity#79, ws_list_price#80, ws_sold_date_sk#81] +Condition : isnotnull(ws_item_sk#78) + +(84) ColumnarToRow [codegen id : 77] +Input [4]: [ws_item_sk#78, ws_quantity#79, ws_list_price#80, ws_sold_date_sk#81] + +(85) ReusedExchange [Reuses operator id: 50] +Output [1]: [ss_item_sk#83] + +(86) BroadcastHashJoin [codegen id : 77] +Left keys [1]: [ws_item_sk#78] +Right keys [1]: [ss_item_sk#83] +Join type: LeftSemi +Join condition: None + +(87) ReusedExchange [Reuses operator id: 57] +Output [4]: [i_item_sk#84, i_brand_id#85, i_class_id#86, i_category_id#87] + +(88) BroadcastHashJoin [codegen id : 77] +Left keys [1]: [ws_item_sk#78] +Right keys [1]: [i_item_sk#84] +Join type: Inner +Join condition: None + +(89) Project [codegen id : 77] +Output [6]: [ws_quantity#79, ws_list_price#80, ws_sold_date_sk#81, i_brand_id#85, i_class_id#86, i_category_id#87] +Input [8]: [ws_item_sk#78, ws_quantity#79, ws_list_price#80, ws_sold_date_sk#81, i_item_sk#84, i_brand_id#85, i_class_id#86, i_category_id#87] + +(90) ReusedExchange [Reuses operator id: 154] +Output [1]: [d_date_sk#88] + +(91) BroadcastHashJoin [codegen id : 77] +Left keys [1]: [ws_sold_date_sk#81] +Right keys [1]: [d_date_sk#88] +Join type: Inner +Join condition: None + +(92) Project [codegen id : 77] +Output [5]: [ws_quantity#79, ws_list_price#80, i_brand_id#85, i_class_id#86, i_category_id#87] +Input [7]: [ws_quantity#79, ws_list_price#80, ws_sold_date_sk#81, i_brand_id#85, i_class_id#86, i_category_id#87, d_date_sk#88] + +(93) HashAggregate [codegen id : 77] +Input [5]: [ws_quantity#79, ws_list_price#80, i_brand_id#85, i_class_id#86, i_category_id#87] +Keys [3]: [i_brand_id#85, i_class_id#86, i_category_id#87] +Functions [2]: [partial_sum((cast(ws_quantity#79 as decimal(10,0)) * ws_list_price#80)), partial_count(1)] +Aggregate Attributes [3]: [sum#89, isEmpty#90, count#91] +Results [6]: [i_brand_id#85, i_class_id#86, i_category_id#87, sum#92, isEmpty#93, count#94] + +(94) Exchange +Input [6]: [i_brand_id#85, i_class_id#86, i_category_id#87, sum#92, isEmpty#93, count#94] +Arguments: hashpartitioning(i_brand_id#85, i_class_id#86, i_category_id#87, 5), ENSURE_REQUIREMENTS, [plan_id=11] + +(95) HashAggregate [codegen id : 78] +Input [6]: [i_brand_id#85, i_class_id#86, i_category_id#87, sum#92, isEmpty#93, count#94] +Keys [3]: [i_brand_id#85, i_class_id#86, i_category_id#87] +Functions [2]: [sum((cast(ws_quantity#79 as decimal(10,0)) * ws_list_price#80)), count(1)] +Aggregate Attributes [2]: [sum((cast(ws_quantity#79 as decimal(10,0)) * ws_list_price#80))#95, count(1)#96] +Results [6]: [web AS channel#97, i_brand_id#85, i_class_id#86, i_category_id#87, sum((cast(ws_quantity#79 as decimal(10,0)) * ws_list_price#80))#95 AS sales#98, count(1)#96 AS number_sales#99] + +(96) Filter [codegen id : 78] +Input [6]: [channel#97, i_brand_id#85, i_class_id#86, i_category_id#87, sales#98, number_sales#99] +Condition : (isnotnull(sales#98) AND (cast(sales#98 as decimal(32,6)) > cast(ReusedSubquery Subquery scalar-subquery#54, [id=#55] as decimal(32,6)))) + +(97) Union + +(98) HashAggregate [codegen id : 79] +Input [6]: [channel#51, i_brand_id#39, i_class_id#40, i_category_id#41, sales#52, number_sales#53] +Keys [4]: [channel#51, i_brand_id#39, i_class_id#40, i_category_id#41] +Functions [2]: [partial_sum(sales#52), partial_sum(number_sales#53)] +Aggregate Attributes [3]: [sum#100, isEmpty#101, sum#102] +Results [7]: [channel#51, i_brand_id#39, i_class_id#40, i_category_id#41, sum#103, isEmpty#104, sum#105] + +(99) Exchange +Input [7]: [channel#51, i_brand_id#39, i_class_id#40, i_category_id#41, sum#103, isEmpty#104, sum#105] +Arguments: hashpartitioning(channel#51, i_brand_id#39, i_class_id#40, i_category_id#41, 5), ENSURE_REQUIREMENTS, [plan_id=12] + +(100) HashAggregate [codegen id : 80] +Input [7]: [channel#51, i_brand_id#39, i_class_id#40, i_category_id#41, sum#103, isEmpty#104, sum#105] +Keys [4]: [channel#51, i_brand_id#39, i_class_id#40, i_category_id#41] +Functions [2]: [sum(sales#52), sum(number_sales#53)] +Aggregate Attributes [2]: [sum(sales#52)#106, sum(number_sales#53)#107] +Results [6]: [channel#51, i_brand_id#39, i_class_id#40, i_category_id#41, sum(sales#52)#106 AS sum_sales#108, sum(number_sales#53)#107 AS number_sales#109] + +(101) ReusedExchange [Reuses operator id: 99] +Output [7]: [channel#110, i_brand_id#111, i_class_id#112, i_category_id#113, sum#114, isEmpty#115, sum#116] + +(102) HashAggregate [codegen id : 160] +Input [7]: [channel#110, i_brand_id#111, i_class_id#112, i_category_id#113, sum#114, isEmpty#115, sum#116] +Keys [4]: [channel#110, i_brand_id#111, i_class_id#112, i_category_id#113] +Functions [2]: [sum(sales#117), sum(number_sales#118)] +Aggregate Attributes [2]: [sum(sales#117)#106, sum(number_sales#118)#107] +Results [5]: [channel#110, i_brand_id#111, i_class_id#112, sum(sales#117)#106 AS sum_sales#119, sum(number_sales#118)#107 AS number_sales#120] + +(103) HashAggregate [codegen id : 160] +Input [5]: [channel#110, i_brand_id#111, i_class_id#112, sum_sales#119, number_sales#120] +Keys [3]: [channel#110, i_brand_id#111, i_class_id#112] +Functions [2]: [partial_sum(sum_sales#119), partial_sum(number_sales#120)] +Aggregate Attributes [3]: [sum#121, isEmpty#122, sum#123] +Results [6]: [channel#110, i_brand_id#111, i_class_id#112, sum#124, isEmpty#125, sum#126] + +(104) Exchange +Input [6]: [channel#110, i_brand_id#111, i_class_id#112, sum#124, isEmpty#125, sum#126] +Arguments: hashpartitioning(channel#110, i_brand_id#111, i_class_id#112, 5), ENSURE_REQUIREMENTS, [plan_id=13] + +(105) HashAggregate [codegen id : 161] +Input [6]: [channel#110, i_brand_id#111, i_class_id#112, sum#124, isEmpty#125, sum#126] +Keys [3]: [channel#110, i_brand_id#111, i_class_id#112] +Functions [2]: [sum(sum_sales#119), sum(number_sales#120)] +Aggregate Attributes [2]: [sum(sum_sales#119)#127, sum(number_sales#120)#128] +Results [6]: [channel#110, i_brand_id#111, i_class_id#112, null AS i_category_id#129, sum(sum_sales#119)#127 AS sum(sum_sales)#130, sum(number_sales#120)#128 AS sum(number_sales)#131] + +(106) ReusedExchange [Reuses operator id: 99] +Output [7]: [channel#132, i_brand_id#133, i_class_id#134, i_category_id#135, sum#136, isEmpty#137, sum#138] + +(107) HashAggregate [codegen id : 241] +Input [7]: [channel#132, i_brand_id#133, i_class_id#134, i_category_id#135, sum#136, isEmpty#137, sum#138] +Keys [4]: [channel#132, i_brand_id#133, i_class_id#134, i_category_id#135] +Functions [2]: [sum(sales#139), sum(number_sales#140)] +Aggregate Attributes [2]: [sum(sales#139)#106, sum(number_sales#140)#107] +Results [4]: [channel#132, i_brand_id#133, sum(sales#139)#106 AS sum_sales#141, sum(number_sales#140)#107 AS number_sales#142] + +(108) HashAggregate [codegen id : 241] +Input [4]: [channel#132, i_brand_id#133, sum_sales#141, number_sales#142] +Keys [2]: [channel#132, i_brand_id#133] +Functions [2]: [partial_sum(sum_sales#141), partial_sum(number_sales#142)] +Aggregate Attributes [3]: [sum#143, isEmpty#144, sum#145] +Results [5]: [channel#132, i_brand_id#133, sum#146, isEmpty#147, sum#148] + +(109) Exchange +Input [5]: [channel#132, i_brand_id#133, sum#146, isEmpty#147, sum#148] +Arguments: hashpartitioning(channel#132, i_brand_id#133, 5), ENSURE_REQUIREMENTS, [plan_id=14] + +(110) HashAggregate [codegen id : 242] +Input [5]: [channel#132, i_brand_id#133, sum#146, isEmpty#147, sum#148] +Keys [2]: [channel#132, i_brand_id#133] +Functions [2]: [sum(sum_sales#141), sum(number_sales#142)] +Aggregate Attributes [2]: [sum(sum_sales#141)#149, sum(number_sales#142)#150] +Results [6]: [channel#132, i_brand_id#133, null AS i_class_id#151, null AS i_category_id#152, sum(sum_sales#141)#149 AS sum(sum_sales)#153, sum(number_sales#142)#150 AS sum(number_sales)#154] + +(111) ReusedExchange [Reuses operator id: 99] +Output [7]: [channel#155, i_brand_id#156, i_class_id#157, i_category_id#158, sum#159, isEmpty#160, sum#161] + +(112) HashAggregate [codegen id : 322] +Input [7]: [channel#155, i_brand_id#156, i_class_id#157, i_category_id#158, sum#159, isEmpty#160, sum#161] +Keys [4]: [channel#155, i_brand_id#156, i_class_id#157, i_category_id#158] +Functions [2]: [sum(sales#162), sum(number_sales#163)] +Aggregate Attributes [2]: [sum(sales#162)#106, sum(number_sales#163)#107] +Results [3]: [channel#155, sum(sales#162)#106 AS sum_sales#164, sum(number_sales#163)#107 AS number_sales#165] + +(113) HashAggregate [codegen id : 322] +Input [3]: [channel#155, sum_sales#164, number_sales#165] +Keys [1]: [channel#155] +Functions [2]: [partial_sum(sum_sales#164), partial_sum(number_sales#165)] +Aggregate Attributes [3]: [sum#166, isEmpty#167, sum#168] +Results [4]: [channel#155, sum#169, isEmpty#170, sum#171] + +(114) Exchange +Input [4]: [channel#155, sum#169, isEmpty#170, sum#171] +Arguments: hashpartitioning(channel#155, 5), ENSURE_REQUIREMENTS, [plan_id=15] + +(115) HashAggregate [codegen id : 323] +Input [4]: [channel#155, sum#169, isEmpty#170, sum#171] +Keys [1]: [channel#155] +Functions [2]: [sum(sum_sales#164), sum(number_sales#165)] +Aggregate Attributes [2]: [sum(sum_sales#164)#172, sum(number_sales#165)#173] +Results [6]: [channel#155, null AS i_brand_id#174, null AS i_class_id#175, null AS i_category_id#176, sum(sum_sales#164)#172 AS sum(sum_sales)#177, sum(number_sales#165)#173 AS sum(number_sales)#178] + +(116) ReusedExchange [Reuses operator id: 99] +Output [7]: [channel#179, i_brand_id#180, i_class_id#181, i_category_id#182, sum#183, isEmpty#184, sum#185] + +(117) HashAggregate [codegen id : 403] +Input [7]: [channel#179, i_brand_id#180, i_class_id#181, i_category_id#182, sum#183, isEmpty#184, sum#185] +Keys [4]: [channel#179, i_brand_id#180, i_class_id#181, i_category_id#182] +Functions [2]: [sum(sales#186), sum(number_sales#187)] +Aggregate Attributes [2]: [sum(sales#186)#106, sum(number_sales#187)#107] +Results [2]: [sum(sales#186)#106 AS sum_sales#188, sum(number_sales#187)#107 AS number_sales#189] + +(118) HashAggregate [codegen id : 403] +Input [2]: [sum_sales#188, number_sales#189] +Keys: [] +Functions [2]: [partial_sum(sum_sales#188), partial_sum(number_sales#189)] +Aggregate Attributes [3]: [sum#190, isEmpty#191, sum#192] +Results [3]: [sum#193, isEmpty#194, sum#195] + +(119) Exchange +Input [3]: [sum#193, isEmpty#194, sum#195] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=16] + +(120) HashAggregate [codegen id : 404] +Input [3]: [sum#193, isEmpty#194, sum#195] +Keys: [] +Functions [2]: [sum(sum_sales#188), sum(number_sales#189)] +Aggregate Attributes [2]: [sum(sum_sales#188)#196, sum(number_sales#189)#197] +Results [6]: [null AS channel#198, null AS i_brand_id#199, null AS i_class_id#200, null AS i_category_id#201, sum(sum_sales#188)#196 AS sum(sum_sales)#202, sum(number_sales#189)#197 AS sum(number_sales)#203] + +(121) Union + +(122) HashAggregate [codegen id : 405] +Input [6]: [channel#51, i_brand_id#39, i_class_id#40, i_category_id#41, sum_sales#108, number_sales#109] +Keys [6]: [channel#51, i_brand_id#39, i_class_id#40, i_category_id#41, sum_sales#108, number_sales#109] +Functions: [] +Aggregate Attributes: [] +Results [6]: [channel#51, i_brand_id#39, i_class_id#40, i_category_id#41, sum_sales#108, number_sales#109] + +(123) Exchange +Input [6]: [channel#51, i_brand_id#39, i_class_id#40, i_category_id#41, sum_sales#108, number_sales#109] +Arguments: hashpartitioning(channel#51, i_brand_id#39, i_class_id#40, i_category_id#41, sum_sales#108, number_sales#109, 5), ENSURE_REQUIREMENTS, [plan_id=17] + +(124) HashAggregate [codegen id : 406] +Input [6]: [channel#51, i_brand_id#39, i_class_id#40, i_category_id#41, sum_sales#108, number_sales#109] +Keys [6]: [channel#51, i_brand_id#39, i_class_id#40, i_category_id#41, sum_sales#108, number_sales#109] +Functions: [] +Aggregate Attributes: [] +Results [6]: [channel#51, i_brand_id#39, i_class_id#40, i_category_id#41, sum_sales#108, number_sales#109] + +(125) TakeOrderedAndProject +Input [6]: [channel#51, i_brand_id#39, i_class_id#40, i_category_id#41, sum_sales#108, number_sales#109] +Arguments: 100, [channel#51 ASC NULLS FIRST, i_brand_id#39 ASC NULLS FIRST, i_class_id#40 ASC NULLS FIRST, i_category_id#41 ASC NULLS FIRST], [channel#51, i_brand_id#39, i_class_id#40, i_category_id#41, sum_sales#108, number_sales#109] + +===== Subqueries ===== + +Subquery:1 Hosting operator id = 66 Hosting Expression = Subquery scalar-subquery#54, [id=#55] +* HashAggregate (144) ++- Exchange (143) + +- * HashAggregate (142) + +- Union (141) + :- * Project (130) + : +- * BroadcastHashJoin Inner BuildRight (129) + : :- * ColumnarToRow (127) + : : +- CometScan parquet spark_catalog.default.store_sales (126) + : +- ReusedExchange (128) + :- * Project (135) + : +- * BroadcastHashJoin Inner BuildRight (134) + : :- * ColumnarToRow (132) + : : +- CometScan parquet spark_catalog.default.catalog_sales (131) + : +- ReusedExchange (133) + +- * Project (140) + +- * BroadcastHashJoin Inner BuildRight (139) + :- * ColumnarToRow (137) + : +- CometScan parquet spark_catalog.default.web_sales (136) + +- ReusedExchange (138) + + +(126) Scan parquet spark_catalog.default.store_sales +Output [3]: [ss_quantity#204, ss_list_price#205, ss_sold_date_sk#206] +Batched: true +Location: InMemoryFileIndex [] +PartitionFilters: [isnotnull(ss_sold_date_sk#206), dynamicpruningexpression(ss_sold_date_sk#206 IN dynamicpruning#207)] +ReadSchema: struct<ss_quantity:int,ss_list_price:decimal(7,2)> + +(127) ColumnarToRow [codegen id : 2] +Input [3]: [ss_quantity#204, ss_list_price#205, ss_sold_date_sk#206] + +(128) ReusedExchange [Reuses operator id: 159] +Output [1]: [d_date_sk#208] + +(129) BroadcastHashJoin [codegen id : 2] +Left keys [1]: [ss_sold_date_sk#206] +Right keys [1]: [d_date_sk#208] +Join type: Inner +Join condition: None + +(130) Project [codegen id : 2] +Output [2]: [ss_quantity#204 AS quantity#209, ss_list_price#205 AS list_price#210] +Input [4]: [ss_quantity#204, ss_list_price#205, ss_sold_date_sk#206, d_date_sk#208] + +(131) Scan parquet spark_catalog.default.catalog_sales +Output [3]: [cs_quantity#211, cs_list_price#212, cs_sold_date_sk#213] +Batched: true +Location: InMemoryFileIndex [] +PartitionFilters: [isnotnull(cs_sold_date_sk#213), dynamicpruningexpression(cs_sold_date_sk#213 IN dynamicpruning#214)] +ReadSchema: struct<cs_quantity:int,cs_list_price:decimal(7,2)> + +(132) ColumnarToRow [codegen id : 4] +Input [3]: [cs_quantity#211, cs_list_price#212, cs_sold_date_sk#213] + +(133) ReusedExchange [Reuses operator id: 149] +Output [1]: [d_date_sk#215] + +(134) BroadcastHashJoin [codegen id : 4] +Left keys [1]: [cs_sold_date_sk#213] +Right keys [1]: [d_date_sk#215] +Join type: Inner +Join condition: None + +(135) Project [codegen id : 4] +Output [2]: [cs_quantity#211 AS quantity#216, cs_list_price#212 AS list_price#217] +Input [4]: [cs_quantity#211, cs_list_price#212, cs_sold_date_sk#213, d_date_sk#215] + +(136) Scan parquet spark_catalog.default.web_sales +Output [3]: [ws_quantity#218, ws_list_price#219, ws_sold_date_sk#220] +Batched: true +Location: InMemoryFileIndex [] +PartitionFilters: [isnotnull(ws_sold_date_sk#220), dynamicpruningexpression(ws_sold_date_sk#220 IN dynamicpruning#221)] +ReadSchema: struct<ws_quantity:int,ws_list_price:decimal(7,2)> + +(137) ColumnarToRow [codegen id : 6] +Input [3]: [ws_quantity#218, ws_list_price#219, ws_sold_date_sk#220] + +(138) ReusedExchange [Reuses operator id: 149] +Output [1]: [d_date_sk#222] + +(139) BroadcastHashJoin [codegen id : 6] +Left keys [1]: [ws_sold_date_sk#220] +Right keys [1]: [d_date_sk#222] +Join type: Inner +Join condition: None + +(140) Project [codegen id : 6] +Output [2]: [ws_quantity#218 AS quantity#223, ws_list_price#219 AS list_price#224] +Input [4]: [ws_quantity#218, ws_list_price#219, ws_sold_date_sk#220, d_date_sk#222] + +(141) Union + +(142) HashAggregate [codegen id : 7] +Input [2]: [quantity#209, list_price#210] +Keys: [] +Functions [1]: [partial_avg((cast(quantity#209 as decimal(10,0)) * list_price#210))] +Aggregate Attributes [2]: [sum#225, count#226] +Results [2]: [sum#227, count#228] + +(143) Exchange +Input [2]: [sum#227, count#228] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=18] + +(144) HashAggregate [codegen id : 8] +Input [2]: [sum#227, count#228] +Keys: [] +Functions [1]: [avg((cast(quantity#209 as decimal(10,0)) * list_price#210))] +Aggregate Attributes [1]: [avg((cast(quantity#209 as decimal(10,0)) * list_price#210))#229] +Results [1]: [avg((cast(quantity#209 as decimal(10,0)) * list_price#210))#229 AS average_sales#230] + +Subquery:2 Hosting operator id = 126 Hosting Expression = ss_sold_date_sk#206 IN dynamicpruning#12 + +Subquery:3 Hosting operator id = 131 Hosting Expression = cs_sold_date_sk#213 IN dynamicpruning#214 +BroadcastExchange (149) ++- * ColumnarToRow (148) + +- CometProject (147) + +- CometFilter (146) + +- CometScan parquet spark_catalog.default.date_dim (145) + + +(145) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#215, d_year#231] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_year), GreaterThanOrEqual(d_year,1998), LessThanOrEqual(d_year,2000), IsNotNull(d_date_sk)] +ReadSchema: struct<d_date_sk:int,d_year:int> + +(146) CometFilter +Input [2]: [d_date_sk#215, d_year#231] +Condition : (((isnotnull(d_year#231) AND (d_year#231 >= 1998)) AND (d_year#231 <= 2000)) AND isnotnull(d_date_sk#215)) + +(147) CometProject +Input [2]: [d_date_sk#215, d_year#231] +Arguments: [d_date_sk#215], [d_date_sk#215] + +(148) ColumnarToRow [codegen id : 1] +Input [1]: [d_date_sk#215] + +(149) BroadcastExchange +Input [1]: [d_date_sk#215] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=19] + +Subquery:4 Hosting operator id = 136 Hosting Expression = ws_sold_date_sk#220 IN dynamicpruning#214 + +Subquery:5 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 +BroadcastExchange (154) ++- * ColumnarToRow (153) + +- CometProject (152) + +- CometFilter (151) + +- CometScan parquet spark_catalog.default.date_dim (150) + + +(150) Scan parquet spark_catalog.default.date_dim +Output [3]: [d_date_sk#42, d_year#232, d_moy#233] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2000), EqualTo(d_moy,11), IsNotNull(d_date_sk)] +ReadSchema: struct<d_date_sk:int,d_year:int,d_moy:int> + +(151) CometFilter +Input [3]: [d_date_sk#42, d_year#232, d_moy#233] +Condition : ((((isnotnull(d_year#232) AND isnotnull(d_moy#233)) AND (d_year#232 = 2000)) AND (d_moy#233 = 11)) AND isnotnull(d_date_sk#42)) + +(152) CometProject +Input [3]: [d_date_sk#42, d_year#232, d_moy#233] +Arguments: [d_date_sk#42], [d_date_sk#42] + +(153) ColumnarToRow [codegen id : 1] +Input [1]: [d_date_sk#42] + +(154) BroadcastExchange +Input [1]: [d_date_sk#42] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=20] + +Subquery:6 Hosting operator id = 7 Hosting Expression = ss_sold_date_sk#11 IN dynamicpruning#12 +BroadcastExchange (159) ++- * ColumnarToRow (158) + +- CometProject (157) + +- CometFilter (156) + +- CometScan parquet spark_catalog.default.date_dim (155) + + +(155) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#25, d_year#234] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_year), GreaterThanOrEqual(d_year,1999), LessThanOrEqual(d_year,2001), IsNotNull(d_date_sk)] +ReadSchema: struct<d_date_sk:int,d_year:int> + +(156) CometFilter +Input [2]: [d_date_sk#25, d_year#234] +Condition : (((isnotnull(d_year#234) AND (d_year#234 >= 1999)) AND (d_year#234 <= 2001)) AND isnotnull(d_date_sk#25)) + +(157) CometProject +Input [2]: [d_date_sk#25, d_year#234] +Arguments: [d_date_sk#25], [d_date_sk#25] + +(158) ColumnarToRow [codegen id : 1] +Input [1]: [d_date_sk#25] + +(159) BroadcastExchange +Input [1]: [d_date_sk#25] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=21] + +Subquery:7 Hosting operator id = 13 Hosting Expression = cs_sold_date_sk#18 IN dynamicpruning#12 + +Subquery:8 Hosting operator id = 36 Hosting Expression = ws_sold_date_sk#30 IN dynamicpruning#12 + +Subquery:9 Hosting operator id = 81 Hosting Expression = ReusedSubquery Subquery scalar-subquery#54, [id=#55] + +Subquery:10 Hosting operator id = 67 Hosting Expression = cs_sold_date_sk#59 IN dynamicpruning#5 + +Subquery:11 Hosting operator id = 96 Hosting Expression = ReusedSubquery Subquery scalar-subquery#54, [id=#55] + +Subquery:12 Hosting operator id = 82 Hosting Expression = ws_sold_date_sk#81 IN dynamicpruning#5 + + diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q14a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q14a/simplified.txt new file mode 100644 index 000000000..a203f9620 --- /dev/null +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q14a/simplified.txt @@ -0,0 +1,261 @@ +TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum_sales,number_sales] + WholeStageCodegen (406) + HashAggregate [channel,i_brand_id,i_class_id,i_category_id,sum_sales,number_sales] + InputAdapter + Exchange [channel,i_brand_id,i_class_id,i_category_id,sum_sales,number_sales] #1 + WholeStageCodegen (405) + 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] + InputAdapter + Exchange [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] + InputAdapter + Exchange #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 + CometScan parquet spark_catalog.default.store_sales [ss_quantity,ss_list_price,ss_sold_date_sk] + ReusedSubquery [d_date_sk] #2 + InputAdapter + ReusedExchange [d_date_sk] #8 + WholeStageCodegen (4) + Project [cs_quantity,cs_list_price] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + ColumnarToRow + InputAdapter + CometScan parquet spark_catalog.default.catalog_sales [cs_quantity,cs_list_price,cs_sold_date_sk] + SubqueryBroadcast [d_date_sk] #4 + BroadcastExchange #15 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_year,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + InputAdapter + ReusedExchange [d_date_sk] #15 + WholeStageCodegen (6) + Project [ws_quantity,ws_list_price] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + ColumnarToRow + InputAdapter + CometScan parquet spark_catalog.default.web_sales [ws_quantity,ws_list_price,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #4 + InputAdapter + ReusedExchange [d_date_sk] #15 + HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum((cast(ss_quantity as decimal(10,0)) * ss_list_price)),count(1),channel,sales,number_sales,sum,isEmpty,count] + InputAdapter + Exchange [i_brand_id,i_class_id,i_category_id] #3 + WholeStageCodegen (25) + HashAggregate [i_brand_id,i_class_id,i_category_id,ss_quantity,ss_list_price] [sum,isEmpty,count,sum,isEmpty,count] + Project [ss_quantity,ss_list_price,i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [ss_quantity,ss_list_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [ss_item_sk,i_item_sk] + BroadcastHashJoin [ss_item_sk,ss_item_sk] + ColumnarToRow + InputAdapter + CometFilter [ss_item_sk] + CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_quantity,ss_list_price,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #4 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_year,d_moy,d_date_sk] + CometScan 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] + ColumnarToRow + InputAdapter + CometFilter [i_brand_id,i_class_id,i_category_id] + CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] + InputAdapter + BroadcastExchange #6 + WholeStageCodegen (10) + BroadcastHashJoin [brand_id,class_id,category_id,i_brand_id,i_class_id,i_category_id] + HashAggregate [brand_id,class_id,category_id] + InputAdapter + Exchange [brand_id,class_id,category_id] #7 + WholeStageCodegen (6) + HashAggregate [brand_id,class_id,category_id] + Project [i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [ss_sold_date_sk,i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [ss_item_sk,i_item_sk] + ColumnarToRow + InputAdapter + CometFilter [ss_item_sk] + CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #2 + BroadcastExchange #8 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_year,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + InputAdapter + BroadcastExchange #9 + WholeStageCodegen (4) + BroadcastHashJoin [i_brand_id,i_class_id,i_category_id,i_brand_id,i_class_id,i_category_id] + ColumnarToRow + InputAdapter + CometFilter [i_item_sk,i_brand_id,i_class_id,i_category_id] + CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] + InputAdapter + BroadcastExchange #10 + WholeStageCodegen (3) + Project [i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + Project [cs_sold_date_sk,i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [cs_item_sk,i_item_sk] + ColumnarToRow + InputAdapter + CometFilter [cs_item_sk] + CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #2 + InputAdapter + BroadcastExchange #11 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometFilter [i_item_sk] + CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] + InputAdapter + ReusedExchange [d_date_sk] #8 + InputAdapter + ReusedExchange [d_date_sk] #8 + InputAdapter + BroadcastExchange #12 + WholeStageCodegen (9) + Project [i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + Project [ws_sold_date_sk,i_brand_id,i_class_id,i_category_id] + BroadcastHashJoin [ws_item_sk,i_item_sk] + ColumnarToRow + InputAdapter + CometFilter [ws_item_sk] + CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #2 + InputAdapter + ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #11 + InputAdapter + ReusedExchange [d_date_sk] #8 + InputAdapter + BroadcastExchange #13 + WholeStageCodegen (23) + BroadcastHashJoin [i_item_sk,ss_item_sk] + ColumnarToRow + InputAdapter + CometFilter [i_item_sk] + CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id] + InputAdapter + ReusedExchange [ss_item_sk] #5 + 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] + InputAdapter + Exchange [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] + ColumnarToRow + InputAdapter + CometFilter [cs_item_sk] + CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_quantity,cs_list_price,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + InputAdapter + ReusedExchange [ss_item_sk] #5 + InputAdapter + ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #13 + 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] + InputAdapter + Exchange [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] + ColumnarToRow + InputAdapter + CometFilter [ws_item_sk] + CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_quantity,ws_list_price,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + InputAdapter + ReusedExchange [ss_item_sk] #5 + InputAdapter + ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id] #13 + 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] + InputAdapter + Exchange [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] + 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] + InputAdapter + Exchange [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] + 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] + InputAdapter + Exchange [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] + 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] + InputAdapter + Exchange #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] + InputAdapter + ReusedExchange [channel,i_brand_id,i_class_id,i_category_id,sum,isEmpty,sum] #2 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q18a/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q18a/explain.txt new file mode 100644 index 000000000..8bcc0c6ba --- /dev/null +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q18a/explain.txt @@ -0,0 +1,909 @@ +== Physical Plan == +TakeOrderedAndProject (153) ++- Union (152) + :- * HashAggregate (41) + : +- Exchange (40) + : +- * HashAggregate (39) + : +- * Project (38) + : +- * BroadcastHashJoin Inner BuildRight (37) + : :- * Project (32) + : : +- * BroadcastHashJoin Inner BuildRight (31) + : : :- * Project (29) + : : : +- * BroadcastHashJoin Inner BuildRight (28) + : : : :- * Project (23) + : : : : +- * BroadcastHashJoin Inner BuildRight (22) + : : : : :- * Project (17) + : : : : : +- * BroadcastHashJoin Inner BuildRight (16) + : : : : : :- * Project (10) + : : : : : : +- * BroadcastHashJoin Inner BuildRight (9) + : : : : : : :- * ColumnarToRow (3) + : : : : : : : +- CometFilter (2) + : : : : : : : +- CometScan parquet spark_catalog.default.catalog_sales (1) + : : : : : : +- BroadcastExchange (8) + : : : : : : +- * ColumnarToRow (7) + : : : : : : +- CometProject (6) + : : : : : : +- CometFilter (5) + : : : : : : +- CometScan parquet spark_catalog.default.customer_demographics (4) + : : : : : +- BroadcastExchange (15) + : : : : : +- * ColumnarToRow (14) + : : : : : +- CometProject (13) + : : : : : +- CometFilter (12) + : : : : : +- CometScan parquet spark_catalog.default.customer (11) + : : : : +- BroadcastExchange (21) + : : : : +- * ColumnarToRow (20) + : : : : +- CometFilter (19) + : : : : +- CometScan parquet spark_catalog.default.customer_demographics (18) + : : : +- BroadcastExchange (27) + : : : +- * ColumnarToRow (26) + : : : +- CometFilter (25) + : : : +- CometScan parquet spark_catalog.default.customer_address (24) + : : +- ReusedExchange (30) + : +- BroadcastExchange (36) + : +- * ColumnarToRow (35) + : +- CometFilter (34) + : +- CometScan parquet spark_catalog.default.item (33) + :- * HashAggregate (68) + : +- Exchange (67) + : +- * HashAggregate (66) + : +- * Project (65) + : +- * BroadcastHashJoin Inner BuildRight (64) + : :- * Project (62) + : : +- * BroadcastHashJoin Inner BuildRight (61) + : : :- * Project (59) + : : : +- * BroadcastHashJoin Inner BuildRight (58) + : : : :- * Project (53) + : : : : +- * BroadcastHashJoin Inner BuildRight (52) + : : : : :- * Project (50) + : : : : : +- * BroadcastHashJoin Inner BuildRight (49) + : : : : : :- * Project (47) + : : : : : : +- * BroadcastHashJoin Inner BuildRight (46) + : : : : : : :- * ColumnarToRow (44) + : : : : : : : +- CometFilter (43) + : : : : : : : +- CometScan parquet spark_catalog.default.catalog_sales (42) + : : : : : : +- ReusedExchange (45) + : : : : : +- ReusedExchange (48) + : : : : +- ReusedExchange (51) + : : : +- BroadcastExchange (57) + : : : +- * ColumnarToRow (56) + : : : +- CometFilter (55) + : : : +- CometScan parquet spark_catalog.default.customer_address (54) + : : +- ReusedExchange (60) + : +- ReusedExchange (63) + :- * HashAggregate (96) + : +- Exchange (95) + : +- * HashAggregate (94) + : +- * Project (93) + : +- * BroadcastHashJoin Inner BuildRight (92) + : :- * Project (90) + : : +- * BroadcastHashJoin Inner BuildRight (89) + : : :- * Project (87) + : : : +- * BroadcastHashJoin Inner BuildRight (86) + : : : :- * Project (80) + : : : : +- * BroadcastHashJoin Inner BuildRight (79) + : : : : :- * Project (77) + : : : : : +- * BroadcastHashJoin Inner BuildRight (76) + : : : : : :- * Project (74) + : : : : : : +- * BroadcastHashJoin Inner BuildRight (73) + : : : : : : :- * ColumnarToRow (71) + : : : : : : : +- CometFilter (70) + : : : : : : : +- CometScan parquet spark_catalog.default.catalog_sales (69) + : : : : : : +- ReusedExchange (72) + : : : : : +- ReusedExchange (75) + : : : : +- ReusedExchange (78) + : : : +- BroadcastExchange (85) + : : : +- * ColumnarToRow (84) + : : : +- CometProject (83) + : : : +- CometFilter (82) + : : : +- CometScan parquet spark_catalog.default.customer_address (81) + : : +- ReusedExchange (88) + : +- ReusedExchange (91) + :- * HashAggregate (124) + : +- Exchange (123) + : +- * HashAggregate (122) + : +- * Project (121) + : +- * BroadcastHashJoin Inner BuildRight (120) + : :- * Project (118) + : : +- * BroadcastHashJoin Inner BuildRight (117) + : : :- * Project (115) + : : : +- * BroadcastHashJoin Inner BuildRight (114) + : : : :- * Project (108) + : : : : +- * BroadcastHashJoin Inner BuildRight (107) + : : : : :- * Project (105) + : : : : : +- * BroadcastHashJoin Inner BuildRight (104) + : : : : : :- * Project (102) + : : : : : : +- * BroadcastHashJoin Inner BuildRight (101) + : : : : : : :- * ColumnarToRow (99) + : : : : : : : +- CometFilter (98) + : : : : : : : +- CometScan parquet spark_catalog.default.catalog_sales (97) + : : : : : : +- ReusedExchange (100) + : : : : : +- ReusedExchange (103) + : : : : +- ReusedExchange (106) + : : : +- BroadcastExchange (113) + : : : +- * ColumnarToRow (112) + : : : +- CometProject (111) + : : : +- CometFilter (110) + : : : +- CometScan parquet spark_catalog.default.customer_address (109) + : : +- ReusedExchange (116) + : +- ReusedExchange (119) + +- * HashAggregate (151) + +- Exchange (150) + +- * HashAggregate (149) + +- * Project (148) + +- * BroadcastHashJoin Inner BuildRight (147) + :- * Project (142) + : +- * BroadcastHashJoin Inner BuildRight (141) + : :- * Project (139) + : : +- * BroadcastHashJoin Inner BuildRight (138) + : : :- * Project (136) + : : : +- * BroadcastHashJoin Inner BuildRight (135) + : : : :- * Project (133) + : : : : +- * BroadcastHashJoin Inner BuildRight (132) + : : : : :- * Project (130) + : : : : : +- * BroadcastHashJoin Inner BuildRight (129) + : : : : : :- * ColumnarToRow (127) + : : : : : : +- CometFilter (126) + : : : : : : +- CometScan parquet spark_catalog.default.catalog_sales (125) + : : : : : +- ReusedExchange (128) + : : : : +- ReusedExchange (131) + : : : +- ReusedExchange (134) + : : +- ReusedExchange (137) + : +- ReusedExchange (140) + +- BroadcastExchange (146) + +- * ColumnarToRow (145) + +- CometFilter (144) + +- CometScan parquet spark_catalog.default.item (143) + + +(1) Scan parquet spark_catalog.default.catalog_sales +Output [9]: [cs_bill_customer_sk#1, cs_bill_cdemo_sk#2, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9] +Batched: true +Location: InMemoryFileIndex [] +PartitionFilters: [isnotnull(cs_sold_date_sk#9), dynamicpruningexpression(cs_sold_date_sk#9 IN dynamicpruning#10)] +PushedFilters: [IsNotNull(cs_bill_cdemo_sk), IsNotNull(cs_bill_customer_sk), IsNotNull(cs_item_sk)] +ReadSchema: struct<cs_bill_customer_sk:int,cs_bill_cdemo_sk:int,cs_item_sk:int,cs_quantity:int,cs_list_price:decimal(7,2),cs_sales_price:decimal(7,2),cs_coupon_amt:decimal(7,2),cs_net_profit:decimal(7,2)> + +(2) CometFilter +Input [9]: [cs_bill_customer_sk#1, cs_bill_cdemo_sk#2, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9] +Condition : ((isnotnull(cs_bill_cdemo_sk#2) AND isnotnull(cs_bill_customer_sk#1)) AND isnotnull(cs_item_sk#3)) + +(3) ColumnarToRow [codegen id : 7] +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] + +(4) Scan parquet spark_catalog.default.customer_demographics +Output [4]: [cd_demo_sk#11, cd_gender#12, cd_education_status#13, cd_dep_count#14] +Batched: true +Location [not included in comparison]/{warehouse_dir}/customer_demographics] +PushedFilters: [IsNotNull(cd_gender), IsNotNull(cd_education_status), EqualTo(cd_gender,M), EqualTo(cd_education_status,College ), IsNotNull(cd_demo_sk)] +ReadSchema: struct<cd_demo_sk:int,cd_gender:string,cd_education_status:string,cd_dep_count:int> + +(5) CometFilter +Input [4]: [cd_demo_sk#11, cd_gender#12, cd_education_status#13, cd_dep_count#14] +Condition : ((((isnotnull(cd_gender#12) AND isnotnull(cd_education_status#13)) AND (cd_gender#12 = M)) AND (cd_education_status#13 = College )) AND isnotnull(cd_demo_sk#11)) + +(6) CometProject +Input [4]: [cd_demo_sk#11, cd_gender#12, cd_education_status#13, cd_dep_count#14] +Arguments: [cd_demo_sk#11, cd_dep_count#14], [cd_demo_sk#11, cd_dep_count#14] + +(7) ColumnarToRow [codegen id : 1] +Input [2]: [cd_demo_sk#11, cd_dep_count#14] + +(8) BroadcastExchange +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] +Left keys [1]: [cs_bill_cdemo_sk#2] +Right keys [1]: [cd_demo_sk#11] +Join type: Inner +Join condition: None + +(10) Project [codegen id : 7] +Output [9]: [cs_bill_customer_sk#1, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14] +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] + +(11) Scan parquet spark_catalog.default.customer +Output [5]: [c_customer_sk#15, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_month#18, c_birth_year#19] +Batched: true +Location [not included in comparison]/{warehouse_dir}/customer] +PushedFilters: [In(c_birth_month, [1,10,12,4,5,9]), IsNotNull(c_customer_sk), IsNotNull(c_current_cdemo_sk), IsNotNull(c_current_addr_sk)] +ReadSchema: struct<c_customer_sk:int,c_current_cdemo_sk:int,c_current_addr_sk:int,c_birth_month:int,c_birth_year:int> + +(12) CometFilter +Input [5]: [c_customer_sk#15, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_month#18, c_birth_year#19] +Condition : (((c_birth_month#18 IN (9,5,12,4,1,10) AND isnotnull(c_customer_sk#15)) AND isnotnull(c_current_cdemo_sk#16)) AND isnotnull(c_current_addr_sk#17)) + +(13) CometProject +Input [5]: [c_customer_sk#15, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_month#18, c_birth_year#19] +Arguments: [c_customer_sk#15, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_year#19], [c_customer_sk#15, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_year#19] + +(14) ColumnarToRow [codegen id : 2] +Input [4]: [c_customer_sk#15, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_year#19] + +(15) BroadcastExchange +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] +Left keys [1]: [cs_bill_customer_sk#1] +Right keys [1]: [c_customer_sk#15] +Join type: Inner +Join condition: None + +(17) Project [codegen id : 7] +Output [11]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_current_cdemo_sk#16, c_current_addr_sk#17, c_birth_year#19] +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] + +(18) Scan parquet spark_catalog.default.customer_demographics +Output [1]: [cd_demo_sk#20] +Batched: true +Location [not included in comparison]/{warehouse_dir}/customer_demographics] +PushedFilters: [IsNotNull(cd_demo_sk)] +ReadSchema: struct<cd_demo_sk:int> + +(19) CometFilter +Input [1]: [cd_demo_sk#20] +Condition : isnotnull(cd_demo_sk#20) + +(20) ColumnarToRow [codegen id : 3] +Input [1]: [cd_demo_sk#20] + +(21) BroadcastExchange +Input [1]: [cd_demo_sk#20] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=3] + +(22) BroadcastHashJoin [codegen id : 7] +Left keys [1]: [c_current_cdemo_sk#16] +Right keys [1]: [cd_demo_sk#20] +Join type: Inner +Join condition: None + +(23) Project [codegen id : 7] +Output [10]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_current_addr_sk#17, c_birth_year#19] +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] + +(24) Scan parquet spark_catalog.default.customer_address +Output [4]: [ca_address_sk#21, ca_county#22, ca_state#23, ca_country#24] +Batched: true +Location [not included in comparison]/{warehouse_dir}/customer_address] +PushedFilters: [In(ca_state, [AL,MS,NC,ND,OK,TN,WI]), IsNotNull(ca_address_sk)] +ReadSchema: struct<ca_address_sk:int,ca_county:string,ca_state:string,ca_country:string> + +(25) CometFilter +Input [4]: [ca_address_sk#21, ca_county#22, ca_state#23, ca_country#24] +Condition : (ca_state#23 IN (ND,WI,AL,NC,OK,MS,TN) AND isnotnull(ca_address_sk#21)) + +(26) ColumnarToRow [codegen id : 4] +Input [4]: [ca_address_sk#21, ca_county#22, ca_state#23, ca_country#24] + +(27) BroadcastExchange +Input [4]: [ca_address_sk#21, ca_county#22, ca_state#23, ca_country#24] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=4] + +(28) BroadcastHashJoin [codegen id : 7] +Left keys [1]: [c_current_addr_sk#17] +Right keys [1]: [ca_address_sk#21] +Join type: Inner +Join condition: None + +(29) Project [codegen id : 7] +Output [12]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_birth_year#19, ca_county#22, ca_state#23, ca_country#24] +Input [14]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_current_addr_sk#17, c_birth_year#19, ca_address_sk#21, ca_county#22, ca_state#23, ca_country#24] + +(30) ReusedExchange [Reuses operator id: 158] +Output [1]: [d_date_sk#25] + +(31) BroadcastHashJoin [codegen id : 7] +Left keys [1]: [cs_sold_date_sk#9] +Right keys [1]: [d_date_sk#25] +Join type: Inner +Join condition: None + +(32) Project [codegen id : 7] +Output [11]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#19, ca_county#22, ca_state#23, ca_country#24] +Input [13]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, c_birth_year#19, ca_county#22, ca_state#23, ca_country#24, d_date_sk#25] + +(33) Scan parquet spark_catalog.default.item +Output [2]: [i_item_sk#26, i_item_id#27] +Batched: true +Location [not included in comparison]/{warehouse_dir}/item] +PushedFilters: [IsNotNull(i_item_sk)] +ReadSchema: struct<i_item_sk:int,i_item_id:string> + +(34) CometFilter +Input [2]: [i_item_sk#26, i_item_id#27] +Condition : isnotnull(i_item_sk#26) + +(35) ColumnarToRow [codegen id : 6] +Input [2]: [i_item_sk#26, i_item_id#27] + +(36) BroadcastExchange +Input [2]: [i_item_sk#26, i_item_id#27] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=5] + +(37) BroadcastHashJoin [codegen id : 7] +Left keys [1]: [cs_item_sk#3] +Right keys [1]: [i_item_sk#26] +Join type: Inner +Join condition: None + +(38) Project [codegen id : 7] +Output [11]: [i_item_id#27, ca_country#24, ca_state#23, ca_county#22, cast(cs_quantity#4 as decimal(12,2)) AS agg1#28, cast(cs_list_price#5 as decimal(12,2)) AS agg2#29, cast(cs_coupon_amt#7 as decimal(12,2)) AS agg3#30, cast(cs_sales_price#6 as decimal(12,2)) AS agg4#31, cast(cs_net_profit#8 as decimal(12,2)) AS agg5#32, cast(c_birth_year#19 as decimal(12,2)) AS agg6#33, cast(cd_dep_count#14 as decimal(12,2)) AS agg7#34] +Input [13]: [cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#19, ca_county#22, ca_state#23, ca_country#24, i_item_sk#26, i_item_id#27] + +(39) HashAggregate [codegen id : 7] +Input [11]: [i_item_id#27, ca_country#24, ca_state#23, ca_county#22, agg1#28, agg2#29, agg3#30, agg4#31, agg5#32, agg6#33, agg7#34] +Keys [4]: [i_item_id#27, ca_country#24, ca_state#23, ca_county#22] +Functions [7]: [partial_avg(agg1#28), partial_avg(agg2#29), partial_avg(agg3#30), partial_avg(agg4#31), partial_avg(agg5#32), partial_avg(agg6#33), partial_avg(agg7#34)] +Aggregate Attributes [14]: [sum#35, count#36, sum#37, count#38, sum#39, count#40, sum#41, count#42, sum#43, count#44, sum#45, count#46, sum#47, count#48] +Results [18]: [i_item_id#27, ca_country#24, ca_state#23, ca_county#22, sum#49, count#50, sum#51, count#52, sum#53, count#54, sum#55, count#56, sum#57, count#58, sum#59, count#60, sum#61, count#62] + +(40) Exchange +Input [18]: [i_item_id#27, ca_country#24, ca_state#23, ca_county#22, sum#49, count#50, sum#51, count#52, sum#53, count#54, sum#55, count#56, sum#57, count#58, sum#59, count#60, sum#61, count#62] +Arguments: hashpartitioning(i_item_id#27, ca_country#24, ca_state#23, ca_county#22, 5), ENSURE_REQUIREMENTS, [plan_id=6] + +(41) HashAggregate [codegen id : 8] +Input [18]: [i_item_id#27, ca_country#24, ca_state#23, ca_county#22, sum#49, count#50, sum#51, count#52, sum#53, count#54, sum#55, count#56, sum#57, count#58, sum#59, count#60, sum#61, count#62] +Keys [4]: [i_item_id#27, ca_country#24, ca_state#23, ca_county#22] +Functions [7]: [avg(agg1#28), avg(agg2#29), avg(agg3#30), avg(agg4#31), avg(agg5#32), avg(agg6#33), avg(agg7#34)] +Aggregate Attributes [7]: [avg(agg1#28)#63, avg(agg2#29)#64, avg(agg3#30)#65, avg(agg4#31)#66, avg(agg5#32)#67, avg(agg6#33)#68, avg(agg7#34)#69] +Results [11]: [i_item_id#27, ca_country#24, ca_state#23, ca_county#22, avg(agg1#28)#63 AS agg1#70, avg(agg2#29)#64 AS agg2#71, avg(agg3#30)#65 AS agg3#72, avg(agg4#31)#66 AS agg4#73, avg(agg5#32)#67 AS agg5#74, avg(agg6#33)#68 AS agg6#75, avg(agg7#34)#69 AS agg7#76] + +(42) Scan parquet spark_catalog.default.catalog_sales +Output [9]: [cs_bill_customer_sk#77, cs_bill_cdemo_sk#78, cs_item_sk#79, cs_quantity#80, cs_list_price#81, cs_sales_price#82, cs_coupon_amt#83, cs_net_profit#84, cs_sold_date_sk#85] +Batched: true +Location: InMemoryFileIndex [] +PartitionFilters: [isnotnull(cs_sold_date_sk#85), dynamicpruningexpression(cs_sold_date_sk#85 IN dynamicpruning#86)] +PushedFilters: [IsNotNull(cs_bill_cdemo_sk), IsNotNull(cs_bill_customer_sk), IsNotNull(cs_item_sk)] +ReadSchema: struct<cs_bill_customer_sk:int,cs_bill_cdemo_sk:int,cs_item_sk:int,cs_quantity:int,cs_list_price:decimal(7,2),cs_sales_price:decimal(7,2),cs_coupon_amt:decimal(7,2),cs_net_profit:decimal(7,2)> + +(43) CometFilter +Input [9]: [cs_bill_customer_sk#77, cs_bill_cdemo_sk#78, cs_item_sk#79, cs_quantity#80, cs_list_price#81, cs_sales_price#82, cs_coupon_amt#83, cs_net_profit#84, cs_sold_date_sk#85] +Condition : ((isnotnull(cs_bill_cdemo_sk#78) AND isnotnull(cs_bill_customer_sk#77)) AND isnotnull(cs_item_sk#79)) + +(44) ColumnarToRow [codegen id : 15] +Input [9]: [cs_bill_customer_sk#77, cs_bill_cdemo_sk#78, cs_item_sk#79, cs_quantity#80, cs_list_price#81, cs_sales_price#82, cs_coupon_amt#83, cs_net_profit#84, cs_sold_date_sk#85] + +(45) ReusedExchange [Reuses operator id: 8] +Output [2]: [cd_demo_sk#87, cd_dep_count#88] + +(46) BroadcastHashJoin [codegen id : 15] +Left keys [1]: [cs_bill_cdemo_sk#78] +Right keys [1]: [cd_demo_sk#87] +Join type: Inner +Join condition: None + +(47) Project [codegen id : 15] +Output [9]: [cs_bill_customer_sk#77, cs_item_sk#79, cs_quantity#80, cs_list_price#81, cs_sales_price#82, cs_coupon_amt#83, cs_net_profit#84, cs_sold_date_sk#85, cd_dep_count#88] +Input [11]: [cs_bill_customer_sk#77, cs_bill_cdemo_sk#78, cs_item_sk#79, cs_quantity#80, cs_list_price#81, cs_sales_price#82, cs_coupon_amt#83, cs_net_profit#84, cs_sold_date_sk#85, cd_demo_sk#87, cd_dep_count#88] + +(48) ReusedExchange [Reuses operator id: 15] +Output [4]: [c_customer_sk#89, c_current_cdemo_sk#90, c_current_addr_sk#91, c_birth_year#92] + +(49) BroadcastHashJoin [codegen id : 15] +Left keys [1]: [cs_bill_customer_sk#77] +Right keys [1]: [c_customer_sk#89] +Join type: Inner +Join condition: None + +(50) Project [codegen id : 15] +Output [11]: [cs_item_sk#79, cs_quantity#80, cs_list_price#81, cs_sales_price#82, cs_coupon_amt#83, cs_net_profit#84, cs_sold_date_sk#85, cd_dep_count#88, c_current_cdemo_sk#90, c_current_addr_sk#91, c_birth_year#92] +Input [13]: [cs_bill_customer_sk#77, cs_item_sk#79, cs_quantity#80, cs_list_price#81, cs_sales_price#82, cs_coupon_amt#83, cs_net_profit#84, cs_sold_date_sk#85, cd_dep_count#88, c_customer_sk#89, c_current_cdemo_sk#90, c_current_addr_sk#91, c_birth_year#92] + +(51) ReusedExchange [Reuses operator id: 21] +Output [1]: [cd_demo_sk#93] + +(52) BroadcastHashJoin [codegen id : 15] +Left keys [1]: [c_current_cdemo_sk#90] +Right keys [1]: [cd_demo_sk#93] +Join type: Inner +Join condition: None + +(53) Project [codegen id : 15] +Output [10]: [cs_item_sk#79, cs_quantity#80, cs_list_price#81, cs_sales_price#82, cs_coupon_amt#83, cs_net_profit#84, cs_sold_date_sk#85, cd_dep_count#88, c_current_addr_sk#91, c_birth_year#92] +Input [12]: [cs_item_sk#79, cs_quantity#80, cs_list_price#81, cs_sales_price#82, cs_coupon_amt#83, cs_net_profit#84, cs_sold_date_sk#85, cd_dep_count#88, c_current_cdemo_sk#90, c_current_addr_sk#91, c_birth_year#92, cd_demo_sk#93] + +(54) Scan parquet spark_catalog.default.customer_address +Output [3]: [ca_address_sk#94, ca_state#95, ca_country#96] +Batched: true +Location [not included in comparison]/{warehouse_dir}/customer_address] +PushedFilters: [In(ca_state, [AL,MS,NC,ND,OK,TN,WI]), IsNotNull(ca_address_sk)] +ReadSchema: struct<ca_address_sk:int,ca_state:string,ca_country:string> + +(55) CometFilter +Input [3]: [ca_address_sk#94, ca_state#95, ca_country#96] +Condition : (ca_state#95 IN (ND,WI,AL,NC,OK,MS,TN) AND isnotnull(ca_address_sk#94)) + +(56) ColumnarToRow [codegen id : 12] +Input [3]: [ca_address_sk#94, ca_state#95, ca_country#96] + +(57) BroadcastExchange +Input [3]: [ca_address_sk#94, ca_state#95, ca_country#96] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=7] + +(58) BroadcastHashJoin [codegen id : 15] +Left keys [1]: [c_current_addr_sk#91] +Right keys [1]: [ca_address_sk#94] +Join type: Inner +Join condition: None + +(59) Project [codegen id : 15] +Output [11]: [cs_item_sk#79, cs_quantity#80, cs_list_price#81, cs_sales_price#82, cs_coupon_amt#83, cs_net_profit#84, cs_sold_date_sk#85, cd_dep_count#88, c_birth_year#92, ca_state#95, ca_country#96] +Input [13]: [cs_item_sk#79, cs_quantity#80, cs_list_price#81, cs_sales_price#82, cs_coupon_amt#83, cs_net_profit#84, cs_sold_date_sk#85, cd_dep_count#88, c_current_addr_sk#91, c_birth_year#92, ca_address_sk#94, ca_state#95, ca_country#96] + +(60) ReusedExchange [Reuses operator id: 158] +Output [1]: [d_date_sk#97] + +(61) BroadcastHashJoin [codegen id : 15] +Left keys [1]: [cs_sold_date_sk#85] +Right keys [1]: [d_date_sk#97] +Join type: Inner +Join condition: None + +(62) Project [codegen id : 15] +Output [10]: [cs_item_sk#79, cs_quantity#80, cs_list_price#81, cs_sales_price#82, cs_coupon_amt#83, cs_net_profit#84, cd_dep_count#88, c_birth_year#92, ca_state#95, ca_country#96] +Input [12]: [cs_item_sk#79, cs_quantity#80, cs_list_price#81, cs_sales_price#82, cs_coupon_amt#83, cs_net_profit#84, cs_sold_date_sk#85, cd_dep_count#88, c_birth_year#92, ca_state#95, ca_country#96, d_date_sk#97] + +(63) ReusedExchange [Reuses operator id: 36] +Output [2]: [i_item_sk#98, i_item_id#99] + +(64) BroadcastHashJoin [codegen id : 15] +Left keys [1]: [cs_item_sk#79] +Right keys [1]: [i_item_sk#98] +Join type: Inner +Join condition: None + +(65) Project [codegen id : 15] +Output [10]: [i_item_id#99, ca_country#96, ca_state#95, cast(cs_quantity#80 as decimal(12,2)) AS agg1#100, cast(cs_list_price#81 as decimal(12,2)) AS agg2#101, cast(cs_coupon_amt#83 as decimal(12,2)) AS agg3#102, cast(cs_sales_price#82 as decimal(12,2)) AS agg4#103, cast(cs_net_profit#84 as decimal(12,2)) AS agg5#104, cast(c_birth_year#92 as decimal(12,2)) AS agg6#105, cast(cd_dep_count#88 as decimal(12,2)) AS agg7#106] +Input [12]: [cs_item_sk#79, cs_quantity#80, cs_list_price#81, cs_sales_price#82, cs_coupon_amt#83, cs_net_profit#84, cd_dep_count#88, c_birth_year#92, ca_state#95, ca_country#96, i_item_sk#98, i_item_id#99] + +(66) HashAggregate [codegen id : 15] +Input [10]: [i_item_id#99, ca_country#96, ca_state#95, agg1#100, agg2#101, agg3#102, agg4#103, agg5#104, agg6#105, agg7#106] +Keys [3]: [i_item_id#99, ca_country#96, ca_state#95] +Functions [7]: [partial_avg(agg1#100), partial_avg(agg2#101), partial_avg(agg3#102), partial_avg(agg4#103), partial_avg(agg5#104), partial_avg(agg6#105), partial_avg(agg7#106)] +Aggregate Attributes [14]: [sum#107, count#108, sum#109, count#110, sum#111, count#112, sum#113, count#114, sum#115, count#116, sum#117, count#118, sum#119, count#120] +Results [17]: [i_item_id#99, ca_country#96, ca_state#95, sum#121, count#122, sum#123, count#124, sum#125, count#126, sum#127, count#128, sum#129, count#130, sum#131, count#132, sum#133, count#134] + +(67) Exchange +Input [17]: [i_item_id#99, ca_country#96, ca_state#95, sum#121, count#122, sum#123, count#124, sum#125, count#126, sum#127, count#128, sum#129, count#130, sum#131, count#132, sum#133, count#134] +Arguments: hashpartitioning(i_item_id#99, ca_country#96, ca_state#95, 5), ENSURE_REQUIREMENTS, [plan_id=8] + +(68) HashAggregate [codegen id : 16] +Input [17]: [i_item_id#99, ca_country#96, ca_state#95, sum#121, count#122, sum#123, count#124, sum#125, count#126, sum#127, count#128, sum#129, count#130, sum#131, count#132, sum#133, count#134] +Keys [3]: [i_item_id#99, ca_country#96, ca_state#95] +Functions [7]: [avg(agg1#100), avg(agg2#101), avg(agg3#102), avg(agg4#103), avg(agg5#104), avg(agg6#105), avg(agg7#106)] +Aggregate Attributes [7]: [avg(agg1#100)#135, avg(agg2#101)#136, avg(agg3#102)#137, avg(agg4#103)#138, avg(agg5#104)#139, avg(agg6#105)#140, avg(agg7#106)#141] +Results [11]: [i_item_id#99, ca_country#96, ca_state#95, null AS county#142, avg(agg1#100)#135 AS agg1#143, avg(agg2#101)#136 AS agg2#144, avg(agg3#102)#137 AS agg3#145, avg(agg4#103)#138 AS agg4#146, avg(agg5#104)#139 AS agg5#147, avg(agg6#105)#140 AS agg6#148, avg(agg7#106)#141 AS agg7#149] + +(69) Scan parquet spark_catalog.default.catalog_sales +Output [9]: [cs_bill_customer_sk#150, cs_bill_cdemo_sk#151, cs_item_sk#152, cs_quantity#153, cs_list_price#154, cs_sales_price#155, cs_coupon_amt#156, cs_net_profit#157, cs_sold_date_sk#158] +Batched: true +Location: InMemoryFileIndex [] +PartitionFilters: [isnotnull(cs_sold_date_sk#158), dynamicpruningexpression(cs_sold_date_sk#158 IN dynamicpruning#159)] +PushedFilters: [IsNotNull(cs_bill_cdemo_sk), IsNotNull(cs_bill_customer_sk), IsNotNull(cs_item_sk)] +ReadSchema: struct<cs_bill_customer_sk:int,cs_bill_cdemo_sk:int,cs_item_sk:int,cs_quantity:int,cs_list_price:decimal(7,2),cs_sales_price:decimal(7,2),cs_coupon_amt:decimal(7,2),cs_net_profit:decimal(7,2)> + +(70) CometFilter +Input [9]: [cs_bill_customer_sk#150, cs_bill_cdemo_sk#151, cs_item_sk#152, cs_quantity#153, cs_list_price#154, cs_sales_price#155, cs_coupon_amt#156, cs_net_profit#157, cs_sold_date_sk#158] +Condition : ((isnotnull(cs_bill_cdemo_sk#151) AND isnotnull(cs_bill_customer_sk#150)) AND isnotnull(cs_item_sk#152)) + +(71) ColumnarToRow [codegen id : 23] +Input [9]: [cs_bill_customer_sk#150, cs_bill_cdemo_sk#151, cs_item_sk#152, cs_quantity#153, cs_list_price#154, cs_sales_price#155, cs_coupon_amt#156, cs_net_profit#157, cs_sold_date_sk#158] + +(72) ReusedExchange [Reuses operator id: 8] +Output [2]: [cd_demo_sk#160, cd_dep_count#161] + +(73) BroadcastHashJoin [codegen id : 23] +Left keys [1]: [cs_bill_cdemo_sk#151] +Right keys [1]: [cd_demo_sk#160] +Join type: Inner +Join condition: None + +(74) Project [codegen id : 23] +Output [9]: [cs_bill_customer_sk#150, cs_item_sk#152, cs_quantity#153, cs_list_price#154, cs_sales_price#155, cs_coupon_amt#156, cs_net_profit#157, cs_sold_date_sk#158, cd_dep_count#161] +Input [11]: [cs_bill_customer_sk#150, cs_bill_cdemo_sk#151, cs_item_sk#152, cs_quantity#153, cs_list_price#154, cs_sales_price#155, cs_coupon_amt#156, cs_net_profit#157, cs_sold_date_sk#158, cd_demo_sk#160, cd_dep_count#161] + +(75) ReusedExchange [Reuses operator id: 15] +Output [4]: [c_customer_sk#162, c_current_cdemo_sk#163, c_current_addr_sk#164, c_birth_year#165] + +(76) BroadcastHashJoin [codegen id : 23] +Left keys [1]: [cs_bill_customer_sk#150] +Right keys [1]: [c_customer_sk#162] +Join type: Inner +Join condition: None + +(77) Project [codegen id : 23] +Output [11]: [cs_item_sk#152, cs_quantity#153, cs_list_price#154, cs_sales_price#155, cs_coupon_amt#156, cs_net_profit#157, cs_sold_date_sk#158, cd_dep_count#161, c_current_cdemo_sk#163, c_current_addr_sk#164, c_birth_year#165] +Input [13]: [cs_bill_customer_sk#150, cs_item_sk#152, cs_quantity#153, cs_list_price#154, cs_sales_price#155, cs_coupon_amt#156, cs_net_profit#157, cs_sold_date_sk#158, cd_dep_count#161, c_customer_sk#162, c_current_cdemo_sk#163, c_current_addr_sk#164, c_birth_year#165] + +(78) ReusedExchange [Reuses operator id: 21] +Output [1]: [cd_demo_sk#166] + +(79) BroadcastHashJoin [codegen id : 23] +Left keys [1]: [c_current_cdemo_sk#163] +Right keys [1]: [cd_demo_sk#166] +Join type: Inner +Join condition: None + +(80) Project [codegen id : 23] +Output [10]: [cs_item_sk#152, cs_quantity#153, cs_list_price#154, cs_sales_price#155, cs_coupon_amt#156, cs_net_profit#157, cs_sold_date_sk#158, cd_dep_count#161, c_current_addr_sk#164, c_birth_year#165] +Input [12]: [cs_item_sk#152, cs_quantity#153, cs_list_price#154, cs_sales_price#155, cs_coupon_amt#156, cs_net_profit#157, cs_sold_date_sk#158, cd_dep_count#161, c_current_cdemo_sk#163, c_current_addr_sk#164, c_birth_year#165, cd_demo_sk#166] + +(81) Scan parquet spark_catalog.default.customer_address +Output [3]: [ca_address_sk#167, ca_state#168, ca_country#169] +Batched: true +Location [not included in comparison]/{warehouse_dir}/customer_address] +PushedFilters: [In(ca_state, [AL,MS,NC,ND,OK,TN,WI]), IsNotNull(ca_address_sk)] +ReadSchema: struct<ca_address_sk:int,ca_state:string,ca_country:string> + +(82) CometFilter +Input [3]: [ca_address_sk#167, ca_state#168, ca_country#169] +Condition : (ca_state#168 IN (ND,WI,AL,NC,OK,MS,TN) AND isnotnull(ca_address_sk#167)) + +(83) CometProject +Input [3]: [ca_address_sk#167, ca_state#168, ca_country#169] +Arguments: [ca_address_sk#167, ca_country#169], [ca_address_sk#167, ca_country#169] + +(84) ColumnarToRow [codegen id : 20] +Input [2]: [ca_address_sk#167, ca_country#169] + +(85) BroadcastExchange +Input [2]: [ca_address_sk#167, ca_country#169] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=9] + +(86) BroadcastHashJoin [codegen id : 23] +Left keys [1]: [c_current_addr_sk#164] +Right keys [1]: [ca_address_sk#167] +Join type: Inner +Join condition: None + +(87) Project [codegen id : 23] +Output [10]: [cs_item_sk#152, cs_quantity#153, cs_list_price#154, cs_sales_price#155, cs_coupon_amt#156, cs_net_profit#157, cs_sold_date_sk#158, cd_dep_count#161, c_birth_year#165, ca_country#169] +Input [12]: [cs_item_sk#152, cs_quantity#153, cs_list_price#154, cs_sales_price#155, cs_coupon_amt#156, cs_net_profit#157, cs_sold_date_sk#158, cd_dep_count#161, c_current_addr_sk#164, c_birth_year#165, ca_address_sk#167, ca_country#169] + +(88) ReusedExchange [Reuses operator id: 158] +Output [1]: [d_date_sk#170] + +(89) BroadcastHashJoin [codegen id : 23] +Left keys [1]: [cs_sold_date_sk#158] +Right keys [1]: [d_date_sk#170] +Join type: Inner +Join condition: None + +(90) Project [codegen id : 23] +Output [9]: [cs_item_sk#152, cs_quantity#153, cs_list_price#154, cs_sales_price#155, cs_coupon_amt#156, cs_net_profit#157, cd_dep_count#161, c_birth_year#165, ca_country#169] +Input [11]: [cs_item_sk#152, cs_quantity#153, cs_list_price#154, cs_sales_price#155, cs_coupon_amt#156, cs_net_profit#157, cs_sold_date_sk#158, cd_dep_count#161, c_birth_year#165, ca_country#169, d_date_sk#170] + +(91) ReusedExchange [Reuses operator id: 36] +Output [2]: [i_item_sk#171, i_item_id#172] + +(92) BroadcastHashJoin [codegen id : 23] +Left keys [1]: [cs_item_sk#152] +Right keys [1]: [i_item_sk#171] +Join type: Inner +Join condition: None + +(93) Project [codegen id : 23] +Output [9]: [i_item_id#172, ca_country#169, cast(cs_quantity#153 as decimal(12,2)) AS agg1#173, cast(cs_list_price#154 as decimal(12,2)) AS agg2#174, cast(cs_coupon_amt#156 as decimal(12,2)) AS agg3#175, cast(cs_sales_price#155 as decimal(12,2)) AS agg4#176, cast(cs_net_profit#157 as decimal(12,2)) AS agg5#177, cast(c_birth_year#165 as decimal(12,2)) AS agg6#178, cast(cd_dep_count#161 as decimal(12,2)) AS agg7#179] +Input [11]: [cs_item_sk#152, cs_quantity#153, cs_list_price#154, cs_sales_price#155, cs_coupon_amt#156, cs_net_profit#157, cd_dep_count#161, c_birth_year#165, ca_country#169, i_item_sk#171, i_item_id#172] + +(94) HashAggregate [codegen id : 23] +Input [9]: [i_item_id#172, ca_country#169, agg1#173, agg2#174, agg3#175, agg4#176, agg5#177, agg6#178, agg7#179] +Keys [2]: [i_item_id#172, ca_country#169] +Functions [7]: [partial_avg(agg1#173), partial_avg(agg2#174), partial_avg(agg3#175), partial_avg(agg4#176), partial_avg(agg5#177), partial_avg(agg6#178), partial_avg(agg7#179)] +Aggregate Attributes [14]: [sum#180, count#181, sum#182, count#183, sum#184, count#185, sum#186, count#187, sum#188, count#189, sum#190, count#191, sum#192, count#193] +Results [16]: [i_item_id#172, ca_country#169, sum#194, count#195, sum#196, count#197, sum#198, count#199, sum#200, count#201, sum#202, count#203, sum#204, count#205, sum#206, count#207] + +(95) Exchange +Input [16]: [i_item_id#172, ca_country#169, sum#194, count#195, sum#196, count#197, sum#198, count#199, sum#200, count#201, sum#202, count#203, sum#204, count#205, sum#206, count#207] +Arguments: hashpartitioning(i_item_id#172, ca_country#169, 5), ENSURE_REQUIREMENTS, [plan_id=10] + +(96) HashAggregate [codegen id : 24] +Input [16]: [i_item_id#172, ca_country#169, sum#194, count#195, sum#196, count#197, sum#198, count#199, sum#200, count#201, sum#202, count#203, sum#204, count#205, sum#206, count#207] +Keys [2]: [i_item_id#172, ca_country#169] +Functions [7]: [avg(agg1#173), avg(agg2#174), avg(agg3#175), avg(agg4#176), avg(agg5#177), avg(agg6#178), avg(agg7#179)] +Aggregate Attributes [7]: [avg(agg1#173)#208, avg(agg2#174)#209, avg(agg3#175)#210, avg(agg4#176)#211, avg(agg5#177)#212, avg(agg6#178)#213, avg(agg7#179)#214] +Results [11]: [i_item_id#172, ca_country#169, null AS ca_state#215, null AS county#216, avg(agg1#173)#208 AS agg1#217, avg(agg2#174)#209 AS agg2#218, avg(agg3#175)#210 AS agg3#219, avg(agg4#176)#211 AS agg4#220, avg(agg5#177)#212 AS agg5#221, avg(agg6#178)#213 AS agg6#222, avg(agg7#179)#214 AS agg7#223] + +(97) Scan parquet spark_catalog.default.catalog_sales +Output [9]: [cs_bill_customer_sk#224, cs_bill_cdemo_sk#225, cs_item_sk#226, cs_quantity#227, cs_list_price#228, cs_sales_price#229, cs_coupon_amt#230, cs_net_profit#231, cs_sold_date_sk#232] +Batched: true +Location: InMemoryFileIndex [] +PartitionFilters: [isnotnull(cs_sold_date_sk#232), dynamicpruningexpression(cs_sold_date_sk#232 IN dynamicpruning#233)] +PushedFilters: [IsNotNull(cs_bill_cdemo_sk), IsNotNull(cs_bill_customer_sk), IsNotNull(cs_item_sk)] +ReadSchema: struct<cs_bill_customer_sk:int,cs_bill_cdemo_sk:int,cs_item_sk:int,cs_quantity:int,cs_list_price:decimal(7,2),cs_sales_price:decimal(7,2),cs_coupon_amt:decimal(7,2),cs_net_profit:decimal(7,2)> + +(98) CometFilter +Input [9]: [cs_bill_customer_sk#224, cs_bill_cdemo_sk#225, cs_item_sk#226, cs_quantity#227, cs_list_price#228, cs_sales_price#229, cs_coupon_amt#230, cs_net_profit#231, cs_sold_date_sk#232] +Condition : ((isnotnull(cs_bill_cdemo_sk#225) AND isnotnull(cs_bill_customer_sk#224)) AND isnotnull(cs_item_sk#226)) + +(99) ColumnarToRow [codegen id : 31] +Input [9]: [cs_bill_customer_sk#224, cs_bill_cdemo_sk#225, cs_item_sk#226, cs_quantity#227, cs_list_price#228, cs_sales_price#229, cs_coupon_amt#230, cs_net_profit#231, cs_sold_date_sk#232] + +(100) ReusedExchange [Reuses operator id: 8] +Output [2]: [cd_demo_sk#234, cd_dep_count#235] + +(101) BroadcastHashJoin [codegen id : 31] +Left keys [1]: [cs_bill_cdemo_sk#225] +Right keys [1]: [cd_demo_sk#234] +Join type: Inner +Join condition: None + +(102) Project [codegen id : 31] +Output [9]: [cs_bill_customer_sk#224, cs_item_sk#226, cs_quantity#227, cs_list_price#228, cs_sales_price#229, cs_coupon_amt#230, cs_net_profit#231, cs_sold_date_sk#232, cd_dep_count#235] +Input [11]: [cs_bill_customer_sk#224, cs_bill_cdemo_sk#225, cs_item_sk#226, cs_quantity#227, cs_list_price#228, cs_sales_price#229, cs_coupon_amt#230, cs_net_profit#231, cs_sold_date_sk#232, cd_demo_sk#234, cd_dep_count#235] + +(103) ReusedExchange [Reuses operator id: 15] +Output [4]: [c_customer_sk#236, c_current_cdemo_sk#237, c_current_addr_sk#238, c_birth_year#239] + +(104) BroadcastHashJoin [codegen id : 31] +Left keys [1]: [cs_bill_customer_sk#224] +Right keys [1]: [c_customer_sk#236] +Join type: Inner +Join condition: None + +(105) Project [codegen id : 31] +Output [11]: [cs_item_sk#226, cs_quantity#227, cs_list_price#228, cs_sales_price#229, cs_coupon_amt#230, cs_net_profit#231, cs_sold_date_sk#232, cd_dep_count#235, c_current_cdemo_sk#237, c_current_addr_sk#238, c_birth_year#239] +Input [13]: [cs_bill_customer_sk#224, cs_item_sk#226, cs_quantity#227, cs_list_price#228, cs_sales_price#229, cs_coupon_amt#230, cs_net_profit#231, cs_sold_date_sk#232, cd_dep_count#235, c_customer_sk#236, c_current_cdemo_sk#237, c_current_addr_sk#238, c_birth_year#239] + +(106) ReusedExchange [Reuses operator id: 21] +Output [1]: [cd_demo_sk#240] + +(107) BroadcastHashJoin [codegen id : 31] +Left keys [1]: [c_current_cdemo_sk#237] +Right keys [1]: [cd_demo_sk#240] +Join type: Inner +Join condition: None + +(108) Project [codegen id : 31] +Output [10]: [cs_item_sk#226, cs_quantity#227, cs_list_price#228, cs_sales_price#229, cs_coupon_amt#230, cs_net_profit#231, cs_sold_date_sk#232, cd_dep_count#235, c_current_addr_sk#238, c_birth_year#239] +Input [12]: [cs_item_sk#226, cs_quantity#227, cs_list_price#228, cs_sales_price#229, cs_coupon_amt#230, cs_net_profit#231, cs_sold_date_sk#232, cd_dep_count#235, c_current_cdemo_sk#237, c_current_addr_sk#238, c_birth_year#239, cd_demo_sk#240] + +(109) Scan parquet spark_catalog.default.customer_address +Output [2]: [ca_address_sk#241, ca_state#242] +Batched: true +Location [not included in comparison]/{warehouse_dir}/customer_address] +PushedFilters: [In(ca_state, [AL,MS,NC,ND,OK,TN,WI]), IsNotNull(ca_address_sk)] +ReadSchema: struct<ca_address_sk:int,ca_state:string> + +(110) CometFilter +Input [2]: [ca_address_sk#241, ca_state#242] +Condition : (ca_state#242 IN (ND,WI,AL,NC,OK,MS,TN) AND isnotnull(ca_address_sk#241)) + +(111) CometProject +Input [2]: [ca_address_sk#241, ca_state#242] +Arguments: [ca_address_sk#241], [ca_address_sk#241] + +(112) ColumnarToRow [codegen id : 28] +Input [1]: [ca_address_sk#241] + +(113) BroadcastExchange +Input [1]: [ca_address_sk#241] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=11] + +(114) BroadcastHashJoin [codegen id : 31] +Left keys [1]: [c_current_addr_sk#238] +Right keys [1]: [ca_address_sk#241] +Join type: Inner +Join condition: None + +(115) Project [codegen id : 31] +Output [9]: [cs_item_sk#226, cs_quantity#227, cs_list_price#228, cs_sales_price#229, cs_coupon_amt#230, cs_net_profit#231, cs_sold_date_sk#232, cd_dep_count#235, c_birth_year#239] +Input [11]: [cs_item_sk#226, cs_quantity#227, cs_list_price#228, cs_sales_price#229, cs_coupon_amt#230, cs_net_profit#231, cs_sold_date_sk#232, cd_dep_count#235, c_current_addr_sk#238, c_birth_year#239, ca_address_sk#241] + +(116) ReusedExchange [Reuses operator id: 158] +Output [1]: [d_date_sk#243] + +(117) BroadcastHashJoin [codegen id : 31] +Left keys [1]: [cs_sold_date_sk#232] +Right keys [1]: [d_date_sk#243] +Join type: Inner +Join condition: None + +(118) Project [codegen id : 31] +Output [8]: [cs_item_sk#226, cs_quantity#227, cs_list_price#228, cs_sales_price#229, cs_coupon_amt#230, cs_net_profit#231, cd_dep_count#235, c_birth_year#239] +Input [10]: [cs_item_sk#226, cs_quantity#227, cs_list_price#228, cs_sales_price#229, cs_coupon_amt#230, cs_net_profit#231, cs_sold_date_sk#232, cd_dep_count#235, c_birth_year#239, d_date_sk#243] + +(119) ReusedExchange [Reuses operator id: 36] +Output [2]: [i_item_sk#244, i_item_id#245] + +(120) BroadcastHashJoin [codegen id : 31] +Left keys [1]: [cs_item_sk#226] +Right keys [1]: [i_item_sk#244] +Join type: Inner +Join condition: None + +(121) Project [codegen id : 31] +Output [8]: [i_item_id#245, cast(cs_quantity#227 as decimal(12,2)) AS agg1#246, cast(cs_list_price#228 as decimal(12,2)) AS agg2#247, cast(cs_coupon_amt#230 as decimal(12,2)) AS agg3#248, cast(cs_sales_price#229 as decimal(12,2)) AS agg4#249, cast(cs_net_profit#231 as decimal(12,2)) AS agg5#250, cast(c_birth_year#239 as decimal(12,2)) AS agg6#251, cast(cd_dep_count#235 as decimal(12,2)) AS agg7#252] +Input [10]: [cs_item_sk#226, cs_quantity#227, cs_list_price#228, cs_sales_price#229, cs_coupon_amt#230, cs_net_profit#231, cd_dep_count#235, c_birth_year#239, i_item_sk#244, i_item_id#245] + +(122) HashAggregate [codegen id : 31] +Input [8]: [i_item_id#245, agg1#246, agg2#247, agg3#248, agg4#249, agg5#250, agg6#251, agg7#252] +Keys [1]: [i_item_id#245] +Functions [7]: [partial_avg(agg1#246), partial_avg(agg2#247), partial_avg(agg3#248), partial_avg(agg4#249), partial_avg(agg5#250), partial_avg(agg6#251), partial_avg(agg7#252)] +Aggregate Attributes [14]: [sum#253, count#254, sum#255, count#256, sum#257, count#258, sum#259, count#260, sum#261, count#262, sum#263, count#264, sum#265, count#266] +Results [15]: [i_item_id#245, sum#267, count#268, sum#269, count#270, sum#271, count#272, sum#273, count#274, sum#275, count#276, sum#277, count#278, sum#279, count#280] + +(123) Exchange +Input [15]: [i_item_id#245, sum#267, count#268, sum#269, count#270, sum#271, count#272, sum#273, count#274, sum#275, count#276, sum#277, count#278, sum#279, count#280] +Arguments: hashpartitioning(i_item_id#245, 5), ENSURE_REQUIREMENTS, [plan_id=12] + +(124) HashAggregate [codegen id : 32] +Input [15]: [i_item_id#245, sum#267, count#268, sum#269, count#270, sum#271, count#272, sum#273, count#274, sum#275, count#276, sum#277, count#278, sum#279, count#280] +Keys [1]: [i_item_id#245] +Functions [7]: [avg(agg1#246), avg(agg2#247), avg(agg3#248), avg(agg4#249), avg(agg5#250), avg(agg6#251), avg(agg7#252)] +Aggregate Attributes [7]: [avg(agg1#246)#281, avg(agg2#247)#282, avg(agg3#248)#283, avg(agg4#249)#284, avg(agg5#250)#285, avg(agg6#251)#286, avg(agg7#252)#287] +Results [11]: [i_item_id#245, null AS ca_country#288, null AS ca_state#289, null AS county#290, avg(agg1#246)#281 AS agg1#291, avg(agg2#247)#282 AS agg2#292, avg(agg3#248)#283 AS agg3#293, avg(agg4#249)#284 AS agg4#294, avg(agg5#250)#285 AS agg5#295, avg(agg6#251)#286 AS agg6#296, avg(agg7#252)#287 AS agg7#297] + +(125) Scan parquet spark_catalog.default.catalog_sales +Output [9]: [cs_bill_customer_sk#298, cs_bill_cdemo_sk#299, cs_item_sk#300, cs_quantity#301, cs_list_price#302, cs_sales_price#303, cs_coupon_amt#304, cs_net_profit#305, cs_sold_date_sk#306] +Batched: true +Location: InMemoryFileIndex [] +PartitionFilters: [isnotnull(cs_sold_date_sk#306), dynamicpruningexpression(cs_sold_date_sk#306 IN dynamicpruning#307)] +PushedFilters: [IsNotNull(cs_bill_cdemo_sk), IsNotNull(cs_bill_customer_sk), IsNotNull(cs_item_sk)] +ReadSchema: struct<cs_bill_customer_sk:int,cs_bill_cdemo_sk:int,cs_item_sk:int,cs_quantity:int,cs_list_price:decimal(7,2),cs_sales_price:decimal(7,2),cs_coupon_amt:decimal(7,2),cs_net_profit:decimal(7,2)> + +(126) CometFilter +Input [9]: [cs_bill_customer_sk#298, cs_bill_cdemo_sk#299, cs_item_sk#300, cs_quantity#301, cs_list_price#302, cs_sales_price#303, cs_coupon_amt#304, cs_net_profit#305, cs_sold_date_sk#306] +Condition : ((isnotnull(cs_bill_cdemo_sk#299) AND isnotnull(cs_bill_customer_sk#298)) AND isnotnull(cs_item_sk#300)) + +(127) ColumnarToRow [codegen id : 39] +Input [9]: [cs_bill_customer_sk#298, cs_bill_cdemo_sk#299, cs_item_sk#300, cs_quantity#301, cs_list_price#302, cs_sales_price#303, cs_coupon_amt#304, cs_net_profit#305, cs_sold_date_sk#306] + +(128) ReusedExchange [Reuses operator id: 8] +Output [2]: [cd_demo_sk#308, cd_dep_count#309] + +(129) BroadcastHashJoin [codegen id : 39] +Left keys [1]: [cs_bill_cdemo_sk#299] +Right keys [1]: [cd_demo_sk#308] +Join type: Inner +Join condition: None + +(130) Project [codegen id : 39] +Output [9]: [cs_bill_customer_sk#298, cs_item_sk#300, cs_quantity#301, cs_list_price#302, cs_sales_price#303, cs_coupon_amt#304, cs_net_profit#305, cs_sold_date_sk#306, cd_dep_count#309] +Input [11]: [cs_bill_customer_sk#298, cs_bill_cdemo_sk#299, cs_item_sk#300, cs_quantity#301, cs_list_price#302, cs_sales_price#303, cs_coupon_amt#304, cs_net_profit#305, cs_sold_date_sk#306, cd_demo_sk#308, cd_dep_count#309] + +(131) ReusedExchange [Reuses operator id: 15] +Output [4]: [c_customer_sk#310, c_current_cdemo_sk#311, c_current_addr_sk#312, c_birth_year#313] + +(132) BroadcastHashJoin [codegen id : 39] +Left keys [1]: [cs_bill_customer_sk#298] +Right keys [1]: [c_customer_sk#310] +Join type: Inner +Join condition: None + +(133) Project [codegen id : 39] +Output [11]: [cs_item_sk#300, cs_quantity#301, cs_list_price#302, cs_sales_price#303, cs_coupon_amt#304, cs_net_profit#305, cs_sold_date_sk#306, cd_dep_count#309, c_current_cdemo_sk#311, c_current_addr_sk#312, c_birth_year#313] +Input [13]: [cs_bill_customer_sk#298, cs_item_sk#300, cs_quantity#301, cs_list_price#302, cs_sales_price#303, cs_coupon_amt#304, cs_net_profit#305, cs_sold_date_sk#306, cd_dep_count#309, c_customer_sk#310, c_current_cdemo_sk#311, c_current_addr_sk#312, c_birth_year#313] + +(134) ReusedExchange [Reuses operator id: 21] +Output [1]: [cd_demo_sk#314] + +(135) BroadcastHashJoin [codegen id : 39] +Left keys [1]: [c_current_cdemo_sk#311] +Right keys [1]: [cd_demo_sk#314] +Join type: Inner +Join condition: None + +(136) Project [codegen id : 39] +Output [10]: [cs_item_sk#300, cs_quantity#301, cs_list_price#302, cs_sales_price#303, cs_coupon_amt#304, cs_net_profit#305, cs_sold_date_sk#306, cd_dep_count#309, c_current_addr_sk#312, c_birth_year#313] +Input [12]: [cs_item_sk#300, cs_quantity#301, cs_list_price#302, cs_sales_price#303, cs_coupon_amt#304, cs_net_profit#305, cs_sold_date_sk#306, cd_dep_count#309, c_current_cdemo_sk#311, c_current_addr_sk#312, c_birth_year#313, cd_demo_sk#314] + +(137) ReusedExchange [Reuses operator id: 113] +Output [1]: [ca_address_sk#315] + +(138) BroadcastHashJoin [codegen id : 39] +Left keys [1]: [c_current_addr_sk#312] +Right keys [1]: [ca_address_sk#315] +Join type: Inner +Join condition: None + +(139) Project [codegen id : 39] +Output [9]: [cs_item_sk#300, cs_quantity#301, cs_list_price#302, cs_sales_price#303, cs_coupon_amt#304, cs_net_profit#305, cs_sold_date_sk#306, cd_dep_count#309, c_birth_year#313] +Input [11]: [cs_item_sk#300, cs_quantity#301, cs_list_price#302, cs_sales_price#303, cs_coupon_amt#304, cs_net_profit#305, cs_sold_date_sk#306, cd_dep_count#309, c_current_addr_sk#312, c_birth_year#313, ca_address_sk#315] + +(140) ReusedExchange [Reuses operator id: 158] +Output [1]: [d_date_sk#316] + +(141) BroadcastHashJoin [codegen id : 39] +Left keys [1]: [cs_sold_date_sk#306] +Right keys [1]: [d_date_sk#316] +Join type: Inner +Join condition: None + +(142) Project [codegen id : 39] +Output [8]: [cs_item_sk#300, cs_quantity#301, cs_list_price#302, cs_sales_price#303, cs_coupon_amt#304, cs_net_profit#305, cd_dep_count#309, c_birth_year#313] +Input [10]: [cs_item_sk#300, cs_quantity#301, cs_list_price#302, cs_sales_price#303, cs_coupon_amt#304, cs_net_profit#305, cs_sold_date_sk#306, cd_dep_count#309, c_birth_year#313, d_date_sk#316] + +(143) Scan parquet spark_catalog.default.item +Output [1]: [i_item_sk#317] +Batched: true +Location [not included in comparison]/{warehouse_dir}/item] +PushedFilters: [IsNotNull(i_item_sk)] +ReadSchema: struct<i_item_sk:int> + +(144) CometFilter +Input [1]: [i_item_sk#317] +Condition : isnotnull(i_item_sk#317) + +(145) ColumnarToRow [codegen id : 38] +Input [1]: [i_item_sk#317] + +(146) BroadcastExchange +Input [1]: [i_item_sk#317] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=13] + +(147) BroadcastHashJoin [codegen id : 39] +Left keys [1]: [cs_item_sk#300] +Right keys [1]: [i_item_sk#317] +Join type: Inner +Join condition: None + +(148) Project [codegen id : 39] +Output [7]: [cast(cs_quantity#301 as decimal(12,2)) AS agg1#318, cast(cs_list_price#302 as decimal(12,2)) AS agg2#319, cast(cs_coupon_amt#304 as decimal(12,2)) AS agg3#320, cast(cs_sales_price#303 as decimal(12,2)) AS agg4#321, cast(cs_net_profit#305 as decimal(12,2)) AS agg5#322, cast(c_birth_year#313 as decimal(12,2)) AS agg6#323, cast(cd_dep_count#309 as decimal(12,2)) AS agg7#324] +Input [9]: [cs_item_sk#300, cs_quantity#301, cs_list_price#302, cs_sales_price#303, cs_coupon_amt#304, cs_net_profit#305, cd_dep_count#309, c_birth_year#313, i_item_sk#317] + +(149) HashAggregate [codegen id : 39] +Input [7]: [agg1#318, agg2#319, agg3#320, agg4#321, agg5#322, agg6#323, agg7#324] +Keys: [] +Functions [7]: [partial_avg(agg1#318), partial_avg(agg2#319), partial_avg(agg3#320), partial_avg(agg4#321), partial_avg(agg5#322), partial_avg(agg6#323), partial_avg(agg7#324)] +Aggregate Attributes [14]: [sum#325, count#326, sum#327, count#328, sum#329, count#330, sum#331, count#332, sum#333, count#334, sum#335, count#336, sum#337, count#338] +Results [14]: [sum#339, count#340, sum#341, count#342, sum#343, count#344, sum#345, count#346, sum#347, count#348, sum#349, count#350, sum#351, count#352] + +(150) Exchange +Input [14]: [sum#339, count#340, sum#341, count#342, sum#343, count#344, sum#345, count#346, sum#347, count#348, sum#349, count#350, sum#351, count#352] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=14] + +(151) HashAggregate [codegen id : 40] +Input [14]: [sum#339, count#340, sum#341, count#342, sum#343, count#344, sum#345, count#346, sum#347, count#348, sum#349, count#350, sum#351, count#352] +Keys: [] +Functions [7]: [avg(agg1#318), avg(agg2#319), avg(agg3#320), avg(agg4#321), avg(agg5#322), avg(agg6#323), avg(agg7#324)] +Aggregate Attributes [7]: [avg(agg1#318)#353, avg(agg2#319)#354, avg(agg3#320)#355, avg(agg4#321)#356, avg(agg5#322)#357, avg(agg6#323)#358, avg(agg7#324)#359] +Results [11]: [null AS i_item_id#360, null AS ca_country#361, null AS ca_state#362, null AS county#363, avg(agg1#318)#353 AS agg1#364, avg(agg2#319)#354 AS agg2#365, avg(agg3#320)#355 AS agg3#366, avg(agg4#321)#356 AS agg4#367, avg(agg5#322)#357 AS agg5#368, avg(agg6#323)#358 AS agg6#369, avg(agg7#324)#359 AS agg7#370] + +(152) Union + +(153) TakeOrderedAndProject +Input [11]: [i_item_id#27, ca_country#24, ca_state#23, ca_county#22, agg1#70, agg2#71, agg3#72, agg4#73, agg5#74, agg6#75, agg7#76] +Arguments: 100, [ca_country#24 ASC NULLS FIRST, ca_state#23 ASC NULLS FIRST, ca_county#22 ASC NULLS FIRST, i_item_id#27 ASC NULLS FIRST], [i_item_id#27, ca_country#24, ca_state#23, ca_county#22, agg1#70, agg2#71, agg3#72, agg4#73, agg5#74, agg6#75, agg7#76] + +===== Subqueries ===== + +Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#9 IN dynamicpruning#10 +BroadcastExchange (158) ++- * ColumnarToRow (157) + +- CometProject (156) + +- CometFilter (155) + +- CometScan parquet spark_catalog.default.date_dim (154) + + +(154) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#25, d_year#371] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), IsNotNull(d_date_sk)] +ReadSchema: struct<d_date_sk:int,d_year:int> + +(155) CometFilter +Input [2]: [d_date_sk#25, d_year#371] +Condition : ((isnotnull(d_year#371) AND (d_year#371 = 2001)) AND isnotnull(d_date_sk#25)) + +(156) CometProject +Input [2]: [d_date_sk#25, d_year#371] +Arguments: [d_date_sk#25], [d_date_sk#25] + +(157) ColumnarToRow [codegen id : 1] +Input [1]: [d_date_sk#25] + +(158) BroadcastExchange +Input [1]: [d_date_sk#25] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=15] + +Subquery:2 Hosting operator id = 42 Hosting Expression = cs_sold_date_sk#85 IN dynamicpruning#10 + +Subquery:3 Hosting operator id = 69 Hosting Expression = cs_sold_date_sk#158 IN dynamicpruning#10 + +Subquery:4 Hosting operator id = 97 Hosting Expression = cs_sold_date_sk#232 IN dynamicpruning#10 + +Subquery:5 Hosting operator id = 125 Hosting Expression = cs_sold_date_sk#306 IN dynamicpruning#10 + + diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q18a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q18a/simplified.txt new file mode 100644 index 000000000..f02809572 --- /dev/null +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q18a/simplified.txt @@ -0,0 +1,233 @@ +TakeOrderedAndProject [ca_country,ca_state,ca_county,i_item_id,agg1,agg2,agg3,agg4,agg5,agg6,agg7] + Union + WholeStageCodegen (8) + HashAggregate [i_item_id,ca_country,ca_state,ca_county,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] [avg(agg1),avg(agg2),avg(agg3),avg(agg4),avg(agg5),avg(agg6),avg(agg7),agg1,agg2,agg3,agg4,agg5,agg6,agg7,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] + InputAdapter + Exchange [i_item_id,ca_country,ca_state,ca_county] #1 + WholeStageCodegen (7) + HashAggregate [i_item_id,ca_country,ca_state,ca_county,agg1,agg2,agg3,agg4,agg5,agg6,agg7] [sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] + Project [i_item_id,ca_country,ca_state,ca_county,cs_quantity,cs_list_price,cs_coupon_amt,cs_sales_price,cs_net_profit,c_birth_year,cd_dep_count] + BroadcastHashJoin [cs_item_sk,i_item_sk] + Project [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count,c_birth_year,ca_county,ca_state,ca_country] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + Project [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_birth_year,ca_county,ca_state,ca_country] + BroadcastHashJoin [c_current_addr_sk,ca_address_sk] + Project [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_addr_sk,c_birth_year] + BroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] + Project [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] + BroadcastHashJoin [cs_bill_customer_sk,c_customer_sk] + Project [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count] + BroadcastHashJoin [cs_bill_cdemo_sk,cd_demo_sk] + ColumnarToRow + InputAdapter + CometFilter [cs_bill_cdemo_sk,cs_bill_customer_sk,cs_item_sk] + CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_bill_cdemo_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #2 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_year,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + InputAdapter + BroadcastExchange #3 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [cd_demo_sk,cd_dep_count] + CometFilter [cd_gender,cd_education_status,cd_demo_sk] + CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_education_status,cd_dep_count] + InputAdapter + BroadcastExchange #4 + WholeStageCodegen (2) + ColumnarToRow + InputAdapter + CometProject [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] + CometFilter [c_birth_month,c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] + CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_month,c_birth_year] + InputAdapter + BroadcastExchange #5 + WholeStageCodegen (3) + ColumnarToRow + InputAdapter + CometFilter [cd_demo_sk] + CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk] + InputAdapter + BroadcastExchange #6 + WholeStageCodegen (4) + ColumnarToRow + InputAdapter + CometFilter [ca_state,ca_address_sk] + CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_county,ca_state,ca_country] + InputAdapter + ReusedExchange [d_date_sk] #2 + InputAdapter + BroadcastExchange #7 + WholeStageCodegen (6) + ColumnarToRow + InputAdapter + CometFilter [i_item_sk] + CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id] + WholeStageCodegen (16) + HashAggregate [i_item_id,ca_country,ca_state,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] [avg(agg1),avg(agg2),avg(agg3),avg(agg4),avg(agg5),avg(agg6),avg(agg7),county,agg1,agg2,agg3,agg4,agg5,agg6,agg7,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] + InputAdapter + Exchange [i_item_id,ca_country,ca_state] #8 + WholeStageCodegen (15) + HashAggregate [i_item_id,ca_country,ca_state,agg1,agg2,agg3,agg4,agg5,agg6,agg7] [sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] + Project [i_item_id,ca_country,ca_state,cs_quantity,cs_list_price,cs_coupon_amt,cs_sales_price,cs_net_profit,c_birth_year,cd_dep_count] + BroadcastHashJoin [cs_item_sk,i_item_sk] + Project [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count,c_birth_year,ca_state,ca_country] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + Project [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_birth_year,ca_state,ca_country] + BroadcastHashJoin [c_current_addr_sk,ca_address_sk] + Project [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_addr_sk,c_birth_year] + BroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] + Project [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] + BroadcastHashJoin [cs_bill_customer_sk,c_customer_sk] + Project [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count] + BroadcastHashJoin [cs_bill_cdemo_sk,cd_demo_sk] + ColumnarToRow + InputAdapter + CometFilter [cs_bill_cdemo_sk,cs_bill_customer_sk,cs_item_sk] + CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_bill_cdemo_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + InputAdapter + ReusedExchange [cd_demo_sk,cd_dep_count] #3 + InputAdapter + ReusedExchange [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] #4 + InputAdapter + ReusedExchange [cd_demo_sk] #5 + InputAdapter + BroadcastExchange #9 + WholeStageCodegen (12) + ColumnarToRow + InputAdapter + CometFilter [ca_state,ca_address_sk] + CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state,ca_country] + InputAdapter + ReusedExchange [d_date_sk] #2 + InputAdapter + ReusedExchange [i_item_sk,i_item_id] #7 + WholeStageCodegen (24) + HashAggregate [i_item_id,ca_country,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] [avg(agg1),avg(agg2),avg(agg3),avg(agg4),avg(agg5),avg(agg6),avg(agg7),ca_state,county,agg1,agg2,agg3,agg4,agg5,agg6,agg7,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] + InputAdapter + Exchange [i_item_id,ca_country] #10 + WholeStageCodegen (23) + HashAggregate [i_item_id,ca_country,agg1,agg2,agg3,agg4,agg5,agg6,agg7] [sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] + Project [i_item_id,ca_country,cs_quantity,cs_list_price,cs_coupon_amt,cs_sales_price,cs_net_profit,c_birth_year,cd_dep_count] + BroadcastHashJoin [cs_item_sk,i_item_sk] + Project [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count,c_birth_year,ca_country] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + Project [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_birth_year,ca_country] + BroadcastHashJoin [c_current_addr_sk,ca_address_sk] + Project [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_addr_sk,c_birth_year] + BroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] + Project [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] + BroadcastHashJoin [cs_bill_customer_sk,c_customer_sk] + Project [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count] + BroadcastHashJoin [cs_bill_cdemo_sk,cd_demo_sk] + ColumnarToRow + InputAdapter + CometFilter [cs_bill_cdemo_sk,cs_bill_customer_sk,cs_item_sk] + CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_bill_cdemo_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + InputAdapter + ReusedExchange [cd_demo_sk,cd_dep_count] #3 + InputAdapter + ReusedExchange [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] #4 + InputAdapter + ReusedExchange [cd_demo_sk] #5 + InputAdapter + BroadcastExchange #11 + WholeStageCodegen (20) + ColumnarToRow + InputAdapter + CometProject [ca_address_sk,ca_country] + CometFilter [ca_state,ca_address_sk] + CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state,ca_country] + InputAdapter + ReusedExchange [d_date_sk] #2 + InputAdapter + ReusedExchange [i_item_sk,i_item_id] #7 + WholeStageCodegen (32) + HashAggregate [i_item_id,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] [avg(agg1),avg(agg2),avg(agg3),avg(agg4),avg(agg5),avg(agg6),avg(agg7),ca_country,ca_state,county,agg1,agg2,agg3,agg4,agg5,agg6,agg7,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] + InputAdapter + Exchange [i_item_id] #12 + WholeStageCodegen (31) + HashAggregate [i_item_id,agg1,agg2,agg3,agg4,agg5,agg6,agg7] [sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] + Project [i_item_id,cs_quantity,cs_list_price,cs_coupon_amt,cs_sales_price,cs_net_profit,c_birth_year,cd_dep_count] + BroadcastHashJoin [cs_item_sk,i_item_sk] + Project [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count,c_birth_year] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + Project [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_birth_year] + BroadcastHashJoin [c_current_addr_sk,ca_address_sk] + Project [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_addr_sk,c_birth_year] + BroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] + Project [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] + BroadcastHashJoin [cs_bill_customer_sk,c_customer_sk] + Project [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count] + BroadcastHashJoin [cs_bill_cdemo_sk,cd_demo_sk] + ColumnarToRow + InputAdapter + CometFilter [cs_bill_cdemo_sk,cs_bill_customer_sk,cs_item_sk] + CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_bill_cdemo_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + InputAdapter + ReusedExchange [cd_demo_sk,cd_dep_count] #3 + InputAdapter + ReusedExchange [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] #4 + InputAdapter + ReusedExchange [cd_demo_sk] #5 + InputAdapter + BroadcastExchange #13 + WholeStageCodegen (28) + ColumnarToRow + InputAdapter + CometProject [ca_address_sk] + CometFilter [ca_state,ca_address_sk] + CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] + InputAdapter + ReusedExchange [d_date_sk] #2 + InputAdapter + ReusedExchange [i_item_sk,i_item_id] #7 + WholeStageCodegen (40) + HashAggregate [sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] [avg(agg1),avg(agg2),avg(agg3),avg(agg4),avg(agg5),avg(agg6),avg(agg7),i_item_id,ca_country,ca_state,county,agg1,agg2,agg3,agg4,agg5,agg6,agg7,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] + InputAdapter + Exchange #14 + WholeStageCodegen (39) + HashAggregate [agg1,agg2,agg3,agg4,agg5,agg6,agg7] [sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] + Project [cs_quantity,cs_list_price,cs_coupon_amt,cs_sales_price,cs_net_profit,c_birth_year,cd_dep_count] + BroadcastHashJoin [cs_item_sk,i_item_sk] + Project [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cd_dep_count,c_birth_year] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + Project [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_birth_year] + BroadcastHashJoin [c_current_addr_sk,ca_address_sk] + Project [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_addr_sk,c_birth_year] + BroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] + Project [cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] + BroadcastHashJoin [cs_bill_customer_sk,c_customer_sk] + Project [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk,cd_dep_count] + BroadcastHashJoin [cs_bill_cdemo_sk,cd_demo_sk] + ColumnarToRow + InputAdapter + CometFilter [cs_bill_cdemo_sk,cs_bill_customer_sk,cs_item_sk] + CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_bill_cdemo_sk,cs_item_sk,cs_quantity,cs_list_price,cs_sales_price,cs_coupon_amt,cs_net_profit,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + InputAdapter + ReusedExchange [cd_demo_sk,cd_dep_count] #3 + InputAdapter + ReusedExchange [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] #4 + InputAdapter + ReusedExchange [cd_demo_sk] #5 + InputAdapter + ReusedExchange [ca_address_sk] #13 + InputAdapter + ReusedExchange [d_date_sk] #2 + InputAdapter + BroadcastExchange #15 + WholeStageCodegen (38) + ColumnarToRow + InputAdapter + CometFilter [i_item_sk] + CometScan parquet spark_catalog.default.item [i_item_sk] 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 new file mode 100644 index 000000000..1304af2e1 --- /dev/null +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q20/explain.txt @@ -0,0 +1,150 @@ +== Physical Plan == +TakeOrderedAndProject (20) ++- * Project (19) + +- Window (18) + +- * Sort (17) + +- Exchange (16) + +- * HashAggregate (15) + +- Exchange (14) + +- * HashAggregate (13) + +- * Project (12) + +- * BroadcastHashJoin Inner BuildRight (11) + :- * Project (9) + : +- * BroadcastHashJoin Inner BuildRight (8) + : :- * ColumnarToRow (3) + : : +- CometFilter (2) + : : +- CometScan parquet spark_catalog.default.catalog_sales (1) + : +- BroadcastExchange (7) + : +- * ColumnarToRow (6) + : +- CometFilter (5) + : +- CometScan parquet spark_catalog.default.item (4) + +- ReusedExchange (10) + + +(1) Scan parquet spark_catalog.default.catalog_sales +Output [3]: [cs_item_sk#1, cs_ext_sales_price#2, cs_sold_date_sk#3] +Batched: true +Location: InMemoryFileIndex [] +PartitionFilters: [isnotnull(cs_sold_date_sk#3), dynamicpruningexpression(cs_sold_date_sk#3 IN dynamicpruning#4)] +PushedFilters: [IsNotNull(cs_item_sk)] +ReadSchema: struct<cs_item_sk:int,cs_ext_sales_price:decimal(7,2)> + +(2) CometFilter +Input [3]: [cs_item_sk#1, cs_ext_sales_price#2, cs_sold_date_sk#3] +Condition : isnotnull(cs_item_sk#1) + +(3) ColumnarToRow [codegen id : 3] +Input [3]: [cs_item_sk#1, cs_ext_sales_price#2, cs_sold_date_sk#3] + +(4) Scan parquet spark_catalog.default.item +Output [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] +Batched: true +Location [not included in comparison]/{warehouse_dir}/item] +PushedFilters: [In(i_category, [Books ,Home ,Sports ]), IsNotNull(i_item_sk)] +ReadSchema: struct<i_item_sk:int,i_item_id:string,i_item_desc:string,i_current_price:decimal(7,2),i_class:string,i_category:string> + +(5) CometFilter +Input [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] +Condition : (i_category#10 IN (Sports ,Books ,Home ) AND isnotnull(i_item_sk#5)) + +(6) ColumnarToRow [codegen id : 1] +Input [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] + +(7) BroadcastExchange +Input [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=1] + +(8) BroadcastHashJoin [codegen id : 3] +Left keys [1]: [cs_item_sk#1] +Right keys [1]: [i_item_sk#5] +Join type: Inner +Join condition: None + +(9) Project [codegen id : 3] +Output [7]: [cs_ext_sales_price#2, cs_sold_date_sk#3, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] +Input [9]: [cs_item_sk#1, cs_ext_sales_price#2, cs_sold_date_sk#3, i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] + +(10) ReusedExchange [Reuses operator id: 25] +Output [1]: [d_date_sk#11] + +(11) BroadcastHashJoin [codegen id : 3] +Left keys [1]: [cs_sold_date_sk#3] +Right keys [1]: [d_date_sk#11] +Join type: Inner +Join condition: None + +(12) Project [codegen id : 3] +Output [6]: [cs_ext_sales_price#2, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] +Input [8]: [cs_ext_sales_price#2, cs_sold_date_sk#3, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10, d_date_sk#11] + +(13) HashAggregate [codegen id : 3] +Input [6]: [cs_ext_sales_price#2, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] +Keys [5]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8] +Functions [1]: [partial_sum(UnscaledValue(cs_ext_sales_price#2))] +Aggregate Attributes [1]: [sum#12] +Results [6]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, sum#13] + +(14) Exchange +Input [6]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, sum#13] +Arguments: hashpartitioning(i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, 5), ENSURE_REQUIREMENTS, [plan_id=2] + +(15) HashAggregate [codegen id : 4] +Input [6]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, sum#13] +Keys [5]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8] +Functions [1]: [sum(UnscaledValue(cs_ext_sales_price#2))] +Aggregate Attributes [1]: [sum(UnscaledValue(cs_ext_sales_price#2))#14] +Results [7]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#2))#14,17,2) AS itemrevenue#15, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#2))#14,17,2) AS _w0#16] + +(16) Exchange +Input [7]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#15, _w0#16] +Arguments: hashpartitioning(i_class#9, 5), ENSURE_REQUIREMENTS, [plan_id=3] + +(17) Sort [codegen id : 5] +Input [7]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#15, _w0#16] +Arguments: [i_class#9 ASC NULLS FIRST], false, 0 + +(18) Window +Input [7]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#15, _w0#16] +Arguments: [sum(_w0#16) windowspecdefinition(i_class#9, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS _we0#17], [i_class#9] + +(19) Project [codegen id : 6] +Output [7]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#15, ((_w0#16 * 100) / _we0#17) AS revenueratio#18] +Input [8]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#15, _w0#16, _we0#17] + +(20) TakeOrderedAndProject +Input [7]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#15, revenueratio#18] +Arguments: 100, [i_category#10 ASC NULLS FIRST, i_class#9 ASC NULLS FIRST, i_item_id#6 ASC NULLS FIRST, i_item_desc#7 ASC NULLS FIRST, revenueratio#18 ASC NULLS FIRST], [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#15, revenueratio#18] + +===== Subqueries ===== + +Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#3 IN dynamicpruning#4 +BroadcastExchange (25) ++- * ColumnarToRow (24) + +- CometProject (23) + +- CometFilter (22) + +- CometScan parquet spark_catalog.default.date_dim (21) + + +(21) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#11, d_date#19] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,1999-02-22), LessThanOrEqual(d_date,1999-03-24), IsNotNull(d_date_sk)] +ReadSchema: struct<d_date_sk:int,d_date:date> + +(22) CometFilter +Input [2]: [d_date_sk#11, d_date#19] +Condition : (((isnotnull(d_date#19) AND (d_date#19 >= 1999-02-22)) AND (d_date#19 <= 1999-03-24)) AND isnotnull(d_date_sk#11)) + +(23) CometProject +Input [2]: [d_date_sk#11, d_date#19] +Arguments: [d_date_sk#11], [d_date_sk#11] + +(24) ColumnarToRow [codegen id : 1] +Input [1]: [d_date_sk#11] + +(25) BroadcastExchange +Input [1]: [d_date_sk#11] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] + + diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q20/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q20/simplified.txt new file mode 100644 index 000000000..2a2a392cd --- /dev/null +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q20/simplified.txt @@ -0,0 +1,40 @@ +TakeOrderedAndProject [i_category,i_class,i_item_id,i_item_desc,revenueratio,i_current_price,itemrevenue] + 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) + Sort [i_class] + InputAdapter + Exchange [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] + InputAdapter + Exchange [i_item_id,i_item_desc,i_category,i_class,i_current_price] #2 + WholeStageCodegen (3) + HashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,cs_ext_sales_price] [sum,sum] + Project [cs_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + Project [cs_ext_sales_price,cs_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + BroadcastHashJoin [cs_item_sk,i_item_sk] + ColumnarToRow + InputAdapter + CometFilter [cs_item_sk] + CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] + InputAdapter + BroadcastExchange #4 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometFilter [i_category,i_item_sk] + CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + InputAdapter + ReusedExchange [d_date_sk] #3 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 new file mode 100644 index 000000000..fdebdc8a4 --- /dev/null +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q22/explain.txt @@ -0,0 +1,161 @@ +== Physical Plan == +TakeOrderedAndProject (22) ++- * HashAggregate (21) + +- Exchange (20) + +- * HashAggregate (19) + +- * Expand (18) + +- * Project (17) + +- * BroadcastNestedLoopJoin Inner BuildRight (16) + :- * Project (12) + : +- * BroadcastHashJoin Inner BuildRight (11) + : :- * Project (6) + : : +- * BroadcastHashJoin Inner BuildRight (5) + : : :- * ColumnarToRow (3) + : : : +- CometFilter (2) + : : : +- CometScan parquet spark_catalog.default.inventory (1) + : : +- ReusedExchange (4) + : +- BroadcastExchange (10) + : +- * ColumnarToRow (9) + : +- CometFilter (8) + : +- CometScan parquet spark_catalog.default.item (7) + +- BroadcastExchange (15) + +- * ColumnarToRow (14) + +- CometScan parquet spark_catalog.default.warehouse (13) + + +(1) Scan parquet spark_catalog.default.inventory +Output [3]: [inv_item_sk#1, inv_quantity_on_hand#2, inv_date_sk#3] +Batched: true +Location: InMemoryFileIndex [] +PartitionFilters: [isnotnull(inv_date_sk#3), dynamicpruningexpression(inv_date_sk#3 IN dynamicpruning#4)] +PushedFilters: [IsNotNull(inv_item_sk)] +ReadSchema: struct<inv_item_sk:int,inv_quantity_on_hand:int> + +(2) CometFilter +Input [3]: [inv_item_sk#1, inv_quantity_on_hand#2, inv_date_sk#3] +Condition : isnotnull(inv_item_sk#1) + +(3) ColumnarToRow [codegen id : 4] +Input [3]: [inv_item_sk#1, inv_quantity_on_hand#2, inv_date_sk#3] + +(4) ReusedExchange [Reuses operator id: 27] +Output [1]: [d_date_sk#5] + +(5) BroadcastHashJoin [codegen id : 4] +Left keys [1]: [inv_date_sk#3] +Right keys [1]: [d_date_sk#5] +Join type: Inner +Join condition: None + +(6) Project [codegen id : 4] +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] + +(7) Scan parquet spark_catalog.default.item +Output [5]: [i_item_sk#6, i_brand#7, i_class#8, i_category#9, i_product_name#10] +Batched: true +Location [not included in comparison]/{warehouse_dir}/item] +PushedFilters: [IsNotNull(i_item_sk)] +ReadSchema: struct<i_item_sk:int,i_brand:string,i_class:string,i_category:string,i_product_name:string> + +(8) CometFilter +Input [5]: [i_item_sk#6, i_brand#7, i_class#8, i_category#9, i_product_name#10] +Condition : isnotnull(i_item_sk#6) + +(9) ColumnarToRow [codegen id : 2] +Input [5]: [i_item_sk#6, i_brand#7, i_class#8, i_category#9, i_product_name#10] + +(10) BroadcastExchange +Input [5]: [i_item_sk#6, i_brand#7, i_class#8, i_category#9, i_product_name#10] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=1] + +(11) BroadcastHashJoin [codegen id : 4] +Left keys [1]: [inv_item_sk#1] +Right keys [1]: [i_item_sk#6] +Join type: Inner +Join condition: None + +(12) Project [codegen id : 4] +Output [5]: [inv_quantity_on_hand#2, i_brand#7, i_class#8, i_category#9, i_product_name#10] +Input [7]: [inv_item_sk#1, inv_quantity_on_hand#2, i_item_sk#6, i_brand#7, i_class#8, i_category#9, i_product_name#10] + +(13) Scan parquet spark_catalog.default.warehouse +Output: [] +Batched: true +Location [not included in comparison]/{warehouse_dir}/warehouse] +ReadSchema: struct<> + +(14) ColumnarToRow [codegen id : 3] +Input: [] + +(15) BroadcastExchange +Input: [] +Arguments: IdentityBroadcastMode, [plan_id=2] + +(16) BroadcastNestedLoopJoin [codegen id : 4] +Join type: Inner +Join condition: None + +(17) Project [codegen id : 4] +Output [5]: [inv_quantity_on_hand#2, i_product_name#10, i_brand#7, i_class#8, i_category#9] +Input [5]: [inv_quantity_on_hand#2, i_brand#7, i_class#8, i_category#9, i_product_name#10] + +(18) Expand [codegen id : 4] +Input [5]: [inv_quantity_on_hand#2, i_product_name#10, i_brand#7, i_class#8, i_category#9] +Arguments: [[inv_quantity_on_hand#2, i_product_name#10, i_brand#7, i_class#8, i_category#9, 0], [inv_quantity_on_hand#2, i_product_name#10, i_brand#7, i_class#8, null, 1], [inv_quantity_on_hand#2, i_product_name#10, i_brand#7, null, null, 3], [inv_quantity_on_hand#2, i_product_name#10, null, null, null, 7], [inv_quantity_on_hand#2, null, null, null, null, 15]], [inv_quantity_on_hand#2, i_product_name#11, i_brand#12, i_class#13, i_category#14, spark_grouping_id#15] + +(19) HashAggregate [codegen id : 4] +Input [6]: [inv_quantity_on_hand#2, i_product_name#11, i_brand#12, i_class#13, i_category#14, spark_grouping_id#15] +Keys [5]: [i_product_name#11, i_brand#12, i_class#13, i_category#14, spark_grouping_id#15] +Functions [1]: [partial_avg(inv_quantity_on_hand#2)] +Aggregate Attributes [2]: [sum#16, count#17] +Results [7]: [i_product_name#11, i_brand#12, i_class#13, i_category#14, spark_grouping_id#15, sum#18, count#19] + +(20) Exchange +Input [7]: [i_product_name#11, i_brand#12, i_class#13, i_category#14, spark_grouping_id#15, sum#18, count#19] +Arguments: hashpartitioning(i_product_name#11, i_brand#12, i_class#13, i_category#14, spark_grouping_id#15, 5), ENSURE_REQUIREMENTS, [plan_id=3] + +(21) HashAggregate [codegen id : 5] +Input [7]: [i_product_name#11, i_brand#12, i_class#13, i_category#14, spark_grouping_id#15, sum#18, count#19] +Keys [5]: [i_product_name#11, i_brand#12, i_class#13, i_category#14, spark_grouping_id#15] +Functions [1]: [avg(inv_quantity_on_hand#2)] +Aggregate Attributes [1]: [avg(inv_quantity_on_hand#2)#20] +Results [5]: [i_product_name#11, i_brand#12, i_class#13, i_category#14, avg(inv_quantity_on_hand#2)#20 AS qoh#21] + +(22) TakeOrderedAndProject +Input [5]: [i_product_name#11, i_brand#12, i_class#13, i_category#14, qoh#21] +Arguments: 100, [qoh#21 ASC NULLS FIRST, i_product_name#11 ASC NULLS FIRST, i_brand#12 ASC NULLS FIRST, i_class#13 ASC NULLS FIRST, i_category#14 ASC NULLS FIRST], [i_product_name#11, i_brand#12, i_class#13, i_category#14, qoh#21] + +===== Subqueries ===== + +Subquery:1 Hosting operator id = 1 Hosting Expression = inv_date_sk#3 IN dynamicpruning#4 +BroadcastExchange (27) ++- * ColumnarToRow (26) + +- CometProject (25) + +- CometFilter (24) + +- CometScan parquet spark_catalog.default.date_dim (23) + + +(23) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#5, d_month_seq#22] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1200), LessThanOrEqual(d_month_seq,1211), IsNotNull(d_date_sk)] +ReadSchema: struct<d_date_sk:int,d_month_seq:int> + +(24) CometFilter +Input [2]: [d_date_sk#5, d_month_seq#22] +Condition : (((isnotnull(d_month_seq#22) AND (d_month_seq#22 >= 1200)) AND (d_month_seq#22 <= 1211)) AND isnotnull(d_date_sk#5)) + +(25) CometProject +Input [2]: [d_date_sk#5, d_month_seq#22] +Arguments: [d_date_sk#5], [d_date_sk#5] + +(26) ColumnarToRow [codegen id : 1] +Input [1]: [d_date_sk#5] + +(27) BroadcastExchange +Input [1]: [d_date_sk#5] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] + + diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q22/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q22/simplified.txt new file mode 100644 index 000000000..63a428d4e --- /dev/null +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q22/simplified.txt @@ -0,0 +1,41 @@ +TakeOrderedAndProject [qoh,i_product_name,i_brand,i_class,i_category] + WholeStageCodegen (5) + HashAggregate [i_product_name,i_brand,i_class,i_category,spark_grouping_id,sum,count] [avg(inv_quantity_on_hand),qoh,sum,count] + InputAdapter + Exchange [i_product_name,i_brand,i_class,i_category,spark_grouping_id] #1 + WholeStageCodegen (4) + HashAggregate [i_product_name,i_brand,i_class,i_category,spark_grouping_id,inv_quantity_on_hand] [sum,count,sum,count] + Expand [inv_quantity_on_hand,i_product_name,i_brand,i_class,i_category] + Project [inv_quantity_on_hand,i_product_name,i_brand,i_class,i_category] + BroadcastNestedLoopJoin + Project [inv_quantity_on_hand,i_brand,i_class,i_category,i_product_name] + BroadcastHashJoin [inv_item_sk,i_item_sk] + Project [inv_item_sk,inv_quantity_on_hand] + BroadcastHashJoin [inv_date_sk,d_date_sk] + ColumnarToRow + InputAdapter + CometFilter [inv_item_sk] + CometScan parquet spark_catalog.default.inventory [inv_item_sk,inv_quantity_on_hand,inv_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #2 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_month_seq,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] + InputAdapter + ReusedExchange [d_date_sk] #2 + InputAdapter + BroadcastExchange #3 + WholeStageCodegen (2) + ColumnarToRow + InputAdapter + CometFilter [i_item_sk] + CometScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category,i_product_name] + InputAdapter + BroadcastExchange #4 + WholeStageCodegen (3) + ColumnarToRow + InputAdapter + CometScan parquet spark_catalog.default.warehouse diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q22a/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q22a/explain.txt new file mode 100644 index 000000000..1e5f5c2f7 --- /dev/null +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q22a/explain.txt @@ -0,0 +1,315 @@ +== Physical Plan == +TakeOrderedAndProject (45) ++- Union (44) + :- * HashAggregate (23) + : +- * HashAggregate (22) + : +- * HashAggregate (21) + : +- Exchange (20) + : +- * HashAggregate (19) + : +- * Project (18) + : +- * BroadcastHashJoin Inner BuildRight (17) + : :- * Project (12) + : : +- * BroadcastHashJoin Inner BuildRight (11) + : : :- * Project (6) + : : : +- * BroadcastHashJoin Inner BuildRight (5) + : : : :- * ColumnarToRow (3) + : : : : +- CometFilter (2) + : : : : +- CometScan parquet spark_catalog.default.inventory (1) + : : : +- ReusedExchange (4) + : : +- BroadcastExchange (10) + : : +- * ColumnarToRow (9) + : : +- CometFilter (8) + : : +- CometScan parquet spark_catalog.default.item (7) + : +- BroadcastExchange (16) + : +- * ColumnarToRow (15) + : +- CometFilter (14) + : +- CometScan parquet spark_catalog.default.warehouse (13) + :- * HashAggregate (28) + : +- Exchange (27) + : +- * HashAggregate (26) + : +- * HashAggregate (25) + : +- ReusedExchange (24) + :- * HashAggregate (33) + : +- Exchange (32) + : +- * HashAggregate (31) + : +- * HashAggregate (30) + : +- ReusedExchange (29) + :- * HashAggregate (38) + : +- Exchange (37) + : +- * HashAggregate (36) + : +- * HashAggregate (35) + : +- ReusedExchange (34) + +- * HashAggregate (43) + +- Exchange (42) + +- * HashAggregate (41) + +- * HashAggregate (40) + +- ReusedExchange (39) + + +(1) Scan parquet spark_catalog.default.inventory +Output [4]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4] +Batched: true +Location: InMemoryFileIndex [] +PartitionFilters: [isnotnull(inv_date_sk#4), dynamicpruningexpression(inv_date_sk#4 IN dynamicpruning#5)] +PushedFilters: [IsNotNull(inv_item_sk), IsNotNull(inv_warehouse_sk)] +ReadSchema: struct<inv_item_sk:int,inv_warehouse_sk:int,inv_quantity_on_hand:int> + +(2) CometFilter +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)) + +(3) ColumnarToRow [codegen id : 4] +Input [4]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3, inv_date_sk#4] + +(4) ReusedExchange [Reuses operator id: 50] +Output [1]: [d_date_sk#6] + +(5) BroadcastHashJoin [codegen id : 4] +Left keys [1]: [inv_date_sk#4] +Right keys [1]: [d_date_sk#6] +Join type: Inner +Join condition: None + +(6) Project [codegen id : 4] +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] + +(7) Scan parquet spark_catalog.default.item +Output [5]: [i_item_sk#7, i_brand#8, i_class#9, i_category#10, i_product_name#11] +Batched: true +Location [not included in comparison]/{warehouse_dir}/item] +PushedFilters: [IsNotNull(i_item_sk)] +ReadSchema: struct<i_item_sk:int,i_brand:string,i_class:string,i_category:string,i_product_name:string> + +(8) CometFilter +Input [5]: [i_item_sk#7, i_brand#8, i_class#9, i_category#10, i_product_name#11] +Condition : isnotnull(i_item_sk#7) + +(9) ColumnarToRow [codegen id : 2] +Input [5]: [i_item_sk#7, i_brand#8, i_class#9, i_category#10, i_product_name#11] + +(10) BroadcastExchange +Input [5]: [i_item_sk#7, i_brand#8, i_class#9, i_category#10, i_product_name#11] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=1] + +(11) BroadcastHashJoin [codegen id : 4] +Left keys [1]: [inv_item_sk#1] +Right keys [1]: [i_item_sk#7] +Join type: Inner +Join condition: None + +(12) Project [codegen id : 4] +Output [6]: [inv_warehouse_sk#2, inv_quantity_on_hand#3, i_brand#8, i_class#9, i_category#10, i_product_name#11] +Input [8]: [inv_item_sk#1, inv_warehouse_sk#2, inv_quantity_on_hand#3, i_item_sk#7, i_brand#8, i_class#9, i_category#10, i_product_name#11] + +(13) Scan parquet spark_catalog.default.warehouse +Output [1]: [w_warehouse_sk#12] +Batched: true +Location [not included in comparison]/{warehouse_dir}/warehouse] +PushedFilters: [IsNotNull(w_warehouse_sk)] +ReadSchema: struct<w_warehouse_sk:int> + +(14) CometFilter +Input [1]: [w_warehouse_sk#12] +Condition : isnotnull(w_warehouse_sk#12) + +(15) ColumnarToRow [codegen id : 3] +Input [1]: [w_warehouse_sk#12] + +(16) BroadcastExchange +Input [1]: [w_warehouse_sk#12] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=2] + +(17) BroadcastHashJoin [codegen id : 4] +Left keys [1]: [inv_warehouse_sk#2] +Right keys [1]: [w_warehouse_sk#12] +Join type: Inner +Join condition: None + +(18) Project [codegen id : 4] +Output [5]: [inv_quantity_on_hand#3, i_brand#8, i_class#9, i_category#10, i_product_name#11] +Input [7]: [inv_warehouse_sk#2, inv_quantity_on_hand#3, i_brand#8, i_class#9, i_category#10, i_product_name#11, w_warehouse_sk#12] + +(19) HashAggregate [codegen id : 4] +Input [5]: [inv_quantity_on_hand#3, i_brand#8, i_class#9, i_category#10, i_product_name#11] +Keys [4]: [i_product_name#11, i_brand#8, i_class#9, i_category#10] +Functions [1]: [partial_avg(inv_quantity_on_hand#3)] +Aggregate Attributes [2]: [sum#13, count#14] +Results [6]: [i_product_name#11, i_brand#8, i_class#9, i_category#10, sum#15, count#16] + +(20) Exchange +Input [6]: [i_product_name#11, i_brand#8, i_class#9, i_category#10, sum#15, count#16] +Arguments: hashpartitioning(i_product_name#11, i_brand#8, i_class#9, i_category#10, 5), ENSURE_REQUIREMENTS, [plan_id=3] + +(21) HashAggregate [codegen id : 5] +Input [6]: [i_product_name#11, i_brand#8, i_class#9, i_category#10, sum#15, count#16] +Keys [4]: [i_product_name#11, i_brand#8, i_class#9, i_category#10] +Functions [1]: [avg(inv_quantity_on_hand#3)] +Aggregate Attributes [1]: [avg(inv_quantity_on_hand#3)#17] +Results [5]: [i_product_name#11, i_brand#8, i_class#9, i_category#10, avg(inv_quantity_on_hand#3)#17 AS qoh#18] + +(22) HashAggregate [codegen id : 5] +Input [5]: [i_product_name#11, i_brand#8, i_class#9, i_category#10, qoh#18] +Keys [4]: [i_product_name#11, i_brand#8, i_class#9, i_category#10] +Functions [1]: [partial_avg(qoh#18)] +Aggregate Attributes [2]: [sum#19, count#20] +Results [6]: [i_product_name#11, i_brand#8, i_class#9, i_category#10, sum#21, count#22] + +(23) HashAggregate [codegen id : 5] +Input [6]: [i_product_name#11, i_brand#8, i_class#9, i_category#10, sum#21, count#22] +Keys [4]: [i_product_name#11, i_brand#8, i_class#9, i_category#10] +Functions [1]: [avg(qoh#18)] +Aggregate Attributes [1]: [avg(qoh#18)#23] +Results [5]: [i_product_name#11 AS i_product_name#24, i_brand#8 AS i_brand#25, i_class#9 AS i_class#26, i_category#10 AS i_category#27, avg(qoh#18)#23 AS qoh#28] + +(24) ReusedExchange [Reuses operator id: 20] +Output [6]: [i_product_name#29, i_brand#30, i_class#31, i_category#32, sum#33, count#34] + +(25) HashAggregate [codegen id : 10] +Input [6]: [i_product_name#29, i_brand#30, i_class#31, i_category#32, sum#33, count#34] +Keys [4]: [i_product_name#29, i_brand#30, i_class#31, i_category#32] +Functions [1]: [avg(inv_quantity_on_hand#35)] +Aggregate Attributes [1]: [avg(inv_quantity_on_hand#35)#17] +Results [4]: [i_product_name#29, i_brand#30, i_class#31, avg(inv_quantity_on_hand#35)#17 AS qoh#36] + +(26) HashAggregate [codegen id : 10] +Input [4]: [i_product_name#29, i_brand#30, i_class#31, qoh#36] +Keys [3]: [i_product_name#29, i_brand#30, i_class#31] +Functions [1]: [partial_avg(qoh#36)] +Aggregate Attributes [2]: [sum#37, count#38] +Results [5]: [i_product_name#29, i_brand#30, i_class#31, sum#39, count#40] + +(27) Exchange +Input [5]: [i_product_name#29, i_brand#30, i_class#31, sum#39, count#40] +Arguments: hashpartitioning(i_product_name#29, i_brand#30, i_class#31, 5), ENSURE_REQUIREMENTS, [plan_id=4] + +(28) HashAggregate [codegen id : 11] +Input [5]: [i_product_name#29, i_brand#30, i_class#31, sum#39, count#40] +Keys [3]: [i_product_name#29, i_brand#30, i_class#31] +Functions [1]: [avg(qoh#36)] +Aggregate Attributes [1]: [avg(qoh#36)#41] +Results [5]: [i_product_name#29, i_brand#30, i_class#31, null AS i_category#42, avg(qoh#36)#41 AS qoh#43] + +(29) ReusedExchange [Reuses operator id: 20] +Output [6]: [i_product_name#44, i_brand#45, i_class#46, i_category#47, sum#48, count#49] + +(30) HashAggregate [codegen id : 16] +Input [6]: [i_product_name#44, i_brand#45, i_class#46, i_category#47, sum#48, count#49] +Keys [4]: [i_product_name#44, i_brand#45, i_class#46, i_category#47] +Functions [1]: [avg(inv_quantity_on_hand#50)] +Aggregate Attributes [1]: [avg(inv_quantity_on_hand#50)#17] +Results [3]: [i_product_name#44, i_brand#45, avg(inv_quantity_on_hand#50)#17 AS qoh#51] + +(31) HashAggregate [codegen id : 16] +Input [3]: [i_product_name#44, i_brand#45, qoh#51] +Keys [2]: [i_product_name#44, i_brand#45] +Functions [1]: [partial_avg(qoh#51)] +Aggregate Attributes [2]: [sum#52, count#53] +Results [4]: [i_product_name#44, i_brand#45, sum#54, count#55] + +(32) Exchange +Input [4]: [i_product_name#44, i_brand#45, sum#54, count#55] +Arguments: hashpartitioning(i_product_name#44, i_brand#45, 5), ENSURE_REQUIREMENTS, [plan_id=5] + +(33) HashAggregate [codegen id : 17] +Input [4]: [i_product_name#44, i_brand#45, sum#54, count#55] +Keys [2]: [i_product_name#44, i_brand#45] +Functions [1]: [avg(qoh#51)] +Aggregate Attributes [1]: [avg(qoh#51)#56] +Results [5]: [i_product_name#44, i_brand#45, null AS i_class#57, null AS i_category#58, avg(qoh#51)#56 AS qoh#59] + +(34) ReusedExchange [Reuses operator id: 20] +Output [6]: [i_product_name#60, i_brand#61, i_class#62, i_category#63, sum#64, count#65] + +(35) HashAggregate [codegen id : 22] +Input [6]: [i_product_name#60, i_brand#61, i_class#62, i_category#63, sum#64, count#65] +Keys [4]: [i_product_name#60, i_brand#61, i_class#62, i_category#63] +Functions [1]: [avg(inv_quantity_on_hand#66)] +Aggregate Attributes [1]: [avg(inv_quantity_on_hand#66)#17] +Results [2]: [i_product_name#60, avg(inv_quantity_on_hand#66)#17 AS qoh#67] + +(36) HashAggregate [codegen id : 22] +Input [2]: [i_product_name#60, qoh#67] +Keys [1]: [i_product_name#60] +Functions [1]: [partial_avg(qoh#67)] +Aggregate Attributes [2]: [sum#68, count#69] +Results [3]: [i_product_name#60, sum#70, count#71] + +(37) Exchange +Input [3]: [i_product_name#60, sum#70, count#71] +Arguments: hashpartitioning(i_product_name#60, 5), ENSURE_REQUIREMENTS, [plan_id=6] + +(38) HashAggregate [codegen id : 23] +Input [3]: [i_product_name#60, sum#70, count#71] +Keys [1]: [i_product_name#60] +Functions [1]: [avg(qoh#67)] +Aggregate Attributes [1]: [avg(qoh#67)#72] +Results [5]: [i_product_name#60, null AS i_brand#73, null AS i_class#74, null AS i_category#75, avg(qoh#67)#72 AS qoh#76] + +(39) ReusedExchange [Reuses operator id: 20] +Output [6]: [i_product_name#77, i_brand#78, i_class#79, i_category#80, sum#81, count#82] + +(40) HashAggregate [codegen id : 28] +Input [6]: [i_product_name#77, i_brand#78, i_class#79, i_category#80, sum#81, count#82] +Keys [4]: [i_product_name#77, i_brand#78, i_class#79, i_category#80] +Functions [1]: [avg(inv_quantity_on_hand#83)] +Aggregate Attributes [1]: [avg(inv_quantity_on_hand#83)#17] +Results [1]: [avg(inv_quantity_on_hand#83)#17 AS qoh#84] + +(41) HashAggregate [codegen id : 28] +Input [1]: [qoh#84] +Keys: [] +Functions [1]: [partial_avg(qoh#84)] +Aggregate Attributes [2]: [sum#85, count#86] +Results [2]: [sum#87, count#88] + +(42) Exchange +Input [2]: [sum#87, count#88] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=7] + +(43) HashAggregate [codegen id : 29] +Input [2]: [sum#87, count#88] +Keys: [] +Functions [1]: [avg(qoh#84)] +Aggregate Attributes [1]: [avg(qoh#84)#89] +Results [5]: [null AS i_product_name#90, null AS i_brand#91, null AS i_class#92, null AS i_category#93, avg(qoh#84)#89 AS qoh#94] + +(44) Union + +(45) TakeOrderedAndProject +Input [5]: [i_product_name#24, i_brand#25, i_class#26, i_category#27, qoh#28] +Arguments: 100, [qoh#28 ASC NULLS FIRST, i_product_name#24 ASC NULLS FIRST, i_brand#25 ASC NULLS FIRST, i_class#26 ASC NULLS FIRST, i_category#27 ASC NULLS FIRST], [i_product_name#24, i_brand#25, i_class#26, i_category#27, qoh#28] + +===== Subqueries ===== + +Subquery:1 Hosting operator id = 1 Hosting Expression = inv_date_sk#4 IN dynamicpruning#5 +BroadcastExchange (50) ++- * ColumnarToRow (49) + +- CometProject (48) + +- CometFilter (47) + +- CometScan parquet spark_catalog.default.date_dim (46) + + +(46) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#6, d_month_seq#95] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1212), LessThanOrEqual(d_month_seq,1223), IsNotNull(d_date_sk)] +ReadSchema: struct<d_date_sk:int,d_month_seq:int> + +(47) CometFilter +Input [2]: [d_date_sk#6, d_month_seq#95] +Condition : (((isnotnull(d_month_seq#95) AND (d_month_seq#95 >= 1212)) AND (d_month_seq#95 <= 1223)) AND isnotnull(d_date_sk#6)) + +(48) CometProject +Input [2]: [d_date_sk#6, d_month_seq#95] +Arguments: [d_date_sk#6], [d_date_sk#6] + +(49) ColumnarToRow [codegen id : 1] +Input [1]: [d_date_sk#6] + +(50) BroadcastExchange +Input [1]: [d_date_sk#6] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=8] + + diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q22a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q22a/simplified.txt new file mode 100644 index 000000000..3aa0745ad --- /dev/null +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q22a/simplified.txt @@ -0,0 +1,80 @@ +TakeOrderedAndProject [qoh,i_product_name,i_brand,i_class,i_category] + Union + WholeStageCodegen (5) + HashAggregate [i_product_name,i_brand,i_class,i_category,sum,count] [avg(qoh),i_product_name,i_brand,i_class,i_category,qoh,sum,count] + HashAggregate [i_product_name,i_brand,i_class,i_category,qoh] [sum,count,sum,count] + HashAggregate [i_product_name,i_brand,i_class,i_category,sum,count] [avg(inv_quantity_on_hand),qoh,sum,count] + InputAdapter + Exchange [i_product_name,i_brand,i_class,i_category] #1 + WholeStageCodegen (4) + HashAggregate [i_product_name,i_brand,i_class,i_category,inv_quantity_on_hand] [sum,count,sum,count] + Project [inv_quantity_on_hand,i_brand,i_class,i_category,i_product_name] + BroadcastHashJoin [inv_warehouse_sk,w_warehouse_sk] + Project [inv_warehouse_sk,inv_quantity_on_hand,i_brand,i_class,i_category,i_product_name] + BroadcastHashJoin [inv_item_sk,i_item_sk] + Project [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand] + BroadcastHashJoin [inv_date_sk,d_date_sk] + ColumnarToRow + InputAdapter + CometFilter [inv_item_sk,inv_warehouse_sk] + CometScan parquet spark_catalog.default.inventory [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #2 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_month_seq,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] + InputAdapter + ReusedExchange [d_date_sk] #2 + InputAdapter + BroadcastExchange #3 + WholeStageCodegen (2) + ColumnarToRow + InputAdapter + CometFilter [i_item_sk] + CometScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category,i_product_name] + InputAdapter + BroadcastExchange #4 + WholeStageCodegen (3) + ColumnarToRow + InputAdapter + CometFilter [w_warehouse_sk] + CometScan parquet spark_catalog.default.warehouse [w_warehouse_sk] + WholeStageCodegen (11) + HashAggregate [i_product_name,i_brand,i_class,sum,count] [avg(qoh),i_category,qoh,sum,count] + InputAdapter + Exchange [i_product_name,i_brand,i_class] #5 + WholeStageCodegen (10) + HashAggregate [i_product_name,i_brand,i_class,qoh] [sum,count,sum,count] + HashAggregate [i_product_name,i_brand,i_class,i_category,sum,count] [avg(inv_quantity_on_hand),qoh,sum,count] + InputAdapter + ReusedExchange [i_product_name,i_brand,i_class,i_category,sum,count] #1 + WholeStageCodegen (17) + HashAggregate [i_product_name,i_brand,sum,count] [avg(qoh),i_class,i_category,qoh,sum,count] + InputAdapter + Exchange [i_product_name,i_brand] #6 + WholeStageCodegen (16) + HashAggregate [i_product_name,i_brand,qoh] [sum,count,sum,count] + HashAggregate [i_product_name,i_brand,i_class,i_category,sum,count] [avg(inv_quantity_on_hand),qoh,sum,count] + InputAdapter + ReusedExchange [i_product_name,i_brand,i_class,i_category,sum,count] #1 + WholeStageCodegen (23) + HashAggregate [i_product_name,sum,count] [avg(qoh),i_brand,i_class,i_category,qoh,sum,count] + InputAdapter + Exchange [i_product_name] #7 + WholeStageCodegen (22) + HashAggregate [i_product_name,qoh] [sum,count,sum,count] + HashAggregate [i_product_name,i_brand,i_class,i_category,sum,count] [avg(inv_quantity_on_hand),qoh,sum,count] + InputAdapter + ReusedExchange [i_product_name,i_brand,i_class,i_category,sum,count] #1 + WholeStageCodegen (29) + HashAggregate [sum,count] [avg(qoh),i_product_name,i_brand,i_class,i_category,qoh,sum,count] + InputAdapter + Exchange #8 + WholeStageCodegen (28) + HashAggregate [qoh] [sum,count,sum,count] + HashAggregate [i_product_name,i_brand,i_class,i_category,sum,count] [avg(inv_quantity_on_hand),qoh,sum,count] + InputAdapter + ReusedExchange [i_product_name,i_brand,i_class,i_category,sum,count] #1 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 new file mode 100644 index 000000000..0cba9b059 --- /dev/null +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q24/explain.txt @@ -0,0 +1,437 @@ +== Physical Plan == +* Sort (48) ++- Exchange (47) + +- * Filter (46) + +- * HashAggregate (45) + +- Exchange (44) + +- * HashAggregate (43) + +- * HashAggregate (42) + +- Exchange (41) + +- * HashAggregate (40) + +- * Project (39) + +- * BroadcastHashJoin Inner BuildRight (38) + :- * Project (33) + : +- * BroadcastHashJoin Inner BuildRight (32) + : :- * Project (27) + : : +- * BroadcastHashJoin Inner BuildRight (26) + : : :- * Project (21) + : : : +- * BroadcastHashJoin Inner BuildRight (20) + : : : :- * Project (14) + : : : : +- * SortMergeJoin Inner (13) + : : : : :- * Sort (6) + : : : : : +- Exchange (5) + : : : : : +- * ColumnarToRow (4) + : : : : : +- CometProject (3) + : : : : : +- CometFilter (2) + : : : : : +- CometScan parquet spark_catalog.default.store_sales (1) + : : : : +- * Sort (12) + : : : : +- Exchange (11) + : : : : +- * ColumnarToRow (10) + : : : : +- CometProject (9) + : : : : +- CometFilter (8) + : : : : +- CometScan parquet spark_catalog.default.store_returns (7) + : : : +- BroadcastExchange (19) + : : : +- * ColumnarToRow (18) + : : : +- CometProject (17) + : : : +- CometFilter (16) + : : : +- CometScan parquet spark_catalog.default.store (15) + : : +- BroadcastExchange (25) + : : +- * ColumnarToRow (24) + : : +- CometFilter (23) + : : +- CometScan parquet spark_catalog.default.item (22) + : +- BroadcastExchange (31) + : +- * ColumnarToRow (30) + : +- CometFilter (29) + : +- CometScan parquet spark_catalog.default.customer (28) + +- BroadcastExchange (37) + +- * ColumnarToRow (36) + +- CometFilter (35) + +- CometScan parquet spark_catalog.default.customer_address (34) + + +(1) Scan parquet spark_catalog.default.store_sales +Output [6]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5, ss_sold_date_sk#6] +Batched: true +Location [not included in comparison]/{warehouse_dir}/store_sales] +PushedFilters: [IsNotNull(ss_ticket_number), IsNotNull(ss_item_sk), IsNotNull(ss_store_sk), IsNotNull(ss_customer_sk)] +ReadSchema: struct<ss_item_sk:int,ss_customer_sk:int,ss_store_sk:int,ss_ticket_number:int,ss_net_paid:decimal(7,2)> + +(2) CometFilter +Input [6]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5, ss_sold_date_sk#6] +Condition : (((isnotnull(ss_ticket_number#4) AND isnotnull(ss_item_sk#1)) AND isnotnull(ss_store_sk#3)) AND isnotnull(ss_customer_sk#2)) + +(3) CometProject +Input [6]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5, ss_sold_date_sk#6] +Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5], [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5] + +(4) ColumnarToRow [codegen id : 1] +Input [5]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5] + +(5) Exchange +Input [5]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5] +Arguments: hashpartitioning(ss_ticket_number#4, ss_item_sk#1, 5), ENSURE_REQUIREMENTS, [plan_id=1] + +(6) Sort [codegen id : 2] +Input [5]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5] +Arguments: [ss_ticket_number#4 ASC NULLS FIRST, ss_item_sk#1 ASC NULLS FIRST], false, 0 + +(7) Scan parquet spark_catalog.default.store_returns +Output [3]: [sr_item_sk#7, sr_ticket_number#8, sr_returned_date_sk#9] +Batched: true +Location [not included in comparison]/{warehouse_dir}/store_returns] +PushedFilters: [IsNotNull(sr_ticket_number), IsNotNull(sr_item_sk)] +ReadSchema: struct<sr_item_sk:int,sr_ticket_number:int> + +(8) CometFilter +Input [3]: [sr_item_sk#7, sr_ticket_number#8, sr_returned_date_sk#9] +Condition : (isnotnull(sr_ticket_number#8) AND isnotnull(sr_item_sk#7)) + +(9) CometProject +Input [3]: [sr_item_sk#7, sr_ticket_number#8, sr_returned_date_sk#9] +Arguments: [sr_item_sk#7, sr_ticket_number#8], [sr_item_sk#7, sr_ticket_number#8] + +(10) ColumnarToRow [codegen id : 3] +Input [2]: [sr_item_sk#7, sr_ticket_number#8] + +(11) Exchange +Input [2]: [sr_item_sk#7, sr_ticket_number#8] +Arguments: hashpartitioning(sr_ticket_number#8, sr_item_sk#7, 5), ENSURE_REQUIREMENTS, [plan_id=2] + +(12) Sort [codegen id : 4] +Input [2]: [sr_item_sk#7, sr_ticket_number#8] +Arguments: [sr_ticket_number#8 ASC NULLS FIRST, sr_item_sk#7 ASC NULLS FIRST], false, 0 + +(13) SortMergeJoin [codegen id : 9] +Left keys [2]: [ss_ticket_number#4, ss_item_sk#1] +Right keys [2]: [sr_ticket_number#8, sr_item_sk#7] +Join type: Inner +Join condition: None + +(14) Project [codegen id : 9] +Output [4]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_net_paid#5] +Input [7]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5, sr_item_sk#7, sr_ticket_number#8] + +(15) Scan parquet spark_catalog.default.store +Output [5]: [s_store_sk#10, s_store_name#11, s_market_id#12, s_state#13, s_zip#14] +Batched: true +Location [not included in comparison]/{warehouse_dir}/store] +PushedFilters: [IsNotNull(s_market_id), EqualTo(s_market_id,8), IsNotNull(s_store_sk), IsNotNull(s_zip)] +ReadSchema: struct<s_store_sk:int,s_store_name:string,s_market_id:int,s_state:string,s_zip:string> + +(16) CometFilter +Input [5]: [s_store_sk#10, s_store_name#11, s_market_id#12, s_state#13, s_zip#14] +Condition : (((isnotnull(s_market_id#12) AND (s_market_id#12 = 8)) AND isnotnull(s_store_sk#10)) AND isnotnull(s_zip#14)) + +(17) CometProject +Input [5]: [s_store_sk#10, s_store_name#11, s_market_id#12, s_state#13, s_zip#14] +Arguments: [s_store_sk#10, s_store_name#11, s_state#13, s_zip#14], [s_store_sk#10, s_store_name#11, s_state#13, s_zip#14] + +(18) ColumnarToRow [codegen id : 5] +Input [4]: [s_store_sk#10, s_store_name#11, s_state#13, s_zip#14] + +(19) BroadcastExchange +Input [4]: [s_store_sk#10, s_store_name#11, s_state#13, s_zip#14] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] + +(20) BroadcastHashJoin [codegen id : 9] +Left keys [1]: [ss_store_sk#3] +Right keys [1]: [s_store_sk#10] +Join type: Inner +Join condition: None + +(21) Project [codegen id : 9] +Output [6]: [ss_item_sk#1, ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#13, s_zip#14] +Input [8]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_net_paid#5, s_store_sk#10, s_store_name#11, s_state#13, s_zip#14] + +(22) Scan parquet spark_catalog.default.item +Output [6]: [i_item_sk#15, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20] +Batched: true +Location [not included in comparison]/{warehouse_dir}/item] +PushedFilters: [IsNotNull(i_color), EqualTo(i_color,pale ), IsNotNull(i_item_sk)] +ReadSchema: struct<i_item_sk:int,i_current_price:decimal(7,2),i_size:string,i_color:string,i_units:string,i_manager_id:int> + +(23) CometFilter +Input [6]: [i_item_sk#15, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20] +Condition : ((isnotnull(i_color#18) AND (i_color#18 = pale )) AND isnotnull(i_item_sk#15)) + +(24) ColumnarToRow [codegen id : 6] +Input [6]: [i_item_sk#15, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20] + +(25) BroadcastExchange +Input [6]: [i_item_sk#15, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=4] + +(26) BroadcastHashJoin [codegen id : 9] +Left keys [1]: [ss_item_sk#1] +Right keys [1]: [i_item_sk#15] +Join type: Inner +Join condition: None + +(27) Project [codegen id : 9] +Output [10]: [ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#13, s_zip#14, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20] +Input [12]: [ss_item_sk#1, ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#13, s_zip#14, i_item_sk#15, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20] + +(28) Scan parquet spark_catalog.default.customer +Output [5]: [c_customer_sk#21, c_current_addr_sk#22, c_first_name#23, c_last_name#24, c_birth_country#25] +Batched: true +Location [not included in comparison]/{warehouse_dir}/customer] +PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_current_addr_sk), IsNotNull(c_birth_country)] +ReadSchema: struct<c_customer_sk:int,c_current_addr_sk:int,c_first_name:string,c_last_name:string,c_birth_country:string> + +(29) CometFilter +Input [5]: [c_customer_sk#21, c_current_addr_sk#22, c_first_name#23, c_last_name#24, c_birth_country#25] +Condition : ((isnotnull(c_customer_sk#21) AND isnotnull(c_current_addr_sk#22)) AND isnotnull(c_birth_country#25)) + +(30) ColumnarToRow [codegen id : 7] +Input [5]: [c_customer_sk#21, c_current_addr_sk#22, c_first_name#23, c_last_name#24, c_birth_country#25] + +(31) BroadcastExchange +Input [5]: [c_customer_sk#21, c_current_addr_sk#22, c_first_name#23, c_last_name#24, c_birth_country#25] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=5] + +(32) BroadcastHashJoin [codegen id : 9] +Left keys [1]: [ss_customer_sk#2] +Right keys [1]: [c_customer_sk#21] +Join type: Inner +Join condition: None + +(33) Project [codegen id : 9] +Output [13]: [ss_net_paid#5, s_store_name#11, s_state#13, s_zip#14, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20, c_current_addr_sk#22, c_first_name#23, c_last_name#24, c_birth_country#25] +Input [15]: [ss_customer_sk#2, ss_net_paid#5, s_store_name#11, s_state#13, s_zip#14, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20, c_customer_sk#21, c_current_addr_sk#22, c_first_name#23, c_last_name#24, c_birth_country#25] + +(34) Scan parquet spark_catalog.default.customer_address +Output [4]: [ca_address_sk#26, ca_state#27, ca_zip#28, ca_country#29] +Batched: true +Location [not included in comparison]/{warehouse_dir}/customer_address] +PushedFilters: [IsNotNull(ca_address_sk), IsNotNull(ca_country), IsNotNull(ca_zip)] +ReadSchema: struct<ca_address_sk:int,ca_state:string,ca_zip:string,ca_country:string> + +(35) CometFilter +Input [4]: [ca_address_sk#26, ca_state#27, ca_zip#28, ca_country#29] +Condition : ((isnotnull(ca_address_sk#26) AND isnotnull(ca_country#29)) AND isnotnull(ca_zip#28)) + +(36) ColumnarToRow [codegen id : 8] +Input [4]: [ca_address_sk#26, ca_state#27, ca_zip#28, ca_country#29] + +(37) BroadcastExchange +Input [4]: [ca_address_sk#26, ca_state#27, ca_zip#28, ca_country#29] +Arguments: HashedRelationBroadcastMode(List(input[0, int, false], upper(input[3, string, false]), input[2, string, false]),false), [plan_id=6] + +(38) BroadcastHashJoin [codegen id : 9] +Left keys [3]: [c_current_addr_sk#22, c_birth_country#25, s_zip#14] +Right keys [3]: [ca_address_sk#26, upper(ca_country#29), ca_zip#28] +Join type: Inner +Join condition: None + +(39) Project [codegen id : 9] +Output [11]: [ss_net_paid#5, s_store_name#11, s_state#13, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20, c_first_name#23, c_last_name#24, ca_state#27] +Input [17]: [ss_net_paid#5, s_store_name#11, s_state#13, s_zip#14, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20, c_current_addr_sk#22, c_first_name#23, c_last_name#24, c_birth_country#25, ca_address_sk#26, ca_state#27, ca_zip#28, ca_country#29] + +(40) HashAggregate [codegen id : 9] +Input [11]: [ss_net_paid#5, s_store_name#11, s_state#13, i_current_price#16, i_size#17, i_color#18, i_units#19, i_manager_id#20, c_first_name#23, c_last_name#24, ca_state#27] +Keys [10]: [c_last_name#24, c_first_name#23, s_store_name#11, ca_state#27, s_state#13, i_color#18, i_current_price#16, i_manager_id#20, i_units#19, i_size#17] +Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#5))] +Aggregate Attributes [1]: [sum#30] +Results [11]: [c_last_name#24, c_first_name#23, s_store_name#11, ca_state#27, s_state#13, i_color#18, i_current_price#16, i_manager_id#20, i_units#19, i_size#17, sum#31] + +(41) Exchange +Input [11]: [c_last_name#24, c_first_name#23, s_store_name#11, ca_state#27, s_state#13, i_color#18, i_current_price#16, i_manager_id#20, i_units#19, i_size#17, sum#31] +Arguments: hashpartitioning(c_last_name#24, c_first_name#23, s_store_name#11, ca_state#27, s_state#13, i_color#18, i_current_price#16, i_manager_id#20, i_units#19, i_size#17, 5), ENSURE_REQUIREMENTS, [plan_id=7] + +(42) HashAggregate [codegen id : 10] +Input [11]: [c_last_name#24, c_first_name#23, s_store_name#11, ca_state#27, s_state#13, i_color#18, i_current_price#16, i_manager_id#20, i_units#19, i_size#17, sum#31] +Keys [10]: [c_last_name#24, c_first_name#23, s_store_name#11, ca_state#27, s_state#13, i_color#18, i_current_price#16, i_manager_id#20, i_units#19, i_size#17] +Functions [1]: [sum(UnscaledValue(ss_net_paid#5))] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_paid#5))#32] +Results [4]: [c_last_name#24, c_first_name#23, s_store_name#11, MakeDecimal(sum(UnscaledValue(ss_net_paid#5))#32,17,2) AS netpaid#33] + +(43) HashAggregate [codegen id : 10] +Input [4]: [c_last_name#24, c_first_name#23, s_store_name#11, netpaid#33] +Keys [3]: [c_last_name#24, c_first_name#23, s_store_name#11] +Functions [1]: [partial_sum(netpaid#33)] +Aggregate Attributes [2]: [sum#34, isEmpty#35] +Results [5]: [c_last_name#24, c_first_name#23, s_store_name#11, sum#36, isEmpty#37] + +(44) Exchange +Input [5]: [c_last_name#24, c_first_name#23, s_store_name#11, sum#36, isEmpty#37] +Arguments: hashpartitioning(c_last_name#24, c_first_name#23, s_store_name#11, 5), ENSURE_REQUIREMENTS, [plan_id=8] + +(45) HashAggregate [codegen id : 11] +Input [5]: [c_last_name#24, c_first_name#23, s_store_name#11, sum#36, isEmpty#37] +Keys [3]: [c_last_name#24, c_first_name#23, s_store_name#11] +Functions [1]: [sum(netpaid#33)] +Aggregate Attributes [1]: [sum(netpaid#33)#38] +Results [4]: [c_last_name#24, c_first_name#23, s_store_name#11, sum(netpaid#33)#38 AS paid#39] + +(46) Filter [codegen id : 11] +Input [4]: [c_last_name#24, c_first_name#23, s_store_name#11, paid#39] +Condition : (isnotnull(paid#39) AND (cast(paid#39 as decimal(33,8)) > cast(Subquery scalar-subquery#40, [id=#41] as decimal(33,8)))) + +(47) Exchange +Input [4]: [c_last_name#24, c_first_name#23, s_store_name#11, paid#39] +Arguments: rangepartitioning(c_last_name#24 ASC NULLS FIRST, c_first_name#23 ASC NULLS FIRST, s_store_name#11 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, [plan_id=9] + +(48) Sort [codegen id : 12] +Input [4]: [c_last_name#24, c_first_name#23, s_store_name#11, paid#39] +Arguments: [c_last_name#24 ASC NULLS FIRST, c_first_name#23 ASC NULLS FIRST, s_store_name#11 ASC NULLS FIRST], true, 0 + +===== Subqueries ===== + +Subquery:1 Hosting operator id = 46 Hosting Expression = Subquery scalar-subquery#40, [id=#41] +* HashAggregate (75) ++- Exchange (74) + +- * HashAggregate (73) + +- * HashAggregate (72) + +- Exchange (71) + +- * HashAggregate (70) + +- * Project (69) + +- * BroadcastHashJoin Inner BuildRight (68) + :- * Project (66) + : +- * BroadcastHashJoin Inner BuildRight (65) + : :- * Project (63) + : : +- * BroadcastHashJoin Inner BuildRight (62) + : : :- * Project (57) + : : : +- * BroadcastHashJoin Inner BuildRight (56) + : : : :- * Project (54) + : : : : +- * SortMergeJoin Inner (53) + : : : : :- * Sort (50) + : : : : : +- ReusedExchange (49) + : : : : +- * Sort (52) + : : : : +- ReusedExchange (51) + : : : +- ReusedExchange (55) + : : +- BroadcastExchange (61) + : : +- * ColumnarToRow (60) + : : +- CometFilter (59) + : : +- CometScan parquet spark_catalog.default.item (58) + : +- ReusedExchange (64) + +- ReusedExchange (67) + + +(49) ReusedExchange [Reuses operator id: 5] +Output [5]: [ss_item_sk#42, ss_customer_sk#43, ss_store_sk#44, ss_ticket_number#45, ss_net_paid#46] + +(50) Sort [codegen id : 2] +Input [5]: [ss_item_sk#42, ss_customer_sk#43, ss_store_sk#44, ss_ticket_number#45, ss_net_paid#46] +Arguments: [ss_ticket_number#45 ASC NULLS FIRST, ss_item_sk#42 ASC NULLS FIRST], false, 0 + +(51) ReusedExchange [Reuses operator id: 11] +Output [2]: [sr_item_sk#47, sr_ticket_number#48] + +(52) Sort [codegen id : 4] +Input [2]: [sr_item_sk#47, sr_ticket_number#48] +Arguments: [sr_ticket_number#48 ASC NULLS FIRST, sr_item_sk#47 ASC NULLS FIRST], false, 0 + +(53) SortMergeJoin [codegen id : 9] +Left keys [2]: [ss_ticket_number#45, ss_item_sk#42] +Right keys [2]: [sr_ticket_number#48, sr_item_sk#47] +Join type: Inner +Join condition: None + +(54) Project [codegen id : 9] +Output [4]: [ss_item_sk#42, ss_customer_sk#43, ss_store_sk#44, ss_net_paid#46] +Input [7]: [ss_item_sk#42, ss_customer_sk#43, ss_store_sk#44, ss_ticket_number#45, ss_net_paid#46, sr_item_sk#47, sr_ticket_number#48] + +(55) ReusedExchange [Reuses operator id: 19] +Output [4]: [s_store_sk#49, s_store_name#50, s_state#51, s_zip#52] + +(56) BroadcastHashJoin [codegen id : 9] +Left keys [1]: [ss_store_sk#44] +Right keys [1]: [s_store_sk#49] +Join type: Inner +Join condition: None + +(57) Project [codegen id : 9] +Output [6]: [ss_item_sk#42, ss_customer_sk#43, ss_net_paid#46, s_store_name#50, s_state#51, s_zip#52] +Input [8]: [ss_item_sk#42, ss_customer_sk#43, ss_store_sk#44, ss_net_paid#46, s_store_sk#49, s_store_name#50, s_state#51, s_zip#52] + +(58) Scan parquet spark_catalog.default.item +Output [6]: [i_item_sk#53, i_current_price#54, i_size#55, i_color#56, i_units#57, i_manager_id#58] +Batched: true +Location [not included in comparison]/{warehouse_dir}/item] +PushedFilters: [IsNotNull(i_item_sk)] +ReadSchema: struct<i_item_sk:int,i_current_price:decimal(7,2),i_size:string,i_color:string,i_units:string,i_manager_id:int> + +(59) CometFilter +Input [6]: [i_item_sk#53, i_current_price#54, i_size#55, i_color#56, i_units#57, i_manager_id#58] +Condition : isnotnull(i_item_sk#53) + +(60) ColumnarToRow [codegen id : 6] +Input [6]: [i_item_sk#53, i_current_price#54, i_size#55, i_color#56, i_units#57, i_manager_id#58] + +(61) BroadcastExchange +Input [6]: [i_item_sk#53, i_current_price#54, i_size#55, i_color#56, i_units#57, i_manager_id#58] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=10] + +(62) BroadcastHashJoin [codegen id : 9] +Left keys [1]: [ss_item_sk#42] +Right keys [1]: [i_item_sk#53] +Join type: Inner +Join condition: None + +(63) Project [codegen id : 9] +Output [10]: [ss_customer_sk#43, ss_net_paid#46, s_store_name#50, s_state#51, s_zip#52, i_current_price#54, i_size#55, i_color#56, i_units#57, i_manager_id#58] +Input [12]: [ss_item_sk#42, ss_customer_sk#43, ss_net_paid#46, s_store_name#50, s_state#51, s_zip#52, i_item_sk#53, i_current_price#54, i_size#55, i_color#56, i_units#57, i_manager_id#58] + +(64) ReusedExchange [Reuses operator id: 31] +Output [5]: [c_customer_sk#59, c_current_addr_sk#60, c_first_name#61, c_last_name#62, c_birth_country#63] + +(65) BroadcastHashJoin [codegen id : 9] +Left keys [1]: [ss_customer_sk#43] +Right keys [1]: [c_customer_sk#59] +Join type: Inner +Join condition: None + +(66) Project [codegen id : 9] +Output [13]: [ss_net_paid#46, s_store_name#50, s_state#51, s_zip#52, i_current_price#54, i_size#55, i_color#56, i_units#57, i_manager_id#58, c_current_addr_sk#60, c_first_name#61, c_last_name#62, c_birth_country#63] +Input [15]: [ss_customer_sk#43, ss_net_paid#46, s_store_name#50, s_state#51, s_zip#52, i_current_price#54, i_size#55, i_color#56, i_units#57, i_manager_id#58, c_customer_sk#59, c_current_addr_sk#60, c_first_name#61, c_last_name#62, c_birth_country#63] + +(67) ReusedExchange [Reuses operator id: 37] +Output [4]: [ca_address_sk#64, ca_state#65, ca_zip#66, ca_country#67] + +(68) BroadcastHashJoin [codegen id : 9] +Left keys [3]: [c_current_addr_sk#60, c_birth_country#63, s_zip#52] +Right keys [3]: [ca_address_sk#64, upper(ca_country#67), ca_zip#66] +Join type: Inner +Join condition: None + +(69) Project [codegen id : 9] +Output [11]: [ss_net_paid#46, s_store_name#50, s_state#51, i_current_price#54, i_size#55, i_color#56, i_units#57, i_manager_id#58, c_first_name#61, c_last_name#62, ca_state#65] +Input [17]: [ss_net_paid#46, s_store_name#50, s_state#51, s_zip#52, i_current_price#54, i_size#55, i_color#56, i_units#57, i_manager_id#58, c_current_addr_sk#60, c_first_name#61, c_last_name#62, c_birth_country#63, ca_address_sk#64, ca_state#65, ca_zip#66, ca_country#67] + +(70) HashAggregate [codegen id : 9] +Input [11]: [ss_net_paid#46, s_store_name#50, s_state#51, i_current_price#54, i_size#55, i_color#56, i_units#57, i_manager_id#58, c_first_name#61, c_last_name#62, ca_state#65] +Keys [10]: [c_last_name#62, c_first_name#61, s_store_name#50, ca_state#65, s_state#51, i_color#56, i_current_price#54, i_manager_id#58, i_units#57, i_size#55] +Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#46))] +Aggregate Attributes [1]: [sum#68] +Results [11]: [c_last_name#62, c_first_name#61, s_store_name#50, ca_state#65, s_state#51, i_color#56, i_current_price#54, i_manager_id#58, i_units#57, i_size#55, sum#69] + +(71) Exchange +Input [11]: [c_last_name#62, c_first_name#61, s_store_name#50, ca_state#65, s_state#51, i_color#56, i_current_price#54, i_manager_id#58, i_units#57, i_size#55, sum#69] +Arguments: hashpartitioning(c_last_name#62, c_first_name#61, s_store_name#50, ca_state#65, s_state#51, i_color#56, i_current_price#54, i_manager_id#58, i_units#57, i_size#55, 5), ENSURE_REQUIREMENTS, [plan_id=11] + +(72) HashAggregate [codegen id : 10] +Input [11]: [c_last_name#62, c_first_name#61, s_store_name#50, ca_state#65, s_state#51, i_color#56, i_current_price#54, i_manager_id#58, i_units#57, i_size#55, sum#69] +Keys [10]: [c_last_name#62, c_first_name#61, s_store_name#50, ca_state#65, s_state#51, i_color#56, i_current_price#54, i_manager_id#58, i_units#57, i_size#55] +Functions [1]: [sum(UnscaledValue(ss_net_paid#46))] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_paid#46))#32] +Results [1]: [MakeDecimal(sum(UnscaledValue(ss_net_paid#46))#32,17,2) AS netpaid#70] + +(73) HashAggregate [codegen id : 10] +Input [1]: [netpaid#70] +Keys: [] +Functions [1]: [partial_avg(netpaid#70)] +Aggregate Attributes [2]: [sum#71, count#72] +Results [2]: [sum#73, count#74] + +(74) Exchange +Input [2]: [sum#73, count#74] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=12] + +(75) HashAggregate [codegen id : 11] +Input [2]: [sum#73, count#74] +Keys: [] +Functions [1]: [avg(netpaid#70)] +Aggregate Attributes [1]: [avg(netpaid#70)#75] +Results [1]: [(0.05 * avg(netpaid#70)#75) AS (0.05 * avg(netpaid))#76] + + 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 new file mode 100644 index 000000000..7024f439f --- /dev/null +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q24/simplified.txt @@ -0,0 +1,122 @@ +WholeStageCodegen (12) + Sort [c_last_name,c_first_name,s_store_name] + InputAdapter + Exchange [c_last_name,c_first_name,s_store_name] #1 + WholeStageCodegen (11) + Filter [paid] + Subquery #1 + WholeStageCodegen (11) + HashAggregate [sum,count] [avg(netpaid),(0.05 * avg(netpaid)),sum,count] + InputAdapter + Exchange #10 + WholeStageCodegen (10) + HashAggregate [netpaid] [sum,count,sum,count] + HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,sum] [sum(UnscaledValue(ss_net_paid)),netpaid,sum] + InputAdapter + Exchange [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size] #11 + WholeStageCodegen (9) + HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,ss_net_paid] [sum,sum] + Project [ss_net_paid,s_store_name,s_state,i_current_price,i_size,i_color,i_units,i_manager_id,c_first_name,c_last_name,ca_state] + BroadcastHashJoin [c_current_addr_sk,c_birth_country,s_zip,ca_address_sk,ca_country,ca_zip] + Project [ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id,c_current_addr_sk,c_first_name,c_last_name,c_birth_country] + BroadcastHashJoin [ss_customer_sk,c_customer_sk] + Project [ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id] + BroadcastHashJoin [ss_item_sk,i_item_sk] + Project [ss_item_sk,ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip] + BroadcastHashJoin [ss_store_sk,s_store_sk] + Project [ss_item_sk,ss_customer_sk,ss_store_sk,ss_net_paid] + SortMergeJoin [ss_ticket_number,ss_item_sk,sr_ticket_number,sr_item_sk] + InputAdapter + WholeStageCodegen (2) + Sort [ss_ticket_number,ss_item_sk] + InputAdapter + ReusedExchange [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid] #4 + InputAdapter + WholeStageCodegen (4) + Sort [sr_ticket_number,sr_item_sk] + InputAdapter + ReusedExchange [sr_item_sk,sr_ticket_number] #5 + InputAdapter + ReusedExchange [s_store_sk,s_store_name,s_state,s_zip] #6 + InputAdapter + BroadcastExchange #12 + WholeStageCodegen (6) + ColumnarToRow + InputAdapter + CometFilter [i_item_sk] + CometScan parquet spark_catalog.default.item [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] + InputAdapter + ReusedExchange [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name,c_birth_country] #8 + InputAdapter + ReusedExchange [ca_address_sk,ca_state,ca_zip,ca_country] #9 + HashAggregate [c_last_name,c_first_name,s_store_name,sum,isEmpty] [sum(netpaid),paid,sum,isEmpty] + InputAdapter + Exchange [c_last_name,c_first_name,s_store_name] #2 + WholeStageCodegen (10) + HashAggregate [c_last_name,c_first_name,s_store_name,netpaid] [sum,isEmpty,sum,isEmpty] + HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,sum] [sum(UnscaledValue(ss_net_paid)),netpaid,sum] + InputAdapter + Exchange [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size] #3 + WholeStageCodegen (9) + HashAggregate [c_last_name,c_first_name,s_store_name,ca_state,s_state,i_color,i_current_price,i_manager_id,i_units,i_size,ss_net_paid] [sum,sum] + Project [ss_net_paid,s_store_name,s_state,i_current_price,i_size,i_color,i_units,i_manager_id,c_first_name,c_last_name,ca_state] + BroadcastHashJoin [c_current_addr_sk,c_birth_country,s_zip,ca_address_sk,ca_country,ca_zip] + Project [ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id,c_current_addr_sk,c_first_name,c_last_name,c_birth_country] + BroadcastHashJoin [ss_customer_sk,c_customer_sk] + Project [ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id] + BroadcastHashJoin [ss_item_sk,i_item_sk] + Project [ss_item_sk,ss_customer_sk,ss_net_paid,s_store_name,s_state,s_zip] + BroadcastHashJoin [ss_store_sk,s_store_sk] + Project [ss_item_sk,ss_customer_sk,ss_store_sk,ss_net_paid] + SortMergeJoin [ss_ticket_number,ss_item_sk,sr_ticket_number,sr_item_sk] + InputAdapter + WholeStageCodegen (2) + Sort [ss_ticket_number,ss_item_sk] + InputAdapter + Exchange [ss_ticket_number,ss_item_sk] #4 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid] + CometFilter [ss_ticket_number,ss_item_sk,ss_store_sk,ss_customer_sk] + CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_paid,ss_sold_date_sk] + InputAdapter + WholeStageCodegen (4) + Sort [sr_ticket_number,sr_item_sk] + InputAdapter + Exchange [sr_ticket_number,sr_item_sk] #5 + WholeStageCodegen (3) + ColumnarToRow + InputAdapter + CometProject [sr_item_sk,sr_ticket_number] + CometFilter [sr_ticket_number,sr_item_sk] + CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk] + InputAdapter + BroadcastExchange #6 + WholeStageCodegen (5) + ColumnarToRow + InputAdapter + CometProject [s_store_sk,s_store_name,s_state,s_zip] + CometFilter [s_market_id,s_store_sk,s_zip] + CometScan parquet spark_catalog.default.store [s_store_sk,s_store_name,s_market_id,s_state,s_zip] + InputAdapter + BroadcastExchange #7 + WholeStageCodegen (6) + ColumnarToRow + InputAdapter + CometFilter [i_color,i_item_sk] + CometScan parquet spark_catalog.default.item [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] + InputAdapter + BroadcastExchange #8 + WholeStageCodegen (7) + ColumnarToRow + InputAdapter + CometFilter [c_customer_sk,c_current_addr_sk,c_birth_country] + CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk,c_first_name,c_last_name,c_birth_country] + InputAdapter + BroadcastExchange #9 + WholeStageCodegen (8) + ColumnarToRow + InputAdapter + CometFilter [ca_address_sk,ca_country,ca_zip] + CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state,ca_zip,ca_country] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q27a/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q27a/explain.txt new file mode 100644 index 000000000..fc43e7271 --- /dev/null +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q27a/explain.txt @@ -0,0 +1,457 @@ +== Physical Plan == +TakeOrderedAndProject (73) ++- Union (72) + :- * HashAggregate (28) + : +- Exchange (27) + : +- * HashAggregate (26) + : +- * Project (25) + : +- * BroadcastHashJoin Inner BuildRight (24) + : :- * Project (19) + : : +- * BroadcastHashJoin Inner BuildRight (18) + : : :- * Project (13) + : : : +- * BroadcastHashJoin Inner BuildRight (12) + : : : :- * Project (10) + : : : : +- * BroadcastHashJoin Inner BuildRight (9) + : : : : :- * ColumnarToRow (3) + : : : : : +- CometFilter (2) + : : : : : +- CometScan parquet spark_catalog.default.store_sales (1) + : : : : +- BroadcastExchange (8) + : : : : +- * ColumnarToRow (7) + : : : : +- CometProject (6) + : : : : +- CometFilter (5) + : : : : +- CometScan parquet spark_catalog.default.customer_demographics (4) + : : : +- ReusedExchange (11) + : : +- BroadcastExchange (17) + : : +- * ColumnarToRow (16) + : : +- CometFilter (15) + : : +- CometScan parquet spark_catalog.default.store (14) + : +- BroadcastExchange (23) + : +- * ColumnarToRow (22) + : +- CometFilter (21) + : +- CometScan parquet spark_catalog.default.item (20) + :- * HashAggregate (50) + : +- Exchange (49) + : +- * HashAggregate (48) + : +- * Project (47) + : +- * BroadcastHashJoin Inner BuildRight (46) + : :- * Project (44) + : : +- * BroadcastHashJoin Inner BuildRight (43) + : : :- * Project (37) + : : : +- * BroadcastHashJoin Inner BuildRight (36) + : : : :- * Project (34) + : : : : +- * BroadcastHashJoin Inner BuildRight (33) + : : : : :- * ColumnarToRow (31) + : : : : : +- CometFilter (30) + : : : : : +- CometScan parquet spark_catalog.default.store_sales (29) + : : : : +- ReusedExchange (32) + : : : +- ReusedExchange (35) + : : +- BroadcastExchange (42) + : : +- * ColumnarToRow (41) + : : +- CometProject (40) + : : +- CometFilter (39) + : : +- CometScan parquet spark_catalog.default.store (38) + : +- ReusedExchange (45) + +- * HashAggregate (71) + +- Exchange (70) + +- * HashAggregate (69) + +- * Project (68) + +- * BroadcastHashJoin Inner BuildRight (67) + :- * Project (62) + : +- * BroadcastHashJoin Inner BuildRight (61) + : :- * Project (59) + : : +- * BroadcastHashJoin Inner BuildRight (58) + : : :- * Project (56) + : : : +- * BroadcastHashJoin Inner BuildRight (55) + : : : :- * ColumnarToRow (53) + : : : : +- CometFilter (52) + : : : : +- CometScan parquet spark_catalog.default.store_sales (51) + : : : +- ReusedExchange (54) + : : +- ReusedExchange (57) + : +- ReusedExchange (60) + +- BroadcastExchange (66) + +- * ColumnarToRow (65) + +- CometFilter (64) + +- CometScan parquet spark_catalog.default.item (63) + + +(1) Scan parquet spark_catalog.default.store_sales +Output [8]: [ss_item_sk#1, ss_cdemo_sk#2, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8] +Batched: true +Location: InMemoryFileIndex [] +PartitionFilters: [isnotnull(ss_sold_date_sk#8), dynamicpruningexpression(ss_sold_date_sk#8 IN dynamicpruning#9)] +PushedFilters: [IsNotNull(ss_cdemo_sk), IsNotNull(ss_store_sk), IsNotNull(ss_item_sk)] +ReadSchema: struct<ss_item_sk:int,ss_cdemo_sk:int,ss_store_sk:int,ss_quantity:int,ss_list_price:decimal(7,2),ss_sales_price:decimal(7,2),ss_coupon_amt:decimal(7,2)> + +(2) CometFilter +Input [8]: [ss_item_sk#1, ss_cdemo_sk#2, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8] +Condition : ((isnotnull(ss_cdemo_sk#2) AND isnotnull(ss_store_sk#3)) AND isnotnull(ss_item_sk#1)) + +(3) ColumnarToRow [codegen id : 5] +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] + +(4) Scan parquet spark_catalog.default.customer_demographics +Output [4]: [cd_demo_sk#10, cd_gender#11, cd_marital_status#12, cd_education_status#13] +Batched: true +Location [not included in comparison]/{warehouse_dir}/customer_demographics] +PushedFilters: [IsNotNull(cd_gender), IsNotNull(cd_marital_status), IsNotNull(cd_education_status), EqualTo(cd_gender,F), EqualTo(cd_marital_status,W), EqualTo(cd_education_status,Primary ), IsNotNull(cd_demo_sk)] +ReadSchema: struct<cd_demo_sk:int,cd_gender:string,cd_marital_status:string,cd_education_status:string> + +(5) CometFilter +Input [4]: [cd_demo_sk#10, cd_gender#11, cd_marital_status#12, cd_education_status#13] +Condition : ((((((isnotnull(cd_gender#11) AND isnotnull(cd_marital_status#12)) AND isnotnull(cd_education_status#13)) AND (cd_gender#11 = F)) AND (cd_marital_status#12 = W)) AND (cd_education_status#13 = Primary )) AND isnotnull(cd_demo_sk#10)) + +(6) CometProject +Input [4]: [cd_demo_sk#10, cd_gender#11, cd_marital_status#12, cd_education_status#13] +Arguments: [cd_demo_sk#10], [cd_demo_sk#10] + +(7) ColumnarToRow [codegen id : 1] +Input [1]: [cd_demo_sk#10] + +(8) BroadcastExchange +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] +Left keys [1]: [ss_cdemo_sk#2] +Right keys [1]: [cd_demo_sk#10] +Join type: Inner +Join condition: None + +(10) Project [codegen id : 5] +Output [7]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, ss_sold_date_sk#8] +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: 78] +Output [1]: [d_date_sk#14] + +(12) BroadcastHashJoin [codegen id : 5] +Left keys [1]: [ss_sold_date_sk#8] +Right keys [1]: [d_date_sk#14] +Join type: Inner +Join condition: None + +(13) Project [codegen id : 5] +Output [6]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7] +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] + +(14) Scan parquet spark_catalog.default.store +Output [2]: [s_store_sk#15, s_state#16] +Batched: true +Location [not included in comparison]/{warehouse_dir}/store] +PushedFilters: [IsNotNull(s_state), EqualTo(s_state,TN), IsNotNull(s_store_sk)] +ReadSchema: struct<s_store_sk:int,s_state:string> + +(15) CometFilter +Input [2]: [s_store_sk#15, s_state#16] +Condition : ((isnotnull(s_state#16) AND (s_state#16 = TN)) AND isnotnull(s_store_sk#15)) + +(16) ColumnarToRow [codegen id : 3] +Input [2]: [s_store_sk#15, s_state#16] + +(17) BroadcastExchange +Input [2]: [s_store_sk#15, s_state#16] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=2] + +(18) BroadcastHashJoin [codegen id : 5] +Left keys [1]: [ss_store_sk#3] +Right keys [1]: [s_store_sk#15] +Join type: Inner +Join condition: None + +(19) Project [codegen id : 5] +Output [6]: [ss_item_sk#1, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, s_state#16] +Input [8]: [ss_item_sk#1, ss_store_sk#3, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, s_store_sk#15, s_state#16] + +(20) Scan parquet spark_catalog.default.item +Output [2]: [i_item_sk#17, i_item_id#18] +Batched: true +Location [not included in comparison]/{warehouse_dir}/item] +PushedFilters: [IsNotNull(i_item_sk)] +ReadSchema: struct<i_item_sk:int,i_item_id:string> + +(21) CometFilter +Input [2]: [i_item_sk#17, i_item_id#18] +Condition : isnotnull(i_item_sk#17) + +(22) ColumnarToRow [codegen id : 4] +Input [2]: [i_item_sk#17, i_item_id#18] + +(23) BroadcastExchange +Input [2]: [i_item_sk#17, i_item_id#18] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=3] + +(24) BroadcastHashJoin [codegen id : 5] +Left keys [1]: [ss_item_sk#1] +Right keys [1]: [i_item_sk#17] +Join type: Inner +Join condition: None + +(25) Project [codegen id : 5] +Output [6]: [i_item_id#18, s_state#16, ss_quantity#4 AS agg1#19, ss_list_price#5 AS agg2#20, ss_coupon_amt#7 AS agg3#21, ss_sales_price#6 AS agg4#22] +Input [8]: [ss_item_sk#1, ss_quantity#4, ss_list_price#5, ss_sales_price#6, ss_coupon_amt#7, s_state#16, i_item_sk#17, i_item_id#18] + +(26) HashAggregate [codegen id : 5] +Input [6]: [i_item_id#18, s_state#16, agg1#19, agg2#20, agg3#21, agg4#22] +Keys [2]: [i_item_id#18, s_state#16] +Functions [4]: [partial_avg(agg1#19), partial_avg(UnscaledValue(agg2#20)), partial_avg(UnscaledValue(agg3#21)), partial_avg(UnscaledValue(agg4#22))] +Aggregate Attributes [8]: [sum#23, count#24, sum#25, count#26, sum#27, count#28, sum#29, count#30] +Results [10]: [i_item_id#18, s_state#16, sum#31, count#32, sum#33, count#34, sum#35, count#36, sum#37, count#38] + +(27) Exchange +Input [10]: [i_item_id#18, s_state#16, sum#31, count#32, sum#33, count#34, sum#35, count#36, sum#37, count#38] +Arguments: hashpartitioning(i_item_id#18, s_state#16, 5), ENSURE_REQUIREMENTS, [plan_id=4] + +(28) HashAggregate [codegen id : 6] +Input [10]: [i_item_id#18, s_state#16, sum#31, count#32, sum#33, count#34, sum#35, count#36, sum#37, count#38] +Keys [2]: [i_item_id#18, s_state#16] +Functions [4]: [avg(agg1#19), avg(UnscaledValue(agg2#20)), avg(UnscaledValue(agg3#21)), avg(UnscaledValue(agg4#22))] +Aggregate Attributes [4]: [avg(agg1#19)#39, avg(UnscaledValue(agg2#20))#40, avg(UnscaledValue(agg3#21))#41, avg(UnscaledValue(agg4#22))#42] +Results [7]: [i_item_id#18, s_state#16, 0 AS g_state#43, avg(agg1#19)#39 AS agg1#44, cast((avg(UnscaledValue(agg2#20))#40 / 100.0) as decimal(11,6)) AS agg2#45, cast((avg(UnscaledValue(agg3#21))#41 / 100.0) as decimal(11,6)) AS agg3#46, cast((avg(UnscaledValue(agg4#22))#42 / 100.0) as decimal(11,6)) AS agg4#47] + +(29) Scan parquet spark_catalog.default.store_sales +Output [8]: [ss_item_sk#48, ss_cdemo_sk#49, ss_store_sk#50, ss_quantity#51, ss_list_price#52, ss_sales_price#53, ss_coupon_amt#54, ss_sold_date_sk#55] +Batched: true +Location: InMemoryFileIndex [] +PartitionFilters: [isnotnull(ss_sold_date_sk#55), dynamicpruningexpression(ss_sold_date_sk#55 IN dynamicpruning#56)] +PushedFilters: [IsNotNull(ss_cdemo_sk), IsNotNull(ss_store_sk), IsNotNull(ss_item_sk)] +ReadSchema: struct<ss_item_sk:int,ss_cdemo_sk:int,ss_store_sk:int,ss_quantity:int,ss_list_price:decimal(7,2),ss_sales_price:decimal(7,2),ss_coupon_amt:decimal(7,2)> + +(30) CometFilter +Input [8]: [ss_item_sk#48, ss_cdemo_sk#49, ss_store_sk#50, ss_quantity#51, ss_list_price#52, ss_sales_price#53, ss_coupon_amt#54, ss_sold_date_sk#55] +Condition : ((isnotnull(ss_cdemo_sk#49) AND isnotnull(ss_store_sk#50)) AND isnotnull(ss_item_sk#48)) + +(31) ColumnarToRow [codegen id : 11] +Input [8]: [ss_item_sk#48, ss_cdemo_sk#49, ss_store_sk#50, ss_quantity#51, ss_list_price#52, ss_sales_price#53, ss_coupon_amt#54, ss_sold_date_sk#55] + +(32) ReusedExchange [Reuses operator id: 8] +Output [1]: [cd_demo_sk#57] + +(33) BroadcastHashJoin [codegen id : 11] +Left keys [1]: [ss_cdemo_sk#49] +Right keys [1]: [cd_demo_sk#57] +Join type: Inner +Join condition: None + +(34) Project [codegen id : 11] +Output [7]: [ss_item_sk#48, ss_store_sk#50, ss_quantity#51, ss_list_price#52, ss_sales_price#53, ss_coupon_amt#54, ss_sold_date_sk#55] +Input [9]: [ss_item_sk#48, ss_cdemo_sk#49, ss_store_sk#50, ss_quantity#51, ss_list_price#52, ss_sales_price#53, ss_coupon_amt#54, ss_sold_date_sk#55, cd_demo_sk#57] + +(35) ReusedExchange [Reuses operator id: 78] +Output [1]: [d_date_sk#58] + +(36) BroadcastHashJoin [codegen id : 11] +Left keys [1]: [ss_sold_date_sk#55] +Right keys [1]: [d_date_sk#58] +Join type: Inner +Join condition: None + +(37) Project [codegen id : 11] +Output [6]: [ss_item_sk#48, ss_store_sk#50, ss_quantity#51, ss_list_price#52, ss_sales_price#53, ss_coupon_amt#54] +Input [8]: [ss_item_sk#48, ss_store_sk#50, ss_quantity#51, ss_list_price#52, ss_sales_price#53, ss_coupon_amt#54, ss_sold_date_sk#55, d_date_sk#58] + +(38) Scan parquet spark_catalog.default.store +Output [2]: [s_store_sk#59, s_state#60] +Batched: true +Location [not included in comparison]/{warehouse_dir}/store] +PushedFilters: [IsNotNull(s_state), EqualTo(s_state,TN), IsNotNull(s_store_sk)] +ReadSchema: struct<s_store_sk:int,s_state:string> + +(39) CometFilter +Input [2]: [s_store_sk#59, s_state#60] +Condition : ((isnotnull(s_state#60) AND (s_state#60 = TN)) AND isnotnull(s_store_sk#59)) + +(40) CometProject +Input [2]: [s_store_sk#59, s_state#60] +Arguments: [s_store_sk#59], [s_store_sk#59] + +(41) ColumnarToRow [codegen id : 9] +Input [1]: [s_store_sk#59] + +(42) BroadcastExchange +Input [1]: [s_store_sk#59] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] + +(43) BroadcastHashJoin [codegen id : 11] +Left keys [1]: [ss_store_sk#50] +Right keys [1]: [s_store_sk#59] +Join type: Inner +Join condition: None + +(44) Project [codegen id : 11] +Output [5]: [ss_item_sk#48, ss_quantity#51, ss_list_price#52, ss_sales_price#53, ss_coupon_amt#54] +Input [7]: [ss_item_sk#48, ss_store_sk#50, ss_quantity#51, ss_list_price#52, ss_sales_price#53, ss_coupon_amt#54, s_store_sk#59] + +(45) ReusedExchange [Reuses operator id: 23] +Output [2]: [i_item_sk#61, i_item_id#62] + +(46) BroadcastHashJoin [codegen id : 11] +Left keys [1]: [ss_item_sk#48] +Right keys [1]: [i_item_sk#61] +Join type: Inner +Join condition: None + +(47) Project [codegen id : 11] +Output [5]: [i_item_id#62, ss_quantity#51 AS agg1#63, ss_list_price#52 AS agg2#64, ss_coupon_amt#54 AS agg3#65, ss_sales_price#53 AS agg4#66] +Input [7]: [ss_item_sk#48, ss_quantity#51, ss_list_price#52, ss_sales_price#53, ss_coupon_amt#54, i_item_sk#61, i_item_id#62] + +(48) HashAggregate [codegen id : 11] +Input [5]: [i_item_id#62, agg1#63, agg2#64, agg3#65, agg4#66] +Keys [1]: [i_item_id#62] +Functions [4]: [partial_avg(agg1#63), partial_avg(UnscaledValue(agg2#64)), partial_avg(UnscaledValue(agg3#65)), partial_avg(UnscaledValue(agg4#66))] +Aggregate Attributes [8]: [sum#67, count#68, sum#69, count#70, sum#71, count#72, sum#73, count#74] +Results [9]: [i_item_id#62, sum#75, count#76, sum#77, count#78, sum#79, count#80, sum#81, count#82] + +(49) Exchange +Input [9]: [i_item_id#62, sum#75, count#76, sum#77, count#78, sum#79, count#80, sum#81, count#82] +Arguments: hashpartitioning(i_item_id#62, 5), ENSURE_REQUIREMENTS, [plan_id=6] + +(50) HashAggregate [codegen id : 12] +Input [9]: [i_item_id#62, sum#75, count#76, sum#77, count#78, sum#79, count#80, sum#81, count#82] +Keys [1]: [i_item_id#62] +Functions [4]: [avg(agg1#63), avg(UnscaledValue(agg2#64)), avg(UnscaledValue(agg3#65)), avg(UnscaledValue(agg4#66))] +Aggregate Attributes [4]: [avg(agg1#63)#83, avg(UnscaledValue(agg2#64))#84, avg(UnscaledValue(agg3#65))#85, avg(UnscaledValue(agg4#66))#86] +Results [7]: [i_item_id#62, null AS s_state#87, 1 AS g_state#88, avg(agg1#63)#83 AS agg1#89, cast((avg(UnscaledValue(agg2#64))#84 / 100.0) as decimal(11,6)) AS agg2#90, cast((avg(UnscaledValue(agg3#65))#85 / 100.0) as decimal(11,6)) AS agg3#91, cast((avg(UnscaledValue(agg4#66))#86 / 100.0) as decimal(11,6)) AS agg4#92] + +(51) Scan parquet spark_catalog.default.store_sales +Output [8]: [ss_item_sk#93, ss_cdemo_sk#94, ss_store_sk#95, ss_quantity#96, ss_list_price#97, ss_sales_price#98, ss_coupon_amt#99, ss_sold_date_sk#100] +Batched: true +Location: InMemoryFileIndex [] +PartitionFilters: [isnotnull(ss_sold_date_sk#100), dynamicpruningexpression(ss_sold_date_sk#100 IN dynamicpruning#101)] +PushedFilters: [IsNotNull(ss_cdemo_sk), IsNotNull(ss_store_sk), IsNotNull(ss_item_sk)] +ReadSchema: struct<ss_item_sk:int,ss_cdemo_sk:int,ss_store_sk:int,ss_quantity:int,ss_list_price:decimal(7,2),ss_sales_price:decimal(7,2),ss_coupon_amt:decimal(7,2)> + +(52) CometFilter +Input [8]: [ss_item_sk#93, ss_cdemo_sk#94, ss_store_sk#95, ss_quantity#96, ss_list_price#97, ss_sales_price#98, ss_coupon_amt#99, ss_sold_date_sk#100] +Condition : ((isnotnull(ss_cdemo_sk#94) AND isnotnull(ss_store_sk#95)) AND isnotnull(ss_item_sk#93)) + +(53) ColumnarToRow [codegen id : 17] +Input [8]: [ss_item_sk#93, ss_cdemo_sk#94, ss_store_sk#95, ss_quantity#96, ss_list_price#97, ss_sales_price#98, ss_coupon_amt#99, ss_sold_date_sk#100] + +(54) ReusedExchange [Reuses operator id: 8] +Output [1]: [cd_demo_sk#102] + +(55) BroadcastHashJoin [codegen id : 17] +Left keys [1]: [ss_cdemo_sk#94] +Right keys [1]: [cd_demo_sk#102] +Join type: Inner +Join condition: None + +(56) Project [codegen id : 17] +Output [7]: [ss_item_sk#93, ss_store_sk#95, ss_quantity#96, ss_list_price#97, ss_sales_price#98, ss_coupon_amt#99, ss_sold_date_sk#100] +Input [9]: [ss_item_sk#93, ss_cdemo_sk#94, ss_store_sk#95, ss_quantity#96, ss_list_price#97, ss_sales_price#98, ss_coupon_amt#99, ss_sold_date_sk#100, cd_demo_sk#102] + +(57) ReusedExchange [Reuses operator id: 78] +Output [1]: [d_date_sk#103] + +(58) BroadcastHashJoin [codegen id : 17] +Left keys [1]: [ss_sold_date_sk#100] +Right keys [1]: [d_date_sk#103] +Join type: Inner +Join condition: None + +(59) Project [codegen id : 17] +Output [6]: [ss_item_sk#93, ss_store_sk#95, ss_quantity#96, ss_list_price#97, ss_sales_price#98, ss_coupon_amt#99] +Input [8]: [ss_item_sk#93, ss_store_sk#95, ss_quantity#96, ss_list_price#97, ss_sales_price#98, ss_coupon_amt#99, ss_sold_date_sk#100, d_date_sk#103] + +(60) ReusedExchange [Reuses operator id: 42] +Output [1]: [s_store_sk#104] + +(61) BroadcastHashJoin [codegen id : 17] +Left keys [1]: [ss_store_sk#95] +Right keys [1]: [s_store_sk#104] +Join type: Inner +Join condition: None + +(62) Project [codegen id : 17] +Output [5]: [ss_item_sk#93, ss_quantity#96, ss_list_price#97, ss_sales_price#98, ss_coupon_amt#99] +Input [7]: [ss_item_sk#93, ss_store_sk#95, ss_quantity#96, ss_list_price#97, ss_sales_price#98, ss_coupon_amt#99, s_store_sk#104] + +(63) Scan parquet spark_catalog.default.item +Output [1]: [i_item_sk#105] +Batched: true +Location [not included in comparison]/{warehouse_dir}/item] +PushedFilters: [IsNotNull(i_item_sk)] +ReadSchema: struct<i_item_sk:int> + +(64) CometFilter +Input [1]: [i_item_sk#105] +Condition : isnotnull(i_item_sk#105) + +(65) ColumnarToRow [codegen id : 16] +Input [1]: [i_item_sk#105] + +(66) BroadcastExchange +Input [1]: [i_item_sk#105] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=7] + +(67) BroadcastHashJoin [codegen id : 17] +Left keys [1]: [ss_item_sk#93] +Right keys [1]: [i_item_sk#105] +Join type: Inner +Join condition: None + +(68) Project [codegen id : 17] +Output [4]: [ss_quantity#96 AS agg1#106, ss_list_price#97 AS agg2#107, ss_coupon_amt#99 AS agg3#108, ss_sales_price#98 AS agg4#109] +Input [6]: [ss_item_sk#93, ss_quantity#96, ss_list_price#97, ss_sales_price#98, ss_coupon_amt#99, i_item_sk#105] + +(69) HashAggregate [codegen id : 17] +Input [4]: [agg1#106, agg2#107, agg3#108, agg4#109] +Keys: [] +Functions [4]: [partial_avg(agg1#106), partial_avg(UnscaledValue(agg2#107)), partial_avg(UnscaledValue(agg3#108)), partial_avg(UnscaledValue(agg4#109))] +Aggregate Attributes [8]: [sum#110, count#111, sum#112, count#113, sum#114, count#115, sum#116, count#117] +Results [8]: [sum#118, count#119, sum#120, count#121, sum#122, count#123, sum#124, count#125] + +(70) Exchange +Input [8]: [sum#118, count#119, sum#120, count#121, sum#122, count#123, sum#124, count#125] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=8] + +(71) HashAggregate [codegen id : 18] +Input [8]: [sum#118, count#119, sum#120, count#121, sum#122, count#123, sum#124, count#125] +Keys: [] +Functions [4]: [avg(agg1#106), avg(UnscaledValue(agg2#107)), avg(UnscaledValue(agg3#108)), avg(UnscaledValue(agg4#109))] +Aggregate Attributes [4]: [avg(agg1#106)#126, avg(UnscaledValue(agg2#107))#127, avg(UnscaledValue(agg3#108))#128, avg(UnscaledValue(agg4#109))#129] +Results [7]: [null AS i_item_id#130, null AS s_state#131, 1 AS g_state#132, avg(agg1#106)#126 AS agg1#133, cast((avg(UnscaledValue(agg2#107))#127 / 100.0) as decimal(11,6)) AS agg2#134, cast((avg(UnscaledValue(agg3#108))#128 / 100.0) as decimal(11,6)) AS agg3#135, cast((avg(UnscaledValue(agg4#109))#129 / 100.0) as decimal(11,6)) AS agg4#136] + +(72) Union + +(73) TakeOrderedAndProject +Input [7]: [i_item_id#18, s_state#16, g_state#43, agg1#44, agg2#45, agg3#46, agg4#47] +Arguments: 100, [i_item_id#18 ASC NULLS FIRST, s_state#16 ASC NULLS FIRST], [i_item_id#18, s_state#16, g_state#43, agg1#44, agg2#45, agg3#46, agg4#47] + +===== Subqueries ===== + +Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#8 IN dynamicpruning#9 +BroadcastExchange (78) ++- * ColumnarToRow (77) + +- CometProject (76) + +- CometFilter (75) + +- CometScan parquet spark_catalog.default.date_dim (74) + + +(74) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#14, d_year#137] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_year), EqualTo(d_year,1998), IsNotNull(d_date_sk)] +ReadSchema: struct<d_date_sk:int,d_year:int> + +(75) CometFilter +Input [2]: [d_date_sk#14, d_year#137] +Condition : ((isnotnull(d_year#137) AND (d_year#137 = 1998)) AND isnotnull(d_date_sk#14)) + +(76) CometProject +Input [2]: [d_date_sk#14, d_year#137] +Arguments: [d_date_sk#14], [d_date_sk#14] + +(77) ColumnarToRow [codegen id : 1] +Input [1]: [d_date_sk#14] + +(78) BroadcastExchange +Input [1]: [d_date_sk#14] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=9] + +Subquery:2 Hosting operator id = 29 Hosting Expression = ss_sold_date_sk#55 IN dynamicpruning#9 + +Subquery:3 Hosting operator id = 51 Hosting Expression = ss_sold_date_sk#100 IN dynamicpruning#9 + + diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q27a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q27a/simplified.txt new file mode 100644 index 000000000..32f003798 --- /dev/null +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q27a/simplified.txt @@ -0,0 +1,117 @@ +TakeOrderedAndProject [i_item_id,s_state,g_state,agg1,agg2,agg3,agg4] + Union + WholeStageCodegen (6) + HashAggregate [i_item_id,s_state,sum,count,sum,count,sum,count,sum,count] [avg(agg1),avg(UnscaledValue(agg2)),avg(UnscaledValue(agg3)),avg(UnscaledValue(agg4)),g_state,agg1,agg2,agg3,agg4,sum,count,sum,count,sum,count,sum,count] + InputAdapter + Exchange [i_item_id,s_state] #1 + WholeStageCodegen (5) + HashAggregate [i_item_id,s_state,agg1,agg2,agg3,agg4] [sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] + Project [i_item_id,s_state,ss_quantity,ss_list_price,ss_coupon_amt,ss_sales_price] + BroadcastHashJoin [ss_item_sk,i_item_sk] + Project [ss_item_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,s_state] + BroadcastHashJoin [ss_store_sk,s_store_sk] + Project [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk] + BroadcastHashJoin [ss_cdemo_sk,cd_demo_sk] + ColumnarToRow + InputAdapter + CometFilter [ss_cdemo_sk,ss_store_sk,ss_item_sk] + CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_cdemo_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #2 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_year,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + InputAdapter + BroadcastExchange #3 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [cd_demo_sk] + CometFilter [cd_gender,cd_marital_status,cd_education_status,cd_demo_sk] + CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_education_status] + InputAdapter + ReusedExchange [d_date_sk] #2 + InputAdapter + BroadcastExchange #4 + WholeStageCodegen (3) + ColumnarToRow + InputAdapter + CometFilter [s_state,s_store_sk] + CometScan parquet spark_catalog.default.store [s_store_sk,s_state] + InputAdapter + BroadcastExchange #5 + WholeStageCodegen (4) + ColumnarToRow + InputAdapter + CometFilter [i_item_sk] + CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id] + WholeStageCodegen (12) + HashAggregate [i_item_id,sum,count,sum,count,sum,count,sum,count] [avg(agg1),avg(UnscaledValue(agg2)),avg(UnscaledValue(agg3)),avg(UnscaledValue(agg4)),s_state,g_state,agg1,agg2,agg3,agg4,sum,count,sum,count,sum,count,sum,count] + InputAdapter + Exchange [i_item_id] #6 + WholeStageCodegen (11) + HashAggregate [i_item_id,agg1,agg2,agg3,agg4] [sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] + Project [i_item_id,ss_quantity,ss_list_price,ss_coupon_amt,ss_sales_price] + BroadcastHashJoin [ss_item_sk,i_item_sk] + Project [ss_item_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt] + BroadcastHashJoin [ss_store_sk,s_store_sk] + Project [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk] + BroadcastHashJoin [ss_cdemo_sk,cd_demo_sk] + ColumnarToRow + InputAdapter + CometFilter [ss_cdemo_sk,ss_store_sk,ss_item_sk] + CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_cdemo_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + InputAdapter + ReusedExchange [cd_demo_sk] #3 + InputAdapter + ReusedExchange [d_date_sk] #2 + InputAdapter + BroadcastExchange #7 + WholeStageCodegen (9) + ColumnarToRow + InputAdapter + CometProject [s_store_sk] + CometFilter [s_state,s_store_sk] + CometScan parquet spark_catalog.default.store [s_store_sk,s_state] + InputAdapter + ReusedExchange [i_item_sk,i_item_id] #5 + WholeStageCodegen (18) + HashAggregate [sum,count,sum,count,sum,count,sum,count] [avg(agg1),avg(UnscaledValue(agg2)),avg(UnscaledValue(agg3)),avg(UnscaledValue(agg4)),i_item_id,s_state,g_state,agg1,agg2,agg3,agg4,sum,count,sum,count,sum,count,sum,count] + InputAdapter + Exchange #8 + WholeStageCodegen (17) + HashAggregate [agg1,agg2,agg3,agg4] [sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] + Project [ss_quantity,ss_list_price,ss_coupon_amt,ss_sales_price] + BroadcastHashJoin [ss_item_sk,i_item_sk] + Project [ss_item_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt] + BroadcastHashJoin [ss_store_sk,s_store_sk] + Project [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [ss_item_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk] + BroadcastHashJoin [ss_cdemo_sk,cd_demo_sk] + ColumnarToRow + InputAdapter + CometFilter [ss_cdemo_sk,ss_store_sk,ss_item_sk] + CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_cdemo_sk,ss_store_sk,ss_quantity,ss_list_price,ss_sales_price,ss_coupon_amt,ss_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + InputAdapter + ReusedExchange [cd_demo_sk] #3 + InputAdapter + ReusedExchange [d_date_sk] #2 + InputAdapter + ReusedExchange [s_store_sk] #7 + InputAdapter + BroadcastExchange #9 + WholeStageCodegen (16) + ColumnarToRow + InputAdapter + CometFilter [i_item_sk] + CometScan parquet spark_catalog.default.item [i_item_sk] 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 new file mode 100644 index 000000000..387fa47f9 --- /dev/null +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q34/explain.txt @@ -0,0 +1,218 @@ +== Physical Plan == +* Sort (32) ++- Exchange (31) + +- * Project (30) + +- * BroadcastHashJoin Inner BuildRight (29) + :- * Filter (24) + : +- * HashAggregate (23) + : +- Exchange (22) + : +- * HashAggregate (21) + : +- * Project (20) + : +- * BroadcastHashJoin Inner BuildRight (19) + : :- * Project (13) + : : +- * BroadcastHashJoin Inner BuildRight (12) + : : :- * Project (6) + : : : +- * BroadcastHashJoin Inner BuildRight (5) + : : : :- * ColumnarToRow (3) + : : : : +- CometFilter (2) + : : : : +- CometScan parquet spark_catalog.default.store_sales (1) + : : : +- ReusedExchange (4) + : : +- BroadcastExchange (11) + : : +- * ColumnarToRow (10) + : : +- CometProject (9) + : : +- CometFilter (8) + : : +- CometScan parquet spark_catalog.default.store (7) + : +- BroadcastExchange (18) + : +- * ColumnarToRow (17) + : +- CometProject (16) + : +- CometFilter (15) + : +- CometScan parquet spark_catalog.default.household_demographics (14) + +- BroadcastExchange (28) + +- * ColumnarToRow (27) + +- CometFilter (26) + +- CometScan parquet spark_catalog.default.customer (25) + + +(1) Scan parquet spark_catalog.default.store_sales +Output [5]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_sold_date_sk#5] +Batched: true +Location: InMemoryFileIndex [] +PartitionFilters: [isnotnull(ss_sold_date_sk#5), dynamicpruningexpression(ss_sold_date_sk#5 IN dynamicpruning#6)] +PushedFilters: [IsNotNull(ss_store_sk), IsNotNull(ss_hdemo_sk), IsNotNull(ss_customer_sk)] +ReadSchema: struct<ss_customer_sk:int,ss_hdemo_sk:int,ss_store_sk:int,ss_ticket_number:int> + +(2) CometFilter +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)) + +(3) ColumnarToRow [codegen id : 4] +Input [5]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_sold_date_sk#5] + +(4) ReusedExchange [Reuses operator id: 37] +Output [1]: [d_date_sk#7] + +(5) BroadcastHashJoin [codegen id : 4] +Left keys [1]: [ss_sold_date_sk#5] +Right keys [1]: [d_date_sk#7] +Join type: Inner +Join condition: None + +(6) Project [codegen id : 4] +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] + +(7) Scan parquet spark_catalog.default.store +Output [2]: [s_store_sk#8, s_county#9] +Batched: true +Location [not included in comparison]/{warehouse_dir}/store] +PushedFilters: [IsNotNull(s_county), EqualTo(s_county,Williamson County), IsNotNull(s_store_sk)] +ReadSchema: struct<s_store_sk:int,s_county:string> + +(8) CometFilter +Input [2]: [s_store_sk#8, s_county#9] +Condition : ((isnotnull(s_county#9) AND (s_county#9 = Williamson County)) AND isnotnull(s_store_sk#8)) + +(9) CometProject +Input [2]: [s_store_sk#8, s_county#9] +Arguments: [s_store_sk#8], [s_store_sk#8] + +(10) ColumnarToRow [codegen id : 2] +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] +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] +Output [3]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_ticket_number#4] +Input [5]: [ss_customer_sk#1, ss_hdemo_sk#2, ss_store_sk#3, ss_ticket_number#4, s_store_sk#8] + +(14) Scan parquet spark_catalog.default.household_demographics +Output [4]: [hd_demo_sk#10, hd_buy_potential#11, hd_dep_count#12, hd_vehicle_count#13] +Batched: true +Location [not included in comparison]/{warehouse_dir}/household_demographics] +PushedFilters: [IsNotNull(hd_vehicle_count), Or(EqualTo(hd_buy_potential,>10000 ),EqualTo(hd_buy_potential,unknown )), GreaterThan(hd_vehicle_count,0), IsNotNull(hd_demo_sk)] +ReadSchema: struct<hd_demo_sk:int,hd_buy_potential:string,hd_dep_count:int,hd_vehicle_count:int> + +(15) CometFilter +Input [4]: [hd_demo_sk#10, hd_buy_potential#11, hd_dep_count#12, hd_vehicle_count#13] +Condition : ((((isnotnull(hd_vehicle_count#13) AND ((hd_buy_potential#11 = >10000 ) OR (hd_buy_potential#11 = unknown ))) AND (hd_vehicle_count#13 > 0)) AND CASE WHEN (hd_vehicle_count#13 > 0) THEN ((cast(hd_dep_count#12 as double) / cast(hd_vehicle_count#13 as double)) > 1.2) END) AND isnotnull(hd_demo_sk#10)) + +(16) CometProject +Input [4]: [hd_demo_sk#10, hd_buy_potential#11, hd_dep_count#12, hd_vehicle_count#13] +Arguments: [hd_demo_sk#10], [hd_demo_sk#10] + +(17) ColumnarToRow [codegen id : 3] +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] +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] +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] +Input [2]: [ss_customer_sk#1, ss_ticket_number#4] +Keys [2]: [ss_ticket_number#4, ss_customer_sk#1] +Functions [1]: [partial_count(1)] +Aggregate Attributes [1]: [count#14] +Results [3]: [ss_ticket_number#4, ss_customer_sk#1, count#15] + +(22) Exchange +Input [3]: [ss_ticket_number#4, ss_customer_sk#1, count#15] +Arguments: hashpartitioning(ss_ticket_number#4, ss_customer_sk#1, 5), ENSURE_REQUIREMENTS, [plan_id=3] + +(23) HashAggregate [codegen id : 6] +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] + +(24) Filter [codegen id : 6] +Input [3]: [ss_ticket_number#4, ss_customer_sk#1, cnt#17] +Condition : ((cnt#17 >= 15) AND (cnt#17 <= 20)) + +(25) Scan parquet spark_catalog.default.customer +Output [5]: [c_customer_sk#18, c_salutation#19, c_first_name#20, c_last_name#21, c_preferred_cust_flag#22] +Batched: true +Location [not included in comparison]/{warehouse_dir}/customer] +PushedFilters: [IsNotNull(c_customer_sk)] +ReadSchema: struct<c_customer_sk:int,c_salutation:string,c_first_name:string,c_last_name:string,c_preferred_cust_flag:string> + +(26) CometFilter +Input [5]: [c_customer_sk#18, c_salutation#19, c_first_name#20, c_last_name#21, c_preferred_cust_flag#22] +Condition : isnotnull(c_customer_sk#18) + +(27) ColumnarToRow [codegen id : 5] +Input [5]: [c_customer_sk#18, c_salutation#19, c_first_name#20, c_last_name#21, c_preferred_cust_flag#22] + +(28) BroadcastExchange +Input [5]: [c_customer_sk#18, c_salutation#19, c_first_name#20, c_last_name#21, c_preferred_cust_flag#22] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=4] + +(29) BroadcastHashJoin [codegen id : 6] +Left keys [1]: [ss_customer_sk#1] +Right keys [1]: [c_customer_sk#18] +Join type: Inner +Join condition: None + +(30) Project [codegen id : 6] +Output [6]: [c_last_name#21, c_first_name#20, c_salutation#19, c_preferred_cust_flag#22, ss_ticket_number#4, cnt#17] +Input [8]: [ss_ticket_number#4, ss_customer_sk#1, cnt#17, c_customer_sk#18, c_salutation#19, c_first_name#20, c_last_name#21, c_preferred_cust_flag#22] + +(31) Exchange +Input [6]: [c_last_name#21, c_first_name#20, c_salutation#19, c_preferred_cust_flag#22, ss_ticket_number#4, cnt#17] +Arguments: rangepartitioning(c_last_name#21 ASC NULLS FIRST, c_first_name#20 ASC NULLS FIRST, c_salutation#19 ASC NULLS FIRST, c_preferred_cust_flag#22 DESC NULLS LAST, ss_ticket_number#4 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, [plan_id=5] + +(32) Sort [codegen id : 7] +Input [6]: [c_last_name#21, c_first_name#20, c_salutation#19, c_preferred_cust_flag#22, ss_ticket_number#4, cnt#17] +Arguments: [c_last_name#21 ASC NULLS FIRST, c_first_name#20 ASC NULLS FIRST, c_salutation#19 ASC NULLS FIRST, c_preferred_cust_flag#22 DESC NULLS LAST, ss_ticket_number#4 ASC NULLS FIRST], true, 0 + +===== Subqueries ===== + +Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#5 IN dynamicpruning#6 +BroadcastExchange (37) ++- * ColumnarToRow (36) + +- CometProject (35) + +- CometFilter (34) + +- CometScan parquet spark_catalog.default.date_dim (33) + + +(33) Scan parquet spark_catalog.default.date_dim +Output [3]: [d_date_sk#7, d_year#23, d_dom#24] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [Or(And(GreaterThanOrEqual(d_dom,1),LessThanOrEqual(d_dom,3)),And(GreaterThanOrEqual(d_dom,25),LessThanOrEqual(d_dom,28))), In(d_year, [1999,2000,2001]), IsNotNull(d_date_sk)] +ReadSchema: struct<d_date_sk:int,d_year:int,d_dom:int> + +(34) CometFilter +Input [3]: [d_date_sk#7, d_year#23, d_dom#24] +Condition : (((((d_dom#24 >= 1) AND (d_dom#24 <= 3)) OR ((d_dom#24 >= 25) AND (d_dom#24 <= 28))) AND d_year#23 IN (1999,2000,2001)) AND isnotnull(d_date_sk#7)) + +(35) CometProject +Input [3]: [d_date_sk#7, d_year#23, d_dom#24] +Arguments: [d_date_sk#7], [d_date_sk#7] + +(36) ColumnarToRow [codegen id : 1] +Input [1]: [d_date_sk#7] + +(37) BroadcastExchange +Input [1]: [d_date_sk#7] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=6] + + diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q34/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q34/simplified.txt new file mode 100644 index 000000000..b473e4892 --- /dev/null +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q34/simplified.txt @@ -0,0 +1,56 @@ +WholeStageCodegen (7) + Sort [c_last_name,c_first_name,c_salutation,c_preferred_cust_flag,ss_ticket_number] + InputAdapter + Exchange [c_last_name,c_first_name,c_salutation,c_preferred_cust_flag,ss_ticket_number] #1 + WholeStageCodegen (6) + Project [c_last_name,c_first_name,c_salutation,c_preferred_cust_flag,ss_ticket_number,cnt] + BroadcastHashJoin [ss_customer_sk,c_customer_sk] + Filter [cnt] + HashAggregate [ss_ticket_number,ss_customer_sk,count] [count(1),cnt,count] + InputAdapter + Exchange [ss_ticket_number,ss_customer_sk] #2 + WholeStageCodegen (4) + HashAggregate [ss_ticket_number,ss_customer_sk] [count,count] + Project [ss_customer_sk,ss_ticket_number] + BroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] + Project [ss_customer_sk,ss_hdemo_sk,ss_ticket_number] + BroadcastHashJoin [ss_store_sk,s_store_sk] + Project [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + ColumnarToRow + InputAdapter + CometFilter [ss_store_sk,ss_hdemo_sk,ss_customer_sk] + CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_hdemo_sk,ss_store_sk,ss_ticket_number,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_dom,d_year,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_dom] + InputAdapter + ReusedExchange [d_date_sk] #3 + InputAdapter + BroadcastExchange #4 + WholeStageCodegen (2) + ColumnarToRow + InputAdapter + CometProject [s_store_sk] + CometFilter [s_county,s_store_sk] + CometScan parquet spark_catalog.default.store [s_store_sk,s_county] + InputAdapter + BroadcastExchange #5 + WholeStageCodegen (3) + ColumnarToRow + InputAdapter + CometProject [hd_demo_sk] + CometFilter [hd_vehicle_count,hd_buy_potential,hd_dep_count,hd_demo_sk] + CometScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_buy_potential,hd_dep_count,hd_vehicle_count] + InputAdapter + BroadcastExchange #6 + WholeStageCodegen (5) + ColumnarToRow + InputAdapter + CometFilter [c_customer_sk] + CometScan parquet spark_catalog.default.customer [c_customer_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q35/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q35/explain.txt new file mode 100644 index 000000000..ce8753277 --- /dev/null +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q35/explain.txt @@ -0,0 +1,281 @@ +== Physical Plan == +TakeOrderedAndProject (42) ++- * HashAggregate (41) + +- Exchange (40) + +- * HashAggregate (39) + +- * Project (38) + +- * BroadcastHashJoin Inner BuildRight (37) + :- * Project (32) + : +- * BroadcastHashJoin Inner BuildRight (31) + : :- * Project (26) + : : +- * Filter (25) + : : +- * BroadcastHashJoin ExistenceJoin(exists#1) BuildRight (24) + : : :- * BroadcastHashJoin ExistenceJoin(exists#2) BuildRight (17) + : : : :- * BroadcastHashJoin LeftSemi BuildRight (10) + : : : : :- * ColumnarToRow (3) + : : : : : +- CometFilter (2) + : : : : : +- CometScan parquet spark_catalog.default.customer (1) + : : : : +- BroadcastExchange (9) + : : : : +- * Project (8) + : : : : +- * BroadcastHashJoin Inner BuildRight (7) + : : : : :- * ColumnarToRow (5) + : : : : : +- CometScan parquet spark_catalog.default.store_sales (4) + : : : : +- ReusedExchange (6) + : : : +- BroadcastExchange (16) + : : : +- * Project (15) + : : : +- * BroadcastHashJoin Inner BuildRight (14) + : : : :- * ColumnarToRow (12) + : : : : +- CometScan parquet spark_catalog.default.web_sales (11) + : : : +- ReusedExchange (13) + : : +- BroadcastExchange (23) + : : +- * Project (22) + : : +- * BroadcastHashJoin Inner BuildRight (21) + : : :- * ColumnarToRow (19) + : : : +- CometScan parquet spark_catalog.default.catalog_sales (18) + : : +- ReusedExchange (20) + : +- BroadcastExchange (30) + : +- * ColumnarToRow (29) + : +- CometFilter (28) + : +- CometScan parquet spark_catalog.default.customer_address (27) + +- BroadcastExchange (36) + +- * ColumnarToRow (35) + +- CometFilter (34) + +- CometScan parquet spark_catalog.default.customer_demographics (33) + + +(1) Scan parquet spark_catalog.default.customer +Output [3]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5] +Batched: true +Location [not included in comparison]/{warehouse_dir}/customer] +PushedFilters: [IsNotNull(c_current_addr_sk), IsNotNull(c_current_cdemo_sk)] +ReadSchema: struct<c_customer_sk:int,c_current_cdemo_sk:int,c_current_addr_sk:int> + +(2) CometFilter +Input [3]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5] +Condition : (isnotnull(c_current_addr_sk#5) AND isnotnull(c_current_cdemo_sk#4)) + +(3) ColumnarToRow [codegen id : 9] +Input [3]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5] + +(4) Scan parquet spark_catalog.default.store_sales +Output [2]: [ss_customer_sk#6, ss_sold_date_sk#7] +Batched: true +Location: InMemoryFileIndex [] +PartitionFilters: [isnotnull(ss_sold_date_sk#7), dynamicpruningexpression(ss_sold_date_sk#7 IN dynamicpruning#8)] +ReadSchema: struct<ss_customer_sk:int> + +(5) ColumnarToRow [codegen id : 2] +Input [2]: [ss_customer_sk#6, ss_sold_date_sk#7] + +(6) ReusedExchange [Reuses operator id: 47] +Output [1]: [d_date_sk#9] + +(7) BroadcastHashJoin [codegen id : 2] +Left keys [1]: [ss_sold_date_sk#7] +Right keys [1]: [d_date_sk#9] +Join type: Inner +Join condition: None + +(8) Project [codegen id : 2] +Output [1]: [ss_customer_sk#6] +Input [3]: [ss_customer_sk#6, ss_sold_date_sk#7, d_date_sk#9] + +(9) BroadcastExchange +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] +Left keys [1]: [c_customer_sk#3] +Right keys [1]: [ss_customer_sk#6] +Join type: LeftSemi +Join condition: None + +(11) Scan parquet spark_catalog.default.web_sales +Output [2]: [ws_bill_customer_sk#10, ws_sold_date_sk#11] +Batched: true +Location: InMemoryFileIndex [] +PartitionFilters: [isnotnull(ws_sold_date_sk#11), dynamicpruningexpression(ws_sold_date_sk#11 IN dynamicpruning#12)] +ReadSchema: struct<ws_bill_customer_sk:int> + +(12) ColumnarToRow [codegen id : 4] +Input [2]: [ws_bill_customer_sk#10, ws_sold_date_sk#11] + +(13) ReusedExchange [Reuses operator id: 47] +Output [1]: [d_date_sk#13] + +(14) BroadcastHashJoin [codegen id : 4] +Left keys [1]: [ws_sold_date_sk#11] +Right keys [1]: [d_date_sk#13] +Join type: Inner +Join condition: None + +(15) Project [codegen id : 4] +Output [1]: [ws_bill_customer_sk#10] +Input [3]: [ws_bill_customer_sk#10, ws_sold_date_sk#11, d_date_sk#13] + +(16) BroadcastExchange +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] +Left keys [1]: [c_customer_sk#3] +Right keys [1]: [ws_bill_customer_sk#10] +Join type: ExistenceJoin(exists#2) +Join condition: None + +(18) Scan parquet spark_catalog.default.catalog_sales +Output [2]: [cs_ship_customer_sk#14, cs_sold_date_sk#15] +Batched: true +Location: InMemoryFileIndex [] +PartitionFilters: [isnotnull(cs_sold_date_sk#15), dynamicpruningexpression(cs_sold_date_sk#15 IN dynamicpruning#16)] +ReadSchema: struct<cs_ship_customer_sk:int> + +(19) ColumnarToRow [codegen id : 6] +Input [2]: [cs_ship_customer_sk#14, cs_sold_date_sk#15] + +(20) ReusedExchange [Reuses operator id: 47] +Output [1]: [d_date_sk#17] + +(21) BroadcastHashJoin [codegen id : 6] +Left keys [1]: [cs_sold_date_sk#15] +Right keys [1]: [d_date_sk#17] +Join type: Inner +Join condition: None + +(22) Project [codegen id : 6] +Output [1]: [cs_ship_customer_sk#14] +Input [3]: [cs_ship_customer_sk#14, cs_sold_date_sk#15, d_date_sk#17] + +(23) BroadcastExchange +Input [1]: [cs_ship_customer_sk#14] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] + +(24) BroadcastHashJoin [codegen id : 9] +Left keys [1]: [c_customer_sk#3] +Right keys [1]: [cs_ship_customer_sk#14] +Join type: ExistenceJoin(exists#1) +Join condition: None + +(25) Filter [codegen id : 9] +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] +Output [2]: [c_current_cdemo_sk#4, c_current_addr_sk#5] +Input [5]: [c_customer_sk#3, c_current_cdemo_sk#4, c_current_addr_sk#5, exists#2, exists#1] + +(27) Scan parquet spark_catalog.default.customer_address +Output [2]: [ca_address_sk#18, ca_state#19] +Batched: true +Location [not included in comparison]/{warehouse_dir}/customer_address] +PushedFilters: [IsNotNull(ca_address_sk)] +ReadSchema: struct<ca_address_sk:int,ca_state:string> + +(28) CometFilter +Input [2]: [ca_address_sk#18, ca_state#19] +Condition : isnotnull(ca_address_sk#18) + +(29) ColumnarToRow [codegen id : 7] +Input [2]: [ca_address_sk#18, ca_state#19] + +(30) BroadcastExchange +Input [2]: [ca_address_sk#18, ca_state#19] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=4] + +(31) BroadcastHashJoin [codegen id : 9] +Left keys [1]: [c_current_addr_sk#5] +Right keys [1]: [ca_address_sk#18] +Join type: Inner +Join condition: None + +(32) Project [codegen id : 9] +Output [2]: [c_current_cdemo_sk#4, ca_state#19] +Input [4]: [c_current_cdemo_sk#4, c_current_addr_sk#5, ca_address_sk#18, ca_state#19] + +(33) Scan parquet spark_catalog.default.customer_demographics +Output [6]: [cd_demo_sk#20, cd_gender#21, cd_marital_status#22, cd_dep_count#23, cd_dep_employed_count#24, cd_dep_college_count#25] +Batched: true +Location [not included in comparison]/{warehouse_dir}/customer_demographics] +PushedFilters: [IsNotNull(cd_demo_sk)] +ReadSchema: struct<cd_demo_sk:int,cd_gender:string,cd_marital_status:string,cd_dep_count:int,cd_dep_employed_count:int,cd_dep_college_count:int> + +(34) CometFilter +Input [6]: [cd_demo_sk#20, cd_gender#21, cd_marital_status#22, cd_dep_count#23, cd_dep_employed_count#24, cd_dep_college_count#25] +Condition : isnotnull(cd_demo_sk#20) + +(35) ColumnarToRow [codegen id : 8] +Input [6]: [cd_demo_sk#20, cd_gender#21, cd_marital_status#22, cd_dep_count#23, cd_dep_employed_count#24, cd_dep_college_count#25] + +(36) BroadcastExchange +Input [6]: [cd_demo_sk#20, cd_gender#21, cd_marital_status#22, cd_dep_count#23, cd_dep_employed_count#24, cd_dep_college_count#25] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=5] + +(37) BroadcastHashJoin [codegen id : 9] +Left keys [1]: [c_current_cdemo_sk#4] +Right keys [1]: [cd_demo_sk#20] +Join type: Inner +Join condition: None + +(38) Project [codegen id : 9] +Output [6]: [ca_state#19, cd_gender#21, cd_marital_status#22, cd_dep_count#23, cd_dep_employed_count#24, cd_dep_college_count#25] +Input [8]: [c_current_cdemo_sk#4, ca_state#19, cd_demo_sk#20, cd_gender#21, cd_marital_status#22, cd_dep_count#23, cd_dep_employed_count#24, cd_dep_college_count#25] + +(39) HashAggregate [codegen id : 9] +Input [6]: [ca_state#19, cd_gender#21, cd_marital_status#22, cd_dep_count#23, cd_dep_employed_count#24, cd_dep_college_count#25] +Keys [6]: [ca_state#19, cd_gender#21, cd_marital_status#22, cd_dep_count#23, cd_dep_employed_count#24, cd_dep_college_count#25] +Functions [10]: [partial_count(1), partial_avg(cd_dep_count#23), partial_max(cd_dep_count#23), partial_sum(cd_dep_count#23), partial_avg(cd_dep_employed_count#24), partial_max(cd_dep_employed_count#24), partial_sum(cd_dep_employed_count#24), partial_avg(cd_dep_college_count#25), partial_max(cd_dep_college_count#25), partial_sum(cd_dep_college_count#25)] +Aggregate Attributes [13]: [count#26, sum#27, count#28, max#29, sum#30, sum#31, count#32, max#33, sum#34, sum#35, count#36, max#37, sum#38] +Results [19]: [ca_state#19, cd_gender#21, cd_marital_status#22, cd_dep_count#23, cd_dep_employed_count#24, cd_dep_college_count#25, count#39, sum#40, count#41, max#42, sum#43, sum#44, count#45, max#46, sum#47, sum#48, count#49, max#50, sum#51] + +(40) Exchange +Input [19]: [ca_state#19, cd_gender#21, cd_marital_status#22, cd_dep_count#23, cd_dep_employed_count#24, cd_dep_college_count#25, count#39, sum#40, count#41, max#42, sum#43, sum#44, count#45, max#46, sum#47, sum#48, count#49, max#50, sum#51] +Arguments: hashpartitioning(ca_state#19, cd_gender#21, cd_marital_status#22, cd_dep_count#23, cd_dep_employed_count#24, cd_dep_college_count#25, 5), ENSURE_REQUIREMENTS, [plan_id=6] + +(41) HashAggregate [codegen id : 10] +Input [19]: [ca_state#19, cd_gender#21, cd_marital_status#22, cd_dep_count#23, cd_dep_employed_count#24, cd_dep_college_count#25, count#39, sum#40, count#41, max#42, sum#43, sum#44, count#45, max#46, sum#47, sum#48, count#49, max#50, sum#51] +Keys [6]: [ca_state#19, cd_gender#21, cd_marital_status#22, cd_dep_count#23, cd_dep_employed_count#24, cd_dep_college_count#25] +Functions [10]: [count(1), avg(cd_dep_count#23), max(cd_dep_count#23), sum(cd_dep_count#23), avg(cd_dep_employed_count#24), max(cd_dep_employed_count#24), sum(cd_dep_employed_count#24), avg(cd_dep_college_count#25), max(cd_dep_college_count#25), sum(cd_dep_college_count#25)] +Aggregate Attributes [10]: [count(1)#52, avg(cd_dep_count#23)#53, max(cd_dep_count#23)#54, sum(cd_dep_count#23)#55, avg(cd_dep_employed_count#24)#56, max(cd_dep_employed_count#24)#57, sum(cd_dep_employed_count#24)#58, avg(cd_dep_college_count#25)#59, max(cd_dep_college_count#25)#60, sum(cd_dep_college_count#25)#61] +Results [18]: [ca_state#19, cd_gender#21, cd_marital_status#22, cd_dep_count#23, count(1)#52 AS cnt1#62, avg(cd_dep_count#23)#53 AS avg(cd_dep_count)#63, max(cd_dep_count#23)#54 AS max(cd_dep_count)#64, sum(cd_dep_count#23)#55 AS sum(cd_dep_count)#65, cd_dep_employed_count#24, count(1)#52 AS cnt2#66, avg(cd_dep_employed_count#24)#56 AS avg(cd_dep_employed_count)#67, max(cd_dep_employed_count#24)#57 AS max(cd_dep_employed_count)#68, sum(cd_dep_employed_count#24)#58 AS sum(cd_dep_employed_count)#69, cd_dep_college_count#25, count(1)#52 AS cnt3#70, avg(cd_dep_college_count#25)#59 AS avg(cd_dep_college_count)#71, max(cd_dep_college_count#25)#60 AS max(cd_dep_college_count)#72, sum(cd_dep_college_count#25)#61 AS sum(cd_dep_college_count)#73] + +(42) TakeOrderedAndProject +Input [18]: [ca_state#19, cd_gender#21, cd_marital_status#22, cd_dep_count#23, cnt1#62, avg(cd_dep_count)#63, max(cd_dep_count)#64, sum(cd_dep_count)#65, cd_dep_employed_count#24, cnt2#66, avg(cd_dep_employed_count)#67, max(cd_dep_employed_count)#68, sum(cd_dep_employed_count)#69, cd_dep_college_count#25, cnt3#70, avg(cd_dep_college_count)#71, max(cd_dep_college_count)#72, sum(cd_dep_college_count)#73] +Arguments: 100, [ca_state#19 ASC NULLS FIRST, cd_gender#21 ASC NULLS FIRST, cd_marital_status#22 ASC NULLS FIRST, cd_dep_count#23 ASC NULLS FIRST, cd_dep_employed_count#24 ASC NULLS FIRST, cd_dep_college_count#25 ASC NULLS FIRST], [ca_state#19, cd_gender#21, cd_marital_status#22, cd_dep_count#23, cnt1#62, avg(cd_dep_count)#63, max(cd_dep_count)#64, sum(cd_dep_count)#65, cd_dep_employed_count#24, cnt2#66, avg(cd_dep_employed_count)#67, max(cd_dep_employed_count)#68, sum(cd_dep_employed_count)#69, cd_dep_college_count#25, cnt3#70, avg(cd_dep_college_count)#71, max(cd_dep_college_count)#72, sum(cd_dep_college_count)#73] + +===== Subqueries ===== + +Subquery:1 Hosting operator id = 4 Hosting Expression = ss_sold_date_sk#7 IN dynamicpruning#8 +BroadcastExchange (47) ++- * ColumnarToRow (46) + +- CometProject (45) + +- CometFilter (44) + +- CometScan parquet spark_catalog.default.date_dim (43) + + +(43) Scan parquet spark_catalog.default.date_dim +Output [3]: [d_date_sk#9, d_year#74, d_qoy#75] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_year), IsNotNull(d_qoy), EqualTo(d_year,2002), LessThan(d_qoy,4), IsNotNull(d_date_sk)] +ReadSchema: struct<d_date_sk:int,d_year:int,d_qoy:int> + +(44) CometFilter +Input [3]: [d_date_sk#9, d_year#74, d_qoy#75] +Condition : ((((isnotnull(d_year#74) AND isnotnull(d_qoy#75)) AND (d_year#74 = 2002)) AND (d_qoy#75 < 4)) AND isnotnull(d_date_sk#9)) + +(45) CometProject +Input [3]: [d_date_sk#9, d_year#74, d_qoy#75] +Arguments: [d_date_sk#9], [d_date_sk#9] + +(46) ColumnarToRow [codegen id : 1] +Input [1]: [d_date_sk#9] + +(47) BroadcastExchange +Input [1]: [d_date_sk#9] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=7] + +Subquery:2 Hosting operator id = 11 Hosting Expression = ws_sold_date_sk#11 IN dynamicpruning#8 + +Subquery:3 Hosting operator id = 18 Hosting Expression = cs_sold_date_sk#15 IN dynamicpruning#8 + + diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q35/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q35/simplified.txt new file mode 100644 index 000000000..dc724ca91 --- /dev/null +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q35/simplified.txt @@ -0,0 +1,74 @@ +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) + HashAggregate [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count,count,sum,count,max,sum,sum,count,max,sum,sum,count,max,sum] [count(1),avg(cd_dep_count),max(cd_dep_count),sum(cd_dep_count),avg(cd_dep_employed_count),max(cd_dep_employed_count),sum(cd_dep_employed_count),avg(cd_dep_college_count),max(cd_dep_college_count),sum(cd_dep_college_count),cnt1,avg(cd_dep_count),max(cd_dep_count),sum(cd_dep_count),cnt2,avg(cd_dep_employed_count),max(cd_dep_employed_count),sum(cd_dep_employed_count),cnt3,avg(cd_dep_college_count),max(cd_dep_college_count),sum(cd_dep_college_count),count,sum,count,max,sum,sum,count,max,sum,sum,count,max,sum] + InputAdapter + Exchange [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] #1 + WholeStageCodegen (9) + HashAggregate [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] [count,sum,count,max,sum,sum,count,max,sum,sum,count,max,sum,count,sum,count,max,sum,sum,count,max,sum,sum,count,max,sum] + Project [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] + BroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] + Project [c_current_cdemo_sk,ca_state] + BroadcastHashJoin [c_current_addr_sk,ca_address_sk] + Project [c_current_cdemo_sk,c_current_addr_sk] + Filter [exists,exists] + BroadcastHashJoin [c_customer_sk,cs_ship_customer_sk] + BroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] + BroadcastHashJoin [c_customer_sk,ss_customer_sk] + ColumnarToRow + InputAdapter + CometFilter [c_current_addr_sk,c_current_cdemo_sk] + CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] + InputAdapter + BroadcastExchange #2 + WholeStageCodegen (2) + Project [ss_customer_sk] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + ColumnarToRow + InputAdapter + CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_year,d_qoy,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] + InputAdapter + ReusedExchange [d_date_sk] #3 + InputAdapter + BroadcastExchange #4 + WholeStageCodegen (4) + Project [ws_bill_customer_sk] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + ColumnarToRow + InputAdapter + CometScan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + InputAdapter + ReusedExchange [d_date_sk] #3 + InputAdapter + BroadcastExchange #5 + WholeStageCodegen (6) + Project [cs_ship_customer_sk] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + ColumnarToRow + InputAdapter + CometScan parquet spark_catalog.default.catalog_sales [cs_ship_customer_sk,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + InputAdapter + ReusedExchange [d_date_sk] #3 + InputAdapter + BroadcastExchange #6 + WholeStageCodegen (7) + ColumnarToRow + InputAdapter + CometFilter [ca_address_sk] + CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] + InputAdapter + BroadcastExchange #7 + WholeStageCodegen (8) + ColumnarToRow + InputAdapter + CometFilter [cd_demo_sk] + CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] 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 new file mode 100644 index 000000000..648b19933 --- /dev/null +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q35a/explain.txt @@ -0,0 +1,267 @@ +== Physical Plan == +TakeOrderedAndProject (40) ++- * HashAggregate (39) + +- Exchange (38) + +- * HashAggregate (37) + +- * Project (36) + +- * BroadcastHashJoin Inner BuildRight (35) + :- * Project (30) + : +- * BroadcastHashJoin Inner BuildRight (29) + : :- * Project (24) + : : +- * BroadcastHashJoin LeftSemi BuildRight (23) + : : :- * BroadcastHashJoin LeftSemi BuildRight (10) + : : : :- * ColumnarToRow (3) + : : : : +- CometFilter (2) + : : : : +- CometScan parquet spark_catalog.default.customer (1) + : : : +- BroadcastExchange (9) + : : : +- * Project (8) + : : : +- * BroadcastHashJoin Inner BuildRight (7) + : : : :- * ColumnarToRow (5) + : : : : +- CometScan parquet spark_catalog.default.store_sales (4) + : : : +- ReusedExchange (6) + : : +- BroadcastExchange (22) + : : +- Union (21) + : : :- * Project (15) + : : : +- * BroadcastHashJoin Inner BuildRight (14) + : : : :- * ColumnarToRow (12) + : : : : +- CometScan parquet spark_catalog.default.web_sales (11) + : : : +- ReusedExchange (13) + : : +- * Project (20) + : : +- * BroadcastHashJoin Inner BuildRight (19) + : : :- * ColumnarToRow (17) + : : : +- CometScan parquet spark_catalog.default.catalog_sales (16) + : : +- ReusedExchange (18) + : +- BroadcastExchange (28) + : +- * ColumnarToRow (27) + : +- CometFilter (26) + : +- CometScan parquet spark_catalog.default.customer_address (25) + +- BroadcastExchange (34) + +- * ColumnarToRow (33) + +- CometFilter (32) + +- CometScan parquet spark_catalog.default.customer_demographics (31) + + +(1) Scan parquet spark_catalog.default.customer +Output [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] +Batched: true +Location [not included in comparison]/{warehouse_dir}/customer] +PushedFilters: [IsNotNull(c_current_addr_sk), IsNotNull(c_current_cdemo_sk)] +ReadSchema: struct<c_customer_sk:int,c_current_cdemo_sk:int,c_current_addr_sk:int> + +(2) CometFilter +Input [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] +Condition : (isnotnull(c_current_addr_sk#3) AND isnotnull(c_current_cdemo_sk#2)) + +(3) ColumnarToRow [codegen id : 9] +Input [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] + +(4) Scan parquet spark_catalog.default.store_sales +Output [2]: [ss_customer_sk#4, ss_sold_date_sk#5] +Batched: true +Location: InMemoryFileIndex [] +PartitionFilters: [isnotnull(ss_sold_date_sk#5), dynamicpruningexpression(ss_sold_date_sk#5 IN dynamicpruning#6)] +ReadSchema: struct<ss_customer_sk:int> + +(5) ColumnarToRow [codegen id : 2] +Input [2]: [ss_customer_sk#4, ss_sold_date_sk#5] + +(6) ReusedExchange [Reuses operator id: 45] +Output [1]: [d_date_sk#7] + +(7) BroadcastHashJoin [codegen id : 2] +Left keys [1]: [ss_sold_date_sk#5] +Right keys [1]: [d_date_sk#7] +Join type: Inner +Join condition: None + +(8) Project [codegen id : 2] +Output [1]: [ss_customer_sk#4] +Input [3]: [ss_customer_sk#4, ss_sold_date_sk#5, d_date_sk#7] + +(9) BroadcastExchange +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] +Left keys [1]: [c_customer_sk#1] +Right keys [1]: [ss_customer_sk#4] +Join type: LeftSemi +Join condition: None + +(11) Scan parquet spark_catalog.default.web_sales +Output [2]: [ws_bill_customer_sk#8, ws_sold_date_sk#9] +Batched: true +Location: InMemoryFileIndex [] +PartitionFilters: [isnotnull(ws_sold_date_sk#9), dynamicpruningexpression(ws_sold_date_sk#9 IN dynamicpruning#10)] +ReadSchema: struct<ws_bill_customer_sk:int> + +(12) ColumnarToRow [codegen id : 4] +Input [2]: [ws_bill_customer_sk#8, ws_sold_date_sk#9] + +(13) ReusedExchange [Reuses operator id: 45] +Output [1]: [d_date_sk#11] + +(14) BroadcastHashJoin [codegen id : 4] +Left keys [1]: [ws_sold_date_sk#9] +Right keys [1]: [d_date_sk#11] +Join type: Inner +Join condition: None + +(15) Project [codegen id : 4] +Output [1]: [ws_bill_customer_sk#8 AS customsk#12] +Input [3]: [ws_bill_customer_sk#8, ws_sold_date_sk#9, d_date_sk#11] + +(16) Scan parquet spark_catalog.default.catalog_sales +Output [2]: [cs_ship_customer_sk#13, cs_sold_date_sk#14] +Batched: true +Location: InMemoryFileIndex [] +PartitionFilters: [isnotnull(cs_sold_date_sk#14), dynamicpruningexpression(cs_sold_date_sk#14 IN dynamicpruning#15)] +ReadSchema: struct<cs_ship_customer_sk:int> + +(17) ColumnarToRow [codegen id : 6] +Input [2]: [cs_ship_customer_sk#13, cs_sold_date_sk#14] + +(18) ReusedExchange [Reuses operator id: 45] +Output [1]: [d_date_sk#16] + +(19) BroadcastHashJoin [codegen id : 6] +Left keys [1]: [cs_sold_date_sk#14] +Right keys [1]: [d_date_sk#16] +Join type: Inner +Join condition: None + +(20) Project [codegen id : 6] +Output [1]: [cs_ship_customer_sk#13 AS customsk#17] +Input [3]: [cs_ship_customer_sk#13, cs_sold_date_sk#14, d_date_sk#16] + +(21) Union + +(22) BroadcastExchange +Input [1]: [customsk#12] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] + +(23) BroadcastHashJoin [codegen id : 9] +Left keys [1]: [c_customer_sk#1] +Right keys [1]: [customsk#12] +Join type: LeftSemi +Join condition: None + +(24) Project [codegen id : 9] +Output [2]: [c_current_cdemo_sk#2, c_current_addr_sk#3] +Input [3]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_addr_sk#3] + +(25) Scan parquet spark_catalog.default.customer_address +Output [2]: [ca_address_sk#18, ca_state#19] +Batched: true +Location [not included in comparison]/{warehouse_dir}/customer_address] +PushedFilters: [IsNotNull(ca_address_sk)] +ReadSchema: struct<ca_address_sk:int,ca_state:string> + +(26) CometFilter +Input [2]: [ca_address_sk#18, ca_state#19] +Condition : isnotnull(ca_address_sk#18) + +(27) ColumnarToRow [codegen id : 7] +Input [2]: [ca_address_sk#18, ca_state#19] + +(28) BroadcastExchange +Input [2]: [ca_address_sk#18, ca_state#19] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=3] + +(29) BroadcastHashJoin [codegen id : 9] +Left keys [1]: [c_current_addr_sk#3] +Right keys [1]: [ca_address_sk#18] +Join type: Inner +Join condition: None + +(30) Project [codegen id : 9] +Output [2]: [c_current_cdemo_sk#2, ca_state#19] +Input [4]: [c_current_cdemo_sk#2, c_current_addr_sk#3, ca_address_sk#18, ca_state#19] + +(31) Scan parquet spark_catalog.default.customer_demographics +Output [6]: [cd_demo_sk#20, cd_gender#21, cd_marital_status#22, cd_dep_count#23, cd_dep_employed_count#24, cd_dep_college_count#25] +Batched: true +Location [not included in comparison]/{warehouse_dir}/customer_demographics] +PushedFilters: [IsNotNull(cd_demo_sk)] +ReadSchema: struct<cd_demo_sk:int,cd_gender:string,cd_marital_status:string,cd_dep_count:int,cd_dep_employed_count:int,cd_dep_college_count:int> + +(32) CometFilter +Input [6]: [cd_demo_sk#20, cd_gender#21, cd_marital_status#22, cd_dep_count#23, cd_dep_employed_count#24, cd_dep_college_count#25] +Condition : isnotnull(cd_demo_sk#20) + +(33) ColumnarToRow [codegen id : 8] +Input [6]: [cd_demo_sk#20, cd_gender#21, cd_marital_status#22, cd_dep_count#23, cd_dep_employed_count#24, cd_dep_college_count#25] + +(34) BroadcastExchange +Input [6]: [cd_demo_sk#20, cd_gender#21, cd_marital_status#22, cd_dep_count#23, cd_dep_employed_count#24, cd_dep_college_count#25] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=4] + +(35) BroadcastHashJoin [codegen id : 9] +Left keys [1]: [c_current_cdemo_sk#2] +Right keys [1]: [cd_demo_sk#20] +Join type: Inner +Join condition: None + +(36) Project [codegen id : 9] +Output [6]: [ca_state#19, cd_gender#21, cd_marital_status#22, cd_dep_count#23, cd_dep_employed_count#24, cd_dep_college_count#25] +Input [8]: [c_current_cdemo_sk#2, ca_state#19, cd_demo_sk#20, cd_gender#21, cd_marital_status#22, cd_dep_count#23, cd_dep_employed_count#24, cd_dep_college_count#25] + +(37) HashAggregate [codegen id : 9] +Input [6]: [ca_state#19, cd_gender#21, cd_marital_status#22, cd_dep_count#23, cd_dep_employed_count#24, cd_dep_college_count#25] +Keys [6]: [ca_state#19, cd_gender#21, cd_marital_status#22, cd_dep_count#23, cd_dep_employed_count#24, cd_dep_college_count#25] +Functions [10]: [partial_count(1), partial_avg(cd_dep_count#23), partial_max(cd_dep_count#23), partial_sum(cd_dep_count#23), partial_avg(cd_dep_employed_count#24), partial_max(cd_dep_employed_count#24), partial_sum(cd_dep_employed_count#24), partial_avg(cd_dep_college_count#25), partial_max(cd_dep_college_count#25), partial_sum(cd_dep_college_count#25)] +Aggregate Attributes [13]: [count#26, sum#27, count#28, max#29, sum#30, sum#31, count#32, max#33, sum#34, sum#35, count#36, max#37, sum#38] +Results [19]: [ca_state#19, cd_gender#21, cd_marital_status#22, cd_dep_count#23, cd_dep_employed_count#24, cd_dep_college_count#25, count#39, sum#40, count#41, max#42, sum#43, sum#44, count#45, max#46, sum#47, sum#48, count#49, max#50, sum#51] + +(38) Exchange +Input [19]: [ca_state#19, cd_gender#21, cd_marital_status#22, cd_dep_count#23, cd_dep_employed_count#24, cd_dep_college_count#25, count#39, sum#40, count#41, max#42, sum#43, sum#44, count#45, max#46, sum#47, sum#48, count#49, max#50, sum#51] +Arguments: hashpartitioning(ca_state#19, cd_gender#21, cd_marital_status#22, cd_dep_count#23, cd_dep_employed_count#24, cd_dep_college_count#25, 5), ENSURE_REQUIREMENTS, [plan_id=5] + +(39) HashAggregate [codegen id : 10] +Input [19]: [ca_state#19, cd_gender#21, cd_marital_status#22, cd_dep_count#23, cd_dep_employed_count#24, cd_dep_college_count#25, count#39, sum#40, count#41, max#42, sum#43, sum#44, count#45, max#46, sum#47, sum#48, count#49, max#50, sum#51] +Keys [6]: [ca_state#19, cd_gender#21, cd_marital_status#22, cd_dep_count#23, cd_dep_employed_count#24, cd_dep_college_count#25] +Functions [10]: [count(1), avg(cd_dep_count#23), max(cd_dep_count#23), sum(cd_dep_count#23), avg(cd_dep_employed_count#24), max(cd_dep_employed_count#24), sum(cd_dep_employed_count#24), avg(cd_dep_college_count#25), max(cd_dep_college_count#25), sum(cd_dep_college_count#25)] +Aggregate Attributes [10]: [count(1)#52, avg(cd_dep_count#23)#53, max(cd_dep_count#23)#54, sum(cd_dep_count#23)#55, avg(cd_dep_employed_count#24)#56, max(cd_dep_employed_count#24)#57, sum(cd_dep_employed_count#24)#58, avg(cd_dep_college_count#25)#59, max(cd_dep_college_count#25)#60, sum(cd_dep_college_count#25)#61] +Results [18]: [ca_state#19, cd_gender#21, cd_marital_status#22, cd_dep_count#23, count(1)#52 AS cnt1#62, avg(cd_dep_count#23)#53 AS avg(cd_dep_count)#63, max(cd_dep_count#23)#54 AS max(cd_dep_count)#64, sum(cd_dep_count#23)#55 AS sum(cd_dep_count)#65, cd_dep_employed_count#24, count(1)#52 AS cnt2#66, avg(cd_dep_employed_count#24)#56 AS avg(cd_dep_employed_count)#67, max(cd_dep_employed_count#24)#57 AS max(cd_dep_employed_count)#68, sum(cd_dep_employed_count#24)#58 AS sum(cd_dep_employed_count)#69, cd_dep_college_count#25, count(1)#52 AS cnt3#70, avg(cd_dep_college_count#25)#59 AS avg(cd_dep_college_count)#71, max(cd_dep_college_count#25)#60 AS max(cd_dep_college_count)#72, sum(cd_dep_college_count#25)#61 AS sum(cd_dep_college_count)#73] + +(40) TakeOrderedAndProject +Input [18]: [ca_state#19, cd_gender#21, cd_marital_status#22, cd_dep_count#23, cnt1#62, avg(cd_dep_count)#63, max(cd_dep_count)#64, sum(cd_dep_count)#65, cd_dep_employed_count#24, cnt2#66, avg(cd_dep_employed_count)#67, max(cd_dep_employed_count)#68, sum(cd_dep_employed_count)#69, cd_dep_college_count#25, cnt3#70, avg(cd_dep_college_count)#71, max(cd_dep_college_count)#72, sum(cd_dep_college_count)#73] +Arguments: 100, [ca_state#19 ASC NULLS FIRST, cd_gender#21 ASC NULLS FIRST, cd_marital_status#22 ASC NULLS FIRST, cd_dep_count#23 ASC NULLS FIRST, cd_dep_employed_count#24 ASC NULLS FIRST, cd_dep_college_count#25 ASC NULLS FIRST], [ca_state#19, cd_gender#21, cd_marital_status#22, cd_dep_count#23, cnt1#62, avg(cd_dep_count)#63, max(cd_dep_count)#64, sum(cd_dep_count)#65, cd_dep_employed_count#24, cnt2#66, avg(cd_dep_employed_count)#67, max(cd_dep_employed_count)#68, sum(cd_dep_employed_count)#69, cd_dep_college_count#25, cnt3#70, avg(cd_dep_college_count)#71, max(cd_dep_college_count)#72, sum(cd_dep_college_count)#73] + +===== Subqueries ===== + +Subquery:1 Hosting operator id = 4 Hosting Expression = ss_sold_date_sk#5 IN dynamicpruning#6 +BroadcastExchange (45) ++- * ColumnarToRow (44) + +- CometProject (43) + +- CometFilter (42) + +- CometScan parquet spark_catalog.default.date_dim (41) + + +(41) Scan parquet spark_catalog.default.date_dim +Output [3]: [d_date_sk#7, d_year#74, d_qoy#75] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_year), IsNotNull(d_qoy), EqualTo(d_year,1999), LessThan(d_qoy,4), IsNotNull(d_date_sk)] +ReadSchema: struct<d_date_sk:int,d_year:int,d_qoy:int> + +(42) CometFilter +Input [3]: [d_date_sk#7, d_year#74, d_qoy#75] +Condition : ((((isnotnull(d_year#74) AND isnotnull(d_qoy#75)) AND (d_year#74 = 1999)) AND (d_qoy#75 < 4)) AND isnotnull(d_date_sk#7)) + +(43) CometProject +Input [3]: [d_date_sk#7, d_year#74, d_qoy#75] +Arguments: [d_date_sk#7], [d_date_sk#7] + +(44) ColumnarToRow [codegen id : 1] +Input [1]: [d_date_sk#7] + +(45) BroadcastExchange +Input [1]: [d_date_sk#7] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=6] + +Subquery:2 Hosting operator id = 11 Hosting Expression = ws_sold_date_sk#9 IN dynamicpruning#6 + +Subquery:3 Hosting operator id = 16 Hosting Expression = cs_sold_date_sk#14 IN dynamicpruning#6 + + diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q35a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q35a/simplified.txt new file mode 100644 index 000000000..e5cb94055 --- /dev/null +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q35a/simplified.txt @@ -0,0 +1,71 @@ +TakeOrderedAndProject [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count,cnt1,avg(cd_dep_count),max(cd_dep_count),sum(cd_dep_count),cnt2,avg(cd_dep_employed_count),max(cd_dep_employed_count),sum(cd_dep_employed_count),cnt3,avg(cd_dep_college_count),max(cd_dep_college_count),sum(cd_dep_college_count)] + WholeStageCodegen (10) + HashAggregate [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count,count,sum,count,max,sum,sum,count,max,sum,sum,count,max,sum] [count(1),avg(cd_dep_count),max(cd_dep_count),sum(cd_dep_count),avg(cd_dep_employed_count),max(cd_dep_employed_count),sum(cd_dep_employed_count),avg(cd_dep_college_count),max(cd_dep_college_count),sum(cd_dep_college_count),cnt1,avg(cd_dep_count),max(cd_dep_count),sum(cd_dep_count),cnt2,avg(cd_dep_employed_count),max(cd_dep_employed_count),sum(cd_dep_employed_count),cnt3,avg(cd_dep_college_count),max(cd_dep_college_count),sum(cd_dep_college_count),count,sum,count,max,sum,sum,count,max,sum,sum,count,max,sum] + InputAdapter + Exchange [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] #1 + WholeStageCodegen (9) + HashAggregate [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] [count,sum,count,max,sum,sum,count,max,sum,sum,count,max,sum,count,sum,count,max,sum,sum,count,max,sum,sum,count,max,sum] + Project [ca_state,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] + BroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] + Project [c_current_cdemo_sk,ca_state] + BroadcastHashJoin [c_current_addr_sk,ca_address_sk] + Project [c_current_cdemo_sk,c_current_addr_sk] + BroadcastHashJoin [c_customer_sk,customsk] + BroadcastHashJoin [c_customer_sk,ss_customer_sk] + ColumnarToRow + InputAdapter + CometFilter [c_current_addr_sk,c_current_cdemo_sk] + CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] + InputAdapter + BroadcastExchange #2 + WholeStageCodegen (2) + Project [ss_customer_sk] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + ColumnarToRow + InputAdapter + CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_year,d_qoy,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_qoy] + InputAdapter + ReusedExchange [d_date_sk] #3 + InputAdapter + BroadcastExchange #4 + Union + WholeStageCodegen (4) + Project [ws_bill_customer_sk] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + ColumnarToRow + InputAdapter + CometScan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + InputAdapter + ReusedExchange [d_date_sk] #3 + WholeStageCodegen (6) + Project [cs_ship_customer_sk] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + ColumnarToRow + InputAdapter + CometScan parquet spark_catalog.default.catalog_sales [cs_ship_customer_sk,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + InputAdapter + ReusedExchange [d_date_sk] #3 + InputAdapter + BroadcastExchange #5 + WholeStageCodegen (7) + ColumnarToRow + InputAdapter + CometFilter [ca_address_sk] + CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] + InputAdapter + BroadcastExchange #6 + WholeStageCodegen (8) + ColumnarToRow + InputAdapter + CometFilter [cd_demo_sk] + CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_gender,cd_marital_status,cd_dep_count,cd_dep_employed_count,cd_dep_college_count] 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 new file mode 100644 index 000000000..2610a698c --- /dev/null +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q36a/explain.txt @@ -0,0 +1,279 @@ +== Physical Plan == +TakeOrderedAndProject (41) ++- * Project (40) + +- Window (39) + +- * Sort (38) + +- Exchange (37) + +- * HashAggregate (36) + +- Exchange (35) + +- * HashAggregate (34) + +- Union (33) + :- * HashAggregate (22) + : +- Exchange (21) + : +- * HashAggregate (20) + : +- * Project (19) + : +- * BroadcastHashJoin Inner BuildRight (18) + : :- * Project (12) + : : +- * BroadcastHashJoin Inner BuildRight (11) + : : :- * Project (6) + : : : +- * BroadcastHashJoin Inner BuildRight (5) + : : : :- * ColumnarToRow (3) + : : : : +- CometFilter (2) + : : : : +- CometScan parquet spark_catalog.default.store_sales (1) + : : : +- ReusedExchange (4) + : : +- BroadcastExchange (10) + : : +- * ColumnarToRow (9) + : : +- CometFilter (8) + : : +- CometScan parquet spark_catalog.default.item (7) + : +- BroadcastExchange (17) + : +- * ColumnarToRow (16) + : +- CometProject (15) + : +- CometFilter (14) + : +- CometScan parquet spark_catalog.default.store (13) + :- * HashAggregate (27) + : +- Exchange (26) + : +- * HashAggregate (25) + : +- * HashAggregate (24) + : +- ReusedExchange (23) + +- * HashAggregate (32) + +- Exchange (31) + +- * HashAggregate (30) + +- * HashAggregate (29) + +- ReusedExchange (28) + + +(1) Scan parquet spark_catalog.default.store_sales +Output [5]: [ss_item_sk#1, ss_store_sk#2, ss_ext_sales_price#3, ss_net_profit#4, ss_sold_date_sk#5] +Batched: true +Location: InMemoryFileIndex [] +PartitionFilters: [isnotnull(ss_sold_date_sk#5), dynamicpruningexpression(ss_sold_date_sk#5 IN dynamicpruning#6)] +PushedFilters: [IsNotNull(ss_item_sk), IsNotNull(ss_store_sk)] +ReadSchema: struct<ss_item_sk:int,ss_store_sk:int,ss_ext_sales_price:decimal(7,2),ss_net_profit:decimal(7,2)> + +(2) CometFilter +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)) + +(3) ColumnarToRow [codegen id : 4] +Input [5]: [ss_item_sk#1, ss_store_sk#2, ss_ext_sales_price#3, ss_net_profit#4, ss_sold_date_sk#5] + +(4) ReusedExchange [Reuses operator id: 46] +Output [1]: [d_date_sk#7] + +(5) BroadcastHashJoin [codegen id : 4] +Left keys [1]: [ss_sold_date_sk#5] +Right keys [1]: [d_date_sk#7] +Join type: Inner +Join condition: None + +(6) Project [codegen id : 4] +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] + +(7) Scan parquet spark_catalog.default.item +Output [3]: [i_item_sk#8, i_class#9, i_category#10] +Batched: true +Location [not included in comparison]/{warehouse_dir}/item] +PushedFilters: [IsNotNull(i_item_sk)] +ReadSchema: struct<i_item_sk:int,i_class:string,i_category:string> + +(8) CometFilter +Input [3]: [i_item_sk#8, i_class#9, i_category#10] +Condition : isnotnull(i_item_sk#8) + +(9) ColumnarToRow [codegen id : 2] +Input [3]: [i_item_sk#8, i_class#9, i_category#10] + +(10) BroadcastExchange +Input [3]: [i_item_sk#8, i_class#9, i_category#10] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=1] + +(11) BroadcastHashJoin [codegen id : 4] +Left keys [1]: [ss_item_sk#1] +Right keys [1]: [i_item_sk#8] +Join type: Inner +Join condition: None + +(12) Project [codegen id : 4] +Output [5]: [ss_store_sk#2, ss_ext_sales_price#3, ss_net_profit#4, i_class#9, i_category#10] +Input [7]: [ss_item_sk#1, ss_store_sk#2, ss_ext_sales_price#3, ss_net_profit#4, i_item_sk#8, i_class#9, i_category#10] + +(13) Scan parquet spark_catalog.default.store +Output [2]: [s_store_sk#11, s_state#12] +Batched: true +Location [not included in comparison]/{warehouse_dir}/store] +PushedFilters: [IsNotNull(s_state), EqualTo(s_state,TN), IsNotNull(s_store_sk)] +ReadSchema: struct<s_store_sk:int,s_state:string> + +(14) CometFilter +Input [2]: [s_store_sk#11, s_state#12] +Condition : ((isnotnull(s_state#12) AND (s_state#12 = TN)) AND isnotnull(s_store_sk#11)) + +(15) CometProject +Input [2]: [s_store_sk#11, s_state#12] +Arguments: [s_store_sk#11], [s_store_sk#11] + +(16) ColumnarToRow [codegen id : 3] +Input [1]: [s_store_sk#11] + +(17) BroadcastExchange +Input [1]: [s_store_sk#11] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=2] + +(18) BroadcastHashJoin [codegen id : 4] +Left keys [1]: [ss_store_sk#2] +Right keys [1]: [s_store_sk#11] +Join type: Inner +Join condition: None + +(19) Project [codegen id : 4] +Output [4]: [ss_ext_sales_price#3, ss_net_profit#4, i_class#9, i_category#10] +Input [6]: [ss_store_sk#2, ss_ext_sales_price#3, ss_net_profit#4, i_class#9, i_category#10, s_store_sk#11] + +(20) HashAggregate [codegen id : 4] +Input [4]: [ss_ext_sales_price#3, ss_net_profit#4, i_class#9, i_category#10] +Keys [2]: [i_category#10, i_class#9] +Functions [2]: [partial_sum(UnscaledValue(ss_net_profit#4)), partial_sum(UnscaledValue(ss_ext_sales_price#3))] +Aggregate Attributes [2]: [sum#13, sum#14] +Results [4]: [i_category#10, i_class#9, sum#15, sum#16] + +(21) Exchange +Input [4]: [i_category#10, i_class#9, sum#15, sum#16] +Arguments: hashpartitioning(i_category#10, i_class#9, 5), ENSURE_REQUIREMENTS, [plan_id=3] + +(22) HashAggregate [codegen id : 5] +Input [4]: [i_category#10, i_class#9, sum#15, sum#16] +Keys [2]: [i_category#10, i_class#9] +Functions [2]: [sum(UnscaledValue(ss_net_profit#4)), sum(UnscaledValue(ss_ext_sales_price#3))] +Aggregate Attributes [2]: [sum(UnscaledValue(ss_net_profit#4))#17, sum(UnscaledValue(ss_ext_sales_price#3))#18] +Results [6]: [cast((MakeDecimal(sum(UnscaledValue(ss_net_profit#4))#17,17,2) / MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#3))#18,17,2)) as decimal(38,11)) AS gross_margin#19, i_category#10, i_class#9, 0 AS t_category#20, 0 AS t_class#21, 0 AS lochierarchy#22] + +(23) ReusedExchange [Reuses operator id: 21] +Output [4]: [i_category#23, i_class#24, sum#25, sum#26] + +(24) HashAggregate [codegen id : 10] +Input [4]: [i_category#23, i_class#24, sum#25, sum#26] +Keys [2]: [i_category#23, i_class#24] +Functions [2]: [sum(UnscaledValue(ss_net_profit#27)), sum(UnscaledValue(ss_ext_sales_price#28))] +Aggregate Attributes [2]: [sum(UnscaledValue(ss_net_profit#27))#29, sum(UnscaledValue(ss_ext_sales_price#28))#30] +Results [3]: [MakeDecimal(sum(UnscaledValue(ss_net_profit#27))#29,17,2) AS ss_net_profit#31, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#28))#30,17,2) AS ss_ext_sales_price#32, i_category#23] + +(25) HashAggregate [codegen id : 10] +Input [3]: [ss_net_profit#31, ss_ext_sales_price#32, i_category#23] +Keys [1]: [i_category#23] +Functions [2]: [partial_sum(ss_net_profit#31), partial_sum(ss_ext_sales_price#32)] +Aggregate Attributes [4]: [sum#33, isEmpty#34, sum#35, isEmpty#36] +Results [5]: [i_category#23, sum#37, isEmpty#38, sum#39, isEmpty#40] + +(26) Exchange +Input [5]: [i_category#23, sum#37, isEmpty#38, sum#39, isEmpty#40] +Arguments: hashpartitioning(i_category#23, 5), ENSURE_REQUIREMENTS, [plan_id=4] + +(27) HashAggregate [codegen id : 11] +Input [5]: [i_category#23, sum#37, isEmpty#38, sum#39, isEmpty#40] +Keys [1]: [i_category#23] +Functions [2]: [sum(ss_net_profit#31), sum(ss_ext_sales_price#32)] +Aggregate Attributes [2]: [sum(ss_net_profit#31)#41, sum(ss_ext_sales_price#32)#42] +Results [6]: [(sum(ss_net_profit#31)#41 / sum(ss_ext_sales_price#32)#42) AS gross_margin#43, i_category#23, null AS i_class#44, 0 AS t_category#45, 1 AS t_class#46, 1 AS lochierarchy#47] + +(28) ReusedExchange [Reuses operator id: 21] +Output [4]: [i_category#48, i_class#49, sum#50, sum#51] + +(29) HashAggregate [codegen id : 16] +Input [4]: [i_category#48, i_class#49, sum#50, sum#51] +Keys [2]: [i_category#48, i_class#49] +Functions [2]: [sum(UnscaledValue(ss_net_profit#52)), sum(UnscaledValue(ss_ext_sales_price#53))] +Aggregate Attributes [2]: [sum(UnscaledValue(ss_net_profit#52))#29, sum(UnscaledValue(ss_ext_sales_price#53))#30] +Results [2]: [MakeDecimal(sum(UnscaledValue(ss_net_profit#52))#29,17,2) AS ss_net_profit#54, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#53))#30,17,2) AS ss_ext_sales_price#55] + +(30) HashAggregate [codegen id : 16] +Input [2]: [ss_net_profit#54, ss_ext_sales_price#55] +Keys: [] +Functions [2]: [partial_sum(ss_net_profit#54), partial_sum(ss_ext_sales_price#55)] +Aggregate Attributes [4]: [sum#56, isEmpty#57, sum#58, isEmpty#59] +Results [4]: [sum#60, isEmpty#61, sum#62, isEmpty#63] + +(31) Exchange +Input [4]: [sum#60, isEmpty#61, sum#62, isEmpty#63] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=5] + +(32) HashAggregate [codegen id : 17] +Input [4]: [sum#60, isEmpty#61, sum#62, isEmpty#63] +Keys: [] +Functions [2]: [sum(ss_net_profit#54), sum(ss_ext_sales_price#55)] +Aggregate Attributes [2]: [sum(ss_net_profit#54)#64, sum(ss_ext_sales_price#55)#65] +Results [6]: [(sum(ss_net_profit#54)#64 / sum(ss_ext_sales_price#55)#65) AS gross_margin#66, null AS i_category#67, null AS i_class#68, 1 AS t_category#69, 1 AS t_class#70, 2 AS lochierarchy#71] + +(33) Union + +(34) HashAggregate [codegen id : 18] +Input [6]: [gross_margin#19, i_category#10, i_class#9, t_category#20, t_class#21, lochierarchy#22] +Keys [6]: [gross_margin#19, i_category#10, i_class#9, t_category#20, t_class#21, lochierarchy#22] +Functions: [] +Aggregate Attributes: [] +Results [6]: [gross_margin#19, i_category#10, i_class#9, t_category#20, t_class#21, lochierarchy#22] + +(35) Exchange +Input [6]: [gross_margin#19, i_category#10, i_class#9, t_category#20, t_class#21, lochierarchy#22] +Arguments: hashpartitioning(gross_margin#19, i_category#10, i_class#9, t_category#20, t_class#21, lochierarchy#22, 5), ENSURE_REQUIREMENTS, [plan_id=6] + +(36) HashAggregate [codegen id : 19] +Input [6]: [gross_margin#19, i_category#10, i_class#9, t_category#20, t_class#21, lochierarchy#22] +Keys [6]: [gross_margin#19, i_category#10, i_class#9, t_category#20, t_class#21, lochierarchy#22] +Functions: [] +Aggregate Attributes: [] +Results [5]: [gross_margin#19, i_category#10, i_class#9, lochierarchy#22, CASE WHEN (t_class#21 = 0) THEN i_category#10 END AS _w0#72] + +(37) Exchange +Input [5]: [gross_margin#19, i_category#10, i_class#9, lochierarchy#22, _w0#72] +Arguments: hashpartitioning(lochierarchy#22, _w0#72, 5), ENSURE_REQUIREMENTS, [plan_id=7] + +(38) Sort [codegen id : 20] +Input [5]: [gross_margin#19, i_category#10, i_class#9, lochierarchy#22, _w0#72] +Arguments: [lochierarchy#22 ASC NULLS FIRST, _w0#72 ASC NULLS FIRST, gross_margin#19 ASC NULLS FIRST], false, 0 + +(39) Window +Input [5]: [gross_margin#19, i_category#10, i_class#9, lochierarchy#22, _w0#72] +Arguments: [rank(gross_margin#19) windowspecdefinition(lochierarchy#22, _w0#72, gross_margin#19 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rank_within_parent#73], [lochierarchy#22, _w0#72], [gross_margin#19 ASC NULLS FIRST] + +(40) Project [codegen id : 21] +Output [5]: [gross_margin#19, i_category#10, i_class#9, lochierarchy#22, rank_within_parent#73] +Input [6]: [gross_margin#19, i_category#10, i_class#9, lochierarchy#22, _w0#72, rank_within_parent#73] + +(41) TakeOrderedAndProject +Input [5]: [gross_margin#19, i_category#10, i_class#9, lochierarchy#22, rank_within_parent#73] +Arguments: 100, [lochierarchy#22 DESC NULLS LAST, CASE WHEN (lochierarchy#22 = 0) THEN i_category#10 END ASC NULLS FIRST, rank_within_parent#73 ASC NULLS FIRST], [gross_margin#19, i_category#10, i_class#9, lochierarchy#22, rank_within_parent#73] + +===== Subqueries ===== + +Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#5 IN dynamicpruning#6 +BroadcastExchange (46) ++- * ColumnarToRow (45) + +- CometProject (44) + +- CometFilter (43) + +- CometScan parquet spark_catalog.default.date_dim (42) + + +(42) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#7, d_year#74] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), IsNotNull(d_date_sk)] +ReadSchema: struct<d_date_sk:int,d_year:int> + +(43) CometFilter +Input [2]: [d_date_sk#7, d_year#74] +Condition : ((isnotnull(d_year#74) AND (d_year#74 = 2001)) AND isnotnull(d_date_sk#7)) + +(44) CometProject +Input [2]: [d_date_sk#7, d_year#74] +Arguments: [d_date_sk#7], [d_date_sk#7] + +(45) ColumnarToRow [codegen id : 1] +Input [1]: [d_date_sk#7] + +(46) BroadcastExchange +Input [1]: [d_date_sk#7] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=8] + + diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q36a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q36a/simplified.txt new file mode 100644 index 000000000..f265d2099 --- /dev/null +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q36a/simplified.txt @@ -0,0 +1,76 @@ +TakeOrderedAndProject [lochierarchy,i_category,rank_within_parent,gross_margin,i_class] + WholeStageCodegen (21) + Project [gross_margin,i_category,i_class,lochierarchy,rank_within_parent] + InputAdapter + Window [gross_margin,lochierarchy,_w0] + WholeStageCodegen (20) + Sort [lochierarchy,_w0,gross_margin] + InputAdapter + Exchange [lochierarchy,_w0] #1 + WholeStageCodegen (19) + HashAggregate [gross_margin,i_category,i_class,t_category,t_class,lochierarchy] [_w0] + InputAdapter + Exchange [gross_margin,i_category,i_class,t_category,t_class,lochierarchy] #2 + WholeStageCodegen (18) + 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,t_category,t_class,lochierarchy,sum,sum] + InputAdapter + Exchange [i_category,i_class] #3 + WholeStageCodegen (4) + HashAggregate [i_category,i_class,ss_net_profit,ss_ext_sales_price] [sum,sum,sum,sum] + Project [ss_ext_sales_price,ss_net_profit,i_class,i_category] + BroadcastHashJoin [ss_store_sk,s_store_sk] + Project [ss_store_sk,ss_ext_sales_price,ss_net_profit,i_class,i_category] + BroadcastHashJoin [ss_item_sk,i_item_sk] + Project [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + ColumnarToRow + InputAdapter + CometFilter [ss_item_sk,ss_store_sk] + CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #4 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_year,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + InputAdapter + ReusedExchange [d_date_sk] #4 + InputAdapter + BroadcastExchange #5 + WholeStageCodegen (2) + ColumnarToRow + InputAdapter + CometFilter [i_item_sk] + CometScan parquet spark_catalog.default.item [i_item_sk,i_class,i_category] + InputAdapter + BroadcastExchange #6 + WholeStageCodegen (3) + ColumnarToRow + InputAdapter + CometProject [s_store_sk] + CometFilter [s_state,s_store_sk] + CometScan parquet spark_catalog.default.store [s_store_sk,s_state] + WholeStageCodegen (11) + HashAggregate [i_category,sum,isEmpty,sum,isEmpty] [sum(ss_net_profit),sum(ss_ext_sales_price),gross_margin,i_class,t_category,t_class,lochierarchy,sum,isEmpty,sum,isEmpty] + InputAdapter + Exchange [i_category] #7 + WholeStageCodegen (10) + HashAggregate [i_category,ss_net_profit,ss_ext_sales_price] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] + HashAggregate [i_category,i_class,sum,sum] [sum(UnscaledValue(ss_net_profit)),sum(UnscaledValue(ss_ext_sales_price)),ss_net_profit,ss_ext_sales_price,sum,sum] + InputAdapter + ReusedExchange [i_category,i_class,sum,sum] #3 + WholeStageCodegen (17) + HashAggregate [sum,isEmpty,sum,isEmpty] [sum(ss_net_profit),sum(ss_ext_sales_price),gross_margin,i_category,i_class,t_category,t_class,lochierarchy,sum,isEmpty,sum,isEmpty] + InputAdapter + Exchange #8 + WholeStageCodegen (16) + HashAggregate [ss_net_profit,ss_ext_sales_price] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] + HashAggregate [i_category,i_class,sum,sum] [sum(UnscaledValue(ss_net_profit)),sum(UnscaledValue(ss_ext_sales_price)),ss_net_profit,ss_ext_sales_price,sum,sum] + InputAdapter + ReusedExchange [i_category,i_class,sum,sum] #3 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 new file mode 100644 index 000000000..9cb7b35cb --- /dev/null +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q47/explain.txt @@ -0,0 +1,279 @@ +== Physical Plan == +TakeOrderedAndProject (45) ++- * Project (44) + +- * BroadcastHashJoin Inner BuildRight (43) + :- * Project (37) + : +- * BroadcastHashJoin Inner BuildRight (36) + : :- * Project (28) + : : +- * Filter (27) + : : +- Window (26) + : : +- * Filter (25) + : : +- Window (24) + : : +- * Sort (23) + : : +- Exchange (22) + : : +- * HashAggregate (21) + : : +- Exchange (20) + : : +- * HashAggregate (19) + : : +- * Project (18) + : : +- * BroadcastHashJoin Inner BuildRight (17) + : : :- * Project (12) + : : : +- * BroadcastHashJoin Inner BuildRight (11) + : : : :- * Project (9) + : : : : +- * BroadcastHashJoin Inner BuildRight (8) + : : : : :- * ColumnarToRow (3) + : : : : : +- CometFilter (2) + : : : : : +- CometScan parquet spark_catalog.default.item (1) + : : : : +- BroadcastExchange (7) + : : : : +- * ColumnarToRow (6) + : : : : +- CometFilter (5) + : : : : +- CometScan parquet spark_catalog.default.store_sales (4) + : : : +- ReusedExchange (10) + : : +- BroadcastExchange (16) + : : +- * ColumnarToRow (15) + : : +- CometFilter (14) + : : +- CometScan parquet spark_catalog.default.store (13) + : +- BroadcastExchange (35) + : +- * Project (34) + : +- Window (33) + : +- * Sort (32) + : +- Exchange (31) + : +- * HashAggregate (30) + : +- ReusedExchange (29) + +- BroadcastExchange (42) + +- * Project (41) + +- Window (40) + +- * Sort (39) + +- ReusedExchange (38) + + +(1) Scan parquet spark_catalog.default.item +Output [3]: [i_item_sk#1, i_brand#2, i_category#3] +Batched: true +Location [not included in comparison]/{warehouse_dir}/item] +PushedFilters: [IsNotNull(i_item_sk), IsNotNull(i_category), IsNotNull(i_brand)] +ReadSchema: struct<i_item_sk:int,i_brand:string,i_category:string> + +(2) CometFilter +Input [3]: [i_item_sk#1, i_brand#2, i_category#3] +Condition : ((isnotnull(i_item_sk#1) AND isnotnull(i_category#3)) AND isnotnull(i_brand#2)) + +(3) ColumnarToRow [codegen id : 4] +Input [3]: [i_item_sk#1, i_brand#2, i_category#3] + +(4) Scan parquet spark_catalog.default.store_sales +Output [4]: [ss_item_sk#4, ss_store_sk#5, ss_sales_price#6, ss_sold_date_sk#7] +Batched: true +Location: InMemoryFileIndex [] +PartitionFilters: [isnotnull(ss_sold_date_sk#7), dynamicpruningexpression(ss_sold_date_sk#7 IN dynamicpruning#8)] +PushedFilters: [IsNotNull(ss_item_sk), IsNotNull(ss_store_sk)] +ReadSchema: struct<ss_item_sk:int,ss_store_sk:int,ss_sales_price:decimal(7,2)> + +(5) CometFilter +Input [4]: [ss_item_sk#4, ss_store_sk#5, ss_sales_price#6, ss_sold_date_sk#7] +Condition : (isnotnull(ss_item_sk#4) AND isnotnull(ss_store_sk#5)) + +(6) ColumnarToRow [codegen id : 1] +Input [4]: [ss_item_sk#4, ss_store_sk#5, ss_sales_price#6, ss_sold_date_sk#7] + +(7) BroadcastExchange +Input [4]: [ss_item_sk#4, ss_store_sk#5, ss_sales_price#6, ss_sold_date_sk#7] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=1] + +(8) BroadcastHashJoin [codegen id : 4] +Left keys [1]: [i_item_sk#1] +Right keys [1]: [ss_item_sk#4] +Join type: Inner +Join condition: None + +(9) Project [codegen id : 4] +Output [5]: [i_brand#2, i_category#3, ss_store_sk#5, ss_sales_price#6, ss_sold_date_sk#7] +Input [7]: [i_item_sk#1, i_brand#2, i_category#3, ss_item_sk#4, ss_store_sk#5, ss_sales_price#6, ss_sold_date_sk#7] + +(10) ReusedExchange [Reuses operator id: 49] +Output [3]: [d_date_sk#9, d_year#10, d_moy#11] + +(11) BroadcastHashJoin [codegen id : 4] +Left keys [1]: [ss_sold_date_sk#7] +Right keys [1]: [d_date_sk#9] +Join type: Inner +Join condition: None + +(12) Project [codegen id : 4] +Output [6]: [i_brand#2, i_category#3, ss_store_sk#5, ss_sales_price#6, d_year#10, d_moy#11] +Input [8]: [i_brand#2, i_category#3, ss_store_sk#5, ss_sales_price#6, ss_sold_date_sk#7, d_date_sk#9, d_year#10, d_moy#11] + +(13) Scan parquet spark_catalog.default.store +Output [3]: [s_store_sk#12, s_store_name#13, s_company_name#14] +Batched: true +Location [not included in comparison]/{warehouse_dir}/store] +PushedFilters: [IsNotNull(s_store_sk), IsNotNull(s_store_name), IsNotNull(s_company_name)] +ReadSchema: struct<s_store_sk:int,s_store_name:string,s_company_name:string> + +(14) CometFilter +Input [3]: [s_store_sk#12, s_store_name#13, s_company_name#14] +Condition : ((isnotnull(s_store_sk#12) AND isnotnull(s_store_name#13)) AND isnotnull(s_company_name#14)) + +(15) ColumnarToRow [codegen id : 3] +Input [3]: [s_store_sk#12, s_store_name#13, s_company_name#14] + +(16) BroadcastExchange +Input [3]: [s_store_sk#12, s_store_name#13, s_company_name#14] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=2] + +(17) BroadcastHashJoin [codegen id : 4] +Left keys [1]: [ss_store_sk#5] +Right keys [1]: [s_store_sk#12] +Join type: Inner +Join condition: None + +(18) Project [codegen id : 4] +Output [7]: [i_brand#2, i_category#3, ss_sales_price#6, d_year#10, d_moy#11, s_store_name#13, s_company_name#14] +Input [9]: [i_brand#2, i_category#3, ss_store_sk#5, ss_sales_price#6, d_year#10, d_moy#11, s_store_sk#12, s_store_name#13, s_company_name#14] + +(19) HashAggregate [codegen id : 4] +Input [7]: [i_brand#2, i_category#3, ss_sales_price#6, d_year#10, d_moy#11, s_store_name#13, s_company_name#14] +Keys [6]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11] +Functions [1]: [partial_sum(UnscaledValue(ss_sales_price#6))] +Aggregate Attributes [1]: [sum#15] +Results [7]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, sum#16] + +(20) Exchange +Input [7]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, sum#16] +Arguments: hashpartitioning(i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, 5), ENSURE_REQUIREMENTS, [plan_id=3] + +(21) HashAggregate [codegen id : 5] +Input [7]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, sum#16] +Keys [6]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11] +Functions [1]: [sum(UnscaledValue(ss_sales_price#6))] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_sales_price#6))#17] +Results [8]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, MakeDecimal(sum(UnscaledValue(ss_sales_price#6))#17,17,2) AS sum_sales#18, MakeDecimal(sum(UnscaledValue(ss_sales_price#6))#17,17,2) AS _w0#19] + +(22) Exchange +Input [8]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, sum_sales#18, _w0#19] +Arguments: hashpartitioning(i_category#3, i_brand#2, s_store_name#13, s_company_name#14, 5), ENSURE_REQUIREMENTS, [plan_id=4] + +(23) Sort [codegen id : 6] +Input [8]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, sum_sales#18, _w0#19] +Arguments: [i_category#3 ASC NULLS FIRST, i_brand#2 ASC NULLS FIRST, s_store_name#13 ASC NULLS FIRST, s_company_name#14 ASC NULLS FIRST, d_year#10 ASC NULLS FIRST, d_moy#11 ASC NULLS FIRST], false, 0 + +(24) Window +Input [8]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, sum_sales#18, _w0#19] +Arguments: [rank(d_year#10, d_moy#11) windowspecdefinition(i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10 ASC NULLS FIRST, d_moy#11 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#20], [i_category#3, i_brand#2, s_store_name#13, s_company_name#14], [d_year#10 ASC NULLS FIRST, d_moy#11 ASC NULLS FIRST] + +(25) Filter [codegen id : 7] +Input [9]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, sum_sales#18, _w0#19, rn#20] +Condition : (isnotnull(d_year#10) AND (d_year#10 = 1999)) + +(26) Window +Input [9]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, sum_sales#18, _w0#19, rn#20] +Arguments: [avg(_w0#19) windowspecdefinition(i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS avg_monthly_sales#21], [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10] + +(27) Filter [codegen id : 22] +Input [10]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, sum_sales#18, _w0#19, rn#20, avg_monthly_sales#21] +Condition : ((isnotnull(avg_monthly_sales#21) AND (avg_monthly_sales#21 > 0.000000)) AND CASE WHEN (avg_monthly_sales#21 > 0.000000) THEN ((abs((sum_sales#18 - avg_monthly_sales#21)) / avg_monthly_sales#21) > 0.1000000000000000) END) + +(28) Project [codegen id : 22] +Output [9]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, sum_sales#18, avg_monthly_sales#21, rn#20] +Input [10]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, sum_sales#18, _w0#19, rn#20, avg_monthly_sales#21] + +(29) ReusedExchange [Reuses operator id: 20] +Output [7]: [i_category#22, i_brand#23, s_store_name#24, s_company_name#25, d_year#26, d_moy#27, sum#28] + +(30) HashAggregate [codegen id : 12] +Input [7]: [i_category#22, i_brand#23, s_store_name#24, s_company_name#25, d_year#26, d_moy#27, sum#28] +Keys [6]: [i_category#22, i_brand#23, s_store_name#24, s_company_name#25, d_year#26, d_moy#27] +Functions [1]: [sum(UnscaledValue(ss_sales_price#29))] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_sales_price#29))#17] +Results [7]: [i_category#22, i_brand#23, s_store_name#24, s_company_name#25, d_year#26, d_moy#27, MakeDecimal(sum(UnscaledValue(ss_sales_price#29))#17,17,2) AS sum_sales#30] + +(31) Exchange +Input [7]: [i_category#22, i_brand#23, s_store_name#24, s_company_name#25, d_year#26, d_moy#27, sum_sales#30] +Arguments: hashpartitioning(i_category#22, i_brand#23, s_store_name#24, s_company_name#25, 5), ENSURE_REQUIREMENTS, [plan_id=5] + +(32) Sort [codegen id : 13] +Input [7]: [i_category#22, i_brand#23, s_store_name#24, s_company_name#25, d_year#26, d_moy#27, sum_sales#30] +Arguments: [i_category#22 ASC NULLS FIRST, i_brand#23 ASC NULLS FIRST, s_store_name#24 ASC NULLS FIRST, s_company_name#25 ASC NULLS FIRST, d_year#26 ASC NULLS FIRST, d_moy#27 ASC NULLS FIRST], false, 0 + +(33) Window +Input [7]: [i_category#22, i_brand#23, s_store_name#24, s_company_name#25, d_year#26, d_moy#27, sum_sales#30] +Arguments: [rank(d_year#26, d_moy#27) windowspecdefinition(i_category#22, i_brand#23, s_store_name#24, s_company_name#25, d_year#26 ASC NULLS FIRST, d_moy#27 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#31], [i_category#22, i_brand#23, s_store_name#24, s_company_name#25], [d_year#26 ASC NULLS FIRST, d_moy#27 ASC NULLS FIRST] + +(34) Project [codegen id : 14] +Output [6]: [i_category#22, i_brand#23, s_store_name#24, s_company_name#25, sum_sales#30, rn#31] +Input [8]: [i_category#22, i_brand#23, s_store_name#24, s_company_name#25, d_year#26, d_moy#27, sum_sales#30, rn#31] + +(35) BroadcastExchange +Input [6]: [i_category#22, i_brand#23, s_store_name#24, s_company_name#25, sum_sales#30, rn#31] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], input[3, string, true], (input[5, int, false] + 1)),false), [plan_id=6] + +(36) BroadcastHashJoin [codegen id : 22] +Left keys [5]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, rn#20] +Right keys [5]: [i_category#22, i_brand#23, s_store_name#24, s_company_name#25, (rn#31 + 1)] +Join type: Inner +Join condition: None + +(37) Project [codegen id : 22] +Output [10]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, sum_sales#18, avg_monthly_sales#21, rn#20, sum_sales#30] +Input [15]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, sum_sales#18, avg_monthly_sales#21, rn#20, i_category#22, i_brand#23, s_store_name#24, s_company_name#25, sum_sales#30, rn#31] + +(38) ReusedExchange [Reuses operator id: 31] +Output [7]: [i_category#32, i_brand#33, s_store_name#34, s_company_name#35, d_year#36, d_moy#37, sum_sales#38] + +(39) Sort [codegen id : 20] +Input [7]: [i_category#32, i_brand#33, s_store_name#34, s_company_name#35, d_year#36, d_moy#37, sum_sales#38] +Arguments: [i_category#32 ASC NULLS FIRST, i_brand#33 ASC NULLS FIRST, s_store_name#34 ASC NULLS FIRST, s_company_name#35 ASC NULLS FIRST, d_year#36 ASC NULLS FIRST, d_moy#37 ASC NULLS FIRST], false, 0 + +(40) Window +Input [7]: [i_category#32, i_brand#33, s_store_name#34, s_company_name#35, d_year#36, d_moy#37, sum_sales#38] +Arguments: [rank(d_year#36, d_moy#37) windowspecdefinition(i_category#32, i_brand#33, s_store_name#34, s_company_name#35, d_year#36 ASC NULLS FIRST, d_moy#37 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#39], [i_category#32, i_brand#33, s_store_name#34, s_company_name#35], [d_year#36 ASC NULLS FIRST, d_moy#37 ASC NULLS FIRST] + +(41) Project [codegen id : 21] +Output [6]: [i_category#32, i_brand#33, s_store_name#34, s_company_name#35, sum_sales#38, rn#39] +Input [8]: [i_category#32, i_brand#33, s_store_name#34, s_company_name#35, d_year#36, d_moy#37, sum_sales#38, rn#39] + +(42) BroadcastExchange +Input [6]: [i_category#32, i_brand#33, s_store_name#34, s_company_name#35, sum_sales#38, rn#39] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], input[3, string, true], (input[5, int, false] - 1)),false), [plan_id=7] + +(43) BroadcastHashJoin [codegen id : 22] +Left keys [5]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, rn#20] +Right keys [5]: [i_category#32, i_brand#33, s_store_name#34, s_company_name#35, (rn#39 - 1)] +Join type: Inner +Join condition: None + +(44) Project [codegen id : 22] +Output [7]: [i_category#3, d_year#10, d_moy#11, avg_monthly_sales#21, sum_sales#18, sum_sales#30 AS psum#40, sum_sales#38 AS nsum#41] +Input [16]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, sum_sales#18, avg_monthly_sales#21, rn#20, sum_sales#30, i_category#32, i_brand#33, s_store_name#34, s_company_name#35, sum_sales#38, rn#39] + +(45) TakeOrderedAndProject +Input [7]: [i_category#3, d_year#10, d_moy#11, avg_monthly_sales#21, sum_sales#18, psum#40, nsum#41] +Arguments: 100, [(sum_sales#18 - avg_monthly_sales#21) ASC NULLS FIRST, d_moy#11 ASC NULLS FIRST], [i_category#3, d_year#10, d_moy#11, avg_monthly_sales#21, sum_sales#18, psum#40, nsum#41] + +===== Subqueries ===== + +Subquery:1 Hosting operator id = 4 Hosting Expression = ss_sold_date_sk#7 IN dynamicpruning#8 +BroadcastExchange (49) ++- * ColumnarToRow (48) + +- CometFilter (47) + +- CometScan parquet spark_catalog.default.date_dim (46) + + +(46) Scan parquet spark_catalog.default.date_dim +Output [3]: [d_date_sk#9, d_year#10, d_moy#11] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [Or(Or(EqualTo(d_year,1999),And(EqualTo(d_year,1998),EqualTo(d_moy,12))),And(EqualTo(d_year,2000),EqualTo(d_moy,1))), IsNotNull(d_date_sk)] +ReadSchema: struct<d_date_sk:int,d_year:int,d_moy:int> + +(47) CometFilter +Input [3]: [d_date_sk#9, d_year#10, d_moy#11] +Condition : ((((d_year#10 = 1999) OR ((d_year#10 = 1998) AND (d_moy#11 = 12))) OR ((d_year#10 = 2000) AND (d_moy#11 = 1))) AND isnotnull(d_date_sk#9)) + +(48) ColumnarToRow [codegen id : 1] +Input [3]: [d_date_sk#9, d_year#10, d_moy#11] + +(49) BroadcastExchange +Input [3]: [d_date_sk#9, d_year#10, d_moy#11] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=8] + + 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 new file mode 100644 index 000000000..a54895305 --- /dev/null +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q47/simplified.txt @@ -0,0 +1,81 @@ +TakeOrderedAndProject [sum_sales,avg_monthly_sales,d_moy,i_category,d_year,psum,nsum] + WholeStageCodegen (22) + 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] + 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] + Filter [avg_monthly_sales,sum_sales] + InputAdapter + Window [_w0,i_category,i_brand,s_store_name,s_company_name,d_year] + WholeStageCodegen (7) + Filter [d_year] + InputAdapter + Window [d_year,d_moy,i_category,i_brand,s_store_name,s_company_name] + WholeStageCodegen (6) + Sort [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy] + InputAdapter + Exchange [i_category,i_brand,s_store_name,s_company_name] #1 + WholeStageCodegen (5) + HashAggregate [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum] [sum(UnscaledValue(ss_sales_price)),sum_sales,_w0,sum] + InputAdapter + Exchange [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy] #2 + WholeStageCodegen (4) + HashAggregate [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,ss_sales_price] [sum,sum] + Project [i_brand,i_category,ss_sales_price,d_year,d_moy,s_store_name,s_company_name] + BroadcastHashJoin [ss_store_sk,s_store_sk] + Project [i_brand,i_category,ss_store_sk,ss_sales_price,d_year,d_moy] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [i_brand,i_category,ss_store_sk,ss_sales_price,ss_sold_date_sk] + BroadcastHashJoin [i_item_sk,ss_item_sk] + ColumnarToRow + InputAdapter + CometFilter [i_item_sk,i_category,i_brand] + CometScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_category] + InputAdapter + BroadcastExchange #3 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometFilter [ss_item_sk,ss_store_sk] + CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #4 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometFilter [d_year,d_moy,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + InputAdapter + ReusedExchange [d_date_sk,d_year,d_moy] #4 + InputAdapter + BroadcastExchange #5 + WholeStageCodegen (3) + ColumnarToRow + InputAdapter + CometFilter [s_store_sk,s_store_name,s_company_name] + CometScan parquet spark_catalog.default.store [s_store_sk,s_store_name,s_company_name] + InputAdapter + BroadcastExchange #6 + WholeStageCodegen (14) + 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) + Sort [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy] + InputAdapter + Exchange [i_category,i_brand,s_store_name,s_company_name] #7 + WholeStageCodegen (12) + HashAggregate [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum] [sum(UnscaledValue(ss_sales_price)),sum_sales,sum] + InputAdapter + ReusedExchange [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum] #2 + InputAdapter + BroadcastExchange #8 + WholeStageCodegen (21) + 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) + Sort [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy] + InputAdapter + ReusedExchange [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales] #7 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 new file mode 100644 index 000000000..6591c8b8a --- /dev/null +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q49/explain.txt @@ -0,0 +1,456 @@ +== Physical Plan == +TakeOrderedAndProject (74) ++- * HashAggregate (73) + +- Exchange (72) + +- * HashAggregate (71) + +- Union (70) + :- * Project (23) + : +- * Filter (22) + : +- Window (21) + : +- * Sort (20) + : +- Window (19) + : +- * Sort (18) + : +- Exchange (17) + : +- * HashAggregate (16) + : +- Exchange (15) + : +- * HashAggregate (14) + : +- * Project (13) + : +- * BroadcastHashJoin Inner BuildRight (12) + : :- * ColumnarToRow (10) + : : +- CometProject (9) + : : +- CometBroadcastHashJoin (8) + : : :- CometBroadcastExchange (4) + : : : +- CometProject (3) + : : : +- CometFilter (2) + : : : +- CometScan parquet spark_catalog.default.web_sales (1) + : : +- CometProject (7) + : : +- CometFilter (6) + : : +- CometScan parquet spark_catalog.default.web_returns (5) + : +- ReusedExchange (11) + :- * Project (46) + : +- * Filter (45) + : +- Window (44) + : +- * Sort (43) + : +- Window (42) + : +- * Sort (41) + : +- Exchange (40) + : +- * HashAggregate (39) + : +- Exchange (38) + : +- * HashAggregate (37) + : +- * Project (36) + : +- * BroadcastHashJoin Inner BuildRight (35) + : :- * ColumnarToRow (33) + : : +- CometProject (32) + : : +- CometBroadcastHashJoin (31) + : : :- CometBroadcastExchange (27) + : : : +- CometProject (26) + : : : +- CometFilter (25) + : : : +- CometScan parquet spark_catalog.default.catalog_sales (24) + : : +- CometProject (30) + : : +- CometFilter (29) + : : +- CometScan parquet spark_catalog.default.catalog_returns (28) + : +- ReusedExchange (34) + +- * Project (69) + +- * Filter (68) + +- Window (67) + +- * Sort (66) + +- Window (65) + +- * Sort (64) + +- Exchange (63) + +- * HashAggregate (62) + +- Exchange (61) + +- * HashAggregate (60) + +- * Project (59) + +- * BroadcastHashJoin Inner BuildRight (58) + :- * ColumnarToRow (56) + : +- CometProject (55) + : +- CometBroadcastHashJoin (54) + : :- CometBroadcastExchange (50) + : : +- CometProject (49) + : : +- CometFilter (48) + : : +- CometScan parquet spark_catalog.default.store_sales (47) + : +- CometProject (53) + : +- CometFilter (52) + : +- CometScan parquet spark_catalog.default.store_returns (51) + +- ReusedExchange (57) + + +(1) Scan parquet spark_catalog.default.web_sales +Output [6]: [ws_item_sk#1, ws_order_number#2, ws_quantity#3, ws_net_paid#4, ws_net_profit#5, ws_sold_date_sk#6] +Batched: true +Location: InMemoryFileIndex [] +PartitionFilters: [isnotnull(ws_sold_date_sk#6), dynamicpruningexpression(ws_sold_date_sk#6 IN dynamicpruning#7)] +PushedFilters: [IsNotNull(ws_net_profit), IsNotNull(ws_net_paid), IsNotNull(ws_quantity), GreaterThan(ws_net_profit,1.00), GreaterThan(ws_net_paid,0.00), GreaterThan(ws_quantity,0), IsNotNull(ws_order_number), IsNotNull(ws_item_sk)] +ReadSchema: struct<ws_item_sk:int,ws_order_number:int,ws_quantity:int,ws_net_paid:decimal(7,2),ws_net_profit:decimal(7,2)> + +(2) CometFilter +Input [6]: [ws_item_sk#1, ws_order_number#2, ws_quantity#3, ws_net_paid#4, ws_net_profit#5, ws_sold_date_sk#6] +Condition : (((((((isnotnull(ws_net_profit#5) AND isnotnull(ws_net_paid#4)) AND isnotnull(ws_quantity#3)) AND (ws_net_profit#5 > 1.00)) AND (ws_net_paid#4 > 0.00)) AND (ws_quantity#3 > 0)) AND isnotnull(ws_order_number#2)) AND isnotnull(ws_item_sk#1)) + +(3) CometProject +Input [6]: [ws_item_sk#1, ws_order_number#2, ws_quantity#3, ws_net_paid#4, ws_net_profit#5, ws_sold_date_sk#6] +Arguments: [ws_item_sk#1, ws_order_number#2, ws_quantity#3, ws_net_paid#4, ws_sold_date_sk#6], [ws_item_sk#1, ws_order_number#2, ws_quantity#3, ws_net_paid#4, ws_sold_date_sk#6] + +(4) CometBroadcastExchange +Input [5]: [ws_item_sk#1, ws_order_number#2, ws_quantity#3, ws_net_paid#4, ws_sold_date_sk#6] +Arguments: [ws_item_sk#1, ws_order_number#2, ws_quantity#3, ws_net_paid#4, ws_sold_date_sk#6] + +(5) Scan parquet spark_catalog.default.web_returns +Output [5]: [wr_item_sk#8, wr_order_number#9, wr_return_quantity#10, wr_return_amt#11, wr_returned_date_sk#12] +Batched: true +Location [not included in comparison]/{warehouse_dir}/web_returns] +PushedFilters: [IsNotNull(wr_return_amt), GreaterThan(wr_return_amt,10000.00), IsNotNull(wr_order_number), IsNotNull(wr_item_sk)] +ReadSchema: struct<wr_item_sk:int,wr_order_number:int,wr_return_quantity:int,wr_return_amt:decimal(7,2)> + +(6) CometFilter +Input [5]: [wr_item_sk#8, wr_order_number#9, wr_return_quantity#10, wr_return_amt#11, wr_returned_date_sk#12] +Condition : (((isnotnull(wr_return_amt#11) AND (wr_return_amt#11 > 10000.00)) AND isnotnull(wr_order_number#9)) AND isnotnull(wr_item_sk#8)) + +(7) CometProject +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] + +(8) CometBroadcastHashJoin +Left output [5]: [ws_item_sk#1, ws_order_number#2, ws_quantity#3, ws_net_paid#4, ws_sold_date_sk#6] +Right output [4]: [wr_item_sk#8, wr_order_number#9, wr_return_quantity#10, wr_return_amt#11] +Arguments: [ws_order_number#2, ws_item_sk#1], [wr_order_number#9, wr_item_sk#8], Inner + +(9) CometProject +Input [9]: [ws_item_sk#1, ws_order_number#2, ws_quantity#3, ws_net_paid#4, ws_sold_date_sk#6, wr_item_sk#8, wr_order_number#9, wr_return_quantity#10, wr_return_amt#11] +Arguments: [ws_item_sk#1, ws_quantity#3, ws_net_paid#4, ws_sold_date_sk#6, wr_return_quantity#10, wr_return_amt#11], [ws_item_sk#1, ws_quantity#3, ws_net_paid#4, ws_sold_date_sk#6, wr_return_quantity#10, wr_return_amt#11] + +(10) ColumnarToRow [codegen id : 2] +Input [6]: [ws_item_sk#1, ws_quantity#3, ws_net_paid#4, ws_sold_date_sk#6, wr_return_quantity#10, wr_return_amt#11] + +(11) ReusedExchange [Reuses operator id: 79] +Output [1]: [d_date_sk#13] + +(12) BroadcastHashJoin [codegen id : 2] +Left keys [1]: [ws_sold_date_sk#6] +Right keys [1]: [d_date_sk#13] +Join type: Inner +Join condition: None + +(13) Project [codegen id : 2] +Output [5]: [ws_item_sk#1, ws_quantity#3, ws_net_paid#4, wr_return_quantity#10, wr_return_amt#11] +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] + +(14) 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))] +Aggregate Attributes [6]: [sum#14, sum#15, sum#16, isEmpty#17, sum#18, isEmpty#19] +Results [7]: [ws_item_sk#1, sum#20, sum#21, sum#22, isEmpty#23, sum#24, isEmpty#25] + +(15) Exchange +Input [7]: [ws_item_sk#1, sum#20, sum#21, sum#22, isEmpty#23, sum#24, isEmpty#25] +Arguments: hashpartitioning(ws_item_sk#1, 5), ENSURE_REQUIREMENTS, [plan_id=1] + +(16) HashAggregate [codegen id : 3] +Input [7]: [ws_item_sk#1, sum#20, sum#21, sum#22, isEmpty#23, sum#24, isEmpty#25] +Keys [1]: [ws_item_sk#1] +Functions [4]: [sum(coalesce(wr_return_quantity#10, 0)), sum(coalesce(ws_quantity#3, 0)), sum(coalesce(cast(wr_return_amt#11 as decimal(12,2)), 0.00)), sum(coalesce(cast(ws_net_paid#4 as decimal(12,2)), 0.00))] +Aggregate Attributes [4]: [sum(coalesce(wr_return_quantity#10, 0))#26, sum(coalesce(ws_quantity#3, 0))#27, sum(coalesce(cast(wr_return_amt#11 as decimal(12,2)), 0.00))#28, sum(coalesce(cast(ws_net_paid#4 as decimal(12,2)), 0.00))#29] +Results [3]: [ws_item_sk#1 AS item#30, (cast(sum(coalesce(wr_return_quantity#10, 0))#26 as decimal(15,4)) / cast(sum(coalesce(ws_quantity#3, 0))#27 as decimal(15,4))) AS return_ratio#31, (cast(sum(coalesce(cast(wr_return_amt#11 as decimal(12,2)), 0.00))#28 as decimal(15,4)) / cast(sum(coalesce(cast(ws_net_paid#4 as decimal(12,2)), 0.00))#29 as decimal(15,4))) AS currency_ratio#32] + +(17) Exchange +Input [3]: [item#30, return_ratio#31, currency_ratio#32] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=2] + +(18) Sort [codegen id : 4] +Input [3]: [item#30, return_ratio#31, currency_ratio#32] +Arguments: [return_ratio#31 ASC NULLS FIRST], false, 0 + +(19) Window +Input [3]: [item#30, return_ratio#31, currency_ratio#32] +Arguments: [rank(return_ratio#31) windowspecdefinition(return_ratio#31 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS return_rank#33], [return_ratio#31 ASC NULLS FIRST] + +(20) Sort [codegen id : 5] +Input [4]: [item#30, return_ratio#31, currency_ratio#32, return_rank#33] +Arguments: [currency_ratio#32 ASC NULLS FIRST], false, 0 + +(21) Window +Input [4]: [item#30, return_ratio#31, currency_ratio#32, return_rank#33] +Arguments: [rank(currency_ratio#32) windowspecdefinition(currency_ratio#32 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS currency_rank#34], [currency_ratio#32 ASC NULLS FIRST] + +(22) Filter [codegen id : 6] +Input [5]: [item#30, return_ratio#31, currency_ratio#32, return_rank#33, currency_rank#34] +Condition : ((return_rank#33 <= 10) OR (currency_rank#34 <= 10)) + +(23) Project [codegen id : 6] +Output [5]: [web AS channel#35, item#30, return_ratio#31, return_rank#33, currency_rank#34] +Input [5]: [item#30, return_ratio#31, currency_ratio#32, return_rank#33, currency_rank#34] + +(24) Scan parquet spark_catalog.default.catalog_sales +Output [6]: [cs_item_sk#36, cs_order_number#37, cs_quantity#38, cs_net_paid#39, cs_net_profit#40, cs_sold_date_sk#41] +Batched: true +Location: InMemoryFileIndex [] +PartitionFilters: [isnotnull(cs_sold_date_sk#41), dynamicpruningexpression(cs_sold_date_sk#41 IN dynamicpruning#42)] +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<cs_item_sk:int,cs_order_number:int,cs_quantity:int,cs_net_paid:decimal(7,2),cs_net_profit:decimal(7,2)> + +(25) CometFilter +Input [6]: [cs_item_sk#36, cs_order_number#37, cs_quantity#38, cs_net_paid#39, cs_net_profit#40, cs_sold_date_sk#41] +Condition : (((((((isnotnull(cs_net_profit#40) AND isnotnull(cs_net_paid#39)) AND isnotnull(cs_quantity#38)) AND (cs_net_profit#40 > 1.00)) AND (cs_net_paid#39 > 0.00)) AND (cs_quantity#38 > 0)) AND isnotnull(cs_order_number#37)) AND isnotnull(cs_item_sk#36)) + +(26) CometProject +Input [6]: [cs_item_sk#36, cs_order_number#37, cs_quantity#38, cs_net_paid#39, cs_net_profit#40, cs_sold_date_sk#41] +Arguments: [cs_item_sk#36, cs_order_number#37, cs_quantity#38, cs_net_paid#39, cs_sold_date_sk#41], [cs_item_sk#36, cs_order_number#37, cs_quantity#38, cs_net_paid#39, cs_sold_date_sk#41] + +(27) CometBroadcastExchange +Input [5]: [cs_item_sk#36, cs_order_number#37, cs_quantity#38, cs_net_paid#39, cs_sold_date_sk#41] +Arguments: [cs_item_sk#36, cs_order_number#37, cs_quantity#38, cs_net_paid#39, cs_sold_date_sk#41] + +(28) Scan parquet spark_catalog.default.catalog_returns +Output [5]: [cr_item_sk#43, cr_order_number#44, cr_return_quantity#45, cr_return_amount#46, cr_returned_date_sk#47] +Batched: true +Location [not included in comparison]/{warehouse_dir}/catalog_returns] +PushedFilters: [IsNotNull(cr_return_amount), GreaterThan(cr_return_amount,10000.00), IsNotNull(cr_order_number), IsNotNull(cr_item_sk)] +ReadSchema: struct<cr_item_sk:int,cr_order_number:int,cr_return_quantity:int,cr_return_amount:decimal(7,2)> + +(29) CometFilter +Input [5]: [cr_item_sk#43, cr_order_number#44, cr_return_quantity#45, cr_return_amount#46, cr_returned_date_sk#47] +Condition : (((isnotnull(cr_return_amount#46) AND (cr_return_amount#46 > 10000.00)) AND isnotnull(cr_order_number#44)) AND isnotnull(cr_item_sk#43)) + +(30) CometProject +Input [5]: [cr_item_sk#43, cr_order_number#44, cr_return_quantity#45, cr_return_amount#46, cr_returned_date_sk#47] +Arguments: [cr_item_sk#43, cr_order_number#44, cr_return_quantity#45, cr_return_amount#46], [cr_item_sk#43, cr_order_number#44, cr_return_quantity#45, cr_return_amount#46] + +(31) CometBroadcastHashJoin +Left output [5]: [cs_item_sk#36, cs_order_number#37, cs_quantity#38, cs_net_paid#39, cs_sold_date_sk#41] +Right output [4]: [cr_item_sk#43, cr_order_number#44, cr_return_quantity#45, cr_return_amount#46] +Arguments: [cs_order_number#37, cs_item_sk#36], [cr_order_number#44, cr_item_sk#43], Inner + +(32) CometProject +Input [9]: [cs_item_sk#36, cs_order_number#37, cs_quantity#38, cs_net_paid#39, cs_sold_date_sk#41, cr_item_sk#43, cr_order_number#44, cr_return_quantity#45, cr_return_amount#46] +Arguments: [cs_item_sk#36, cs_quantity#38, cs_net_paid#39, cs_sold_date_sk#41, cr_return_quantity#45, cr_return_amount#46], [cs_item_sk#36, cs_quantity#38, cs_net_paid#39, cs_sold_date_sk#41, cr_return_quantity#45, cr_return_amount#46] + +(33) ColumnarToRow [codegen id : 8] +Input [6]: [cs_item_sk#36, cs_quantity#38, cs_net_paid#39, cs_sold_date_sk#41, cr_return_quantity#45, cr_return_amount#46] + +(34) ReusedExchange [Reuses operator id: 79] +Output [1]: [d_date_sk#48] + +(35) BroadcastHashJoin [codegen id : 8] +Left keys [1]: [cs_sold_date_sk#41] +Right keys [1]: [d_date_sk#48] +Join type: Inner +Join condition: None + +(36) Project [codegen id : 8] +Output [5]: [cs_item_sk#36, cs_quantity#38, cs_net_paid#39, cr_return_quantity#45, cr_return_amount#46] +Input [7]: [cs_item_sk#36, cs_quantity#38, cs_net_paid#39, cs_sold_date_sk#41, cr_return_quantity#45, cr_return_amount#46, d_date_sk#48] + +(37) HashAggregate [codegen id : 8] +Input [5]: [cs_item_sk#36, cs_quantity#38, cs_net_paid#39, cr_return_quantity#45, cr_return_amount#46] +Keys [1]: [cs_item_sk#36] +Functions [4]: [partial_sum(coalesce(cr_return_quantity#45, 0)), partial_sum(coalesce(cs_quantity#38, 0)), partial_sum(coalesce(cast(cr_return_amount#46 as decimal(12,2)), 0.00)), partial_sum(coalesce(cast(cs_net_paid#39 as decimal(12,2)), 0.00))] +Aggregate Attributes [6]: [sum#49, sum#50, sum#51, isEmpty#52, sum#53, isEmpty#54] +Results [7]: [cs_item_sk#36, sum#55, sum#56, sum#57, isEmpty#58, sum#59, isEmpty#60] + +(38) Exchange +Input [7]: [cs_item_sk#36, sum#55, sum#56, sum#57, isEmpty#58, sum#59, isEmpty#60] +Arguments: hashpartitioning(cs_item_sk#36, 5), ENSURE_REQUIREMENTS, [plan_id=3] + +(39) HashAggregate [codegen id : 9] +Input [7]: [cs_item_sk#36, sum#55, sum#56, sum#57, isEmpty#58, sum#59, isEmpty#60] +Keys [1]: [cs_item_sk#36] +Functions [4]: [sum(coalesce(cr_return_quantity#45, 0)), sum(coalesce(cs_quantity#38, 0)), sum(coalesce(cast(cr_return_amount#46 as decimal(12,2)), 0.00)), sum(coalesce(cast(cs_net_paid#39 as decimal(12,2)), 0.00))] +Aggregate Attributes [4]: [sum(coalesce(cr_return_quantity#45, 0))#61, sum(coalesce(cs_quantity#38, 0))#62, sum(coalesce(cast(cr_return_amount#46 as decimal(12,2)), 0.00))#63, sum(coalesce(cast(cs_net_paid#39 as decimal(12,2)), 0.00))#64] +Results [3]: [cs_item_sk#36 AS item#65, (cast(sum(coalesce(cr_return_quantity#45, 0))#61 as decimal(15,4)) / cast(sum(coalesce(cs_quantity#38, 0))#62 as decimal(15,4))) AS return_ratio#66, (cast(sum(coalesce(cast(cr_return_amount#46 as decimal(12,2)), 0.00))#63 as decimal(15,4)) / cast(sum(coalesce(cast(cs_net_paid#39 as decimal(12,2)), 0.00))#64 as decimal(15,4))) AS currency_ratio#67] + +(40) Exchange +Input [3]: [item#65, return_ratio#66, currency_ratio#67] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=4] + +(41) Sort [codegen id : 10] +Input [3]: [item#65, return_ratio#66, currency_ratio#67] +Arguments: [return_ratio#66 ASC NULLS FIRST], false, 0 + +(42) Window +Input [3]: [item#65, return_ratio#66, currency_ratio#67] +Arguments: [rank(return_ratio#66) windowspecdefinition(return_ratio#66 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS return_rank#68], [return_ratio#66 ASC NULLS FIRST] + +(43) Sort [codegen id : 11] +Input [4]: [item#65, return_ratio#66, currency_ratio#67, return_rank#68] +Arguments: [currency_ratio#67 ASC NULLS FIRST], false, 0 + +(44) Window +Input [4]: [item#65, return_ratio#66, currency_ratio#67, return_rank#68] +Arguments: [rank(currency_ratio#67) windowspecdefinition(currency_ratio#67 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS currency_rank#69], [currency_ratio#67 ASC NULLS FIRST] + +(45) Filter [codegen id : 12] +Input [5]: [item#65, return_ratio#66, currency_ratio#67, return_rank#68, currency_rank#69] +Condition : ((return_rank#68 <= 10) OR (currency_rank#69 <= 10)) + +(46) Project [codegen id : 12] +Output [5]: [catalog AS channel#70, item#65, return_ratio#66, return_rank#68, currency_rank#69] +Input [5]: [item#65, return_ratio#66, currency_ratio#67, return_rank#68, currency_rank#69] + +(47) Scan parquet spark_catalog.default.store_sales +Output [6]: [ss_item_sk#71, ss_ticket_number#72, ss_quantity#73, ss_net_paid#74, ss_net_profit#75, ss_sold_date_sk#76] +Batched: true +Location: InMemoryFileIndex [] +PartitionFilters: [isnotnull(ss_sold_date_sk#76), dynamicpruningexpression(ss_sold_date_sk#76 IN dynamicpruning#77)] +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<ss_item_sk:int,ss_ticket_number:int,ss_quantity:int,ss_net_paid:decimal(7,2),ss_net_profit:decimal(7,2)> + +(48) CometFilter +Input [6]: [ss_item_sk#71, ss_ticket_number#72, ss_quantity#73, ss_net_paid#74, ss_net_profit#75, ss_sold_date_sk#76] +Condition : (((((((isnotnull(ss_net_profit#75) AND isnotnull(ss_net_paid#74)) AND isnotnull(ss_quantity#73)) AND (ss_net_profit#75 > 1.00)) AND (ss_net_paid#74 > 0.00)) AND (ss_quantity#73 > 0)) AND isnotnull(ss_ticket_number#72)) AND isnotnull(ss_item_sk#71)) + +(49) CometProject +Input [6]: [ss_item_sk#71, ss_ticket_number#72, ss_quantity#73, ss_net_paid#74, ss_net_profit#75, ss_sold_date_sk#76] +Arguments: [ss_item_sk#71, ss_ticket_number#72, ss_quantity#73, ss_net_paid#74, ss_sold_date_sk#76], [ss_item_sk#71, ss_ticket_number#72, ss_quantity#73, ss_net_paid#74, ss_sold_date_sk#76] + +(50) CometBroadcastExchange +Input [5]: [ss_item_sk#71, ss_ticket_number#72, ss_quantity#73, ss_net_paid#74, ss_sold_date_sk#76] +Arguments: [ss_item_sk#71, ss_ticket_number#72, ss_quantity#73, ss_net_paid#74, ss_sold_date_sk#76] + +(51) Scan parquet spark_catalog.default.store_returns +Output [5]: [sr_item_sk#78, sr_ticket_number#79, sr_return_quantity#80, sr_return_amt#81, sr_returned_date_sk#82] +Batched: true +Location [not included in comparison]/{warehouse_dir}/store_returns] +PushedFilters: [IsNotNull(sr_return_amt), GreaterThan(sr_return_amt,10000.00), IsNotNull(sr_ticket_number), IsNotNull(sr_item_sk)] +ReadSchema: struct<sr_item_sk:int,sr_ticket_number:int,sr_return_quantity:int,sr_return_amt:decimal(7,2)> + +(52) CometFilter +Input [5]: [sr_item_sk#78, sr_ticket_number#79, sr_return_quantity#80, sr_return_amt#81, sr_returned_date_sk#82] +Condition : (((isnotnull(sr_return_amt#81) AND (sr_return_amt#81 > 10000.00)) AND isnotnull(sr_ticket_number#79)) AND isnotnull(sr_item_sk#78)) + +(53) CometProject +Input [5]: [sr_item_sk#78, sr_ticket_number#79, sr_return_quantity#80, sr_return_amt#81, sr_returned_date_sk#82] +Arguments: [sr_item_sk#78, sr_ticket_number#79, sr_return_quantity#80, sr_return_amt#81], [sr_item_sk#78, sr_ticket_number#79, sr_return_quantity#80, sr_return_amt#81] + +(54) CometBroadcastHashJoin +Left output [5]: [ss_item_sk#71, ss_ticket_number#72, ss_quantity#73, ss_net_paid#74, ss_sold_date_sk#76] +Right output [4]: [sr_item_sk#78, sr_ticket_number#79, sr_return_quantity#80, sr_return_amt#81] +Arguments: [ss_ticket_number#72, ss_item_sk#71], [sr_ticket_number#79, sr_item_sk#78], Inner + +(55) CometProject +Input [9]: [ss_item_sk#71, ss_ticket_number#72, ss_quantity#73, ss_net_paid#74, ss_sold_date_sk#76, sr_item_sk#78, sr_ticket_number#79, sr_return_quantity#80, sr_return_amt#81] +Arguments: [ss_item_sk#71, ss_quantity#73, ss_net_paid#74, ss_sold_date_sk#76, sr_return_quantity#80, sr_return_amt#81], [ss_item_sk#71, ss_quantity#73, ss_net_paid#74, ss_sold_date_sk#76, sr_return_quantity#80, sr_return_amt#81] + +(56) ColumnarToRow [codegen id : 14] +Input [6]: [ss_item_sk#71, ss_quantity#73, ss_net_paid#74, ss_sold_date_sk#76, sr_return_quantity#80, sr_return_amt#81] + +(57) ReusedExchange [Reuses operator id: 79] +Output [1]: [d_date_sk#83] + +(58) BroadcastHashJoin [codegen id : 14] +Left keys [1]: [ss_sold_date_sk#76] +Right keys [1]: [d_date_sk#83] +Join type: Inner +Join condition: None + +(59) Project [codegen id : 14] +Output [5]: [ss_item_sk#71, ss_quantity#73, ss_net_paid#74, sr_return_quantity#80, sr_return_amt#81] +Input [7]: [ss_item_sk#71, ss_quantity#73, ss_net_paid#74, ss_sold_date_sk#76, sr_return_quantity#80, sr_return_amt#81, d_date_sk#83] + +(60) HashAggregate [codegen id : 14] +Input [5]: [ss_item_sk#71, ss_quantity#73, ss_net_paid#74, sr_return_quantity#80, sr_return_amt#81] +Keys [1]: [ss_item_sk#71] +Functions [4]: [partial_sum(coalesce(sr_return_quantity#80, 0)), partial_sum(coalesce(ss_quantity#73, 0)), partial_sum(coalesce(cast(sr_return_amt#81 as decimal(12,2)), 0.00)), partial_sum(coalesce(cast(ss_net_paid#74 as decimal(12,2)), 0.00))] +Aggregate Attributes [6]: [sum#84, sum#85, sum#86, isEmpty#87, sum#88, isEmpty#89] +Results [7]: [ss_item_sk#71, sum#90, sum#91, sum#92, isEmpty#93, sum#94, isEmpty#95] + +(61) Exchange +Input [7]: [ss_item_sk#71, sum#90, sum#91, sum#92, isEmpty#93, sum#94, isEmpty#95] +Arguments: hashpartitioning(ss_item_sk#71, 5), ENSURE_REQUIREMENTS, [plan_id=5] + +(62) HashAggregate [codegen id : 15] +Input [7]: [ss_item_sk#71, sum#90, sum#91, sum#92, isEmpty#93, sum#94, isEmpty#95] +Keys [1]: [ss_item_sk#71] +Functions [4]: [sum(coalesce(sr_return_quantity#80, 0)), sum(coalesce(ss_quantity#73, 0)), sum(coalesce(cast(sr_return_amt#81 as decimal(12,2)), 0.00)), sum(coalesce(cast(ss_net_paid#74 as decimal(12,2)), 0.00))] +Aggregate Attributes [4]: [sum(coalesce(sr_return_quantity#80, 0))#96, sum(coalesce(ss_quantity#73, 0))#97, sum(coalesce(cast(sr_return_amt#81 as decimal(12,2)), 0.00))#98, sum(coalesce(cast(ss_net_paid#74 as decimal(12,2)), 0.00))#99] +Results [3]: [ss_item_sk#71 AS item#100, (cast(sum(coalesce(sr_return_quantity#80, 0))#96 as decimal(15,4)) / cast(sum(coalesce(ss_quantity#73, 0))#97 as decimal(15,4))) AS return_ratio#101, (cast(sum(coalesce(cast(sr_return_amt#81 as decimal(12,2)), 0.00))#98 as decimal(15,4)) / cast(sum(coalesce(cast(ss_net_paid#74 as decimal(12,2)), 0.00))#99 as decimal(15,4))) AS currency_ratio#102] + +(63) Exchange +Input [3]: [item#100, return_ratio#101, currency_ratio#102] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=6] + +(64) Sort [codegen id : 16] +Input [3]: [item#100, return_ratio#101, currency_ratio#102] +Arguments: [return_ratio#101 ASC NULLS FIRST], false, 0 + +(65) Window +Input [3]: [item#100, return_ratio#101, currency_ratio#102] +Arguments: [rank(return_ratio#101) windowspecdefinition(return_ratio#101 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS return_rank#103], [return_ratio#101 ASC NULLS FIRST] + +(66) Sort [codegen id : 17] +Input [4]: [item#100, return_ratio#101, currency_ratio#102, return_rank#103] +Arguments: [currency_ratio#102 ASC NULLS FIRST], false, 0 + +(67) Window +Input [4]: [item#100, return_ratio#101, currency_ratio#102, return_rank#103] +Arguments: [rank(currency_ratio#102) windowspecdefinition(currency_ratio#102 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS currency_rank#104], [currency_ratio#102 ASC NULLS FIRST] + +(68) Filter [codegen id : 18] +Input [5]: [item#100, return_ratio#101, currency_ratio#102, return_rank#103, currency_rank#104] +Condition : ((return_rank#103 <= 10) OR (currency_rank#104 <= 10)) + +(69) Project [codegen id : 18] +Output [5]: [store AS channel#105, item#100, return_ratio#101, return_rank#103, currency_rank#104] +Input [5]: [item#100, return_ratio#101, currency_ratio#102, return_rank#103, currency_rank#104] + +(70) Union + +(71) HashAggregate [codegen id : 19] +Input [5]: [channel#35, item#30, return_ratio#31, return_rank#33, currency_rank#34] +Keys [5]: [channel#35, item#30, return_ratio#31, return_rank#33, currency_rank#34] +Functions: [] +Aggregate Attributes: [] +Results [5]: [channel#35, item#30, return_ratio#31, return_rank#33, currency_rank#34] + +(72) Exchange +Input [5]: [channel#35, item#30, return_ratio#31, return_rank#33, currency_rank#34] +Arguments: hashpartitioning(channel#35, item#30, return_ratio#31, return_rank#33, currency_rank#34, 5), ENSURE_REQUIREMENTS, [plan_id=7] + +(73) HashAggregate [codegen id : 20] +Input [5]: [channel#35, item#30, return_ratio#31, return_rank#33, currency_rank#34] +Keys [5]: [channel#35, item#30, return_ratio#31, return_rank#33, currency_rank#34] +Functions: [] +Aggregate Attributes: [] +Results [5]: [channel#35, item#30, return_ratio#31, return_rank#33, currency_rank#34] + +(74) TakeOrderedAndProject +Input [5]: [channel#35, item#30, return_ratio#31, return_rank#33, currency_rank#34] +Arguments: 100, [channel#35 ASC NULLS FIRST, return_rank#33 ASC NULLS FIRST, currency_rank#34 ASC NULLS FIRST, item#30 ASC NULLS FIRST], [channel#35, item#30, return_ratio#31, return_rank#33, currency_rank#34] + +===== Subqueries ===== + +Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#6 IN dynamicpruning#7 +BroadcastExchange (79) ++- * ColumnarToRow (78) + +- CometProject (77) + +- CometFilter (76) + +- CometScan parquet spark_catalog.default.date_dim (75) + + +(75) Scan parquet spark_catalog.default.date_dim +Output [3]: [d_date_sk#13, d_year#106, d_moy#107] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2001), EqualTo(d_moy,12), IsNotNull(d_date_sk)] +ReadSchema: struct<d_date_sk:int,d_year:int,d_moy:int> + +(76) CometFilter +Input [3]: [d_date_sk#13, d_year#106, d_moy#107] +Condition : ((((isnotnull(d_year#106) AND isnotnull(d_moy#107)) AND (d_year#106 = 2001)) AND (d_moy#107 = 12)) AND isnotnull(d_date_sk#13)) + +(77) CometProject +Input [3]: [d_date_sk#13, d_year#106, d_moy#107] +Arguments: [d_date_sk#13], [d_date_sk#13] + +(78) ColumnarToRow [codegen id : 1] +Input [1]: [d_date_sk#13] + +(79) BroadcastExchange +Input [1]: [d_date_sk#13] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=8] + +Subquery:2 Hosting operator id = 24 Hosting Expression = cs_sold_date_sk#41 IN dynamicpruning#7 + +Subquery:3 Hosting operator id = 47 Hosting Expression = ss_sold_date_sk#76 IN dynamicpruning#7 + + diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q49/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q49/simplified.txt new file mode 100644 index 000000000..43ebf34cc --- /dev/null +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q49/simplified.txt @@ -0,0 +1,121 @@ +TakeOrderedAndProject [channel,return_rank,currency_rank,item,return_ratio] + WholeStageCodegen (20) + HashAggregate [channel,item,return_ratio,return_rank,currency_rank] + InputAdapter + Exchange [channel,item,return_ratio,return_rank,currency_rank] #1 + WholeStageCodegen (19) + HashAggregate [channel,item,return_ratio,return_rank,currency_rank] + InputAdapter + Union + WholeStageCodegen (6) + Project [item,return_ratio,return_rank,currency_rank] + Filter [return_rank,currency_rank] + InputAdapter + Window [currency_ratio] + WholeStageCodegen (5) + Sort [currency_ratio] + InputAdapter + Window [return_ratio] + WholeStageCodegen (4) + Sort [return_ratio] + InputAdapter + Exchange #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 + Exchange [ws_item_sk] #3 + WholeStageCodegen (2) + HashAggregate [ws_item_sk,wr_return_quantity,ws_quantity,wr_return_amt,ws_net_paid] [sum,sum,sum,isEmpty,sum,isEmpty,sum,sum,sum,isEmpty,sum,isEmpty] + Project [ws_item_sk,ws_quantity,ws_net_paid,wr_return_quantity,wr_return_amt] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + ColumnarToRow + InputAdapter + CometProject [ws_item_sk,ws_quantity,ws_net_paid,ws_sold_date_sk,wr_return_quantity,wr_return_amt] + CometBroadcastHashJoin [ws_order_number,ws_item_sk,wr_order_number,wr_item_sk] + CometBroadcastExchange #4 + CometProject [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_sold_date_sk] + CometFilter [ws_net_profit,ws_net_paid,ws_quantity,ws_order_number,ws_item_sk] + CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_net_profit,ws_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #5 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_year,d_moy,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometProject [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] + CometFilter [wr_return_amt,wr_order_number,wr_item_sk] + CometScan parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk] + InputAdapter + ReusedExchange [d_date_sk] #5 + WholeStageCodegen (12) + Project [item,return_ratio,return_rank,currency_rank] + Filter [return_rank,currency_rank] + InputAdapter + Window [currency_ratio] + WholeStageCodegen (11) + Sort [currency_ratio] + InputAdapter + Window [return_ratio] + WholeStageCodegen (10) + Sort [return_ratio] + InputAdapter + Exchange #6 + WholeStageCodegen (9) + HashAggregate [cs_item_sk,sum,sum,sum,isEmpty,sum,isEmpty] [sum(coalesce(cr_return_quantity, 0)),sum(coalesce(cs_quantity, 0)),sum(coalesce(cast(cr_return_amount as decimal(12,2)), 0.00)),sum(coalesce(cast(cs_net_paid as decimal(12,2)), 0.00)),item,return_ratio,currency_ratio,sum,sum,sum,isEmpty,sum,isEmpty] + InputAdapter + Exchange [cs_item_sk] #7 + WholeStageCodegen (8) + HashAggregate [cs_item_sk,cr_return_quantity,cs_quantity,cr_return_amount,cs_net_paid] [sum,sum,sum,isEmpty,sum,isEmpty,sum,sum,sum,isEmpty,sum,isEmpty] + Project [cs_item_sk,cs_quantity,cs_net_paid,cr_return_quantity,cr_return_amount] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + ColumnarToRow + InputAdapter + CometProject [cs_item_sk,cs_quantity,cs_net_paid,cs_sold_date_sk,cr_return_quantity,cr_return_amount] + CometBroadcastHashJoin [cs_order_number,cs_item_sk,cr_order_number,cr_item_sk] + CometBroadcastExchange #8 + CometProject [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_sold_date_sk] + CometFilter [cs_net_profit,cs_net_paid,cs_quantity,cs_order_number,cs_item_sk] + CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_net_profit,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + CometProject [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] + CometFilter [cr_return_amount,cr_order_number,cr_item_sk] + CometScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount,cr_returned_date_sk] + InputAdapter + ReusedExchange [d_date_sk] #5 + WholeStageCodegen (18) + Project [item,return_ratio,return_rank,currency_rank] + Filter [return_rank,currency_rank] + InputAdapter + Window [currency_ratio] + WholeStageCodegen (17) + Sort [currency_ratio] + InputAdapter + Window [return_ratio] + WholeStageCodegen (16) + Sort [return_ratio] + InputAdapter + Exchange #9 + WholeStageCodegen (15) + HashAggregate [ss_item_sk,sum,sum,sum,isEmpty,sum,isEmpty] [sum(coalesce(sr_return_quantity, 0)),sum(coalesce(ss_quantity, 0)),sum(coalesce(cast(sr_return_amt as decimal(12,2)), 0.00)),sum(coalesce(cast(ss_net_paid as decimal(12,2)), 0.00)),item,return_ratio,currency_ratio,sum,sum,sum,isEmpty,sum,isEmpty] + InputAdapter + Exchange [ss_item_sk] #10 + WholeStageCodegen (14) + HashAggregate [ss_item_sk,sr_return_quantity,ss_quantity,sr_return_amt,ss_net_paid] [sum,sum,sum,isEmpty,sum,isEmpty,sum,sum,sum,isEmpty,sum,isEmpty] + Project [ss_item_sk,ss_quantity,ss_net_paid,sr_return_quantity,sr_return_amt] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + ColumnarToRow + InputAdapter + CometProject [ss_item_sk,ss_quantity,ss_net_paid,ss_sold_date_sk,sr_return_quantity,sr_return_amt] + CometBroadcastHashJoin [ss_ticket_number,ss_item_sk,sr_ticket_number,sr_item_sk] + CometBroadcastExchange #11 + CometProject [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_sold_date_sk] + CometFilter [ss_net_profit,ss_net_paid,ss_quantity,ss_ticket_number,ss_item_sk] + CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_net_profit,ss_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + CometProject [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] + CometFilter [sr_return_amt,sr_ticket_number,sr_item_sk] + CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt,sr_returned_date_sk] + InputAdapter + ReusedExchange [d_date_sk] #5 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 new file mode 100644 index 000000000..3892f250a --- /dev/null +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q51a/explain.txt @@ -0,0 +1,416 @@ +== Physical Plan == +TakeOrderedAndProject (67) ++- * Filter (66) + +- * HashAggregate (65) + +- * HashAggregate (64) + +- * Project (63) + +- * BroadcastHashJoin Inner BuildRight (62) + :- Window (56) + : +- * Sort (55) + : +- Exchange (54) + : +- * Project (53) + : +- * Filter (52) + : +- * SortMergeJoin FullOuter (51) + : :- * Sort (25) + : : +- Exchange (24) + : : +- * HashAggregate (23) + : : +- Exchange (22) + : : +- * HashAggregate (21) + : : +- * Project (20) + : : +- * BroadcastHashJoin Inner BuildRight (19) + : : :- * Project (13) + : : : +- Window (12) + : : : +- * Sort (11) + : : : +- Exchange (10) + : : : +- * HashAggregate (9) + : : : +- Exchange (8) + : : : +- * HashAggregate (7) + : : : +- * Project (6) + : : : +- * BroadcastHashJoin Inner BuildRight (5) + : : : :- * ColumnarToRow (3) + : : : : +- CometFilter (2) + : : : : +- CometScan parquet spark_catalog.default.web_sales (1) + : : : +- ReusedExchange (4) + : : +- BroadcastExchange (18) + : : +- * Project (17) + : : +- Window (16) + : : +- * Sort (15) + : : +- ReusedExchange (14) + : +- * Sort (50) + : +- Exchange (49) + : +- * HashAggregate (48) + : +- Exchange (47) + : +- * HashAggregate (46) + : +- * Project (45) + : +- * BroadcastHashJoin Inner BuildRight (44) + : :- * Project (38) + : : +- Window (37) + : : +- * Sort (36) + : : +- Exchange (35) + : : +- * HashAggregate (34) + : : +- Exchange (33) + : : +- * HashAggregate (32) + : : +- * Project (31) + : : +- * BroadcastHashJoin Inner BuildRight (30) + : : :- * ColumnarToRow (28) + : : : +- CometFilter (27) + : : : +- CometScan parquet spark_catalog.default.store_sales (26) + : : +- ReusedExchange (29) + : +- BroadcastExchange (43) + : +- * Project (42) + : +- Window (41) + : +- * Sort (40) + : +- ReusedExchange (39) + +- BroadcastExchange (61) + +- * Project (60) + +- Window (59) + +- * Sort (58) + +- ReusedExchange (57) + + +(1) Scan parquet spark_catalog.default.web_sales +Output [3]: [ws_item_sk#1, ws_sales_price#2, ws_sold_date_sk#3] +Batched: true +Location: InMemoryFileIndex [] +PartitionFilters: [isnotnull(ws_sold_date_sk#3), dynamicpruningexpression(ws_sold_date_sk#3 IN dynamicpruning#4)] +PushedFilters: [IsNotNull(ws_item_sk)] +ReadSchema: struct<ws_item_sk:int,ws_sales_price:decimal(7,2)> + +(2) CometFilter +Input [3]: [ws_item_sk#1, ws_sales_price#2, ws_sold_date_sk#3] +Condition : isnotnull(ws_item_sk#1) + +(3) ColumnarToRow [codegen id : 2] +Input [3]: [ws_item_sk#1, ws_sales_price#2, ws_sold_date_sk#3] + +(4) ReusedExchange [Reuses operator id: 72] +Output [2]: [d_date_sk#5, d_date#6] + +(5) BroadcastHashJoin [codegen id : 2] +Left keys [1]: [ws_sold_date_sk#3] +Right keys [1]: [d_date_sk#5] +Join type: Inner +Join condition: None + +(6) Project [codegen id : 2] +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] +Input [3]: [ws_item_sk#1, ws_sales_price#2, d_date#6] +Keys [2]: [ws_item_sk#1, d_date#6] +Functions [1]: [partial_sum(UnscaledValue(ws_sales_price#2))] +Aggregate Attributes [1]: [sum#7] +Results [3]: [ws_item_sk#1, d_date#6, sum#8] + +(8) Exchange +Input [3]: [ws_item_sk#1, d_date#6, sum#8] +Arguments: hashpartitioning(ws_item_sk#1, d_date#6, 5), ENSURE_REQUIREMENTS, [plan_id=1] + +(9) HashAggregate [codegen id : 3] +Input [3]: [ws_item_sk#1, d_date#6, sum#8] +Keys [2]: [ws_item_sk#1, d_date#6] +Functions [1]: [sum(UnscaledValue(ws_sales_price#2))] +Aggregate Attributes [1]: [sum(UnscaledValue(ws_sales_price#2))#9] +Results [4]: [ws_item_sk#1 AS item_sk#10, d_date#6, MakeDecimal(sum(UnscaledValue(ws_sales_price#2))#9,17,2) AS sumws#11, ws_item_sk#1] + +(10) Exchange +Input [4]: [item_sk#10, d_date#6, sumws#11, ws_item_sk#1] +Arguments: hashpartitioning(ws_item_sk#1, 5), ENSURE_REQUIREMENTS, [plan_id=2] + +(11) Sort [codegen id : 4] +Input [4]: [item_sk#10, d_date#6, sumws#11, ws_item_sk#1] +Arguments: [ws_item_sk#1 ASC NULLS FIRST, d_date#6 ASC NULLS FIRST], false, 0 + +(12) Window +Input [4]: [item_sk#10, d_date#6, sumws#11, ws_item_sk#1] +Arguments: [row_number() windowspecdefinition(ws_item_sk#1, d_date#6 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#12], [ws_item_sk#1], [d_date#6 ASC NULLS FIRST] + +(13) Project [codegen id : 10] +Output [4]: [item_sk#10, d_date#6, sumws#11, rk#12] +Input [5]: [item_sk#10, d_date#6, sumws#11, ws_item_sk#1, rk#12] + +(14) ReusedExchange [Reuses operator id: 10] +Output [4]: [item_sk#13, d_date#14, sumws#15, ws_item_sk#16] + +(15) Sort [codegen id : 8] +Input [4]: [item_sk#13, d_date#14, sumws#15, ws_item_sk#16] +Arguments: [ws_item_sk#16 ASC NULLS FIRST, d_date#14 ASC NULLS FIRST], false, 0 + +(16) Window +Input [4]: [item_sk#13, d_date#14, sumws#15, ws_item_sk#16] +Arguments: [row_number() windowspecdefinition(ws_item_sk#16, d_date#14 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#17], [ws_item_sk#16], [d_date#14 ASC NULLS FIRST] + +(17) Project [codegen id : 9] +Output [3]: [item_sk#13, sumws#15, rk#17] +Input [5]: [item_sk#13, d_date#14, sumws#15, ws_item_sk#16, rk#17] + +(18) BroadcastExchange +Input [3]: [item_sk#13, sumws#15, rk#17] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] + +(19) BroadcastHashJoin [codegen id : 10] +Left keys [1]: [item_sk#10] +Right keys [1]: [item_sk#13] +Join type: Inner +Join condition: (rk#12 >= rk#17) + +(20) Project [codegen id : 10] +Output [4]: [item_sk#10, d_date#6, sumws#11, sumws#15] +Input [7]: [item_sk#10, d_date#6, sumws#11, rk#12, item_sk#13, sumws#15, rk#17] + +(21) HashAggregate [codegen id : 10] +Input [4]: [item_sk#10, d_date#6, sumws#11, sumws#15] +Keys [3]: [item_sk#10, d_date#6, sumws#11] +Functions [1]: [partial_sum(sumws#15)] +Aggregate Attributes [2]: [sum#18, isEmpty#19] +Results [5]: [item_sk#10, d_date#6, sumws#11, sum#20, isEmpty#21] + +(22) Exchange +Input [5]: [item_sk#10, d_date#6, sumws#11, sum#20, isEmpty#21] +Arguments: hashpartitioning(item_sk#10, d_date#6, sumws#11, 5), ENSURE_REQUIREMENTS, [plan_id=4] + +(23) HashAggregate [codegen id : 11] +Input [5]: [item_sk#10, d_date#6, sumws#11, sum#20, isEmpty#21] +Keys [3]: [item_sk#10, d_date#6, sumws#11] +Functions [1]: [sum(sumws#15)] +Aggregate Attributes [1]: [sum(sumws#15)#22] +Results [3]: [item_sk#10, d_date#6, sum(sumws#15)#22 AS cume_sales#23] + +(24) Exchange +Input [3]: [item_sk#10, d_date#6, cume_sales#23] +Arguments: hashpartitioning(item_sk#10, d_date#6, 5), ENSURE_REQUIREMENTS, [plan_id=5] + +(25) Sort [codegen id : 12] +Input [3]: [item_sk#10, d_date#6, cume_sales#23] +Arguments: [item_sk#10 ASC NULLS FIRST, d_date#6 ASC NULLS FIRST], false, 0 + +(26) Scan parquet spark_catalog.default.store_sales +Output [3]: [ss_item_sk#24, ss_sales_price#25, ss_sold_date_sk#26] +Batched: true +Location: InMemoryFileIndex [] +PartitionFilters: [isnotnull(ss_sold_date_sk#26), dynamicpruningexpression(ss_sold_date_sk#26 IN dynamicpruning#27)] +PushedFilters: [IsNotNull(ss_item_sk)] +ReadSchema: struct<ss_item_sk:int,ss_sales_price:decimal(7,2)> + +(27) CometFilter +Input [3]: [ss_item_sk#24, ss_sales_price#25, ss_sold_date_sk#26] +Condition : isnotnull(ss_item_sk#24) + +(28) ColumnarToRow [codegen id : 14] +Input [3]: [ss_item_sk#24, ss_sales_price#25, ss_sold_date_sk#26] + +(29) ReusedExchange [Reuses operator id: 72] +Output [2]: [d_date_sk#28, d_date#29] + +(30) BroadcastHashJoin [codegen id : 14] +Left keys [1]: [ss_sold_date_sk#26] +Right keys [1]: [d_date_sk#28] +Join type: Inner +Join condition: None + +(31) Project [codegen id : 14] +Output [3]: [ss_item_sk#24, ss_sales_price#25, d_date#29] +Input [5]: [ss_item_sk#24, ss_sales_price#25, ss_sold_date_sk#26, d_date_sk#28, d_date#29] + +(32) HashAggregate [codegen id : 14] +Input [3]: [ss_item_sk#24, ss_sales_price#25, d_date#29] +Keys [2]: [ss_item_sk#24, d_date#29] +Functions [1]: [partial_sum(UnscaledValue(ss_sales_price#25))] +Aggregate Attributes [1]: [sum#30] +Results [3]: [ss_item_sk#24, d_date#29, sum#31] + +(33) Exchange +Input [3]: [ss_item_sk#24, d_date#29, sum#31] +Arguments: hashpartitioning(ss_item_sk#24, d_date#29, 5), ENSURE_REQUIREMENTS, [plan_id=6] + +(34) HashAggregate [codegen id : 15] +Input [3]: [ss_item_sk#24, d_date#29, sum#31] +Keys [2]: [ss_item_sk#24, d_date#29] +Functions [1]: [sum(UnscaledValue(ss_sales_price#25))] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_sales_price#25))#32] +Results [4]: [ss_item_sk#24 AS item_sk#33, d_date#29, MakeDecimal(sum(UnscaledValue(ss_sales_price#25))#32,17,2) AS sumss#34, ss_item_sk#24] + +(35) Exchange +Input [4]: [item_sk#33, d_date#29, sumss#34, ss_item_sk#24] +Arguments: hashpartitioning(ss_item_sk#24, 5), ENSURE_REQUIREMENTS, [plan_id=7] + +(36) Sort [codegen id : 16] +Input [4]: [item_sk#33, d_date#29, sumss#34, ss_item_sk#24] +Arguments: [ss_item_sk#24 ASC NULLS FIRST, d_date#29 ASC NULLS FIRST], false, 0 + +(37) Window +Input [4]: [item_sk#33, d_date#29, sumss#34, ss_item_sk#24] +Arguments: [row_number() windowspecdefinition(ss_item_sk#24, d_date#29 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#35], [ss_item_sk#24], [d_date#29 ASC NULLS FIRST] + +(38) Project [codegen id : 22] +Output [4]: [item_sk#33, d_date#29, sumss#34, rk#35] +Input [5]: [item_sk#33, d_date#29, sumss#34, ss_item_sk#24, rk#35] + +(39) ReusedExchange [Reuses operator id: 35] +Output [4]: [item_sk#36, d_date#37, sumss#38, ss_item_sk#39] + +(40) Sort [codegen id : 20] +Input [4]: [item_sk#36, d_date#37, sumss#38, ss_item_sk#39] +Arguments: [ss_item_sk#39 ASC NULLS FIRST, d_date#37 ASC NULLS FIRST], false, 0 + +(41) Window +Input [4]: [item_sk#36, d_date#37, sumss#38, ss_item_sk#39] +Arguments: [row_number() windowspecdefinition(ss_item_sk#39, d_date#37 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#40], [ss_item_sk#39], [d_date#37 ASC NULLS FIRST] + +(42) Project [codegen id : 21] +Output [3]: [item_sk#36, sumss#38, rk#40] +Input [5]: [item_sk#36, d_date#37, sumss#38, ss_item_sk#39, rk#40] + +(43) BroadcastExchange +Input [3]: [item_sk#36, sumss#38, rk#40] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=8] + +(44) BroadcastHashJoin [codegen id : 22] +Left keys [1]: [item_sk#33] +Right keys [1]: [item_sk#36] +Join type: Inner +Join condition: (rk#35 >= rk#40) + +(45) Project [codegen id : 22] +Output [4]: [item_sk#33, d_date#29, sumss#34, sumss#38] +Input [7]: [item_sk#33, d_date#29, sumss#34, rk#35, item_sk#36, sumss#38, rk#40] + +(46) HashAggregate [codegen id : 22] +Input [4]: [item_sk#33, d_date#29, sumss#34, sumss#38] +Keys [3]: [item_sk#33, d_date#29, sumss#34] +Functions [1]: [partial_sum(sumss#38)] +Aggregate Attributes [2]: [sum#41, isEmpty#42] +Results [5]: [item_sk#33, d_date#29, sumss#34, sum#43, isEmpty#44] + +(47) Exchange +Input [5]: [item_sk#33, d_date#29, sumss#34, sum#43, isEmpty#44] +Arguments: hashpartitioning(item_sk#33, d_date#29, sumss#34, 5), ENSURE_REQUIREMENTS, [plan_id=9] + +(48) HashAggregate [codegen id : 23] +Input [5]: [item_sk#33, d_date#29, sumss#34, sum#43, isEmpty#44] +Keys [3]: [item_sk#33, d_date#29, sumss#34] +Functions [1]: [sum(sumss#38)] +Aggregate Attributes [1]: [sum(sumss#38)#45] +Results [3]: [item_sk#33, d_date#29, sum(sumss#38)#45 AS cume_sales#46] + +(49) Exchange +Input [3]: [item_sk#33, d_date#29, cume_sales#46] +Arguments: hashpartitioning(item_sk#33, d_date#29, 5), ENSURE_REQUIREMENTS, [plan_id=10] + +(50) Sort [codegen id : 24] +Input [3]: [item_sk#33, d_date#29, cume_sales#46] +Arguments: [item_sk#33 ASC NULLS FIRST, d_date#29 ASC NULLS FIRST], false, 0 + +(51) SortMergeJoin [codegen id : 25] +Left keys [2]: [item_sk#10, d_date#6] +Right keys [2]: [item_sk#33, d_date#29] +Join type: FullOuter +Join condition: None + +(52) Filter [codegen id : 25] +Input [6]: [item_sk#10, d_date#6, cume_sales#23, item_sk#33, d_date#29, cume_sales#46] +Condition : isnotnull(CASE WHEN isnotnull(item_sk#10) THEN item_sk#10 ELSE item_sk#33 END) + +(53) Project [codegen id : 25] +Output [4]: [CASE WHEN isnotnull(item_sk#10) THEN item_sk#10 ELSE item_sk#33 END AS item_sk#47, CASE WHEN isnotnull(d_date#6) THEN d_date#6 ELSE d_date#29 END AS d_date#48, cume_sales#23 AS web_sales#49, cume_sales#46 AS store_sales#50] +Input [6]: [item_sk#10, d_date#6, cume_sales#23, item_sk#33, d_date#29, cume_sales#46] + +(54) Exchange +Input [4]: [item_sk#47, d_date#48, web_sales#49, store_sales#50] +Arguments: hashpartitioning(item_sk#47, 5), ENSURE_REQUIREMENTS, [plan_id=11] + +(55) Sort [codegen id : 26] +Input [4]: [item_sk#47, d_date#48, web_sales#49, store_sales#50] +Arguments: [item_sk#47 ASC NULLS FIRST, d_date#48 ASC NULLS FIRST], false, 0 + +(56) Window +Input [4]: [item_sk#47, d_date#48, web_sales#49, store_sales#50] +Arguments: [row_number() windowspecdefinition(item_sk#47, d_date#48 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#51], [item_sk#47], [d_date#48 ASC NULLS FIRST] + +(57) ReusedExchange [Reuses operator id: 54] +Output [4]: [item_sk#52, d_date#53, web_sales#54, store_sales#55] + +(58) Sort [codegen id : 52] +Input [4]: [item_sk#52, d_date#53, web_sales#54, store_sales#55] +Arguments: [item_sk#52 ASC NULLS FIRST, d_date#53 ASC NULLS FIRST], false, 0 + +(59) Window +Input [4]: [item_sk#52, d_date#53, web_sales#54, store_sales#55] +Arguments: [row_number() windowspecdefinition(item_sk#52, d_date#53 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#56], [item_sk#52], [d_date#53 ASC NULLS FIRST] + +(60) Project [codegen id : 53] +Output [4]: [item_sk#52, web_sales#54, store_sales#55, rk#56] +Input [5]: [item_sk#52, d_date#53, web_sales#54, store_sales#55, rk#56] + +(61) BroadcastExchange +Input [4]: [item_sk#52, web_sales#54, store_sales#55, rk#56] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=12] + +(62) BroadcastHashJoin [codegen id : 54] +Left keys [1]: [item_sk#47] +Right keys [1]: [item_sk#52] +Join type: Inner +Join condition: (rk#51 >= rk#56) + +(63) Project [codegen id : 54] +Output [6]: [item_sk#47, d_date#48, web_sales#49, store_sales#50, web_sales#54, store_sales#55] +Input [9]: [item_sk#47, d_date#48, web_sales#49, store_sales#50, rk#51, item_sk#52, web_sales#54, store_sales#55, rk#56] + +(64) HashAggregate [codegen id : 54] +Input [6]: [item_sk#47, d_date#48, web_sales#49, store_sales#50, web_sales#54, store_sales#55] +Keys [4]: [item_sk#47, d_date#48, web_sales#49, store_sales#50] +Functions [2]: [partial_max(web_sales#54), partial_max(store_sales#55)] +Aggregate Attributes [2]: [max#57, max#58] +Results [6]: [item_sk#47, d_date#48, web_sales#49, store_sales#50, max#59, max#60] + +(65) HashAggregate [codegen id : 54] +Input [6]: [item_sk#47, d_date#48, web_sales#49, store_sales#50, max#59, max#60] +Keys [4]: [item_sk#47, d_date#48, web_sales#49, store_sales#50] +Functions [2]: [max(web_sales#54), max(store_sales#55)] +Aggregate Attributes [2]: [max(web_sales#54)#61, max(store_sales#55)#62] +Results [6]: [item_sk#47, d_date#48, web_sales#49, store_sales#50, max(web_sales#54)#61 AS web_cumulative#63, max(store_sales#55)#62 AS store_cumulative#64] + +(66) Filter [codegen id : 54] +Input [6]: [item_sk#47, d_date#48, web_sales#49, store_sales#50, web_cumulative#63, store_cumulative#64] +Condition : ((isnotnull(web_cumulative#63) AND isnotnull(store_cumulative#64)) AND (web_cumulative#63 > store_cumulative#64)) + +(67) TakeOrderedAndProject +Input [6]: [item_sk#47, d_date#48, web_sales#49, store_sales#50, web_cumulative#63, store_cumulative#64] +Arguments: 100, [item_sk#47 ASC NULLS FIRST, d_date#48 ASC NULLS FIRST], [item_sk#47, d_date#48, web_sales#49, store_sales#50, web_cumulative#63, store_cumulative#64] + +===== Subqueries ===== + +Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#3 IN dynamicpruning#4 +BroadcastExchange (72) ++- * ColumnarToRow (71) + +- CometProject (70) + +- CometFilter (69) + +- CometScan parquet spark_catalog.default.date_dim (68) + + +(68) Scan parquet spark_catalog.default.date_dim +Output [3]: [d_date_sk#5, d_date#6, d_month_seq#65] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1212), LessThanOrEqual(d_month_seq,1223), IsNotNull(d_date_sk)] +ReadSchema: struct<d_date_sk:int,d_date:date,d_month_seq:int> + +(69) CometFilter +Input [3]: [d_date_sk#5, d_date#6, d_month_seq#65] +Condition : (((isnotnull(d_month_seq#65) AND (d_month_seq#65 >= 1212)) AND (d_month_seq#65 <= 1223)) AND isnotnull(d_date_sk#5)) + +(70) CometProject +Input [3]: [d_date_sk#5, d_date#6, d_month_seq#65] +Arguments: [d_date_sk#5, d_date#6], [d_date_sk#5, d_date#6] + +(71) ColumnarToRow [codegen id : 1] +Input [2]: [d_date_sk#5, d_date#6] + +(72) BroadcastExchange +Input [2]: [d_date_sk#5, d_date#6] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=13] + +Subquery:2 Hosting operator id = 26 Hosting Expression = ss_sold_date_sk#26 IN dynamicpruning#4 + + 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 new file mode 100644 index 000000000..3109290dc --- /dev/null +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q51a/simplified.txt @@ -0,0 +1,124 @@ +TakeOrderedAndProject [item_sk,d_date,web_sales,store_sales,web_cumulative,store_cumulative] + WholeStageCodegen (54) + 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] + Project [item_sk,d_date,web_sales,store_sales,web_sales,store_sales] + BroadcastHashJoin [item_sk,item_sk,rk,rk] + InputAdapter + Window [item_sk,d_date] + WholeStageCodegen (26) + Sort [item_sk,d_date] + InputAdapter + Exchange [item_sk] #1 + WholeStageCodegen (25) + Project [item_sk,item_sk,d_date,d_date,cume_sales,cume_sales] + Filter [item_sk,item_sk] + SortMergeJoin [item_sk,d_date,item_sk,d_date] + InputAdapter + WholeStageCodegen (12) + Sort [item_sk,d_date] + InputAdapter + Exchange [item_sk,d_date] #2 + WholeStageCodegen (11) + HashAggregate [item_sk,d_date,sumws,sum,isEmpty] [sum(sumws),cume_sales,sum,isEmpty] + InputAdapter + Exchange [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) + Sort [ws_item_sk,d_date] + InputAdapter + Exchange [ws_item_sk] #4 + WholeStageCodegen (3) + HashAggregate [ws_item_sk,d_date,sum] [sum(UnscaledValue(ws_sales_price)),item_sk,sumws,sum] + InputAdapter + Exchange [ws_item_sk,d_date] #5 + WholeStageCodegen (2) + HashAggregate [ws_item_sk,d_date,ws_sales_price] [sum,sum] + Project [ws_item_sk,ws_sales_price,d_date] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + ColumnarToRow + InputAdapter + CometFilter [ws_item_sk] + CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_sales_price,ws_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #6 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk,d_date] + CometFilter [d_month_seq,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_month_seq] + InputAdapter + ReusedExchange [d_date_sk,d_date] #6 + InputAdapter + BroadcastExchange #7 + WholeStageCodegen (9) + Project [item_sk,sumws,rk] + InputAdapter + Window [ws_item_sk,d_date] + WholeStageCodegen (8) + Sort [ws_item_sk,d_date] + InputAdapter + ReusedExchange [item_sk,d_date,sumws,ws_item_sk] #4 + InputAdapter + WholeStageCodegen (24) + Sort [item_sk,d_date] + InputAdapter + Exchange [item_sk,d_date] #8 + WholeStageCodegen (23) + HashAggregate [item_sk,d_date,sumss,sum,isEmpty] [sum(sumss),cume_sales,sum,isEmpty] + InputAdapter + Exchange [item_sk,d_date,sumss] #9 + WholeStageCodegen (22) + HashAggregate [item_sk,d_date,sumss,sumss] [sum,isEmpty,sum,isEmpty] + Project [item_sk,d_date,sumss,sumss] + BroadcastHashJoin [item_sk,item_sk,rk,rk] + Project [item_sk,d_date,sumss,rk] + InputAdapter + Window [ss_item_sk,d_date] + WholeStageCodegen (16) + Sort [ss_item_sk,d_date] + InputAdapter + Exchange [ss_item_sk] #10 + WholeStageCodegen (15) + HashAggregate [ss_item_sk,d_date,sum] [sum(UnscaledValue(ss_sales_price)),item_sk,sumss,sum] + InputAdapter + Exchange [ss_item_sk,d_date] #11 + WholeStageCodegen (14) + HashAggregate [ss_item_sk,d_date,ss_sales_price] [sum,sum] + Project [ss_item_sk,ss_sales_price,d_date] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + ColumnarToRow + InputAdapter + CometFilter [ss_item_sk] + CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_sales_price,ss_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + InputAdapter + ReusedExchange [d_date_sk,d_date] #6 + InputAdapter + BroadcastExchange #12 + WholeStageCodegen (21) + Project [item_sk,sumss,rk] + InputAdapter + Window [ss_item_sk,d_date] + WholeStageCodegen (20) + Sort [ss_item_sk,d_date] + InputAdapter + ReusedExchange [item_sk,d_date,sumss,ss_item_sk] #10 + InputAdapter + BroadcastExchange #13 + WholeStageCodegen (53) + Project [item_sk,web_sales,store_sales,rk] + InputAdapter + Window [item_sk,d_date] + WholeStageCodegen (52) + Sort [item_sk,d_date] + InputAdapter + ReusedExchange [item_sk,d_date,web_sales,store_sales] #1 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q57/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q57/explain.txt new file mode 100644 index 000000000..dd64cc7bc --- /dev/null +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q57/explain.txt @@ -0,0 +1,279 @@ +== Physical Plan == +TakeOrderedAndProject (45) ++- * Project (44) + +- * BroadcastHashJoin Inner BuildRight (43) + :- * Project (37) + : +- * BroadcastHashJoin Inner BuildRight (36) + : :- * Project (28) + : : +- * Filter (27) + : : +- Window (26) + : : +- * Filter (25) + : : +- Window (24) + : : +- * Sort (23) + : : +- Exchange (22) + : : +- * HashAggregate (21) + : : +- Exchange (20) + : : +- * HashAggregate (19) + : : +- * Project (18) + : : +- * BroadcastHashJoin Inner BuildRight (17) + : : :- * Project (12) + : : : +- * BroadcastHashJoin Inner BuildRight (11) + : : : :- * Project (9) + : : : : +- * BroadcastHashJoin Inner BuildRight (8) + : : : : :- * ColumnarToRow (3) + : : : : : +- CometFilter (2) + : : : : : +- CometScan parquet spark_catalog.default.item (1) + : : : : +- BroadcastExchange (7) + : : : : +- * ColumnarToRow (6) + : : : : +- CometFilter (5) + : : : : +- CometScan parquet spark_catalog.default.catalog_sales (4) + : : : +- ReusedExchange (10) + : : +- BroadcastExchange (16) + : : +- * ColumnarToRow (15) + : : +- CometFilter (14) + : : +- CometScan parquet spark_catalog.default.call_center (13) + : +- BroadcastExchange (35) + : +- * Project (34) + : +- Window (33) + : +- * Sort (32) + : +- Exchange (31) + : +- * HashAggregate (30) + : +- ReusedExchange (29) + +- BroadcastExchange (42) + +- * Project (41) + +- Window (40) + +- * Sort (39) + +- ReusedExchange (38) + + +(1) Scan parquet spark_catalog.default.item +Output [3]: [i_item_sk#1, i_brand#2, i_category#3] +Batched: true +Location [not included in comparison]/{warehouse_dir}/item] +PushedFilters: [IsNotNull(i_item_sk), IsNotNull(i_category), IsNotNull(i_brand)] +ReadSchema: struct<i_item_sk:int,i_brand:string,i_category:string> + +(2) CometFilter +Input [3]: [i_item_sk#1, i_brand#2, i_category#3] +Condition : ((isnotnull(i_item_sk#1) AND isnotnull(i_category#3)) AND isnotnull(i_brand#2)) + +(3) ColumnarToRow [codegen id : 4] +Input [3]: [i_item_sk#1, i_brand#2, i_category#3] + +(4) Scan parquet spark_catalog.default.catalog_sales +Output [4]: [cs_call_center_sk#4, cs_item_sk#5, cs_sales_price#6, cs_sold_date_sk#7] +Batched: true +Location: InMemoryFileIndex [] +PartitionFilters: [isnotnull(cs_sold_date_sk#7), dynamicpruningexpression(cs_sold_date_sk#7 IN dynamicpruning#8)] +PushedFilters: [IsNotNull(cs_item_sk), IsNotNull(cs_call_center_sk)] +ReadSchema: struct<cs_call_center_sk:int,cs_item_sk:int,cs_sales_price:decimal(7,2)> + +(5) CometFilter +Input [4]: [cs_call_center_sk#4, cs_item_sk#5, cs_sales_price#6, cs_sold_date_sk#7] +Condition : (isnotnull(cs_item_sk#5) AND isnotnull(cs_call_center_sk#4)) + +(6) ColumnarToRow [codegen id : 1] +Input [4]: [cs_call_center_sk#4, cs_item_sk#5, cs_sales_price#6, cs_sold_date_sk#7] + +(7) BroadcastExchange +Input [4]: [cs_call_center_sk#4, cs_item_sk#5, cs_sales_price#6, cs_sold_date_sk#7] +Arguments: HashedRelationBroadcastMode(List(cast(input[1, int, false] as bigint)),false), [plan_id=1] + +(8) BroadcastHashJoin [codegen id : 4] +Left keys [1]: [i_item_sk#1] +Right keys [1]: [cs_item_sk#5] +Join type: Inner +Join condition: None + +(9) Project [codegen id : 4] +Output [5]: [i_brand#2, i_category#3, cs_call_center_sk#4, cs_sales_price#6, cs_sold_date_sk#7] +Input [7]: [i_item_sk#1, i_brand#2, i_category#3, cs_call_center_sk#4, cs_item_sk#5, cs_sales_price#6, cs_sold_date_sk#7] + +(10) ReusedExchange [Reuses operator id: 49] +Output [3]: [d_date_sk#9, d_year#10, d_moy#11] + +(11) BroadcastHashJoin [codegen id : 4] +Left keys [1]: [cs_sold_date_sk#7] +Right keys [1]: [d_date_sk#9] +Join type: Inner +Join condition: None + +(12) Project [codegen id : 4] +Output [6]: [i_brand#2, i_category#3, cs_call_center_sk#4, cs_sales_price#6, d_year#10, d_moy#11] +Input [8]: [i_brand#2, i_category#3, cs_call_center_sk#4, cs_sales_price#6, cs_sold_date_sk#7, d_date_sk#9, d_year#10, d_moy#11] + +(13) Scan parquet spark_catalog.default.call_center +Output [2]: [cc_call_center_sk#12, cc_name#13] +Batched: true +Location [not included in comparison]/{warehouse_dir}/call_center] +PushedFilters: [IsNotNull(cc_call_center_sk), IsNotNull(cc_name)] +ReadSchema: struct<cc_call_center_sk:int,cc_name:string> + +(14) CometFilter +Input [2]: [cc_call_center_sk#12, cc_name#13] +Condition : (isnotnull(cc_call_center_sk#12) AND isnotnull(cc_name#13)) + +(15) ColumnarToRow [codegen id : 3] +Input [2]: [cc_call_center_sk#12, cc_name#13] + +(16) BroadcastExchange +Input [2]: [cc_call_center_sk#12, cc_name#13] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=2] + +(17) BroadcastHashJoin [codegen id : 4] +Left keys [1]: [cs_call_center_sk#4] +Right keys [1]: [cc_call_center_sk#12] +Join type: Inner +Join condition: None + +(18) Project [codegen id : 4] +Output [6]: [i_brand#2, i_category#3, cs_sales_price#6, d_year#10, d_moy#11, cc_name#13] +Input [8]: [i_brand#2, i_category#3, cs_call_center_sk#4, cs_sales_price#6, d_year#10, d_moy#11, cc_call_center_sk#12, cc_name#13] + +(19) HashAggregate [codegen id : 4] +Input [6]: [i_brand#2, i_category#3, cs_sales_price#6, d_year#10, d_moy#11, cc_name#13] +Keys [5]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11] +Functions [1]: [partial_sum(UnscaledValue(cs_sales_price#6))] +Aggregate Attributes [1]: [sum#14] +Results [6]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, sum#15] + +(20) Exchange +Input [6]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, sum#15] +Arguments: hashpartitioning(i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, 5), ENSURE_REQUIREMENTS, [plan_id=3] + +(21) HashAggregate [codegen id : 5] +Input [6]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, sum#15] +Keys [5]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11] +Functions [1]: [sum(UnscaledValue(cs_sales_price#6))] +Aggregate Attributes [1]: [sum(UnscaledValue(cs_sales_price#6))#16] +Results [7]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, MakeDecimal(sum(UnscaledValue(cs_sales_price#6))#16,17,2) AS sum_sales#17, MakeDecimal(sum(UnscaledValue(cs_sales_price#6))#16,17,2) AS _w0#18] + +(22) Exchange +Input [7]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, sum_sales#17, _w0#18] +Arguments: hashpartitioning(i_category#3, i_brand#2, cc_name#13, 5), ENSURE_REQUIREMENTS, [plan_id=4] + +(23) Sort [codegen id : 6] +Input [7]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, sum_sales#17, _w0#18] +Arguments: [i_category#3 ASC NULLS FIRST, i_brand#2 ASC NULLS FIRST, cc_name#13 ASC NULLS FIRST, d_year#10 ASC NULLS FIRST, d_moy#11 ASC NULLS FIRST], false, 0 + +(24) Window +Input [7]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, sum_sales#17, _w0#18] +Arguments: [rank(d_year#10, d_moy#11) windowspecdefinition(i_category#3, i_brand#2, cc_name#13, d_year#10 ASC NULLS FIRST, d_moy#11 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#19], [i_category#3, i_brand#2, cc_name#13], [d_year#10 ASC NULLS FIRST, d_moy#11 ASC NULLS FIRST] + +(25) Filter [codegen id : 7] +Input [8]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, sum_sales#17, _w0#18, rn#19] +Condition : (isnotnull(d_year#10) AND (d_year#10 = 1999)) + +(26) Window +Input [8]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, sum_sales#17, _w0#18, rn#19] +Arguments: [avg(_w0#18) windowspecdefinition(i_category#3, i_brand#2, cc_name#13, d_year#10, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS avg_monthly_sales#20], [i_category#3, i_brand#2, cc_name#13, d_year#10] + +(27) Filter [codegen id : 22] +Input [9]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, sum_sales#17, _w0#18, rn#19, avg_monthly_sales#20] +Condition : ((isnotnull(avg_monthly_sales#20) AND (avg_monthly_sales#20 > 0.000000)) AND CASE WHEN (avg_monthly_sales#20 > 0.000000) THEN ((abs((sum_sales#17 - avg_monthly_sales#20)) / avg_monthly_sales#20) > 0.1000000000000000) END) + +(28) Project [codegen id : 22] +Output [8]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, sum_sales#17, avg_monthly_sales#20, rn#19] +Input [9]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, sum_sales#17, _w0#18, rn#19, avg_monthly_sales#20] + +(29) ReusedExchange [Reuses operator id: 20] +Output [6]: [i_category#21, i_brand#22, cc_name#23, d_year#24, d_moy#25, sum#26] + +(30) HashAggregate [codegen id : 12] +Input [6]: [i_category#21, i_brand#22, cc_name#23, d_year#24, d_moy#25, sum#26] +Keys [5]: [i_category#21, i_brand#22, cc_name#23, d_year#24, d_moy#25] +Functions [1]: [sum(UnscaledValue(cs_sales_price#27))] +Aggregate Attributes [1]: [sum(UnscaledValue(cs_sales_price#27))#16] +Results [6]: [i_category#21, i_brand#22, cc_name#23, d_year#24, d_moy#25, MakeDecimal(sum(UnscaledValue(cs_sales_price#27))#16,17,2) AS sum_sales#28] + +(31) Exchange +Input [6]: [i_category#21, i_brand#22, cc_name#23, d_year#24, d_moy#25, sum_sales#28] +Arguments: hashpartitioning(i_category#21, i_brand#22, cc_name#23, 5), ENSURE_REQUIREMENTS, [plan_id=5] + +(32) Sort [codegen id : 13] +Input [6]: [i_category#21, i_brand#22, cc_name#23, d_year#24, d_moy#25, sum_sales#28] +Arguments: [i_category#21 ASC NULLS FIRST, i_brand#22 ASC NULLS FIRST, cc_name#23 ASC NULLS FIRST, d_year#24 ASC NULLS FIRST, d_moy#25 ASC NULLS FIRST], false, 0 + +(33) Window +Input [6]: [i_category#21, i_brand#22, cc_name#23, d_year#24, d_moy#25, sum_sales#28] +Arguments: [rank(d_year#24, d_moy#25) windowspecdefinition(i_category#21, i_brand#22, cc_name#23, d_year#24 ASC NULLS FIRST, d_moy#25 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#29], [i_category#21, i_brand#22, cc_name#23], [d_year#24 ASC NULLS FIRST, d_moy#25 ASC NULLS FIRST] + +(34) Project [codegen id : 14] +Output [5]: [i_category#21, i_brand#22, cc_name#23, sum_sales#28, rn#29] +Input [7]: [i_category#21, i_brand#22, cc_name#23, d_year#24, d_moy#25, sum_sales#28, rn#29] + +(35) BroadcastExchange +Input [5]: [i_category#21, i_brand#22, cc_name#23, sum_sales#28, rn#29] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], (input[4, int, false] + 1)),false), [plan_id=6] + +(36) BroadcastHashJoin [codegen id : 22] +Left keys [4]: [i_category#3, i_brand#2, cc_name#13, rn#19] +Right keys [4]: [i_category#21, i_brand#22, cc_name#23, (rn#29 + 1)] +Join type: Inner +Join condition: None + +(37) Project [codegen id : 22] +Output [9]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, sum_sales#17, avg_monthly_sales#20, rn#19, sum_sales#28] +Input [13]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, sum_sales#17, avg_monthly_sales#20, rn#19, i_category#21, i_brand#22, cc_name#23, sum_sales#28, rn#29] + +(38) ReusedExchange [Reuses operator id: 31] +Output [6]: [i_category#30, i_brand#31, cc_name#32, d_year#33, d_moy#34, sum_sales#35] + +(39) Sort [codegen id : 20] +Input [6]: [i_category#30, i_brand#31, cc_name#32, d_year#33, d_moy#34, sum_sales#35] +Arguments: [i_category#30 ASC NULLS FIRST, i_brand#31 ASC NULLS FIRST, cc_name#32 ASC NULLS FIRST, d_year#33 ASC NULLS FIRST, d_moy#34 ASC NULLS FIRST], false, 0 + +(40) Window +Input [6]: [i_category#30, i_brand#31, cc_name#32, d_year#33, d_moy#34, sum_sales#35] +Arguments: [rank(d_year#33, d_moy#34) windowspecdefinition(i_category#30, i_brand#31, cc_name#32, d_year#33 ASC NULLS FIRST, d_moy#34 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#36], [i_category#30, i_brand#31, cc_name#32], [d_year#33 ASC NULLS FIRST, d_moy#34 ASC NULLS FIRST] + +(41) Project [codegen id : 21] +Output [5]: [i_category#30, i_brand#31, cc_name#32, sum_sales#35, rn#36] +Input [7]: [i_category#30, i_brand#31, cc_name#32, d_year#33, d_moy#34, sum_sales#35, rn#36] + +(42) BroadcastExchange +Input [5]: [i_category#30, i_brand#31, cc_name#32, sum_sales#35, rn#36] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], (input[4, int, false] - 1)),false), [plan_id=7] + +(43) BroadcastHashJoin [codegen id : 22] +Left keys [4]: [i_category#3, i_brand#2, cc_name#13, rn#19] +Right keys [4]: [i_category#30, i_brand#31, cc_name#32, (rn#36 - 1)] +Join type: Inner +Join condition: None + +(44) Project [codegen id : 22] +Output [8]: [i_category#3, i_brand#2, d_year#10, d_moy#11, avg_monthly_sales#20, sum_sales#17, sum_sales#28 AS psum#37, sum_sales#35 AS nsum#38] +Input [14]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, sum_sales#17, avg_monthly_sales#20, rn#19, sum_sales#28, i_category#30, i_brand#31, cc_name#32, sum_sales#35, rn#36] + +(45) TakeOrderedAndProject +Input [8]: [i_category#3, i_brand#2, d_year#10, d_moy#11, avg_monthly_sales#20, sum_sales#17, psum#37, nsum#38] +Arguments: 100, [(sum_sales#17 - avg_monthly_sales#20) ASC NULLS FIRST, d_year#10 ASC NULLS FIRST], [i_category#3, i_brand#2, d_year#10, d_moy#11, avg_monthly_sales#20, sum_sales#17, psum#37, nsum#38] + +===== Subqueries ===== + +Subquery:1 Hosting operator id = 4 Hosting Expression = cs_sold_date_sk#7 IN dynamicpruning#8 +BroadcastExchange (49) ++- * ColumnarToRow (48) + +- CometFilter (47) + +- CometScan parquet spark_catalog.default.date_dim (46) + + +(46) Scan parquet spark_catalog.default.date_dim +Output [3]: [d_date_sk#9, d_year#10, d_moy#11] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [Or(Or(EqualTo(d_year,1999),And(EqualTo(d_year,1998),EqualTo(d_moy,12))),And(EqualTo(d_year,2000),EqualTo(d_moy,1))), IsNotNull(d_date_sk)] +ReadSchema: struct<d_date_sk:int,d_year:int,d_moy:int> + +(47) CometFilter +Input [3]: [d_date_sk#9, d_year#10, d_moy#11] +Condition : ((((d_year#10 = 1999) OR ((d_year#10 = 1998) AND (d_moy#11 = 12))) OR ((d_year#10 = 2000) AND (d_moy#11 = 1))) AND isnotnull(d_date_sk#9)) + +(48) ColumnarToRow [codegen id : 1] +Input [3]: [d_date_sk#9, d_year#10, d_moy#11] + +(49) BroadcastExchange +Input [3]: [d_date_sk#9, d_year#10, d_moy#11] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=8] + + 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 new file mode 100644 index 000000000..56e33be9e --- /dev/null +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q57/simplified.txt @@ -0,0 +1,81 @@ +TakeOrderedAndProject [sum_sales,avg_monthly_sales,d_year,i_category,i_brand,d_moy,psum,nsum] + WholeStageCodegen (22) + 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] + 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] + Filter [avg_monthly_sales,sum_sales] + InputAdapter + Window [_w0,i_category,i_brand,cc_name,d_year] + WholeStageCodegen (7) + Filter [d_year] + InputAdapter + Window [d_year,d_moy,i_category,i_brand,cc_name] + WholeStageCodegen (6) + Sort [i_category,i_brand,cc_name,d_year,d_moy] + InputAdapter + Exchange [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] + InputAdapter + Exchange [i_category,i_brand,cc_name,d_year,d_moy] #2 + WholeStageCodegen (4) + HashAggregate [i_category,i_brand,cc_name,d_year,d_moy,cs_sales_price] [sum,sum] + Project [i_brand,i_category,cs_sales_price,d_year,d_moy,cc_name] + BroadcastHashJoin [cs_call_center_sk,cc_call_center_sk] + Project [i_brand,i_category,cs_call_center_sk,cs_sales_price,d_year,d_moy] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + Project [i_brand,i_category,cs_call_center_sk,cs_sales_price,cs_sold_date_sk] + BroadcastHashJoin [i_item_sk,cs_item_sk] + ColumnarToRow + InputAdapter + CometFilter [i_item_sk,i_category,i_brand] + CometScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_category] + InputAdapter + BroadcastExchange #3 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometFilter [cs_item_sk,cs_call_center_sk] + CometScan parquet spark_catalog.default.catalog_sales [cs_call_center_sk,cs_item_sk,cs_sales_price,cs_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #4 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometFilter [d_year,d_moy,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + InputAdapter + ReusedExchange [d_date_sk,d_year,d_moy] #4 + InputAdapter + BroadcastExchange #5 + WholeStageCodegen (3) + ColumnarToRow + InputAdapter + CometFilter [cc_call_center_sk,cc_name] + CometScan parquet spark_catalog.default.call_center [cc_call_center_sk,cc_name] + InputAdapter + BroadcastExchange #6 + WholeStageCodegen (14) + Project [i_category,i_brand,cc_name,sum_sales,rn] + InputAdapter + Window [d_year,d_moy,i_category,i_brand,cc_name] + WholeStageCodegen (13) + Sort [i_category,i_brand,cc_name,d_year,d_moy] + InputAdapter + Exchange [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] + InputAdapter + ReusedExchange [i_category,i_brand,cc_name,d_year,d_moy,sum] #2 + InputAdapter + BroadcastExchange #8 + WholeStageCodegen (21) + Project [i_category,i_brand,cc_name,sum_sales,rn] + InputAdapter + Window [d_year,d_moy,i_category,i_brand,cc_name] + WholeStageCodegen (20) + Sort [i_category,i_brand,cc_name,d_year,d_moy] + InputAdapter + ReusedExchange [i_category,i_brand,cc_name,d_year,d_moy,sum_sales] #7 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 new file mode 100644 index 000000000..2769e772a --- /dev/null +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q5a/explain.txt @@ -0,0 +1,542 @@ +== Physical Plan == +TakeOrderedAndProject (83) ++- * HashAggregate (82) + +- Exchange (81) + +- * HashAggregate (80) + +- Union (79) + :- * HashAggregate (68) + : +- Exchange (67) + : +- * HashAggregate (66) + : +- Union (65) + : :- * HashAggregate (20) + : : +- Exchange (19) + : : +- * HashAggregate (18) + : : +- * Project (17) + : : +- * BroadcastHashJoin Inner BuildRight (16) + : : :- * Project (11) + : : : +- * BroadcastHashJoin Inner BuildRight (10) + : : : :- * ColumnarToRow (8) + : : : : +- CometUnion (7) + : : : : :- CometProject (3) + : : : : : +- CometFilter (2) + : : : : : +- CometScan parquet spark_catalog.default.store_sales (1) + : : : : +- CometProject (6) + : : : : +- CometFilter (5) + : : : : +- CometScan parquet spark_catalog.default.store_returns (4) + : : : +- ReusedExchange (9) + : : +- BroadcastExchange (15) + : : +- * ColumnarToRow (14) + : : +- CometFilter (13) + : : +- CometScan parquet spark_catalog.default.store (12) + : :- * HashAggregate (40) + : : +- Exchange (39) + : : +- * HashAggregate (38) + : : +- * Project (37) + : : +- * BroadcastHashJoin Inner BuildRight (36) + : : :- * Project (31) + : : : +- * BroadcastHashJoin Inner BuildRight (30) + : : : :- * ColumnarToRow (28) + : : : : +- CometUnion (27) + : : : : :- CometProject (23) + : : : : : +- CometFilter (22) + : : : : : +- CometScan parquet spark_catalog.default.catalog_sales (21) + : : : : +- CometProject (26) + : : : : +- CometFilter (25) + : : : : +- CometScan parquet spark_catalog.default.catalog_returns (24) + : : : +- ReusedExchange (29) + : : +- BroadcastExchange (35) + : : +- * ColumnarToRow (34) + : : +- CometFilter (33) + : : +- CometScan parquet spark_catalog.default.catalog_page (32) + : +- * HashAggregate (64) + : +- Exchange (63) + : +- * HashAggregate (62) + : +- * Project (61) + : +- * BroadcastHashJoin Inner BuildRight (60) + : :- * Project (55) + : : +- * BroadcastHashJoin Inner BuildRight (54) + : : :- * ColumnarToRow (52) + : : : +- CometUnion (51) + : : : :- CometProject (43) + : : : : +- CometFilter (42) + : : : : +- CometScan parquet spark_catalog.default.web_sales (41) + : : : +- CometProject (50) + : : : +- CometBroadcastHashJoin (49) + : : : :- CometBroadcastExchange (45) + : : : : +- CometScan parquet spark_catalog.default.web_returns (44) + : : : +- CometProject (48) + : : : +- CometFilter (47) + : : : +- CometScan parquet spark_catalog.default.web_sales (46) + : : +- ReusedExchange (53) + : +- BroadcastExchange (59) + : +- * ColumnarToRow (58) + : +- CometFilter (57) + : +- CometScan parquet spark_catalog.default.web_site (56) + :- * HashAggregate (73) + : +- Exchange (72) + : +- * HashAggregate (71) + : +- * HashAggregate (70) + : +- ReusedExchange (69) + +- * HashAggregate (78) + +- Exchange (77) + +- * HashAggregate (76) + +- * HashAggregate (75) + +- ReusedExchange (74) + + +(1) Scan parquet spark_catalog.default.store_sales +Output [4]: [ss_store_sk#1, ss_ext_sales_price#2, ss_net_profit#3, ss_sold_date_sk#4] +Batched: true +Location: InMemoryFileIndex [] +PartitionFilters: [isnotnull(ss_sold_date_sk#4), dynamicpruningexpression(ss_sold_date_sk#4 IN dynamicpruning#5)] +PushedFilters: [IsNotNull(ss_store_sk)] +ReadSchema: struct<ss_store_sk:int,ss_ext_sales_price:decimal(7,2),ss_net_profit:decimal(7,2)> + +(2) CometFilter +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) + +(3) CometProject +Input [4]: [ss_store_sk#1, ss_ext_sales_price#2, ss_net_profit#3, ss_sold_date_sk#4] +Arguments: [store_sk#6, date_sk#7, sales_price#8, profit#9, return_amt#10, net_loss#11], [ss_store_sk#1 AS store_sk#6, ss_sold_date_sk#4 AS date_sk#7, ss_ext_sales_price#2 AS sales_price#8, ss_net_profit#3 AS profit#9, 0.00 AS return_amt#10, 0.00 AS net_loss#11] + +(4) Scan parquet spark_catalog.default.store_returns +Output [4]: [sr_store_sk#12, sr_return_amt#13, sr_net_loss#14, sr_returned_date_sk#15] +Batched: true +Location: InMemoryFileIndex [] +PartitionFilters: [isnotnull(sr_returned_date_sk#15), dynamicpruningexpression(sr_returned_date_sk#15 IN dynamicpruning#5)] +PushedFilters: [IsNotNull(sr_store_sk)] +ReadSchema: struct<sr_store_sk:int,sr_return_amt:decimal(7,2),sr_net_loss:decimal(7,2)> + +(5) CometFilter +Input [4]: [sr_store_sk#12, sr_return_amt#13, sr_net_loss#14, sr_returned_date_sk#15] +Condition : isnotnull(sr_store_sk#12) + +(6) CometProject +Input [4]: [sr_store_sk#12, sr_return_amt#13, sr_net_loss#14, sr_returned_date_sk#15] +Arguments: [store_sk#16, date_sk#17, sales_price#18, profit#19, return_amt#20, net_loss#21], [sr_store_sk#12 AS store_sk#16, sr_returned_date_sk#15 AS date_sk#17, 0.00 AS sales_price#18, 0.00 AS profit#19, sr_return_amt#13 AS return_amt#20, sr_net_loss#14 AS net_loss#21] + +(7) CometUnion +Child 0 Input [6]: [store_sk#6, date_sk#7, sales_price#8, profit#9, return_amt#10, net_loss#11] +Child 1 Input [6]: [store_sk#16, date_sk#17, sales_price#18, profit#19, return_amt#20, net_loss#21] + +(8) ColumnarToRow [codegen id : 3] +Input [6]: [store_sk#6, date_sk#7, sales_price#8, profit#9, return_amt#10, net_loss#11] + +(9) ReusedExchange [Reuses operator id: 88] +Output [1]: [d_date_sk#22] + +(10) BroadcastHashJoin [codegen id : 3] +Left keys [1]: [date_sk#7] +Right keys [1]: [d_date_sk#22] +Join type: Inner +Join condition: None + +(11) 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] + +(12) Scan parquet spark_catalog.default.store +Output [2]: [s_store_sk#23, s_store_id#24] +Batched: true +Location [not included in comparison]/{warehouse_dir}/store] +PushedFilters: [IsNotNull(s_store_sk)] +ReadSchema: struct<s_store_sk:int,s_store_id:string> + +(13) CometFilter +Input [2]: [s_store_sk#23, s_store_id#24] +Condition : isnotnull(s_store_sk#23) + +(14) ColumnarToRow [codegen id : 2] +Input [2]: [s_store_sk#23, s_store_id#24] + +(15) BroadcastExchange +Input [2]: [s_store_sk#23, s_store_id#24] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=1] + +(16) BroadcastHashJoin [codegen id : 3] +Left keys [1]: [store_sk#6] +Right keys [1]: [s_store_sk#23] +Join type: Inner +Join condition: None + +(17) Project [codegen id : 3] +Output [5]: [sales_price#8, profit#9, return_amt#10, net_loss#11, s_store_id#24] +Input [7]: [store_sk#6, sales_price#8, profit#9, return_amt#10, net_loss#11, s_store_sk#23, s_store_id#24] + +(18) HashAggregate [codegen id : 3] +Input [5]: [sales_price#8, profit#9, return_amt#10, net_loss#11, s_store_id#24] +Keys [1]: [s_store_id#24] +Functions [4]: [partial_sum(UnscaledValue(sales_price#8)), partial_sum(UnscaledValue(return_amt#10)), partial_sum(UnscaledValue(profit#9)), partial_sum(UnscaledValue(net_loss#11))] +Aggregate Attributes [4]: [sum#25, sum#26, sum#27, sum#28] +Results [5]: [s_store_id#24, sum#29, sum#30, sum#31, sum#32] + +(19) Exchange +Input [5]: [s_store_id#24, sum#29, sum#30, sum#31, sum#32] +Arguments: hashpartitioning(s_store_id#24, 5), ENSURE_REQUIREMENTS, [plan_id=2] + +(20) HashAggregate [codegen id : 4] +Input [5]: [s_store_id#24, sum#29, sum#30, sum#31, sum#32] +Keys [1]: [s_store_id#24] +Functions [4]: [sum(UnscaledValue(sales_price#8)), sum(UnscaledValue(return_amt#10)), sum(UnscaledValue(profit#9)), sum(UnscaledValue(net_loss#11))] +Aggregate Attributes [4]: [sum(UnscaledValue(sales_price#8))#33, sum(UnscaledValue(return_amt#10))#34, sum(UnscaledValue(profit#9))#35, sum(UnscaledValue(net_loss#11))#36] +Results [5]: [store channel AS channel#37, concat(store, s_store_id#24) AS id#38, MakeDecimal(sum(UnscaledValue(sales_price#8))#33,17,2) AS sales#39, MakeDecimal(sum(UnscaledValue(return_amt#10))#34,17,2) AS returns#40, (MakeDecimal(sum(UnscaledValue(profit#9))#35,17,2) - MakeDecimal(sum(UnscaledValue(net_loss#11))#36,17,2)) AS profit#41] + +(21) Scan parquet spark_catalog.default.catalog_sales +Output [4]: [cs_catalog_page_sk#42, cs_ext_sales_price#43, cs_net_profit#44, cs_sold_date_sk#45] +Batched: true +Location: InMemoryFileIndex [] +PartitionFilters: [isnotnull(cs_sold_date_sk#45), dynamicpruningexpression(cs_sold_date_sk#45 IN dynamicpruning#46)] +PushedFilters: [IsNotNull(cs_catalog_page_sk)] +ReadSchema: struct<cs_catalog_page_sk:int,cs_ext_sales_price:decimal(7,2),cs_net_profit:decimal(7,2)> + +(22) CometFilter +Input [4]: [cs_catalog_page_sk#42, cs_ext_sales_price#43, cs_net_profit#44, cs_sold_date_sk#45] +Condition : isnotnull(cs_catalog_page_sk#42) + +(23) CometProject +Input [4]: [cs_catalog_page_sk#42, cs_ext_sales_price#43, cs_net_profit#44, cs_sold_date_sk#45] +Arguments: [page_sk#47, date_sk#48, sales_price#49, profit#50, return_amt#51, net_loss#52], [cs_catalog_page_sk#42 AS page_sk#47, cs_sold_date_sk#45 AS date_sk#48, cs_ext_sales_price#43 AS sales_price#49, cs_net_profit#44 AS profit#50, 0.00 AS return_amt#51, 0.00 AS net_loss#52] + +(24) Scan parquet spark_catalog.default.catalog_returns +Output [4]: [cr_catalog_page_sk#53, cr_return_amount#54, cr_net_loss#55, cr_returned_date_sk#56] +Batched: true +Location: InMemoryFileIndex [] +PartitionFilters: [isnotnull(cr_returned_date_sk#56), dynamicpruningexpression(cr_returned_date_sk#56 IN dynamicpruning#46)] +PushedFilters: [IsNotNull(cr_catalog_page_sk)] +ReadSchema: struct<cr_catalog_page_sk:int,cr_return_amount:decimal(7,2),cr_net_loss:decimal(7,2)> + +(25) CometFilter +Input [4]: [cr_catalog_page_sk#53, cr_return_amount#54, cr_net_loss#55, cr_returned_date_sk#56] +Condition : isnotnull(cr_catalog_page_sk#53) + +(26) CometProject +Input [4]: [cr_catalog_page_sk#53, cr_return_amount#54, cr_net_loss#55, cr_returned_date_sk#56] +Arguments: [page_sk#57, date_sk#58, sales_price#59, profit#60, return_amt#61, net_loss#62], [cr_catalog_page_sk#53 AS page_sk#57, cr_returned_date_sk#56 AS date_sk#58, 0.00 AS sales_price#59, 0.00 AS profit#60, cr_return_amount#54 AS return_amt#61, cr_net_loss#55 AS net_loss#62] + +(27) CometUnion +Child 0 Input [6]: [page_sk#47, date_sk#48, sales_price#49, profit#50, return_amt#51, net_loss#52] +Child 1 Input [6]: [page_sk#57, date_sk#58, sales_price#59, profit#60, return_amt#61, net_loss#62] + +(28) ColumnarToRow [codegen id : 7] +Input [6]: [page_sk#47, date_sk#48, sales_price#49, profit#50, return_amt#51, net_loss#52] + +(29) ReusedExchange [Reuses operator id: 88] +Output [1]: [d_date_sk#63] + +(30) BroadcastHashJoin [codegen id : 7] +Left keys [1]: [date_sk#48] +Right keys [1]: [d_date_sk#63] +Join type: Inner +Join condition: None + +(31) Project [codegen id : 7] +Output [5]: [page_sk#47, sales_price#49, profit#50, return_amt#51, net_loss#52] +Input [7]: [page_sk#47, date_sk#48, sales_price#49, profit#50, return_amt#51, net_loss#52, d_date_sk#63] + +(32) Scan parquet spark_catalog.default.catalog_page +Output [2]: [cp_catalog_page_sk#64, cp_catalog_page_id#65] +Batched: true +Location [not included in comparison]/{warehouse_dir}/catalog_page] +PushedFilters: [IsNotNull(cp_catalog_page_sk)] +ReadSchema: struct<cp_catalog_page_sk:int,cp_catalog_page_id:string> + +(33) CometFilter +Input [2]: [cp_catalog_page_sk#64, cp_catalog_page_id#65] +Condition : isnotnull(cp_catalog_page_sk#64) + +(34) ColumnarToRow [codegen id : 6] +Input [2]: [cp_catalog_page_sk#64, cp_catalog_page_id#65] + +(35) BroadcastExchange +Input [2]: [cp_catalog_page_sk#64, cp_catalog_page_id#65] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=3] + +(36) BroadcastHashJoin [codegen id : 7] +Left keys [1]: [page_sk#47] +Right keys [1]: [cp_catalog_page_sk#64] +Join type: Inner +Join condition: None + +(37) Project [codegen id : 7] +Output [5]: [sales_price#49, profit#50, return_amt#51, net_loss#52, cp_catalog_page_id#65] +Input [7]: [page_sk#47, sales_price#49, profit#50, return_amt#51, net_loss#52, cp_catalog_page_sk#64, cp_catalog_page_id#65] + +(38) HashAggregate [codegen id : 7] +Input [5]: [sales_price#49, profit#50, return_amt#51, net_loss#52, cp_catalog_page_id#65] +Keys [1]: [cp_catalog_page_id#65] +Functions [4]: [partial_sum(UnscaledValue(sales_price#49)), partial_sum(UnscaledValue(return_amt#51)), partial_sum(UnscaledValue(profit#50)), partial_sum(UnscaledValue(net_loss#52))] +Aggregate Attributes [4]: [sum#66, sum#67, sum#68, sum#69] +Results [5]: [cp_catalog_page_id#65, sum#70, sum#71, sum#72, sum#73] + +(39) Exchange +Input [5]: [cp_catalog_page_id#65, sum#70, sum#71, sum#72, sum#73] +Arguments: hashpartitioning(cp_catalog_page_id#65, 5), ENSURE_REQUIREMENTS, [plan_id=4] + +(40) HashAggregate [codegen id : 8] +Input [5]: [cp_catalog_page_id#65, sum#70, sum#71, sum#72, sum#73] +Keys [1]: [cp_catalog_page_id#65] +Functions [4]: [sum(UnscaledValue(sales_price#49)), sum(UnscaledValue(return_amt#51)), sum(UnscaledValue(profit#50)), sum(UnscaledValue(net_loss#52))] +Aggregate Attributes [4]: [sum(UnscaledValue(sales_price#49))#74, sum(UnscaledValue(return_amt#51))#75, sum(UnscaledValue(profit#50))#76, sum(UnscaledValue(net_loss#52))#77] +Results [5]: [catalog channel AS channel#78, concat(catalog_page, cp_catalog_page_id#65) AS id#79, MakeDecimal(sum(UnscaledValue(sales_price#49))#74,17,2) AS sales#80, MakeDecimal(sum(UnscaledValue(return_amt#51))#75,17,2) AS returns#81, (MakeDecimal(sum(UnscaledValue(profit#50))#76,17,2) - MakeDecimal(sum(UnscaledValue(net_loss#52))#77,17,2)) AS profit#82] + +(41) Scan parquet spark_catalog.default.web_sales +Output [4]: [ws_web_site_sk#83, ws_ext_sales_price#84, ws_net_profit#85, ws_sold_date_sk#86] +Batched: true +Location: InMemoryFileIndex [] +PartitionFilters: [isnotnull(ws_sold_date_sk#86), dynamicpruningexpression(ws_sold_date_sk#86 IN dynamicpruning#87)] +PushedFilters: [IsNotNull(ws_web_site_sk)] +ReadSchema: struct<ws_web_site_sk:int,ws_ext_sales_price:decimal(7,2),ws_net_profit:decimal(7,2)> + +(42) CometFilter +Input [4]: [ws_web_site_sk#83, ws_ext_sales_price#84, ws_net_profit#85, ws_sold_date_sk#86] +Condition : isnotnull(ws_web_site_sk#83) + +(43) CometProject +Input [4]: [ws_web_site_sk#83, ws_ext_sales_price#84, ws_net_profit#85, ws_sold_date_sk#86] +Arguments: [wsr_web_site_sk#88, date_sk#89, sales_price#90, profit#91, return_amt#92, net_loss#93], [ws_web_site_sk#83 AS wsr_web_site_sk#88, ws_sold_date_sk#86 AS date_sk#89, ws_ext_sales_price#84 AS sales_price#90, ws_net_profit#85 AS profit#91, 0.00 AS return_amt#92, 0.00 AS net_loss#93] + +(44) Scan parquet spark_catalog.default.web_returns +Output [5]: [wr_item_sk#94, wr_order_number#95, wr_return_amt#96, wr_net_loss#97, wr_returned_date_sk#98] +Batched: true +Location: InMemoryFileIndex [] +PartitionFilters: [isnotnull(wr_returned_date_sk#98), dynamicpruningexpression(wr_returned_date_sk#98 IN dynamicpruning#87)] +ReadSchema: struct<wr_item_sk:int,wr_order_number:int,wr_return_amt:decimal(7,2),wr_net_loss:decimal(7,2)> + +(45) CometBroadcastExchange +Input [5]: [wr_item_sk#94, wr_order_number#95, wr_return_amt#96, wr_net_loss#97, wr_returned_date_sk#98] +Arguments: [wr_item_sk#94, wr_order_number#95, wr_return_amt#96, wr_net_loss#97, wr_returned_date_sk#98] + +(46) Scan parquet spark_catalog.default.web_sales +Output [4]: [ws_item_sk#99, ws_web_site_sk#100, ws_order_number#101, ws_sold_date_sk#102] +Batched: true +Location [not included in comparison]/{warehouse_dir}/web_sales] +PushedFilters: [IsNotNull(ws_item_sk), IsNotNull(ws_order_number), IsNotNull(ws_web_site_sk)] +ReadSchema: struct<ws_item_sk:int,ws_web_site_sk:int,ws_order_number:int> + +(47) CometFilter +Input [4]: [ws_item_sk#99, ws_web_site_sk#100, ws_order_number#101, ws_sold_date_sk#102] +Condition : ((isnotnull(ws_item_sk#99) AND isnotnull(ws_order_number#101)) AND isnotnull(ws_web_site_sk#100)) + +(48) CometProject +Input [4]: [ws_item_sk#99, ws_web_site_sk#100, ws_order_number#101, ws_sold_date_sk#102] +Arguments: [ws_item_sk#99, ws_web_site_sk#100, ws_order_number#101], [ws_item_sk#99, ws_web_site_sk#100, ws_order_number#101] + +(49) CometBroadcastHashJoin +Left output [5]: [wr_item_sk#94, wr_order_number#95, wr_return_amt#96, wr_net_loss#97, wr_returned_date_sk#98] +Right output [3]: [ws_item_sk#99, ws_web_site_sk#100, ws_order_number#101] +Arguments: [wr_item_sk#94, wr_order_number#95], [ws_item_sk#99, ws_order_number#101], Inner + +(50) CometProject +Input [8]: [wr_item_sk#94, wr_order_number#95, wr_return_amt#96, wr_net_loss#97, wr_returned_date_sk#98, ws_item_sk#99, ws_web_site_sk#100, ws_order_number#101] +Arguments: [wsr_web_site_sk#103, date_sk#104, sales_price#105, profit#106, return_amt#107, net_loss#108], [ws_web_site_sk#100 AS wsr_web_site_sk#103, wr_returned_date_sk#98 AS date_sk#104, 0.00 AS sales_price#105, 0.00 AS profit#106, wr_return_amt#96 AS return_amt#107, wr_net_loss#97 AS net_loss#108] + +(51) CometUnion +Child 0 Input [6]: [wsr_web_site_sk#88, date_sk#89, sales_price#90, profit#91, return_amt#92, net_loss#93] +Child 1 Input [6]: [wsr_web_site_sk#103, date_sk#104, sales_price#105, profit#106, return_amt#107, net_loss#108] + +(52) ColumnarToRow [codegen id : 11] +Input [6]: [wsr_web_site_sk#88, date_sk#89, sales_price#90, profit#91, return_amt#92, net_loss#93] + +(53) ReusedExchange [Reuses operator id: 88] +Output [1]: [d_date_sk#109] + +(54) BroadcastHashJoin [codegen id : 11] +Left keys [1]: [date_sk#89] +Right keys [1]: [d_date_sk#109] +Join type: Inner +Join condition: None + +(55) Project [codegen id : 11] +Output [5]: [wsr_web_site_sk#88, sales_price#90, profit#91, return_amt#92, net_loss#93] +Input [7]: [wsr_web_site_sk#88, date_sk#89, sales_price#90, profit#91, return_amt#92, net_loss#93, d_date_sk#109] + +(56) Scan parquet spark_catalog.default.web_site +Output [2]: [web_site_sk#110, web_site_id#111] +Batched: true +Location [not included in comparison]/{warehouse_dir}/web_site] +PushedFilters: [IsNotNull(web_site_sk)] +ReadSchema: struct<web_site_sk:int,web_site_id:string> + +(57) CometFilter +Input [2]: [web_site_sk#110, web_site_id#111] +Condition : isnotnull(web_site_sk#110) + +(58) ColumnarToRow [codegen id : 10] +Input [2]: [web_site_sk#110, web_site_id#111] + +(59) BroadcastExchange +Input [2]: [web_site_sk#110, web_site_id#111] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=5] + +(60) BroadcastHashJoin [codegen id : 11] +Left keys [1]: [wsr_web_site_sk#88] +Right keys [1]: [web_site_sk#110] +Join type: Inner +Join condition: None + +(61) Project [codegen id : 11] +Output [5]: [sales_price#90, profit#91, return_amt#92, net_loss#93, web_site_id#111] +Input [7]: [wsr_web_site_sk#88, sales_price#90, profit#91, return_amt#92, net_loss#93, web_site_sk#110, web_site_id#111] + +(62) HashAggregate [codegen id : 11] +Input [5]: [sales_price#90, profit#91, return_amt#92, net_loss#93, web_site_id#111] +Keys [1]: [web_site_id#111] +Functions [4]: [partial_sum(UnscaledValue(sales_price#90)), partial_sum(UnscaledValue(return_amt#92)), partial_sum(UnscaledValue(profit#91)), partial_sum(UnscaledValue(net_loss#93))] +Aggregate Attributes [4]: [sum#112, sum#113, sum#114, sum#115] +Results [5]: [web_site_id#111, sum#116, sum#117, sum#118, sum#119] + +(63) Exchange +Input [5]: [web_site_id#111, sum#116, sum#117, sum#118, sum#119] +Arguments: hashpartitioning(web_site_id#111, 5), ENSURE_REQUIREMENTS, [plan_id=6] + +(64) HashAggregate [codegen id : 12] +Input [5]: [web_site_id#111, sum#116, sum#117, sum#118, sum#119] +Keys [1]: [web_site_id#111] +Functions [4]: [sum(UnscaledValue(sales_price#90)), sum(UnscaledValue(return_amt#92)), sum(UnscaledValue(profit#91)), sum(UnscaledValue(net_loss#93))] +Aggregate Attributes [4]: [sum(UnscaledValue(sales_price#90))#120, sum(UnscaledValue(return_amt#92))#121, sum(UnscaledValue(profit#91))#122, sum(UnscaledValue(net_loss#93))#123] +Results [5]: [web channel AS channel#124, concat(web_site, web_site_id#111) AS id#125, MakeDecimal(sum(UnscaledValue(sales_price#90))#120,17,2) AS sales#126, MakeDecimal(sum(UnscaledValue(return_amt#92))#121,17,2) AS returns#127, (MakeDecimal(sum(UnscaledValue(profit#91))#122,17,2) - MakeDecimal(sum(UnscaledValue(net_loss#93))#123,17,2)) AS profit#128] + +(65) Union + +(66) HashAggregate [codegen id : 13] +Input [5]: [channel#37, id#38, sales#39, returns#40, profit#41] +Keys [2]: [channel#37, id#38] +Functions [3]: [partial_sum(sales#39), partial_sum(returns#40), partial_sum(profit#41)] +Aggregate Attributes [6]: [sum#129, isEmpty#130, sum#131, isEmpty#132, sum#133, isEmpty#134] +Results [8]: [channel#37, id#38, sum#135, isEmpty#136, sum#137, isEmpty#138, sum#139, isEmpty#140] + +(67) Exchange +Input [8]: [channel#37, id#38, sum#135, isEmpty#136, sum#137, isEmpty#138, sum#139, isEmpty#140] +Arguments: hashpartitioning(channel#37, id#38, 5), ENSURE_REQUIREMENTS, [plan_id=7] + +(68) HashAggregate [codegen id : 14] +Input [8]: [channel#37, id#38, sum#135, isEmpty#136, sum#137, isEmpty#138, sum#139, isEmpty#140] +Keys [2]: [channel#37, id#38] +Functions [3]: [sum(sales#39), sum(returns#40), sum(profit#41)] +Aggregate Attributes [3]: [sum(sales#39)#141, sum(returns#40)#142, sum(profit#41)#143] +Results [5]: [channel#37, id#38, cast(sum(sales#39)#141 as decimal(37,2)) AS sales#144, cast(sum(returns#40)#142 as decimal(37,2)) AS returns#145, cast(sum(profit#41)#143 as decimal(38,2)) AS profit#146] + +(69) ReusedExchange [Reuses operator id: 67] +Output [8]: [channel#147, id#148, sum#149, isEmpty#150, sum#151, isEmpty#152, sum#153, isEmpty#154] + +(70) HashAggregate [codegen id : 28] +Input [8]: [channel#147, id#148, sum#149, isEmpty#150, sum#151, isEmpty#152, sum#153, isEmpty#154] +Keys [2]: [channel#147, id#148] +Functions [3]: [sum(sales#155), sum(returns#156), sum(profit#157)] +Aggregate Attributes [3]: [sum(sales#155)#141, sum(returns#156)#142, sum(profit#157)#143] +Results [4]: [channel#147, sum(sales#155)#141 AS sales#158, sum(returns#156)#142 AS returns#159, sum(profit#157)#143 AS profit#160] + +(71) HashAggregate [codegen id : 28] +Input [4]: [channel#147, sales#158, returns#159, profit#160] +Keys [1]: [channel#147] +Functions [3]: [partial_sum(sales#158), partial_sum(returns#159), partial_sum(profit#160)] +Aggregate Attributes [6]: [sum#161, isEmpty#162, sum#163, isEmpty#164, sum#165, isEmpty#166] +Results [7]: [channel#147, sum#167, isEmpty#168, sum#169, isEmpty#170, sum#171, isEmpty#172] + +(72) Exchange +Input [7]: [channel#147, sum#167, isEmpty#168, sum#169, isEmpty#170, sum#171, isEmpty#172] +Arguments: hashpartitioning(channel#147, 5), ENSURE_REQUIREMENTS, [plan_id=8] + +(73) HashAggregate [codegen id : 29] +Input [7]: [channel#147, sum#167, isEmpty#168, sum#169, isEmpty#170, sum#171, isEmpty#172] +Keys [1]: [channel#147] +Functions [3]: [sum(sales#158), sum(returns#159), sum(profit#160)] +Aggregate Attributes [3]: [sum(sales#158)#173, sum(returns#159)#174, sum(profit#160)#175] +Results [5]: [channel#147, null AS id#176, sum(sales#158)#173 AS sum(sales)#177, sum(returns#159)#174 AS sum(returns)#178, sum(profit#160)#175 AS sum(profit)#179] + +(74) ReusedExchange [Reuses operator id: 67] +Output [8]: [channel#180, id#181, sum#182, isEmpty#183, sum#184, isEmpty#185, sum#186, isEmpty#187] + +(75) HashAggregate [codegen id : 43] +Input [8]: [channel#180, id#181, sum#182, isEmpty#183, sum#184, isEmpty#185, sum#186, isEmpty#187] +Keys [2]: [channel#180, id#181] +Functions [3]: [sum(sales#188), sum(returns#189), sum(profit#190)] +Aggregate Attributes [3]: [sum(sales#188)#141, sum(returns#189)#142, sum(profit#190)#143] +Results [3]: [sum(sales#188)#141 AS sales#191, sum(returns#189)#142 AS returns#192, sum(profit#190)#143 AS profit#193] + +(76) HashAggregate [codegen id : 43] +Input [3]: [sales#191, returns#192, profit#193] +Keys: [] +Functions [3]: [partial_sum(sales#191), partial_sum(returns#192), partial_sum(profit#193)] +Aggregate Attributes [6]: [sum#194, isEmpty#195, sum#196, isEmpty#197, sum#198, isEmpty#199] +Results [6]: [sum#200, isEmpty#201, sum#202, isEmpty#203, sum#204, isEmpty#205] + +(77) Exchange +Input [6]: [sum#200, isEmpty#201, sum#202, isEmpty#203, sum#204, isEmpty#205] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=9] + +(78) HashAggregate [codegen id : 44] +Input [6]: [sum#200, isEmpty#201, sum#202, isEmpty#203, sum#204, isEmpty#205] +Keys: [] +Functions [3]: [sum(sales#191), sum(returns#192), sum(profit#193)] +Aggregate Attributes [3]: [sum(sales#191)#206, sum(returns#192)#207, sum(profit#193)#208] +Results [5]: [null AS channel#209, null AS id#210, sum(sales#191)#206 AS sum(sales)#211, sum(returns#192)#207 AS sum(returns)#212, sum(profit#193)#208 AS sum(profit)#213] + +(79) Union + +(80) HashAggregate [codegen id : 45] +Input [5]: [channel#37, id#38, sales#144, returns#145, profit#146] +Keys [5]: [channel#37, id#38, sales#144, returns#145, profit#146] +Functions: [] +Aggregate Attributes: [] +Results [5]: [channel#37, id#38, sales#144, returns#145, profit#146] + +(81) Exchange +Input [5]: [channel#37, id#38, sales#144, returns#145, profit#146] +Arguments: hashpartitioning(channel#37, id#38, sales#144, returns#145, profit#146, 5), ENSURE_REQUIREMENTS, [plan_id=10] + +(82) HashAggregate [codegen id : 46] +Input [5]: [channel#37, id#38, sales#144, returns#145, profit#146] +Keys [5]: [channel#37, id#38, sales#144, returns#145, profit#146] +Functions: [] +Aggregate Attributes: [] +Results [5]: [channel#37, id#38, sales#144, returns#145, profit#146] + +(83) TakeOrderedAndProject +Input [5]: [channel#37, id#38, sales#144, returns#145, profit#146] +Arguments: 100, [channel#37 ASC NULLS FIRST, id#38 ASC NULLS FIRST], [channel#37, id#38, sales#144, returns#145, profit#146] + +===== Subqueries ===== + +Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 +BroadcastExchange (88) ++- * ColumnarToRow (87) + +- CometProject (86) + +- CometFilter (85) + +- CometScan parquet spark_catalog.default.date_dim (84) + + +(84) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#22, d_date#214] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,1998-08-04), LessThanOrEqual(d_date,1998-08-18), IsNotNull(d_date_sk)] +ReadSchema: struct<d_date_sk:int,d_date:date> + +(85) CometFilter +Input [2]: [d_date_sk#22, d_date#214] +Condition : (((isnotnull(d_date#214) AND (d_date#214 >= 1998-08-04)) AND (d_date#214 <= 1998-08-18)) AND isnotnull(d_date_sk#22)) + +(86) CometProject +Input [2]: [d_date_sk#22, d_date#214] +Arguments: [d_date_sk#22], [d_date_sk#22] + +(87) ColumnarToRow [codegen id : 1] +Input [1]: [d_date_sk#22] + +(88) BroadcastExchange +Input [1]: [d_date_sk#22] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=11] + +Subquery:2 Hosting operator id = 4 Hosting Expression = sr_returned_date_sk#15 IN dynamicpruning#5 + +Subquery:3 Hosting operator id = 21 Hosting Expression = cs_sold_date_sk#45 IN dynamicpruning#5 + +Subquery:4 Hosting operator id = 24 Hosting Expression = cr_returned_date_sk#56 IN dynamicpruning#5 + +Subquery:5 Hosting operator id = 41 Hosting Expression = ws_sold_date_sk#86 IN dynamicpruning#5 + +Subquery:6 Hosting operator id = 44 Hosting Expression = wr_returned_date_sk#98 IN dynamicpruning#5 + + diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q5a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q5a/simplified.txt new file mode 100644 index 000000000..aaec304fd --- /dev/null +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q5a/simplified.txt @@ -0,0 +1,136 @@ +TakeOrderedAndProject [channel,id,sales,returns,profit] + WholeStageCodegen (46) + HashAggregate [channel,id,sales,returns,profit] + InputAdapter + Exchange [channel,id,sales,returns,profit] #1 + WholeStageCodegen (45) + HashAggregate [channel,id,sales,returns,profit] + InputAdapter + Union + WholeStageCodegen (14) + HashAggregate [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),sales,returns,profit,sum,isEmpty,sum,isEmpty,sum,isEmpty] + InputAdapter + Exchange [channel,id] #2 + WholeStageCodegen (13) + HashAggregate [channel,id,sales,returns,profit] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] + InputAdapter + Union + WholeStageCodegen (4) + HashAggregate [s_store_id,sum,sum,sum,sum] [sum(UnscaledValue(sales_price)),sum(UnscaledValue(return_amt)),sum(UnscaledValue(profit)),sum(UnscaledValue(net_loss)),channel,id,sales,returns,profit,sum,sum,sum,sum] + InputAdapter + Exchange [s_store_id] #3 + WholeStageCodegen (3) + HashAggregate [s_store_id,sales_price,return_amt,profit,net_loss] [sum,sum,sum,sum,sum,sum,sum,sum] + Project [sales_price,profit,return_amt,net_loss,s_store_id] + BroadcastHashJoin [store_sk,s_store_sk] + Project [store_sk,sales_price,profit,return_amt,net_loss] + BroadcastHashJoin [date_sk,d_date_sk] + ColumnarToRow + InputAdapter + CometUnion + CometProject [ss_store_sk,ss_sold_date_sk,ss_ext_sales_price,ss_net_profit] [store_sk,date_sk,sales_price,profit,return_amt,net_loss] + CometFilter [ss_store_sk] + CometScan parquet spark_catalog.default.store_sales [ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #4 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometProject [sr_store_sk,sr_returned_date_sk,sr_return_amt,sr_net_loss] [store_sk,date_sk,sales_price,profit,return_amt,net_loss] + CometFilter [sr_store_sk] + CometScan parquet spark_catalog.default.store_returns [sr_store_sk,sr_return_amt,sr_net_loss,sr_returned_date_sk] + ReusedSubquery [d_date_sk] #1 + InputAdapter + ReusedExchange [d_date_sk] #4 + InputAdapter + BroadcastExchange #5 + WholeStageCodegen (2) + ColumnarToRow + InputAdapter + CometFilter [s_store_sk] + CometScan parquet spark_catalog.default.store [s_store_sk,s_store_id] + WholeStageCodegen (8) + HashAggregate [cp_catalog_page_id,sum,sum,sum,sum] [sum(UnscaledValue(sales_price)),sum(UnscaledValue(return_amt)),sum(UnscaledValue(profit)),sum(UnscaledValue(net_loss)),channel,id,sales,returns,profit,sum,sum,sum,sum] + InputAdapter + Exchange [cp_catalog_page_id] #6 + WholeStageCodegen (7) + HashAggregate [cp_catalog_page_id,sales_price,return_amt,profit,net_loss] [sum,sum,sum,sum,sum,sum,sum,sum] + Project [sales_price,profit,return_amt,net_loss,cp_catalog_page_id] + BroadcastHashJoin [page_sk,cp_catalog_page_sk] + Project [page_sk,sales_price,profit,return_amt,net_loss] + BroadcastHashJoin [date_sk,d_date_sk] + ColumnarToRow + InputAdapter + CometUnion + CometProject [cs_catalog_page_sk,cs_sold_date_sk,cs_ext_sales_price,cs_net_profit] [page_sk,date_sk,sales_price,profit,return_amt,net_loss] + CometFilter [cs_catalog_page_sk] + CometScan parquet spark_catalog.default.catalog_sales [cs_catalog_page_sk,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + CometProject [cr_catalog_page_sk,cr_returned_date_sk,cr_return_amount,cr_net_loss] [page_sk,date_sk,sales_price,profit,return_amt,net_loss] + CometFilter [cr_catalog_page_sk] + CometScan parquet spark_catalog.default.catalog_returns [cr_catalog_page_sk,cr_return_amount,cr_net_loss,cr_returned_date_sk] + ReusedSubquery [d_date_sk] #1 + InputAdapter + ReusedExchange [d_date_sk] #4 + InputAdapter + BroadcastExchange #7 + WholeStageCodegen (6) + ColumnarToRow + InputAdapter + CometFilter [cp_catalog_page_sk] + CometScan parquet spark_catalog.default.catalog_page [cp_catalog_page_sk,cp_catalog_page_id] + WholeStageCodegen (12) + HashAggregate [web_site_id,sum,sum,sum,sum] [sum(UnscaledValue(sales_price)),sum(UnscaledValue(return_amt)),sum(UnscaledValue(profit)),sum(UnscaledValue(net_loss)),channel,id,sales,returns,profit,sum,sum,sum,sum] + InputAdapter + Exchange [web_site_id] #8 + WholeStageCodegen (11) + HashAggregate [web_site_id,sales_price,return_amt,profit,net_loss] [sum,sum,sum,sum,sum,sum,sum,sum] + Project [sales_price,profit,return_amt,net_loss,web_site_id] + BroadcastHashJoin [wsr_web_site_sk,web_site_sk] + Project [wsr_web_site_sk,sales_price,profit,return_amt,net_loss] + BroadcastHashJoin [date_sk,d_date_sk] + ColumnarToRow + InputAdapter + CometUnion + CometProject [ws_web_site_sk,ws_sold_date_sk,ws_ext_sales_price,ws_net_profit] [wsr_web_site_sk,date_sk,sales_price,profit,return_amt,net_loss] + CometFilter [ws_web_site_sk] + CometScan parquet spark_catalog.default.web_sales [ws_web_site_sk,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + CometProject [ws_web_site_sk,wr_returned_date_sk,wr_return_amt,wr_net_loss] [wsr_web_site_sk,date_sk,sales_price,profit,return_amt,net_loss] + CometBroadcastHashJoin [wr_item_sk,wr_order_number,ws_item_sk,ws_order_number] + CometBroadcastExchange #9 + CometScan parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk] + ReusedSubquery [d_date_sk] #1 + CometProject [ws_item_sk,ws_web_site_sk,ws_order_number] + CometFilter [ws_item_sk,ws_order_number,ws_web_site_sk] + CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_web_site_sk,ws_order_number,ws_sold_date_sk] + InputAdapter + ReusedExchange [d_date_sk] #4 + InputAdapter + BroadcastExchange #10 + WholeStageCodegen (10) + ColumnarToRow + InputAdapter + CometFilter [web_site_sk] + CometScan parquet spark_catalog.default.web_site [web_site_sk,web_site_id] + WholeStageCodegen (29) + HashAggregate [channel,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),id,sum(sales),sum(returns),sum(profit),sum,isEmpty,sum,isEmpty,sum,isEmpty] + InputAdapter + Exchange [channel] #11 + WholeStageCodegen (28) + HashAggregate [channel,sales,returns,profit] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] + HashAggregate [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),sales,returns,profit,sum,isEmpty,sum,isEmpty,sum,isEmpty] + InputAdapter + ReusedExchange [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] #2 + WholeStageCodegen (44) + HashAggregate [sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),channel,id,sum(sales),sum(returns),sum(profit),sum,isEmpty,sum,isEmpty,sum,isEmpty] + InputAdapter + Exchange #12 + WholeStageCodegen (43) + HashAggregate [sales,returns,profit] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] + HashAggregate [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),sales,returns,profit,sum,isEmpty,sum,isEmpty,sum,isEmpty] + InputAdapter + ReusedExchange [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] #2 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 new file mode 100644 index 000000000..a71a3a875 --- /dev/null +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q6/explain.txt @@ -0,0 +1,309 @@ +== Physical Plan == +TakeOrderedAndProject (39) ++- * Filter (38) + +- * HashAggregate (37) + +- Exchange (36) + +- * HashAggregate (35) + +- * Project (34) + +- * BroadcastHashJoin Inner BuildRight (33) + :- * Project (18) + : +- * BroadcastHashJoin Inner BuildRight (17) + : :- * Project (15) + : : +- * BroadcastHashJoin Inner BuildRight (14) + : : :- * Project (9) + : : : +- * BroadcastHashJoin Inner BuildRight (8) + : : : :- * ColumnarToRow (3) + : : : : +- CometFilter (2) + : : : : +- CometScan parquet spark_catalog.default.customer_address (1) + : : : +- BroadcastExchange (7) + : : : +- * ColumnarToRow (6) + : : : +- CometFilter (5) + : : : +- CometScan parquet spark_catalog.default.customer (4) + : : +- BroadcastExchange (13) + : : +- * ColumnarToRow (12) + : : +- CometFilter (11) + : : +- CometScan parquet spark_catalog.default.store_sales (10) + : +- ReusedExchange (16) + +- BroadcastExchange (32) + +- * Project (31) + +- * BroadcastHashJoin Inner BuildRight (30) + :- * ColumnarToRow (21) + : +- CometFilter (20) + : +- CometScan parquet spark_catalog.default.item (19) + +- BroadcastExchange (29) + +- * Filter (28) + +- * HashAggregate (27) + +- Exchange (26) + +- * HashAggregate (25) + +- * ColumnarToRow (24) + +- CometFilter (23) + +- CometScan parquet spark_catalog.default.item (22) + + +(1) Scan parquet spark_catalog.default.customer_address +Output [2]: [ca_address_sk#1, ca_state#2] +Batched: true +Location [not included in comparison]/{warehouse_dir}/customer_address] +PushedFilters: [IsNotNull(ca_address_sk)] +ReadSchema: struct<ca_address_sk:int,ca_state:string> + +(2) CometFilter +Input [2]: [ca_address_sk#1, ca_state#2] +Condition : isnotnull(ca_address_sk#1) + +(3) ColumnarToRow [codegen id : 7] +Input [2]: [ca_address_sk#1, ca_state#2] + +(4) Scan parquet spark_catalog.default.customer +Output [2]: [c_customer_sk#3, c_current_addr_sk#4] +Batched: true +Location [not included in comparison]/{warehouse_dir}/customer] +PushedFilters: [IsNotNull(c_current_addr_sk), IsNotNull(c_customer_sk)] +ReadSchema: struct<c_customer_sk:int,c_current_addr_sk:int> + +(5) CometFilter +Input [2]: [c_customer_sk#3, c_current_addr_sk#4] +Condition : (isnotnull(c_current_addr_sk#4) AND isnotnull(c_customer_sk#3)) + +(6) ColumnarToRow [codegen id : 1] +Input [2]: [c_customer_sk#3, c_current_addr_sk#4] + +(7) BroadcastExchange +Input [2]: [c_customer_sk#3, c_current_addr_sk#4] +Arguments: HashedRelationBroadcastMode(List(cast(input[1, int, false] as bigint)),false), [plan_id=1] + +(8) BroadcastHashJoin [codegen id : 7] +Left keys [1]: [ca_address_sk#1] +Right keys [1]: [c_current_addr_sk#4] +Join type: Inner +Join condition: None + +(9) Project [codegen id : 7] +Output [2]: [ca_state#2, c_customer_sk#3] +Input [4]: [ca_address_sk#1, ca_state#2, c_customer_sk#3, c_current_addr_sk#4] + +(10) Scan parquet spark_catalog.default.store_sales +Output [3]: [ss_item_sk#5, ss_customer_sk#6, ss_sold_date_sk#7] +Batched: true +Location: InMemoryFileIndex [] +PartitionFilters: [isnotnull(ss_sold_date_sk#7), dynamicpruningexpression(ss_sold_date_sk#7 IN dynamicpruning#8)] +PushedFilters: [IsNotNull(ss_customer_sk), IsNotNull(ss_item_sk)] +ReadSchema: struct<ss_item_sk:int,ss_customer_sk:int> + +(11) CometFilter +Input [3]: [ss_item_sk#5, ss_customer_sk#6, ss_sold_date_sk#7] +Condition : (isnotnull(ss_customer_sk#6) AND isnotnull(ss_item_sk#5)) + +(12) ColumnarToRow [codegen id : 2] +Input [3]: [ss_item_sk#5, ss_customer_sk#6, ss_sold_date_sk#7] + +(13) BroadcastExchange +Input [3]: [ss_item_sk#5, ss_customer_sk#6, ss_sold_date_sk#7] +Arguments: HashedRelationBroadcastMode(List(cast(input[1, int, false] as bigint)),false), [plan_id=2] + +(14) BroadcastHashJoin [codegen id : 7] +Left keys [1]: [c_customer_sk#3] +Right keys [1]: [ss_customer_sk#6] +Join type: Inner +Join condition: None + +(15) Project [codegen id : 7] +Output [3]: [ca_state#2, ss_item_sk#5, ss_sold_date_sk#7] +Input [5]: [ca_state#2, c_customer_sk#3, ss_item_sk#5, ss_customer_sk#6, ss_sold_date_sk#7] + +(16) ReusedExchange [Reuses operator id: 44] +Output [1]: [d_date_sk#9] + +(17) BroadcastHashJoin [codegen id : 7] +Left keys [1]: [ss_sold_date_sk#7] +Right keys [1]: [d_date_sk#9] +Join type: Inner +Join condition: None + +(18) Project [codegen id : 7] +Output [2]: [ca_state#2, ss_item_sk#5] +Input [4]: [ca_state#2, ss_item_sk#5, ss_sold_date_sk#7, d_date_sk#9] + +(19) Scan parquet spark_catalog.default.item +Output [3]: [i_item_sk#10, i_current_price#11, i_category#12] +Batched: true +Location [not included in comparison]/{warehouse_dir}/item] +PushedFilters: [IsNotNull(i_current_price), IsNotNull(i_category), IsNotNull(i_item_sk)] +ReadSchema: struct<i_item_sk:int,i_current_price:decimal(7,2),i_category:string> + +(20) CometFilter +Input [3]: [i_item_sk#10, i_current_price#11, i_category#12] +Condition : ((isnotnull(i_current_price#11) AND isnotnull(i_category#12)) AND isnotnull(i_item_sk#10)) + +(21) ColumnarToRow [codegen id : 6] +Input [3]: [i_item_sk#10, i_current_price#11, i_category#12] + +(22) Scan parquet spark_catalog.default.item +Output [2]: [i_current_price#13, i_category#14] +Batched: true +Location [not included in comparison]/{warehouse_dir}/item] +PushedFilters: [IsNotNull(i_category)] +ReadSchema: struct<i_current_price:decimal(7,2),i_category:string> + +(23) CometFilter +Input [2]: [i_current_price#13, i_category#14] +Condition : isnotnull(i_category#14) + +(24) ColumnarToRow [codegen id : 4] +Input [2]: [i_current_price#13, i_category#14] + +(25) HashAggregate [codegen id : 4] +Input [2]: [i_current_price#13, i_category#14] +Keys [1]: [i_category#14] +Functions [1]: [partial_avg(UnscaledValue(i_current_price#13))] +Aggregate Attributes [2]: [sum#15, count#16] +Results [3]: [i_category#14, sum#17, count#18] + +(26) Exchange +Input [3]: [i_category#14, sum#17, count#18] +Arguments: hashpartitioning(i_category#14, 5), ENSURE_REQUIREMENTS, [plan_id=3] + +(27) HashAggregate [codegen id : 5] +Input [3]: [i_category#14, sum#17, count#18] +Keys [1]: [i_category#14] +Functions [1]: [avg(UnscaledValue(i_current_price#13))] +Aggregate Attributes [1]: [avg(UnscaledValue(i_current_price#13))#19] +Results [2]: [cast((avg(UnscaledValue(i_current_price#13))#19 / 100.0) as decimal(11,6)) AS avg(i_current_price)#20, i_category#14] + +(28) Filter [codegen id : 5] +Input [2]: [avg(i_current_price)#20, i_category#14] +Condition : isnotnull(avg(i_current_price)#20) + +(29) BroadcastExchange +Input [2]: [avg(i_current_price)#20, i_category#14] +Arguments: HashedRelationBroadcastMode(List(input[1, string, true]),false), [plan_id=4] + +(30) BroadcastHashJoin [codegen id : 6] +Left keys [1]: [i_category#12] +Right keys [1]: [i_category#14] +Join type: Inner +Join condition: (cast(i_current_price#11 as decimal(14,7)) > (1.2 * avg(i_current_price)#20)) + +(31) Project [codegen id : 6] +Output [1]: [i_item_sk#10] +Input [5]: [i_item_sk#10, i_current_price#11, i_category#12, avg(i_current_price)#20, i_category#14] + +(32) BroadcastExchange +Input [1]: [i_item_sk#10] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] + +(33) BroadcastHashJoin [codegen id : 7] +Left keys [1]: [ss_item_sk#5] +Right keys [1]: [i_item_sk#10] +Join type: Inner +Join condition: None + +(34) Project [codegen id : 7] +Output [1]: [ca_state#2] +Input [3]: [ca_state#2, ss_item_sk#5, i_item_sk#10] + +(35) HashAggregate [codegen id : 7] +Input [1]: [ca_state#2] +Keys [1]: [ca_state#2] +Functions [1]: [partial_count(1)] +Aggregate Attributes [1]: [count#21] +Results [2]: [ca_state#2, count#22] + +(36) Exchange +Input [2]: [ca_state#2, count#22] +Arguments: hashpartitioning(ca_state#2, 5), ENSURE_REQUIREMENTS, [plan_id=6] + +(37) HashAggregate [codegen id : 8] +Input [2]: [ca_state#2, count#22] +Keys [1]: [ca_state#2] +Functions [1]: [count(1)] +Aggregate Attributes [1]: [count(1)#23] +Results [2]: [ca_state#2 AS state#24, count(1)#23 AS cnt#25] + +(38) Filter [codegen id : 8] +Input [2]: [state#24, cnt#25] +Condition : (cnt#25 >= 10) + +(39) TakeOrderedAndProject +Input [2]: [state#24, cnt#25] +Arguments: 100, [cnt#25 ASC NULLS FIRST, state#24 ASC NULLS FIRST], [state#24, cnt#25] + +===== Subqueries ===== + +Subquery:1 Hosting operator id = 10 Hosting Expression = ss_sold_date_sk#7 IN dynamicpruning#8 +BroadcastExchange (44) ++- * ColumnarToRow (43) + +- CometProject (42) + +- CometFilter (41) + +- CometScan parquet spark_catalog.default.date_dim (40) + + +(40) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#9, d_month_seq#26] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_month_seq), EqualTo(d_month_seq,ScalarSubquery#27), IsNotNull(d_date_sk)] +ReadSchema: struct<d_date_sk:int,d_month_seq:int> + +(41) CometFilter +Input [2]: [d_date_sk#9, d_month_seq#26] +Condition : ((isnotnull(d_month_seq#26) AND (d_month_seq#26 = ReusedSubquery Subquery scalar-subquery#27, [id=#28])) AND isnotnull(d_date_sk#9)) + +(42) CometProject +Input [2]: [d_date_sk#9, d_month_seq#26] +Arguments: [d_date_sk#9], [d_date_sk#9] + +(43) ColumnarToRow [codegen id : 1] +Input [1]: [d_date_sk#9] + +(44) BroadcastExchange +Input [1]: [d_date_sk#9] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=7] + +Subquery:2 Hosting operator id = 41 Hosting Expression = ReusedSubquery Subquery scalar-subquery#27, [id=#28] + +Subquery:3 Hosting operator id = 40 Hosting Expression = Subquery scalar-subquery#27, [id=#28] +* HashAggregate (51) ++- Exchange (50) + +- * ColumnarToRow (49) + +- CometHashAggregate (48) + +- CometProject (47) + +- CometFilter (46) + +- CometScan parquet spark_catalog.default.date_dim (45) + + +(45) Scan parquet spark_catalog.default.date_dim +Output [3]: [d_month_seq#29, d_year#30, d_moy#31] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2000), EqualTo(d_moy,1)] +ReadSchema: struct<d_month_seq:int,d_year:int,d_moy:int> + +(46) CometFilter +Input [3]: [d_month_seq#29, d_year#30, d_moy#31] +Condition : (((isnotnull(d_year#30) AND isnotnull(d_moy#31)) AND (d_year#30 = 2000)) AND (d_moy#31 = 1)) + +(47) CometProject +Input [3]: [d_month_seq#29, d_year#30, d_moy#31] +Arguments: [d_month_seq#29], [d_month_seq#29] + +(48) CometHashAggregate +Input [1]: [d_month_seq#29] +Keys [1]: [d_month_seq#29] +Functions: [] + +(49) ColumnarToRow [codegen id : 1] +Input [1]: [d_month_seq#29] + +(50) Exchange +Input [1]: [d_month_seq#29] +Arguments: hashpartitioning(d_month_seq#29, 5), ENSURE_REQUIREMENTS, [plan_id=8] + +(51) HashAggregate [codegen id : 2] +Input [1]: [d_month_seq#29] +Keys [1]: [d_month_seq#29] +Functions: [] +Aggregate Attributes: [] +Results [1]: [d_month_seq#29] + + 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 new file mode 100644 index 000000000..c2d5a6ce8 --- /dev/null +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q6/simplified.txt @@ -0,0 +1,79 @@ +TakeOrderedAndProject [cnt,state] + WholeStageCodegen (8) + Filter [cnt] + HashAggregate [ca_state,count] [count(1),state,cnt,count] + InputAdapter + Exchange [ca_state] #1 + WholeStageCodegen (7) + HashAggregate [ca_state] [count,count] + Project [ca_state] + BroadcastHashJoin [ss_item_sk,i_item_sk] + Project [ca_state,ss_item_sk] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [ca_state,ss_item_sk,ss_sold_date_sk] + BroadcastHashJoin [c_customer_sk,ss_customer_sk] + Project [ca_state,c_customer_sk] + BroadcastHashJoin [ca_address_sk,c_current_addr_sk] + ColumnarToRow + InputAdapter + CometFilter [ca_address_sk] + CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_state] + InputAdapter + BroadcastExchange #2 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometFilter [c_current_addr_sk,c_customer_sk] + CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_addr_sk] + InputAdapter + BroadcastExchange #3 + WholeStageCodegen (2) + ColumnarToRow + InputAdapter + CometFilter [ss_customer_sk,ss_item_sk] + CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #4 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_month_seq,d_date_sk] + ReusedSubquery [d_month_seq] #2 + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] + Subquery #2 + WholeStageCodegen (2) + HashAggregate [d_month_seq] + InputAdapter + Exchange [d_month_seq] #5 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometHashAggregate [d_month_seq] + CometProject [d_month_seq] + CometFilter [d_year,d_moy] + CometScan parquet spark_catalog.default.date_dim [d_month_seq,d_year,d_moy] + InputAdapter + ReusedExchange [d_date_sk] #4 + InputAdapter + BroadcastExchange #6 + WholeStageCodegen (6) + Project [i_item_sk] + BroadcastHashJoin [i_category,i_category,i_current_price,avg(i_current_price)] + ColumnarToRow + InputAdapter + CometFilter [i_current_price,i_category,i_item_sk] + CometScan parquet spark_catalog.default.item [i_item_sk,i_current_price,i_category] + InputAdapter + BroadcastExchange #7 + WholeStageCodegen (5) + Filter [avg(i_current_price)] + HashAggregate [i_category,sum,count] [avg(UnscaledValue(i_current_price)),avg(i_current_price),sum,count] + InputAdapter + Exchange [i_category] #8 + WholeStageCodegen (4) + HashAggregate [i_category,i_current_price] [sum,count,sum,count] + ColumnarToRow + InputAdapter + CometFilter [i_category] + CometScan parquet spark_catalog.default.item [i_current_price,i_category] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q64/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q64/explain.txt new file mode 100644 index 000000000..e50a522b8 --- /dev/null +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q64/explain.txt @@ -0,0 +1,1064 @@ +== Physical Plan == +* Sort (181) ++- Exchange (180) + +- * Project (179) + +- * SortMergeJoin Inner (178) + :- * Sort (110) + : +- Exchange (109) + : +- * HashAggregate (108) + : +- * HashAggregate (107) + : +- * Project (106) + : +- * BroadcastHashJoin Inner BuildRight (105) + : :- * Project (99) + : : +- * BroadcastHashJoin Inner BuildRight (98) + : : :- * Project (96) + : : : +- * BroadcastHashJoin Inner BuildRight (95) + : : : :- * Project (90) + : : : : +- * BroadcastHashJoin Inner BuildRight (89) + : : : : :- * Project (87) + : : : : : +- * BroadcastHashJoin Inner BuildRight (86) + : : : : : :- * Project (81) + : : : : : : +- * BroadcastHashJoin Inner BuildRight (80) + : : : : : : :- * Project (78) + : : : : : : : +- * BroadcastHashJoin Inner BuildRight (77) + : : : : : : : :- * Project (72) + : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (71) + : : : : : : : : :- * Project (66) + : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (65) + : : : : : : : : : :- * Project (63) + : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (62) + : : : : : : : : : : :- * Project (57) + : : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (56) + : : : : : : : : : : : :- * Project (54) + : : : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (53) + : : : : : : : : : : : : :- * Project (48) + : : : : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (47) + : : : : : : : : : : : : : :- * Project (42) + : : : : : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (41) + : : : : : : : : : : : : : : :- * Project (36) + : : : : : : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (35) + : : : : : : : : : : : : : : : :- * Project (33) + : : : : : : : : : : : : : : : : +- * SortMergeJoin Inner (32) + : : : : : : : : : : : : : : : : :- * Sort (11) + : : : : : : : : : : : : : : : : : +- Exchange (10) + : : : : : : : : : : : : : : : : : +- * ColumnarToRow (9) + : : : : : : : : : : : : : : : : : +- CometProject (8) + : : : : : : : : : : : : : : : : : +- CometBroadcastHashJoin (7) + : : : : : : : : : : : : : : : : : :- CometBroadcastExchange (3) + : : : : : : : : : : : : : : : : : : +- CometFilter (2) + : : : : : : : : : : : : : : : : : : +- CometScan parquet spark_catalog.default.store_sales (1) + : : : : : : : : : : : : : : : : : +- CometProject (6) + : : : : : : : : : : : : : : : : : +- CometFilter (5) + : : : : : : : : : : : : : : : : : +- CometScan parquet spark_catalog.default.store_returns (4) + : : : : : : : : : : : : : : : : +- * Sort (31) + : : : : : : : : : : : : : : : : +- * Project (30) + : : : : : : : : : : : : : : : : +- * Filter (29) + : : : : : : : : : : : : : : : : +- * HashAggregate (28) + : : : : : : : : : : : : : : : : +- Exchange (27) + : : : : : : : : : : : : : : : : +- * HashAggregate (26) + : : : : : : : : : : : : : : : : +- * Project (25) + : : : : : : : : : : : : : : : : +- * SortMergeJoin Inner (24) + : : : : : : : : : : : : : : : : :- * Sort (17) + : : : : : : : : : : : : : : : : : +- Exchange (16) + : : : : : : : : : : : : : : : : : +- * ColumnarToRow (15) + : : : : : : : : : : : : : : : : : +- CometProject (14) + : : : : : : : : : : : : : : : : : +- CometFilter (13) + : : : : : : : : : : : : : : : : : +- CometScan parquet spark_catalog.default.catalog_sales (12) + : : : : : : : : : : : : : : : : +- * Sort (23) + : : : : : : : : : : : : : : : : +- Exchange (22) + : : : : : : : : : : : : : : : : +- * ColumnarToRow (21) + : : : : : : : : : : : : : : : : +- CometProject (20) + : : : : : : : : : : : : : : : : +- CometFilter (19) + : : : : : : : : : : : : : : : : +- CometScan parquet spark_catalog.default.catalog_returns (18) + : : : : : : : : : : : : : : : +- ReusedExchange (34) + : : : : : : : : : : : : : : +- BroadcastExchange (40) + : : : : : : : : : : : : : : +- * ColumnarToRow (39) + : : : : : : : : : : : : : : +- CometFilter (38) + : : : : : : : : : : : : : : +- CometScan parquet spark_catalog.default.store (37) + : : : : : : : : : : : : : +- BroadcastExchange (46) + : : : : : : : : : : : : : +- * ColumnarToRow (45) + : : : : : : : : : : : : : +- CometFilter (44) + : : : : : : : : : : : : : +- CometScan parquet spark_catalog.default.customer (43) + : : : : : : : : : : : : +- BroadcastExchange (52) + : : : : : : : : : : : : +- * ColumnarToRow (51) + : : : : : : : : : : : : +- CometFilter (50) + : : : : : : : : : : : : +- CometScan parquet spark_catalog.default.date_dim (49) + : : : : : : : : : : : +- ReusedExchange (55) + : : : : : : : : : : +- BroadcastExchange (61) + : : : : : : : : : : +- * ColumnarToRow (60) + : : : : : : : : : : +- CometFilter (59) + : : : : : : : : : : +- CometScan parquet spark_catalog.default.customer_demographics (58) + : : : : : : : : : +- ReusedExchange (64) + : : : : : : : : +- BroadcastExchange (70) + : : : : : : : : +- * ColumnarToRow (69) + : : : : : : : : +- CometFilter (68) + : : : : : : : : +- CometScan parquet spark_catalog.default.promotion (67) + : : : : : : : +- BroadcastExchange (76) + : : : : : : : +- * ColumnarToRow (75) + : : : : : : : +- CometFilter (74) + : : : : : : : +- CometScan parquet spark_catalog.default.household_demographics (73) + : : : : : : +- ReusedExchange (79) + : : : : : +- BroadcastExchange (85) + : : : : : +- * ColumnarToRow (84) + : : : : : +- CometFilter (83) + : : : : : +- CometScan parquet spark_catalog.default.customer_address (82) + : : : : +- ReusedExchange (88) + : : : +- BroadcastExchange (94) + : : : +- * ColumnarToRow (93) + : : : +- CometFilter (92) + : : : +- CometScan parquet spark_catalog.default.income_band (91) + : : +- ReusedExchange (97) + : +- BroadcastExchange (104) + : +- * ColumnarToRow (103) + : +- CometProject (102) + : +- CometFilter (101) + : +- CometScan parquet spark_catalog.default.item (100) + +- * Sort (177) + +- Exchange (176) + +- * HashAggregate (175) + +- * HashAggregate (174) + +- * Project (173) + +- * BroadcastHashJoin Inner BuildRight (172) + :- * Project (170) + : +- * BroadcastHashJoin Inner BuildRight (169) + : :- * Project (167) + : : +- * BroadcastHashJoin Inner BuildRight (166) + : : :- * Project (164) + : : : +- * BroadcastHashJoin Inner BuildRight (163) + : : : :- * Project (161) + : : : : +- * BroadcastHashJoin Inner BuildRight (160) + : : : : :- * Project (158) + : : : : : +- * BroadcastHashJoin Inner BuildRight (157) + : : : : : :- * Project (155) + : : : : : : +- * BroadcastHashJoin Inner BuildRight (154) + : : : : : : :- * Project (152) + : : : : : : : +- * BroadcastHashJoin Inner BuildRight (151) + : : : : : : : :- * Project (149) + : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (148) + : : : : : : : : :- * Project (146) + : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (145) + : : : : : : : : : :- * Project (143) + : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (142) + : : : : : : : : : : :- * Project (140) + : : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (139) + : : : : : : : : : : : :- * Project (137) + : : : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (136) + : : : : : : : : : : : : :- * Project (134) + : : : : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (133) + : : : : : : : : : : : : : :- * Project (131) + : : : : : : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (130) + : : : : : : : : : : : : : : :- * Project (128) + : : : : : : : : : : : : : : : +- * SortMergeJoin Inner (127) + : : : : : : : : : : : : : : : :- * Sort (121) + : : : : : : : : : : : : : : : : +- Exchange (120) + : : : : : : : : : : : : : : : : +- * ColumnarToRow (119) + : : : : : : : : : : : : : : : : +- CometProject (118) + : : : : : : : : : : : : : : : : +- CometBroadcastHashJoin (117) + : : : : : : : : : : : : : : : : :- CometBroadcastExchange (113) + : : : : : : : : : : : : : : : : : +- CometFilter (112) + : : : : : : : : : : : : : : : : : +- CometScan parquet spark_catalog.default.store_sales (111) + : : : : : : : : : : : : : : : : +- CometProject (116) + : : : : : : : : : : : : : : : : +- CometFilter (115) + : : : : : : : : : : : : : : : : +- CometScan parquet spark_catalog.default.store_returns (114) + : : : : : : : : : : : : : : : +- * Sort (126) + : : : : : : : : : : : : : : : +- * Project (125) + : : : : : : : : : : : : : : : +- * Filter (124) + : : : : : : : : : : : : : : : +- * HashAggregate (123) + : : : : : : : : : : : : : : : +- ReusedExchange (122) + : : : : : : : : : : : : : : +- ReusedExchange (129) + : : : : : : : : : : : : : +- ReusedExchange (132) + : : : : : : : : : : : : +- ReusedExchange (135) + : : : : : : : : : : : +- ReusedExchange (138) + : : : : : : : : : : +- ReusedExchange (141) + : : : : : : : : : +- ReusedExchange (144) + : : : : : : : : +- ReusedExchange (147) + : : : : : : : +- ReusedExchange (150) + : : : : : : +- ReusedExchange (153) + : : : : : +- ReusedExchange (156) + : : : : +- ReusedExchange (159) + : : : +- ReusedExchange (162) + : : +- ReusedExchange (165) + : +- ReusedExchange (168) + +- ReusedExchange (171) + + +(1) Scan parquet spark_catalog.default.store_sales +Output [12]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_ticket_number#8, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] +Batched: true +Location: InMemoryFileIndex [] +PartitionFilters: [isnotnull(ss_sold_date_sk#12), dynamicpruningexpression(ss_sold_date_sk#12 IN dynamicpruning#13)] +PushedFilters: [IsNotNull(ss_item_sk), IsNotNull(ss_ticket_number), IsNotNull(ss_store_sk), IsNotNull(ss_customer_sk), IsNotNull(ss_cdemo_sk), IsNotNull(ss_promo_sk), IsNotNull(ss_hdemo_sk), IsNotNull(ss_addr_sk)] +ReadSchema: struct<ss_item_sk:int,ss_customer_sk:int,ss_cdemo_sk:int,ss_hdemo_sk:int,ss_addr_sk:int,ss_store_sk:int,ss_promo_sk:int,ss_ticket_number:int,ss_wholesale_cost:decimal(7,2),ss_list_price:decimal(7,2),ss_coupon_amt:decimal(7,2)> + +(2) CometFilter +Input [12]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_ticket_number#8, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] +Condition : (((((((isnotnull(ss_item_sk#1) AND isnotnull(ss_ticket_number#8)) AND isnotnull(ss_store_sk#6)) AND isnotnull(ss_customer_sk#2)) AND isnotnull(ss_cdemo_sk#3)) AND isnotnull(ss_promo_sk#7)) AND isnotnull(ss_hdemo_sk#4)) AND isnotnull(ss_addr_sk#5)) + +(3) CometBroadcastExchange +Input [12]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_ticket_number#8, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] +Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_ticket_number#8, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] + +(4) Scan parquet spark_catalog.default.store_returns +Output [3]: [sr_item_sk#14, sr_ticket_number#15, sr_returned_date_sk#16] +Batched: true +Location [not included in comparison]/{warehouse_dir}/store_returns] +PushedFilters: [IsNotNull(sr_item_sk), IsNotNull(sr_ticket_number)] +ReadSchema: struct<sr_item_sk:int,sr_ticket_number:int> + +(5) CometFilter +Input [3]: [sr_item_sk#14, sr_ticket_number#15, sr_returned_date_sk#16] +Condition : (isnotnull(sr_item_sk#14) AND isnotnull(sr_ticket_number#15)) + +(6) CometProject +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] + +(7) CometBroadcastHashJoin +Left output [12]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_ticket_number#8, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] +Right output [2]: [sr_item_sk#14, sr_ticket_number#15] +Arguments: [ss_item_sk#1, ss_ticket_number#8], [sr_item_sk#14, sr_ticket_number#15], Inner + +(8) CometProject +Input [14]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_ticket_number#8, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12, sr_item_sk#14, sr_ticket_number#15] +Arguments: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12], [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] + +(9) ColumnarToRow [codegen id : 1] +Input [11]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] + +(10) Exchange +Input [11]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] +Arguments: hashpartitioning(ss_item_sk#1, 5), ENSURE_REQUIREMENTS, [plan_id=1] + +(11) Sort [codegen id : 2] +Input [11]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] +Arguments: [ss_item_sk#1 ASC NULLS FIRST], false, 0 + +(12) Scan parquet spark_catalog.default.catalog_sales +Output [4]: [cs_item_sk#17, cs_order_number#18, cs_ext_list_price#19, cs_sold_date_sk#20] +Batched: true +Location [not included in comparison]/{warehouse_dir}/catalog_sales] +PushedFilters: [IsNotNull(cs_item_sk), IsNotNull(cs_order_number)] +ReadSchema: struct<cs_item_sk:int,cs_order_number:int,cs_ext_list_price:decimal(7,2)> + +(13) CometFilter +Input [4]: [cs_item_sk#17, cs_order_number#18, cs_ext_list_price#19, cs_sold_date_sk#20] +Condition : (isnotnull(cs_item_sk#17) AND isnotnull(cs_order_number#18)) + +(14) CometProject +Input [4]: [cs_item_sk#17, cs_order_number#18, cs_ext_list_price#19, cs_sold_date_sk#20] +Arguments: [cs_item_sk#17, cs_order_number#18, cs_ext_list_price#19], [cs_item_sk#17, cs_order_number#18, cs_ext_list_price#19] + +(15) ColumnarToRow [codegen id : 3] +Input [3]: [cs_item_sk#17, cs_order_number#18, cs_ext_list_price#19] + +(16) Exchange +Input [3]: [cs_item_sk#17, cs_order_number#18, cs_ext_list_price#19] +Arguments: hashpartitioning(cs_item_sk#17, cs_order_number#18, 5), ENSURE_REQUIREMENTS, [plan_id=2] + +(17) Sort [codegen id : 4] +Input [3]: [cs_item_sk#17, cs_order_number#18, cs_ext_list_price#19] +Arguments: [cs_item_sk#17 ASC NULLS FIRST, cs_order_number#18 ASC NULLS FIRST], false, 0 + +(18) Scan parquet spark_catalog.default.catalog_returns +Output [6]: [cr_item_sk#21, cr_order_number#22, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25, cr_returned_date_sk#26] +Batched: true +Location [not included in comparison]/{warehouse_dir}/catalog_returns] +PushedFilters: [IsNotNull(cr_item_sk), IsNotNull(cr_order_number)] +ReadSchema: struct<cr_item_sk:int,cr_order_number:int,cr_refunded_cash:decimal(7,2),cr_reversed_charge:decimal(7,2),cr_store_credit:decimal(7,2)> + +(19) CometFilter +Input [6]: [cr_item_sk#21, cr_order_number#22, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25, cr_returned_date_sk#26] +Condition : (isnotnull(cr_item_sk#21) AND isnotnull(cr_order_number#22)) + +(20) CometProject +Input [6]: [cr_item_sk#21, cr_order_number#22, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25, cr_returned_date_sk#26] +Arguments: [cr_item_sk#21, cr_order_number#22, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25], [cr_item_sk#21, cr_order_number#22, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25] + +(21) ColumnarToRow [codegen id : 5] +Input [5]: [cr_item_sk#21, cr_order_number#22, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25] + +(22) Exchange +Input [5]: [cr_item_sk#21, cr_order_number#22, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25] +Arguments: hashpartitioning(cr_item_sk#21, cr_order_number#22, 5), ENSURE_REQUIREMENTS, [plan_id=3] + +(23) Sort [codegen id : 6] +Input [5]: [cr_item_sk#21, cr_order_number#22, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25] +Arguments: [cr_item_sk#21 ASC NULLS FIRST, cr_order_number#22 ASC NULLS FIRST], false, 0 + +(24) SortMergeJoin [codegen id : 7] +Left keys [2]: [cs_item_sk#17, cs_order_number#18] +Right keys [2]: [cr_item_sk#21, cr_order_number#22] +Join type: Inner +Join condition: None + +(25) Project [codegen id : 7] +Output [5]: [cs_item_sk#17, cs_ext_list_price#19, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25] +Input [8]: [cs_item_sk#17, cs_order_number#18, cs_ext_list_price#19, cr_item_sk#21, cr_order_number#22, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25] + +(26) HashAggregate [codegen id : 7] +Input [5]: [cs_item_sk#17, cs_ext_list_price#19, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25] +Keys [1]: [cs_item_sk#17] +Functions [2]: [partial_sum(UnscaledValue(cs_ext_list_price#19)), partial_sum(((cr_refunded_cash#23 + cr_reversed_charge#24) + cr_store_credit#25))] +Aggregate Attributes [3]: [sum#27, sum#28, isEmpty#29] +Results [4]: [cs_item_sk#17, sum#30, sum#31, isEmpty#32] + +(27) Exchange +Input [4]: [cs_item_sk#17, sum#30, sum#31, isEmpty#32] +Arguments: hashpartitioning(cs_item_sk#17, 5), ENSURE_REQUIREMENTS, [plan_id=4] + +(28) HashAggregate [codegen id : 8] +Input [4]: [cs_item_sk#17, sum#30, sum#31, isEmpty#32] +Keys [1]: [cs_item_sk#17] +Functions [2]: [sum(UnscaledValue(cs_ext_list_price#19)), sum(((cr_refunded_cash#23 + cr_reversed_charge#24) + cr_store_credit#25))] +Aggregate Attributes [2]: [sum(UnscaledValue(cs_ext_list_price#19))#33, sum(((cr_refunded_cash#23 + cr_reversed_charge#24) + cr_store_credit#25))#34] +Results [3]: [cs_item_sk#17, MakeDecimal(sum(UnscaledValue(cs_ext_list_price#19))#33,17,2) AS sale#35, sum(((cr_refunded_cash#23 + cr_reversed_charge#24) + cr_store_credit#25))#34 AS refund#36] + +(29) Filter [codegen id : 8] +Input [3]: [cs_item_sk#17, sale#35, refund#36] +Condition : ((isnotnull(sale#35) AND isnotnull(refund#36)) AND (cast(sale#35 as decimal(21,2)) > (2 * refund#36))) + +(30) Project [codegen id : 8] +Output [1]: [cs_item_sk#17] +Input [3]: [cs_item_sk#17, sale#35, refund#36] + +(31) Sort [codegen id : 8] +Input [1]: [cs_item_sk#17] +Arguments: [cs_item_sk#17 ASC NULLS FIRST], false, 0 + +(32) SortMergeJoin [codegen id : 24] +Left keys [1]: [ss_item_sk#1] +Right keys [1]: [cs_item_sk#17] +Join type: Inner +Join condition: None + +(33) Project [codegen id : 24] +Output [11]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12] +Input [12]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12, cs_item_sk#17] + +(34) ReusedExchange [Reuses operator id: 185] +Output [2]: [d_date_sk#37, d_year#38] + +(35) BroadcastHashJoin [codegen id : 24] +Left keys [1]: [ss_sold_date_sk#12] +Right keys [1]: [d_date_sk#37] +Join type: Inner +Join condition: None + +(36) Project [codegen id : 24] +Output [11]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#38] +Input [13]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, ss_sold_date_sk#12, d_date_sk#37, d_year#38] + +(37) Scan parquet spark_catalog.default.store +Output [3]: [s_store_sk#39, s_store_name#40, s_zip#41] +Batched: true +Location [not included in comparison]/{warehouse_dir}/store] +PushedFilters: [IsNotNull(s_store_sk), IsNotNull(s_store_name), IsNotNull(s_zip)] +ReadSchema: struct<s_store_sk:int,s_store_name:string,s_zip:string> + +(38) CometFilter +Input [3]: [s_store_sk#39, s_store_name#40, s_zip#41] +Condition : ((isnotnull(s_store_sk#39) AND isnotnull(s_store_name#40)) AND isnotnull(s_zip#41)) + +(39) ColumnarToRow [codegen id : 10] +Input [3]: [s_store_sk#39, s_store_name#40, s_zip#41] + +(40) BroadcastExchange +Input [3]: [s_store_sk#39, s_store_name#40, s_zip#41] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=5] + +(41) BroadcastHashJoin [codegen id : 24] +Left keys [1]: [ss_store_sk#6] +Right keys [1]: [s_store_sk#39] +Join type: Inner +Join condition: None + +(42) Project [codegen id : 24] +Output [12]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#38, s_store_name#40, s_zip#41] +Input [14]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_store_sk#6, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#38, s_store_sk#39, s_store_name#40, s_zip#41] + +(43) Scan parquet spark_catalog.default.customer +Output [6]: [c_customer_sk#42, c_current_cdemo_sk#43, c_current_hdemo_sk#44, c_current_addr_sk#45, c_first_shipto_date_sk#46, c_first_sales_date_sk#47] +Batched: true +Location [not included in comparison]/{warehouse_dir}/customer] +PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_first_sales_date_sk), IsNotNull(c_first_shipto_date_sk), IsNotNull(c_current_cdemo_sk), IsNotNull(c_current_hdemo_sk), IsNotNull(c_current_addr_sk)] +ReadSchema: struct<c_customer_sk:int,c_current_cdemo_sk:int,c_current_hdemo_sk:int,c_current_addr_sk:int,c_first_shipto_date_sk:int,c_first_sales_date_sk:int> + +(44) CometFilter +Input [6]: [c_customer_sk#42, c_current_cdemo_sk#43, c_current_hdemo_sk#44, c_current_addr_sk#45, c_first_shipto_date_sk#46, c_first_sales_date_sk#47] +Condition : (((((isnotnull(c_customer_sk#42) AND isnotnull(c_first_sales_date_sk#47)) AND isnotnull(c_first_shipto_date_sk#46)) AND isnotnull(c_current_cdemo_sk#43)) AND isnotnull(c_current_hdemo_sk#44)) AND isnotnull(c_current_addr_sk#45)) + +(45) ColumnarToRow [codegen id : 11] +Input [6]: [c_customer_sk#42, c_current_cdemo_sk#43, c_current_hdemo_sk#44, c_current_addr_sk#45, c_first_shipto_date_sk#46, c_first_sales_date_sk#47] + +(46) BroadcastExchange +Input [6]: [c_customer_sk#42, c_current_cdemo_sk#43, c_current_hdemo_sk#44, c_current_addr_sk#45, c_first_shipto_date_sk#46, c_first_sales_date_sk#47] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=6] + +(47) BroadcastHashJoin [codegen id : 24] +Left keys [1]: [ss_customer_sk#2] +Right keys [1]: [c_customer_sk#42] +Join type: Inner +Join condition: None + +(48) Project [codegen id : 24] +Output [16]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#38, s_store_name#40, s_zip#41, c_current_cdemo_sk#43, c_current_hdemo_sk#44, c_current_addr_sk#45, c_first_shipto_date_sk#46, c_first_sales_date_sk#47] +Input [18]: [ss_item_sk#1, ss_customer_sk#2, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#38, s_store_name#40, s_zip#41, c_customer_sk#42, c_current_cdemo_sk#43, c_current_hdemo_sk#44, c_current_addr_sk#45, c_first_shipto_date_sk#46, c_first_sales_date_sk#47] + +(49) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#48, d_year#49] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_date_sk)] +ReadSchema: struct<d_date_sk:int,d_year:int> + +(50) CometFilter +Input [2]: [d_date_sk#48, d_year#49] +Condition : isnotnull(d_date_sk#48) + +(51) ColumnarToRow [codegen id : 12] +Input [2]: [d_date_sk#48, d_year#49] + +(52) BroadcastExchange +Input [2]: [d_date_sk#48, d_year#49] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=7] + +(53) BroadcastHashJoin [codegen id : 24] +Left keys [1]: [c_first_sales_date_sk#47] +Right keys [1]: [d_date_sk#48] +Join type: Inner +Join condition: None + +(54) Project [codegen id : 24] +Output [16]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#38, s_store_name#40, s_zip#41, c_current_cdemo_sk#43, c_current_hdemo_sk#44, c_current_addr_sk#45, c_first_shipto_date_sk#46, d_year#49] +Input [18]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#38, s_store_name#40, s_zip#41, c_current_cdemo_sk#43, c_current_hdemo_sk#44, c_current_addr_sk#45, c_first_shipto_date_sk#46, c_first_sales_date_sk#47, d_date_sk#48, d_year#49] + +(55) ReusedExchange [Reuses operator id: 52] +Output [2]: [d_date_sk#50, d_year#51] + +(56) BroadcastHashJoin [codegen id : 24] +Left keys [1]: [c_first_shipto_date_sk#46] +Right keys [1]: [d_date_sk#50] +Join type: Inner +Join condition: None + +(57) Project [codegen id : 24] +Output [16]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#38, s_store_name#40, s_zip#41, c_current_cdemo_sk#43, c_current_hdemo_sk#44, c_current_addr_sk#45, d_year#49, d_year#51] +Input [18]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#38, s_store_name#40, s_zip#41, c_current_cdemo_sk#43, c_current_hdemo_sk#44, c_current_addr_sk#45, c_first_shipto_date_sk#46, d_year#49, d_date_sk#50, d_year#51] + +(58) Scan parquet spark_catalog.default.customer_demographics +Output [2]: [cd_demo_sk#52, cd_marital_status#53] +Batched: true +Location [not included in comparison]/{warehouse_dir}/customer_demographics] +PushedFilters: [IsNotNull(cd_demo_sk), IsNotNull(cd_marital_status)] +ReadSchema: struct<cd_demo_sk:int,cd_marital_status:string> + +(59) CometFilter +Input [2]: [cd_demo_sk#52, cd_marital_status#53] +Condition : (isnotnull(cd_demo_sk#52) AND isnotnull(cd_marital_status#53)) + +(60) ColumnarToRow [codegen id : 14] +Input [2]: [cd_demo_sk#52, cd_marital_status#53] + +(61) BroadcastExchange +Input [2]: [cd_demo_sk#52, cd_marital_status#53] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=8] + +(62) BroadcastHashJoin [codegen id : 24] +Left keys [1]: [ss_cdemo_sk#3] +Right keys [1]: [cd_demo_sk#52] +Join type: Inner +Join condition: None + +(63) Project [codegen id : 24] +Output [16]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#38, s_store_name#40, s_zip#41, c_current_cdemo_sk#43, c_current_hdemo_sk#44, c_current_addr_sk#45, d_year#49, d_year#51, cd_marital_status#53] +Input [18]: [ss_item_sk#1, ss_cdemo_sk#3, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#38, s_store_name#40, s_zip#41, c_current_cdemo_sk#43, c_current_hdemo_sk#44, c_current_addr_sk#45, d_year#49, d_year#51, cd_demo_sk#52, cd_marital_status#53] + +(64) ReusedExchange [Reuses operator id: 61] +Output [2]: [cd_demo_sk#54, cd_marital_status#55] + +(65) BroadcastHashJoin [codegen id : 24] +Left keys [1]: [c_current_cdemo_sk#43] +Right keys [1]: [cd_demo_sk#54] +Join type: Inner +Join condition: NOT (cd_marital_status#53 = cd_marital_status#55) + +(66) Project [codegen id : 24] +Output [14]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#38, s_store_name#40, s_zip#41, c_current_hdemo_sk#44, c_current_addr_sk#45, d_year#49, d_year#51] +Input [18]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#38, s_store_name#40, s_zip#41, c_current_cdemo_sk#43, c_current_hdemo_sk#44, c_current_addr_sk#45, d_year#49, d_year#51, cd_marital_status#53, cd_demo_sk#54, cd_marital_status#55] + +(67) Scan parquet spark_catalog.default.promotion +Output [1]: [p_promo_sk#56] +Batched: true +Location [not included in comparison]/{warehouse_dir}/promotion] +PushedFilters: [IsNotNull(p_promo_sk)] +ReadSchema: struct<p_promo_sk:int> + +(68) CometFilter +Input [1]: [p_promo_sk#56] +Condition : isnotnull(p_promo_sk#56) + +(69) ColumnarToRow [codegen id : 16] +Input [1]: [p_promo_sk#56] + +(70) BroadcastExchange +Input [1]: [p_promo_sk#56] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=9] + +(71) BroadcastHashJoin [codegen id : 24] +Left keys [1]: [ss_promo_sk#7] +Right keys [1]: [p_promo_sk#56] +Join type: Inner +Join condition: None + +(72) Project [codegen id : 24] +Output [13]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#38, s_store_name#40, s_zip#41, c_current_hdemo_sk#44, c_current_addr_sk#45, d_year#49, d_year#51] +Input [15]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_promo_sk#7, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#38, s_store_name#40, s_zip#41, c_current_hdemo_sk#44, c_current_addr_sk#45, d_year#49, d_year#51, p_promo_sk#56] + +(73) Scan parquet spark_catalog.default.household_demographics +Output [2]: [hd_demo_sk#57, hd_income_band_sk#58] +Batched: true +Location [not included in comparison]/{warehouse_dir}/household_demographics] +PushedFilters: [IsNotNull(hd_demo_sk), IsNotNull(hd_income_band_sk)] +ReadSchema: struct<hd_demo_sk:int,hd_income_band_sk:int> + +(74) CometFilter +Input [2]: [hd_demo_sk#57, hd_income_band_sk#58] +Condition : (isnotnull(hd_demo_sk#57) AND isnotnull(hd_income_band_sk#58)) + +(75) ColumnarToRow [codegen id : 17] +Input [2]: [hd_demo_sk#57, hd_income_band_sk#58] + +(76) BroadcastExchange +Input [2]: [hd_demo_sk#57, hd_income_band_sk#58] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=10] + +(77) BroadcastHashJoin [codegen id : 24] +Left keys [1]: [ss_hdemo_sk#4] +Right keys [1]: [hd_demo_sk#57] +Join type: Inner +Join condition: None + +(78) Project [codegen id : 24] +Output [13]: [ss_item_sk#1, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#38, s_store_name#40, s_zip#41, c_current_hdemo_sk#44, c_current_addr_sk#45, d_year#49, d_year#51, hd_income_band_sk#58] +Input [15]: [ss_item_sk#1, ss_hdemo_sk#4, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#38, s_store_name#40, s_zip#41, c_current_hdemo_sk#44, c_current_addr_sk#45, d_year#49, d_year#51, hd_demo_sk#57, hd_income_band_sk#58] + +(79) ReusedExchange [Reuses operator id: 76] +Output [2]: [hd_demo_sk#59, hd_income_band_sk#60] + +(80) BroadcastHashJoin [codegen id : 24] +Left keys [1]: [c_current_hdemo_sk#44] +Right keys [1]: [hd_demo_sk#59] +Join type: Inner +Join condition: None + +(81) Project [codegen id : 24] +Output [13]: [ss_item_sk#1, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#38, s_store_name#40, s_zip#41, c_current_addr_sk#45, d_year#49, d_year#51, hd_income_band_sk#58, hd_income_band_sk#60] +Input [15]: [ss_item_sk#1, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#38, s_store_name#40, s_zip#41, c_current_hdemo_sk#44, c_current_addr_sk#45, d_year#49, d_year#51, hd_income_band_sk#58, hd_demo_sk#59, hd_income_band_sk#60] + +(82) Scan parquet spark_catalog.default.customer_address +Output [5]: [ca_address_sk#61, ca_street_number#62, ca_street_name#63, ca_city#64, ca_zip#65] +Batched: true +Location [not included in comparison]/{warehouse_dir}/customer_address] +PushedFilters: [IsNotNull(ca_address_sk)] +ReadSchema: struct<ca_address_sk:int,ca_street_number:string,ca_street_name:string,ca_city:string,ca_zip:string> + +(83) CometFilter +Input [5]: [ca_address_sk#61, ca_street_number#62, ca_street_name#63, ca_city#64, ca_zip#65] +Condition : isnotnull(ca_address_sk#61) + +(84) ColumnarToRow [codegen id : 19] +Input [5]: [ca_address_sk#61, ca_street_number#62, ca_street_name#63, ca_city#64, ca_zip#65] + +(85) BroadcastExchange +Input [5]: [ca_address_sk#61, ca_street_number#62, ca_street_name#63, ca_city#64, ca_zip#65] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=11] + +(86) BroadcastHashJoin [codegen id : 24] +Left keys [1]: [ss_addr_sk#5] +Right keys [1]: [ca_address_sk#61] +Join type: Inner +Join condition: None + +(87) Project [codegen id : 24] +Output [16]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#38, s_store_name#40, s_zip#41, c_current_addr_sk#45, d_year#49, d_year#51, hd_income_band_sk#58, hd_income_band_sk#60, ca_street_number#62, ca_street_name#63, ca_city#64, ca_zip#65] +Input [18]: [ss_item_sk#1, ss_addr_sk#5, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#38, s_store_name#40, s_zip#41, c_current_addr_sk#45, d_year#49, d_year#51, hd_income_band_sk#58, hd_income_band_sk#60, ca_address_sk#61, ca_street_number#62, ca_street_name#63, ca_city#64, ca_zip#65] + +(88) ReusedExchange [Reuses operator id: 85] +Output [5]: [ca_address_sk#66, ca_street_number#67, ca_street_name#68, ca_city#69, ca_zip#70] + +(89) BroadcastHashJoin [codegen id : 24] +Left keys [1]: [c_current_addr_sk#45] +Right keys [1]: [ca_address_sk#66] +Join type: Inner +Join condition: None + +(90) Project [codegen id : 24] +Output [19]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#38, s_store_name#40, s_zip#41, d_year#49, d_year#51, hd_income_band_sk#58, hd_income_band_sk#60, ca_street_number#62, ca_street_name#63, ca_city#64, ca_zip#65, ca_street_number#67, ca_street_name#68, ca_city#69, ca_zip#70] +Input [21]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#38, s_store_name#40, s_zip#41, c_current_addr_sk#45, d_year#49, d_year#51, hd_income_band_sk#58, hd_income_band_sk#60, ca_street_number#62, ca_street_name#63, ca_city#64, ca_zip#65, ca_address_sk#66, ca_street_number#67, ca_street_name#68, ca_city#69, ca_zip#70] + +(91) Scan parquet spark_catalog.default.income_band +Output [1]: [ib_income_band_sk#71] +Batched: true +Location [not included in comparison]/{warehouse_dir}/income_band] +PushedFilters: [IsNotNull(ib_income_band_sk)] +ReadSchema: struct<ib_income_band_sk:int> + +(92) CometFilter +Input [1]: [ib_income_band_sk#71] +Condition : isnotnull(ib_income_band_sk#71) + +(93) ColumnarToRow [codegen id : 21] +Input [1]: [ib_income_band_sk#71] + +(94) BroadcastExchange +Input [1]: [ib_income_band_sk#71] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=12] + +(95) BroadcastHashJoin [codegen id : 24] +Left keys [1]: [hd_income_band_sk#58] +Right keys [1]: [ib_income_band_sk#71] +Join type: Inner +Join condition: None + +(96) Project [codegen id : 24] +Output [18]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#38, s_store_name#40, s_zip#41, d_year#49, d_year#51, hd_income_band_sk#60, ca_street_number#62, ca_street_name#63, ca_city#64, ca_zip#65, ca_street_number#67, ca_street_name#68, ca_city#69, ca_zip#70] +Input [20]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#38, s_store_name#40, s_zip#41, d_year#49, d_year#51, hd_income_band_sk#58, hd_income_band_sk#60, ca_street_number#62, ca_street_name#63, ca_city#64, ca_zip#65, ca_street_number#67, ca_street_name#68, ca_city#69, ca_zip#70, ib_income_band_sk#71] + +(97) ReusedExchange [Reuses operator id: 94] +Output [1]: [ib_income_band_sk#72] + +(98) BroadcastHashJoin [codegen id : 24] +Left keys [1]: [hd_income_band_sk#60] +Right keys [1]: [ib_income_band_sk#72] +Join type: Inner +Join condition: None + +(99) Project [codegen id : 24] +Output [17]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#38, s_store_name#40, s_zip#41, d_year#49, d_year#51, ca_street_number#62, ca_street_name#63, ca_city#64, ca_zip#65, ca_street_number#67, ca_street_name#68, ca_city#69, ca_zip#70] +Input [19]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#38, s_store_name#40, s_zip#41, d_year#49, d_year#51, hd_income_band_sk#60, ca_street_number#62, ca_street_name#63, ca_city#64, ca_zip#65, ca_street_number#67, ca_street_name#68, ca_city#69, ca_zip#70, ib_income_band_sk#72] + +(100) Scan parquet spark_catalog.default.item +Output [4]: [i_item_sk#73, i_current_price#74, i_color#75, i_product_name#76] +Batched: true +Location [not included in comparison]/{warehouse_dir}/item] +PushedFilters: [IsNotNull(i_current_price), In(i_color, [burlywood ,floral ,indian ,medium ,purple ,spring ]), GreaterThanOrEqual(i_current_price,64.00), LessThanOrEqual(i_current_price,74.00), GreaterThanOrEqual(i_current_price,65.00), LessThanOrEqual(i_current_price,79.00), IsNotNull(i_item_sk)] +ReadSchema: struct<i_item_sk:int,i_current_price:decimal(7,2),i_color:string,i_product_name:string> + +(101) CometFilter +Input [4]: [i_item_sk#73, i_current_price#74, i_color#75, i_product_name#76] +Condition : ((((((isnotnull(i_current_price#74) AND i_color#75 IN (purple ,burlywood ,indian ,spring ,floral ,medium )) AND (i_current_price#74 >= 64.00)) AND (i_current_price#74 <= 74.00)) AND (i_current_price#74 >= 65.00)) AND (i_current_price#74 <= 79.00)) AND isnotnull(i_item_sk#73)) + +(102) CometProject +Input [4]: [i_item_sk#73, i_current_price#74, i_color#75, i_product_name#76] +Arguments: [i_item_sk#73, i_product_name#76], [i_item_sk#73, i_product_name#76] + +(103) ColumnarToRow [codegen id : 23] +Input [2]: [i_item_sk#73, i_product_name#76] + +(104) BroadcastExchange +Input [2]: [i_item_sk#73, i_product_name#76] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=13] + +(105) BroadcastHashJoin [codegen id : 24] +Left keys [1]: [ss_item_sk#1] +Right keys [1]: [i_item_sk#73] +Join type: Inner +Join condition: None + +(106) Project [codegen id : 24] +Output [18]: [ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#38, d_year#49, d_year#51, s_store_name#40, s_zip#41, ca_street_number#62, ca_street_name#63, ca_city#64, ca_zip#65, ca_street_number#67, ca_street_name#68, ca_city#69, ca_zip#70, i_item_sk#73, i_product_name#76] +Input [19]: [ss_item_sk#1, ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#38, s_store_name#40, s_zip#41, d_year#49, d_year#51, ca_street_number#62, ca_street_name#63, ca_city#64, ca_zip#65, ca_street_number#67, ca_street_name#68, ca_city#69, ca_zip#70, i_item_sk#73, i_product_name#76] + +(107) HashAggregate [codegen id : 24] +Input [18]: [ss_wholesale_cost#9, ss_list_price#10, ss_coupon_amt#11, d_year#38, d_year#49, d_year#51, s_store_name#40, s_zip#41, ca_street_number#62, ca_street_name#63, ca_city#64, ca_zip#65, ca_street_number#67, ca_street_name#68, ca_city#69, ca_zip#70, i_item_sk#73, i_product_name#76] +Keys [15]: [i_product_name#76, i_item_sk#73, s_store_name#40, s_zip#41, ca_street_number#62, ca_street_name#63, ca_city#64, ca_zip#65, ca_street_number#67, ca_street_name#68, ca_city#69, ca_zip#70, d_year#38, d_year#49, d_year#51] +Functions [4]: [partial_count(1), partial_sum(UnscaledValue(ss_wholesale_cost#9)), partial_sum(UnscaledValue(ss_list_price#10)), partial_sum(UnscaledValue(ss_coupon_amt#11))] +Aggregate Attributes [4]: [count#77, sum#78, sum#79, sum#80] +Results [19]: [i_product_name#76, i_item_sk#73, s_store_name#40, s_zip#41, ca_street_number#62, ca_street_name#63, ca_city#64, ca_zip#65, ca_street_number#67, ca_street_name#68, ca_city#69, ca_zip#70, d_year#38, d_year#49, d_year#51, count#81, sum#82, sum#83, sum#84] + +(108) HashAggregate [codegen id : 24] +Input [19]: [i_product_name#76, i_item_sk#73, s_store_name#40, s_zip#41, ca_street_number#62, ca_street_name#63, ca_city#64, ca_zip#65, ca_street_number#67, ca_street_name#68, ca_city#69, ca_zip#70, d_year#38, d_year#49, d_year#51, count#81, sum#82, sum#83, sum#84] +Keys [15]: [i_product_name#76, i_item_sk#73, s_store_name#40, s_zip#41, ca_street_number#62, ca_street_name#63, ca_city#64, ca_zip#65, ca_street_number#67, ca_street_name#68, ca_city#69, ca_zip#70, d_year#38, d_year#49, d_year#51] +Functions [4]: [count(1), sum(UnscaledValue(ss_wholesale_cost#9)), sum(UnscaledValue(ss_list_price#10)), sum(UnscaledValue(ss_coupon_amt#11))] +Aggregate Attributes [4]: [count(1)#85, sum(UnscaledValue(ss_wholesale_cost#9))#86, sum(UnscaledValue(ss_list_price#10))#87, sum(UnscaledValue(ss_coupon_amt#11))#88] +Results [17]: [i_product_name#76 AS product_name#89, i_item_sk#73 AS item_sk#90, s_store_name#40 AS store_name#91, s_zip#41 AS store_zip#92, ca_street_number#62 AS b_street_number#93, ca_street_name#63 AS b_streen_name#94, ca_city#64 AS b_city#95, ca_zip#65 AS b_zip#96, ca_street_number#67 AS c_street_number#97, ca_street_name#68 AS c_street_name#98, ca_city#69 AS c_city#99, ca_zip#70 AS c_zip#100, d_year#38 AS syear#101, count(1)#85 AS cnt#102, MakeDecimal(sum(UnscaledValue(ss_wholesale_cost#9))#86,17,2) AS s1#103, MakeDecimal(sum(UnscaledValue(ss_list_price#10))#87,17,2) AS s2#104, MakeDecimal(sum(UnscaledValue(ss_coupon_amt#11))#88,17,2) AS s3#105] + +(109) Exchange +Input [17]: [product_name#89, item_sk#90, store_name#91, store_zip#92, b_street_number#93, b_streen_name#94, b_city#95, b_zip#96, c_street_number#97, c_street_name#98, c_city#99, c_zip#100, syear#101, cnt#102, s1#103, s2#104, s3#105] +Arguments: hashpartitioning(item_sk#90, store_name#91, store_zip#92, 5), ENSURE_REQUIREMENTS, [plan_id=14] + +(110) Sort [codegen id : 25] +Input [17]: [product_name#89, item_sk#90, store_name#91, store_zip#92, b_street_number#93, b_streen_name#94, b_city#95, b_zip#96, c_street_number#97, c_street_name#98, c_city#99, c_zip#100, syear#101, cnt#102, s1#103, s2#104, s3#105] +Arguments: [item_sk#90 ASC NULLS FIRST, store_name#91 ASC NULLS FIRST, store_zip#92 ASC NULLS FIRST], false, 0 + +(111) Scan parquet spark_catalog.default.store_sales +Output [12]: [ss_item_sk#106, ss_customer_sk#107, ss_cdemo_sk#108, ss_hdemo_sk#109, ss_addr_sk#110, ss_store_sk#111, ss_promo_sk#112, ss_ticket_number#113, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, ss_sold_date_sk#117] +Batched: true +Location: InMemoryFileIndex [] +PartitionFilters: [isnotnull(ss_sold_date_sk#117), dynamicpruningexpression(ss_sold_date_sk#117 IN dynamicpruning#118)] +PushedFilters: [IsNotNull(ss_item_sk), IsNotNull(ss_ticket_number), IsNotNull(ss_store_sk), IsNotNull(ss_customer_sk), IsNotNull(ss_cdemo_sk), IsNotNull(ss_promo_sk), IsNotNull(ss_hdemo_sk), IsNotNull(ss_addr_sk)] +ReadSchema: struct<ss_item_sk:int,ss_customer_sk:int,ss_cdemo_sk:int,ss_hdemo_sk:int,ss_addr_sk:int,ss_store_sk:int,ss_promo_sk:int,ss_ticket_number:int,ss_wholesale_cost:decimal(7,2),ss_list_price:decimal(7,2),ss_coupon_amt:decimal(7,2)> + +(112) CometFilter +Input [12]: [ss_item_sk#106, ss_customer_sk#107, ss_cdemo_sk#108, ss_hdemo_sk#109, ss_addr_sk#110, ss_store_sk#111, ss_promo_sk#112, ss_ticket_number#113, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, ss_sold_date_sk#117] +Condition : (((((((isnotnull(ss_item_sk#106) AND isnotnull(ss_ticket_number#113)) AND isnotnull(ss_store_sk#111)) AND isnotnull(ss_customer_sk#107)) AND isnotnull(ss_cdemo_sk#108)) AND isnotnull(ss_promo_sk#112)) AND isnotnull(ss_hdemo_sk#109)) AND isnotnull(ss_addr_sk#110)) + +(113) CometBroadcastExchange +Input [12]: [ss_item_sk#106, ss_customer_sk#107, ss_cdemo_sk#108, ss_hdemo_sk#109, ss_addr_sk#110, ss_store_sk#111, ss_promo_sk#112, ss_ticket_number#113, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, ss_sold_date_sk#117] +Arguments: [ss_item_sk#106, ss_customer_sk#107, ss_cdemo_sk#108, ss_hdemo_sk#109, ss_addr_sk#110, ss_store_sk#111, ss_promo_sk#112, ss_ticket_number#113, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, ss_sold_date_sk#117] + +(114) Scan parquet spark_catalog.default.store_returns +Output [3]: [sr_item_sk#119, sr_ticket_number#120, sr_returned_date_sk#121] +Batched: true +Location [not included in comparison]/{warehouse_dir}/store_returns] +PushedFilters: [IsNotNull(sr_item_sk), IsNotNull(sr_ticket_number)] +ReadSchema: struct<sr_item_sk:int,sr_ticket_number:int> + +(115) CometFilter +Input [3]: [sr_item_sk#119, sr_ticket_number#120, sr_returned_date_sk#121] +Condition : (isnotnull(sr_item_sk#119) AND isnotnull(sr_ticket_number#120)) + +(116) CometProject +Input [3]: [sr_item_sk#119, sr_ticket_number#120, sr_returned_date_sk#121] +Arguments: [sr_item_sk#119, sr_ticket_number#120], [sr_item_sk#119, sr_ticket_number#120] + +(117) CometBroadcastHashJoin +Left output [12]: [ss_item_sk#106, ss_customer_sk#107, ss_cdemo_sk#108, ss_hdemo_sk#109, ss_addr_sk#110, ss_store_sk#111, ss_promo_sk#112, ss_ticket_number#113, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, ss_sold_date_sk#117] +Right output [2]: [sr_item_sk#119, sr_ticket_number#120] +Arguments: [ss_item_sk#106, ss_ticket_number#113], [sr_item_sk#119, sr_ticket_number#120], Inner + +(118) CometProject +Input [14]: [ss_item_sk#106, ss_customer_sk#107, ss_cdemo_sk#108, ss_hdemo_sk#109, ss_addr_sk#110, ss_store_sk#111, ss_promo_sk#112, ss_ticket_number#113, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, ss_sold_date_sk#117, sr_item_sk#119, sr_ticket_number#120] +Arguments: [ss_item_sk#106, ss_customer_sk#107, ss_cdemo_sk#108, ss_hdemo_sk#109, ss_addr_sk#110, ss_store_sk#111, ss_promo_sk#112, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, ss_sold_date_sk#117], [ss_item_sk#106, ss_customer_sk#107, ss_cdemo_sk#108, ss_hdemo_sk#109, ss_addr_sk#110, ss_store_sk#111, ss_promo_sk#112, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, ss_sold_date_sk#117] + +(119) ColumnarToRow [codegen id : 26] +Input [11]: [ss_item_sk#106, ss_customer_sk#107, ss_cdemo_sk#108, ss_hdemo_sk#109, ss_addr_sk#110, ss_store_sk#111, ss_promo_sk#112, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, ss_sold_date_sk#117] + +(120) Exchange +Input [11]: [ss_item_sk#106, ss_customer_sk#107, ss_cdemo_sk#108, ss_hdemo_sk#109, ss_addr_sk#110, ss_store_sk#111, ss_promo_sk#112, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, ss_sold_date_sk#117] +Arguments: hashpartitioning(ss_item_sk#106, 5), ENSURE_REQUIREMENTS, [plan_id=15] + +(121) Sort [codegen id : 27] +Input [11]: [ss_item_sk#106, ss_customer_sk#107, ss_cdemo_sk#108, ss_hdemo_sk#109, ss_addr_sk#110, ss_store_sk#111, ss_promo_sk#112, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, ss_sold_date_sk#117] +Arguments: [ss_item_sk#106 ASC NULLS FIRST], false, 0 + +(122) ReusedExchange [Reuses operator id: 27] +Output [4]: [cs_item_sk#122, sum#123, sum#124, isEmpty#125] + +(123) HashAggregate [codegen id : 33] +Input [4]: [cs_item_sk#122, sum#123, sum#124, isEmpty#125] +Keys [1]: [cs_item_sk#122] +Functions [2]: [sum(UnscaledValue(cs_ext_list_price#126)), sum(((cr_refunded_cash#127 + cr_reversed_charge#128) + cr_store_credit#129))] +Aggregate Attributes [2]: [sum(UnscaledValue(cs_ext_list_price#126))#33, sum(((cr_refunded_cash#127 + cr_reversed_charge#128) + cr_store_credit#129))#34] +Results [3]: [cs_item_sk#122, MakeDecimal(sum(UnscaledValue(cs_ext_list_price#126))#33,17,2) AS sale#130, sum(((cr_refunded_cash#127 + cr_reversed_charge#128) + cr_store_credit#129))#34 AS refund#131] + +(124) Filter [codegen id : 33] +Input [3]: [cs_item_sk#122, sale#130, refund#131] +Condition : ((isnotnull(sale#130) AND isnotnull(refund#131)) AND (cast(sale#130 as decimal(21,2)) > (2 * refund#131))) + +(125) Project [codegen id : 33] +Output [1]: [cs_item_sk#122] +Input [3]: [cs_item_sk#122, sale#130, refund#131] + +(126) Sort [codegen id : 33] +Input [1]: [cs_item_sk#122] +Arguments: [cs_item_sk#122 ASC NULLS FIRST], false, 0 + +(127) SortMergeJoin [codegen id : 49] +Left keys [1]: [ss_item_sk#106] +Right keys [1]: [cs_item_sk#122] +Join type: Inner +Join condition: None + +(128) Project [codegen id : 49] +Output [11]: [ss_item_sk#106, ss_customer_sk#107, ss_cdemo_sk#108, ss_hdemo_sk#109, ss_addr_sk#110, ss_store_sk#111, ss_promo_sk#112, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, ss_sold_date_sk#117] +Input [12]: [ss_item_sk#106, ss_customer_sk#107, ss_cdemo_sk#108, ss_hdemo_sk#109, ss_addr_sk#110, ss_store_sk#111, ss_promo_sk#112, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, ss_sold_date_sk#117, cs_item_sk#122] + +(129) ReusedExchange [Reuses operator id: 189] +Output [2]: [d_date_sk#132, d_year#133] + +(130) BroadcastHashJoin [codegen id : 49] +Left keys [1]: [ss_sold_date_sk#117] +Right keys [1]: [d_date_sk#132] +Join type: Inner +Join condition: None + +(131) Project [codegen id : 49] +Output [11]: [ss_item_sk#106, ss_customer_sk#107, ss_cdemo_sk#108, ss_hdemo_sk#109, ss_addr_sk#110, ss_store_sk#111, ss_promo_sk#112, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#133] +Input [13]: [ss_item_sk#106, ss_customer_sk#107, ss_cdemo_sk#108, ss_hdemo_sk#109, ss_addr_sk#110, ss_store_sk#111, ss_promo_sk#112, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, ss_sold_date_sk#117, d_date_sk#132, d_year#133] + +(132) ReusedExchange [Reuses operator id: 40] +Output [3]: [s_store_sk#134, s_store_name#135, s_zip#136] + +(133) BroadcastHashJoin [codegen id : 49] +Left keys [1]: [ss_store_sk#111] +Right keys [1]: [s_store_sk#134] +Join type: Inner +Join condition: None + +(134) Project [codegen id : 49] +Output [12]: [ss_item_sk#106, ss_customer_sk#107, ss_cdemo_sk#108, ss_hdemo_sk#109, ss_addr_sk#110, ss_promo_sk#112, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#133, s_store_name#135, s_zip#136] +Input [14]: [ss_item_sk#106, ss_customer_sk#107, ss_cdemo_sk#108, ss_hdemo_sk#109, ss_addr_sk#110, ss_store_sk#111, ss_promo_sk#112, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#133, s_store_sk#134, s_store_name#135, s_zip#136] + +(135) ReusedExchange [Reuses operator id: 46] +Output [6]: [c_customer_sk#137, c_current_cdemo_sk#138, c_current_hdemo_sk#139, c_current_addr_sk#140, c_first_shipto_date_sk#141, c_first_sales_date_sk#142] + +(136) BroadcastHashJoin [codegen id : 49] +Left keys [1]: [ss_customer_sk#107] +Right keys [1]: [c_customer_sk#137] +Join type: Inner +Join condition: None + +(137) Project [codegen id : 49] +Output [16]: [ss_item_sk#106, ss_cdemo_sk#108, ss_hdemo_sk#109, ss_addr_sk#110, ss_promo_sk#112, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#133, s_store_name#135, s_zip#136, c_current_cdemo_sk#138, c_current_hdemo_sk#139, c_current_addr_sk#140, c_first_shipto_date_sk#141, c_first_sales_date_sk#142] +Input [18]: [ss_item_sk#106, ss_customer_sk#107, ss_cdemo_sk#108, ss_hdemo_sk#109, ss_addr_sk#110, ss_promo_sk#112, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#133, s_store_name#135, s_zip#136, c_customer_sk#137, c_current_cdemo_sk#138, c_current_hdemo_sk#139, c_current_addr_sk#140, c_first_shipto_date_sk#141, c_first_sales_date_sk#142] + +(138) ReusedExchange [Reuses operator id: 52] +Output [2]: [d_date_sk#143, d_year#144] + +(139) BroadcastHashJoin [codegen id : 49] +Left keys [1]: [c_first_sales_date_sk#142] +Right keys [1]: [d_date_sk#143] +Join type: Inner +Join condition: None + +(140) Project [codegen id : 49] +Output [16]: [ss_item_sk#106, ss_cdemo_sk#108, ss_hdemo_sk#109, ss_addr_sk#110, ss_promo_sk#112, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#133, s_store_name#135, s_zip#136, c_current_cdemo_sk#138, c_current_hdemo_sk#139, c_current_addr_sk#140, c_first_shipto_date_sk#141, d_year#144] +Input [18]: [ss_item_sk#106, ss_cdemo_sk#108, ss_hdemo_sk#109, ss_addr_sk#110, ss_promo_sk#112, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#133, s_store_name#135, s_zip#136, c_current_cdemo_sk#138, c_current_hdemo_sk#139, c_current_addr_sk#140, c_first_shipto_date_sk#141, c_first_sales_date_sk#142, d_date_sk#143, d_year#144] + +(141) ReusedExchange [Reuses operator id: 52] +Output [2]: [d_date_sk#145, d_year#146] + +(142) BroadcastHashJoin [codegen id : 49] +Left keys [1]: [c_first_shipto_date_sk#141] +Right keys [1]: [d_date_sk#145] +Join type: Inner +Join condition: None + +(143) Project [codegen id : 49] +Output [16]: [ss_item_sk#106, ss_cdemo_sk#108, ss_hdemo_sk#109, ss_addr_sk#110, ss_promo_sk#112, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#133, s_store_name#135, s_zip#136, c_current_cdemo_sk#138, c_current_hdemo_sk#139, c_current_addr_sk#140, d_year#144, d_year#146] +Input [18]: [ss_item_sk#106, ss_cdemo_sk#108, ss_hdemo_sk#109, ss_addr_sk#110, ss_promo_sk#112, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#133, s_store_name#135, s_zip#136, c_current_cdemo_sk#138, c_current_hdemo_sk#139, c_current_addr_sk#140, c_first_shipto_date_sk#141, d_year#144, d_date_sk#145, d_year#146] + +(144) ReusedExchange [Reuses operator id: 61] +Output [2]: [cd_demo_sk#147, cd_marital_status#148] + +(145) BroadcastHashJoin [codegen id : 49] +Left keys [1]: [ss_cdemo_sk#108] +Right keys [1]: [cd_demo_sk#147] +Join type: Inner +Join condition: None + +(146) Project [codegen id : 49] +Output [16]: [ss_item_sk#106, ss_hdemo_sk#109, ss_addr_sk#110, ss_promo_sk#112, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#133, s_store_name#135, s_zip#136, c_current_cdemo_sk#138, c_current_hdemo_sk#139, c_current_addr_sk#140, d_year#144, d_year#146, cd_marital_status#148] +Input [18]: [ss_item_sk#106, ss_cdemo_sk#108, ss_hdemo_sk#109, ss_addr_sk#110, ss_promo_sk#112, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#133, s_store_name#135, s_zip#136, c_current_cdemo_sk#138, c_current_hdemo_sk#139, c_current_addr_sk#140, d_year#144, d_year#146, cd_demo_sk#147, cd_marital_status#148] + +(147) ReusedExchange [Reuses operator id: 61] +Output [2]: [cd_demo_sk#149, cd_marital_status#150] + +(148) BroadcastHashJoin [codegen id : 49] +Left keys [1]: [c_current_cdemo_sk#138] +Right keys [1]: [cd_demo_sk#149] +Join type: Inner +Join condition: NOT (cd_marital_status#148 = cd_marital_status#150) + +(149) Project [codegen id : 49] +Output [14]: [ss_item_sk#106, ss_hdemo_sk#109, ss_addr_sk#110, ss_promo_sk#112, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#133, s_store_name#135, s_zip#136, c_current_hdemo_sk#139, c_current_addr_sk#140, d_year#144, d_year#146] +Input [18]: [ss_item_sk#106, ss_hdemo_sk#109, ss_addr_sk#110, ss_promo_sk#112, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#133, s_store_name#135, s_zip#136, c_current_cdemo_sk#138, c_current_hdemo_sk#139, c_current_addr_sk#140, d_year#144, d_year#146, cd_marital_status#148, cd_demo_sk#149, cd_marital_status#150] + +(150) ReusedExchange [Reuses operator id: 70] +Output [1]: [p_promo_sk#151] + +(151) BroadcastHashJoin [codegen id : 49] +Left keys [1]: [ss_promo_sk#112] +Right keys [1]: [p_promo_sk#151] +Join type: Inner +Join condition: None + +(152) Project [codegen id : 49] +Output [13]: [ss_item_sk#106, ss_hdemo_sk#109, ss_addr_sk#110, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#133, s_store_name#135, s_zip#136, c_current_hdemo_sk#139, c_current_addr_sk#140, d_year#144, d_year#146] +Input [15]: [ss_item_sk#106, ss_hdemo_sk#109, ss_addr_sk#110, ss_promo_sk#112, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#133, s_store_name#135, s_zip#136, c_current_hdemo_sk#139, c_current_addr_sk#140, d_year#144, d_year#146, p_promo_sk#151] + +(153) ReusedExchange [Reuses operator id: 76] +Output [2]: [hd_demo_sk#152, hd_income_band_sk#153] + +(154) BroadcastHashJoin [codegen id : 49] +Left keys [1]: [ss_hdemo_sk#109] +Right keys [1]: [hd_demo_sk#152] +Join type: Inner +Join condition: None + +(155) Project [codegen id : 49] +Output [13]: [ss_item_sk#106, ss_addr_sk#110, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#133, s_store_name#135, s_zip#136, c_current_hdemo_sk#139, c_current_addr_sk#140, d_year#144, d_year#146, hd_income_band_sk#153] +Input [15]: [ss_item_sk#106, ss_hdemo_sk#109, ss_addr_sk#110, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#133, s_store_name#135, s_zip#136, c_current_hdemo_sk#139, c_current_addr_sk#140, d_year#144, d_year#146, hd_demo_sk#152, hd_income_band_sk#153] + +(156) ReusedExchange [Reuses operator id: 76] +Output [2]: [hd_demo_sk#154, hd_income_band_sk#155] + +(157) BroadcastHashJoin [codegen id : 49] +Left keys [1]: [c_current_hdemo_sk#139] +Right keys [1]: [hd_demo_sk#154] +Join type: Inner +Join condition: None + +(158) Project [codegen id : 49] +Output [13]: [ss_item_sk#106, ss_addr_sk#110, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#133, s_store_name#135, s_zip#136, c_current_addr_sk#140, d_year#144, d_year#146, hd_income_band_sk#153, hd_income_band_sk#155] +Input [15]: [ss_item_sk#106, ss_addr_sk#110, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#133, s_store_name#135, s_zip#136, c_current_hdemo_sk#139, c_current_addr_sk#140, d_year#144, d_year#146, hd_income_band_sk#153, hd_demo_sk#154, hd_income_band_sk#155] + +(159) ReusedExchange [Reuses operator id: 85] +Output [5]: [ca_address_sk#156, ca_street_number#157, ca_street_name#158, ca_city#159, ca_zip#160] + +(160) BroadcastHashJoin [codegen id : 49] +Left keys [1]: [ss_addr_sk#110] +Right keys [1]: [ca_address_sk#156] +Join type: Inner +Join condition: None + +(161) Project [codegen id : 49] +Output [16]: [ss_item_sk#106, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#133, s_store_name#135, s_zip#136, c_current_addr_sk#140, d_year#144, d_year#146, hd_income_band_sk#153, hd_income_band_sk#155, ca_street_number#157, ca_street_name#158, ca_city#159, ca_zip#160] +Input [18]: [ss_item_sk#106, ss_addr_sk#110, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#133, s_store_name#135, s_zip#136, c_current_addr_sk#140, d_year#144, d_year#146, hd_income_band_sk#153, hd_income_band_sk#155, ca_address_sk#156, ca_street_number#157, ca_street_name#158, ca_city#159, ca_zip#160] + +(162) ReusedExchange [Reuses operator id: 85] +Output [5]: [ca_address_sk#161, ca_street_number#162, ca_street_name#163, ca_city#164, ca_zip#165] + +(163) BroadcastHashJoin [codegen id : 49] +Left keys [1]: [c_current_addr_sk#140] +Right keys [1]: [ca_address_sk#161] +Join type: Inner +Join condition: None + +(164) Project [codegen id : 49] +Output [19]: [ss_item_sk#106, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#133, s_store_name#135, s_zip#136, d_year#144, d_year#146, hd_income_band_sk#153, hd_income_band_sk#155, ca_street_number#157, ca_street_name#158, ca_city#159, ca_zip#160, ca_street_number#162, ca_street_name#163, ca_city#164, ca_zip#165] +Input [21]: [ss_item_sk#106, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#133, s_store_name#135, s_zip#136, c_current_addr_sk#140, d_year#144, d_year#146, hd_income_band_sk#153, hd_income_band_sk#155, ca_street_number#157, ca_street_name#158, ca_city#159, ca_zip#160, ca_address_sk#161, ca_street_number#162, ca_street_name#163, ca_city#164, ca_zip#165] + +(165) ReusedExchange [Reuses operator id: 94] +Output [1]: [ib_income_band_sk#166] + +(166) BroadcastHashJoin [codegen id : 49] +Left keys [1]: [hd_income_band_sk#153] +Right keys [1]: [ib_income_band_sk#166] +Join type: Inner +Join condition: None + +(167) Project [codegen id : 49] +Output [18]: [ss_item_sk#106, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#133, s_store_name#135, s_zip#136, d_year#144, d_year#146, hd_income_band_sk#155, ca_street_number#157, ca_street_name#158, ca_city#159, ca_zip#160, ca_street_number#162, ca_street_name#163, ca_city#164, ca_zip#165] +Input [20]: [ss_item_sk#106, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#133, s_store_name#135, s_zip#136, d_year#144, d_year#146, hd_income_band_sk#153, hd_income_band_sk#155, ca_street_number#157, ca_street_name#158, ca_city#159, ca_zip#160, ca_street_number#162, ca_street_name#163, ca_city#164, ca_zip#165, ib_income_band_sk#166] + +(168) ReusedExchange [Reuses operator id: 94] +Output [1]: [ib_income_band_sk#167] + +(169) BroadcastHashJoin [codegen id : 49] +Left keys [1]: [hd_income_band_sk#155] +Right keys [1]: [ib_income_band_sk#167] +Join type: Inner +Join condition: None + +(170) Project [codegen id : 49] +Output [17]: [ss_item_sk#106, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#133, s_store_name#135, s_zip#136, d_year#144, d_year#146, ca_street_number#157, ca_street_name#158, ca_city#159, ca_zip#160, ca_street_number#162, ca_street_name#163, ca_city#164, ca_zip#165] +Input [19]: [ss_item_sk#106, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#133, s_store_name#135, s_zip#136, d_year#144, d_year#146, hd_income_band_sk#155, ca_street_number#157, ca_street_name#158, ca_city#159, ca_zip#160, ca_street_number#162, ca_street_name#163, ca_city#164, ca_zip#165, ib_income_band_sk#167] + +(171) ReusedExchange [Reuses operator id: 104] +Output [2]: [i_item_sk#168, i_product_name#169] + +(172) BroadcastHashJoin [codegen id : 49] +Left keys [1]: [ss_item_sk#106] +Right keys [1]: [i_item_sk#168] +Join type: Inner +Join condition: None + +(173) Project [codegen id : 49] +Output [18]: [ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#133, d_year#144, d_year#146, s_store_name#135, s_zip#136, ca_street_number#157, ca_street_name#158, ca_city#159, ca_zip#160, ca_street_number#162, ca_street_name#163, ca_city#164, ca_zip#165, i_item_sk#168, i_product_name#169] +Input [19]: [ss_item_sk#106, ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#133, s_store_name#135, s_zip#136, d_year#144, d_year#146, ca_street_number#157, ca_street_name#158, ca_city#159, ca_zip#160, ca_street_number#162, ca_street_name#163, ca_city#164, ca_zip#165, i_item_sk#168, i_product_name#169] + +(174) HashAggregate [codegen id : 49] +Input [18]: [ss_wholesale_cost#114, ss_list_price#115, ss_coupon_amt#116, d_year#133, d_year#144, d_year#146, s_store_name#135, s_zip#136, ca_street_number#157, ca_street_name#158, ca_city#159, ca_zip#160, ca_street_number#162, ca_street_name#163, ca_city#164, ca_zip#165, i_item_sk#168, i_product_name#169] +Keys [15]: [i_product_name#169, i_item_sk#168, s_store_name#135, s_zip#136, ca_street_number#157, ca_street_name#158, ca_city#159, ca_zip#160, ca_street_number#162, ca_street_name#163, ca_city#164, ca_zip#165, d_year#133, d_year#144, d_year#146] +Functions [4]: [partial_count(1), partial_sum(UnscaledValue(ss_wholesale_cost#114)), partial_sum(UnscaledValue(ss_list_price#115)), partial_sum(UnscaledValue(ss_coupon_amt#116))] +Aggregate Attributes [4]: [count#77, sum#170, sum#171, sum#172] +Results [19]: [i_product_name#169, i_item_sk#168, s_store_name#135, s_zip#136, ca_street_number#157, ca_street_name#158, ca_city#159, ca_zip#160, ca_street_number#162, ca_street_name#163, ca_city#164, ca_zip#165, d_year#133, d_year#144, d_year#146, count#81, sum#173, sum#174, sum#175] + +(175) HashAggregate [codegen id : 49] +Input [19]: [i_product_name#169, i_item_sk#168, s_store_name#135, s_zip#136, ca_street_number#157, ca_street_name#158, ca_city#159, ca_zip#160, ca_street_number#162, ca_street_name#163, ca_city#164, ca_zip#165, d_year#133, d_year#144, d_year#146, count#81, sum#173, sum#174, sum#175] +Keys [15]: [i_product_name#169, i_item_sk#168, s_store_name#135, s_zip#136, ca_street_number#157, ca_street_name#158, ca_city#159, ca_zip#160, ca_street_number#162, ca_street_name#163, ca_city#164, ca_zip#165, d_year#133, d_year#144, d_year#146] +Functions [4]: [count(1), sum(UnscaledValue(ss_wholesale_cost#114)), sum(UnscaledValue(ss_list_price#115)), sum(UnscaledValue(ss_coupon_amt#116))] +Aggregate Attributes [4]: [count(1)#85, sum(UnscaledValue(ss_wholesale_cost#114))#86, sum(UnscaledValue(ss_list_price#115))#87, sum(UnscaledValue(ss_coupon_amt#116))#88] +Results [8]: [i_item_sk#168 AS item_sk#176, s_store_name#135 AS store_name#177, s_zip#136 AS store_zip#178, d_year#133 AS syear#179, count(1)#85 AS cnt#180, MakeDecimal(sum(UnscaledValue(ss_wholesale_cost#114))#86,17,2) AS s1#181, MakeDecimal(sum(UnscaledValue(ss_list_price#115))#87,17,2) AS s2#182, MakeDecimal(sum(UnscaledValue(ss_coupon_amt#116))#88,17,2) AS s3#183] + +(176) Exchange +Input [8]: [item_sk#176, store_name#177, store_zip#178, syear#179, cnt#180, s1#181, s2#182, s3#183] +Arguments: hashpartitioning(item_sk#176, store_name#177, store_zip#178, 5), ENSURE_REQUIREMENTS, [plan_id=16] + +(177) Sort [codegen id : 50] +Input [8]: [item_sk#176, store_name#177, store_zip#178, syear#179, cnt#180, s1#181, s2#182, s3#183] +Arguments: [item_sk#176 ASC NULLS FIRST, store_name#177 ASC NULLS FIRST, store_zip#178 ASC NULLS FIRST], false, 0 + +(178) SortMergeJoin [codegen id : 51] +Left keys [3]: [item_sk#90, store_name#91, store_zip#92] +Right keys [3]: [item_sk#176, store_name#177, store_zip#178] +Join type: Inner +Join condition: (cnt#180 <= cnt#102) + +(179) Project [codegen id : 51] +Output [21]: [product_name#89, store_name#91, store_zip#92, b_street_number#93, b_streen_name#94, b_city#95, b_zip#96, c_street_number#97, c_street_name#98, c_city#99, c_zip#100, syear#101, cnt#102, s1#103, s2#104, s3#105, s1#181, s2#182, s3#183, syear#179, cnt#180] +Input [25]: [product_name#89, item_sk#90, store_name#91, store_zip#92, b_street_number#93, b_streen_name#94, b_city#95, b_zip#96, c_street_number#97, c_street_name#98, c_city#99, c_zip#100, syear#101, cnt#102, s1#103, s2#104, s3#105, item_sk#176, store_name#177, store_zip#178, syear#179, cnt#180, s1#181, s2#182, s3#183] + +(180) Exchange +Input [21]: [product_name#89, store_name#91, store_zip#92, b_street_number#93, b_streen_name#94, b_city#95, b_zip#96, c_street_number#97, c_street_name#98, c_city#99, c_zip#100, syear#101, cnt#102, s1#103, s2#104, s3#105, s1#181, s2#182, s3#183, syear#179, cnt#180] +Arguments: rangepartitioning(product_name#89 ASC NULLS FIRST, store_name#91 ASC NULLS FIRST, cnt#180 ASC NULLS FIRST, s1#103 ASC NULLS FIRST, s1#181 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, [plan_id=17] + +(181) Sort [codegen id : 52] +Input [21]: [product_name#89, store_name#91, store_zip#92, b_street_number#93, b_streen_name#94, b_city#95, b_zip#96, c_street_number#97, c_street_name#98, c_city#99, c_zip#100, syear#101, cnt#102, s1#103, s2#104, s3#105, s1#181, s2#182, s3#183, syear#179, cnt#180] +Arguments: [product_name#89 ASC NULLS FIRST, store_name#91 ASC NULLS FIRST, cnt#180 ASC NULLS FIRST, s1#103 ASC NULLS FIRST, s1#181 ASC NULLS FIRST], true, 0 + +===== Subqueries ===== + +Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#12 IN dynamicpruning#13 +BroadcastExchange (185) ++- * ColumnarToRow (184) + +- CometFilter (183) + +- CometScan parquet spark_catalog.default.date_dim (182) + + +(182) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#37, d_year#38] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_year), EqualTo(d_year,1999), IsNotNull(d_date_sk)] +ReadSchema: struct<d_date_sk:int,d_year:int> + +(183) CometFilter +Input [2]: [d_date_sk#37, d_year#38] +Condition : ((isnotnull(d_year#38) AND (d_year#38 = 1999)) AND isnotnull(d_date_sk#37)) + +(184) ColumnarToRow [codegen id : 1] +Input [2]: [d_date_sk#37, d_year#38] + +(185) BroadcastExchange +Input [2]: [d_date_sk#37, d_year#38] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=18] + +Subquery:2 Hosting operator id = 111 Hosting Expression = ss_sold_date_sk#117 IN dynamicpruning#118 +BroadcastExchange (189) ++- * ColumnarToRow (188) + +- CometFilter (187) + +- CometScan parquet spark_catalog.default.date_dim (186) + + +(186) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#132, d_year#133] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2000), IsNotNull(d_date_sk)] +ReadSchema: struct<d_date_sk:int,d_year:int> + +(187) CometFilter +Input [2]: [d_date_sk#132, d_year#133] +Condition : ((isnotnull(d_year#133) AND (d_year#133 = 2000)) AND isnotnull(d_date_sk#132)) + +(188) ColumnarToRow [codegen id : 1] +Input [2]: [d_date_sk#132, d_year#133] + +(189) BroadcastExchange +Input [2]: [d_date_sk#132, d_year#133] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=19] + + diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q64/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q64/simplified.txt new file mode 100644 index 000000000..2a0bc5bce --- /dev/null +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q64/simplified.txt @@ -0,0 +1,281 @@ +WholeStageCodegen (52) + Sort [product_name,store_name,cnt,s1,s1] + InputAdapter + Exchange [product_name,store_name,cnt,s1,s1] #1 + WholeStageCodegen (51) + Project [product_name,store_name,store_zip,b_street_number,b_streen_name,b_city,b_zip,c_street_number,c_street_name,c_city,c_zip,syear,cnt,s1,s2,s3,s1,s2,s3,syear,cnt] + SortMergeJoin [item_sk,store_name,store_zip,item_sk,store_name,store_zip,cnt,cnt] + InputAdapter + WholeStageCodegen (25) + Sort [item_sk,store_name,store_zip] + InputAdapter + Exchange [item_sk,store_name,store_zip] #2 + WholeStageCodegen (24) + HashAggregate [i_product_name,i_item_sk,s_store_name,s_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,d_year,d_year,d_year,count,sum,sum,sum] [count(1),sum(UnscaledValue(ss_wholesale_cost)),sum(UnscaledValue(ss_list_price)),sum(UnscaledValue(ss_coupon_amt)),product_name,item_sk,store_name,store_zip,b_street_number,b_streen_name,b_city,b_zip,c_street_number,c_street_name,c_city,c_zip,syear,cnt,s1,s2,s3,count,sum,sum,sum] + HashAggregate [i_product_name,i_item_sk,s_store_name,s_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,d_year,d_year,d_year,ss_wholesale_cost,ss_list_price,ss_coupon_amt] [count,sum,sum,sum,count,sum,sum,sum] + Project [ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,d_year,d_year,s_store_name,s_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,i_item_sk,i_product_name] + BroadcastHashJoin [ss_item_sk,i_item_sk] + Project [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip] + BroadcastHashJoin [hd_income_band_sk,ib_income_band_sk] + Project [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip] + BroadcastHashJoin [hd_income_band_sk,ib_income_band_sk] + Project [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip] + BroadcastHashJoin [c_current_addr_sk,ca_address_sk] + Project [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip] + BroadcastHashJoin [ss_addr_sk,ca_address_sk] + Project [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk] + BroadcastHashJoin [c_current_hdemo_sk,hd_demo_sk] + Project [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,hd_income_band_sk] + BroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] + Project [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year] + BroadcastHashJoin [ss_promo_sk,p_promo_sk] + Project [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year] + BroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk,cd_marital_status,cd_marital_status] + Project [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,cd_marital_status] + BroadcastHashJoin [ss_cdemo_sk,cd_demo_sk] + Project [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year] + BroadcastHashJoin [c_first_shipto_date_sk,d_date_sk] + Project [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,d_year] + BroadcastHashJoin [c_first_sales_date_sk,d_date_sk] + Project [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk] + BroadcastHashJoin [ss_customer_sk,c_customer_sk] + Project [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip] + BroadcastHashJoin [ss_store_sk,s_store_sk] + Project [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] + SortMergeJoin [ss_item_sk,cs_item_sk] + InputAdapter + WholeStageCodegen (2) + Sort [ss_item_sk] + InputAdapter + Exchange [ss_item_sk] #3 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] + CometBroadcastHashJoin [ss_item_sk,ss_ticket_number,sr_item_sk,sr_ticket_number] + CometBroadcastExchange #4 + CometFilter [ss_item_sk,ss_ticket_number,ss_store_sk,ss_customer_sk,ss_cdemo_sk,ss_promo_sk,ss_hdemo_sk,ss_addr_sk] + CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #5 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometFilter [d_year,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometProject [sr_item_sk,sr_ticket_number] + CometFilter [sr_item_sk,sr_ticket_number] + CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk] + InputAdapter + WholeStageCodegen (8) + Sort [cs_item_sk] + Project [cs_item_sk] + Filter [sale,refund] + HashAggregate [cs_item_sk,sum,sum,isEmpty] [sum(UnscaledValue(cs_ext_list_price)),sum(((cr_refunded_cash + cr_reversed_charge) + cr_store_credit)),sale,refund,sum,sum,isEmpty] + InputAdapter + Exchange [cs_item_sk] #6 + WholeStageCodegen (7) + HashAggregate [cs_item_sk,cs_ext_list_price,cr_refunded_cash,cr_reversed_charge,cr_store_credit] [sum,sum,isEmpty,sum,sum,isEmpty] + Project [cs_item_sk,cs_ext_list_price,cr_refunded_cash,cr_reversed_charge,cr_store_credit] + SortMergeJoin [cs_item_sk,cs_order_number,cr_item_sk,cr_order_number] + InputAdapter + WholeStageCodegen (4) + Sort [cs_item_sk,cs_order_number] + InputAdapter + Exchange [cs_item_sk,cs_order_number] #7 + WholeStageCodegen (3) + ColumnarToRow + InputAdapter + CometProject [cs_item_sk,cs_order_number,cs_ext_list_price] + CometFilter [cs_item_sk,cs_order_number] + CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_order_number,cs_ext_list_price,cs_sold_date_sk] + InputAdapter + WholeStageCodegen (6) + Sort [cr_item_sk,cr_order_number] + InputAdapter + Exchange [cr_item_sk,cr_order_number] #8 + WholeStageCodegen (5) + ColumnarToRow + InputAdapter + CometProject [cr_item_sk,cr_order_number,cr_refunded_cash,cr_reversed_charge,cr_store_credit] + CometFilter [cr_item_sk,cr_order_number] + CometScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_refunded_cash,cr_reversed_charge,cr_store_credit,cr_returned_date_sk] + InputAdapter + ReusedExchange [d_date_sk,d_year] #5 + InputAdapter + BroadcastExchange #9 + WholeStageCodegen (10) + ColumnarToRow + InputAdapter + CometFilter [s_store_sk,s_store_name,s_zip] + CometScan parquet spark_catalog.default.store [s_store_sk,s_store_name,s_zip] + InputAdapter + BroadcastExchange #10 + WholeStageCodegen (11) + ColumnarToRow + InputAdapter + CometFilter [c_customer_sk,c_first_sales_date_sk,c_first_shipto_date_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk] + CometScan parquet spark_catalog.default.customer [c_customer_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk] + InputAdapter + BroadcastExchange #11 + WholeStageCodegen (12) + ColumnarToRow + InputAdapter + CometFilter [d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + InputAdapter + ReusedExchange [d_date_sk,d_year] #11 + InputAdapter + BroadcastExchange #12 + WholeStageCodegen (14) + ColumnarToRow + InputAdapter + CometFilter [cd_demo_sk,cd_marital_status] + CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_marital_status] + InputAdapter + ReusedExchange [cd_demo_sk,cd_marital_status] #12 + InputAdapter + BroadcastExchange #13 + WholeStageCodegen (16) + ColumnarToRow + InputAdapter + CometFilter [p_promo_sk] + CometScan parquet spark_catalog.default.promotion [p_promo_sk] + InputAdapter + BroadcastExchange #14 + WholeStageCodegen (17) + ColumnarToRow + InputAdapter + CometFilter [hd_demo_sk,hd_income_band_sk] + CometScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_income_band_sk] + InputAdapter + ReusedExchange [hd_demo_sk,hd_income_band_sk] #14 + InputAdapter + BroadcastExchange #15 + WholeStageCodegen (19) + ColumnarToRow + InputAdapter + CometFilter [ca_address_sk] + CometScan parquet spark_catalog.default.customer_address [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] + InputAdapter + ReusedExchange [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] #15 + InputAdapter + BroadcastExchange #16 + WholeStageCodegen (21) + ColumnarToRow + InputAdapter + CometFilter [ib_income_band_sk] + CometScan parquet spark_catalog.default.income_band [ib_income_band_sk] + InputAdapter + ReusedExchange [ib_income_band_sk] #16 + InputAdapter + BroadcastExchange #17 + WholeStageCodegen (23) + ColumnarToRow + InputAdapter + CometProject [i_item_sk,i_product_name] + CometFilter [i_current_price,i_color,i_item_sk] + CometScan parquet spark_catalog.default.item [i_item_sk,i_current_price,i_color,i_product_name] + InputAdapter + WholeStageCodegen (50) + Sort [item_sk,store_name,store_zip] + InputAdapter + Exchange [item_sk,store_name,store_zip] #18 + WholeStageCodegen (49) + HashAggregate [i_product_name,i_item_sk,s_store_name,s_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,d_year,d_year,d_year,count,sum,sum,sum] [count(1),sum(UnscaledValue(ss_wholesale_cost)),sum(UnscaledValue(ss_list_price)),sum(UnscaledValue(ss_coupon_amt)),item_sk,store_name,store_zip,syear,cnt,s1,s2,s3,count,sum,sum,sum] + HashAggregate [i_product_name,i_item_sk,s_store_name,s_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,d_year,d_year,d_year,ss_wholesale_cost,ss_list_price,ss_coupon_amt] [count,sum,sum,sum,count,sum,sum,sum] + Project [ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,d_year,d_year,s_store_name,s_zip,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip,i_item_sk,i_product_name] + BroadcastHashJoin [ss_item_sk,i_item_sk] + Project [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip] + BroadcastHashJoin [hd_income_band_sk,ib_income_band_sk] + Project [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip] + BroadcastHashJoin [hd_income_band_sk,ib_income_band_sk] + Project [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip,ca_street_number,ca_street_name,ca_city,ca_zip] + BroadcastHashJoin [c_current_addr_sk,ca_address_sk] + Project [ss_item_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk,ca_street_number,ca_street_name,ca_city,ca_zip] + BroadcastHashJoin [ss_addr_sk,ca_address_sk] + Project [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_addr_sk,d_year,d_year,hd_income_band_sk,hd_income_band_sk] + BroadcastHashJoin [c_current_hdemo_sk,hd_demo_sk] + Project [ss_item_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,hd_income_band_sk] + BroadcastHashJoin [ss_hdemo_sk,hd_demo_sk] + Project [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year] + BroadcastHashJoin [ss_promo_sk,p_promo_sk] + Project [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year] + BroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk,cd_marital_status,cd_marital_status] + Project [ss_item_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year,cd_marital_status] + BroadcastHashJoin [ss_cdemo_sk,cd_demo_sk] + Project [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,d_year,d_year] + BroadcastHashJoin [c_first_shipto_date_sk,d_date_sk] + Project [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,d_year] + BroadcastHashJoin [c_first_sales_date_sk,d_date_sk] + Project [ss_item_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk] + BroadcastHashJoin [ss_customer_sk,c_customer_sk] + Project [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year,s_store_name,s_zip] + BroadcastHashJoin [ss_store_sk,s_store_sk] + Project [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,d_year] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] + SortMergeJoin [ss_item_sk,cs_item_sk] + InputAdapter + WholeStageCodegen (27) + Sort [ss_item_sk] + InputAdapter + Exchange [ss_item_sk] #19 + WholeStageCodegen (26) + ColumnarToRow + InputAdapter + CometProject [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] + CometBroadcastHashJoin [ss_item_sk,ss_ticket_number,sr_item_sk,sr_ticket_number] + CometBroadcastExchange #20 + CometFilter [ss_item_sk,ss_ticket_number,ss_store_sk,ss_customer_sk,ss_cdemo_sk,ss_promo_sk,ss_hdemo_sk,ss_addr_sk] + CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_cdemo_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_wholesale_cost,ss_list_price,ss_coupon_amt,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #2 + BroadcastExchange #21 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometFilter [d_year,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometProject [sr_item_sk,sr_ticket_number] + CometFilter [sr_item_sk,sr_ticket_number] + CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk] + InputAdapter + WholeStageCodegen (33) + Sort [cs_item_sk] + Project [cs_item_sk] + Filter [sale,refund] + HashAggregate [cs_item_sk,sum,sum,isEmpty] [sum(UnscaledValue(cs_ext_list_price)),sum(((cr_refunded_cash + cr_reversed_charge) + cr_store_credit)),sale,refund,sum,sum,isEmpty] + InputAdapter + ReusedExchange [cs_item_sk,sum,sum,isEmpty] #6 + InputAdapter + ReusedExchange [d_date_sk,d_year] #21 + InputAdapter + ReusedExchange [s_store_sk,s_store_name,s_zip] #9 + InputAdapter + ReusedExchange [c_customer_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk,c_first_shipto_date_sk,c_first_sales_date_sk] #10 + InputAdapter + ReusedExchange [d_date_sk,d_year] #11 + InputAdapter + ReusedExchange [d_date_sk,d_year] #11 + InputAdapter + ReusedExchange [cd_demo_sk,cd_marital_status] #12 + InputAdapter + ReusedExchange [cd_demo_sk,cd_marital_status] #12 + InputAdapter + ReusedExchange [p_promo_sk] #13 + InputAdapter + ReusedExchange [hd_demo_sk,hd_income_band_sk] #14 + InputAdapter + ReusedExchange [hd_demo_sk,hd_income_band_sk] #14 + InputAdapter + ReusedExchange [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] #15 + InputAdapter + ReusedExchange [ca_address_sk,ca_street_number,ca_street_name,ca_city,ca_zip] #15 + InputAdapter + ReusedExchange [ib_income_band_sk] #16 + InputAdapter + ReusedExchange [ib_income_band_sk] #16 + InputAdapter + ReusedExchange [i_item_sk,i_product_name] #17 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 new file mode 100644 index 000000000..6ca27323a --- /dev/null +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q67a/explain.txt @@ -0,0 +1,466 @@ +== Physical Plan == +TakeOrderedAndProject (70) ++- * Filter (69) + +- Window (68) + +- WindowGroupLimit (67) + +- * Sort (66) + +- Exchange (65) + +- WindowGroupLimit (64) + +- * Sort (63) + +- Union (62) + :- * HashAggregate (21) + : +- Exchange (20) + : +- * HashAggregate (19) + : +- * Project (18) + : +- * BroadcastHashJoin Inner BuildRight (17) + : :- * Project (12) + : : +- * BroadcastHashJoin Inner BuildRight (11) + : : :- * Project (6) + : : : +- * BroadcastHashJoin Inner BuildRight (5) + : : : :- * ColumnarToRow (3) + : : : : +- CometFilter (2) + : : : : +- CometScan parquet spark_catalog.default.store_sales (1) + : : : +- ReusedExchange (4) + : : +- BroadcastExchange (10) + : : +- * ColumnarToRow (9) + : : +- CometFilter (8) + : : +- CometScan parquet spark_catalog.default.store (7) + : +- BroadcastExchange (16) + : +- * ColumnarToRow (15) + : +- CometFilter (14) + : +- CometScan parquet spark_catalog.default.item (13) + :- * HashAggregate (26) + : +- Exchange (25) + : +- * HashAggregate (24) + : +- * HashAggregate (23) + : +- ReusedExchange (22) + :- * HashAggregate (31) + : +- Exchange (30) + : +- * HashAggregate (29) + : +- * HashAggregate (28) + : +- ReusedExchange (27) + :- * HashAggregate (36) + : +- Exchange (35) + : +- * HashAggregate (34) + : +- * HashAggregate (33) + : +- ReusedExchange (32) + :- * HashAggregate (41) + : +- Exchange (40) + : +- * HashAggregate (39) + : +- * HashAggregate (38) + : +- ReusedExchange (37) + :- * HashAggregate (46) + : +- Exchange (45) + : +- * HashAggregate (44) + : +- * HashAggregate (43) + : +- ReusedExchange (42) + :- * HashAggregate (51) + : +- Exchange (50) + : +- * HashAggregate (49) + : +- * HashAggregate (48) + : +- ReusedExchange (47) + :- * HashAggregate (56) + : +- Exchange (55) + : +- * HashAggregate (54) + : +- * HashAggregate (53) + : +- ReusedExchange (52) + +- * HashAggregate (61) + +- Exchange (60) + +- * HashAggregate (59) + +- * HashAggregate (58) + +- ReusedExchange (57) + + +(1) Scan parquet spark_catalog.default.store_sales +Output [5]: [ss_item_sk#1, ss_store_sk#2, ss_quantity#3, ss_sales_price#4, ss_sold_date_sk#5] +Batched: true +Location: InMemoryFileIndex [] +PartitionFilters: [isnotnull(ss_sold_date_sk#5), dynamicpruningexpression(ss_sold_date_sk#5 IN dynamicpruning#6)] +PushedFilters: [IsNotNull(ss_store_sk), IsNotNull(ss_item_sk)] +ReadSchema: struct<ss_item_sk:int,ss_store_sk:int,ss_quantity:int,ss_sales_price:decimal(7,2)> + +(2) CometFilter +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)) + +(3) ColumnarToRow [codegen id : 4] +Input [5]: [ss_item_sk#1, ss_store_sk#2, ss_quantity#3, ss_sales_price#4, ss_sold_date_sk#5] + +(4) ReusedExchange [Reuses operator id: 75] +Output [4]: [d_date_sk#7, d_year#8, d_moy#9, d_qoy#10] + +(5) BroadcastHashJoin [codegen id : 4] +Left keys [1]: [ss_sold_date_sk#5] +Right keys [1]: [d_date_sk#7] +Join type: Inner +Join condition: None + +(6) Project [codegen id : 4] +Output [7]: [ss_item_sk#1, ss_store_sk#2, ss_quantity#3, ss_sales_price#4, d_year#8, d_moy#9, d_qoy#10] +Input [9]: [ss_item_sk#1, ss_store_sk#2, ss_quantity#3, ss_sales_price#4, ss_sold_date_sk#5, d_date_sk#7, d_year#8, d_moy#9, d_qoy#10] + +(7) Scan parquet spark_catalog.default.store +Output [2]: [s_store_sk#11, s_store_id#12] +Batched: true +Location [not included in comparison]/{warehouse_dir}/store] +PushedFilters: [IsNotNull(s_store_sk)] +ReadSchema: struct<s_store_sk:int,s_store_id:string> + +(8) CometFilter +Input [2]: [s_store_sk#11, s_store_id#12] +Condition : isnotnull(s_store_sk#11) + +(9) ColumnarToRow [codegen id : 2] +Input [2]: [s_store_sk#11, s_store_id#12] + +(10) BroadcastExchange +Input [2]: [s_store_sk#11, s_store_id#12] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=1] + +(11) BroadcastHashJoin [codegen id : 4] +Left keys [1]: [ss_store_sk#2] +Right keys [1]: [s_store_sk#11] +Join type: Inner +Join condition: None + +(12) Project [codegen id : 4] +Output [7]: [ss_item_sk#1, ss_quantity#3, ss_sales_price#4, d_year#8, d_moy#9, d_qoy#10, s_store_id#12] +Input [9]: [ss_item_sk#1, ss_store_sk#2, ss_quantity#3, ss_sales_price#4, d_year#8, d_moy#9, d_qoy#10, s_store_sk#11, s_store_id#12] + +(13) Scan parquet spark_catalog.default.item +Output [5]: [i_item_sk#13, i_brand#14, i_class#15, i_category#16, i_product_name#17] +Batched: true +Location [not included in comparison]/{warehouse_dir}/item] +PushedFilters: [IsNotNull(i_item_sk)] +ReadSchema: struct<i_item_sk:int,i_brand:string,i_class:string,i_category:string,i_product_name:string> + +(14) CometFilter +Input [5]: [i_item_sk#13, i_brand#14, i_class#15, i_category#16, i_product_name#17] +Condition : isnotnull(i_item_sk#13) + +(15) ColumnarToRow [codegen id : 3] +Input [5]: [i_item_sk#13, i_brand#14, i_class#15, i_category#16, i_product_name#17] + +(16) BroadcastExchange +Input [5]: [i_item_sk#13, i_brand#14, i_class#15, i_category#16, i_product_name#17] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=2] + +(17) BroadcastHashJoin [codegen id : 4] +Left keys [1]: [ss_item_sk#1] +Right keys [1]: [i_item_sk#13] +Join type: Inner +Join condition: None + +(18) Project [codegen id : 4] +Output [10]: [ss_quantity#3, ss_sales_price#4, d_year#8, d_moy#9, d_qoy#10, s_store_id#12, i_brand#14, i_class#15, i_category#16, i_product_name#17] +Input [12]: [ss_item_sk#1, ss_quantity#3, ss_sales_price#4, d_year#8, d_moy#9, d_qoy#10, s_store_id#12, i_item_sk#13, i_brand#14, i_class#15, i_category#16, i_product_name#17] + +(19) HashAggregate [codegen id : 4] +Input [10]: [ss_quantity#3, ss_sales_price#4, d_year#8, d_moy#9, d_qoy#10, s_store_id#12, i_brand#14, i_class#15, i_category#16, i_product_name#17] +Keys [8]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, s_store_id#12] +Functions [1]: [partial_sum(coalesce((ss_sales_price#4 * cast(ss_quantity#3 as decimal(10,0))), 0.00))] +Aggregate Attributes [2]: [sum#18, isEmpty#19] +Results [10]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, s_store_id#12, sum#20, isEmpty#21] + +(20) Exchange +Input [10]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, s_store_id#12, sum#20, isEmpty#21] +Arguments: hashpartitioning(i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, s_store_id#12, 5), ENSURE_REQUIREMENTS, [plan_id=3] + +(21) HashAggregate [codegen id : 5] +Input [10]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, s_store_id#12, sum#20, isEmpty#21] +Keys [8]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, s_store_id#12] +Functions [1]: [sum(coalesce((ss_sales_price#4 * cast(ss_quantity#3 as decimal(10,0))), 0.00))] +Aggregate Attributes [1]: [sum(coalesce((ss_sales_price#4 * cast(ss_quantity#3 as decimal(10,0))), 0.00))#22] +Results [9]: [i_category#16 AS i_category#23, i_class#15 AS i_class#24, i_brand#14 AS i_brand#25, i_product_name#17 AS i_product_name#26, d_year#8 AS d_year#27, d_qoy#10 AS d_qoy#28, d_moy#9 AS d_moy#29, s_store_id#12 AS s_store_id#30, cast(sum(coalesce((ss_sales_price#4 * cast(ss_quantity#3 as decimal(10,0))), 0.00))#22 as decimal(38,2)) AS sumsales#31] + +(22) ReusedExchange [Reuses operator id: 20] +Output [10]: [i_category#32, i_class#33, i_brand#34, i_product_name#35, d_year#36, d_qoy#37, d_moy#38, s_store_id#39, sum#40, isEmpty#41] + +(23) HashAggregate [codegen id : 10] +Input [10]: [i_category#32, i_class#33, i_brand#34, i_product_name#35, d_year#36, d_qoy#37, d_moy#38, s_store_id#39, sum#40, isEmpty#41] +Keys [8]: [i_category#32, i_class#33, i_brand#34, i_product_name#35, d_year#36, d_qoy#37, d_moy#38, s_store_id#39] +Functions [1]: [sum(coalesce((ss_sales_price#42 * cast(ss_quantity#43 as decimal(10,0))), 0.00))] +Aggregate Attributes [1]: [sum(coalesce((ss_sales_price#42 * cast(ss_quantity#43 as decimal(10,0))), 0.00))#22] +Results [8]: [i_category#32, i_class#33, i_brand#34, i_product_name#35, d_year#36, d_qoy#37, d_moy#38, sum(coalesce((ss_sales_price#42 * cast(ss_quantity#43 as decimal(10,0))), 0.00))#22 AS sumsales#44] + +(24) HashAggregate [codegen id : 10] +Input [8]: [i_category#32, i_class#33, i_brand#34, i_product_name#35, d_year#36, d_qoy#37, d_moy#38, sumsales#44] +Keys [7]: [i_category#32, i_class#33, i_brand#34, i_product_name#35, d_year#36, d_qoy#37, d_moy#38] +Functions [1]: [partial_sum(sumsales#44)] +Aggregate Attributes [2]: [sum#45, isEmpty#46] +Results [9]: [i_category#32, i_class#33, i_brand#34, i_product_name#35, d_year#36, d_qoy#37, d_moy#38, sum#47, isEmpty#48] + +(25) Exchange +Input [9]: [i_category#32, i_class#33, i_brand#34, i_product_name#35, d_year#36, d_qoy#37, d_moy#38, sum#47, isEmpty#48] +Arguments: hashpartitioning(i_category#32, i_class#33, i_brand#34, i_product_name#35, d_year#36, d_qoy#37, d_moy#38, 5), ENSURE_REQUIREMENTS, [plan_id=4] + +(26) HashAggregate [codegen id : 11] +Input [9]: [i_category#32, i_class#33, i_brand#34, i_product_name#35, d_year#36, d_qoy#37, d_moy#38, sum#47, isEmpty#48] +Keys [7]: [i_category#32, i_class#33, i_brand#34, i_product_name#35, d_year#36, d_qoy#37, d_moy#38] +Functions [1]: [sum(sumsales#44)] +Aggregate Attributes [1]: [sum(sumsales#44)#49] +Results [9]: [i_category#32, i_class#33, i_brand#34, i_product_name#35, d_year#36, d_qoy#37, d_moy#38, null AS s_store_id#50, sum(sumsales#44)#49 AS sumsales#51] + +(27) ReusedExchange [Reuses operator id: 20] +Output [10]: [i_category#52, i_class#53, i_brand#54, i_product_name#55, d_year#56, d_qoy#57, d_moy#58, s_store_id#59, sum#60, isEmpty#61] + +(28) HashAggregate [codegen id : 16] +Input [10]: [i_category#52, i_class#53, i_brand#54, i_product_name#55, d_year#56, d_qoy#57, d_moy#58, s_store_id#59, sum#60, isEmpty#61] +Keys [8]: [i_category#52, i_class#53, i_brand#54, i_product_name#55, d_year#56, d_qoy#57, d_moy#58, s_store_id#59] +Functions [1]: [sum(coalesce((ss_sales_price#62 * cast(ss_quantity#63 as decimal(10,0))), 0.00))] +Aggregate Attributes [1]: [sum(coalesce((ss_sales_price#62 * cast(ss_quantity#63 as decimal(10,0))), 0.00))#22] +Results [7]: [i_category#52, i_class#53, i_brand#54, i_product_name#55, d_year#56, d_qoy#57, sum(coalesce((ss_sales_price#62 * cast(ss_quantity#63 as decimal(10,0))), 0.00))#22 AS sumsales#64] + +(29) HashAggregate [codegen id : 16] +Input [7]: [i_category#52, i_class#53, i_brand#54, i_product_name#55, d_year#56, d_qoy#57, sumsales#64] +Keys [6]: [i_category#52, i_class#53, i_brand#54, i_product_name#55, d_year#56, d_qoy#57] +Functions [1]: [partial_sum(sumsales#64)] +Aggregate Attributes [2]: [sum#65, isEmpty#66] +Results [8]: [i_category#52, i_class#53, i_brand#54, i_product_name#55, d_year#56, d_qoy#57, sum#67, isEmpty#68] + +(30) Exchange +Input [8]: [i_category#52, i_class#53, i_brand#54, i_product_name#55, d_year#56, d_qoy#57, sum#67, isEmpty#68] +Arguments: hashpartitioning(i_category#52, i_class#53, i_brand#54, i_product_name#55, d_year#56, d_qoy#57, 5), ENSURE_REQUIREMENTS, [plan_id=5] + +(31) HashAggregate [codegen id : 17] +Input [8]: [i_category#52, i_class#53, i_brand#54, i_product_name#55, d_year#56, d_qoy#57, sum#67, isEmpty#68] +Keys [6]: [i_category#52, i_class#53, i_brand#54, i_product_name#55, d_year#56, d_qoy#57] +Functions [1]: [sum(sumsales#64)] +Aggregate Attributes [1]: [sum(sumsales#64)#69] +Results [9]: [i_category#52, i_class#53, i_brand#54, i_product_name#55, d_year#56, d_qoy#57, null AS d_moy#70, null AS s_store_id#71, sum(sumsales#64)#69 AS sumsales#72] + +(32) ReusedExchange [Reuses operator id: 20] +Output [10]: [i_category#73, i_class#74, i_brand#75, i_product_name#76, d_year#77, d_qoy#78, d_moy#79, s_store_id#80, sum#81, isEmpty#82] + +(33) HashAggregate [codegen id : 22] +Input [10]: [i_category#73, i_class#74, i_brand#75, i_product_name#76, d_year#77, d_qoy#78, d_moy#79, s_store_id#80, sum#81, isEmpty#82] +Keys [8]: [i_category#73, i_class#74, i_brand#75, i_product_name#76, d_year#77, d_qoy#78, d_moy#79, s_store_id#80] +Functions [1]: [sum(coalesce((ss_sales_price#83 * cast(ss_quantity#84 as decimal(10,0))), 0.00))] +Aggregate Attributes [1]: [sum(coalesce((ss_sales_price#83 * cast(ss_quantity#84 as decimal(10,0))), 0.00))#22] +Results [6]: [i_category#73, i_class#74, i_brand#75, i_product_name#76, d_year#77, sum(coalesce((ss_sales_price#83 * cast(ss_quantity#84 as decimal(10,0))), 0.00))#22 AS sumsales#85] + +(34) HashAggregate [codegen id : 22] +Input [6]: [i_category#73, i_class#74, i_brand#75, i_product_name#76, d_year#77, sumsales#85] +Keys [5]: [i_category#73, i_class#74, i_brand#75, i_product_name#76, d_year#77] +Functions [1]: [partial_sum(sumsales#85)] +Aggregate Attributes [2]: [sum#86, isEmpty#87] +Results [7]: [i_category#73, i_class#74, i_brand#75, i_product_name#76, d_year#77, sum#88, isEmpty#89] + +(35) Exchange +Input [7]: [i_category#73, i_class#74, i_brand#75, i_product_name#76, d_year#77, sum#88, isEmpty#89] +Arguments: hashpartitioning(i_category#73, i_class#74, i_brand#75, i_product_name#76, d_year#77, 5), ENSURE_REQUIREMENTS, [plan_id=6] + +(36) HashAggregate [codegen id : 23] +Input [7]: [i_category#73, i_class#74, i_brand#75, i_product_name#76, d_year#77, sum#88, isEmpty#89] +Keys [5]: [i_category#73, i_class#74, i_brand#75, i_product_name#76, d_year#77] +Functions [1]: [sum(sumsales#85)] +Aggregate Attributes [1]: [sum(sumsales#85)#90] +Results [9]: [i_category#73, i_class#74, i_brand#75, i_product_name#76, d_year#77, null AS d_qoy#91, null AS d_moy#92, null AS s_store_id#93, sum(sumsales#85)#90 AS sumsales#94] + +(37) ReusedExchange [Reuses operator id: 20] +Output [10]: [i_category#95, i_class#96, i_brand#97, i_product_name#98, d_year#99, d_qoy#100, d_moy#101, s_store_id#102, sum#103, isEmpty#104] + +(38) HashAggregate [codegen id : 28] +Input [10]: [i_category#95, i_class#96, i_brand#97, i_product_name#98, d_year#99, d_qoy#100, d_moy#101, s_store_id#102, sum#103, isEmpty#104] +Keys [8]: [i_category#95, i_class#96, i_brand#97, i_product_name#98, d_year#99, d_qoy#100, d_moy#101, s_store_id#102] +Functions [1]: [sum(coalesce((ss_sales_price#105 * cast(ss_quantity#106 as decimal(10,0))), 0.00))] +Aggregate Attributes [1]: [sum(coalesce((ss_sales_price#105 * cast(ss_quantity#106 as decimal(10,0))), 0.00))#22] +Results [5]: [i_category#95, i_class#96, i_brand#97, i_product_name#98, sum(coalesce((ss_sales_price#105 * cast(ss_quantity#106 as decimal(10,0))), 0.00))#22 AS sumsales#107] + +(39) HashAggregate [codegen id : 28] +Input [5]: [i_category#95, i_class#96, i_brand#97, i_product_name#98, sumsales#107] +Keys [4]: [i_category#95, i_class#96, i_brand#97, i_product_name#98] +Functions [1]: [partial_sum(sumsales#107)] +Aggregate Attributes [2]: [sum#108, isEmpty#109] +Results [6]: [i_category#95, i_class#96, i_brand#97, i_product_name#98, sum#110, isEmpty#111] + +(40) Exchange +Input [6]: [i_category#95, i_class#96, i_brand#97, i_product_name#98, sum#110, isEmpty#111] +Arguments: hashpartitioning(i_category#95, i_class#96, i_brand#97, i_product_name#98, 5), ENSURE_REQUIREMENTS, [plan_id=7] + +(41) HashAggregate [codegen id : 29] +Input [6]: [i_category#95, i_class#96, i_brand#97, i_product_name#98, sum#110, isEmpty#111] +Keys [4]: [i_category#95, i_class#96, i_brand#97, i_product_name#98] +Functions [1]: [sum(sumsales#107)] +Aggregate Attributes [1]: [sum(sumsales#107)#112] +Results [9]: [i_category#95, i_class#96, i_brand#97, i_product_name#98, null AS d_year#113, null AS d_qoy#114, null AS d_moy#115, null AS s_store_id#116, sum(sumsales#107)#112 AS sumsales#117] + +(42) ReusedExchange [Reuses operator id: 20] +Output [10]: [i_category#118, i_class#119, i_brand#120, i_product_name#121, d_year#122, d_qoy#123, d_moy#124, s_store_id#125, sum#126, isEmpty#127] + +(43) HashAggregate [codegen id : 34] +Input [10]: [i_category#118, i_class#119, i_brand#120, i_product_name#121, d_year#122, d_qoy#123, d_moy#124, s_store_id#125, sum#126, isEmpty#127] +Keys [8]: [i_category#118, i_class#119, i_brand#120, i_product_name#121, d_year#122, d_qoy#123, d_moy#124, s_store_id#125] +Functions [1]: [sum(coalesce((ss_sales_price#128 * cast(ss_quantity#129 as decimal(10,0))), 0.00))] +Aggregate Attributes [1]: [sum(coalesce((ss_sales_price#128 * cast(ss_quantity#129 as decimal(10,0))), 0.00))#22] +Results [4]: [i_category#118, i_class#119, i_brand#120, sum(coalesce((ss_sales_price#128 * cast(ss_quantity#129 as decimal(10,0))), 0.00))#22 AS sumsales#130] + +(44) HashAggregate [codegen id : 34] +Input [4]: [i_category#118, i_class#119, i_brand#120, sumsales#130] +Keys [3]: [i_category#118, i_class#119, i_brand#120] +Functions [1]: [partial_sum(sumsales#130)] +Aggregate Attributes [2]: [sum#131, isEmpty#132] +Results [5]: [i_category#118, i_class#119, i_brand#120, sum#133, isEmpty#134] + +(45) Exchange +Input [5]: [i_category#118, i_class#119, i_brand#120, sum#133, isEmpty#134] +Arguments: hashpartitioning(i_category#118, i_class#119, i_brand#120, 5), ENSURE_REQUIREMENTS, [plan_id=8] + +(46) HashAggregate [codegen id : 35] +Input [5]: [i_category#118, i_class#119, i_brand#120, sum#133, isEmpty#134] +Keys [3]: [i_category#118, i_class#119, i_brand#120] +Functions [1]: [sum(sumsales#130)] +Aggregate Attributes [1]: [sum(sumsales#130)#135] +Results [9]: [i_category#118, i_class#119, i_brand#120, null AS i_product_name#136, null AS d_year#137, null AS d_qoy#138, null AS d_moy#139, null AS s_store_id#140, sum(sumsales#130)#135 AS sumsales#141] + +(47) ReusedExchange [Reuses operator id: 20] +Output [10]: [i_category#142, i_class#143, i_brand#144, i_product_name#145, d_year#146, d_qoy#147, d_moy#148, s_store_id#149, sum#150, isEmpty#151] + +(48) HashAggregate [codegen id : 40] +Input [10]: [i_category#142, i_class#143, i_brand#144, i_product_name#145, d_year#146, d_qoy#147, d_moy#148, s_store_id#149, sum#150, isEmpty#151] +Keys [8]: [i_category#142, i_class#143, i_brand#144, i_product_name#145, d_year#146, d_qoy#147, d_moy#148, s_store_id#149] +Functions [1]: [sum(coalesce((ss_sales_price#152 * cast(ss_quantity#153 as decimal(10,0))), 0.00))] +Aggregate Attributes [1]: [sum(coalesce((ss_sales_price#152 * cast(ss_quantity#153 as decimal(10,0))), 0.00))#22] +Results [3]: [i_category#142, i_class#143, sum(coalesce((ss_sales_price#152 * cast(ss_quantity#153 as decimal(10,0))), 0.00))#22 AS sumsales#154] + +(49) HashAggregate [codegen id : 40] +Input [3]: [i_category#142, i_class#143, sumsales#154] +Keys [2]: [i_category#142, i_class#143] +Functions [1]: [partial_sum(sumsales#154)] +Aggregate Attributes [2]: [sum#155, isEmpty#156] +Results [4]: [i_category#142, i_class#143, sum#157, isEmpty#158] + +(50) Exchange +Input [4]: [i_category#142, i_class#143, sum#157, isEmpty#158] +Arguments: hashpartitioning(i_category#142, i_class#143, 5), ENSURE_REQUIREMENTS, [plan_id=9] + +(51) HashAggregate [codegen id : 41] +Input [4]: [i_category#142, i_class#143, sum#157, isEmpty#158] +Keys [2]: [i_category#142, i_class#143] +Functions [1]: [sum(sumsales#154)] +Aggregate Attributes [1]: [sum(sumsales#154)#159] +Results [9]: [i_category#142, i_class#143, null AS i_brand#160, null AS i_product_name#161, null AS d_year#162, null AS d_qoy#163, null AS d_moy#164, null AS s_store_id#165, sum(sumsales#154)#159 AS sumsales#166] + +(52) ReusedExchange [Reuses operator id: 20] +Output [10]: [i_category#167, i_class#168, i_brand#169, i_product_name#170, d_year#171, d_qoy#172, d_moy#173, s_store_id#174, sum#175, isEmpty#176] + +(53) HashAggregate [codegen id : 46] +Input [10]: [i_category#167, i_class#168, i_brand#169, i_product_name#170, d_year#171, d_qoy#172, d_moy#173, s_store_id#174, sum#175, isEmpty#176] +Keys [8]: [i_category#167, i_class#168, i_brand#169, i_product_name#170, d_year#171, d_qoy#172, d_moy#173, s_store_id#174] +Functions [1]: [sum(coalesce((ss_sales_price#177 * cast(ss_quantity#178 as decimal(10,0))), 0.00))] +Aggregate Attributes [1]: [sum(coalesce((ss_sales_price#177 * cast(ss_quantity#178 as decimal(10,0))), 0.00))#22] +Results [2]: [i_category#167, sum(coalesce((ss_sales_price#177 * cast(ss_quantity#178 as decimal(10,0))), 0.00))#22 AS sumsales#179] + +(54) HashAggregate [codegen id : 46] +Input [2]: [i_category#167, sumsales#179] +Keys [1]: [i_category#167] +Functions [1]: [partial_sum(sumsales#179)] +Aggregate Attributes [2]: [sum#180, isEmpty#181] +Results [3]: [i_category#167, sum#182, isEmpty#183] + +(55) Exchange +Input [3]: [i_category#167, sum#182, isEmpty#183] +Arguments: hashpartitioning(i_category#167, 5), ENSURE_REQUIREMENTS, [plan_id=10] + +(56) HashAggregate [codegen id : 47] +Input [3]: [i_category#167, sum#182, isEmpty#183] +Keys [1]: [i_category#167] +Functions [1]: [sum(sumsales#179)] +Aggregate Attributes [1]: [sum(sumsales#179)#184] +Results [9]: [i_category#167, null AS i_class#185, null AS i_brand#186, null AS i_product_name#187, null AS d_year#188, null AS d_qoy#189, null AS d_moy#190, null AS s_store_id#191, sum(sumsales#179)#184 AS sumsales#192] + +(57) ReusedExchange [Reuses operator id: 20] +Output [10]: [i_category#193, i_class#194, i_brand#195, i_product_name#196, d_year#197, d_qoy#198, d_moy#199, s_store_id#200, sum#201, isEmpty#202] + +(58) HashAggregate [codegen id : 52] +Input [10]: [i_category#193, i_class#194, i_brand#195, i_product_name#196, d_year#197, d_qoy#198, d_moy#199, s_store_id#200, sum#201, isEmpty#202] +Keys [8]: [i_category#193, i_class#194, i_brand#195, i_product_name#196, d_year#197, d_qoy#198, d_moy#199, s_store_id#200] +Functions [1]: [sum(coalesce((ss_sales_price#203 * cast(ss_quantity#204 as decimal(10,0))), 0.00))] +Aggregate Attributes [1]: [sum(coalesce((ss_sales_price#203 * cast(ss_quantity#204 as decimal(10,0))), 0.00))#22] +Results [1]: [sum(coalesce((ss_sales_price#203 * cast(ss_quantity#204 as decimal(10,0))), 0.00))#22 AS sumsales#205] + +(59) HashAggregate [codegen id : 52] +Input [1]: [sumsales#205] +Keys: [] +Functions [1]: [partial_sum(sumsales#205)] +Aggregate Attributes [2]: [sum#206, isEmpty#207] +Results [2]: [sum#208, isEmpty#209] + +(60) Exchange +Input [2]: [sum#208, isEmpty#209] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=11] + +(61) HashAggregate [codegen id : 53] +Input [2]: [sum#208, isEmpty#209] +Keys: [] +Functions [1]: [sum(sumsales#205)] +Aggregate Attributes [1]: [sum(sumsales#205)#210] +Results [9]: [null AS i_category#211, null AS i_class#212, null AS i_brand#213, null AS i_product_name#214, null AS d_year#215, null AS d_qoy#216, null AS d_moy#217, null AS s_store_id#218, sum(sumsales#205)#210 AS sumsales#219] + +(62) Union + +(63) Sort [codegen id : 54] +Input [9]: [i_category#23, i_class#24, i_brand#25, i_product_name#26, d_year#27, d_qoy#28, d_moy#29, s_store_id#30, sumsales#31] +Arguments: [i_category#23 ASC NULLS FIRST, sumsales#31 DESC NULLS LAST], false, 0 + +(64) WindowGroupLimit +Input [9]: [i_category#23, i_class#24, i_brand#25, i_product_name#26, d_year#27, d_qoy#28, d_moy#29, s_store_id#30, sumsales#31] +Arguments: [i_category#23], [sumsales#31 DESC NULLS LAST], rank(sumsales#31), 100, Partial + +(65) Exchange +Input [9]: [i_category#23, i_class#24, i_brand#25, i_product_name#26, d_year#27, d_qoy#28, d_moy#29, s_store_id#30, sumsales#31] +Arguments: hashpartitioning(i_category#23, 5), ENSURE_REQUIREMENTS, [plan_id=12] + +(66) Sort [codegen id : 55] +Input [9]: [i_category#23, i_class#24, i_brand#25, i_product_name#26, d_year#27, d_qoy#28, d_moy#29, s_store_id#30, sumsales#31] +Arguments: [i_category#23 ASC NULLS FIRST, sumsales#31 DESC NULLS LAST], false, 0 + +(67) WindowGroupLimit +Input [9]: [i_category#23, i_class#24, i_brand#25, i_product_name#26, d_year#27, d_qoy#28, d_moy#29, s_store_id#30, sumsales#31] +Arguments: [i_category#23], [sumsales#31 DESC NULLS LAST], rank(sumsales#31), 100, Final + +(68) Window +Input [9]: [i_category#23, i_class#24, i_brand#25, i_product_name#26, d_year#27, d_qoy#28, d_moy#29, s_store_id#30, sumsales#31] +Arguments: [rank(sumsales#31) windowspecdefinition(i_category#23, sumsales#31 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#220], [i_category#23], [sumsales#31 DESC NULLS LAST] + +(69) Filter [codegen id : 56] +Input [10]: [i_category#23, i_class#24, i_brand#25, i_product_name#26, d_year#27, d_qoy#28, d_moy#29, s_store_id#30, sumsales#31, rk#220] +Condition : (rk#220 <= 100) + +(70) TakeOrderedAndProject +Input [10]: [i_category#23, i_class#24, i_brand#25, i_product_name#26, d_year#27, d_qoy#28, d_moy#29, s_store_id#30, sumsales#31, rk#220] +Arguments: 100, [i_category#23 ASC NULLS FIRST, i_class#24 ASC NULLS FIRST, i_brand#25 ASC NULLS FIRST, i_product_name#26 ASC NULLS FIRST, d_year#27 ASC NULLS FIRST, d_qoy#28 ASC NULLS FIRST, d_moy#29 ASC NULLS FIRST, s_store_id#30 ASC NULLS FIRST, sumsales#31 ASC NULLS FIRST, rk#220 ASC NULLS FIRST], [i_category#23, i_class#24, i_brand#25, i_product_name#26, d_year#27, d_qoy#28, d_moy#29, s_store_id#30, sumsales#31, rk#220] + +===== Subqueries ===== + +Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#5 IN dynamicpruning#6 +BroadcastExchange (75) ++- * ColumnarToRow (74) + +- CometProject (73) + +- CometFilter (72) + +- CometScan parquet spark_catalog.default.date_dim (71) + + +(71) Scan parquet spark_catalog.default.date_dim +Output [5]: [d_date_sk#7, d_month_seq#221, d_year#8, d_moy#9, d_qoy#10] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1212), LessThanOrEqual(d_month_seq,1223), IsNotNull(d_date_sk)] +ReadSchema: struct<d_date_sk:int,d_month_seq:int,d_year:int,d_moy:int,d_qoy:int> + +(72) CometFilter +Input [5]: [d_date_sk#7, d_month_seq#221, d_year#8, d_moy#9, d_qoy#10] +Condition : (((isnotnull(d_month_seq#221) AND (d_month_seq#221 >= 1212)) AND (d_month_seq#221 <= 1223)) AND isnotnull(d_date_sk#7)) + +(73) CometProject +Input [5]: [d_date_sk#7, d_month_seq#221, d_year#8, d_moy#9, d_qoy#10] +Arguments: [d_date_sk#7, d_year#8, d_moy#9, d_qoy#10], [d_date_sk#7, d_year#8, d_moy#9, d_qoy#10] + +(74) ColumnarToRow [codegen id : 1] +Input [4]: [d_date_sk#7, d_year#8, d_moy#9, d_qoy#10] + +(75) BroadcastExchange +Input [4]: [d_date_sk#7, d_year#8, d_moy#9, d_qoy#10] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=13] + + 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 new file mode 100644 index 000000000..4d2d3972d --- /dev/null +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q67a/simplified.txt @@ -0,0 +1,127 @@ +TakeOrderedAndProject [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,rk] + WholeStageCodegen (56) + Filter [rk] + InputAdapter + Window [sumsales,i_category] + WindowGroupLimit [i_category,sumsales] + WholeStageCodegen (55) + Sort [i_category,sumsales] + InputAdapter + Exchange [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] + InputAdapter + Exchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id] #2 + WholeStageCodegen (4) + HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,ss_sales_price,ss_quantity] [sum,isEmpty,sum,isEmpty] + Project [ss_quantity,ss_sales_price,d_year,d_moy,d_qoy,s_store_id,i_brand,i_class,i_category,i_product_name] + BroadcastHashJoin [ss_item_sk,i_item_sk] + Project [ss_item_sk,ss_quantity,ss_sales_price,d_year,d_moy,d_qoy,s_store_id] + BroadcastHashJoin [ss_store_sk,s_store_sk] + Project [ss_item_sk,ss_store_sk,ss_quantity,ss_sales_price,d_year,d_moy,d_qoy] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + ColumnarToRow + InputAdapter + CometFilter [ss_store_sk,ss_item_sk] + CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_quantity,ss_sales_price,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk,d_year,d_moy,d_qoy] + CometFilter [d_month_seq,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_year,d_moy,d_qoy] + InputAdapter + ReusedExchange [d_date_sk,d_year,d_moy,d_qoy] #3 + InputAdapter + BroadcastExchange #4 + WholeStageCodegen (2) + ColumnarToRow + InputAdapter + CometFilter [s_store_sk] + CometScan parquet spark_catalog.default.store [s_store_sk,s_store_id] + InputAdapter + BroadcastExchange #5 + WholeStageCodegen (3) + ColumnarToRow + InputAdapter + CometFilter [i_item_sk] + CometScan parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category,i_product_name] + WholeStageCodegen (11) + HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,sum,isEmpty] [sum(sumsales),s_store_id,sumsales,sum,isEmpty] + InputAdapter + Exchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy] #6 + WholeStageCodegen (10) + HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,sumsales] [sum,isEmpty,sum,isEmpty] + HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] [sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00)),sumsales,sum,isEmpty] + InputAdapter + ReusedExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] #2 + WholeStageCodegen (17) + HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,sum,isEmpty] [sum(sumsales),d_moy,s_store_id,sumsales,sum,isEmpty] + InputAdapter + Exchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy] #7 + WholeStageCodegen (16) + HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,sumsales] [sum,isEmpty,sum,isEmpty] + HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] [sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00)),sumsales,sum,isEmpty] + InputAdapter + ReusedExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] #2 + WholeStageCodegen (23) + HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,sum,isEmpty] [sum(sumsales),d_qoy,d_moy,s_store_id,sumsales,sum,isEmpty] + InputAdapter + Exchange [i_category,i_class,i_brand,i_product_name,d_year] #8 + WholeStageCodegen (22) + HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,sumsales] [sum,isEmpty,sum,isEmpty] + HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] [sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00)),sumsales,sum,isEmpty] + InputAdapter + ReusedExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] #2 + WholeStageCodegen (29) + HashAggregate [i_category,i_class,i_brand,i_product_name,sum,isEmpty] [sum(sumsales),d_year,d_qoy,d_moy,s_store_id,sumsales,sum,isEmpty] + InputAdapter + Exchange [i_category,i_class,i_brand,i_product_name] #9 + WholeStageCodegen (28) + HashAggregate [i_category,i_class,i_brand,i_product_name,sumsales] [sum,isEmpty,sum,isEmpty] + HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] [sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00)),sumsales,sum,isEmpty] + InputAdapter + ReusedExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] #2 + WholeStageCodegen (35) + HashAggregate [i_category,i_class,i_brand,sum,isEmpty] [sum(sumsales),i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,sum,isEmpty] + InputAdapter + Exchange [i_category,i_class,i_brand] #10 + WholeStageCodegen (34) + HashAggregate [i_category,i_class,i_brand,sumsales] [sum,isEmpty,sum,isEmpty] + HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] [sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00)),sumsales,sum,isEmpty] + InputAdapter + ReusedExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] #2 + WholeStageCodegen (41) + HashAggregate [i_category,i_class,sum,isEmpty] [sum(sumsales),i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,sum,isEmpty] + InputAdapter + Exchange [i_category,i_class] #11 + WholeStageCodegen (40) + HashAggregate [i_category,i_class,sumsales] [sum,isEmpty,sum,isEmpty] + HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] [sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00)),sumsales,sum,isEmpty] + InputAdapter + ReusedExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] #2 + WholeStageCodegen (47) + HashAggregate [i_category,sum,isEmpty] [sum(sumsales),i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,sum,isEmpty] + InputAdapter + Exchange [i_category] #12 + WholeStageCodegen (46) + HashAggregate [i_category,sumsales] [sum,isEmpty,sum,isEmpty] + HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] [sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00)),sumsales,sum,isEmpty] + InputAdapter + ReusedExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] #2 + WholeStageCodegen (53) + HashAggregate [sum,isEmpty] [sum(sumsales),i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,sum,isEmpty] + InputAdapter + Exchange #13 + WholeStageCodegen (52) + HashAggregate [sumsales] [sum,isEmpty,sum,isEmpty] + HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] [sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00)),sumsales,sum,isEmpty] + InputAdapter + ReusedExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] #2 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 new file mode 100644 index 000000000..6a81a2ec1 --- /dev/null +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q70a/explain.txt @@ -0,0 +1,368 @@ +== Physical Plan == +TakeOrderedAndProject (56) ++- * Project (55) + +- Window (54) + +- * Sort (53) + +- Exchange (52) + +- * HashAggregate (51) + +- Exchange (50) + +- * HashAggregate (49) + +- Union (48) + :- * HashAggregate (37) + : +- Exchange (36) + : +- * HashAggregate (35) + : +- * Project (34) + : +- * BroadcastHashJoin Inner BuildRight (33) + : :- * Project (6) + : : +- * BroadcastHashJoin Inner BuildRight (5) + : : :- * ColumnarToRow (3) + : : : +- CometFilter (2) + : : : +- CometScan parquet spark_catalog.default.store_sales (1) + : : +- ReusedExchange (4) + : +- BroadcastExchange (32) + : +- * BroadcastHashJoin LeftSemi BuildRight (31) + : :- * ColumnarToRow (9) + : : +- CometFilter (8) + : : +- CometScan parquet spark_catalog.default.store (7) + : +- BroadcastExchange (30) + : +- * Project (29) + : +- * Filter (28) + : +- Window (27) + : +- WindowGroupLimit (26) + : +- * Sort (25) + : +- * HashAggregate (24) + : +- Exchange (23) + : +- * HashAggregate (22) + : +- * Project (21) + : +- * BroadcastHashJoin Inner BuildRight (20) + : :- * Project (18) + : : +- * BroadcastHashJoin Inner BuildRight (17) + : : :- * ColumnarToRow (12) + : : : +- CometFilter (11) + : : : +- CometScan parquet spark_catalog.default.store_sales (10) + : : +- BroadcastExchange (16) + : : +- * ColumnarToRow (15) + : : +- CometFilter (14) + : : +- CometScan parquet spark_catalog.default.store (13) + : +- ReusedExchange (19) + :- * HashAggregate (42) + : +- Exchange (41) + : +- * HashAggregate (40) + : +- * HashAggregate (39) + : +- ReusedExchange (38) + +- * HashAggregate (47) + +- Exchange (46) + +- * HashAggregate (45) + +- * HashAggregate (44) + +- ReusedExchange (43) + + +(1) Scan parquet spark_catalog.default.store_sales +Output [3]: [ss_store_sk#1, ss_net_profit#2, ss_sold_date_sk#3] +Batched: true +Location: InMemoryFileIndex [] +PartitionFilters: [isnotnull(ss_sold_date_sk#3), dynamicpruningexpression(ss_sold_date_sk#3 IN dynamicpruning#4)] +PushedFilters: [IsNotNull(ss_store_sk)] +ReadSchema: struct<ss_store_sk:int,ss_net_profit:decimal(7,2)> + +(2) CometFilter +Input [3]: [ss_store_sk#1, ss_net_profit#2, ss_sold_date_sk#3] +Condition : isnotnull(ss_store_sk#1) + +(3) ColumnarToRow [codegen id : 8] +Input [3]: [ss_store_sk#1, ss_net_profit#2, ss_sold_date_sk#3] + +(4) ReusedExchange [Reuses operator id: 61] +Output [1]: [d_date_sk#5] + +(5) BroadcastHashJoin [codegen id : 8] +Left keys [1]: [ss_sold_date_sk#3] +Right keys [1]: [d_date_sk#5] +Join type: Inner +Join condition: None + +(6) Project [codegen id : 8] +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] + +(7) Scan parquet spark_catalog.default.store +Output [3]: [s_store_sk#6, s_county#7, s_state#8] +Batched: true +Location [not included in comparison]/{warehouse_dir}/store] +PushedFilters: [IsNotNull(s_store_sk)] +ReadSchema: struct<s_store_sk:int,s_county:string,s_state:string> + +(8) CometFilter +Input [3]: [s_store_sk#6, s_county#7, s_state#8] +Condition : isnotnull(s_store_sk#6) + +(9) ColumnarToRow [codegen id : 7] +Input [3]: [s_store_sk#6, s_county#7, s_state#8] + +(10) Scan parquet spark_catalog.default.store_sales +Output [3]: [ss_store_sk#9, ss_net_profit#10, ss_sold_date_sk#11] +Batched: true +Location: InMemoryFileIndex [] +PartitionFilters: [isnotnull(ss_sold_date_sk#11), dynamicpruningexpression(ss_sold_date_sk#11 IN dynamicpruning#12)] +PushedFilters: [IsNotNull(ss_store_sk)] +ReadSchema: struct<ss_store_sk:int,ss_net_profit:decimal(7,2)> + +(11) CometFilter +Input [3]: [ss_store_sk#9, ss_net_profit#10, ss_sold_date_sk#11] +Condition : isnotnull(ss_store_sk#9) + +(12) ColumnarToRow [codegen id : 4] +Input [3]: [ss_store_sk#9, ss_net_profit#10, ss_sold_date_sk#11] + +(13) Scan parquet spark_catalog.default.store +Output [2]: [s_store_sk#13, s_state#14] +Batched: true +Location [not included in comparison]/{warehouse_dir}/store] +PushedFilters: [IsNotNull(s_store_sk)] +ReadSchema: struct<s_store_sk:int,s_state:string> + +(14) CometFilter +Input [2]: [s_store_sk#13, s_state#14] +Condition : isnotnull(s_store_sk#13) + +(15) ColumnarToRow [codegen id : 2] +Input [2]: [s_store_sk#13, s_state#14] + +(16) BroadcastExchange +Input [2]: [s_store_sk#13, s_state#14] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=1] + +(17) BroadcastHashJoin [codegen id : 4] +Left keys [1]: [ss_store_sk#9] +Right keys [1]: [s_store_sk#13] +Join type: Inner +Join condition: None + +(18) Project [codegen id : 4] +Output [3]: [ss_net_profit#10, ss_sold_date_sk#11, s_state#14] +Input [5]: [ss_store_sk#9, ss_net_profit#10, ss_sold_date_sk#11, s_store_sk#13, s_state#14] + +(19) ReusedExchange [Reuses operator id: 61] +Output [1]: [d_date_sk#15] + +(20) BroadcastHashJoin [codegen id : 4] +Left keys [1]: [ss_sold_date_sk#11] +Right keys [1]: [d_date_sk#15] +Join type: Inner +Join condition: None + +(21) Project [codegen id : 4] +Output [2]: [ss_net_profit#10, s_state#14] +Input [4]: [ss_net_profit#10, ss_sold_date_sk#11, s_state#14, d_date_sk#15] + +(22) HashAggregate [codegen id : 4] +Input [2]: [ss_net_profit#10, s_state#14] +Keys [1]: [s_state#14] +Functions [1]: [partial_sum(UnscaledValue(ss_net_profit#10))] +Aggregate Attributes [1]: [sum#16] +Results [2]: [s_state#14, sum#17] + +(23) Exchange +Input [2]: [s_state#14, sum#17] +Arguments: hashpartitioning(s_state#14, 5), ENSURE_REQUIREMENTS, [plan_id=2] + +(24) HashAggregate [codegen id : 5] +Input [2]: [s_state#14, sum#17] +Keys [1]: [s_state#14] +Functions [1]: [sum(UnscaledValue(ss_net_profit#10))] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_profit#10))#18] +Results [3]: [s_state#14, MakeDecimal(sum(UnscaledValue(ss_net_profit#10))#18,17,2) AS _w0#19, s_state#14] + +(25) Sort [codegen id : 5] +Input [3]: [s_state#14, _w0#19, s_state#14] +Arguments: [s_state#14 ASC NULLS FIRST, _w0#19 DESC NULLS LAST], false, 0 + +(26) WindowGroupLimit +Input [3]: [s_state#14, _w0#19, s_state#14] +Arguments: [s_state#14], [_w0#19 DESC NULLS LAST], rank(_w0#19), 5, Final + +(27) Window +Input [3]: [s_state#14, _w0#19, s_state#14] +Arguments: [rank(_w0#19) windowspecdefinition(s_state#14, _w0#19 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS ranking#20], [s_state#14], [_w0#19 DESC NULLS LAST] + +(28) Filter [codegen id : 6] +Input [4]: [s_state#14, _w0#19, s_state#14, ranking#20] +Condition : (ranking#20 <= 5) + +(29) Project [codegen id : 6] +Output [1]: [s_state#14] +Input [4]: [s_state#14, _w0#19, s_state#14, ranking#20] + +(30) BroadcastExchange +Input [1]: [s_state#14] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=3] + +(31) BroadcastHashJoin [codegen id : 7] +Left keys [1]: [s_state#8] +Right keys [1]: [s_state#14] +Join type: LeftSemi +Join condition: None + +(32) BroadcastExchange +Input [3]: [s_store_sk#6, s_county#7, s_state#8] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=4] + +(33) BroadcastHashJoin [codegen id : 8] +Left keys [1]: [ss_store_sk#1] +Right keys [1]: [s_store_sk#6] +Join type: Inner +Join condition: None + +(34) Project [codegen id : 8] +Output [3]: [ss_net_profit#2, s_county#7, s_state#8] +Input [5]: [ss_store_sk#1, ss_net_profit#2, s_store_sk#6, s_county#7, s_state#8] + +(35) HashAggregate [codegen id : 8] +Input [3]: [ss_net_profit#2, s_county#7, s_state#8] +Keys [2]: [s_state#8, s_county#7] +Functions [1]: [partial_sum(UnscaledValue(ss_net_profit#2))] +Aggregate Attributes [1]: [sum#21] +Results [3]: [s_state#8, s_county#7, sum#22] + +(36) Exchange +Input [3]: [s_state#8, s_county#7, sum#22] +Arguments: hashpartitioning(s_state#8, s_county#7, 5), ENSURE_REQUIREMENTS, [plan_id=5] + +(37) HashAggregate [codegen id : 9] +Input [3]: [s_state#8, s_county#7, sum#22] +Keys [2]: [s_state#8, s_county#7] +Functions [1]: [sum(UnscaledValue(ss_net_profit#2))] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_profit#2))#23] +Results [6]: [cast(MakeDecimal(sum(UnscaledValue(ss_net_profit#2))#23,17,2) as decimal(27,2)) AS total_sum#24, s_state#8, s_county#7, 0 AS g_state#25, 0 AS g_county#26, 0 AS lochierarchy#27] + +(38) ReusedExchange [Reuses operator id: 36] +Output [3]: [s_state#28, s_county#29, sum#30] + +(39) HashAggregate [codegen id : 18] +Input [3]: [s_state#28, s_county#29, sum#30] +Keys [2]: [s_state#28, s_county#29] +Functions [1]: [sum(UnscaledValue(ss_net_profit#31))] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_profit#31))#23] +Results [2]: [MakeDecimal(sum(UnscaledValue(ss_net_profit#31))#23,17,2) AS total_sum#32, s_state#28] + +(40) HashAggregate [codegen id : 18] +Input [2]: [total_sum#32, s_state#28] +Keys [1]: [s_state#28] +Functions [1]: [partial_sum(total_sum#32)] +Aggregate Attributes [2]: [sum#33, isEmpty#34] +Results [3]: [s_state#28, sum#35, isEmpty#36] + +(41) Exchange +Input [3]: [s_state#28, sum#35, isEmpty#36] +Arguments: hashpartitioning(s_state#28, 5), ENSURE_REQUIREMENTS, [plan_id=6] + +(42) HashAggregate [codegen id : 19] +Input [3]: [s_state#28, sum#35, isEmpty#36] +Keys [1]: [s_state#28] +Functions [1]: [sum(total_sum#32)] +Aggregate Attributes [1]: [sum(total_sum#32)#37] +Results [6]: [sum(total_sum#32)#37 AS total_sum#38, s_state#28, null AS s_county#39, 0 AS g_state#40, 1 AS g_county#41, 1 AS lochierarchy#42] + +(43) ReusedExchange [Reuses operator id: 36] +Output [3]: [s_state#43, s_county#44, sum#45] + +(44) HashAggregate [codegen id : 28] +Input [3]: [s_state#43, s_county#44, sum#45] +Keys [2]: [s_state#43, s_county#44] +Functions [1]: [sum(UnscaledValue(ss_net_profit#46))] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_profit#46))#23] +Results [1]: [MakeDecimal(sum(UnscaledValue(ss_net_profit#46))#23,17,2) AS total_sum#47] + +(45) HashAggregate [codegen id : 28] +Input [1]: [total_sum#47] +Keys: [] +Functions [1]: [partial_sum(total_sum#47)] +Aggregate Attributes [2]: [sum#48, isEmpty#49] +Results [2]: [sum#50, isEmpty#51] + +(46) Exchange +Input [2]: [sum#50, isEmpty#51] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=7] + +(47) HashAggregate [codegen id : 29] +Input [2]: [sum#50, isEmpty#51] +Keys: [] +Functions [1]: [sum(total_sum#47)] +Aggregate Attributes [1]: [sum(total_sum#47)#52] +Results [6]: [sum(total_sum#47)#52 AS total_sum#53, null AS s_state#54, null AS s_county#55, 1 AS g_state#56, 1 AS g_county#57, 2 AS lochierarchy#58] + +(48) Union + +(49) HashAggregate [codegen id : 30] +Input [6]: [total_sum#24, s_state#8, s_county#7, g_state#25, g_county#26, lochierarchy#27] +Keys [6]: [total_sum#24, s_state#8, s_county#7, g_state#25, g_county#26, lochierarchy#27] +Functions: [] +Aggregate Attributes: [] +Results [6]: [total_sum#24, s_state#8, s_county#7, g_state#25, g_county#26, lochierarchy#27] + +(50) Exchange +Input [6]: [total_sum#24, s_state#8, s_county#7, g_state#25, g_county#26, lochierarchy#27] +Arguments: hashpartitioning(total_sum#24, s_state#8, s_county#7, g_state#25, g_county#26, lochierarchy#27, 5), ENSURE_REQUIREMENTS, [plan_id=8] + +(51) HashAggregate [codegen id : 31] +Input [6]: [total_sum#24, s_state#8, s_county#7, g_state#25, g_county#26, lochierarchy#27] +Keys [6]: [total_sum#24, s_state#8, s_county#7, g_state#25, g_county#26, lochierarchy#27] +Functions: [] +Aggregate Attributes: [] +Results [5]: [total_sum#24, s_state#8, s_county#7, lochierarchy#27, CASE WHEN (g_county#26 = 0) THEN s_state#8 END AS _w0#59] + +(52) Exchange +Input [5]: [total_sum#24, s_state#8, s_county#7, lochierarchy#27, _w0#59] +Arguments: hashpartitioning(lochierarchy#27, _w0#59, 5), ENSURE_REQUIREMENTS, [plan_id=9] + +(53) Sort [codegen id : 32] +Input [5]: [total_sum#24, s_state#8, s_county#7, lochierarchy#27, _w0#59] +Arguments: [lochierarchy#27 ASC NULLS FIRST, _w0#59 ASC NULLS FIRST, total_sum#24 DESC NULLS LAST], false, 0 + +(54) Window +Input [5]: [total_sum#24, s_state#8, s_county#7, lochierarchy#27, _w0#59] +Arguments: [rank(total_sum#24) windowspecdefinition(lochierarchy#27, _w0#59, total_sum#24 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rank_within_parent#60], [lochierarchy#27, _w0#59], [total_sum#24 DESC NULLS LAST] + +(55) Project [codegen id : 33] +Output [5]: [total_sum#24, s_state#8, s_county#7, lochierarchy#27, rank_within_parent#60] +Input [6]: [total_sum#24, s_state#8, s_county#7, lochierarchy#27, _w0#59, rank_within_parent#60] + +(56) TakeOrderedAndProject +Input [5]: [total_sum#24, s_state#8, s_county#7, lochierarchy#27, rank_within_parent#60] +Arguments: 100, [lochierarchy#27 DESC NULLS LAST, CASE WHEN (lochierarchy#27 = 0) THEN s_state#8 END ASC NULLS FIRST, rank_within_parent#60 ASC NULLS FIRST], [total_sum#24, s_state#8, s_county#7, lochierarchy#27, rank_within_parent#60] + +===== Subqueries ===== + +Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#3 IN dynamicpruning#4 +BroadcastExchange (61) ++- * ColumnarToRow (60) + +- CometProject (59) + +- CometFilter (58) + +- CometScan parquet spark_catalog.default.date_dim (57) + + +(57) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#5, d_month_seq#61] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1212), LessThanOrEqual(d_month_seq,1223), IsNotNull(d_date_sk)] +ReadSchema: struct<d_date_sk:int,d_month_seq:int> + +(58) CometFilter +Input [2]: [d_date_sk#5, d_month_seq#61] +Condition : (((isnotnull(d_month_seq#61) AND (d_month_seq#61 >= 1212)) AND (d_month_seq#61 <= 1223)) AND isnotnull(d_date_sk#5)) + +(59) CometProject +Input [2]: [d_date_sk#5, d_month_seq#61] +Arguments: [d_date_sk#5], [d_date_sk#5] + +(60) ColumnarToRow [codegen id : 1] +Input [1]: [d_date_sk#5] + +(61) BroadcastExchange +Input [1]: [d_date_sk#5] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=10] + +Subquery:2 Hosting operator id = 10 Hosting Expression = ss_sold_date_sk#11 IN dynamicpruning#4 + + diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q70a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q70a/simplified.txt new file mode 100644 index 000000000..2fd04badf --- /dev/null +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q70a/simplified.txt @@ -0,0 +1,100 @@ +TakeOrderedAndProject [lochierarchy,s_state,rank_within_parent,total_sum,s_county] + WholeStageCodegen (33) + Project [total_sum,s_state,s_county,lochierarchy,rank_within_parent] + InputAdapter + Window [total_sum,lochierarchy,_w0] + WholeStageCodegen (32) + Sort [lochierarchy,_w0,total_sum] + InputAdapter + Exchange [lochierarchy,_w0] #1 + WholeStageCodegen (31) + HashAggregate [total_sum,s_state,s_county,g_state,g_county,lochierarchy] [_w0] + InputAdapter + Exchange [total_sum,s_state,s_county,g_state,g_county,lochierarchy] #2 + WholeStageCodegen (30) + 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,g_state,g_county,lochierarchy,sum] + InputAdapter + Exchange [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] + ColumnarToRow + InputAdapter + CometFilter [ss_store_sk] + CometScan parquet spark_catalog.default.store_sales [ss_store_sk,ss_net_profit,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #4 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_month_seq,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] + InputAdapter + ReusedExchange [d_date_sk] #4 + InputAdapter + BroadcastExchange #5 + WholeStageCodegen (7) + BroadcastHashJoin [s_state,s_state] + ColumnarToRow + InputAdapter + CometFilter [s_store_sk] + CometScan parquet spark_catalog.default.store [s_store_sk,s_county,s_state] + InputAdapter + BroadcastExchange #6 + WholeStageCodegen (6) + 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] + InputAdapter + Exchange [s_state] #7 + WholeStageCodegen (4) + HashAggregate [s_state,ss_net_profit] [sum,sum] + Project [ss_net_profit,s_state] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [ss_net_profit,ss_sold_date_sk,s_state] + BroadcastHashJoin [ss_store_sk,s_store_sk] + ColumnarToRow + InputAdapter + CometFilter [ss_store_sk] + CometScan parquet spark_catalog.default.store_sales [ss_store_sk,ss_net_profit,ss_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + InputAdapter + BroadcastExchange #8 + WholeStageCodegen (2) + ColumnarToRow + InputAdapter + CometFilter [s_store_sk] + CometScan parquet spark_catalog.default.store [s_store_sk,s_state] + InputAdapter + ReusedExchange [d_date_sk] #4 + WholeStageCodegen (19) + HashAggregate [s_state,sum,isEmpty] [sum(total_sum),total_sum,s_county,g_state,g_county,lochierarchy,sum,isEmpty] + InputAdapter + Exchange [s_state] #9 + WholeStageCodegen (18) + HashAggregate [s_state,total_sum] [sum,isEmpty,sum,isEmpty] + HashAggregate [s_state,s_county,sum] [sum(UnscaledValue(ss_net_profit)),total_sum,sum] + InputAdapter + ReusedExchange [s_state,s_county,sum] #3 + WholeStageCodegen (29) + HashAggregate [sum,isEmpty] [sum(total_sum),total_sum,s_state,s_county,g_state,g_county,lochierarchy,sum,isEmpty] + InputAdapter + Exchange #10 + WholeStageCodegen (28) + HashAggregate [total_sum] [sum,isEmpty,sum,isEmpty] + HashAggregate [s_state,s_county,sum] [sum(UnscaledValue(ss_net_profit)),total_sum,sum] + InputAdapter + ReusedExchange [s_state,s_county,sum] #3 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 new file mode 100644 index 000000000..c88573838 --- /dev/null +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q72/explain.txt @@ -0,0 +1,433 @@ +== Physical Plan == +TakeOrderedAndProject (70) ++- * HashAggregate (69) + +- Exchange (68) + +- * HashAggregate (67) + +- * Project (66) + +- * SortMergeJoin LeftOuter (65) + :- * Sort (58) + : +- Exchange (57) + : +- * Project (56) + : +- * BroadcastHashJoin LeftOuter BuildRight (55) + : :- * Project (50) + : : +- * BroadcastHashJoin Inner BuildRight (49) + : : :- * Project (44) + : : : +- * BroadcastHashJoin Inner BuildRight (43) + : : : :- * Project (38) + : : : : +- * BroadcastHashJoin Inner BuildRight (37) + : : : : :- * Project (35) + : : : : : +- * BroadcastHashJoin Inner BuildRight (34) + : : : : : :- * Project (28) + : : : : : : +- * BroadcastHashJoin Inner BuildRight (27) + : : : : : : :- * Project (21) + : : : : : : : +- * BroadcastHashJoin Inner BuildRight (20) + : : : : : : : :- * Project (15) + : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (14) + : : : : : : : : :- * Project (9) + : : : : : : : : : +- * BroadcastHashJoin Inner BuildRight (8) + : : : : : : : : : :- * ColumnarToRow (3) + : : : : : : : : : : +- CometFilter (2) + : : : : : : : : : : +- CometScan parquet spark_catalog.default.catalog_sales (1) + : : : : : : : : : +- BroadcastExchange (7) + : : : : : : : : : +- * ColumnarToRow (6) + : : : : : : : : : +- CometFilter (5) + : : : : : : : : : +- CometScan parquet spark_catalog.default.inventory (4) + : : : : : : : : +- BroadcastExchange (13) + : : : : : : : : +- * ColumnarToRow (12) + : : : : : : : : +- CometFilter (11) + : : : : : : : : +- CometScan parquet spark_catalog.default.warehouse (10) + : : : : : : : +- BroadcastExchange (19) + : : : : : : : +- * ColumnarToRow (18) + : : : : : : : +- CometFilter (17) + : : : : : : : +- CometScan parquet spark_catalog.default.item (16) + : : : : : : +- BroadcastExchange (26) + : : : : : : +- * ColumnarToRow (25) + : : : : : : +- CometProject (24) + : : : : : : +- CometFilter (23) + : : : : : : +- CometScan parquet spark_catalog.default.customer_demographics (22) + : : : : : +- BroadcastExchange (33) + : : : : : +- * ColumnarToRow (32) + : : : : : +- CometProject (31) + : : : : : +- CometFilter (30) + : : : : : +- CometScan parquet spark_catalog.default.household_demographics (29) + : : : : +- ReusedExchange (36) + : : : +- BroadcastExchange (42) + : : : +- * ColumnarToRow (41) + : : : +- CometFilter (40) + : : : +- CometScan parquet spark_catalog.default.date_dim (39) + : : +- BroadcastExchange (48) + : : +- * ColumnarToRow (47) + : : +- CometFilter (46) + : : +- CometScan parquet spark_catalog.default.date_dim (45) + : +- BroadcastExchange (54) + : +- * ColumnarToRow (53) + : +- CometFilter (52) + : +- CometScan parquet spark_catalog.default.promotion (51) + +- * Sort (64) + +- Exchange (63) + +- * ColumnarToRow (62) + +- CometProject (61) + +- CometFilter (60) + +- CometScan parquet spark_catalog.default.catalog_returns (59) + + +(1) Scan parquet spark_catalog.default.catalog_sales +Output [8]: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_quantity#7, cs_sold_date_sk#8] +Batched: true +Location: InMemoryFileIndex [] +PartitionFilters: [isnotnull(cs_sold_date_sk#8), dynamicpruningexpression(cs_sold_date_sk#8 IN dynamicpruning#9)] +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<cs_ship_date_sk:int,cs_bill_cdemo_sk:int,cs_bill_hdemo_sk:int,cs_item_sk:int,cs_promo_sk:int,cs_order_number:int,cs_quantity:int> + +(2) CometFilter +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)) + +(3) ColumnarToRow [codegen id : 10] +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] + +(4) Scan parquet spark_catalog.default.inventory +Output [4]: [inv_item_sk#10, inv_warehouse_sk#11, inv_quantity_on_hand#12, inv_date_sk#13] +Batched: true +Location: InMemoryFileIndex [] +PartitionFilters: [isnotnull(inv_date_sk#13)] +PushedFilters: [IsNotNull(inv_quantity_on_hand), IsNotNull(inv_item_sk), IsNotNull(inv_warehouse_sk)] +ReadSchema: struct<inv_item_sk:int,inv_warehouse_sk:int,inv_quantity_on_hand:int> + +(5) CometFilter +Input [4]: [inv_item_sk#10, inv_warehouse_sk#11, inv_quantity_on_hand#12, inv_date_sk#13] +Condition : ((isnotnull(inv_quantity_on_hand#12) AND isnotnull(inv_item_sk#10)) AND isnotnull(inv_warehouse_sk#11)) + +(6) ColumnarToRow [codegen id : 1] +Input [4]: [inv_item_sk#10, inv_warehouse_sk#11, inv_quantity_on_hand#12, inv_date_sk#13] + +(7) BroadcastExchange +Input [4]: [inv_item_sk#10, inv_warehouse_sk#11, inv_quantity_on_hand#12, inv_date_sk#13] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=1] + +(8) BroadcastHashJoin [codegen id : 10] +Left keys [1]: [cs_item_sk#4] +Right keys [1]: [inv_item_sk#10] +Join type: Inner +Join condition: (inv_quantity_on_hand#12 < cs_quantity#7) + +(9) Project [codegen id : 10] +Output [9]: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_warehouse_sk#11, inv_date_sk#13] +Input [12]: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_quantity#7, cs_sold_date_sk#8, inv_item_sk#10, inv_warehouse_sk#11, inv_quantity_on_hand#12, inv_date_sk#13] + +(10) Scan parquet spark_catalog.default.warehouse +Output [2]: [w_warehouse_sk#14, w_warehouse_name#15] +Batched: true +Location [not included in comparison]/{warehouse_dir}/warehouse] +PushedFilters: [IsNotNull(w_warehouse_sk)] +ReadSchema: struct<w_warehouse_sk:int,w_warehouse_name:string> + +(11) CometFilter +Input [2]: [w_warehouse_sk#14, w_warehouse_name#15] +Condition : isnotnull(w_warehouse_sk#14) + +(12) ColumnarToRow [codegen id : 2] +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] +Left keys [1]: [inv_warehouse_sk#11] +Right keys [1]: [w_warehouse_sk#14] +Join type: Inner +Join condition: None + +(15) Project [codegen id : 10] +Output [9]: [cs_ship_date_sk#1, cs_bill_cdemo_sk#2, cs_bill_hdemo_sk#3, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15] +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] + +(16) Scan parquet spark_catalog.default.item +Output [2]: [i_item_sk#16, i_item_desc#17] +Batched: true +Location [not included in comparison]/{warehouse_dir}/item] +PushedFilters: [IsNotNull(i_item_sk)] +ReadSchema: struct<i_item_sk:int,i_item_desc:string> + +(17) CometFilter +Input [2]: [i_item_sk#16, i_item_desc#17] +Condition : isnotnull(i_item_sk#16) + +(18) ColumnarToRow [codegen id : 3] +Input [2]: [i_item_sk#16, i_item_desc#17] + +(19) BroadcastExchange +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] +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] +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] + +(22) Scan parquet spark_catalog.default.customer_demographics +Output [2]: [cd_demo_sk#18, cd_marital_status#19] +Batched: true +Location [not included in comparison]/{warehouse_dir}/customer_demographics] +PushedFilters: [IsNotNull(cd_marital_status), EqualTo(cd_marital_status,M), IsNotNull(cd_demo_sk)] +ReadSchema: struct<cd_demo_sk:int,cd_marital_status:string> + +(23) CometFilter +Input [2]: [cd_demo_sk#18, cd_marital_status#19] +Condition : ((isnotnull(cd_marital_status#19) AND (cd_marital_status#19 = M)) AND isnotnull(cd_demo_sk#18)) + +(24) CometProject +Input [2]: [cd_demo_sk#18, cd_marital_status#19] +Arguments: [cd_demo_sk#18], [cd_demo_sk#18] + +(25) ColumnarToRow [codegen id : 4] +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] +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] +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] + +(29) Scan parquet spark_catalog.default.household_demographics +Output [2]: [hd_demo_sk#20, hd_buy_potential#21] +Batched: true +Location [not included in comparison]/{warehouse_dir}/household_demographics] +PushedFilters: [IsNotNull(hd_buy_potential), EqualTo(hd_buy_potential,1001-5000 ), IsNotNull(hd_demo_sk)] +ReadSchema: struct<hd_demo_sk:int,hd_buy_potential:string> + +(30) CometFilter +Input [2]: [hd_demo_sk#20, hd_buy_potential#21] +Condition : ((isnotnull(hd_buy_potential#21) AND (hd_buy_potential#21 = 1001-5000 )) AND isnotnull(hd_demo_sk#20)) + +(31) CometProject +Input [2]: [hd_demo_sk#20, hd_buy_potential#21] +Arguments: [hd_demo_sk#20], [hd_demo_sk#20] + +(32) ColumnarToRow [codegen id : 5] +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] +Left keys [1]: [cs_bill_hdemo_sk#3] +Right keys [1]: [hd_demo_sk#20] +Join type: Inner +Join condition: None + +(35) Project [codegen id : 10] +Output [8]: [cs_ship_date_sk#1, cs_item_sk#4, cs_promo_sk#5, cs_order_number#6, cs_sold_date_sk#8, inv_date_sk#13, w_warehouse_name#15, i_item_desc#17] +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] +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] +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] + +(39) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#25, d_week_seq#26] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_week_seq), IsNotNull(d_date_sk)] +ReadSchema: struct<d_date_sk:int,d_week_seq:int> + +(40) CometFilter +Input [2]: [d_date_sk#25, d_week_seq#26] +Condition : (isnotnull(d_week_seq#26) AND isnotnull(d_date_sk#25)) + +(41) ColumnarToRow [codegen id : 7] +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] +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] +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] + +(45) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#27, d_date#28] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_date), IsNotNull(d_date_sk)] +ReadSchema: struct<d_date_sk:int,d_date:date> + +(46) CometFilter +Input [2]: [d_date_sk#27, d_date#28] +Condition : (isnotnull(d_date#28) AND isnotnull(d_date_sk#27)) + +(47) ColumnarToRow [codegen id : 8] +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] +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] +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] + +(51) Scan parquet spark_catalog.default.promotion +Output [1]: [p_promo_sk#29] +Batched: true +Location [not included in comparison]/{warehouse_dir}/promotion] +PushedFilters: [IsNotNull(p_promo_sk)] +ReadSchema: struct<p_promo_sk:int> + +(52) CometFilter +Input [1]: [p_promo_sk#29] +Condition : isnotnull(p_promo_sk#29) + +(53) ColumnarToRow [codegen id : 9] +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] +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] +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] + +(57) Exchange +Input [5]: [cs_item_sk#4, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_week_seq#24] +Arguments: hashpartitioning(cs_item_sk#4, cs_order_number#6, 5), ENSURE_REQUIREMENTS, [plan_id=9] + +(58) Sort [codegen id : 11] +Input [5]: [cs_item_sk#4, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_week_seq#24] +Arguments: [cs_item_sk#4 ASC NULLS FIRST, cs_order_number#6 ASC NULLS FIRST], false, 0 + +(59) Scan parquet spark_catalog.default.catalog_returns +Output [3]: [cr_item_sk#30, cr_order_number#31, cr_returned_date_sk#32] +Batched: true +Location [not included in comparison]/{warehouse_dir}/catalog_returns] +PushedFilters: [IsNotNull(cr_item_sk), IsNotNull(cr_order_number)] +ReadSchema: struct<cr_item_sk:int,cr_order_number:int> + +(60) CometFilter +Input [3]: [cr_item_sk#30, cr_order_number#31, cr_returned_date_sk#32] +Condition : (isnotnull(cr_item_sk#30) AND isnotnull(cr_order_number#31)) + +(61) CometProject +Input [3]: [cr_item_sk#30, cr_order_number#31, cr_returned_date_sk#32] +Arguments: [cr_item_sk#30, cr_order_number#31], [cr_item_sk#30, cr_order_number#31] + +(62) ColumnarToRow [codegen id : 12] +Input [2]: [cr_item_sk#30, cr_order_number#31] + +(63) Exchange +Input [2]: [cr_item_sk#30, cr_order_number#31] +Arguments: hashpartitioning(cr_item_sk#30, cr_order_number#31, 5), ENSURE_REQUIREMENTS, [plan_id=10] + +(64) Sort [codegen id : 13] +Input [2]: [cr_item_sk#30, cr_order_number#31] +Arguments: [cr_item_sk#30 ASC NULLS FIRST, cr_order_number#31 ASC NULLS FIRST], false, 0 + +(65) SortMergeJoin [codegen id : 14] +Left keys [2]: [cs_item_sk#4, cs_order_number#6] +Right keys [2]: [cr_item_sk#30, cr_order_number#31] +Join type: LeftOuter +Join condition: None + +(66) Project [codegen id : 14] +Output [3]: [w_warehouse_name#15, i_item_desc#17, d_week_seq#24] +Input [7]: [cs_item_sk#4, cs_order_number#6, w_warehouse_name#15, i_item_desc#17, d_week_seq#24, cr_item_sk#30, cr_order_number#31] + +(67) HashAggregate [codegen id : 14] +Input [3]: [w_warehouse_name#15, i_item_desc#17, d_week_seq#24] +Keys [3]: [i_item_desc#17, w_warehouse_name#15, d_week_seq#24] +Functions [1]: [partial_count(1)] +Aggregate Attributes [1]: [count#33] +Results [4]: [i_item_desc#17, w_warehouse_name#15, d_week_seq#24, count#34] + +(68) Exchange +Input [4]: [i_item_desc#17, w_warehouse_name#15, d_week_seq#24, count#34] +Arguments: hashpartitioning(i_item_desc#17, w_warehouse_name#15, d_week_seq#24, 5), ENSURE_REQUIREMENTS, [plan_id=11] + +(69) HashAggregate [codegen id : 15] +Input [4]: [i_item_desc#17, w_warehouse_name#15, d_week_seq#24, count#34] +Keys [3]: [i_item_desc#17, w_warehouse_name#15, d_week_seq#24] +Functions [1]: [count(1)] +Aggregate Attributes [1]: [count(1)#35] +Results [6]: [i_item_desc#17, w_warehouse_name#15, d_week_seq#24, count(1)#35 AS no_promo#36, count(1)#35 AS promo#37, count(1)#35 AS total_cnt#38] + +(70) TakeOrderedAndProject +Input [6]: [i_item_desc#17, w_warehouse_name#15, d_week_seq#24, no_promo#36, promo#37, total_cnt#38] +Arguments: 100, [total_cnt#38 DESC NULLS LAST, i_item_desc#17 ASC NULLS FIRST, w_warehouse_name#15 ASC NULLS FIRST, d_week_seq#24 ASC NULLS FIRST], [i_item_desc#17, w_warehouse_name#15, d_week_seq#24, no_promo#36, promo#37, total_cnt#38] + +===== Subqueries ===== + +Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#8 IN dynamicpruning#9 +BroadcastExchange (75) ++- * ColumnarToRow (74) + +- CometProject (73) + +- CometFilter (72) + +- CometScan parquet spark_catalog.default.date_dim (71) + + +(71) Scan parquet spark_catalog.default.date_dim +Output [4]: [d_date_sk#22, d_date#23, d_week_seq#24, d_year#39] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), IsNotNull(d_date_sk), IsNotNull(d_week_seq), IsNotNull(d_date)] +ReadSchema: struct<d_date_sk:int,d_date:date,d_week_seq:int,d_year:int> + +(72) CometFilter +Input [4]: [d_date_sk#22, d_date#23, d_week_seq#24, d_year#39] +Condition : ((((isnotnull(d_year#39) AND (d_year#39 = 2001)) AND isnotnull(d_date_sk#22)) AND isnotnull(d_week_seq#24)) AND isnotnull(d_date#23)) + +(73) CometProject +Input [4]: [d_date_sk#22, d_date#23, d_week_seq#24, d_year#39] +Arguments: [d_date_sk#22, d_date#23, d_week_seq#24], [d_date_sk#22, d_date#23, d_week_seq#24] + +(74) ColumnarToRow [codegen id : 1] +Input [3]: [d_date_sk#22, d_date#23, d_week_seq#24] + +(75) BroadcastExchange +Input [3]: [d_date_sk#22, d_date#23, d_week_seq#24] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=12] + + 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 new file mode 100644 index 000000000..17fc9dee7 --- /dev/null +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q72/simplified.txt @@ -0,0 +1,116 @@ +TakeOrderedAndProject [total_cnt,i_item_desc,w_warehouse_name,d_week_seq,no_promo,promo] + WholeStageCodegen (15) + HashAggregate [i_item_desc,w_warehouse_name,d_week_seq,count] [count(1),no_promo,promo,total_cnt,count] + InputAdapter + Exchange [i_item_desc,w_warehouse_name,d_week_seq] #1 + WholeStageCodegen (14) + HashAggregate [i_item_desc,w_warehouse_name,d_week_seq] [count,count] + Project [w_warehouse_name,i_item_desc,d_week_seq] + SortMergeJoin [cs_item_sk,cs_order_number,cr_item_sk,cr_order_number] + InputAdapter + WholeStageCodegen (11) + Sort [cs_item_sk,cs_order_number] + InputAdapter + Exchange [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] + ColumnarToRow + InputAdapter + CometFilter [cs_quantity,cs_item_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_ship_date_sk] + CometScan parquet spark_catalog.default.catalog_sales [cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_quantity,cs_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk,d_date,d_week_seq] + CometFilter [d_year,d_date_sk,d_week_seq,d_date] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_week_seq,d_year] + InputAdapter + BroadcastExchange #4 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometFilter [inv_quantity_on_hand,inv_item_sk,inv_warehouse_sk] + CometScan parquet spark_catalog.default.inventory [inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand,inv_date_sk] + InputAdapter + BroadcastExchange #5 + WholeStageCodegen (2) + ColumnarToRow + InputAdapter + CometFilter [w_warehouse_sk] + CometScan parquet spark_catalog.default.warehouse [w_warehouse_sk,w_warehouse_name] + InputAdapter + BroadcastExchange #6 + WholeStageCodegen (3) + ColumnarToRow + InputAdapter + CometFilter [i_item_sk] + CometScan parquet spark_catalog.default.item [i_item_sk,i_item_desc] + InputAdapter + BroadcastExchange #7 + WholeStageCodegen (4) + ColumnarToRow + InputAdapter + CometProject [cd_demo_sk] + CometFilter [cd_marital_status,cd_demo_sk] + CometScan parquet spark_catalog.default.customer_demographics [cd_demo_sk,cd_marital_status] + InputAdapter + BroadcastExchange #8 + WholeStageCodegen (5) + ColumnarToRow + InputAdapter + CometProject [hd_demo_sk] + CometFilter [hd_buy_potential,hd_demo_sk] + CometScan parquet spark_catalog.default.household_demographics [hd_demo_sk,hd_buy_potential] + InputAdapter + ReusedExchange [d_date_sk,d_date,d_week_seq] #3 + InputAdapter + BroadcastExchange #9 + WholeStageCodegen (7) + ColumnarToRow + InputAdapter + CometFilter [d_week_seq,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_week_seq] + InputAdapter + BroadcastExchange #10 + WholeStageCodegen (8) + ColumnarToRow + InputAdapter + CometFilter [d_date,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] + InputAdapter + BroadcastExchange #11 + WholeStageCodegen (9) + ColumnarToRow + InputAdapter + CometFilter [p_promo_sk] + CometScan parquet spark_catalog.default.promotion [p_promo_sk] + InputAdapter + WholeStageCodegen (13) + Sort [cr_item_sk,cr_order_number] + InputAdapter + Exchange [cr_item_sk,cr_order_number] #12 + WholeStageCodegen (12) + ColumnarToRow + InputAdapter + CometProject [cr_item_sk,cr_order_number] + CometFilter [cr_item_sk,cr_order_number] + CometScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_returned_date_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q74/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q74/explain.txt new file mode 100644 index 000000000..ad8c33ca1 --- /dev/null +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q74/explain.txt @@ -0,0 +1,477 @@ +== Physical Plan == +TakeOrderedAndProject (71) ++- * Project (70) + +- * BroadcastHashJoin Inner BuildRight (69) + :- * Project (52) + : +- * BroadcastHashJoin Inner BuildRight (51) + : :- * BroadcastHashJoin Inner BuildRight (33) + : : :- * Filter (16) + : : : +- * HashAggregate (15) + : : : +- Exchange (14) + : : : +- * HashAggregate (13) + : : : +- * Project (12) + : : : +- * BroadcastHashJoin Inner BuildRight (11) + : : : :- * Project (9) + : : : : +- * BroadcastHashJoin Inner BuildRight (8) + : : : : :- * ColumnarToRow (3) + : : : : : +- CometFilter (2) + : : : : : +- CometScan parquet spark_catalog.default.customer (1) + : : : : +- BroadcastExchange (7) + : : : : +- * ColumnarToRow (6) + : : : : +- CometFilter (5) + : : : : +- CometScan parquet spark_catalog.default.store_sales (4) + : : : +- ReusedExchange (10) + : : +- BroadcastExchange (32) + : : +- * HashAggregate (31) + : : +- Exchange (30) + : : +- * HashAggregate (29) + : : +- * Project (28) + : : +- * BroadcastHashJoin Inner BuildRight (27) + : : :- * Project (25) + : : : +- * BroadcastHashJoin Inner BuildRight (24) + : : : :- * ColumnarToRow (19) + : : : : +- CometFilter (18) + : : : : +- CometScan parquet spark_catalog.default.customer (17) + : : : +- BroadcastExchange (23) + : : : +- * ColumnarToRow (22) + : : : +- CometFilter (21) + : : : +- CometScan parquet spark_catalog.default.store_sales (20) + : : +- ReusedExchange (26) + : +- BroadcastExchange (50) + : +- * Filter (49) + : +- * HashAggregate (48) + : +- Exchange (47) + : +- * HashAggregate (46) + : +- * Project (45) + : +- * BroadcastHashJoin Inner BuildRight (44) + : :- * Project (42) + : : +- * BroadcastHashJoin Inner BuildRight (41) + : : :- * ColumnarToRow (36) + : : : +- CometFilter (35) + : : : +- CometScan parquet spark_catalog.default.customer (34) + : : +- BroadcastExchange (40) + : : +- * ColumnarToRow (39) + : : +- CometFilter (38) + : : +- CometScan parquet spark_catalog.default.web_sales (37) + : +- ReusedExchange (43) + +- BroadcastExchange (68) + +- * HashAggregate (67) + +- Exchange (66) + +- * HashAggregate (65) + +- * Project (64) + +- * BroadcastHashJoin Inner BuildRight (63) + :- * Project (61) + : +- * BroadcastHashJoin Inner BuildRight (60) + : :- * ColumnarToRow (55) + : : +- CometFilter (54) + : : +- CometScan parquet spark_catalog.default.customer (53) + : +- BroadcastExchange (59) + : +- * ColumnarToRow (58) + : +- CometFilter (57) + : +- CometScan parquet spark_catalog.default.web_sales (56) + +- ReusedExchange (62) + + +(1) Scan parquet spark_catalog.default.customer +Output [4]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4] +Batched: true +Location [not included in comparison]/{warehouse_dir}/customer] +PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_customer_id)] +ReadSchema: struct<c_customer_sk:int,c_customer_id:string,c_first_name:string,c_last_name:string> + +(2) CometFilter +Input [4]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4] +Condition : (isnotnull(c_customer_sk#1) AND isnotnull(c_customer_id#2)) + +(3) ColumnarToRow [codegen id : 3] +Input [4]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4] + +(4) Scan parquet spark_catalog.default.store_sales +Output [3]: [ss_customer_sk#5, ss_net_paid#6, ss_sold_date_sk#7] +Batched: true +Location: InMemoryFileIndex [] +PartitionFilters: [isnotnull(ss_sold_date_sk#7), dynamicpruningexpression(ss_sold_date_sk#7 IN dynamicpruning#8)] +PushedFilters: [IsNotNull(ss_customer_sk)] +ReadSchema: struct<ss_customer_sk:int,ss_net_paid:decimal(7,2)> + +(5) CometFilter +Input [3]: [ss_customer_sk#5, ss_net_paid#6, ss_sold_date_sk#7] +Condition : isnotnull(ss_customer_sk#5) + +(6) ColumnarToRow [codegen id : 1] +Input [3]: [ss_customer_sk#5, ss_net_paid#6, ss_sold_date_sk#7] + +(7) BroadcastExchange +Input [3]: [ss_customer_sk#5, ss_net_paid#6, ss_sold_date_sk#7] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=1] + +(8) BroadcastHashJoin [codegen id : 3] +Left keys [1]: [c_customer_sk#1] +Right keys [1]: [ss_customer_sk#5] +Join type: Inner +Join condition: None + +(9) Project [codegen id : 3] +Output [5]: [c_customer_id#2, c_first_name#3, c_last_name#4, ss_net_paid#6, ss_sold_date_sk#7] +Input [7]: [c_customer_sk#1, c_customer_id#2, c_first_name#3, c_last_name#4, ss_customer_sk#5, ss_net_paid#6, ss_sold_date_sk#7] + +(10) ReusedExchange [Reuses operator id: 75] +Output [2]: [d_date_sk#9, d_year#10] + +(11) BroadcastHashJoin [codegen id : 3] +Left keys [1]: [ss_sold_date_sk#7] +Right keys [1]: [d_date_sk#9] +Join type: Inner +Join condition: None + +(12) Project [codegen id : 3] +Output [5]: [c_customer_id#2, c_first_name#3, c_last_name#4, ss_net_paid#6, d_year#10] +Input [7]: [c_customer_id#2, c_first_name#3, c_last_name#4, ss_net_paid#6, ss_sold_date_sk#7, d_date_sk#9, d_year#10] + +(13) HashAggregate [codegen id : 3] +Input [5]: [c_customer_id#2, c_first_name#3, c_last_name#4, ss_net_paid#6, d_year#10] +Keys [4]: [c_customer_id#2, c_first_name#3, c_last_name#4, d_year#10] +Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#6))] +Aggregate Attributes [1]: [sum#11] +Results [5]: [c_customer_id#2, c_first_name#3, c_last_name#4, d_year#10, sum#12] + +(14) Exchange +Input [5]: [c_customer_id#2, c_first_name#3, c_last_name#4, d_year#10, sum#12] +Arguments: hashpartitioning(c_customer_id#2, c_first_name#3, c_last_name#4, d_year#10, 5), ENSURE_REQUIREMENTS, [plan_id=2] + +(15) HashAggregate [codegen id : 16] +Input [5]: [c_customer_id#2, c_first_name#3, c_last_name#4, d_year#10, sum#12] +Keys [4]: [c_customer_id#2, c_first_name#3, c_last_name#4, d_year#10] +Functions [1]: [sum(UnscaledValue(ss_net_paid#6))] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_paid#6))#13] +Results [2]: [c_customer_id#2 AS customer_id#14, MakeDecimal(sum(UnscaledValue(ss_net_paid#6))#13,17,2) AS year_total#15] + +(16) Filter [codegen id : 16] +Input [2]: [customer_id#14, year_total#15] +Condition : (isnotnull(year_total#15) AND (year_total#15 > 0.00)) + +(17) Scan parquet spark_catalog.default.customer +Output [4]: [c_customer_sk#16, c_customer_id#17, c_first_name#18, c_last_name#19] +Batched: true +Location [not included in comparison]/{warehouse_dir}/customer] +PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_customer_id)] +ReadSchema: struct<c_customer_sk:int,c_customer_id:string,c_first_name:string,c_last_name:string> + +(18) CometFilter +Input [4]: [c_customer_sk#16, c_customer_id#17, c_first_name#18, c_last_name#19] +Condition : (isnotnull(c_customer_sk#16) AND isnotnull(c_customer_id#17)) + +(19) ColumnarToRow [codegen id : 6] +Input [4]: [c_customer_sk#16, c_customer_id#17, c_first_name#18, c_last_name#19] + +(20) Scan parquet spark_catalog.default.store_sales +Output [3]: [ss_customer_sk#20, ss_net_paid#21, ss_sold_date_sk#22] +Batched: true +Location: InMemoryFileIndex [] +PartitionFilters: [isnotnull(ss_sold_date_sk#22), dynamicpruningexpression(ss_sold_date_sk#22 IN dynamicpruning#23)] +PushedFilters: [IsNotNull(ss_customer_sk)] +ReadSchema: struct<ss_customer_sk:int,ss_net_paid:decimal(7,2)> + +(21) CometFilter +Input [3]: [ss_customer_sk#20, ss_net_paid#21, ss_sold_date_sk#22] +Condition : isnotnull(ss_customer_sk#20) + +(22) ColumnarToRow [codegen id : 4] +Input [3]: [ss_customer_sk#20, ss_net_paid#21, ss_sold_date_sk#22] + +(23) BroadcastExchange +Input [3]: [ss_customer_sk#20, ss_net_paid#21, ss_sold_date_sk#22] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=3] + +(24) BroadcastHashJoin [codegen id : 6] +Left keys [1]: [c_customer_sk#16] +Right keys [1]: [ss_customer_sk#20] +Join type: Inner +Join condition: None + +(25) Project [codegen id : 6] +Output [5]: [c_customer_id#17, c_first_name#18, c_last_name#19, ss_net_paid#21, ss_sold_date_sk#22] +Input [7]: [c_customer_sk#16, c_customer_id#17, c_first_name#18, c_last_name#19, ss_customer_sk#20, ss_net_paid#21, ss_sold_date_sk#22] + +(26) ReusedExchange [Reuses operator id: 79] +Output [2]: [d_date_sk#24, d_year#25] + +(27) BroadcastHashJoin [codegen id : 6] +Left keys [1]: [ss_sold_date_sk#22] +Right keys [1]: [d_date_sk#24] +Join type: Inner +Join condition: None + +(28) Project [codegen id : 6] +Output [5]: [c_customer_id#17, c_first_name#18, c_last_name#19, ss_net_paid#21, d_year#25] +Input [7]: [c_customer_id#17, c_first_name#18, c_last_name#19, ss_net_paid#21, ss_sold_date_sk#22, d_date_sk#24, d_year#25] + +(29) HashAggregate [codegen id : 6] +Input [5]: [c_customer_id#17, c_first_name#18, c_last_name#19, ss_net_paid#21, d_year#25] +Keys [4]: [c_customer_id#17, c_first_name#18, c_last_name#19, d_year#25] +Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#21))] +Aggregate Attributes [1]: [sum#26] +Results [5]: [c_customer_id#17, c_first_name#18, c_last_name#19, d_year#25, sum#27] + +(30) Exchange +Input [5]: [c_customer_id#17, c_first_name#18, c_last_name#19, d_year#25, sum#27] +Arguments: hashpartitioning(c_customer_id#17, c_first_name#18, c_last_name#19, d_year#25, 5), ENSURE_REQUIREMENTS, [plan_id=4] + +(31) HashAggregate [codegen id : 7] +Input [5]: [c_customer_id#17, c_first_name#18, c_last_name#19, d_year#25, sum#27] +Keys [4]: [c_customer_id#17, c_first_name#18, c_last_name#19, d_year#25] +Functions [1]: [sum(UnscaledValue(ss_net_paid#21))] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_paid#21))#13] +Results [4]: [c_customer_id#17 AS customer_id#28, c_first_name#18 AS customer_first_name#29, c_last_name#19 AS customer_last_name#30, MakeDecimal(sum(UnscaledValue(ss_net_paid#21))#13,17,2) AS year_total#31] + +(32) BroadcastExchange +Input [4]: [customer_id#28, customer_first_name#29, customer_last_name#30, year_total#31] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=5] + +(33) BroadcastHashJoin [codegen id : 16] +Left keys [1]: [customer_id#14] +Right keys [1]: [customer_id#28] +Join type: Inner +Join condition: None + +(34) Scan parquet spark_catalog.default.customer +Output [4]: [c_customer_sk#32, c_customer_id#33, c_first_name#34, c_last_name#35] +Batched: true +Location [not included in comparison]/{warehouse_dir}/customer] +PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_customer_id)] +ReadSchema: struct<c_customer_sk:int,c_customer_id:string,c_first_name:string,c_last_name:string> + +(35) CometFilter +Input [4]: [c_customer_sk#32, c_customer_id#33, c_first_name#34, c_last_name#35] +Condition : (isnotnull(c_customer_sk#32) AND isnotnull(c_customer_id#33)) + +(36) ColumnarToRow [codegen id : 10] +Input [4]: [c_customer_sk#32, c_customer_id#33, c_first_name#34, c_last_name#35] + +(37) Scan parquet spark_catalog.default.web_sales +Output [3]: [ws_bill_customer_sk#36, ws_net_paid#37, ws_sold_date_sk#38] +Batched: true +Location: InMemoryFileIndex [] +PartitionFilters: [isnotnull(ws_sold_date_sk#38), dynamicpruningexpression(ws_sold_date_sk#38 IN dynamicpruning#39)] +PushedFilters: [IsNotNull(ws_bill_customer_sk)] +ReadSchema: struct<ws_bill_customer_sk:int,ws_net_paid:decimal(7,2)> + +(38) CometFilter +Input [3]: [ws_bill_customer_sk#36, ws_net_paid#37, ws_sold_date_sk#38] +Condition : isnotnull(ws_bill_customer_sk#36) + +(39) ColumnarToRow [codegen id : 8] +Input [3]: [ws_bill_customer_sk#36, ws_net_paid#37, ws_sold_date_sk#38] + +(40) BroadcastExchange +Input [3]: [ws_bill_customer_sk#36, ws_net_paid#37, ws_sold_date_sk#38] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=6] + +(41) BroadcastHashJoin [codegen id : 10] +Left keys [1]: [c_customer_sk#32] +Right keys [1]: [ws_bill_customer_sk#36] +Join type: Inner +Join condition: None + +(42) Project [codegen id : 10] +Output [5]: [c_customer_id#33, c_first_name#34, c_last_name#35, ws_net_paid#37, ws_sold_date_sk#38] +Input [7]: [c_customer_sk#32, c_customer_id#33, c_first_name#34, c_last_name#35, ws_bill_customer_sk#36, ws_net_paid#37, ws_sold_date_sk#38] + +(43) ReusedExchange [Reuses operator id: 75] +Output [2]: [d_date_sk#40, d_year#41] + +(44) BroadcastHashJoin [codegen id : 10] +Left keys [1]: [ws_sold_date_sk#38] +Right keys [1]: [d_date_sk#40] +Join type: Inner +Join condition: None + +(45) Project [codegen id : 10] +Output [5]: [c_customer_id#33, c_first_name#34, c_last_name#35, ws_net_paid#37, d_year#41] +Input [7]: [c_customer_id#33, c_first_name#34, c_last_name#35, ws_net_paid#37, ws_sold_date_sk#38, d_date_sk#40, d_year#41] + +(46) HashAggregate [codegen id : 10] +Input [5]: [c_customer_id#33, c_first_name#34, c_last_name#35, ws_net_paid#37, d_year#41] +Keys [4]: [c_customer_id#33, c_first_name#34, c_last_name#35, d_year#41] +Functions [1]: [partial_sum(UnscaledValue(ws_net_paid#37))] +Aggregate Attributes [1]: [sum#42] +Results [5]: [c_customer_id#33, c_first_name#34, c_last_name#35, d_year#41, sum#43] + +(47) Exchange +Input [5]: [c_customer_id#33, c_first_name#34, c_last_name#35, d_year#41, sum#43] +Arguments: hashpartitioning(c_customer_id#33, c_first_name#34, c_last_name#35, d_year#41, 5), ENSURE_REQUIREMENTS, [plan_id=7] + +(48) HashAggregate [codegen id : 11] +Input [5]: [c_customer_id#33, c_first_name#34, c_last_name#35, d_year#41, sum#43] +Keys [4]: [c_customer_id#33, c_first_name#34, c_last_name#35, d_year#41] +Functions [1]: [sum(UnscaledValue(ws_net_paid#37))] +Aggregate Attributes [1]: [sum(UnscaledValue(ws_net_paid#37))#44] +Results [2]: [c_customer_id#33 AS customer_id#45, MakeDecimal(sum(UnscaledValue(ws_net_paid#37))#44,17,2) AS year_total#46] + +(49) Filter [codegen id : 11] +Input [2]: [customer_id#45, year_total#46] +Condition : (isnotnull(year_total#46) AND (year_total#46 > 0.00)) + +(50) BroadcastExchange +Input [2]: [customer_id#45, year_total#46] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=8] + +(51) BroadcastHashJoin [codegen id : 16] +Left keys [1]: [customer_id#14] +Right keys [1]: [customer_id#45] +Join type: Inner +Join condition: None + +(52) Project [codegen id : 16] +Output [7]: [customer_id#14, year_total#15, customer_id#28, customer_first_name#29, customer_last_name#30, year_total#31, year_total#46] +Input [8]: [customer_id#14, year_total#15, customer_id#28, customer_first_name#29, customer_last_name#30, year_total#31, customer_id#45, year_total#46] + +(53) Scan parquet spark_catalog.default.customer +Output [4]: [c_customer_sk#47, c_customer_id#48, c_first_name#49, c_last_name#50] +Batched: true +Location [not included in comparison]/{warehouse_dir}/customer] +PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_customer_id)] +ReadSchema: struct<c_customer_sk:int,c_customer_id:string,c_first_name:string,c_last_name:string> + +(54) CometFilter +Input [4]: [c_customer_sk#47, c_customer_id#48, c_first_name#49, c_last_name#50] +Condition : (isnotnull(c_customer_sk#47) AND isnotnull(c_customer_id#48)) + +(55) ColumnarToRow [codegen id : 14] +Input [4]: [c_customer_sk#47, c_customer_id#48, c_first_name#49, c_last_name#50] + +(56) Scan parquet spark_catalog.default.web_sales +Output [3]: [ws_bill_customer_sk#51, ws_net_paid#52, ws_sold_date_sk#53] +Batched: true +Location: InMemoryFileIndex [] +PartitionFilters: [isnotnull(ws_sold_date_sk#53), dynamicpruningexpression(ws_sold_date_sk#53 IN dynamicpruning#54)] +PushedFilters: [IsNotNull(ws_bill_customer_sk)] +ReadSchema: struct<ws_bill_customer_sk:int,ws_net_paid:decimal(7,2)> + +(57) CometFilter +Input [3]: [ws_bill_customer_sk#51, ws_net_paid#52, ws_sold_date_sk#53] +Condition : isnotnull(ws_bill_customer_sk#51) + +(58) ColumnarToRow [codegen id : 12] +Input [3]: [ws_bill_customer_sk#51, ws_net_paid#52, ws_sold_date_sk#53] + +(59) BroadcastExchange +Input [3]: [ws_bill_customer_sk#51, ws_net_paid#52, ws_sold_date_sk#53] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=9] + +(60) BroadcastHashJoin [codegen id : 14] +Left keys [1]: [c_customer_sk#47] +Right keys [1]: [ws_bill_customer_sk#51] +Join type: Inner +Join condition: None + +(61) Project [codegen id : 14] +Output [5]: [c_customer_id#48, c_first_name#49, c_last_name#50, ws_net_paid#52, ws_sold_date_sk#53] +Input [7]: [c_customer_sk#47, c_customer_id#48, c_first_name#49, c_last_name#50, ws_bill_customer_sk#51, ws_net_paid#52, ws_sold_date_sk#53] + +(62) ReusedExchange [Reuses operator id: 79] +Output [2]: [d_date_sk#55, d_year#56] + +(63) BroadcastHashJoin [codegen id : 14] +Left keys [1]: [ws_sold_date_sk#53] +Right keys [1]: [d_date_sk#55] +Join type: Inner +Join condition: None + +(64) Project [codegen id : 14] +Output [5]: [c_customer_id#48, c_first_name#49, c_last_name#50, ws_net_paid#52, d_year#56] +Input [7]: [c_customer_id#48, c_first_name#49, c_last_name#50, ws_net_paid#52, ws_sold_date_sk#53, d_date_sk#55, d_year#56] + +(65) HashAggregate [codegen id : 14] +Input [5]: [c_customer_id#48, c_first_name#49, c_last_name#50, ws_net_paid#52, d_year#56] +Keys [4]: [c_customer_id#48, c_first_name#49, c_last_name#50, d_year#56] +Functions [1]: [partial_sum(UnscaledValue(ws_net_paid#52))] +Aggregate Attributes [1]: [sum#57] +Results [5]: [c_customer_id#48, c_first_name#49, c_last_name#50, d_year#56, sum#58] + +(66) Exchange +Input [5]: [c_customer_id#48, c_first_name#49, c_last_name#50, d_year#56, sum#58] +Arguments: hashpartitioning(c_customer_id#48, c_first_name#49, c_last_name#50, d_year#56, 5), ENSURE_REQUIREMENTS, [plan_id=10] + +(67) HashAggregate [codegen id : 15] +Input [5]: [c_customer_id#48, c_first_name#49, c_last_name#50, d_year#56, sum#58] +Keys [4]: [c_customer_id#48, c_first_name#49, c_last_name#50, d_year#56] +Functions [1]: [sum(UnscaledValue(ws_net_paid#52))] +Aggregate Attributes [1]: [sum(UnscaledValue(ws_net_paid#52))#44] +Results [2]: [c_customer_id#48 AS customer_id#59, MakeDecimal(sum(UnscaledValue(ws_net_paid#52))#44,17,2) AS year_total#60] + +(68) BroadcastExchange +Input [2]: [customer_id#59, year_total#60] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=11] + +(69) BroadcastHashJoin [codegen id : 16] +Left keys [1]: [customer_id#14] +Right keys [1]: [customer_id#59] +Join type: Inner +Join condition: (CASE WHEN (year_total#46 > 0.00) THEN (year_total#60 / year_total#46) END > CASE WHEN (year_total#15 > 0.00) THEN (year_total#31 / year_total#15) END) + +(70) Project [codegen id : 16] +Output [3]: [customer_id#28, customer_first_name#29, customer_last_name#30] +Input [9]: [customer_id#14, year_total#15, customer_id#28, customer_first_name#29, customer_last_name#30, year_total#31, year_total#46, customer_id#59, year_total#60] + +(71) TakeOrderedAndProject +Input [3]: [customer_id#28, customer_first_name#29, customer_last_name#30] +Arguments: 100, [customer_first_name#29 ASC NULLS FIRST, customer_id#28 ASC NULLS FIRST, customer_last_name#30 ASC NULLS FIRST], [customer_id#28, customer_first_name#29, customer_last_name#30] + +===== Subqueries ===== + +Subquery:1 Hosting operator id = 4 Hosting Expression = ss_sold_date_sk#7 IN dynamicpruning#8 +BroadcastExchange (75) ++- * ColumnarToRow (74) + +- CometFilter (73) + +- CometScan parquet spark_catalog.default.date_dim (72) + + +(72) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#9, d_year#10] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), In(d_year, [2001,2002]), IsNotNull(d_date_sk)] +ReadSchema: struct<d_date_sk:int,d_year:int> + +(73) CometFilter +Input [2]: [d_date_sk#9, d_year#10] +Condition : (((isnotnull(d_year#10) AND (d_year#10 = 2001)) AND d_year#10 IN (2001,2002)) AND isnotnull(d_date_sk#9)) + +(74) ColumnarToRow [codegen id : 1] +Input [2]: [d_date_sk#9, d_year#10] + +(75) BroadcastExchange +Input [2]: [d_date_sk#9, d_year#10] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=12] + +Subquery:2 Hosting operator id = 20 Hosting Expression = ss_sold_date_sk#22 IN dynamicpruning#23 +BroadcastExchange (79) ++- * ColumnarToRow (78) + +- CometFilter (77) + +- CometScan parquet spark_catalog.default.date_dim (76) + + +(76) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#24, d_year#25] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2002), In(d_year, [2001,2002]), IsNotNull(d_date_sk)] +ReadSchema: struct<d_date_sk:int,d_year:int> + +(77) CometFilter +Input [2]: [d_date_sk#24, d_year#25] +Condition : (((isnotnull(d_year#25) AND (d_year#25 = 2002)) AND d_year#25 IN (2001,2002)) AND isnotnull(d_date_sk#24)) + +(78) ColumnarToRow [codegen id : 1] +Input [2]: [d_date_sk#24, d_year#25] + +(79) BroadcastExchange +Input [2]: [d_date_sk#24, d_year#25] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=13] + +Subquery:3 Hosting operator id = 37 Hosting Expression = ws_sold_date_sk#38 IN dynamicpruning#8 + +Subquery:4 Hosting operator id = 56 Hosting Expression = ws_sold_date_sk#53 IN dynamicpruning#23 + + diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q74/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q74/simplified.txt new file mode 100644 index 000000000..26989b0c0 --- /dev/null +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q74/simplified.txt @@ -0,0 +1,122 @@ +TakeOrderedAndProject [customer_first_name,customer_id,customer_last_name] + WholeStageCodegen (16) + Project [customer_id,customer_first_name,customer_last_name] + BroadcastHashJoin [customer_id,customer_id,year_total,year_total,year_total,year_total] + Project [customer_id,year_total,customer_id,customer_first_name,customer_last_name,year_total,year_total] + BroadcastHashJoin [customer_id,customer_id] + BroadcastHashJoin [customer_id,customer_id] + Filter [year_total] + HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,sum] [sum(UnscaledValue(ss_net_paid)),customer_id,year_total,sum] + InputAdapter + Exchange [c_customer_id,c_first_name,c_last_name,d_year] #1 + WholeStageCodegen (3) + HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,ss_net_paid] [sum,sum] + Project [c_customer_id,c_first_name,c_last_name,ss_net_paid,d_year] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [c_customer_id,c_first_name,c_last_name,ss_net_paid,ss_sold_date_sk] + BroadcastHashJoin [c_customer_sk,ss_customer_sk] + ColumnarToRow + InputAdapter + CometFilter [c_customer_sk,c_customer_id] + CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name] + InputAdapter + BroadcastExchange #2 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometFilter [ss_customer_sk] + CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_net_paid,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometFilter [d_year,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + InputAdapter + ReusedExchange [d_date_sk,d_year] #3 + InputAdapter + BroadcastExchange #4 + WholeStageCodegen (7) + HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,sum] [sum(UnscaledValue(ss_net_paid)),customer_id,customer_first_name,customer_last_name,year_total,sum] + InputAdapter + Exchange [c_customer_id,c_first_name,c_last_name,d_year] #5 + WholeStageCodegen (6) + HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,ss_net_paid] [sum,sum] + Project [c_customer_id,c_first_name,c_last_name,ss_net_paid,d_year] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [c_customer_id,c_first_name,c_last_name,ss_net_paid,ss_sold_date_sk] + BroadcastHashJoin [c_customer_sk,ss_customer_sk] + ColumnarToRow + InputAdapter + CometFilter [c_customer_sk,c_customer_id] + CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name] + InputAdapter + BroadcastExchange #6 + WholeStageCodegen (4) + ColumnarToRow + InputAdapter + CometFilter [ss_customer_sk] + CometScan parquet spark_catalog.default.store_sales [ss_customer_sk,ss_net_paid,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #2 + BroadcastExchange #7 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometFilter [d_year,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + InputAdapter + ReusedExchange [d_date_sk,d_year] #7 + InputAdapter + BroadcastExchange #8 + WholeStageCodegen (11) + Filter [year_total] + HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,sum] [sum(UnscaledValue(ws_net_paid)),customer_id,year_total,sum] + InputAdapter + Exchange [c_customer_id,c_first_name,c_last_name,d_year] #9 + WholeStageCodegen (10) + HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,ws_net_paid] [sum,sum] + Project [c_customer_id,c_first_name,c_last_name,ws_net_paid,d_year] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + Project [c_customer_id,c_first_name,c_last_name,ws_net_paid,ws_sold_date_sk] + BroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] + ColumnarToRow + InputAdapter + CometFilter [c_customer_sk,c_customer_id] + CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name] + InputAdapter + BroadcastExchange #10 + WholeStageCodegen (8) + ColumnarToRow + InputAdapter + CometFilter [ws_bill_customer_sk] + CometScan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + InputAdapter + ReusedExchange [d_date_sk,d_year] #3 + InputAdapter + BroadcastExchange #11 + WholeStageCodegen (15) + HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,sum] [sum(UnscaledValue(ws_net_paid)),customer_id,year_total,sum] + InputAdapter + Exchange [c_customer_id,c_first_name,c_last_name,d_year] #12 + WholeStageCodegen (14) + HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,ws_net_paid] [sum,sum] + Project [c_customer_id,c_first_name,c_last_name,ws_net_paid,d_year] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + Project [c_customer_id,c_first_name,c_last_name,ws_net_paid,ws_sold_date_sk] + BroadcastHashJoin [c_customer_sk,ws_bill_customer_sk] + ColumnarToRow + InputAdapter + CometFilter [c_customer_sk,c_customer_id] + CometScan parquet spark_catalog.default.customer [c_customer_sk,c_customer_id,c_first_name,c_last_name] + InputAdapter + BroadcastExchange #13 + WholeStageCodegen (12) + ColumnarToRow + InputAdapter + CometFilter [ws_bill_customer_sk] + CometScan parquet spark_catalog.default.web_sales [ws_bill_customer_sk,ws_net_paid,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #2 + InputAdapter + ReusedExchange [d_date_sk,d_year] #7 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 new file mode 100644 index 000000000..5c8480a96 --- /dev/null +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q75/explain.txt @@ -0,0 +1,779 @@ +== Physical Plan == +TakeOrderedAndProject (129) ++- * Project (128) + +- * SortMergeJoin Inner (127) + :- * Sort (71) + : +- Exchange (70) + : +- * Filter (69) + : +- * HashAggregate (68) + : +- Exchange (67) + : +- * HashAggregate (66) + : +- * HashAggregate (65) + : +- Exchange (64) + : +- * HashAggregate (63) + : +- Union (62) + : :- * Project (23) + : : +- * SortMergeJoin LeftOuter (22) + : : :- * Sort (15) + : : : +- Exchange (14) + : : : +- * Project (13) + : : : +- * BroadcastHashJoin Inner BuildRight (12) + : : : :- * Project (10) + : : : : +- * BroadcastHashJoin Inner BuildRight (9) + : : : : :- * ColumnarToRow (3) + : : : : : +- CometFilter (2) + : : : : : +- CometScan parquet spark_catalog.default.catalog_sales (1) + : : : : +- BroadcastExchange (8) + : : : : +- * ColumnarToRow (7) + : : : : +- CometProject (6) + : : : : +- CometFilter (5) + : : : : +- CometScan parquet spark_catalog.default.item (4) + : : : +- ReusedExchange (11) + : : +- * Sort (21) + : : +- Exchange (20) + : : +- * ColumnarToRow (19) + : : +- CometProject (18) + : : +- CometFilter (17) + : : +- CometScan parquet spark_catalog.default.catalog_returns (16) + : :- * Project (42) + : : +- * SortMergeJoin LeftOuter (41) + : : :- * Sort (34) + : : : +- Exchange (33) + : : : +- * Project (32) + : : : +- * BroadcastHashJoin Inner BuildRight (31) + : : : :- * Project (29) + : : : : +- * BroadcastHashJoin Inner BuildRight (28) + : : : : :- * ColumnarToRow (26) + : : : : : +- CometFilter (25) + : : : : : +- CometScan parquet spark_catalog.default.store_sales (24) + : : : : +- ReusedExchange (27) + : : : +- ReusedExchange (30) + : : +- * Sort (40) + : : +- Exchange (39) + : : +- * ColumnarToRow (38) + : : +- CometProject (37) + : : +- CometFilter (36) + : : +- CometScan parquet spark_catalog.default.store_returns (35) + : +- * Project (61) + : +- * SortMergeJoin LeftOuter (60) + : :- * Sort (53) + : : +- Exchange (52) + : : +- * Project (51) + : : +- * BroadcastHashJoin Inner BuildRight (50) + : : :- * Project (48) + : : : +- * BroadcastHashJoin Inner BuildRight (47) + : : : :- * ColumnarToRow (45) + : : : : +- CometFilter (44) + : : : : +- CometScan parquet spark_catalog.default.web_sales (43) + : : : +- ReusedExchange (46) + : : +- ReusedExchange (49) + : +- * Sort (59) + : +- Exchange (58) + : +- * ColumnarToRow (57) + : +- CometProject (56) + : +- CometFilter (55) + : +- CometScan parquet spark_catalog.default.web_returns (54) + +- * Sort (126) + +- Exchange (125) + +- * Filter (124) + +- * HashAggregate (123) + +- Exchange (122) + +- * HashAggregate (121) + +- * HashAggregate (120) + +- Exchange (119) + +- * HashAggregate (118) + +- Union (117) + :- * Project (86) + : +- * SortMergeJoin LeftOuter (85) + : :- * Sort (82) + : : +- Exchange (81) + : : +- * Project (80) + : : +- * BroadcastHashJoin Inner BuildRight (79) + : : :- * Project (77) + : : : +- * BroadcastHashJoin Inner BuildRight (76) + : : : :- * ColumnarToRow (74) + : : : : +- CometFilter (73) + : : : : +- CometScan parquet spark_catalog.default.catalog_sales (72) + : : : +- ReusedExchange (75) + : : +- ReusedExchange (78) + : +- * Sort (84) + : +- ReusedExchange (83) + :- * Project (101) + : +- * SortMergeJoin LeftOuter (100) + : :- * Sort (97) + : : +- Exchange (96) + : : +- * Project (95) + : : +- * BroadcastHashJoin Inner BuildRight (94) + : : :- * Project (92) + : : : +- * BroadcastHashJoin Inner BuildRight (91) + : : : :- * ColumnarToRow (89) + : : : : +- CometFilter (88) + : : : : +- CometScan parquet spark_catalog.default.store_sales (87) + : : : +- ReusedExchange (90) + : : +- ReusedExchange (93) + : +- * Sort (99) + : +- ReusedExchange (98) + +- * Project (116) + +- * SortMergeJoin LeftOuter (115) + :- * Sort (112) + : +- Exchange (111) + : +- * Project (110) + : +- * BroadcastHashJoin Inner BuildRight (109) + : :- * Project (107) + : : +- * BroadcastHashJoin Inner BuildRight (106) + : : :- * ColumnarToRow (104) + : : : +- CometFilter (103) + : : : +- CometScan parquet spark_catalog.default.web_sales (102) + : : +- ReusedExchange (105) + : +- ReusedExchange (108) + +- * Sort (114) + +- ReusedExchange (113) + + +(1) Scan parquet spark_catalog.default.catalog_sales +Output [5]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, cs_sold_date_sk#5] +Batched: true +Location: InMemoryFileIndex [] +PartitionFilters: [isnotnull(cs_sold_date_sk#5), dynamicpruningexpression(cs_sold_date_sk#5 IN dynamicpruning#6)] +PushedFilters: [IsNotNull(cs_item_sk)] +ReadSchema: struct<cs_item_sk:int,cs_order_number:int,cs_quantity:int,cs_ext_sales_price:decimal(7,2)> + +(2) CometFilter +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) + +(3) ColumnarToRow [codegen id : 3] +Input [5]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, cs_sold_date_sk#5] + +(4) Scan parquet spark_catalog.default.item +Output [6]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_category#11, i_manufact_id#12] +Batched: true +Location [not included in comparison]/{warehouse_dir}/item] +PushedFilters: [IsNotNull(i_category), EqualTo(i_category,Books ), IsNotNull(i_item_sk), IsNotNull(i_brand_id), IsNotNull(i_class_id), IsNotNull(i_category_id), IsNotNull(i_manufact_id)] +ReadSchema: struct<i_item_sk:int,i_brand_id:int,i_class_id:int,i_category_id:int,i_category:string,i_manufact_id:int> + +(5) CometFilter +Input [6]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_category#11, i_manufact_id#12] +Condition : ((((((isnotnull(i_category#11) AND (i_category#11 = Books )) AND isnotnull(i_item_sk#7)) AND isnotnull(i_brand_id#8)) AND isnotnull(i_class_id#9)) AND isnotnull(i_category_id#10)) AND isnotnull(i_manufact_id#12)) + +(6) CometProject +Input [6]: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_category#11, i_manufact_id#12] +Arguments: [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12], [i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] + +(7) ColumnarToRow [codegen id : 1] +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] +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] +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: 133] +Output [2]: [d_date_sk#13, d_year#14] + +(12) BroadcastHashJoin [codegen id : 3] +Left keys [1]: [cs_sold_date_sk#5] +Right keys [1]: [d_date_sk#13] +Join type: Inner +Join condition: None + +(13) Project [codegen id : 3] +Output [9]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_year#14] +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] + +(14) Exchange +Input [9]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_year#14] +Arguments: hashpartitioning(cs_order_number#2, cs_item_sk#1, 5), ENSURE_REQUIREMENTS, [plan_id=2] + +(15) Sort [codegen id : 4] +Input [9]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_year#14] +Arguments: [cs_order_number#2 ASC NULLS FIRST, cs_item_sk#1 ASC NULLS FIRST], false, 0 + +(16) Scan parquet spark_catalog.default.catalog_returns +Output [5]: [cr_item_sk#15, cr_order_number#16, cr_return_quantity#17, cr_return_amount#18, cr_returned_date_sk#19] +Batched: true +Location [not included in comparison]/{warehouse_dir}/catalog_returns] +PushedFilters: [IsNotNull(cr_order_number), IsNotNull(cr_item_sk)] +ReadSchema: struct<cr_item_sk:int,cr_order_number:int,cr_return_quantity:int,cr_return_amount:decimal(7,2)> + +(17) CometFilter +Input [5]: [cr_item_sk#15, cr_order_number#16, cr_return_quantity#17, cr_return_amount#18, cr_returned_date_sk#19] +Condition : (isnotnull(cr_order_number#16) AND isnotnull(cr_item_sk#15)) + +(18) CometProject +Input [5]: [cr_item_sk#15, cr_order_number#16, cr_return_quantity#17, cr_return_amount#18, cr_returned_date_sk#19] +Arguments: [cr_item_sk#15, cr_order_number#16, cr_return_quantity#17, cr_return_amount#18], [cr_item_sk#15, cr_order_number#16, cr_return_quantity#17, cr_return_amount#18] + +(19) ColumnarToRow [codegen id : 5] +Input [4]: [cr_item_sk#15, cr_order_number#16, cr_return_quantity#17, cr_return_amount#18] + +(20) Exchange +Input [4]: [cr_item_sk#15, cr_order_number#16, cr_return_quantity#17, cr_return_amount#18] +Arguments: hashpartitioning(cr_order_number#16, cr_item_sk#15, 5), ENSURE_REQUIREMENTS, [plan_id=3] + +(21) Sort [codegen id : 6] +Input [4]: [cr_item_sk#15, cr_order_number#16, cr_return_quantity#17, cr_return_amount#18] +Arguments: [cr_order_number#16 ASC NULLS FIRST, cr_item_sk#15 ASC NULLS FIRST], false, 0 + +(22) SortMergeJoin [codegen id : 7] +Left keys [2]: [cs_order_number#2, cs_item_sk#1] +Right keys [2]: [cr_order_number#16, cr_item_sk#15] +Join type: LeftOuter +Join condition: None + +(23) Project [codegen id : 7] +Output [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, (cs_quantity#3 - coalesce(cr_return_quantity#17, 0)) AS sales_cnt#20, (cs_ext_sales_price#4 - coalesce(cr_return_amount#18, 0.00)) AS sales_amt#21] +Input [13]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_year#14, cr_item_sk#15, cr_order_number#16, cr_return_quantity#17, cr_return_amount#18] + +(24) Scan parquet spark_catalog.default.store_sales +Output [5]: [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, ss_sold_date_sk#26] +Batched: true +Location: InMemoryFileIndex [] +PartitionFilters: [isnotnull(ss_sold_date_sk#26), dynamicpruningexpression(ss_sold_date_sk#26 IN dynamicpruning#27)] +PushedFilters: [IsNotNull(ss_item_sk)] +ReadSchema: struct<ss_item_sk:int,ss_ticket_number:int,ss_quantity:int,ss_ext_sales_price:decimal(7,2)> + +(25) CometFilter +Input [5]: [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, ss_sold_date_sk#26] +Condition : isnotnull(ss_item_sk#22) + +(26) ColumnarToRow [codegen id : 10] +Input [5]: [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, ss_sold_date_sk#26] + +(27) ReusedExchange [Reuses operator id: 8] +Output [5]: [i_item_sk#28, i_brand_id#29, i_class_id#30, i_category_id#31, i_manufact_id#32] + +(28) BroadcastHashJoin [codegen id : 10] +Left keys [1]: [ss_item_sk#22] +Right keys [1]: [i_item_sk#28] +Join type: Inner +Join condition: None + +(29) Project [codegen id : 10] +Output [9]: [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, ss_sold_date_sk#26, i_brand_id#29, i_class_id#30, i_category_id#31, i_manufact_id#32] +Input [10]: [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, ss_sold_date_sk#26, i_item_sk#28, i_brand_id#29, i_class_id#30, i_category_id#31, i_manufact_id#32] + +(30) ReusedExchange [Reuses operator id: 133] +Output [2]: [d_date_sk#33, d_year#34] + +(31) BroadcastHashJoin [codegen id : 10] +Left keys [1]: [ss_sold_date_sk#26] +Right keys [1]: [d_date_sk#33] +Join type: Inner +Join condition: None + +(32) Project [codegen id : 10] +Output [9]: [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, i_brand_id#29, i_class_id#30, i_category_id#31, i_manufact_id#32, d_year#34] +Input [11]: [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, ss_sold_date_sk#26, i_brand_id#29, i_class_id#30, i_category_id#31, i_manufact_id#32, d_date_sk#33, d_year#34] + +(33) Exchange +Input [9]: [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, i_brand_id#29, i_class_id#30, i_category_id#31, i_manufact_id#32, d_year#34] +Arguments: hashpartitioning(ss_ticket_number#23, ss_item_sk#22, 5), ENSURE_REQUIREMENTS, [plan_id=4] + +(34) Sort [codegen id : 11] +Input [9]: [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, i_brand_id#29, i_class_id#30, i_category_id#31, i_manufact_id#32, d_year#34] +Arguments: [ss_ticket_number#23 ASC NULLS FIRST, ss_item_sk#22 ASC NULLS FIRST], false, 0 + +(35) Scan parquet spark_catalog.default.store_returns +Output [5]: [sr_item_sk#35, sr_ticket_number#36, sr_return_quantity#37, sr_return_amt#38, sr_returned_date_sk#39] +Batched: true +Location [not included in comparison]/{warehouse_dir}/store_returns] +PushedFilters: [IsNotNull(sr_ticket_number), IsNotNull(sr_item_sk)] +ReadSchema: struct<sr_item_sk:int,sr_ticket_number:int,sr_return_quantity:int,sr_return_amt:decimal(7,2)> + +(36) CometFilter +Input [5]: [sr_item_sk#35, sr_ticket_number#36, sr_return_quantity#37, sr_return_amt#38, sr_returned_date_sk#39] +Condition : (isnotnull(sr_ticket_number#36) AND isnotnull(sr_item_sk#35)) + +(37) CometProject +Input [5]: [sr_item_sk#35, sr_ticket_number#36, sr_return_quantity#37, sr_return_amt#38, sr_returned_date_sk#39] +Arguments: [sr_item_sk#35, sr_ticket_number#36, sr_return_quantity#37, sr_return_amt#38], [sr_item_sk#35, sr_ticket_number#36, sr_return_quantity#37, sr_return_amt#38] + +(38) ColumnarToRow [codegen id : 12] +Input [4]: [sr_item_sk#35, sr_ticket_number#36, sr_return_quantity#37, sr_return_amt#38] + +(39) Exchange +Input [4]: [sr_item_sk#35, sr_ticket_number#36, sr_return_quantity#37, sr_return_amt#38] +Arguments: hashpartitioning(sr_ticket_number#36, sr_item_sk#35, 5), ENSURE_REQUIREMENTS, [plan_id=5] + +(40) Sort [codegen id : 13] +Input [4]: [sr_item_sk#35, sr_ticket_number#36, sr_return_quantity#37, sr_return_amt#38] +Arguments: [sr_ticket_number#36 ASC NULLS FIRST, sr_item_sk#35 ASC NULLS FIRST], false, 0 + +(41) SortMergeJoin [codegen id : 14] +Left keys [2]: [ss_ticket_number#23, ss_item_sk#22] +Right keys [2]: [sr_ticket_number#36, sr_item_sk#35] +Join type: LeftOuter +Join condition: None + +(42) Project [codegen id : 14] +Output [7]: [d_year#34, i_brand_id#29, i_class_id#30, i_category_id#31, i_manufact_id#32, (ss_quantity#24 - coalesce(sr_return_quantity#37, 0)) AS sales_cnt#40, (ss_ext_sales_price#25 - coalesce(sr_return_amt#38, 0.00)) AS sales_amt#41] +Input [13]: [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, i_brand_id#29, i_class_id#30, i_category_id#31, i_manufact_id#32, d_year#34, sr_item_sk#35, sr_ticket_number#36, sr_return_quantity#37, sr_return_amt#38] + +(43) Scan parquet spark_catalog.default.web_sales +Output [5]: [ws_item_sk#42, ws_order_number#43, ws_quantity#44, ws_ext_sales_price#45, ws_sold_date_sk#46] +Batched: true +Location: InMemoryFileIndex [] +PartitionFilters: [isnotnull(ws_sold_date_sk#46), dynamicpruningexpression(ws_sold_date_sk#46 IN dynamicpruning#47)] +PushedFilters: [IsNotNull(ws_item_sk)] +ReadSchema: struct<ws_item_sk:int,ws_order_number:int,ws_quantity:int,ws_ext_sales_price:decimal(7,2)> + +(44) CometFilter +Input [5]: [ws_item_sk#42, ws_order_number#43, ws_quantity#44, ws_ext_sales_price#45, ws_sold_date_sk#46] +Condition : isnotnull(ws_item_sk#42) + +(45) ColumnarToRow [codegen id : 17] +Input [5]: [ws_item_sk#42, ws_order_number#43, ws_quantity#44, ws_ext_sales_price#45, ws_sold_date_sk#46] + +(46) ReusedExchange [Reuses operator id: 8] +Output [5]: [i_item_sk#48, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52] + +(47) BroadcastHashJoin [codegen id : 17] +Left keys [1]: [ws_item_sk#42] +Right keys [1]: [i_item_sk#48] +Join type: Inner +Join condition: None + +(48) Project [codegen id : 17] +Output [9]: [ws_item_sk#42, ws_order_number#43, ws_quantity#44, ws_ext_sales_price#45, ws_sold_date_sk#46, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52] +Input [10]: [ws_item_sk#42, ws_order_number#43, ws_quantity#44, ws_ext_sales_price#45, ws_sold_date_sk#46, i_item_sk#48, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52] + +(49) ReusedExchange [Reuses operator id: 133] +Output [2]: [d_date_sk#53, d_year#54] + +(50) BroadcastHashJoin [codegen id : 17] +Left keys [1]: [ws_sold_date_sk#46] +Right keys [1]: [d_date_sk#53] +Join type: Inner +Join condition: None + +(51) Project [codegen id : 17] +Output [9]: [ws_item_sk#42, ws_order_number#43, ws_quantity#44, ws_ext_sales_price#45, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52, d_year#54] +Input [11]: [ws_item_sk#42, ws_order_number#43, ws_quantity#44, ws_ext_sales_price#45, ws_sold_date_sk#46, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52, d_date_sk#53, d_year#54] + +(52) Exchange +Input [9]: [ws_item_sk#42, ws_order_number#43, ws_quantity#44, ws_ext_sales_price#45, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52, d_year#54] +Arguments: hashpartitioning(ws_order_number#43, ws_item_sk#42, 5), ENSURE_REQUIREMENTS, [plan_id=6] + +(53) Sort [codegen id : 18] +Input [9]: [ws_item_sk#42, ws_order_number#43, ws_quantity#44, ws_ext_sales_price#45, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52, d_year#54] +Arguments: [ws_order_number#43 ASC NULLS FIRST, ws_item_sk#42 ASC NULLS FIRST], false, 0 + +(54) Scan parquet spark_catalog.default.web_returns +Output [5]: [wr_item_sk#55, wr_order_number#56, wr_return_quantity#57, wr_return_amt#58, wr_returned_date_sk#59] +Batched: true +Location [not included in comparison]/{warehouse_dir}/web_returns] +PushedFilters: [IsNotNull(wr_order_number), IsNotNull(wr_item_sk)] +ReadSchema: struct<wr_item_sk:int,wr_order_number:int,wr_return_quantity:int,wr_return_amt:decimal(7,2)> + +(55) CometFilter +Input [5]: [wr_item_sk#55, wr_order_number#56, wr_return_quantity#57, wr_return_amt#58, wr_returned_date_sk#59] +Condition : (isnotnull(wr_order_number#56) AND isnotnull(wr_item_sk#55)) + +(56) CometProject +Input [5]: [wr_item_sk#55, wr_order_number#56, wr_return_quantity#57, wr_return_amt#58, wr_returned_date_sk#59] +Arguments: [wr_item_sk#55, wr_order_number#56, wr_return_quantity#57, wr_return_amt#58], [wr_item_sk#55, wr_order_number#56, wr_return_quantity#57, wr_return_amt#58] + +(57) ColumnarToRow [codegen id : 19] +Input [4]: [wr_item_sk#55, wr_order_number#56, wr_return_quantity#57, wr_return_amt#58] + +(58) Exchange +Input [4]: [wr_item_sk#55, wr_order_number#56, wr_return_quantity#57, wr_return_amt#58] +Arguments: hashpartitioning(wr_order_number#56, wr_item_sk#55, 5), ENSURE_REQUIREMENTS, [plan_id=7] + +(59) Sort [codegen id : 20] +Input [4]: [wr_item_sk#55, wr_order_number#56, wr_return_quantity#57, wr_return_amt#58] +Arguments: [wr_order_number#56 ASC NULLS FIRST, wr_item_sk#55 ASC NULLS FIRST], false, 0 + +(60) SortMergeJoin [codegen id : 21] +Left keys [2]: [ws_order_number#43, ws_item_sk#42] +Right keys [2]: [wr_order_number#56, wr_item_sk#55] +Join type: LeftOuter +Join condition: None + +(61) Project [codegen id : 21] +Output [7]: [d_year#54, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52, (ws_quantity#44 - coalesce(wr_return_quantity#57, 0)) AS sales_cnt#60, (ws_ext_sales_price#45 - coalesce(wr_return_amt#58, 0.00)) AS sales_amt#61] +Input [13]: [ws_item_sk#42, ws_order_number#43, ws_quantity#44, ws_ext_sales_price#45, i_brand_id#49, i_class_id#50, i_category_id#51, i_manufact_id#52, d_year#54, wr_item_sk#55, wr_order_number#56, wr_return_quantity#57, wr_return_amt#58] + +(62) Union + +(63) HashAggregate [codegen id : 22] +Input [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#20, sales_amt#21] +Keys [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#20, sales_amt#21] +Functions: [] +Aggregate Attributes: [] +Results [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#20, sales_amt#21] + +(64) Exchange +Input [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#20, sales_amt#21] +Arguments: hashpartitioning(d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#20, sales_amt#21, 5), ENSURE_REQUIREMENTS, [plan_id=8] + +(65) HashAggregate [codegen id : 23] +Input [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#20, sales_amt#21] +Keys [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#20, sales_amt#21] +Functions: [] +Aggregate Attributes: [] +Results [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#20, sales_amt#21] + +(66) HashAggregate [codegen id : 23] +Input [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#20, sales_amt#21] +Keys [5]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] +Functions [2]: [partial_sum(sales_cnt#20), partial_sum(UnscaledValue(sales_amt#21))] +Aggregate Attributes [2]: [sum#62, sum#63] +Results [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sum#64, sum#65] + +(67) Exchange +Input [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sum#64, sum#65] +Arguments: hashpartitioning(d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, 5), ENSURE_REQUIREMENTS, [plan_id=9] + +(68) HashAggregate [codegen id : 24] +Input [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sum#64, sum#65] +Keys [5]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] +Functions [2]: [sum(sales_cnt#20), sum(UnscaledValue(sales_amt#21))] +Aggregate Attributes [2]: [sum(sales_cnt#20)#66, sum(UnscaledValue(sales_amt#21))#67] +Results [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sum(sales_cnt#20)#66 AS sales_cnt#68, MakeDecimal(sum(UnscaledValue(sales_amt#21))#67,18,2) AS sales_amt#69] + +(69) Filter [codegen id : 24] +Input [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#68, sales_amt#69] +Condition : isnotnull(sales_cnt#68) + +(70) Exchange +Input [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#68, sales_amt#69] +Arguments: hashpartitioning(i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, 5), ENSURE_REQUIREMENTS, [plan_id=10] + +(71) Sort [codegen id : 25] +Input [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#68, sales_amt#69] +Arguments: [i_brand_id#8 ASC NULLS FIRST, i_class_id#9 ASC NULLS FIRST, i_category_id#10 ASC NULLS FIRST, i_manufact_id#12 ASC NULLS FIRST], false, 0 + +(72) Scan parquet spark_catalog.default.catalog_sales +Output [5]: [cs_item_sk#70, cs_order_number#71, cs_quantity#72, cs_ext_sales_price#73, cs_sold_date_sk#74] +Batched: true +Location: InMemoryFileIndex [] +PartitionFilters: [isnotnull(cs_sold_date_sk#74), dynamicpruningexpression(cs_sold_date_sk#74 IN dynamicpruning#75)] +PushedFilters: [IsNotNull(cs_item_sk)] +ReadSchema: struct<cs_item_sk:int,cs_order_number:int,cs_quantity:int,cs_ext_sales_price:decimal(7,2)> + +(73) CometFilter +Input [5]: [cs_item_sk#70, cs_order_number#71, cs_quantity#72, cs_ext_sales_price#73, cs_sold_date_sk#74] +Condition : isnotnull(cs_item_sk#70) + +(74) ColumnarToRow [codegen id : 28] +Input [5]: [cs_item_sk#70, cs_order_number#71, cs_quantity#72, cs_ext_sales_price#73, cs_sold_date_sk#74] + +(75) ReusedExchange [Reuses operator id: 8] +Output [5]: [i_item_sk#76, i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80] + +(76) BroadcastHashJoin [codegen id : 28] +Left keys [1]: [cs_item_sk#70] +Right keys [1]: [i_item_sk#76] +Join type: Inner +Join condition: None + +(77) Project [codegen id : 28] +Output [9]: [cs_item_sk#70, cs_order_number#71, cs_quantity#72, cs_ext_sales_price#73, cs_sold_date_sk#74, i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80] +Input [10]: [cs_item_sk#70, cs_order_number#71, cs_quantity#72, cs_ext_sales_price#73, cs_sold_date_sk#74, i_item_sk#76, i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80] + +(78) ReusedExchange [Reuses operator id: 137] +Output [2]: [d_date_sk#81, d_year#82] + +(79) BroadcastHashJoin [codegen id : 28] +Left keys [1]: [cs_sold_date_sk#74] +Right keys [1]: [d_date_sk#81] +Join type: Inner +Join condition: None + +(80) Project [codegen id : 28] +Output [9]: [cs_item_sk#70, cs_order_number#71, cs_quantity#72, cs_ext_sales_price#73, i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80, d_year#82] +Input [11]: [cs_item_sk#70, cs_order_number#71, cs_quantity#72, cs_ext_sales_price#73, cs_sold_date_sk#74, i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80, d_date_sk#81, d_year#82] + +(81) Exchange +Input [9]: [cs_item_sk#70, cs_order_number#71, cs_quantity#72, cs_ext_sales_price#73, i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80, d_year#82] +Arguments: hashpartitioning(cs_order_number#71, cs_item_sk#70, 5), ENSURE_REQUIREMENTS, [plan_id=11] + +(82) Sort [codegen id : 29] +Input [9]: [cs_item_sk#70, cs_order_number#71, cs_quantity#72, cs_ext_sales_price#73, i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80, d_year#82] +Arguments: [cs_order_number#71 ASC NULLS FIRST, cs_item_sk#70 ASC NULLS FIRST], false, 0 + +(83) ReusedExchange [Reuses operator id: 20] +Output [4]: [cr_item_sk#83, cr_order_number#84, cr_return_quantity#85, cr_return_amount#86] + +(84) Sort [codegen id : 31] +Input [4]: [cr_item_sk#83, cr_order_number#84, cr_return_quantity#85, cr_return_amount#86] +Arguments: [cr_order_number#84 ASC NULLS FIRST, cr_item_sk#83 ASC NULLS FIRST], false, 0 + +(85) SortMergeJoin [codegen id : 32] +Left keys [2]: [cs_order_number#71, cs_item_sk#70] +Right keys [2]: [cr_order_number#84, cr_item_sk#83] +Join type: LeftOuter +Join condition: None + +(86) Project [codegen id : 32] +Output [7]: [d_year#82, i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80, (cs_quantity#72 - coalesce(cr_return_quantity#85, 0)) AS sales_cnt#87, (cs_ext_sales_price#73 - coalesce(cr_return_amount#86, 0.00)) AS sales_amt#88] +Input [13]: [cs_item_sk#70, cs_order_number#71, cs_quantity#72, cs_ext_sales_price#73, i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80, d_year#82, cr_item_sk#83, cr_order_number#84, cr_return_quantity#85, cr_return_amount#86] + +(87) Scan parquet spark_catalog.default.store_sales +Output [5]: [ss_item_sk#89, ss_ticket_number#90, ss_quantity#91, ss_ext_sales_price#92, ss_sold_date_sk#93] +Batched: true +Location: InMemoryFileIndex [] +PartitionFilters: [isnotnull(ss_sold_date_sk#93), dynamicpruningexpression(ss_sold_date_sk#93 IN dynamicpruning#94)] +PushedFilters: [IsNotNull(ss_item_sk)] +ReadSchema: struct<ss_item_sk:int,ss_ticket_number:int,ss_quantity:int,ss_ext_sales_price:decimal(7,2)> + +(88) CometFilter +Input [5]: [ss_item_sk#89, ss_ticket_number#90, ss_quantity#91, ss_ext_sales_price#92, ss_sold_date_sk#93] +Condition : isnotnull(ss_item_sk#89) + +(89) ColumnarToRow [codegen id : 35] +Input [5]: [ss_item_sk#89, ss_ticket_number#90, ss_quantity#91, ss_ext_sales_price#92, ss_sold_date_sk#93] + +(90) ReusedExchange [Reuses operator id: 8] +Output [5]: [i_item_sk#95, i_brand_id#96, i_class_id#97, i_category_id#98, i_manufact_id#99] + +(91) BroadcastHashJoin [codegen id : 35] +Left keys [1]: [ss_item_sk#89] +Right keys [1]: [i_item_sk#95] +Join type: Inner +Join condition: None + +(92) Project [codegen id : 35] +Output [9]: [ss_item_sk#89, ss_ticket_number#90, ss_quantity#91, ss_ext_sales_price#92, ss_sold_date_sk#93, i_brand_id#96, i_class_id#97, i_category_id#98, i_manufact_id#99] +Input [10]: [ss_item_sk#89, ss_ticket_number#90, ss_quantity#91, ss_ext_sales_price#92, ss_sold_date_sk#93, i_item_sk#95, i_brand_id#96, i_class_id#97, i_category_id#98, i_manufact_id#99] + +(93) ReusedExchange [Reuses operator id: 137] +Output [2]: [d_date_sk#100, d_year#101] + +(94) BroadcastHashJoin [codegen id : 35] +Left keys [1]: [ss_sold_date_sk#93] +Right keys [1]: [d_date_sk#100] +Join type: Inner +Join condition: None + +(95) Project [codegen id : 35] +Output [9]: [ss_item_sk#89, ss_ticket_number#90, ss_quantity#91, ss_ext_sales_price#92, i_brand_id#96, i_class_id#97, i_category_id#98, i_manufact_id#99, d_year#101] +Input [11]: [ss_item_sk#89, ss_ticket_number#90, ss_quantity#91, ss_ext_sales_price#92, ss_sold_date_sk#93, i_brand_id#96, i_class_id#97, i_category_id#98, i_manufact_id#99, d_date_sk#100, d_year#101] + +(96) Exchange +Input [9]: [ss_item_sk#89, ss_ticket_number#90, ss_quantity#91, ss_ext_sales_price#92, i_brand_id#96, i_class_id#97, i_category_id#98, i_manufact_id#99, d_year#101] +Arguments: hashpartitioning(ss_ticket_number#90, ss_item_sk#89, 5), ENSURE_REQUIREMENTS, [plan_id=12] + +(97) Sort [codegen id : 36] +Input [9]: [ss_item_sk#89, ss_ticket_number#90, ss_quantity#91, ss_ext_sales_price#92, i_brand_id#96, i_class_id#97, i_category_id#98, i_manufact_id#99, d_year#101] +Arguments: [ss_ticket_number#90 ASC NULLS FIRST, ss_item_sk#89 ASC NULLS FIRST], false, 0 + +(98) ReusedExchange [Reuses operator id: 39] +Output [4]: [sr_item_sk#102, sr_ticket_number#103, sr_return_quantity#104, sr_return_amt#105] + +(99) Sort [codegen id : 38] +Input [4]: [sr_item_sk#102, sr_ticket_number#103, sr_return_quantity#104, sr_return_amt#105] +Arguments: [sr_ticket_number#103 ASC NULLS FIRST, sr_item_sk#102 ASC NULLS FIRST], false, 0 + +(100) SortMergeJoin [codegen id : 39] +Left keys [2]: [ss_ticket_number#90, ss_item_sk#89] +Right keys [2]: [sr_ticket_number#103, sr_item_sk#102] +Join type: LeftOuter +Join condition: None + +(101) Project [codegen id : 39] +Output [7]: [d_year#101, i_brand_id#96, i_class_id#97, i_category_id#98, i_manufact_id#99, (ss_quantity#91 - coalesce(sr_return_quantity#104, 0)) AS sales_cnt#106, (ss_ext_sales_price#92 - coalesce(sr_return_amt#105, 0.00)) AS sales_amt#107] +Input [13]: [ss_item_sk#89, ss_ticket_number#90, ss_quantity#91, ss_ext_sales_price#92, i_brand_id#96, i_class_id#97, i_category_id#98, i_manufact_id#99, d_year#101, sr_item_sk#102, sr_ticket_number#103, sr_return_quantity#104, sr_return_amt#105] + +(102) Scan parquet spark_catalog.default.web_sales +Output [5]: [ws_item_sk#108, ws_order_number#109, ws_quantity#110, ws_ext_sales_price#111, ws_sold_date_sk#112] +Batched: true +Location: InMemoryFileIndex [] +PartitionFilters: [isnotnull(ws_sold_date_sk#112), dynamicpruningexpression(ws_sold_date_sk#112 IN dynamicpruning#113)] +PushedFilters: [IsNotNull(ws_item_sk)] +ReadSchema: struct<ws_item_sk:int,ws_order_number:int,ws_quantity:int,ws_ext_sales_price:decimal(7,2)> + +(103) CometFilter +Input [5]: [ws_item_sk#108, ws_order_number#109, ws_quantity#110, ws_ext_sales_price#111, ws_sold_date_sk#112] +Condition : isnotnull(ws_item_sk#108) + +(104) ColumnarToRow [codegen id : 42] +Input [5]: [ws_item_sk#108, ws_order_number#109, ws_quantity#110, ws_ext_sales_price#111, ws_sold_date_sk#112] + +(105) ReusedExchange [Reuses operator id: 8] +Output [5]: [i_item_sk#114, i_brand_id#115, i_class_id#116, i_category_id#117, i_manufact_id#118] + +(106) BroadcastHashJoin [codegen id : 42] +Left keys [1]: [ws_item_sk#108] +Right keys [1]: [i_item_sk#114] +Join type: Inner +Join condition: None + +(107) Project [codegen id : 42] +Output [9]: [ws_item_sk#108, ws_order_number#109, ws_quantity#110, ws_ext_sales_price#111, ws_sold_date_sk#112, i_brand_id#115, i_class_id#116, i_category_id#117, i_manufact_id#118] +Input [10]: [ws_item_sk#108, ws_order_number#109, ws_quantity#110, ws_ext_sales_price#111, ws_sold_date_sk#112, i_item_sk#114, i_brand_id#115, i_class_id#116, i_category_id#117, i_manufact_id#118] + +(108) ReusedExchange [Reuses operator id: 137] +Output [2]: [d_date_sk#119, d_year#120] + +(109) BroadcastHashJoin [codegen id : 42] +Left keys [1]: [ws_sold_date_sk#112] +Right keys [1]: [d_date_sk#119] +Join type: Inner +Join condition: None + +(110) Project [codegen id : 42] +Output [9]: [ws_item_sk#108, ws_order_number#109, ws_quantity#110, ws_ext_sales_price#111, i_brand_id#115, i_class_id#116, i_category_id#117, i_manufact_id#118, d_year#120] +Input [11]: [ws_item_sk#108, ws_order_number#109, ws_quantity#110, ws_ext_sales_price#111, ws_sold_date_sk#112, i_brand_id#115, i_class_id#116, i_category_id#117, i_manufact_id#118, d_date_sk#119, d_year#120] + +(111) Exchange +Input [9]: [ws_item_sk#108, ws_order_number#109, ws_quantity#110, ws_ext_sales_price#111, i_brand_id#115, i_class_id#116, i_category_id#117, i_manufact_id#118, d_year#120] +Arguments: hashpartitioning(ws_order_number#109, ws_item_sk#108, 5), ENSURE_REQUIREMENTS, [plan_id=13] + +(112) Sort [codegen id : 43] +Input [9]: [ws_item_sk#108, ws_order_number#109, ws_quantity#110, ws_ext_sales_price#111, i_brand_id#115, i_class_id#116, i_category_id#117, i_manufact_id#118, d_year#120] +Arguments: [ws_order_number#109 ASC NULLS FIRST, ws_item_sk#108 ASC NULLS FIRST], false, 0 + +(113) ReusedExchange [Reuses operator id: 58] +Output [4]: [wr_item_sk#121, wr_order_number#122, wr_return_quantity#123, wr_return_amt#124] + +(114) Sort [codegen id : 45] +Input [4]: [wr_item_sk#121, wr_order_number#122, wr_return_quantity#123, wr_return_amt#124] +Arguments: [wr_order_number#122 ASC NULLS FIRST, wr_item_sk#121 ASC NULLS FIRST], false, 0 + +(115) SortMergeJoin [codegen id : 46] +Left keys [2]: [ws_order_number#109, ws_item_sk#108] +Right keys [2]: [wr_order_number#122, wr_item_sk#121] +Join type: LeftOuter +Join condition: None + +(116) Project [codegen id : 46] +Output [7]: [d_year#120, i_brand_id#115, i_class_id#116, i_category_id#117, i_manufact_id#118, (ws_quantity#110 - coalesce(wr_return_quantity#123, 0)) AS sales_cnt#125, (ws_ext_sales_price#111 - coalesce(wr_return_amt#124, 0.00)) AS sales_amt#126] +Input [13]: [ws_item_sk#108, ws_order_number#109, ws_quantity#110, ws_ext_sales_price#111, i_brand_id#115, i_class_id#116, i_category_id#117, i_manufact_id#118, d_year#120, wr_item_sk#121, wr_order_number#122, wr_return_quantity#123, wr_return_amt#124] + +(117) Union + +(118) HashAggregate [codegen id : 47] +Input [7]: [d_year#82, i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80, sales_cnt#87, sales_amt#88] +Keys [7]: [d_year#82, i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80, sales_cnt#87, sales_amt#88] +Functions: [] +Aggregate Attributes: [] +Results [7]: [d_year#82, i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80, sales_cnt#87, sales_amt#88] + +(119) Exchange +Input [7]: [d_year#82, i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80, sales_cnt#87, sales_amt#88] +Arguments: hashpartitioning(d_year#82, i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80, sales_cnt#87, sales_amt#88, 5), ENSURE_REQUIREMENTS, [plan_id=14] + +(120) HashAggregate [codegen id : 48] +Input [7]: [d_year#82, i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80, sales_cnt#87, sales_amt#88] +Keys [7]: [d_year#82, i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80, sales_cnt#87, sales_amt#88] +Functions: [] +Aggregate Attributes: [] +Results [7]: [d_year#82, i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80, sales_cnt#87, sales_amt#88] + +(121) HashAggregate [codegen id : 48] +Input [7]: [d_year#82, i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80, sales_cnt#87, sales_amt#88] +Keys [5]: [d_year#82, i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80] +Functions [2]: [partial_sum(sales_cnt#87), partial_sum(UnscaledValue(sales_amt#88))] +Aggregate Attributes [2]: [sum#127, sum#128] +Results [7]: [d_year#82, i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80, sum#129, sum#130] + +(122) Exchange +Input [7]: [d_year#82, i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80, sum#129, sum#130] +Arguments: hashpartitioning(d_year#82, i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80, 5), ENSURE_REQUIREMENTS, [plan_id=15] + +(123) HashAggregate [codegen id : 49] +Input [7]: [d_year#82, i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80, sum#129, sum#130] +Keys [5]: [d_year#82, i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80] +Functions [2]: [sum(sales_cnt#87), sum(UnscaledValue(sales_amt#88))] +Aggregate Attributes [2]: [sum(sales_cnt#87)#66, sum(UnscaledValue(sales_amt#88))#67] +Results [7]: [d_year#82, i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80, sum(sales_cnt#87)#66 AS sales_cnt#131, MakeDecimal(sum(UnscaledValue(sales_amt#88))#67,18,2) AS sales_amt#132] + +(124) Filter [codegen id : 49] +Input [7]: [d_year#82, i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80, sales_cnt#131, sales_amt#132] +Condition : isnotnull(sales_cnt#131) + +(125) Exchange +Input [7]: [d_year#82, i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80, sales_cnt#131, sales_amt#132] +Arguments: hashpartitioning(i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80, 5), ENSURE_REQUIREMENTS, [plan_id=16] + +(126) Sort [codegen id : 50] +Input [7]: [d_year#82, i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80, sales_cnt#131, sales_amt#132] +Arguments: [i_brand_id#77 ASC NULLS FIRST, i_class_id#78 ASC NULLS FIRST, i_category_id#79 ASC NULLS FIRST, i_manufact_id#80 ASC NULLS FIRST], false, 0 + +(127) SortMergeJoin [codegen id : 51] +Left keys [4]: [i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12] +Right keys [4]: [i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80] +Join type: Inner +Join condition: ((cast(sales_cnt#68 as decimal(17,2)) / cast(sales_cnt#131 as decimal(17,2))) < 0.90000000000000000000) + +(128) Project [codegen id : 51] +Output [10]: [d_year#82 AS prev_year#133, d_year#14 AS year#134, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#131 AS prev_yr_cnt#135, sales_cnt#68 AS curr_yr_cnt#136, (sales_cnt#68 - sales_cnt#131) AS sales_cnt_diff#137, (sales_amt#69 - sales_amt#132) AS sales_amt_diff#138] +Input [14]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#68, sales_amt#69, d_year#82, i_brand_id#77, i_class_id#78, i_category_id#79, i_manufact_id#80, sales_cnt#131, sales_amt#132] + +(129) TakeOrderedAndProject +Input [10]: [prev_year#133, year#134, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, prev_yr_cnt#135, curr_yr_cnt#136, sales_cnt_diff#137, sales_amt_diff#138] +Arguments: 100, [sales_cnt_diff#137 ASC NULLS FIRST, sales_amt_diff#138 ASC NULLS FIRST], [prev_year#133, year#134, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, prev_yr_cnt#135, curr_yr_cnt#136, sales_cnt_diff#137, sales_amt_diff#138] + +===== Subqueries ===== + +Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#5 IN dynamicpruning#6 +BroadcastExchange (133) ++- * ColumnarToRow (132) + +- CometFilter (131) + +- CometScan parquet spark_catalog.default.date_dim (130) + + +(130) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#13, d_year#14] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2002), IsNotNull(d_date_sk)] +ReadSchema: struct<d_date_sk:int,d_year:int> + +(131) CometFilter +Input [2]: [d_date_sk#13, d_year#14] +Condition : ((isnotnull(d_year#14) AND (d_year#14 = 2002)) AND isnotnull(d_date_sk#13)) + +(132) ColumnarToRow [codegen id : 1] +Input [2]: [d_date_sk#13, d_year#14] + +(133) BroadcastExchange +Input [2]: [d_date_sk#13, d_year#14] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=17] + +Subquery:2 Hosting operator id = 24 Hosting Expression = ss_sold_date_sk#26 IN dynamicpruning#6 + +Subquery:3 Hosting operator id = 43 Hosting Expression = ws_sold_date_sk#46 IN dynamicpruning#6 + +Subquery:4 Hosting operator id = 72 Hosting Expression = cs_sold_date_sk#74 IN dynamicpruning#75 +BroadcastExchange (137) ++- * ColumnarToRow (136) + +- CometFilter (135) + +- CometScan parquet spark_catalog.default.date_dim (134) + + +(134) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#81, d_year#82] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), IsNotNull(d_date_sk)] +ReadSchema: struct<d_date_sk:int,d_year:int> + +(135) CometFilter +Input [2]: [d_date_sk#81, d_year#82] +Condition : ((isnotnull(d_year#82) AND (d_year#82 = 2001)) AND isnotnull(d_date_sk#81)) + +(136) ColumnarToRow [codegen id : 1] +Input [2]: [d_date_sk#81, d_year#82] + +(137) BroadcastExchange +Input [2]: [d_date_sk#81, d_year#82] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=18] + +Subquery:5 Hosting operator id = 87 Hosting Expression = ss_sold_date_sk#93 IN dynamicpruning#75 + +Subquery:6 Hosting operator id = 102 Hosting Expression = ws_sold_date_sk#112 IN dynamicpruning#75 + + 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 new file mode 100644 index 000000000..9939b2fe2 --- /dev/null +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q75/simplified.txt @@ -0,0 +1,240 @@ +TakeOrderedAndProject [sales_cnt_diff,sales_amt_diff,prev_year,year,i_brand_id,i_class_id,i_category_id,i_manufact_id,prev_yr_cnt,curr_yr_cnt] + WholeStageCodegen (51) + Project [d_year,d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_cnt,sales_amt,sales_amt] + SortMergeJoin [i_brand_id,i_class_id,i_category_id,i_manufact_id,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_cnt] + InputAdapter + WholeStageCodegen (25) + Sort [i_brand_id,i_class_id,i_category_id,i_manufact_id] + InputAdapter + Exchange [i_brand_id,i_class_id,i_category_id,i_manufact_id] #1 + WholeStageCodegen (24) + Filter [sales_cnt] + HashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sum,sum] [sum(sales_cnt),sum(UnscaledValue(sales_amt)),sales_cnt,sales_amt,sum,sum] + InputAdapter + Exchange [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id] #2 + WholeStageCodegen (23) + HashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] [sum,sum,sum,sum] + HashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] + InputAdapter + Exchange [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] #3 + WholeStageCodegen (22) + HashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] + InputAdapter + Union + WholeStageCodegen (7) + Project [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,cs_quantity,cr_return_quantity,cs_ext_sales_price,cr_return_amount] + SortMergeJoin [cs_order_number,cs_item_sk,cr_order_number,cr_item_sk] + InputAdapter + WholeStageCodegen (4) + Sort [cs_order_number,cs_item_sk] + InputAdapter + Exchange [cs_order_number,cs_item_sk] #4 + WholeStageCodegen (3) + Project [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + Project [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] + BroadcastHashJoin [cs_item_sk,i_item_sk] + ColumnarToRow + InputAdapter + CometFilter [cs_item_sk] + CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #5 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometFilter [d_year,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + InputAdapter + BroadcastExchange #6 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] + CometFilter [i_category,i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] + CometScan parquet spark_catalog.default.item [i_item_sk,i_brand_id,i_class_id,i_category_id,i_category,i_manufact_id] + InputAdapter + ReusedExchange [d_date_sk,d_year] #5 + InputAdapter + WholeStageCodegen (6) + Sort [cr_order_number,cr_item_sk] + InputAdapter + Exchange [cr_order_number,cr_item_sk] #7 + WholeStageCodegen (5) + ColumnarToRow + InputAdapter + CometProject [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] + CometFilter [cr_order_number,cr_item_sk] + CometScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount,cr_returned_date_sk] + WholeStageCodegen (14) + Project [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,ss_quantity,sr_return_quantity,ss_ext_sales_price,sr_return_amt] + SortMergeJoin [ss_ticket_number,ss_item_sk,sr_ticket_number,sr_item_sk] + InputAdapter + WholeStageCodegen (11) + Sort [ss_ticket_number,ss_item_sk] + InputAdapter + Exchange [ss_ticket_number,ss_item_sk] #8 + WholeStageCodegen (10) + Project [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] + BroadcastHashJoin [ss_item_sk,i_item_sk] + ColumnarToRow + InputAdapter + CometFilter [ss_item_sk] + CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + InputAdapter + ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 + InputAdapter + ReusedExchange [d_date_sk,d_year] #5 + InputAdapter + WholeStageCodegen (13) + Sort [sr_ticket_number,sr_item_sk] + InputAdapter + Exchange [sr_ticket_number,sr_item_sk] #9 + WholeStageCodegen (12) + ColumnarToRow + InputAdapter + CometProject [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] + CometFilter [sr_ticket_number,sr_item_sk] + CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt,sr_returned_date_sk] + WholeStageCodegen (21) + Project [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,ws_quantity,wr_return_quantity,ws_ext_sales_price,wr_return_amt] + SortMergeJoin [ws_order_number,ws_item_sk,wr_order_number,wr_item_sk] + InputAdapter + WholeStageCodegen (18) + Sort [ws_order_number,ws_item_sk] + InputAdapter + Exchange [ws_order_number,ws_item_sk] #10 + WholeStageCodegen (17) + Project [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + Project [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] + BroadcastHashJoin [ws_item_sk,i_item_sk] + ColumnarToRow + InputAdapter + CometFilter [ws_item_sk] + CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + InputAdapter + ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 + InputAdapter + ReusedExchange [d_date_sk,d_year] #5 + InputAdapter + WholeStageCodegen (20) + Sort [wr_order_number,wr_item_sk] + InputAdapter + Exchange [wr_order_number,wr_item_sk] #11 + WholeStageCodegen (19) + ColumnarToRow + InputAdapter + CometProject [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] + CometFilter [wr_order_number,wr_item_sk] + CometScan parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk] + InputAdapter + WholeStageCodegen (50) + Sort [i_brand_id,i_class_id,i_category_id,i_manufact_id] + InputAdapter + Exchange [i_brand_id,i_class_id,i_category_id,i_manufact_id] #12 + WholeStageCodegen (49) + Filter [sales_cnt] + HashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sum,sum] [sum(sales_cnt),sum(UnscaledValue(sales_amt)),sales_cnt,sales_amt,sum,sum] + InputAdapter + Exchange [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id] #13 + WholeStageCodegen (48) + HashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] [sum,sum,sum,sum] + HashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] + InputAdapter + Exchange [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] #14 + WholeStageCodegen (47) + HashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] + InputAdapter + Union + WholeStageCodegen (32) + Project [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,cs_quantity,cr_return_quantity,cs_ext_sales_price,cr_return_amount] + SortMergeJoin [cs_order_number,cs_item_sk,cr_order_number,cr_item_sk] + InputAdapter + WholeStageCodegen (29) + Sort [cs_order_number,cs_item_sk] + InputAdapter + Exchange [cs_order_number,cs_item_sk] #15 + WholeStageCodegen (28) + Project [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + Project [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] + BroadcastHashJoin [cs_item_sk,i_item_sk] + ColumnarToRow + InputAdapter + CometFilter [cs_item_sk] + CometScan parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk] + SubqueryBroadcast [d_date_sk] #2 + BroadcastExchange #16 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometFilter [d_year,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + InputAdapter + ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 + InputAdapter + ReusedExchange [d_date_sk,d_year] #16 + InputAdapter + WholeStageCodegen (31) + Sort [cr_order_number,cr_item_sk] + InputAdapter + ReusedExchange [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] #7 + WholeStageCodegen (39) + Project [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,ss_quantity,sr_return_quantity,ss_ext_sales_price,sr_return_amt] + SortMergeJoin [ss_ticket_number,ss_item_sk,sr_ticket_number,sr_item_sk] + InputAdapter + WholeStageCodegen (36) + Sort [ss_ticket_number,ss_item_sk] + InputAdapter + Exchange [ss_ticket_number,ss_item_sk] #17 + WholeStageCodegen (35) + Project [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] + BroadcastHashJoin [ss_item_sk,i_item_sk] + ColumnarToRow + InputAdapter + CometFilter [ss_item_sk] + CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk] + ReusedSubquery [d_date_sk] #2 + InputAdapter + ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 + InputAdapter + ReusedExchange [d_date_sk,d_year] #16 + InputAdapter + WholeStageCodegen (38) + Sort [sr_ticket_number,sr_item_sk] + InputAdapter + ReusedExchange [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] #9 + WholeStageCodegen (46) + Project [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,ws_quantity,wr_return_quantity,ws_ext_sales_price,wr_return_amt] + SortMergeJoin [ws_order_number,ws_item_sk,wr_order_number,wr_item_sk] + InputAdapter + WholeStageCodegen (43) + Sort [ws_order_number,ws_item_sk] + InputAdapter + Exchange [ws_order_number,ws_item_sk] #18 + WholeStageCodegen (42) + Project [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + Project [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] + BroadcastHashJoin [ws_item_sk,i_item_sk] + ColumnarToRow + InputAdapter + CometFilter [ws_item_sk] + CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #2 + InputAdapter + ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6 + InputAdapter + ReusedExchange [d_date_sk,d_year] #16 + InputAdapter + WholeStageCodegen (45) + Sort [wr_order_number,wr_item_sk] + InputAdapter + 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/q77a/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q77a/explain.txt new file mode 100644 index 000000000..5c1865267 --- /dev/null +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q77a/explain.txt @@ -0,0 +1,632 @@ +== Physical Plan == +TakeOrderedAndProject (98) ++- * HashAggregate (97) + +- Exchange (96) + +- * HashAggregate (95) + +- Union (94) + :- * HashAggregate (83) + : +- Exchange (82) + : +- * HashAggregate (81) + : +- Union (80) + : :- * Project (30) + : : +- * BroadcastHashJoin LeftOuter BuildRight (29) + : : :- * HashAggregate (15) + : : : +- Exchange (14) + : : : +- * HashAggregate (13) + : : : +- * Project (12) + : : : +- * BroadcastHashJoin Inner BuildRight (11) + : : : :- * Project (6) + : : : : +- * BroadcastHashJoin Inner BuildRight (5) + : : : : :- * ColumnarToRow (3) + : : : : : +- CometFilter (2) + : : : : : +- CometScan parquet spark_catalog.default.store_sales (1) + : : : : +- ReusedExchange (4) + : : : +- BroadcastExchange (10) + : : : +- * ColumnarToRow (9) + : : : +- CometFilter (8) + : : : +- CometScan parquet spark_catalog.default.store (7) + : : +- BroadcastExchange (28) + : : +- * HashAggregate (27) + : : +- Exchange (26) + : : +- * HashAggregate (25) + : : +- * Project (24) + : : +- * BroadcastHashJoin Inner BuildRight (23) + : : :- * Project (21) + : : : +- * BroadcastHashJoin Inner BuildRight (20) + : : : :- * ColumnarToRow (18) + : : : : +- CometFilter (17) + : : : : +- CometScan parquet spark_catalog.default.store_returns (16) + : : : +- ReusedExchange (19) + : : +- ReusedExchange (22) + : :- * Project (49) + : : +- * BroadcastNestedLoopJoin Inner BuildLeft (48) + : : :- BroadcastExchange (39) + : : : +- * HashAggregate (38) + : : : +- Exchange (37) + : : : +- * HashAggregate (36) + : : : +- * Project (35) + : : : +- * BroadcastHashJoin Inner BuildRight (34) + : : : :- * ColumnarToRow (32) + : : : : +- CometScan parquet spark_catalog.default.catalog_sales (31) + : : : +- ReusedExchange (33) + : : +- * HashAggregate (47) + : : +- Exchange (46) + : : +- * HashAggregate (45) + : : +- * Project (44) + : : +- * BroadcastHashJoin Inner BuildRight (43) + : : :- * ColumnarToRow (41) + : : : +- CometScan parquet spark_catalog.default.catalog_returns (40) + : : +- ReusedExchange (42) + : +- * Project (79) + : +- * BroadcastHashJoin LeftOuter BuildRight (78) + : :- * HashAggregate (64) + : : +- Exchange (63) + : : +- * HashAggregate (62) + : : +- * Project (61) + : : +- * BroadcastHashJoin Inner BuildRight (60) + : : :- * Project (55) + : : : +- * BroadcastHashJoin Inner BuildRight (54) + : : : :- * ColumnarToRow (52) + : : : : +- CometFilter (51) + : : : : +- CometScan parquet spark_catalog.default.web_sales (50) + : : : +- ReusedExchange (53) + : : +- BroadcastExchange (59) + : : +- * ColumnarToRow (58) + : : +- CometFilter (57) + : : +- CometScan parquet spark_catalog.default.web_page (56) + : +- BroadcastExchange (77) + : +- * HashAggregate (76) + : +- Exchange (75) + : +- * HashAggregate (74) + : +- * Project (73) + : +- * BroadcastHashJoin Inner BuildRight (72) + : :- * Project (70) + : : +- * BroadcastHashJoin Inner BuildRight (69) + : : :- * ColumnarToRow (67) + : : : +- CometFilter (66) + : : : +- CometScan parquet spark_catalog.default.web_returns (65) + : : +- ReusedExchange (68) + : +- ReusedExchange (71) + :- * HashAggregate (88) + : +- Exchange (87) + : +- * HashAggregate (86) + : +- * HashAggregate (85) + : +- ReusedExchange (84) + +- * HashAggregate (93) + +- Exchange (92) + +- * HashAggregate (91) + +- * HashAggregate (90) + +- ReusedExchange (89) + + +(1) Scan parquet spark_catalog.default.store_sales +Output [4]: [ss_store_sk#1, ss_ext_sales_price#2, ss_net_profit#3, ss_sold_date_sk#4] +Batched: true +Location: InMemoryFileIndex [] +PartitionFilters: [isnotnull(ss_sold_date_sk#4), dynamicpruningexpression(ss_sold_date_sk#4 IN dynamicpruning#5)] +PushedFilters: [IsNotNull(ss_store_sk)] +ReadSchema: struct<ss_store_sk:int,ss_ext_sales_price:decimal(7,2),ss_net_profit:decimal(7,2)> + +(2) CometFilter +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) + +(3) ColumnarToRow [codegen id : 3] +Input [4]: [ss_store_sk#1, ss_ext_sales_price#2, ss_net_profit#3, ss_sold_date_sk#4] + +(4) ReusedExchange [Reuses operator id: 103] +Output [1]: [d_date_sk#6] + +(5) BroadcastHashJoin [codegen id : 3] +Left keys [1]: [ss_sold_date_sk#4] +Right keys [1]: [d_date_sk#6] +Join type: Inner +Join condition: None + +(6) Project [codegen id : 3] +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] + +(7) Scan parquet spark_catalog.default.store +Output [1]: [s_store_sk#7] +Batched: true +Location [not included in comparison]/{warehouse_dir}/store] +PushedFilters: [IsNotNull(s_store_sk)] +ReadSchema: struct<s_store_sk:int> + +(8) CometFilter +Input [1]: [s_store_sk#7] +Condition : isnotnull(s_store_sk#7) + +(9) ColumnarToRow [codegen id : 2] +Input [1]: [s_store_sk#7] + +(10) BroadcastExchange +Input [1]: [s_store_sk#7] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=1] + +(11) BroadcastHashJoin [codegen id : 3] +Left keys [1]: [ss_store_sk#1] +Right keys [1]: [s_store_sk#7] +Join type: Inner +Join condition: None + +(12) Project [codegen id : 3] +Output [3]: [ss_ext_sales_price#2, ss_net_profit#3, s_store_sk#7] +Input [4]: [ss_store_sk#1, ss_ext_sales_price#2, ss_net_profit#3, s_store_sk#7] + +(13) HashAggregate [codegen id : 3] +Input [3]: [ss_ext_sales_price#2, ss_net_profit#3, s_store_sk#7] +Keys [1]: [s_store_sk#7] +Functions [2]: [partial_sum(UnscaledValue(ss_ext_sales_price#2)), partial_sum(UnscaledValue(ss_net_profit#3))] +Aggregate Attributes [2]: [sum#8, sum#9] +Results [3]: [s_store_sk#7, sum#10, sum#11] + +(14) Exchange +Input [3]: [s_store_sk#7, sum#10, sum#11] +Arguments: hashpartitioning(s_store_sk#7, 5), ENSURE_REQUIREMENTS, [plan_id=2] + +(15) HashAggregate [codegen id : 8] +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))] +Aggregate Attributes [2]: [sum(UnscaledValue(ss_ext_sales_price#2))#12, sum(UnscaledValue(ss_net_profit#3))#13] +Results [3]: [s_store_sk#7, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#2))#12,17,2) AS sales#14, MakeDecimal(sum(UnscaledValue(ss_net_profit#3))#13,17,2) AS profit#15] + +(16) Scan parquet spark_catalog.default.store_returns +Output [4]: [sr_store_sk#16, sr_return_amt#17, sr_net_loss#18, sr_returned_date_sk#19] +Batched: true +Location: InMemoryFileIndex [] +PartitionFilters: [isnotnull(sr_returned_date_sk#19), dynamicpruningexpression(sr_returned_date_sk#19 IN dynamicpruning#20)] +PushedFilters: [IsNotNull(sr_store_sk)] +ReadSchema: struct<sr_store_sk:int,sr_return_amt:decimal(7,2),sr_net_loss:decimal(7,2)> + +(17) CometFilter +Input [4]: [sr_store_sk#16, sr_return_amt#17, sr_net_loss#18, sr_returned_date_sk#19] +Condition : isnotnull(sr_store_sk#16) + +(18) ColumnarToRow [codegen id : 6] +Input [4]: [sr_store_sk#16, sr_return_amt#17, sr_net_loss#18, sr_returned_date_sk#19] + +(19) ReusedExchange [Reuses operator id: 103] +Output [1]: [d_date_sk#21] + +(20) BroadcastHashJoin [codegen id : 6] +Left keys [1]: [sr_returned_date_sk#19] +Right keys [1]: [d_date_sk#21] +Join type: Inner +Join condition: None + +(21) Project [codegen id : 6] +Output [3]: [sr_store_sk#16, sr_return_amt#17, sr_net_loss#18] +Input [5]: [sr_store_sk#16, sr_return_amt#17, sr_net_loss#18, sr_returned_date_sk#19, d_date_sk#21] + +(22) ReusedExchange [Reuses operator id: 10] +Output [1]: [s_store_sk#22] + +(23) BroadcastHashJoin [codegen id : 6] +Left keys [1]: [sr_store_sk#16] +Right keys [1]: [s_store_sk#22] +Join type: Inner +Join condition: None + +(24) Project [codegen id : 6] +Output [3]: [sr_return_amt#17, sr_net_loss#18, s_store_sk#22] +Input [4]: [sr_store_sk#16, sr_return_amt#17, sr_net_loss#18, s_store_sk#22] + +(25) HashAggregate [codegen id : 6] +Input [3]: [sr_return_amt#17, sr_net_loss#18, s_store_sk#22] +Keys [1]: [s_store_sk#22] +Functions [2]: [partial_sum(UnscaledValue(sr_return_amt#17)), partial_sum(UnscaledValue(sr_net_loss#18))] +Aggregate Attributes [2]: [sum#23, sum#24] +Results [3]: [s_store_sk#22, sum#25, sum#26] + +(26) Exchange +Input [3]: [s_store_sk#22, sum#25, sum#26] +Arguments: hashpartitioning(s_store_sk#22, 5), ENSURE_REQUIREMENTS, [plan_id=3] + +(27) HashAggregate [codegen id : 7] +Input [3]: [s_store_sk#22, sum#25, sum#26] +Keys [1]: [s_store_sk#22] +Functions [2]: [sum(UnscaledValue(sr_return_amt#17)), sum(UnscaledValue(sr_net_loss#18))] +Aggregate Attributes [2]: [sum(UnscaledValue(sr_return_amt#17))#27, sum(UnscaledValue(sr_net_loss#18))#28] +Results [3]: [s_store_sk#22, MakeDecimal(sum(UnscaledValue(sr_return_amt#17))#27,17,2) AS returns#29, MakeDecimal(sum(UnscaledValue(sr_net_loss#18))#28,17,2) AS profit_loss#30] + +(28) BroadcastExchange +Input [3]: [s_store_sk#22, returns#29, profit_loss#30] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] + +(29) BroadcastHashJoin [codegen id : 8] +Left keys [1]: [s_store_sk#7] +Right keys [1]: [s_store_sk#22] +Join type: LeftOuter +Join condition: None + +(30) Project [codegen id : 8] +Output [5]: [store channel AS channel#31, s_store_sk#7 AS id#32, sales#14, coalesce(returns#29, 0.00) AS returns#33, (profit#15 - coalesce(profit_loss#30, 0.00)) AS profit#34] +Input [6]: [s_store_sk#7, sales#14, profit#15, s_store_sk#22, returns#29, profit_loss#30] + +(31) Scan parquet spark_catalog.default.catalog_sales +Output [4]: [cs_call_center_sk#35, cs_ext_sales_price#36, cs_net_profit#37, cs_sold_date_sk#38] +Batched: true +Location: InMemoryFileIndex [] +PartitionFilters: [isnotnull(cs_sold_date_sk#38), dynamicpruningexpression(cs_sold_date_sk#38 IN dynamicpruning#39)] +ReadSchema: struct<cs_call_center_sk:int,cs_ext_sales_price:decimal(7,2),cs_net_profit:decimal(7,2)> + +(32) ColumnarToRow [codegen id : 10] +Input [4]: [cs_call_center_sk#35, cs_ext_sales_price#36, cs_net_profit#37, cs_sold_date_sk#38] + +(33) ReusedExchange [Reuses operator id: 103] +Output [1]: [d_date_sk#40] + +(34) BroadcastHashJoin [codegen id : 10] +Left keys [1]: [cs_sold_date_sk#38] +Right keys [1]: [d_date_sk#40] +Join type: Inner +Join condition: None + +(35) Project [codegen id : 10] +Output [3]: [cs_call_center_sk#35, cs_ext_sales_price#36, cs_net_profit#37] +Input [5]: [cs_call_center_sk#35, cs_ext_sales_price#36, cs_net_profit#37, cs_sold_date_sk#38, d_date_sk#40] + +(36) HashAggregate [codegen id : 10] +Input [3]: [cs_call_center_sk#35, cs_ext_sales_price#36, cs_net_profit#37] +Keys [1]: [cs_call_center_sk#35] +Functions [2]: [partial_sum(UnscaledValue(cs_ext_sales_price#36)), partial_sum(UnscaledValue(cs_net_profit#37))] +Aggregate Attributes [2]: [sum#41, sum#42] +Results [3]: [cs_call_center_sk#35, sum#43, sum#44] + +(37) Exchange +Input [3]: [cs_call_center_sk#35, sum#43, sum#44] +Arguments: hashpartitioning(cs_call_center_sk#35, 5), ENSURE_REQUIREMENTS, [plan_id=5] + +(38) HashAggregate [codegen id : 11] +Input [3]: [cs_call_center_sk#35, sum#43, sum#44] +Keys [1]: [cs_call_center_sk#35] +Functions [2]: [sum(UnscaledValue(cs_ext_sales_price#36)), sum(UnscaledValue(cs_net_profit#37))] +Aggregate Attributes [2]: [sum(UnscaledValue(cs_ext_sales_price#36))#45, sum(UnscaledValue(cs_net_profit#37))#46] +Results [3]: [cs_call_center_sk#35, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#36))#45,17,2) AS sales#47, MakeDecimal(sum(UnscaledValue(cs_net_profit#37))#46,17,2) AS profit#48] + +(39) BroadcastExchange +Input [3]: [cs_call_center_sk#35, sales#47, profit#48] +Arguments: IdentityBroadcastMode, [plan_id=6] + +(40) Scan parquet spark_catalog.default.catalog_returns +Output [3]: [cr_return_amount#49, cr_net_loss#50, cr_returned_date_sk#51] +Batched: true +Location: InMemoryFileIndex [] +PartitionFilters: [isnotnull(cr_returned_date_sk#51), dynamicpruningexpression(cr_returned_date_sk#51 IN dynamicpruning#52)] +ReadSchema: struct<cr_return_amount:decimal(7,2),cr_net_loss:decimal(7,2)> + +(41) ColumnarToRow [codegen id : 13] +Input [3]: [cr_return_amount#49, cr_net_loss#50, cr_returned_date_sk#51] + +(42) ReusedExchange [Reuses operator id: 103] +Output [1]: [d_date_sk#53] + +(43) BroadcastHashJoin [codegen id : 13] +Left keys [1]: [cr_returned_date_sk#51] +Right keys [1]: [d_date_sk#53] +Join type: Inner +Join condition: None + +(44) Project [codegen id : 13] +Output [2]: [cr_return_amount#49, cr_net_loss#50] +Input [4]: [cr_return_amount#49, cr_net_loss#50, cr_returned_date_sk#51, d_date_sk#53] + +(45) HashAggregate [codegen id : 13] +Input [2]: [cr_return_amount#49, cr_net_loss#50] +Keys: [] +Functions [2]: [partial_sum(UnscaledValue(cr_return_amount#49)), partial_sum(UnscaledValue(cr_net_loss#50))] +Aggregate Attributes [2]: [sum#54, sum#55] +Results [2]: [sum#56, sum#57] + +(46) Exchange +Input [2]: [sum#56, sum#57] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=7] + +(47) HashAggregate +Input [2]: [sum#56, sum#57] +Keys: [] +Functions [2]: [sum(UnscaledValue(cr_return_amount#49)), sum(UnscaledValue(cr_net_loss#50))] +Aggregate Attributes [2]: [sum(UnscaledValue(cr_return_amount#49))#58, sum(UnscaledValue(cr_net_loss#50))#59] +Results [2]: [MakeDecimal(sum(UnscaledValue(cr_return_amount#49))#58,17,2) AS returns#60, MakeDecimal(sum(UnscaledValue(cr_net_loss#50))#59,17,2) AS profit_loss#61] + +(48) BroadcastNestedLoopJoin [codegen id : 14] +Join type: Inner +Join condition: None + +(49) Project [codegen id : 14] +Output [5]: [catalog channel AS channel#62, cs_call_center_sk#35 AS id#63, sales#47, returns#60, (profit#48 - profit_loss#61) AS profit#64] +Input [5]: [cs_call_center_sk#35, sales#47, profit#48, returns#60, profit_loss#61] + +(50) Scan parquet spark_catalog.default.web_sales +Output [4]: [ws_web_page_sk#65, ws_ext_sales_price#66, ws_net_profit#67, ws_sold_date_sk#68] +Batched: true +Location: InMemoryFileIndex [] +PartitionFilters: [isnotnull(ws_sold_date_sk#68), dynamicpruningexpression(ws_sold_date_sk#68 IN dynamicpruning#69)] +PushedFilters: [IsNotNull(ws_web_page_sk)] +ReadSchema: struct<ws_web_page_sk:int,ws_ext_sales_price:decimal(7,2),ws_net_profit:decimal(7,2)> + +(51) CometFilter +Input [4]: [ws_web_page_sk#65, ws_ext_sales_price#66, ws_net_profit#67, ws_sold_date_sk#68] +Condition : isnotnull(ws_web_page_sk#65) + +(52) ColumnarToRow [codegen id : 17] +Input [4]: [ws_web_page_sk#65, ws_ext_sales_price#66, ws_net_profit#67, ws_sold_date_sk#68] + +(53) ReusedExchange [Reuses operator id: 103] +Output [1]: [d_date_sk#70] + +(54) BroadcastHashJoin [codegen id : 17] +Left keys [1]: [ws_sold_date_sk#68] +Right keys [1]: [d_date_sk#70] +Join type: Inner +Join condition: None + +(55) Project [codegen id : 17] +Output [3]: [ws_web_page_sk#65, ws_ext_sales_price#66, ws_net_profit#67] +Input [5]: [ws_web_page_sk#65, ws_ext_sales_price#66, ws_net_profit#67, ws_sold_date_sk#68, d_date_sk#70] + +(56) Scan parquet spark_catalog.default.web_page +Output [1]: [wp_web_page_sk#71] +Batched: true +Location [not included in comparison]/{warehouse_dir}/web_page] +PushedFilters: [IsNotNull(wp_web_page_sk)] +ReadSchema: struct<wp_web_page_sk:int> + +(57) CometFilter +Input [1]: [wp_web_page_sk#71] +Condition : isnotnull(wp_web_page_sk#71) + +(58) ColumnarToRow [codegen id : 16] +Input [1]: [wp_web_page_sk#71] + +(59) BroadcastExchange +Input [1]: [wp_web_page_sk#71] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=8] + +(60) BroadcastHashJoin [codegen id : 17] +Left keys [1]: [ws_web_page_sk#65] +Right keys [1]: [wp_web_page_sk#71] +Join type: Inner +Join condition: None + +(61) Project [codegen id : 17] +Output [3]: [ws_ext_sales_price#66, ws_net_profit#67, wp_web_page_sk#71] +Input [4]: [ws_web_page_sk#65, ws_ext_sales_price#66, ws_net_profit#67, wp_web_page_sk#71] + +(62) HashAggregate [codegen id : 17] +Input [3]: [ws_ext_sales_price#66, ws_net_profit#67, wp_web_page_sk#71] +Keys [1]: [wp_web_page_sk#71] +Functions [2]: [partial_sum(UnscaledValue(ws_ext_sales_price#66)), partial_sum(UnscaledValue(ws_net_profit#67))] +Aggregate Attributes [2]: [sum#72, sum#73] +Results [3]: [wp_web_page_sk#71, sum#74, sum#75] + +(63) Exchange +Input [3]: [wp_web_page_sk#71, sum#74, sum#75] +Arguments: hashpartitioning(wp_web_page_sk#71, 5), ENSURE_REQUIREMENTS, [plan_id=9] + +(64) HashAggregate [codegen id : 22] +Input [3]: [wp_web_page_sk#71, sum#74, sum#75] +Keys [1]: [wp_web_page_sk#71] +Functions [2]: [sum(UnscaledValue(ws_ext_sales_price#66)), sum(UnscaledValue(ws_net_profit#67))] +Aggregate Attributes [2]: [sum(UnscaledValue(ws_ext_sales_price#66))#76, sum(UnscaledValue(ws_net_profit#67))#77] +Results [3]: [wp_web_page_sk#71, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#66))#76,17,2) AS sales#78, MakeDecimal(sum(UnscaledValue(ws_net_profit#67))#77,17,2) AS profit#79] + +(65) Scan parquet spark_catalog.default.web_returns +Output [4]: [wr_web_page_sk#80, wr_return_amt#81, wr_net_loss#82, wr_returned_date_sk#83] +Batched: true +Location: InMemoryFileIndex [] +PartitionFilters: [isnotnull(wr_returned_date_sk#83), dynamicpruningexpression(wr_returned_date_sk#83 IN dynamicpruning#84)] +PushedFilters: [IsNotNull(wr_web_page_sk)] +ReadSchema: struct<wr_web_page_sk:int,wr_return_amt:decimal(7,2),wr_net_loss:decimal(7,2)> + +(66) CometFilter +Input [4]: [wr_web_page_sk#80, wr_return_amt#81, wr_net_loss#82, wr_returned_date_sk#83] +Condition : isnotnull(wr_web_page_sk#80) + +(67) ColumnarToRow [codegen id : 20] +Input [4]: [wr_web_page_sk#80, wr_return_amt#81, wr_net_loss#82, wr_returned_date_sk#83] + +(68) ReusedExchange [Reuses operator id: 103] +Output [1]: [d_date_sk#85] + +(69) BroadcastHashJoin [codegen id : 20] +Left keys [1]: [wr_returned_date_sk#83] +Right keys [1]: [d_date_sk#85] +Join type: Inner +Join condition: None + +(70) Project [codegen id : 20] +Output [3]: [wr_web_page_sk#80, wr_return_amt#81, wr_net_loss#82] +Input [5]: [wr_web_page_sk#80, wr_return_amt#81, wr_net_loss#82, wr_returned_date_sk#83, d_date_sk#85] + +(71) ReusedExchange [Reuses operator id: 59] +Output [1]: [wp_web_page_sk#86] + +(72) BroadcastHashJoin [codegen id : 20] +Left keys [1]: [wr_web_page_sk#80] +Right keys [1]: [wp_web_page_sk#86] +Join type: Inner +Join condition: None + +(73) Project [codegen id : 20] +Output [3]: [wr_return_amt#81, wr_net_loss#82, wp_web_page_sk#86] +Input [4]: [wr_web_page_sk#80, wr_return_amt#81, wr_net_loss#82, wp_web_page_sk#86] + +(74) HashAggregate [codegen id : 20] +Input [3]: [wr_return_amt#81, wr_net_loss#82, wp_web_page_sk#86] +Keys [1]: [wp_web_page_sk#86] +Functions [2]: [partial_sum(UnscaledValue(wr_return_amt#81)), partial_sum(UnscaledValue(wr_net_loss#82))] +Aggregate Attributes [2]: [sum#87, sum#88] +Results [3]: [wp_web_page_sk#86, sum#89, sum#90] + +(75) Exchange +Input [3]: [wp_web_page_sk#86, sum#89, sum#90] +Arguments: hashpartitioning(wp_web_page_sk#86, 5), ENSURE_REQUIREMENTS, [plan_id=10] + +(76) HashAggregate [codegen id : 21] +Input [3]: [wp_web_page_sk#86, sum#89, sum#90] +Keys [1]: [wp_web_page_sk#86] +Functions [2]: [sum(UnscaledValue(wr_return_amt#81)), sum(UnscaledValue(wr_net_loss#82))] +Aggregate Attributes [2]: [sum(UnscaledValue(wr_return_amt#81))#91, sum(UnscaledValue(wr_net_loss#82))#92] +Results [3]: [wp_web_page_sk#86, MakeDecimal(sum(UnscaledValue(wr_return_amt#81))#91,17,2) AS returns#93, MakeDecimal(sum(UnscaledValue(wr_net_loss#82))#92,17,2) AS profit_loss#94] + +(77) BroadcastExchange +Input [3]: [wp_web_page_sk#86, returns#93, profit_loss#94] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=11] + +(78) BroadcastHashJoin [codegen id : 22] +Left keys [1]: [wp_web_page_sk#71] +Right keys [1]: [wp_web_page_sk#86] +Join type: LeftOuter +Join condition: None + +(79) Project [codegen id : 22] +Output [5]: [web channel AS channel#95, wp_web_page_sk#71 AS id#96, sales#78, coalesce(returns#93, 0.00) AS returns#97, (profit#79 - coalesce(profit_loss#94, 0.00)) AS profit#98] +Input [6]: [wp_web_page_sk#71, sales#78, profit#79, wp_web_page_sk#86, returns#93, profit_loss#94] + +(80) Union + +(81) HashAggregate [codegen id : 23] +Input [5]: [channel#31, id#32, sales#14, returns#33, profit#34] +Keys [2]: [channel#31, id#32] +Functions [3]: [partial_sum(sales#14), partial_sum(returns#33), partial_sum(profit#34)] +Aggregate Attributes [6]: [sum#99, isEmpty#100, sum#101, isEmpty#102, sum#103, isEmpty#104] +Results [8]: [channel#31, id#32, sum#105, isEmpty#106, sum#107, isEmpty#108, sum#109, isEmpty#110] + +(82) Exchange +Input [8]: [channel#31, id#32, sum#105, isEmpty#106, sum#107, isEmpty#108, sum#109, isEmpty#110] +Arguments: hashpartitioning(channel#31, id#32, 5), ENSURE_REQUIREMENTS, [plan_id=12] + +(83) HashAggregate [codegen id : 24] +Input [8]: [channel#31, id#32, sum#105, isEmpty#106, sum#107, isEmpty#108, sum#109, isEmpty#110] +Keys [2]: [channel#31, id#32] +Functions [3]: [sum(sales#14), sum(returns#33), sum(profit#34)] +Aggregate Attributes [3]: [sum(sales#14)#111, sum(returns#33)#112, sum(profit#34)#113] +Results [5]: [channel#31, id#32, cast(sum(sales#14)#111 as decimal(37,2)) AS sales#114, cast(sum(returns#33)#112 as decimal(37,2)) AS returns#115, cast(sum(profit#34)#113 as decimal(38,2)) AS profit#116] + +(84) ReusedExchange [Reuses operator id: 82] +Output [8]: [channel#117, id#118, sum#119, isEmpty#120, sum#121, isEmpty#122, sum#123, isEmpty#124] + +(85) HashAggregate [codegen id : 48] +Input [8]: [channel#117, id#118, sum#119, isEmpty#120, sum#121, isEmpty#122, sum#123, isEmpty#124] +Keys [2]: [channel#117, id#118] +Functions [3]: [sum(sales#125), sum(returns#126), sum(profit#127)] +Aggregate Attributes [3]: [sum(sales#125)#111, sum(returns#126)#112, sum(profit#127)#113] +Results [4]: [channel#117, sum(sales#125)#111 AS sales#128, sum(returns#126)#112 AS returns#129, sum(profit#127)#113 AS profit#130] + +(86) HashAggregate [codegen id : 48] +Input [4]: [channel#117, sales#128, returns#129, profit#130] +Keys [1]: [channel#117] +Functions [3]: [partial_sum(sales#128), partial_sum(returns#129), partial_sum(profit#130)] +Aggregate Attributes [6]: [sum#131, isEmpty#132, sum#133, isEmpty#134, sum#135, isEmpty#136] +Results [7]: [channel#117, sum#137, isEmpty#138, sum#139, isEmpty#140, sum#141, isEmpty#142] + +(87) Exchange +Input [7]: [channel#117, sum#137, isEmpty#138, sum#139, isEmpty#140, sum#141, isEmpty#142] +Arguments: hashpartitioning(channel#117, 5), ENSURE_REQUIREMENTS, [plan_id=13] + +(88) HashAggregate [codegen id : 49] +Input [7]: [channel#117, sum#137, isEmpty#138, sum#139, isEmpty#140, sum#141, isEmpty#142] +Keys [1]: [channel#117] +Functions [3]: [sum(sales#128), sum(returns#129), sum(profit#130)] +Aggregate Attributes [3]: [sum(sales#128)#143, sum(returns#129)#144, sum(profit#130)#145] +Results [5]: [channel#117, null AS id#146, sum(sales#128)#143 AS sales#147, sum(returns#129)#144 AS returns#148, sum(profit#130)#145 AS profit#149] + +(89) ReusedExchange [Reuses operator id: 82] +Output [8]: [channel#150, id#151, sum#152, isEmpty#153, sum#154, isEmpty#155, sum#156, isEmpty#157] + +(90) HashAggregate [codegen id : 73] +Input [8]: [channel#150, id#151, sum#152, isEmpty#153, sum#154, isEmpty#155, sum#156, isEmpty#157] +Keys [2]: [channel#150, id#151] +Functions [3]: [sum(sales#158), sum(returns#159), sum(profit#160)] +Aggregate Attributes [3]: [sum(sales#158)#111, sum(returns#159)#112, sum(profit#160)#113] +Results [3]: [sum(sales#158)#111 AS sales#161, sum(returns#159)#112 AS returns#162, sum(profit#160)#113 AS profit#163] + +(91) HashAggregate [codegen id : 73] +Input [3]: [sales#161, returns#162, profit#163] +Keys: [] +Functions [3]: [partial_sum(sales#161), partial_sum(returns#162), partial_sum(profit#163)] +Aggregate Attributes [6]: [sum#164, isEmpty#165, sum#166, isEmpty#167, sum#168, isEmpty#169] +Results [6]: [sum#170, isEmpty#171, sum#172, isEmpty#173, sum#174, isEmpty#175] + +(92) Exchange +Input [6]: [sum#170, isEmpty#171, sum#172, isEmpty#173, sum#174, isEmpty#175] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=14] + +(93) HashAggregate [codegen id : 74] +Input [6]: [sum#170, isEmpty#171, sum#172, isEmpty#173, sum#174, isEmpty#175] +Keys: [] +Functions [3]: [sum(sales#161), sum(returns#162), sum(profit#163)] +Aggregate Attributes [3]: [sum(sales#161)#176, sum(returns#162)#177, sum(profit#163)#178] +Results [5]: [null AS channel#179, null AS id#180, sum(sales#161)#176 AS sales#181, sum(returns#162)#177 AS returns#182, sum(profit#163)#178 AS profit#183] + +(94) Union + +(95) HashAggregate [codegen id : 75] +Input [5]: [channel#31, id#32, sales#114, returns#115, profit#116] +Keys [5]: [channel#31, id#32, sales#114, returns#115, profit#116] +Functions: [] +Aggregate Attributes: [] +Results [5]: [channel#31, id#32, sales#114, returns#115, profit#116] + +(96) Exchange +Input [5]: [channel#31, id#32, sales#114, returns#115, profit#116] +Arguments: hashpartitioning(channel#31, id#32, sales#114, returns#115, profit#116, 5), ENSURE_REQUIREMENTS, [plan_id=15] + +(97) HashAggregate [codegen id : 76] +Input [5]: [channel#31, id#32, sales#114, returns#115, profit#116] +Keys [5]: [channel#31, id#32, sales#114, returns#115, profit#116] +Functions: [] +Aggregate Attributes: [] +Results [5]: [channel#31, id#32, sales#114, returns#115, profit#116] + +(98) TakeOrderedAndProject +Input [5]: [channel#31, id#32, sales#114, returns#115, profit#116] +Arguments: 100, [channel#31 ASC NULLS FIRST, id#32 ASC NULLS FIRST], [channel#31, id#32, sales#114, returns#115, profit#116] + +===== Subqueries ===== + +Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 +BroadcastExchange (103) ++- * ColumnarToRow (102) + +- CometProject (101) + +- CometFilter (100) + +- CometScan parquet spark_catalog.default.date_dim (99) + + +(99) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#6, d_date#184] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,1998-08-04), LessThanOrEqual(d_date,1998-09-03), IsNotNull(d_date_sk)] +ReadSchema: struct<d_date_sk:int,d_date:date> + +(100) CometFilter +Input [2]: [d_date_sk#6, d_date#184] +Condition : (((isnotnull(d_date#184) AND (d_date#184 >= 1998-08-04)) AND (d_date#184 <= 1998-09-03)) AND isnotnull(d_date_sk#6)) + +(101) CometProject +Input [2]: [d_date_sk#6, d_date#184] +Arguments: [d_date_sk#6], [d_date_sk#6] + +(102) ColumnarToRow [codegen id : 1] +Input [1]: [d_date_sk#6] + +(103) BroadcastExchange +Input [1]: [d_date_sk#6] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=16] + +Subquery:2 Hosting operator id = 16 Hosting Expression = sr_returned_date_sk#19 IN dynamicpruning#5 + +Subquery:3 Hosting operator id = 31 Hosting Expression = cs_sold_date_sk#38 IN dynamicpruning#5 + +Subquery:4 Hosting operator id = 40 Hosting Expression = cr_returned_date_sk#51 IN dynamicpruning#5 + +Subquery:5 Hosting operator id = 50 Hosting Expression = ws_sold_date_sk#68 IN dynamicpruning#5 + +Subquery:6 Hosting operator id = 65 Hosting Expression = wr_returned_date_sk#83 IN dynamicpruning#5 + + diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q77a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q77a/simplified.txt new file mode 100644 index 000000000..670a7e6c3 --- /dev/null +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q77a/simplified.txt @@ -0,0 +1,168 @@ +TakeOrderedAndProject [channel,id,sales,returns,profit] + WholeStageCodegen (76) + HashAggregate [channel,id,sales,returns,profit] + InputAdapter + Exchange [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] + InputAdapter + Exchange [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] + InputAdapter + Exchange [s_store_sk] #3 + WholeStageCodegen (3) + HashAggregate [s_store_sk,ss_ext_sales_price,ss_net_profit] [sum,sum,sum,sum] + Project [ss_ext_sales_price,ss_net_profit,s_store_sk] + BroadcastHashJoin [ss_store_sk,s_store_sk] + Project [ss_store_sk,ss_ext_sales_price,ss_net_profit] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + ColumnarToRow + InputAdapter + CometFilter [ss_store_sk] + CometScan parquet spark_catalog.default.store_sales [ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #4 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] + InputAdapter + ReusedExchange [d_date_sk] #4 + InputAdapter + BroadcastExchange #5 + WholeStageCodegen (2) + ColumnarToRow + InputAdapter + CometFilter [s_store_sk] + CometScan 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] + InputAdapter + Exchange [s_store_sk] #7 + WholeStageCodegen (6) + HashAggregate [s_store_sk,sr_return_amt,sr_net_loss] [sum,sum,sum,sum] + Project [sr_return_amt,sr_net_loss,s_store_sk] + BroadcastHashJoin [sr_store_sk,s_store_sk] + Project [sr_store_sk,sr_return_amt,sr_net_loss] + BroadcastHashJoin [sr_returned_date_sk,d_date_sk] + ColumnarToRow + InputAdapter + CometFilter [sr_store_sk] + CometScan parquet spark_catalog.default.store_returns [sr_store_sk,sr_return_amt,sr_net_loss,sr_returned_date_sk] + ReusedSubquery [d_date_sk] #1 + InputAdapter + ReusedExchange [d_date_sk] #4 + InputAdapter + ReusedExchange [s_store_sk] #5 + WholeStageCodegen (14) + 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] + InputAdapter + Exchange [cs_call_center_sk] #9 + WholeStageCodegen (10) + HashAggregate [cs_call_center_sk,cs_ext_sales_price,cs_net_profit] [sum,sum,sum,sum] + Project [cs_call_center_sk,cs_ext_sales_price,cs_net_profit] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + ColumnarToRow + InputAdapter + CometScan parquet spark_catalog.default.catalog_sales [cs_call_center_sk,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + InputAdapter + ReusedExchange [d_date_sk] #4 + HashAggregate [sum,sum] [sum(UnscaledValue(cr_return_amount)),sum(UnscaledValue(cr_net_loss)),returns,profit_loss,sum,sum] + InputAdapter + Exchange #10 + WholeStageCodegen (13) + HashAggregate [cr_return_amount,cr_net_loss] [sum,sum,sum,sum] + Project [cr_return_amount,cr_net_loss] + BroadcastHashJoin [cr_returned_date_sk,d_date_sk] + ColumnarToRow + InputAdapter + CometScan parquet spark_catalog.default.catalog_returns [cr_return_amount,cr_net_loss,cr_returned_date_sk] + ReusedSubquery [d_date_sk] #1 + InputAdapter + ReusedExchange [d_date_sk] #4 + WholeStageCodegen (22) + Project [wp_web_page_sk,sales,returns,profit,profit_loss] + BroadcastHashJoin [wp_web_page_sk,wp_web_page_sk] + HashAggregate [wp_web_page_sk,sum,sum] [sum(UnscaledValue(ws_ext_sales_price)),sum(UnscaledValue(ws_net_profit)),sales,profit,sum,sum] + InputAdapter + Exchange [wp_web_page_sk] #11 + WholeStageCodegen (17) + HashAggregate [wp_web_page_sk,ws_ext_sales_price,ws_net_profit] [sum,sum,sum,sum] + Project [ws_ext_sales_price,ws_net_profit,wp_web_page_sk] + BroadcastHashJoin [ws_web_page_sk,wp_web_page_sk] + Project [ws_web_page_sk,ws_ext_sales_price,ws_net_profit] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + ColumnarToRow + InputAdapter + CometFilter [ws_web_page_sk] + CometScan parquet spark_catalog.default.web_sales [ws_web_page_sk,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + InputAdapter + ReusedExchange [d_date_sk] #4 + InputAdapter + BroadcastExchange #12 + WholeStageCodegen (16) + ColumnarToRow + InputAdapter + CometFilter [wp_web_page_sk] + CometScan parquet spark_catalog.default.web_page [wp_web_page_sk] + 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] + InputAdapter + Exchange [wp_web_page_sk] #14 + WholeStageCodegen (20) + HashAggregate [wp_web_page_sk,wr_return_amt,wr_net_loss] [sum,sum,sum,sum] + Project [wr_return_amt,wr_net_loss,wp_web_page_sk] + BroadcastHashJoin [wr_web_page_sk,wp_web_page_sk] + Project [wr_web_page_sk,wr_return_amt,wr_net_loss] + BroadcastHashJoin [wr_returned_date_sk,d_date_sk] + ColumnarToRow + InputAdapter + CometFilter [wr_web_page_sk] + CometScan parquet spark_catalog.default.web_returns [wr_web_page_sk,wr_return_amt,wr_net_loss,wr_returned_date_sk] + ReusedSubquery [d_date_sk] #1 + InputAdapter + ReusedExchange [d_date_sk] #4 + InputAdapter + ReusedExchange [wp_web_page_sk] #12 + WholeStageCodegen (49) + HashAggregate [channel,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),id,sales,returns,profit,sum,isEmpty,sum,isEmpty,sum,isEmpty] + InputAdapter + Exchange [channel] #15 + WholeStageCodegen (48) + HashAggregate [channel,sales,returns,profit] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] + HashAggregate [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),sales,returns,profit,sum,isEmpty,sum,isEmpty,sum,isEmpty] + InputAdapter + ReusedExchange [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] #2 + WholeStageCodegen (74) + HashAggregate [sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),channel,id,sales,returns,profit,sum,isEmpty,sum,isEmpty,sum,isEmpty] + InputAdapter + Exchange #16 + WholeStageCodegen (73) + HashAggregate [sales,returns,profit] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] + HashAggregate [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),sales,returns,profit,sum,isEmpty,sum,isEmpty,sum,isEmpty] + InputAdapter + ReusedExchange [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] #2 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 new file mode 100644 index 000000000..c7ee5b1c9 --- /dev/null +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q78/explain.txt @@ -0,0 +1,431 @@ +== Physical Plan == +TakeOrderedAndProject (70) ++- * Project (69) + +- * SortMergeJoin Inner (68) + :- * Project (45) + : +- * SortMergeJoin Inner (44) + : :- * Sort (21) + : : +- * HashAggregate (20) + : : +- Exchange (19) + : : +- * HashAggregate (18) + : : +- * Project (17) + : : +- * BroadcastHashJoin Inner BuildRight (16) + : : :- * Project (14) + : : : +- * Filter (13) + : : : +- * SortMergeJoin LeftOuter (12) + : : : :- * Sort (5) + : : : : +- Exchange (4) + : : : : +- * ColumnarToRow (3) + : : : : +- CometFilter (2) + : : : : +- CometScan parquet spark_catalog.default.store_sales (1) + : : : +- * Sort (11) + : : : +- Exchange (10) + : : : +- * ColumnarToRow (9) + : : : +- CometProject (8) + : : : +- CometFilter (7) + : : : +- CometScan parquet spark_catalog.default.store_returns (6) + : : +- ReusedExchange (15) + : +- * Sort (43) + : +- * Filter (42) + : +- * HashAggregate (41) + : +- Exchange (40) + : +- * HashAggregate (39) + : +- * Project (38) + : +- * BroadcastHashJoin Inner BuildRight (37) + : :- * Project (35) + : : +- * Filter (34) + : : +- * SortMergeJoin LeftOuter (33) + : : :- * Sort (26) + : : : +- Exchange (25) + : : : +- * ColumnarToRow (24) + : : : +- CometFilter (23) + : : : +- CometScan parquet spark_catalog.default.web_sales (22) + : : +- * Sort (32) + : : +- Exchange (31) + : : +- * ColumnarToRow (30) + : : +- CometProject (29) + : : +- CometFilter (28) + : : +- CometScan parquet spark_catalog.default.web_returns (27) + : +- ReusedExchange (36) + +- * Sort (67) + +- * Filter (66) + +- * HashAggregate (65) + +- Exchange (64) + +- * HashAggregate (63) + +- * Project (62) + +- * BroadcastHashJoin Inner BuildRight (61) + :- * Project (59) + : +- * Filter (58) + : +- * SortMergeJoin LeftOuter (57) + : :- * Sort (50) + : : +- Exchange (49) + : : +- * ColumnarToRow (48) + : : +- CometFilter (47) + : : +- CometScan parquet spark_catalog.default.catalog_sales (46) + : +- * Sort (56) + : +- Exchange (55) + : +- * ColumnarToRow (54) + : +- CometProject (53) + : +- CometFilter (52) + : +- CometScan parquet spark_catalog.default.catalog_returns (51) + +- ReusedExchange (60) + + +(1) Scan parquet spark_catalog.default.store_sales +Output [7]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#3, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, ss_sold_date_sk#7] +Batched: true +Location: InMemoryFileIndex [] +PartitionFilters: [isnotnull(ss_sold_date_sk#7), dynamicpruningexpression(ss_sold_date_sk#7 IN dynamicpruning#8)] +PushedFilters: [IsNotNull(ss_item_sk), IsNotNull(ss_customer_sk)] +ReadSchema: struct<ss_item_sk:int,ss_customer_sk:int,ss_ticket_number:int,ss_quantity:int,ss_wholesale_cost:decimal(7,2),ss_sales_price:decimal(7,2)> + +(2) CometFilter +Input [7]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#3, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, ss_sold_date_sk#7] +Condition : (isnotnull(ss_item_sk#1) AND isnotnull(ss_customer_sk#2)) + +(3) ColumnarToRow [codegen id : 1] +Input [7]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#3, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, ss_sold_date_sk#7] + +(4) Exchange +Input [7]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#3, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, ss_sold_date_sk#7] +Arguments: hashpartitioning(ss_ticket_number#3, ss_item_sk#1, 5), ENSURE_REQUIREMENTS, [plan_id=1] + +(5) Sort [codegen id : 2] +Input [7]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#3, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, ss_sold_date_sk#7] +Arguments: [ss_ticket_number#3 ASC NULLS FIRST, ss_item_sk#1 ASC NULLS FIRST], false, 0 + +(6) Scan parquet spark_catalog.default.store_returns +Output [3]: [sr_item_sk#9, sr_ticket_number#10, sr_returned_date_sk#11] +Batched: true +Location [not included in comparison]/{warehouse_dir}/store_returns] +PushedFilters: [IsNotNull(sr_ticket_number), IsNotNull(sr_item_sk)] +ReadSchema: struct<sr_item_sk:int,sr_ticket_number:int> + +(7) CometFilter +Input [3]: [sr_item_sk#9, sr_ticket_number#10, sr_returned_date_sk#11] +Condition : (isnotnull(sr_ticket_number#10) AND isnotnull(sr_item_sk#9)) + +(8) CometProject +Input [3]: [sr_item_sk#9, sr_ticket_number#10, sr_returned_date_sk#11] +Arguments: [sr_item_sk#9, sr_ticket_number#10], [sr_item_sk#9, sr_ticket_number#10] + +(9) ColumnarToRow [codegen id : 3] +Input [2]: [sr_item_sk#9, sr_ticket_number#10] + +(10) Exchange +Input [2]: [sr_item_sk#9, sr_ticket_number#10] +Arguments: hashpartitioning(sr_ticket_number#10, sr_item_sk#9, 5), ENSURE_REQUIREMENTS, [plan_id=2] + +(11) Sort [codegen id : 4] +Input [2]: [sr_item_sk#9, sr_ticket_number#10] +Arguments: [sr_ticket_number#10 ASC NULLS FIRST, sr_item_sk#9 ASC NULLS FIRST], false, 0 + +(12) SortMergeJoin [codegen id : 6] +Left keys [2]: [ss_ticket_number#3, ss_item_sk#1] +Right keys [2]: [sr_ticket_number#10, sr_item_sk#9] +Join type: LeftOuter +Join condition: None + +(13) Filter [codegen id : 6] +Input [9]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#3, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, ss_sold_date_sk#7, sr_item_sk#9, sr_ticket_number#10] +Condition : isnull(sr_ticket_number#10) + +(14) Project [codegen id : 6] +Output [6]: [ss_item_sk#1, ss_customer_sk#2, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, ss_sold_date_sk#7] +Input [9]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#3, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, ss_sold_date_sk#7, sr_item_sk#9, sr_ticket_number#10] + +(15) ReusedExchange [Reuses operator id: 74] +Output [2]: [d_date_sk#12, d_year#13] + +(16) BroadcastHashJoin [codegen id : 6] +Left keys [1]: [ss_sold_date_sk#7] +Right keys [1]: [d_date_sk#12] +Join type: Inner +Join condition: None + +(17) Project [codegen id : 6] +Output [6]: [ss_item_sk#1, ss_customer_sk#2, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, d_year#13] +Input [8]: [ss_item_sk#1, ss_customer_sk#2, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, ss_sold_date_sk#7, d_date_sk#12, d_year#13] + +(18) HashAggregate [codegen id : 6] +Input [6]: [ss_item_sk#1, ss_customer_sk#2, ss_quantity#4, ss_wholesale_cost#5, ss_sales_price#6, d_year#13] +Keys [3]: [d_year#13, ss_item_sk#1, ss_customer_sk#2] +Functions [3]: [partial_sum(ss_quantity#4), partial_sum(UnscaledValue(ss_wholesale_cost#5)), partial_sum(UnscaledValue(ss_sales_price#6))] +Aggregate Attributes [3]: [sum#14, sum#15, sum#16] +Results [6]: [d_year#13, ss_item_sk#1, ss_customer_sk#2, sum#17, sum#18, sum#19] + +(19) Exchange +Input [6]: [d_year#13, ss_item_sk#1, ss_customer_sk#2, sum#17, sum#18, sum#19] +Arguments: hashpartitioning(d_year#13, ss_item_sk#1, ss_customer_sk#2, 5), ENSURE_REQUIREMENTS, [plan_id=3] + +(20) HashAggregate [codegen id : 7] +Input [6]: [d_year#13, ss_item_sk#1, ss_customer_sk#2, sum#17, sum#18, sum#19] +Keys [3]: [d_year#13, ss_item_sk#1, ss_customer_sk#2] +Functions [3]: [sum(ss_quantity#4), sum(UnscaledValue(ss_wholesale_cost#5)), sum(UnscaledValue(ss_sales_price#6))] +Aggregate Attributes [3]: [sum(ss_quantity#4)#20, sum(UnscaledValue(ss_wholesale_cost#5))#21, sum(UnscaledValue(ss_sales_price#6))#22] +Results [6]: [d_year#13 AS ss_sold_year#23, ss_item_sk#1, ss_customer_sk#2, sum(ss_quantity#4)#20 AS ss_qty#24, MakeDecimal(sum(UnscaledValue(ss_wholesale_cost#5))#21,17,2) AS ss_wc#25, MakeDecimal(sum(UnscaledValue(ss_sales_price#6))#22,17,2) AS ss_sp#26] + +(21) Sort [codegen id : 7] +Input [6]: [ss_sold_year#23, ss_item_sk#1, ss_customer_sk#2, ss_qty#24, ss_wc#25, ss_sp#26] +Arguments: [ss_sold_year#23 ASC NULLS FIRST, ss_item_sk#1 ASC NULLS FIRST, ss_customer_sk#2 ASC NULLS FIRST], false, 0 + +(22) Scan parquet spark_catalog.default.web_sales +Output [7]: [ws_item_sk#27, ws_bill_customer_sk#28, ws_order_number#29, ws_quantity#30, ws_wholesale_cost#31, ws_sales_price#32, ws_sold_date_sk#33] +Batched: true +Location: InMemoryFileIndex [] +PartitionFilters: [isnotnull(ws_sold_date_sk#33), dynamicpruningexpression(ws_sold_date_sk#33 IN dynamicpruning#34)] +PushedFilters: [IsNotNull(ws_item_sk), IsNotNull(ws_bill_customer_sk)] +ReadSchema: struct<ws_item_sk:int,ws_bill_customer_sk:int,ws_order_number:int,ws_quantity:int,ws_wholesale_cost:decimal(7,2),ws_sales_price:decimal(7,2)> + +(23) CometFilter +Input [7]: [ws_item_sk#27, ws_bill_customer_sk#28, ws_order_number#29, ws_quantity#30, ws_wholesale_cost#31, ws_sales_price#32, ws_sold_date_sk#33] +Condition : (isnotnull(ws_item_sk#27) AND isnotnull(ws_bill_customer_sk#28)) + +(24) ColumnarToRow [codegen id : 8] +Input [7]: [ws_item_sk#27, ws_bill_customer_sk#28, ws_order_number#29, ws_quantity#30, ws_wholesale_cost#31, ws_sales_price#32, ws_sold_date_sk#33] + +(25) Exchange +Input [7]: [ws_item_sk#27, ws_bill_customer_sk#28, ws_order_number#29, ws_quantity#30, ws_wholesale_cost#31, ws_sales_price#32, ws_sold_date_sk#33] +Arguments: hashpartitioning(ws_order_number#29, ws_item_sk#27, 5), ENSURE_REQUIREMENTS, [plan_id=4] + +(26) Sort [codegen id : 9] +Input [7]: [ws_item_sk#27, ws_bill_customer_sk#28, ws_order_number#29, ws_quantity#30, ws_wholesale_cost#31, ws_sales_price#32, ws_sold_date_sk#33] +Arguments: [ws_order_number#29 ASC NULLS FIRST, ws_item_sk#27 ASC NULLS FIRST], false, 0 + +(27) Scan parquet spark_catalog.default.web_returns +Output [3]: [wr_item_sk#35, wr_order_number#36, wr_returned_date_sk#37] +Batched: true +Location [not included in comparison]/{warehouse_dir}/web_returns] +PushedFilters: [IsNotNull(wr_order_number), IsNotNull(wr_item_sk)] +ReadSchema: struct<wr_item_sk:int,wr_order_number:int> + +(28) CometFilter +Input [3]: [wr_item_sk#35, wr_order_number#36, wr_returned_date_sk#37] +Condition : (isnotnull(wr_order_number#36) AND isnotnull(wr_item_sk#35)) + +(29) CometProject +Input [3]: [wr_item_sk#35, wr_order_number#36, wr_returned_date_sk#37] +Arguments: [wr_item_sk#35, wr_order_number#36], [wr_item_sk#35, wr_order_number#36] + +(30) ColumnarToRow [codegen id : 10] +Input [2]: [wr_item_sk#35, wr_order_number#36] + +(31) Exchange +Input [2]: [wr_item_sk#35, wr_order_number#36] +Arguments: hashpartitioning(wr_order_number#36, wr_item_sk#35, 5), ENSURE_REQUIREMENTS, [plan_id=5] + +(32) Sort [codegen id : 11] +Input [2]: [wr_item_sk#35, wr_order_number#36] +Arguments: [wr_order_number#36 ASC NULLS FIRST, wr_item_sk#35 ASC NULLS FIRST], false, 0 + +(33) SortMergeJoin [codegen id : 13] +Left keys [2]: [ws_order_number#29, ws_item_sk#27] +Right keys [2]: [wr_order_number#36, wr_item_sk#35] +Join type: LeftOuter +Join condition: None + +(34) Filter [codegen id : 13] +Input [9]: [ws_item_sk#27, ws_bill_customer_sk#28, ws_order_number#29, ws_quantity#30, ws_wholesale_cost#31, ws_sales_price#32, ws_sold_date_sk#33, wr_item_sk#35, wr_order_number#36] +Condition : isnull(wr_order_number#36) + +(35) Project [codegen id : 13] +Output [6]: [ws_item_sk#27, ws_bill_customer_sk#28, ws_quantity#30, ws_wholesale_cost#31, ws_sales_price#32, ws_sold_date_sk#33] +Input [9]: [ws_item_sk#27, ws_bill_customer_sk#28, ws_order_number#29, ws_quantity#30, ws_wholesale_cost#31, ws_sales_price#32, ws_sold_date_sk#33, wr_item_sk#35, wr_order_number#36] + +(36) ReusedExchange [Reuses operator id: 74] +Output [2]: [d_date_sk#38, d_year#39] + +(37) BroadcastHashJoin [codegen id : 13] +Left keys [1]: [ws_sold_date_sk#33] +Right keys [1]: [d_date_sk#38] +Join type: Inner +Join condition: None + +(38) Project [codegen id : 13] +Output [6]: [ws_item_sk#27, ws_bill_customer_sk#28, ws_quantity#30, ws_wholesale_cost#31, ws_sales_price#32, d_year#39] +Input [8]: [ws_item_sk#27, ws_bill_customer_sk#28, ws_quantity#30, ws_wholesale_cost#31, ws_sales_price#32, ws_sold_date_sk#33, d_date_sk#38, d_year#39] + +(39) HashAggregate [codegen id : 13] +Input [6]: [ws_item_sk#27, ws_bill_customer_sk#28, ws_quantity#30, ws_wholesale_cost#31, ws_sales_price#32, d_year#39] +Keys [3]: [d_year#39, ws_item_sk#27, ws_bill_customer_sk#28] +Functions [3]: [partial_sum(ws_quantity#30), partial_sum(UnscaledValue(ws_wholesale_cost#31)), partial_sum(UnscaledValue(ws_sales_price#32))] +Aggregate Attributes [3]: [sum#40, sum#41, sum#42] +Results [6]: [d_year#39, ws_item_sk#27, ws_bill_customer_sk#28, sum#43, sum#44, sum#45] + +(40) Exchange +Input [6]: [d_year#39, ws_item_sk#27, ws_bill_customer_sk#28, sum#43, sum#44, sum#45] +Arguments: hashpartitioning(d_year#39, ws_item_sk#27, ws_bill_customer_sk#28, 5), ENSURE_REQUIREMENTS, [plan_id=6] + +(41) HashAggregate [codegen id : 14] +Input [6]: [d_year#39, ws_item_sk#27, ws_bill_customer_sk#28, sum#43, sum#44, sum#45] +Keys [3]: [d_year#39, ws_item_sk#27, ws_bill_customer_sk#28] +Functions [3]: [sum(ws_quantity#30), sum(UnscaledValue(ws_wholesale_cost#31)), sum(UnscaledValue(ws_sales_price#32))] +Aggregate Attributes [3]: [sum(ws_quantity#30)#46, sum(UnscaledValue(ws_wholesale_cost#31))#47, sum(UnscaledValue(ws_sales_price#32))#48] +Results [6]: [d_year#39 AS ws_sold_year#49, ws_item_sk#27, ws_bill_customer_sk#28 AS ws_customer_sk#50, sum(ws_quantity#30)#46 AS ws_qty#51, MakeDecimal(sum(UnscaledValue(ws_wholesale_cost#31))#47,17,2) AS ws_wc#52, MakeDecimal(sum(UnscaledValue(ws_sales_price#32))#48,17,2) AS ws_sp#53] + +(42) Filter [codegen id : 14] +Input [6]: [ws_sold_year#49, ws_item_sk#27, ws_customer_sk#50, ws_qty#51, ws_wc#52, ws_sp#53] +Condition : (coalesce(ws_qty#51, 0) > 0) + +(43) Sort [codegen id : 14] +Input [6]: [ws_sold_year#49, ws_item_sk#27, ws_customer_sk#50, ws_qty#51, ws_wc#52, ws_sp#53] +Arguments: [ws_sold_year#49 ASC NULLS FIRST, ws_item_sk#27 ASC NULLS FIRST, ws_customer_sk#50 ASC NULLS FIRST], false, 0 + +(44) SortMergeJoin [codegen id : 15] +Left keys [3]: [ss_sold_year#23, ss_item_sk#1, ss_customer_sk#2] +Right keys [3]: [ws_sold_year#49, ws_item_sk#27, ws_customer_sk#50] +Join type: Inner +Join condition: None + +(45) Project [codegen id : 15] +Output [9]: [ss_sold_year#23, ss_item_sk#1, ss_customer_sk#2, ss_qty#24, ss_wc#25, ss_sp#26, ws_qty#51, ws_wc#52, ws_sp#53] +Input [12]: [ss_sold_year#23, ss_item_sk#1, ss_customer_sk#2, ss_qty#24, ss_wc#25, ss_sp#26, ws_sold_year#49, ws_item_sk#27, ws_customer_sk#50, ws_qty#51, ws_wc#52, ws_sp#53] + +(46) Scan parquet spark_catalog.default.catalog_sales +Output [7]: [cs_bill_customer_sk#54, cs_item_sk#55, cs_order_number#56, cs_quantity#57, cs_wholesale_cost#58, cs_sales_price#59, cs_sold_date_sk#60] +Batched: true +Location: InMemoryFileIndex [] +PartitionFilters: [isnotnull(cs_sold_date_sk#60), dynamicpruningexpression(cs_sold_date_sk#60 IN dynamicpruning#61)] +PushedFilters: [IsNotNull(cs_item_sk), IsNotNull(cs_bill_customer_sk)] +ReadSchema: struct<cs_bill_customer_sk:int,cs_item_sk:int,cs_order_number:int,cs_quantity:int,cs_wholesale_cost:decimal(7,2),cs_sales_price:decimal(7,2)> + +(47) CometFilter +Input [7]: [cs_bill_customer_sk#54, cs_item_sk#55, cs_order_number#56, cs_quantity#57, cs_wholesale_cost#58, cs_sales_price#59, cs_sold_date_sk#60] +Condition : (isnotnull(cs_item_sk#55) AND isnotnull(cs_bill_customer_sk#54)) + +(48) ColumnarToRow [codegen id : 16] +Input [7]: [cs_bill_customer_sk#54, cs_item_sk#55, cs_order_number#56, cs_quantity#57, cs_wholesale_cost#58, cs_sales_price#59, cs_sold_date_sk#60] + +(49) Exchange +Input [7]: [cs_bill_customer_sk#54, cs_item_sk#55, cs_order_number#56, cs_quantity#57, cs_wholesale_cost#58, cs_sales_price#59, cs_sold_date_sk#60] +Arguments: hashpartitioning(cs_order_number#56, cs_item_sk#55, 5), ENSURE_REQUIREMENTS, [plan_id=7] + +(50) Sort [codegen id : 17] +Input [7]: [cs_bill_customer_sk#54, cs_item_sk#55, cs_order_number#56, cs_quantity#57, cs_wholesale_cost#58, cs_sales_price#59, cs_sold_date_sk#60] +Arguments: [cs_order_number#56 ASC NULLS FIRST, cs_item_sk#55 ASC NULLS FIRST], false, 0 + +(51) Scan parquet spark_catalog.default.catalog_returns +Output [3]: [cr_item_sk#62, cr_order_number#63, cr_returned_date_sk#64] +Batched: true +Location [not included in comparison]/{warehouse_dir}/catalog_returns] +PushedFilters: [IsNotNull(cr_order_number), IsNotNull(cr_item_sk)] +ReadSchema: struct<cr_item_sk:int,cr_order_number:int> + +(52) CometFilter +Input [3]: [cr_item_sk#62, cr_order_number#63, cr_returned_date_sk#64] +Condition : (isnotnull(cr_order_number#63) AND isnotnull(cr_item_sk#62)) + +(53) CometProject +Input [3]: [cr_item_sk#62, cr_order_number#63, cr_returned_date_sk#64] +Arguments: [cr_item_sk#62, cr_order_number#63], [cr_item_sk#62, cr_order_number#63] + +(54) ColumnarToRow [codegen id : 18] +Input [2]: [cr_item_sk#62, cr_order_number#63] + +(55) Exchange +Input [2]: [cr_item_sk#62, cr_order_number#63] +Arguments: hashpartitioning(cr_order_number#63, cr_item_sk#62, 5), ENSURE_REQUIREMENTS, [plan_id=8] + +(56) Sort [codegen id : 19] +Input [2]: [cr_item_sk#62, cr_order_number#63] +Arguments: [cr_order_number#63 ASC NULLS FIRST, cr_item_sk#62 ASC NULLS FIRST], false, 0 + +(57) SortMergeJoin [codegen id : 21] +Left keys [2]: [cs_order_number#56, cs_item_sk#55] +Right keys [2]: [cr_order_number#63, cr_item_sk#62] +Join type: LeftOuter +Join condition: None + +(58) Filter [codegen id : 21] +Input [9]: [cs_bill_customer_sk#54, cs_item_sk#55, cs_order_number#56, cs_quantity#57, cs_wholesale_cost#58, cs_sales_price#59, cs_sold_date_sk#60, cr_item_sk#62, cr_order_number#63] +Condition : isnull(cr_order_number#63) + +(59) Project [codegen id : 21] +Output [6]: [cs_bill_customer_sk#54, cs_item_sk#55, cs_quantity#57, cs_wholesale_cost#58, cs_sales_price#59, cs_sold_date_sk#60] +Input [9]: [cs_bill_customer_sk#54, cs_item_sk#55, cs_order_number#56, cs_quantity#57, cs_wholesale_cost#58, cs_sales_price#59, cs_sold_date_sk#60, cr_item_sk#62, cr_order_number#63] + +(60) ReusedExchange [Reuses operator id: 74] +Output [2]: [d_date_sk#65, d_year#66] + +(61) BroadcastHashJoin [codegen id : 21] +Left keys [1]: [cs_sold_date_sk#60] +Right keys [1]: [d_date_sk#65] +Join type: Inner +Join condition: None + +(62) Project [codegen id : 21] +Output [6]: [cs_bill_customer_sk#54, cs_item_sk#55, cs_quantity#57, cs_wholesale_cost#58, cs_sales_price#59, d_year#66] +Input [8]: [cs_bill_customer_sk#54, cs_item_sk#55, cs_quantity#57, cs_wholesale_cost#58, cs_sales_price#59, cs_sold_date_sk#60, d_date_sk#65, d_year#66] + +(63) HashAggregate [codegen id : 21] +Input [6]: [cs_bill_customer_sk#54, cs_item_sk#55, cs_quantity#57, cs_wholesale_cost#58, cs_sales_price#59, d_year#66] +Keys [3]: [d_year#66, cs_item_sk#55, cs_bill_customer_sk#54] +Functions [3]: [partial_sum(cs_quantity#57), partial_sum(UnscaledValue(cs_wholesale_cost#58)), partial_sum(UnscaledValue(cs_sales_price#59))] +Aggregate Attributes [3]: [sum#67, sum#68, sum#69] +Results [6]: [d_year#66, cs_item_sk#55, cs_bill_customer_sk#54, sum#70, sum#71, sum#72] + +(64) Exchange +Input [6]: [d_year#66, cs_item_sk#55, cs_bill_customer_sk#54, sum#70, sum#71, sum#72] +Arguments: hashpartitioning(d_year#66, cs_item_sk#55, cs_bill_customer_sk#54, 5), ENSURE_REQUIREMENTS, [plan_id=9] + +(65) HashAggregate [codegen id : 22] +Input [6]: [d_year#66, cs_item_sk#55, cs_bill_customer_sk#54, sum#70, sum#71, sum#72] +Keys [3]: [d_year#66, cs_item_sk#55, cs_bill_customer_sk#54] +Functions [3]: [sum(cs_quantity#57), sum(UnscaledValue(cs_wholesale_cost#58)), sum(UnscaledValue(cs_sales_price#59))] +Aggregate Attributes [3]: [sum(cs_quantity#57)#73, sum(UnscaledValue(cs_wholesale_cost#58))#74, sum(UnscaledValue(cs_sales_price#59))#75] +Results [6]: [d_year#66 AS cs_sold_year#76, cs_item_sk#55, cs_bill_customer_sk#54 AS cs_customer_sk#77, sum(cs_quantity#57)#73 AS cs_qty#78, MakeDecimal(sum(UnscaledValue(cs_wholesale_cost#58))#74,17,2) AS cs_wc#79, MakeDecimal(sum(UnscaledValue(cs_sales_price#59))#75,17,2) AS cs_sp#80] + +(66) Filter [codegen id : 22] +Input [6]: [cs_sold_year#76, cs_item_sk#55, cs_customer_sk#77, cs_qty#78, cs_wc#79, cs_sp#80] +Condition : (coalesce(cs_qty#78, 0) > 0) + +(67) Sort [codegen id : 22] +Input [6]: [cs_sold_year#76, cs_item_sk#55, cs_customer_sk#77, cs_qty#78, cs_wc#79, cs_sp#80] +Arguments: [cs_sold_year#76 ASC NULLS FIRST, cs_item_sk#55 ASC NULLS FIRST, cs_customer_sk#77 ASC NULLS FIRST], false, 0 + +(68) SortMergeJoin [codegen id : 23] +Left keys [3]: [ss_sold_year#23, ss_item_sk#1, ss_customer_sk#2] +Right keys [3]: [cs_sold_year#76, cs_item_sk#55, cs_customer_sk#77] +Join type: Inner +Join condition: None + +(69) Project [codegen id : 23] +Output [13]: [round((cast(ss_qty#24 as double) / cast(coalesce((ws_qty#51 + cs_qty#78), 1) as double)), 2) AS ratio#81, ss_qty#24 AS store_qty#82, ss_wc#25 AS store_wholesale_cost#83, ss_sp#26 AS store_sales_price#84, (coalesce(ws_qty#51, 0) + coalesce(cs_qty#78, 0)) AS other_chan_qty#85, (coalesce(ws_wc#52, 0.00) + coalesce(cs_wc#79, 0.00)) AS other_chan_wholesale_cost#86, (coalesce(ws_sp#53, 0.00) + coalesce(cs_sp#80, 0.00)) AS other_chan_sales_price#87, ss_sold_year#23, ss_item_sk#1, ss_customer_sk#2, ss_qty#24, ss_wc#25, ss_sp#26] +Input [15]: [ss_sold_year#23, ss_item_sk#1, ss_customer_sk#2, ss_qty#24, ss_wc#25, ss_sp#26, ws_qty#51, ws_wc#52, ws_sp#53, cs_sold_year#76, cs_item_sk#55, cs_customer_sk#77, cs_qty#78, cs_wc#79, cs_sp#80] + +(70) TakeOrderedAndProject +Input [13]: [ratio#81, store_qty#82, store_wholesale_cost#83, store_sales_price#84, other_chan_qty#85, other_chan_wholesale_cost#86, other_chan_sales_price#87, ss_sold_year#23, ss_item_sk#1, ss_customer_sk#2, ss_qty#24, ss_wc#25, ss_sp#26] +Arguments: 100, [ss_sold_year#23 ASC NULLS FIRST, ss_item_sk#1 ASC NULLS FIRST, ss_customer_sk#2 ASC NULLS FIRST, ss_qty#24 DESC NULLS LAST, ss_wc#25 DESC NULLS LAST, ss_sp#26 DESC NULLS LAST, other_chan_qty#85 ASC NULLS FIRST, other_chan_wholesale_cost#86 ASC NULLS FIRST, other_chan_sales_price#87 ASC NULLS FIRST, ratio#81 ASC NULLS FIRST], [ratio#81, store_qty#82, store_wholesale_cost#83, store_sales_price#84, other_chan_qty#85, other_chan_wholesale_cost#86, other_chan_sales_price#87] + +===== Subqueries ===== + +Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#7 IN dynamicpruning#8 +BroadcastExchange (74) ++- * ColumnarToRow (73) + +- CometFilter (72) + +- CometScan parquet spark_catalog.default.date_dim (71) + + +(71) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#12, d_year#13] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2000), IsNotNull(d_date_sk)] +ReadSchema: struct<d_date_sk:int,d_year:int> + +(72) CometFilter +Input [2]: [d_date_sk#12, d_year#13] +Condition : ((isnotnull(d_year#13) AND (d_year#13 = 2000)) AND isnotnull(d_date_sk#12)) + +(73) ColumnarToRow [codegen id : 1] +Input [2]: [d_date_sk#12, d_year#13] + +(74) BroadcastExchange +Input [2]: [d_date_sk#12, d_year#13] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=10] + +Subquery:2 Hosting operator id = 22 Hosting Expression = ws_sold_date_sk#33 IN dynamicpruning#8 + +Subquery:3 Hosting operator id = 46 Hosting Expression = cs_sold_date_sk#60 IN dynamicpruning#8 + + diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q78/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q78/simplified.txt new file mode 100644 index 000000000..49bd173f6 --- /dev/null +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q78/simplified.txt @@ -0,0 +1,127 @@ +TakeOrderedAndProject [ss_sold_year,ss_item_sk,ss_customer_sk,ss_qty,ss_wc,ss_sp,other_chan_qty,other_chan_wholesale_cost,other_chan_sales_price,ratio,store_qty,store_wholesale_cost,store_sales_price] + WholeStageCodegen (23) + Project [ss_qty,ws_qty,cs_qty,ss_wc,ss_sp,ws_wc,cs_wc,ws_sp,cs_sp,ss_sold_year,ss_item_sk,ss_customer_sk] + SortMergeJoin [ss_sold_year,ss_item_sk,ss_customer_sk,cs_sold_year,cs_item_sk,cs_customer_sk] + InputAdapter + WholeStageCodegen (15) + Project [ss_sold_year,ss_item_sk,ss_customer_sk,ss_qty,ss_wc,ss_sp,ws_qty,ws_wc,ws_sp] + SortMergeJoin [ss_sold_year,ss_item_sk,ss_customer_sk,ws_sold_year,ws_item_sk,ws_customer_sk] + InputAdapter + WholeStageCodegen (7) + Sort [ss_sold_year,ss_item_sk,ss_customer_sk] + HashAggregate [d_year,ss_item_sk,ss_customer_sk,sum,sum,sum] [sum(ss_quantity),sum(UnscaledValue(ss_wholesale_cost)),sum(UnscaledValue(ss_sales_price)),ss_sold_year,ss_qty,ss_wc,ss_sp,sum,sum,sum] + InputAdapter + Exchange [d_year,ss_item_sk,ss_customer_sk] #1 + WholeStageCodegen (6) + HashAggregate [d_year,ss_item_sk,ss_customer_sk,ss_quantity,ss_wholesale_cost,ss_sales_price] [sum,sum,sum,sum,sum,sum] + Project [ss_item_sk,ss_customer_sk,ss_quantity,ss_wholesale_cost,ss_sales_price,d_year] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [ss_item_sk,ss_customer_sk,ss_quantity,ss_wholesale_cost,ss_sales_price,ss_sold_date_sk] + Filter [sr_ticket_number] + SortMergeJoin [ss_ticket_number,ss_item_sk,sr_ticket_number,sr_item_sk] + InputAdapter + WholeStageCodegen (2) + Sort [ss_ticket_number,ss_item_sk] + InputAdapter + Exchange [ss_ticket_number,ss_item_sk] #2 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometFilter [ss_item_sk,ss_customer_sk] + CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_quantity,ss_wholesale_cost,ss_sales_price,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometFilter [d_year,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_year] + InputAdapter + WholeStageCodegen (4) + Sort [sr_ticket_number,sr_item_sk] + InputAdapter + Exchange [sr_ticket_number,sr_item_sk] #4 + WholeStageCodegen (3) + ColumnarToRow + InputAdapter + CometProject [sr_item_sk,sr_ticket_number] + CometFilter [sr_ticket_number,sr_item_sk] + CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_returned_date_sk] + InputAdapter + ReusedExchange [d_date_sk,d_year] #3 + InputAdapter + WholeStageCodegen (14) + Sort [ws_sold_year,ws_item_sk,ws_customer_sk] + Filter [ws_qty] + HashAggregate [d_year,ws_item_sk,ws_bill_customer_sk,sum,sum,sum] [sum(ws_quantity),sum(UnscaledValue(ws_wholesale_cost)),sum(UnscaledValue(ws_sales_price)),ws_sold_year,ws_customer_sk,ws_qty,ws_wc,ws_sp,sum,sum,sum] + InputAdapter + Exchange [d_year,ws_item_sk,ws_bill_customer_sk] #5 + WholeStageCodegen (13) + HashAggregate [d_year,ws_item_sk,ws_bill_customer_sk,ws_quantity,ws_wholesale_cost,ws_sales_price] [sum,sum,sum,sum,sum,sum] + Project [ws_item_sk,ws_bill_customer_sk,ws_quantity,ws_wholesale_cost,ws_sales_price,d_year] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + Project [ws_item_sk,ws_bill_customer_sk,ws_quantity,ws_wholesale_cost,ws_sales_price,ws_sold_date_sk] + Filter [wr_order_number] + SortMergeJoin [ws_order_number,ws_item_sk,wr_order_number,wr_item_sk] + InputAdapter + WholeStageCodegen (9) + Sort [ws_order_number,ws_item_sk] + InputAdapter + Exchange [ws_order_number,ws_item_sk] #6 + WholeStageCodegen (8) + ColumnarToRow + InputAdapter + CometFilter [ws_item_sk,ws_bill_customer_sk] + CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_bill_customer_sk,ws_order_number,ws_quantity,ws_wholesale_cost,ws_sales_price,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + InputAdapter + WholeStageCodegen (11) + Sort [wr_order_number,wr_item_sk] + InputAdapter + Exchange [wr_order_number,wr_item_sk] #7 + WholeStageCodegen (10) + ColumnarToRow + InputAdapter + CometProject [wr_item_sk,wr_order_number] + CometFilter [wr_order_number,wr_item_sk] + CometScan parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_returned_date_sk] + InputAdapter + ReusedExchange [d_date_sk,d_year] #3 + InputAdapter + WholeStageCodegen (22) + Sort [cs_sold_year,cs_item_sk,cs_customer_sk] + Filter [cs_qty] + HashAggregate [d_year,cs_item_sk,cs_bill_customer_sk,sum,sum,sum] [sum(cs_quantity),sum(UnscaledValue(cs_wholesale_cost)),sum(UnscaledValue(cs_sales_price)),cs_sold_year,cs_customer_sk,cs_qty,cs_wc,cs_sp,sum,sum,sum] + InputAdapter + Exchange [d_year,cs_item_sk,cs_bill_customer_sk] #8 + WholeStageCodegen (21) + HashAggregate [d_year,cs_item_sk,cs_bill_customer_sk,cs_quantity,cs_wholesale_cost,cs_sales_price] [sum,sum,sum,sum,sum,sum] + Project [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_wholesale_cost,cs_sales_price,d_year] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + Project [cs_bill_customer_sk,cs_item_sk,cs_quantity,cs_wholesale_cost,cs_sales_price,cs_sold_date_sk] + Filter [cr_order_number] + SortMergeJoin [cs_order_number,cs_item_sk,cr_order_number,cr_item_sk] + InputAdapter + WholeStageCodegen (17) + Sort [cs_order_number,cs_item_sk] + InputAdapter + Exchange [cs_order_number,cs_item_sk] #9 + WholeStageCodegen (16) + ColumnarToRow + InputAdapter + CometFilter [cs_item_sk,cs_bill_customer_sk] + CometScan parquet spark_catalog.default.catalog_sales [cs_bill_customer_sk,cs_item_sk,cs_order_number,cs_quantity,cs_wholesale_cost,cs_sales_price,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + InputAdapter + WholeStageCodegen (19) + Sort [cr_order_number,cr_item_sk] + InputAdapter + Exchange [cr_order_number,cr_item_sk] #10 + WholeStageCodegen (18) + ColumnarToRow + InputAdapter + CometProject [cr_item_sk,cr_order_number] + CometFilter [cr_order_number,cr_item_sk] + CometScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_returned_date_sk] + InputAdapter + ReusedExchange [d_date_sk,d_year] #3 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 new file mode 100644 index 000000000..8f47f4f0e --- /dev/null +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q80a/explain.txt @@ -0,0 +1,730 @@ +== Physical Plan == +TakeOrderedAndProject (120) ++- * HashAggregate (119) + +- Exchange (118) + +- * HashAggregate (117) + +- Union (116) + :- * HashAggregate (105) + : +- Exchange (104) + : +- * HashAggregate (103) + : +- Union (102) + : :- * HashAggregate (39) + : : +- Exchange (38) + : : +- * HashAggregate (37) + : : +- * Project (36) + : : +- * BroadcastHashJoin Inner BuildRight (35) + : : :- * Project (29) + : : : +- * BroadcastHashJoin Inner BuildRight (28) + : : : :- * Project (22) + : : : : +- * BroadcastHashJoin Inner BuildRight (21) + : : : : :- * Project (16) + : : : : : +- * BroadcastHashJoin Inner BuildRight (15) + : : : : : :- * Project (13) + : : : : : : +- * SortMergeJoin LeftOuter (12) + : : : : : : :- * Sort (5) + : : : : : : : +- Exchange (4) + : : : : : : : +- * ColumnarToRow (3) + : : : : : : : +- CometFilter (2) + : : : : : : : +- CometScan parquet spark_catalog.default.store_sales (1) + : : : : : : +- * Sort (11) + : : : : : : +- Exchange (10) + : : : : : : +- * ColumnarToRow (9) + : : : : : : +- CometProject (8) + : : : : : : +- CometFilter (7) + : : : : : : +- CometScan parquet spark_catalog.default.store_returns (6) + : : : : : +- ReusedExchange (14) + : : : : +- BroadcastExchange (20) + : : : : +- * ColumnarToRow (19) + : : : : +- CometFilter (18) + : : : : +- CometScan parquet spark_catalog.default.store (17) + : : : +- BroadcastExchange (27) + : : : +- * ColumnarToRow (26) + : : : +- CometProject (25) + : : : +- CometFilter (24) + : : : +- CometScan parquet spark_catalog.default.item (23) + : : +- BroadcastExchange (34) + : : +- * ColumnarToRow (33) + : : +- CometProject (32) + : : +- CometFilter (31) + : : +- CometScan parquet spark_catalog.default.promotion (30) + : :- * HashAggregate (70) + : : +- Exchange (69) + : : +- * HashAggregate (68) + : : +- * Project (67) + : : +- * BroadcastHashJoin Inner BuildRight (66) + : : :- * Project (64) + : : : +- * BroadcastHashJoin Inner BuildRight (63) + : : : :- * Project (61) + : : : : +- * BroadcastHashJoin Inner BuildRight (60) + : : : : :- * Project (55) + : : : : : +- * BroadcastHashJoin Inner BuildRight (54) + : : : : : :- * Project (52) + : : : : : : +- * SortMergeJoin LeftOuter (51) + : : : : : : :- * Sort (44) + : : : : : : : +- Exchange (43) + : : : : : : : +- * ColumnarToRow (42) + : : : : : : : +- CometFilter (41) + : : : : : : : +- CometScan parquet spark_catalog.default.catalog_sales (40) + : : : : : : +- * Sort (50) + : : : : : : +- Exchange (49) + : : : : : : +- * ColumnarToRow (48) + : : : : : : +- CometProject (47) + : : : : : : +- CometFilter (46) + : : : : : : +- CometScan parquet spark_catalog.default.catalog_returns (45) + : : : : : +- ReusedExchange (53) + : : : : +- BroadcastExchange (59) + : : : : +- * ColumnarToRow (58) + : : : : +- CometFilter (57) + : : : : +- CometScan parquet spark_catalog.default.catalog_page (56) + : : : +- ReusedExchange (62) + : : +- ReusedExchange (65) + : +- * HashAggregate (101) + : +- Exchange (100) + : +- * HashAggregate (99) + : +- * Project (98) + : +- * BroadcastHashJoin Inner BuildRight (97) + : :- * Project (95) + : : +- * BroadcastHashJoin Inner BuildRight (94) + : : :- * Project (92) + : : : +- * BroadcastHashJoin Inner BuildRight (91) + : : : :- * Project (86) + : : : : +- * BroadcastHashJoin Inner BuildRight (85) + : : : : :- * Project (83) + : : : : : +- * SortMergeJoin LeftOuter (82) + : : : : : :- * Sort (75) + : : : : : : +- Exchange (74) + : : : : : : +- * ColumnarToRow (73) + : : : : : : +- CometFilter (72) + : : : : : : +- CometScan parquet spark_catalog.default.web_sales (71) + : : : : : +- * Sort (81) + : : : : : +- Exchange (80) + : : : : : +- * ColumnarToRow (79) + : : : : : +- CometProject (78) + : : : : : +- CometFilter (77) + : : : : : +- CometScan parquet spark_catalog.default.web_returns (76) + : : : : +- ReusedExchange (84) + : : : +- BroadcastExchange (90) + : : : +- * ColumnarToRow (89) + : : : +- CometFilter (88) + : : : +- CometScan parquet spark_catalog.default.web_site (87) + : : +- ReusedExchange (93) + : +- ReusedExchange (96) + :- * HashAggregate (110) + : +- Exchange (109) + : +- * HashAggregate (108) + : +- * HashAggregate (107) + : +- ReusedExchange (106) + +- * HashAggregate (115) + +- Exchange (114) + +- * HashAggregate (113) + +- * HashAggregate (112) + +- ReusedExchange (111) + + +(1) Scan parquet spark_catalog.default.store_sales +Output [7]: [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ticket_number#4, ss_ext_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7] +Batched: true +Location: InMemoryFileIndex [] +PartitionFilters: [isnotnull(ss_sold_date_sk#7), dynamicpruningexpression(ss_sold_date_sk#7 IN dynamicpruning#8)] +PushedFilters: [IsNotNull(ss_store_sk), IsNotNull(ss_item_sk), IsNotNull(ss_promo_sk)] +ReadSchema: struct<ss_item_sk:int,ss_store_sk:int,ss_promo_sk:int,ss_ticket_number:int,ss_ext_sales_price:decimal(7,2),ss_net_profit:decimal(7,2)> + +(2) CometFilter +Input [7]: [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ticket_number#4, ss_ext_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7] +Condition : ((isnotnull(ss_store_sk#2) AND isnotnull(ss_item_sk#1)) AND isnotnull(ss_promo_sk#3)) + +(3) ColumnarToRow [codegen id : 1] +Input [7]: [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ticket_number#4, ss_ext_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7] + +(4) Exchange +Input [7]: [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ticket_number#4, ss_ext_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7] +Arguments: hashpartitioning(ss_item_sk#1, ss_ticket_number#4, 5), ENSURE_REQUIREMENTS, [plan_id=1] + +(5) Sort [codegen id : 2] +Input [7]: [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ticket_number#4, ss_ext_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7] +Arguments: [ss_item_sk#1 ASC NULLS FIRST, ss_ticket_number#4 ASC NULLS FIRST], false, 0 + +(6) Scan parquet spark_catalog.default.store_returns +Output [5]: [sr_item_sk#9, sr_ticket_number#10, sr_return_amt#11, sr_net_loss#12, sr_returned_date_sk#13] +Batched: true +Location [not included in comparison]/{warehouse_dir}/store_returns] +PushedFilters: [IsNotNull(sr_item_sk), IsNotNull(sr_ticket_number)] +ReadSchema: struct<sr_item_sk:int,sr_ticket_number:int,sr_return_amt:decimal(7,2),sr_net_loss:decimal(7,2)> + +(7) CometFilter +Input [5]: [sr_item_sk#9, sr_ticket_number#10, sr_return_amt#11, sr_net_loss#12, sr_returned_date_sk#13] +Condition : (isnotnull(sr_item_sk#9) AND isnotnull(sr_ticket_number#10)) + +(8) CometProject +Input [5]: [sr_item_sk#9, sr_ticket_number#10, sr_return_amt#11, sr_net_loss#12, sr_returned_date_sk#13] +Arguments: [sr_item_sk#9, sr_ticket_number#10, sr_return_amt#11, sr_net_loss#12], [sr_item_sk#9, sr_ticket_number#10, sr_return_amt#11, sr_net_loss#12] + +(9) ColumnarToRow [codegen id : 3] +Input [4]: [sr_item_sk#9, sr_ticket_number#10, sr_return_amt#11, sr_net_loss#12] + +(10) Exchange +Input [4]: [sr_item_sk#9, sr_ticket_number#10, sr_return_amt#11, sr_net_loss#12] +Arguments: hashpartitioning(sr_item_sk#9, sr_ticket_number#10, 5), ENSURE_REQUIREMENTS, [plan_id=2] + +(11) Sort [codegen id : 4] +Input [4]: [sr_item_sk#9, sr_ticket_number#10, sr_return_amt#11, sr_net_loss#12] +Arguments: [sr_item_sk#9 ASC NULLS FIRST, sr_ticket_number#10 ASC NULLS FIRST], false, 0 + +(12) SortMergeJoin [codegen id : 9] +Left keys [2]: [ss_item_sk#1, ss_ticket_number#4] +Right keys [2]: [sr_item_sk#9, sr_ticket_number#10] +Join type: LeftOuter +Join condition: None + +(13) Project [codegen id : 9] +Output [8]: [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7, sr_return_amt#11, sr_net_loss#12] +Input [11]: [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ticket_number#4, ss_ext_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7, sr_item_sk#9, sr_ticket_number#10, sr_return_amt#11, sr_net_loss#12] + +(14) ReusedExchange [Reuses operator id: 125] +Output [1]: [d_date_sk#14] + +(15) BroadcastHashJoin [codegen id : 9] +Left keys [1]: [ss_sold_date_sk#7] +Right keys [1]: [d_date_sk#14] +Join type: Inner +Join condition: None + +(16) Project [codegen id : 9] +Output [7]: [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12] +Input [9]: [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, ss_sold_date_sk#7, sr_return_amt#11, sr_net_loss#12, d_date_sk#14] + +(17) Scan parquet spark_catalog.default.store +Output [2]: [s_store_sk#15, s_store_id#16] +Batched: true +Location [not included in comparison]/{warehouse_dir}/store] +PushedFilters: [IsNotNull(s_store_sk)] +ReadSchema: struct<s_store_sk:int,s_store_id:string> + +(18) CometFilter +Input [2]: [s_store_sk#15, s_store_id#16] +Condition : isnotnull(s_store_sk#15) + +(19) ColumnarToRow [codegen id : 6] +Input [2]: [s_store_sk#15, s_store_id#16] + +(20) BroadcastExchange +Input [2]: [s_store_sk#15, s_store_id#16] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=3] + +(21) BroadcastHashJoin [codegen id : 9] +Left keys [1]: [ss_store_sk#2] +Right keys [1]: [s_store_sk#15] +Join type: Inner +Join condition: None + +(22) Project [codegen id : 9] +Output [7]: [ss_item_sk#1, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#16] +Input [9]: [ss_item_sk#1, ss_store_sk#2, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_sk#15, s_store_id#16] + +(23) Scan parquet spark_catalog.default.item +Output [2]: [i_item_sk#17, i_current_price#18] +Batched: true +Location [not included in comparison]/{warehouse_dir}/item] +PushedFilters: [IsNotNull(i_current_price), GreaterThan(i_current_price,50.00), IsNotNull(i_item_sk)] +ReadSchema: struct<i_item_sk:int,i_current_price:decimal(7,2)> + +(24) CometFilter +Input [2]: [i_item_sk#17, i_current_price#18] +Condition : ((isnotnull(i_current_price#18) AND (i_current_price#18 > 50.00)) AND isnotnull(i_item_sk#17)) + +(25) CometProject +Input [2]: [i_item_sk#17, i_current_price#18] +Arguments: [i_item_sk#17], [i_item_sk#17] + +(26) ColumnarToRow [codegen id : 7] +Input [1]: [i_item_sk#17] + +(27) BroadcastExchange +Input [1]: [i_item_sk#17] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] + +(28) BroadcastHashJoin [codegen id : 9] +Left keys [1]: [ss_item_sk#1] +Right keys [1]: [i_item_sk#17] +Join type: Inner +Join condition: None + +(29) Project [codegen id : 9] +Output [6]: [ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#16] +Input [8]: [ss_item_sk#1, ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#16, i_item_sk#17] + +(30) Scan parquet spark_catalog.default.promotion +Output [2]: [p_promo_sk#19, p_channel_tv#20] +Batched: true +Location [not included in comparison]/{warehouse_dir}/promotion] +PushedFilters: [IsNotNull(p_channel_tv), EqualTo(p_channel_tv,N), IsNotNull(p_promo_sk)] +ReadSchema: struct<p_promo_sk:int,p_channel_tv:string> + +(31) CometFilter +Input [2]: [p_promo_sk#19, p_channel_tv#20] +Condition : ((isnotnull(p_channel_tv#20) AND (p_channel_tv#20 = N)) AND isnotnull(p_promo_sk#19)) + +(32) CometProject +Input [2]: [p_promo_sk#19, p_channel_tv#20] +Arguments: [p_promo_sk#19], [p_promo_sk#19] + +(33) ColumnarToRow [codegen id : 8] +Input [1]: [p_promo_sk#19] + +(34) BroadcastExchange +Input [1]: [p_promo_sk#19] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] + +(35) BroadcastHashJoin [codegen id : 9] +Left keys [1]: [ss_promo_sk#3] +Right keys [1]: [p_promo_sk#19] +Join type: Inner +Join condition: None + +(36) Project [codegen id : 9] +Output [5]: [ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#16] +Input [7]: [ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#16, p_promo_sk#19] + +(37) HashAggregate [codegen id : 9] +Input [5]: [ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#16] +Keys [1]: [s_store_id#16] +Functions [3]: [partial_sum(UnscaledValue(ss_ext_sales_price#5)), partial_sum(coalesce(cast(sr_return_amt#11 as decimal(12,2)), 0.00)), partial_sum((ss_net_profit#6 - coalesce(cast(sr_net_loss#12 as decimal(12,2)), 0.00)))] +Aggregate Attributes [5]: [sum#21, sum#22, isEmpty#23, sum#24, isEmpty#25] +Results [6]: [s_store_id#16, sum#26, sum#27, isEmpty#28, sum#29, isEmpty#30] + +(38) Exchange +Input [6]: [s_store_id#16, sum#26, sum#27, isEmpty#28, sum#29, isEmpty#30] +Arguments: hashpartitioning(s_store_id#16, 5), ENSURE_REQUIREMENTS, [plan_id=6] + +(39) HashAggregate [codegen id : 10] +Input [6]: [s_store_id#16, sum#26, sum#27, isEmpty#28, sum#29, isEmpty#30] +Keys [1]: [s_store_id#16] +Functions [3]: [sum(UnscaledValue(ss_ext_sales_price#5)), sum(coalesce(cast(sr_return_amt#11 as decimal(12,2)), 0.00)), sum((ss_net_profit#6 - coalesce(cast(sr_net_loss#12 as decimal(12,2)), 0.00)))] +Aggregate Attributes [3]: [sum(UnscaledValue(ss_ext_sales_price#5))#31, sum(coalesce(cast(sr_return_amt#11 as decimal(12,2)), 0.00))#32, sum((ss_net_profit#6 - coalesce(cast(sr_net_loss#12 as decimal(12,2)), 0.00)))#33] +Results [5]: [store channel AS channel#34, concat(store, s_store_id#16) AS id#35, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#5))#31,17,2) AS sales#36, sum(coalesce(cast(sr_return_amt#11 as decimal(12,2)), 0.00))#32 AS returns#37, sum((ss_net_profit#6 - coalesce(cast(sr_net_loss#12 as decimal(12,2)), 0.00)))#33 AS profit#38] + +(40) Scan parquet spark_catalog.default.catalog_sales +Output [7]: [cs_catalog_page_sk#39, cs_item_sk#40, cs_promo_sk#41, cs_order_number#42, cs_ext_sales_price#43, cs_net_profit#44, cs_sold_date_sk#45] +Batched: true +Location: InMemoryFileIndex [] +PartitionFilters: [isnotnull(cs_sold_date_sk#45), dynamicpruningexpression(cs_sold_date_sk#45 IN dynamicpruning#46)] +PushedFilters: [IsNotNull(cs_catalog_page_sk), IsNotNull(cs_item_sk), IsNotNull(cs_promo_sk)] +ReadSchema: struct<cs_catalog_page_sk:int,cs_item_sk:int,cs_promo_sk:int,cs_order_number:int,cs_ext_sales_price:decimal(7,2),cs_net_profit:decimal(7,2)> + +(41) CometFilter +Input [7]: [cs_catalog_page_sk#39, cs_item_sk#40, cs_promo_sk#41, cs_order_number#42, cs_ext_sales_price#43, cs_net_profit#44, cs_sold_date_sk#45] +Condition : ((isnotnull(cs_catalog_page_sk#39) AND isnotnull(cs_item_sk#40)) AND isnotnull(cs_promo_sk#41)) + +(42) ColumnarToRow [codegen id : 11] +Input [7]: [cs_catalog_page_sk#39, cs_item_sk#40, cs_promo_sk#41, cs_order_number#42, cs_ext_sales_price#43, cs_net_profit#44, cs_sold_date_sk#45] + +(43) Exchange +Input [7]: [cs_catalog_page_sk#39, cs_item_sk#40, cs_promo_sk#41, cs_order_number#42, cs_ext_sales_price#43, cs_net_profit#44, cs_sold_date_sk#45] +Arguments: hashpartitioning(cs_item_sk#40, cs_order_number#42, 5), ENSURE_REQUIREMENTS, [plan_id=7] + +(44) Sort [codegen id : 12] +Input [7]: [cs_catalog_page_sk#39, cs_item_sk#40, cs_promo_sk#41, cs_order_number#42, cs_ext_sales_price#43, cs_net_profit#44, cs_sold_date_sk#45] +Arguments: [cs_item_sk#40 ASC NULLS FIRST, cs_order_number#42 ASC NULLS FIRST], false, 0 + +(45) Scan parquet spark_catalog.default.catalog_returns +Output [5]: [cr_item_sk#47, cr_order_number#48, cr_return_amount#49, cr_net_loss#50, cr_returned_date_sk#51] +Batched: true +Location [not included in comparison]/{warehouse_dir}/catalog_returns] +PushedFilters: [IsNotNull(cr_item_sk), IsNotNull(cr_order_number)] +ReadSchema: struct<cr_item_sk:int,cr_order_number:int,cr_return_amount:decimal(7,2),cr_net_loss:decimal(7,2)> + +(46) CometFilter +Input [5]: [cr_item_sk#47, cr_order_number#48, cr_return_amount#49, cr_net_loss#50, cr_returned_date_sk#51] +Condition : (isnotnull(cr_item_sk#47) AND isnotnull(cr_order_number#48)) + +(47) CometProject +Input [5]: [cr_item_sk#47, cr_order_number#48, cr_return_amount#49, cr_net_loss#50, cr_returned_date_sk#51] +Arguments: [cr_item_sk#47, cr_order_number#48, cr_return_amount#49, cr_net_loss#50], [cr_item_sk#47, cr_order_number#48, cr_return_amount#49, cr_net_loss#50] + +(48) ColumnarToRow [codegen id : 13] +Input [4]: [cr_item_sk#47, cr_order_number#48, cr_return_amount#49, cr_net_loss#50] + +(49) Exchange +Input [4]: [cr_item_sk#47, cr_order_number#48, cr_return_amount#49, cr_net_loss#50] +Arguments: hashpartitioning(cr_item_sk#47, cr_order_number#48, 5), ENSURE_REQUIREMENTS, [plan_id=8] + +(50) Sort [codegen id : 14] +Input [4]: [cr_item_sk#47, cr_order_number#48, cr_return_amount#49, cr_net_loss#50] +Arguments: [cr_item_sk#47 ASC NULLS FIRST, cr_order_number#48 ASC NULLS FIRST], false, 0 + +(51) SortMergeJoin [codegen id : 19] +Left keys [2]: [cs_item_sk#40, cs_order_number#42] +Right keys [2]: [cr_item_sk#47, cr_order_number#48] +Join type: LeftOuter +Join condition: None + +(52) Project [codegen id : 19] +Output [8]: [cs_catalog_page_sk#39, cs_item_sk#40, cs_promo_sk#41, cs_ext_sales_price#43, cs_net_profit#44, cs_sold_date_sk#45, cr_return_amount#49, cr_net_loss#50] +Input [11]: [cs_catalog_page_sk#39, cs_item_sk#40, cs_promo_sk#41, cs_order_number#42, cs_ext_sales_price#43, cs_net_profit#44, cs_sold_date_sk#45, cr_item_sk#47, cr_order_number#48, cr_return_amount#49, cr_net_loss#50] + +(53) ReusedExchange [Reuses operator id: 125] +Output [1]: [d_date_sk#52] + +(54) BroadcastHashJoin [codegen id : 19] +Left keys [1]: [cs_sold_date_sk#45] +Right keys [1]: [d_date_sk#52] +Join type: Inner +Join condition: None + +(55) Project [codegen id : 19] +Output [7]: [cs_catalog_page_sk#39, cs_item_sk#40, cs_promo_sk#41, cs_ext_sales_price#43, cs_net_profit#44, cr_return_amount#49, cr_net_loss#50] +Input [9]: [cs_catalog_page_sk#39, cs_item_sk#40, cs_promo_sk#41, cs_ext_sales_price#43, cs_net_profit#44, cs_sold_date_sk#45, cr_return_amount#49, cr_net_loss#50, d_date_sk#52] + +(56) Scan parquet spark_catalog.default.catalog_page +Output [2]: [cp_catalog_page_sk#53, cp_catalog_page_id#54] +Batched: true +Location [not included in comparison]/{warehouse_dir}/catalog_page] +PushedFilters: [IsNotNull(cp_catalog_page_sk)] +ReadSchema: struct<cp_catalog_page_sk:int,cp_catalog_page_id:string> + +(57) CometFilter +Input [2]: [cp_catalog_page_sk#53, cp_catalog_page_id#54] +Condition : isnotnull(cp_catalog_page_sk#53) + +(58) ColumnarToRow [codegen id : 16] +Input [2]: [cp_catalog_page_sk#53, cp_catalog_page_id#54] + +(59) BroadcastExchange +Input [2]: [cp_catalog_page_sk#53, cp_catalog_page_id#54] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=9] + +(60) BroadcastHashJoin [codegen id : 19] +Left keys [1]: [cs_catalog_page_sk#39] +Right keys [1]: [cp_catalog_page_sk#53] +Join type: Inner +Join condition: None + +(61) Project [codegen id : 19] +Output [7]: [cs_item_sk#40, cs_promo_sk#41, cs_ext_sales_price#43, cs_net_profit#44, cr_return_amount#49, cr_net_loss#50, cp_catalog_page_id#54] +Input [9]: [cs_catalog_page_sk#39, cs_item_sk#40, cs_promo_sk#41, cs_ext_sales_price#43, cs_net_profit#44, cr_return_amount#49, cr_net_loss#50, cp_catalog_page_sk#53, cp_catalog_page_id#54] + +(62) ReusedExchange [Reuses operator id: 27] +Output [1]: [i_item_sk#55] + +(63) BroadcastHashJoin [codegen id : 19] +Left keys [1]: [cs_item_sk#40] +Right keys [1]: [i_item_sk#55] +Join type: Inner +Join condition: None + +(64) Project [codegen id : 19] +Output [6]: [cs_promo_sk#41, cs_ext_sales_price#43, cs_net_profit#44, cr_return_amount#49, cr_net_loss#50, cp_catalog_page_id#54] +Input [8]: [cs_item_sk#40, cs_promo_sk#41, cs_ext_sales_price#43, cs_net_profit#44, cr_return_amount#49, cr_net_loss#50, cp_catalog_page_id#54, i_item_sk#55] + +(65) ReusedExchange [Reuses operator id: 34] +Output [1]: [p_promo_sk#56] + +(66) BroadcastHashJoin [codegen id : 19] +Left keys [1]: [cs_promo_sk#41] +Right keys [1]: [p_promo_sk#56] +Join type: Inner +Join condition: None + +(67) Project [codegen id : 19] +Output [5]: [cs_ext_sales_price#43, cs_net_profit#44, cr_return_amount#49, cr_net_loss#50, cp_catalog_page_id#54] +Input [7]: [cs_promo_sk#41, cs_ext_sales_price#43, cs_net_profit#44, cr_return_amount#49, cr_net_loss#50, cp_catalog_page_id#54, p_promo_sk#56] + +(68) HashAggregate [codegen id : 19] +Input [5]: [cs_ext_sales_price#43, cs_net_profit#44, cr_return_amount#49, cr_net_loss#50, cp_catalog_page_id#54] +Keys [1]: [cp_catalog_page_id#54] +Functions [3]: [partial_sum(UnscaledValue(cs_ext_sales_price#43)), partial_sum(coalesce(cast(cr_return_amount#49 as decimal(12,2)), 0.00)), partial_sum((cs_net_profit#44 - coalesce(cast(cr_net_loss#50 as decimal(12,2)), 0.00)))] +Aggregate Attributes [5]: [sum#57, sum#58, isEmpty#59, sum#60, isEmpty#61] +Results [6]: [cp_catalog_page_id#54, sum#62, sum#63, isEmpty#64, sum#65, isEmpty#66] + +(69) Exchange +Input [6]: [cp_catalog_page_id#54, sum#62, sum#63, isEmpty#64, sum#65, isEmpty#66] +Arguments: hashpartitioning(cp_catalog_page_id#54, 5), ENSURE_REQUIREMENTS, [plan_id=10] + +(70) HashAggregate [codegen id : 20] +Input [6]: [cp_catalog_page_id#54, sum#62, sum#63, isEmpty#64, sum#65, isEmpty#66] +Keys [1]: [cp_catalog_page_id#54] +Functions [3]: [sum(UnscaledValue(cs_ext_sales_price#43)), sum(coalesce(cast(cr_return_amount#49 as decimal(12,2)), 0.00)), sum((cs_net_profit#44 - coalesce(cast(cr_net_loss#50 as decimal(12,2)), 0.00)))] +Aggregate Attributes [3]: [sum(UnscaledValue(cs_ext_sales_price#43))#67, sum(coalesce(cast(cr_return_amount#49 as decimal(12,2)), 0.00))#68, sum((cs_net_profit#44 - coalesce(cast(cr_net_loss#50 as decimal(12,2)), 0.00)))#69] +Results [5]: [catalog channel AS channel#70, concat(catalog_page, cp_catalog_page_id#54) AS id#71, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#43))#67,17,2) AS sales#72, sum(coalesce(cast(cr_return_amount#49 as decimal(12,2)), 0.00))#68 AS returns#73, sum((cs_net_profit#44 - coalesce(cast(cr_net_loss#50 as decimal(12,2)), 0.00)))#69 AS profit#74] + +(71) Scan parquet spark_catalog.default.web_sales +Output [7]: [ws_item_sk#75, ws_web_site_sk#76, ws_promo_sk#77, ws_order_number#78, ws_ext_sales_price#79, ws_net_profit#80, ws_sold_date_sk#81] +Batched: true +Location: InMemoryFileIndex [] +PartitionFilters: [isnotnull(ws_sold_date_sk#81), dynamicpruningexpression(ws_sold_date_sk#81 IN dynamicpruning#82)] +PushedFilters: [IsNotNull(ws_web_site_sk), IsNotNull(ws_item_sk), IsNotNull(ws_promo_sk)] +ReadSchema: struct<ws_item_sk:int,ws_web_site_sk:int,ws_promo_sk:int,ws_order_number:int,ws_ext_sales_price:decimal(7,2),ws_net_profit:decimal(7,2)> + +(72) CometFilter +Input [7]: [ws_item_sk#75, ws_web_site_sk#76, ws_promo_sk#77, ws_order_number#78, ws_ext_sales_price#79, ws_net_profit#80, ws_sold_date_sk#81] +Condition : ((isnotnull(ws_web_site_sk#76) AND isnotnull(ws_item_sk#75)) AND isnotnull(ws_promo_sk#77)) + +(73) ColumnarToRow [codegen id : 21] +Input [7]: [ws_item_sk#75, ws_web_site_sk#76, ws_promo_sk#77, ws_order_number#78, ws_ext_sales_price#79, ws_net_profit#80, ws_sold_date_sk#81] + +(74) Exchange +Input [7]: [ws_item_sk#75, ws_web_site_sk#76, ws_promo_sk#77, ws_order_number#78, ws_ext_sales_price#79, ws_net_profit#80, ws_sold_date_sk#81] +Arguments: hashpartitioning(ws_item_sk#75, ws_order_number#78, 5), ENSURE_REQUIREMENTS, [plan_id=11] + +(75) Sort [codegen id : 22] +Input [7]: [ws_item_sk#75, ws_web_site_sk#76, ws_promo_sk#77, ws_order_number#78, ws_ext_sales_price#79, ws_net_profit#80, ws_sold_date_sk#81] +Arguments: [ws_item_sk#75 ASC NULLS FIRST, ws_order_number#78 ASC NULLS FIRST], false, 0 + +(76) Scan parquet spark_catalog.default.web_returns +Output [5]: [wr_item_sk#83, wr_order_number#84, wr_return_amt#85, wr_net_loss#86, wr_returned_date_sk#87] +Batched: true +Location [not included in comparison]/{warehouse_dir}/web_returns] +PushedFilters: [IsNotNull(wr_item_sk), IsNotNull(wr_order_number)] +ReadSchema: struct<wr_item_sk:int,wr_order_number:int,wr_return_amt:decimal(7,2),wr_net_loss:decimal(7,2)> + +(77) CometFilter +Input [5]: [wr_item_sk#83, wr_order_number#84, wr_return_amt#85, wr_net_loss#86, wr_returned_date_sk#87] +Condition : (isnotnull(wr_item_sk#83) AND isnotnull(wr_order_number#84)) + +(78) CometProject +Input [5]: [wr_item_sk#83, wr_order_number#84, wr_return_amt#85, wr_net_loss#86, wr_returned_date_sk#87] +Arguments: [wr_item_sk#83, wr_order_number#84, wr_return_amt#85, wr_net_loss#86], [wr_item_sk#83, wr_order_number#84, wr_return_amt#85, wr_net_loss#86] + +(79) ColumnarToRow [codegen id : 23] +Input [4]: [wr_item_sk#83, wr_order_number#84, wr_return_amt#85, wr_net_loss#86] + +(80) Exchange +Input [4]: [wr_item_sk#83, wr_order_number#84, wr_return_amt#85, wr_net_loss#86] +Arguments: hashpartitioning(wr_item_sk#83, wr_order_number#84, 5), ENSURE_REQUIREMENTS, [plan_id=12] + +(81) Sort [codegen id : 24] +Input [4]: [wr_item_sk#83, wr_order_number#84, wr_return_amt#85, wr_net_loss#86] +Arguments: [wr_item_sk#83 ASC NULLS FIRST, wr_order_number#84 ASC NULLS FIRST], false, 0 + +(82) SortMergeJoin [codegen id : 29] +Left keys [2]: [ws_item_sk#75, ws_order_number#78] +Right keys [2]: [wr_item_sk#83, wr_order_number#84] +Join type: LeftOuter +Join condition: None + +(83) Project [codegen id : 29] +Output [8]: [ws_item_sk#75, ws_web_site_sk#76, ws_promo_sk#77, ws_ext_sales_price#79, ws_net_profit#80, ws_sold_date_sk#81, wr_return_amt#85, wr_net_loss#86] +Input [11]: [ws_item_sk#75, ws_web_site_sk#76, ws_promo_sk#77, ws_order_number#78, ws_ext_sales_price#79, ws_net_profit#80, ws_sold_date_sk#81, wr_item_sk#83, wr_order_number#84, wr_return_amt#85, wr_net_loss#86] + +(84) ReusedExchange [Reuses operator id: 125] +Output [1]: [d_date_sk#88] + +(85) BroadcastHashJoin [codegen id : 29] +Left keys [1]: [ws_sold_date_sk#81] +Right keys [1]: [d_date_sk#88] +Join type: Inner +Join condition: None + +(86) Project [codegen id : 29] +Output [7]: [ws_item_sk#75, ws_web_site_sk#76, ws_promo_sk#77, ws_ext_sales_price#79, ws_net_profit#80, wr_return_amt#85, wr_net_loss#86] +Input [9]: [ws_item_sk#75, ws_web_site_sk#76, ws_promo_sk#77, ws_ext_sales_price#79, ws_net_profit#80, ws_sold_date_sk#81, wr_return_amt#85, wr_net_loss#86, d_date_sk#88] + +(87) Scan parquet spark_catalog.default.web_site +Output [2]: [web_site_sk#89, web_site_id#90] +Batched: true +Location [not included in comparison]/{warehouse_dir}/web_site] +PushedFilters: [IsNotNull(web_site_sk)] +ReadSchema: struct<web_site_sk:int,web_site_id:string> + +(88) CometFilter +Input [2]: [web_site_sk#89, web_site_id#90] +Condition : isnotnull(web_site_sk#89) + +(89) ColumnarToRow [codegen id : 26] +Input [2]: [web_site_sk#89, web_site_id#90] + +(90) BroadcastExchange +Input [2]: [web_site_sk#89, web_site_id#90] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=13] + +(91) BroadcastHashJoin [codegen id : 29] +Left keys [1]: [ws_web_site_sk#76] +Right keys [1]: [web_site_sk#89] +Join type: Inner +Join condition: None + +(92) Project [codegen id : 29] +Output [7]: [ws_item_sk#75, ws_promo_sk#77, ws_ext_sales_price#79, ws_net_profit#80, wr_return_amt#85, wr_net_loss#86, web_site_id#90] +Input [9]: [ws_item_sk#75, ws_web_site_sk#76, ws_promo_sk#77, ws_ext_sales_price#79, ws_net_profit#80, wr_return_amt#85, wr_net_loss#86, web_site_sk#89, web_site_id#90] + +(93) ReusedExchange [Reuses operator id: 27] +Output [1]: [i_item_sk#91] + +(94) BroadcastHashJoin [codegen id : 29] +Left keys [1]: [ws_item_sk#75] +Right keys [1]: [i_item_sk#91] +Join type: Inner +Join condition: None + +(95) Project [codegen id : 29] +Output [6]: [ws_promo_sk#77, ws_ext_sales_price#79, ws_net_profit#80, wr_return_amt#85, wr_net_loss#86, web_site_id#90] +Input [8]: [ws_item_sk#75, ws_promo_sk#77, ws_ext_sales_price#79, ws_net_profit#80, wr_return_amt#85, wr_net_loss#86, web_site_id#90, i_item_sk#91] + +(96) ReusedExchange [Reuses operator id: 34] +Output [1]: [p_promo_sk#92] + +(97) BroadcastHashJoin [codegen id : 29] +Left keys [1]: [ws_promo_sk#77] +Right keys [1]: [p_promo_sk#92] +Join type: Inner +Join condition: None + +(98) Project [codegen id : 29] +Output [5]: [ws_ext_sales_price#79, ws_net_profit#80, wr_return_amt#85, wr_net_loss#86, web_site_id#90] +Input [7]: [ws_promo_sk#77, ws_ext_sales_price#79, ws_net_profit#80, wr_return_amt#85, wr_net_loss#86, web_site_id#90, p_promo_sk#92] + +(99) HashAggregate [codegen id : 29] +Input [5]: [ws_ext_sales_price#79, ws_net_profit#80, wr_return_amt#85, wr_net_loss#86, web_site_id#90] +Keys [1]: [web_site_id#90] +Functions [3]: [partial_sum(UnscaledValue(ws_ext_sales_price#79)), partial_sum(coalesce(cast(wr_return_amt#85 as decimal(12,2)), 0.00)), partial_sum((ws_net_profit#80 - coalesce(cast(wr_net_loss#86 as decimal(12,2)), 0.00)))] +Aggregate Attributes [5]: [sum#93, sum#94, isEmpty#95, sum#96, isEmpty#97] +Results [6]: [web_site_id#90, sum#98, sum#99, isEmpty#100, sum#101, isEmpty#102] + +(100) Exchange +Input [6]: [web_site_id#90, sum#98, sum#99, isEmpty#100, sum#101, isEmpty#102] +Arguments: hashpartitioning(web_site_id#90, 5), ENSURE_REQUIREMENTS, [plan_id=14] + +(101) HashAggregate [codegen id : 30] +Input [6]: [web_site_id#90, sum#98, sum#99, isEmpty#100, sum#101, isEmpty#102] +Keys [1]: [web_site_id#90] +Functions [3]: [sum(UnscaledValue(ws_ext_sales_price#79)), sum(coalesce(cast(wr_return_amt#85 as decimal(12,2)), 0.00)), sum((ws_net_profit#80 - coalesce(cast(wr_net_loss#86 as decimal(12,2)), 0.00)))] +Aggregate Attributes [3]: [sum(UnscaledValue(ws_ext_sales_price#79))#103, sum(coalesce(cast(wr_return_amt#85 as decimal(12,2)), 0.00))#104, sum((ws_net_profit#80 - coalesce(cast(wr_net_loss#86 as decimal(12,2)), 0.00)))#105] +Results [5]: [web channel AS channel#106, concat(web_site, web_site_id#90) AS id#107, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#79))#103,17,2) AS sales#108, sum(coalesce(cast(wr_return_amt#85 as decimal(12,2)), 0.00))#104 AS returns#109, sum((ws_net_profit#80 - coalesce(cast(wr_net_loss#86 as decimal(12,2)), 0.00)))#105 AS profit#110] + +(102) Union + +(103) HashAggregate [codegen id : 31] +Input [5]: [channel#34, id#35, sales#36, returns#37, profit#38] +Keys [2]: [channel#34, id#35] +Functions [3]: [partial_sum(sales#36), partial_sum(returns#37), partial_sum(profit#38)] +Aggregate Attributes [6]: [sum#111, isEmpty#112, sum#113, isEmpty#114, sum#115, isEmpty#116] +Results [8]: [channel#34, id#35, sum#117, isEmpty#118, sum#119, isEmpty#120, sum#121, isEmpty#122] + +(104) Exchange +Input [8]: [channel#34, id#35, sum#117, isEmpty#118, sum#119, isEmpty#120, sum#121, isEmpty#122] +Arguments: hashpartitioning(channel#34, id#35, 5), ENSURE_REQUIREMENTS, [plan_id=15] + +(105) HashAggregate [codegen id : 32] +Input [8]: [channel#34, id#35, sum#117, isEmpty#118, sum#119, isEmpty#120, sum#121, isEmpty#122] +Keys [2]: [channel#34, id#35] +Functions [3]: [sum(sales#36), sum(returns#37), sum(profit#38)] +Aggregate Attributes [3]: [sum(sales#36)#123, sum(returns#37)#124, sum(profit#38)#125] +Results [5]: [channel#34, id#35, cast(sum(sales#36)#123 as decimal(37,2)) AS sales#126, cast(sum(returns#37)#124 as decimal(38,2)) AS returns#127, cast(sum(profit#38)#125 as decimal(38,2)) AS profit#128] + +(106) ReusedExchange [Reuses operator id: 104] +Output [8]: [channel#129, id#130, sum#131, isEmpty#132, sum#133, isEmpty#134, sum#135, isEmpty#136] + +(107) HashAggregate [codegen id : 64] +Input [8]: [channel#129, id#130, sum#131, isEmpty#132, sum#133, isEmpty#134, sum#135, isEmpty#136] +Keys [2]: [channel#129, id#130] +Functions [3]: [sum(sales#137), sum(returns#138), sum(profit#139)] +Aggregate Attributes [3]: [sum(sales#137)#123, sum(returns#138)#124, sum(profit#139)#125] +Results [4]: [channel#129, sum(sales#137)#123 AS sales#140, sum(returns#138)#124 AS returns#141, sum(profit#139)#125 AS profit#142] + +(108) HashAggregate [codegen id : 64] +Input [4]: [channel#129, sales#140, returns#141, profit#142] +Keys [1]: [channel#129] +Functions [3]: [partial_sum(sales#140), partial_sum(returns#141), partial_sum(profit#142)] +Aggregate Attributes [6]: [sum#143, isEmpty#144, sum#145, isEmpty#146, sum#147, isEmpty#148] +Results [7]: [channel#129, sum#149, isEmpty#150, sum#151, isEmpty#152, sum#153, isEmpty#154] + +(109) Exchange +Input [7]: [channel#129, sum#149, isEmpty#150, sum#151, isEmpty#152, sum#153, isEmpty#154] +Arguments: hashpartitioning(channel#129, 5), ENSURE_REQUIREMENTS, [plan_id=16] + +(110) HashAggregate [codegen id : 65] +Input [7]: [channel#129, sum#149, isEmpty#150, sum#151, isEmpty#152, sum#153, isEmpty#154] +Keys [1]: [channel#129] +Functions [3]: [sum(sales#140), sum(returns#141), sum(profit#142)] +Aggregate Attributes [3]: [sum(sales#140)#155, sum(returns#141)#156, sum(profit#142)#157] +Results [5]: [channel#129, null AS id#158, sum(sales#140)#155 AS sales#159, sum(returns#141)#156 AS returns#160, sum(profit#142)#157 AS profit#161] + +(111) ReusedExchange [Reuses operator id: 104] +Output [8]: [channel#162, id#163, sum#164, isEmpty#165, sum#166, isEmpty#167, sum#168, isEmpty#169] + +(112) HashAggregate [codegen id : 97] +Input [8]: [channel#162, id#163, sum#164, isEmpty#165, sum#166, isEmpty#167, sum#168, isEmpty#169] +Keys [2]: [channel#162, id#163] +Functions [3]: [sum(sales#170), sum(returns#171), sum(profit#172)] +Aggregate Attributes [3]: [sum(sales#170)#123, sum(returns#171)#124, sum(profit#172)#125] +Results [3]: [sum(sales#170)#123 AS sales#173, sum(returns#171)#124 AS returns#174, sum(profit#172)#125 AS profit#175] + +(113) HashAggregate [codegen id : 97] +Input [3]: [sales#173, returns#174, profit#175] +Keys: [] +Functions [3]: [partial_sum(sales#173), partial_sum(returns#174), partial_sum(profit#175)] +Aggregate Attributes [6]: [sum#176, isEmpty#177, sum#178, isEmpty#179, sum#180, isEmpty#181] +Results [6]: [sum#182, isEmpty#183, sum#184, isEmpty#185, sum#186, isEmpty#187] + +(114) Exchange +Input [6]: [sum#182, isEmpty#183, sum#184, isEmpty#185, sum#186, isEmpty#187] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=17] + +(115) HashAggregate [codegen id : 98] +Input [6]: [sum#182, isEmpty#183, sum#184, isEmpty#185, sum#186, isEmpty#187] +Keys: [] +Functions [3]: [sum(sales#173), sum(returns#174), sum(profit#175)] +Aggregate Attributes [3]: [sum(sales#173)#188, sum(returns#174)#189, sum(profit#175)#190] +Results [5]: [null AS channel#191, null AS id#192, sum(sales#173)#188 AS sales#193, sum(returns#174)#189 AS returns#194, sum(profit#175)#190 AS profit#195] + +(116) Union + +(117) HashAggregate [codegen id : 99] +Input [5]: [channel#34, id#35, sales#126, returns#127, profit#128] +Keys [5]: [channel#34, id#35, sales#126, returns#127, profit#128] +Functions: [] +Aggregate Attributes: [] +Results [5]: [channel#34, id#35, sales#126, returns#127, profit#128] + +(118) Exchange +Input [5]: [channel#34, id#35, sales#126, returns#127, profit#128] +Arguments: hashpartitioning(channel#34, id#35, sales#126, returns#127, profit#128, 5), ENSURE_REQUIREMENTS, [plan_id=18] + +(119) HashAggregate [codegen id : 100] +Input [5]: [channel#34, id#35, sales#126, returns#127, profit#128] +Keys [5]: [channel#34, id#35, sales#126, returns#127, profit#128] +Functions: [] +Aggregate Attributes: [] +Results [5]: [channel#34, id#35, sales#126, returns#127, profit#128] + +(120) TakeOrderedAndProject +Input [5]: [channel#34, id#35, sales#126, returns#127, profit#128] +Arguments: 100, [channel#34 ASC NULLS FIRST, id#35 ASC NULLS FIRST], [channel#34, id#35, sales#126, returns#127, profit#128] + +===== Subqueries ===== + +Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#7 IN dynamicpruning#8 +BroadcastExchange (125) ++- * ColumnarToRow (124) + +- CometProject (123) + +- CometFilter (122) + +- CometScan parquet spark_catalog.default.date_dim (121) + + +(121) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#14, d_date#196] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,1998-08-04), LessThanOrEqual(d_date,1998-09-03), IsNotNull(d_date_sk)] +ReadSchema: struct<d_date_sk:int,d_date:date> + +(122) CometFilter +Input [2]: [d_date_sk#14, d_date#196] +Condition : (((isnotnull(d_date#196) AND (d_date#196 >= 1998-08-04)) AND (d_date#196 <= 1998-09-03)) AND isnotnull(d_date_sk#14)) + +(123) CometProject +Input [2]: [d_date_sk#14, d_date#196] +Arguments: [d_date_sk#14], [d_date_sk#14] + +(124) ColumnarToRow [codegen id : 1] +Input [1]: [d_date_sk#14] + +(125) BroadcastExchange +Input [1]: [d_date_sk#14] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=19] + +Subquery:2 Hosting operator id = 40 Hosting Expression = cs_sold_date_sk#45 IN dynamicpruning#8 + +Subquery:3 Hosting operator id = 71 Hosting Expression = ws_sold_date_sk#81 IN dynamicpruning#8 + + diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q80a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q80a/simplified.txt new file mode 100644 index 000000000..34e47dcba --- /dev/null +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q80a/simplified.txt @@ -0,0 +1,207 @@ +TakeOrderedAndProject [channel,id,sales,returns,profit] + WholeStageCodegen (100) + HashAggregate [channel,id,sales,returns,profit] + InputAdapter + Exchange [channel,id,sales,returns,profit] #1 + WholeStageCodegen (99) + HashAggregate [channel,id,sales,returns,profit] + InputAdapter + Union + WholeStageCodegen (32) + HashAggregate [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),sales,returns,profit,sum,isEmpty,sum,isEmpty,sum,isEmpty] + InputAdapter + Exchange [channel,id] #2 + WholeStageCodegen (31) + HashAggregate [channel,id,sales,returns,profit] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] + InputAdapter + Union + WholeStageCodegen (10) + HashAggregate [s_store_id,sum,sum,isEmpty,sum,isEmpty] [sum(UnscaledValue(ss_ext_sales_price)),sum(coalesce(cast(sr_return_amt as decimal(12,2)), 0.00)),sum((ss_net_profit - coalesce(cast(sr_net_loss as decimal(12,2)), 0.00))),channel,id,sales,returns,profit,sum,sum,isEmpty,sum,isEmpty] + InputAdapter + Exchange [s_store_id] #3 + WholeStageCodegen (9) + HashAggregate [s_store_id,ss_ext_sales_price,sr_return_amt,ss_net_profit,sr_net_loss] [sum,sum,isEmpty,sum,isEmpty,sum,sum,isEmpty,sum,isEmpty] + Project [ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss,s_store_id] + BroadcastHashJoin [ss_promo_sk,p_promo_sk] + Project [ss_promo_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss,s_store_id] + BroadcastHashJoin [ss_item_sk,i_item_sk] + Project [ss_item_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss,s_store_id] + BroadcastHashJoin [ss_store_sk,s_store_sk] + Project [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk,sr_return_amt,sr_net_loss] + SortMergeJoin [ss_item_sk,ss_ticket_number,sr_item_sk,sr_ticket_number] + InputAdapter + WholeStageCodegen (2) + Sort [ss_item_sk,ss_ticket_number] + InputAdapter + Exchange [ss_item_sk,ss_ticket_number] #4 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometFilter [ss_store_sk,ss_item_sk,ss_promo_sk] + CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #5 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] + InputAdapter + WholeStageCodegen (4) + Sort [sr_item_sk,sr_ticket_number] + InputAdapter + Exchange [sr_item_sk,sr_ticket_number] #6 + WholeStageCodegen (3) + ColumnarToRow + InputAdapter + CometProject [sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss] + CometFilter [sr_item_sk,sr_ticket_number] + CometScan parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss,sr_returned_date_sk] + InputAdapter + ReusedExchange [d_date_sk] #5 + InputAdapter + BroadcastExchange #7 + WholeStageCodegen (6) + ColumnarToRow + InputAdapter + CometFilter [s_store_sk] + CometScan parquet spark_catalog.default.store [s_store_sk,s_store_id] + InputAdapter + BroadcastExchange #8 + WholeStageCodegen (7) + ColumnarToRow + InputAdapter + CometProject [i_item_sk] + CometFilter [i_current_price,i_item_sk] + CometScan parquet spark_catalog.default.item [i_item_sk,i_current_price] + InputAdapter + BroadcastExchange #9 + WholeStageCodegen (8) + ColumnarToRow + InputAdapter + CometProject [p_promo_sk] + CometFilter [p_channel_tv,p_promo_sk] + CometScan parquet spark_catalog.default.promotion [p_promo_sk,p_channel_tv] + WholeStageCodegen (20) + HashAggregate [cp_catalog_page_id,sum,sum,isEmpty,sum,isEmpty] [sum(UnscaledValue(cs_ext_sales_price)),sum(coalesce(cast(cr_return_amount as decimal(12,2)), 0.00)),sum((cs_net_profit - coalesce(cast(cr_net_loss as decimal(12,2)), 0.00))),channel,id,sales,returns,profit,sum,sum,isEmpty,sum,isEmpty] + InputAdapter + Exchange [cp_catalog_page_id] #10 + WholeStageCodegen (19) + HashAggregate [cp_catalog_page_id,cs_ext_sales_price,cr_return_amount,cs_net_profit,cr_net_loss] [sum,sum,isEmpty,sum,isEmpty,sum,sum,isEmpty,sum,isEmpty] + Project [cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss,cp_catalog_page_id] + BroadcastHashJoin [cs_promo_sk,p_promo_sk] + Project [cs_promo_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss,cp_catalog_page_id] + BroadcastHashJoin [cs_item_sk,i_item_sk] + Project [cs_item_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss,cp_catalog_page_id] + BroadcastHashJoin [cs_catalog_page_sk,cp_catalog_page_sk] + Project [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + Project [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk,cr_return_amount,cr_net_loss] + SortMergeJoin [cs_item_sk,cs_order_number,cr_item_sk,cr_order_number] + InputAdapter + WholeStageCodegen (12) + Sort [cs_item_sk,cs_order_number] + InputAdapter + Exchange [cs_item_sk,cs_order_number] #11 + WholeStageCodegen (11) + ColumnarToRow + InputAdapter + CometFilter [cs_catalog_page_sk,cs_item_sk,cs_promo_sk] + CometScan parquet spark_catalog.default.catalog_sales [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + InputAdapter + WholeStageCodegen (14) + Sort [cr_item_sk,cr_order_number] + InputAdapter + Exchange [cr_item_sk,cr_order_number] #12 + WholeStageCodegen (13) + ColumnarToRow + InputAdapter + CometProject [cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss] + CometFilter [cr_item_sk,cr_order_number] + CometScan parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss,cr_returned_date_sk] + InputAdapter + ReusedExchange [d_date_sk] #5 + InputAdapter + BroadcastExchange #13 + WholeStageCodegen (16) + ColumnarToRow + InputAdapter + CometFilter [cp_catalog_page_sk] + CometScan parquet spark_catalog.default.catalog_page [cp_catalog_page_sk,cp_catalog_page_id] + InputAdapter + ReusedExchange [i_item_sk] #8 + InputAdapter + ReusedExchange [p_promo_sk] #9 + WholeStageCodegen (30) + HashAggregate [web_site_id,sum,sum,isEmpty,sum,isEmpty] [sum(UnscaledValue(ws_ext_sales_price)),sum(coalesce(cast(wr_return_amt as decimal(12,2)), 0.00)),sum((ws_net_profit - coalesce(cast(wr_net_loss as decimal(12,2)), 0.00))),channel,id,sales,returns,profit,sum,sum,isEmpty,sum,isEmpty] + InputAdapter + Exchange [web_site_id] #14 + WholeStageCodegen (29) + HashAggregate [web_site_id,ws_ext_sales_price,wr_return_amt,ws_net_profit,wr_net_loss] [sum,sum,isEmpty,sum,isEmpty,sum,sum,isEmpty,sum,isEmpty] + Project [ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss,web_site_id] + BroadcastHashJoin [ws_promo_sk,p_promo_sk] + Project [ws_promo_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss,web_site_id] + BroadcastHashJoin [ws_item_sk,i_item_sk] + Project [ws_item_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss,web_site_id] + BroadcastHashJoin [ws_web_site_sk,web_site_sk] + Project [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + Project [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk,wr_return_amt,wr_net_loss] + SortMergeJoin [ws_item_sk,ws_order_number,wr_item_sk,wr_order_number] + InputAdapter + WholeStageCodegen (22) + Sort [ws_item_sk,ws_order_number] + InputAdapter + Exchange [ws_item_sk,ws_order_number] #15 + WholeStageCodegen (21) + ColumnarToRow + InputAdapter + CometFilter [ws_web_site_sk,ws_item_sk,ws_promo_sk] + CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_order_number,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + InputAdapter + WholeStageCodegen (24) + Sort [wr_item_sk,wr_order_number] + InputAdapter + Exchange [wr_item_sk,wr_order_number] #16 + WholeStageCodegen (23) + ColumnarToRow + InputAdapter + CometProject [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss] + CometFilter [wr_item_sk,wr_order_number] + CometScan parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk] + InputAdapter + ReusedExchange [d_date_sk] #5 + InputAdapter + BroadcastExchange #17 + WholeStageCodegen (26) + ColumnarToRow + InputAdapter + CometFilter [web_site_sk] + CometScan parquet spark_catalog.default.web_site [web_site_sk,web_site_id] + InputAdapter + ReusedExchange [i_item_sk] #8 + InputAdapter + ReusedExchange [p_promo_sk] #9 + WholeStageCodegen (65) + HashAggregate [channel,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),id,sales,returns,profit,sum,isEmpty,sum,isEmpty,sum,isEmpty] + InputAdapter + Exchange [channel] #18 + WholeStageCodegen (64) + HashAggregate [channel,sales,returns,profit] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] + HashAggregate [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),sales,returns,profit,sum,isEmpty,sum,isEmpty,sum,isEmpty] + InputAdapter + ReusedExchange [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] #2 + WholeStageCodegen (98) + HashAggregate [sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),channel,id,sales,returns,profit,sum,isEmpty,sum,isEmpty,sum,isEmpty] + InputAdapter + Exchange #19 + WholeStageCodegen (97) + HashAggregate [sales,returns,profit] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] + HashAggregate [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),sales,returns,profit,sum,isEmpty,sum,isEmpty,sum,isEmpty] + InputAdapter + ReusedExchange [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] #2 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 new file mode 100644 index 000000000..58e7f5825 --- /dev/null +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q86a/explain.txt @@ -0,0 +1,240 @@ +== Physical Plan == +TakeOrderedAndProject (34) ++- * Project (33) + +- Window (32) + +- * Sort (31) + +- Exchange (30) + +- * HashAggregate (29) + +- Exchange (28) + +- * HashAggregate (27) + +- Union (26) + :- * HashAggregate (15) + : +- Exchange (14) + : +- * HashAggregate (13) + : +- * Project (12) + : +- * BroadcastHashJoin Inner BuildRight (11) + : :- * Project (6) + : : +- * BroadcastHashJoin Inner BuildRight (5) + : : :- * ColumnarToRow (3) + : : : +- CometFilter (2) + : : : +- CometScan parquet spark_catalog.default.web_sales (1) + : : +- ReusedExchange (4) + : +- BroadcastExchange (10) + : +- * ColumnarToRow (9) + : +- CometFilter (8) + : +- CometScan parquet spark_catalog.default.item (7) + :- * HashAggregate (20) + : +- Exchange (19) + : +- * HashAggregate (18) + : +- * HashAggregate (17) + : +- ReusedExchange (16) + +- * HashAggregate (25) + +- Exchange (24) + +- * HashAggregate (23) + +- * HashAggregate (22) + +- ReusedExchange (21) + + +(1) Scan parquet spark_catalog.default.web_sales +Output [3]: [ws_item_sk#1, ws_net_paid#2, ws_sold_date_sk#3] +Batched: true +Location: InMemoryFileIndex [] +PartitionFilters: [isnotnull(ws_sold_date_sk#3), dynamicpruningexpression(ws_sold_date_sk#3 IN dynamicpruning#4)] +PushedFilters: [IsNotNull(ws_item_sk)] +ReadSchema: struct<ws_item_sk:int,ws_net_paid:decimal(7,2)> + +(2) CometFilter +Input [3]: [ws_item_sk#1, ws_net_paid#2, ws_sold_date_sk#3] +Condition : isnotnull(ws_item_sk#1) + +(3) ColumnarToRow [codegen id : 3] +Input [3]: [ws_item_sk#1, ws_net_paid#2, ws_sold_date_sk#3] + +(4) ReusedExchange [Reuses operator id: 39] +Output [1]: [d_date_sk#5] + +(5) BroadcastHashJoin [codegen id : 3] +Left keys [1]: [ws_sold_date_sk#3] +Right keys [1]: [d_date_sk#5] +Join type: Inner +Join condition: None + +(6) Project [codegen id : 3] +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] + +(7) Scan parquet spark_catalog.default.item +Output [3]: [i_item_sk#6, i_class#7, i_category#8] +Batched: true +Location [not included in comparison]/{warehouse_dir}/item] +PushedFilters: [IsNotNull(i_item_sk)] +ReadSchema: struct<i_item_sk:int,i_class:string,i_category:string> + +(8) CometFilter +Input [3]: [i_item_sk#6, i_class#7, i_category#8] +Condition : isnotnull(i_item_sk#6) + +(9) ColumnarToRow [codegen id : 2] +Input [3]: [i_item_sk#6, i_class#7, i_category#8] + +(10) BroadcastExchange +Input [3]: [i_item_sk#6, i_class#7, i_category#8] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=1] + +(11) BroadcastHashJoin [codegen id : 3] +Left keys [1]: [ws_item_sk#1] +Right keys [1]: [i_item_sk#6] +Join type: Inner +Join condition: None + +(12) Project [codegen id : 3] +Output [3]: [ws_net_paid#2, i_class#7, i_category#8] +Input [5]: [ws_item_sk#1, ws_net_paid#2, i_item_sk#6, i_class#7, i_category#8] + +(13) HashAggregate [codegen id : 3] +Input [3]: [ws_net_paid#2, i_class#7, i_category#8] +Keys [2]: [i_category#8, i_class#7] +Functions [1]: [partial_sum(UnscaledValue(ws_net_paid#2))] +Aggregate Attributes [1]: [sum#9] +Results [3]: [i_category#8, i_class#7, sum#10] + +(14) Exchange +Input [3]: [i_category#8, i_class#7, sum#10] +Arguments: hashpartitioning(i_category#8, i_class#7, 5), ENSURE_REQUIREMENTS, [plan_id=2] + +(15) HashAggregate [codegen id : 4] +Input [3]: [i_category#8, i_class#7, sum#10] +Keys [2]: [i_category#8, i_class#7] +Functions [1]: [sum(UnscaledValue(ws_net_paid#2))] +Aggregate Attributes [1]: [sum(UnscaledValue(ws_net_paid#2))#11] +Results [6]: [cast(MakeDecimal(sum(UnscaledValue(ws_net_paid#2))#11,17,2) as decimal(27,2)) AS total_sum#12, i_category#8, i_class#7, 0 AS g_category#13, 0 AS g_class#14, 0 AS lochierarchy#15] + +(16) ReusedExchange [Reuses operator id: 14] +Output [3]: [i_category#16, i_class#17, sum#18] + +(17) HashAggregate [codegen id : 8] +Input [3]: [i_category#16, i_class#17, sum#18] +Keys [2]: [i_category#16, i_class#17] +Functions [1]: [sum(UnscaledValue(ws_net_paid#19))] +Aggregate Attributes [1]: [sum(UnscaledValue(ws_net_paid#19))#11] +Results [2]: [MakeDecimal(sum(UnscaledValue(ws_net_paid#19))#11,17,2) AS total_sum#20, i_category#16] + +(18) HashAggregate [codegen id : 8] +Input [2]: [total_sum#20, i_category#16] +Keys [1]: [i_category#16] +Functions [1]: [partial_sum(total_sum#20)] +Aggregate Attributes [2]: [sum#21, isEmpty#22] +Results [3]: [i_category#16, sum#23, isEmpty#24] + +(19) Exchange +Input [3]: [i_category#16, sum#23, isEmpty#24] +Arguments: hashpartitioning(i_category#16, 5), ENSURE_REQUIREMENTS, [plan_id=3] + +(20) HashAggregate [codegen id : 9] +Input [3]: [i_category#16, sum#23, isEmpty#24] +Keys [1]: [i_category#16] +Functions [1]: [sum(total_sum#20)] +Aggregate Attributes [1]: [sum(total_sum#20)#25] +Results [6]: [sum(total_sum#20)#25 AS total_sum#26, i_category#16, null AS i_class#27, 0 AS g_category#28, 1 AS g_class#29, 1 AS lochierarchy#30] + +(21) ReusedExchange [Reuses operator id: 14] +Output [3]: [i_category#31, i_class#32, sum#33] + +(22) HashAggregate [codegen id : 13] +Input [3]: [i_category#31, i_class#32, sum#33] +Keys [2]: [i_category#31, i_class#32] +Functions [1]: [sum(UnscaledValue(ws_net_paid#34))] +Aggregate Attributes [1]: [sum(UnscaledValue(ws_net_paid#34))#11] +Results [1]: [MakeDecimal(sum(UnscaledValue(ws_net_paid#34))#11,17,2) AS total_sum#35] + +(23) HashAggregate [codegen id : 13] +Input [1]: [total_sum#35] +Keys: [] +Functions [1]: [partial_sum(total_sum#35)] +Aggregate Attributes [2]: [sum#36, isEmpty#37] +Results [2]: [sum#38, isEmpty#39] + +(24) Exchange +Input [2]: [sum#38, isEmpty#39] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=4] + +(25) HashAggregate [codegen id : 14] +Input [2]: [sum#38, isEmpty#39] +Keys: [] +Functions [1]: [sum(total_sum#35)] +Aggregate Attributes [1]: [sum(total_sum#35)#40] +Results [6]: [sum(total_sum#35)#40 AS total_sum#41, null AS i_category#42, null AS i_class#43, 1 AS g_category#44, 1 AS g_class#45, 2 AS lochierarchy#46] + +(26) Union + +(27) HashAggregate [codegen id : 15] +Input [6]: [total_sum#12, i_category#8, i_class#7, g_category#13, g_class#14, lochierarchy#15] +Keys [6]: [total_sum#12, i_category#8, i_class#7, g_category#13, g_class#14, lochierarchy#15] +Functions: [] +Aggregate Attributes: [] +Results [6]: [total_sum#12, i_category#8, i_class#7, g_category#13, g_class#14, lochierarchy#15] + +(28) Exchange +Input [6]: [total_sum#12, i_category#8, i_class#7, g_category#13, g_class#14, lochierarchy#15] +Arguments: hashpartitioning(total_sum#12, i_category#8, i_class#7, g_category#13, g_class#14, lochierarchy#15, 5), ENSURE_REQUIREMENTS, [plan_id=5] + +(29) HashAggregate [codegen id : 16] +Input [6]: [total_sum#12, i_category#8, i_class#7, g_category#13, g_class#14, lochierarchy#15] +Keys [6]: [total_sum#12, i_category#8, i_class#7, g_category#13, g_class#14, lochierarchy#15] +Functions: [] +Aggregate Attributes: [] +Results [5]: [total_sum#12, i_category#8, i_class#7, lochierarchy#15, CASE WHEN (g_class#14 = 0) THEN i_category#8 END AS _w0#47] + +(30) Exchange +Input [5]: [total_sum#12, i_category#8, i_class#7, lochierarchy#15, _w0#47] +Arguments: hashpartitioning(lochierarchy#15, _w0#47, 5), ENSURE_REQUIREMENTS, [plan_id=6] + +(31) Sort [codegen id : 17] +Input [5]: [total_sum#12, i_category#8, i_class#7, lochierarchy#15, _w0#47] +Arguments: [lochierarchy#15 ASC NULLS FIRST, _w0#47 ASC NULLS FIRST, total_sum#12 DESC NULLS LAST], false, 0 + +(32) Window +Input [5]: [total_sum#12, i_category#8, i_class#7, lochierarchy#15, _w0#47] +Arguments: [rank(total_sum#12) windowspecdefinition(lochierarchy#15, _w0#47, total_sum#12 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rank_within_parent#48], [lochierarchy#15, _w0#47], [total_sum#12 DESC NULLS LAST] + +(33) Project [codegen id : 18] +Output [5]: [total_sum#12, i_category#8, i_class#7, lochierarchy#15, rank_within_parent#48] +Input [6]: [total_sum#12, i_category#8, i_class#7, lochierarchy#15, _w0#47, rank_within_parent#48] + +(34) TakeOrderedAndProject +Input [5]: [total_sum#12, i_category#8, i_class#7, lochierarchy#15, rank_within_parent#48] +Arguments: 100, [lochierarchy#15 DESC NULLS LAST, CASE WHEN (lochierarchy#15 = 0) THEN i_category#8 END ASC NULLS FIRST, rank_within_parent#48 ASC NULLS FIRST], [total_sum#12, i_category#8, i_class#7, lochierarchy#15, rank_within_parent#48] + +===== Subqueries ===== + +Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#3 IN dynamicpruning#4 +BroadcastExchange (39) ++- * ColumnarToRow (38) + +- CometProject (37) + +- CometFilter (36) + +- CometScan parquet spark_catalog.default.date_dim (35) + + +(35) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#5, d_month_seq#49] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1212), LessThanOrEqual(d_month_seq,1223), IsNotNull(d_date_sk)] +ReadSchema: struct<d_date_sk:int,d_month_seq:int> + +(36) CometFilter +Input [2]: [d_date_sk#5, d_month_seq#49] +Condition : (((isnotnull(d_month_seq#49) AND (d_month_seq#49 >= 1212)) AND (d_month_seq#49 <= 1223)) AND isnotnull(d_date_sk#5)) + +(37) CometProject +Input [2]: [d_date_sk#5, d_month_seq#49] +Arguments: [d_date_sk#5], [d_date_sk#5] + +(38) ColumnarToRow [codegen id : 1] +Input [1]: [d_date_sk#5] + +(39) BroadcastExchange +Input [1]: [d_date_sk#5] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=7] + + diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q86a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q86a/simplified.txt new file mode 100644 index 000000000..5c5e08885 --- /dev/null +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q86a/simplified.txt @@ -0,0 +1,66 @@ +TakeOrderedAndProject [lochierarchy,i_category,rank_within_parent,total_sum,i_class] + WholeStageCodegen (18) + Project [total_sum,i_category,i_class,lochierarchy,rank_within_parent] + InputAdapter + Window [total_sum,lochierarchy,_w0] + WholeStageCodegen (17) + Sort [lochierarchy,_w0,total_sum] + InputAdapter + Exchange [lochierarchy,_w0] #1 + WholeStageCodegen (16) + HashAggregate [total_sum,i_category,i_class,g_category,g_class,lochierarchy] [_w0] + InputAdapter + Exchange [total_sum,i_category,i_class,g_category,g_class,lochierarchy] #2 + WholeStageCodegen (15) + 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,g_category,g_class,lochierarchy,sum] + InputAdapter + Exchange [i_category,i_class] #3 + WholeStageCodegen (3) + HashAggregate [i_category,i_class,ws_net_paid] [sum,sum] + Project [ws_net_paid,i_class,i_category] + BroadcastHashJoin [ws_item_sk,i_item_sk] + Project [ws_item_sk,ws_net_paid] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + ColumnarToRow + InputAdapter + CometFilter [ws_item_sk] + CometScan parquet spark_catalog.default.web_sales [ws_item_sk,ws_net_paid,ws_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #4 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_month_seq,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] + InputAdapter + ReusedExchange [d_date_sk] #4 + InputAdapter + BroadcastExchange #5 + WholeStageCodegen (2) + ColumnarToRow + InputAdapter + CometFilter [i_item_sk] + CometScan parquet spark_catalog.default.item [i_item_sk,i_class,i_category] + WholeStageCodegen (9) + HashAggregate [i_category,sum,isEmpty] [sum(total_sum),total_sum,i_class,g_category,g_class,lochierarchy,sum,isEmpty] + InputAdapter + Exchange [i_category] #6 + WholeStageCodegen (8) + HashAggregate [i_category,total_sum] [sum,isEmpty,sum,isEmpty] + HashAggregate [i_category,i_class,sum] [sum(UnscaledValue(ws_net_paid)),total_sum,sum] + InputAdapter + ReusedExchange [i_category,i_class,sum] #3 + WholeStageCodegen (14) + HashAggregate [sum,isEmpty] [sum(total_sum),total_sum,i_category,i_class,g_category,g_class,lochierarchy,sum,isEmpty] + InputAdapter + Exchange #7 + WholeStageCodegen (13) + HashAggregate [total_sum] [sum,isEmpty,sum,isEmpty] + HashAggregate [i_category,i_class,sum] [sum(UnscaledValue(ws_net_paid)),total_sum,sum] + InputAdapter + ReusedExchange [i_category,i_class,sum] #3 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 new file mode 100644 index 000000000..f77b7ec93 --- /dev/null +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q98/explain.txt @@ -0,0 +1,155 @@ +== Physical Plan == +* Sort (21) ++- Exchange (20) + +- * Project (19) + +- Window (18) + +- * Sort (17) + +- Exchange (16) + +- * HashAggregate (15) + +- Exchange (14) + +- * HashAggregate (13) + +- * Project (12) + +- * BroadcastHashJoin Inner BuildRight (11) + :- * Project (9) + : +- * BroadcastHashJoin Inner BuildRight (8) + : :- * ColumnarToRow (3) + : : +- CometFilter (2) + : : +- CometScan parquet spark_catalog.default.store_sales (1) + : +- BroadcastExchange (7) + : +- * ColumnarToRow (6) + : +- CometFilter (5) + : +- CometScan parquet spark_catalog.default.item (4) + +- ReusedExchange (10) + + +(1) Scan parquet spark_catalog.default.store_sales +Output [3]: [ss_item_sk#1, ss_ext_sales_price#2, ss_sold_date_sk#3] +Batched: true +Location: InMemoryFileIndex [] +PartitionFilters: [isnotnull(ss_sold_date_sk#3), dynamicpruningexpression(ss_sold_date_sk#3 IN dynamicpruning#4)] +PushedFilters: [IsNotNull(ss_item_sk)] +ReadSchema: struct<ss_item_sk:int,ss_ext_sales_price:decimal(7,2)> + +(2) CometFilter +Input [3]: [ss_item_sk#1, ss_ext_sales_price#2, ss_sold_date_sk#3] +Condition : isnotnull(ss_item_sk#1) + +(3) ColumnarToRow [codegen id : 3] +Input [3]: [ss_item_sk#1, ss_ext_sales_price#2, ss_sold_date_sk#3] + +(4) Scan parquet spark_catalog.default.item +Output [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] +Batched: true +Location [not included in comparison]/{warehouse_dir}/item] +PushedFilters: [In(i_category, [Books ,Home ,Sports ]), IsNotNull(i_item_sk)] +ReadSchema: struct<i_item_sk:int,i_item_id:string,i_item_desc:string,i_current_price:decimal(7,2),i_class:string,i_category:string> + +(5) CometFilter +Input [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] +Condition : (i_category#10 IN (Sports ,Books ,Home ) AND isnotnull(i_item_sk#5)) + +(6) ColumnarToRow [codegen id : 1] +Input [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] + +(7) BroadcastExchange +Input [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=1] + +(8) BroadcastHashJoin [codegen id : 3] +Left keys [1]: [ss_item_sk#1] +Right keys [1]: [i_item_sk#5] +Join type: Inner +Join condition: None + +(9) Project [codegen id : 3] +Output [7]: [ss_ext_sales_price#2, ss_sold_date_sk#3, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] +Input [9]: [ss_item_sk#1, ss_ext_sales_price#2, ss_sold_date_sk#3, i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] + +(10) ReusedExchange [Reuses operator id: 26] +Output [1]: [d_date_sk#11] + +(11) BroadcastHashJoin [codegen id : 3] +Left keys [1]: [ss_sold_date_sk#3] +Right keys [1]: [d_date_sk#11] +Join type: Inner +Join condition: None + +(12) Project [codegen id : 3] +Output [6]: [ss_ext_sales_price#2, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] +Input [8]: [ss_ext_sales_price#2, ss_sold_date_sk#3, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10, d_date_sk#11] + +(13) HashAggregate [codegen id : 3] +Input [6]: [ss_ext_sales_price#2, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] +Keys [5]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8] +Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#2))] +Aggregate Attributes [1]: [sum#12] +Results [6]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, sum#13] + +(14) Exchange +Input [6]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, sum#13] +Arguments: hashpartitioning(i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, 5), ENSURE_REQUIREMENTS, [plan_id=2] + +(15) HashAggregate [codegen id : 4] +Input [6]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, sum#13] +Keys [5]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8] +Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#2))] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#2))#14] +Results [7]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#2))#14,17,2) AS itemrevenue#15, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#2))#14,17,2) AS _w0#16] + +(16) Exchange +Input [7]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#15, _w0#16] +Arguments: hashpartitioning(i_class#9, 5), ENSURE_REQUIREMENTS, [plan_id=3] + +(17) Sort [codegen id : 5] +Input [7]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#15, _w0#16] +Arguments: [i_class#9 ASC NULLS FIRST], false, 0 + +(18) Window +Input [7]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#15, _w0#16] +Arguments: [sum(_w0#16) windowspecdefinition(i_class#9, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS _we0#17], [i_class#9] + +(19) Project [codegen id : 6] +Output [7]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#15, ((_w0#16 * 100) / _we0#17) AS revenueratio#18] +Input [8]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#15, _w0#16, _we0#17] + +(20) Exchange +Input [7]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#15, revenueratio#18] +Arguments: rangepartitioning(i_category#10 ASC NULLS FIRST, i_class#9 ASC NULLS FIRST, i_item_id#6 ASC NULLS FIRST, i_item_desc#7 ASC NULLS FIRST, revenueratio#18 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, [plan_id=4] + +(21) Sort [codegen id : 7] +Input [7]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#15, revenueratio#18] +Arguments: [i_category#10 ASC NULLS FIRST, i_class#9 ASC NULLS FIRST, i_item_id#6 ASC NULLS FIRST, i_item_desc#7 ASC NULLS FIRST, revenueratio#18 ASC NULLS FIRST], true, 0 + +===== Subqueries ===== + +Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#3 IN dynamicpruning#4 +BroadcastExchange (26) ++- * ColumnarToRow (25) + +- CometProject (24) + +- CometFilter (23) + +- CometScan parquet spark_catalog.default.date_dim (22) + + +(22) Scan parquet spark_catalog.default.date_dim +Output [2]: [d_date_sk#11, d_date#19] +Batched: true +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,1999-02-22), LessThanOrEqual(d_date,1999-03-24), IsNotNull(d_date_sk)] +ReadSchema: struct<d_date_sk:int,d_date:date> + +(23) CometFilter +Input [2]: [d_date_sk#11, d_date#19] +Condition : (((isnotnull(d_date#19) AND (d_date#19 >= 1999-02-22)) AND (d_date#19 <= 1999-03-24)) AND isnotnull(d_date_sk#11)) + +(24) CometProject +Input [2]: [d_date_sk#11, d_date#19] +Arguments: [d_date_sk#11], [d_date_sk#11] + +(25) ColumnarToRow [codegen id : 1] +Input [1]: [d_date_sk#11] + +(26) BroadcastExchange +Input [1]: [d_date_sk#11] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] + + diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q98/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q98/simplified.txt new file mode 100644 index 000000000..b7489a0af --- /dev/null +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q98/simplified.txt @@ -0,0 +1,43 @@ +WholeStageCodegen (7) + Sort [i_category,i_class,i_item_id,i_item_desc,revenueratio] + InputAdapter + Exchange [i_category,i_class,i_item_id,i_item_desc,revenueratio] #1 + WholeStageCodegen (6) + Project [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,_we0] + InputAdapter + Window [_w0,i_class] + WholeStageCodegen (5) + Sort [i_class] + InputAdapter + Exchange [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] + InputAdapter + Exchange [i_item_id,i_item_desc,i_category,i_class,i_current_price] #3 + WholeStageCodegen (3) + HashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,ss_ext_sales_price] [sum,sum] + Project [ss_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [ss_ext_sales_price,ss_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + BroadcastHashJoin [ss_item_sk,i_item_sk] + ColumnarToRow + InputAdapter + CometFilter [ss_item_sk] + CometScan parquet spark_catalog.default.store_sales [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #4 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date,d_date_sk] + CometScan parquet spark_catalog.default.date_dim [d_date_sk,d_date] + InputAdapter + BroadcastExchange #5 + WholeStageCodegen (1) + ColumnarToRow + InputAdapter + CometFilter [i_category,i_item_sk] + CometScan parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + InputAdapter + ReusedExchange [d_date_sk] #4 diff --git a/spark/src/test/scala/org/apache/comet/CometCastSuite.scala b/spark/src/test/scala/org/apache/comet/CometCastSuite.scala index 1710090e2..fd2218965 100644 --- a/spark/src/test/scala/org/apache/comet/CometCastSuite.scala +++ b/spark/src/test/scala/org/apache/comet/CometCastSuite.scala @@ -982,12 +982,18 @@ class CometCastSuite extends CometTestBase with AdaptiveSparkPlanHelper { fail(s"Comet should have failed with ${e.getCause.getMessage}") case (Some(sparkException), Some(cometException)) => // both systems threw an exception so we make sure they are the same - val sparkMessage = sparkException.getCause.getMessage + val sparkMessage = + if (sparkException.getCause != null) sparkException.getCause.getMessage else null // We have to workaround https://github.com/apache/datafusion-comet/issues/293 here by // removing the "Execution error: " error message prefix that is added by DataFusion - val cometMessage = cometException.getCause.getMessage - .replace("Execution error: ", "") - if (CometSparkSessionExtensions.isSpark34Plus) { + val cometMessage = cometException.getCause.getMessage.replace("Execution error: ", "") + if (CometSparkSessionExtensions.isSpark40Plus) { + // for Spark 4 we expect to sparkException carries the message + assert( + sparkException.getMessage + .replace(".WITH_SUGGESTION] ", "]") + .startsWith(cometMessage)) + } else if (CometSparkSessionExtensions.isSpark34Plus) { // for Spark 3.4 we expect to reproduce the error message exactly assert(cometMessage == sparkMessage) } else if (CometSparkSessionExtensions.isSpark33Plus) { 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 7c19890d3..a0afe6b0c 100644 --- a/spark/src/test/scala/org/apache/comet/exec/CometExecSuite.scala +++ b/spark/src/test/scala/org/apache/comet/exec/CometExecSuite.scala @@ -49,7 +49,7 @@ import org.apache.spark.sql.internal.SQLConf.SESSION_LOCAL_TIMEZONE import org.apache.spark.unsafe.types.UTF8String import org.apache.comet.CometConf -import org.apache.comet.CometSparkSessionExtensions.{isSpark33Plus, isSpark34Plus} +import org.apache.comet.CometSparkSessionExtensions.{isSpark33Plus, isSpark34Plus, isSpark40Plus} class CometExecSuite extends CometTestBase { import testImplicits._ @@ -1055,7 +1055,11 @@ class CometExecSuite extends CometTestBase { val e = intercept[AnalysisException] { sql("CREATE TABLE t2(name STRING, part INTERVAL) USING PARQUET PARTITIONED BY (part)") }.getMessage - assert(e.contains("Cannot use interval")) + if (isSpark40Plus) { + assert(e.contains(" Cannot use \"INTERVAL\"")) + } else { + assert(e.contains("Cannot use interval")) + } } } } diff --git a/spark/src/test/scala/org/apache/comet/parquet/ParquetReadSuite.scala b/spark/src/test/scala/org/apache/comet/parquet/ParquetReadSuite.scala index f44752297..0b37f5ccf 100644 --- a/spark/src/test/scala/org/apache/comet/parquet/ParquetReadSuite.scala +++ b/spark/src/test/scala/org/apache/comet/parquet/ParquetReadSuite.scala @@ -34,9 +34,7 @@ import org.apache.hadoop.fs.Path import org.apache.parquet.example.data.simple.SimpleGroup import org.apache.parquet.schema.MessageTypeParser import org.apache.spark.SparkException -import org.apache.spark.sql.CometTestBase -import org.apache.spark.sql.DataFrame -import org.apache.spark.sql.Row +import org.apache.spark.sql.{CometTestBase, DataFrame, Row} import org.apache.spark.sql.catalyst.expressions.GenericInternalRow import org.apache.spark.sql.catalyst.util.DateTimeUtils import org.apache.spark.sql.comet.CometBatchScanExec @@ -49,7 +47,7 @@ import org.apache.spark.unsafe.types.UTF8String import com.google.common.primitives.UnsignedLong import org.apache.comet.CometConf -import org.apache.comet.CometSparkSessionExtensions.isSpark34Plus +import org.apache.comet.CometSparkSessionExtensions.{isSpark34Plus, isSpark40Plus} abstract class ParquetReadSuite extends CometTestBase { import testImplicits._ @@ -1125,7 +1123,9 @@ abstract class ParquetReadSuite extends CometTestBase { } test("row group skipping doesn't overflow when reading into larger type") { - assume(isSpark34Plus) + // Spark 4.0 no longer fails for widening types + // https://github.com/apache/spark/commit/3361f25dc0ff6e5233903c26ee105711b79ba967 + assume(isSpark34Plus && !isSpark40Plus) withTempPath { path => Seq(0).toDF("a").write.parquet(path.toString) diff --git a/spark/src/test/scala/org/apache/spark/sql/comet/CometPlanStabilitySuite.scala b/spark/src/test/scala/org/apache/spark/sql/comet/CometPlanStabilitySuite.scala index 79d61f524..691d2cd63 100644 --- a/spark/src/test/scala/org/apache/spark/sql/comet/CometPlanStabilitySuite.scala +++ b/spark/src/test/scala/org/apache/spark/sql/comet/CometPlanStabilitySuite.scala @@ -36,7 +36,7 @@ import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.test.TestSparkSession import org.apache.comet.CometConf -import org.apache.comet.CometSparkSessionExtensions.isSpark34Plus +import org.apache.comet.CometSparkSessionExtensions.{isSpark34Plus, isSpark40Plus} /** * Similar to [[org.apache.spark.sql.PlanStabilitySuite]], checks that TPC-DS Comet plans don't @@ -298,8 +298,10 @@ trait CometPlanStabilitySuite extends DisableAdaptiveExecutionSuite with TPCDSBa } class CometTPCDSV1_4_PlanStabilitySuite extends CometPlanStabilitySuite { + private val planName = + if (isSpark40Plus) "approved-plans-v1_4-spark4_0" else "approved-plans-v1_4" override val goldenFilePath: String = - new File(baseResourcePath, "approved-plans-v1_4").getAbsolutePath + new File(baseResourcePath, planName).getAbsolutePath tpcdsQueries.foreach { q => test(s"check simplified (tpcds-v1.4/$q)") { @@ -309,8 +311,10 @@ class CometTPCDSV1_4_PlanStabilitySuite extends CometPlanStabilitySuite { } class CometTPCDSV2_7_PlanStabilitySuite extends CometPlanStabilitySuite { + private val planName = + if (isSpark40Plus) "approved-plans-v2_7-spark4_0" else "approved-plans-v2_7" override val goldenFilePath: String = - new File(baseResourcePath, "approved-plans-v2_7").getAbsolutePath + new File(baseResourcePath, planName).getAbsolutePath tpcdsQueriesV2_7_0.foreach { q => test(s"check simplified (tpcds-v2.7.0/$q)") { diff --git a/spark/src/test/scala/org/apache/spark/sql/comet/ParquetDatetimeRebaseSuite.scala b/spark/src/test/scala/org/apache/spark/sql/comet/ParquetDatetimeRebaseSuite.scala index 5bb7e8f70..a829d7449 100644 --- a/spark/src/test/scala/org/apache/spark/sql/comet/ParquetDatetimeRebaseSuite.scala +++ b/spark/src/test/scala/org/apache/spark/sql/comet/ParquetDatetimeRebaseSuite.scala @@ -27,6 +27,7 @@ import org.apache.spark.sql.{CometTestBase, DataFrame, Dataset, Row} import org.apache.spark.sql.internal.SQLConf import org.apache.comet.CometConf +import org.apache.comet.CometSparkSessionExtensions.isSpark40Plus // This test checks if Comet reads ancient dates & timestamps that are before 1582, as if they are // read according to the `LegacyBehaviorPolicy.CORRECTED` mode (i.e., no rebase) in Spark. @@ -48,7 +49,8 @@ abstract class ParquetDatetimeRebaseSuite extends CometTestBase { val df = spark.read.parquet(file) // Parquet file written by 2.4.5 should throw exception for both Spark and Comet - if (exceptionOnRebase || sparkVersion == "2_4_5") { + // For Spark 4.0+, Parquet file written by 2.4.5 should not throw exception + if ((exceptionOnRebase || sparkVersion == "2_4_5") && (!isSpark40Plus || sparkVersion != "2_4_5")) { intercept[SparkException](df.collect()) } else { checkSparkNoRebaseAnswer(df) @@ -70,7 +72,8 @@ abstract class ParquetDatetimeRebaseSuite extends CometTestBase { val df = spark.read.parquet(file) // Parquet file written by 2.4.5 should throw exception for both Spark and Comet - if (exceptionOnRebase || sparkVersion == "2_4_5") { + // For Spark 4.0+, Parquet file written by 2.4.5 should not throw exception + if ((exceptionOnRebase || sparkVersion == "2_4_5") && (!isSpark40Plus || sparkVersion != "2_4_5")) { intercept[SparkException](df.collect()) } else { checkSparkNoRebaseAnswer(df) @@ -93,7 +96,8 @@ abstract class ParquetDatetimeRebaseSuite extends CometTestBase { val df = spark.read.parquet(file) // Parquet file written by 2.4.5 should throw exception for both Spark and Comet - if (exceptionOnRebase || sparkVersion == "2_4_5") { + // For Spark 4.0+, Parquet file written by 2.4.5 should not throw exception + if ((exceptionOnRebase || sparkVersion == "2_4_5") && (!isSpark40Plus || sparkVersion != "2_4_5")) { intercept[SparkException](df.collect()) } else { checkSparkNoRebaseAnswer(df) diff --git a/spark/src/test/spark-3.3-plus/org/apache/comet/CometExpression3_3PlusSuite.scala b/spark/src/test/spark-3.3-plus/org/apache/comet/CometExpression3_3PlusSuite.scala index 6102777fc..d54c8dad0 100644 --- a/spark/src/test/spark-3.3-plus/org/apache/comet/CometExpression3_3PlusSuite.scala +++ b/spark/src/test/spark-3.3-plus/org/apache/comet/CometExpression3_3PlusSuite.scala @@ -25,7 +25,6 @@ import org.apache.spark.sql.catalyst.FunctionIdentifier import org.apache.spark.sql.catalyst.expressions.{BloomFilterMightContain, Expression, ExpressionInfo} import org.apache.spark.sql.functions.{col, lit} import org.apache.spark.util.sketch.BloomFilter - import java.io.ByteArrayOutputStream import scala.util.Random diff --git a/spark/src/test/spark-3.4-plus/org/apache/comet/exec/CometExec3_4PlusSuite.scala b/spark/src/test/spark-3.4-plus/org/apache/comet/exec/CometExec3_4PlusSuite.scala index 31d1ffbf7..764f7b18d 100644 --- a/spark/src/test/spark-3.4-plus/org/apache/comet/exec/CometExec3_4PlusSuite.scala +++ b/spark/src/test/spark-3.4-plus/org/apache/comet/exec/CometExec3_4PlusSuite.scala @@ -23,7 +23,6 @@ import org.scalactic.source.Position import org.scalatest.Tag import org.apache.spark.sql.CometTestBase - import org.apache.comet.CometConf /**