Skip to content
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
Show all changes
47 commits
Select commit Hold shift + click to select a range
709d6e9
Use Spark's RangePartitioning to compute boundary rows and serialize …
mbutrovich Aug 28, 2025
4ee3d8e
Fix warnings and benchmark compilation.
mbutrovich Aug 28, 2025
6f34e35
Fix benchmark bug.
mbutrovich Aug 28, 2025
332e76a
Minor refactor.
mbutrovich Aug 28, 2025
0eb1134
Cleanup to make it more clear what code came from Spark.
mbutrovich Aug 28, 2025
bb67f73
Fix errant comment.
mbutrovich Aug 28, 2025
abd8958
Override partitioning scheme at serialization when num_partitions is 1.
mbutrovich Aug 28, 2025
967d1a1
Override partitioning scheme at serialization when computed bounds re…
mbutrovich Aug 28, 2025
7af9474
Merge branch 'main' into fix_range_partitioning
mbutrovich Aug 29, 2025
522ef80
Remove string and binary range partitioning types until we sort out h…
mbutrovich Aug 29, 2025
1a956a5
Merge branch 'main' into fix_range_partitioning
mbutrovich Sep 11, 2025
58e35b0
Fix circular dependency in execution.
mbutrovich Sep 11, 2025
08a4b51
Update plans.
mbutrovich Sep 11, 2025
2f4280f
fix Spark SQL test "change SQLConf should not change view behavior - …
mbutrovich Sep 11, 2025
58f2eda
Fix bug with indexing into boundary rows.
mbutrovich Sep 12, 2025
2803842
Merge branch 'main' into fix_range_partitioning
mbutrovich Sep 13, 2025
5bceb41
Merge branch 'main' into fix_range_partitioning
mbutrovich Sep 16, 2025
044e098
Remove range_partitioner.rs (native bounds calculation and reservoir …
mbutrovich Sep 16, 2025
21c4665
remove errant collection.JavaConverters
mbutrovich Sep 16, 2025
4e86961
Merge branch 'main' into fix_range_partitioning
mbutrovich Sep 16, 2025
c9acdfc
Remove redundant config (setting to default).
mbutrovich Sep 16, 2025
386aa6c
Merge branch 'main' into fix_range_partitioning
mbutrovich Sep 17, 2025
261ea33
Hoist wrapping the child of ShuffleWriter in a CopyExec before we gen…
mbutrovich Sep 17, 2025
4077f7d
Update test after last commit.
mbutrovich Sep 17, 2025
dd0939f
Merge branch 'main' into fix_range_partitioning
mbutrovich Sep 19, 2025
764675b
Add a reduced test case for the Spark SQL test failure.
mbutrovich Sep 19, 2025
91fef94
More complicated test case.
mbutrovich Sep 19, 2025
3a83258
More complicated test case.
mbutrovich Sep 19, 2025
a1adf7f
Test fix for duplicate expression references in range partitioning.
mbutrovich Sep 19, 2025
f287dd1
New plans?
mbutrovich Sep 20, 2025
a36ab4f
Merge branch 'main' into fix_range_partitioning
mbutrovich Sep 20, 2025
ae7ea4c
Merge branch 'main' into fix_range_partitioning
mbutrovich Sep 20, 2025
1a1d329
new plans.
mbutrovich Sep 20, 2025
6456d4e
New tests to replicate Spark SQL failure.
mbutrovich Sep 20, 2025
e24a36b
switch to using hash-based deduplication like DataFusion's LexOrderin…
mbutrovich Sep 20, 2025
b7a078a
Minor refactor.
mbutrovich Sep 20, 2025
5305366
Merge branch 'main' into fix_range_partitioning
mbutrovich Sep 20, 2025
4233b42
add benchmark.
mbutrovich Sep 21, 2025
f6e5c90
Merge branch 'apache:main' into fix_range_partitioning
mbutrovich Sep 22, 2025
b5b286b
Remove development test, add more tests for duplicates in columns and…
mbutrovich Sep 22, 2025
d12c855
Merge remote-tracking branch 'origin/fix_range_partitioning' into fix…
mbutrovich Sep 22, 2025
fa2c20b
Update docs.
mbutrovich Sep 23, 2025
8781264
PR feedback.
mbutrovich Sep 23, 2025
318adbd
Fix clippy.
mbutrovich Sep 23, 2025
2aa3f0f
Update comments based on PR feedback.
mbutrovich Sep 24, 2025
3d359ba
Update spark/src/main/scala/org/apache/spark/sql/comet/execution/shuf…
mbutrovich Sep 24, 2025
c87aba7
Fix formatting after accepting change on GitHub.
mbutrovich Sep 24, 2025
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
7 changes: 2 additions & 5 deletions common/src/main/scala/org/apache/comet/CometConf.scala
Original file line number Diff line number Diff line change
Expand Up @@ -325,14 +325,11 @@ object CometConf extends ShimCometConf {
.booleanConf
.createWithDefault(true)

// RangePartitioning contains bugs https://github.com/apache/datafusion-comet/issues/1906
val COMET_EXEC_SHUFFLE_WITH_RANGE_PARTITIONING_ENABLED: ConfigEntry[Boolean] =
conf("spark.comet.native.shuffle.partitioning.range.enabled")
.doc("Experimental feature to enable range partitioning for Comet native shuffle. " +
"This feature is experimental while we investigate scenarios that don't partition data " +
"correctly.")
.doc("Whether to enable range partitioning for Comet native shuffle.")
.booleanConf
.createWithDefault(false)
.createWithDefault(true)
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

should we keep it as false

then run some benches and real tests with this param true and later enable it by default?

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I discussed with @andygrove and we were comfortable merging with true back in June. I think if you're opting into native shuffle we should try to accelerate all partitioning schemes, and if we discover issues it can be toggled off.

Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Enabling it by default now gives us more opportunities to find bugs over the next few weeks before we release 0.11.0 and we can always disable if we find issues in that time.


val COMET_EXEC_SHUFFLE_COMPRESSION_CODEC: ConfigEntry[String] =
conf(s"$COMET_EXEC_CONFIG_PREFIX.shuffle.compression.codec")
Expand Down
2 changes: 1 addition & 1 deletion docs/source/user-guide/latest/configs.md
Original file line number Diff line number Diff line change
Expand Up @@ -72,7 +72,7 @@ Comet provides the following configuration settings.
| spark.comet.memoryOverhead | The amount of additional memory to be allocated per executor process for Comet, in MiB, when running Spark in on-heap mode. This config is optional. If this is not specified, it will be set to `spark.comet.memory.overhead.factor` * `spark.executor.memory`. For more information, refer to the Comet Tuning Guide (https://datafusion.apache.org/comet/user-guide/tuning.html). | |
| spark.comet.metrics.updateInterval | The interval in milliseconds to update metrics. If interval is negative, metrics will be updated upon task completion. | 3000 |
| spark.comet.native.shuffle.partitioning.hash.enabled | Whether to enable hash partitioning for Comet native shuffle. | true |
| spark.comet.native.shuffle.partitioning.range.enabled | Experimental feature to enable range partitioning for Comet native shuffle. This feature is experimental while we investigate scenarios that don't partition data correctly. | false |
| spark.comet.native.shuffle.partitioning.range.enabled | Whether to enable range partitioning for Comet native shuffle. | true |
| spark.comet.nativeLoadRequired | Whether to require Comet native library to load successfully when Comet is enabled. If not, Comet will silently fallback to Spark when it fails to load the native lib. Otherwise, an error will be thrown and the Spark job will be aborted. | false |
| spark.comet.parquet.enable.directBuffer | Whether to use Java direct byte buffer when reading Parquet. | false |
| spark.comet.parquet.read.io.adjust.readRange.skew | In the parallel reader, if the read ranges submitted are skewed in sizes, this option will cause the reader to break up larger read ranges into smaller ranges to reduce the skew. This will result in a slightly larger number of connections opened to the file system but may give improved performance. | false |
Expand Down
41 changes: 32 additions & 9 deletions native/core/benches/shuffle_writer.rs
Original file line number Diff line number Diff line change
Expand Up @@ -16,8 +16,9 @@
// under the License.

use arrow::array::builder::{Date32Builder, Decimal128Builder, Int32Builder};
use arrow::array::{builder::StringBuilder, RecordBatch};
use arrow::array::{builder::StringBuilder, Array, Int32Array, RecordBatch};
use arrow::datatypes::{DataType, Field, Schema};
use arrow::row::{RowConverter, SortField};
use comet::execution::shuffle::{
CometPartitioning, CompressionCodec, ShuffleBlockWriter, ShuffleWriterExec,
};
Expand All @@ -31,6 +32,7 @@ use datafusion::{
physical_plan::{common::collect, ExecutionPlan},
prelude::SessionContext,
};
use itertools::Itertools;
use std::io::Cursor;
use std::sync::Arc;
use tokio::runtime::Runtime;
Expand Down Expand Up @@ -84,16 +86,37 @@ fn criterion_benchmark(c: &mut Criterion) {
);
}

let lex_ordering = LexOrdering::new(vec![PhysicalSortExpr::new_default(
col("c0", batch.schema().as_ref()).unwrap(),
)])
.unwrap();

let sort_fields: Vec<SortField> = batch
.columns()
.iter()
.zip(&lex_ordering)
.map(|(array, sort_expr)| {
SortField::new_with_options(array.data_type().clone(), sort_expr.options)
})
.collect();
let row_converter = RowConverter::new(sort_fields).unwrap();

// These are hard-coded values based on the benchmark params of 8192 rows per batch, and 16
// partitions. If these change, these values need to be recalculated, or bring over the
// bounds-finding logic from shuffle_write_test in shuffle_writer.rs.
let bounds_ints = vec![
512, 1024, 1536, 2048, 2560, 3072, 3584, 4096, 4608, 5120, 5632, 6144, 6656, 7168, 7680,
];
let bounds_array: Arc<dyn Array> = Arc::new(Int32Array::from(bounds_ints));
let bounds_rows = row_converter
.convert_columns(vec![bounds_array].as_slice())
.unwrap();

let owned_rows = bounds_rows.iter().map(|row| row.owned()).collect_vec();

for partitioning in [
CometPartitioning::Hash(vec![Arc::new(Column::new("a", 0))], 16),
CometPartitioning::RangePartitioning(
LexOrdering::new(vec![PhysicalSortExpr::new_default(
col("c0", batch.schema().as_ref()).unwrap(),
)])
.unwrap(),
16,
100,
),
CometPartitioning::RangePartitioning(lex_ordering, 16, Arc::new(row_converter), owned_rows),
] {
let compression_codec = CompressionCodec::None;
group.bench_function(
Expand Down
68 changes: 62 additions & 6 deletions native/core/src/execution/planner.rs
Original file line number Diff line number Diff line change
Expand Up @@ -92,6 +92,7 @@ use arrow::array::{
NullArray, StringBuilder, TimestampMicrosecondArray,
};
use arrow::buffer::{BooleanBuffer, NullBuffer, OffsetBuffer};
use arrow::row::{OwnedRow, RowConverter, SortField};
use datafusion::common::utils::SingleRowListArrayBuilder;
use datafusion::physical_plan::coalesce_batches::CoalesceBatchesExec;
use datafusion::physical_plan::filter::FilterExec;
Expand Down Expand Up @@ -484,14 +485,14 @@ impl PhysicalPlanner {
)))
}
}
},
}
Value::ListVal(values) => {
if let DataType::List(_) = data_type {
SingleRowListArrayBuilder::new(literal_to_array_ref(data_type, values.clone())?).build_list_scalar()
} else {
return Err(GeneralError(format!(
"Expected DataType::List but got {data_type:?}"
)))
)));
}
}
}
Expand Down Expand Up @@ -1402,8 +1403,14 @@ impl PhysicalPlanner {
assert_eq!(children.len(), 1);
let (scans, child) = self.create_plan(&children[0], inputs, partition_count)?;

let partitioning = self
.create_partitioning(writer.partitioning.as_ref().unwrap(), child.schema())?;
// We wrap native shuffle in a CopyExec. This existed previously, but for
// RangePartitioning at least we want to ensure that dictionaries are unpacked.
let wrapped_child = Self::wrap_in_copy_exec(Arc::clone(&child.native_plan));

let partitioning = self.create_partitioning(
writer.partitioning.as_ref().unwrap(),
wrapped_child.schema(),
)?;

let codec = match writer.codec.try_into() {
Ok(SparkCompressionCodec::None) => Ok(CompressionCodec::None),
Expand All @@ -1419,7 +1426,7 @@ impl PhysicalPlanner {
}?;

let shuffle_writer = Arc::new(ShuffleWriterExec::try_new(
Self::wrap_in_copy_exec(Arc::clone(&child.native_plan)),
wrapped_child,
partitioning,
codec,
writer.output_data_file.clone(),
Expand Down Expand Up @@ -2344,16 +2351,65 @@ impl PhysicalPlanner {
))
}
PartitioningStruct::RangePartition(range_partition) => {
// Generate the lexical ordering for comparisons
let exprs: Result<Vec<PhysicalSortExpr>, ExecutionError> = range_partition
.sort_orders
.iter()
.map(|expr| self.create_sort_expr(expr, Arc::clone(&input_schema)))
.collect();
let lex_ordering = LexOrdering::new(exprs?).unwrap();

// Generate the row converter for comparing incoming batches to boundary rows
let sort_fields: Vec<SortField> = lex_ordering
.iter()
.map(|sort_expr| {
sort_expr
.expr
.data_type(input_schema.as_ref())
.map(|dt| SortField::new_with_options(dt, sort_expr.options))
})
.collect::<Result<Vec<_>, _>>()?;

// Deserialize the literals to columnar collections of ScalarValues
let mut scalar_values: Vec<Vec<ScalarValue>> = vec![vec![]; lex_ordering.len()];
for boundary_row in &range_partition.boundary_rows {
// For each serialized expr in a boundary row, convert to a Literal
// expression, then extract the ScalarValue from the Literal and push it
// into the collection of ScalarValues
for (col_idx, col_values) in scalar_values
.iter_mut()
.enumerate()
.take(lex_ordering.len())
{
let expr = self.create_expr(
&boundary_row.partition_bounds[col_idx],
Arc::clone(&input_schema),
)?;
let literal_expr =
expr.as_any().downcast_ref::<Literal>().expect("Literal");
col_values.push(literal_expr.value().clone());
}
}

// Convert the collection of ScalarValues to collection of Arrow Arrays
let arrays: Vec<ArrayRef> = scalar_values
.iter()
.map(|scalar_vec| ScalarValue::iter_to_array(scalar_vec.iter().cloned()))
.collect::<Result<Vec<_>, _>>()?;

// Create a RowConverter and use to create OwnedRows from the Arrays
let converter = RowConverter::new(sort_fields)?;
let boundary_rows = converter.convert_columns(&arrays)?;
// Rows are only a view into Arrow Arrays. We need to create OwnedRows with their
// own internal memory ownership to pass as our boundary values to the partitioner.
let boundary_owned_rows: Vec<OwnedRow> =
boundary_rows.iter().map(|row| row.owned()).collect();

Ok(CometPartitioning::RangePartitioning(
lex_ordering,
range_partition.num_partitions as usize,
range_partition.sample_size as usize,
Arc::new(converter),
boundary_owned_rows,
))
}
PartitioningStruct::SinglePartition(_) => Ok(CometPartitioning::SinglePartition),
Expand Down
12 changes: 8 additions & 4 deletions native/core/src/execution/shuffle/comet_partitioning.rs
Original file line number Diff line number Diff line change
Expand Up @@ -15,26 +15,30 @@
// specific language governing permissions and limitations
// under the License.

use arrow::row::{OwnedRow, RowConverter};
use datafusion::physical_expr::{LexOrdering, PhysicalExpr};
use std::sync::Arc;

#[derive(Debug, Clone)]
pub enum CometPartitioning {
SinglePartition,
/// Allocate rows based on a hash of one of more expressions and the specified number of
/// partitions
/// partitions. Args are 1) the expression to hash on, and 2) the number of partitions.
Hash(Vec<Arc<dyn PhysicalExpr>>, usize),
/// Allocate rows based on the lexical order of one of more expressions and the specified number of
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I'm thinking would be that intuitive for the user to have

Arc<RowConverter>, Vec<OwnedRow>

here? 🤔

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Could you expand on this please? I'm not sure I understand the requested change.

Copy link
Copy Markdown
Contributor

@comphead comphead Sep 24, 2025

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Sorry for misleading comment.
I was thinking if to compare with others variants like Hash

 Hash(Vec<Arc<dyn PhysicalExpr>>, usize),

it is quite intuitive that hash depends on numPartitions and expression that supposed to be hashed.

for Range it is

RangePartitioning(LexOrdering, usize, Arc<RowConverter>, Vec<OwnedRow>),

which looks no so intuitive IMO, because cannot say when reading what is the meaning of last 2 params.
Anyway, this design question can be addressed in follow up if needed

/// partitions
RangePartitioning(LexOrdering, usize, usize),
/// partitions. Args are 1) the LexOrdering to use to compare values and split into partitions,
/// 2) the number of partitions, 3) the RowConverter used to view incoming RecordBatches as Arrow
/// Rows for comparing to 4) OwnedRows that represent the boundaries of each partition, used with
/// LexOrdering to bin each value in the RecordBatch to a partition.
RangePartitioning(LexOrdering, usize, Arc<RowConverter>, Vec<OwnedRow>),
}

impl CometPartitioning {
pub fn partition_count(&self) -> usize {
use CometPartitioning::*;
match self {
SinglePartition => 1,
Hash(_, n) | RangePartitioning(_, n, _) => *n,
Hash(_, n) | RangePartitioning(_, n, _, _) => *n,
}
}
}
1 change: 0 additions & 1 deletion native/core/src/execution/shuffle/mod.rs
Original file line number Diff line number Diff line change
Expand Up @@ -19,7 +19,6 @@ pub(crate) mod codec;
mod comet_partitioning;
mod list;
mod map;
mod range_partitioner;
pub mod row;
mod shuffle_writer;

Expand Down
Loading
Loading