datafusion-physical-plan 53.1.0

Physical (ExecutionPlan) implementations for DataFusion query engine
Documentation
1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
143
144
145
146
147
148
149
150
151
152
153
154
155
156
157
158
159
160
161
162
163
164
165
166
167
168
169
170
171
172
173
174
175
176
177
178
179
180
181
182
183
184
185
186
187
188
189
190
191
192
193
194
195
196
197
198
199
200
201
202
203
204
205
206
207
208
209
210
211
212
213
214
215
216
217
218
219
220
221
222
223
224
225
226
227
228
229
230
231
232
233
234
235
236
237
238
239
240
241
242
243
244
245
246
247
248
249
250
251
252
253
254
255
256
257
258
259
260
261
262
263
264
265
266
267
268
269
270
271
272
273
274
275
276
277
278
279
280
281
282
283
284
285
286
287
288
289
290
291
292
293
294
295
296
297
298
299
300
301
302
303
304
305
306
307
308
309
310
311
312
313
314
315
316
317
318
319
320
321
322
323
324
325
326
327
328
329
330
331
332
333
334
335
336
337
338
339
340
341
342
343
344
345
346
347
348
349
350
351
352
353
354
355
356
357
358
359
360
361
362
363
364
365
366
367
368
369
370
371
372
373
374
375
376
377
378
379
380
381
382
383
384
385
386
387
388
389
390
391
392
393
394
395
396
397
398
399
400
401
402
403
404
405
406
407
408
409
410
411
412
413
414
415
416
417
418
419
420
421
422
423
424
425
426
427
428
429
430
431
432
433
434
435
436
437
438
439
440
441
442
443
444
445
446
447
448
449
450
451
452
453
454
455
456
457
458
459
460
461
462
463
464
465
466
467
468
469
470
471
472
473
474
475
476
477
478
479
480
481
482
483
484
485
486
487
488
489
490
491
492
493
494
495
496
497
498
499
500
501
502
503
504
505
506
507
508
509
510
511
512
513
514
515
516
517
518
519
520
521
522
523
524
525
526
527
528
529
530
531
// 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.

//! Defines the merge plan for executing partitions in parallel and then merging the results
//! into a single partition

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::execution_plan::{CardinalityEffect, EvaluationType, SchedulingType};
use crate::filter_pushdown::{FilterDescription, FilterPushdownPhase};
use crate::projection::{ProjectionExec, make_with_child};
use crate::sort_pushdown::SortOrderPushdownResult;
use crate::{DisplayFormatType, ExecutionPlan, Partitioning, check_if_same_properties};
use datafusion_physical_expr_common::sort_expr::PhysicalSortExpr;

use datafusion_common::config::ConfigOptions;
use datafusion_common::{Result, assert_eq_or_internal_err, internal_err};
use datafusion_execution::TaskContext;
use datafusion_physical_expr::PhysicalExpr;

/// Merge execution plan executes partitions in parallel and combines them into a single
/// partition. No guarantees are made about the order of the resulting partition.
#[derive(Debug, Clone)]
pub struct CoalescePartitionsExec {
    /// Input execution plan
    input: Arc<dyn ExecutionPlan>,
    /// Execution metrics
    metrics: ExecutionPlanMetricsSet,
    cache: Arc<PlanProperties>,
    /// Optional number of rows to fetch. Stops producing rows after this fetch
    pub(crate) fetch: Option<usize>,
}

impl CoalescePartitionsExec {
    /// Create a new CoalescePartitionsExec
    pub fn new(input: Arc<dyn ExecutionPlan>) -> Self {
        let cache = Self::compute_properties(&input);
        CoalescePartitionsExec {
            input,
            metrics: ExecutionPlanMetricsSet::new(),
            cache: Arc::new(cache),
            fetch: None,
        }
    }

    /// Update fetch with the argument
    pub fn with_fetch(mut self, fetch: Option<usize>) -> Self {
        self.fetch = fetch;
        self
    }

    /// Input execution plan
    pub fn input(&self) -> &Arc<dyn ExecutionPlan> {
        &self.input
    }

