Skip to main content

datafusion_physical_plan/
memory.rs

1// Licensed to the Apache Software Foundation (ASF) under one
2// or more contributor license agreements.  See the NOTICE file
3// distributed with this work for additional information
4// regarding copyright ownership.  The ASF licenses this file
5// to you under the Apache License, Version 2.0 (the
6// "License"); you may not use this file except in compliance
7// with the License.  You may obtain a copy of the License at
8//
9//   http://www.apache.org/licenses/LICENSE-2.0
10//
11// Unless required by applicable law or agreed to in writing,
12// software distributed under the License is distributed on an
13// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
14// KIND, either express or implied.  See the License for the
15// specific language governing permissions and limitations
16// under the License.
17
18//! Execution plan for reading in-memory batches of data
19
20use std::any::Any;
21use std::fmt;
22use std::sync::Arc;
23use std::task::{Context, Poll};
24
25use crate::coop::cooperative;
26use crate::execution_plan::{Boundedness, EmissionType, SchedulingType};
27use crate::metrics::{BaselineMetrics, ExecutionPlanMetricsSet, MetricsSet};
28use crate::{
29    DisplayAs, DisplayFormatType, ExecutionPlan, Partitioning, PlanProperties,
30    RecordBatchStream, SendableRecordBatchStream,
31};
32
33use arrow::array::RecordBatch;
34use arrow::datatypes::SchemaRef;
35use datafusion_common::{Result, assert_eq_or_internal_err, assert_or_internal_err};
36use datafusion_execution::TaskContext;
37use datafusion_execution::memory_pool::MemoryReservation;
38use datafusion_physical_expr::EquivalenceProperties;
39
40use datafusion_physical_expr_common::sort_expr::PhysicalSortExpr;
41use futures::Stream;
42use parking_lot::RwLock;
43
44/// Iterator over batches
45pub struct MemoryStream {
46    /// Vector of record batches
47    data: Vec<RecordBatch>,
48    /// Optional memory reservation bound to the data, freed on drop
49    reservation: Option<MemoryReservation>,
50    /// Schema representing the data
51    schema: SchemaRef,
52    /// Optional projection for which columns to load
53    projection: Option<Vec<usize>>,
54    /// Index into the data
55    index: usize,
56    /// The remaining number of rows to return. If None, all rows are returned
57    fetch: Option<usize>,
58}
59
60impl MemoryStream {
61    /// Create an iterator for a vector of record batches
62    pub fn try_new(
63        data: Vec<RecordBatch>,
64        schema: SchemaRef,
65        projection: Option<Vec<usize>>,
66    ) -> Result<Self> {
67        Ok(Self {
68            data,
69            reservation: None,
70            schema,
71            projection,
72            index: 0,
73            fetch: None,
74        })
75    }
76
77    /// Set the memory reservation for the data
78    pub fn with_reservation(mut self, reservation: MemoryReservation) -> Self {
79        self.reservation = Some(reservation);
80        self
81    }
82
83    /// Set the number of rows to produce
84    pub fn with_fetch(mut self, fetch: Option<usize>) -> Self {
85        self.fetch = fetch;
86        self
87    }
88}
89
90impl Stream for MemoryStream {
91    type Item = Result<RecordBatch>;
92
93    fn poll_next(
94        mut self: std::pin::Pin<&mut Self>,
95        _: &mut Context<'_>,
96    ) -> Poll<Option<Self::Item>> {
97        if self.index >= self.data.len() {
98            return Poll::Ready(None);
99        }
100        self.index += 1;
101        let batch = &self.data[self.index - 1];
102        // return just the columns requested
103        let batch = match self.projection.as_ref() {
104            Some(columns) => batch.project(columns)?,
105            None => batch.clone(),
106        };
107
108        let Some(&fetch) = self.fetch.as_ref() else {
109            return Poll::Ready(Some(Ok(batch)));
110        };
111        if fetch == 0 {
112            return Poll::Ready(None);
113        }
114
115        let batch = if batch.num_rows() > fetch {
116            batch.slice(0, fetch)
117        } else {
118            batch
119        };
120        self.fetch = Some(fetch - batch.num_rows());
121        Poll::Ready(Some(Ok(batch)))
122    }
123
124    fn size_hint(&self) -> (usize, Option<usize>) {
125        (self.data.len(), Some(self.data.len()))
126    }
127}
128
129impl RecordBatchStream for MemoryStream {
130    /// Get the schema
131    fn schema(&self) -> SchemaRef {
132        Arc::clone(&self.schema)
133    }
134}
135
136pub trait LazyBatchGenerator: Send + Sync + fmt::Debug + fmt::Display {
137    /// Returns the generator as [`Any`] so that it can be
138    /// downcast to a specific implementation.
139    fn as_any(&self) -> &dyn Any;
140
141    fn boundedness(&self) -> Boundedness {
142        Boundedness::Bounded
143    }
144
145    /// Generate the next batch, return `None` when no more batches are available
146    fn generate_next_batch(&mut self) -> Result<Option<RecordBatch>>;
147
148    /// Returns a new instance with the state reset.
149    fn reset_state(&self) -> Arc<RwLock<dyn LazyBatchGenerator>>;
150}
151
152/// Execution plan for lazy in-memory batches of data
153///
154/// This plan generates output batches lazily, it doesn't have to buffer all batches
155/// in memory up front (compared to `MemorySourceConfig`), thus consuming constant memory.
156pub struct LazyMemoryExec {
157    /// Schema representing the data
158    schema: SchemaRef,
159    /// Optional projection for which columns to load
160    projection: Option<Vec<usize>>,
161    /// Functions to generate batches for each partition
162    batch_generators: Vec<Arc<RwLock<dyn LazyBatchGenerator>>>,
163    /// Plan properties cache storing equivalence properties, partitioning, and execution mode
164    cache: Arc<PlanProperties>,
165    /// Execution metrics
166    metrics: ExecutionPlanMetricsSet,
167}
168
169impl LazyMemoryExec {
170    /// Create a new lazy memory execution plan
171    pub fn try_new(
172        schema: SchemaRef,
173        generators: Vec<Arc<RwLock<dyn LazyBatchGenerator>>>,
174    ) -> Result<Self> {
175        let boundedness = generators
176            .iter()
177            .map(|g| g.read().boundedness())
178            .reduce(|acc, b| match acc {
179                Boundedness::Bounded => b,
180                Boundedness::Unbounded {
181                    requires_infinite_memory,
182                } => {
183                    let acc_infinite_memory = requires_infinite_memory;
184                    match b {
185                        Boundedness::Bounded => acc,
186                        Boundedness::Unbounded {
187                            requires_infinite_memory,
188                        } => Boundedness::Unbounded {
189                            requires_infinite_memory: requires_infinite_memory
190                                || acc_infinite_memory,
191                        },
192                    }
193                }
194            })
195            .unwrap_or(Boundedness::Bounded);
196
197        let cache = PlanProperties::new(
198            EquivalenceProperties::new(Arc::clone(&schema)),
199            Partitioning::RoundRobinBatch(generators.len()),
200            EmissionType::Incremental,
201            boundedness,
202        )
203        .with_scheduling_type(SchedulingType::Cooperative)
204        .into();
205
206        Ok(Self {
207            schema,
208            projection: None,
209            batch_generators: generators,
210            cache,
211            metrics: ExecutionPlanMetricsSet::new(),
212        })
213    }
214
215    pub fn with_projection(mut self, projection: Option<Vec<usize>>) -> Self {
216        match projection.as_ref() {
217            Some(columns) => {
218                let projected = Arc::new(self.schema.project(columns).unwrap());
219                Arc::make_mut(&mut self.cache).set_eq_properties(
220                    EquivalenceProperties::new(Arc::clone(&projected)),
221                );
222                self.schema = projected;
223                self.projection = projection;
224                self
225            }
226            _ => self,
227        }
228    }
229
230    pub fn try_set_partitioning(&mut self, partitioning: Partitioning) -> Result<()> {
231        let partition_count = partitioning.partition_count();
232        let generator_count = self.batch_generators.len();
233        assert_eq_or_internal_err!(
234            partition_count,
235            generator_count,
236            "Partition count must match generator count: {} != {}",
237            partition_count,
238            generator_count
239        );
240        Arc::make_mut(&mut self.cache).partitioning = partitioning;
241        Ok(())
242    }
243
244    pub fn add_ordering(&mut self, ordering: impl IntoIterator<Item = PhysicalSortExpr>) {
245        Arc::make_mut(&mut self.cache)
246            .eq_properties
247            .add_orderings(std::iter::once(ordering));
248    }
249
250    /// Get the batch generators
251    pub fn generators(&self) -> &Vec<Arc<RwLock<dyn LazyBatchGenerator>>> {
252        &self.batch_generators
253    }
254}
255
256impl fmt::Debug for LazyMemoryExec {
257    fn fmt(&self, f: &mut fmt::Formatter) -> fmt::Result {
258        f.debug_struct("LazyMemoryExec")
259            .field("schema", &self.schema)
260            .field("batch_generators", &self.batch_generators)
261            .finish()
262    }
263}
264
265impl DisplayAs for LazyMemoryExec {
266    fn fmt_as(&self, t: DisplayFormatType, f: &mut fmt::Formatter) -> fmt::Result {
267        match t {
268            DisplayFormatType::Default | DisplayFormatType::Verbose => {
269                write!(
270                    f,
271                    "LazyMemoryExec: partitions={}, batch_generators=[{}]",
272                    self.batch_generators.len(),
273                    self.batch_generators
274                        .iter()
275                        .map(|g| g.read().to_string())
276                        .collect::<Vec<_>>()
277                        .join(", ")
278                )
279            }
280            DisplayFormatType::TreeRender => {
281                //TODO: remove batch_size, add one line per generator
282                writeln!(
283                    f,
284                    "batch_generators={}",
285                    self.batch_generators
286                        .iter()
287                        .map(|g| g.read().to_string())
288                        .collect::<Vec<String>>()
289                        .join(", ")
290                )?;
291                Ok(())
292            }
293        }
294    }
295}
296
297impl ExecutionPlan for LazyMemoryExec {
298    fn name(&self) -> &'static str {
299        "LazyMemoryExec"
300    }
301
302    fn as_any(&self) -> &dyn Any {
303        self
304    }
305
306    fn schema(&self) -> SchemaRef {
307        Arc::clone(&self.schema)
308    }
309
310    fn properties(&self) -> &Arc<PlanProperties> {
311        &self.cache
312    }
313
314    fn children(&self) -> Vec<&Arc<dyn ExecutionPlan>> {
315        vec![]
316    }
317
318    fn with_new_children(
319        self: Arc<Self>,
320        children: Vec<Arc<dyn ExecutionPlan>>,
321    ) -> Result<Arc<dyn ExecutionPlan>> {
322        assert_or_internal_err!(
323            children.is_empty(),
324            "Children cannot be replaced in LazyMemoryExec"
325        );
326        Ok(self)
327    }
328
329    fn execute(
330        &self,
331        partition: usize,
332        _context: Arc<TaskContext>,
333    ) -> Result<SendableRecordBatchStream> {
334        assert_or_internal_err!(
335            partition < self.batch_generators.len(),
336            "Invalid partition {} for LazyMemoryExec with {} partitions",
337            partition,
338            self.batch_generators.len()
339        );
340
341        let baseline_metrics = BaselineMetrics::new(&self.metrics, partition);
342
343        let stream = LazyMemoryStream {
344            schema: Arc::clone(&self.schema),
345            projection: self.projection.clone(),
346            generator: Arc::clone(&self.batch_generators[partition]),
347            baseline_metrics,
348        };
349        Ok(Box::pin(cooperative(stream)))
350    }
351
352    fn metrics(&self) -> Option<MetricsSet> {
353        Some(self.metrics.clone_inner())
354    }
355
356    fn reset_state(self: Arc<Self>) -> Result<Arc<dyn ExecutionPlan>> {
357        let generators = self
358            .generators()
359            .iter()
360            .map(|g| g.read().reset_state())
361            .collect::<Vec<_>>();
362        Ok(Arc::new(LazyMemoryExec {
363            schema: Arc::clone(&self.schema),
364            batch_generators: generators,
365            cache: Arc::clone(&self.cache),
366            metrics: ExecutionPlanMetricsSet::new(),
367            projection: self.projection.clone(),
368        }))
369    }
370}
371
372/// Stream that generates record batches on demand
373pub struct LazyMemoryStream {
374    schema: SchemaRef,
375    /// Optional projection for which columns to load
376    projection: Option<Vec<usize>>,
377    /// Generator to produce batches
378    ///
379    /// Note: Idiomatically, DataFusion uses plan-time parallelism - each stream
380    /// should have a unique `LazyBatchGenerator`. Use RepartitionExec or
381    /// construct multiple `LazyMemoryStream`s during planning to enable
382    /// parallel execution.
383    /// Sharing generators between streams should be used with caution.
384    generator: Arc<RwLock<dyn LazyBatchGenerator>>,
385    /// Execution metrics
386    baseline_metrics: BaselineMetrics,
387}
388
389impl Stream for LazyMemoryStream {
390    type Item = Result<RecordBatch>;
391
392    fn poll_next(
393        self: std::pin::Pin<&mut Self>,
394        _: &mut Context<'_>,
395    ) -> Poll<Option<Self::Item>> {
396        let _timer_guard = self.baseline_metrics.elapsed_compute().timer();
397        let batch = self.generator.write().generate_next_batch();
398
399        let poll = match batch {
400            Ok(Some(batch)) => {
401                // return just the columns requested
402                let batch = match self.projection.as_ref() {
403                    Some(columns) => batch.project(columns)?,
404                    None => batch,
405                };
406                Poll::Ready(Some(Ok(batch)))
407            }
408            Ok(None) => Poll::Ready(None),
409            Err(e) => Poll::Ready(Some(Err(e))),
410        };
411
412        self.baseline_metrics.record_poll(poll)
413    }
414}
415
416impl RecordBatchStream for LazyMemoryStream {
417    fn schema(&self) -> SchemaRef {
418        Arc::clone(&self.schema)
419    }
420}
421
422#[cfg(test)]
423mod lazy_memory_tests {
424    use super::*;
425    use crate::common::collect;
426    use arrow::array::Int64Array;
427    use arrow::datatypes::{DataType, Field, Schema};
428    use futures::StreamExt;
429
430    #[derive(Debug, Clone)]
431    struct TestGenerator {
432        counter: i64,
433        max_batches: i64,
434        batch_size: usize,
435        schema: SchemaRef,
436    }
437
438    impl fmt::Display for TestGenerator {
439        fn fmt(&self, f: &mut fmt::Formatter) -> fmt::Result {
440            write!(
441                f,
442                "TestGenerator: counter={}, max_batches={}, batch_size={}",
443                self.counter, self.max_batches, self.batch_size
444            )
445        }
446    }
447
448    impl LazyBatchGenerator for TestGenerator {
449        fn as_any(&self) -> &dyn Any {
450            self
451        }
452
453        fn generate_next_batch(&mut self) -> Result<Option<RecordBatch>> {
454            if self.counter >= self.max_batches {
455                return Ok(None);
456            }
457
458            let array = Int64Array::from_iter_values(
459                (self.counter * self.batch_size as i64)
460                    ..(self.counter * self.batch_size as i64 + self.batch_size as i64),
461            );
462            self.counter += 1;
463            Ok(Some(RecordBatch::try_new(
464                Arc::clone(&self.schema),
465                vec![Arc::new(array)],
466            )?))
467        }
468
469        fn reset_state(&self) -> Arc<RwLock<dyn LazyBatchGenerator>> {
470            Arc::new(RwLock::new(TestGenerator {
471                counter: 0,
472                max_batches: self.max_batches,
473                batch_size: self.batch_size,
474                schema: Arc::clone(&self.schema),
475            }))
476        }
477    }
478
479    #[tokio::test]
480    async fn test_lazy_memory_exec() -> Result<()> {
481        let schema = Arc::new(Schema::new(vec![Field::new("a", DataType::Int64, false)]));
482        let generator = TestGenerator {
483            counter: 0,
484            max_batches: 3,
485            batch_size: 2,
486            schema: Arc::clone(&schema),
487        };
488
489        let exec =
490            LazyMemoryExec::try_new(schema, vec![Arc::new(RwLock::new(generator))])?;
491
492        // Test schema
493        assert_eq!(exec.schema().fields().len(), 1);
494        assert_eq!(exec.schema().field(0).name(), "a");
495
496        // Test execution
497        let stream = exec.execute(0, Arc::new(TaskContext::default()))?;
498        let batches: Vec<_> = stream.collect::<Vec<_>>().await;
499
500        assert_eq!(batches.len(), 3);
501
502        // Verify batch contents
503        let batch0 = batches[0].as_ref().unwrap();
504        let array0 = batch0
505            .column(0)
506            .as_any()
507            .downcast_ref::<Int64Array>()
508            .unwrap();
509        assert_eq!(array0.values(), &[0, 1]);
510
511        let batch1 = batches[1].as_ref().unwrap();
512        let array1 = batch1
513            .column(0)
514            .as_any()
515            .downcast_ref::<Int64Array>()
516            .unwrap();
517        assert_eq!(array1.values(), &[2, 3]);
518
519        let batch2 = batches[2].as_ref().unwrap();
520        let array2 = batch2
521            .column(0)
522            .as_any()
523            .downcast_ref::<Int64Array>()
524            .unwrap();
525        assert_eq!(array2.values(), &[4, 5]);
526
527        Ok(())
528    }
529
530    #[tokio::test]
531    async fn test_lazy_memory_exec_invalid_partition() -> Result<()> {
532        let schema = Arc::new(Schema::new(vec![Field::new("a", DataType::Int64, false)]));
533        let generator = TestGenerator {
534            counter: 0,
535            max_batches: 1,
536            batch_size: 1,
537            schema: Arc::clone(&schema),
538        };
539
540        let exec =
541            LazyMemoryExec::try_new(schema, vec![Arc::new(RwLock::new(generator))])?;
542
543        // Test invalid partition
544        let result = exec.execute(1, Arc::new(TaskContext::default()));
545
546        // partition is 0-indexed, so there only should be partition 0
547        assert!(matches!(
548            result,
549            Err(e) if e.to_string().contains("Invalid partition 1 for LazyMemoryExec with 1 partitions")
550        ));
551
552        Ok(())
553    }
554
555    #[tokio::test]
556    async fn test_generate_series_metrics_integration() -> Result<()> {
557        // Test LazyMemoryExec metrics with different configurations
558        let test_cases = vec![
559            (10, 2, 10),    // 10 rows, batch size 2, expected 10 rows
560            (100, 10, 100), // 100 rows, batch size 10, expected 100 rows
561            (5, 1, 5),      // 5 rows, batch size 1, expected 5 rows
562        ];
563
564        for (total_rows, batch_size, expected_rows) in test_cases {
565            let schema =
566                Arc::new(Schema::new(vec![Field::new("a", DataType::Int64, false)]));
567            let generator = TestGenerator {
568                counter: 0,
569                max_batches: (total_rows + batch_size - 1) / batch_size, // ceiling division
570                batch_size: batch_size as usize,
571                schema: Arc::clone(&schema),
572            };
573
574            let exec =
575                LazyMemoryExec::try_new(schema, vec![Arc::new(RwLock::new(generator))])?;
576            let task_ctx = Arc::new(TaskContext::default());
577
578            let stream = exec.execute(0, task_ctx)?;
579            let batches = collect(stream).await?;
580
581            // Verify metrics exist with actual expected numbers
582            let metrics = exec.metrics().unwrap();
583
584            // Count actual rows returned
585            let actual_rows: usize = batches.iter().map(|b| b.num_rows()).sum();
586            assert_eq!(actual_rows, expected_rows);
587
588            // Verify metrics match actual output
589            assert_eq!(metrics.output_rows().unwrap(), expected_rows);
590            assert!(metrics.elapsed_compute().unwrap() > 0);
591        }
592
593        Ok(())
594    }
595
596    #[tokio::test]
597    async fn test_lazy_memory_exec_reset_state() -> Result<()> {
598        let schema = Arc::new(Schema::new(vec![Field::new("a", DataType::Int64, false)]));
599        let generator = TestGenerator {
600            counter: 0,
601            max_batches: 3,
602            batch_size: 2,
603            schema: Arc::clone(&schema),
604        };
605
606        let exec = Arc::new(LazyMemoryExec::try_new(
607            schema,
608            vec![Arc::new(RwLock::new(generator))],
609        )?);
610        let stream = exec.execute(0, Arc::new(TaskContext::default()))?;
611        let batches = collect(stream).await?;
612
613        let exec_reset = exec.reset_state()?;
614        let stream = exec_reset.execute(0, Arc::new(TaskContext::default()))?;
615        let batches_reset = collect(stream).await?;
616
617        // if the reset_state is not correct, the batches_reset will be empty
618        assert_eq!(batches, batches_reset);
619
620        Ok(())
621    }
622}