diff --git a/common/src/main/scala/org/apache/comet/CometConf.scala b/common/src/main/scala/org/apache/comet/CometConf.scala index 22af76f6e4..15c160ed6c 100644 --- a/common/src/main/scala/org/apache/comet/CometConf.scala +++ b/common/src/main/scala/org/apache/comet/CometConf.scala @@ -325,14 +325,11 @@ object CometConf extends ShimCometConf { .booleanConf .createWithDefault(true) - // RangePartitioning contains bugs https://github.com/apache/datafusion-comet/issues/1906 val COMET_EXEC_SHUFFLE_WITH_RANGE_PARTITIONING_ENABLED: ConfigEntry[Boolean] = conf("spark.comet.native.shuffle.partitioning.range.enabled") - .doc("Experimental feature to enable range partitioning for Comet native shuffle. " + - "This feature is experimental while we investigate scenarios that don't partition data " + - "correctly.") + .doc("Whether to enable range partitioning for Comet native shuffle.") .booleanConf - .createWithDefault(false) + .createWithDefault(true) val COMET_EXEC_SHUFFLE_COMPRESSION_CODEC: ConfigEntry[String] = conf(s"$COMET_EXEC_CONFIG_PREFIX.shuffle.compression.codec") diff --git a/docs/source/user-guide/latest/configs.md b/docs/source/user-guide/latest/configs.md index fdfc2da35c..410386e958 100644 --- a/docs/source/user-guide/latest/configs.md +++ b/docs/source/user-guide/latest/configs.md @@ -72,7 +72,7 @@ Comet provides the following configuration settings. | spark.comet.memoryOverhead | The amount of additional memory to be allocated per executor process for Comet, in MiB, when running Spark in on-heap mode. This config is optional. If this is not specified, it will be set to `spark.comet.memory.overhead.factor` * `spark.executor.memory`. For more information, refer to the Comet Tuning Guide (https://datafusion.apache.org/comet/user-guide/tuning.html). | | | spark.comet.metrics.updateInterval | The interval in milliseconds to update metrics. If interval is negative, metrics will be updated upon task completion. | 3000 | | spark.comet.native.shuffle.partitioning.hash.enabled | Whether to enable hash partitioning for Comet native shuffle. | true | -| spark.comet.native.shuffle.partitioning.range.enabled | Experimental feature to enable range partitioning for Comet native shuffle. This feature is experimental while we investigate scenarios that don't partition data correctly. | false | +| spark.comet.native.shuffle.partitioning.range.enabled | Whether to enable range partitioning for Comet native shuffle. | true | | spark.comet.nativeLoadRequired | Whether to require Comet native library to load successfully when Comet is enabled. If not, Comet will silently fallback to Spark when it fails to load the native lib. Otherwise, an error will be thrown and the Spark job will be aborted. | false | | spark.comet.parquet.enable.directBuffer | Whether to use Java direct byte buffer when reading Parquet. | false | | spark.comet.parquet.read.io.adjust.readRange.skew | In the parallel reader, if the read ranges submitted are skewed in sizes, this option will cause the reader to break up larger read ranges into smaller ranges to reduce the skew. This will result in a slightly larger number of connections opened to the file system but may give improved performance. | false | diff --git a/native/core/benches/shuffle_writer.rs b/native/core/benches/shuffle_writer.rs index 52638d92a9..70f1acc8db 100644 --- a/native/core/benches/shuffle_writer.rs +++ b/native/core/benches/shuffle_writer.rs @@ -16,8 +16,9 @@ // under the License. use arrow::array::builder::{Date32Builder, Decimal128Builder, Int32Builder}; -use arrow::array::{builder::StringBuilder, RecordBatch}; +use arrow::array::{builder::StringBuilder, Array, Int32Array, RecordBatch}; use arrow::datatypes::{DataType, Field, Schema}; +use arrow::row::{RowConverter, SortField}; use comet::execution::shuffle::{ CometPartitioning, CompressionCodec, ShuffleBlockWriter, ShuffleWriterExec, }; @@ -31,6 +32,7 @@ use datafusion::{ physical_plan::{common::collect, ExecutionPlan}, prelude::SessionContext, }; +use itertools::Itertools; use std::io::Cursor; use std::sync::Arc; use tokio::runtime::Runtime; @@ -84,16 +86,37 @@ fn criterion_benchmark(c: &mut Criterion) { ); } + let lex_ordering = LexOrdering::new(vec![PhysicalSortExpr::new_default( + col("c0", batch.schema().as_ref()).unwrap(), + )]) + .unwrap(); + + let sort_fields: Vec = batch + .columns() + .iter() + .zip(&lex_ordering) + .map(|(array, sort_expr)| { + SortField::new_with_options(array.data_type().clone(), sort_expr.options) + }) + .collect(); + let row_converter = RowConverter::new(sort_fields).unwrap(); + + // These are hard-coded values based on the benchmark params of 8192 rows per batch, and 16 + // partitions. If these change, these values need to be recalculated, or bring over the + // bounds-finding logic from shuffle_write_test in shuffle_writer.rs. + let bounds_ints = vec![ + 512, 1024, 1536, 2048, 2560, 3072, 3584, 4096, 4608, 5120, 5632, 6144, 6656, 7168, 7680, + ]; + let bounds_array: Arc = Arc::new(Int32Array::from(bounds_ints)); + let bounds_rows = row_converter + .convert_columns(vec![bounds_array].as_slice()) + .unwrap(); + + let owned_rows = bounds_rows.iter().map(|row| row.owned()).collect_vec(); + for partitioning in [ CometPartitioning::Hash(vec![Arc::new(Column::new("a", 0))], 16), - CometPartitioning::RangePartitioning( - LexOrdering::new(vec![PhysicalSortExpr::new_default( - col("c0", batch.schema().as_ref()).unwrap(), - )]) - .unwrap(), - 16, - 100, - ), + CometPartitioning::RangePartitioning(lex_ordering, 16, Arc::new(row_converter), owned_rows), ] { let compression_codec = CompressionCodec::None; group.bench_function( diff --git a/native/core/src/execution/planner.rs b/native/core/src/execution/planner.rs index 5aa6ece3bc..64efa31d52 100644 --- a/native/core/src/execution/planner.rs +++ b/native/core/src/execution/planner.rs @@ -92,6 +92,7 @@ use arrow::array::{ NullArray, StringBuilder, TimestampMicrosecondArray, }; use arrow::buffer::{BooleanBuffer, NullBuffer, OffsetBuffer}; +use arrow::row::{OwnedRow, RowConverter, SortField}; use datafusion::common::utils::SingleRowListArrayBuilder; use datafusion::physical_plan::coalesce_batches::CoalesceBatchesExec; use datafusion::physical_plan::filter::FilterExec; @@ -484,14 +485,14 @@ impl PhysicalPlanner { ))) } } - }, + } Value::ListVal(values) => { if let DataType::List(_) = data_type { SingleRowListArrayBuilder::new(literal_to_array_ref(data_type, values.clone())?).build_list_scalar() } else { return Err(GeneralError(format!( "Expected DataType::List but got {data_type:?}" - ))) + ))); } } } @@ -1402,8 +1403,14 @@ impl PhysicalPlanner { assert_eq!(children.len(), 1); let (scans, child) = self.create_plan(&children[0], inputs, partition_count)?; - let partitioning = self - .create_partitioning(writer.partitioning.as_ref().unwrap(), child.schema())?; + // We wrap native shuffle in a CopyExec. This existed previously, but for + // RangePartitioning at least we want to ensure that dictionaries are unpacked. + let wrapped_child = Self::wrap_in_copy_exec(Arc::clone(&child.native_plan)); + + let partitioning = self.create_partitioning( + writer.partitioning.as_ref().unwrap(), + wrapped_child.schema(), + )?; let codec = match writer.codec.try_into() { Ok(SparkCompressionCodec::None) => Ok(CompressionCodec::None), @@ -1419,7 +1426,7 @@ impl PhysicalPlanner { }?; let shuffle_writer = Arc::new(ShuffleWriterExec::try_new( - Self::wrap_in_copy_exec(Arc::clone(&child.native_plan)), + wrapped_child, partitioning, codec, writer.output_data_file.clone(), @@ -2344,16 +2351,65 @@ impl PhysicalPlanner { )) } PartitioningStruct::RangePartition(range_partition) => { + // Generate the lexical ordering for comparisons let exprs: Result, ExecutionError> = range_partition .sort_orders .iter() .map(|expr| self.create_sort_expr(expr, Arc::clone(&input_schema))) .collect(); let lex_ordering = LexOrdering::new(exprs?).unwrap(); + + // Generate the row converter for comparing incoming batches to boundary rows + let sort_fields: Vec = lex_ordering + .iter() + .map(|sort_expr| { + sort_expr + .expr + .data_type(input_schema.as_ref()) + .map(|dt| SortField::new_with_options(dt, sort_expr.options)) + }) + .collect::, _>>()?; + + // Deserialize the literals to columnar collections of ScalarValues + let mut scalar_values: Vec> = vec![vec![]; lex_ordering.len()]; + for boundary_row in &range_partition.boundary_rows { + // For each serialized expr in a boundary row, convert to a Literal + // expression, then extract the ScalarValue from the Literal and push it + // into the collection of ScalarValues + for (col_idx, col_values) in scalar_values + .iter_mut() + .enumerate() + .take(lex_ordering.len()) + { + let expr = self.create_expr( + &boundary_row.partition_bounds[col_idx], + Arc::clone(&input_schema), + )?; + let literal_expr = + expr.as_any().downcast_ref::().expect("Literal"); + col_values.push(literal_expr.value().clone()); + } + } + + // Convert the collection of ScalarValues to collection of Arrow Arrays + let arrays: Vec = scalar_values + .iter() + .map(|scalar_vec| ScalarValue::iter_to_array(scalar_vec.iter().cloned())) + .collect::, _>>()?; + + // Create a RowConverter and use to create OwnedRows from the Arrays + let converter = RowConverter::new(sort_fields)?; + let boundary_rows = converter.convert_columns(&arrays)?; + // Rows are only a view into Arrow Arrays. We need to create OwnedRows with their + // own internal memory ownership to pass as our boundary values to the partitioner. + let boundary_owned_rows: Vec = + boundary_rows.iter().map(|row| row.owned()).collect(); + Ok(CometPartitioning::RangePartitioning( lex_ordering, range_partition.num_partitions as usize, - range_partition.sample_size as usize, + Arc::new(converter), + boundary_owned_rows, )) } PartitioningStruct::SinglePartition(_) => Ok(CometPartitioning::SinglePartition), diff --git a/native/core/src/execution/shuffle/comet_partitioning.rs b/native/core/src/execution/shuffle/comet_partitioning.rs index 9c33da8e93..a2422cf9e6 100644 --- a/native/core/src/execution/shuffle/comet_partitioning.rs +++ b/native/core/src/execution/shuffle/comet_partitioning.rs @@ -15,6 +15,7 @@ // specific language governing permissions and limitations // under the License. +use arrow::row::{OwnedRow, RowConverter}; use datafusion::physical_expr::{LexOrdering, PhysicalExpr}; use std::sync::Arc; @@ -22,11 +23,14 @@ use std::sync::Arc; pub enum CometPartitioning { SinglePartition, /// Allocate rows based on a hash of one of more expressions and the specified number of - /// partitions + /// partitions. Args are 1) the expression to hash on, and 2) the number of partitions. Hash(Vec>, usize), /// Allocate rows based on the lexical order of one of more expressions and the specified number of - /// partitions - RangePartitioning(LexOrdering, usize, usize), + /// partitions. Args are 1) the LexOrdering to use to compare values and split into partitions, + /// 2) the number of partitions, 3) the RowConverter used to view incoming RecordBatches as Arrow + /// Rows for comparing to 4) OwnedRows that represent the boundaries of each partition, used with + /// LexOrdering to bin each value in the RecordBatch to a partition. + RangePartitioning(LexOrdering, usize, Arc, Vec), } impl CometPartitioning { @@ -34,7 +38,7 @@ impl CometPartitioning { use CometPartitioning::*; match self { SinglePartition => 1, - Hash(_, n) | RangePartitioning(_, n, _) => *n, + Hash(_, n) | RangePartitioning(_, n, _, _) => *n, } } } diff --git a/native/core/src/execution/shuffle/mod.rs b/native/core/src/execution/shuffle/mod.rs index c3cb610afa..e2798df63e 100644 --- a/native/core/src/execution/shuffle/mod.rs +++ b/native/core/src/execution/shuffle/mod.rs @@ -19,7 +19,6 @@ pub(crate) mod codec; mod comet_partitioning; mod list; mod map; -mod range_partitioner; pub mod row; mod shuffle_writer; diff --git a/native/core/src/execution/shuffle/range_partitioner.rs b/native/core/src/execution/shuffle/range_partitioner.rs deleted file mode 100644 index e14cf61482..0000000000 --- a/native/core/src/execution/shuffle/range_partitioner.rs +++ /dev/null @@ -1,474 +0,0 @@ -// Licensed to the Apache Software Foundation (ASF) under one -// or more contributor license agreements. See the NOTICE file -// distributed with this work for additional information -// regarding copyright ownership. The ASF licenses this file -// to you under the Apache License, Version 2.0 (the -// "License"); you may not use this file except in compliance -// with the License. You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, -// software distributed under the License is distributed on an -// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -// KIND, either express or implied. See the License for the -// specific language governing permissions and limitations -// under the License. - -use arrow::array::{ArrayRef, UInt64Array}; -use arrow::compute::{take_arrays, TakeOptions}; -use arrow::row::{OwnedRow, Row, RowConverter, Rows, SortField}; -use datafusion::error::{DataFusionError, Result}; -use datafusion::physical_expr::LexOrdering; -use rand::{rngs::SmallRng, Rng, SeedableRng}; - -pub struct RangePartitioner; - -impl RangePartitioner { - /// Given a number of rows, sample size, and a random seed, generates unique indices to take - /// from an input batch to act as a random sample. - /// Adapted from https://en.wikipedia.org/wiki/Reservoir_sampling#Optimal:_Algorithm_L - /// We use sample_size instead of k and num_rows instead of n. - /// We use indices instead of actual values in the reservoir since we'll do one take() on the - /// input arrays at the end. - fn reservoir_sample_indices(num_rows: usize, sample_size: usize, seed: u64) -> Vec { - assert!(sample_size > 0); - assert!( - num_rows > sample_size, - "Sample size > num_rows yields original batch." - ); - - // Initialize our reservoir with indices of the first |sample_size| elements. - let mut reservoir: Vec = (0..sample_size as u64).collect(); - - let mut rng = SmallRng::seed_from_u64(seed); - let mut w = (rng.random::().ln() / sample_size as f64).exp(); - let mut i = sample_size - 1; - - while i < num_rows { - i += (rng.random::().ln() / (1.0 - w).ln()).floor() as usize + 1; - - if i < num_rows { - // Replace a random item in the reservoir with i - let random_index = rng.random_range(0..sample_size); - reservoir[random_index] = i as u64; - w *= (rng.random::().ln() / sample_size as f64).exp(); - } - } - - reservoir - } - - /// Given a batch of Rows, an ordered vector of Rows that represent partition boundaries, and - /// a slice with enough space for the input batch, determines a partition id for every input - /// Row using binary search. - pub fn partition_indices_for_batch( - row_batch: &Rows, - partition_bounds_vec: &[OwnedRow], - partition_ids: &mut [u32], - ) { - row_batch.iter().enumerate().for_each(|(row_idx, row)| { - partition_ids[row_idx] = - partition_bounds_vec.partition_point(|bound| bound.row() <= row) as u32 - }); - } - - /// Given input arrays and range partitioning metadata: samples the input arrays, generates - /// partition bounds, and returns Rows (for comparison against) and a RowConverter (for - /// adapting future incoming batches). - pub fn generate_bounds( - partition_arrays: &Vec, - lex_ordering: &LexOrdering, - num_output_partitions: usize, - num_rows: usize, - sample_size: usize, - seed: u64, - ) -> Result<(Rows, RowConverter), DataFusionError> { - let sampled_columns = if sample_size < num_rows { - // Construct our sample indices. - let sample_indices = UInt64Array::from(RangePartitioner::reservoir_sample_indices( - num_rows, - sample_size, - seed, - )); - - // Extract our sampled data from the input data. - take_arrays( - partition_arrays, - &sample_indices, - Some(TakeOptions { - check_bounds: false, - }), - )? - } else { - // Requested sample_size is larger than the batch, so just use the batch. - partition_arrays.clone() - }; - - // Generate our bounds indices. - let sort_fields: Vec = partition_arrays - .iter() - .zip(lex_ordering) - .map(|(array, sort_expr)| { - SortField::new_with_options(array.data_type().clone(), sort_expr.options) - }) - .collect(); - - let (bounds_indices, row_converter) = RangePartitioner::determine_bounds_for_rows( - sort_fields, - sampled_columns.as_slice(), - num_output_partitions, - )?; - - // Extract our bounds data from the sampled data. - let bounds_indices_array = UInt64Array::from(bounds_indices); - let bounds_arrays = take_arrays( - sampled_columns.as_slice(), - &bounds_indices_array, - Some(TakeOptions { - check_bounds: false, - }), - )?; - - // Convert the bounds data to Rows and return with RowConverter. - Ok(( - row_converter.convert_columns(bounds_arrays.as_slice())?, - row_converter, - )) - } - - /// Given a sort ordering, sampled data, and a number of target partitions, finds the partition - /// bounds and returns them as indices into the sampled data. - /// Adapted from org.apache.spark.RangePartitioner.determineBounds but without weighted - /// values since we don't have cross-partition samples to merge. - /// We normalize the math to use ints instead of floating point by replacing 1.0 with a - /// (imagined) num_candidates * partitions range. - fn determine_bounds_for_rows( - sort_fields: Vec, - sampled_columns: &[ArrayRef], - partitions: usize, - ) -> Result<(Vec, RowConverter), DataFusionError> { - assert!(partitions > 1); - - let converter = RowConverter::new(sort_fields)?; - let sampled_rows = converter.convert_columns(sampled_columns)?; - let mut sorted_sampled_rows: Vec<(usize, Row)> = sampled_rows.iter().enumerate().collect(); - sorted_sampled_rows.sort_unstable_by(|(_, a), (_, b)| a.cmp(b)); - - let num_candidates = sampled_rows.num_rows(); - let step = 1.0 / partitions as f64; - let mut cumulative_weights = 0.0; - let mut target = step; - let mut bounds_indices: Vec = Vec::with_capacity(partitions - 1); - let mut i = 0; - let mut j = 0; - let mut previous_bound = None; - let sample_weight = 1.0 / num_candidates as f64; - while (i < num_candidates) && (j < partitions - 1) { - let key = sorted_sampled_rows[i]; - cumulative_weights += sample_weight; - if cumulative_weights >= target { - // Skip duplicate values. - if previous_bound.is_none() || key.1 > previous_bound.unwrap() { - bounds_indices.push(key.0 as u64); - target += step; - j += 1; - previous_bound = Some(key.1) - } - } - i += 1 - } - - Ok((bounds_indices, converter)) - } -} - -#[cfg(test)] -mod test { - use super::*; - use arrow::array::{Array, AsArray, Int64Array, RecordBatch, UInt64Array}; - use arrow::compute::take_record_batch; - use arrow::datatypes::DataType::{Float64, Int64}; - use arrow::datatypes::{Field, Float64Type, Int32Type, Int64Type, Schema}; - use datafusion::common::record_batch; - use datafusion::physical_expr::expressions::col; - use datafusion::physical_expr::PhysicalSortExpr; - use itertools::Itertools; - use std::sync::Arc; - - fn sample_batch(input: RecordBatch, indices: Vec) -> RecordBatch { - let indices = UInt64Array::from(indices); - take_record_batch(&input, &indices).unwrap() - } - - fn check_sample_indices(indices: &[u64], batch_size: usize, sample_size: usize) { - // sample indices size should never exceed the batch size - assert!(indices.len() <= batch_size); - // number of samples should be the smaller of batch size and sample size - assert_eq!(indices.len(), batch_size.min(sample_size)); - // Check that indices are not out of bounds - indices - .iter() - .for_each(|&idx| assert!(idx < batch_size as u64)); - // Check that values are distinct - let sorted_indices = indices.iter().sorted().collect_vec(); - assert_eq!( - sorted_indices.len(), - sorted_indices.iter().dedup().collect_vec().len() - ); - } - - fn check_bounds_indices(indices: &[u64], sample_size: usize) { - // bounds indices size should never exceed the sample size - assert!(indices.len() <= sample_size); - // Check that indices are not out of bounds - indices - .iter() - .for_each(|&idx| assert!(idx < sample_size as u64)); - // Check that values are distinct - let sorted_indices = indices.iter().sorted().collect_vec(); - assert_eq!( - sorted_indices.len(), - sorted_indices.iter().dedup().collect_vec().len() - ); - } - - #[test] - // Reservoir sampling from a sorted batch can appear like an adversarial case, so we want to - // ensure that we still generate reasonable bounds. A sorted batch with partitioning column of - // [0,8192) is sampled and we generate bounds for 10 partitions. A perfect split would have - // ~819 between each partition, so we just assert that the difference in bounds is <1000. - fn generate_bounds_for_sorted_batch() { - let input_batch = create_random_batch(8192, true, Some((0, 8192))); - - let lex_ordering = LexOrdering::new(vec![PhysicalSortExpr::new_default( - col("a", input_batch.schema().as_ref()).unwrap(), - )]); - - let (rows, row_converter) = RangePartitioner::generate_bounds( - input_batch.columns().to_vec().as_ref(), - &lex_ordering.unwrap(), - 10, - input_batch.num_rows(), - 1000, - 42, - ) - .unwrap(); - - let rows_array = row_converter.convert_rows(&rows).unwrap(); - - let primitive_array = rows_array[0].as_primitive::(); - - for i in 0..8 { - assert!( - (primitive_array.values().get(i + 1).unwrap() - - primitive_array.values().get(i).unwrap()) - .abs() - < 1000 - ); - } - } - - #[test] - // We want to verify that with hand-written bounds for a distribution of data that we get - // reasonable partition indices for a random batch. For this scenario, we create a full - // RecordBatch with one partition column. The values in the column are uniform randomly - // distributed between [0,10). We request 10 partitions with bounds of [1,2,3,4,5,6,7,8,9], - // and the result should be 10 bins with reasonably close counts. - fn partition_indices_for_batch() { - let sort_fields = vec![SortField::new(Int64)]; - let row_converter = RowConverter::new(sort_fields).unwrap(); - let mut partition_ids = vec![0u32; 8192]; - let mut partition_counts = [0u32; 10]; - - let input_batch = create_random_batch(8192, false, Some((0, 10))); - let bounds = record_batch!(("a", Int64, (1..=9).collect_vec())).unwrap(); - - let input_rows = row_converter - .convert_columns(input_batch.columns()) - .unwrap(); - - let bounds_rows = row_converter.convert_columns(bounds.columns()).unwrap(); - - let owned_rows_vec = bounds_rows.iter().map(|row| row.owned()).collect_vec(); - - RangePartitioner::partition_indices_for_batch( - &input_rows, - owned_rows_vec.as_slice(), - &mut partition_ids, - ); - - partition_ids - .iter() - .for_each(|&partition_id| partition_counts[partition_id as usize] += 1); - - // The RecordBatch won't be perfectly distributed, so for 8192 / 10 possible values we - // check that each partition just has >700 values in them. - partition_counts - .iter() - .for_each(|&partition_count| assert!(partition_count > 700)); - } - - #[test] - // We want to verify that reservoir sampling yields valid indices for different size input - // batches. We randomly generate batch sizes and sample sizes, and then construct reservoir - // samples for each scenario. Finally, we validate the indices. - fn reservoir_sample_random() { - let mut rng = SmallRng::seed_from_u64(42); - - // These functions don't rely on unsafe Rust, so we reduce the number iterations for CI - // when testing with Miri. - for _ in 0..if cfg!(miri) { 8 } else { 8192 } { - let batch_size: usize = rng.random_range(1..=8192); - // We don't test sample size > batch_size since in that case you would just take the - // entire batch as the sample. - let sample_size: usize = rng.random_range(1..batch_size); - let indices = RangePartitioner::reservoir_sample_indices(batch_size, sample_size, 42); - - check_sample_indices(&indices, batch_size, sample_size); - } - } - - #[test] - // org.apache.spark.util.random.SamplingUtilsSuite - // "SPARK-18678 reservoirSampleAndCount with tiny input" - fn reservoir_sample_and_count_with_tiny_input() { - let batch = record_batch!(("a", Int32, vec![0, 1])).unwrap(); - let mut counts: Vec = vec![0; 2]; - for i in 0..500 { - let indices = RangePartitioner::reservoir_sample_indices(batch.num_rows(), 1, i); - let result = sample_batch(batch.clone(), indices); - assert_eq!(result.num_rows(), 1); - counts[result.column(0).as_primitive::().value(0) as usize] += 1; - } - // If correct, should be true with prob ~ 0.99999707 according to original Spark test. - assert!((counts[0] - counts[1]).abs() <= 100) - } - - #[test] - // org.apache.spark.PartitioningSuite - // "RangePartitioner.determineBounds" - fn determine_bounds_for_rows() { - // The original test had weights on the values. We just duplicate them because our - // determine_bounds function is unweighted. - let batch = record_batch!(( - "a", - Float64, - vec![ - Some(0.7), - Some(0.7), - Some(0.1), - Some(0.4), - Some(0.3), - Some(0.2), - Some(0.5), - Some(1.0), - Some(1.0), - Some(1.0), - ] - )) - .unwrap(); - - let sort_fields = vec![SortField::new(Float64)]; - - let (rows, _) = - RangePartitioner::determine_bounds_for_rows(sort_fields, batch.columns(), 3).unwrap(); - - check_bounds_indices(rows.as_slice(), batch.num_rows()); - - assert_eq!(rows.len(), 2); - - let indices = UInt64Array::from(rows); - - let bounds = take_record_batch(&batch, &indices).unwrap(); - let bounds_array = bounds.column(0).as_primitive::(); - assert_eq!(bounds_array.values(), &[0.4, 0.7]); - } - - #[test] - // We want to verify that determining bounds yields valid indices for different size sample - // batches. We randomly generate batches and number of partitions, and then construct - // bounds for each scenario. Finally, we validate the indices. - fn determine_bounds_random() { - let mut rng = SmallRng::seed_from_u64(42); - - let sort_fields = vec![SortField::new(Int64)]; - - // These functions don't rely on unsafe Rust, so we reduce the number iterations for CI - // when testing with Miri. - for _ in 0..if cfg!(miri) { 8 } else { 2048 } { - let batch_size = rng.random_range(0..=8192); - // We don't test fewer than 2 partitions since this is used by the - // MultiPartitionShuffleRepartitioner which is for >1 partitions. - let num_partitions = rng.random_range(2..=10000); - - let batch = create_random_batch(batch_size, false, None); - - let (rows, _) = RangePartitioner::determine_bounds_for_rows( - sort_fields.clone(), - batch.columns(), - num_partitions, - ) - .unwrap(); - - check_bounds_indices(rows.as_slice(), batch_size as usize); - - let rows_array = UInt64Array::from(rows); - - let bounds = take_record_batch(&batch, &rows_array).unwrap(); - - let bounds_vec: Vec = bounds - .column(0) - .as_primitive::() - .values() - .to_vec(); - - // Bounds should be sorted. - assert!(bounds_vec.is_sorted()); - // Bounds should be unique. - assert_eq!( - bounds_vec.len(), - bounds_vec.iter().dedup().collect_vec().len() - ); - } - } - - #[test] - // We want to make sure that finding bounds works with nulls. DF has more exhaustive tests for - // sorting with nulls, so we defer to those for more coverage. This is just a small - // deterministic test to verify that nulls can be partition boundaries. - fn determine_bounds_with_nulls() { - let batch = record_batch!(("a", Float64, vec![None, None, Some(0.1),])).unwrap(); - - let sort_fields = vec![SortField::new(Float64)]; - - let (rows, _) = - RangePartitioner::determine_bounds_for_rows(sort_fields, batch.columns(), 2).unwrap(); - - assert_eq!(rows.len(), 1); - - let indices = UInt64Array::from(rows); - - let bounds = take_record_batch(&batch, &indices).unwrap(); - let bounds_array = bounds.column(0).as_primitive::(); - assert!(bounds_array.is_null(0)); - } - - fn create_random_batch(batch_size: u32, sort: bool, range: Option<(i64, i64)>) -> RecordBatch { - let mut rng = SmallRng::seed_from_u64(42); - let mut column: Vec = if let Some((min, max)) = range { - assert!(min <= max); - (0..batch_size) - .map(|_| rng.random_range(min..max)) - .collect() - } else { - (0..batch_size).map(|_| rng.random::()).collect() - }; - if sort { - column.sort(); - } - let array = Int64Array::from(column); - let schema = Arc::new(Schema::new(vec![Field::new("a", Int64, true)])); - RecordBatch::try_new(Arc::clone(&schema), vec![Arc::new(array)]).unwrap() - } -} diff --git a/native/core/src/execution/shuffle/shuffle_writer.rs b/native/core/src/execution/shuffle/shuffle_writer.rs index e96d0574d2..bf2d3017f5 100644 --- a/native/core/src/execution/shuffle/shuffle_writer.rs +++ b/native/core/src/execution/shuffle/shuffle_writer.rs @@ -17,11 +17,9 @@ //! Defines the External shuffle repartition plan. -use crate::execution::shuffle::range_partitioner::RangePartitioner; use crate::execution::shuffle::{CometPartitioning, CompressionCodec, ShuffleBlockWriter}; use crate::execution::tracing::{with_trace, with_trace_async}; use arrow::compute::interleave_record_batch; -use arrow::row::{OwnedRow, RowConverter}; use async_trait::async_trait; use datafusion::common::utils::proxy::VecAllocExt; use datafusion::physical_expr::{EquivalenceProperties, Partitioning}; @@ -333,10 +331,6 @@ struct MultiPartitionShuffleRepartitioner { /// Reservation for repartitioning reservation: MemoryReservation, tracing_enabled: bool, - /// RangePartitioning-specific state - bounds_rows: Option>, - row_converter: Option, - seed: u64, } #[derive(Default)] @@ -413,10 +407,6 @@ impl MultiPartitionShuffleRepartitioner { batch_size, reservation, tracing_enabled, - bounds_rows: None, - row_converter: None, - // Spark RangePartitioner seeds off of partition number. - seed: partition as u64, }) } @@ -546,7 +536,8 @@ impl MultiPartitionShuffleRepartitioner { CometPartitioning::RangePartitioning( lex_ordering, num_output_partitions, - sample_size, + row_converter, + bounds, ) => { let mut scratch = std::mem::take(&mut self.scratch); let (partition_starts, partition_row_indices): (&Vec, &Vec) = { @@ -560,37 +551,20 @@ impl MultiPartitionShuffleRepartitioner { let num_rows = arrays[0].len(); - // If necessary (i.e., when first batch arrives) generate the bounds (as Rows) - // for range partitioning based on randomly reservoir sampling the batch. - if self.row_converter.is_none() { - let (bounds_rows, row_converter) = RangePartitioner::generate_bounds( - &arrays, - lex_ordering, - *num_output_partitions, - input.num_rows(), - *sample_size, - self.seed, - )?; - - self.bounds_rows = - Some(bounds_rows.iter().map(|row| row.owned()).collect_vec()); - self.row_converter = Some(row_converter); - } - // Generate partition ids for every row, first by converting the partition // arrays to Rows, and then doing binary search for each Row against the // bounds Rows. - let row_batch = self - .row_converter - .as_ref() - .unwrap() - .convert_columns(arrays.as_slice())?; - - RangePartitioner::partition_indices_for_batch( - &row_batch, - self.bounds_rows.as_ref().unwrap().as_slice(), - &mut scratch.partition_ids[..num_rows], - ); + { + let row_batch = row_converter.convert_columns(arrays.as_slice())?; + let partition_ids = &mut scratch.partition_ids[..num_rows]; + + row_batch.iter().enumerate().for_each(|(row_idx, row)| { + partition_ids[row_idx] = bounds + .as_slice() + .partition_point(|bound| bound.row() <= row) + as u32 + }); + } // We now have partition ids for every input row, map that to partition starts // and partition indices to eventually right these rows to partition buffers. @@ -1278,6 +1252,7 @@ mod test { use super::*; use crate::execution::shuffle::read_ipc_compressed; use arrow::datatypes::{DataType, Field, Schema}; + use arrow::row::{RowConverter, SortField}; use datafusion::datasource::memory::MemorySourceConfig; use datafusion::datasource::source::DataSourceExec; use datafusion::execution::config::SessionConfig; @@ -1404,15 +1379,51 @@ mod test { ) { let batch = create_batch(batch_size); + let lex_ordering = LexOrdering::new(vec![PhysicalSortExpr::new_default( + col("a", batch.schema().as_ref()).unwrap(), + )]) + .unwrap(); + + let sort_fields: Vec = batch + .columns() + .iter() + .zip(&lex_ordering) + .map(|(array, sort_expr)| { + SortField::new_with_options(array.data_type().clone(), sort_expr.options) + }) + .collect(); + let row_converter = RowConverter::new(sort_fields).unwrap(); + + let owned_rows = if num_partitions == 1 { + vec![] + } else { + // Determine range boundaries based on create_batch implementation. We just divide the + // domain of values in the batch equally to find partition bounds. + let bounds_strings = { + let mut boundaries = Vec::with_capacity(num_partitions - 1); + let step = batch_size as f64 / num_partitions as f64; + + for i in 1..(num_partitions) { + boundaries.push(Some((step * i as f64).round().to_string())); + } + boundaries + }; + let bounds_array: Arc = Arc::new(StringArray::from(bounds_strings)); + let bounds_rows = row_converter + .convert_columns(vec![bounds_array].as_slice()) + .unwrap(); + + let owned_rows_vec = bounds_rows.iter().map(|row| row.owned()).collect_vec(); + owned_rows_vec + }; + for partitioning in [ CometPartitioning::Hash(vec![Arc::new(Column::new("a", 0))], num_partitions), CometPartitioning::RangePartitioning( - LexOrdering::new(vec![PhysicalSortExpr::new_default( - col("a", batch.schema().as_ref()).unwrap(), - )]) - .unwrap(), + lex_ordering, num_partitions, - 100, + Arc::new(row_converter), + owned_rows, ), ] { let batches = (0..num_batches).map(|_| batch.clone()).collect::>(); diff --git a/native/proto/src/proto/partitioning.proto b/native/proto/src/proto/partitioning.proto index ea0e586fb1..e11d7a384b 100644 --- a/native/proto/src/proto/partitioning.proto +++ b/native/proto/src/proto/partitioning.proto @@ -42,8 +42,12 @@ message HashPartition { message SinglePartition { } +message BoundaryRow { + repeated spark.spark_expression.Expr partition_bounds = 1; +} + message RangePartition { repeated spark.spark_expression.Expr sort_orders = 1; int32 num_partitions = 2; - int32 sample_size = 3; + repeated BoundaryRow boundary_rows = 4; } diff --git a/spark/src/main/scala/org/apache/comet/rules/CometExecRule.scala b/spark/src/main/scala/org/apache/comet/rules/CometExecRule.scala index 091f70fdc2..0f5fe6269f 100644 --- a/spark/src/main/scala/org/apache/comet/rules/CometExecRule.scala +++ b/spark/src/main/scala/org/apache/comet/rules/CometExecRule.scala @@ -774,7 +774,16 @@ case class CometExecRule(session: SparkSession) extends Rule[SparkPlan] { * operations like `groupByKey`, `reduceByKey` or `join`. Native code does not support hashing * complex types, see hash_funcs/utils.rs */ - def supportedPartitioningDataType(dt: DataType): Boolean = dt match { + def supportedHashPartitioningDataType(dt: DataType): Boolean = dt match { + case _: BooleanType | _: ByteType | _: ShortType | _: IntegerType | _: LongType | + _: FloatType | _: DoubleType | _: StringType | _: BinaryType | _: TimestampType | + _: TimestampNTZType | _: DecimalType | _: DateType => + true + case _ => + false + } + + def supportedRangePartitioningDataType(dt: DataType): Boolean = dt match { case _: BooleanType | _: ByteType | _: ShortType | _: IntegerType | _: LongType | _: FloatType | _: DoubleType | _: StringType | _: BinaryType | _: TimestampType | _: TimestampNTZType | _: DecimalType | _: DateType => @@ -847,7 +856,7 @@ case class CometExecRule(session: SparkSession) extends Rule[SparkPlan] { } } for (dt <- expressions.map(_.dataType).distinct) { - if (!supportedPartitioningDataType(dt)) { + if (!supportedHashPartitioningDataType(dt)) { withInfo(s, s"unsupported hash partitioning data type for native shuffle: $dt") supported = false } @@ -873,7 +882,7 @@ case class CometExecRule(session: SparkSession) extends Rule[SparkPlan] { } } for (dt <- orderings.map(_.dataType).distinct) { - if (!supportedPartitioningDataType(dt)) { + if (!supportedRangePartitioningDataType(dt)) { withInfo(s, s"unsupported range partitioning data type for native shuffle: $dt") supported = false } diff --git a/spark/src/main/scala/org/apache/spark/sql/comet/execution/shuffle/CometNativeShuffleWriter.scala b/spark/src/main/scala/org/apache/spark/sql/comet/execution/shuffle/CometNativeShuffleWriter.scala index 018c9f7c10..3dfd1f8d03 100644 --- a/spark/src/main/scala/org/apache/spark/sql/comet/execution/shuffle/CometNativeShuffleWriter.scala +++ b/spark/src/main/scala/org/apache/spark/sql/comet/execution/shuffle/CometNativeShuffleWriter.scala @@ -22,17 +22,18 @@ package org.apache.spark.sql.comet.execution.shuffle import java.nio.{ByteBuffer, ByteOrder} import java.nio.file.{Files, Paths} +import scala.collection.mutable import scala.jdk.CollectionConverters._ import org.apache.spark.{SparkEnv, TaskContext} import org.apache.spark.internal.Logging import org.apache.spark.scheduler.MapStatus import org.apache.spark.shuffle.{IndexShuffleBlockResolver, ShuffleWriteMetricsReporter, ShuffleWriter} -import org.apache.spark.sql.catalyst.expressions.Attribute +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.expressions.{Attribute, Expression, Literal} import org.apache.spark.sql.catalyst.plans.physical.{HashPartitioning, Partitioning, RangePartitioning, SinglePartition} import org.apache.spark.sql.comet.{CometExec, CometMetricNode} import org.apache.spark.sql.execution.metric.SQLMetric -import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.vectorized.ColumnarBatch import org.apache.comet.CometConf @@ -51,7 +52,8 @@ class CometNativeShuffleWriter[K, V]( shuffleId: Int, mapId: Long, context: TaskContext, - metricsReporter: ShuffleWriteMetricsReporter) + metricsReporter: ShuffleWriteMetricsReporter, + rangePartitionBounds: Option[Seq[InternalRow]] = None) extends ShuffleWriter[K, V] with Logging { @@ -140,6 +142,17 @@ class CometNativeShuffleWriter[K, V]( MapStatus.apply(SparkEnv.get.blockManager.shuffleServerId, partitionLengths, mapId) } + private def isSinglePartitioning(p: Partitioning): Boolean = p match { + case SinglePartition => true + case rp: RangePartitioning => + // Spark sometimes generates RangePartitioning schemes with numPartitions == 1, + // or the computed bounds results in a single target partition. + // In this case Comet just serializes a SinglePartition scheme to native. + rp.numPartitions == 1 || rangePartitionBounds.forall(_.isEmpty) + case hp: HashPartitioning => hp.numPartitions == 1 + case _ => false + } + private def getNativePlan(dataFile: String, indexFile: String): Operator = { val scanBuilder = OperatorOuterClass.Scan.newBuilder().setSource("ShuffleWriterInput") val opBuilder = OperatorOuterClass.Operator.newBuilder() @@ -170,6 +183,12 @@ class CometNativeShuffleWriter[K, V]( CometConf.COMET_EXEC_SHUFFLE_COMPRESSION_ZSTD_LEVEL.get) outputPartitioning match { + case p if isSinglePartitioning(p) => + val partitioning = PartitioningOuterClass.SinglePartition.newBuilder() + + val partitioningBuilder = PartitioningOuterClass.Partitioning.newBuilder() + shuffleWriterBuilder.setPartitioning( + partitioningBuilder.setSinglePartition(partitioning).build()) case _: HashPartitioning => val hashPartitioning = outputPartitioning.asInstanceOf[HashPartitioning] @@ -194,45 +213,73 @@ class CometNativeShuffleWriter[K, V]( val partitioning = PartitioningOuterClass.RangePartition.newBuilder() partitioning.setNumPartitions(outputPartitioning.numPartitions) - val sampleSize = { - // taken from org.apache.spark.RangePartitioner#rangeBounds - // This is the sample size we need to have roughly balanced output partitions, - // capped at 1M. - // Cast to double to avoid overflowing ints or longs - val sampleSize = math.min( - SQLConf.get - .getConf(SQLConf.RANGE_EXCHANGE_SAMPLE_SIZE_PER_PARTITION) - .toDouble * outputPartitioning.numPartitions, - 1e6) - // Assume the input partitions are roughly balanced and over-sample a little bit. - // Comet: we don't divide by numPartitions since each DF plan handles one partition. - math.ceil(3.0 * sampleSize).toInt - } - if (sampleSize > 8192) { - logWarning( - s"RangePartitioning sampleSize of s$sampleSize exceeds Comet RecordBatch size.") - } - partitioning.setSampleSize(sampleSize) - val orderingExprs = rangePartitioning.ordering - .flatMap(e => QueryPlanSerde.exprToProto(e, outputAttributes)) + // Detect duplicates by tracking expressions directly, similar to DataFusion's LexOrdering + // DataFusion will deduplicate identical sort expressions in LexOrdering, + // so we need to transform boundary rows to match the deduplicated structure + val seenExprs = mutable.HashSet[Expression]() + val deduplicationMap = mutable.ArrayBuffer[(Int, Boolean)]() // (originalIndex, isKept) + + rangePartitioning.ordering.zipWithIndex.foreach { case (sortOrder, idx) => + if (seenExprs.contains(sortOrder.child)) { + deduplicationMap += (idx -> false) // Will be deduplicated by DataFusion + } else { + seenExprs += sortOrder.child + deduplicationMap += (idx -> true) // Will be kept by DataFusion + } + } - if (orderingExprs.length != rangePartitioning.ordering.length) { - throw new UnsupportedOperationException( - s"Partitioning $rangePartitioning is not supported.") + { + // Serialize the ordering expressions for comparisons + val orderingExprs = rangePartitioning.ordering + .flatMap(e => QueryPlanSerde.exprToProto(e, outputAttributes)) + if (orderingExprs.length != rangePartitioning.ordering.length) { + throw new UnsupportedOperationException( + s"Partitioning $rangePartitioning is not supported.") + } + partitioning.addAllSortOrders(orderingExprs.asJava) } - partitioning.addAllSortOrders(orderingExprs.asJava) + // Convert Spark's sequence of InternalRows that represent partitioning boundaries to + // sequences of Literals, where each outer entry represents a boundary row, and each + // internal entry is a value in that row. In other words, these are stored in row major + // order, not column major + val boundarySchema = rangePartitioning.ordering.flatMap(e => Some(e.dataType)) + + // Transform boundary rows to match DataFusion's deduplicated structure + val transformedBoundaryExprs: Seq[Seq[Literal]] = + rangePartitionBounds.get.map((row: InternalRow) => { + // For every InternalRow, map its values to Literals + val allLiterals = + row.toSeq(boundarySchema).zip(boundarySchema).map { case (value, valueType) => + Literal(value, valueType) + } + + // Keep only the literals that correspond to non-deduplicated expressions + allLiterals + .zip(deduplicationMap) + .filter(_._2._2) // Keep only where isKept = true + .map(_._1) // Extract the literal + }) + + { + // Convert the sequences of Literals to a collection of serialized BoundaryRows + val boundaryRows: Seq[PartitioningOuterClass.BoundaryRow] = transformedBoundaryExprs + .map((rowLiterals: Seq[Literal]) => { + // Serialize each sequence of Literals as a BoundaryRow + val rowBuilder = PartitioningOuterClass.BoundaryRow.newBuilder(); + val serializedExprs = + rowLiterals.map(lit_value => + QueryPlanSerde.exprToProto(lit_value, outputAttributes).get) + rowBuilder.addAllPartitionBounds(serializedExprs.asJava) + rowBuilder.build() + }) + partitioning.addAllBoundaryRows(boundaryRows.asJava) + } val partitioningBuilder = PartitioningOuterClass.Partitioning.newBuilder() shuffleWriterBuilder.setPartitioning( partitioningBuilder.setRangePartition(partitioning).build()) - case SinglePartition => - val partitioning = PartitioningOuterClass.SinglePartition.newBuilder() - - val partitioningBuilder = PartitioningOuterClass.Partitioning.newBuilder() - shuffleWriterBuilder.setPartitioning( - partitioningBuilder.setSinglePartition(partitioning).build()) case _ => throw new UnsupportedOperationException( diff --git a/spark/src/main/scala/org/apache/spark/sql/comet/execution/shuffle/CometShuffleDependency.scala b/spark/src/main/scala/org/apache/spark/sql/comet/execution/shuffle/CometShuffleDependency.scala index ff35b10eb6..2b74e5a168 100644 --- a/spark/src/main/scala/org/apache/spark/sql/comet/execution/shuffle/CometShuffleDependency.scala +++ b/spark/src/main/scala/org/apache/spark/sql/comet/execution/shuffle/CometShuffleDependency.scala @@ -25,6 +25,7 @@ import org.apache.spark.{Aggregator, Partitioner, ShuffleDependency, SparkEnv} import org.apache.spark.rdd.RDD import org.apache.spark.serializer.Serializer import org.apache.spark.shuffle.ShuffleWriteProcessor +import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.Attribute import org.apache.spark.sql.catalyst.plans.physical.Partitioning import org.apache.spark.sql.execution.metric.SQLMetric @@ -47,7 +48,8 @@ class CometShuffleDependency[K: ClassTag, V: ClassTag, C: ClassTag]( val outputPartitioning: Option[Partitioning] = None, val outputAttributes: Seq[Attribute] = Seq.empty, val shuffleWriteMetrics: Map[String, SQLMetric] = Map.empty, - val numParts: Int = 0) + val numParts: Int = 0, + val rangePartitionBounds: Option[Seq[InternalRow]] = None) extends ShuffleDependency[K, V, C]( _rdd, partitioner, diff --git a/spark/src/main/scala/org/apache/spark/sql/comet/execution/shuffle/CometShuffleExchangeExec.scala b/spark/src/main/scala/org/apache/spark/sql/comet/execution/shuffle/CometShuffleExchangeExec.scala index df67e7b7f2..1f7d37a108 100644 --- a/spark/src/main/scala/org/apache/spark/sql/comet/execution/shuffle/CometShuffleExchangeExec.scala +++ b/spark/src/main/scala/org/apache/spark/sql/comet/execution/shuffle/CometShuffleExchangeExec.scala @@ -22,6 +22,7 @@ package org.apache.spark.sql.comet.execution.shuffle import java.util.function.Supplier import scala.concurrent.Future +import scala.jdk.CollectionConverters._ import org.apache.spark._ import org.apache.spark.internal.config @@ -35,8 +36,7 @@ import org.apache.spark.sql.catalyst.plans.logical.Statistics import org.apache.spark.sql.catalyst.plans.physical._ import org.apache.spark.sql.comet.{CometMetricNode, CometPlan} import org.apache.spark.sql.execution._ -import org.apache.spark.sql.execution.exchange.{ENSURE_REQUIREMENTS, ShuffleExchangeLike, ShuffleOrigin} -import org.apache.spark.sql.execution.exchange.ShuffleExchangeExec +import org.apache.spark.sql.execution.exchange.{ENSURE_REQUIREMENTS, ShuffleExchangeExec, ShuffleExchangeLike, ShuffleOrigin} import org.apache.spark.sql.execution.metric.{SQLMetric, SQLMetrics, SQLShuffleReadMetricsReporter, SQLShuffleWriteMetricsReporter} import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.vectorized.ColumnarBatch @@ -211,6 +211,7 @@ case class CometShuffleExchangeExec( } object CometShuffleExchangeExec extends ShimCometShuffleExchangeExec { + def prepareShuffleDependency( rdd: RDD[ColumnarBatch], outputAttributes: Seq[Attribute], @@ -218,6 +219,58 @@ object CometShuffleExchangeExec extends ShimCometShuffleExchangeExec { serializer: Serializer, metrics: Map[String, SQLMetric]): ShuffleDependency[Int, ColumnarBatch, ColumnarBatch] = { val numParts = rdd.getNumPartitions + + // The code block below is mostly brought over from + // ShuffleExchangeExec::prepareShuffleDependency + val (partitioner, rangePartitionBounds) = outputPartitioning match { + case rangePartitioning: RangePartitioning => + // Extract only fields used for sorting to avoid collecting large fields that does not + // affect sorting result when deciding partition bounds in RangePartitioner + val rddForSampling = rdd.mapPartitionsInternal { iter => + val projection = + UnsafeProjection.create(rangePartitioning.ordering.map(_.child), outputAttributes) + val mutablePair = new MutablePair[InternalRow, Null]() + + // Internally, RangePartitioner runs a job on the RDD that samples keys to compute + // partition bounds. To get accurate samples, we need to copy the mutable keys. + iter.flatMap { batch => + val rowIter = batch.rowIterator().asScala + rowIter.map { row => + mutablePair.update(projection(row).copy(), null) + } + } + } + + // Construct ordering on extracted sort key. + val orderingAttributes = rangePartitioning.ordering.zipWithIndex.map { case (ord, i) => + ord.copy(child = BoundReference(i, ord.dataType, ord.nullable)) + } + implicit val ordering = new LazilyGeneratedOrdering(orderingAttributes) + // Use Spark's RangePartitioner to compute bounds from global samples + val rangePartitioner = new RangePartitioner( + rangePartitioning.numPartitions, + rddForSampling, + ascending = true, + samplePointsPerPartitionHint = SQLConf.get.rangeExchangeSampleSizePerPartition) + + // Use reflection to access the private rangeBounds field + val rangeBoundsField = rangePartitioner.getClass.getDeclaredField("rangeBounds") + rangeBoundsField.setAccessible(true) + val rangeBounds = + rangeBoundsField.get(rangePartitioner).asInstanceOf[Array[InternalRow]].toSeq + + (rangePartitioner.asInstanceOf[Partitioner], Some(rangeBounds)) + + case _ => + ( + new Partitioner { + override def numPartitions: Int = outputPartitioning.numPartitions + + override def getPartition(key: Any): Int = key.asInstanceOf[Int] + }, + None) + } + val dependency = new CometShuffleDependency[Int, ColumnarBatch, ColumnarBatch]( rdd.map( (0, _) @@ -225,15 +278,13 @@ object CometShuffleExchangeExec extends ShimCometShuffleExchangeExec { serializer = serializer, shuffleWriterProcessor = ShuffleExchangeExec.createShuffleWriteProcessor(metrics), shuffleType = CometNativeShuffle, - partitioner = new Partitioner { - override def numPartitions: Int = outputPartitioning.numPartitions - override def getPartition(key: Any): Int = key.asInstanceOf[Int] - }, + partitioner = partitioner, decodeTime = metrics("decode_time"), outputPartitioning = Some(outputPartitioning), outputAttributes = outputAttributes, shuffleWriteMetrics = metrics, - numParts = numParts) + numParts = numParts, + rangePartitionBounds = rangePartitionBounds) dependency } @@ -323,6 +374,7 @@ object CometShuffleExchangeExec extends ShimCometShuffleExchangeExec { case _ => throw new IllegalStateException(s"Exchange not implemented for $newPartitioning") // TODO: Handle BroadcastPartitioning. } + def getPartitionKeyExtractor(): InternalRow => Any = newPartitioning match { case RoundRobinPartitioning(numPartitions) => // Distributes elements evenly across output partitions, starting from a random partition. @@ -375,6 +427,7 @@ object CometShuffleExchangeExec extends ShimCometShuffleExchangeExec { // limited range. val prefixComputer = new UnsafeExternalRowSorter.PrefixComputer { private val result = new UnsafeExternalRowSorter.PrefixComputer.Prefix + override def computePrefix( row: InternalRow): UnsafeExternalRowSorter.PrefixComputer.Prefix = { // The hashcode generated from the binary form of a [[UnsafeRow]] should not be null. diff --git a/spark/src/main/scala/org/apache/spark/sql/comet/execution/shuffle/CometShuffleManager.scala b/spark/src/main/scala/org/apache/spark/sql/comet/execution/shuffle/CometShuffleManager.scala index d733c00fe4..a2af185e53 100644 --- a/spark/src/main/scala/org/apache/spark/sql/comet/execution/shuffle/CometShuffleManager.scala +++ b/spark/src/main/scala/org/apache/spark/sql/comet/execution/shuffle/CometShuffleManager.scala @@ -209,7 +209,8 @@ class CometShuffleManager(conf: SparkConf) extends ShuffleManager with Logging { dep.shuffleId, mapId, context, - metrics) + metrics, + dep.rangePartitionBounds) case bypassMergeSortHandle: CometBypassMergeSortShuffleHandle[K @unchecked, V @unchecked] => new CometBypassMergeSortShuffleWriter( env.blockManager, diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q2/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q2/explain.txt index 213f1c61b4..58e7d31f47 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q2/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q2/explain.txt @@ -1,7 +1,7 @@ == Physical Plan == * CometColumnarToRow (34) +- CometSort (33) - +- CometColumnarExchange (32) + +- CometExchange (32) +- CometProject (31) +- CometBroadcastHashJoin (30) :- CometProject (20) @@ -180,9 +180,9 @@ Arguments: [d_week_seq1#29], [(d_week_seq2#54 - 53)], Inner, BuildRight Input [16]: [d_week_seq1#29, sun_sales1#30, mon_sales1#31, tue_sales1#32, wed_sales1#33, thu_sales1#34, fri_sales1#35, sat_sales1#36, d_week_seq2#54, sun_sales2#55, mon_sales2#56, tue_sales2#57, wed_sales2#58, thu_sales2#59, fri_sales2#60, sat_sales2#61] Arguments: [d_week_seq1#29, round((sun_sales1 / sun_sales2), 2)#62, round((mon_sales1 / mon_sales2), 2)#63, round((tue_sales1 / tue_sales2), 2)#64, round((wed_sales1 / wed_sales2), 2)#65, round((thu_sales1 / thu_sales2), 2)#66, round((fri_sales1 / fri_sales2), 2)#67, round((sat_sales1 / sat_sales2), 2)#68], [d_week_seq1#29, round((sun_sales1#30 / sun_sales2#55), 2) AS round((sun_sales1 / sun_sales2), 2)#62, round((mon_sales1#31 / mon_sales2#56), 2) AS round((mon_sales1 / mon_sales2), 2)#63, round((tue_sales1#32 / tue_sales2#57), 2) AS round((tue_sales1 / tue_sales2), 2)#64, round((wed_sales1#33 / wed_sales2#58), 2) AS round((wed_sales1 / wed_sales2), 2)#65, round((thu_sales1#34 / thu_sales2#59), 2) AS round((thu_sales1 / thu_sales2), 2)#66, round((fri_sales1#35 / fri_sales2#60), 2) AS round((fri_sales1 / fri_sales2), 2)#67, round((sat_sales1#36 / sat_sales2#61), 2) AS round((sat_sales1 / sat_sales2), 2)#68] -(32) CometColumnarExchange +(32) CometExchange Input [8]: [d_week_seq1#29, round((sun_sales1 / sun_sales2), 2)#62, round((mon_sales1 / mon_sales2), 2)#63, round((tue_sales1 / tue_sales2), 2)#64, round((wed_sales1 / wed_sales2), 2)#65, round((thu_sales1 / thu_sales2), 2)#66, round((fri_sales1 / fri_sales2), 2)#67, round((sat_sales1 / sat_sales2), 2)#68] -Arguments: rangepartitioning(d_week_seq1#29 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] +Arguments: rangepartitioning(d_week_seq1#29 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] (33) CometSort Input [8]: [d_week_seq1#29, round((sun_sales1 / sun_sales2), 2)#62, round((mon_sales1 / mon_sales2), 2)#63, round((tue_sales1 / tue_sales2), 2)#64, round((wed_sales1 / wed_sales2), 2)#65, round((thu_sales1 / thu_sales2), 2)#66, round((fri_sales1 / fri_sales2), 2)#67, round((sat_sales1 / sat_sales2), 2)#68] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q2/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q2/simplified.txt index e9a2499f38..852c5fca0f 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q2/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q2/simplified.txt @@ -2,7 +2,7 @@ WholeStageCodegen (1) CometColumnarToRow InputAdapter CometSort [d_week_seq1,round((sun_sales1 / sun_sales2), 2),round((mon_sales1 / mon_sales2), 2),round((tue_sales1 / tue_sales2), 2),round((wed_sales1 / wed_sales2), 2),round((thu_sales1 / thu_sales2), 2),round((fri_sales1 / fri_sales2), 2),round((sat_sales1 / sat_sales2), 2)] - CometColumnarExchange [d_week_seq1] #1 + CometExchange [d_week_seq1] #1 CometProject [sun_sales1,sun_sales2,mon_sales1,mon_sales2,tue_sales1,tue_sales2,wed_sales1,wed_sales2,thu_sales1,thu_sales2,fri_sales1,fri_sales2,sat_sales1,sat_sales2] [d_week_seq1,round((sun_sales1 / sun_sales2), 2),round((mon_sales1 / mon_sales2), 2),round((tue_sales1 / tue_sales2), 2),round((wed_sales1 / wed_sales2), 2),round((thu_sales1 / thu_sales2), 2),round((fri_sales1 / fri_sales2), 2),round((sat_sales1 / sat_sales2), 2)] CometBroadcastHashJoin [d_week_seq1,sun_sales1,mon_sales1,tue_sales1,wed_sales1,thu_sales1,fri_sales1,sat_sales1,d_week_seq2,sun_sales2,mon_sales2,tue_sales2,wed_sales2,thu_sales2,fri_sales2,sat_sales2] CometProject [d_week_seq,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales] [d_week_seq1,sun_sales1,mon_sales1,tue_sales1,wed_sales1,thu_sales1,fri_sales1,sat_sales1] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q31/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q31/explain.txt index 4f065b40bd..83d717e295 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q31/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q31/explain.txt @@ -1,7 +1,7 @@ == Physical Plan == * CometColumnarToRow (90) +- CometSort (89) - +- CometColumnarExchange (88) + +- CometExchange (88) +- CometProject (87) +- CometBroadcastHashJoin (86) :- CometProject (73) @@ -496,9 +496,9 @@ Arguments: [ca_county#42], [ca_county#64], Inner, (CASE WHEN (web_sales#55 > 0.0 Input [10]: [ca_county#9, d_year#6, store_sales#22, store_sales#21, store_sales#33, ca_county#42, web_sales#44, web_sales#55, ca_county#64, web_sales#66] Arguments: [ca_county#9, d_year#6, web_q1_q2_increase#67, store_q1_q2_increase#68, web_q2_q3_increase#69, store_q2_q3_increase#70], [ca_county#9, d_year#6, (web_sales#55 / web_sales#44) AS web_q1_q2_increase#67, (store_sales#21 / store_sales#22) AS store_q1_q2_increase#68, (web_sales#66 / web_sales#55) AS web_q2_q3_increase#69, (store_sales#33 / store_sales#21) AS store_q2_q3_increase#70] -(88) CometColumnarExchange +(88) CometExchange Input [6]: [ca_county#9, d_year#6, web_q1_q2_increase#67, store_q1_q2_increase#68, web_q2_q3_increase#69, store_q2_q3_increase#70] -Arguments: rangepartitioning(ca_county#9 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] +Arguments: rangepartitioning(ca_county#9 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=7] (89) CometSort Input [6]: [ca_county#9, d_year#6, web_q1_q2_increase#67, store_q1_q2_increase#68, web_q2_q3_increase#69, store_q2_q3_increase#70] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q31/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q31/simplified.txt index edc4dc3923..2f97384320 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q31/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q31/simplified.txt @@ -2,7 +2,7 @@ WholeStageCodegen (1) CometColumnarToRow InputAdapter CometSort [ca_county,d_year,web_q1_q2_increase,store_q1_q2_increase,web_q2_q3_increase,store_q2_q3_increase] - CometColumnarExchange [ca_county] #1 + CometExchange [ca_county] #1 CometProject [web_sales,web_sales,store_sales,store_sales,web_sales,store_sales] [ca_county,d_year,web_q1_q2_increase,store_q1_q2_increase,web_q2_q3_increase,store_q2_q3_increase] CometBroadcastHashJoin [ca_county,d_year,store_sales,store_sales,store_sales,ca_county,web_sales,web_sales,ca_county,web_sales] CometProject [ca_county,d_year,store_sales,store_sales,store_sales,ca_county,web_sales,web_sales] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q34/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q34/explain.txt index ceab9d3f0a..287e323952 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q34/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q34/explain.txt @@ -1,7 +1,7 @@ == Physical Plan == * CometColumnarToRow (33) +- CometSort (32) - +- CometColumnarExchange (31) + +- CometExchange (31) +- CometProject (30) +- CometBroadcastHashJoin (29) :- CometFilter (24) @@ -176,9 +176,9 @@ Arguments: [ss_customer_sk#1], [c_customer_sk#18], Inner, BuildRight Input [8]: [ss_ticket_number#4, ss_customer_sk#1, cnt#17, c_customer_sk#18, c_salutation#23, c_first_name#24, c_last_name#25, c_preferred_cust_flag#26] Arguments: [c_last_name#25, c_first_name#24, c_salutation#23, c_preferred_cust_flag#26, ss_ticket_number#4, cnt#17], [c_last_name#25, c_first_name#24, c_salutation#23, c_preferred_cust_flag#26, ss_ticket_number#4, cnt#17] -(31) CometColumnarExchange +(31) CometExchange Input [6]: [c_last_name#25, c_first_name#24, c_salutation#23, c_preferred_cust_flag#26, ss_ticket_number#4, cnt#17] -Arguments: rangepartitioning(c_last_name#25 ASC NULLS FIRST, c_first_name#24 ASC NULLS FIRST, c_salutation#23 ASC NULLS FIRST, c_preferred_cust_flag#26 DESC NULLS LAST, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] +Arguments: rangepartitioning(c_last_name#25 ASC NULLS FIRST, c_first_name#24 ASC NULLS FIRST, c_salutation#23 ASC NULLS FIRST, c_preferred_cust_flag#26 DESC NULLS LAST, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] (32) CometSort Input [6]: [c_last_name#25, c_first_name#24, c_salutation#23, c_preferred_cust_flag#26, ss_ticket_number#4, cnt#17] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q34/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q34/simplified.txt index 9e9ee7fa05..47d0e35dd2 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q34/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q34/simplified.txt @@ -2,7 +2,7 @@ WholeStageCodegen (1) CometColumnarToRow InputAdapter CometSort [c_last_name,c_first_name,c_salutation,c_preferred_cust_flag,ss_ticket_number,cnt] - CometColumnarExchange [c_last_name,c_first_name,c_salutation,c_preferred_cust_flag] #1 + CometExchange [c_last_name,c_first_name,c_salutation,c_preferred_cust_flag] #1 CometProject [c_last_name,c_first_name,c_salutation,c_preferred_cust_flag,ss_ticket_number,cnt] CometBroadcastHashJoin [ss_ticket_number,ss_customer_sk,cnt,c_customer_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag] CometFilter [ss_ticket_number,ss_customer_sk,cnt] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q39a/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q39a/explain.txt index 8e6d9265d6..ed74780c2c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q39a/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q39a/explain.txt @@ -1,7 +1,7 @@ == Physical Plan == * CometColumnarToRow (47) +- CometSort (46) - +- CometColumnarExchange (45) + +- CometExchange (45) +- CometBroadcastHashJoin (44) :- CometProject (23) : +- CometFilter (22) @@ -253,9 +253,9 @@ Left output [5]: [w_warehouse_sk#7, i_item_sk#6, d_moy#11, mean#18, cov#19] Right output [5]: [w_warehouse_sk#26, i_item_sk#25, d_moy#30, mean#36, cov#37] Arguments: [i_item_sk#6, w_warehouse_sk#7], [i_item_sk#25, w_warehouse_sk#26], Inner, BuildRight -(45) CometColumnarExchange +(45) CometExchange Input [10]: [w_warehouse_sk#7, i_item_sk#6, d_moy#11, mean#18, cov#19, w_warehouse_sk#26, i_item_sk#25, d_moy#30, mean#36, cov#37] -Arguments: rangepartitioning(w_warehouse_sk#7 ASC NULLS FIRST, i_item_sk#6 ASC NULLS FIRST, d_moy#11 ASC NULLS FIRST, mean#18 ASC NULLS FIRST, cov#19 ASC NULLS FIRST, d_moy#30 ASC NULLS FIRST, mean#36 ASC NULLS FIRST, cov#37 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] +Arguments: rangepartitioning(w_warehouse_sk#7 ASC NULLS FIRST, i_item_sk#6 ASC NULLS FIRST, d_moy#11 ASC NULLS FIRST, mean#18 ASC NULLS FIRST, cov#19 ASC NULLS FIRST, d_moy#30 ASC NULLS FIRST, mean#36 ASC NULLS FIRST, cov#37 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] (46) CometSort Input [10]: [w_warehouse_sk#7, i_item_sk#6, d_moy#11, mean#18, cov#19, w_warehouse_sk#26, i_item_sk#25, d_moy#30, mean#36, cov#37] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q39a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q39a/simplified.txt index 1e0af57591..2cd1f70c8a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q39a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q39a/simplified.txt @@ -2,7 +2,7 @@ WholeStageCodegen (1) CometColumnarToRow InputAdapter CometSort [w_warehouse_sk,i_item_sk,d_moy,mean,cov,w_warehouse_sk,i_item_sk,d_moy,mean,cov] - CometColumnarExchange [w_warehouse_sk,i_item_sk,d_moy,mean,cov,d_moy,mean,cov] #1 + CometExchange [w_warehouse_sk,i_item_sk,d_moy,mean,cov,d_moy,mean,cov] #1 CometBroadcastHashJoin [w_warehouse_sk,i_item_sk,d_moy,mean,cov,w_warehouse_sk,i_item_sk,d_moy,mean,cov] CometProject [stdev] [w_warehouse_sk,i_item_sk,d_moy,mean,cov] CometFilter [w_warehouse_sk,i_item_sk,d_moy,stdev,mean] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q39b/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q39b/explain.txt index 69b6185645..2a31b0d8b2 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q39b/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q39b/explain.txt @@ -1,7 +1,7 @@ == Physical Plan == * CometColumnarToRow (47) +- CometSort (46) - +- CometColumnarExchange (45) + +- CometExchange (45) +- CometBroadcastHashJoin (44) :- CometProject (23) : +- CometFilter (22) @@ -253,9 +253,9 @@ Left output [5]: [w_warehouse_sk#7, i_item_sk#6, d_moy#11, mean#18, cov#19] Right output [5]: [w_warehouse_sk#26, i_item_sk#25, d_moy#30, mean#36, cov#37] Arguments: [i_item_sk#6, w_warehouse_sk#7], [i_item_sk#25, w_warehouse_sk#26], Inner, BuildRight -(45) CometColumnarExchange +(45) CometExchange Input [10]: [w_warehouse_sk#7, i_item_sk#6, d_moy#11, mean#18, cov#19, w_warehouse_sk#26, i_item_sk#25, d_moy#30, mean#36, cov#37] -Arguments: rangepartitioning(w_warehouse_sk#7 ASC NULLS FIRST, i_item_sk#6 ASC NULLS FIRST, d_moy#11 ASC NULLS FIRST, mean#18 ASC NULLS FIRST, cov#19 ASC NULLS FIRST, d_moy#30 ASC NULLS FIRST, mean#36 ASC NULLS FIRST, cov#37 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] +Arguments: rangepartitioning(w_warehouse_sk#7 ASC NULLS FIRST, i_item_sk#6 ASC NULLS FIRST, d_moy#11 ASC NULLS FIRST, mean#18 ASC NULLS FIRST, cov#19 ASC NULLS FIRST, d_moy#30 ASC NULLS FIRST, mean#36 ASC NULLS FIRST, cov#37 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] (46) CometSort Input [10]: [w_warehouse_sk#7, i_item_sk#6, d_moy#11, mean#18, cov#19, w_warehouse_sk#26, i_item_sk#25, d_moy#30, mean#36, cov#37] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q39b/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q39b/simplified.txt index 1e0af57591..2cd1f70c8a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q39b/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q39b/simplified.txt @@ -2,7 +2,7 @@ WholeStageCodegen (1) CometColumnarToRow InputAdapter CometSort [w_warehouse_sk,i_item_sk,d_moy,mean,cov,w_warehouse_sk,i_item_sk,d_moy,mean,cov] - CometColumnarExchange [w_warehouse_sk,i_item_sk,d_moy,mean,cov,d_moy,mean,cov] #1 + CometExchange [w_warehouse_sk,i_item_sk,d_moy,mean,cov,d_moy,mean,cov] #1 CometBroadcastHashJoin [w_warehouse_sk,i_item_sk,d_moy,mean,cov,w_warehouse_sk,i_item_sk,d_moy,mean,cov] CometProject [stdev] [w_warehouse_sk,i_item_sk,d_moy,mean,cov] CometFilter [w_warehouse_sk,i_item_sk,d_moy,stdev,mean] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q64/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q64/explain.txt index ed591f262b..cbe790cd00 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q64/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q64/explain.txt @@ -1,7 +1,7 @@ == Physical Plan == * CometColumnarToRow (176) +- CometSort (175) - +- CometColumnarExchange (174) + +- CometExchange (174) +- CometProject (173) +- CometSortMergeJoin (172) :- CometSort (103) @@ -945,9 +945,9 @@ Arguments: [item_sk#82, store_name#83, store_zip#84], [item_sk#155, store_name#1 Input [25]: [product_name#81, item_sk#82, store_name#83, store_zip#84, b_street_number#85, b_streen_name#86, b_city#87, b_zip#88, c_street_number#89, c_street_name#90, c_city#91, c_zip#92, syear#93, cnt#94, s1#95, s2#96, s3#97, item_sk#155, store_name#156, store_zip#157, syear#158, cnt#159, s1#160, s2#161, s3#162] Arguments: [product_name#81, store_name#83, store_zip#84, b_street_number#85, b_streen_name#86, b_city#87, b_zip#88, c_street_number#89, c_street_name#90, c_city#91, c_zip#92, syear#93, cnt#94, s1#95, s2#96, s3#97, s1#160, s2#161, s3#162, syear#158, cnt#159], [product_name#81, store_name#83, store_zip#84, b_street_number#85, b_streen_name#86, b_city#87, b_zip#88, c_street_number#89, c_street_name#90, c_city#91, c_zip#92, syear#93, cnt#94, s1#95, s2#96, s3#97, s1#160, s2#161, s3#162, syear#158, cnt#159] -(174) CometColumnarExchange +(174) CometExchange Input [21]: [product_name#81, store_name#83, store_zip#84, b_street_number#85, b_streen_name#86, b_city#87, b_zip#88, c_street_number#89, c_street_name#90, c_city#91, c_zip#92, syear#93, cnt#94, s1#95, s2#96, s3#97, s1#160, s2#161, s3#162, syear#158, cnt#159] -Arguments: rangepartitioning(product_name#81 ASC NULLS FIRST, store_name#83 ASC NULLS FIRST, cnt#159 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=8] +Arguments: rangepartitioning(product_name#81 ASC NULLS FIRST, store_name#83 ASC NULLS FIRST, cnt#159 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=8] (175) CometSort Input [21]: [product_name#81, store_name#83, store_zip#84, b_street_number#85, b_streen_name#86, b_city#87, b_zip#88, c_street_number#89, c_street_name#90, c_city#91, c_zip#92, syear#93, cnt#94, s1#95, s2#96, s3#97, s1#160, s2#161, s3#162, syear#158, cnt#159] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q64/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q64/simplified.txt index acd00bedb5..43527978d8 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q64/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q64/simplified.txt @@ -2,7 +2,7 @@ WholeStageCodegen (1) CometColumnarToRow InputAdapter CometSort [product_name,store_name,store_zip,b_street_number,b_streen_name,b_city,b_zip,c_street_number,c_street_name,c_city,c_zip,syear,cnt,s1,s2,s3,s1,s2,s3,syear,cnt] - CometColumnarExchange [product_name,store_name,cnt] #1 + CometExchange [product_name,store_name,cnt] #1 CometProject [product_name,store_name,store_zip,b_street_number,b_streen_name,b_city,b_zip,c_street_number,c_street_name,c_city,c_zip,syear,cnt,s1,s2,s3,s1,s2,s3,syear,cnt] CometSortMergeJoin [product_name,item_sk,store_name,store_zip,b_street_number,b_streen_name,b_city,b_zip,c_street_number,c_street_name,c_city,c_zip,syear,cnt,s1,s2,s3,item_sk,store_name,store_zip,syear,cnt,s1,s2,s3] CometSort [product_name,item_sk,store_name,store_zip,b_street_number,b_streen_name,b_city,b_zip,c_street_number,c_street_name,c_city,c_zip,syear,cnt,s1,s2,s3] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q71/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q71/explain.txt index 9ecf9be21f..bb217d6417 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q71/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q71/explain.txt @@ -1,7 +1,7 @@ == Physical Plan == * CometColumnarToRow (37) +- CometSort (36) - +- CometColumnarExchange (35) + +- CometExchange (35) +- CometHashAggregate (34) +- CometExchange (33) +- CometHashAggregate (32) @@ -201,9 +201,9 @@ Input [5]: [i_brand#5, i_brand_id#2, t_hour#36, t_minute#37, sum#39] Keys [4]: [i_brand#5, i_brand_id#2, t_hour#36, t_minute#37] Functions [1]: [sum(UnscaledValue(ext_price#14))] -(35) CometColumnarExchange +(35) CometExchange Input [5]: [brand_id#40, brand#41, t_hour#36, t_minute#37, ext_price#42] -Arguments: rangepartitioning(ext_price#42 DESC NULLS LAST, brand_id#40 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] +Arguments: rangepartitioning(ext_price#42 DESC NULLS LAST, brand_id#40 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] (36) CometSort Input [5]: [brand_id#40, brand#41, t_hour#36, t_minute#37, ext_price#42] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q71/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q71/simplified.txt index 1dbf04331b..63e7c353ac 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q71/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q71/simplified.txt @@ -2,7 +2,7 @@ WholeStageCodegen (1) CometColumnarToRow InputAdapter CometSort [brand_id,brand,t_hour,t_minute,ext_price] - CometColumnarExchange [ext_price,brand_id] #1 + CometExchange [ext_price,brand_id] #1 CometHashAggregate [sum] [brand_id,brand,t_hour,t_minute,ext_price,i_brand_id,i_brand,sum(UnscaledValue(ext_price))] CometExchange [i_brand,i_brand_id,t_hour,t_minute] #2 CometHashAggregate [ext_price] [i_brand,i_brand_id,t_hour,t_minute,sum] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q73/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q73/explain.txt index 2de3317bc8..346e110371 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q73/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q73/explain.txt @@ -1,7 +1,7 @@ == Physical Plan == * CometColumnarToRow (33) +- CometSort (32) - +- CometColumnarExchange (31) + +- CometExchange (31) +- CometProject (30) +- CometBroadcastHashJoin (29) :- CometFilter (24) @@ -176,9 +176,9 @@ Arguments: [ss_customer_sk#1], [c_customer_sk#18], Inner, BuildRight Input [8]: [ss_ticket_number#4, ss_customer_sk#1, cnt#17, c_customer_sk#18, c_salutation#23, c_first_name#24, c_last_name#25, c_preferred_cust_flag#26] Arguments: [c_last_name#25, c_first_name#24, c_salutation#23, c_preferred_cust_flag#26, ss_ticket_number#4, cnt#17], [c_last_name#25, c_first_name#24, c_salutation#23, c_preferred_cust_flag#26, ss_ticket_number#4, cnt#17] -(31) CometColumnarExchange +(31) CometExchange Input [6]: [c_last_name#25, c_first_name#24, c_salutation#23, c_preferred_cust_flag#26, ss_ticket_number#4, cnt#17] -Arguments: rangepartitioning(cnt#17 DESC NULLS LAST, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] +Arguments: rangepartitioning(cnt#17 DESC NULLS LAST, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] (32) CometSort Input [6]: [c_last_name#25, c_first_name#24, c_salutation#23, c_preferred_cust_flag#26, ss_ticket_number#4, cnt#17] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q73/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q73/simplified.txt index fcd0c060e8..227dd00f2b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q73/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q73/simplified.txt @@ -2,7 +2,7 @@ WholeStageCodegen (1) CometColumnarToRow InputAdapter CometSort [c_last_name,c_first_name,c_salutation,c_preferred_cust_flag,ss_ticket_number,cnt] - CometColumnarExchange [cnt] #1 + CometExchange [cnt] #1 CometProject [c_last_name,c_first_name,c_salutation,c_preferred_cust_flag,ss_ticket_number,cnt] CometBroadcastHashJoin [ss_ticket_number,ss_customer_sk,cnt,c_customer_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag] CometFilter [ss_ticket_number,ss_customer_sk,cnt] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q91/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q91/explain.txt index 0ba113dd4d..cf532d2b41 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q91/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q91/explain.txt @@ -1,7 +1,7 @@ == Physical Plan == * CometColumnarToRow (43) +- CometSort (42) - +- CometColumnarExchange (41) + +- CometExchange (41) +- CometHashAggregate (40) +- CometExchange (39) +- CometHashAggregate (38) @@ -234,9 +234,9 @@ Input [6]: [cc_call_center_id#5, cc_name#3, cc_manager#4, cd_marital_status#23, Keys [5]: [cc_call_center_id#5, cc_name#3, cc_manager#4, cd_marital_status#23, cd_education_status#24] Functions [1]: [sum(UnscaledValue(cr_net_loss#8))] -(41) CometColumnarExchange +(41) CometExchange Input [4]: [Call_Center#28, Call_Center_Name#29, Manager#30, Returns_Loss#31] -Arguments: rangepartitioning(Returns_Loss#31 DESC NULLS LAST, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] +Arguments: rangepartitioning(Returns_Loss#31 DESC NULLS LAST, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] (42) CometSort Input [4]: [Call_Center#28, Call_Center_Name#29, Manager#30, Returns_Loss#31] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q91/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q91/simplified.txt index 575d92032a..3e9b8945da 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q91/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q91/simplified.txt @@ -2,7 +2,7 @@ WholeStageCodegen (1) CometColumnarToRow InputAdapter CometSort [Call_Center,Call_Center_Name,Manager,Returns_Loss] - CometColumnarExchange [Returns_Loss] #1 + CometExchange [Returns_Loss] #1 CometHashAggregate [cd_marital_status,cd_education_status,sum] [Call_Center,Call_Center_Name,Manager,Returns_Loss,cc_call_center_id,cc_name,cc_manager,sum(UnscaledValue(cr_net_loss))] CometExchange [cc_call_center_id,cc_name,cc_manager,cd_marital_status,cd_education_status] #2 CometHashAggregate [cr_net_loss] [cc_call_center_id,cc_name,cc_manager,cd_marital_status,cd_education_status,sum] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q34/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q34/explain.txt index 25339900c5..d8ee4f981e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q34/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q34/explain.txt @@ -1,7 +1,7 @@ == Physical Plan == * CometColumnarToRow (33) +- CometSort (32) - +- CometColumnarExchange (31) + +- CometExchange (31) +- CometProject (30) +- CometBroadcastHashJoin (29) :- CometFilter (24) @@ -176,9 +176,9 @@ Arguments: [ss_customer_sk#1], [c_customer_sk#18], Inner, BuildRight Input [8]: [ss_ticket_number#4, ss_customer_sk#1, cnt#17, c_customer_sk#18, c_salutation#23, c_first_name#24, c_last_name#25, c_preferred_cust_flag#26] Arguments: [c_last_name#25, c_first_name#24, c_salutation#23, c_preferred_cust_flag#26, ss_ticket_number#4, cnt#17], [c_last_name#25, c_first_name#24, c_salutation#23, c_preferred_cust_flag#26, ss_ticket_number#4, cnt#17] -(31) CometColumnarExchange +(31) CometExchange Input [6]: [c_last_name#25, c_first_name#24, c_salutation#23, c_preferred_cust_flag#26, ss_ticket_number#4, cnt#17] -Arguments: rangepartitioning(c_last_name#25 ASC NULLS FIRST, c_first_name#24 ASC NULLS FIRST, c_salutation#23 ASC NULLS FIRST, c_preferred_cust_flag#26 DESC NULLS LAST, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] +Arguments: rangepartitioning(c_last_name#25 ASC NULLS FIRST, c_first_name#24 ASC NULLS FIRST, c_salutation#23 ASC NULLS FIRST, c_preferred_cust_flag#26 DESC NULLS LAST, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] (32) CometSort Input [6]: [c_last_name#25, c_first_name#24, c_salutation#23, c_preferred_cust_flag#26, ss_ticket_number#4, cnt#17] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q34/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q34/simplified.txt index 9e9ee7fa05..47d0e35dd2 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q34/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q34/simplified.txt @@ -2,7 +2,7 @@ WholeStageCodegen (1) CometColumnarToRow InputAdapter CometSort [c_last_name,c_first_name,c_salutation,c_preferred_cust_flag,ss_ticket_number,cnt] - CometColumnarExchange [c_last_name,c_first_name,c_salutation,c_preferred_cust_flag] #1 + CometExchange [c_last_name,c_first_name,c_salutation,c_preferred_cust_flag] #1 CometProject [c_last_name,c_first_name,c_salutation,c_preferred_cust_flag,ss_ticket_number,cnt] CometBroadcastHashJoin [ss_ticket_number,ss_customer_sk,cnt,c_customer_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag] CometFilter [ss_ticket_number,ss_customer_sk,cnt] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q39a/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q39a/explain.txt index e5b1fc495c..4e7b77d7ec 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q39a/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q39a/explain.txt @@ -1,7 +1,7 @@ == Physical Plan == * CometColumnarToRow (47) +- CometSort (46) - +- CometColumnarExchange (45) + +- CometExchange (45) +- CometBroadcastHashJoin (44) :- CometProject (23) : +- CometFilter (22) @@ -253,9 +253,9 @@ Left output [5]: [w_warehouse_sk#7, i_item_sk#6, d_moy#11, mean#18, cov#19] Right output [5]: [w_warehouse_sk#26, i_item_sk#25, d_moy#30, mean#37, cov#38] Arguments: [i_item_sk#6, w_warehouse_sk#7], [i_item_sk#25, w_warehouse_sk#26], Inner, BuildRight -(45) CometColumnarExchange +(45) CometExchange Input [10]: [w_warehouse_sk#7, i_item_sk#6, d_moy#11, mean#18, cov#19, w_warehouse_sk#26, i_item_sk#25, d_moy#30, mean#37, cov#38] -Arguments: rangepartitioning(w_warehouse_sk#7 ASC NULLS FIRST, i_item_sk#6 ASC NULLS FIRST, d_moy#11 ASC NULLS FIRST, mean#18 ASC NULLS FIRST, cov#19 ASC NULLS FIRST, d_moy#30 ASC NULLS FIRST, mean#37 ASC NULLS FIRST, cov#38 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] +Arguments: rangepartitioning(w_warehouse_sk#7 ASC NULLS FIRST, i_item_sk#6 ASC NULLS FIRST, d_moy#11 ASC NULLS FIRST, mean#18 ASC NULLS FIRST, cov#19 ASC NULLS FIRST, d_moy#30 ASC NULLS FIRST, mean#37 ASC NULLS FIRST, cov#38 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] (46) CometSort Input [10]: [w_warehouse_sk#7, i_item_sk#6, d_moy#11, mean#18, cov#19, w_warehouse_sk#26, i_item_sk#25, d_moy#30, mean#37, cov#38] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q39a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q39a/simplified.txt index fd173660dd..e54435ae10 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q39a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q39a/simplified.txt @@ -2,7 +2,7 @@ WholeStageCodegen (1) CometColumnarToRow InputAdapter CometSort [w_warehouse_sk,i_item_sk,d_moy,mean,cov,w_warehouse_sk,i_item_sk,d_moy,mean,cov] - CometColumnarExchange [w_warehouse_sk,i_item_sk,d_moy,mean,cov,d_moy,mean,cov] #1 + CometExchange [w_warehouse_sk,i_item_sk,d_moy,mean,cov,d_moy,mean,cov] #1 CometBroadcastHashJoin [w_warehouse_sk,i_item_sk,d_moy,mean,cov,w_warehouse_sk,i_item_sk,d_moy,mean,cov] CometProject [stdev] [w_warehouse_sk,i_item_sk,d_moy,mean,cov] CometFilter [w_warehouse_sk,i_item_sk,d_moy,stdev,mean] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q39b/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q39b/explain.txt index 80ef707d35..7d8df40ba0 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q39b/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q39b/explain.txt @@ -1,7 +1,7 @@ == Physical Plan == * CometColumnarToRow (47) +- CometSort (46) - +- CometColumnarExchange (45) + +- CometExchange (45) +- CometBroadcastHashJoin (44) :- CometProject (23) : +- CometFilter (22) @@ -253,9 +253,9 @@ Left output [5]: [w_warehouse_sk#7, i_item_sk#6, d_moy#11, mean#18, cov#19] Right output [5]: [w_warehouse_sk#26, i_item_sk#25, d_moy#30, mean#37, cov#38] Arguments: [i_item_sk#6, w_warehouse_sk#7], [i_item_sk#25, w_warehouse_sk#26], Inner, BuildRight -(45) CometColumnarExchange +(45) CometExchange Input [10]: [w_warehouse_sk#7, i_item_sk#6, d_moy#11, mean#18, cov#19, w_warehouse_sk#26, i_item_sk#25, d_moy#30, mean#37, cov#38] -Arguments: rangepartitioning(w_warehouse_sk#7 ASC NULLS FIRST, i_item_sk#6 ASC NULLS FIRST, d_moy#11 ASC NULLS FIRST, mean#18 ASC NULLS FIRST, cov#19 ASC NULLS FIRST, d_moy#30 ASC NULLS FIRST, mean#37 ASC NULLS FIRST, cov#38 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] +Arguments: rangepartitioning(w_warehouse_sk#7 ASC NULLS FIRST, i_item_sk#6 ASC NULLS FIRST, d_moy#11 ASC NULLS FIRST, mean#18 ASC NULLS FIRST, cov#19 ASC NULLS FIRST, d_moy#30 ASC NULLS FIRST, mean#37 ASC NULLS FIRST, cov#38 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] (46) CometSort Input [10]: [w_warehouse_sk#7, i_item_sk#6, d_moy#11, mean#18, cov#19, w_warehouse_sk#26, i_item_sk#25, d_moy#30, mean#37, cov#38] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q39b/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q39b/simplified.txt index fd173660dd..e54435ae10 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q39b/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q39b/simplified.txt @@ -2,7 +2,7 @@ WholeStageCodegen (1) CometColumnarToRow InputAdapter CometSort [w_warehouse_sk,i_item_sk,d_moy,mean,cov,w_warehouse_sk,i_item_sk,d_moy,mean,cov] - CometColumnarExchange [w_warehouse_sk,i_item_sk,d_moy,mean,cov,d_moy,mean,cov] #1 + CometExchange [w_warehouse_sk,i_item_sk,d_moy,mean,cov,d_moy,mean,cov] #1 CometBroadcastHashJoin [w_warehouse_sk,i_item_sk,d_moy,mean,cov,w_warehouse_sk,i_item_sk,d_moy,mean,cov] CometProject [stdev] [w_warehouse_sk,i_item_sk,d_moy,mean,cov] CometFilter [w_warehouse_sk,i_item_sk,d_moy,stdev,mean] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q64/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q64/explain.txt index c098795280..44c02a061d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q64/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q64/explain.txt @@ -1,7 +1,7 @@ == Physical Plan == * CometColumnarToRow (185) +- CometSort (184) - +- CometColumnarExchange (183) + +- CometExchange (183) +- CometProject (182) +- CometSortMergeJoin (181) :- CometSort (112) @@ -1013,9 +1013,9 @@ Arguments: [item_sk#92, store_name#93, store_zip#94], [item_sk#178, store_name#1 Input [25]: [product_name#91, item_sk#92, store_name#93, store_zip#94, b_street_number#95, b_streen_name#96, b_city#97, b_zip#98, c_street_number#99, c_street_name#100, c_city#101, c_zip#102, syear#103, cnt#104, s1#105, s2#106, s3#107, item_sk#178, store_name#179, store_zip#180, syear#181, cnt#182, s1#183, s2#184, s3#185] Arguments: [product_name#91, store_name#93, store_zip#94, b_street_number#95, b_streen_name#96, b_city#97, b_zip#98, c_street_number#99, c_street_name#100, c_city#101, c_zip#102, syear#103, cnt#104, s1#105, s2#106, s3#107, s1#183, s2#184, s3#185, syear#181, cnt#182], [product_name#91, store_name#93, store_zip#94, b_street_number#95, b_streen_name#96, b_city#97, b_zip#98, c_street_number#99, c_street_name#100, c_city#101, c_zip#102, syear#103, cnt#104, s1#105, s2#106, s3#107, s1#183, s2#184, s3#185, syear#181, cnt#182] -(183) CometColumnarExchange +(183) CometExchange Input [21]: [product_name#91, store_name#93, store_zip#94, b_street_number#95, b_streen_name#96, b_city#97, b_zip#98, c_street_number#99, c_street_name#100, c_city#101, c_zip#102, syear#103, cnt#104, s1#105, s2#106, s3#107, s1#183, s2#184, s3#185, syear#181, cnt#182] -Arguments: rangepartitioning(product_name#91 ASC NULLS FIRST, store_name#93 ASC NULLS FIRST, cnt#182 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=17] +Arguments: rangepartitioning(product_name#91 ASC NULLS FIRST, store_name#93 ASC NULLS FIRST, cnt#182 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=17] (184) CometSort Input [21]: [product_name#91, store_name#93, store_zip#94, b_street_number#95, b_streen_name#96, b_city#97, b_zip#98, c_street_number#99, c_street_name#100, c_city#101, c_zip#102, syear#103, cnt#104, s1#105, s2#106, s3#107, s1#183, s2#184, s3#185, syear#181, cnt#182] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q64/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q64/simplified.txt index 6f68f25d4f..944f6edbde 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q64/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q64/simplified.txt @@ -2,7 +2,7 @@ WholeStageCodegen (37) CometColumnarToRow InputAdapter CometSort [product_name,store_name,store_zip,b_street_number,b_streen_name,b_city,b_zip,c_street_number,c_street_name,c_city,c_zip,syear,cnt,s1,s2,s3,s1,s2,s3,syear,cnt] - CometColumnarExchange [product_name,store_name,cnt] #1 + CometExchange [product_name,store_name,cnt] #1 CometProject [product_name,store_name,store_zip,b_street_number,b_streen_name,b_city,b_zip,c_street_number,c_street_name,c_city,c_zip,syear,cnt,s1,s2,s3,s1,s2,s3,syear,cnt] CometSortMergeJoin [product_name,item_sk,store_name,store_zip,b_street_number,b_streen_name,b_city,b_zip,c_street_number,c_street_name,c_city,c_zip,syear,cnt,s1,s2,s3,item_sk,store_name,store_zip,syear,cnt,s1,s2,s3] CometSort [product_name,item_sk,store_name,store_zip,b_street_number,b_streen_name,b_city,b_zip,c_street_number,c_street_name,c_city,c_zip,syear,cnt,s1,s2,s3] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q73/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q73/explain.txt index 87b805556e..3bb13dcb30 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q73/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q73/explain.txt @@ -1,7 +1,7 @@ == Physical Plan == * CometColumnarToRow (33) +- CometSort (32) - +- CometColumnarExchange (31) + +- CometExchange (31) +- CometProject (30) +- CometBroadcastHashJoin (29) :- CometFilter (24) @@ -176,9 +176,9 @@ Arguments: [ss_customer_sk#1], [c_customer_sk#18], Inner, BuildRight Input [8]: [ss_ticket_number#4, ss_customer_sk#1, cnt#17, c_customer_sk#18, c_salutation#23, c_first_name#24, c_last_name#25, c_preferred_cust_flag#26] Arguments: [c_last_name#25, c_first_name#24, c_salutation#23, c_preferred_cust_flag#26, ss_ticket_number#4, cnt#17], [c_last_name#25, c_first_name#24, c_salutation#23, c_preferred_cust_flag#26, ss_ticket_number#4, cnt#17] -(31) CometColumnarExchange +(31) CometExchange Input [6]: [c_last_name#25, c_first_name#24, c_salutation#23, c_preferred_cust_flag#26, ss_ticket_number#4, cnt#17] -Arguments: rangepartitioning(cnt#17 DESC NULLS LAST, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] +Arguments: rangepartitioning(cnt#17 DESC NULLS LAST, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] (32) CometSort Input [6]: [c_last_name#25, c_first_name#24, c_salutation#23, c_preferred_cust_flag#26, ss_ticket_number#4, cnt#17] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q73/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q73/simplified.txt index fcd0c060e8..227dd00f2b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q73/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q73/simplified.txt @@ -2,7 +2,7 @@ WholeStageCodegen (1) CometColumnarToRow InputAdapter CometSort [c_last_name,c_first_name,c_salutation,c_preferred_cust_flag,ss_ticket_number,cnt] - CometColumnarExchange [cnt] #1 + CometExchange [cnt] #1 CometProject [c_last_name,c_first_name,c_salutation,c_preferred_cust_flag,ss_ticket_number,cnt] CometBroadcastHashJoin [ss_ticket_number,ss_customer_sk,cnt,c_customer_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag] CometFilter [ss_ticket_number,ss_customer_sk,cnt] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q2/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q2/explain.txt index 00c1602acc..e13ae7371e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q2/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q2/explain.txt @@ -1,7 +1,7 @@ == Physical Plan == * CometColumnarToRow (34) +- CometSort (33) - +- CometColumnarExchange (32) + +- CometExchange (32) +- CometProject (31) +- CometBroadcastHashJoin (30) :- CometProject (20) @@ -180,9 +180,9 @@ Arguments: [d_week_seq1#29], [(d_week_seq2#46 - 53)], Inner, BuildRight Input [16]: [d_week_seq1#29, sun_sales1#30, mon_sales1#31, tue_sales1#32, wed_sales1#33, thu_sales1#34, fri_sales1#35, sat_sales1#36, d_week_seq2#46, sun_sales2#47, mon_sales2#48, tue_sales2#49, wed_sales2#50, thu_sales2#51, fri_sales2#52, sat_sales2#53] Arguments: [d_week_seq1#29, round((sun_sales1 / sun_sales2), 2)#54, round((mon_sales1 / mon_sales2), 2)#55, round((tue_sales1 / tue_sales2), 2)#56, round((wed_sales1 / wed_sales2), 2)#57, round((thu_sales1 / thu_sales2), 2)#58, round((fri_sales1 / fri_sales2), 2)#59, round((sat_sales1 / sat_sales2), 2)#60], [d_week_seq1#29, round((sun_sales1#30 / sun_sales2#47), 2) AS round((sun_sales1 / sun_sales2), 2)#54, round((mon_sales1#31 / mon_sales2#48), 2) AS round((mon_sales1 / mon_sales2), 2)#55, round((tue_sales1#32 / tue_sales2#49), 2) AS round((tue_sales1 / tue_sales2), 2)#56, round((wed_sales1#33 / wed_sales2#50), 2) AS round((wed_sales1 / wed_sales2), 2)#57, round((thu_sales1#34 / thu_sales2#51), 2) AS round((thu_sales1 / thu_sales2), 2)#58, round((fri_sales1#35 / fri_sales2#52), 2) AS round((fri_sales1 / fri_sales2), 2)#59, round((sat_sales1#36 / sat_sales2#53), 2) AS round((sat_sales1 / sat_sales2), 2)#60] -(32) CometColumnarExchange +(32) CometExchange Input [8]: [d_week_seq1#29, round((sun_sales1 / sun_sales2), 2)#54, round((mon_sales1 / mon_sales2), 2)#55, round((tue_sales1 / tue_sales2), 2)#56, round((wed_sales1 / wed_sales2), 2)#57, round((thu_sales1 / thu_sales2), 2)#58, round((fri_sales1 / fri_sales2), 2)#59, round((sat_sales1 / sat_sales2), 2)#60] -Arguments: rangepartitioning(d_week_seq1#29 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] +Arguments: rangepartitioning(d_week_seq1#29 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] (33) CometSort Input [8]: [d_week_seq1#29, round((sun_sales1 / sun_sales2), 2)#54, round((mon_sales1 / mon_sales2), 2)#55, round((tue_sales1 / tue_sales2), 2)#56, round((wed_sales1 / wed_sales2), 2)#57, round((thu_sales1 / thu_sales2), 2)#58, round((fri_sales1 / fri_sales2), 2)#59, round((sat_sales1 / sat_sales2), 2)#60] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q2/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q2/simplified.txt index e9a2499f38..852c5fca0f 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q2/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q2/simplified.txt @@ -2,7 +2,7 @@ WholeStageCodegen (1) CometColumnarToRow InputAdapter CometSort [d_week_seq1,round((sun_sales1 / sun_sales2), 2),round((mon_sales1 / mon_sales2), 2),round((tue_sales1 / tue_sales2), 2),round((wed_sales1 / wed_sales2), 2),round((thu_sales1 / thu_sales2), 2),round((fri_sales1 / fri_sales2), 2),round((sat_sales1 / sat_sales2), 2)] - CometColumnarExchange [d_week_seq1] #1 + CometExchange [d_week_seq1] #1 CometProject [sun_sales1,sun_sales2,mon_sales1,mon_sales2,tue_sales1,tue_sales2,wed_sales1,wed_sales2,thu_sales1,thu_sales2,fri_sales1,fri_sales2,sat_sales1,sat_sales2] [d_week_seq1,round((sun_sales1 / sun_sales2), 2),round((mon_sales1 / mon_sales2), 2),round((tue_sales1 / tue_sales2), 2),round((wed_sales1 / wed_sales2), 2),round((thu_sales1 / thu_sales2), 2),round((fri_sales1 / fri_sales2), 2),round((sat_sales1 / sat_sales2), 2)] CometBroadcastHashJoin [d_week_seq1,sun_sales1,mon_sales1,tue_sales1,wed_sales1,thu_sales1,fri_sales1,sat_sales1,d_week_seq2,sun_sales2,mon_sales2,tue_sales2,wed_sales2,thu_sales2,fri_sales2,sat_sales2] CometProject [d_week_seq,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales] [d_week_seq1,sun_sales1,mon_sales1,tue_sales1,wed_sales1,thu_sales1,fri_sales1,sat_sales1] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q31/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q31/explain.txt index 4f065b40bd..83d717e295 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q31/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q31/explain.txt @@ -1,7 +1,7 @@ == Physical Plan == * CometColumnarToRow (90) +- CometSort (89) - +- CometColumnarExchange (88) + +- CometExchange (88) +- CometProject (87) +- CometBroadcastHashJoin (86) :- CometProject (73) @@ -496,9 +496,9 @@ Arguments: [ca_county#42], [ca_county#64], Inner, (CASE WHEN (web_sales#55 > 0.0 Input [10]: [ca_county#9, d_year#6, store_sales#22, store_sales#21, store_sales#33, ca_county#42, web_sales#44, web_sales#55, ca_county#64, web_sales#66] Arguments: [ca_county#9, d_year#6, web_q1_q2_increase#67, store_q1_q2_increase#68, web_q2_q3_increase#69, store_q2_q3_increase#70], [ca_county#9, d_year#6, (web_sales#55 / web_sales#44) AS web_q1_q2_increase#67, (store_sales#21 / store_sales#22) AS store_q1_q2_increase#68, (web_sales#66 / web_sales#55) AS web_q2_q3_increase#69, (store_sales#33 / store_sales#21) AS store_q2_q3_increase#70] -(88) CometColumnarExchange +(88) CometExchange Input [6]: [ca_county#9, d_year#6, web_q1_q2_increase#67, store_q1_q2_increase#68, web_q2_q3_increase#69, store_q2_q3_increase#70] -Arguments: rangepartitioning(ca_county#9 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] +Arguments: rangepartitioning(ca_county#9 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=7] (89) CometSort Input [6]: [ca_county#9, d_year#6, web_q1_q2_increase#67, store_q1_q2_increase#68, web_q2_q3_increase#69, store_q2_q3_increase#70] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q31/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q31/simplified.txt index edc4dc3923..2f97384320 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q31/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q31/simplified.txt @@ -2,7 +2,7 @@ WholeStageCodegen (1) CometColumnarToRow InputAdapter CometSort [ca_county,d_year,web_q1_q2_increase,store_q1_q2_increase,web_q2_q3_increase,store_q2_q3_increase] - CometColumnarExchange [ca_county] #1 + CometExchange [ca_county] #1 CometProject [web_sales,web_sales,store_sales,store_sales,web_sales,store_sales] [ca_county,d_year,web_q1_q2_increase,store_q1_q2_increase,web_q2_q3_increase,store_q2_q3_increase] CometBroadcastHashJoin [ca_county,d_year,store_sales,store_sales,store_sales,ca_county,web_sales,web_sales,ca_county,web_sales] CometProject [ca_county,d_year,store_sales,store_sales,store_sales,ca_county,web_sales,web_sales] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q34/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q34/explain.txt index ceab9d3f0a..287e323952 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q34/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q34/explain.txt @@ -1,7 +1,7 @@ == Physical Plan == * CometColumnarToRow (33) +- CometSort (32) - +- CometColumnarExchange (31) + +- CometExchange (31) +- CometProject (30) +- CometBroadcastHashJoin (29) :- CometFilter (24) @@ -176,9 +176,9 @@ Arguments: [ss_customer_sk#1], [c_customer_sk#18], Inner, BuildRight Input [8]: [ss_ticket_number#4, ss_customer_sk#1, cnt#17, c_customer_sk#18, c_salutation#23, c_first_name#24, c_last_name#25, c_preferred_cust_flag#26] Arguments: [c_last_name#25, c_first_name#24, c_salutation#23, c_preferred_cust_flag#26, ss_ticket_number#4, cnt#17], [c_last_name#25, c_first_name#24, c_salutation#23, c_preferred_cust_flag#26, ss_ticket_number#4, cnt#17] -(31) CometColumnarExchange +(31) CometExchange Input [6]: [c_last_name#25, c_first_name#24, c_salutation#23, c_preferred_cust_flag#26, ss_ticket_number#4, cnt#17] -Arguments: rangepartitioning(c_last_name#25 ASC NULLS FIRST, c_first_name#24 ASC NULLS FIRST, c_salutation#23 ASC NULLS FIRST, c_preferred_cust_flag#26 DESC NULLS LAST, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] +Arguments: rangepartitioning(c_last_name#25 ASC NULLS FIRST, c_first_name#24 ASC NULLS FIRST, c_salutation#23 ASC NULLS FIRST, c_preferred_cust_flag#26 DESC NULLS LAST, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] (32) CometSort Input [6]: [c_last_name#25, c_first_name#24, c_salutation#23, c_preferred_cust_flag#26, ss_ticket_number#4, cnt#17] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q34/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q34/simplified.txt index 9e9ee7fa05..47d0e35dd2 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q34/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q34/simplified.txt @@ -2,7 +2,7 @@ WholeStageCodegen (1) CometColumnarToRow InputAdapter CometSort [c_last_name,c_first_name,c_salutation,c_preferred_cust_flag,ss_ticket_number,cnt] - CometColumnarExchange [c_last_name,c_first_name,c_salutation,c_preferred_cust_flag] #1 + CometExchange [c_last_name,c_first_name,c_salutation,c_preferred_cust_flag] #1 CometProject [c_last_name,c_first_name,c_salutation,c_preferred_cust_flag,ss_ticket_number,cnt] CometBroadcastHashJoin [ss_ticket_number,ss_customer_sk,cnt,c_customer_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag] CometFilter [ss_ticket_number,ss_customer_sk,cnt] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39a/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39a/explain.txt index 8e6d9265d6..ed74780c2c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39a/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39a/explain.txt @@ -1,7 +1,7 @@ == Physical Plan == * CometColumnarToRow (47) +- CometSort (46) - +- CometColumnarExchange (45) + +- CometExchange (45) +- CometBroadcastHashJoin (44) :- CometProject (23) : +- CometFilter (22) @@ -253,9 +253,9 @@ Left output [5]: [w_warehouse_sk#7, i_item_sk#6, d_moy#11, mean#18, cov#19] Right output [5]: [w_warehouse_sk#26, i_item_sk#25, d_moy#30, mean#36, cov#37] Arguments: [i_item_sk#6, w_warehouse_sk#7], [i_item_sk#25, w_warehouse_sk#26], Inner, BuildRight -(45) CometColumnarExchange +(45) CometExchange Input [10]: [w_warehouse_sk#7, i_item_sk#6, d_moy#11, mean#18, cov#19, w_warehouse_sk#26, i_item_sk#25, d_moy#30, mean#36, cov#37] -Arguments: rangepartitioning(w_warehouse_sk#7 ASC NULLS FIRST, i_item_sk#6 ASC NULLS FIRST, d_moy#11 ASC NULLS FIRST, mean#18 ASC NULLS FIRST, cov#19 ASC NULLS FIRST, d_moy#30 ASC NULLS FIRST, mean#36 ASC NULLS FIRST, cov#37 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] +Arguments: rangepartitioning(w_warehouse_sk#7 ASC NULLS FIRST, i_item_sk#6 ASC NULLS FIRST, d_moy#11 ASC NULLS FIRST, mean#18 ASC NULLS FIRST, cov#19 ASC NULLS FIRST, d_moy#30 ASC NULLS FIRST, mean#36 ASC NULLS FIRST, cov#37 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] (46) CometSort Input [10]: [w_warehouse_sk#7, i_item_sk#6, d_moy#11, mean#18, cov#19, w_warehouse_sk#26, i_item_sk#25, d_moy#30, mean#36, cov#37] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39a/simplified.txt index 1e0af57591..2cd1f70c8a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39a/simplified.txt @@ -2,7 +2,7 @@ WholeStageCodegen (1) CometColumnarToRow InputAdapter CometSort [w_warehouse_sk,i_item_sk,d_moy,mean,cov,w_warehouse_sk,i_item_sk,d_moy,mean,cov] - CometColumnarExchange [w_warehouse_sk,i_item_sk,d_moy,mean,cov,d_moy,mean,cov] #1 + CometExchange [w_warehouse_sk,i_item_sk,d_moy,mean,cov,d_moy,mean,cov] #1 CometBroadcastHashJoin [w_warehouse_sk,i_item_sk,d_moy,mean,cov,w_warehouse_sk,i_item_sk,d_moy,mean,cov] CometProject [stdev] [w_warehouse_sk,i_item_sk,d_moy,mean,cov] CometFilter [w_warehouse_sk,i_item_sk,d_moy,stdev,mean] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39b/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39b/explain.txt index 69b6185645..2a31b0d8b2 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39b/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39b/explain.txt @@ -1,7 +1,7 @@ == Physical Plan == * CometColumnarToRow (47) +- CometSort (46) - +- CometColumnarExchange (45) + +- CometExchange (45) +- CometBroadcastHashJoin (44) :- CometProject (23) : +- CometFilter (22) @@ -253,9 +253,9 @@ Left output [5]: [w_warehouse_sk#7, i_item_sk#6, d_moy#11, mean#18, cov#19] Right output [5]: [w_warehouse_sk#26, i_item_sk#25, d_moy#30, mean#36, cov#37] Arguments: [i_item_sk#6, w_warehouse_sk#7], [i_item_sk#25, w_warehouse_sk#26], Inner, BuildRight -(45) CometColumnarExchange +(45) CometExchange Input [10]: [w_warehouse_sk#7, i_item_sk#6, d_moy#11, mean#18, cov#19, w_warehouse_sk#26, i_item_sk#25, d_moy#30, mean#36, cov#37] -Arguments: rangepartitioning(w_warehouse_sk#7 ASC NULLS FIRST, i_item_sk#6 ASC NULLS FIRST, d_moy#11 ASC NULLS FIRST, mean#18 ASC NULLS FIRST, cov#19 ASC NULLS FIRST, d_moy#30 ASC NULLS FIRST, mean#36 ASC NULLS FIRST, cov#37 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] +Arguments: rangepartitioning(w_warehouse_sk#7 ASC NULLS FIRST, i_item_sk#6 ASC NULLS FIRST, d_moy#11 ASC NULLS FIRST, mean#18 ASC NULLS FIRST, cov#19 ASC NULLS FIRST, d_moy#30 ASC NULLS FIRST, mean#36 ASC NULLS FIRST, cov#37 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] (46) CometSort Input [10]: [w_warehouse_sk#7, i_item_sk#6, d_moy#11, mean#18, cov#19, w_warehouse_sk#26, i_item_sk#25, d_moy#30, mean#36, cov#37] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39b/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39b/simplified.txt index 1e0af57591..2cd1f70c8a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39b/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q39b/simplified.txt @@ -2,7 +2,7 @@ WholeStageCodegen (1) CometColumnarToRow InputAdapter CometSort [w_warehouse_sk,i_item_sk,d_moy,mean,cov,w_warehouse_sk,i_item_sk,d_moy,mean,cov] - CometColumnarExchange [w_warehouse_sk,i_item_sk,d_moy,mean,cov,d_moy,mean,cov] #1 + CometExchange [w_warehouse_sk,i_item_sk,d_moy,mean,cov,d_moy,mean,cov] #1 CometBroadcastHashJoin [w_warehouse_sk,i_item_sk,d_moy,mean,cov,w_warehouse_sk,i_item_sk,d_moy,mean,cov] CometProject [stdev] [w_warehouse_sk,i_item_sk,d_moy,mean,cov] CometFilter [w_warehouse_sk,i_item_sk,d_moy,stdev,mean] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q64/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q64/explain.txt index ed591f262b..cbe790cd00 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q64/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q64/explain.txt @@ -1,7 +1,7 @@ == Physical Plan == * CometColumnarToRow (176) +- CometSort (175) - +- CometColumnarExchange (174) + +- CometExchange (174) +- CometProject (173) +- CometSortMergeJoin (172) :- CometSort (103) @@ -945,9 +945,9 @@ Arguments: [item_sk#82, store_name#83, store_zip#84], [item_sk#155, store_name#1 Input [25]: [product_name#81, item_sk#82, store_name#83, store_zip#84, b_street_number#85, b_streen_name#86, b_city#87, b_zip#88, c_street_number#89, c_street_name#90, c_city#91, c_zip#92, syear#93, cnt#94, s1#95, s2#96, s3#97, item_sk#155, store_name#156, store_zip#157, syear#158, cnt#159, s1#160, s2#161, s3#162] Arguments: [product_name#81, store_name#83, store_zip#84, b_street_number#85, b_streen_name#86, b_city#87, b_zip#88, c_street_number#89, c_street_name#90, c_city#91, c_zip#92, syear#93, cnt#94, s1#95, s2#96, s3#97, s1#160, s2#161, s3#162, syear#158, cnt#159], [product_name#81, store_name#83, store_zip#84, b_street_number#85, b_streen_name#86, b_city#87, b_zip#88, c_street_number#89, c_street_name#90, c_city#91, c_zip#92, syear#93, cnt#94, s1#95, s2#96, s3#97, s1#160, s2#161, s3#162, syear#158, cnt#159] -(174) CometColumnarExchange +(174) CometExchange Input [21]: [product_name#81, store_name#83, store_zip#84, b_street_number#85, b_streen_name#86, b_city#87, b_zip#88, c_street_number#89, c_street_name#90, c_city#91, c_zip#92, syear#93, cnt#94, s1#95, s2#96, s3#97, s1#160, s2#161, s3#162, syear#158, cnt#159] -Arguments: rangepartitioning(product_name#81 ASC NULLS FIRST, store_name#83 ASC NULLS FIRST, cnt#159 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=8] +Arguments: rangepartitioning(product_name#81 ASC NULLS FIRST, store_name#83 ASC NULLS FIRST, cnt#159 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=8] (175) CometSort Input [21]: [product_name#81, store_name#83, store_zip#84, b_street_number#85, b_streen_name#86, b_city#87, b_zip#88, c_street_number#89, c_street_name#90, c_city#91, c_zip#92, syear#93, cnt#94, s1#95, s2#96, s3#97, s1#160, s2#161, s3#162, syear#158, cnt#159] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q64/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q64/simplified.txt index acd00bedb5..43527978d8 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q64/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q64/simplified.txt @@ -2,7 +2,7 @@ WholeStageCodegen (1) CometColumnarToRow InputAdapter CometSort [product_name,store_name,store_zip,b_street_number,b_streen_name,b_city,b_zip,c_street_number,c_street_name,c_city,c_zip,syear,cnt,s1,s2,s3,s1,s2,s3,syear,cnt] - CometColumnarExchange [product_name,store_name,cnt] #1 + CometExchange [product_name,store_name,cnt] #1 CometProject [product_name,store_name,store_zip,b_street_number,b_streen_name,b_city,b_zip,c_street_number,c_street_name,c_city,c_zip,syear,cnt,s1,s2,s3,s1,s2,s3,syear,cnt] CometSortMergeJoin [product_name,item_sk,store_name,store_zip,b_street_number,b_streen_name,b_city,b_zip,c_street_number,c_street_name,c_city,c_zip,syear,cnt,s1,s2,s3,item_sk,store_name,store_zip,syear,cnt,s1,s2,s3] CometSort [product_name,item_sk,store_name,store_zip,b_street_number,b_streen_name,b_city,b_zip,c_street_number,c_street_name,c_city,c_zip,syear,cnt,s1,s2,s3] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q71/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q71/explain.txt index 9ecf9be21f..bb217d6417 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q71/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q71/explain.txt @@ -1,7 +1,7 @@ == Physical Plan == * CometColumnarToRow (37) +- CometSort (36) - +- CometColumnarExchange (35) + +- CometExchange (35) +- CometHashAggregate (34) +- CometExchange (33) +- CometHashAggregate (32) @@ -201,9 +201,9 @@ Input [5]: [i_brand#5, i_brand_id#2, t_hour#36, t_minute#37, sum#39] Keys [4]: [i_brand#5, i_brand_id#2, t_hour#36, t_minute#37] Functions [1]: [sum(UnscaledValue(ext_price#14))] -(35) CometColumnarExchange +(35) CometExchange Input [5]: [brand_id#40, brand#41, t_hour#36, t_minute#37, ext_price#42] -Arguments: rangepartitioning(ext_price#42 DESC NULLS LAST, brand_id#40 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] +Arguments: rangepartitioning(ext_price#42 DESC NULLS LAST, brand_id#40 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] (36) CometSort Input [5]: [brand_id#40, brand#41, t_hour#36, t_minute#37, ext_price#42] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q71/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q71/simplified.txt index 1dbf04331b..63e7c353ac 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q71/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q71/simplified.txt @@ -2,7 +2,7 @@ WholeStageCodegen (1) CometColumnarToRow InputAdapter CometSort [brand_id,brand,t_hour,t_minute,ext_price] - CometColumnarExchange [ext_price,brand_id] #1 + CometExchange [ext_price,brand_id] #1 CometHashAggregate [sum] [brand_id,brand,t_hour,t_minute,ext_price,i_brand_id,i_brand,sum(UnscaledValue(ext_price))] CometExchange [i_brand,i_brand_id,t_hour,t_minute] #2 CometHashAggregate [ext_price] [i_brand,i_brand_id,t_hour,t_minute,sum] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q73/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q73/explain.txt index 2de3317bc8..346e110371 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q73/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q73/explain.txt @@ -1,7 +1,7 @@ == Physical Plan == * CometColumnarToRow (33) +- CometSort (32) - +- CometColumnarExchange (31) + +- CometExchange (31) +- CometProject (30) +- CometBroadcastHashJoin (29) :- CometFilter (24) @@ -176,9 +176,9 @@ Arguments: [ss_customer_sk#1], [c_customer_sk#18], Inner, BuildRight Input [8]: [ss_ticket_number#4, ss_customer_sk#1, cnt#17, c_customer_sk#18, c_salutation#23, c_first_name#24, c_last_name#25, c_preferred_cust_flag#26] Arguments: [c_last_name#25, c_first_name#24, c_salutation#23, c_preferred_cust_flag#26, ss_ticket_number#4, cnt#17], [c_last_name#25, c_first_name#24, c_salutation#23, c_preferred_cust_flag#26, ss_ticket_number#4, cnt#17] -(31) CometColumnarExchange +(31) CometExchange Input [6]: [c_last_name#25, c_first_name#24, c_salutation#23, c_preferred_cust_flag#26, ss_ticket_number#4, cnt#17] -Arguments: rangepartitioning(cnt#17 DESC NULLS LAST, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] +Arguments: rangepartitioning(cnt#17 DESC NULLS LAST, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] (32) CometSort Input [6]: [c_last_name#25, c_first_name#24, c_salutation#23, c_preferred_cust_flag#26, ss_ticket_number#4, cnt#17] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q73/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q73/simplified.txt index fcd0c060e8..227dd00f2b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q73/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q73/simplified.txt @@ -2,7 +2,7 @@ WholeStageCodegen (1) CometColumnarToRow InputAdapter CometSort [c_last_name,c_first_name,c_salutation,c_preferred_cust_flag,ss_ticket_number,cnt] - CometColumnarExchange [cnt] #1 + CometExchange [cnt] #1 CometProject [c_last_name,c_first_name,c_salutation,c_preferred_cust_flag,ss_ticket_number,cnt] CometBroadcastHashJoin [ss_ticket_number,ss_customer_sk,cnt,c_customer_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag] CometFilter [ss_ticket_number,ss_customer_sk,cnt] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q91/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q91/explain.txt index 0ba113dd4d..cf532d2b41 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q91/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q91/explain.txt @@ -1,7 +1,7 @@ == Physical Plan == * CometColumnarToRow (43) +- CometSort (42) - +- CometColumnarExchange (41) + +- CometExchange (41) +- CometHashAggregate (40) +- CometExchange (39) +- CometHashAggregate (38) @@ -234,9 +234,9 @@ Input [6]: [cc_call_center_id#5, cc_name#3, cc_manager#4, cd_marital_status#23, Keys [5]: [cc_call_center_id#5, cc_name#3, cc_manager#4, cd_marital_status#23, cd_education_status#24] Functions [1]: [sum(UnscaledValue(cr_net_loss#8))] -(41) CometColumnarExchange +(41) CometExchange Input [4]: [Call_Center#28, Call_Center_Name#29, Manager#30, Returns_Loss#31] -Arguments: rangepartitioning(Returns_Loss#31 DESC NULLS LAST, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] +Arguments: rangepartitioning(Returns_Loss#31 DESC NULLS LAST, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] (42) CometSort Input [4]: [Call_Center#28, Call_Center_Name#29, Manager#30, Returns_Loss#31] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q91/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q91/simplified.txt index 575d92032a..3e9b8945da 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q91/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q91/simplified.txt @@ -2,7 +2,7 @@ WholeStageCodegen (1) CometColumnarToRow InputAdapter CometSort [Call_Center,Call_Center_Name,Manager,Returns_Loss] - CometColumnarExchange [Returns_Loss] #1 + CometExchange [Returns_Loss] #1 CometHashAggregate [cd_marital_status,cd_education_status,sum] [Call_Center,Call_Center_Name,Manager,Returns_Loss,cc_call_center_id,cc_name,cc_manager,sum(UnscaledValue(cr_net_loss))] CometExchange [cc_call_center_id,cc_name,cc_manager,cd_marital_status,cd_education_status] #2 CometHashAggregate [cr_net_loss] [cc_call_center_id,cc_name,cc_manager,cd_marital_status,cd_education_status,sum] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q34/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q34/explain.txt index 692b84d605..50af5f4f75 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q34/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q34/explain.txt @@ -1,7 +1,7 @@ == Physical Plan == * CometColumnarToRow (33) +- CometSort (32) - +- CometColumnarExchange (31) + +- CometExchange (31) +- CometProject (30) +- CometBroadcastHashJoin (29) :- CometFilter (24) @@ -176,9 +176,9 @@ Arguments: [ss_customer_sk#1], [c_customer_sk#18], Inner, BuildRight Input [8]: [ss_ticket_number#4, ss_customer_sk#1, cnt#17, c_customer_sk#18, c_salutation#23, c_first_name#24, c_last_name#25, c_preferred_cust_flag#26] Arguments: [c_last_name#25, c_first_name#24, c_salutation#23, c_preferred_cust_flag#26, ss_ticket_number#4, cnt#17], [c_last_name#25, c_first_name#24, c_salutation#23, c_preferred_cust_flag#26, ss_ticket_number#4, cnt#17] -(31) CometColumnarExchange +(31) CometExchange Input [6]: [c_last_name#25, c_first_name#24, c_salutation#23, c_preferred_cust_flag#26, ss_ticket_number#4, cnt#17] -Arguments: rangepartitioning(c_last_name#25 ASC NULLS FIRST, c_first_name#24 ASC NULLS FIRST, c_salutation#23 ASC NULLS FIRST, c_preferred_cust_flag#26 DESC NULLS LAST, ss_ticket_number#4 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] +Arguments: rangepartitioning(c_last_name#25 ASC NULLS FIRST, c_first_name#24 ASC NULLS FIRST, c_salutation#23 ASC NULLS FIRST, c_preferred_cust_flag#26 DESC NULLS LAST, ss_ticket_number#4 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] (32) CometSort Input [6]: [c_last_name#25, c_first_name#24, c_salutation#23, c_preferred_cust_flag#26, ss_ticket_number#4, cnt#17] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q34/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q34/simplified.txt index cdc4b42fb2..d57afd4fcf 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q34/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q34/simplified.txt @@ -2,7 +2,7 @@ WholeStageCodegen (1) CometColumnarToRow InputAdapter CometSort [c_last_name,c_first_name,c_salutation,c_preferred_cust_flag,ss_ticket_number,cnt] - CometColumnarExchange [c_last_name,c_first_name,c_salutation,c_preferred_cust_flag,ss_ticket_number] #1 + CometExchange [c_last_name,c_first_name,c_salutation,c_preferred_cust_flag,ss_ticket_number] #1 CometProject [c_last_name,c_first_name,c_salutation,c_preferred_cust_flag,ss_ticket_number,cnt] CometBroadcastHashJoin [ss_ticket_number,ss_customer_sk,cnt,c_customer_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag] CometFilter [ss_ticket_number,ss_customer_sk,cnt] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q64/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q64/explain.txt index 1990c685ce..077daeca3a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q64/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q64/explain.txt @@ -1,7 +1,7 @@ == Physical Plan == * CometColumnarToRow (176) +- CometSort (175) - +- CometColumnarExchange (174) + +- CometExchange (174) +- CometProject (173) +- CometSortMergeJoin (172) :- CometSort (103) @@ -945,9 +945,9 @@ Arguments: [item_sk#82, store_name#83, store_zip#84], [item_sk#155, store_name#1 Input [25]: [product_name#81, item_sk#82, store_name#83, store_zip#84, b_street_number#85, b_streen_name#86, b_city#87, b_zip#88, c_street_number#89, c_street_name#90, c_city#91, c_zip#92, syear#93, cnt#94, s1#95, s2#96, s3#97, item_sk#155, store_name#156, store_zip#157, syear#158, cnt#159, s1#160, s2#161, s3#162] Arguments: [product_name#81, store_name#83, store_zip#84, b_street_number#85, b_streen_name#86, b_city#87, b_zip#88, c_street_number#89, c_street_name#90, c_city#91, c_zip#92, syear#93, cnt#94, s1#95, s2#96, s3#97, s1#160, s2#161, s3#162, syear#158, cnt#159], [product_name#81, store_name#83, store_zip#84, b_street_number#85, b_streen_name#86, b_city#87, b_zip#88, c_street_number#89, c_street_name#90, c_city#91, c_zip#92, syear#93, cnt#94, s1#95, s2#96, s3#97, s1#160, s2#161, s3#162, syear#158, cnt#159] -(174) CometColumnarExchange +(174) CometExchange Input [21]: [product_name#81, store_name#83, store_zip#84, b_street_number#85, b_streen_name#86, b_city#87, b_zip#88, c_street_number#89, c_street_name#90, c_city#91, c_zip#92, syear#93, cnt#94, s1#95, s2#96, s3#97, s1#160, s2#161, s3#162, syear#158, cnt#159] -Arguments: rangepartitioning(product_name#81 ASC NULLS FIRST, store_name#83 ASC NULLS FIRST, cnt#159 ASC NULLS FIRST, s1#95 ASC NULLS FIRST, s1#160 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=8] +Arguments: rangepartitioning(product_name#81 ASC NULLS FIRST, store_name#83 ASC NULLS FIRST, cnt#159 ASC NULLS FIRST, s1#95 ASC NULLS FIRST, s1#160 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=8] (175) CometSort Input [21]: [product_name#81, store_name#83, store_zip#84, b_street_number#85, b_streen_name#86, b_city#87, b_zip#88, c_street_number#89, c_street_name#90, c_city#91, c_zip#92, syear#93, cnt#94, s1#95, s2#96, s3#97, s1#160, s2#161, s3#162, syear#158, cnt#159] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q64/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q64/simplified.txt index 7cec32adee..46fe063430 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q64/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q64/simplified.txt @@ -2,7 +2,7 @@ WholeStageCodegen (1) CometColumnarToRow InputAdapter CometSort [product_name,store_name,store_zip,b_street_number,b_streen_name,b_city,b_zip,c_street_number,c_street_name,c_city,c_zip,syear,cnt,s1,s2,s3,s1,s2,s3,syear,cnt] - CometColumnarExchange [product_name,store_name,cnt,s1,s1] #1 + CometExchange [product_name,store_name,cnt,s1,s1] #1 CometProject [product_name,store_name,store_zip,b_street_number,b_streen_name,b_city,b_zip,c_street_number,c_street_name,c_city,c_zip,syear,cnt,s1,s2,s3,s1,s2,s3,syear,cnt] CometSortMergeJoin [product_name,item_sk,store_name,store_zip,b_street_number,b_streen_name,b_city,b_zip,c_street_number,c_street_name,c_city,c_zip,syear,cnt,s1,s2,s3,item_sk,store_name,store_zip,syear,cnt,s1,s2,s3] CometSort [product_name,item_sk,store_name,store_zip,b_street_number,b_streen_name,b_city,b_zip,c_street_number,c_street_name,c_city,c_zip,syear,cnt,s1,s2,s3] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q34/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q34/explain.txt index a526a8835c..94be39468a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q34/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q34/explain.txt @@ -1,7 +1,7 @@ == Physical Plan == * CometColumnarToRow (33) +- CometSort (32) - +- CometColumnarExchange (31) + +- CometExchange (31) +- CometProject (30) +- CometBroadcastHashJoin (29) :- CometFilter (24) @@ -176,9 +176,9 @@ Arguments: [ss_customer_sk#1], [c_customer_sk#18], Inner, BuildRight Input [8]: [ss_ticket_number#4, ss_customer_sk#1, cnt#17, c_customer_sk#18, c_salutation#23, c_first_name#24, c_last_name#25, c_preferred_cust_flag#26] Arguments: [c_last_name#25, c_first_name#24, c_salutation#23, c_preferred_cust_flag#26, ss_ticket_number#4, cnt#17], [c_last_name#25, c_first_name#24, c_salutation#23, c_preferred_cust_flag#26, ss_ticket_number#4, cnt#17] -(31) CometColumnarExchange +(31) CometExchange Input [6]: [c_last_name#25, c_first_name#24, c_salutation#23, c_preferred_cust_flag#26, ss_ticket_number#4, cnt#17] -Arguments: rangepartitioning(c_last_name#25 ASC NULLS FIRST, c_first_name#24 ASC NULLS FIRST, c_salutation#23 ASC NULLS FIRST, c_preferred_cust_flag#26 DESC NULLS LAST, ss_ticket_number#4 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] +Arguments: rangepartitioning(c_last_name#25 ASC NULLS FIRST, c_first_name#24 ASC NULLS FIRST, c_salutation#23 ASC NULLS FIRST, c_preferred_cust_flag#26 DESC NULLS LAST, ss_ticket_number#4 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] (32) CometSort Input [6]: [c_last_name#25, c_first_name#24, c_salutation#23, c_preferred_cust_flag#26, ss_ticket_number#4, cnt#17] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q34/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q34/simplified.txt index cdc4b42fb2..d57afd4fcf 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q34/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q34/simplified.txt @@ -2,7 +2,7 @@ WholeStageCodegen (1) CometColumnarToRow InputAdapter CometSort [c_last_name,c_first_name,c_salutation,c_preferred_cust_flag,ss_ticket_number,cnt] - CometColumnarExchange [c_last_name,c_first_name,c_salutation,c_preferred_cust_flag,ss_ticket_number] #1 + CometExchange [c_last_name,c_first_name,c_salutation,c_preferred_cust_flag,ss_ticket_number] #1 CometProject [c_last_name,c_first_name,c_salutation,c_preferred_cust_flag,ss_ticket_number,cnt] CometBroadcastHashJoin [ss_ticket_number,ss_customer_sk,cnt,c_customer_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag] CometFilter [ss_ticket_number,ss_customer_sk,cnt] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q64/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q64/explain.txt index c236c65822..e2784d3e5b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q64/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q64/explain.txt @@ -1,7 +1,7 @@ == Physical Plan == * CometColumnarToRow (185) +- CometSort (184) - +- CometColumnarExchange (183) + +- CometExchange (183) +- CometProject (182) +- CometSortMergeJoin (181) :- CometSort (112) @@ -1013,9 +1013,9 @@ Arguments: [item_sk#92, store_name#93, store_zip#94], [item_sk#178, store_name#1 Input [25]: [product_name#91, item_sk#92, store_name#93, store_zip#94, b_street_number#95, b_streen_name#96, b_city#97, b_zip#98, c_street_number#99, c_street_name#100, c_city#101, c_zip#102, syear#103, cnt#104, s1#105, s2#106, s3#107, item_sk#178, store_name#179, store_zip#180, syear#181, cnt#182, s1#183, s2#184, s3#185] Arguments: [product_name#91, store_name#93, store_zip#94, b_street_number#95, b_streen_name#96, b_city#97, b_zip#98, c_street_number#99, c_street_name#100, c_city#101, c_zip#102, syear#103, cnt#104, s1#105, s2#106, s3#107, s1#183, s2#184, s3#185, syear#181, cnt#182], [product_name#91, store_name#93, store_zip#94, b_street_number#95, b_streen_name#96, b_city#97, b_zip#98, c_street_number#99, c_street_name#100, c_city#101, c_zip#102, syear#103, cnt#104, s1#105, s2#106, s3#107, s1#183, s2#184, s3#185, syear#181, cnt#182] -(183) CometColumnarExchange +(183) CometExchange Input [21]: [product_name#91, store_name#93, store_zip#94, b_street_number#95, b_streen_name#96, b_city#97, b_zip#98, c_street_number#99, c_street_name#100, c_city#101, c_zip#102, syear#103, cnt#104, s1#105, s2#106, s3#107, s1#183, s2#184, s3#185, syear#181, cnt#182] -Arguments: rangepartitioning(product_name#91 ASC NULLS FIRST, store_name#93 ASC NULLS FIRST, cnt#182 ASC NULLS FIRST, s1#105 ASC NULLS FIRST, s1#183 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=17] +Arguments: rangepartitioning(product_name#91 ASC NULLS FIRST, store_name#93 ASC NULLS FIRST, cnt#182 ASC NULLS FIRST, s1#105 ASC NULLS FIRST, s1#183 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=17] (184) CometSort Input [21]: [product_name#91, store_name#93, store_zip#94, b_street_number#95, b_streen_name#96, b_city#97, b_zip#98, c_street_number#99, c_street_name#100, c_city#101, c_zip#102, syear#103, cnt#104, s1#105, s2#106, s3#107, s1#183, s2#184, s3#185, syear#181, cnt#182] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q64/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q64/simplified.txt index 23e49e9dd7..29748bfd47 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q64/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q64/simplified.txt @@ -2,7 +2,7 @@ WholeStageCodegen (37) CometColumnarToRow InputAdapter CometSort [product_name,store_name,store_zip,b_street_number,b_streen_name,b_city,b_zip,c_street_number,c_street_name,c_city,c_zip,syear,cnt,s1,s2,s3,s1,s2,s3,syear,cnt] - CometColumnarExchange [product_name,store_name,cnt,s1,s1] #1 + CometExchange [product_name,store_name,cnt,s1,s1] #1 CometProject [product_name,store_name,store_zip,b_street_number,b_streen_name,b_city,b_zip,c_street_number,c_street_name,c_city,c_zip,syear,cnt,s1,s2,s3,s1,s2,s3,syear,cnt] CometSortMergeJoin [product_name,item_sk,store_name,store_zip,b_street_number,b_streen_name,b_city,b_zip,c_street_number,c_street_name,c_city,c_zip,syear,cnt,s1,s2,s3,item_sk,store_name,store_zip,syear,cnt,s1,s2,s3] CometSort [product_name,item_sk,store_name,store_zip,b_street_number,b_streen_name,b_city,b_zip,c_street_number,c_street_name,c_city,c_zip,syear,cnt,s1,s2,s3] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q34/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q34/explain.txt index 692b84d605..50af5f4f75 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q34/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q34/explain.txt @@ -1,7 +1,7 @@ == Physical Plan == * CometColumnarToRow (33) +- CometSort (32) - +- CometColumnarExchange (31) + +- CometExchange (31) +- CometProject (30) +- CometBroadcastHashJoin (29) :- CometFilter (24) @@ -176,9 +176,9 @@ Arguments: [ss_customer_sk#1], [c_customer_sk#18], Inner, BuildRight Input [8]: [ss_ticket_number#4, ss_customer_sk#1, cnt#17, c_customer_sk#18, c_salutation#23, c_first_name#24, c_last_name#25, c_preferred_cust_flag#26] Arguments: [c_last_name#25, c_first_name#24, c_salutation#23, c_preferred_cust_flag#26, ss_ticket_number#4, cnt#17], [c_last_name#25, c_first_name#24, c_salutation#23, c_preferred_cust_flag#26, ss_ticket_number#4, cnt#17] -(31) CometColumnarExchange +(31) CometExchange Input [6]: [c_last_name#25, c_first_name#24, c_salutation#23, c_preferred_cust_flag#26, ss_ticket_number#4, cnt#17] -Arguments: rangepartitioning(c_last_name#25 ASC NULLS FIRST, c_first_name#24 ASC NULLS FIRST, c_salutation#23 ASC NULLS FIRST, c_preferred_cust_flag#26 DESC NULLS LAST, ss_ticket_number#4 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=2] +Arguments: rangepartitioning(c_last_name#25 ASC NULLS FIRST, c_first_name#24 ASC NULLS FIRST, c_salutation#23 ASC NULLS FIRST, c_preferred_cust_flag#26 DESC NULLS LAST, ss_ticket_number#4 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] (32) CometSort Input [6]: [c_last_name#25, c_first_name#24, c_salutation#23, c_preferred_cust_flag#26, ss_ticket_number#4, cnt#17] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q34/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q34/simplified.txt index cdc4b42fb2..d57afd4fcf 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q34/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q34/simplified.txt @@ -2,7 +2,7 @@ WholeStageCodegen (1) CometColumnarToRow InputAdapter CometSort [c_last_name,c_first_name,c_salutation,c_preferred_cust_flag,ss_ticket_number,cnt] - CometColumnarExchange [c_last_name,c_first_name,c_salutation,c_preferred_cust_flag,ss_ticket_number] #1 + CometExchange [c_last_name,c_first_name,c_salutation,c_preferred_cust_flag,ss_ticket_number] #1 CometProject [c_last_name,c_first_name,c_salutation,c_preferred_cust_flag,ss_ticket_number,cnt] CometBroadcastHashJoin [ss_ticket_number,ss_customer_sk,cnt,c_customer_sk,c_salutation,c_first_name,c_last_name,c_preferred_cust_flag] CometFilter [ss_ticket_number,ss_customer_sk,cnt] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q64/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q64/explain.txt index 1990c685ce..077daeca3a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q64/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q64/explain.txt @@ -1,7 +1,7 @@ == Physical Plan == * CometColumnarToRow (176) +- CometSort (175) - +- CometColumnarExchange (174) + +- CometExchange (174) +- CometProject (173) +- CometSortMergeJoin (172) :- CometSort (103) @@ -945,9 +945,9 @@ Arguments: [item_sk#82, store_name#83, store_zip#84], [item_sk#155, store_name#1 Input [25]: [product_name#81, item_sk#82, store_name#83, store_zip#84, b_street_number#85, b_streen_name#86, b_city#87, b_zip#88, c_street_number#89, c_street_name#90, c_city#91, c_zip#92, syear#93, cnt#94, s1#95, s2#96, s3#97, item_sk#155, store_name#156, store_zip#157, syear#158, cnt#159, s1#160, s2#161, s3#162] Arguments: [product_name#81, store_name#83, store_zip#84, b_street_number#85, b_streen_name#86, b_city#87, b_zip#88, c_street_number#89, c_street_name#90, c_city#91, c_zip#92, syear#93, cnt#94, s1#95, s2#96, s3#97, s1#160, s2#161, s3#162, syear#158, cnt#159], [product_name#81, store_name#83, store_zip#84, b_street_number#85, b_streen_name#86, b_city#87, b_zip#88, c_street_number#89, c_street_name#90, c_city#91, c_zip#92, syear#93, cnt#94, s1#95, s2#96, s3#97, s1#160, s2#161, s3#162, syear#158, cnt#159] -(174) CometColumnarExchange +(174) CometExchange Input [21]: [product_name#81, store_name#83, store_zip#84, b_street_number#85, b_streen_name#86, b_city#87, b_zip#88, c_street_number#89, c_street_name#90, c_city#91, c_zip#92, syear#93, cnt#94, s1#95, s2#96, s3#97, s1#160, s2#161, s3#162, syear#158, cnt#159] -Arguments: rangepartitioning(product_name#81 ASC NULLS FIRST, store_name#83 ASC NULLS FIRST, cnt#159 ASC NULLS FIRST, s1#95 ASC NULLS FIRST, s1#160 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=8] +Arguments: rangepartitioning(product_name#81 ASC NULLS FIRST, store_name#83 ASC NULLS FIRST, cnt#159 ASC NULLS FIRST, s1#95 ASC NULLS FIRST, s1#160 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=8] (175) CometSort Input [21]: [product_name#81, store_name#83, store_zip#84, b_street_number#85, b_streen_name#86, b_city#87, b_zip#88, c_street_number#89, c_street_name#90, c_city#91, c_zip#92, syear#93, cnt#94, s1#95, s2#96, s3#97, s1#160, s2#161, s3#162, syear#158, cnt#159] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q64/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q64/simplified.txt index 7cec32adee..46fe063430 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q64/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q64/simplified.txt @@ -2,7 +2,7 @@ WholeStageCodegen (1) CometColumnarToRow InputAdapter CometSort [product_name,store_name,store_zip,b_street_number,b_streen_name,b_city,b_zip,c_street_number,c_street_name,c_city,c_zip,syear,cnt,s1,s2,s3,s1,s2,s3,syear,cnt] - CometColumnarExchange [product_name,store_name,cnt,s1,s1] #1 + CometExchange [product_name,store_name,cnt,s1,s1] #1 CometProject [product_name,store_name,store_zip,b_street_number,b_streen_name,b_city,b_zip,c_street_number,c_street_name,c_city,c_zip,syear,cnt,s1,s2,s3,s1,s2,s3,syear,cnt] CometSortMergeJoin [product_name,item_sk,store_name,store_zip,b_street_number,b_streen_name,b_city,b_zip,c_street_number,c_street_name,c_city,c_zip,syear,cnt,s1,s2,s3,item_sk,store_name,store_zip,syear,cnt,s1,s2,s3] CometSort [product_name,item_sk,store_name,store_zip,b_street_number,b_streen_name,b_city,b_zip,c_street_number,c_street_name,c_city,c_zip,syear,cnt,s1,s2,s3] diff --git a/spark/src/test/scala/org/apache/comet/CometFuzzTestSuite.scala b/spark/src/test/scala/org/apache/comet/CometFuzzTestSuite.scala index 1b65dd8d44..398d960136 100644 --- a/spark/src/test/scala/org/apache/comet/CometFuzzTestSuite.scala +++ b/spark/src/test/scala/org/apache/comet/CometFuzzTestSuite.scala @@ -133,6 +133,21 @@ class CometFuzzTestSuite extends CometFuzzTestBase { } } + test("order by random columns") { + val df = spark.read.parquet(filename) + df.createOrReplaceTempView("t1") + + for (_ <- 1 to 10) { + // We only do order by permutations of primitive types to exercise native shuffle's + // RangePartitioning which only supports those types. + val shuffledPrimitiveCols = Random.shuffle(df.columns.slice(0, 14).toList) + val randomSize = Random.nextInt(shuffledPrimitiveCols.length) + 1 + val randomColsSubset = shuffledPrimitiveCols.take(randomSize).toArray.mkString(",") + val sql = s"SELECT $randomColsSubset FROM t1 ORDER BY $randomColsSubset" + checkSparkAnswerAndOperator(sql) + } + } + test("distribute by single column (complex types)") { val df = spark.read.parquet(filename) df.createOrReplaceTempView("t1") @@ -168,7 +183,19 @@ class CometFuzzTestSuite extends CometFuzzTestBase { df2.collect() if (usingDataSourceExec) { val cometShuffles = collectCometShuffleExchanges(df2.queryExecution.executedPlan) - assert(1 == cometShuffles.length) + val expectedNumCometShuffles = CometConf.COMET_NATIVE_SCAN_IMPL.get() match { + case CometConf.SCAN_NATIVE_COMET => + // native_comet does not support reading complex types + 0 + case CometConf.SCAN_NATIVE_ICEBERG_COMPAT | CometConf.SCAN_NATIVE_DATAFUSION => + CometConf.COMET_SHUFFLE_MODE.get() match { + case "jvm" => + 1 + case "native" => + 2 + } + } + assert(cometShuffles.length == expectedNumCometShuffles) } } diff --git a/spark/src/test/scala/org/apache/comet/exec/CometNativeShuffleSuite.scala b/spark/src/test/scala/org/apache/comet/exec/CometNativeShuffleSuite.scala index 5e2abd7c33..a682ff91a5 100644 --- a/spark/src/test/scala/org/apache/comet/exec/CometNativeShuffleSuite.scala +++ b/spark/src/test/scala/org/apache/comet/exec/CometNativeShuffleSuite.scala @@ -20,13 +20,14 @@ package org.apache.comet.exec import scala.concurrent.duration.DurationInt +import scala.util.Random import org.scalactic.source.Position import org.scalatest.Tag import org.apache.hadoop.fs.Path import org.apache.spark.SparkEnv -import org.apache.spark.sql.{CometTestBase, DataFrame} +import org.apache.spark.sql.{CometTestBase, DataFrame, Dataset, Row} import org.apache.spark.sql.comet.execution.shuffle.CometShuffleExchangeExec import org.apache.spark.sql.execution.adaptive.AdaptiveSparkPlanHelper import org.apache.spark.sql.functions.col @@ -248,6 +249,129 @@ class CometNativeShuffleSuite extends CometTestBase with AdaptiveSparkPlanHelper } } + // This duplicates behavior seen in a much more complicated Spark SQL test + // "SPARK-44647: test join key is the second cluster key" + test("range partitioning with duplicate column references") { + // Test with wider schema and non-adjacent duplicate columns + withParquetTable( + (0 until 100).map(i => (i % 10, (i % 5).toByte, i % 3, i % 7, (i % 4).toShort, i.toString)), + "tbl") { + + val df = sql("SELECT * FROM tbl") + + // Test case 1: Adjacent duplicates (original case) + val rangePartitioned1 = df.repartitionByRange(3, df.col("_1"), df.col("_1"), df.col("_2")) + checkShuffleAnswer(rangePartitioned1, 1) + + // Test case 2: Non-adjacent duplicates in wider schema + // Duplicate _1 at positions 0 and 3, with different columns in between + val rangePartitioned2 = + df.repartitionByRange(4, df.col("_1"), df.col("_3"), df.col("_5"), df.col("_1")) + checkShuffleAnswer(rangePartitioned2, 1) + + // Test case 3: Multiple duplicate pairs + // _1 duplicated at positions 0,2 and _4 duplicated at positions 1,3 + val rangePartitioned3 = + df.repartitionByRange(4, df.col("_1"), df.col("_4"), df.col("_1"), df.col("_4")) + checkShuffleAnswer(rangePartitioned3, 1) + + // Test case 4: Triple duplicates with gaps + val rangePartitioned4 = df.repartitionByRange( + 5, + df.col("_1"), + df.col("_2"), + df.col("_1"), + df.col("_3"), + df.col("_1")) + checkShuffleAnswer(rangePartitioned4, 1) + } + } + + // Asserts ordering properties of partitions in a Dataset that has been RangePartitioned + private def checkRangePartitionedDataset(df_range_partitioned: Dataset[Row]): Unit = { + val partition_bounds = df_range_partitioned.rdd + .mapPartitionsWithIndex((idx: Int, iterator: Iterator[Row]) => { + // Find the min and max value in each partition + var min: Option[Int] = None + var max: Option[Int] = None + iterator.foreach((row: Row) => { + val row_val = row.get(0).asInstanceOf[Int] + if (min.isEmpty || row_val < min.get) { + min = Some(row_val) + } + if (max.isEmpty || row_val > max.get) { + max = Some(row_val) + } + }) + Iterator.single((idx, min, max)) + }) + .collect() + + // Check min and max values in each partition + for (i <- partition_bounds.indices.init) { + val currentPartition = partition_bounds(i) + val nextPartition = partition_bounds(i + 1) + + if (currentPartition._2.isDefined && currentPartition._3.isDefined) { + val currentMin = currentPartition._2.get + val currentMax = currentPartition._3.get + assert(currentMin <= currentMax) + } + + if (currentPartition._3.isDefined && nextPartition._2.isDefined) { + val currentMax = currentPartition._3.get + val nextMin = nextPartition._2.get + assert(currentMax < nextMin) + } + } + } + + // This adapts the PySpark example in https://github.com/apache/datafusion-comet/issues/1906 to + // test for incorrect partition values after native RangePartitioning + test("fix: range partitioning #1906") { + withSQLConf(CometConf.COMET_EXEC_SHUFFLE_WITH_RANGE_PARTITIONING_ENABLED.key -> "true") { + withParquetTable((0 until 100000).map(i => (i, i + 1)), "tbl") { + val df = sql("SELECT * from tbl") + + // Repartition with two sort columns + val repartitioned_df = df.repartitionByRange(10, df.col("_1")) + checkSparkAnswerAndOperator(repartitioned_df) + checkRangePartitionedDataset(repartitioned_df) + } + } + } + + // This adapts the PySpark example in https://github.com/apache/datafusion-comet/issues/1906 to + // test for incorrect partition values after native RangePartitioning + test("fix: range partitioning #1906, two columns") { + withSQLConf(CometConf.COMET_EXEC_SHUFFLE_WITH_RANGE_PARTITIONING_ENABLED.key -> "true") { + withParquetTable((0 until 100000).map(i => (i, i + 1)), "tbl") { + val df = sql("SELECT * from tbl") + + // Repartition with two sort columns + val repartitioned_df = df.repartitionByRange(10, df.col("_1"), df.col("_2")) + checkSparkAnswerAndOperator(repartitioned_df) + checkRangePartitionedDataset(repartitioned_df) + } + } + } + + // This adapts the PySpark example in https://github.com/apache/datafusion-comet/issues/1906 to + // test for incorrect partition values after native RangePartitioning + test("fix: range partitioning #1906, random sort column with duplicates") { + withSQLConf(CometConf.COMET_EXEC_SHUFFLE_WITH_RANGE_PARTITIONING_ENABLED.key -> "true") { + val random = new Random(42) + withParquetTable((0 until 100000).map(i => (random.nextInt(10000), i)), "tbl") { + val df = sql("SELECT * from tbl") + + // Repartition with two sort columns + val repartitioned_df = df.repartitionByRange(10, df.col("_1")) + checkSparkAnswerAndOperator(repartitioned_df) + checkRangePartitionedDataset(repartitioned_df) + } + } + } + /** * Checks that `df` produces the same answer as Spark does, and has the `expectedNum` Comet * exchange operators. When `checkNativeOperators` is true, this also checks that all operators diff --git a/spark/src/test/scala/org/apache/spark/sql/benchmark/CometShuffleBenchmark.scala b/spark/src/test/scala/org/apache/spark/sql/benchmark/CometShuffleBenchmark.scala index 46af7115cc..103866331c 100644 --- a/spark/src/test/scala/org/apache/spark/sql/benchmark/CometShuffleBenchmark.scala +++ b/spark/src/test/scala/org/apache/spark/sql/benchmark/CometShuffleBenchmark.scala @@ -421,6 +421,76 @@ object CometShuffleBenchmark extends CometBenchmarkBase { } } + def shuffleRangePartitionBenchmark( + values: Int, + dataType: DataType, + width: Int, + partitionNum: Int): Unit = { + val benchmark = + new Benchmark( + s"SQL Wide ($width cols) ${dataType.sql} Range Partition Shuffle($partitionNum Partition)", + values, + output = output) + + val projection = (1 to width) + .map(i => s"CAST(CAST(RAND(1) * 100 AS INTEGER) AS ${dataType.sql}) AS c$i") + .mkString(", ") + val columns = (1 to width).map(i => s"c$i").mkString(", ") + + withTempPath { dir => + withTempTable("parquetV1Table") { + prepareTable(dir, spark.sql(s"SELECT $projection FROM $tbl")) + + benchmark.addCase("SQL Parquet - Spark") { _ => + spark + .sql(s"select $columns from parquetV1Table") + .repartitionByRange(partitionNum, Column("c1")) + .noop() + } + + benchmark.addCase("SQL Parquet - Comet (Spark Shuffle)") { _ => + withSQLConf( + CometConf.COMET_ENABLED.key -> "true", + CometConf.COMET_EXEC_ENABLED.key -> "true", + CometConf.COMET_EXEC_SHUFFLE_ENABLED.key -> "false") { + spark + .sql(s"select $columns from parquetV1Table") + .repartitionByRange(partitionNum, Column("c1")) + .noop() + } + } + + benchmark.addCase("SQL Parquet - Comet (Comet Arrow Shuffle)") { _ => + withSQLConf( + CometConf.COMET_ENABLED.key -> "true", + CometConf.COMET_EXEC_ENABLED.key -> "true", + CometConf.COMET_EXEC_SHUFFLE_ENABLED.key -> "true", + CometConf.COMET_SHUFFLE_MODE.key -> "jvm") { + spark + .sql(s"select $columns from parquetV1Table") + .repartitionByRange(partitionNum, Column("c1")) + .noop() + } + } + + benchmark.addCase("SQL Parquet - Comet (Comet Shuffle)") { _ => + withSQLConf( + CometConf.COMET_ENABLED.key -> "true", + CometConf.COMET_EXEC_ENABLED.key -> "true", + CometConf.COMET_EXEC_SHUFFLE_ENABLED.key -> "true", + CometConf.COMET_SHUFFLE_MODE.key -> "native") { + spark + .sql(s"select $columns from parquetV1Table") + .repartitionByRange(partitionNum, Column("c1")) + .noop() + } + } + + benchmark.run() + } + } + } + override def runCometBenchmark(mainArgs: Array[String]): Unit = { runBenchmarkWithTable("Shuffle on array", 1024 * 1024 * 1) { v => Seq( @@ -591,5 +661,69 @@ object CometShuffleBenchmark extends CometBenchmarkBase { shuffleWideBenchmark(v, dataType, 20, 201) } } + + runBenchmarkWithTable("Wide Range Partition Shuffle (10 cols)", 1024 * 1024 * 10) { v => + Seq( + BooleanType, + ByteType, + ShortType, + IntegerType, + LongType, + FloatType, + DoubleType, + StringType, + DecimalType(10, 0)) + .foreach { dataType => + shuffleRangePartitionBenchmark(v, dataType, 10, 5) + } + } + + runBenchmarkWithTable("Wide Range Partition Shuffle (20 cols)", 1024 * 1024 * 10) { v => + Seq( + BooleanType, + ByteType, + ShortType, + IntegerType, + LongType, + FloatType, + DoubleType, + StringType, + DecimalType(10, 0)) + .foreach { dataType => + shuffleRangePartitionBenchmark(v, dataType, 20, 5) + } + } + + runBenchmarkWithTable("Wide Range Partition Shuffle (10 cols)", 1024 * 1024 * 10) { v => + Seq( + BooleanType, + ByteType, + ShortType, + IntegerType, + LongType, + FloatType, + DoubleType, + StringType, + DecimalType(10, 0)) + .foreach { dataType => + shuffleRangePartitionBenchmark(v, dataType, 10, 201) + } + } + + runBenchmarkWithTable("Wide Range Partition Shuffle (20 cols)", 1024 * 1024 * 10) { v => + Seq( + BooleanType, + ByteType, + ShortType, + IntegerType, + LongType, + FloatType, + DoubleType, + StringType, + DecimalType(10, 0)) + .foreach { dataType => + shuffleRangePartitionBenchmark(v, dataType, 20, 201) + } + } } }