    /// This function creates the cache object that stores the plan properties such as schema, equivalence properties, ordering, partitioning, etc.
    fn compute_properties(input: &Arc<dyn ExecutionPlan>) -> PlanProperties {
        let input_partitions = input.output_partitioning().partition_count();
        let (drive, scheduling) = if input_partitions > 1 {
            (EvaluationType::Eager, SchedulingType::Cooperative)
        } else {
            (
                input.properties().evaluation_type,
                input.properties().scheduling_type,
            )
        };

        // Coalescing partitions loses existing orderings:
        let mut eq_properties = input.equivalence_properties().clone();
        eq_properties.clear_orderings();
        eq_properties.clear_per_partition_constants();
        PlanProperties::new(
            eq_properties,                        // Equivalence Properties
            Partitioning::UnknownPartitioning(1), // Output Partitioning
            input.pipeline_behavior(),
            input.boundedness(),
        )
        .with_evaluation_type(drive)
        .with_scheduling_type(scheduling)
    }

    fn with_new_children_and_same_properties(
        &self,
        mut children: Vec<Arc<dyn ExecutionPlan>>,
    ) -> Self {
        Self {
            input: children.swap_remove(0),
            metrics: ExecutionPlanMetricsSet::new(),
            ..Self::clone(self)
        }
    }
}

impl DisplayAs for CoalescePartitionsExec {
    fn fmt_as(
        &self,
        t: DisplayFormatType,
        f: &mut std::fmt::Formatter,
    ) -> std::fmt::Result {
        match t {
            DisplayFormatType::Default | DisplayFormatType::Verbose => match self.fetch {
                Some(fetch) => {
                    write!(f, "CoalescePartitionsExec: fetch={fetch}")
                }
                None => write!(f, "CoalescePartitionsExec"),
            },
            DisplayFormatType::TreeRender => match self.fetch {
                Some(fetch) => {
                    write!(f, "limit: {fetch}")
                }
                None => write!(f, ""),
            },
        }
    }
}

