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
532
533
534
535
536
537
538
539
540
541
542
543
544
545
546
547
548
549
550
551
552
553
554
555
556
557
558
559
560
561
562
563
564
565
566
567
568
569
570
571
572
573
574
575
576
577
578
579
580
581
582
583
584
585
586
587
588
589
590
591
592
593
594
595
596
597
598
599
600
601
602
603
604
605
606
607
608
609
610
611
612
613
614
615
616
617
618
619
620
621
622
// 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.

//! Execution plan for reading in-memory batches of data

use std::any::Any;
use std::fmt;
use std::sync::Arc;
use std::task::{Context, Poll};

use crate::coop::cooperative;
use crate::execution_plan::{Boundedness, EmissionType, SchedulingType};
use crate::metrics::{BaselineMetrics, ExecutionPlanMetricsSet, MetricsSet};
use crate::{
    DisplayAs, DisplayFormatType, ExecutionPlan, Partitioning, PlanProperties,
    RecordBatchStream, SendableRecordBatchStream,
};

use arrow::array::RecordBatch;
use arrow::datatypes::SchemaRef;
use datafusion_common::{Result, assert_eq_or_internal_err, assert_or_internal_err};
use datafusion_execution::TaskContext;
use datafusion_execution::memory_pool::MemoryReservation;
use datafusion_physical_expr::EquivalenceProperties;

use datafusion_physical_expr_common::sort_expr::PhysicalSortExpr;
use futures::Stream;
use parking_lot::RwLock;

/// Iterator over batches
pub struct MemoryStream {
    /// Vector of record batches
    data: Vec<RecordBatch>,
    /// Optional memory reservation bound to the data, freed on drop
    reservation: Option<MemoryReservation>,
    /// Schema representing the data
    schema: SchemaRef,
    /// Optional projection for which columns to load
    projection: Option<Vec<usize>>,
    /// Index into the data
    index: usize,
    /// The remaining number of rows to return. If None, all rows are returned
    fetch: Option<usize>,
}

impl MemoryStream {
    /// Create an iterator for a vector of record batches
    pub fn try_new(
        data: Vec<RecordBatch>,
        schema: SchemaRef,
        projection: Option<Vec<usize>>,
    ) -> Result<Self> {
        Ok(Self {
            data,
            reservation: None,
            schema,
            projection,
            index: 0,
            fetch: None,
        })
    }

    /// Set the memory reservation for the data
    pub fn with_reservation(mut self, reservation: MemoryReservation) -> Self {
        self.reservation = Some(reservation);
        self
    }

    /// Set the number of rows to produce
    pub fn with_fetch(mut self, fetch: Option<usize>) -> Self {
        self.fetch = fetch;
        self
    }
}

impl Stream for MemoryStream {
    type Item = Result<RecordBatch>;

    fn poll_next(
        mut self: std::pin::Pin<&mut Self>,
        _: &mut Context<'_>,
    ) -> Poll<Option<Self::Item>> {
        if self.index >= self.data.len() {
            return Poll::Ready(None);
        }
        self.index += 1;
        let batch = &self.data[self.index - 1];
        // return just the columns requested
        let batch = match self.projection.as_ref() {
            Some(columns) => batch.project(columns)?,
            None => batch.clone(),
        };

        let Some(&fetch) = self.fetch.as_ref() else {
            return Poll::Ready(Some(Ok(batch)));
        };
        if fetch == 0 {
            return Poll::Ready(None);
        }

        let batch = if batch.num_rows() > fetch {
            batch.slice(0, fetch)
        } else {
            batch
        };
        self.fetch = Some(fetch - batch.num_rows());
        Poll::Ready(Some(Ok(batch)))
    }

    fn size_hint(&self) -> (usize, Option<usize>) {
        (self.data.len(), Some(self.data.len()))
    }
}

impl RecordBatchStream for MemoryStream {
    /// Get the schema
    fn schema(&self) -> SchemaRef {
        Arc::clone(&self.schema)
    }
}

pub trait LazyBatchGenerator: Send + Sync + fmt::Debug + fmt::Display {
    /// Returns the generator as [`Any`] so that it can be
    /// downcast to a specific implementation.
    fn as_any(&self) -> &dyn Any;

    fn boundedness(&self) -> Boundedness {
        Boundedness::Bounded
    }

    /// Generate the next batch, return `None` when no more batches are available
    fn generate_next_batch(&mut self) -> Result<Option<RecordBatch>>;

    /// Returns a new instance with the state reset.
    fn reset_state(&self) -> Arc<RwLock<dyn LazyBatchGenerator>>;
}

