Skip to content
Closed
Show file tree
Hide file tree
Changes from all commits
Commits
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
4 changes: 4 additions & 0 deletions datafusion/physical-plan/Cargo.toml
Original file line number Diff line number Diff line change
Expand Up @@ -91,6 +91,10 @@ tokio = { workspace = true, features = [
harness = false
name = "partial_ordering"

[[bench]]
harness = false
name = "repartition"

[[bench]]
harness = false
name = "spill_io"
Expand Down
165 changes: 165 additions & 0 deletions datafusion/physical-plan/benches/repartition.rs
Original file line number Diff line number Diff line change
@@ -0,0 +1,165 @@
// 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.

//! Microbenchmark for `RepartitionExec` in isolation.
//!
//! Stages a pre-built in-memory set of batches across N input partitions and
//! runs them through `RepartitionExec` to N output partitions. Drains all
//! output partitions concurrently so the measurement reflects the true
//! per-batch cost of the breaker: mpsc `send().await`, reservation lock,
//! metric timers, and `SpawnedTask`-per-input scheduling.
//!
//! Run with:
//! ```sh
//! cargo bench --bench repartition -- --sample-size=20
//! ```

use std::sync::Arc;

use arrow::array::{ArrayRef, UInt64Array};
use arrow::datatypes::{DataType, Field, Schema, SchemaRef};
use arrow::record_batch::RecordBatch;
use criterion::{BatchSize, Criterion, criterion_group, criterion_main};
use datafusion_execution::TaskContext;
use datafusion_physical_expr::Partitioning;
use datafusion_physical_expr::expressions::col;
use datafusion_physical_plan::repartition::RepartitionExec;
use datafusion_physical_plan::test::TestMemoryExec;
use datafusion_physical_plan::{ExecutionPlan, ExecutionPlanProperties};
use futures::StreamExt;

const TOTAL_ROWS: usize = 1_000_000;
const BATCH_SIZE: usize = 8_192;
const PARTITIONS: usize = 16;

fn schema() -> SchemaRef {
Arc::new(Schema::new(vec![Field::new("c0", DataType::UInt64, false)]))
}

/// Build `PARTITIONS` input partitions, each containing
/// `TOTAL_ROWS / PARTITIONS` rows split into `BATCH_SIZE`-sized batches.
fn make_partitions(schema: &SchemaRef) -> Vec<Vec<RecordBatch>> {
let per_partition = TOTAL_ROWS / PARTITIONS;
(0..PARTITIONS)
.map(|p| {
let mut batches = Vec::new();
let mut offset: u64 = (p * per_partition) as u64;
let end: u64 = offset + per_partition as u64;
while offset < end {
let n = std::cmp::min(BATCH_SIZE as u64, end - offset) as usize;
let arr: ArrayRef = Arc::new(UInt64Array::from_iter_values(
(offset..offset + n as u64).collect::<Vec<_>>(),
));
batches
.push(RecordBatch::try_new(Arc::clone(schema), vec![arr]).unwrap());
offset += n as u64;
}
batches
})
.collect()
}

fn build_plan(
partitions: &[Vec<RecordBatch>],
schema: &SchemaRef,
partitioning: Partitioning,
) -> Arc<dyn ExecutionPlan> {
let src = TestMemoryExec::try_new_exec(partitions, Arc::clone(schema), None).unwrap();
Arc::new(RepartitionExec::try_new(src, partitioning).unwrap())
}

fn drain_all(
rt: &tokio::runtime::Runtime,
plan: Arc<dyn ExecutionPlan>,
task_ctx: Arc<TaskContext>,
) {
rt.block_on(async move {
let out = plan.output_partitioning().partition_count();
let mut handles = Vec::with_capacity(out);
for p in 0..out {
let mut stream = plan.execute(p, Arc::clone(&task_ctx)).unwrap();
handles.push(tokio::spawn(async move {
let mut rows = 0usize;
while let Some(batch) = stream.next().await {
rows += batch.unwrap().num_rows();
}
rows
}));
}
for h in handles {
let _ = h.await.unwrap();
}
});
}

fn bench_repartition(c: &mut Criterion) {
let schema = schema();
let partitions = make_partitions(&schema);
let hash_expr = vec![col("c0", &schema).unwrap()];

let rt = tokio::runtime::Builder::new_multi_thread()
.worker_threads(PARTITIONS)
.enable_all()
.build()
.unwrap();
let task_ctx = Arc::new(TaskContext::default());

c.bench_function("repartition/hash_16_to_16", |b| {
b.iter_batched(
|| {
build_plan(
&partitions,
&schema,
Partitioning::Hash(hash_expr.clone(), PARTITIONS),
)
},
|plan| drain_all(&rt, plan, Arc::clone(&task_ctx)),
BatchSize::LargeInput,
)
});

c.bench_function("repartition/round_robin_16_to_16", |b| {
b.iter_batched(
|| {
build_plan(
&partitions,
&schema,
Partitioning::RoundRobinBatch(PARTITIONS),
)
},
|plan| drain_all(&rt, plan, Arc::clone(&task_ctx)),
BatchSize::LargeInput,
)
});

c.bench_function("repartition/hash_16_to_1_coalesce", |b| {
b.iter_batched(
|| {
build_plan(
&partitions,
&schema,
Partitioning::Hash(hash_expr.clone(), 1),
)
},
|plan| drain_all(&rt, plan, Arc::clone(&task_ctx)),
BatchSize::LargeInput,
)
});
}

criterion_group!(benches, bench_repartition);
criterion_main!(benches);
39 changes: 23 additions & 16 deletions datafusion/physical-plan/src/repartition/mod.rs
Original file line number Diff line number Diff line change
Expand Up @@ -25,7 +25,6 @@ use std::sync::Arc;
use std::task::{Context, Poll};
use std::vec;

use super::common::SharedMemoryReservation;
use super::metrics::{self, ExecutionPlanMetricsSet, MetricBuilder, MetricsSet};
use super::{
DisplayAs, ExecutionPlanProperties, RecordBatchStream, SendableRecordBatchStream,
Expand Down Expand Up @@ -58,7 +57,7 @@ use datafusion_common::{
use datafusion_common::{Result, not_impl_err};
use datafusion_common_runtime::SpawnedTask;
use datafusion_execution::TaskContext;
use datafusion_execution::memory_pool::MemoryConsumer;
use datafusion_execution::memory_pool::{MemoryConsumer, MemoryReservation};
use datafusion_physical_expr::{EquivalenceProperties, PhysicalExpr};
use datafusion_physical_expr_common::sort_expr::LexOrdering;

Expand Down Expand Up @@ -143,10 +142,15 @@ type MaybeBatch = Option<Result<RepartitionBatch>>;
type InputPartitionsToCurrentPartitionSender = Vec<DistributionSender<MaybeBatch>>;
type InputPartitionsToCurrentPartitionReceiver = Vec<DistributionReceiver<MaybeBatch>>;

/// Output channel with its associated memory reservation and spill writer
/// Output channel with its associated memory reservation and spill writer.
///
/// Every method on [`MemoryReservation`] takes `&self` and mutates atomic
/// counters internally, so the shared reservation needs no extra lock on the
/// hot path — both the producer (`try_grow`) and the consumer (`shrink`)
/// operate concurrently via atomics.
struct OutputChannel {
sender: DistributionSender<MaybeBatch>,
reservation: SharedMemoryReservation,
reservation: Arc<MemoryReservation>,
spill_writer: SpillPoolWriter,
}

Expand Down Expand Up @@ -177,7 +181,7 @@ struct PartitionChannels {
/// Receivers for each input partition sending data to this output partition
rx: InputPartitionsToCurrentPartitionReceiver,
/// Memory reservation for this output partition
reservation: SharedMemoryReservation,
reservation: Arc<MemoryReservation>,
/// Spill writers for writing spilled data.
/// SpillPoolWriter is Clone, so multiple writers can share state in non-preserve-order mode.
spill_writers: Vec<SpillPoolWriter>,
Expand Down Expand Up @@ -322,11 +326,11 @@ impl RepartitionExecState {

let mut channels = HashMap::with_capacity(txs.len());
for (partition, (tx, rx)) in txs.into_iter().zip(rxs).enumerate() {
let reservation = Arc::new(Mutex::new(
let reservation = Arc::new(
MemoryConsumer::new(format!("{name}[{partition}]"))
.with_can_spill(true)
.register(context.memory_pool()),
));
);

// Create spill channels based on mode:
// - preserve_order: one spill channel per (input, output) pair for proper FIFO ordering
Expand Down Expand Up @@ -1393,15 +1397,18 @@ impl RepartitionExec {
continue;
}

// Track per-partition send time by advancing a single `Instant`
// through the inner loop — one `Instant::now()` per sub-batch,
// instead of two via `ScopedTimerGuard::{timer, done}`.
let mut last = datafusion_common::instant::Instant::now();
for res in partitioner.partition_iter(batch)? {
let (partition, batch) = res?;
let size = batch.get_array_memory_size();

let timer = metrics.send_time[partition].timer();
// if there is still a receiver, send to it
if let Some(channel) = output_channels.get_mut(&partition) {
let (batch_to_send, is_memory_batch) =
match channel.reservation.lock().try_grow(size) {
match channel.reservation.try_grow(size) {
Ok(_) => {
// Memory available - send in-memory batch
(RepartitionBatch::Memory(batch), true)
Expand All @@ -1419,12 +1426,14 @@ impl RepartitionExec {
// If the other end has hung up, it was an early shutdown (e.g. LIMIT)
// Only shrink memory if it was a memory batch
if is_memory_batch {
channel.reservation.lock().shrink(size);
channel.reservation.shrink(size);
}
output_channels.remove(&partition);
}
}
timer.done();
let now = datafusion_common::instant::Instant::now();
metrics.send_time[partition].add_duration(now - last);
last = now;
}

// If the input stream is endless, we may spin forever and
Expand Down Expand Up @@ -1567,7 +1576,7 @@ struct PerPartitionStream {
_drop_helper: Arc<Vec<SpawnedTask<()>>>,

/// Memory reservation.
reservation: SharedMemoryReservation,
reservation: Arc<MemoryReservation>,

/// Infinite stream for reading from the spill pool
spill_stream: SendableRecordBatchStream,
Expand All @@ -1593,7 +1602,7 @@ impl PerPartitionStream {
schema: SchemaRef,
receiver: DistributionReceiver<MaybeBatch>,
drop_helper: Arc<Vec<SpawnedTask<()>>>,
reservation: SharedMemoryReservation,
reservation: Arc<MemoryReservation>,
spill_stream: SendableRecordBatchStream,
num_input_partitions: usize,
baseline_metrics: BaselineMetrics,
Expand Down Expand Up @@ -1638,9 +1647,7 @@ impl PerPartitionStream {
Some(Some(v)) => match v {
Ok(RepartitionBatch::Memory(batch)) => {
// Release memory and return batch
self.reservation
.lock()
.shrink(batch.get_array_memory_size());
self.reservation.shrink(batch.get_array_memory_size());
return Poll::Ready(Some(Ok(batch)));
}
Ok(RepartitionBatch::Spilled) => {
Expand Down
Loading