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
// 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.

//! Utilities for improved cooperative scheduling.
//!
//! # Cooperative scheduling
//!
//! A single call to `poll_next` on a top-level [`Stream`] may potentially perform a lot of work
//! before it returns a `Poll::Pending`. Think for instance of calculating an aggregation over a
//! large dataset.
//!
//! If a `Stream` runs for a long period of time without yielding back to the Tokio executor,
//! it can starve other tasks waiting on that executor to execute them.
//! Additionally, this prevents the query execution from being cancelled.
//!
//! For more background, please also see the [Using Rust async for Query Execution and Cancelling Long-Running Queries blog]
//!
//! [Using Rust async for Query Execution and Cancelling Long-Running Queries blog]: https://datafusion.apache.org/blog/2025/06/30/cancellation
//!
//! To ensure that `Stream` implementations yield regularly, operators can insert explicit yield
//! points using the utilities in this module. For most operators this is **not** necessary. The
//! `Stream`s of the built-in DataFusion operators that generate (rather than manipulate)
//! `RecordBatch`es such as `DataSourceExec` and those that eagerly consume `RecordBatch`es
//! (for instance, `RepartitionExec`) contain yield points that will make most query `Stream`s yield
//! periodically.
//!
//! There are a couple of types of operators that _should_ insert yield points:
//! - New source operators that do not make use of Tokio resources
//! - Exchange like operators that do not use Tokio's `Channel` implementation to pass data between
//!   tasks
//!
//! ## Adding yield points
//!
//! Yield points can be inserted manually using the facilities provided by the
//! [Tokio coop module](https://docs.rs/tokio/latest/tokio/task/coop/index.html) such as
//! [`tokio::task::coop::consume_budget`](https://docs.rs/tokio/latest/tokio/task/coop/fn.consume_budget.html).
//!
//! Another option is to use the wrapper `Stream` implementation provided by this module which will
//! consume a unit of task budget every time a `RecordBatch` is produced.
//! Wrapper `Stream`s can be created using the [`cooperative`] and [`make_cooperative`] functions.
//!
//! [`cooperative`] is a generic function that takes ownership of the wrapped [`RecordBatchStream`].
//! This function has the benefit of not requiring an additional heap allocation and can avoid
//! dynamic dispatch.
//!
//! [`make_cooperative`] is a non-generic function that wraps a [`SendableRecordBatchStream`]. This
//! can be used to wrap dynamically typed, heap allocated [`RecordBatchStream`]s.
//!
//! ## Automatic cooperation
//!
//! The `EnsureCooperative` physical optimizer rule, which is included in the default set of
//! optimizer rules, inspects query plans for potential cooperative scheduling issues.
//! It injects the [`CooperativeExec`] wrapper `ExecutionPlan` into the query plan where necessary.
//! This `ExecutionPlan` uses [`make_cooperative`] to wrap the `Stream` of its input.
//!
//! The optimizer rule currently checks the plan for exchange-like operators and leave operators
//! that report [`SchedulingType::NonCooperative`] in their [plan properties](ExecutionPlan::properties).

use datafusion_common::config::ConfigOptions;
use datafusion_physical_expr::PhysicalExpr;
#[cfg(datafusion_coop = "tokio_fallback")]
use futures::Future;
use std::any::Any;
use std::pin::Pin;
use std::sync::Arc;
use std::task::{Context, Poll};

use crate::execution_plan::CardinalityEffect::{self, Equal};
use crate::filter_pushdown::{
    ChildPushdownResult, FilterDescription, FilterPushdownPhase,
    FilterPushdownPropagation,
};
use crate::projection::ProjectionExec;
use crate::{
    DisplayAs, DisplayFormatType, ExecutionPlan, PlanProperties, RecordBatchStream,
    SendableRecordBatchStream, SortOrderPushdownResult, check_if_same_properties,
};
use arrow::record_batch::RecordBatch;
use arrow_schema::Schema;
use datafusion_common::{Result, Statistics, assert_eq_or_internal_err};
use datafusion_execution::TaskContext;

use crate::execution_plan::SchedulingType;
use crate::stream::RecordBatchStreamAdapter;
use datafusion_physical_expr_common::sort_expr::PhysicalSortExpr;
use futures::{Stream, StreamExt};

/// A stream that passes record batches through unchanged while cooperating with the Tokio runtime.
/// It consumes cooperative scheduling budget for each returned [`RecordBatch`],
/// allowing other tasks to execute when the budget is exhausted.
///
/// See the [module level documentation](crate::coop) for an in-depth discussion.
pub struct CooperativeStream<T>
where
    T: RecordBatchStream + Unpin,
{
    inner: T,
    #[cfg(datafusion_coop = "per_stream")]
    budget: u8,
}

#[cfg(datafusion_coop = "per_stream")]
// Magic value that matches Tokio's task budget value
const YIELD_FREQUENCY: u8 = 128;