/// Execution plan for lazy in-memory batches of data
///
/// This plan generates output batches lazily, it doesn't have to buffer all batches
/// in memory up front (compared to `MemorySourceConfig`), thus consuming constant memory.
pub struct LazyMemoryExec {
    /// Schema representing the data
    schema: SchemaRef,
    /// Optional projection for which columns to load
    projection: Option<Vec<usize>>,
    /// Functions to generate batches for each partition
    batch_generators: Vec<Arc<RwLock<dyn LazyBatchGenerator>>>,
    /// Plan properties cache storing equivalence properties, partitioning, and execution mode
    cache: Arc<PlanProperties>,
    /// Execution metrics
    metrics: ExecutionPlanMetricsSet,
}

impl LazyMemoryExec {
    /// Create a new lazy memory execution plan
    pub fn try_new(
        schema: SchemaRef,
        generators: Vec<Arc<RwLock<dyn LazyBatchGenerator>>>,
    ) -> Result<Self> {
        let boundedness = generators
            .iter()
            .map(|g| g.read().boundedness())
            .reduce(|acc, b| match acc {
                Boundedness::Bounded => b,
                Boundedness::Unbounded {
                    requires_infinite_memory,
                } => {
                    let acc_infinite_memory = requires_infinite_memory;
                    match b {
                        Boundedness::Bounded => acc,
                        Boundedness::Unbounded {
                            requires_infinite_memory,
                        } => Boundedness::Unbounded {
                            requires_infinite_memory: requires_infinite_memory
                                || acc_infinite_memory,
                        },
                    }
                }
            })
            .unwrap_or(Boundedness::Bounded);

        let cache = PlanProperties::new(
            EquivalenceProperties::new(Arc::clone(&schema)),
            Partitioning::RoundRobinBatch(generators.len()),
            EmissionType::Incremental,
            boundedness,
        )
        .with_scheduling_type(SchedulingType::Cooperative)
        .into();

        Ok(Self {
            schema,
            projection: None,
            batch_generators: generators,
            cache,
            metrics: ExecutionPlanMetricsSet::new(),
        })
    }

    pub fn with_projection(mut self, projection: Option<Vec<usize>>) -> Self {
        match projection.as_ref() {
            Some(columns) => {
                let projected = Arc::new(self.schema.project(columns).unwrap());
                Arc::make_mut(&mut self.cache).set_eq_properties(
                    EquivalenceProperties::new(Arc::clone(&projected)),
                );
                self.schema = projected;
                self.projection = projection;
                self
            }
            _ => self,
        }
    }

    pub fn try_set_partitioning(&mut self, partitioning: Partitioning) -> Result<()> {
        let partition_count = partitioning.partition_count();
        let generator_count = self.batch_generators.len();
        assert_eq_or_internal_err!(
            partition_count,
            generator_count,
            "Partition count must match generator count: {} != {}",
            partition_count,
            generator_count
        );
        Arc::make_mut(&mut self.cache).partitioning = partitioning;
        Ok(())
    }

    pub fn add_ordering(&mut self, ordering: impl IntoIterator<Item = PhysicalSortExpr>) {
        Arc::make_mut(&mut self.cache)
            .eq_properties
            .add_orderings(std::iter::once(ordering));
    }

    /// Get the batch generators
    pub fn generators(&self) -> &Vec<Arc<RwLock<dyn LazyBatchGenerator>>> {
        &self.batch_generators
    }
}

impl fmt::Debug for LazyMemoryExec {
    fn fmt(&self, f: &mut fmt::Formatter) -> fmt::Result {
        f.debug_struct("LazyMemoryExec")
            .field("schema", &self.schema)
            .field("batch_generators", &self.batch_generators)
            .finish()
    }
}

