use std::any::Any;
use std::sync::Arc;
use super::metrics::{BaselineMetrics, ExecutionPlanMetricsSet, MetricsSet};
use super::stream::{ObservedStream, RecordBatchReceiverStream};
use super::{
DisplayAs, ExecutionPlanProperties, PlanProperties, SendableRecordBatchStream,
Statistics,
};
use crate::{DisplayFormatType, ExecutionPlan, Partitioning};
use datafusion_common::{internal_err, Result};
use datafusion_execution::TaskContext;
#[derive(Debug)]
pub struct CoalescePartitionsExec {
input: Arc<dyn ExecutionPlan>,
metrics: ExecutionPlanMetricsSet,
cache: PlanProperties,
}
impl CoalescePartitionsExec {
pub fn new(input: Arc<dyn ExecutionPlan>) -> Self {
let cache = Self::compute_properties(&input);
CoalescePartitionsExec {
input,
metrics: ExecutionPlanMetricsSet::new(),
cache,
}
}
pub fn input(&self) -> &Arc<dyn ExecutionPlan> {
&self.input
}
fn compute_properties(input: &Arc<dyn ExecutionPlan>) -> PlanProperties {
let mut eq_properties = input.equivalence_properties().clone();
eq_properties.clear_orderings();
eq_properties.clear_per_partition_constants();
PlanProperties::new(
eq_properties, Partitioning::UnknownPartitioning(1), input.execution_mode(), )
}
}
impl DisplayAs for CoalescePartitionsExec {
fn fmt_as(
&self,
t: DisplayFormatType,
f: &mut std::fmt::Formatter,
) -> std::fmt::Result {
match t {
DisplayFormatType::Default | DisplayFormatType::Verbose => {
write!(f, "CoalescePartitionsExec")
}
}
}
}
impl ExecutionPlan for CoalescePartitionsExec {
fn name(&self) -> &'static str {
"CoalescePartitionsExec"
}
fn as_any(&self) -> &dyn Any {
self
}
fn properties(&self) -> &PlanProperties {
&self.cache
}
fn children(&self) -> Vec<&Arc<dyn ExecutionPlan>> {
vec![&self.input]
}
fn benefits_from_input_partitioning(&self) -> Vec<bool> {
vec![false]
}
fn with_new_children(
self: Arc<Self>,
children: Vec<Arc<dyn ExecutionPlan>>,
) -> Result<Arc<dyn ExecutionPlan>> {
Ok(Arc::new(CoalescePartitionsExec::new(Arc::clone(
&children[0],
))))
}
fn execute(
&self,
partition: usize,
context: Arc<TaskContext>,
) -> Result<SendableRecordBatchStream> {
if 0 != partition {
return internal_err!("CoalescePartitionsExec invalid partition {partition}");
}
let input_partitions = self.input.output_partitioning().partition_count();
match input_partitions {
0 => internal_err!(
"CoalescePartitionsExec requires at least one input partition"
),
1 => {
self.input.execute(0, context)
}
_ => {
let baseline_metrics = BaselineMetrics::new(&self.metrics, partition);
let elapsed_compute = baseline_metrics.elapsed_compute().clone();
let _timer = elapsed_compute.timer();
let mut builder =
RecordBatchReceiverStream::builder(self.schema(), input_partitions);
for part_i in 0..input_partitions {
builder.run_input(
Arc::clone(&self.input),
part_i,
Arc::clone(&context),
);
}
let stream = builder.build();
Ok(Box::pin(ObservedStream::new(stream, baseline_metrics)))
}
}
}
fn metrics(&self) -> Option<MetricsSet> {
Some(self.metrics.clone_inner())
}
fn statistics(&self) -> Result<Statistics> {
self.input.statistics()
}
fn supports_limit_pushdown(&self) -> bool {
true
}
}
#[cfg(test)]
mod tests {
use super::*;
use crate::test::exec::{
assert_strong_count_converges_to_zero, BlockingExec, PanicExec,
};
use crate::test::{self, assert_is_pending};
use crate::{collect, common};
use arrow::datatypes::{DataType, Field, Schema};
use futures::FutureExt;
#[tokio::test]
async fn merge() -> Result<()> {
let task_ctx = Arc::new(TaskContext::default());
let num_partitions = 4;
let csv = test::scan_partitioned(num_partitions);
assert_eq!(csv.output_partitioning().partition_count(), num_partitions);
let merge = CoalescePartitionsExec::new(csv);
assert_eq!(
merge.properties().output_partitioning().partition_count(),
1
);
let iter = merge.execute(0, task_ctx)?;
let batches = common::collect(iter).await?;
assert_eq!(batches.len(), num_partitions);
let row_count: usize = batches.iter().map(|batch| batch.num_rows()).sum();
assert_eq!(row_count, 400);
Ok(())
}
#[tokio::test]
async fn test_drop_cancel() -> Result<()> {
let task_ctx = Arc::new(TaskContext::default());
let schema =
Arc::new(Schema::new(vec![Field::new("a", DataType::Float32, true)]));
let blocking_exec = Arc::new(BlockingExec::new(Arc::clone(&schema), 2));
let refs = blocking_exec.refs();
let coaelesce_partitions_exec =
Arc::new(CoalescePartitionsExec::new(blocking_exec));
let fut = collect(coaelesce_partitions_exec, task_ctx);
let mut fut = fut.boxed();
assert_is_pending(&mut fut);
drop(fut);
assert_strong_count_converges_to_zero(refs).await;
Ok(())
}
#[tokio::test]
#[should_panic(expected = "PanickingStream did panic")]
async fn test_panic() {
let task_ctx = Arc::new(TaskContext::default());
let schema =
Arc::new(Schema::new(vec![Field::new("a", DataType::Float32, true)]));
let panicking_exec = Arc::new(PanicExec::new(Arc::clone(&schema), 2));
let coalesce_partitions_exec =
Arc::new(CoalescePartitionsExec::new(panicking_exec));
collect(coalesce_partitions_exec, task_ctx).await.unwrap();
}
}