impl<T> CooperativeStream<T>
where
    T: RecordBatchStream + Unpin,
{
    /// Creates a new `CooperativeStream` that wraps the provided stream.
    /// The resulting stream will cooperate with the Tokio scheduler by consuming a unit of
    /// scheduling budget when the wrapped `Stream` returns a record batch.
    pub fn new(inner: T) -> Self {
        Self {
            inner,
            #[cfg(datafusion_coop = "per_stream")]
            budget: YIELD_FREQUENCY,
        }
    }
}

impl<T> Stream for CooperativeStream<T>
where
    T: RecordBatchStream + Unpin,
{
    type Item = Result<RecordBatch>;

    fn poll_next(
        mut self: Pin<&mut Self>,
        cx: &mut Context<'_>,
    ) -> Poll<Option<Self::Item>> {
        #[cfg(any(
            datafusion_coop = "tokio",
            not(any(
                datafusion_coop = "tokio_fallback",
                datafusion_coop = "per_stream"
            ))
        ))]
        {
            let coop = std::task::ready!(tokio::task::coop::poll_proceed(cx));
            let value = self.inner.poll_next_unpin(cx);
            if value.is_ready() {
                coop.made_progress();
            }
            value
        }

        #[cfg(datafusion_coop = "tokio_fallback")]
        {
            // This is a temporary placeholder implementation that may have slightly
            // worse performance compared to `poll_proceed`
            if !tokio::task::coop::has_budget_remaining() {
                cx.waker().wake_by_ref();
                return Poll::Pending;
            }

            let value = self.inner.poll_next_unpin(cx);
            if value.is_ready() {
                // In contrast to `poll_proceed` we are not able to consume
                // budget before proceeding to do work. Instead, we try to consume budget
                // after the work has been done and just assume that that succeeded.
                // The poll result is ignored because we don't want to discard
                // or buffer the Ready result we got from the inner stream.
                let consume = tokio::task::coop::consume_budget();
                let consume_ref = std::pin::pin!(consume);
                let _ = consume_ref.poll(cx);
            }
            value
        }

        #[cfg(datafusion_coop = "per_stream")]
        {
            if self.budget == 0 {
                self.budget = YIELD_FREQUENCY;
                cx.waker().wake_by_ref();
                return Poll::Pending;
            }

            let value = { self.inner.poll_next_unpin(cx) };

            if value.is_ready() {
                self.budget -= 1;
            } else {
                self.budget = YIELD_FREQUENCY;
            }
            value
        }
    }
}

impl<T> RecordBatchStream for CooperativeStream<T>
where
    T: RecordBatchStream + Unpin,
{
    fn schema(&self) -> Arc<Schema> {
        self.inner.schema()
    }
}

/// An execution plan decorator that enables cooperative multitasking.
/// It wraps the streams produced by its input execution plan using the [`make_cooperative`] function,
/// which makes the stream participate in Tokio cooperative scheduling.
#[derive(Debug, Clone)]
pub struct CooperativeExec {
    input: Arc<dyn ExecutionPlan>,
    properties: Arc<PlanProperties>,
}

impl CooperativeExec {
    /// Creates a new `CooperativeExec` operator that wraps the given input execution plan.
    pub fn new(input: Arc<dyn ExecutionPlan>) -> Self {
        let properties = PlanProperties::clone(input.properties())
            .with_scheduling_type(SchedulingType::Cooperative)
            .into();

        Self { input, properties }
    }

    /// Returns a reference to the wrapped input execution plan.
    pub fn input(&self) -> &Arc<dyn ExecutionPlan> {
        &self.input
    }

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

impl DisplayAs for CooperativeExec {
    fn fmt_as(
        &self,
        _t: DisplayFormatType,
        f: &mut std::fmt::Formatter<'_>,
    ) -> std::fmt::Result {
        write!(f, "CooperativeExec")
    }
}

impl ExecutionPlan for CooperativeExec {
    fn name(&self) -> &str {
        "CooperativeExec"
    }

    fn as_any(&self) -> &dyn Any {
        self
    }

    fn schema(&self) -> Arc<Schema> {
        self.input.schema()
    }

    fn properties(&self) -> &Arc<PlanProperties> {
        &self.properties
    }

    fn maintains_input_order(&self) -> Vec<bool> {
        vec![true; self.children().len()]
    }

    fn children(&self) -> Vec<&Arc<dyn ExecutionPlan>> {
        vec![&self.input]
    }

    fn with_new_children(
        self: Arc<Self>,
        mut children: Vec<Arc<dyn ExecutionPlan>>,
    ) -> Result<Arc<dyn ExecutionPlan>> {
        assert_eq_or_internal_err!(
            children.len(),
            1,
            "CooperativeExec requires exactly one child"
        );
        check_if_same_properties!(self, children);
        Ok(Arc::new(CooperativeExec::new(children.swap_remove(0))))
    }