impl ExecutionPlan for CoalescePartitionsExec {
    fn name(&self) -> &'static str {
        "CoalescePartitionsExec"
    }

    /// Return a reference to Any that can be used for downcasting
    fn as_any(&self) -> &dyn Any {
        self
    }

    fn properties(&self) -> &Arc<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>,
        mut children: Vec<Arc<dyn ExecutionPlan>>,
    ) -> Result<Arc<dyn ExecutionPlan>> {
        check_if_same_properties!(self, children);
        let mut plan = CoalescePartitionsExec::new(children.swap_remove(0));
        plan.fetch = self.fetch;
        Ok(Arc::new(plan))
    }

    fn execute(
        &self,
        partition: usize,
        context: Arc<TaskContext>,
    ) -> Result<SendableRecordBatchStream> {
        // CoalescePartitionsExec produces a single partition
        assert_eq_or_internal_err!(
            partition,
            0,
            "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 => {
                // single-partition path: execute child directly, but ensure fetch is respected
                // (wrap with ObservedStream only if fetch is present so we don't add overhead otherwise)
                let child_stream = self.input.execute(0, context)?;
                if self.fetch.is_some() {
                    let baseline_metrics = BaselineMetrics::new(&self.metrics, partition);
                    return Ok(Box::pin(ObservedStream::new(
                        child_stream,
                        baseline_metrics,
                        self.fetch,
                    )));
                }
                Ok(child_stream)
            }
            _ => {
                let baseline_metrics = BaselineMetrics::new(&self.metrics, partition);
                // record the (very) minimal work done so that
                // elapsed_compute is not reported as 0
                let elapsed_compute = baseline_metrics.elapsed_compute().clone();
                let _timer = elapsed_compute.timer();

                // use a stream that allows each sender to put in at
                // least one result in an attempt to maximize
                // parallelism.
                let mut builder =
                    RecordBatchReceiverStream::builder(self.schema(), input_partitions);

                // spawn independent tasks whose resulting streams (of batches)
                // are sent to the channel for consumption.
                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,
                    self.fetch,
                )))
            }
        }
    }

    fn metrics(&self) -> Option<MetricsSet> {
        Some(self.metrics.clone_inner())
    }

    fn partition_statistics(&self, _partition: Option<usize>) -> Result<Statistics> {
        self.input
            .partition_statistics(None)?
            .with_fetch(self.fetch, 0, 1)
    }

    fn supports_limit_pushdown(&self) -> bool {
        true
    }

    fn cardinality_effect(&self) -> CardinalityEffect {
        CardinalityEffect::Equal
    }

    /// Tries to swap `projection` with its input, which is known to be a
    /// [`CoalescePartitionsExec`]. If possible, performs the swap and returns
    /// [`CoalescePartitionsExec`] as the top plan. Otherwise, returns `None`.
    fn try_swapping_with_projection(
        &self,
        projection: &ProjectionExec,
    ) -> Result<Option<Arc<dyn ExecutionPlan>>> {
        // If the projection does not narrow the schema, we should not try to push it down:
        if projection.expr().len() >= projection.input().schema().fields().len() {
            return Ok(None);
        }
        // CoalescePartitionsExec always has a single child, so zero indexing is safe.
        make_with_child(projection, projection.input().children()[0]).map(|e| {
            if self.fetch.is_some() {
                let mut plan = CoalescePartitionsExec::new(e);
                plan.fetch = self.fetch;
                Some(Arc::new(plan) as _)
            } else {
                Some(Arc::new(CoalescePartitionsExec::new(e)) as _)
            }
        })
    }

    fn fetch(&self) -> Option<usize> {
        self.fetch
    }

    fn with_fetch(&self, limit: Option<usize>) -> Option<Arc<dyn ExecutionPlan>> {
        Some(Arc::new(CoalescePartitionsExec {
            input: Arc::clone(&self.input),
            fetch: limit,
            metrics: self.metrics.clone(),
            cache: Arc::clone(&self.cache),
        }))
    }

    fn with_preserve_order(
        &self,
        preserve_order: bool,
    ) -> Option<Arc<dyn ExecutionPlan>> {
        self.input
            .with_preserve_order(preserve_order)
            .and_then(|new_input| {
                Arc::new(self.clone())
                    .with_new_children(vec![new_input])
                    .ok()
            })
    }

    fn gather_filters_for_pushdown(
        &self,
        _phase: FilterPushdownPhase,
        parent_filters: Vec<Arc<dyn PhysicalExpr>>,
        _config: &ConfigOptions,
    ) -> Result<FilterDescription> {
        FilterDescription::from_children(parent_filters, &self.children())
    }

    fn try_pushdown_sort(
        &self,
        order: &[PhysicalSortExpr],
    ) -> Result<SortOrderPushdownResult<Arc<dyn ExecutionPlan>>> {
        // CoalescePartitionsExec merges multiple partitions into one, which loses
        // global ordering. However, we can still push the sort requirement down
        // to optimize individual partitions - the Sort operator above will handle
        // the global ordering.
        //
        // Note: The result will always be at most Inexact (never Exact) when there
        // are multiple partitions, because merging destroys global ordering.
        let result = self.input.try_pushdown_sort(order)?;

        // If we have multiple partitions, we can't return Exact even if the
        // underlying source claims Exact - merging destroys global ordering
        let has_multiple_partitions =
            self.input.output_partitioning().partition_count() > 1;

        result
            .try_map(|new_input| {
                Ok(
                    Arc::new(
                        CoalescePartitionsExec::new(new_input).with_fetch(self.fetch),
                    ) as Arc<dyn ExecutionPlan>,
                )
            })
            .map(|r| {
                if has_multiple_partitions {
                    // Downgrade Exact to Inexact when merging multiple partitions
                    r.into_inexact()
                } else {
                    r
                }
            })
    }
}

