From 93d77eef56496a90c59646508ad14c0379d251ef Mon Sep 17 00:00:00 2001 From: Matt Butrovich Date: Thu, 5 Jun 2025 11:06:28 -0400 Subject: [PATCH 01/72] determine_bounds for f64. --- native/core/src/execution/shuffle/mod.rs | 1 + .../execution/shuffle/range_partitioner.rs | 83 +++++++++++++++++++ 2 files changed, 84 insertions(+) create mode 100644 native/core/src/execution/shuffle/range_partitioner.rs diff --git a/native/core/src/execution/shuffle/mod.rs b/native/core/src/execution/shuffle/mod.rs index 4864ffc062..fdecd6a066 100644 --- a/native/core/src/execution/shuffle/mod.rs +++ b/native/core/src/execution/shuffle/mod.rs @@ -20,6 +20,7 @@ mod list; mod map; pub mod row; mod shuffle_writer; +mod range_partitioner; pub use codec::{read_ipc_compressed, CompressionCodec, ShuffleBlockWriter}; pub use shuffle_writer::ShuffleWriterExec; diff --git a/native/core/src/execution/shuffle/range_partitioner.rs b/native/core/src/execution/shuffle/range_partitioner.rs new file mode 100644 index 0000000000..dbc5934e55 --- /dev/null +++ b/native/core/src/execution/shuffle/range_partitioner.rs @@ -0,0 +1,83 @@ +use arrow::array::RecordBatch; +use num::ToPrimitive; + +struct RangePartitioner {} + +impl RangePartitioner { + // Adapted from org.apache.spark.RangePartitioner.sketch + fn sketch(input: RecordBatch, sample_size_per_partition: i32) {} + + // Adapted from org.apache.spark.RangePartitioner.determineBounds + fn determine_bounds(mut candidates: Vec<(f64, f64)>, partitions: i32) -> Vec { + candidates.sort_by(|a, b| a.0.partial_cmp(&b.0).unwrap()); + println!("{:?}", candidates); + let num_candidates = candidates.len(); + let sum_weights = candidates + .iter() + .map(|x| x.1.to_f64().unwrap()) + .sum::(); + let step = sum_weights / partitions as f64; + let mut cumulative_weights = 0.0; + let mut target = step; + let mut bounds = Vec::with_capacity((partitions - 1) as usize); + let mut i = 0; + let mut j = 0; + let mut previous_bound: Option = None; + while (i < num_candidates) && (j < partitions - 1) { + let (key, weight) = candidates[i]; + cumulative_weights += weight; + if cumulative_weights >= target { + // Skip duplicate values. + if previous_bound.is_none() || key > previous_bound.unwrap() { + bounds.push(key); + target += step; + j += 1; + previous_bound = Some(key) + } + } + i += 1 + } + bounds + } +} + +#[cfg(test)] +mod test { + use super::*; + use arrow::array::Int32Array; + use arrow::datatypes::{DataType, Field, Schema}; + use std::sync::Arc; + + #[test] + // org.apache.spark.PartitioningSuite + // "RangPartitioner.sketch" + fn sketch_test() { + let batch = create_batch(30); + println!("{:?}", batch); + RangePartitioner::sketch(batch, 30); + } + + #[test] + // org.apache.spark.PartitioningSuite + // "RangePartitioner.determineBounds" + fn determine_bounds_test() { + let candidates = vec![ + (0.7, 2.0), + (0.1, 1.0), + (0.4, 1.0), + (0.3, 1.0), + (0.2, 1.0), + (0.5, 1.0), + (1.0, 3.0), + ]; + let result = RangePartitioner::determine_bounds(candidates, 3); + assert_eq!(vec![0.4, 0.7], result); + } + + fn create_batch(batch_size: i32) -> RecordBatch { + let column: Vec = (0..batch_size).collect(); + let array = Int32Array::from(column); + let schema = Arc::new(Schema::new(vec![Field::new("a", DataType::Int32, true)])); + RecordBatch::try_new(Arc::clone(&schema), vec![Arc::new(array)]).unwrap() + } +} From 29fa8cc515ec2bebbd7c9015d928a0fefd6339a9 Mon Sep 17 00:00:00 2001 From: Matt Butrovich Date: Thu, 5 Jun 2025 11:14:32 -0400 Subject: [PATCH 02/72] Generic determine_bounds. --- .../src/execution/shuffle/range_partitioner.rs | 16 +++++++++------- 1 file changed, 9 insertions(+), 7 deletions(-) diff --git a/native/core/src/execution/shuffle/range_partitioner.rs b/native/core/src/execution/shuffle/range_partitioner.rs index dbc5934e55..1198c4474c 100644 --- a/native/core/src/execution/shuffle/range_partitioner.rs +++ b/native/core/src/execution/shuffle/range_partitioner.rs @@ -8,9 +8,11 @@ impl RangePartitioner { fn sketch(input: RecordBatch, sample_size_per_partition: i32) {} // Adapted from org.apache.spark.RangePartitioner.determineBounds - fn determine_bounds(mut candidates: Vec<(f64, f64)>, partitions: i32) -> Vec { + fn determine_bounds(mut candidates: Vec<(T, f64)>, partitions: i32) -> Vec + where + T: PartialOrd + Copy, + { candidates.sort_by(|a, b| a.0.partial_cmp(&b.0).unwrap()); - println!("{:?}", candidates); let num_candidates = candidates.len(); let sum_weights = candidates .iter() @@ -22,17 +24,17 @@ impl RangePartitioner { let mut bounds = Vec::with_capacity((partitions - 1) as usize); let mut i = 0; let mut j = 0; - let mut previous_bound: Option = None; + let mut previous_bound: Option = None; while (i < num_candidates) && (j < partitions - 1) { - let (key, weight) = candidates[i]; + let (key, weight) = &candidates[i]; cumulative_weights += weight; if cumulative_weights >= target { // Skip duplicate values. - if previous_bound.is_none() || key > previous_bound.unwrap() { - bounds.push(key); + if previous_bound.is_none() || *key > previous_bound.unwrap() { + bounds.push(*key); target += step; j += 1; - previous_bound = Some(key) + previous_bound = Some(*key) } } i += 1 From 0871dab8b2176ec1441598e6c09fd6566663b7ea Mon Sep 17 00:00:00 2001 From: Matt Butrovich Date: Thu, 5 Jun 2025 14:47:40 -0400 Subject: [PATCH 03/72] reservoir_sample. --- native/core/src/execution/shuffle/mod.rs | 2 +- .../execution/shuffle/range_partitioner.rs | 101 +++++++++++++++--- 2 files changed, 85 insertions(+), 18 deletions(-) diff --git a/native/core/src/execution/shuffle/mod.rs b/native/core/src/execution/shuffle/mod.rs index fdecd6a066..a980482f7d 100644 --- a/native/core/src/execution/shuffle/mod.rs +++ b/native/core/src/execution/shuffle/mod.rs @@ -18,9 +18,9 @@ pub(crate) mod codec; mod list; mod map; +mod range_partitioner; pub mod row; mod shuffle_writer; -mod range_partitioner; pub use codec::{read_ipc_compressed, CompressionCodec, ShuffleBlockWriter}; pub use shuffle_writer::ShuffleWriterExec; diff --git a/native/core/src/execution/shuffle/range_partitioner.rs b/native/core/src/execution/shuffle/range_partitioner.rs index 1198c4474c..d60dfcde88 100644 --- a/native/core/src/execution/shuffle/range_partitioner.rs +++ b/native/core/src/execution/shuffle/range_partitioner.rs @@ -1,11 +1,56 @@ -use arrow::array::RecordBatch; +use arrow::array::{ArrayRef, UInt64Array}; +use arrow::compute::{take, TakeOptions}; use num::ToPrimitive; +use rand::Rng; +use std::sync::Arc; -struct RangePartitioner {} +struct RangePartitioner; impl RangePartitioner { - // Adapted from org.apache.spark.RangePartitioner.sketch - fn sketch(input: RecordBatch, sample_size_per_partition: i32) {} + // Adapted from https://en.wikipedia.org/wiki/Reservoir_sampling#Optimal:_Algorithm_L + // We use sample_size instead of k and input_length instead of n. + // We use indices in the reservoir instead of actual values since we'll do one take() on the + // input array at the end. + fn reservoir_sample(input: ArrayRef, sample_size: usize) -> ArrayRef { + let input_length = input.len(); + + if input_length <= sample_size { + // Just return the original input since we can't create a bigger sample. + return Arc::clone(&input); + } + + // Initialize our reservoir with indices of the first |sample_size| elements. + let mut reservoir: Vec = (0..sample_size as u64).collect(); + + let mut rng = rand::rng(); + let mut w = (rng.random::().ln() / sample_size as f64).exp(); + let mut i = sample_size; + + while i < input_length { + i += (rng.random::().ln() / (1.0 - w).ln()).floor() as usize; + + if i < input_length { + // 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(); + } + } + + // Build our indices array and then take the values from the input. + let indices = UInt64Array::from(reservoir); + + ArrayRef::from( + take( + &*input, + &indices, + Some(TakeOptions { + check_bounds: false, + }), + ) + .unwrap(), + ) + } // Adapted from org.apache.spark.RangePartitioner.determineBounds fn determine_bounds(mut candidates: Vec<(T, f64)>, partitions: i32) -> Vec @@ -46,23 +91,44 @@ impl RangePartitioner { #[cfg(test)] mod test { use super::*; - use arrow::array::Int32Array; - use arrow::datatypes::{DataType, Field, Schema}; + use arrow::array::{AsArray, Float64Array, Int32Array, RecordBatch}; + use arrow::datatypes::{DataType, Field, Int32Type, Schema}; use std::sync::Arc; #[test] - // org.apache.spark.PartitioningSuite - // "RangPartitioner.sketch" - fn sketch_test() { - let batch = create_batch(30); - println!("{:?}", batch); - RangePartitioner::sketch(batch, 30); + // org.apache.spark.util.random.SamplingUtilsSuite + // "reservoirSampleAndCount" + fn reservoir_sample() { + let batch = create_random_batch(100); + let column = batch.column(0).clone(); + let sample1 = RangePartitioner::reservoir_sample(column.clone(), 150); + assert_eq!(column, sample1.into()); + let sample2 = RangePartitioner::reservoir_sample(column.clone(), 100); + assert_eq!(column, sample2.into()); + let sample3 = RangePartitioner::reservoir_sample(column.clone(), 10); + assert_eq!(sample3.len(), 10); + } + + #[test] + // org.apache.spark.util.random.SamplingUtilsSuite + // "SPARK-18678 reservoirSampleAndCount with tiny input" + fn reservoir_sample_and_count_with_tiny_input() { + let column = vec![0, 1]; + let array = Arc::new(Int32Array::from(column)); + let mut counts: Vec = vec![0; array.len()]; + for _i in 0..500 { + let result = RangePartitioner::reservoir_sample(array.clone(), 1); + assert_eq!(result.len(), 1); + counts[result.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_test() { + fn determine_bounds() { let candidates = vec![ (0.7, 2.0), (0.1, 1.0), @@ -76,10 +142,11 @@ mod test { assert_eq!(vec![0.4, 0.7], result); } - fn create_batch(batch_size: i32) -> RecordBatch { - let column: Vec = (0..batch_size).collect(); - let array = Int32Array::from(column); - let schema = Arc::new(Schema::new(vec![Field::new("a", DataType::Int32, true)])); + fn create_random_batch(batch_size: i32) -> RecordBatch { + let mut rng = rand::rng(); + let column: Vec = (0..batch_size).map(|_| rng.random::()).collect(); + let array = Float64Array::from(column); + let schema = Arc::new(Schema::new(vec![Field::new("a", DataType::Float64, true)])); RecordBatch::try_new(Arc::clone(&schema), vec![Arc::new(array)]).unwrap() } } From ff18a391ea82d8f6a81aac58b9be2b36b86e6d95 Mon Sep 17 00:00:00 2001 From: Matt Butrovich Date: Fri, 6 Jun 2025 10:17:22 -0400 Subject: [PATCH 04/72] Change reservoir_sample to return a RecordBatch since that makes arbitrary sorting easier. --- .../execution/shuffle/range_partitioner.rs | 76 +++++++++++++------ 1 file changed, 52 insertions(+), 24 deletions(-) diff --git a/native/core/src/execution/shuffle/range_partitioner.rs b/native/core/src/execution/shuffle/range_partitioner.rs index d60dfcde88..f96ef3c7d9 100644 --- a/native/core/src/execution/shuffle/range_partitioner.rs +++ b/native/core/src/execution/shuffle/range_partitioner.rs @@ -1,5 +1,8 @@ -use arrow::array::{ArrayRef, UInt64Array}; +use arrow::array::{RecordBatch, UInt64Array}; use arrow::compute::{take, TakeOptions}; +use datafusion::physical_expr::expressions::col; +use datafusion::physical_expr::{LexOrdering, PhysicalSortExpr}; +use datafusion::physical_plan::sorts::sort::sort_batch; use num::ToPrimitive; use rand::Rng; use std::sync::Arc; @@ -11,12 +14,12 @@ impl RangePartitioner { // We use sample_size instead of k and input_length instead of n. // We use indices in the reservoir instead of actual values since we'll do one take() on the // input array at the end. - fn reservoir_sample(input: ArrayRef, sample_size: usize) -> ArrayRef { - let input_length = input.len(); + fn reservoir_sample(input: RecordBatch, sample_size: usize) -> RecordBatch { + let input_length = input.num_rows(); if input_length <= sample_size { // Just return the original input since we can't create a bigger sample. - return Arc::clone(&input); + return input.clone(); } // Initialize our reservoir with indices of the first |sample_size| elements. @@ -40,16 +43,24 @@ impl RangePartitioner { // Build our indices array and then take the values from the input. let indices = UInt64Array::from(reservoir); - ArrayRef::from( - take( - &*input, - &indices, - Some(TakeOptions { - check_bounds: false, - }), - ) - .unwrap(), - ) + // let result = take_record_batch(&input, &indices).unwrap(); + + // We don't use take_record_batch because it needlessly bounds checks + let columns = input + .columns() + .iter() + .map(|c| { + take( + c, + &indices, + Some(TakeOptions { + check_bounds: false, + }), + ) + }) + .collect::, _>>() + .unwrap(); + RecordBatch::try_new(input.schema(), columns).unwrap() } // Adapted from org.apache.spark.RangePartitioner.determineBounds @@ -93,20 +104,35 @@ mod test { use super::*; use arrow::array::{AsArray, Float64Array, Int32Array, RecordBatch}; use arrow::datatypes::{DataType, Field, Int32Type, Schema}; + use datafusion::physical_expr::expressions::col; + use datafusion::physical_expr::{LexOrdering, PhysicalSortExpr}; + use datafusion::physical_plan::sorts::sort::sort_batch; use std::sync::Arc; + #[test] + fn test_sort_record_batch() { + let batch = create_random_batch(10000); + let sample_size = 10; + let lex_ordering = LexOrdering::new(vec![PhysicalSortExpr::new_default( + col("a", batch.schema().as_ref()).unwrap(), + )]); + let batch = sort_batch(&batch, &lex_ordering, None).unwrap(); + let sample = RangePartitioner::reservoir_sample(batch, sample_size); + let sorted_sample = sort_batch(&sample, &lex_ordering, None).unwrap(); + println!("sorted_sample: {:?}", sorted_sample); + } + #[test] // org.apache.spark.util.random.SamplingUtilsSuite // "reservoirSampleAndCount" fn reservoir_sample() { let batch = create_random_batch(100); - let column = batch.column(0).clone(); - let sample1 = RangePartitioner::reservoir_sample(column.clone(), 150); - assert_eq!(column, sample1.into()); - let sample2 = RangePartitioner::reservoir_sample(column.clone(), 100); - assert_eq!(column, sample2.into()); - let sample3 = RangePartitioner::reservoir_sample(column.clone(), 10); - assert_eq!(sample3.len(), 10); + let sample1 = RangePartitioner::reservoir_sample(batch.clone(), 150); + assert_eq!(batch, sample1.into()); + let sample2 = RangePartitioner::reservoir_sample(batch.clone(), 100); + assert_eq!(batch, sample2.into()); + let sample3 = RangePartitioner::reservoir_sample(batch.clone(), 10); + assert_eq!(sample3.num_rows(), 10); } #[test] @@ -115,11 +141,13 @@ mod test { fn reservoir_sample_and_count_with_tiny_input() { let column = vec![0, 1]; let array = Arc::new(Int32Array::from(column)); + let schema = Arc::new(Schema::new(vec![Field::new("a", DataType::Int32, true)])); + let batch = RecordBatch::try_new(Arc::clone(&schema), vec![array.clone()]).unwrap(); let mut counts: Vec = vec![0; array.len()]; for _i in 0..500 { - let result = RangePartitioner::reservoir_sample(array.clone(), 1); - assert_eq!(result.len(), 1); - counts[result.as_primitive::().value(0) as usize] += 1; + let result = RangePartitioner::reservoir_sample(batch.clone(), 1); + 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) From 784b416f697df41a3b6329119866a25c1db92a1a Mon Sep 17 00:00:00 2001 From: Matt Butrovich Date: Fri, 6 Jun 2025 13:11:38 -0400 Subject: [PATCH 05/72] Checkpoint with plan serialization. --- native/core/src/execution/planner.rs | 10 +++++++ native/proto/src/proto/partitioning.proto | 12 ++++++--- .../apache/comet/serde/QueryPlanSerde.scala | 9 +++++-- .../shuffle/CometNativeShuffleWriter.scala | 26 +++++++++++++++++-- 4 files changed, 50 insertions(+), 7 deletions(-) diff --git a/native/core/src/execution/planner.rs b/native/core/src/execution/planner.rs index 8ae8c9c027..4b329e9eef 100644 --- a/native/core/src/execution/planner.rs +++ b/native/core/src/execution/planner.rs @@ -2202,6 +2202,16 @@ impl PhysicalPlanner { hash_partition.num_partitions as usize, )) } + PartitioningStruct::RangePartition(range_partition) => { + 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::from(exprs?); + println!("{:?}", lex_ordering); + todo!(); + } PartitioningStruct::SinglePartition(_) => Ok(Partitioning::UnknownPartitioning(1)), } } diff --git a/native/proto/src/proto/partitioning.proto b/native/proto/src/proto/partitioning.proto index 21bd505609..db28a64df5 100644 --- a/native/proto/src/proto/partitioning.proto +++ b/native/proto/src/proto/partitioning.proto @@ -28,15 +28,21 @@ option java_package = "org.apache.comet.serde"; // The basic message representing a Spark partitioning. message Partitioning { oneof partitioning_struct { - HashRepartition hash_partition = 2; - SinglePartition single_partition = 3; + HashPartition hash_partition = 1; + SinglePartition single_partition = 2; + RangePartition range_partition = 3; } } -message HashRepartition { +message HashPartition { repeated spark.spark_expression.Expr hash_expression = 1; int32 num_partitions = 2; } message SinglePartition { } + +message RangePartition { + repeated spark.spark_expression.Expr sort_orders = 1; + int32 num_partitions = 2; +} diff --git a/spark/src/main/scala/org/apache/comet/serde/QueryPlanSerde.scala b/spark/src/main/scala/org/apache/comet/serde/QueryPlanSerde.scala index 3fedaa7e36..a0da46fb66 100644 --- a/spark/src/main/scala/org/apache/comet/serde/QueryPlanSerde.scala +++ b/spark/src/main/scala/org/apache/comet/serde/QueryPlanSerde.scala @@ -39,8 +39,7 @@ import org.apache.spark.sql.execution import org.apache.spark.sql.execution._ import org.apache.spark.sql.execution.adaptive.{BroadcastQueryStageExec, ShuffleQueryStageExec} import org.apache.spark.sql.execution.aggregate.{BaseAggregateExec, HashAggregateExec, ObjectHashAggregateExec} -import org.apache.spark.sql.execution.datasources.{FilePartition, FileScanRDD} -import org.apache.spark.sql.execution.datasources.PartitionedFile +import org.apache.spark.sql.execution.datasources.{FilePartition, FileScanRDD, PartitionedFile} import org.apache.spark.sql.execution.datasources.v2.{DataSourceRDD, DataSourceRDDPartition} import org.apache.spark.sql.execution.exchange.{BroadcastExchangeExec, ReusedExchangeExec, ShuffleExchangeExec} import org.apache.spark.sql.execution.joins.{BroadcastHashJoinExec, HashJoin, ShuffledHashJoinExec, SortMergeJoinExec} @@ -2897,6 +2896,12 @@ object QueryPlanSerde extends Logging with CometExprShim { supported case SinglePartition => inputs.forall(attr => supportedShuffleDataType(attr.dataType)) + case RangePartitioning(ordering, numPartitions) => + // scalastyle:off + println(s"trying to do RangePartitioning $ordering $numPartitions") + // scalastyle:on + // TODO: Apply any expression constraints similar to HashPartitioning above. + true case _ => msg = s"unsupported Spark partitioning: ${partitioning.getClass.getName}" 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 66d2fac89c..70a08a2ef9 100644 --- a/spark/src/main/scala/org/apache/spark/sql/comet/execution/shuffle/CometNativeShuffleWriter.scala +++ b/spark/src/main/scala/org/apache/spark/sql/comet/execution/shuffle/CometNativeShuffleWriter.scala @@ -28,7 +28,7 @@ import org.apache.spark.{SparkEnv, TaskContext} 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.plans.physical.{HashPartitioning, Partitioning, SinglePartition} +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.vectorized.ColumnarBatch @@ -170,7 +170,7 @@ class CometNativeShuffleWriter[K, V]( case _: HashPartitioning => val hashPartitioning = outputPartitioning.asInstanceOf[HashPartitioning] - val partitioning = PartitioningOuterClass.HashRepartition.newBuilder() + val partitioning = PartitioningOuterClass.HashPartition.newBuilder() partitioning.setNumPartitions(outputPartitioning.numPartitions) val partitionExprs = hashPartitioning.expressions @@ -186,7 +186,29 @@ class CometNativeShuffleWriter[K, V]( val partitioningBuilder = PartitioningOuterClass.Partitioning.newBuilder() shuffleWriterBuilder.setPartitioning( partitioningBuilder.setHashPartition(partitioning).build()) + case _: RangePartitioning => + val rangePartitioning = outputPartitioning.asInstanceOf[RangePartitioning] + // scalastyle:off + println(rangePartitioning) + // scalastyle:on + + val partitioning = PartitioningOuterClass.RangePartition.newBuilder() + partitioning.setNumPartitions(outputPartitioning.numPartitions) + + 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) + + val partitioningBuilder = PartitioningOuterClass.Partitioning.newBuilder() + shuffleWriterBuilder.setPartitioning( + partitioningBuilder.setRangePartition(partitioning).build()) case SinglePartition => val partitioning = PartitioningOuterClass.SinglePartition.newBuilder() From 48710587e06270e6ef94e70de26871da0cd910f6 Mon Sep 17 00:00:00 2001 From: Matt Butrovich Date: Fri, 6 Jun 2025 13:42:18 -0400 Subject: [PATCH 06/72] Add CometPartitioning. --- native/core/src/execution/planner.rs | 31 ++++---- .../execution/shuffle/comet_partitioning.rs | 74 +++++++++++++++++++ native/core/src/execution/shuffle/mod.rs | 2 + .../execution/shuffle/range_partitioner.rs | 17 +++++ .../src/execution/shuffle/shuffle_writer.rs | 24 +++--- 5 files changed, 122 insertions(+), 26 deletions(-) create mode 100644 native/core/src/execution/shuffle/comet_partitioning.rs diff --git a/native/core/src/execution/planner.rs b/native/core/src/execution/planner.rs index 4b329e9eef..e215abe220 100644 --- a/native/core/src/execution/planner.rs +++ b/native/core/src/execution/planner.rs @@ -60,7 +60,7 @@ use datafusion::{ limit::LocalLimitExec, projection::ProjectionExec, sorts::sort::SortExec, - ExecutionPlan, Partitioning, + ExecutionPlan, }, prelude::SessionContext, }; @@ -69,7 +69,7 @@ use datafusion_comet_spark_expr::{ }; use crate::execution::operators::ExecutionError::GeneralError; -use crate::execution::shuffle::CompressionCodec; +use crate::execution::shuffle::{CometPartitioning, CompressionCodec}; use crate::execution::spark_plan::SparkPlan; use crate::parquet::parquet_support::prepare_object_store_with_configs; use datafusion::common::scalar::ScalarStructBuilder; @@ -921,7 +921,7 @@ impl PhysicalPlanner { let children = &spark_plan.children; match spark_plan.op_struct.as_ref().unwrap() { OpStruct::Projection(project) => { - assert!(children.len() == 1); + assert_eq!(children.len(), 1); let (scans, child) = self.create_plan(&children[0], inputs, partition_count)?; let exprs: PhyExprResult = project .project_list @@ -942,7 +942,7 @@ impl PhysicalPlanner { )) } OpStruct::Filter(filter) => { - assert!(children.len() == 1); + assert_eq!(children.len(), 1); let (scans, child) = self.create_plan(&children[0], inputs, partition_count)?; let predicate = self.create_expr(filter.predicate.as_ref().unwrap(), child.schema())?; @@ -965,7 +965,7 @@ impl PhysicalPlanner { )) } OpStruct::HashAgg(agg) => { - assert!(children.len() == 1); + assert_eq!(children.len(), 1); let (scans, child) = self.create_plan(&children[0], inputs, partition_count)?; let group_exprs: PhyExprResult = agg @@ -1043,7 +1043,7 @@ impl PhysicalPlanner { } } OpStruct::Limit(limit) => { - assert!(children.len() == 1); + assert_eq!(children.len(), 1); let num = limit.limit; let (scans, child) = self.create_plan(&children[0], inputs, partition_count)?; @@ -1057,7 +1057,7 @@ impl PhysicalPlanner { )) } OpStruct::Sort(sort) => { - assert!(children.len() == 1); + assert_eq!(children.len(), 1); let (scans, child) = self.create_plan(&children[0], inputs, partition_count)?; let exprs: Result, ExecutionError> = sort @@ -1225,7 +1225,7 @@ impl PhysicalPlanner { )) } OpStruct::ShuffleWriter(writer) => { - assert!(children.len() == 1); + assert_eq!(children.len(), 1); let (scans, child) = self.create_plan(&children[0], inputs, partition_count)?; let partitioning = self @@ -1263,7 +1263,7 @@ impl PhysicalPlanner { )) } OpStruct::Expand(expand) => { - assert!(children.len() == 1); + assert_eq!(children.len(), 1); let (scans, child) = self.create_plan(&children[0], inputs, partition_count)?; let mut projections = vec![]; @@ -1509,7 +1509,7 @@ impl PhysicalPlanner { condition: &Option, partition_count: usize, ) -> Result<(JoinParameters, Vec), ExecutionError> { - assert!(children.len() == 2); + assert_eq!(children.len(), 2); let (mut left_scans, left) = self.create_plan(&children[0], inputs, partition_count)?; let (mut right_scans, right) = self.create_plan(&children[1], inputs, partition_count)?; @@ -2189,7 +2189,7 @@ impl PhysicalPlanner { &self, spark_partitioning: &SparkPartitioning, input_schema: SchemaRef, - ) -> Result { + ) -> Result { match spark_partitioning.partitioning_struct.as_ref().unwrap() { PartitioningStruct::HashPartition(hash_partition) => { let exprs: PartitionPhyExprResult = hash_partition @@ -2197,7 +2197,7 @@ impl PhysicalPlanner { .iter() .map(|x| self.create_expr(x, Arc::clone(&input_schema))) .collect(); - Ok(Partitioning::Hash( + Ok(CometPartitioning::Hash( exprs?, hash_partition.num_partitions as usize, )) @@ -2210,9 +2210,12 @@ impl PhysicalPlanner { .collect(); let lex_ordering = LexOrdering::from(exprs?); println!("{:?}", lex_ordering); - todo!(); + Ok(CometPartitioning::RangePartitioning( + lex_ordering, + range_partition.num_partitions as usize, + )) } - PartitioningStruct::SinglePartition(_) => Ok(Partitioning::UnknownPartitioning(1)), + PartitioningStruct::SinglePartition(_) => Ok(CometPartitioning::UnknownPartitioning(1)), } } diff --git a/native/core/src/execution/shuffle/comet_partitioning.rs b/native/core/src/execution/shuffle/comet_partitioning.rs new file mode 100644 index 0000000000..d4f2337c22 --- /dev/null +++ b/native/core/src/execution/shuffle/comet_partitioning.rs @@ -0,0 +1,74 @@ +// 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 datafusion::physical_expr::{LexOrdering, Partitioning, PhysicalExpr}; +use std::sync::Arc; + +#[derive(Debug, Clone)] +pub enum CometPartitioning { + /// Allocate batches using a round-robin algorithm and the specified number of partitions + RoundRobinBatch(usize), + /// Allocate rows based on a hash of one of more expressions and the specified number of + /// partitions + Hash(Vec>, usize), + /// + RangePartitioning(LexOrdering, usize), + /// Unknown partitioning scheme with a known number of partitions + UnknownPartitioning(usize), +} + +impl CometPartitioning { + pub fn partition_count(&self) -> usize { + use CometPartitioning::*; + match self { + RoundRobinBatch(n) | Hash(_, n) | UnknownPartitioning(n) | RangePartitioning(_, n) => { + *n + } + } + } +} + +impl From for CometPartitioning { + fn from(df_partitioning: Partitioning) -> Self { + match df_partitioning { + Partitioning::RoundRobinBatch(partitions) => { + CometPartitioning::RoundRobinBatch(partitions) + } + Partitioning::Hash(exprs, partitions) => CometPartitioning::Hash(exprs, partitions), + Partitioning::UnknownPartitioning(partitions) => { + CometPartitioning::UnknownPartitioning(partitions) + } + } + } +} + +impl Into for CometPartitioning { + fn into(self) -> Partitioning { + match self { + CometPartitioning::RoundRobinBatch(partitions) => { + Partitioning::RoundRobinBatch(partitions) + } + CometPartitioning::Hash(exprs, partitions) => Partitioning::Hash(exprs, partitions), + CometPartitioning::UnknownPartitioning(partitions) => { + Partitioning::UnknownPartitioning(partitions) + } + CometPartitioning::RangePartitioning(_lex_ordering, usize) => { + Partitioning::UnknownPartitioning(usize) + } + } + } +} diff --git a/native/core/src/execution/shuffle/mod.rs b/native/core/src/execution/shuffle/mod.rs index a980482f7d..c3cb610afa 100644 --- a/native/core/src/execution/shuffle/mod.rs +++ b/native/core/src/execution/shuffle/mod.rs @@ -16,6 +16,7 @@ // under the License. pub(crate) mod codec; +mod comet_partitioning; mod list; mod map; mod range_partitioner; @@ -23,4 +24,5 @@ pub mod row; mod shuffle_writer; pub use codec::{read_ipc_compressed, CompressionCodec, ShuffleBlockWriter}; +pub use comet_partitioning::CometPartitioning; pub use shuffle_writer::ShuffleWriterExec; diff --git a/native/core/src/execution/shuffle/range_partitioner.rs b/native/core/src/execution/shuffle/range_partitioner.rs index f96ef3c7d9..4c18ee8cd7 100644 --- a/native/core/src/execution/shuffle/range_partitioner.rs +++ b/native/core/src/execution/shuffle/range_partitioner.rs @@ -1,3 +1,20 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + use arrow::array::{RecordBatch, UInt64Array}; use arrow::compute::{take, TakeOptions}; use datafusion::physical_expr::expressions::col; diff --git a/native/core/src/execution/shuffle/shuffle_writer.rs b/native/core/src/execution/shuffle/shuffle_writer.rs index 16e5516f48..bf896206ee 100644 --- a/native/core/src/execution/shuffle/shuffle_writer.rs +++ b/native/core/src/execution/shuffle/shuffle_writer.rs @@ -17,12 +17,12 @@ //! Defines the External shuffle repartition plan. -use crate::execution::shuffle::{CompressionCodec, ShuffleBlockWriter}; +use crate::execution::shuffle::{CometPartitioning, CompressionCodec, ShuffleBlockWriter}; use crate::execution::tracing::{with_trace, with_trace_async}; use arrow::compute::interleave_record_batch; use async_trait::async_trait; use datafusion::common::utils::proxy::VecAllocExt; -use datafusion::physical_expr::EquivalenceProperties; +use datafusion::physical_expr::{EquivalenceProperties, LexOrdering, PhysicalExpr}; use datafusion::physical_plan::execution_plan::{Boundedness, EmissionType}; use datafusion::physical_plan::EmptyRecordBatchStream; use datafusion::{ @@ -66,7 +66,7 @@ pub struct ShuffleWriterExec { /// Input execution plan input: Arc, /// Partitioning scheme to use - partitioning: Partitioning, + partitioning: CometPartitioning, /// Output data file path output_data_file: String, /// Output index file path @@ -84,7 +84,7 @@ impl ShuffleWriterExec { /// Create a new ShuffleWriterExec pub fn try_new( input: Arc, - partitioning: Partitioning, + partitioning: CometPartitioning, codec: CompressionCodec, output_data_file: String, output_index_file: String, @@ -92,7 +92,7 @@ impl ShuffleWriterExec { ) -> Result { let cache = PlanProperties::new( EquivalenceProperties::new(Arc::clone(&input.schema())), - partitioning.clone(), + partitioning.clone().into(), EmissionType::Final, Boundedness::Bounded, ); @@ -209,7 +209,7 @@ async fn external_shuffle( partition: usize, output_data_file: String, output_index_file: String, - partitioning: Partitioning, + partitioning: CometPartitioning, metrics: ShuffleRepartitionerMetrics, context: Arc, codec: CompressionCodec, @@ -321,7 +321,7 @@ struct MultiPartitionShuffleRepartitioner { partition_writers: Vec, shuffle_block_writer: ShuffleBlockWriter, /// Partitioning scheme to use - partitioning: Partitioning, + partitioning: CometPartitioning, runtime: Arc, metrics: ShuffleRepartitionerMetrics, /// Reused scratch space for computing partition indices @@ -356,7 +356,7 @@ impl MultiPartitionShuffleRepartitioner { output_data_file: String, output_index_file: String, schema: SchemaRef, - partitioning: Partitioning, + partitioning: CometPartitioning, metrics: ShuffleRepartitionerMetrics, runtime: Arc, batch_size: usize, @@ -431,7 +431,7 @@ impl MultiPartitionShuffleRepartitioner { self.metrics.baseline.record_output(input.num_rows()); match &self.partitioning { - Partitioning::Hash(exprs, num_output_partitions) => { + CometPartitioning::Hash(exprs, num_output_partitions) => { let mut scratch = std::mem::take(&mut self.scratch); let (partition_starts, partition_row_indices): (&Vec, &Vec) = { let mut timer = self.metrics.repart_time.timer(); @@ -512,7 +512,7 @@ impl MultiPartitionShuffleRepartitioner { // this should be unreachable as long as the validation logic // in the constructor is kept up-to-date return Err(DataFusionError::NotImplemented(format!( - "Unsupported repartitioning scheme {:?}", + "Unsupported shuffle partitioning scheme {:?}", other ))); } @@ -1253,7 +1253,7 @@ mod test { "/tmp/data.out".to_string(), "/tmp/index.out".to_string(), batch.schema(), - Partitioning::Hash(vec![Arc::new(Column::new("a", 0))], num_partitions), + CometPartitioning::Hash(vec![Arc::new(Column::new("a", 0))], num_partitions), ShuffleRepartitionerMetrics::new(&metrics_set, 0), runtime_env, 1024, @@ -1303,7 +1303,7 @@ mod test { Arc::new(DataSourceExec::new(Arc::new( MemorySourceConfig::try_new(partitions, batch.schema(), None).unwrap(), ))), - Partitioning::Hash(vec![Arc::new(Column::new("a", 0))], num_partitions), + CometPartitioning::Hash(vec![Arc::new(Column::new("a", 0))], num_partitions), CompressionCodec::Zstd(1), "/tmp/data.out".to_string(), "/tmp/index.out".to_string(), From d7cbb5490e99b405d5601e92d0bb3f48fef7c9ff Mon Sep 17 00:00:00 2001 From: Matt Butrovich Date: Fri, 6 Jun 2025 17:18:40 -0400 Subject: [PATCH 07/72] reservoir_sample_indices. --- .../execution/shuffle/range_partitioner.rs | 63 ++++++++++--------- 1 file changed, 32 insertions(+), 31 deletions(-) diff --git a/native/core/src/execution/shuffle/range_partitioner.rs b/native/core/src/execution/shuffle/range_partitioner.rs index 4c18ee8cd7..1c5b9b22ca 100644 --- a/native/core/src/execution/shuffle/range_partitioner.rs +++ b/native/core/src/execution/shuffle/range_partitioner.rs @@ -15,8 +15,10 @@ // specific language governing permissions and limitations // under the License. -use arrow::array::{RecordBatch, UInt64Array}; -use arrow::compute::{take, TakeOptions}; +use arrow::array::{ArrayRef, PrimitiveArray, RecordBatch, UInt64Array}; +use arrow::compute::{take, take_record_batch, TakeOptions}; +use arrow::row::{RowConverter, SortField}; +use datafusion::common::HashSet; use datafusion::physical_expr::expressions::col; use datafusion::physical_expr::{LexOrdering, PhysicalSortExpr}; use datafusion::physical_plan::sorts::sort::sort_batch; @@ -24,19 +26,25 @@ use num::ToPrimitive; use rand::Rng; use std::sync::Arc; -struct RangePartitioner; +pub struct RangePartitioner; impl RangePartitioner { // Adapted from https://en.wikipedia.org/wiki/Reservoir_sampling#Optimal:_Algorithm_L // We use sample_size instead of k and input_length instead of n. // We use indices in the reservoir instead of actual values since we'll do one take() on the // input array at the end. - fn reservoir_sample(input: RecordBatch, sample_size: usize) -> RecordBatch { - let input_length = input.num_rows(); + pub fn reservoir_sample(input: RecordBatch, sample_size: usize) -> RecordBatch { + // Build our indices array and then take the values from the input. + let indices = UInt64Array::from(Self::reservoir_sample_indices(input.num_rows(), sample_size)); + + // TODO: This bounds checks, probably not necessary. + take_record_batch(&input, &indices).unwrap() + } - if input_length <= sample_size { + pub fn reservoir_sample_indices(num_rows: usize, sample_size: usize) -> Vec { + if num_rows <= sample_size { // Just return the original input since we can't create a bigger sample. - return input.clone(); + return (0..num_rows as u64).collect(); } // Initialize our reservoir with indices of the first |sample_size| elements. @@ -44,12 +52,12 @@ impl RangePartitioner { let mut rng = rand::rng(); let mut w = (rng.random::().ln() / sample_size as f64).exp(); - let mut i = sample_size; + let mut i = sample_size - 1; - while i < input_length { - i += (rng.random::().ln() / (1.0 - w).ln()).floor() as usize; + while i < num_rows { + i += (rng.random::().ln() / (1.0 - w).ln()).floor() as usize + 1; - if i < input_length { + 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; @@ -57,27 +65,20 @@ impl RangePartitioner { } } - // Build our indices array and then take the values from the input. - let indices = UInt64Array::from(reservoir); + let set: HashSet = reservoir.iter().map(|e| *e).collect(); + assert_eq!(set.len(), reservoir.len()); - // let result = take_record_batch(&input, &indices).unwrap(); + reservoir + } - // We don't use take_record_batch because it needlessly bounds checks - let columns = input - .columns() - .iter() - .map(|c| { - take( - c, - &indices, - Some(TakeOptions { - check_bounds: false, - }), - ) - }) - .collect::, _>>() - .unwrap(); - RecordBatch::try_new(input.schema(), columns).unwrap() + // Adapted from org.apache.spark.RangePartitioner.determineBounds + pub fn determine_bounds_for_batch( + input: RecordBatch, + lex_ordering: &LexOrdering, + partitions: i32, + ) -> ArrayRef { + // sort.sort_unstable_by(|(_, a), (_, b)| a.cmp(b)); + todo!() } // Adapted from org.apache.spark.RangePartitioner.determineBounds @@ -129,7 +130,7 @@ mod test { #[test] fn test_sort_record_batch() { let batch = create_random_batch(10000); - let sample_size = 10; + let sample_size = 20; let lex_ordering = LexOrdering::new(vec![PhysicalSortExpr::new_default( col("a", batch.schema().as_ref()).unwrap(), )]); From a2b4d29003590afde89dcef67a8e72ee320d6667 Mon Sep 17 00:00:00 2001 From: Matt Butrovich Date: Fri, 6 Jun 2025 17:49:42 -0400 Subject: [PATCH 08/72] checkpoint on sampling batch, converting it to rows, sorting, and then taking the result. Next step is to generate the boundaries from Rows input. --- .../execution/shuffle/range_partitioner.rs | 5 +- .../src/execution/shuffle/shuffle_writer.rs | 55 ++++++++++++++++++- 2 files changed, 58 insertions(+), 2 deletions(-) diff --git a/native/core/src/execution/shuffle/range_partitioner.rs b/native/core/src/execution/shuffle/range_partitioner.rs index 1c5b9b22ca..8bdb2e8f28 100644 --- a/native/core/src/execution/shuffle/range_partitioner.rs +++ b/native/core/src/execution/shuffle/range_partitioner.rs @@ -35,7 +35,10 @@ impl RangePartitioner { // input array at the end. pub fn reservoir_sample(input: RecordBatch, sample_size: usize) -> RecordBatch { // Build our indices array and then take the values from the input. - let indices = UInt64Array::from(Self::reservoir_sample_indices(input.num_rows(), sample_size)); + let indices = UInt64Array::from(Self::reservoir_sample_indices( + input.num_rows(), + sample_size, + )); // TODO: This bounds checks, probably not necessary. take_record_batch(&input, &indices).unwrap() diff --git a/native/core/src/execution/shuffle/shuffle_writer.rs b/native/core/src/execution/shuffle/shuffle_writer.rs index bf896206ee..52b871b2ad 100644 --- a/native/core/src/execution/shuffle/shuffle_writer.rs +++ b/native/core/src/execution/shuffle/shuffle_writer.rs @@ -17,9 +17,13 @@ //! Defines the External shuffle repartition plan. +use crate::execution::shuffle::range_partitioner; +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::compute::{interleave_record_batch, partition, take, take_arrays}; +use arrow::datatypes::Schema; +use arrow::row::{RowConverter, SortField}; use async_trait::async_trait; use datafusion::common::utils::proxy::VecAllocExt; use datafusion::physical_expr::{EquivalenceProperties, LexOrdering, PhysicalExpr}; @@ -508,6 +512,55 @@ impl MultiPartitionShuffleRepartitioner { .await?; self.scratch = scratch; } + CometPartitioning::RangePartitioning(lex_ordering, num_output_partitions) => { + println!("{:?}", lex_ordering); + println!("{:?}", num_output_partitions); + println!("{:?}", input.schema()); + println!("{:?}", input.num_rows()); + + // evaluate partition expressions + let partition_arrays = lex_ordering + .iter() + .map(|expr| expr.expr.evaluate(&input)?.into_array(input.num_rows())) + .collect::>>()?; + + println!("partition_arrays: {:?}", partition_arrays); + + let sample_indices = UInt64Array::from(RangePartitioner::reservoir_sample_indices( + input.num_rows(), + 100, + )); + + println!("sample_indices: {:?}", sample_indices); + + let sampled_columns = partition_arrays + .iter() + .map(|c| take(c, &sample_indices, None)) + .collect::, _>>()?; + + println!("sampled_columns: {:?}", sampled_columns); + + let sort_fields = partition_arrays + .iter() + .zip(lex_ordering) + .map(|(array, sort_expr)| { + SortField::new_with_options(array.data_type().clone(), sort_expr.options) + }) + .collect(); + + println!("{:?}", sort_fields); + let converter = RowConverter::new(sort_fields)?; + let rows = converter.convert_columns(sampled_columns.as_slice())?; + let mut sort: Vec<_> = rows.iter().enumerate().collect(); + sort.sort_unstable_by(|(_, a), (_, b)| a.cmp(b)); + let sort_indices = + UInt32Array::from_iter_values(sort.iter().map(|(i, _)| *i as u32)); + println!("indices: {:?}", sort_indices); + let sorted_sample = take_arrays(sampled_columns.as_slice(), &sort_indices, None)?; + println!("sorted_sample: {:?}", sorted_sample); + + todo!(); + } other => { // this should be unreachable as long as the validation logic // in the constructor is kept up-to-date From fc3dea6ad3d38818fd6e786020d2a98036bd184d Mon Sep 17 00:00:00 2001 From: Matt Butrovich Date: Fri, 6 Jun 2025 19:35:54 -0400 Subject: [PATCH 09/72] Checkpoint on bounds for rows. --- .../execution/shuffle/range_partitioner.rs | 59 ++++++++++++++++++- .../src/execution/shuffle/shuffle_writer.rs | 33 +++++------ 2 files changed, 71 insertions(+), 21 deletions(-) diff --git a/native/core/src/execution/shuffle/range_partitioner.rs b/native/core/src/execution/shuffle/range_partitioner.rs index 8bdb2e8f28..0ed43c24c9 100644 --- a/native/core/src/execution/shuffle/range_partitioner.rs +++ b/native/core/src/execution/shuffle/range_partitioner.rs @@ -15,13 +15,14 @@ // specific language governing permissions and limitations // under the License. -use arrow::array::{ArrayRef, PrimitiveArray, RecordBatch, UInt64Array}; +use arrow::array::{ArrayRef, AsArray, PrimitiveArray, RecordBatch, UInt64Array}; use arrow::compute::{take, take_record_batch, TakeOptions}; -use arrow::row::{RowConverter, SortField}; +use arrow::row::{Row, RowConverter, Rows, SortField}; use datafusion::common::HashSet; use datafusion::physical_expr::expressions::col; use datafusion::physical_expr::{LexOrdering, PhysicalSortExpr}; use datafusion::physical_plan::sorts::sort::sort_batch; +use itertools::Itertools; use num::ToPrimitive; use rand::Rng; use std::sync::Arc; @@ -118,6 +119,60 @@ impl RangePartitioner { } bounds } + + // Adapted from org.apache.spark.RangePartitioner.determineBounds + pub fn determine_bounds_for_rows( + sort_fields: Vec, + sampled_columns: Vec, + partitions: i32, + ) { + // println!("{:?}", partitions); + let converter = RowConverter::new(sort_fields).unwrap(); + let rows = converter + .convert_columns(sampled_columns.as_slice()) + .unwrap(); + // println!("rows: {:?}", rows); + let mut thing: Vec<_> = rows.iter().enumerate().collect(); + thing.sort_unstable_by(|(_, a), (_, b)| a.cmp(b)); + // println!("rows.sorted(): {:?}", thing); + + let num_candidates = 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::with_capacity((partitions - 1) as usize); + 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 = thing[i]; + cumulative_weights += sample_weight; + if cumulative_weights >= target { + // Skip duplicate values. + if previous_bound.is_none() || key.1 > previous_bound.unwrap() { + // bounds.push(key.1); + bounds_indices.push(key.0); + target += step; + println!("{}", i); + j += 1; + previous_bound = Some(key.1) + } + } + i += 1 + } + + println!("bounds_indices.len(): {:?}", bounds_indices.len()); + + let selection: Vec = bounds_indices.iter().map(|&idx| rows.row(idx)).collect(); + + // let selection = [rows.row(0), rows2.row(1), rows.row(2), rows2.row(0)]; + let converted = converter.convert_rows(selection).unwrap(); + + println!("{:?}", converted); + + todo!() + } } #[cfg(test)] diff --git a/native/core/src/execution/shuffle/shuffle_writer.rs b/native/core/src/execution/shuffle/shuffle_writer.rs index 52b871b2ad..9558c29511 100644 --- a/native/core/src/execution/shuffle/shuffle_writer.rs +++ b/native/core/src/execution/shuffle/shuffle_writer.rs @@ -23,7 +23,7 @@ use crate::execution::shuffle::{CometPartitioning, CompressionCodec, ShuffleBloc use crate::execution::tracing::{with_trace, with_trace_async}; use arrow::compute::{interleave_record_batch, partition, take, take_arrays}; use arrow::datatypes::Schema; -use arrow::row::{RowConverter, SortField}; +use arrow::row::{RowConverter, Rows, SortField}; use async_trait::async_trait; use datafusion::common::utils::proxy::VecAllocExt; use datafusion::physical_expr::{EquivalenceProperties, LexOrdering, PhysicalExpr}; @@ -513,10 +513,10 @@ impl MultiPartitionShuffleRepartitioner { self.scratch = scratch; } CometPartitioning::RangePartitioning(lex_ordering, num_output_partitions) => { - println!("{:?}", lex_ordering); - println!("{:?}", num_output_partitions); - println!("{:?}", input.schema()); - println!("{:?}", input.num_rows()); + // println!("{:?}", lex_ordering); + // println!("{:?}", num_output_partitions); + // println!("{:?}", input.schema()); + // println!("{:?}", input.num_rows()); // evaluate partition expressions let partition_arrays = lex_ordering @@ -524,23 +524,23 @@ impl MultiPartitionShuffleRepartitioner { .map(|expr| expr.expr.evaluate(&input)?.into_array(input.num_rows())) .collect::>>()?; - println!("partition_arrays: {:?}", partition_arrays); + // println!("partition_arrays: {:?}", partition_arrays); let sample_indices = UInt64Array::from(RangePartitioner::reservoir_sample_indices( input.num_rows(), 100, )); - println!("sample_indices: {:?}", sample_indices); + // println!("sample_indices: {:?}", sample_indices); let sampled_columns = partition_arrays .iter() .map(|c| take(c, &sample_indices, None)) .collect::, _>>()?; - println!("sampled_columns: {:?}", sampled_columns); + // println!("sampled_columns: {:?}", sampled_columns); - let sort_fields = partition_arrays + let sort_fields: Vec = partition_arrays .iter() .zip(lex_ordering) .map(|(array, sort_expr)| { @@ -548,16 +548,11 @@ impl MultiPartitionShuffleRepartitioner { }) .collect(); - println!("{:?}", sort_fields); - let converter = RowConverter::new(sort_fields)?; - let rows = converter.convert_columns(sampled_columns.as_slice())?; - let mut sort: Vec<_> = rows.iter().enumerate().collect(); - sort.sort_unstable_by(|(_, a), (_, b)| a.cmp(b)); - let sort_indices = - UInt32Array::from_iter_values(sort.iter().map(|(i, _)| *i as u32)); - println!("indices: {:?}", sort_indices); - let sorted_sample = take_arrays(sampled_columns.as_slice(), &sort_indices, None)?; - println!("sorted_sample: {:?}", sorted_sample); + RangePartitioner::determine_bounds_for_rows( + sort_fields, + sampled_columns, + *num_output_partitions as i32, + ); todo!(); } From a50aad957f884649b4016ed0f44b79dca7dbfbb8 Mon Sep 17 00:00:00 2001 From: Matt Butrovich Date: Fri, 6 Jun 2025 19:49:20 -0400 Subject: [PATCH 10/72] Cleanup. --- .../src/execution/shuffle/range_partitioner.rs | 18 ++++-------------- 1 file changed, 4 insertions(+), 14 deletions(-) diff --git a/native/core/src/execution/shuffle/range_partitioner.rs b/native/core/src/execution/shuffle/range_partitioner.rs index 0ed43c24c9..6375e78174 100644 --- a/native/core/src/execution/shuffle/range_partitioner.rs +++ b/native/core/src/execution/shuffle/range_partitioner.rs @@ -15,9 +15,9 @@ // specific language governing permissions and limitations // under the License. -use arrow::array::{ArrayRef, AsArray, PrimitiveArray, RecordBatch, UInt64Array}; -use arrow::compute::{take, take_record_batch, TakeOptions}; -use arrow::row::{Row, RowConverter, Rows, SortField}; +use arrow::array::{ArrayRef, AsArray, RecordBatch, UInt64Array}; +use arrow::compute::take_record_batch; +use arrow::row::{Row, RowConverter, SortField}; use datafusion::common::HashSet; use datafusion::physical_expr::expressions::col; use datafusion::physical_expr::{LexOrdering, PhysicalSortExpr}; @@ -75,16 +75,6 @@ impl RangePartitioner { reservoir } - // Adapted from org.apache.spark.RangePartitioner.determineBounds - pub fn determine_bounds_for_batch( - input: RecordBatch, - lex_ordering: &LexOrdering, - partitions: i32, - ) -> ArrayRef { - // sort.sort_unstable_by(|(_, a), (_, b)| a.cmp(b)); - todo!() - } - // Adapted from org.apache.spark.RangePartitioner.determineBounds fn determine_bounds(mut candidates: Vec<(T, f64)>, partitions: i32) -> Vec where @@ -161,7 +151,7 @@ impl RangePartitioner { } i += 1 } - + println!("bounds_indices.len(): {:?}", bounds_indices.len()); let selection: Vec = bounds_indices.iter().map(|&idx| rows.row(idx)).collect(); From cbdaf8267c57176a72367f2f7488ad452b067ff9 Mon Sep 17 00:00:00 2001 From: Matt Butrovich Date: Fri, 6 Jun 2025 20:28:07 -0400 Subject: [PATCH 11/72] Cleanup. --- native/core/src/execution/planner.rs | 1 - .../execution/shuffle/range_partitioner.rs | 29 +++++++------------ .../src/execution/shuffle/shuffle_writer.rs | 15 ++-------- 3 files changed, 14 insertions(+), 31 deletions(-) diff --git a/native/core/src/execution/planner.rs b/native/core/src/execution/planner.rs index e215abe220..476988f44e 100644 --- a/native/core/src/execution/planner.rs +++ b/native/core/src/execution/planner.rs @@ -2209,7 +2209,6 @@ impl PhysicalPlanner { .map(|expr| self.create_sort_expr(expr, Arc::clone(&input_schema))) .collect(); let lex_ordering = LexOrdering::from(exprs?); - println!("{:?}", lex_ordering); Ok(CometPartitioning::RangePartitioning( lex_ordering, range_partition.num_partitions as usize, diff --git a/native/core/src/execution/shuffle/range_partitioner.rs b/native/core/src/execution/shuffle/range_partitioner.rs index 6375e78174..9ba7022eac 100644 --- a/native/core/src/execution/shuffle/range_partitioner.rs +++ b/native/core/src/execution/shuffle/range_partitioner.rs @@ -115,18 +115,15 @@ impl RangePartitioner { sort_fields: Vec, sampled_columns: Vec, partitions: i32, - ) { - // println!("{:?}", partitions); + ) -> (Vec, RowConverter) { let converter = RowConverter::new(sort_fields).unwrap(); - let rows = converter + let sampled_rows = converter .convert_columns(sampled_columns.as_slice()) .unwrap(); - // println!("rows: {:?}", rows); - let mut thing: Vec<_> = rows.iter().enumerate().collect(); - thing.sort_unstable_by(|(_, a), (_, b)| a.cmp(b)); - // println!("rows.sorted(): {:?}", thing); + 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 = rows.num_rows(); + let num_candidates = sampled_rows.num_rows(); let step = 1.0 / partitions as f64; let mut cumulative_weights = 0.0; let mut target = step; @@ -136,7 +133,7 @@ impl RangePartitioner { let mut previous_bound = None; let sample_weight = 1.0 / num_candidates as f64; while (i < num_candidates) && (j < partitions - 1) { - let key = thing[i]; + let key = sorted_sampled_rows[i]; cumulative_weights += sample_weight; if cumulative_weights >= target { // Skip duplicate values. @@ -144,7 +141,6 @@ impl RangePartitioner { // bounds.push(key.1); bounds_indices.push(key.0); target += step; - println!("{}", i); j += 1; previous_bound = Some(key.1) } @@ -152,16 +148,14 @@ impl RangePartitioner { i += 1 } - println!("bounds_indices.len(): {:?}", bounds_indices.len()); - - let selection: Vec = bounds_indices.iter().map(|&idx| rows.row(idx)).collect(); + let selection: Vec = bounds_indices + .iter() + .map(|idx| sampled_rows.row(*idx)) + .collect(); - // let selection = [rows.row(0), rows2.row(1), rows.row(2), rows2.row(0)]; let converted = converter.convert_rows(selection).unwrap(); - println!("{:?}", converted); - - todo!() + (converted, converter) } } @@ -185,7 +179,6 @@ mod test { let batch = sort_batch(&batch, &lex_ordering, None).unwrap(); let sample = RangePartitioner::reservoir_sample(batch, sample_size); let sorted_sample = sort_batch(&sample, &lex_ordering, None).unwrap(); - println!("sorted_sample: {:?}", sorted_sample); } #[test] diff --git a/native/core/src/execution/shuffle/shuffle_writer.rs b/native/core/src/execution/shuffle/shuffle_writer.rs index 9558c29511..d3d89998a3 100644 --- a/native/core/src/execution/shuffle/shuffle_writer.rs +++ b/native/core/src/execution/shuffle/shuffle_writer.rs @@ -513,33 +513,22 @@ impl MultiPartitionShuffleRepartitioner { self.scratch = scratch; } CometPartitioning::RangePartitioning(lex_ordering, num_output_partitions) => { - // println!("{:?}", lex_ordering); - // println!("{:?}", num_output_partitions); - // println!("{:?}", input.schema()); - // println!("{:?}", input.num_rows()); - // evaluate partition expressions let partition_arrays = lex_ordering .iter() .map(|expr| expr.expr.evaluate(&input)?.into_array(input.num_rows())) .collect::>>()?; - // println!("partition_arrays: {:?}", partition_arrays); - let sample_indices = UInt64Array::from(RangePartitioner::reservoir_sample_indices( input.num_rows(), 100, )); - // println!("sample_indices: {:?}", sample_indices); - let sampled_columns = partition_arrays .iter() .map(|c| take(c, &sample_indices, None)) .collect::, _>>()?; - // println!("sampled_columns: {:?}", sampled_columns); - let sort_fields: Vec = partition_arrays .iter() .zip(lex_ordering) @@ -548,12 +537,14 @@ impl MultiPartitionShuffleRepartitioner { }) .collect(); - RangePartitioner::determine_bounds_for_rows( + let (bounds, converter) = RangePartitioner::determine_bounds_for_rows( sort_fields, sampled_columns, *num_output_partitions as i32, ); + println!("{:?}", bounds); + todo!(); } other => { From c17a028164e13feaf0069ef58629c879bf853440 Mon Sep 17 00:00:00 2001 From: Matt Butrovich Date: Fri, 6 Jun 2025 21:08:17 -0400 Subject: [PATCH 12/72] Stash bounds and converter. --- .../execution/shuffle/range_partitioner.rs | 15 +++-- .../src/execution/shuffle/shuffle_writer.rs | 60 +++++++++++-------- 2 files changed, 43 insertions(+), 32 deletions(-) diff --git a/native/core/src/execution/shuffle/range_partitioner.rs b/native/core/src/execution/shuffle/range_partitioner.rs index 9ba7022eac..d131620ca7 100644 --- a/native/core/src/execution/shuffle/range_partitioner.rs +++ b/native/core/src/execution/shuffle/range_partitioner.rs @@ -17,7 +17,7 @@ use arrow::array::{ArrayRef, AsArray, RecordBatch, UInt64Array}; use arrow::compute::take_record_batch; -use arrow::row::{Row, RowConverter, SortField}; +use arrow::row::{OwnedRow, Row, RowConverter, SortField}; use datafusion::common::HashSet; use datafusion::physical_expr::expressions::col; use datafusion::physical_expr::{LexOrdering, PhysicalSortExpr}; @@ -115,7 +115,7 @@ impl RangePartitioner { sort_fields: Vec, sampled_columns: Vec, partitions: i32, - ) -> (Vec, RowConverter) { + ) -> (Vec, RowConverter) { let converter = RowConverter::new(sort_fields).unwrap(); let sampled_rows = converter .convert_columns(sampled_columns.as_slice()) @@ -148,14 +148,13 @@ impl RangePartitioner { i += 1 } - let selection: Vec = bounds_indices - .iter() - .map(|idx| sampled_rows.row(*idx)) - .collect(); + let mut result: Vec = Vec::with_capacity(bounds_indices.len()); - let converted = converter.convert_rows(selection).unwrap(); + bounds_indices + .iter() + .for_each(|idx| result.push(sampled_rows.row(*idx).owned())); - (converted, converter) + (result, converter) } } diff --git a/native/core/src/execution/shuffle/shuffle_writer.rs b/native/core/src/execution/shuffle/shuffle_writer.rs index d3d89998a3..5fe2864ad1 100644 --- a/native/core/src/execution/shuffle/shuffle_writer.rs +++ b/native/core/src/execution/shuffle/shuffle_writer.rs @@ -23,7 +23,7 @@ use crate::execution::shuffle::{CometPartitioning, CompressionCodec, ShuffleBloc use crate::execution::tracing::{with_trace, with_trace_async}; use arrow::compute::{interleave_record_batch, partition, take, take_arrays}; use arrow::datatypes::Schema; -use arrow::row::{RowConverter, Rows, SortField}; +use arrow::row::{OwnedRow, RowConverter, Rows, SortField}; use async_trait::async_trait; use datafusion::common::utils::proxy::VecAllocExt; use datafusion::physical_expr::{EquivalenceProperties, LexOrdering, PhysicalExpr}; @@ -335,6 +335,9 @@ struct MultiPartitionShuffleRepartitioner { /// Reservation for repartitioning reservation: MemoryReservation, tracing_enabled: bool, + /// RangePartitioning-specific state + range_boundaries: Option>, + row_converter: Option, } #[derive(Default)] @@ -407,6 +410,8 @@ impl MultiPartitionShuffleRepartitioner { batch_size, reservation, tracing_enabled, + range_boundaries: None, + row_converter: None, }) } @@ -519,32 +524,39 @@ impl MultiPartitionShuffleRepartitioner { .map(|expr| expr.expr.evaluate(&input)?.into_array(input.num_rows())) .collect::>>()?; - let sample_indices = UInt64Array::from(RangePartitioner::reservoir_sample_indices( - input.num_rows(), - 100, - )); + if self.range_boundaries.is_none() { + assert!(self.row_converter.is_none()); - let sampled_columns = partition_arrays - .iter() - .map(|c| take(c, &sample_indices, None)) - .collect::, _>>()?; - - 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, converter) = RangePartitioner::determine_bounds_for_rows( - sort_fields, - sampled_columns, - *num_output_partitions as i32, - ); + // TODO: Adjust sample size. + let sample_indices = UInt64Array::from( + RangePartitioner::reservoir_sample_indices(input.num_rows(), 100), + ); - println!("{:?}", bounds); + let sampled_columns = partition_arrays + .iter() + .map(|c| take(c, &sample_indices, None)) + .collect::, _>>()?; + 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, converter) = RangePartitioner::determine_bounds_for_rows( + sort_fields, + sampled_columns, + *num_output_partitions as i32, + ); + + self.range_boundaries = Some(bounds); + self.row_converter = Some(converter); + } todo!(); } other => { From 090d7b02a8a34d5b578dd8c97530a37656423f3c Mon Sep 17 00:00:00 2001 From: Matt Butrovich Date: Fri, 6 Jun 2025 21:32:25 -0400 Subject: [PATCH 13/72] Add partition_indices. --- .../src/execution/shuffle/shuffle_writer.rs | 21 +++++++++++++++++++ 1 file changed, 21 insertions(+) diff --git a/native/core/src/execution/shuffle/shuffle_writer.rs b/native/core/src/execution/shuffle/shuffle_writer.rs index 5fe2864ad1..7d0de06199 100644 --- a/native/core/src/execution/shuffle/shuffle_writer.rs +++ b/native/core/src/execution/shuffle/shuffle_writer.rs @@ -557,6 +557,27 @@ impl MultiPartitionShuffleRepartitioner { self.range_boundaries = Some(bounds); self.row_converter = Some(converter); } + + let row_batch = self + .row_converter + .as_ref() + .unwrap() + .convert_columns(partition_arrays.as_slice())?; + + let partition_indices: Vec = row_batch + .iter() + .map(|row| { + let search_result = self + .range_boundaries + .as_ref() + .unwrap() + .binary_search(&row.owned()); + search_result.unwrap_or_else(|idx| idx) + }) + .collect(); + + println!("{:?}", partition_indices); + todo!(); } other => { From f71ff3982946616dd0ab489ab73ebe301256de70 Mon Sep 17 00:00:00 2001 From: Matt Butrovich Date: Fri, 6 Jun 2025 21:44:19 -0400 Subject: [PATCH 14/72] Use scratch space. --- .../src/execution/shuffle/shuffle_writer.rs | 27 ++++++++++--------- 1 file changed, 14 insertions(+), 13 deletions(-) diff --git a/native/core/src/execution/shuffle/shuffle_writer.rs b/native/core/src/execution/shuffle/shuffle_writer.rs index 7d0de06199..3b92cddd31 100644 --- a/native/core/src/execution/shuffle/shuffle_writer.rs +++ b/native/core/src/execution/shuffle/shuffle_writer.rs @@ -518,6 +518,7 @@ impl MultiPartitionShuffleRepartitioner { self.scratch = scratch; } CometPartitioning::RangePartitioning(lex_ordering, num_output_partitions) => { + let mut scratch = std::mem::take(&mut self.scratch); // evaluate partition expressions let partition_arrays = lex_ordering .iter() @@ -564,19 +565,19 @@ impl MultiPartitionShuffleRepartitioner { .unwrap() .convert_columns(partition_arrays.as_slice())?; - let partition_indices: Vec = row_batch - .iter() - .map(|row| { - let search_result = self - .range_boundaries - .as_ref() - .unwrap() - .binary_search(&row.owned()); - search_result.unwrap_or_else(|idx| idx) - }) - .collect(); - - println!("{:?}", partition_indices); + let partition_ids = &mut scratch.partition_ids[..partition_arrays[0].len()]; + row_batch.iter().enumerate().for_each(|(idx, row)| { + let search_result = self + .range_boundaries + .as_ref() + .unwrap() + .binary_search(&row.owned()); + let partition = search_result.unwrap_or_else(|idx| idx); + assert!(partition < *num_output_partitions); + partition_ids[idx] = partition as u32; + }); + + println!("{:?}", partition_ids); todo!(); } From 9fa063af131c953dfb6ecc582418578ef3fc2f67 Mon Sep 17 00:00:00 2001 From: Matt Butrovich Date: Fri, 6 Jun 2025 21:53:12 -0400 Subject: [PATCH 15/72] It works with a lot of copy-pasted code. --- .../src/execution/shuffle/shuffle_writer.rs | 160 ++++++++++++------ 1 file changed, 108 insertions(+), 52 deletions(-) diff --git a/native/core/src/execution/shuffle/shuffle_writer.rs b/native/core/src/execution/shuffle/shuffle_writer.rs index 3b92cddd31..10e8b8d0d7 100644 --- a/native/core/src/execution/shuffle/shuffle_writer.rs +++ b/native/core/src/execution/shuffle/shuffle_writer.rs @@ -519,67 +519,123 @@ impl MultiPartitionShuffleRepartitioner { } CometPartitioning::RangePartitioning(lex_ordering, num_output_partitions) => { let mut scratch = std::mem::take(&mut self.scratch); - // evaluate partition expressions - let partition_arrays = lex_ordering - .iter() - .map(|expr| expr.expr.evaluate(&input)?.into_array(input.num_rows())) - .collect::>>()?; + let (partition_starts, partition_row_indices): (&Vec, &Vec) = { + let mut timer = self.metrics.repart_time.timer(); - if self.range_boundaries.is_none() { - assert!(self.row_converter.is_none()); + // evaluate partition expressions + let partition_arrays = lex_ordering + .iter() + .map(|expr| expr.expr.evaluate(&input)?.into_array(input.num_rows())) + .collect::>>()?; - // TODO: Adjust sample size. - let sample_indices = UInt64Array::from( - RangePartitioner::reservoir_sample_indices(input.num_rows(), 100), - ); + if self.range_boundaries.is_none() { + assert!(self.row_converter.is_none()); + + // TODO: Adjust sample size. + let sample_indices = UInt64Array::from( + RangePartitioner::reservoir_sample_indices(input.num_rows(), 100), + ); + + let sampled_columns = partition_arrays + .iter() + .map(|c| take(c, &sample_indices, None)) + .collect::, _>>()?; + + 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, converter) = RangePartitioner::determine_bounds_for_rows( + sort_fields, + sampled_columns, + *num_output_partitions as i32, + ); + + self.range_boundaries = Some(bounds); + self.row_converter = Some(converter); + } - let sampled_columns = partition_arrays - .iter() - .map(|c| take(c, &sample_indices, None)) - .collect::, _>>()?; + let row_batch = self + .row_converter + .as_ref() + .unwrap() + .convert_columns(partition_arrays.as_slice())?; + + let partition_ids = &mut scratch.partition_ids[..partition_arrays[0].len()]; + row_batch.iter().enumerate().for_each(|(idx, row)| { + let search_result = self + .range_boundaries + .as_ref() + .unwrap() + .binary_search(&row.owned()); + let partition = search_result.unwrap_or_else(|idx| idx); + assert!(partition < *num_output_partitions); + partition_ids[idx] = partition as u32; + }); - let sort_fields: Vec = partition_arrays + println!("{:?}", partition_ids); + + // count each partition size, while leaving the last extra element as 0 + let partition_counters = &mut scratch.partition_starts; + partition_counters.resize(num_output_partitions + 1, 0); + partition_counters.fill(0); + partition_ids .iter() - .zip(lex_ordering) - .map(|(array, sort_expr)| { - SortField::new_with_options( - array.data_type().clone(), - sort_expr.options, - ) - }) - .collect(); - - let (bounds, converter) = RangePartitioner::determine_bounds_for_rows( - sort_fields, - sampled_columns, - *num_output_partitions as i32, - ); - - self.range_boundaries = Some(bounds); - self.row_converter = Some(converter); - } + .for_each(|partition_id| partition_counters[*partition_id as usize] += 1); - let row_batch = self - .row_converter - .as_ref() - .unwrap() - .convert_columns(partition_arrays.as_slice())?; + println!("{:?}", partition_counters); - let partition_ids = &mut scratch.partition_ids[..partition_arrays[0].len()]; - row_batch.iter().enumerate().for_each(|(idx, row)| { - let search_result = self - .range_boundaries - .as_ref() - .unwrap() - .binary_search(&row.owned()); - let partition = search_result.unwrap_or_else(|idx| idx); - assert!(partition < *num_output_partitions); - partition_ids[idx] = partition as u32; - }); + // accumulate partition counters into partition ends + // e.g. partition counter: [1, 3, 2, 1, 0] => [1, 4, 6, 7, 7] + let partition_ends = partition_counters; + let mut accum = 0; + partition_ends.iter_mut().for_each(|v| { + *v += accum; + accum = *v; + }); + + // calculate partition row indices and partition starts + // e.g. partition ids: [3, 1, 1, 1, 2, 2, 0] will produce the following partition_row_indices + // and partition_starts arrays: + // + // partition_row_indices: [6, 1, 2, 3, 4, 5, 0] + // partition_starts: [0, 1, 4, 6, 7] + // + // partition_starts conceptually splits partition_row_indices into smaller slices. + // Each slice partition_row_indices[partition_starts[K]..partition_starts[K + 1]] contains the + // row indices of the input batch that are partitioned into partition K. For example, + // first partition 0 has one row index [6], partition 1 has row indices [1, 2, 3], etc. + let partition_row_indices = &mut scratch.partition_row_indices; + partition_row_indices.resize(input.num_rows(), 0); + for (index, partition_id) in partition_ids.iter().enumerate().rev() { + partition_ends[*partition_id as usize] -= 1; + let end = partition_ends[*partition_id as usize]; + partition_row_indices[end as usize] = index as u32; + } - println!("{:?}", partition_ids); + // after calculating, partition ends become partition starts + let partition_starts = partition_ends; + timer.stop(); + Ok::<(&Vec, &Vec), DataFusionError>(( + partition_starts, + partition_row_indices, + )) + }?; - todo!(); + self.buffer_partitioned_batch_may_spill( + input, + partition_row_indices, + partition_starts, + ) + .await?; + self.scratch = scratch; } other => { // this should be unreachable as long as the validation logic From 402f4467a54359737183c733a60b6c80e998847f Mon Sep 17 00:00:00 2001 From: Matt Butrovich Date: Fri, 6 Jun 2025 22:00:18 -0400 Subject: [PATCH 16/72] Cleanup. --- native/core/src/execution/shuffle/shuffle_writer.rs | 4 ---- .../main/scala/org/apache/comet/serde/QueryPlanSerde.scala | 3 --- .../comet/execution/shuffle/CometNativeShuffleWriter.scala | 4 ---- 3 files changed, 11 deletions(-) diff --git a/native/core/src/execution/shuffle/shuffle_writer.rs b/native/core/src/execution/shuffle/shuffle_writer.rs index 10e8b8d0d7..8e0f39bc8b 100644 --- a/native/core/src/execution/shuffle/shuffle_writer.rs +++ b/native/core/src/execution/shuffle/shuffle_writer.rs @@ -580,8 +580,6 @@ impl MultiPartitionShuffleRepartitioner { partition_ids[idx] = partition as u32; }); - println!("{:?}", partition_ids); - // count each partition size, while leaving the last extra element as 0 let partition_counters = &mut scratch.partition_starts; partition_counters.resize(num_output_partitions + 1, 0); @@ -590,8 +588,6 @@ impl MultiPartitionShuffleRepartitioner { .iter() .for_each(|partition_id| partition_counters[*partition_id as usize] += 1); - println!("{:?}", partition_counters); - // accumulate partition counters into partition ends // e.g. partition counter: [1, 3, 2, 1, 0] => [1, 4, 6, 7, 7] let partition_ends = partition_counters; diff --git a/spark/src/main/scala/org/apache/comet/serde/QueryPlanSerde.scala b/spark/src/main/scala/org/apache/comet/serde/QueryPlanSerde.scala index a0da46fb66..9226dcc4ae 100644 --- a/spark/src/main/scala/org/apache/comet/serde/QueryPlanSerde.scala +++ b/spark/src/main/scala/org/apache/comet/serde/QueryPlanSerde.scala @@ -2897,9 +2897,6 @@ object QueryPlanSerde extends Logging with CometExprShim { case SinglePartition => inputs.forall(attr => supportedShuffleDataType(attr.dataType)) case RangePartitioning(ordering, numPartitions) => - // scalastyle:off - println(s"trying to do RangePartitioning $ordering $numPartitions") - // scalastyle:on // TODO: Apply any expression constraints similar to HashPartitioning above. true case _ => 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 70a08a2ef9..e99d2d77c3 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 @@ -189,10 +189,6 @@ class CometNativeShuffleWriter[K, V]( case _: RangePartitioning => val rangePartitioning = outputPartitioning.asInstanceOf[RangePartitioning] - // scalastyle:off - println(rangePartitioning) - // scalastyle:on - val partitioning = PartitioningOuterClass.RangePartition.newBuilder() partitioning.setNumPartitions(outputPartitioning.numPartitions) From c9b49f90b0189390697e461a1597cbb49b1377f0 Mon Sep 17 00:00:00 2001 From: Matt Butrovich Date: Fri, 6 Jun 2025 22:44:44 -0400 Subject: [PATCH 17/72] More testings. --- .../src/execution/shuffle/shuffle_writer.rs | 65 +++++++++++-------- 1 file changed, 38 insertions(+), 27 deletions(-) diff --git a/native/core/src/execution/shuffle/shuffle_writer.rs b/native/core/src/execution/shuffle/shuffle_writer.rs index 8e0f39bc8b..24ce7f5e4d 100644 --- a/native/core/src/execution/shuffle/shuffle_writer.rs +++ b/native/core/src/execution/shuffle/shuffle_writer.rs @@ -1300,7 +1300,8 @@ mod test { use datafusion::datasource::source::DataSourceExec; use datafusion::execution::config::SessionConfig; use datafusion::execution::runtime_env::RuntimeEnvBuilder; - use datafusion::physical_expr::expressions::Column; + use datafusion::physical_expr::expressions::{col, Column}; + use datafusion::physical_expr::PhysicalSortExpr; use datafusion::physical_plan::common::collect; use datafusion::prelude::SessionContext; use std::io::Cursor; @@ -1421,34 +1422,44 @@ mod test { ) { let batch = create_batch(batch_size); - let batches = (0..num_batches).map(|_| batch.clone()).collect::>(); - - let partitions = &[batches]; - let exec = ShuffleWriterExec::try_new( - Arc::new(DataSourceExec::new(Arc::new( - MemorySourceConfig::try_new(partitions, batch.schema(), None).unwrap(), - ))), + for partitioning in vec![ CometPartitioning::Hash(vec![Arc::new(Column::new("a", 0))], num_partitions), - CompressionCodec::Zstd(1), - "/tmp/data.out".to_string(), - "/tmp/index.out".to_string(), - false, - ) - .unwrap(); + CometPartitioning::RangePartitioning( + LexOrdering::new(vec![PhysicalSortExpr::new_default( + col("a", batch.schema().as_ref()).unwrap(), + )]), + num_partitions, + ), + ] { + let batches = (0..num_batches).map(|_| batch.clone()).collect::>(); - // 10MB memory should be enough for running this test - let config = SessionConfig::new(); - let mut runtime_env_builder = RuntimeEnvBuilder::new(); - runtime_env_builder = match memory_limit { - Some(limit) => runtime_env_builder.with_memory_limit(limit, 1.0), - None => runtime_env_builder, - }; - let runtime_env = Arc::new(runtime_env_builder.build().unwrap()); - let ctx = SessionContext::new_with_config_rt(config, runtime_env); - let task_ctx = ctx.task_ctx(); - let stream = exec.execute(0, task_ctx).unwrap(); - let rt = Runtime::new().unwrap(); - rt.block_on(collect(stream)).unwrap(); + let partitions = &[batches]; + let exec = ShuffleWriterExec::try_new( + Arc::new(DataSourceExec::new(Arc::new( + MemorySourceConfig::try_new(partitions, batch.schema(), None).unwrap(), + ))), + partitioning, + CompressionCodec::Zstd(1), + "/tmp/data.out".to_string(), + "/tmp/index.out".to_string(), + false, + ) + .unwrap(); + + // 10MB memory should be enough for running this test + let config = SessionConfig::new(); + let mut runtime_env_builder = RuntimeEnvBuilder::new(); + runtime_env_builder = match memory_limit { + Some(limit) => runtime_env_builder.with_memory_limit(limit, 1.0), + None => runtime_env_builder, + }; + let runtime_env = Arc::new(runtime_env_builder.build().unwrap()); + let ctx = SessionContext::new_with_config_rt(config, runtime_env); + let task_ctx = ctx.task_ctx(); + let stream = exec.execute(0, task_ctx).unwrap(); + let rt = Runtime::new().unwrap(); + rt.block_on(collect(stream)).unwrap(); + } } fn create_batch(batch_size: usize) -> RecordBatch { From 509ef66c06e2469baf3e678a6b349dee907347ac Mon Sep 17 00:00:00 2001 From: Matt Butrovich Date: Fri, 6 Jun 2025 22:49:57 -0400 Subject: [PATCH 18/72] clean up warnings. --- .../core/src/execution/shuffle/range_partitioner.rs | 7 +------ native/core/src/execution/shuffle/shuffle_writer.rs | 12 +++++------- 2 files changed, 6 insertions(+), 13 deletions(-) diff --git a/native/core/src/execution/shuffle/range_partitioner.rs b/native/core/src/execution/shuffle/range_partitioner.rs index d131620ca7..1e4b346ee3 100644 --- a/native/core/src/execution/shuffle/range_partitioner.rs +++ b/native/core/src/execution/shuffle/range_partitioner.rs @@ -15,17 +15,12 @@ // specific language governing permissions and limitations // under the License. -use arrow::array::{ArrayRef, AsArray, RecordBatch, UInt64Array}; +use arrow::array::{ArrayRef, RecordBatch, UInt64Array}; use arrow::compute::take_record_batch; use arrow::row::{OwnedRow, Row, RowConverter, SortField}; use datafusion::common::HashSet; -use datafusion::physical_expr::expressions::col; -use datafusion::physical_expr::{LexOrdering, PhysicalSortExpr}; -use datafusion::physical_plan::sorts::sort::sort_batch; -use itertools::Itertools; use num::ToPrimitive; use rand::Rng; -use std::sync::Arc; pub struct RangePartitioner; diff --git a/native/core/src/execution/shuffle/shuffle_writer.rs b/native/core/src/execution/shuffle/shuffle_writer.rs index 24ce7f5e4d..11d12a3d39 100644 --- a/native/core/src/execution/shuffle/shuffle_writer.rs +++ b/native/core/src/execution/shuffle/shuffle_writer.rs @@ -17,16 +17,14 @@ //! Defines the External shuffle repartition plan. -use crate::execution::shuffle::range_partitioner; 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, partition, take, take_arrays}; -use arrow::datatypes::Schema; -use arrow::row::{OwnedRow, RowConverter, Rows, SortField}; +use arrow::compute::{interleave_record_batch, take}; +use arrow::row::{OwnedRow, RowConverter, SortField}; use async_trait::async_trait; use datafusion::common::utils::proxy::VecAllocExt; -use datafusion::physical_expr::{EquivalenceProperties, LexOrdering, PhysicalExpr}; +use datafusion::physical_expr::EquivalenceProperties; use datafusion::physical_plan::execution_plan::{Boundedness, EmissionType}; use datafusion::physical_plan::EmptyRecordBatchStream; use datafusion::{ @@ -43,8 +41,8 @@ use datafusion::{ BaselineMetrics, Count, ExecutionPlanMetricsSet, MetricBuilder, MetricsSet, Time, }, stream::RecordBatchStreamAdapter, - DisplayAs, DisplayFormatType, ExecutionPlan, Partitioning, PlanProperties, - SendableRecordBatchStream, Statistics, + DisplayAs, DisplayFormatType, ExecutionPlan, PlanProperties, SendableRecordBatchStream, + Statistics, }, }; use datafusion_comet_spark_expr::hash_funcs::murmur3::create_murmur3_hashes; From 13b542d8cc74b0d4a8d85fdf03b7013d0d900041 Mon Sep 17 00:00:00 2001 From: Matt Butrovich Date: Fri, 6 Jun 2025 22:53:32 -0400 Subject: [PATCH 19/72] clean up warnings. --- native/core/benches/shuffle_writer.rs | 8 +++++--- native/core/src/execution/shuffle/comet_partitioning.rs | 6 +++--- native/core/src/execution/shuffle/range_partitioner.rs | 6 +++--- native/core/src/execution/shuffle/shuffle_writer.rs | 8 +++----- 4 files changed, 14 insertions(+), 14 deletions(-) diff --git a/native/core/benches/shuffle_writer.rs b/native/core/benches/shuffle_writer.rs index 0fd7502351..b1b5b39ed9 100644 --- a/native/core/benches/shuffle_writer.rs +++ b/native/core/benches/shuffle_writer.rs @@ -18,11 +18,13 @@ use arrow::array::builder::{Date32Builder, Decimal128Builder, Int32Builder}; use arrow::array::{builder::StringBuilder, RecordBatch}; use arrow::datatypes::{DataType, Field, Schema}; -use comet::execution::shuffle::{CompressionCodec, ShuffleBlockWriter, ShuffleWriterExec}; +use comet::execution::shuffle::{ + CometPartitioning, CompressionCodec, ShuffleBlockWriter, ShuffleWriterExec, +}; use criterion::{criterion_group, criterion_main, Criterion}; use datafusion::datasource::memory::MemorySourceConfig; use datafusion::datasource::source::DataSourceExec; -use datafusion::physical_expr::{expressions::Column, Partitioning}; +use datafusion::physical_expr::expressions::Column; use datafusion::physical_plan::metrics::Time; use datafusion::{ physical_plan::{common::collect, ExecutionPlan}, @@ -89,7 +91,7 @@ fn create_shuffle_writer_exec(compression_codec: CompressionCodec) -> ShuffleWri Arc::new(DataSourceExec::new(Arc::new( MemorySourceConfig::try_new(partitions, Arc::clone(&schema), None).unwrap(), ))), - Partitioning::Hash(vec![Arc::new(Column::new("a", 0))], 16), + CometPartitioning::Hash(vec![Arc::new(Column::new("a", 0))], 16), compression_codec, "/tmp/data.out".to_string(), "/tmp/index.out".to_string(), diff --git a/native/core/src/execution/shuffle/comet_partitioning.rs b/native/core/src/execution/shuffle/comet_partitioning.rs index d4f2337c22..22f362b0a0 100644 --- a/native/core/src/execution/shuffle/comet_partitioning.rs +++ b/native/core/src/execution/shuffle/comet_partitioning.rs @@ -56,9 +56,9 @@ impl From for CometPartitioning { } } -impl Into for CometPartitioning { - fn into(self) -> Partitioning { - match self { +impl From for Partitioning { + fn from(val: CometPartitioning) -> Self { + match val { CometPartitioning::RoundRobinBatch(partitions) => { Partitioning::RoundRobinBatch(partitions) } diff --git a/native/core/src/execution/shuffle/range_partitioner.rs b/native/core/src/execution/shuffle/range_partitioner.rs index 1e4b346ee3..aa1b28050c 100644 --- a/native/core/src/execution/shuffle/range_partitioner.rs +++ b/native/core/src/execution/shuffle/range_partitioner.rs @@ -64,7 +64,7 @@ impl RangePartitioner { } } - let set: HashSet = reservoir.iter().map(|e| *e).collect(); + let set: HashSet = reservoir.iter().copied().collect(); assert_eq!(set.len(), reservoir.len()); reservoir @@ -181,9 +181,9 @@ mod test { fn reservoir_sample() { let batch = create_random_batch(100); let sample1 = RangePartitioner::reservoir_sample(batch.clone(), 150); - assert_eq!(batch, sample1.into()); + assert_eq!(batch, sample1); let sample2 = RangePartitioner::reservoir_sample(batch.clone(), 100); - assert_eq!(batch, sample2.into()); + assert_eq!(batch, sample2); let sample3 = RangePartitioner::reservoir_sample(batch.clone(), 10); assert_eq!(sample3.num_rows(), 10); } diff --git a/native/core/src/execution/shuffle/shuffle_writer.rs b/native/core/src/execution/shuffle/shuffle_writer.rs index 11d12a3d39..1719e66fb8 100644 --- a/native/core/src/execution/shuffle/shuffle_writer.rs +++ b/native/core/src/execution/shuffle/shuffle_writer.rs @@ -1299,7 +1299,7 @@ mod test { use datafusion::execution::config::SessionConfig; use datafusion::execution::runtime_env::RuntimeEnvBuilder; use datafusion::physical_expr::expressions::{col, Column}; - use datafusion::physical_expr::PhysicalSortExpr; + use datafusion::physical_expr::{LexOrdering, PhysicalSortExpr}; use datafusion::physical_plan::common::collect; use datafusion::prelude::SessionContext; use std::io::Cursor; @@ -1420,15 +1420,13 @@ mod test { ) { let batch = create_batch(batch_size); - for partitioning in vec![ - CometPartitioning::Hash(vec![Arc::new(Column::new("a", 0))], num_partitions), + 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(), )]), num_partitions, - ), - ] { + )] { let batches = (0..num_batches).map(|_| batch.clone()).collect::>(); let partitions = &[batches]; From e3a28e3323b5214e58e4644329ad5d06e349df6b Mon Sep 17 00:00:00 2001 From: Matt Butrovich Date: Sat, 7 Jun 2025 07:00:33 -0400 Subject: [PATCH 20/72] Update test. --- .../org/apache/comet/exec/CometNativeShuffleSuite.scala | 6 ++---- 1 file changed, 2 insertions(+), 4 deletions(-) diff --git a/spark/src/test/scala/org/apache/comet/exec/CometNativeShuffleSuite.scala b/spark/src/test/scala/org/apache/comet/exec/CometNativeShuffleSuite.scala index e1cbd7406a..3758989702 100644 --- a/spark/src/test/scala/org/apache/comet/exec/CometNativeShuffleSuite.scala +++ b/spark/src/test/scala/org/apache/comet/exec/CometNativeShuffleSuite.scala @@ -139,10 +139,8 @@ class CometNativeShuffleSuite extends CometTestBase with AdaptiveSparkPlanHelper .repartition(10, $"_1") .filter($"_1" > 1) - // Because the first exchange from the bottom is range exchange which native shuffle - // doesn't support. So Comet exec operators stop before the first exchange and thus - // there is no Comet exchange. - checkShuffleAnswer(shuffled2, 0) + // native shuffle supports RangePartitioning, so 2 Comet shuffle exchanges are expected + checkShuffleAnswer(shuffled2, 2) } } From ab03b5c1440b6090651f7c3507c4ea19292cfa8b Mon Sep 17 00:00:00 2001 From: Matt Butrovich Date: Sat, 7 Jun 2025 07:57:24 -0400 Subject: [PATCH 21/72] More testing. --- .../execution/shuffle/range_partitioner.rs | 154 ++++++++---------- 1 file changed, 71 insertions(+), 83 deletions(-) diff --git a/native/core/src/execution/shuffle/range_partitioner.rs b/native/core/src/execution/shuffle/range_partitioner.rs index aa1b28050c..0e19d119e3 100644 --- a/native/core/src/execution/shuffle/range_partitioner.rs +++ b/native/core/src/execution/shuffle/range_partitioner.rs @@ -15,11 +15,9 @@ // specific language governing permissions and limitations // under the License. -use arrow::array::{ArrayRef, RecordBatch, UInt64Array}; -use arrow::compute::take_record_batch; +use arrow::array::ArrayRef; use arrow::row::{OwnedRow, Row, RowConverter, SortField}; use datafusion::common::HashSet; -use num::ToPrimitive; use rand::Rng; pub struct RangePartitioner; @@ -29,17 +27,6 @@ impl RangePartitioner { // We use sample_size instead of k and input_length instead of n. // We use indices in the reservoir instead of actual values since we'll do one take() on the // input array at the end. - pub fn reservoir_sample(input: RecordBatch, sample_size: usize) -> RecordBatch { - // Build our indices array and then take the values from the input. - let indices = UInt64Array::from(Self::reservoir_sample_indices( - input.num_rows(), - sample_size, - )); - - // TODO: This bounds checks, probably not necessary. - take_record_batch(&input, &indices).unwrap() - } - pub fn reservoir_sample_indices(num_rows: usize, sample_size: usize) -> Vec { if num_rows <= sample_size { // Just return the original input since we can't create a bigger sample. @@ -70,41 +57,6 @@ impl RangePartitioner { reservoir } - // Adapted from org.apache.spark.RangePartitioner.determineBounds - fn determine_bounds(mut candidates: Vec<(T, f64)>, partitions: i32) -> Vec - where - T: PartialOrd + Copy, - { - candidates.sort_by(|a, b| a.0.partial_cmp(&b.0).unwrap()); - let num_candidates = candidates.len(); - let sum_weights = candidates - .iter() - .map(|x| x.1.to_f64().unwrap()) - .sum::(); - let step = sum_weights / partitions as f64; - let mut cumulative_weights = 0.0; - let mut target = step; - let mut bounds = Vec::with_capacity((partitions - 1) as usize); - let mut i = 0; - let mut j = 0; - let mut previous_bound: Option = None; - while (i < num_candidates) && (j < partitions - 1) { - let (key, weight) = &candidates[i]; - cumulative_weights += weight; - if cumulative_weights >= target { - // Skip duplicate values. - if previous_bound.is_none() || *key > previous_bound.unwrap() { - bounds.push(*key); - target += step; - j += 1; - previous_bound = Some(*key) - } - } - i += 1 - } - bounds - } - // Adapted from org.apache.spark.RangePartitioner.determineBounds pub fn determine_bounds_for_rows( sort_fields: Vec, @@ -156,23 +108,32 @@ impl RangePartitioner { #[cfg(test)] mod test { use super::*; - use arrow::array::{AsArray, Float64Array, Int32Array, RecordBatch}; - use arrow::datatypes::{DataType, Field, Int32Type, Schema}; - use datafusion::physical_expr::expressions::col; - use datafusion::physical_expr::{LexOrdering, PhysicalSortExpr}; - use datafusion::physical_plan::sorts::sort::sort_batch; + use arrow::array::{AsArray, Float64Array, Int32Array, RecordBatch, UInt64Array}; + use arrow::compute::take_record_batch; + use arrow::datatypes::DataType::Float64; + use arrow::datatypes::{DataType, Field, Float64Type, Int32Type, Schema}; + use datafusion::common::record_batch; + use itertools::Itertools; use std::sync::Arc; - #[test] - fn test_sort_record_batch() { - let batch = create_random_batch(10000); - let sample_size = 20; - let lex_ordering = LexOrdering::new(vec![PhysicalSortExpr::new_default( - col("a", batch.schema().as_ref()).unwrap(), - )]); - let batch = sort_batch(&batch, &lex_ordering, None).unwrap(); - let sample = RangePartitioner::reservoir_sample(batch, sample_size); - let sorted_sample = sort_batch(&sample, &lex_ordering, None).unwrap(); + fn sample_batch(input: RecordBatch, indices: Vec) -> RecordBatch { + let indices = UInt64Array::from(indices); + take_record_batch(&input, &indices).unwrap() + } + + fn check_indices(indices: &Vec, batch_size: usize, sample_size: usize) { + // sample indices size should never exceed the batch size + assert!(indices.len() <= batch_size); + assert_eq!(indices.len(), batch_size.min(sample_size)); + // Check that values are distinct and not out of bounds + let sorted_indices = indices.into_iter().sorted().collect_vec(); + sorted_indices + .iter() + .for_each(|&&idx| assert!(idx < batch_size as u64)); + assert_eq!( + sorted_indices.len(), + sorted_indices.iter().dedup().collect_vec().len() + ); } #[test] @@ -180,12 +141,18 @@ mod test { // "reservoirSampleAndCount" fn reservoir_sample() { let batch = create_random_batch(100); - let sample1 = RangePartitioner::reservoir_sample(batch.clone(), 150); - assert_eq!(batch, sample1); - let sample2 = RangePartitioner::reservoir_sample(batch.clone(), 100); - assert_eq!(batch, sample2); - let sample3 = RangePartitioner::reservoir_sample(batch.clone(), 10); - assert_eq!(sample3.num_rows(), 10); + // sample_size > batch.num_rows returns entire batch after sampling + let sample1_indices = RangePartitioner::reservoir_sample_indices(batch.num_rows(), 150); + check_indices(&sample1_indices, batch.num_rows(), 150); + assert_eq!(batch, sample_batch(batch.clone(), sample1_indices)); + // sample_size == batch.num_rows returns entire batch after sampling + let sample2_indices = RangePartitioner::reservoir_sample_indices(batch.num_rows(), 100); + check_indices(&sample2_indices, batch.num_rows(), 100); + assert_eq!(batch, sample_batch(batch.clone(), sample2_indices)); + // sample_size < batch.num_rows returns a random subset, so can't compare to original batch + let sample3_indices = RangePartitioner::reservoir_sample_indices(batch.num_rows(), 10); + check_indices(&sample3_indices, batch.num_rows(), 10); + assert_eq!(sample3_indices.len(), 10); } #[test] @@ -198,7 +165,8 @@ mod test { let batch = RecordBatch::try_new(Arc::clone(&schema), vec![array.clone()]).unwrap(); let mut counts: Vec = vec![0; array.len()]; for _i in 0..500 { - let result = RangePartitioner::reservoir_sample(batch.clone(), 1); + let indices = RangePartitioner::reservoir_sample_indices(batch.num_rows(), 1); + let result = sample_batch(batch.clone(), indices); assert_eq!(result.num_rows(), 1); counts[result.column(0).as_primitive::().value(0) as usize] += 1; } @@ -209,18 +177,38 @@ mod test { #[test] // org.apache.spark.PartitioningSuite // "RangePartitioner.determineBounds" - fn determine_bounds() { - let candidates = vec![ - (0.7, 2.0), - (0.1, 1.0), - (0.4, 1.0), - (0.3, 1.0), - (0.2, 1.0), - (0.5, 1.0), - (1.0, 3.0), - ]; - let result = RangePartitioner::determine_bounds(candidates, 3); - assert_eq!(vec![0.4, 0.7], result); + 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, row_converter) = + RangePartitioner::determine_bounds_for_rows(sort_fields, Vec::from(batch.columns()), 3); + + assert_eq!(rows.len(), 2); + + let selection = [rows[0].row(), rows[1].row()]; + let converted = row_converter.convert_rows(selection).unwrap(); + let c1 = converted[0].as_primitive::(); + assert_eq!(c1.values(), &[0.4, 0.7]); } fn create_random_batch(batch_size: i32) -> RecordBatch { From 1a887b5546da909236a934d38a61455525e3d432 Mon Sep 17 00:00:00 2001 From: Matt Butrovich Date: Sat, 7 Jun 2025 08:06:46 -0400 Subject: [PATCH 22/72] More testing. --- .../execution/shuffle/comet_partitioning.rs | 3 +- .../execution/shuffle/range_partitioner.rs | 34 ++++++++++++++----- .../src/execution/shuffle/shuffle_writer.rs | 6 ++-- 3 files changed, 32 insertions(+), 11 deletions(-) diff --git a/native/core/src/execution/shuffle/comet_partitioning.rs b/native/core/src/execution/shuffle/comet_partitioning.rs index 22f362b0a0..a369dffb40 100644 --- a/native/core/src/execution/shuffle/comet_partitioning.rs +++ b/native/core/src/execution/shuffle/comet_partitioning.rs @@ -25,7 +25,8 @@ pub enum CometPartitioning { /// Allocate rows based on a hash of one of more expressions and the specified number of /// partitions Hash(Vec>, usize), - /// + /// Allocate rows based on lexical order of one of more expressions and the specified number of + /// partitions RangePartitioning(LexOrdering, usize), /// Unknown partitioning scheme with a known number of partitions UnknownPartitioning(usize), diff --git a/native/core/src/execution/shuffle/range_partitioner.rs b/native/core/src/execution/shuffle/range_partitioner.rs index 0e19d119e3..41e9df4ac4 100644 --- a/native/core/src/execution/shuffle/range_partitioner.rs +++ b/native/core/src/execution/shuffle/range_partitioner.rs @@ -108,7 +108,7 @@ impl RangePartitioner { #[cfg(test)] mod test { use super::*; - use arrow::array::{AsArray, Float64Array, Int32Array, RecordBatch, UInt64Array}; + use arrow::array::{Array, AsArray, Float64Array, Int32Array, RecordBatch, UInt64Array}; use arrow::compute::take_record_batch; use arrow::datatypes::DataType::Float64; use arrow::datatypes::{DataType, Field, Float64Type, Int32Type, Schema}; @@ -126,10 +126,11 @@ mod test { assert!(indices.len() <= batch_size); assert_eq!(indices.len(), batch_size.min(sample_size)); // Check that values are distinct and not out of bounds - let sorted_indices = indices.into_iter().sorted().collect_vec(); - sorted_indices + indices .iter() - .for_each(|&&idx| assert!(idx < batch_size as u64)); + .for_each(|&idx| assert!(idx < batch_size as u64)); + // Check that values are distinct and not out of bounds + let sorted_indices = indices.into_iter().sorted().collect_vec(); assert_eq!( sorted_indices.len(), sorted_indices.iter().dedup().collect_vec().len() @@ -205,10 +206,27 @@ mod test { assert_eq!(rows.len(), 2); - let selection = [rows[0].row(), rows[1].row()]; - let converted = row_converter.convert_rows(selection).unwrap(); - let c1 = converted[0].as_primitive::(); - assert_eq!(c1.values(), &[0.4, 0.7]); + let bounds = row_converter + .convert_rows([rows[0].row(), rows[1].row()]) + .unwrap(); + let bounds_array = bounds[0].as_primitive::(); + assert_eq!(bounds_array.values(), &[0.4, 0.7]); + } + + #[test] + 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, row_converter) = + RangePartitioner::determine_bounds_for_rows(sort_fields, Vec::from(batch.columns()), 2); + + assert_eq!(rows.len(), 1); + + let bounds = row_converter.convert_rows([rows[0].row()]).unwrap(); + let bounds_array = bounds[0].as_primitive::(); + assert!(bounds_array.is_null(0)); } fn create_random_batch(batch_size: i32) -> RecordBatch { diff --git a/native/core/src/execution/shuffle/shuffle_writer.rs b/native/core/src/execution/shuffle/shuffle_writer.rs index 1719e66fb8..bddfd80178 100644 --- a/native/core/src/execution/shuffle/shuffle_writer.rs +++ b/native/core/src/execution/shuffle/shuffle_writer.rs @@ -1420,13 +1420,15 @@ mod test { ) { let batch = create_batch(batch_size); - for partitioning in [CometPartitioning::Hash(vec![Arc::new(Column::new("a", 0))], num_partitions), + 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(), )]), num_partitions, - )] { + ), + ] { let batches = (0..num_batches).map(|_| batch.clone()).collect::>(); let partitions = &[batches]; From 3fe5ad02c43acf64af8b45e181785aea28130038 Mon Sep 17 00:00:00 2001 From: Matt Butrovich Date: Sat, 7 Jun 2025 08:18:05 -0400 Subject: [PATCH 23/72] More testing. --- .../execution/shuffle/range_partitioner.rs | 34 +++++++++++++++++++ 1 file changed, 34 insertions(+) diff --git a/native/core/src/execution/shuffle/range_partitioner.rs b/native/core/src/execution/shuffle/range_partitioner.rs index 41e9df4ac4..20fec57112 100644 --- a/native/core/src/execution/shuffle/range_partitioner.rs +++ b/native/core/src/execution/shuffle/range_partitioner.rs @@ -213,6 +213,40 @@ mod test { assert_eq!(bounds_array.values(), &[0.4, 0.7]); } + #[test] + fn determine_bounds_sizes() { + let batch = record_batch!(("a", Float64, vec![Some(0.1), Some(0.2), Some(0.3),])).unwrap(); + + let sort_fields = vec![SortField::new(Float64)]; + + // num_partitions < sample size + let mut num_partitions = (batch.num_rows() - 1) as i32; + let (rows, _) = RangePartitioner::determine_bounds_for_rows( + sort_fields.clone(), + Vec::from(batch.columns()), + num_partitions, + ); + assert_eq!(rows.len() as i32, num_partitions - 1); + + // num_partitions == sample size + num_partitions = batch.num_rows() as i32; + let (rows, _) = RangePartitioner::determine_bounds_for_rows( + sort_fields.clone(), + Vec::from(batch.columns()), + num_partitions, + ); + assert_eq!(rows.len() as i32, num_partitions - 1); + + // num_partitions > sample size + num_partitions = (batch.num_rows() + 1) as i32; + let (rows, _) = RangePartitioner::determine_bounds_for_rows( + sort_fields.clone(), + Vec::from(batch.columns()), + num_partitions, + ); + assert_eq!(rows.len(), batch.num_rows()); + } + #[test] fn determine_bounds_with_nulls() { let batch = record_batch!(("a", Float64, vec![None, None, Some(0.1),])).unwrap(); From cb5931dd401010cbbd2fccd897a98e6d9ad63cf3 Mon Sep 17 00:00:00 2001 From: Matt Butrovich Date: Sat, 7 Jun 2025 09:12:52 -0400 Subject: [PATCH 24/72] Update shuffle_writer benchmark. --- native/core/benches/shuffle_writer.rs | 67 ++++++++++++++++++++------- 1 file changed, 49 insertions(+), 18 deletions(-) diff --git a/native/core/benches/shuffle_writer.rs b/native/core/benches/shuffle_writer.rs index b1b5b39ed9..246154804e 100644 --- a/native/core/benches/shuffle_writer.rs +++ b/native/core/benches/shuffle_writer.rs @@ -24,7 +24,8 @@ use comet::execution::shuffle::{ use criterion::{criterion_group, criterion_main, Criterion}; use datafusion::datasource::memory::MemorySourceConfig; use datafusion::datasource::source::DataSourceExec; -use datafusion::physical_expr::expressions::Column; +use datafusion::physical_expr::expressions::{col, Column}; +use datafusion::physical_expr::{LexOrdering, PhysicalSortExpr}; use datafusion::physical_plan::metrics::Time; use datafusion::{ physical_plan::{common::collect, ExecutionPlan}, @@ -44,20 +45,18 @@ fn criterion_benchmark(c: &mut Criterion) { CompressionCodec::Zstd(1), CompressionCodec::Zstd(6), ] { - for enable_fast_encoding in [true, false] { - let name = format!("shuffle_writer: write encoded (enable_fast_encoding={enable_fast_encoding}, compression={compression_codec:?})"); - group.bench_function(name, |b| { - let mut buffer = vec![]; - let ipc_time = Time::default(); - let w = ShuffleBlockWriter::try_new(&batch.schema(), compression_codec.clone()) - .unwrap(); - b.iter(|| { - buffer.clear(); - let mut cursor = Cursor::new(&mut buffer); - w.write_batch(&batch, &mut cursor, &ipc_time).unwrap(); - }); + let name = format!("shuffle_writer: write encoded (compression={compression_codec:?})"); + group.bench_function(name, |b| { + let mut buffer = vec![]; + let ipc_time = Time::default(); + let w = + ShuffleBlockWriter::try_new(&batch.schema(), compression_codec.clone()).unwrap(); + b.iter(|| { + buffer.clear(); + let mut cursor = Cursor::new(&mut buffer); + w.write_batch(&batch, &mut cursor, &ipc_time).unwrap(); }); - } + }); } for compression_codec in [ @@ -68,10 +67,39 @@ fn criterion_benchmark(c: &mut Criterion) { CompressionCodec::Zstd(6), ] { group.bench_function( - format!("shuffle_writer: end to end (compression = {compression_codec:?}"), + format!("shuffle_writer: end to end (compression = {compression_codec:?})"), |b| { let ctx = SessionContext::new(); - let exec = create_shuffle_writer_exec(compression_codec.clone()); + let exec = create_shuffle_writer_exec( + compression_codec.clone(), + CometPartitioning::Hash(vec![Arc::new(Column::new("a", 0))], 16), + ); + b.iter(|| { + let task_ctx = ctx.task_ctx(); + let stream = exec.execute(0, task_ctx).unwrap(); + let rt = Runtime::new().unwrap(); + rt.block_on(collect(stream)).unwrap(); + }); + }, + ); + } + + 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(), + )]), + 16, + ), + ] { + let compression_codec = CompressionCodec::None; + group.bench_function( + format!("shuffle_writer: end to end (partitioning={partitioning:?})"), + |b| { + let ctx = SessionContext::new(); + let exec = + create_shuffle_writer_exec(compression_codec.clone(), partitioning.clone()); b.iter(|| { let task_ctx = ctx.task_ctx(); let stream = exec.execute(0, task_ctx).unwrap(); @@ -83,7 +111,10 @@ fn criterion_benchmark(c: &mut Criterion) { } } -fn create_shuffle_writer_exec(compression_codec: CompressionCodec) -> ShuffleWriterExec { +fn create_shuffle_writer_exec( + compression_codec: CompressionCodec, + partitioning: CometPartitioning, +) -> ShuffleWriterExec { let batches = create_batches(8192, 10); let schema = batches[0].schema(); let partitions = &[batches]; @@ -91,7 +122,7 @@ fn create_shuffle_writer_exec(compression_codec: CompressionCodec) -> ShuffleWri Arc::new(DataSourceExec::new(Arc::new( MemorySourceConfig::try_new(partitions, Arc::clone(&schema), None).unwrap(), ))), - CometPartitioning::Hash(vec![Arc::new(Column::new("a", 0))], 16), + partitioning, compression_codec, "/tmp/data.out".to_string(), "/tmp/index.out".to_string(), From 45cafd20dbca6cdff0da8e666112edd9e85b64c1 Mon Sep 17 00:00:00 2001 From: Matt Butrovich Date: Sat, 7 Jun 2025 09:42:30 -0400 Subject: [PATCH 25/72] Remove assertion in tight loop. --- native/core/src/execution/shuffle/shuffle_writer.rs | 1 - 1 file changed, 1 deletion(-) diff --git a/native/core/src/execution/shuffle/shuffle_writer.rs b/native/core/src/execution/shuffle/shuffle_writer.rs index bddfd80178..9cb5216908 100644 --- a/native/core/src/execution/shuffle/shuffle_writer.rs +++ b/native/core/src/execution/shuffle/shuffle_writer.rs @@ -574,7 +574,6 @@ impl MultiPartitionShuffleRepartitioner { .unwrap() .binary_search(&row.owned()); let partition = search_result.unwrap_or_else(|idx| idx); - assert!(partition < *num_output_partitions); partition_ids[idx] = partition as u32; }); From 842ab428d424d4d10da999480c13a9daedef13a7 Mon Sep 17 00:00:00 2001 From: Matt Butrovich Date: Sat, 7 Jun 2025 15:20:40 -0400 Subject: [PATCH 26/72] Remove OwnedRows transformations to improve performance. --- .../execution/shuffle/range_partitioner.rs | 34 +++++----- .../src/execution/shuffle/shuffle_writer.rs | 66 ++++++++++++------- 2 files changed, 58 insertions(+), 42 deletions(-) diff --git a/native/core/src/execution/shuffle/range_partitioner.rs b/native/core/src/execution/shuffle/range_partitioner.rs index 20fec57112..541ef5425d 100644 --- a/native/core/src/execution/shuffle/range_partitioner.rs +++ b/native/core/src/execution/shuffle/range_partitioner.rs @@ -16,7 +16,7 @@ // under the License. use arrow::array::ArrayRef; -use arrow::row::{OwnedRow, Row, RowConverter, SortField}; +use arrow::row::{Row, RowConverter, SortField}; use datafusion::common::HashSet; use rand::Rng; @@ -62,7 +62,7 @@ impl RangePartitioner { sort_fields: Vec, sampled_columns: Vec, partitions: i32, - ) -> (Vec, RowConverter) { + ) -> (Vec, RowConverter) { let converter = RowConverter::new(sort_fields).unwrap(); let sampled_rows = converter .convert_columns(sampled_columns.as_slice()) @@ -74,7 +74,7 @@ impl RangePartitioner { let step = 1.0 / partitions as f64; let mut cumulative_weights = 0.0; let mut target = step; - let mut bounds_indices = Vec::with_capacity((partitions - 1) as usize); + let mut bounds_indices: Vec = Vec::with_capacity((partitions - 1) as usize); let mut i = 0; let mut j = 0; let mut previous_bound = None; @@ -86,7 +86,7 @@ impl RangePartitioner { // Skip duplicate values. if previous_bound.is_none() || key.1 > previous_bound.unwrap() { // bounds.push(key.1); - bounds_indices.push(key.0); + bounds_indices.push(key.0 as u64); target += step; j += 1; previous_bound = Some(key.1) @@ -95,13 +95,7 @@ impl RangePartitioner { i += 1 } - let mut result: Vec = Vec::with_capacity(bounds_indices.len()); - - bounds_indices - .iter() - .for_each(|idx| result.push(sampled_rows.row(*idx).owned())); - - (result, converter) + (bounds_indices, converter) } } @@ -201,15 +195,15 @@ mod test { let sort_fields = vec![SortField::new(Float64)]; - let (rows, row_converter) = + let (rows, _) = RangePartitioner::determine_bounds_for_rows(sort_fields, Vec::from(batch.columns()), 3); assert_eq!(rows.len(), 2); - let bounds = row_converter - .convert_rows([rows[0].row(), rows[1].row()]) - .unwrap(); - let bounds_array = bounds[0].as_primitive::(); + 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]); } @@ -253,13 +247,15 @@ mod test { let sort_fields = vec![SortField::new(Float64)]; - let (rows, row_converter) = + let (rows, _) = RangePartitioner::determine_bounds_for_rows(sort_fields, Vec::from(batch.columns()), 2); assert_eq!(rows.len(), 1); - let bounds = row_converter.convert_rows([rows[0].row()]).unwrap(); - let bounds_array = bounds[0].as_primitive::(); + 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)); } diff --git a/native/core/src/execution/shuffle/shuffle_writer.rs b/native/core/src/execution/shuffle/shuffle_writer.rs index 9cb5216908..c496c415e9 100644 --- a/native/core/src/execution/shuffle/shuffle_writer.rs +++ b/native/core/src/execution/shuffle/shuffle_writer.rs @@ -21,7 +21,7 @@ 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, take}; -use arrow::row::{OwnedRow, RowConverter, SortField}; +use arrow::row::{RowConverter, Rows, SortField}; use async_trait::async_trait; use datafusion::common::utils::proxy::VecAllocExt; use datafusion::physical_expr::EquivalenceProperties; @@ -113,7 +113,7 @@ impl ShuffleWriterExec { } impl DisplayAs for ShuffleWriterExec { - fn fmt_as(&self, t: DisplayFormatType, f: &mut std::fmt::Formatter) -> std::fmt::Result { + fn fmt_as(&self, t: DisplayFormatType, f: &mut Formatter) -> fmt::Result { match t { DisplayFormatType::Default | DisplayFormatType::Verbose => { write!( @@ -334,7 +334,7 @@ struct MultiPartitionShuffleRepartitioner { reservation: MemoryReservation, tracing_enabled: bool, /// RangePartitioning-specific state - range_boundaries: Option>, + bounds_rows: Option, row_converter: Option, } @@ -378,7 +378,10 @@ impl MultiPartitionShuffleRepartitioner { // initialization code is simply initializing the vectors to the desired size. // The initial values are not used. let scratch = ScratchSpace { - hashes_buf: vec![0; batch_size], + hashes_buf: match partitioning { + CometPartitioning::Hash(_, _) => vec![0; batch_size], + _ => vec![], + }, partition_ids: vec![0; batch_size], partition_row_indices: vec![0; batch_size], partition_starts: vec![0; num_output_partitions + 1], @@ -408,7 +411,7 @@ impl MultiPartitionShuffleRepartitioner { batch_size, reservation, tracing_enabled, - range_boundaries: None, + bounds_rows: None, row_converter: None, }) } @@ -526,9 +529,7 @@ impl MultiPartitionShuffleRepartitioner { .map(|expr| expr.expr.evaluate(&input)?.into_array(input.num_rows())) .collect::>>()?; - if self.range_boundaries.is_none() { - assert!(self.row_converter.is_none()); - + if self.row_converter.is_none() { // TODO: Adjust sample size. let sample_indices = UInt64Array::from( RangePartitioner::reservoir_sample_indices(input.num_rows(), 100), @@ -550,14 +551,26 @@ impl MultiPartitionShuffleRepartitioner { }) .collect(); - let (bounds, converter) = RangePartitioner::determine_bounds_for_rows( - sort_fields, - sampled_columns, - *num_output_partitions as i32, - ); + let (bounds_indices, row_converter) = + RangePartitioner::determine_bounds_for_rows( + sort_fields, + sampled_columns, + *num_output_partitions as i32, + ); + + let bounds_indices_array = UInt64Array::from(bounds_indices); + + let bounds_arrays = partition_arrays + .iter() + .map(|c| take(c, &bounds_indices_array, None)) + .collect::, _>>()?; - self.range_boundaries = Some(bounds); - self.row_converter = Some(converter); + self.bounds_rows = Some( + row_converter + .convert_columns(bounds_arrays.as_slice()) + .unwrap(), + ); + self.row_converter = Some(row_converter); } let row_batch = self @@ -567,14 +580,21 @@ impl MultiPartitionShuffleRepartitioner { .convert_columns(partition_arrays.as_slice())?; let partition_ids = &mut scratch.partition_ids[..partition_arrays[0].len()]; - row_batch.iter().enumerate().for_each(|(idx, row)| { - let search_result = self - .range_boundaries - .as_ref() - .unwrap() - .binary_search(&row.owned()); - let partition = search_result.unwrap_or_else(|idx| idx); - partition_ids[idx] = partition as u32; + + let partition_bounds = self.bounds_rows.as_ref().unwrap(); + + row_batch.iter().enumerate().for_each(|(row_idx, row)| { + let mut partition_id = 0; + // TODO: binary search + for bound in partition_bounds { + if row < bound { + partition_id += 1; + } else { + break; + } + } + assert!(partition_id < *num_output_partitions); + partition_ids[row_idx] = partition_id as u32; }); // count each partition size, while leaving the last extra element as 0 From 037daae10387cc45e5e685af0c9ad2894c1deee2 Mon Sep 17 00:00:00 2001 From: Matt Butrovich Date: Sat, 7 Jun 2025 15:56:28 -0400 Subject: [PATCH 27/72] Fix logic. --- native/core/src/execution/shuffle/shuffle_writer.rs | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/native/core/src/execution/shuffle/shuffle_writer.rs b/native/core/src/execution/shuffle/shuffle_writer.rs index c496c415e9..811f3a8d62 100644 --- a/native/core/src/execution/shuffle/shuffle_writer.rs +++ b/native/core/src/execution/shuffle/shuffle_writer.rs @@ -587,13 +587,12 @@ impl MultiPartitionShuffleRepartitioner { let mut partition_id = 0; // TODO: binary search for bound in partition_bounds { - if row < bound { + if row >= bound { partition_id += 1; } else { break; } } - assert!(partition_id < *num_output_partitions); partition_ids[row_idx] = partition_id as u32; }); From caa7cc24a60ca21617be9fcc1a0e610f549d0dce Mon Sep 17 00:00:00 2001 From: Matt Butrovich Date: Sun, 8 Jun 2025 07:15:33 -0400 Subject: [PATCH 28/72] reservoir_sample_fuzz test, remove HashSet validation from reservoir_sample_indices, take_arrays instead of take. --- .../execution/shuffle/range_partitioner.rs | 23 ++++++++++++++---- .../src/execution/shuffle/shuffle_writer.rs | 24 ++++++++++++------- 2 files changed, 34 insertions(+), 13 deletions(-) diff --git a/native/core/src/execution/shuffle/range_partitioner.rs b/native/core/src/execution/shuffle/range_partitioner.rs index 541ef5425d..307928be51 100644 --- a/native/core/src/execution/shuffle/range_partitioner.rs +++ b/native/core/src/execution/shuffle/range_partitioner.rs @@ -17,7 +17,6 @@ use arrow::array::ArrayRef; use arrow::row::{Row, RowConverter, SortField}; -use datafusion::common::HashSet; use rand::Rng; pub struct RangePartitioner; @@ -51,9 +50,6 @@ impl RangePartitioner { } } - let set: HashSet = reservoir.iter().copied().collect(); - assert_eq!(set.len(), reservoir.len()); - reservoir } @@ -131,6 +127,25 @@ mod test { ); } + #[test] + fn reservoir_sample_fuzz() { + let mut rng = rand::rng(); + + for _ in 0..1024 { + let batch_size: usize = rng.random_range(0..=8192); + let sample_size: usize = rng.random_range(0..=8192); + let reservoir = RangePartitioner::reservoir_sample_indices(batch_size, sample_size); + + assert_eq!(reservoir.len(), sample_size.min(batch_size)); + + let mut set: HashSet = HashSet::with_capacity(sample_size); + reservoir.iter().for_each(|&idx| { + assert!(idx < batch_size as u64); + assert!(set.insert(idx)); + }); + } + } + #[test] // org.apache.spark.util.random.SamplingUtilsSuite // "reservoirSampleAndCount" diff --git a/native/core/src/execution/shuffle/shuffle_writer.rs b/native/core/src/execution/shuffle/shuffle_writer.rs index 811f3a8d62..09800d5922 100644 --- a/native/core/src/execution/shuffle/shuffle_writer.rs +++ b/native/core/src/execution/shuffle/shuffle_writer.rs @@ -20,7 +20,7 @@ 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, take}; +use arrow::compute::{interleave_record_batch, take_arrays, TakeOptions}; use arrow::row::{RowConverter, Rows, SortField}; use async_trait::async_trait; use datafusion::common::utils::proxy::VecAllocExt; @@ -535,10 +535,13 @@ impl MultiPartitionShuffleRepartitioner { RangePartitioner::reservoir_sample_indices(input.num_rows(), 100), ); - let sampled_columns = partition_arrays - .iter() - .map(|c| take(c, &sample_indices, None)) - .collect::, _>>()?; + let sampled_columns = take_arrays( + &partition_arrays, + &sample_indices, + Some(TakeOptions { + check_bounds: false, + }), + )?; let sort_fields: Vec = partition_arrays .iter() @@ -560,10 +563,13 @@ impl MultiPartitionShuffleRepartitioner { let bounds_indices_array = UInt64Array::from(bounds_indices); - let bounds_arrays = partition_arrays - .iter() - .map(|c| take(c, &bounds_indices_array, None)) - .collect::, _>>()?; + let bounds_arrays = take_arrays( + &partition_arrays, + &bounds_indices_array, + Some(TakeOptions { + check_bounds: false, + }), + )?; self.bounds_rows = Some( row_converter From 7c5875453d4edcd96d42f9410082af44714301f0 Mon Sep 17 00:00:00 2001 From: Matt Butrovich Date: Sun, 8 Jun 2025 07:19:18 -0400 Subject: [PATCH 29/72] Fix missing include. --- native/core/src/execution/shuffle/range_partitioner.rs | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/native/core/src/execution/shuffle/range_partitioner.rs b/native/core/src/execution/shuffle/range_partitioner.rs index 307928be51..371cad0e89 100644 --- a/native/core/src/execution/shuffle/range_partitioner.rs +++ b/native/core/src/execution/shuffle/range_partitioner.rs @@ -27,6 +27,8 @@ impl RangePartitioner { // We use indices in the reservoir instead of actual values since we'll do one take() on the // input array at the end. pub fn reservoir_sample_indices(num_rows: usize, sample_size: usize) -> Vec { + assert!(sample_size > 0); + if num_rows <= sample_size { // Just return the original input since we can't create a bigger sample. return (0..num_rows as u64).collect(); @@ -102,7 +104,7 @@ mod test { use arrow::compute::take_record_batch; use arrow::datatypes::DataType::Float64; use arrow::datatypes::{DataType, Field, Float64Type, Int32Type, Schema}; - use datafusion::common::record_batch; + use datafusion::common::{record_batch, HashSet}; use itertools::Itertools; use std::sync::Arc; @@ -133,7 +135,7 @@ mod test { for _ in 0..1024 { let batch_size: usize = rng.random_range(0..=8192); - let sample_size: usize = rng.random_range(0..=8192); + let sample_size: usize = rng.random_range(1..=8192); let reservoir = RangePartitioner::reservoir_sample_indices(batch_size, sample_size); assert_eq!(reservoir.len(), sample_size.min(batch_size)); From eb6fb038fc69614027f543d625d32cb21d7d4581 Mon Sep 17 00:00:00 2001 From: Matt Butrovich Date: Sun, 8 Jun 2025 07:53:29 -0400 Subject: [PATCH 30/72] determine_bounds_fuzz test. --- .../execution/shuffle/range_partitioner.rs | 32 +++++++++++++++++-- 1 file changed, 30 insertions(+), 2 deletions(-) diff --git a/native/core/src/execution/shuffle/range_partitioner.rs b/native/core/src/execution/shuffle/range_partitioner.rs index 371cad0e89..187f590f42 100644 --- a/native/core/src/execution/shuffle/range_partitioner.rs +++ b/native/core/src/execution/shuffle/range_partitioner.rs @@ -61,6 +61,8 @@ impl RangePartitioner { sampled_columns: Vec, partitions: i32, ) -> (Vec, RowConverter) { + assert!(partitions > 1); + let converter = RowConverter::new(sort_fields).unwrap(); let sampled_rows = converter .convert_columns(sampled_columns.as_slice()) @@ -133,7 +135,7 @@ mod test { fn reservoir_sample_fuzz() { let mut rng = rand::rng(); - for _ in 0..1024 { + for _ in 0..8192 { let batch_size: usize = rng.random_range(0..=8192); let sample_size: usize = rng.random_range(1..=8192); let reservoir = RangePartitioner::reservoir_sample_indices(batch_size, sample_size); @@ -258,6 +260,32 @@ mod test { assert_eq!(rows.len(), batch.num_rows()); } + #[test] + fn determine_bounds_fuzz() { + let mut rng = rand::rng(); + + let sort_fields = vec![SortField::new(Float64)]; + + for _ in 0..8192 { + let batch_size: i32 = rng.random_range(0..=8192); + let num_partitions: i32 = rng.random_range(2..1048576); + + let batch = create_random_batch(batch_size); + + let (rows, _) = RangePartitioner::determine_bounds_for_rows( + sort_fields.clone(), + Vec::from(batch.columns()), + num_partitions, + ); + + if batch_size < num_partitions { + assert_eq!(rows.len(), batch_size as usize); + } else { + assert_eq!(rows.len(), (num_partitions - 1) as usize); + } + } + } + #[test] fn determine_bounds_with_nulls() { let batch = record_batch!(("a", Float64, vec![None, None, Some(0.1),])).unwrap(); @@ -280,7 +308,7 @@ mod test { let mut rng = rand::rng(); let column: Vec = (0..batch_size).map(|_| rng.random::()).collect(); let array = Float64Array::from(column); - let schema = Arc::new(Schema::new(vec![Field::new("a", DataType::Float64, true)])); + let schema = Arc::new(Schema::new(vec![Field::new("a", Float64, true)])); RecordBatch::try_new(Arc::clone(&schema), vec![Arc::new(array)]).unwrap() } } From 3eca544e42cefeafd86eeeec0b467a7d9cb7b842 Mon Sep 17 00:00:00 2001 From: Matt Butrovich Date: Sun, 8 Jun 2025 08:11:28 -0400 Subject: [PATCH 31/72] More tests. --- .../execution/shuffle/range_partitioner.rs | 36 ++++++++++++++----- 1 file changed, 27 insertions(+), 9 deletions(-) diff --git a/native/core/src/execution/shuffle/range_partitioner.rs b/native/core/src/execution/shuffle/range_partitioner.rs index 187f590f42..6a8e9c1325 100644 --- a/native/core/src/execution/shuffle/range_partitioner.rs +++ b/native/core/src/execution/shuffle/range_partitioner.rs @@ -102,10 +102,10 @@ impl RangePartitioner { #[cfg(test)] mod test { use super::*; - use arrow::array::{Array, AsArray, Float64Array, Int32Array, RecordBatch, UInt64Array}; + use arrow::array::{Array, AsArray, Int32Array, Int64Array, RecordBatch, UInt64Array}; use arrow::compute::take_record_batch; - use arrow::datatypes::DataType::Float64; - use arrow::datatypes::{DataType, Field, Float64Type, Int32Type, Schema}; + use arrow::datatypes::DataType::{Float64, Int64}; + use arrow::datatypes::{DataType, Field, Float64Type, Int32Type, Int64Type, Schema}; use datafusion::common::{record_batch, HashSet}; use itertools::Itertools; use std::sync::Arc; @@ -135,7 +135,7 @@ mod test { fn reservoir_sample_fuzz() { let mut rng = rand::rng(); - for _ in 0..8192 { + for _ in 0..1000 { let batch_size: usize = rng.random_range(0..=8192); let sample_size: usize = rng.random_range(1..=8192); let reservoir = RangePartitioner::reservoir_sample_indices(batch_size, sample_size); @@ -264,9 +264,9 @@ mod test { fn determine_bounds_fuzz() { let mut rng = rand::rng(); - let sort_fields = vec![SortField::new(Float64)]; + let sort_fields = vec![SortField::new(Int64)]; - for _ in 0..8192 { + for _ in 0..1000 { let batch_size: i32 = rng.random_range(0..=8192); let num_partitions: i32 = rng.random_range(2..1048576); @@ -283,6 +283,24 @@ mod test { } else { assert_eq!(rows.len(), (num_partitions - 1) as usize); } + + let mut set: HashSet = HashSet::with_capacity(rows.len()); + rows.iter().for_each(|&idx| { + assert!(idx < batch_size as u64); + assert!(set.insert(idx)); + }); + + 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(); + + assert!(bounds_vec.is_sorted()); } } @@ -306,9 +324,9 @@ mod test { fn create_random_batch(batch_size: i32) -> RecordBatch { let mut rng = rand::rng(); - let column: Vec = (0..batch_size).map(|_| rng.random::()).collect(); - let array = Float64Array::from(column); - let schema = Arc::new(Schema::new(vec![Field::new("a", Float64, true)])); + let column: Vec = (0..batch_size).map(|_| rng.random::()).collect(); + 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() } } From f270ab015cc8ae117b64ec31c583c06b853e5620 Mon Sep 17 00:00:00 2001 From: Matt Butrovich Date: Sun, 8 Jun 2025 10:55:15 -0400 Subject: [PATCH 32/72] Checkpoint pulling partition indices for batch over to RangePartitioner with a test and binary search. --- .../execution/shuffle/range_partitioner.rs | 70 +++++++++++++++++-- 1 file changed, 63 insertions(+), 7 deletions(-) diff --git a/native/core/src/execution/shuffle/range_partitioner.rs b/native/core/src/execution/shuffle/range_partitioner.rs index 6a8e9c1325..de5bd889cf 100644 --- a/native/core/src/execution/shuffle/range_partitioner.rs +++ b/native/core/src/execution/shuffle/range_partitioner.rs @@ -16,7 +16,8 @@ // under the License. use arrow::array::ArrayRef; -use arrow::row::{Row, RowConverter, SortField}; +use arrow::row::{Row, RowConverter, Rows, SortField}; +use itertools::Itertools; use rand::Rng; pub struct RangePartitioner; @@ -55,6 +56,18 @@ impl RangePartitioner { reservoir } + pub fn partition_indices_for_batch( + row_batch: &Rows, + partition_bounds: &Rows, + partition_ids: &mut [u32], + ) { + let partition_bounds_vec = partition_bounds.iter().collect_vec(); + row_batch.iter().enumerate().for_each(|(row_idx, row)| { + partition_ids[row_idx] = + partition_bounds_vec.partition_point(|bound| *bound <= row) as u32 + }); + } + // Adapted from org.apache.spark.RangePartitioner.determineBounds pub fn determine_bounds_for_rows( sort_fields: Vec, @@ -131,6 +144,39 @@ mod test { ); } + #[test] + 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 = vec![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(); + + RangePartitioner::partition_indices_for_batch( + &input_rows, + &bounds_rows, + &mut partition_ids, + ); + + partition_ids + .iter() + .for_each(|&partition_id| partition_counts[partition_id as usize] += 1); + + partition_counts + .iter() + .for_each(|&partition_count| assert!(partition_count > 700)); + + partition_counts.fill(0); + } + #[test] fn reservoir_sample_fuzz() { let mut rng = rand::rng(); @@ -154,7 +200,7 @@ mod test { // org.apache.spark.util.random.SamplingUtilsSuite // "reservoirSampleAndCount" fn reservoir_sample() { - let batch = create_random_batch(100); + let batch = create_random_batch(100, false, None); // sample_size > batch.num_rows returns entire batch after sampling let sample1_indices = RangePartitioner::reservoir_sample_indices(batch.num_rows(), 150); check_indices(&sample1_indices, batch.num_rows(), 150); @@ -267,10 +313,10 @@ mod test { let sort_fields = vec![SortField::new(Int64)]; for _ in 0..1000 { - let batch_size: i32 = rng.random_range(0..=8192); + let batch_size = rng.random_range(0..=8192); let num_partitions: i32 = rng.random_range(2..1048576); - let batch = create_random_batch(batch_size); + let batch = create_random_batch(batch_size, false, None); let (rows, _) = RangePartitioner::determine_bounds_for_rows( sort_fields.clone(), @@ -278,7 +324,7 @@ mod test { num_partitions, ); - if batch_size < num_partitions { + if batch_size < num_partitions as u32 { assert_eq!(rows.len(), batch_size as usize); } else { assert_eq!(rows.len(), (num_partitions - 1) as usize); @@ -322,9 +368,19 @@ mod test { assert!(bounds_array.is_null(0)); } - fn create_random_batch(batch_size: i32) -> RecordBatch { + fn create_random_batch(batch_size: u32, sort: bool, range: Option<(i64, i64)>) -> RecordBatch { let mut rng = rand::rng(); - let column: Vec = (0..batch_size).map(|_| rng.random::()).collect(); + 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() From ed1f8948ffe095c6db845ada1b1778d0f6bcd388 Mon Sep 17 00:00:00 2001 From: Matt Butrovich Date: Sun, 8 Jun 2025 10:57:36 -0400 Subject: [PATCH 33/72] Checkpoint before adding more tests and trying to refactor shuffle_writer. --- native/core/src/execution/shuffle/range_partitioner.rs | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/native/core/src/execution/shuffle/range_partitioner.rs b/native/core/src/execution/shuffle/range_partitioner.rs index de5bd889cf..038c6a7797 100644 --- a/native/core/src/execution/shuffle/range_partitioner.rs +++ b/native/core/src/execution/shuffle/range_partitioner.rs @@ -17,7 +17,6 @@ use arrow::array::ArrayRef; use arrow::row::{Row, RowConverter, Rows, SortField}; -use itertools::Itertools; use rand::Rng; pub struct RangePartitioner; @@ -58,10 +57,9 @@ impl RangePartitioner { pub fn partition_indices_for_batch( row_batch: &Rows, - partition_bounds: &Rows, + partition_bounds_vec: &Vec, partition_ids: &mut [u32], ) { - let partition_bounds_vec = partition_bounds.iter().collect_vec(); row_batch.iter().enumerate().for_each(|(row_idx, row)| { partition_ids[row_idx] = partition_bounds_vec.partition_point(|bound| *bound <= row) as u32 @@ -160,9 +158,11 @@ mod test { let bounds_rows = row_converter.convert_columns(bounds.columns()).unwrap(); + let bounds_rows_vec = bounds_rows.iter().collect_vec(); + RangePartitioner::partition_indices_for_batch( &input_rows, - &bounds_rows, + &bounds_rows_vec, &mut partition_ids, ); From 0c6910b8c71edd6f2b3116ef2a65cb029973413b Mon Sep 17 00:00:00 2001 From: Matt Butrovich Date: Sun, 8 Jun 2025 11:16:15 -0400 Subject: [PATCH 34/72] shuffle writer uses external partition mapper. --- .../execution/shuffle/range_partitioner.rs | 2 -- .../src/execution/shuffle/shuffle_writer.rs | 22 +++++++------------ 2 files changed, 8 insertions(+), 16 deletions(-) diff --git a/native/core/src/execution/shuffle/range_partitioner.rs b/native/core/src/execution/shuffle/range_partitioner.rs index 038c6a7797..a503b5087b 100644 --- a/native/core/src/execution/shuffle/range_partitioner.rs +++ b/native/core/src/execution/shuffle/range_partitioner.rs @@ -173,8 +173,6 @@ mod test { partition_counts .iter() .for_each(|&partition_count| assert!(partition_count > 700)); - - partition_counts.fill(0); } #[test] diff --git a/native/core/src/execution/shuffle/shuffle_writer.rs b/native/core/src/execution/shuffle/shuffle_writer.rs index 09800d5922..5667208500 100644 --- a/native/core/src/execution/shuffle/shuffle_writer.rs +++ b/native/core/src/execution/shuffle/shuffle_writer.rs @@ -587,20 +587,14 @@ impl MultiPartitionShuffleRepartitioner { let partition_ids = &mut scratch.partition_ids[..partition_arrays[0].len()]; - let partition_bounds = self.bounds_rows.as_ref().unwrap(); - - row_batch.iter().enumerate().for_each(|(row_idx, row)| { - let mut partition_id = 0; - // TODO: binary search - for bound in partition_bounds { - if row >= bound { - partition_id += 1; - } else { - break; - } - } - partition_ids[row_idx] = partition_id as u32; - }); + // TODO: Try to cache this vector. + let bounds_rows_vec = self.bounds_rows.as_ref().unwrap().iter().collect_vec(); + + RangePartitioner::partition_indices_for_batch( + &row_batch, + &bounds_rows_vec, + partition_ids, + ); // count each partition size, while leaving the last extra element as 0 let partition_counters = &mut scratch.partition_starts; From 7a0bff9835047891b1ec826418bf568734f1ca48 Mon Sep 17 00:00:00 2001 From: Matt Butrovich Date: Sun, 8 Jun 2025 20:01:58 -0400 Subject: [PATCH 35/72] Move bounds generation to RangePartitioner. --- .../execution/shuffle/range_partitioner.rs | 74 ++++++++++++++++--- .../src/execution/shuffle/shuffle_writer.rs | 54 +++----------- 2 files changed, 74 insertions(+), 54 deletions(-) diff --git a/native/core/src/execution/shuffle/range_partitioner.rs b/native/core/src/execution/shuffle/range_partitioner.rs index a503b5087b..288706051d 100644 --- a/native/core/src/execution/shuffle/range_partitioner.rs +++ b/native/core/src/execution/shuffle/range_partitioner.rs @@ -15,8 +15,10 @@ // specific language governing permissions and limitations // under the License. -use arrow::array::ArrayRef; +use arrow::array::{ArrayRef, UInt64Array}; +use arrow::compute::{take_arrays, TakeOptions}; use arrow::row::{Row, RowConverter, Rows, SortField}; +use datafusion::physical_expr::LexOrdering; use rand::Rng; pub struct RangePartitioner; @@ -66,11 +68,65 @@ impl RangePartitioner { }); } + pub fn generate_bounds( + partition_arrays: &Vec, + lex_ordering: &LexOrdering, + num_output_partitions: usize, + num_rows: usize, // TODO: u16 + sample_size: usize, // TODO: u16 + ) -> (Rows, RowConverter) { + let sample_indices = UInt64Array::from(RangePartitioner::reservoir_sample_indices( + num_rows, + sample_size, + )); + + let sampled_columns = take_arrays( + partition_arrays, + &sample_indices, + Some(TakeOptions { + check_bounds: false, + }), + ) + .unwrap(); + + 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, + num_output_partitions, + ); + + let bounds_indices_array = UInt64Array::from(bounds_indices); + + let bounds_arrays = take_arrays( + partition_arrays, + &bounds_indices_array, + Some(TakeOptions { + check_bounds: false, + }), + ) + .unwrap(); + + ( + row_converter + .convert_columns(bounds_arrays.as_slice()) + .unwrap(), + row_converter, + ) + } + // Adapted from org.apache.spark.RangePartitioner.determineBounds pub fn determine_bounds_for_rows( sort_fields: Vec, sampled_columns: Vec, - partitions: i32, + partitions: usize, ) -> (Vec, RowConverter) { assert!(partitions > 1); @@ -85,7 +141,7 @@ impl RangePartitioner { 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) as usize); + let mut bounds_indices: Vec = Vec::with_capacity(partitions - 1); let mut i = 0; let mut j = 0; let mut previous_bound = None; @@ -277,25 +333,25 @@ mod test { let sort_fields = vec![SortField::new(Float64)]; // num_partitions < sample size - let mut num_partitions = (batch.num_rows() - 1) as i32; + let mut num_partitions = batch.num_rows() - 1; let (rows, _) = RangePartitioner::determine_bounds_for_rows( sort_fields.clone(), Vec::from(batch.columns()), num_partitions, ); - assert_eq!(rows.len() as i32, num_partitions - 1); + assert_eq!(rows.len(), num_partitions - 1); // num_partitions == sample size - num_partitions = batch.num_rows() as i32; + num_partitions = batch.num_rows(); let (rows, _) = RangePartitioner::determine_bounds_for_rows( sort_fields.clone(), Vec::from(batch.columns()), num_partitions, ); - assert_eq!(rows.len() as i32, num_partitions - 1); + assert_eq!(rows.len(), num_partitions - 1); // num_partitions > sample size - num_partitions = (batch.num_rows() + 1) as i32; + num_partitions = batch.num_rows() + 1; let (rows, _) = RangePartitioner::determine_bounds_for_rows( sort_fields.clone(), Vec::from(batch.columns()), @@ -312,7 +368,7 @@ mod test { for _ in 0..1000 { let batch_size = rng.random_range(0..=8192); - let num_partitions: i32 = rng.random_range(2..1048576); + let num_partitions = rng.random_range(2..1048576); let batch = create_random_batch(batch_size, false, None); diff --git a/native/core/src/execution/shuffle/shuffle_writer.rs b/native/core/src/execution/shuffle/shuffle_writer.rs index 5667208500..c183819ab7 100644 --- a/native/core/src/execution/shuffle/shuffle_writer.rs +++ b/native/core/src/execution/shuffle/shuffle_writer.rs @@ -20,8 +20,8 @@ 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, take_arrays, TakeOptions}; -use arrow::row::{RowConverter, Rows, SortField}; +use arrow::compute::interleave_record_batch; +use arrow::row::{RowConverter, Rows}; use async_trait::async_trait; use datafusion::common::utils::proxy::VecAllocExt; use datafusion::physical_expr::EquivalenceProperties; @@ -531,51 +531,15 @@ impl MultiPartitionShuffleRepartitioner { if self.row_converter.is_none() { // TODO: Adjust sample size. - let sample_indices = UInt64Array::from( - RangePartitioner::reservoir_sample_indices(input.num_rows(), 100), - ); - - let sampled_columns = take_arrays( + let (bounds_rows, row_converter) = RangePartitioner::generate_bounds( &partition_arrays, - &sample_indices, - Some(TakeOptions { - check_bounds: false, - }), - )?; - - 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, - *num_output_partitions as i32, - ); - - let bounds_indices_array = UInt64Array::from(bounds_indices); - - let bounds_arrays = take_arrays( - &partition_arrays, - &bounds_indices_array, - Some(TakeOptions { - check_bounds: false, - }), - )?; - - self.bounds_rows = Some( - row_converter - .convert_columns(bounds_arrays.as_slice()) - .unwrap(), + lex_ordering, + *num_output_partitions, + input.num_rows(), + 100, ); + + self.bounds_rows = Some(bounds_rows); self.row_converter = Some(row_converter); } From d2f3c69418b64866dc90001c916920aa18c2d590 Mon Sep 17 00:00:00 2001 From: Matt Butrovich Date: Mon, 9 Jun 2025 11:56:11 -0400 Subject: [PATCH 36/72] Break out some repeated code into functions within MultiPartitionShuffleRepartitioner. --- .../src/execution/shuffle/shuffle_writer.rs | 191 +++++++++--------- 1 file changed, 98 insertions(+), 93 deletions(-) diff --git a/native/core/src/execution/shuffle/shuffle_writer.rs b/native/core/src/execution/shuffle/shuffle_writer.rs index c183819ab7..c554ca3a5a 100644 --- a/native/core/src/execution/shuffle/shuffle_writer.rs +++ b/native/core/src/execution/shuffle/shuffle_writer.rs @@ -426,6 +426,57 @@ impl MultiPartitionShuffleRepartitioner { return Ok(()); } + fn count_partitions( + partition_ids: &[u32], + partition_counters: &mut Vec, + num_output_partitions: usize, + ) { + // count each partition size, while leaving the last extra element as 0 + partition_counters.resize(num_output_partitions + 1, 0); + partition_counters.fill(0); + partition_ids + .iter() + .for_each(|partition_id| partition_counters[*partition_id as usize] += 1); + } + + fn accumulate_partition_counters(partition_ends: &mut Vec) { + // accumulate partition counters into partition ends + // e.g. partition counter: [1, 3, 2, 1, 0] => [1, 4, 6, 7, 7] + // this is basically an inclusive scan/prefix sum + let mut accum = 0; + partition_ends.iter_mut().for_each(|v| { + *v += accum; + accum = *v; + }); + } + + fn calculate_partition_row_indices_and_partition_starts( + scratch: &mut ScratchSpace, + num_rows: usize, + ) { + // calculate partition row indices and partition starts + // e.g. partition ids: [3, 1, 1, 1, 2, 2, 0] will produce the following partition_row_indices + // and partition_starts arrays: + // + // partition_row_indices: [6, 1, 2, 3, 4, 5, 0] + // partition_starts: [0, 1, 4, 6, 7] + // + // partition_starts conceptually splits partition_row_indices into smaller slices. + // Each slice partition_row_indices[partition_starts[K]..partition_starts[K + 1]] contains the + // row indices of the input batch that are partitioned into partition K. For example, + // first partition 0 has one row index [6], partition 1 has row indices [1, 2, 3], etc. + let partition_ids = &mut scratch.partition_ids[..num_rows]; + let partition_row_indices = &mut scratch.partition_row_indices; + let partition_ends = &mut scratch.partition_starts; + partition_row_indices.resize(num_rows, 0); + for (index, partition_id) in partition_ids.iter().enumerate().rev() { + partition_ends[*partition_id as usize] -= 1; + let end = partition_ends[*partition_id as usize]; + partition_row_indices[end as usize] = index as u32; + } + // after calculating, partition ends become partition starts + } + if input.num_rows() > self.batch_size { return Err(DataFusionError::Internal( "Input batch size exceeds configured batch size. Call `insert_batch` instead." @@ -456,57 +507,34 @@ impl MultiPartitionShuffleRepartitioner { let hashes_buf = &mut scratch.hashes_buf[..arrays[0].len()]; hashes_buf.fill(42_u32); - // Hash arrays and compute buckets based on number of partitions - let partition_ids = &mut scratch.partition_ids[..arrays[0].len()]; - create_murmur3_hashes(&arrays, hashes_buf)? - .iter() - .enumerate() - .for_each(|(idx, hash)| { - partition_ids[idx] = pmod(*hash, *num_output_partitions) as u32; - }); - - // count each partition size, while leaving the last extra element as 0 - let partition_counters = &mut scratch.partition_starts; - partition_counters.resize(num_output_partitions + 1, 0); - partition_counters.fill(0); - partition_ids - .iter() - .for_each(|partition_id| partition_counters[*partition_id as usize] += 1); - - // accumulate partition counters into partition ends - // e.g. partition counter: [1, 3, 2, 1, 0] => [1, 4, 6, 7, 7] - let partition_ends = partition_counters; - let mut accum = 0; - partition_ends.iter_mut().for_each(|v| { - *v += accum; - accum = *v; - }); - - // calculate partition row indices and partition starts - // e.g. partition ids: [3, 1, 1, 1, 2, 2, 0] will produce the following partition_row_indices - // and partition_starts arrays: - // - // partition_row_indices: [6, 1, 2, 3, 4, 5, 0] - // partition_starts: [0, 1, 4, 6, 7] - // - // partition_starts conceptually splits partition_row_indices into smaller slices. - // Each slice partition_row_indices[partition_starts[K]..partition_starts[K + 1]] contains the - // row indices of the input batch that are partitioned into partition K. For example, - // first partition 0 has one row index [6], partition 1 has row indices [1, 2, 3], etc. - let partition_row_indices = &mut scratch.partition_row_indices; - partition_row_indices.resize(input.num_rows(), 0); - for (index, partition_id) in partition_ids.iter().enumerate().rev() { - partition_ends[*partition_id as usize] -= 1; - let end = partition_ends[*partition_id as usize]; - partition_row_indices[end as usize] = index as u32; + { + // Hash arrays and compute buckets based on number of partitions + let partition_ids = &mut scratch.partition_ids[..arrays[0].len()]; + create_murmur3_hashes(&arrays, hashes_buf)? + .iter() + .enumerate() + .for_each(|(idx, hash)| { + partition_ids[idx] = pmod(*hash, *num_output_partitions) as u32; + }); } - // after calculating, partition ends become partition starts - let partition_starts = partition_ends; + count_partitions( + &mut scratch.partition_ids[..arrays[0].len()], + &mut scratch.partition_starts, + *num_output_partitions, + ); + + accumulate_partition_counters(&mut scratch.partition_starts); + + calculate_partition_row_indices_and_partition_starts( + &mut scratch, + arrays[0].len(), + ); + timer.stop(); Ok::<(&Vec, &Vec), DataFusionError>(( - partition_starts, - partition_row_indices, + &scratch.partition_starts, + &scratch.partition_row_indices, )) }?; @@ -548,60 +576,37 @@ impl MultiPartitionShuffleRepartitioner { .as_ref() .unwrap() .convert_columns(partition_arrays.as_slice())?; + { + let partition_ids = &mut scratch.partition_ids[..partition_arrays[0].len()]; - let partition_ids = &mut scratch.partition_ids[..partition_arrays[0].len()]; + // TODO: Try to cache this vector. + let bounds_rows_vec = + self.bounds_rows.as_ref().unwrap().iter().collect_vec(); - // TODO: Try to cache this vector. - let bounds_rows_vec = self.bounds_rows.as_ref().unwrap().iter().collect_vec(); + RangePartitioner::partition_indices_for_batch( + &row_batch, + &bounds_rows_vec, + partition_ids, + ); + } - RangePartitioner::partition_indices_for_batch( - &row_batch, - &bounds_rows_vec, - partition_ids, + count_partitions( + &mut scratch.partition_ids[..partition_arrays[0].len()], + &mut scratch.partition_starts, + *num_output_partitions, ); - // count each partition size, while leaving the last extra element as 0 - let partition_counters = &mut scratch.partition_starts; - partition_counters.resize(num_output_partitions + 1, 0); - partition_counters.fill(0); - partition_ids - .iter() - .for_each(|partition_id| partition_counters[*partition_id as usize] += 1); - - // accumulate partition counters into partition ends - // e.g. partition counter: [1, 3, 2, 1, 0] => [1, 4, 6, 7, 7] - let partition_ends = partition_counters; - let mut accum = 0; - partition_ends.iter_mut().for_each(|v| { - *v += accum; - accum = *v; - }); - - // calculate partition row indices and partition starts - // e.g. partition ids: [3, 1, 1, 1, 2, 2, 0] will produce the following partition_row_indices - // and partition_starts arrays: - // - // partition_row_indices: [6, 1, 2, 3, 4, 5, 0] - // partition_starts: [0, 1, 4, 6, 7] - // - // partition_starts conceptually splits partition_row_indices into smaller slices. - // Each slice partition_row_indices[partition_starts[K]..partition_starts[K + 1]] contains the - // row indices of the input batch that are partitioned into partition K. For example, - // first partition 0 has one row index [6], partition 1 has row indices [1, 2, 3], etc. - let partition_row_indices = &mut scratch.partition_row_indices; - partition_row_indices.resize(input.num_rows(), 0); - for (index, partition_id) in partition_ids.iter().enumerate().rev() { - partition_ends[*partition_id as usize] -= 1; - let end = partition_ends[*partition_id as usize]; - partition_row_indices[end as usize] = index as u32; - } + accumulate_partition_counters(&mut scratch.partition_starts); + + calculate_partition_row_indices_and_partition_starts( + &mut scratch, + partition_arrays[0].len(), + ); - // after calculating, partition ends become partition starts - let partition_starts = partition_ends; timer.stop(); Ok::<(&Vec, &Vec), DataFusionError>(( - partition_starts, - partition_row_indices, + &scratch.partition_starts, + &scratch.partition_row_indices, )) }?; From a69b5647eee8ff355ae2e92cc52b9920a032ee09 Mon Sep 17 00:00:00 2001 From: Matt Butrovich Date: Mon, 9 Jun 2025 13:31:13 -0400 Subject: [PATCH 37/72] Update CometFuzzTestSuite. --- native/Cargo.lock | 631 +++++------------- .../org/apache/comet/CometFuzzTestSuite.scala | 21 +- 2 files changed, 187 insertions(+), 465 deletions(-) diff --git a/native/Cargo.lock b/native/Cargo.lock index 6fea0b84ee..71662f97fa 100644 --- a/native/Cargo.lock +++ b/native/Cargo.lock @@ -1265,7 +1265,7 @@ dependencies = [ [[package]] name = "datafusion" version = "48.0.0" -source = "git+https://github.com/apache/datafusion?rev=48.0.0-rc2#85f6621a6b1680b40d483a56b10ff3495861ece3" +source = "git+https://github.com/apache/datafusion?rev=48.0.0-rc3#33a32d4382bee7e3c705d0f55d05c24a115a2f98" dependencies = [ "arrow", "arrow-ipc", @@ -1273,29 +1273,29 @@ dependencies = [ "async-trait", "bytes", "chrono", - "datafusion-catalog 48.0.0 (git+https://github.com/apache/datafusion?rev=48.0.0-rc2)", + "datafusion-catalog", "datafusion-catalog-listing", - "datafusion-common 48.0.0 (git+https://github.com/apache/datafusion?rev=48.0.0-rc2)", - "datafusion-common-runtime 48.0.0 (git+https://github.com/apache/datafusion?rev=48.0.0-rc2)", - "datafusion-datasource 48.0.0 (git+https://github.com/apache/datafusion?rev=48.0.0-rc2)", + "datafusion-common", + "datafusion-common-runtime", + "datafusion-datasource", "datafusion-datasource-csv", "datafusion-datasource-json", "datafusion-datasource-parquet", - "datafusion-execution 48.0.0 (git+https://github.com/apache/datafusion?rev=48.0.0-rc2)", - "datafusion-expr 48.0.0 (git+https://github.com/apache/datafusion?rev=48.0.0-rc2)", - "datafusion-expr-common 48.0.0 (git+https://github.com/apache/datafusion?rev=48.0.0-rc2)", - "datafusion-functions 48.0.0 (git+https://github.com/apache/datafusion?rev=48.0.0-rc2)", + "datafusion-execution", + "datafusion-expr", + "datafusion-expr-common", + "datafusion-functions", "datafusion-functions-aggregate", "datafusion-functions-nested", "datafusion-functions-table", "datafusion-functions-window", "datafusion-optimizer", - "datafusion-physical-expr 48.0.0 (git+https://github.com/apache/datafusion?rev=48.0.0-rc2)", - "datafusion-physical-expr-common 48.0.0 (git+https://github.com/apache/datafusion?rev=48.0.0-rc2)", + "datafusion-physical-expr", + "datafusion-physical-expr-common", "datafusion-physical-optimizer", - "datafusion-physical-plan 48.0.0 (git+https://github.com/apache/datafusion?rev=48.0.0-rc2)", - "datafusion-session 48.0.0 (git+https://github.com/apache/datafusion?rev=48.0.0-rc2)", - "datafusion-sql 48.0.0 (git+https://github.com/apache/datafusion?rev=48.0.0-rc2)", + "datafusion-physical-plan", + "datafusion-session", + "datafusion-sql", "futures", "itertools 0.14.0", "log", @@ -1311,31 +1311,6 @@ dependencies = [ "uuid", ] -[[package]] -name = "datafusion-catalog" -version = "48.0.0" -source = "git+https://github.com/apache/datafusion?rev=48.0.0-rc2#85f6621a6b1680b40d483a56b10ff3495861ece3" -dependencies = [ - "arrow", - "async-trait", - "dashmap", - "datafusion-common 48.0.0 (git+https://github.com/apache/datafusion?rev=48.0.0-rc2)", - "datafusion-common-runtime 48.0.0 (git+https://github.com/apache/datafusion?rev=48.0.0-rc2)", - "datafusion-datasource 48.0.0 (git+https://github.com/apache/datafusion?rev=48.0.0-rc2)", - "datafusion-execution 48.0.0 (git+https://github.com/apache/datafusion?rev=48.0.0-rc2)", - "datafusion-expr 48.0.0 (git+https://github.com/apache/datafusion?rev=48.0.0-rc2)", - "datafusion-physical-expr 48.0.0 (git+https://github.com/apache/datafusion?rev=48.0.0-rc2)", - "datafusion-physical-plan 48.0.0 (git+https://github.com/apache/datafusion?rev=48.0.0-rc2)", - "datafusion-session 48.0.0 (git+https://github.com/apache/datafusion?rev=48.0.0-rc2)", - "datafusion-sql 48.0.0 (git+https://github.com/apache/datafusion?rev=48.0.0-rc2)", - "futures", - "itertools 0.14.0", - "log", - "object_store", - "parking_lot", - "tokio", -] - [[package]] name = "datafusion-catalog" version = "48.0.0" @@ -1344,15 +1319,15 @@ dependencies = [ "arrow", "async-trait", "dashmap", - "datafusion-common 48.0.0 (git+https://github.com/apache/datafusion?rev=48.0.0-rc3)", - "datafusion-common-runtime 48.0.0 (git+https://github.com/apache/datafusion?rev=48.0.0-rc3)", - "datafusion-datasource 48.0.0 (git+https://github.com/apache/datafusion?rev=48.0.0-rc3)", - "datafusion-execution 48.0.0 (git+https://github.com/apache/datafusion?rev=48.0.0-rc3)", - "datafusion-expr 48.0.0 (git+https://github.com/apache/datafusion?rev=48.0.0-rc3)", - "datafusion-physical-expr 48.0.0 (git+https://github.com/apache/datafusion?rev=48.0.0-rc3)", - "datafusion-physical-plan 48.0.0 (git+https://github.com/apache/datafusion?rev=48.0.0-rc3)", - "datafusion-session 48.0.0 (git+https://github.com/apache/datafusion?rev=48.0.0-rc3)", - "datafusion-sql 48.0.0 (git+https://github.com/apache/datafusion?rev=48.0.0-rc3)", + "datafusion-common", + "datafusion-common-runtime", + "datafusion-datasource", + "datafusion-execution", + "datafusion-expr", + "datafusion-physical-expr", + "datafusion-physical-plan", + "datafusion-session", + "datafusion-sql", "futures", "itertools 0.14.0", "log", @@ -1364,19 +1339,19 @@ dependencies = [ [[package]] name = "datafusion-catalog-listing" version = "48.0.0" -source = "git+https://github.com/apache/datafusion?rev=48.0.0-rc2#85f6621a6b1680b40d483a56b10ff3495861ece3" +source = "git+https://github.com/apache/datafusion?rev=48.0.0-rc3#33a32d4382bee7e3c705d0f55d05c24a115a2f98" dependencies = [ "arrow", "async-trait", - "datafusion-catalog 48.0.0 (git+https://github.com/apache/datafusion?rev=48.0.0-rc2)", - "datafusion-common 48.0.0 (git+https://github.com/apache/datafusion?rev=48.0.0-rc2)", - "datafusion-datasource 48.0.0 (git+https://github.com/apache/datafusion?rev=48.0.0-rc2)", - "datafusion-execution 48.0.0 (git+https://github.com/apache/datafusion?rev=48.0.0-rc2)", - "datafusion-expr 48.0.0 (git+https://github.com/apache/datafusion?rev=48.0.0-rc2)", - "datafusion-physical-expr 48.0.0 (git+https://github.com/apache/datafusion?rev=48.0.0-rc2)", - "datafusion-physical-expr-common 48.0.0 (git+https://github.com/apache/datafusion?rev=48.0.0-rc2)", - "datafusion-physical-plan 48.0.0 (git+https://github.com/apache/datafusion?rev=48.0.0-rc2)", - "datafusion-session 48.0.0 (git+https://github.com/apache/datafusion?rev=48.0.0-rc2)", + "datafusion-catalog", + "datafusion-common", + "datafusion-datasource", + "datafusion-execution", + "datafusion-expr", + "datafusion-physical-expr", + "datafusion-physical-expr-common", + "datafusion-physical-plan", + "datafusion-session", "futures", "log", "object_store", @@ -1472,28 +1447,6 @@ dependencies = [ "twox-hash 2.1.0", ] -[[package]] -name = "datafusion-common" -version = "48.0.0" -source = "git+https://github.com/apache/datafusion?rev=48.0.0-rc2#85f6621a6b1680b40d483a56b10ff3495861ece3" -dependencies = [ - "ahash", - "arrow", - "arrow-ipc", - "base64", - "half", - "hashbrown 0.14.5", - "indexmap", - "libc", - "log", - "object_store", - "parquet", - "paste", - "sqlparser", - "tokio", - "web-time", -] - [[package]] name = "datafusion-common" version = "48.0.0" @@ -1509,22 +1462,13 @@ dependencies = [ "libc", "log", "object_store", + "parquet", "paste", "sqlparser", "tokio", "web-time", ] -[[package]] -name = "datafusion-common-runtime" -version = "48.0.0" -source = "git+https://github.com/apache/datafusion?rev=48.0.0-rc2#85f6621a6b1680b40d483a56b10ff3495861ece3" -dependencies = [ - "futures", - "log", - "tokio", -] - [[package]] name = "datafusion-common-runtime" version = "48.0.0" @@ -1538,20 +1482,20 @@ dependencies = [ [[package]] name = "datafusion-datasource" version = "48.0.0" -source = "git+https://github.com/apache/datafusion?rev=48.0.0-rc2#85f6621a6b1680b40d483a56b10ff3495861ece3" +source = "git+https://github.com/apache/datafusion?rev=48.0.0-rc3#33a32d4382bee7e3c705d0f55d05c24a115a2f98" dependencies = [ "arrow", "async-trait", "bytes", "chrono", - "datafusion-common 48.0.0 (git+https://github.com/apache/datafusion?rev=48.0.0-rc2)", - "datafusion-common-runtime 48.0.0 (git+https://github.com/apache/datafusion?rev=48.0.0-rc2)", - "datafusion-execution 48.0.0 (git+https://github.com/apache/datafusion?rev=48.0.0-rc2)", - "datafusion-expr 48.0.0 (git+https://github.com/apache/datafusion?rev=48.0.0-rc2)", - "datafusion-physical-expr 48.0.0 (git+https://github.com/apache/datafusion?rev=48.0.0-rc2)", - "datafusion-physical-expr-common 48.0.0 (git+https://github.com/apache/datafusion?rev=48.0.0-rc2)", - "datafusion-physical-plan 48.0.0 (git+https://github.com/apache/datafusion?rev=48.0.0-rc2)", - "datafusion-session 48.0.0 (git+https://github.com/apache/datafusion?rev=48.0.0-rc2)", + "datafusion-common", + "datafusion-common-runtime", + "datafusion-execution", + "datafusion-expr", + "datafusion-physical-expr", + "datafusion-physical-expr-common", + "datafusion-physical-plan", + "datafusion-session", "futures", "glob", "itertools 0.14.0", @@ -1564,51 +1508,24 @@ dependencies = [ "url", ] -[[package]] -name = "datafusion-datasource" -version = "48.0.0" -source = "git+https://github.com/apache/datafusion?rev=48.0.0-rc3#33a32d4382bee7e3c705d0f55d05c24a115a2f98" -dependencies = [ - "arrow", - "async-trait", - "bytes", - "chrono", - "datafusion-common 48.0.0 (git+https://github.com/apache/datafusion?rev=48.0.0-rc3)", - "datafusion-common-runtime 48.0.0 (git+https://github.com/apache/datafusion?rev=48.0.0-rc3)", - "datafusion-execution 48.0.0 (git+https://github.com/apache/datafusion?rev=48.0.0-rc3)", - "datafusion-expr 48.0.0 (git+https://github.com/apache/datafusion?rev=48.0.0-rc3)", - "datafusion-physical-expr 48.0.0 (git+https://github.com/apache/datafusion?rev=48.0.0-rc3)", - "datafusion-physical-expr-common 48.0.0 (git+https://github.com/apache/datafusion?rev=48.0.0-rc3)", - "datafusion-physical-plan 48.0.0 (git+https://github.com/apache/datafusion?rev=48.0.0-rc3)", - "datafusion-session 48.0.0 (git+https://github.com/apache/datafusion?rev=48.0.0-rc3)", - "futures", - "glob", - "itertools 0.14.0", - "log", - "object_store", - "rand 0.9.1", - "tokio", - "url", -] - [[package]] name = "datafusion-datasource-csv" version = "48.0.0" -source = "git+https://github.com/apache/datafusion?rev=48.0.0-rc2#85f6621a6b1680b40d483a56b10ff3495861ece3" +source = "git+https://github.com/apache/datafusion?rev=48.0.0-rc3#33a32d4382bee7e3c705d0f55d05c24a115a2f98" dependencies = [ "arrow", "async-trait", "bytes", - "datafusion-catalog 48.0.0 (git+https://github.com/apache/datafusion?rev=48.0.0-rc2)", - "datafusion-common 48.0.0 (git+https://github.com/apache/datafusion?rev=48.0.0-rc2)", - "datafusion-common-runtime 48.0.0 (git+https://github.com/apache/datafusion?rev=48.0.0-rc2)", - "datafusion-datasource 48.0.0 (git+https://github.com/apache/datafusion?rev=48.0.0-rc2)", - "datafusion-execution 48.0.0 (git+https://github.com/apache/datafusion?rev=48.0.0-rc2)", - "datafusion-expr 48.0.0 (git+https://github.com/apache/datafusion?rev=48.0.0-rc2)", - "datafusion-physical-expr 48.0.0 (git+https://github.com/apache/datafusion?rev=48.0.0-rc2)", - "datafusion-physical-expr-common 48.0.0 (git+https://github.com/apache/datafusion?rev=48.0.0-rc2)", - "datafusion-physical-plan 48.0.0 (git+https://github.com/apache/datafusion?rev=48.0.0-rc2)", - "datafusion-session 48.0.0 (git+https://github.com/apache/datafusion?rev=48.0.0-rc2)", + "datafusion-catalog", + "datafusion-common", + "datafusion-common-runtime", + "datafusion-datasource", + "datafusion-execution", + "datafusion-expr", + "datafusion-physical-expr", + "datafusion-physical-expr-common", + "datafusion-physical-plan", + "datafusion-session", "futures", "object_store", "regex", @@ -1618,21 +1535,21 @@ dependencies = [ [[package]] name = "datafusion-datasource-json" version = "48.0.0" -source = "git+https://github.com/apache/datafusion?rev=48.0.0-rc2#85f6621a6b1680b40d483a56b10ff3495861ece3" +source = "git+https://github.com/apache/datafusion?rev=48.0.0-rc3#33a32d4382bee7e3c705d0f55d05c24a115a2f98" dependencies = [ "arrow", "async-trait", "bytes", - "datafusion-catalog 48.0.0 (git+https://github.com/apache/datafusion?rev=48.0.0-rc2)", - "datafusion-common 48.0.0 (git+https://github.com/apache/datafusion?rev=48.0.0-rc2)", - "datafusion-common-runtime 48.0.0 (git+https://github.com/apache/datafusion?rev=48.0.0-rc2)", - "datafusion-datasource 48.0.0 (git+https://github.com/apache/datafusion?rev=48.0.0-rc2)", - "datafusion-execution 48.0.0 (git+https://github.com/apache/datafusion?rev=48.0.0-rc2)", - "datafusion-expr 48.0.0 (git+https://github.com/apache/datafusion?rev=48.0.0-rc2)", - "datafusion-physical-expr 48.0.0 (git+https://github.com/apache/datafusion?rev=48.0.0-rc2)", - "datafusion-physical-expr-common 48.0.0 (git+https://github.com/apache/datafusion?rev=48.0.0-rc2)", - "datafusion-physical-plan 48.0.0 (git+https://github.com/apache/datafusion?rev=48.0.0-rc2)", - "datafusion-session 48.0.0 (git+https://github.com/apache/datafusion?rev=48.0.0-rc2)", + "datafusion-catalog", + "datafusion-common", + "datafusion-common-runtime", + "datafusion-datasource", + "datafusion-execution", + "datafusion-expr", + "datafusion-physical-expr", + "datafusion-physical-expr-common", + "datafusion-physical-plan", + "datafusion-session", "futures", "object_store", "serde_json", @@ -1642,23 +1559,23 @@ dependencies = [ [[package]] name = "datafusion-datasource-parquet" version = "48.0.0" -source = "git+https://github.com/apache/datafusion?rev=48.0.0-rc2#85f6621a6b1680b40d483a56b10ff3495861ece3" +source = "git+https://github.com/apache/datafusion?rev=48.0.0-rc3#33a32d4382bee7e3c705d0f55d05c24a115a2f98" dependencies = [ "arrow", "async-trait", "bytes", - "datafusion-catalog 48.0.0 (git+https://github.com/apache/datafusion?rev=48.0.0-rc2)", - "datafusion-common 48.0.0 (git+https://github.com/apache/datafusion?rev=48.0.0-rc2)", - "datafusion-common-runtime 48.0.0 (git+https://github.com/apache/datafusion?rev=48.0.0-rc2)", - "datafusion-datasource 48.0.0 (git+https://github.com/apache/datafusion?rev=48.0.0-rc2)", - "datafusion-execution 48.0.0 (git+https://github.com/apache/datafusion?rev=48.0.0-rc2)", - "datafusion-expr 48.0.0 (git+https://github.com/apache/datafusion?rev=48.0.0-rc2)", + "datafusion-catalog", + "datafusion-common", + "datafusion-common-runtime", + "datafusion-datasource", + "datafusion-execution", + "datafusion-expr", "datafusion-functions-aggregate", - "datafusion-physical-expr 48.0.0 (git+https://github.com/apache/datafusion?rev=48.0.0-rc2)", - "datafusion-physical-expr-common 48.0.0 (git+https://github.com/apache/datafusion?rev=48.0.0-rc2)", + "datafusion-physical-expr", + "datafusion-physical-expr-common", "datafusion-physical-optimizer", - "datafusion-physical-plan 48.0.0 (git+https://github.com/apache/datafusion?rev=48.0.0-rc2)", - "datafusion-session 48.0.0 (git+https://github.com/apache/datafusion?rev=48.0.0-rc2)", + "datafusion-physical-plan", + "datafusion-session", "futures", "itertools 0.14.0", "log", @@ -1669,34 +1586,11 @@ dependencies = [ "tokio", ] -[[package]] -name = "datafusion-doc" -version = "48.0.0" -source = "git+https://github.com/apache/datafusion?rev=48.0.0-rc2#85f6621a6b1680b40d483a56b10ff3495861ece3" - [[package]] name = "datafusion-doc" version = "48.0.0" source = "git+https://github.com/apache/datafusion?rev=48.0.0-rc3#33a32d4382bee7e3c705d0f55d05c24a115a2f98" -[[package]] -name = "datafusion-execution" -version = "48.0.0" -source = "git+https://github.com/apache/datafusion?rev=48.0.0-rc2#85f6621a6b1680b40d483a56b10ff3495861ece3" -dependencies = [ - "arrow", - "dashmap", - "datafusion-common 48.0.0 (git+https://github.com/apache/datafusion?rev=48.0.0-rc2)", - "datafusion-expr 48.0.0 (git+https://github.com/apache/datafusion?rev=48.0.0-rc2)", - "futures", - "log", - "object_store", - "parking_lot", - "rand 0.9.1", - "tempfile", - "url", -] - [[package]] name = "datafusion-execution" version = "48.0.0" @@ -1704,8 +1598,8 @@ source = "git+https://github.com/apache/datafusion?rev=48.0.0-rc3#33a32d4382bee7 dependencies = [ "arrow", "dashmap", - "datafusion-common 48.0.0 (git+https://github.com/apache/datafusion?rev=48.0.0-rc3)", - "datafusion-expr 48.0.0 (git+https://github.com/apache/datafusion?rev=48.0.0-rc3)", + "datafusion-common", + "datafusion-expr", "futures", "log", "object_store", @@ -1715,25 +1609,6 @@ dependencies = [ "url", ] -[[package]] -name = "datafusion-expr" -version = "48.0.0" -source = "git+https://github.com/apache/datafusion?rev=48.0.0-rc2#85f6621a6b1680b40d483a56b10ff3495861ece3" -dependencies = [ - "arrow", - "chrono", - "datafusion-common 48.0.0 (git+https://github.com/apache/datafusion?rev=48.0.0-rc2)", - "datafusion-doc 48.0.0 (git+https://github.com/apache/datafusion?rev=48.0.0-rc2)", - "datafusion-expr-common 48.0.0 (git+https://github.com/apache/datafusion?rev=48.0.0-rc2)", - "datafusion-functions-aggregate-common 48.0.0 (git+https://github.com/apache/datafusion?rev=48.0.0-rc2)", - "datafusion-functions-window-common 48.0.0 (git+https://github.com/apache/datafusion?rev=48.0.0-rc2)", - "datafusion-physical-expr-common 48.0.0 (git+https://github.com/apache/datafusion?rev=48.0.0-rc2)", - "indexmap", - "paste", - "serde_json", - "sqlparser", -] - [[package]] name = "datafusion-expr" version = "48.0.0" @@ -1741,37 +1616,25 @@ source = "git+https://github.com/apache/datafusion?rev=48.0.0-rc3#33a32d4382bee7 dependencies = [ "arrow", "chrono", - "datafusion-common 48.0.0 (git+https://github.com/apache/datafusion?rev=48.0.0-rc3)", - "datafusion-doc 48.0.0 (git+https://github.com/apache/datafusion?rev=48.0.0-rc3)", - "datafusion-expr-common 48.0.0 (git+https://github.com/apache/datafusion?rev=48.0.0-rc3)", - "datafusion-functions-aggregate-common 48.0.0 (git+https://github.com/apache/datafusion?rev=48.0.0-rc3)", - "datafusion-functions-window-common 48.0.0 (git+https://github.com/apache/datafusion?rev=48.0.0-rc3)", - "datafusion-physical-expr-common 48.0.0 (git+https://github.com/apache/datafusion?rev=48.0.0-rc3)", + "datafusion-common", + "datafusion-doc", + "datafusion-expr-common", + "datafusion-functions-aggregate-common", + "datafusion-functions-window-common", + "datafusion-physical-expr-common", "indexmap", "paste", "serde_json", "sqlparser", ] -[[package]] -name = "datafusion-expr-common" -version = "48.0.0" -source = "git+https://github.com/apache/datafusion?rev=48.0.0-rc2#85f6621a6b1680b40d483a56b10ff3495861ece3" -dependencies = [ - "arrow", - "datafusion-common 48.0.0 (git+https://github.com/apache/datafusion?rev=48.0.0-rc2)", - "indexmap", - "itertools 0.14.0", - "paste", -] - [[package]] name = "datafusion-expr-common" version = "48.0.0" source = "git+https://github.com/apache/datafusion?rev=48.0.0-rc3#33a32d4382bee7e3c705d0f55d05c24a115a2f98" dependencies = [ "arrow", - "datafusion-common 48.0.0 (git+https://github.com/apache/datafusion?rev=48.0.0-rc3)", + "datafusion-common", "indexmap", "itertools 0.14.0", "paste", @@ -1780,7 +1643,7 @@ dependencies = [ [[package]] name = "datafusion-functions" version = "48.0.0" -source = "git+https://github.com/apache/datafusion?rev=48.0.0-rc2#85f6621a6b1680b40d483a56b10ff3495861ece3" +source = "git+https://github.com/apache/datafusion?rev=48.0.0-rc3#33a32d4382bee7e3c705d0f55d05c24a115a2f98" dependencies = [ "arrow", "arrow-buffer", @@ -1788,12 +1651,12 @@ dependencies = [ "blake2", "blake3", "chrono", - "datafusion-common 48.0.0 (git+https://github.com/apache/datafusion?rev=48.0.0-rc2)", - "datafusion-doc 48.0.0 (git+https://github.com/apache/datafusion?rev=48.0.0-rc2)", - "datafusion-execution 48.0.0 (git+https://github.com/apache/datafusion?rev=48.0.0-rc2)", - "datafusion-expr 48.0.0 (git+https://github.com/apache/datafusion?rev=48.0.0-rc2)", - "datafusion-expr-common 48.0.0 (git+https://github.com/apache/datafusion?rev=48.0.0-rc2)", - "datafusion-macros 48.0.0 (git+https://github.com/apache/datafusion?rev=48.0.0-rc2)", + "datafusion-common", + "datafusion-doc", + "datafusion-execution", + "datafusion-expr", + "datafusion-expr-common", + "datafusion-macros", "hex", "itertools 0.14.0", "log", @@ -1805,62 +1668,26 @@ dependencies = [ "uuid", ] -[[package]] -name = "datafusion-functions" -version = "48.0.0" -source = "git+https://github.com/apache/datafusion?rev=48.0.0-rc3#33a32d4382bee7e3c705d0f55d05c24a115a2f98" -dependencies = [ - "arrow", - "arrow-buffer", - "base64", - "chrono", - "datafusion-common 48.0.0 (git+https://github.com/apache/datafusion?rev=48.0.0-rc3)", - "datafusion-doc 48.0.0 (git+https://github.com/apache/datafusion?rev=48.0.0-rc3)", - "datafusion-execution 48.0.0 (git+https://github.com/apache/datafusion?rev=48.0.0-rc3)", - "datafusion-expr 48.0.0 (git+https://github.com/apache/datafusion?rev=48.0.0-rc3)", - "datafusion-expr-common 48.0.0 (git+https://github.com/apache/datafusion?rev=48.0.0-rc3)", - "datafusion-macros 48.0.0 (git+https://github.com/apache/datafusion?rev=48.0.0-rc3)", - "hex", - "itertools 0.14.0", - "log", - "rand 0.9.1", - "regex", - "unicode-segmentation", - "uuid", -] - [[package]] name = "datafusion-functions-aggregate" version = "48.0.0" -source = "git+https://github.com/apache/datafusion?rev=48.0.0-rc2#85f6621a6b1680b40d483a56b10ff3495861ece3" +source = "git+https://github.com/apache/datafusion?rev=48.0.0-rc3#33a32d4382bee7e3c705d0f55d05c24a115a2f98" dependencies = [ "ahash", "arrow", - "datafusion-common 48.0.0 (git+https://github.com/apache/datafusion?rev=48.0.0-rc2)", - "datafusion-doc 48.0.0 (git+https://github.com/apache/datafusion?rev=48.0.0-rc2)", - "datafusion-execution 48.0.0 (git+https://github.com/apache/datafusion?rev=48.0.0-rc2)", - "datafusion-expr 48.0.0 (git+https://github.com/apache/datafusion?rev=48.0.0-rc2)", - "datafusion-functions-aggregate-common 48.0.0 (git+https://github.com/apache/datafusion?rev=48.0.0-rc2)", - "datafusion-macros 48.0.0 (git+https://github.com/apache/datafusion?rev=48.0.0-rc2)", - "datafusion-physical-expr 48.0.0 (git+https://github.com/apache/datafusion?rev=48.0.0-rc2)", - "datafusion-physical-expr-common 48.0.0 (git+https://github.com/apache/datafusion?rev=48.0.0-rc2)", + "datafusion-common", + "datafusion-doc", + "datafusion-execution", + "datafusion-expr", + "datafusion-functions-aggregate-common", + "datafusion-macros", + "datafusion-physical-expr", + "datafusion-physical-expr-common", "half", "log", "paste", ] -[[package]] -name = "datafusion-functions-aggregate-common" -version = "48.0.0" -source = "git+https://github.com/apache/datafusion?rev=48.0.0-rc2#85f6621a6b1680b40d483a56b10ff3495861ece3" -dependencies = [ - "ahash", - "arrow", - "datafusion-common 48.0.0 (git+https://github.com/apache/datafusion?rev=48.0.0-rc2)", - "datafusion-expr-common 48.0.0 (git+https://github.com/apache/datafusion?rev=48.0.0-rc2)", - "datafusion-physical-expr-common 48.0.0 (git+https://github.com/apache/datafusion?rev=48.0.0-rc2)", -] - [[package]] name = "datafusion-functions-aggregate-common" version = "48.0.0" @@ -1868,26 +1695,26 @@ source = "git+https://github.com/apache/datafusion?rev=48.0.0-rc3#33a32d4382bee7 dependencies = [ "ahash", "arrow", - "datafusion-common 48.0.0 (git+https://github.com/apache/datafusion?rev=48.0.0-rc3)", - "datafusion-expr-common 48.0.0 (git+https://github.com/apache/datafusion?rev=48.0.0-rc3)", - "datafusion-physical-expr-common 48.0.0 (git+https://github.com/apache/datafusion?rev=48.0.0-rc3)", + "datafusion-common", + "datafusion-expr-common", + "datafusion-physical-expr-common", ] [[package]] name = "datafusion-functions-nested" version = "48.0.0" -source = "git+https://github.com/apache/datafusion?rev=48.0.0-rc2#85f6621a6b1680b40d483a56b10ff3495861ece3" +source = "git+https://github.com/apache/datafusion?rev=48.0.0-rc3#33a32d4382bee7e3c705d0f55d05c24a115a2f98" dependencies = [ "arrow", "arrow-ord", - "datafusion-common 48.0.0 (git+https://github.com/apache/datafusion?rev=48.0.0-rc2)", - "datafusion-doc 48.0.0 (git+https://github.com/apache/datafusion?rev=48.0.0-rc2)", - "datafusion-execution 48.0.0 (git+https://github.com/apache/datafusion?rev=48.0.0-rc2)", - "datafusion-expr 48.0.0 (git+https://github.com/apache/datafusion?rev=48.0.0-rc2)", - "datafusion-functions 48.0.0 (git+https://github.com/apache/datafusion?rev=48.0.0-rc2)", + "datafusion-common", + "datafusion-doc", + "datafusion-execution", + "datafusion-expr", + "datafusion-functions", "datafusion-functions-aggregate", - "datafusion-macros 48.0.0 (git+https://github.com/apache/datafusion?rev=48.0.0-rc2)", - "datafusion-physical-expr-common 48.0.0 (git+https://github.com/apache/datafusion?rev=48.0.0-rc2)", + "datafusion-macros", + "datafusion-physical-expr-common", "itertools 0.14.0", "log", "paste", @@ -1896,14 +1723,14 @@ dependencies = [ [[package]] name = "datafusion-functions-table" version = "48.0.0" -source = "git+https://github.com/apache/datafusion?rev=48.0.0-rc2#85f6621a6b1680b40d483a56b10ff3495861ece3" +source = "git+https://github.com/apache/datafusion?rev=48.0.0-rc3#33a32d4382bee7e3c705d0f55d05c24a115a2f98" dependencies = [ "arrow", "async-trait", - "datafusion-catalog 48.0.0 (git+https://github.com/apache/datafusion?rev=48.0.0-rc2)", - "datafusion-common 48.0.0 (git+https://github.com/apache/datafusion?rev=48.0.0-rc2)", - "datafusion-expr 48.0.0 (git+https://github.com/apache/datafusion?rev=48.0.0-rc2)", - "datafusion-physical-plan 48.0.0 (git+https://github.com/apache/datafusion?rev=48.0.0-rc2)", + "datafusion-catalog", + "datafusion-common", + "datafusion-expr", + "datafusion-physical-plan", "parking_lot", "paste", ] @@ -1911,46 +1738,27 @@ dependencies = [ [[package]] name = "datafusion-functions-window" version = "48.0.0" -source = "git+https://github.com/apache/datafusion?rev=48.0.0-rc2#85f6621a6b1680b40d483a56b10ff3495861ece3" +source = "git+https://github.com/apache/datafusion?rev=48.0.0-rc3#33a32d4382bee7e3c705d0f55d05c24a115a2f98" dependencies = [ "arrow", - "datafusion-common 48.0.0 (git+https://github.com/apache/datafusion?rev=48.0.0-rc2)", - "datafusion-doc 48.0.0 (git+https://github.com/apache/datafusion?rev=48.0.0-rc2)", - "datafusion-expr 48.0.0 (git+https://github.com/apache/datafusion?rev=48.0.0-rc2)", - "datafusion-functions-window-common 48.0.0 (git+https://github.com/apache/datafusion?rev=48.0.0-rc2)", - "datafusion-macros 48.0.0 (git+https://github.com/apache/datafusion?rev=48.0.0-rc2)", - "datafusion-physical-expr 48.0.0 (git+https://github.com/apache/datafusion?rev=48.0.0-rc2)", - "datafusion-physical-expr-common 48.0.0 (git+https://github.com/apache/datafusion?rev=48.0.0-rc2)", + "datafusion-common", + "datafusion-doc", + "datafusion-expr", + "datafusion-functions-window-common", + "datafusion-macros", + "datafusion-physical-expr", + "datafusion-physical-expr-common", "log", "paste", ] -[[package]] -name = "datafusion-functions-window-common" -version = "48.0.0" -source = "git+https://github.com/apache/datafusion?rev=48.0.0-rc2#85f6621a6b1680b40d483a56b10ff3495861ece3" -dependencies = [ - "datafusion-common 48.0.0 (git+https://github.com/apache/datafusion?rev=48.0.0-rc2)", - "datafusion-physical-expr-common 48.0.0 (git+https://github.com/apache/datafusion?rev=48.0.0-rc2)", -] - [[package]] name = "datafusion-functions-window-common" version = "48.0.0" source = "git+https://github.com/apache/datafusion?rev=48.0.0-rc3#33a32d4382bee7e3c705d0f55d05c24a115a2f98" dependencies = [ - "datafusion-common 48.0.0 (git+https://github.com/apache/datafusion?rev=48.0.0-rc3)", - "datafusion-physical-expr-common 48.0.0 (git+https://github.com/apache/datafusion?rev=48.0.0-rc3)", -] - -[[package]] -name = "datafusion-macros" -version = "48.0.0" -source = "git+https://github.com/apache/datafusion?rev=48.0.0-rc2#85f6621a6b1680b40d483a56b10ff3495861ece3" -dependencies = [ - "datafusion-expr 48.0.0 (git+https://github.com/apache/datafusion?rev=48.0.0-rc2)", - "quote", - "syn 2.0.101", + "datafusion-common", + "datafusion-physical-expr-common", ] [[package]] @@ -1958,7 +1766,7 @@ name = "datafusion-macros" version = "48.0.0" source = "git+https://github.com/apache/datafusion?rev=48.0.0-rc3#33a32d4382bee7e3c705d0f55d05c24a115a2f98" dependencies = [ - "datafusion-expr 48.0.0 (git+https://github.com/apache/datafusion?rev=48.0.0-rc3)", + "datafusion-expr", "quote", "syn 2.0.101", ] @@ -1966,13 +1774,13 @@ dependencies = [ [[package]] name = "datafusion-optimizer" version = "48.0.0" -source = "git+https://github.com/apache/datafusion?rev=48.0.0-rc2#85f6621a6b1680b40d483a56b10ff3495861ece3" +source = "git+https://github.com/apache/datafusion?rev=48.0.0-rc3#33a32d4382bee7e3c705d0f55d05c24a115a2f98" dependencies = [ "arrow", "chrono", - "datafusion-common 48.0.0 (git+https://github.com/apache/datafusion?rev=48.0.0-rc2)", - "datafusion-expr 48.0.0 (git+https://github.com/apache/datafusion?rev=48.0.0-rc2)", - "datafusion-physical-expr 48.0.0 (git+https://github.com/apache/datafusion?rev=48.0.0-rc2)", + "datafusion-common", + "datafusion-expr", + "datafusion-physical-expr", "indexmap", "itertools 0.14.0", "log", @@ -1980,27 +1788,6 @@ dependencies = [ "regex-syntax", ] -[[package]] -name = "datafusion-physical-expr" -version = "48.0.0" -source = "git+https://github.com/apache/datafusion?rev=48.0.0-rc2#85f6621a6b1680b40d483a56b10ff3495861ece3" -dependencies = [ - "ahash", - "arrow", - "datafusion-common 48.0.0 (git+https://github.com/apache/datafusion?rev=48.0.0-rc2)", - "datafusion-expr 48.0.0 (git+https://github.com/apache/datafusion?rev=48.0.0-rc2)", - "datafusion-expr-common 48.0.0 (git+https://github.com/apache/datafusion?rev=48.0.0-rc2)", - "datafusion-functions-aggregate-common 48.0.0 (git+https://github.com/apache/datafusion?rev=48.0.0-rc2)", - "datafusion-physical-expr-common 48.0.0 (git+https://github.com/apache/datafusion?rev=48.0.0-rc2)", - "half", - "hashbrown 0.14.5", - "indexmap", - "itertools 0.14.0", - "log", - "paste", - "petgraph 0.8.1", -] - [[package]] name = "datafusion-physical-expr" version = "48.0.0" @@ -2008,11 +1795,11 @@ source = "git+https://github.com/apache/datafusion?rev=48.0.0-rc3#33a32d4382bee7 dependencies = [ "ahash", "arrow", - "datafusion-common 48.0.0 (git+https://github.com/apache/datafusion?rev=48.0.0-rc3)", - "datafusion-expr 48.0.0 (git+https://github.com/apache/datafusion?rev=48.0.0-rc3)", - "datafusion-expr-common 48.0.0 (git+https://github.com/apache/datafusion?rev=48.0.0-rc3)", - "datafusion-functions-aggregate-common 48.0.0 (git+https://github.com/apache/datafusion?rev=48.0.0-rc3)", - "datafusion-physical-expr-common 48.0.0 (git+https://github.com/apache/datafusion?rev=48.0.0-rc3)", + "datafusion-common", + "datafusion-expr", + "datafusion-expr-common", + "datafusion-functions-aggregate-common", + "datafusion-physical-expr-common", "half", "hashbrown 0.14.5", "indexmap", @@ -2022,19 +1809,6 @@ dependencies = [ "petgraph 0.8.1", ] -[[package]] -name = "datafusion-physical-expr-common" -version = "48.0.0" -source = "git+https://github.com/apache/datafusion?rev=48.0.0-rc2#85f6621a6b1680b40d483a56b10ff3495861ece3" -dependencies = [ - "ahash", - "arrow", - "datafusion-common 48.0.0 (git+https://github.com/apache/datafusion?rev=48.0.0-rc2)", - "datafusion-expr-common 48.0.0 (git+https://github.com/apache/datafusion?rev=48.0.0-rc2)", - "hashbrown 0.14.5", - "itertools 0.14.0", -] - [[package]] name = "datafusion-physical-expr-common" version = "48.0.0" @@ -2042,8 +1816,8 @@ source = "git+https://github.com/apache/datafusion?rev=48.0.0-rc3#33a32d4382bee7 dependencies = [ "ahash", "arrow", - "datafusion-common 48.0.0 (git+https://github.com/apache/datafusion?rev=48.0.0-rc3)", - "datafusion-expr-common 48.0.0 (git+https://github.com/apache/datafusion?rev=48.0.0-rc3)", + "datafusion-common", + "datafusion-expr-common", "hashbrown 0.14.5", "itertools 0.14.0", ] @@ -2051,47 +1825,18 @@ dependencies = [ [[package]] name = "datafusion-physical-optimizer" version = "48.0.0" -source = "git+https://github.com/apache/datafusion?rev=48.0.0-rc2#85f6621a6b1680b40d483a56b10ff3495861ece3" -dependencies = [ - "arrow", - "datafusion-common 48.0.0 (git+https://github.com/apache/datafusion?rev=48.0.0-rc2)", - "datafusion-execution 48.0.0 (git+https://github.com/apache/datafusion?rev=48.0.0-rc2)", - "datafusion-expr 48.0.0 (git+https://github.com/apache/datafusion?rev=48.0.0-rc2)", - "datafusion-expr-common 48.0.0 (git+https://github.com/apache/datafusion?rev=48.0.0-rc2)", - "datafusion-physical-expr 48.0.0 (git+https://github.com/apache/datafusion?rev=48.0.0-rc2)", - "datafusion-physical-expr-common 48.0.0 (git+https://github.com/apache/datafusion?rev=48.0.0-rc2)", - "datafusion-physical-plan 48.0.0 (git+https://github.com/apache/datafusion?rev=48.0.0-rc2)", - "itertools 0.14.0", - "log", -] - -[[package]] -name = "datafusion-physical-plan" -version = "48.0.0" -source = "git+https://github.com/apache/datafusion?rev=48.0.0-rc2#85f6621a6b1680b40d483a56b10ff3495861ece3" +source = "git+https://github.com/apache/datafusion?rev=48.0.0-rc3#33a32d4382bee7e3c705d0f55d05c24a115a2f98" dependencies = [ - "ahash", "arrow", - "arrow-ord", - "arrow-schema", - "async-trait", - "chrono", - "datafusion-common 48.0.0 (git+https://github.com/apache/datafusion?rev=48.0.0-rc2)", - "datafusion-common-runtime 48.0.0 (git+https://github.com/apache/datafusion?rev=48.0.0-rc2)", - "datafusion-execution 48.0.0 (git+https://github.com/apache/datafusion?rev=48.0.0-rc2)", - "datafusion-expr 48.0.0 (git+https://github.com/apache/datafusion?rev=48.0.0-rc2)", - "datafusion-functions-window-common 48.0.0 (git+https://github.com/apache/datafusion?rev=48.0.0-rc2)", - "datafusion-physical-expr 48.0.0 (git+https://github.com/apache/datafusion?rev=48.0.0-rc2)", - "datafusion-physical-expr-common 48.0.0 (git+https://github.com/apache/datafusion?rev=48.0.0-rc2)", - "futures", - "half", - "hashbrown 0.14.5", - "indexmap", + "datafusion-common", + "datafusion-execution", + "datafusion-expr", + "datafusion-expr-common", + "datafusion-physical-expr", + "datafusion-physical-expr-common", + "datafusion-physical-plan", "itertools 0.14.0", "log", - "parking_lot", - "pin-project-lite", - "tokio", ] [[package]] @@ -2105,13 +1850,13 @@ dependencies = [ "arrow-schema", "async-trait", "chrono", - "datafusion-common 48.0.0 (git+https://github.com/apache/datafusion?rev=48.0.0-rc3)", - "datafusion-common-runtime 48.0.0 (git+https://github.com/apache/datafusion?rev=48.0.0-rc3)", - "datafusion-execution 48.0.0 (git+https://github.com/apache/datafusion?rev=48.0.0-rc3)", - "datafusion-expr 48.0.0 (git+https://github.com/apache/datafusion?rev=48.0.0-rc3)", - "datafusion-functions-window-common 48.0.0 (git+https://github.com/apache/datafusion?rev=48.0.0-rc3)", - "datafusion-physical-expr 48.0.0 (git+https://github.com/apache/datafusion?rev=48.0.0-rc3)", - "datafusion-physical-expr-common 48.0.0 (git+https://github.com/apache/datafusion?rev=48.0.0-rc3)", + "datafusion-common", + "datafusion-common-runtime", + "datafusion-execution", + "datafusion-expr", + "datafusion-functions-window-common", + "datafusion-physical-expr", + "datafusion-physical-expr-common", "futures", "half", "hashbrown 0.14.5", @@ -2123,29 +1868,6 @@ dependencies = [ "tokio", ] -[[package]] -name = "datafusion-session" -version = "48.0.0" -source = "git+https://github.com/apache/datafusion?rev=48.0.0-rc2#85f6621a6b1680b40d483a56b10ff3495861ece3" -dependencies = [ - "arrow", - "async-trait", - "dashmap", - "datafusion-common 48.0.0 (git+https://github.com/apache/datafusion?rev=48.0.0-rc2)", - "datafusion-common-runtime 48.0.0 (git+https://github.com/apache/datafusion?rev=48.0.0-rc2)", - "datafusion-execution 48.0.0 (git+https://github.com/apache/datafusion?rev=48.0.0-rc2)", - "datafusion-expr 48.0.0 (git+https://github.com/apache/datafusion?rev=48.0.0-rc2)", - "datafusion-physical-expr 48.0.0 (git+https://github.com/apache/datafusion?rev=48.0.0-rc2)", - "datafusion-physical-plan 48.0.0 (git+https://github.com/apache/datafusion?rev=48.0.0-rc2)", - "datafusion-sql 48.0.0 (git+https://github.com/apache/datafusion?rev=48.0.0-rc2)", - "futures", - "itertools 0.14.0", - "log", - "object_store", - "parking_lot", - "tokio", -] - [[package]] name = "datafusion-session" version = "48.0.0" @@ -2154,13 +1876,13 @@ dependencies = [ "arrow", "async-trait", "dashmap", - "datafusion-common 48.0.0 (git+https://github.com/apache/datafusion?rev=48.0.0-rc3)", - "datafusion-common-runtime 48.0.0 (git+https://github.com/apache/datafusion?rev=48.0.0-rc3)", - "datafusion-execution 48.0.0 (git+https://github.com/apache/datafusion?rev=48.0.0-rc3)", - "datafusion-expr 48.0.0 (git+https://github.com/apache/datafusion?rev=48.0.0-rc3)", - "datafusion-physical-expr 48.0.0 (git+https://github.com/apache/datafusion?rev=48.0.0-rc3)", - "datafusion-physical-plan 48.0.0 (git+https://github.com/apache/datafusion?rev=48.0.0-rc3)", - "datafusion-sql 48.0.0 (git+https://github.com/apache/datafusion?rev=48.0.0-rc3)", + "datafusion-common", + "datafusion-common-runtime", + "datafusion-execution", + "datafusion-expr", + "datafusion-physical-expr", + "datafusion-physical-plan", + "datafusion-sql", "futures", "itertools 0.14.0", "log", @@ -2175,28 +1897,13 @@ version = "48.0.0" source = "git+https://github.com/apache/datafusion?rev=48.0.0-rc3#33a32d4382bee7e3c705d0f55d05c24a115a2f98" dependencies = [ "arrow", - "datafusion-catalog 48.0.0 (git+https://github.com/apache/datafusion?rev=48.0.0-rc3)", - "datafusion-common 48.0.0 (git+https://github.com/apache/datafusion?rev=48.0.0-rc3)", - "datafusion-execution 48.0.0 (git+https://github.com/apache/datafusion?rev=48.0.0-rc3)", - "datafusion-expr 48.0.0 (git+https://github.com/apache/datafusion?rev=48.0.0-rc3)", - "datafusion-functions 48.0.0 (git+https://github.com/apache/datafusion?rev=48.0.0-rc3)", - "datafusion-macros 48.0.0 (git+https://github.com/apache/datafusion?rev=48.0.0-rc3)", - "log", -] - -[[package]] -name = "datafusion-sql" -version = "48.0.0" -source = "git+https://github.com/apache/datafusion?rev=48.0.0-rc2#85f6621a6b1680b40d483a56b10ff3495861ece3" -dependencies = [ - "arrow", - "bigdecimal", - "datafusion-common 48.0.0 (git+https://github.com/apache/datafusion?rev=48.0.0-rc2)", - "datafusion-expr 48.0.0 (git+https://github.com/apache/datafusion?rev=48.0.0-rc2)", - "indexmap", + "datafusion-catalog", + "datafusion-common", + "datafusion-execution", + "datafusion-expr", + "datafusion-functions", + "datafusion-macros", "log", - "regex", - "sqlparser", ] [[package]] @@ -2206,8 +1913,8 @@ source = "git+https://github.com/apache/datafusion?rev=48.0.0-rc3#33a32d4382bee7 dependencies = [ "arrow", "bigdecimal", - "datafusion-common 48.0.0 (git+https://github.com/apache/datafusion?rev=48.0.0-rc3)", - "datafusion-expr 48.0.0 (git+https://github.com/apache/datafusion?rev=48.0.0-rc3)", + "datafusion-common", + "datafusion-expr", "indexmap", "log", "regex", diff --git a/spark/src/test/scala/org/apache/comet/CometFuzzTestSuite.scala b/spark/src/test/scala/org/apache/comet/CometFuzzTestSuite.scala index a1b1812b31..d0751aaff2 100644 --- a/spark/src/test/scala/org/apache/comet/CometFuzzTestSuite.scala +++ b/spark/src/test/scala/org/apache/comet/CometFuzzTestSuite.scala @@ -250,10 +250,25 @@ class CometFuzzTestSuite extends CometTestBase with AdaptiveSparkPlanHelper { val df = spark.read.parquet(filename) val df2 = df.repartition(8, df.col("c0")).sort("c1") 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 => + CometConf.COMET_SHUFFLE_MODE.get() match { + case "jvm" => + 1 + case "native" => + 0 + } + case CometConf.SCAN_NATIVE_ICEBERG_COMPAT | CometConf.SCAN_NATIVE_DATAFUSION => + CometConf.COMET_SHUFFLE_MODE.get() match { + case "jvm" => + 1 + case "native" => + 2 + } } + + val cometShuffles = collectCometShuffleExchanges(df2.queryExecution.executedPlan) + assert(expectedNumCometShuffles == cometShuffles.length) } test("join") { From 652688720acd00c749f7559a7a992ad7fa2a0606 Mon Sep 17 00:00:00 2001 From: Matt Butrovich Date: Mon, 9 Jun 2025 15:08:14 -0400 Subject: [PATCH 38/72] Update golden plans. --- .../approved-plans-v1_4-spark3_5/q2/explain.txt | 6 +++--- .../approved-plans-v1_4-spark3_5/q2/simplified.txt | 2 +- .../approved-plans-v1_4-spark3_5/q31/explain.txt | 6 +++--- .../approved-plans-v1_4-spark3_5/q31/simplified.txt | 2 +- .../approved-plans-v1_4-spark3_5/q34/explain.txt | 6 +++--- .../approved-plans-v1_4-spark3_5/q34/simplified.txt | 2 +- .../approved-plans-v1_4-spark3_5/q39a/explain.txt | 6 +++--- .../approved-plans-v1_4-spark3_5/q39a/simplified.txt | 2 +- .../approved-plans-v1_4-spark3_5/q39b/explain.txt | 6 +++--- .../approved-plans-v1_4-spark3_5/q39b/simplified.txt | 2 +- .../approved-plans-v1_4-spark3_5/q64/explain.txt | 6 +++--- .../approved-plans-v1_4-spark3_5/q64/simplified.txt | 2 +- .../approved-plans-v1_4-spark3_5/q71/explain.txt | 6 +++--- .../approved-plans-v1_4-spark3_5/q71/simplified.txt | 2 +- .../approved-plans-v1_4-spark3_5/q73/explain.txt | 6 +++--- .../approved-plans-v1_4-spark3_5/q73/simplified.txt | 2 +- .../approved-plans-v1_4-spark3_5/q91/explain.txt | 6 +++--- .../approved-plans-v1_4-spark3_5/q91/simplified.txt | 2 +- .../approved-plans-v1_4-spark4_0/q34/explain.txt | 6 +++--- .../approved-plans-v1_4-spark4_0/q34/simplified.txt | 2 +- .../approved-plans-v1_4-spark4_0/q64/explain.txt | 6 +++--- .../approved-plans-v1_4-spark4_0/q64/simplified.txt | 2 +- .../approved-plans-v1_4-spark4_0/q73/explain.txt | 6 +++--- .../approved-plans-v1_4-spark4_0/q73/simplified.txt | 2 +- .../tpcds-plan-stability/approved-plans-v1_4/q2/explain.txt | 6 +++--- .../approved-plans-v1_4/q2/simplified.txt | 2 +- .../approved-plans-v1_4/q31/explain.txt | 6 +++--- .../approved-plans-v1_4/q31/simplified.txt | 2 +- .../approved-plans-v1_4/q34/explain.txt | 6 +++--- .../approved-plans-v1_4/q34/simplified.txt | 2 +- .../approved-plans-v1_4/q39a/explain.txt | 6 +++--- .../approved-plans-v1_4/q39a/simplified.txt | 2 +- .../approved-plans-v1_4/q39b/explain.txt | 6 +++--- .../approved-plans-v1_4/q39b/simplified.txt | 2 +- .../approved-plans-v1_4/q64/explain.txt | 6 +++--- .../approved-plans-v1_4/q64/simplified.txt | 2 +- .../approved-plans-v1_4/q71/explain.txt | 6 +++--- .../approved-plans-v1_4/q71/simplified.txt | 2 +- .../approved-plans-v1_4/q73/explain.txt | 6 +++--- .../approved-plans-v1_4/q73/simplified.txt | 2 +- .../approved-plans-v1_4/q91/explain.txt | 6 +++--- .../approved-plans-v1_4/q91/simplified.txt | 2 +- .../approved-plans-v2_7-spark3_5/q34/explain.txt | 6 +++--- .../approved-plans-v2_7-spark3_5/q34/simplified.txt | 2 +- .../approved-plans-v2_7-spark3_5/q64/explain.txt | 6 +++--- .../approved-plans-v2_7-spark3_5/q64/simplified.txt | 2 +- .../approved-plans-v2_7-spark4_0/q34/explain.txt | 6 +++--- .../approved-plans-v2_7-spark4_0/q34/simplified.txt | 2 +- .../approved-plans-v2_7-spark4_0/q64/explain.txt | 6 +++--- .../approved-plans-v2_7-spark4_0/q64/simplified.txt | 2 +- .../approved-plans-v2_7/q34/explain.txt | 6 +++--- .../approved-plans-v2_7/q34/simplified.txt | 2 +- .../approved-plans-v2_7/q64/explain.txt | 6 +++--- .../approved-plans-v2_7/q64/simplified.txt | 2 +- 54 files changed, 108 insertions(+), 108 deletions(-) diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q2/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q2/explain.txt index 5ff3a2a9ba..61c6e2f564 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 8e6713bf26..a38ccedce7 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 6ce03737eb..2a4b0fc580 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 22635bb9f2..4591720269 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 13934b6c66..ce3305fae3 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 7f1cb740c4..9bc47a48a8 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 0b404d5868..ec8969bbd5 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 9df61b1669..8397df62dd 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 683ec549cc..cf2ac9b891 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 9df61b1669..8397df62dd 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 03ba8783bb..46e0ae9610 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 ffcec55ac5..137ee8251e 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 9539905b2c..b596f48235 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 d765255d14..49adad0a02 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q71/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q71/simplified.txt @@ -2,7 +2,7 @@ WholeStageCodegen (1) CometColumnarToRow InputAdapter CometSort [brand_id,brand,t_hour,t_minute,ext_price] - CometColumnarExchange [ext_price,brand_id] #1 + CometExchange [ext_price,brand_id] #1 CometHashAggregate [brand_id,brand,t_hour,t_minute,ext_price,i_brand,i_brand_id,sum,sum(UnscaledValue(ext_price))] CometExchange [i_brand,i_brand_id,t_hour,t_minute] #2 CometHashAggregate [i_brand,i_brand_id,t_hour,t_minute,sum,ext_price] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q73/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q73/explain.txt index 77957c5099..87ceb7a009 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 e3c1c0b82e..92802523fa 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 b5f0f9083b..dc3a05dbbc 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 3b302272e0..c160227efd 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q91/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q91/simplified.txt @@ -2,7 +2,7 @@ WholeStageCodegen (1) CometColumnarToRow InputAdapter CometSort [Call_Center,Call_Center_Name,Manager,Returns_Loss] - CometColumnarExchange [Returns_Loss] #1 + CometExchange [Returns_Loss] #1 CometHashAggregate [Call_Center,Call_Center_Name,Manager,Returns_Loss,cc_call_center_id,cc_name,cc_manager,cd_marital_status,cd_education_status,sum,sum(UnscaledValue(cr_net_loss))] CometExchange [cc_call_center_id,cc_name,cc_manager,cd_marital_status,cd_education_status] #2 CometHashAggregate [cc_call_center_id,cc_name,cc_manager,cd_marital_status,cd_education_status,sum,cr_net_loss] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q34/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q34/explain.txt index 13934b6c66..ce3305fae3 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 7f1cb740c4..9bc47a48a8 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/q64/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q64/explain.txt index 720daf075d..e3c22a1c5c 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 (186) +- CometSort (185) - +- CometColumnarExchange (184) + +- CometExchange (184) +- CometProject (183) +- CometSortMergeJoin (182) :- CometSort (113) @@ -1019,9 +1019,9 @@ Arguments: [item_sk#95, store_name#96, store_zip#97], [item_sk#181, store_name#1 Input [25]: [product_name#94, item_sk#95, store_name#96, store_zip#97, b_street_number#98, b_streen_name#99, b_city#100, b_zip#101, c_street_number#102, c_street_name#103, c_city#104, c_zip#105, syear#106, cnt#107, s1#108, s2#109, s3#110, item_sk#181, store_name#182, store_zip#183, syear#184, cnt#185, s1#186, s2#187, s3#188] Arguments: [product_name#94, store_name#96, store_zip#97, b_street_number#98, b_streen_name#99, b_city#100, b_zip#101, c_street_number#102, c_street_name#103, c_city#104, c_zip#105, syear#106, cnt#107, s1#108, s2#109, s3#110, s1#186, s2#187, s3#188, syear#184, cnt#185], [product_name#94, store_name#96, store_zip#97, b_street_number#98, b_streen_name#99, b_city#100, b_zip#101, c_street_number#102, c_street_name#103, c_city#104, c_zip#105, syear#106, cnt#107, s1#108, s2#109, s3#110, s1#186, s2#187, s3#188, syear#184, cnt#185] -(184) CometColumnarExchange +(184) CometExchange Input [21]: [product_name#94, store_name#96, store_zip#97, b_street_number#98, b_streen_name#99, b_city#100, b_zip#101, c_street_number#102, c_street_name#103, c_city#104, c_zip#105, syear#106, cnt#107, s1#108, s2#109, s3#110, s1#186, s2#187, s3#188, syear#184, cnt#185] -Arguments: rangepartitioning(product_name#94 ASC NULLS FIRST, store_name#96 ASC NULLS FIRST, cnt#185 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=17] +Arguments: rangepartitioning(product_name#94 ASC NULLS FIRST, store_name#96 ASC NULLS FIRST, cnt#185 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=17] (185) CometSort Input [21]: [product_name#94, store_name#96, store_zip#97, b_street_number#98, b_streen_name#99, b_city#100, b_zip#101, c_street_number#102, c_street_name#103, c_city#104, c_zip#105, syear#106, cnt#107, s1#108, s2#109, s3#110, s1#186, s2#187, s3#188, syear#184, cnt#185] 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 043b82dfa2..35f71c1207 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 (39) CometColumnarToRow InputAdapter CometSort [product_name,store_name,store_zip,b_street_number,b_streen_name,b_city,b_zip,c_street_number,c_street_name,c_city,c_zip,syear,cnt,s1,s2,s3,s1,s2,s3,syear,cnt] - CometColumnarExchange [product_name,store_name,cnt] #1 + 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 77957c5099..87ceb7a009 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 e3c1c0b82e..92802523fa 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 c83c8be0a3..5b4a6372c1 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 8e6713bf26..a38ccedce7 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 6ce03737eb..2a4b0fc580 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 22635bb9f2..4591720269 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 13934b6c66..ce3305fae3 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 7f1cb740c4..9bc47a48a8 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 0b404d5868..ec8969bbd5 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 9df61b1669..8397df62dd 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 683ec549cc..cf2ac9b891 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 9df61b1669..8397df62dd 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 03ba8783bb..46e0ae9610 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 ffcec55ac5..137ee8251e 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 9539905b2c..b596f48235 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 d765255d14..49adad0a02 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q71/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q71/simplified.txt @@ -2,7 +2,7 @@ WholeStageCodegen (1) CometColumnarToRow InputAdapter CometSort [brand_id,brand,t_hour,t_minute,ext_price] - CometColumnarExchange [ext_price,brand_id] #1 + CometExchange [ext_price,brand_id] #1 CometHashAggregate [brand_id,brand,t_hour,t_minute,ext_price,i_brand,i_brand_id,sum,sum(UnscaledValue(ext_price))] CometExchange [i_brand,i_brand_id,t_hour,t_minute] #2 CometHashAggregate [i_brand,i_brand_id,t_hour,t_minute,sum,ext_price] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q73/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q73/explain.txt index 77957c5099..87ceb7a009 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 e3c1c0b82e..92802523fa 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 b5f0f9083b..dc3a05dbbc 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 3b302272e0..c160227efd 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q91/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q91/simplified.txt @@ -2,7 +2,7 @@ WholeStageCodegen (1) CometColumnarToRow InputAdapter CometSort [Call_Center,Call_Center_Name,Manager,Returns_Loss] - CometColumnarExchange [Returns_Loss] #1 + CometExchange [Returns_Loss] #1 CometHashAggregate [Call_Center,Call_Center_Name,Manager,Returns_Loss,cc_call_center_id,cc_name,cc_manager,cd_marital_status,cd_education_status,sum,sum(UnscaledValue(cr_net_loss))] CometExchange [cc_call_center_id,cc_name,cc_manager,cd_marital_status,cd_education_status] #2 CometHashAggregate [cc_call_center_id,cc_name,cc_manager,cd_marital_status,cd_education_status,sum,cr_net_loss] 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 37fa3cf15f..1e73653868 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 7bef10cb97..d0b65bd9cd 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 2e195d003a..2d4d9b9dc1 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 e3c8d3cd2c..9c6b8f1985 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 37fa3cf15f..1e73653868 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 7bef10cb97..d0b65bd9cd 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 c5e3046383..f27e04a898 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 (186) +- CometSort (185) - +- CometColumnarExchange (184) + +- CometExchange (184) +- CometProject (183) +- CometSortMergeJoin (182) :- CometSort (113) @@ -1019,9 +1019,9 @@ Arguments: [item_sk#95, store_name#96, store_zip#97], [item_sk#181, store_name#1 Input [25]: [product_name#94, item_sk#95, store_name#96, store_zip#97, b_street_number#98, b_streen_name#99, b_city#100, b_zip#101, c_street_number#102, c_street_name#103, c_city#104, c_zip#105, syear#106, cnt#107, s1#108, s2#109, s3#110, item_sk#181, store_name#182, store_zip#183, syear#184, cnt#185, s1#186, s2#187, s3#188] Arguments: [product_name#94, store_name#96, store_zip#97, b_street_number#98, b_streen_name#99, b_city#100, b_zip#101, c_street_number#102, c_street_name#103, c_city#104, c_zip#105, syear#106, cnt#107, s1#108, s2#109, s3#110, s1#186, s2#187, s3#188, syear#184, cnt#185], [product_name#94, store_name#96, store_zip#97, b_street_number#98, b_streen_name#99, b_city#100, b_zip#101, c_street_number#102, c_street_name#103, c_city#104, c_zip#105, syear#106, cnt#107, s1#108, s2#109, s3#110, s1#186, s2#187, s3#188, syear#184, cnt#185] -(184) CometColumnarExchange +(184) CometExchange Input [21]: [product_name#94, store_name#96, store_zip#97, b_street_number#98, b_streen_name#99, b_city#100, b_zip#101, c_street_number#102, c_street_name#103, c_city#104, c_zip#105, syear#106, cnt#107, s1#108, s2#109, s3#110, s1#186, s2#187, s3#188, syear#184, cnt#185] -Arguments: rangepartitioning(product_name#94 ASC NULLS FIRST, store_name#96 ASC NULLS FIRST, cnt#185 ASC NULLS FIRST, s1#108 ASC NULLS FIRST, s1#186 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=17] +Arguments: rangepartitioning(product_name#94 ASC NULLS FIRST, store_name#96 ASC NULLS FIRST, cnt#185 ASC NULLS FIRST, s1#108 ASC NULLS FIRST, s1#186 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=17] (185) CometSort Input [21]: [product_name#94, store_name#96, store_zip#97, b_street_number#98, b_streen_name#99, b_city#100, b_zip#101, c_street_number#102, c_street_name#103, c_city#104, c_zip#105, syear#106, cnt#107, s1#108, s2#109, s3#110, s1#186, s2#187, s3#188, syear#184, cnt#185] 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 356584ffa5..4e32f307ee 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 (39) CometColumnarToRow InputAdapter CometSort [product_name,store_name,store_zip,b_street_number,b_streen_name,b_city,b_zip,c_street_number,c_street_name,c_city,c_zip,syear,cnt,s1,s2,s3,s1,s2,s3,syear,cnt] - CometColumnarExchange [product_name,store_name,cnt,s1,s1] #1 + 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 37fa3cf15f..1e73653868 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 7bef10cb97..d0b65bd9cd 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 2e195d003a..2d4d9b9dc1 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 e3c8d3cd2c..9c6b8f1985 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] From 19466c1db1c1ad173dcef9fb9394b3f4fd40f067 Mon Sep 17 00:00:00 2001 From: Matt Butrovich Date: Mon, 9 Jun 2025 15:51:02 -0400 Subject: [PATCH 39/72] Update CometFuzzTestSuite. --- .../scala/org/apache/comet/CometFuzzTestSuite.scala | 10 ++++++++++ 1 file changed, 10 insertions(+) diff --git a/spark/src/test/scala/org/apache/comet/CometFuzzTestSuite.scala b/spark/src/test/scala/org/apache/comet/CometFuzzTestSuite.scala index d0751aaff2..433bdf4675 100644 --- a/spark/src/test/scala/org/apache/comet/CometFuzzTestSuite.scala +++ b/spark/src/test/scala/org/apache/comet/CometFuzzTestSuite.scala @@ -162,7 +162,9 @@ class CometFuzzTestSuite extends CometTestBase with AdaptiveSparkPlanHelper { // cannot run fully natively due to range partitioning and sort val (_, cometPlan) = checkSparkAnswer(sql) if (usingDataSourceExec) { + // Native scans and native shuffle should support all data types for this query assert(1 == collectNativeScans(cometPlan).length) + assert(1 == collectCometShuffleExchanges(cometPlan).length) } } } @@ -187,7 +189,9 @@ class CometFuzzTestSuite extends CometTestBase with AdaptiveSparkPlanHelper { // cannot run fully natively due to range partitioning and sort val (_, cometPlan) = checkSparkAnswer(sql) if (usingDataSourceExec) { + // Native scans and native shuffle should support all data types for this query assert(1 == collectNativeScans(cometPlan).length) + assert(1 == collectCometShuffleExchanges(cometPlan).length) } } @@ -254,15 +258,21 @@ class CometFuzzTestSuite extends CometTestBase with AdaptiveSparkPlanHelper { case CometConf.SCAN_NATIVE_COMET => CometConf.COMET_SHUFFLE_MODE.get() match { case "jvm" => + // Uses a single CometColumnarExchange for hash partitioning + // Range partitioning is in Spark 1 case "native" => + // Uses Spark for hash partitioning and range partitioning 0 } case CometConf.SCAN_NATIVE_ICEBERG_COMPAT | CometConf.SCAN_NATIVE_DATAFUSION => CometConf.COMET_SHUFFLE_MODE.get() match { case "jvm" => + // Uses a single CometColumnarExchange for hash partitioning + // Range partitioning is in Spark 1 case "native" => + // Uses Comet for hash partitioning and range partitioning 2 } } From 4a451cf3df845e1674c8275e51abd42c2b72e6bd Mon Sep 17 00:00:00 2001 From: Matt Butrovich Date: Mon, 9 Jun 2025 16:07:41 -0400 Subject: [PATCH 40/72] Reduce warnings. --- .../src/main/scala/org/apache/comet/serde/QueryPlanSerde.scala | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/spark/src/main/scala/org/apache/comet/serde/QueryPlanSerde.scala b/spark/src/main/scala/org/apache/comet/serde/QueryPlanSerde.scala index 4771a865ce..b126a4ed06 100644 --- a/spark/src/main/scala/org/apache/comet/serde/QueryPlanSerde.scala +++ b/spark/src/main/scala/org/apache/comet/serde/QueryPlanSerde.scala @@ -2903,8 +2903,7 @@ object QueryPlanSerde extends Logging with CometExprShim { supported case SinglePartition => inputs.forall(attr => supportedShuffleDataType(attr.dataType)) - case RangePartitioning(ordering, numPartitions) => - // TODO: Apply any expression constraints similar to HashPartitioning above. + case RangePartitioning(_, _) => true case _ => msg = s"unsupported Spark partitioning: ${partitioning.getClass.getName}" From b39ccb08c8b7b04f2aecd2ecf681d82435664eab Mon Sep 17 00:00:00 2001 From: Matt Butrovich Date: Mon, 9 Jun 2025 18:55:45 -0400 Subject: [PATCH 41/72] clippy --- native/core/src/execution/shuffle/range_partitioner.rs | 4 ++-- native/core/src/execution/shuffle/shuffle_writer.rs | 6 +++--- 2 files changed, 5 insertions(+), 5 deletions(-) diff --git a/native/core/src/execution/shuffle/range_partitioner.rs b/native/core/src/execution/shuffle/range_partitioner.rs index 288706051d..9c6b07daf9 100644 --- a/native/core/src/execution/shuffle/range_partitioner.rs +++ b/native/core/src/execution/shuffle/range_partitioner.rs @@ -191,7 +191,7 @@ mod test { .iter() .for_each(|&idx| assert!(idx < batch_size as u64)); // Check that values are distinct and not out of bounds - let sorted_indices = indices.into_iter().sorted().collect_vec(); + let sorted_indices = indices.iter().sorted().collect_vec(); assert_eq!( sorted_indices.len(), sorted_indices.iter().dedup().collect_vec().len() @@ -203,7 +203,7 @@ mod test { 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 = vec![0u32; 10]; + 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(); diff --git a/native/core/src/execution/shuffle/shuffle_writer.rs b/native/core/src/execution/shuffle/shuffle_writer.rs index c554ca3a5a..9e7113e996 100644 --- a/native/core/src/execution/shuffle/shuffle_writer.rs +++ b/native/core/src/execution/shuffle/shuffle_writer.rs @@ -439,7 +439,7 @@ impl MultiPartitionShuffleRepartitioner { .for_each(|partition_id| partition_counters[*partition_id as usize] += 1); } - fn accumulate_partition_counters(partition_ends: &mut Vec) { + fn accumulate_partition_counters(partition_ends: &mut [u32]) { // accumulate partition counters into partition ends // e.g. partition counter: [1, 3, 2, 1, 0] => [1, 4, 6, 7, 7] // this is basically an inclusive scan/prefix sum @@ -519,7 +519,7 @@ impl MultiPartitionShuffleRepartitioner { } count_partitions( - &mut scratch.partition_ids[..arrays[0].len()], + &scratch.partition_ids[..arrays[0].len()], &mut scratch.partition_starts, *num_output_partitions, ); @@ -591,7 +591,7 @@ impl MultiPartitionShuffleRepartitioner { } count_partitions( - &mut scratch.partition_ids[..partition_arrays[0].len()], + &scratch.partition_ids[..partition_arrays[0].len()], &mut scratch.partition_starts, *num_output_partitions, ); From 6f7f0eb51f4bd9419c1106f84b3924980a278229 Mon Sep 17 00:00:00 2001 From: Matt Butrovich Date: Mon, 9 Jun 2025 19:05:50 -0400 Subject: [PATCH 42/72] More clippy. --- .../core/src/execution/shuffle/range_partitioner.rs | 11 ++++------- 1 file changed, 4 insertions(+), 7 deletions(-) diff --git a/native/core/src/execution/shuffle/range_partitioner.rs b/native/core/src/execution/shuffle/range_partitioner.rs index 9c6b07daf9..fa7bdbe17e 100644 --- a/native/core/src/execution/shuffle/range_partitioner.rs +++ b/native/core/src/execution/shuffle/range_partitioner.rs @@ -169,10 +169,10 @@ impl RangePartitioner { #[cfg(test)] mod test { use super::*; - use arrow::array::{Array, AsArray, Int32Array, Int64Array, RecordBatch, UInt64Array}; + use arrow::array::{Array, AsArray, Int64Array, RecordBatch, UInt64Array}; use arrow::compute::take_record_batch; use arrow::datatypes::DataType::{Float64, Int64}; - use arrow::datatypes::{DataType, Field, Float64Type, Int32Type, Int64Type, Schema}; + use arrow::datatypes::{Field, Float64Type, Int32Type, Int64Type, Schema}; use datafusion::common::{record_batch, HashSet}; use itertools::Itertools; use std::sync::Arc; @@ -273,11 +273,8 @@ mod test { // org.apache.spark.util.random.SamplingUtilsSuite // "SPARK-18678 reservoirSampleAndCount with tiny input" fn reservoir_sample_and_count_with_tiny_input() { - let column = vec![0, 1]; - let array = Arc::new(Int32Array::from(column)); - let schema = Arc::new(Schema::new(vec![Field::new("a", DataType::Int32, true)])); - let batch = RecordBatch::try_new(Arc::clone(&schema), vec![array.clone()]).unwrap(); - let mut counts: Vec = vec![0; array.len()]; + 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); let result = sample_batch(batch.clone(), indices); From 49ad6182e0bca76b3fdfe13e6e705f6ab1f29834 Mon Sep 17 00:00:00 2001 From: Matt Butrovich Date: Mon, 9 Jun 2025 19:48:25 -0400 Subject: [PATCH 43/72] More clippy that my local runs do not show. --- native/core/src/execution/shuffle/range_partitioner.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/native/core/src/execution/shuffle/range_partitioner.rs b/native/core/src/execution/shuffle/range_partitioner.rs index fa7bdbe17e..51dd2c5e31 100644 --- a/native/core/src/execution/shuffle/range_partitioner.rs +++ b/native/core/src/execution/shuffle/range_partitioner.rs @@ -182,7 +182,7 @@ mod test { take_record_batch(&input, &indices).unwrap() } - fn check_indices(indices: &Vec, batch_size: usize, sample_size: usize) { + fn check_indices(indices: &[u64], batch_size: usize, sample_size: usize) { // sample indices size should never exceed the batch size assert!(indices.len() <= batch_size); assert_eq!(indices.len(), batch_size.min(sample_size)); From de956eee9b27aa0743b6af03ee9a5717760dcfe1 Mon Sep 17 00:00:00 2001 From: Matt Butrovich Date: Tue, 10 Jun 2025 10:27:13 -0400 Subject: [PATCH 44/72] Get RangePartitioning sample size from SQLConf. --- native/core/benches/shuffle_writer.rs | 1 + native/core/src/execution/planner.rs | 1 + .../src/execution/shuffle/comet_partitioning.rs | 13 +++++++------ .../core/src/execution/shuffle/shuffle_writer.rs | 10 +++++++--- native/proto/src/proto/partitioning.proto | 1 + .../shuffle/CometNativeShuffleWriter.scala | 16 ++++++++++++++++ 6 files changed, 33 insertions(+), 9 deletions(-) diff --git a/native/core/benches/shuffle_writer.rs b/native/core/benches/shuffle_writer.rs index 246154804e..ee07462d00 100644 --- a/native/core/benches/shuffle_writer.rs +++ b/native/core/benches/shuffle_writer.rs @@ -91,6 +91,7 @@ fn criterion_benchmark(c: &mut Criterion) { col("c0", batch.schema().as_ref()).unwrap(), )]), 16, + 100, ), ] { let compression_codec = CompressionCodec::None; diff --git a/native/core/src/execution/planner.rs b/native/core/src/execution/planner.rs index 7e1713bf3a..2c946025b9 100644 --- a/native/core/src/execution/planner.rs +++ b/native/core/src/execution/planner.rs @@ -2212,6 +2212,7 @@ impl PhysicalPlanner { Ok(CometPartitioning::RangePartitioning( lex_ordering, range_partition.num_partitions as usize, + range_partition.sample_size as usize, )) } PartitioningStruct::SinglePartition(_) => Ok(CometPartitioning::UnknownPartitioning(1)), diff --git a/native/core/src/execution/shuffle/comet_partitioning.rs b/native/core/src/execution/shuffle/comet_partitioning.rs index a369dffb40..da9f5c32d7 100644 --- a/native/core/src/execution/shuffle/comet_partitioning.rs +++ b/native/core/src/execution/shuffle/comet_partitioning.rs @@ -27,7 +27,7 @@ pub enum CometPartitioning { Hash(Vec>, usize), /// Allocate rows based on lexical order of one of more expressions and the specified number of /// partitions - RangePartitioning(LexOrdering, usize), + RangePartitioning(LexOrdering, usize, usize), /// Unknown partitioning scheme with a known number of partitions UnknownPartitioning(usize), } @@ -36,9 +36,10 @@ impl CometPartitioning { pub fn partition_count(&self) -> usize { use CometPartitioning::*; match self { - RoundRobinBatch(n) | Hash(_, n) | UnknownPartitioning(n) | RangePartitioning(_, n) => { - *n - } + RoundRobinBatch(n) + | Hash(_, n) + | UnknownPartitioning(n) + | RangePartitioning(_, n, _) => *n, } } } @@ -67,8 +68,8 @@ impl From for Partitioning { CometPartitioning::UnknownPartitioning(partitions) => { Partitioning::UnknownPartitioning(partitions) } - CometPartitioning::RangePartitioning(_lex_ordering, usize) => { - Partitioning::UnknownPartitioning(usize) + CometPartitioning::RangePartitioning(_, partitions, _) => { + Partitioning::UnknownPartitioning(partitions) } } } diff --git a/native/core/src/execution/shuffle/shuffle_writer.rs b/native/core/src/execution/shuffle/shuffle_writer.rs index 9e7113e996..52c2e3e540 100644 --- a/native/core/src/execution/shuffle/shuffle_writer.rs +++ b/native/core/src/execution/shuffle/shuffle_writer.rs @@ -546,7 +546,11 @@ impl MultiPartitionShuffleRepartitioner { .await?; self.scratch = scratch; } - CometPartitioning::RangePartitioning(lex_ordering, num_output_partitions) => { + CometPartitioning::RangePartitioning( + lex_ordering, + num_output_partitions, + sample_size, + ) => { let mut scratch = std::mem::take(&mut self.scratch); let (partition_starts, partition_row_indices): (&Vec, &Vec) = { let mut timer = self.metrics.repart_time.timer(); @@ -558,13 +562,12 @@ impl MultiPartitionShuffleRepartitioner { .collect::>>()?; if self.row_converter.is_none() { - // TODO: Adjust sample size. let (bounds_rows, row_converter) = RangePartitioner::generate_bounds( &partition_arrays, lex_ordering, *num_output_partitions, input.num_rows(), - 100, + *sample_size, ); self.bounds_rows = Some(bounds_rows); @@ -1414,6 +1417,7 @@ mod test { col("a", batch.schema().as_ref()).unwrap(), )]), num_partitions, + 100, ), ] { 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 db28a64df5..ea0e586fb1 100644 --- a/native/proto/src/proto/partitioning.proto +++ b/native/proto/src/proto/partitioning.proto @@ -45,4 +45,5 @@ message SinglePartition { message RangePartition { repeated spark.spark_expression.Expr sort_orders = 1; int32 num_partitions = 2; + int32 sample_size = 3; } 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 e99d2d77c3..5913b7fcb9 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 @@ -31,6 +31,7 @@ import org.apache.spark.sql.catalyst.expressions.Attribute 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 @@ -191,6 +192,21 @@ 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 change: we don't divide by numPartitions since each DF plan handles one partition. + math.ceil(3.0 * sampleSize).toInt + } + partitioning.setSampleSize(sampleSize) val orderingExprs = rangePartitioning.ordering .flatMap(e => QueryPlanSerde.exprToProto(e, outputAttributes)) From dc250fd1aff50db4260875f64d756cebfbd6c8e1 Mon Sep 17 00:00:00 2001 From: Matt Butrovich Date: Tue, 10 Jun 2025 10:47:15 -0400 Subject: [PATCH 45/72] Fast path generate_bounds for sample_size >= num_rows. --- .../execution/shuffle/range_partitioner.rs | 78 +++++++------------ .../src/execution/shuffle/shuffle_writer.rs | 2 +- .../shuffle/CometNativeShuffleWriter.scala | 2 +- 3 files changed, 30 insertions(+), 52 deletions(-) diff --git a/native/core/src/execution/shuffle/range_partitioner.rs b/native/core/src/execution/shuffle/range_partitioner.rs index 51dd2c5e31..cdfc55107f 100644 --- a/native/core/src/execution/shuffle/range_partitioner.rs +++ b/native/core/src/execution/shuffle/range_partitioner.rs @@ -30,11 +30,10 @@ impl RangePartitioner { // input array at the end. pub fn reservoir_sample_indices(num_rows: usize, sample_size: usize) -> Vec { assert!(sample_size > 0); - - if num_rows <= sample_size { - // Just return the original input since we can't create a bigger sample. - return (0..num_rows as u64).collect(); - } + 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(); @@ -72,22 +71,26 @@ impl RangePartitioner { partition_arrays: &Vec, lex_ordering: &LexOrdering, num_output_partitions: usize, - num_rows: usize, // TODO: u16 - sample_size: usize, // TODO: u16 + num_rows: usize, + sample_size: usize, ) -> (Rows, RowConverter) { - let sample_indices = UInt64Array::from(RangePartitioner::reservoir_sample_indices( - num_rows, - sample_size, - )); - - let sampled_columns = take_arrays( - partition_arrays, - &sample_indices, - Some(TakeOptions { - check_bounds: false, - }), - ) - .unwrap(); + let sampled_columns = if sample_size < num_rows { + let sample_indices = UInt64Array::from(RangePartitioner::reservoir_sample_indices( + num_rows, + sample_size, + )); + + take_arrays( + partition_arrays, + &sample_indices, + Some(TakeOptions { + check_bounds: false, + }), + ) + .unwrap() + } else { + partition_arrays.clone() + }; let sort_fields: Vec = partition_arrays .iter() @@ -232,43 +235,18 @@ mod test { } #[test] - fn reservoir_sample_fuzz() { + fn reservoir_sample_random() { let mut rng = rand::rng(); for _ in 0..1000 { let batch_size: usize = rng.random_range(0..=8192); - let sample_size: usize = rng.random_range(1..=8192); - let reservoir = RangePartitioner::reservoir_sample_indices(batch_size, sample_size); - - assert_eq!(reservoir.len(), sample_size.min(batch_size)); + let sample_size: usize = rng.random_range(1..batch_size); + let indices = RangePartitioner::reservoir_sample_indices(batch_size, sample_size); - let mut set: HashSet = HashSet::with_capacity(sample_size); - reservoir.iter().for_each(|&idx| { - assert!(idx < batch_size as u64); - assert!(set.insert(idx)); - }); + check_indices(&indices, batch_size, sample_size); } } - #[test] - // org.apache.spark.util.random.SamplingUtilsSuite - // "reservoirSampleAndCount" - fn reservoir_sample() { - let batch = create_random_batch(100, false, None); - // sample_size > batch.num_rows returns entire batch after sampling - let sample1_indices = RangePartitioner::reservoir_sample_indices(batch.num_rows(), 150); - check_indices(&sample1_indices, batch.num_rows(), 150); - assert_eq!(batch, sample_batch(batch.clone(), sample1_indices)); - // sample_size == batch.num_rows returns entire batch after sampling - let sample2_indices = RangePartitioner::reservoir_sample_indices(batch.num_rows(), 100); - check_indices(&sample2_indices, batch.num_rows(), 100); - assert_eq!(batch, sample_batch(batch.clone(), sample2_indices)); - // sample_size < batch.num_rows returns a random subset, so can't compare to original batch - let sample3_indices = RangePartitioner::reservoir_sample_indices(batch.num_rows(), 10); - check_indices(&sample3_indices, batch.num_rows(), 10); - assert_eq!(sample3_indices.len(), 10); - } - #[test] // org.apache.spark.util.random.SamplingUtilsSuite // "SPARK-18678 reservoirSampleAndCount with tiny input" @@ -358,7 +336,7 @@ mod test { } #[test] - fn determine_bounds_fuzz() { + fn determine_bounds_random() { let mut rng = rand::rng(); let sort_fields = vec![SortField::new(Int64)]; diff --git a/native/core/src/execution/shuffle/shuffle_writer.rs b/native/core/src/execution/shuffle/shuffle_writer.rs index 52c2e3e540..0f8a70859d 100644 --- a/native/core/src/execution/shuffle/shuffle_writer.rs +++ b/native/core/src/execution/shuffle/shuffle_writer.rs @@ -1417,7 +1417,7 @@ mod test { col("a", batch.schema().as_ref()).unwrap(), )]), num_partitions, - 100, + ((100.0 * num_partitions as f64).min(1e6) * 3.0) as usize, ), ] { let batches = (0..num_batches).map(|_| batch.clone()).collect::>(); 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 5913b7fcb9..7986997a05 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 @@ -203,7 +203,7 @@ class CometNativeShuffleWriter[K, V]( .toDouble * outputPartitioning.numPartitions, 1e6) // Assume the input partitions are roughly balanced and over-sample a little bit. - // Comet change: we don't divide by numPartitions since each DF plan handles one partition. + // Comet: we don't divide by numPartitions since each DF plan handles one partition. math.ceil(3.0 * sampleSize).toInt } partitioning.setSampleSize(sampleSize) From c8df6cbd46d89bc823c55cda94a853d2fbe6094d Mon Sep 17 00:00:00 2001 From: Matt Butrovich Date: Tue, 10 Jun 2025 11:37:08 -0400 Subject: [PATCH 46/72] Seed reservoir sampling from partition number like Spark. --- .../execution/shuffle/range_partitioner.rs | 32 ++++++++++--------- .../src/execution/shuffle/shuffle_writer.rs | 3 ++ 2 files changed, 20 insertions(+), 15 deletions(-) diff --git a/native/core/src/execution/shuffle/range_partitioner.rs b/native/core/src/execution/shuffle/range_partitioner.rs index cdfc55107f..206c6ed4ce 100644 --- a/native/core/src/execution/shuffle/range_partitioner.rs +++ b/native/core/src/execution/shuffle/range_partitioner.rs @@ -19,16 +19,16 @@ use arrow::array::{ArrayRef, UInt64Array}; use arrow::compute::{take_arrays, TakeOptions}; use arrow::row::{Row, RowConverter, Rows, SortField}; use datafusion::physical_expr::LexOrdering; -use rand::Rng; +use rand::{rngs::SmallRng, Rng, SeedableRng}; pub struct RangePartitioner; impl RangePartitioner { // Adapted from https://en.wikipedia.org/wiki/Reservoir_sampling#Optimal:_Algorithm_L - // We use sample_size instead of k and input_length instead of n. - // We use indices in the reservoir instead of actual values since we'll do one take() on the - // input array at the end. - pub fn reservoir_sample_indices(num_rows: usize, sample_size: usize) -> Vec { + // 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. + pub fn reservoir_sample_indices(num_rows: usize, sample_size: usize, seed: u64) -> Vec { assert!(sample_size > 0); assert!( num_rows > sample_size, @@ -38,7 +38,7 @@ impl RangePartitioner { // Initialize our reservoir with indices of the first |sample_size| elements. let mut reservoir: Vec = (0..sample_size as u64).collect(); - let mut rng = rand::rng(); + 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; @@ -73,11 +73,13 @@ impl RangePartitioner { num_output_partitions: usize, num_rows: usize, sample_size: usize, + seed: u64, ) -> (Rows, RowConverter) { let sampled_columns = if sample_size < num_rows { let sample_indices = UInt64Array::from(RangePartitioner::reservoir_sample_indices( num_rows, sample_size, + seed, )); take_arrays( @@ -236,12 +238,12 @@ mod test { #[test] fn reservoir_sample_random() { - let mut rng = rand::rng(); + let mut rng = SmallRng::seed_from_u64(42); - for _ in 0..1000 { - let batch_size: usize = rng.random_range(0..=8192); + for _ in 0..8192 { + let batch_size: usize = rng.random_range(1..=8192); let sample_size: usize = rng.random_range(1..batch_size); - let indices = RangePartitioner::reservoir_sample_indices(batch_size, sample_size); + let indices = RangePartitioner::reservoir_sample_indices(batch_size, sample_size, 42); check_indices(&indices, batch_size, sample_size); } @@ -253,8 +255,8 @@ mod test { 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); + 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; @@ -337,11 +339,11 @@ mod test { #[test] fn determine_bounds_random() { - let mut rng = rand::rng(); + let mut rng = SmallRng::seed_from_u64(42); let sort_fields = vec![SortField::new(Int64)]; - for _ in 0..1000 { + for _ in 0..2048 { let batch_size = rng.random_range(0..=8192); let num_partitions = rng.random_range(2..1048576); @@ -398,7 +400,7 @@ mod test { } fn create_random_batch(batch_size: u32, sort: bool, range: Option<(i64, i64)>) -> RecordBatch { - let mut rng = rand::rng(); + let mut rng = SmallRng::seed_from_u64(42); let mut column: Vec = if let Some((min, max)) = range { assert!(min <= max); (0..batch_size) diff --git a/native/core/src/execution/shuffle/shuffle_writer.rs b/native/core/src/execution/shuffle/shuffle_writer.rs index 0f8a70859d..871b524cf8 100644 --- a/native/core/src/execution/shuffle/shuffle_writer.rs +++ b/native/core/src/execution/shuffle/shuffle_writer.rs @@ -336,6 +336,7 @@ struct MultiPartitionShuffleRepartitioner { /// RangePartitioning-specific state bounds_rows: Option, row_converter: Option, + seed: u64, } #[derive(Default)] @@ -413,6 +414,7 @@ impl MultiPartitionShuffleRepartitioner { tracing_enabled, bounds_rows: None, row_converter: None, + seed: partition as u64, }) } @@ -568,6 +570,7 @@ impl MultiPartitionShuffleRepartitioner { *num_output_partitions, input.num_rows(), *sample_size, + self.seed, ); self.bounds_rows = Some(bounds_rows); From 295f5326f7c5e974a385140da2d4bdbd3ad81a80 Mon Sep 17 00:00:00 2001 From: Matt Butrovich Date: Tue, 10 Jun 2025 12:35:47 -0400 Subject: [PATCH 47/72] Docs and refactoring to shuffle_writer. More to do yet. --- .../src/execution/shuffle/shuffle_writer.rs | 83 +++++++++---------- 1 file changed, 40 insertions(+), 43 deletions(-) diff --git a/native/core/src/execution/shuffle/shuffle_writer.rs b/native/core/src/execution/shuffle/shuffle_writer.rs index 871b524cf8..bce0fd2b57 100644 --- a/native/core/src/execution/shuffle/shuffle_writer.rs +++ b/native/core/src/execution/shuffle/shuffle_writer.rs @@ -380,6 +380,7 @@ impl MultiPartitionShuffleRepartitioner { // The initial values are not used. let scratch = ScratchSpace { hashes_buf: match partitioning { + // Only allocate the hashes_buf if hash partitioning. CometPartitioning::Hash(_, _) => vec![0; batch_size], _ => vec![], }, @@ -414,6 +415,7 @@ impl MultiPartitionShuffleRepartitioner { tracing_enabled, bounds_rows: None, row_converter: None, + // Spark RangePartitioner seeds off of partition number. seed: partition as u64, }) } @@ -428,34 +430,30 @@ impl MultiPartitionShuffleRepartitioner { return Ok(()); } - fn count_partitions( - partition_ids: &[u32], - partition_counters: &mut Vec, + fn map_partition_ids_to_starts_and_indices( + scratch: &mut ScratchSpace, num_output_partitions: usize, + num_rows: usize, ) { + let partition_ids = &mut scratch.partition_ids[..num_rows]; + // count each partition size, while leaving the last extra element as 0 + let partition_counters = &mut scratch.partition_starts; partition_counters.resize(num_output_partitions + 1, 0); partition_counters.fill(0); partition_ids .iter() .for_each(|partition_id| partition_counters[*partition_id as usize] += 1); - } - fn accumulate_partition_counters(partition_ends: &mut [u32]) { // accumulate partition counters into partition ends // e.g. partition counter: [1, 3, 2, 1, 0] => [1, 4, 6, 7, 7] - // this is basically an inclusive scan/prefix sum + let partition_ends = partition_counters; let mut accum = 0; partition_ends.iter_mut().for_each(|v| { *v += accum; accum = *v; }); - } - fn calculate_partition_row_indices_and_partition_starts( - scratch: &mut ScratchSpace, - num_rows: usize, - ) { // calculate partition row indices and partition starts // e.g. partition ids: [3, 1, 1, 1, 2, 2, 0] will produce the following partition_row_indices // and partition_starts arrays: @@ -467,15 +465,14 @@ impl MultiPartitionShuffleRepartitioner { // Each slice partition_row_indices[partition_starts[K]..partition_starts[K + 1]] contains the // row indices of the input batch that are partitioned into partition K. For example, // first partition 0 has one row index [6], partition 1 has row indices [1, 2, 3], etc. - let partition_ids = &mut scratch.partition_ids[..num_rows]; let partition_row_indices = &mut scratch.partition_row_indices; - let partition_ends = &mut scratch.partition_starts; partition_row_indices.resize(num_rows, 0); for (index, partition_id) in partition_ids.iter().enumerate().rev() { partition_ends[*partition_id as usize] -= 1; let end = partition_ends[*partition_id as usize]; partition_row_indices[end as usize] = index as u32; } + // after calculating, partition ends become partition starts } @@ -499,19 +496,22 @@ impl MultiPartitionShuffleRepartitioner { let (partition_starts, partition_row_indices): (&Vec, &Vec) = { let mut timer = self.metrics.repart_time.timer(); - // evaluate partition expressions + // Evaluate partition expressions to get rows to apply partitioning scheme. let arrays = exprs .iter() .map(|expr| expr.evaluate(&input)?.into_array(input.num_rows())) .collect::>>()?; - // use identical seed as spark hash partition - let hashes_buf = &mut scratch.hashes_buf[..arrays[0].len()]; + let num_rows = arrays[0].len(); + + // Use identical seed as Spark hash partitioning. + let hashes_buf = &mut scratch.hashes_buf[..num_rows]; hashes_buf.fill(42_u32); + // Generate partition ids for every row. { - // Hash arrays and compute buckets based on number of partitions - let partition_ids = &mut scratch.partition_ids[..arrays[0].len()]; + // Hash arrays and compute partition ids based on number of partitions. + let partition_ids = &mut scratch.partition_ids[..num_rows]; create_murmur3_hashes(&arrays, hashes_buf)? .iter() .enumerate() @@ -520,17 +520,12 @@ impl MultiPartitionShuffleRepartitioner { }); } - count_partitions( - &scratch.partition_ids[..arrays[0].len()], - &mut scratch.partition_starts, - *num_output_partitions, - ); - - accumulate_partition_counters(&mut scratch.partition_starts); - - calculate_partition_row_indices_and_partition_starts( + // 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. + map_partition_ids_to_starts_and_indices( &mut scratch, - arrays[0].len(), + *num_output_partitions, + num_rows, ); timer.stop(); @@ -557,15 +552,19 @@ impl MultiPartitionShuffleRepartitioner { let (partition_starts, partition_row_indices): (&Vec, &Vec) = { let mut timer = self.metrics.repart_time.timer(); - // evaluate partition expressions - let partition_arrays = lex_ordering + // Evaluate partition expressions for values to apply partitioning scheme on. + let arrays = lex_ordering .iter() .map(|expr| expr.expr.evaluate(&input)?.into_array(input.num_rows())) .collect::>>()?; + 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( - &partition_arrays, + &arrays, lex_ordering, *num_output_partitions, input.num_rows(), @@ -577,13 +576,16 @@ impl MultiPartitionShuffleRepartitioner { 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(partition_arrays.as_slice())?; + .convert_columns(arrays.as_slice())?; { - let partition_ids = &mut scratch.partition_ids[..partition_arrays[0].len()]; + let partition_ids = &mut scratch.partition_ids[..num_rows]; // TODO: Try to cache this vector. let bounds_rows_vec = @@ -596,17 +598,12 @@ impl MultiPartitionShuffleRepartitioner { ); } - count_partitions( - &scratch.partition_ids[..partition_arrays[0].len()], - &mut scratch.partition_starts, - *num_output_partitions, - ); - - accumulate_partition_counters(&mut scratch.partition_starts); - - calculate_partition_row_indices_and_partition_starts( + // 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. + map_partition_ids_to_starts_and_indices( &mut scratch, - partition_arrays[0].len(), + *num_output_partitions, + num_rows, ); timer.stop(); From b5e4a6e42e8f61f9e1e5e5cfbf0ce80afd00ad56 Mon Sep 17 00:00:00 2001 From: Matt Butrovich Date: Tue, 10 Jun 2025 13:45:42 -0400 Subject: [PATCH 48/72] Update docs and fix sampled columns in generate_bounds. --- .../execution/shuffle/range_partitioner.rs | 53 ++++++++++++------- 1 file changed, 33 insertions(+), 20 deletions(-) diff --git a/native/core/src/execution/shuffle/range_partitioner.rs b/native/core/src/execution/shuffle/range_partitioner.rs index 206c6ed4ce..2064d48d5b 100644 --- a/native/core/src/execution/shuffle/range_partitioner.rs +++ b/native/core/src/execution/shuffle/range_partitioner.rs @@ -24,10 +24,12 @@ use rand::{rngs::SmallRng, Rng, SeedableRng}; pub struct RangePartitioner; impl RangePartitioner { - // 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. + /// 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. pub fn reservoir_sample_indices(num_rows: usize, sample_size: usize, seed: u64) -> Vec { assert!(sample_size > 0); assert!( @@ -56,6 +58,9 @@ impl RangePartitioner { 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: &Vec, @@ -67,6 +72,9 @@ impl RangePartitioner { }); } + /// Given input arrays and range partitioning metadata: samples the input arrays, generates + /// partition bounds, and returns Rows (for comparison against) and a RowConverter (for + /// adapting future incoming batches). pub fn generate_bounds( partition_arrays: &Vec, lex_ordering: &LexOrdering, @@ -76,12 +84,14 @@ impl RangePartitioner { seed: u64, ) -> (Rows, RowConverter) { 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, @@ -91,9 +101,11 @@ impl RangePartitioner { ) .unwrap() } 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) @@ -104,14 +116,14 @@ impl RangePartitioner { let (bounds_indices, row_converter) = RangePartitioner::determine_bounds_for_rows( sort_fields, - sampled_columns, + 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( - partition_arrays, + sampled_columns.as_slice(), &bounds_indices_array, Some(TakeOptions { check_bounds: false, @@ -119,6 +131,7 @@ impl RangePartitioner { ) .unwrap(); + // Convert the bounds data to Rows and return with RowConverter. ( row_converter .convert_columns(bounds_arrays.as_slice()) @@ -127,18 +140,19 @@ impl RangePartitioner { ) } - // Adapted from org.apache.spark.RangePartitioner.determineBounds + /// 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. pub fn determine_bounds_for_rows( sort_fields: Vec, - sampled_columns: Vec, + sampled_columns: &[ArrayRef], partitions: usize, ) -> (Vec, RowConverter) { assert!(partitions > 1); let converter = RowConverter::new(sort_fields).unwrap(); - let sampled_rows = converter - .convert_columns(sampled_columns.as_slice()) - .unwrap(); + let sampled_rows = converter.convert_columns(sampled_columns).unwrap(); let mut sorted_sampled_rows: Vec<(usize, Row)> = sampled_rows.iter().enumerate().collect(); sorted_sampled_rows.sort_unstable_by(|(_, a), (_, b)| a.cmp(b)); @@ -157,7 +171,6 @@ impl RangePartitioner { if cumulative_weights >= target { // Skip duplicate values. if previous_bound.is_none() || key.1 > previous_bound.unwrap() { - // bounds.push(key.1); bounds_indices.push(key.0 as u64); target += step; j += 1; @@ -292,7 +305,7 @@ mod test { let sort_fields = vec![SortField::new(Float64)]; let (rows, _) = - RangePartitioner::determine_bounds_for_rows(sort_fields, Vec::from(batch.columns()), 3); + RangePartitioner::determine_bounds_for_rows(sort_fields, batch.columns(), 3); assert_eq!(rows.len(), 2); @@ -313,7 +326,7 @@ mod test { let mut num_partitions = batch.num_rows() - 1; let (rows, _) = RangePartitioner::determine_bounds_for_rows( sort_fields.clone(), - Vec::from(batch.columns()), + batch.columns(), num_partitions, ); assert_eq!(rows.len(), num_partitions - 1); @@ -322,7 +335,7 @@ mod test { num_partitions = batch.num_rows(); let (rows, _) = RangePartitioner::determine_bounds_for_rows( sort_fields.clone(), - Vec::from(batch.columns()), + batch.columns(), num_partitions, ); assert_eq!(rows.len(), num_partitions - 1); @@ -331,7 +344,7 @@ mod test { num_partitions = batch.num_rows() + 1; let (rows, _) = RangePartitioner::determine_bounds_for_rows( sort_fields.clone(), - Vec::from(batch.columns()), + batch.columns(), num_partitions, ); assert_eq!(rows.len(), batch.num_rows()); @@ -351,14 +364,14 @@ mod test { let (rows, _) = RangePartitioner::determine_bounds_for_rows( sort_fields.clone(), - Vec::from(batch.columns()), + batch.columns(), num_partitions, ); if batch_size < num_partitions as u32 { assert_eq!(rows.len(), batch_size as usize); } else { - assert_eq!(rows.len(), (num_partitions - 1) as usize); + assert_eq!(rows.len(), num_partitions - 1); } let mut set: HashSet = HashSet::with_capacity(rows.len()); @@ -388,7 +401,7 @@ mod test { let sort_fields = vec![SortField::new(Float64)]; let (rows, _) = - RangePartitioner::determine_bounds_for_rows(sort_fields, Vec::from(batch.columns()), 2); + RangePartitioner::determine_bounds_for_rows(sort_fields, batch.columns(), 2); assert_eq!(rows.len(), 1); From ad79a0f01be9dc47a3e7185ea186dfe5af6091e1 Mon Sep 17 00:00:00 2001 From: Matt Butrovich Date: Tue, 10 Jun 2025 14:29:07 -0400 Subject: [PATCH 49/72] Reduce test time. --- native/core/src/execution/shuffle/range_partitioner.rs | 2 +- native/core/src/execution/shuffle/shuffle_writer.rs | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/native/core/src/execution/shuffle/range_partitioner.rs b/native/core/src/execution/shuffle/range_partitioner.rs index 2064d48d5b..3c419e7889 100644 --- a/native/core/src/execution/shuffle/range_partitioner.rs +++ b/native/core/src/execution/shuffle/range_partitioner.rs @@ -358,7 +358,7 @@ mod test { for _ in 0..2048 { let batch_size = rng.random_range(0..=8192); - let num_partitions = rng.random_range(2..1048576); + let num_partitions = rng.random_range(2..1000); let batch = create_random_batch(batch_size, false, None); diff --git a/native/core/src/execution/shuffle/shuffle_writer.rs b/native/core/src/execution/shuffle/shuffle_writer.rs index bce0fd2b57..0c565c8c78 100644 --- a/native/core/src/execution/shuffle/shuffle_writer.rs +++ b/native/core/src/execution/shuffle/shuffle_writer.rs @@ -1417,7 +1417,7 @@ mod test { col("a", batch.schema().as_ref()).unwrap(), )]), num_partitions, - ((100.0 * num_partitions as f64).min(1e6) * 3.0) as usize, + 100, ), ] { let batches = (0..num_batches).map(|_| batch.clone()).collect::>(); From 50ab5697af7b0ecc46502a85de7b5e1d21e2fe23 Mon Sep 17 00:00:00 2001 From: Matt Butrovich Date: Tue, 10 Jun 2025 15:04:28 -0400 Subject: [PATCH 50/72] More docs and more testing. --- .../execution/shuffle/range_partitioner.rs | 95 +++++++++---------- 1 file changed, 44 insertions(+), 51 deletions(-) diff --git a/native/core/src/execution/shuffle/range_partitioner.rs b/native/core/src/execution/shuffle/range_partitioner.rs index 3c419e7889..33a9a8f8fc 100644 --- a/native/core/src/execution/shuffle/range_partitioner.rs +++ b/native/core/src/execution/shuffle/range_partitioner.rs @@ -191,7 +191,7 @@ mod test { 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, HashSet}; + use datafusion::common::record_batch; use itertools::Itertools; use std::sync::Arc; @@ -200,15 +200,31 @@ mod test { take_record_batch(&input, &indices).unwrap() } - fn check_indices(indices: &[u64], batch_size: usize, sample_size: usize) { + 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 values are distinct and not out of bounds + // Check that indices are not out of bounds indices .iter() .for_each(|&idx| assert!(idx < batch_size as u64)); - // Check that values are distinct and not out of bounds + // 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(), @@ -250,15 +266,20 @@ mod test { } #[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); for _ in 0..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_indices(&indices, batch_size, sample_size); + check_sample_indices(&indices, batch_size, sample_size); } } @@ -307,6 +328,8 @@ mod test { let (rows, _) = RangePartitioner::determine_bounds_for_rows(sort_fields, batch.columns(), 3); + check_bounds_indices(rows.as_slice(), batch.num_rows()); + assert_eq!(rows.len(), 2); let indices = UInt64Array::from(rows); @@ -317,40 +340,9 @@ mod test { } #[test] - fn determine_bounds_sizes() { - let batch = record_batch!(("a", Float64, vec![Some(0.1), Some(0.2), Some(0.3),])).unwrap(); - - let sort_fields = vec![SortField::new(Float64)]; - - // num_partitions < sample size - let mut num_partitions = batch.num_rows() - 1; - let (rows, _) = RangePartitioner::determine_bounds_for_rows( - sort_fields.clone(), - batch.columns(), - num_partitions, - ); - assert_eq!(rows.len(), num_partitions - 1); - - // num_partitions == sample size - num_partitions = batch.num_rows(); - let (rows, _) = RangePartitioner::determine_bounds_for_rows( - sort_fields.clone(), - batch.columns(), - num_partitions, - ); - assert_eq!(rows.len(), num_partitions - 1); - - // num_partitions > sample size - num_partitions = batch.num_rows() + 1; - let (rows, _) = RangePartitioner::determine_bounds_for_rows( - sort_fields.clone(), - batch.columns(), - num_partitions, - ); - assert_eq!(rows.len(), batch.num_rows()); - } - - #[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); @@ -358,7 +350,9 @@ mod test { for _ in 0..2048 { let batch_size = rng.random_range(0..=8192); - let num_partitions = rng.random_range(2..1000); + // 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); @@ -368,17 +362,7 @@ mod test { num_partitions, ); - if batch_size < num_partitions as u32 { - assert_eq!(rows.len(), batch_size as usize); - } else { - assert_eq!(rows.len(), num_partitions - 1); - } - - let mut set: HashSet = HashSet::with_capacity(rows.len()); - rows.iter().for_each(|&idx| { - assert!(idx < batch_size as u64); - assert!(set.insert(idx)); - }); + check_bounds_indices(rows.as_slice(), batch_size as usize); let rows_array = UInt64Array::from(rows); @@ -390,11 +374,20 @@ mod test { .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(); From 18a62cf8ca3b71b82956100e368c5869b5131091 Mon Sep 17 00:00:00 2001 From: Matt Butrovich Date: Tue, 10 Jun 2025 15:32:17 -0400 Subject: [PATCH 51/72] Add warning for large sampleSize. --- .../execution/shuffle/CometNativeShuffleWriter.scala | 8 +++++++- 1 file changed, 7 insertions(+), 1 deletion(-) diff --git a/spark/src/main/scala/org/apache/spark/sql/comet/execution/shuffle/CometNativeShuffleWriter.scala b/spark/src/main/scala/org/apache/spark/sql/comet/execution/shuffle/CometNativeShuffleWriter.scala index 7986997a05..5d772be403 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 @@ -25,6 +25,7 @@ import java.nio.file.{Files, Paths} import scala.collection.JavaConverters.asJavaIterableConverter 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 @@ -51,7 +52,8 @@ class CometNativeShuffleWriter[K, V]( mapId: Long, context: TaskContext, metricsReporter: ShuffleWriteMetricsReporter) - extends ShuffleWriter[K, V] { + extends ShuffleWriter[K, V] + with Logging { private val OFFSET_LENGTH = 8 @@ -206,6 +208,10 @@ class CometNativeShuffleWriter[K, V]( // 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 From 13836992ac03e3d42e106707e2d808a0da6a28ba Mon Sep 17 00:00:00 2001 From: Matt Butrovich Date: Tue, 10 Jun 2025 16:04:49 -0400 Subject: [PATCH 52/72] More test docs. --- native/core/src/execution/shuffle/range_partitioner.rs | 7 +++++++ 1 file changed, 7 insertions(+) diff --git a/native/core/src/execution/shuffle/range_partitioner.rs b/native/core/src/execution/shuffle/range_partitioner.rs index 33a9a8f8fc..68a2a17df6 100644 --- a/native/core/src/execution/shuffle/range_partitioner.rs +++ b/native/core/src/execution/shuffle/range_partitioner.rs @@ -233,6 +233,11 @@ mod test { } #[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(); @@ -260,6 +265,8 @@ mod test { .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)); From 6e850d4674b28b33fba0add2e06ce754b5192094 Mon Sep 17 00:00:00 2001 From: Matt Butrovich Date: Tue, 10 Jun 2025 19:12:13 -0400 Subject: [PATCH 53/72] More tests. --- .../execution/shuffle/range_partitioner.rs | 38 +++++++++++++++++++ 1 file changed, 38 insertions(+) diff --git a/native/core/src/execution/shuffle/range_partitioner.rs b/native/core/src/execution/shuffle/range_partitioner.rs index 68a2a17df6..bfa30a21ae 100644 --- a/native/core/src/execution/shuffle/range_partitioner.rs +++ b/native/core/src/execution/shuffle/range_partitioner.rs @@ -192,6 +192,8 @@ mod test { 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; @@ -232,6 +234,42 @@ mod test { ); } + #[test] + fn partition_sorted_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, 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, + 10, + input_batch.num_rows(), + 8192, + 42, + ); + + 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 From 4663d29982cd80f6e85666c2583f0e6cba0ca362 Mon Sep 17 00:00:00 2001 From: Matt Butrovich Date: Wed, 11 Jun 2025 08:23:49 -0400 Subject: [PATCH 54/72] Update test. --- .../src/execution/shuffle/range_partitioner.rs | 15 +++++++-------- 1 file changed, 7 insertions(+), 8 deletions(-) diff --git a/native/core/src/execution/shuffle/range_partitioner.rs b/native/core/src/execution/shuffle/range_partitioner.rs index bfa30a21ae..5e9fbcebd9 100644 --- a/native/core/src/execution/shuffle/range_partitioner.rs +++ b/native/core/src/execution/shuffle/range_partitioner.rs @@ -235,12 +235,11 @@ mod test { } #[test] - fn partition_sorted_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]; - + // 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( @@ -252,7 +251,7 @@ mod test { &lex_ordering, 10, input_batch.num_rows(), - 8192, + 1000, 42, ); @@ -266,7 +265,7 @@ mod test { - primitive_array.values().get(i).unwrap()) .abs() < 1000 - ) + ); } } From 640c6ef8f6a159a095f975b3854937b1203e95a6 Mon Sep 17 00:00:00 2001 From: Matt Butrovich Date: Wed, 11 Jun 2025 08:51:27 -0400 Subject: [PATCH 55/72] Address feedback. --- .../execution/shuffle/range_partitioner.rs | 41 +++++++++---------- .../src/execution/shuffle/shuffle_writer.rs | 2 +- 2 files changed, 21 insertions(+), 22 deletions(-) diff --git a/native/core/src/execution/shuffle/range_partitioner.rs b/native/core/src/execution/shuffle/range_partitioner.rs index 5e9fbcebd9..029fa13e86 100644 --- a/native/core/src/execution/shuffle/range_partitioner.rs +++ b/native/core/src/execution/shuffle/range_partitioner.rs @@ -18,6 +18,7 @@ use arrow::array::{ArrayRef, UInt64Array}; use arrow::compute::{take_arrays, TakeOptions}; use arrow::row::{Row, RowConverter, Rows, SortField}; +use datafusion::error::{DataFusionError, Result}; use datafusion::physical_expr::LexOrdering; use rand::{rngs::SmallRng, Rng, SeedableRng}; @@ -30,7 +31,7 @@ impl RangePartitioner { /// We use sample_size instead of k and num_rows instead of n. /// We use indices instead of actual values in the reservoir since we'll do one take() on the /// input arrays at the end. - pub fn reservoir_sample_indices(num_rows: usize, sample_size: usize, seed: u64) -> Vec { + fn reservoir_sample_indices(num_rows: usize, sample_size: usize, seed: u64) -> Vec { assert!(sample_size > 0); assert!( num_rows > sample_size, @@ -82,7 +83,7 @@ impl RangePartitioner { num_rows: usize, sample_size: usize, seed: u64, - ) -> (Rows, RowConverter) { + ) -> 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( @@ -98,8 +99,7 @@ impl RangePartitioner { Some(TakeOptions { check_bounds: false, }), - ) - .unwrap() + )? } else { // Requested sample_size is larger than the batch, so just use the batch. partition_arrays.clone() @@ -118,7 +118,7 @@ impl RangePartitioner { 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); @@ -128,31 +128,28 @@ impl RangePartitioner { Some(TakeOptions { check_bounds: false, }), - ) - .unwrap(); + )?; // Convert the bounds data to Rows and return with RowConverter. - ( - row_converter - .convert_columns(bounds_arrays.as_slice()) - .unwrap(), + 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. - pub fn determine_bounds_for_rows( + fn determine_bounds_for_rows( sort_fields: Vec, sampled_columns: &[ArrayRef], partitions: usize, - ) -> (Vec, RowConverter) { + ) -> Result<(Vec, RowConverter), DataFusionError> { assert!(partitions > 1); - let converter = RowConverter::new(sort_fields).unwrap(); - let sampled_rows = converter.convert_columns(sampled_columns).unwrap(); + 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)); @@ -180,7 +177,7 @@ impl RangePartitioner { i += 1 } - (bounds_indices, converter) + Ok((bounds_indices, converter)) } } @@ -253,7 +250,8 @@ mod test { input_batch.num_rows(), 1000, 42, - ); + ) + .unwrap(); let rows_array = row_converter.convert_rows(&rows).unwrap(); @@ -370,7 +368,7 @@ mod test { let sort_fields = vec![SortField::new(Float64)]; let (rows, _) = - RangePartitioner::determine_bounds_for_rows(sort_fields, batch.columns(), 3); + RangePartitioner::determine_bounds_for_rows(sort_fields, batch.columns(), 3).unwrap(); check_bounds_indices(rows.as_slice(), batch.num_rows()); @@ -404,7 +402,8 @@ mod test { sort_fields.clone(), batch.columns(), num_partitions, - ); + ) + .unwrap(); check_bounds_indices(rows.as_slice(), batch_size as usize); @@ -438,7 +437,7 @@ mod test { let sort_fields = vec![SortField::new(Float64)]; let (rows, _) = - RangePartitioner::determine_bounds_for_rows(sort_fields, batch.columns(), 2); + RangePartitioner::determine_bounds_for_rows(sort_fields, batch.columns(), 2).unwrap(); assert_eq!(rows.len(), 1); diff --git a/native/core/src/execution/shuffle/shuffle_writer.rs b/native/core/src/execution/shuffle/shuffle_writer.rs index 0c565c8c78..e02ea9bd73 100644 --- a/native/core/src/execution/shuffle/shuffle_writer.rs +++ b/native/core/src/execution/shuffle/shuffle_writer.rs @@ -570,7 +570,7 @@ impl MultiPartitionShuffleRepartitioner { input.num_rows(), *sample_size, self.seed, - ); + )?; self.bounds_rows = Some(bounds_rows); self.row_converter = Some(row_converter); From a19ac49a5d73bb9d6d077558d58144084a5fa838 Mon Sep 17 00:00:00 2001 From: Matt Butrovich Date: Wed, 11 Jun 2025 11:33:16 -0400 Subject: [PATCH 56/72] Added configs, extended CometNativeShuffleSuite. --- .../scala/org/apache/comet/CometConf.scala | 16 +++++++ .../apache/comet/serde/QueryPlanSerde.scala | 6 ++- .../comet/exec/CometNativeShuffleSuite.scala | 43 ++++++++++++++++--- 3 files changed, 58 insertions(+), 7 deletions(-) diff --git a/common/src/main/scala/org/apache/comet/CometConf.scala b/common/src/main/scala/org/apache/comet/CometConf.scala index 317303eb7b..a82a33151f 100644 --- a/common/src/main/scala/org/apache/comet/CometConf.scala +++ b/common/src/main/scala/org/apache/comet/CometConf.scala @@ -307,6 +307,18 @@ object CometConf extends ShimCometConf { .booleanConf .createWithDefault(false) + val COMET_EXEC_SHUFFLE_WITH_HASH_PARTITIONING_ENABLED: ConfigEntry[Boolean] = + conf("spark.comet.native.shuffle.partitioning.hash.enabled") + .doc("Whether to enable hash partitioning for Comet native shuffle.") + .booleanConf + .createWithDefault(true) + + val COMET_EXEC_SHUFFLE_WITH_RANGE_PARTITIONING_ENABLED: ConfigEntry[Boolean] = + conf("spark.comet.native.shuffle.partitioning.range.enabled") + .doc("Whether to enable range partitioning for Comet native shuffle.") + .booleanConf + .createWithDefault(true) + val COMET_EXEC_SHUFFLE_COMPRESSION_CODEC: ConfigEntry[String] = conf(s"$COMET_EXEC_CONFIG_PREFIX.shuffle.compression.codec") .doc( @@ -770,11 +782,13 @@ private[comet] abstract class ConfigEntry[T]( /** * Retrieves the config value from the current thread-local [[SQLConf]] + * * @return */ def get(): T = get(SQLConf.get) def defaultValue: Option[T] = None + def defaultValueString: String override def toString: String = { @@ -793,6 +807,7 @@ private[comet] class ConfigEntryWithDefault[T]( version: String) extends ConfigEntry(key, valueConverter, stringConverter, doc, isPublic, version) { override def defaultValue: Option[T] = Some(_defaultValue) + override def defaultValueString: String = stringConverter(_defaultValue) def get(conf: SQLConf): T = { @@ -828,6 +843,7 @@ private[comet] class OptionalConfigEntry[T]( } private[comet] case class ConfigBuilder(key: String) { + import ConfigHelpers._ var _public = true diff --git a/spark/src/main/scala/org/apache/comet/serde/QueryPlanSerde.scala b/spark/src/main/scala/org/apache/comet/serde/QueryPlanSerde.scala index b126a4ed06..12f7c0b4ec 100644 --- a/spark/src/main/scala/org/apache/comet/serde/QueryPlanSerde.scala +++ b/spark/src/main/scala/org/apache/comet/serde/QueryPlanSerde.scala @@ -2888,6 +2888,7 @@ object QueryPlanSerde extends Logging with CometExprShim { val inputs = s.child.output val partitioning = s.outputPartitioning + val conf = SQLConf.get var msg = "" val supported = partitioning match { case HashPartitioning(expressions, _) => @@ -2896,7 +2897,8 @@ object QueryPlanSerde extends Logging with CometExprShim { val supported = expressions.map(QueryPlanSerde.exprToProto(_, inputs)).forall(_.isDefined) && expressions.forall(e => supportedPartitionKeyDataType(e.dataType)) && - inputs.forall(attr => supportedShuffleDataType(attr.dataType)) + inputs.forall(attr => supportedShuffleDataType(attr.dataType)) && + CometConf.COMET_EXEC_SHUFFLE_WITH_HASH_PARTITIONING_ENABLED.get(conf) if (!supported) { msg = s"unsupported Spark partitioning expressions: $expressions" } @@ -2904,7 +2906,7 @@ object QueryPlanSerde extends Logging with CometExprShim { case SinglePartition => inputs.forall(attr => supportedShuffleDataType(attr.dataType)) case RangePartitioning(_, _) => - true + CometConf.COMET_EXEC_SHUFFLE_WITH_RANGE_PARTITIONING_ENABLED.get(conf) case _ => msg = s"unsupported Spark partitioning: ${partitioning.getClass.getName}" false 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 3758989702..92c340de83 100644 --- a/spark/src/test/scala/org/apache/comet/exec/CometNativeShuffleSuite.scala +++ b/spark/src/test/scala/org/apache/comet/exec/CometNativeShuffleSuite.scala @@ -27,6 +27,7 @@ import org.scalatest.Tag import org.apache.hadoop.fs.Path import org.apache.spark.SparkEnv import org.apache.spark.sql.{CometTestBase, DataFrame} +import org.apache.spark.sql.catalyst.plans.physical.{HashPartitioning, RangePartitioning} import org.apache.spark.sql.comet.execution.shuffle.CometShuffleExchangeExec import org.apache.spark.sql.execution.adaptive.AdaptiveSparkPlanHelper import org.apache.spark.sql.functions.col @@ -120,27 +121,59 @@ class CometNativeShuffleSuite extends CometTestBase with AdaptiveSparkPlanHelper } } - test("native operator after native shuffle") { + test("native operator after native shuffle with hash partitioning") { withParquetTable((0 until 5).map(i => (i, (i + 1).toLong)), "tbl") { val df = sql("SELECT * FROM tbl") - val shuffled1 = df + val shuffled = df .repartition(10, $"_2") .select($"_1", $"_1" + 1, $"_2" + 2) .repartition(10, $"_1") .filter($"_1" > 1) // 2 Comet shuffle exchanges are expected - checkShuffleAnswer(shuffled1, 2) + checkShuffleAnswer(shuffled, 2) + + // check the partitioning schemes on the native shuffles + val sparkPlan = stripAQEPlan(shuffled.queryExecution.executedPlan) + val cometShuffleExecs = sparkPlan.collect { case b: CometShuffleExchangeExec => b } + var hash_partitions = 0; + var range_partitions = 0; + cometShuffleExecs.foreach(shuffleExec => + shuffleExec.outputPartitioning match { + case RangePartitioning(_, _) => range_partitions += 1 + case HashPartitioning(_, _) => hash_partitions += 1 + }) + + assert(hash_partitions == 2 && range_partitions == 0) + } + } + + test("native operator after native shuffle with range partitioning") { + withParquetTable((0 until 5).map(i => (i, (i + 1).toLong)), "tbl") { + val df = sql("SELECT * FROM tbl") - val shuffled2 = df + val shuffled = df .repartitionByRange(10, $"_2") .select($"_1", $"_1" + 1, $"_2" + 2) .repartition(10, $"_1") .filter($"_1" > 1) // native shuffle supports RangePartitioning, so 2 Comet shuffle exchanges are expected - checkShuffleAnswer(shuffled2, 2) + checkShuffleAnswer(shuffled, 2) + + // check the partitioning schemes on the native shuffles + val sparkPlan = stripAQEPlan(shuffled.queryExecution.executedPlan) + val cometShuffleExecs = sparkPlan.collect { case b: CometShuffleExchangeExec => b } + var hash_partitions = 0; + var range_partitions = 0; + cometShuffleExecs.foreach(shuffleExec => + shuffleExec.outputPartitioning match { + case RangePartitioning(_, _) => range_partitions += 1 + case HashPartitioning(_, _) => hash_partitions += 1 + }) + + assert(hash_partitions == 1 && range_partitions == 1) } } From 47cb983dc943ceb5a7734dfc0e64a7cd274cb877 Mon Sep 17 00:00:00 2001 From: Matt Butrovich Date: Wed, 11 Jun 2025 11:48:58 -0400 Subject: [PATCH 57/72] Update tests. --- .../comet/exec/CometNativeShuffleSuite.scala | 89 +++++++++---------- 1 file changed, 40 insertions(+), 49 deletions(-) diff --git a/spark/src/test/scala/org/apache/comet/exec/CometNativeShuffleSuite.scala b/spark/src/test/scala/org/apache/comet/exec/CometNativeShuffleSuite.scala index 92c340de83..e1028e301d 100644 --- a/spark/src/test/scala/org/apache/comet/exec/CometNativeShuffleSuite.scala +++ b/spark/src/test/scala/org/apache/comet/exec/CometNativeShuffleSuite.scala @@ -27,7 +27,6 @@ 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.catalyst.plans.physical.{HashPartitioning, RangePartitioning} import org.apache.spark.sql.comet.execution.shuffle.CometShuffleExchangeExec import org.apache.spark.sql.execution.adaptive.AdaptiveSparkPlanHelper import org.apache.spark.sql.functions.col @@ -122,58 +121,50 @@ class CometNativeShuffleSuite extends CometTestBase with AdaptiveSparkPlanHelper } test("native operator after native shuffle with hash partitioning") { - withParquetTable((0 until 5).map(i => (i, (i + 1).toLong)), "tbl") { - val df = sql("SELECT * FROM tbl") - - val shuffled = df - .repartition(10, $"_2") - .select($"_1", $"_1" + 1, $"_2" + 2) - .repartition(10, $"_1") - .filter($"_1" > 1) - - // 2 Comet shuffle exchanges are expected - checkShuffleAnswer(shuffled, 2) - - // check the partitioning schemes on the native shuffles - val sparkPlan = stripAQEPlan(shuffled.queryExecution.executedPlan) - val cometShuffleExecs = sparkPlan.collect { case b: CometShuffleExchangeExec => b } - var hash_partitions = 0; - var range_partitions = 0; - cometShuffleExecs.foreach(shuffleExec => - shuffleExec.outputPartitioning match { - case RangePartitioning(_, _) => range_partitions += 1 - case HashPartitioning(_, _) => hash_partitions += 1 - }) - - assert(hash_partitions == 2 && range_partitions == 0) + Seq("true", "false").foreach { hashPartitioningEnabled => + withSQLConf( + CometConf.COMET_EXEC_SHUFFLE_WITH_HASH_PARTITIONING_ENABLED.key -> hashPartitioningEnabled) { + withParquetTable((0 until 5).map(i => (i, (i + 1).toLong)), "tbl") { + val df = sql("SELECT * FROM tbl") + + val shuffled = df + .repartition(10, $"_2") + .select($"_1", $"_1" + 1, $"_2" + 2) + .repartition(10, $"_1") + .filter($"_1" > 1) + + // native shuffle supports HashPartitioning, so 2 Comet shuffle exchanges are expected + if (hashPartitioningEnabled == "true") { + checkShuffleAnswer(shuffled, 2) + } else { + checkShuffleAnswer(shuffled, 0) + } + } + } } } test("native operator after native shuffle with range partitioning") { - withParquetTable((0 until 5).map(i => (i, (i + 1).toLong)), "tbl") { - val df = sql("SELECT * FROM tbl") - - val shuffled = df - .repartitionByRange(10, $"_2") - .select($"_1", $"_1" + 1, $"_2" + 2) - .repartition(10, $"_1") - .filter($"_1" > 1) - - // native shuffle supports RangePartitioning, so 2 Comet shuffle exchanges are expected - checkShuffleAnswer(shuffled, 2) - - // check the partitioning schemes on the native shuffles - val sparkPlan = stripAQEPlan(shuffled.queryExecution.executedPlan) - val cometShuffleExecs = sparkPlan.collect { case b: CometShuffleExchangeExec => b } - var hash_partitions = 0; - var range_partitions = 0; - cometShuffleExecs.foreach(shuffleExec => - shuffleExec.outputPartitioning match { - case RangePartitioning(_, _) => range_partitions += 1 - case HashPartitioning(_, _) => hash_partitions += 1 - }) - - assert(hash_partitions == 1 && range_partitions == 1) + Seq("true", "false").foreach { rangePartitioningEnabled => + withSQLConf( + CometConf.COMET_EXEC_SHUFFLE_WITH_RANGE_PARTITIONING_ENABLED.key -> rangePartitioningEnabled) { + withParquetTable((0 until 5).map(i => (i, (i + 1).toLong)), "tbl") { + val df = sql("SELECT * FROM tbl") + + val shuffled = df + .repartitionByRange(10, $"_2") + .select($"_1", $"_1" + 1, $"_2" + 2) + .repartition(10, $"_1") + .filter($"_1" > 1) + + // native shuffle supports RangePartitioning, so 2 Comet shuffle exchanges are expected + if (rangePartitioningEnabled == "true") { + checkShuffleAnswer(shuffled, 2) + } else { + checkShuffleAnswer(shuffled, 0) + } + } + } } } From 727ef169d152f98849cba50b5070e2f0bcf2bf57 Mon Sep 17 00:00:00 2001 From: Matt Butrovich Date: Wed, 11 Jun 2025 13:31:55 -0400 Subject: [PATCH 58/72] Update docs, update expression checking for range partitioning. --- docs/source/user-guide/configs.md | 2 ++ .../scala/org/apache/comet/serde/QueryPlanSerde.scala | 8 +++++--- 2 files changed, 7 insertions(+), 3 deletions(-) diff --git a/docs/source/user-guide/configs.md b/docs/source/user-guide/configs.md index 517ce960c0..72a3c03548 100644 --- a/docs/source/user-guide/configs.md +++ b/docs/source/user-guide/configs.md @@ -75,6 +75,8 @@ Comet provides the following configuration settings. | spark.comet.memory.overhead.min | Minimum amount of additional memory to be allocated per executor process for Comet, in MiB, when running Spark in on-heap mode. For more information, refer to the Comet Tuning Guide (https://datafusion.apache.org/comet/user-guide/tuning.html). | 402653184b | | 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 | 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/spark/src/main/scala/org/apache/comet/serde/QueryPlanSerde.scala b/spark/src/main/scala/org/apache/comet/serde/QueryPlanSerde.scala index 12f7c0b4ec..d861e6813b 100644 --- a/spark/src/main/scala/org/apache/comet/serde/QueryPlanSerde.scala +++ b/spark/src/main/scala/org/apache/comet/serde/QueryPlanSerde.scala @@ -2877,7 +2877,7 @@ object QueryPlanSerde extends Logging with CometExprShim { * Hash Partition Key determines how data should be collocated for operations like * `groupByKey`, `reduceByKey` or `join`. */ - def supportedPartitionKeyDataType(dt: DataType): Boolean = dt match { + def supportedHashPartitionKeyDataType(dt: DataType): Boolean = dt match { case _: BooleanType | _: ByteType | _: ShortType | _: IntegerType | _: LongType | _: FloatType | _: DoubleType | _: StringType | _: BinaryType | _: TimestampType | _: TimestampNTZType | _: DecimalType | _: DateType => @@ -2896,7 +2896,7 @@ object QueryPlanSerde extends Logging with CometExprShim { // due to lack of hashing support for those types val supported = expressions.map(QueryPlanSerde.exprToProto(_, inputs)).forall(_.isDefined) && - expressions.forall(e => supportedPartitionKeyDataType(e.dataType)) && + expressions.forall(e => supportedHashPartitionKeyDataType(e.dataType)) && inputs.forall(attr => supportedShuffleDataType(attr.dataType)) && CometConf.COMET_EXEC_SHUFFLE_WITH_HASH_PARTITIONING_ENABLED.get(conf) if (!supported) { @@ -2905,7 +2905,9 @@ object QueryPlanSerde extends Logging with CometExprShim { supported case SinglePartition => inputs.forall(attr => supportedShuffleDataType(attr.dataType)) - case RangePartitioning(_, _) => + case RangePartitioning(ordering, _) => + ordering.map(QueryPlanSerde.exprToProto(_, inputs)).forall(_.isDefined) && + inputs.forall(attr => supportedShuffleDataType(attr.dataType)) && CometConf.COMET_EXEC_SHUFFLE_WITH_RANGE_PARTITIONING_ENABLED.get(conf) case _ => msg = s"unsupported Spark partitioning: ${partitioning.getClass.getName}" From 88e80a50186cf07b8a15ecf167c3739ef08e9ade Mon Sep 17 00:00:00 2001 From: Matt Butrovich Date: Wed, 11 Jun 2025 13:57:39 -0400 Subject: [PATCH 59/72] avoid converting to vector of Row for every batch. --- .../execution/shuffle/range_partitioner.rs | 10 ++++---- .../src/execution/shuffle/shuffle_writer.rs | 24 +++++++------------ 2 files changed, 14 insertions(+), 20 deletions(-) diff --git a/native/core/src/execution/shuffle/range_partitioner.rs b/native/core/src/execution/shuffle/range_partitioner.rs index 029fa13e86..eaa4c46021 100644 --- a/native/core/src/execution/shuffle/range_partitioner.rs +++ b/native/core/src/execution/shuffle/range_partitioner.rs @@ -17,7 +17,7 @@ use arrow::array::{ArrayRef, UInt64Array}; use arrow::compute::{take_arrays, TakeOptions}; -use arrow::row::{Row, RowConverter, Rows, SortField}; +use arrow::row::{OwnedRow, Row, RowConverter, Rows, SortField}; use datafusion::error::{DataFusionError, Result}; use datafusion::physical_expr::LexOrdering; use rand::{rngs::SmallRng, Rng, SeedableRng}; @@ -64,12 +64,12 @@ impl RangePartitioner { /// Row using binary search. pub fn partition_indices_for_batch( row_batch: &Rows, - partition_bounds_vec: &Vec, + 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) as u32 + partition_bounds_vec.partition_point(|bound| bound.row() <= row) as u32 }); } @@ -288,11 +288,11 @@ mod test { let bounds_rows = row_converter.convert_columns(bounds.columns()).unwrap(); - let bounds_rows_vec = bounds_rows.iter().collect_vec(); + let owned_rows_vec = bounds_rows.iter().map(|row| row.owned()).collect_vec(); RangePartitioner::partition_indices_for_batch( &input_rows, - &bounds_rows_vec, + &owned_rows_vec.as_slice(), &mut partition_ids, ); diff --git a/native/core/src/execution/shuffle/shuffle_writer.rs b/native/core/src/execution/shuffle/shuffle_writer.rs index e02ea9bd73..29a049ecb5 100644 --- a/native/core/src/execution/shuffle/shuffle_writer.rs +++ b/native/core/src/execution/shuffle/shuffle_writer.rs @@ -21,7 +21,7 @@ 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::{RowConverter, Rows}; +use arrow::row::{OwnedRow, RowConverter}; use async_trait::async_trait; use datafusion::common::utils::proxy::VecAllocExt; use datafusion::physical_expr::EquivalenceProperties; @@ -334,7 +334,7 @@ struct MultiPartitionShuffleRepartitioner { reservation: MemoryReservation, tracing_enabled: bool, /// RangePartitioning-specific state - bounds_rows: Option, + bounds_rows: Option>, row_converter: Option, seed: u64, } @@ -572,7 +572,8 @@ impl MultiPartitionShuffleRepartitioner { self.seed, )?; - self.bounds_rows = Some(bounds_rows); + self.bounds_rows = + Some(bounds_rows.iter().map(|row| row.owned()).collect_vec()); self.row_converter = Some(row_converter); } @@ -584,19 +585,12 @@ impl MultiPartitionShuffleRepartitioner { .as_ref() .unwrap() .convert_columns(arrays.as_slice())?; - { - let partition_ids = &mut scratch.partition_ids[..num_rows]; - - // TODO: Try to cache this vector. - let bounds_rows_vec = - self.bounds_rows.as_ref().unwrap().iter().collect_vec(); - RangePartitioner::partition_indices_for_batch( - &row_batch, - &bounds_rows_vec, - partition_ids, - ); - } + RangePartitioner::partition_indices_for_batch( + &row_batch, + self.bounds_rows.as_ref().unwrap().as_slice(), + &mut scratch.partition_ids[..num_rows], + ); // 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. From 8228429d17dd3c2b981ca0fe53e8e11bbe9e7908 Mon Sep 17 00:00:00 2001 From: Matt Butrovich Date: Wed, 11 Jun 2025 15:06:14 -0400 Subject: [PATCH 60/72] Simplify CometPartitioning enum. --- native/core/src/execution/planner.rs | 2 +- .../execution/shuffle/comet_partitioning.rs | 44 ++----------------- .../src/execution/shuffle/shuffle_writer.rs | 4 +- 3 files changed, 7 insertions(+), 43 deletions(-) diff --git a/native/core/src/execution/planner.rs b/native/core/src/execution/planner.rs index 2c946025b9..656cd9fe40 100644 --- a/native/core/src/execution/planner.rs +++ b/native/core/src/execution/planner.rs @@ -2215,7 +2215,7 @@ impl PhysicalPlanner { range_partition.sample_size as usize, )) } - PartitioningStruct::SinglePartition(_) => Ok(CometPartitioning::UnknownPartitioning(1)), + 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 da9f5c32d7..b063473bdf 100644 --- a/native/core/src/execution/shuffle/comet_partitioning.rs +++ b/native/core/src/execution/shuffle/comet_partitioning.rs @@ -15,62 +15,26 @@ // specific language governing permissions and limitations // under the License. -use datafusion::physical_expr::{LexOrdering, Partitioning, PhysicalExpr}; +use datafusion::physical_expr::{LexOrdering, PhysicalExpr}; use std::sync::Arc; #[derive(Debug, Clone)] pub enum CometPartitioning { - /// Allocate batches using a round-robin algorithm and the specified number of partitions - RoundRobinBatch(usize), + SinglePartition, /// Allocate rows based on a hash of one of more expressions and the specified number of /// partitions Hash(Vec>, usize), /// Allocate rows based on lexical order of one of more expressions and the specified number of /// partitions RangePartitioning(LexOrdering, usize, usize), - /// Unknown partitioning scheme with a known number of partitions - UnknownPartitioning(usize), } impl CometPartitioning { pub fn partition_count(&self) -> usize { use CometPartitioning::*; match self { - RoundRobinBatch(n) - | Hash(_, n) - | UnknownPartitioning(n) - | RangePartitioning(_, n, _) => *n, - } - } -} - -impl From for CometPartitioning { - fn from(df_partitioning: Partitioning) -> Self { - match df_partitioning { - Partitioning::RoundRobinBatch(partitions) => { - CometPartitioning::RoundRobinBatch(partitions) - } - Partitioning::Hash(exprs, partitions) => CometPartitioning::Hash(exprs, partitions), - Partitioning::UnknownPartitioning(partitions) => { - CometPartitioning::UnknownPartitioning(partitions) - } - } - } -} - -impl From for Partitioning { - fn from(val: CometPartitioning) -> Self { - match val { - CometPartitioning::RoundRobinBatch(partitions) => { - Partitioning::RoundRobinBatch(partitions) - } - CometPartitioning::Hash(exprs, partitions) => Partitioning::Hash(exprs, partitions), - CometPartitioning::UnknownPartitioning(partitions) => { - Partitioning::UnknownPartitioning(partitions) - } - CometPartitioning::RangePartitioning(_, partitions, _) => { - Partitioning::UnknownPartitioning(partitions) - } + SinglePartition => 1, + Hash(_, n) | RangePartitioning(_, n, _) => *n, } } } diff --git a/native/core/src/execution/shuffle/shuffle_writer.rs b/native/core/src/execution/shuffle/shuffle_writer.rs index 29a049ecb5..eb3083bff8 100644 --- a/native/core/src/execution/shuffle/shuffle_writer.rs +++ b/native/core/src/execution/shuffle/shuffle_writer.rs @@ -24,7 +24,7 @@ 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; +use datafusion::physical_expr::{EquivalenceProperties, Partitioning}; use datafusion::physical_plan::execution_plan::{Boundedness, EmissionType}; use datafusion::physical_plan::EmptyRecordBatchStream; use datafusion::{ @@ -94,7 +94,7 @@ impl ShuffleWriterExec { ) -> Result { let cache = PlanProperties::new( EquivalenceProperties::new(Arc::clone(&input.schema())), - partitioning.clone().into(), + Partitioning::UnknownPartitioning(1), EmissionType::Final, Boundedness::Bounded, ); From bdb67af2104b3ad20eb3041353a5f4d5a3ecc956 Mon Sep 17 00:00:00 2001 From: Matt Butrovich Date: Fri, 13 Jun 2025 08:34:03 -0400 Subject: [PATCH 61/72] Clippy. --- native/core/src/execution/shuffle/range_partitioner.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/native/core/src/execution/shuffle/range_partitioner.rs b/native/core/src/execution/shuffle/range_partitioner.rs index eaa4c46021..1306d3b7da 100644 --- a/native/core/src/execution/shuffle/range_partitioner.rs +++ b/native/core/src/execution/shuffle/range_partitioner.rs @@ -292,7 +292,7 @@ mod test { RangePartitioner::partition_indices_for_batch( &input_rows, - &owned_rows_vec.as_slice(), + owned_rows_vec.as_slice(), &mut partition_ids, ); From 4746937bad4376017d9d653a1498cfebfdd05bcb Mon Sep 17 00:00:00 2001 From: Matt Butrovich Date: Fri, 13 Jun 2025 10:36:18 -0400 Subject: [PATCH 62/72] Better message on fallback. --- .../org/apache/comet/serde/QueryPlanSerde.scala | 12 ++++++++---- 1 file changed, 8 insertions(+), 4 deletions(-) diff --git a/spark/src/main/scala/org/apache/comet/serde/QueryPlanSerde.scala b/spark/src/main/scala/org/apache/comet/serde/QueryPlanSerde.scala index d861e6813b..9c988bdae3 100644 --- a/spark/src/main/scala/org/apache/comet/serde/QueryPlanSerde.scala +++ b/spark/src/main/scala/org/apache/comet/serde/QueryPlanSerde.scala @@ -2900,15 +2900,19 @@ object QueryPlanSerde extends Logging with CometExprShim { inputs.forall(attr => supportedShuffleDataType(attr.dataType)) && CometConf.COMET_EXEC_SHUFFLE_WITH_HASH_PARTITIONING_ENABLED.get(conf) if (!supported) { - msg = s"unsupported Spark partitioning expressions: $expressions" + msg = s"unsupported Spark partitioning: $expressions" } supported case SinglePartition => inputs.forall(attr => supportedShuffleDataType(attr.dataType)) case RangePartitioning(ordering, _) => - ordering.map(QueryPlanSerde.exprToProto(_, inputs)).forall(_.isDefined) && - inputs.forall(attr => supportedShuffleDataType(attr.dataType)) && - CometConf.COMET_EXEC_SHUFFLE_WITH_RANGE_PARTITIONING_ENABLED.get(conf) + val supported = ordering.map(QueryPlanSerde.exprToProto(_, inputs)).forall(_.isDefined) && + inputs.forall(attr => supportedShuffleDataType(attr.dataType)) && + CometConf.COMET_EXEC_SHUFFLE_WITH_RANGE_PARTITIONING_ENABLED.get(conf) + if (!supported) { + msg = s"unsupported Spark partitioning: $ordering" + } + supported case _ => msg = s"unsupported Spark partitioning: ${partitioning.getClass.getName}" false From 385bad73f67d19fa40cd623b3936959cdc5ddbb2 Mon Sep 17 00:00:00 2001 From: Matt Butrovich Date: Fri, 13 Jun 2025 14:34:36 -0400 Subject: [PATCH 63/72] 3.5.6 diff. --- dev/diffs/3.5.6.diff | 72 +++++++++++++++++++++++++++++++++++++++----- 1 file changed, 65 insertions(+), 7 deletions(-) diff --git a/dev/diffs/3.5.6.diff b/dev/diffs/3.5.6.diff index 4897de62a7..6dc418a1d5 100644 --- a/dev/diffs/3.5.6.diff +++ b/dev/diffs/3.5.6.diff @@ -2278,10 +2278,18 @@ index d083cac48ff..3c11bcde807 100644 import testImplicits._ diff --git a/sql/core/src/test/scala/org/apache/spark/sql/sources/BucketedReadSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/sources/BucketedReadSuite.scala -index 746f289c393..0c99d028163 100644 +index 746f289c393..199ee138375 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/sources/BucketedReadSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/sources/BucketedReadSuite.scala -@@ -25,10 +25,11 @@ import org.apache.spark.sql.catalyst.expressions +@@ -19,16 +19,19 @@ package org.apache.spark.sql.sources + + import scala.util.Random + ++import org.apache.comet.CometConf ++ + import org.apache.spark.sql._ + import org.apache.spark.sql.catalyst.catalog.BucketSpec + import org.apache.spark.sql.catalyst.expressions import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.plans.physical.HashPartitioning import org.apache.spark.sql.catalyst.types.DataTypeUtils @@ -2295,7 +2303,7 @@ index 746f289c393..0c99d028163 100644 import org.apache.spark.sql.execution.joins.SortMergeJoinExec import org.apache.spark.sql.functions._ import org.apache.spark.sql.internal.SQLConf -@@ -102,12 +103,20 @@ abstract class BucketedReadSuite extends QueryTest with SQLTestUtils with Adapti +@@ -102,12 +105,20 @@ abstract class BucketedReadSuite extends QueryTest with SQLTestUtils with Adapti } } @@ -2318,7 +2326,7 @@ index 746f289c393..0c99d028163 100644 // To verify if the bucket pruning works, this function checks two conditions: // 1) Check if the pruned buckets (before filtering) are empty. // 2) Verify the final result is the same as the expected one -@@ -156,7 +165,8 @@ abstract class BucketedReadSuite extends QueryTest with SQLTestUtils with Adapti +@@ -156,7 +167,8 @@ abstract class BucketedReadSuite extends QueryTest with SQLTestUtils with Adapti val planWithoutBucketedScan = bucketedDataFrame.filter(filterCondition) .queryExecution.executedPlan val fileScan = getFileScan(planWithoutBucketedScan) @@ -2328,7 +2336,7 @@ index 746f289c393..0c99d028163 100644 val bucketColumnType = bucketedDataFrame.schema.apply(bucketColumnIndex).dataType val rowsWithInvalidBuckets = fileScan.execute().filter(row => { -@@ -452,28 +462,49 @@ abstract class BucketedReadSuite extends QueryTest with SQLTestUtils with Adapti +@@ -452,28 +464,49 @@ abstract class BucketedReadSuite extends QueryTest with SQLTestUtils with Adapti val joinOperator = if (joined.sqlContext.conf.adaptiveExecutionEnabled) { val executedPlan = joined.queryExecution.executedPlan.asInstanceOf[AdaptiveSparkPlanExec].executedPlan @@ -2386,7 +2394,7 @@ index 746f289c393..0c99d028163 100644 s"expected sort in the right child to be $sortRight but found\n${joinOperator.right}") // check the output partitioning -@@ -836,11 +867,11 @@ abstract class BucketedReadSuite extends QueryTest with SQLTestUtils with Adapti +@@ -836,11 +869,11 @@ abstract class BucketedReadSuite extends QueryTest with SQLTestUtils with Adapti df1.write.format("parquet").bucketBy(8, "i").saveAsTable("bucketed_table") val scanDF = spark.table("bucketed_table").select("j") @@ -2400,7 +2408,31 @@ index 746f289c393..0c99d028163 100644 checkAnswer(aggDF, df1.groupBy("j").agg(max("k"))) } } -@@ -1029,15 +1060,21 @@ abstract class BucketedReadSuite extends QueryTest with SQLTestUtils with Adapti +@@ -896,6 +929,7 @@ abstract class BucketedReadSuite extends QueryTest with SQLTestUtils with Adapti + + test("SPARK-29655 Read bucketed tables obeys spark.sql.shuffle.partitions") { + withSQLConf( ++ CometConf.COMET_EXEC_SHUFFLE_WITH_RANGE_PARTITIONING_ENABLED.key -> "false", + SQLConf.SHUFFLE_PARTITIONS.key -> "5", + SQLConf.COALESCE_PARTITIONS_INITIAL_PARTITION_NUM.key -> "7") { + val bucketSpec = Some(BucketSpec(6, Seq("i", "j"), Nil)) +@@ -915,6 +949,7 @@ abstract class BucketedReadSuite extends QueryTest with SQLTestUtils with Adapti + + test("SPARK-32767 Bucket join should work if SHUFFLE_PARTITIONS larger than bucket number") { + withSQLConf( ++ CometConf.COMET_EXEC_SHUFFLE_WITH_RANGE_PARTITIONING_ENABLED.key -> "false", + SQLConf.SHUFFLE_PARTITIONS.key -> "9", + SQLConf.COALESCE_PARTITIONS_INITIAL_PARTITION_NUM.key -> "10") { + +@@ -945,6 +980,7 @@ abstract class BucketedReadSuite extends QueryTest with SQLTestUtils with Adapti + + test("bucket coalescing eliminates shuffle") { + withSQLConf( ++ CometConf.COMET_EXEC_SHUFFLE_WITH_RANGE_PARTITIONING_ENABLED.key -> "false", + SQLConf.COALESCE_BUCKETS_IN_JOIN_ENABLED.key -> "true", + SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "false") { + // The side with bucketedTableTestSpec1 will be coalesced to have 4 output partitions. +@@ -1029,15 +1065,21 @@ abstract class BucketedReadSuite extends QueryTest with SQLTestUtils with Adapti Seq(true, false).foreach { aqeEnabled => withSQLConf(SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> aqeEnabled.toString) { val plan = sql(query).queryExecution.executedPlan @@ -2830,6 +2862,32 @@ index 549431ef4f4..e48f1730da6 100644 withTempDir { dir => withSQLConf( "parquet.crypto.factory.class" -> +diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala +index 6160c3e5f6c..d6761dc6f7c 100644 +--- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala ++++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala +@@ -24,6 +24,7 @@ import java.sql.{Date, Timestamp} + import java.util.{Locale, Set} + + import com.google.common.io.Files ++import org.apache.comet.CometConf + import org.apache.hadoop.fs.{FileSystem, Path} + + import org.apache.spark.{SparkException, TestUtils} +@@ -838,8 +839,11 @@ abstract class SQLQuerySuiteBase extends QueryTest with SQLTestUtils with TestHi + } + + test("SPARK-2554 SumDistinct partial aggregation") { +- checkAnswer(sql("SELECT sum( distinct key) FROM src group by key order by key"), +- sql("SELECT distinct key FROM src order by key").collect().toSeq) ++ withSQLConf(CometConf.COMET_EXEC_SHUFFLE_WITH_RANGE_PARTITIONING_ENABLED.key -> "false") ++ { ++ checkAnswer(sql("SELECT sum( distinct key) FROM src group by key order by key"), ++ sql("SELECT distinct key FROM src order by key").collect().toSeq) ++ } + } + + test("SPARK-4963 DataFrame sample on mutable row return wrong result") { diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/test/TestHive.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/test/TestHive.scala index 1d646f40b3e..7f2cdb8f061 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/test/TestHive.scala From 20895e26123e19ba2f1d151d8a59cc87f5416ab7 Mon Sep 17 00:00:00 2001 From: Matt Butrovich Date: Fri, 13 Jun 2025 14:46:06 -0400 Subject: [PATCH 64/72] 3.4.3 diff. --- dev/diffs/3.4.3.diff | 71 +++++++++++++++++++++++++++++++++++++++----- 1 file changed, 64 insertions(+), 7 deletions(-) diff --git a/dev/diffs/3.4.3.diff b/dev/diffs/3.4.3.diff index 52aa73e93d..8bd868346c 100644 --- a/dev/diffs/3.4.3.diff +++ b/dev/diffs/3.4.3.diff @@ -2282,10 +2282,17 @@ index d083cac48ff..3c11bcde807 100644 import testImplicits._ diff --git a/sql/core/src/test/scala/org/apache/spark/sql/sources/BucketedReadSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/sources/BucketedReadSuite.scala -index 266bb343526..c3e3d155813 100644 +index 266bb343526..1c600e6e84c 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/sources/BucketedReadSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/sources/BucketedReadSuite.scala -@@ -24,10 +24,11 @@ import org.apache.spark.sql.catalyst.catalog.BucketSpec +@@ -19,15 +19,18 @@ package org.apache.spark.sql.sources + + import scala.util.Random + ++import org.apache.comet.CometConf ++ + import org.apache.spark.sql._ + import org.apache.spark.sql.catalyst.catalog.BucketSpec import org.apache.spark.sql.catalyst.expressions import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.plans.physical.HashPartitioning @@ -2299,7 +2306,7 @@ index 266bb343526..c3e3d155813 100644 import org.apache.spark.sql.execution.joins.SortMergeJoinExec import org.apache.spark.sql.functions._ import org.apache.spark.sql.internal.SQLConf -@@ -101,12 +102,20 @@ abstract class BucketedReadSuite extends QueryTest with SQLTestUtils with Adapti +@@ -101,12 +104,20 @@ abstract class BucketedReadSuite extends QueryTest with SQLTestUtils with Adapti } } @@ -2322,7 +2329,7 @@ index 266bb343526..c3e3d155813 100644 // To verify if the bucket pruning works, this function checks two conditions: // 1) Check if the pruned buckets (before filtering) are empty. // 2) Verify the final result is the same as the expected one -@@ -155,7 +164,8 @@ abstract class BucketedReadSuite extends QueryTest with SQLTestUtils with Adapti +@@ -155,7 +166,8 @@ abstract class BucketedReadSuite extends QueryTest with SQLTestUtils with Adapti val planWithoutBucketedScan = bucketedDataFrame.filter(filterCondition) .queryExecution.executedPlan val fileScan = getFileScan(planWithoutBucketedScan) @@ -2332,7 +2339,7 @@ index 266bb343526..c3e3d155813 100644 val bucketColumnType = bucketedDataFrame.schema.apply(bucketColumnIndex).dataType val rowsWithInvalidBuckets = fileScan.execute().filter(row => { -@@ -451,28 +461,49 @@ abstract class BucketedReadSuite extends QueryTest with SQLTestUtils with Adapti +@@ -451,28 +463,49 @@ abstract class BucketedReadSuite extends QueryTest with SQLTestUtils with Adapti val joinOperator = if (joined.sqlContext.conf.adaptiveExecutionEnabled) { val executedPlan = joined.queryExecution.executedPlan.asInstanceOf[AdaptiveSparkPlanExec].executedPlan @@ -2390,7 +2397,7 @@ index 266bb343526..c3e3d155813 100644 s"expected sort in the right child to be $sortRight but found\n${joinOperator.right}") // check the output partitioning -@@ -835,11 +866,11 @@ abstract class BucketedReadSuite extends QueryTest with SQLTestUtils with Adapti +@@ -835,11 +868,11 @@ abstract class BucketedReadSuite extends QueryTest with SQLTestUtils with Adapti df1.write.format("parquet").bucketBy(8, "i").saveAsTable("bucketed_table") val scanDF = spark.table("bucketed_table").select("j") @@ -2404,7 +2411,31 @@ index 266bb343526..c3e3d155813 100644 checkAnswer(aggDF, df1.groupBy("j").agg(max("k"))) } } -@@ -1026,15 +1057,23 @@ abstract class BucketedReadSuite extends QueryTest with SQLTestUtils with Adapti +@@ -895,6 +928,7 @@ abstract class BucketedReadSuite extends QueryTest with SQLTestUtils with Adapti + + test("SPARK-29655 Read bucketed tables obeys spark.sql.shuffle.partitions") { + withSQLConf( ++ CometConf.COMET_EXEC_SHUFFLE_WITH_RANGE_PARTITIONING_ENABLED.key -> "false", + SQLConf.SHUFFLE_PARTITIONS.key -> "5", + SQLConf.COALESCE_PARTITIONS_INITIAL_PARTITION_NUM.key -> "7") { + val bucketSpec = Some(BucketSpec(6, Seq("i", "j"), Nil)) +@@ -914,6 +948,7 @@ abstract class BucketedReadSuite extends QueryTest with SQLTestUtils with Adapti + + test("SPARK-32767 Bucket join should work if SHUFFLE_PARTITIONS larger than bucket number") { + withSQLConf( ++ CometConf.COMET_EXEC_SHUFFLE_WITH_RANGE_PARTITIONING_ENABLED.key -> "false", + SQLConf.SHUFFLE_PARTITIONS.key -> "9", + SQLConf.COALESCE_PARTITIONS_INITIAL_PARTITION_NUM.key -> "10") { + +@@ -944,6 +979,7 @@ abstract class BucketedReadSuite extends QueryTest with SQLTestUtils with Adapti + + test("bucket coalescing eliminates shuffle") { + withSQLConf( ++ CometConf.COMET_EXEC_SHUFFLE_WITH_RANGE_PARTITIONING_ENABLED.key -> "false", + SQLConf.COALESCE_BUCKETS_IN_JOIN_ENABLED.key -> "true", + SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "false") { + // The side with bucketedTableTestSpec1 will be coalesced to have 4 output partitions. +@@ -1026,15 +1062,23 @@ abstract class BucketedReadSuite extends QueryTest with SQLTestUtils with Adapti expectedNumShuffles: Int, expectedCoalescedNumBuckets: Option[Int]): Unit = { val plan = sql(query).queryExecution.executedPlan @@ -2816,6 +2847,32 @@ index 52abd248f3a..7a199931a08 100644 case h: HiveTableScanExec => h.partitionPruningPred.collect { case d: DynamicPruningExpression => d.child } +diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala +index a902cb3a69e..a39bb7a7658 100644 +--- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala ++++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala +@@ -24,6 +24,7 @@ import java.sql.{Date, Timestamp} + import java.util.{Locale, Set} + + import com.google.common.io.Files ++import org.apache.comet.CometConf + import org.apache.hadoop.fs.{FileSystem, Path} + + import org.apache.spark.{SparkException, TestUtils} +@@ -838,8 +839,11 @@ abstract class SQLQuerySuiteBase extends QueryTest with SQLTestUtils with TestHi + } + + test("SPARK-2554 SumDistinct partial aggregation") { +- checkAnswer(sql("SELECT sum( distinct key) FROM src group by key order by key"), +- sql("SELECT distinct key FROM src order by key").collect().toSeq) ++ withSQLConf(CometConf.COMET_EXEC_SHUFFLE_WITH_RANGE_PARTITIONING_ENABLED.key -> "false") ++ { ++ checkAnswer(sql("SELECT sum( distinct key) FROM src group by key order by key"), ++ sql("SELECT distinct key FROM src order by key").collect().toSeq) ++ } + } + + test("SPARK-4963 DataFrame sample on mutable row return wrong result") { diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/test/TestHive.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/test/TestHive.scala index 07361cfdce9..b4d53dbe900 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/test/TestHive.scala From 907433513edc99f4a3e200b94e994633c637f756 Mon Sep 17 00:00:00 2001 From: Matt Butrovich Date: Fri, 13 Jun 2025 15:07:14 -0400 Subject: [PATCH 65/72] 4.0.0-preview1 diff. --- dev/diffs/4.0.0-preview1.diff | 74 +++++++++++++++++++++++++++++++---- 1 file changed, 66 insertions(+), 8 deletions(-) diff --git a/dev/diffs/4.0.0-preview1.diff b/dev/diffs/4.0.0-preview1.diff index f4cb601e41..2996538afe 100644 --- a/dev/diffs/4.0.0-preview1.diff +++ b/dev/diffs/4.0.0-preview1.diff @@ -595,7 +595,7 @@ index 49a33d1c925..197c93d62b3 100644 } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/IgnoreComet.scala b/sql/core/src/test/scala/org/apache/spark/sql/IgnoreComet.scala new file mode 100644 -index 00000000000..5db1532cf9f +index 00000000000..5691536c114 --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/IgnoreComet.scala @@ -0,0 +1,45 @@ @@ -2517,10 +2517,18 @@ index 6ff07449c0c..9f95cff99e5 100644 import testImplicits._ diff --git a/sql/core/src/test/scala/org/apache/spark/sql/sources/BucketedReadSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/sources/BucketedReadSuite.scala -index 3573bafe482..11d387110ea 100644 +index 3573bafe482..9cfaa0b93c1 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/sources/BucketedReadSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/sources/BucketedReadSuite.scala -@@ -25,10 +25,11 @@ import org.apache.spark.sql.catalyst.expressions +@@ -19,16 +19,19 @@ package org.apache.spark.sql.sources + + import scala.util.Random + ++import org.apache.comet.CometConf ++ + import org.apache.spark.sql._ + import org.apache.spark.sql.catalyst.catalog.BucketSpec + import org.apache.spark.sql.catalyst.expressions import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.plans.physical.HashPartitioning import org.apache.spark.sql.catalyst.types.DataTypeUtils @@ -2534,7 +2542,7 @@ index 3573bafe482..11d387110ea 100644 import org.apache.spark.sql.execution.joins.SortMergeJoinExec import org.apache.spark.sql.functions._ import org.apache.spark.sql.internal.SQLConf -@@ -102,12 +103,20 @@ abstract class BucketedReadSuite extends QueryTest with SQLTestUtils with Adapti +@@ -102,12 +105,20 @@ abstract class BucketedReadSuite extends QueryTest with SQLTestUtils with Adapti } } @@ -2557,7 +2565,7 @@ index 3573bafe482..11d387110ea 100644 // To verify if the bucket pruning works, this function checks two conditions: // 1) Check if the pruned buckets (before filtering) are empty. // 2) Verify the final result is the same as the expected one -@@ -156,7 +165,8 @@ abstract class BucketedReadSuite extends QueryTest with SQLTestUtils with Adapti +@@ -156,7 +167,8 @@ abstract class BucketedReadSuite extends QueryTest with SQLTestUtils with Adapti val planWithoutBucketedScan = bucketedDataFrame.filter(filterCondition) .queryExecution.executedPlan val fileScan = getFileScan(planWithoutBucketedScan) @@ -2567,7 +2575,7 @@ index 3573bafe482..11d387110ea 100644 val bucketColumnType = bucketedDataFrame.schema.apply(bucketColumnIndex).dataType val rowsWithInvalidBuckets = fileScan.execute().filter(row => { -@@ -452,28 +462,49 @@ abstract class BucketedReadSuite extends QueryTest with SQLTestUtils with Adapti +@@ -452,28 +464,49 @@ abstract class BucketedReadSuite extends QueryTest with SQLTestUtils with Adapti val joinOperator = if (joined.sparkSession.sessionState.conf.adaptiveExecutionEnabled) { val executedPlan = joined.queryExecution.executedPlan.asInstanceOf[AdaptiveSparkPlanExec].executedPlan @@ -2625,7 +2633,7 @@ index 3573bafe482..11d387110ea 100644 s"expected sort in the right child to be $sortRight but found\n${joinOperator.right}") // check the output partitioning -@@ -836,11 +867,11 @@ abstract class BucketedReadSuite extends QueryTest with SQLTestUtils with Adapti +@@ -836,11 +869,11 @@ abstract class BucketedReadSuite extends QueryTest with SQLTestUtils with Adapti df1.write.format("parquet").bucketBy(8, "i").saveAsTable("bucketed_table") val scanDF = spark.table("bucketed_table").select("j") @@ -2639,7 +2647,31 @@ index 3573bafe482..11d387110ea 100644 checkAnswer(aggDF, df1.groupBy("j").agg(max("k"))) } } -@@ -1029,15 +1060,21 @@ abstract class BucketedReadSuite extends QueryTest with SQLTestUtils with Adapti +@@ -896,6 +929,7 @@ abstract class BucketedReadSuite extends QueryTest with SQLTestUtils with Adapti + + test("SPARK-29655 Read bucketed tables obeys spark.sql.shuffle.partitions") { + withSQLConf( ++ CometConf.COMET_EXEC_SHUFFLE_WITH_RANGE_PARTITIONING_ENABLED.key -> "false", + SQLConf.SHUFFLE_PARTITIONS.key -> "5", + SQLConf.COALESCE_PARTITIONS_INITIAL_PARTITION_NUM.key -> "7") { + val bucketSpec = Some(BucketSpec(6, Seq("i", "j"), Nil)) +@@ -915,6 +949,7 @@ abstract class BucketedReadSuite extends QueryTest with SQLTestUtils with Adapti + + test("SPARK-32767 Bucket join should work if SHUFFLE_PARTITIONS larger than bucket number") { + withSQLConf( ++ CometConf.COMET_EXEC_SHUFFLE_WITH_RANGE_PARTITIONING_ENABLED.key -> "false", + SQLConf.SHUFFLE_PARTITIONS.key -> "9", + SQLConf.COALESCE_PARTITIONS_INITIAL_PARTITION_NUM.key -> "10") { + +@@ -945,6 +980,7 @@ abstract class BucketedReadSuite extends QueryTest with SQLTestUtils with Adapti + + test("bucket coalescing eliminates shuffle") { + withSQLConf( ++ CometConf.COMET_EXEC_SHUFFLE_WITH_RANGE_PARTITIONING_ENABLED.key -> "false", + SQLConf.COALESCE_BUCKETS_IN_JOIN_ENABLED.key -> "true", + SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "false") { + // The side with bucketedTableTestSpec1 will be coalesced to have 4 output partitions. +@@ -1029,15 +1065,21 @@ abstract class BucketedReadSuite extends QueryTest with SQLTestUtils with Adapti Seq(true, false).foreach { aqeEnabled => withSQLConf(SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> aqeEnabled.toString) { val plan = sql(query).queryExecution.executedPlan @@ -3036,6 +3068,32 @@ index 52abd248f3a..7a199931a08 100644 case h: HiveTableScanExec => h.partitionPruningPred.collect { case d: DynamicPruningExpression => d.child } +diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala +index 0bcac639443..ee441b9b431 100644 +--- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala ++++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala +@@ -24,6 +24,7 @@ import java.sql.{Date, Timestamp} + import java.util.{Locale, Set} + + import com.google.common.io.Files ++import org.apache.comet.CometConf + import org.apache.hadoop.fs.{FileSystem, Path} + + import org.apache.spark.{SparkException, TestUtils} +@@ -838,8 +839,11 @@ abstract class SQLQuerySuiteBase extends QueryTest with SQLTestUtils with TestHi + } + + test("SPARK-2554 SumDistinct partial aggregation") { +- checkAnswer(sql("SELECT sum( distinct key) FROM src group by key order by key"), +- sql("SELECT distinct key FROM src order by key").collect().toSeq) ++ withSQLConf(CometConf.COMET_EXEC_SHUFFLE_WITH_RANGE_PARTITIONING_ENABLED.key -> "false") ++ { ++ checkAnswer(sql("SELECT sum( distinct key) FROM src group by key order by key"), ++ sql("SELECT distinct key FROM src order by key").collect().toSeq) ++ } + } + + test("SPARK-4963 DataFrame sample on mutable row return wrong result") { diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/test/TestHive.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/test/TestHive.scala index 3f8de93b330..53417076481 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/test/TestHive.scala From 0ed15acadc5fbb1c2149d9e1a5b5c5793349c610 Mon Sep 17 00:00:00 2001 From: Matt Butrovich Date: Sat, 14 Jun 2025 09:51:33 -0400 Subject: [PATCH 66/72] Add comments to diffs. --- dev/diffs/3.4.3.diff | 25 ++++++++++++++++++------- dev/diffs/3.5.6.diff | 25 ++++++++++++++++++------- dev/diffs/4.0.0-preview1.diff | 25 ++++++++++++++++++------- 3 files changed, 54 insertions(+), 21 deletions(-) diff --git a/dev/diffs/3.4.3.diff b/dev/diffs/3.4.3.diff index 8bd868346c..5233cdb000 100644 --- a/dev/diffs/3.4.3.diff +++ b/dev/diffs/3.4.3.diff @@ -2282,7 +2282,7 @@ index d083cac48ff..3c11bcde807 100644 import testImplicits._ diff --git a/sql/core/src/test/scala/org/apache/spark/sql/sources/BucketedReadSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/sources/BucketedReadSuite.scala -index 266bb343526..1c600e6e84c 100644 +index 266bb343526..6675cf7b636 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/sources/BucketedReadSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/sources/BucketedReadSuite.scala @@ -19,15 +19,18 @@ package org.apache.spark.sql.sources @@ -2411,31 +2411,40 @@ index 266bb343526..1c600e6e84c 100644 checkAnswer(aggDF, df1.groupBy("j").agg(max("k"))) } } -@@ -895,6 +928,7 @@ abstract class BucketedReadSuite extends QueryTest with SQLTestUtils with Adapti +@@ -894,7 +927,10 @@ abstract class BucketedReadSuite extends QueryTest with SQLTestUtils with Adapti + } test("SPARK-29655 Read bucketed tables obeys spark.sql.shuffle.partitions") { ++ // Range partitioning uses random samples, so per-partition comparisons do not always yield ++ // the same results. Disable Comet native range partitioning. withSQLConf( + CometConf.COMET_EXEC_SHUFFLE_WITH_RANGE_PARTITIONING_ENABLED.key -> "false", SQLConf.SHUFFLE_PARTITIONS.key -> "5", SQLConf.COALESCE_PARTITIONS_INITIAL_PARTITION_NUM.key -> "7") { val bucketSpec = Some(BucketSpec(6, Seq("i", "j"), Nil)) -@@ -914,6 +948,7 @@ abstract class BucketedReadSuite extends QueryTest with SQLTestUtils with Adapti +@@ -913,7 +949,10 @@ abstract class BucketedReadSuite extends QueryTest with SQLTestUtils with Adapti + } test("SPARK-32767 Bucket join should work if SHUFFLE_PARTITIONS larger than bucket number") { ++ // Range partitioning uses random samples, so per-partition comparisons do not always yield ++ // the same results. Disable Comet native range partitioning. withSQLConf( + CometConf.COMET_EXEC_SHUFFLE_WITH_RANGE_PARTITIONING_ENABLED.key -> "false", SQLConf.SHUFFLE_PARTITIONS.key -> "9", SQLConf.COALESCE_PARTITIONS_INITIAL_PARTITION_NUM.key -> "10") { -@@ -944,6 +979,7 @@ abstract class BucketedReadSuite extends QueryTest with SQLTestUtils with Adapti +@@ -943,7 +982,10 @@ abstract class BucketedReadSuite extends QueryTest with SQLTestUtils with Adapti + } test("bucket coalescing eliminates shuffle") { ++ // Range partitioning uses random samples, so per-partition comparisons do not always yield ++ // the same results. Disable Comet native range partitioning. withSQLConf( + CometConf.COMET_EXEC_SHUFFLE_WITH_RANGE_PARTITIONING_ENABLED.key -> "false", SQLConf.COALESCE_BUCKETS_IN_JOIN_ENABLED.key -> "true", SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "false") { // The side with bucketedTableTestSpec1 will be coalesced to have 4 output partitions. -@@ -1026,15 +1062,23 @@ abstract class BucketedReadSuite extends QueryTest with SQLTestUtils with Adapti +@@ -1026,15 +1068,23 @@ abstract class BucketedReadSuite extends QueryTest with SQLTestUtils with Adapti expectedNumShuffles: Int, expectedCoalescedNumBuckets: Option[Int]): Unit = { val plan = sql(query).queryExecution.executedPlan @@ -2848,7 +2857,7 @@ index 52abd248f3a..7a199931a08 100644 case d: DynamicPruningExpression => d.child } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala -index a902cb3a69e..a39bb7a7658 100644 +index a902cb3a69e..800a3acbe99 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala @@ -24,6 +24,7 @@ import java.sql.{Date, Timestamp} @@ -2859,12 +2868,14 @@ index a902cb3a69e..a39bb7a7658 100644 import org.apache.hadoop.fs.{FileSystem, Path} import org.apache.spark.{SparkException, TestUtils} -@@ -838,8 +839,11 @@ abstract class SQLQuerySuiteBase extends QueryTest with SQLTestUtils with TestHi +@@ -838,8 +839,13 @@ abstract class SQLQuerySuiteBase extends QueryTest with SQLTestUtils with TestHi } test("SPARK-2554 SumDistinct partial aggregation") { - checkAnswer(sql("SELECT sum( distinct key) FROM src group by key order by key"), - sql("SELECT distinct key FROM src order by key").collect().toSeq) ++ // Range partitioning uses random samples, so per-partition comparisons do not always yield ++ // the same results. Disable Comet native range partitioning. + withSQLConf(CometConf.COMET_EXEC_SHUFFLE_WITH_RANGE_PARTITIONING_ENABLED.key -> "false") + { + checkAnswer(sql("SELECT sum( distinct key) FROM src group by key order by key"), diff --git a/dev/diffs/3.5.6.diff b/dev/diffs/3.5.6.diff index 6dc418a1d5..0d35cbf7ca 100644 --- a/dev/diffs/3.5.6.diff +++ b/dev/diffs/3.5.6.diff @@ -2278,7 +2278,7 @@ index d083cac48ff..3c11bcde807 100644 import testImplicits._ diff --git a/sql/core/src/test/scala/org/apache/spark/sql/sources/BucketedReadSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/sources/BucketedReadSuite.scala -index 746f289c393..199ee138375 100644 +index 746f289c393..a90106a1463 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/sources/BucketedReadSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/sources/BucketedReadSuite.scala @@ -19,16 +19,19 @@ package org.apache.spark.sql.sources @@ -2408,31 +2408,40 @@ index 746f289c393..199ee138375 100644 checkAnswer(aggDF, df1.groupBy("j").agg(max("k"))) } } -@@ -896,6 +929,7 @@ abstract class BucketedReadSuite extends QueryTest with SQLTestUtils with Adapti +@@ -895,7 +928,10 @@ abstract class BucketedReadSuite extends QueryTest with SQLTestUtils with Adapti + } test("SPARK-29655 Read bucketed tables obeys spark.sql.shuffle.partitions") { ++ // Range partitioning uses random samples, so per-partition comparisons do not always yield ++ // the same results. Disable Comet native range partitioning. withSQLConf( + CometConf.COMET_EXEC_SHUFFLE_WITH_RANGE_PARTITIONING_ENABLED.key -> "false", SQLConf.SHUFFLE_PARTITIONS.key -> "5", SQLConf.COALESCE_PARTITIONS_INITIAL_PARTITION_NUM.key -> "7") { val bucketSpec = Some(BucketSpec(6, Seq("i", "j"), Nil)) -@@ -915,6 +949,7 @@ abstract class BucketedReadSuite extends QueryTest with SQLTestUtils with Adapti +@@ -914,7 +950,10 @@ abstract class BucketedReadSuite extends QueryTest with SQLTestUtils with Adapti + } test("SPARK-32767 Bucket join should work if SHUFFLE_PARTITIONS larger than bucket number") { ++ // Range partitioning uses random samples, so per-partition comparisons do not always yield ++ // the same results. Disable Comet native range partitioning. withSQLConf( + CometConf.COMET_EXEC_SHUFFLE_WITH_RANGE_PARTITIONING_ENABLED.key -> "false", SQLConf.SHUFFLE_PARTITIONS.key -> "9", SQLConf.COALESCE_PARTITIONS_INITIAL_PARTITION_NUM.key -> "10") { -@@ -945,6 +980,7 @@ abstract class BucketedReadSuite extends QueryTest with SQLTestUtils with Adapti +@@ -944,7 +983,10 @@ abstract class BucketedReadSuite extends QueryTest with SQLTestUtils with Adapti + } test("bucket coalescing eliminates shuffle") { ++ // Range partitioning uses random samples, so per-partition comparisons do not always yield ++ // the same results. Disable Comet native range partitioning. withSQLConf( + CometConf.COMET_EXEC_SHUFFLE_WITH_RANGE_PARTITIONING_ENABLED.key -> "false", SQLConf.COALESCE_BUCKETS_IN_JOIN_ENABLED.key -> "true", SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "false") { // The side with bucketedTableTestSpec1 will be coalesced to have 4 output partitions. -@@ -1029,15 +1065,21 @@ abstract class BucketedReadSuite extends QueryTest with SQLTestUtils with Adapti +@@ -1029,15 +1071,21 @@ abstract class BucketedReadSuite extends QueryTest with SQLTestUtils with Adapti Seq(true, false).foreach { aqeEnabled => withSQLConf(SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> aqeEnabled.toString) { val plan = sql(query).queryExecution.executedPlan @@ -2863,7 +2872,7 @@ index 549431ef4f4..e48f1730da6 100644 withSQLConf( "parquet.crypto.factory.class" -> diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala -index 6160c3e5f6c..d6761dc6f7c 100644 +index 6160c3e5f6c..0956d7d9edc 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala @@ -24,6 +24,7 @@ import java.sql.{Date, Timestamp} @@ -2874,12 +2883,14 @@ index 6160c3e5f6c..d6761dc6f7c 100644 import org.apache.hadoop.fs.{FileSystem, Path} import org.apache.spark.{SparkException, TestUtils} -@@ -838,8 +839,11 @@ abstract class SQLQuerySuiteBase extends QueryTest with SQLTestUtils with TestHi +@@ -838,8 +839,13 @@ abstract class SQLQuerySuiteBase extends QueryTest with SQLTestUtils with TestHi } test("SPARK-2554 SumDistinct partial aggregation") { - checkAnswer(sql("SELECT sum( distinct key) FROM src group by key order by key"), - sql("SELECT distinct key FROM src order by key").collect().toSeq) ++ // Range partitioning uses random samples, so per-partition comparisons do not always yield ++ // the same results. Disable Comet native range partitioning. + withSQLConf(CometConf.COMET_EXEC_SHUFFLE_WITH_RANGE_PARTITIONING_ENABLED.key -> "false") + { + checkAnswer(sql("SELECT sum( distinct key) FROM src group by key order by key"), diff --git a/dev/diffs/4.0.0-preview1.diff b/dev/diffs/4.0.0-preview1.diff index 2996538afe..f5d9197481 100644 --- a/dev/diffs/4.0.0-preview1.diff +++ b/dev/diffs/4.0.0-preview1.diff @@ -2517,7 +2517,7 @@ index 6ff07449c0c..9f95cff99e5 100644 import testImplicits._ diff --git a/sql/core/src/test/scala/org/apache/spark/sql/sources/BucketedReadSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/sources/BucketedReadSuite.scala -index 3573bafe482..9cfaa0b93c1 100644 +index 3573bafe482..8bf48a94fba 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/sources/BucketedReadSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/sources/BucketedReadSuite.scala @@ -19,16 +19,19 @@ package org.apache.spark.sql.sources @@ -2647,31 +2647,40 @@ index 3573bafe482..9cfaa0b93c1 100644 checkAnswer(aggDF, df1.groupBy("j").agg(max("k"))) } } -@@ -896,6 +929,7 @@ abstract class BucketedReadSuite extends QueryTest with SQLTestUtils with Adapti +@@ -895,7 +928,10 @@ abstract class BucketedReadSuite extends QueryTest with SQLTestUtils with Adapti + } test("SPARK-29655 Read bucketed tables obeys spark.sql.shuffle.partitions") { ++ // Range partitioning uses random samples, so per-partition comparisons do not always yield ++ // the same results. Disable Comet native range partitioning. withSQLConf( + CometConf.COMET_EXEC_SHUFFLE_WITH_RANGE_PARTITIONING_ENABLED.key -> "false", SQLConf.SHUFFLE_PARTITIONS.key -> "5", SQLConf.COALESCE_PARTITIONS_INITIAL_PARTITION_NUM.key -> "7") { val bucketSpec = Some(BucketSpec(6, Seq("i", "j"), Nil)) -@@ -915,6 +949,7 @@ abstract class BucketedReadSuite extends QueryTest with SQLTestUtils with Adapti +@@ -914,7 +950,10 @@ abstract class BucketedReadSuite extends QueryTest with SQLTestUtils with Adapti + } test("SPARK-32767 Bucket join should work if SHUFFLE_PARTITIONS larger than bucket number") { ++ // Range partitioning uses random samples, so per-partition comparisons do not always yield ++ // the same results. Disable Comet native range partitioning. withSQLConf( + CometConf.COMET_EXEC_SHUFFLE_WITH_RANGE_PARTITIONING_ENABLED.key -> "false", SQLConf.SHUFFLE_PARTITIONS.key -> "9", SQLConf.COALESCE_PARTITIONS_INITIAL_PARTITION_NUM.key -> "10") { -@@ -945,6 +980,7 @@ abstract class BucketedReadSuite extends QueryTest with SQLTestUtils with Adapti +@@ -944,7 +983,10 @@ abstract class BucketedReadSuite extends QueryTest with SQLTestUtils with Adapti + } test("bucket coalescing eliminates shuffle") { ++ // Range partitioning uses random samples, so per-partition comparisons do not always yield ++ // the same results. Disable Comet native range partitioning. withSQLConf( + CometConf.COMET_EXEC_SHUFFLE_WITH_RANGE_PARTITIONING_ENABLED.key -> "false", SQLConf.COALESCE_BUCKETS_IN_JOIN_ENABLED.key -> "true", SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "false") { // The side with bucketedTableTestSpec1 will be coalesced to have 4 output partitions. -@@ -1029,15 +1065,21 @@ abstract class BucketedReadSuite extends QueryTest with SQLTestUtils with Adapti +@@ -1029,15 +1071,21 @@ abstract class BucketedReadSuite extends QueryTest with SQLTestUtils with Adapti Seq(true, false).foreach { aqeEnabled => withSQLConf(SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> aqeEnabled.toString) { val plan = sql(query).queryExecution.executedPlan @@ -3069,7 +3078,7 @@ index 52abd248f3a..7a199931a08 100644 case d: DynamicPruningExpression => d.child } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala -index 0bcac639443..ee441b9b431 100644 +index 0bcac639443..8957c76886f 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala @@ -24,6 +24,7 @@ import java.sql.{Date, Timestamp} @@ -3080,12 +3089,14 @@ index 0bcac639443..ee441b9b431 100644 import org.apache.hadoop.fs.{FileSystem, Path} import org.apache.spark.{SparkException, TestUtils} -@@ -838,8 +839,11 @@ abstract class SQLQuerySuiteBase extends QueryTest with SQLTestUtils with TestHi +@@ -838,8 +839,13 @@ abstract class SQLQuerySuiteBase extends QueryTest with SQLTestUtils with TestHi } test("SPARK-2554 SumDistinct partial aggregation") { - checkAnswer(sql("SELECT sum( distinct key) FROM src group by key order by key"), - sql("SELECT distinct key FROM src order by key").collect().toSeq) ++ // Range partitioning uses random samples, so per-partition comparisons do not always yield ++ // the same results. Disable Comet native range partitioning. + withSQLConf(CometConf.COMET_EXEC_SHUFFLE_WITH_RANGE_PARTITIONING_ENABLED.key -> "false") + { + checkAnswer(sql("SELECT sum( distinct key) FROM src group by key order by key"), From b8d1acafaa58e84f3b98c2d2b175ae58ead43bc1 Mon Sep 17 00:00:00 2001 From: Matt Butrovich Date: Sat, 14 Jun 2025 10:00:46 -0400 Subject: [PATCH 67/72] Merge tests. --- .../comet/exec/CometNativeShuffleSuite.scala | 36 ++++--------------- 1 file changed, 7 insertions(+), 29 deletions(-) diff --git a/spark/src/test/scala/org/apache/comet/exec/CometNativeShuffleSuite.scala b/spark/src/test/scala/org/apache/comet/exec/CometNativeShuffleSuite.scala index e1028e301d..4543f32913 100644 --- a/spark/src/test/scala/org/apache/comet/exec/CometNativeShuffleSuite.scala +++ b/spark/src/test/scala/org/apache/comet/exec/CometNativeShuffleSuite.scala @@ -120,10 +120,11 @@ class CometNativeShuffleSuite extends CometTestBase with AdaptiveSparkPlanHelper } } - test("native operator after native shuffle with hash partitioning") { - Seq("true", "false").foreach { hashPartitioningEnabled => + test("native operator after native shuffle") { + Seq("true", "false").zip(Seq("true", "false")).foreach { partitioning => withSQLConf( - CometConf.COMET_EXEC_SHUFFLE_WITH_HASH_PARTITIONING_ENABLED.key -> hashPartitioningEnabled) { + CometConf.COMET_EXEC_SHUFFLE_WITH_HASH_PARTITIONING_ENABLED.key -> partitioning._1, + CometConf.COMET_EXEC_SHUFFLE_WITH_RANGE_PARTITIONING_ENABLED.key -> partitioning._2) { withParquetTable((0 until 5).map(i => (i, (i + 1).toLong)), "tbl") { val df = sql("SELECT * FROM tbl") @@ -133,32 +134,9 @@ class CometNativeShuffleSuite extends CometTestBase with AdaptiveSparkPlanHelper .repartition(10, $"_1") .filter($"_1" > 1) - // native shuffle supports HashPartitioning, so 2 Comet shuffle exchanges are expected - if (hashPartitioningEnabled == "true") { - checkShuffleAnswer(shuffled, 2) - } else { - checkShuffleAnswer(shuffled, 0) - } - } - } - } - } - - test("native operator after native shuffle with range partitioning") { - Seq("true", "false").foreach { rangePartitioningEnabled => - withSQLConf( - CometConf.COMET_EXEC_SHUFFLE_WITH_RANGE_PARTITIONING_ENABLED.key -> rangePartitioningEnabled) { - withParquetTable((0 until 5).map(i => (i, (i + 1).toLong)), "tbl") { - val df = sql("SELECT * FROM tbl") - - val shuffled = df - .repartitionByRange(10, $"_2") - .select($"_1", $"_1" + 1, $"_2" + 2) - .repartition(10, $"_1") - .filter($"_1" > 1) - - // native shuffle supports RangePartitioning, so 2 Comet shuffle exchanges are expected - if (rangePartitioningEnabled == "true") { + // We expect a hash and range partitioned exchanges. If both are true, we'll get two + // native exchanges. Otherwise both will fall back. + if (partitioning._1 == "true" && partitioning._2 == "true") { checkShuffleAnswer(shuffled, 2) } else { checkShuffleAnswer(shuffled, 0) From cc225481a40b1191549e78212770c30b50c4d937 Mon Sep 17 00:00:00 2001 From: Matt Butrovich Date: Tue, 17 Jun 2025 08:21:37 -0400 Subject: [PATCH 68/72] Change math in determine_bounds_for_rows to see if that makes Miri happy. --- native/core/src/execution/shuffle/range_partitioner.rs | 8 +++++--- 1 file changed, 5 insertions(+), 3 deletions(-) diff --git a/native/core/src/execution/shuffle/range_partitioner.rs b/native/core/src/execution/shuffle/range_partitioner.rs index 1306d3b7da..4c74b9dbff 100644 --- a/native/core/src/execution/shuffle/range_partitioner.rs +++ b/native/core/src/execution/shuffle/range_partitioner.rs @@ -141,6 +141,8 @@ impl RangePartitioner { /// 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], @@ -154,14 +156,14 @@ impl RangePartitioner { 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 step = num_candidates; // this was 1.0 / num_partitions + let mut cumulative_weights = 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; + let sample_weight = partitions; // this was 1.0 / num_candidates while (i < num_candidates) && (j < partitions - 1) { let key = sorted_sampled_rows[i]; cumulative_weights += sample_weight; From 399c87c15f86754863fbf35d000e0920be1c7056 Mon Sep 17 00:00:00 2001 From: Matt Butrovich Date: Tue, 17 Jun 2025 09:39:14 -0400 Subject: [PATCH 69/72] Miri-specific loop bounds. --- native/core/src/execution/shuffle/range_partitioner.rs | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/native/core/src/execution/shuffle/range_partitioner.rs b/native/core/src/execution/shuffle/range_partitioner.rs index 4c74b9dbff..c2f474a3be 100644 --- a/native/core/src/execution/shuffle/range_partitioner.rs +++ b/native/core/src/execution/shuffle/range_partitioner.rs @@ -316,7 +316,7 @@ mod test { fn reservoir_sample_random() { let mut rng = SmallRng::seed_from_u64(42); - for _ in 0..8192 { + for _ in 0..if cfg!(miri) { 128 } 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. @@ -392,7 +392,7 @@ mod test { let sort_fields = vec![SortField::new(Int64)]; - for _ in 0..2048 { + for _ in 0..if cfg!(miri) { 64 } 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. From 5fc29cc2dce181a43cc3e65a02263718559fbb40 Mon Sep 17 00:00:00 2001 From: Matt Butrovich Date: Tue, 17 Jun 2025 09:44:44 -0400 Subject: [PATCH 70/72] Add comments about Miri. --- native/core/src/execution/shuffle/range_partitioner.rs | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/native/core/src/execution/shuffle/range_partitioner.rs b/native/core/src/execution/shuffle/range_partitioner.rs index c2f474a3be..209dea9538 100644 --- a/native/core/src/execution/shuffle/range_partitioner.rs +++ b/native/core/src/execution/shuffle/range_partitioner.rs @@ -316,6 +316,8 @@ mod test { 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) { 128 } 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 @@ -392,6 +394,8 @@ mod test { 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) { 64 } else { 2048 } { let batch_size = rng.random_range(0..=8192); // We don't test fewer than 2 partitions since this is used by the From 46a08420b075a795b61345ca17c8e81901951c69 Mon Sep 17 00:00:00 2001 From: Matt Butrovich Date: Tue, 17 Jun 2025 10:33:12 -0400 Subject: [PATCH 71/72] Reduce Miri loop iterations further. --- native/core/src/execution/shuffle/range_partitioner.rs | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/native/core/src/execution/shuffle/range_partitioner.rs b/native/core/src/execution/shuffle/range_partitioner.rs index 209dea9538..423b14b3eb 100644 --- a/native/core/src/execution/shuffle/range_partitioner.rs +++ b/native/core/src/execution/shuffle/range_partitioner.rs @@ -318,7 +318,7 @@ mod test { // 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) { 128 } else { 8192 } { + 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. @@ -396,7 +396,7 @@ mod test { // 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) { 64 } else { 2048 } { + 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. From d527dfedef4511b6d30aba367f6e9f75b167853e Mon Sep 17 00:00:00 2001 From: Matt Butrovich Date: Tue, 17 Jun 2025 13:39:30 -0400 Subject: [PATCH 72/72] Revert math changes since the Spark SQL test 22160 doesn't like it, and we fixed it with Miri-specific changes. --- native/core/src/execution/shuffle/range_partitioner.rs | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/native/core/src/execution/shuffle/range_partitioner.rs b/native/core/src/execution/shuffle/range_partitioner.rs index 423b14b3eb..8f443232f9 100644 --- a/native/core/src/execution/shuffle/range_partitioner.rs +++ b/native/core/src/execution/shuffle/range_partitioner.rs @@ -156,14 +156,14 @@ impl RangePartitioner { sorted_sampled_rows.sort_unstable_by(|(_, a), (_, b)| a.cmp(b)); let num_candidates = sampled_rows.num_rows(); - let step = num_candidates; // this was 1.0 / num_partitions - let mut cumulative_weights = 0; + 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 = partitions; // this was 1.0 / num_candidates + 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;