    fn execute(
        &self,
        partition: usize,
        task_ctx: Arc<TaskContext>,
    ) -> Result<SendableRecordBatchStream> {
        let child_stream = self.input.execute(partition, task_ctx)?;
        Ok(make_cooperative(child_stream))
    }

    fn partition_statistics(&self, partition: Option<usize>) -> Result<Statistics> {
        self.input.partition_statistics(partition)
    }

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

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

    fn try_swapping_with_projection(
        &self,
        projection: &ProjectionExec,
    ) -> Result<Option<Arc<dyn ExecutionPlan>>> {
        match self.input.try_swapping_with_projection(projection)? {
            Some(new_input) => Ok(Some(
                Arc::new(self.clone()).with_new_children(vec![new_input])?,
            )),
            None => Ok(None),
        }
    }

    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 handle_child_pushdown_result(
        &self,
        _phase: FilterPushdownPhase,
        child_pushdown_result: ChildPushdownResult,
        _config: &ConfigOptions,
    ) -> Result<FilterPushdownPropagation<Arc<dyn ExecutionPlan>>> {
        Ok(FilterPushdownPropagation::if_all(child_pushdown_result))
    }

    fn try_pushdown_sort(
        &self,
        order: &[PhysicalSortExpr],
    ) -> Result<SortOrderPushdownResult<Arc<dyn ExecutionPlan>>> {
        let child = self.input();

        match child.try_pushdown_sort(order)? {
            SortOrderPushdownResult::Exact { inner } => {
                let new_exec = Arc::new(self.clone()).with_new_children(vec![inner])?;
                Ok(SortOrderPushdownResult::Exact { inner: new_exec })
            }
            SortOrderPushdownResult::Inexact { inner } => {
                let new_exec = Arc::new(self.clone()).with_new_children(vec![inner])?;
                Ok(SortOrderPushdownResult::Inexact { inner: new_exec })
            }
            SortOrderPushdownResult::Unsupported => {
                Ok(SortOrderPushdownResult::Unsupported)
            }
        }
    }
}

/// Creates a [`CooperativeStream`] wrapper around the given [`RecordBatchStream`].
/// This wrapper collaborates with the Tokio cooperative scheduler by consuming a unit of
/// scheduling budget for each returned record batch.
pub fn cooperative<T>(stream: T) -> CooperativeStream<T>
where
    T: RecordBatchStream + Unpin + Send + 'static,
{
    CooperativeStream::new(stream)
}

/// Wraps a `SendableRecordBatchStream` inside a [`CooperativeStream`] to enable cooperative multitasking.
/// Since `SendableRecordBatchStream` is a `dyn RecordBatchStream` this requires the use of dynamic
/// method dispatch.
/// When the stream type is statically known, consider use the generic [`cooperative`] function
/// to allow static method dispatch.
pub fn make_cooperative(stream: SendableRecordBatchStream) -> SendableRecordBatchStream {
    // TODO is there a more elegant way to overload cooperative
    Box::pin(cooperative(RecordBatchStreamAdapter::new(
        stream.schema(),
        stream,
    )))
}

#[cfg(test)]
mod tests {
    use super::*;
    use crate::stream::RecordBatchStreamAdapter;

    use arrow_schema::SchemaRef;

    use futures::{StreamExt, stream};

    // This is the hardcoded value Tokio uses
    const TASK_BUDGET: usize = 128;

    /// Helper: construct a SendableRecordBatchStream containing `n` empty batches
    fn make_empty_batches(n: usize) -> SendableRecordBatchStream {
        let schema: SchemaRef = Arc::new(Schema::empty());
        let schema_for_stream = Arc::clone(&schema);

        let s =
            stream::iter((0..n).map(move |_| {
                Ok(RecordBatch::new_empty(Arc::clone(&schema_for_stream)))
            }));

        Box::pin(RecordBatchStreamAdapter::new(schema, s))
    }

    #[tokio::test]
    async fn yield_less_than_threshold() -> Result<()> {
        let count = TASK_BUDGET - 10;
        let inner = make_empty_batches(count);
        let out = make_cooperative(inner).collect::<Vec<_>>().await;
        assert_eq!(out.len(), count);
        Ok(())
    }

    #[tokio::test]
    async fn yield_equal_to_threshold() -> Result<()> {
        let count = TASK_BUDGET;
        let inner = make_empty_batches(count);
        let out = make_cooperative(inner).collect::<Vec<_>>().await;
        assert_eq!(out.len(), count);
        Ok(())
    }

    #[tokio::test]
    async fn yield_more_than_threshold() -> Result<()> {
        let count = TASK_BUDGET + 20;
        let inner = make_empty_batches(count);
        let out = make_cooperative(inner).collect::<Vec<_>>().await;
        assert_eq!(out.len(), count);
        Ok(())
    }
}