impl DisplayAs for LazyMemoryExec {
    fn fmt_as(&self, t: DisplayFormatType, f: &mut fmt::Formatter) -> fmt::Result {
        match t {
            DisplayFormatType::Default | DisplayFormatType::Verbose => {
                write!(
                    f,
                    "LazyMemoryExec: partitions={}, batch_generators=[{}]",
                    self.batch_generators.len(),
                    self.batch_generators
                        .iter()
                        .map(|g| g.read().to_string())
                        .collect::<Vec<_>>()
                        .join(", ")
                )
            }
            DisplayFormatType::TreeRender => {
                //TODO: remove batch_size, add one line per generator
                writeln!(
                    f,
                    "batch_generators={}",
                    self.batch_generators
                        .iter()
                        .map(|g| g.read().to_string())
                        .collect::<Vec<String>>()
                        .join(", ")
                )?;
                Ok(())
            }
        }
    }
}

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

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

    fn schema(&self) -> SchemaRef {
        Arc::clone(&self.schema)
    }

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

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

    fn with_new_children(
        self: Arc<Self>,
        children: Vec<Arc<dyn ExecutionPlan>>,
    ) -> Result<Arc<dyn ExecutionPlan>> {
        assert_or_internal_err!(
            children.is_empty(),
            "Children cannot be replaced in LazyMemoryExec"
        );
        Ok(self)
    }

    fn execute(
        &self,
        partition: usize,
        _context: Arc<TaskContext>,
    ) -> Result<SendableRecordBatchStream> {
        assert_or_internal_err!(
            partition < self.batch_generators.len(),
            "Invalid partition {} for LazyMemoryExec with {} partitions",
            partition,
            self.batch_generators.len()
        );

        let baseline_metrics = BaselineMetrics::new(&self.metrics, partition);

        let stream = LazyMemoryStream {
            schema: Arc::clone(&self.schema),
            projection: self.projection.clone(),
            generator: Arc::clone(&self.batch_generators[partition]),
            baseline_metrics,
        };
        Ok(Box::pin(cooperative(stream)))
    }

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

    fn reset_state(self: Arc<Self>) -> Result<Arc<dyn ExecutionPlan>> {
        let generators = self
            .generators()
            .iter()
            .map(|g| g.read().reset_state())
            .collect::<Vec<_>>();
        Ok(Arc::new(LazyMemoryExec {
            schema: Arc::clone(&self.schema),
            batch_generators: generators,
            cache: Arc::clone(&self.cache),
            metrics: ExecutionPlanMetricsSet::new(),
            projection: self.projection.clone(),
        }))
    }
}

/// Stream that generates record batches on demand
pub struct LazyMemoryStream {
    schema: SchemaRef,
    /// Optional projection for which columns to load
    projection: Option<Vec<usize>>,
    /// Generator to produce batches
    ///
    /// Note: Idiomatically, DataFusion uses plan-time parallelism - each stream
    /// should have a unique `LazyBatchGenerator`. Use RepartitionExec or
    /// construct multiple `LazyMemoryStream`s during planning to enable
    /// parallel execution.
    /// Sharing generators between streams should be used with caution.
    generator: Arc<RwLock<dyn LazyBatchGenerator>>,
    /// Execution metrics
    baseline_metrics: BaselineMetrics,
}

impl Stream for LazyMemoryStream {
    type Item = Result<RecordBatch>;

    fn poll_next(
        self: std::pin::Pin<&mut Self>,
        _: &mut Context<'_>,
    ) -> Poll<Option<Self::Item>> {
        let _timer_guard = self.baseline_metrics.elapsed_compute().timer();
        let batch = self.generator.write().generate_next_batch();

        let poll = match batch {
            Ok(Some(batch)) => {
                // return just the columns requested
                let batch = match self.projection.as_ref() {
                    Some(columns) => batch.project(columns)?,
                    None => batch,
                };
                Poll::Ready(Some(Ok(batch)))
            }
            Ok(None) => Poll::Ready(None),
            Err(e) => Poll::Ready(Some(Err(e))),
        };

        self.baseline_metrics.record_poll(poll)
    }
}

impl RecordBatchStream for LazyMemoryStream {
    fn schema(&self) -> SchemaRef {
        Arc::clone(&self.schema)
    }
}

#[cfg(test)]
mod lazy_memory_tests {
    use super::*;
    use crate::common::collect;
    use arrow::array::Int64Array;
    use arrow::datatypes::{DataType, Field, Schema};
    use futures::StreamExt;

    #[derive(Debug, Clone)]
    struct TestGenerator {
        counter: i64,
        max_batches: i64,
        batch_size: usize,
        schema: SchemaRef,
    }

    impl fmt::Display for TestGenerator {
        fn fmt(&self, f: &mut fmt::Formatter) -> fmt::Result {
            write!(
                f,
                "TestGenerator: counter={}, max_batches={}, batch_size={}",
                self.counter, self.max_batches, self.batch_size
            )
        }
    }

    impl LazyBatchGenerator for TestGenerator {
        fn as_any(&self) -> &dyn Any {
            self
        }

        fn generate_next_batch(&mut self) -> Result<Option<RecordBatch>> {
            if self.counter >= self.max_batches {
                return Ok(None);
            }

            let array = Int64Array::from_iter_values(
                (self.counter * self.batch_size as i64)
                    ..(self.counter * self.batch_size as i64 + self.batch_size as i64),
            );
            self.counter += 1;
            Ok(Some(RecordBatch::try_new(
                Arc::clone(&self.schema),
                vec![Arc::new(array)],
            )?))
        }

        fn reset_state(&self) -> Arc<RwLock<dyn LazyBatchGenerator>> {
            Arc::new(RwLock::new(TestGenerator {
                counter: 0,
                max_batches: self.max_batches,
                batch_size: self.batch_size,
                schema: Arc::clone(&self.schema),
            }))
        }
    }