#[cfg(test)]
mod tests {
    use super::*;
    use crate::test::exec::{
        BlockingExec, PanicExec, assert_strong_count_converges_to_zero,
    };
    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);

        // input should have 4 partitions
        assert_eq!(csv.output_partitioning().partition_count(), num_partitions);

        let merge = CoalescePartitionsExec::new(csv);

        // output of CoalescePartitionsExec should have a single partition
        assert_eq!(
            merge.properties().output_partitioning().partition_count(),
            1
        );

        // the result should contain 4 batches (one per input partition)
        let iter = merge.execute(0, task_ctx)?;
        let batches = common::collect(iter).await?;
        assert_eq!(batches.len(), num_partitions);

        // there should be a total of 400 rows (100 per each partition)
        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 coalesce_partitions_exec =
            Arc::new(CoalescePartitionsExec::new(blocking_exec));

        let fut = collect(coalesce_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();
    }

    #[tokio::test]
    async fn test_single_partition_with_fetch() -> Result<()> {
        let task_ctx = Arc::new(TaskContext::default());

        // Use existing scan_partitioned with 1 partition (returns 100 rows per partition)
        let input = test::scan_partitioned(1);

        // Test with fetch=3
        let coalesce = CoalescePartitionsExec::new(input).with_fetch(Some(3));

        let stream = coalesce.execute(0, task_ctx)?;
        let batches = common::collect(stream).await?;

        let row_count: usize = batches.iter().map(|batch| batch.num_rows()).sum();
        assert_eq!(row_count, 3, "Should only return 3 rows due to fetch=3");

        Ok(())
    }

    #[tokio::test]
    async fn test_multi_partition_with_fetch_one() -> Result<()> {
        let task_ctx = Arc::new(TaskContext::default());

        // Create 4 partitions, each with 100 rows
        // This simulates the real-world scenario where each partition has data
        let input = test::scan_partitioned(4);

        // Test with fetch=1 (the original bug: was returning multiple rows instead of 1)
        let coalesce = CoalescePartitionsExec::new(input).with_fetch(Some(1));

        let stream = coalesce.execute(0, task_ctx)?;
        let batches = common::collect(stream).await?;

        let row_count: usize = batches.iter().map(|batch| batch.num_rows()).sum();
        assert_eq!(
            row_count, 1,
            "Should only return 1 row due to fetch=1, not one per partition"
        );

        Ok(())
    }

    #[tokio::test]
    async fn test_single_partition_without_fetch() -> Result<()> {
        let task_ctx = Arc::new(TaskContext::default());

        // Use scan_partitioned with 1 partition
        let input = test::scan_partitioned(1);

        // Test without fetch (should return all rows)
        let coalesce = CoalescePartitionsExec::new(input);

        let stream = coalesce.execute(0, task_ctx)?;
        let batches = common::collect(stream).await?;

        let row_count: usize = batches.iter().map(|batch| batch.num_rows()).sum();
        assert_eq!(
            row_count, 100,
            "Should return all 100 rows when fetch is None"
        );

        Ok(())
    }

    #[tokio::test]
    async fn test_single_partition_fetch_larger_than_batch() -> Result<()> {
        let task_ctx = Arc::new(TaskContext::default());

        // Use scan_partitioned with 1 partition (returns 100 rows)
        let input = test::scan_partitioned(1);

        // Test with fetch larger than available rows
        let coalesce = CoalescePartitionsExec::new(input).with_fetch(Some(200));

        let stream = coalesce.execute(0, task_ctx)?;
        let batches = common::collect(stream).await?;

        let row_count: usize = batches.iter().map(|batch| batch.num_rows()).sum();
        assert_eq!(
            row_count, 100,
            "Should return all available rows (100) when fetch (200) is larger"
        );

        Ok(())
    }

    #[tokio::test]
    async fn test_multi_partition_fetch_exact_match() -> Result<()> {
        let task_ctx = Arc::new(TaskContext::default());

        // Create 4 partitions, each with 100 rows
        let num_partitions = 4;
        let csv = test::scan_partitioned(num_partitions);

        // Test with fetch=400 (exactly all rows)
        let coalesce = CoalescePartitionsExec::new(csv).with_fetch(Some(400));

        let stream = coalesce.execute(0, task_ctx)?;
        let batches = common::collect(stream).await?;

        let row_count: usize = batches.iter().map(|batch| batch.num_rows()).sum();
        assert_eq!(row_count, 400, "Should return exactly 400 rows");

        Ok(())
    }
}