Coverage Report

Created: 2024-10-13 08:39

/Users/andrewlamb/Software/datafusion/datafusion/physical-plan/src/coalesce_batches.rs
Line
Count
Source (jump to first uncovered line)
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
//! [`CoalesceBatchesExec`] combines small batches into larger batches.
19
20
use std::any::Any;
21
use std::pin::Pin;
22
use std::sync::Arc;
23
use std::task::{Context, Poll};
24
25
use super::metrics::{BaselineMetrics, ExecutionPlanMetricsSet, MetricsSet};
26
use super::{DisplayAs, ExecutionPlanProperties, PlanProperties, Statistics};
27
use crate::{
28
    DisplayFormatType, ExecutionPlan, RecordBatchStream, SendableRecordBatchStream,
29
};
30
31
use arrow::datatypes::SchemaRef;
32
use arrow::record_batch::RecordBatch;
33
use datafusion_common::Result;
34
use datafusion_execution::TaskContext;
35
36
use crate::coalesce::{BatchCoalescer, CoalescerState};
37
use futures::ready;
38
use futures::stream::{Stream, StreamExt};
39
40
/// `CoalesceBatchesExec` combines small batches into larger batches for more
41
/// efficient vectorized processing by later operators.
42
///
43
/// The operator buffers batches until it collects `target_batch_size` rows and
44
/// then emits a single concatenated batch. When only a limited number of rows
45
/// are necessary (specified by the `fetch` parameter), the operator will stop
46
/// buffering and returns the final batch once the number of collected rows
47
/// reaches the `fetch` value.
48
///
49
/// See [`BatchCoalescer`] for more information
50
#[derive(Debug)]
51
pub struct CoalesceBatchesExec {
52
    /// The input plan
53
    input: Arc<dyn ExecutionPlan>,
54
    /// Minimum number of rows for coalesces batches
55
    target_batch_size: usize,
56
    /// Maximum number of rows to fetch, `None` means fetching all rows
57
    fetch: Option<usize>,
58
    /// Execution metrics
59
    metrics: ExecutionPlanMetricsSet,
60
    cache: PlanProperties,
61
}
62
63
impl CoalesceBatchesExec {
64
    /// Create a new CoalesceBatchesExec
65
4
    pub fn new(input: Arc<dyn ExecutionPlan>, target_batch_size: usize) -> Self {
66
4
        let cache = Self::compute_properties(&input);
67
4
        Self {
68
4
            input,
69
4
            target_batch_size,
70
4
            fetch: None,
71
4
            metrics: ExecutionPlanMetricsSet::new(),
72
4
            cache,
73
4
        }
74
4
    }
75
76
    /// Update fetch with the argument
77
0
    pub fn with_fetch(mut self, fetch: Option<usize>) -> Self {
78
0
        self.fetch = fetch;
79
0
        self
80
0
    }
81
82
    /// The input plan
83
0
    pub fn input(&self) -> &Arc<dyn ExecutionPlan> {
84
0
        &self.input
85
0
    }
86
87
    /// Minimum number of rows for coalesces batches
88
0
    pub fn target_batch_size(&self) -> usize {
89
0
        self.target_batch_size
90
0
    }
91
92
    /// This function creates the cache object that stores the plan properties such as schema, equivalence properties, ordering, partitioning, etc.
93
4
    fn compute_properties(input: &Arc<dyn ExecutionPlan>) -> PlanProperties {
94
4
        // The coalesce batches operator does not make any changes to the
95
4
        // partitioning of its input.
96
4
        PlanProperties::new(
97
4
            input.equivalence_properties().clone(), // Equivalence Properties
98
4
            input.output_partitioning().clone(),    // Output Partitioning
99
4
            input.execution_mode(),                 // Execution Mode
100
4
        )
101
4
    }
102
}
103
104
impl DisplayAs for CoalesceBatchesExec {
105
0
    fn fmt_as(
106
0
        &self,
107
0
        t: DisplayFormatType,
108
0
        f: &mut std::fmt::Formatter,
109
0
    ) -> std::fmt::Result {
110
0
        match t {
111
            DisplayFormatType::Default | DisplayFormatType::Verbose => {
112
0
                write!(
113
0
                    f,
114
0
                    "CoalesceBatchesExec: target_batch_size={}",
115
0
                    self.target_batch_size,
116
0
                )?;
117
0
                if let Some(fetch) = self.fetch {
118
0
                    write!(f, ", fetch={fetch}")?;
119
0
                };
120
121
0
                Ok(())
122
            }
123
        }
124
0
    }
125
}
126
127
impl ExecutionPlan for CoalesceBatchesExec {
128
0
    fn name(&self) -> &'static str {
129
0
        "CoalesceBatchesExec"
130
0
    }