    #[tokio::test]
    async fn test_lazy_memory_exec() -> Result<()> {
        let schema = Arc::new(Schema::new(vec![Field::new("a", DataType::Int64, false)]));
        let generator = TestGenerator {
            counter: 0,
            max_batches: 3,
            batch_size: 2,
            schema: Arc::clone(&schema),
        };

        let exec =
            LazyMemoryExec::try_new(schema, vec![Arc::new(RwLock::new(generator))])?;

        // Test schema
        assert_eq!(exec.schema().fields().len(), 1);
        assert_eq!(exec.schema().field(0).name(), "a");

        // Test execution
        let stream = exec.execute(0, Arc::new(TaskContext::default()))?;
        let batches: Vec<_> = stream.collect::<Vec<_>>().await;

        assert_eq!(batches.len(), 3);

        // Verify batch contents
        let batch0 = batches[0].as_ref().unwrap();
        let array0 = batch0
            .column(0)
            .as_any()
            .downcast_ref::<Int64Array>()
            .unwrap();
        assert_eq!(array0.values(), &[0, 1]);

        let batch1 = batches[1].as_ref().unwrap();
        let array1 = batch1
            .column(0)
            .as_any()
            .downcast_ref::<Int64Array>()
            .unwrap();
        assert_eq!(array1.values(), &[2, 3]);

        let batch2 = batches[2].as_ref().unwrap();
        let array2 = batch2
            .column(0)
            .as_any()
            .downcast_ref::<Int64Array>()
            .unwrap();
        assert_eq!(array2.values(), &[4, 5]);

        Ok(())
    }

    #[tokio::test]
    async fn test_lazy_memory_exec_invalid_partition() -> Result<()> {
        let schema = Arc::new(Schema::new(vec![Field::new("a", DataType::Int64, false)]));
        let generator = TestGenerator {
            counter: 0,
            max_batches: 1,
            batch_size: 1,
            schema: Arc::clone(&schema),
        };

        let exec =
            LazyMemoryExec::try_new(schema, vec![Arc::new(RwLock::new(generator))])?;

        // Test invalid partition
        let result = exec.execute(1, Arc::new(TaskContext::default()));

        // partition is 0-indexed, so there only should be partition 0
        assert!(matches!(
            result,
            Err(e) if e.to_string().contains("Invalid partition 1 for LazyMemoryExec with 1 partitions")
        ));

        Ok(())
    }

    #[tokio::test]
    async fn test_generate_series_metrics_integration() -> Result<()> {
        // Test LazyMemoryExec metrics with different configurations
        let test_cases = vec![
            (10, 2, 10),    // 10 rows, batch size 2, expected 10 rows
            (100, 10, 100), // 100 rows, batch size 10, expected 100 rows
            (5, 1, 5),      // 5 rows, batch size 1, expected 5 rows
        ];

        for (total_rows, batch_size, expected_rows) in test_cases {
            let schema =
                Arc::new(Schema::new(vec![Field::new("a", DataType::Int64, false)]));
            let generator = TestGenerator {
                counter: 0,
                max_batches: (total_rows + batch_size - 1) / batch_size, // ceiling division
                batch_size: batch_size as usize,
                schema: Arc::clone(&schema),
            };

            let exec =
                LazyMemoryExec::try_new(schema, vec![Arc::new(RwLock::new(generator))])?;
            let task_ctx = Arc::new(TaskContext::default());

            let stream = exec.execute(0, task_ctx)?;
            let batches = collect(stream).await?;

            // Verify metrics exist with actual expected numbers
            let metrics = exec.metrics().unwrap();

            // Count actual rows returned
            let actual_rows: usize = batches.iter().map(|b| b.num_rows()).sum();
            assert_eq!(actual_rows, expected_rows);

            // Verify metrics match actual output
            assert_eq!(metrics.output_rows().unwrap(), expected_rows);
            assert!(metrics.elapsed_compute().unwrap() > 0);
        }

        Ok(())
    }

    #[tokio::test]
    async fn test_lazy_memory_exec_reset_state() -> Result<()> {
        let schema = Arc::new(Schema::new(vec![Field::new("a", DataType::Int64, false)]));
        let generator = TestGenerator {
            counter: 0,
            max_batches: 3,
            batch_size: 2,
            schema: Arc::clone(&schema),
        };

        let exec = Arc::new(LazyMemoryExec::try_new(
            schema,
            vec![Arc::new(RwLock::new(generator))],
        )?);
        let stream = exec.execute(0, Arc::new(TaskContext::default()))?;
        let batches = collect(stream).await?;

        let exec_reset = exec.reset_state()?;
        let stream = exec_reset.execute(0, Arc::new(TaskContext::default()))?;
        let batches_reset = collect(stream).await?;

        // if the reset_state is not correct, the batches_reset will be empty
        assert_eq!(batches, batches_reset);

        Ok(())
    }
}