131
132
    /// Return a reference to Any that can be used for downcasting
133
0
    fn as_any(&self) -> &dyn Any {
134
0
        self
135
0
    }
136
137
28
    fn properties(&self) -> &PlanProperties {
138
28
        &self.cache
139
28
    }
140
141
0
    fn children(&self) -> Vec<&Arc<dyn ExecutionPlan>> {
142
0
        vec![&self.input]
143
0
    }
144
145
0
    fn maintains_input_order(&self) -> Vec<bool> {
146
0
        vec![true]
147
0
    }
148
149
0
    fn benefits_from_input_partitioning(&self) -> Vec<bool> {
150
0
        vec![false]
151
0
    }
152
153
0
    fn with_new_children(
154
0
        self: Arc<Self>,
155
0
        children: Vec<Arc<dyn ExecutionPlan>>,
156
0
    ) -> Result<Arc<dyn ExecutionPlan>> {
157
0
        Ok(Arc::new(
158
0
            CoalesceBatchesExec::new(Arc::clone(&children[0]), self.target_batch_size)
159
0
                .with_fetch(self.fetch),
160
0
        ))
161
0
    }
162
163
4
    fn execute(
164
4
        &self,
165
4
        partition: usize,
166
4
        context: Arc<TaskContext>,
167
4
    ) -> Result<SendableRecordBatchStream> {
168
4
        Ok(Box::pin(CoalesceBatchesStream {
169
4
            input: self.input.execute(partition, context)
?0
,
170
4
            coalescer: BatchCoalescer::new(
171
4
                self.input.schema(),
172
4
                self.target_batch_size,
173
4
                self.fetch,
174
4
            ),
175
4
            baseline_metrics: BaselineMetrics::new(&self.metrics, partition),
176
4
            // Start by pulling data
177
4
            inner_state: CoalesceBatchesStreamState::Pull,
178
        }))
179
4
    }
180
181
0
    fn metrics(&self) -> Option<MetricsSet> {
182
0
        Some(self.metrics.clone_inner())
183
0
    }
184
185
0
    fn statistics(&self) -> Result<Statistics> {
186
0
        Statistics::with_fetch(self.input.statistics()?, self.schema(), self.fetch, 0, 1)
187
0
    }
188
189
0
    fn with_fetch(&self, limit: Option<usize>) -> Option<Arc<dyn ExecutionPlan>> {
190
0
        Some(Arc::new(CoalesceBatchesExec {
191
0
            input: Arc::clone(&self.input),
192
0
            target_batch_size: self.target_batch_size,
193
0
            fetch: limit,
194
0
            metrics: self.metrics.clone(),
195
0
            cache: self.cache.clone(),
196
0
        }))
197
0
    }
198
199
0
    fn fetch(&self) -> Option<usize> {
200
0
        self.fetch
201
0
    }
202
}
203
204
/// Stream for [`CoalesceBatchesExec`]. See [`CoalesceBatchesExec`] for more details.
205
struct CoalesceBatchesStream {
206
    /// The input plan
207
    input: SendableRecordBatchStream,
208
    /// Buffer for combining batches
209
    coalescer: BatchCoalescer,
210
    /// Execution metrics
211
    baseline_metrics: BaselineMetrics,
212
    /// The current inner state of the stream. This state dictates the current
213
    /// action or operation to be performed in the streaming process.
214
    inner_state: CoalesceBatchesStreamState,
215
}
216
217
impl Stream for CoalesceBatchesStream {
218
    type Item = Result<RecordBatch>;
219
220
14
    fn poll_next(
221
14
        mut self: Pin<&mut Self>,
222
14
        cx: &mut Context<'_>,
223
14
    ) -> Poll<Option<Self::Item>> {
224
14
        let poll = self.poll_next_inner(cx);
225
14
        self.baseline_metrics.record_poll(poll)
226
14
    }
227
228
0
    fn size_hint(&self) -> (usize, Option<usize>) {
229
0
        // we can't predict the size of incoming batches so re-use the size hint from the input
230
0
        self.input.size_hint()
231
0
    }
232
}
233
234
/// Enumeration of possible states for `CoalesceBatchesStream`.
235
/// It represents different stages in the lifecycle of a stream of record batches.
236
///
237
/// An example of state transition:
238
/// Notation:
239
/// `[3000]`: A batch with size 3000
240
/// `{[2000], [3000]}`: `CoalesceBatchStream`'s internal buffer with 2 batches buffered
241
/// Input of `CoalesceBatchStream` will generate three batches `[2000], [3000], [4000]`
242
/// The coalescing procedure will go through the following steps with 4096 coalescing threshold:
243
/// 1. Read the first batch and get it buffered.
244
/// - initial state: `Pull`
245
/// - initial buffer: `{}`
246
/// - updated buffer: `{[2000]}`
247
/// - next state: `Pull`
248
/// 2. Read the second batch, the coalescing target is reached since 2000 + 3000 > 4096
249
/// - initial state: `Pull`
250
/// - initial buffer: `{[2000]}`
251
/// - updated buffer: `{[2000], [3000]}`
252
/// - next state: `ReturnBuffer`
253
/// 4. Two batches in the batch get merged and consumed by the upstream operator.
254
/// - initial state: `ReturnBuffer`
255
/// - initial buffer: `{[2000], [3000]}`
256
/// - updated buffer: `{}`
257
/// - next state: `Pull`
258
/// 5. Read the third input batch.
259
/// - initial state: `Pull`
260
/// - initial buffer: `{}`
261
/// - updated buffer: `{[4000]}`
262
/// - next state: `Pull`
263
/// 5. The input is ended now. Jump to exhaustion state preparing the finalized data.
264
/// - initial state: `Pull`
265
/// - initial buffer: `{[4000]}`
266
/// - updated buffer: `{[4000]}`
267
/// - next state: `Exhausted`
268
#[derive(Debug, Clone, Eq, PartialEq)]
269
enum CoalesceBatchesStreamState {
270
    /// State to pull a new batch from the input stream.
271
    Pull,
272
    /// State to return a buffered batch.
273
    ReturnBuffer,
274
    /// State indicating that the stream is exhausted.
275
    Exhausted,
276
}
277
278
impl CoalesceBatchesStream {
279
14
    fn poll_next_inner(
280
14
        self: &mut Pin<&mut Self>,
281
14
        cx: &mut Context<'_>,
282
14
    ) -> Poll<Option<Result<RecordBatch>>> {
283
14
        let cloned_time = self.baseline_metrics.elapsed_compute().clone();
284
        loop {
285
42
            match &self.inner_state {
286
                CoalesceBatchesStreamState::Pull => {
287
                    // Attempt to pull the next batch from the input stream.
288
34
                    let 
input_batch28
=
ready!6
(self.input.poll_next_unpin(cx));
289
                    // Start timing the operation. The timer records time upon being dropped.
290
28
                    let _timer = cloned_time.timer();
291
292
24
                    match input_batch {
293
24
                        Some(Ok(batch)) => match self.coalescer.push_batch(batch) {
294
24
                            CoalescerState::Continue => {}
295
0
                            CoalescerState::LimitReached => {
296
0
                                self.inner_state = CoalesceBatchesStreamState::Exhausted;
297
0
                            }
298
0
                            CoalescerState::TargetReached => {
299
0
                                self.inner_state =
300
0
                                    CoalesceBatchesStreamState::ReturnBuffer;
301
0
                            }
302
                        },
303
4
                        None => {
304
4
                            // End of input stream, but buffered batches might still be present.
305
4
                            self.inner_state = CoalesceBatchesStreamState::Exhausted;
306
4
                        }
307
0
                        other => return Poll::Ready(other),
308
                    }
309
                }
310
                CoalesceBatchesStreamState::ReturnBuffer => {
311
                    // Combine buffered batches into one batch and return it.
312
0
                    let batch = self.coalescer.finish_batch()?;
313
                    // Set to pull state for the next iteration.
314
0
                    self.inner_state = CoalesceBatchesStreamState::Pull;
315
0
                    return Poll::Ready(Some(Ok(batch)));
316
                }
317
                CoalesceBatchesStreamState::Exhausted => {
318
                    // Handle the end of the input stream.
319
8
                    return if self.coalescer.is_empty() {
320
                        // If buffer is empty, return None indicating the stream is fully consumed.
321
4
                        Poll::Ready(None)
322
                    } else {
323
                        // If the buffer still contains batches, prepare to return them.
324
4
                        let batch = self.coalescer.finish_batch()
?0
;
325
4
                        Poll::Ready(Some(Ok(batch)))
326
                    };
327
                }
328
            }
329
        }
330
14
    }
331
}
332
333
impl RecordBatchStream for CoalesceBatchesStream {
334
0
    fn schema(&self) -> SchemaRef {
335
0
        self.coalescer.schema()
336
0
    }
337
}