datafusion-physical-plan 54.0.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
// 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.

//! Define the `SpillManager` struct, which is responsible for reading and writing `RecordBatch`es to raw files based on the provided configurations.

use super::{SpillReaderStream, in_progress_spill_file::InProgressSpillFile};
use crate::coop::cooperative;
use crate::{common::spawn_buffered, metrics::SpillMetrics};
use arrow::array::{BinaryViewArray, GenericByteViewArray, StringViewArray};
use arrow::datatypes::{ByteViewType, SchemaRef};
use arrow::record_batch::RecordBatch;
use datafusion_common::{DataFusionError, Result, config::SpillCompression};
use datafusion_execution::SendableRecordBatchStream;
use datafusion_execution::disk_manager::RefCountedTempFile;
use datafusion_execution::runtime_env::RuntimeEnv;
use std::borrow::Borrow;
use std::sync::Arc;

/// The `SpillManager` is responsible for the following tasks:
/// - Reading and writing `RecordBatch`es to raw files based on the provided configurations.
/// - Updating the associated metrics.
///
/// Note: The caller (external operators such as `SortExec`) is responsible for interpreting the spilled files.
/// For example, all records within the same spill file are ordered according to a specific order.
#[derive(Debug, Clone)]
pub struct SpillManager {
    env: Arc<RuntimeEnv>,
    pub(crate) metrics: SpillMetrics,
    schema: SchemaRef,
    /// Number of batches to buffer in memory during disk reads
    batch_read_buffer_capacity: usize,
    /// general-purpose compression options
    pub(crate) compression: SpillCompression,
}

impl SpillManager {
    pub fn new(env: Arc<RuntimeEnv>, metrics: SpillMetrics, schema: SchemaRef) -> Self {
        Self {
            env,
            metrics,
            schema,
            batch_read_buffer_capacity: 2,
            compression: SpillCompression::default(),
        }
    }

    pub fn with_batch_read_buffer_capacity(
        mut self,
        batch_read_buffer_capacity: usize,
    ) -> Self {
        self.batch_read_buffer_capacity = batch_read_buffer_capacity;
        self
    }

    pub fn with_compression_type(mut self, spill_compression: SpillCompression) -> Self {
        self.compression = spill_compression;
        self
    }

    /// Returns the schema for batches managed by this SpillManager
    pub fn schema(&self) -> &SchemaRef {
        &self.schema
    }

    /// Creates a temporary file for in-progress operations, returning an error
    /// message if file creation fails. The file can be used to append batches
    /// incrementally and then finish the file when done.
    pub fn create_in_progress_file(
        &self,
        request_msg: &str,
    ) -> Result<InProgressSpillFile> {
        let temp_file = self.env.disk_manager.create_tmp_file(request_msg)?;
        Ok(InProgressSpillFile::new(Arc::new(self.clone()), temp_file))
    }

    /// Spill input `batches` into a single file in a atomic operation. If it is
    /// intended to incrementally write in-memory batches into the same spill file,
    /// use [`Self::create_in_progress_file`] instead.
    /// None is returned if no batches are spilled.
    ///
    /// # Errors
    /// - Returns an error if spilling would exceed the disk usage limit configured
    ///   by `max_temp_directory_size` in `DiskManager`
    pub fn spill_record_batch_and_finish(
        &self,
        batches: &[RecordBatch],
        request_msg: &str,
    ) -> Result<Option<RefCountedTempFile>> {
        let mut in_progress_file = self.create_in_progress_file(request_msg)?;

        for batch in batches {
            in_progress_file.append_batch(batch)?;
        }

        in_progress_file.finish()
    }

    /// Spill an iterator of `RecordBatch`es to disk and return the spill file and the size of the largest batch in memory
    /// Note that this expects the caller to provide *non-sliced* batches, so the memory calculation of each batch is accurate.
    pub(crate) fn spill_record_batch_iter_and_return_max_batch_memory(
        &self,
        mut iter: impl Iterator<Item = Result<impl Borrow<RecordBatch>>>,
        request_description: &str,
    ) -> Result<Option<(RefCountedTempFile, usize)>> {
        let mut in_progress_file = self.create_in_progress_file(request_description)?;

        let mut max_record_batch_size = 0;

        iter.try_for_each(|batch| {
            let batch = batch?;
            let borrowed = batch.borrow();
            if borrowed.num_rows() == 0 {
                return Ok(());
            }
            let gc_sliced_size = in_progress_file.append_batch(borrowed)?;
            max_record_batch_size = max_record_batch_size.max(gc_sliced_size);
            Result::<_, DataFusionError>::Ok(())
        })?;

        let file = in_progress_file.finish()?;

        Ok(file.map(|f| (f, max_record_batch_size)))
    }

    /// Spill a stream of `RecordBatch`es to disk and return the spill file and the size of the largest batch in memory
    pub(crate) async fn spill_record_batch_stream_and_return_max_batch_memory(
        &self,
        stream: &mut SendableRecordBatchStream,
        request_description: &str,
    ) -> Result<Option<(RefCountedTempFile, usize)>> {
        use futures::StreamExt;

        let mut in_progress_file = self.create_in_progress_file(request_description)?;

        let mut max_record_batch_size = 0;

        while let Some(batch) = stream.next().await {
            let batch = batch?;
            let gc_sliced_size = in_progress_file.append_batch(&batch)?;

            max_record_batch_size = max_record_batch_size.max(gc_sliced_size);
        }

        let file = in_progress_file.finish()?;

        Ok(file.map(|f| (f, max_record_batch_size)))
    }

    /// Reads a spill file as a stream. The file must be created by the current
    /// `SpillManager`; otherwise an error will be returned.
    ///
    /// Output is produced in FIFO order: the batch appended first is read first.
    ///
    /// # Arg `max_record_batch_memory`
    ///
    /// Most callers should pass `None`. This is mainly useful for the
    /// memory-limited sort-preserving merge path.
    ///
    /// When provided, this value is used only as a validation hint. If a
    /// decoded batch exceeds this threshold, a debug-level log message is
    /// emitted.
    ///
    /// That path uses the maximum spilled batch size to conservatively estimate
    /// the merge degree when merging multiple sorted runs.
    pub fn read_spill_as_stream(
        &self,
        spill_file_path: RefCountedTempFile,
        max_record_batch_memory: Option<usize>,
    ) -> Result<SendableRecordBatchStream> {
        let stream = Box::pin(cooperative(SpillReaderStream::new(
            Arc::clone(&self.schema),
            spill_file_path,
            max_record_batch_memory,
        )));

        Ok(spawn_buffered(stream, self.batch_read_buffer_capacity))
    }

    /// Same as `read_spill_as_stream`, but without buffering.
    pub fn read_spill_as_stream_unbuffered(
        &self,
        spill_file_path: RefCountedTempFile,
        max_record_batch_memory: Option<usize>,
    ) -> Result<SendableRecordBatchStream> {
        Ok(Box::pin(cooperative(SpillReaderStream::new(
            Arc::clone(&self.schema),
            spill_file_path,
            max_record_batch_memory,
        ))))
    }
}

pub(crate) trait GetSlicedSize {
    /// Returns the size of the `RecordBatch` when sliced.
    /// Note: if multiple arrays or even a single array share the same data buffers, we may double count each buffer.
    /// Therefore, make sure we call gc() or gc_view_arrays() before using this method.
    fn get_sliced_size(&self) -> Result<usize>;
}

impl GetSlicedSize for RecordBatch {
    fn get_sliced_size(&self) -> Result<usize> {
        let mut total = 0;
        for array in self.columns() {
            let data = array.to_data();
            total += data.get_slice_memory_size()?;

            // While StringViewArray holds large data buffer for non inlined string, the Arrow layout (BufferSpec)
            // does not include any data buffers. Currently, ArrayData::get_slice_memory_size()
            // under-counts memory size by accounting only views buffer although data buffer is cloned during slice()
            //
            // Therefore, we manually add the sum of the lengths used by all non inlined views
            // on top of the sliced size for views buffer. This matches the intended semantics of
            // "bytes needed if we materialized exactly this slice into fresh buffers".
            // This is a workaround until https://github.com/apache/arrow-rs/issues/8230
            if let Some(sv) = array.as_any().downcast_ref::<StringViewArray>() {
                total += byte_view_data_buffer_size(sv);
            }
            if let Some(bv) = array.as_any().downcast_ref::<BinaryViewArray>() {
                total += byte_view_data_buffer_size(bv);
            }
        }
        Ok(total)
    }
}

fn byte_view_data_buffer_size<T: ByteViewType>(array: &GenericByteViewArray<T>) -> usize {
    array
        .data_buffers()
        .iter()
        .map(|buffer| buffer.capacity())
        .sum()
}

#[cfg(test)]
mod tests {
    use super::SpillManager;
    use crate::common::collect;
    use crate::metrics::{ExecutionPlanMetricsSet, SpillMetrics};
    use crate::spill::{get_record_batch_memory_size, spill_manager::GetSlicedSize};
    use arrow::datatypes::{DataType, Field, Schema};
    use arrow::{
        array::{ArrayRef, Int32Array, StringArray, StringViewArray},
        record_batch::RecordBatch,
    };
    use datafusion_common::Result;
    use datafusion_execution::runtime_env::RuntimeEnv;
    use std::sync::Arc;

    fn build_test_spill_manager(
        env: Arc<RuntimeEnv>,
        schema: Arc<Schema>,
    ) -> SpillManager {
        let metrics = SpillMetrics::new(&ExecutionPlanMetricsSet::new(), 0);
        SpillManager::new(env, metrics, schema)
    }

    fn build_writer_batch(schema: Arc<Schema>) -> Result<RecordBatch> {
        RecordBatch::try_new(
            schema,
            vec![
                Arc::new(Int32Array::from(vec![1, 2, 3])),
                Arc::new(StringArray::from(vec!["a", "b", "c"])),
            ],
        )
        .map_err(Into::into)
    }

    #[tokio::test]
    async fn test_read_spill_as_stream_from_another_spill_manager_same_schema()
    -> Result<()> {
        let env = Arc::new(RuntimeEnv::default());
        let writer_schema = Arc::new(Schema::new(vec![
            Field::new("id", DataType::Int32, false),
            Field::new("value", DataType::Utf8, false),
        ]));
        let reader_schema = Arc::new(Schema::new(vec![
            Field::new("id", DataType::Int32, false),
            Field::new("value", DataType::Utf8, false),
        ]));

        let writer =
            build_test_spill_manager(Arc::clone(&env), Arc::clone(&writer_schema));
        let reader = build_test_spill_manager(env, Arc::clone(&reader_schema));
        let written_batch = build_writer_batch(Arc::clone(&writer_schema))?;

        let spill_file = writer
            .spill_record_batch_and_finish(
                std::slice::from_ref(&written_batch),
                "writer",
            )?
            .unwrap();

        // Same-schema reads through a different SpillManager currently pass
        // because only schema compatibility is validated. This is not a
        // supported usage pattern.
        let stream = reader.read_spill_as_stream(spill_file, None)?;
        assert_eq!(stream.schema(), reader_schema);

        let batches = collect(stream).await?;
        assert_eq!(batches, vec![written_batch]);

        Ok(())
    }

    #[tokio::test]
    async fn test_read_spill_as_stream_from_another_spill_manager_different_schema()
    -> Result<()> {
        let env = Arc::new(RuntimeEnv::default());
        let writer_schema = Arc::new(Schema::new(vec![
            Field::new("id", DataType::Int32, false),
            Field::new("value", DataType::Utf8, false),
        ]));
        let reader_schema = Arc::new(Schema::new(vec![
            Field::new("other_id", DataType::Int32, true),
            Field::new("other_value", DataType::Utf8, true),
        ]));

        let writer =
            build_test_spill_manager(Arc::clone(&env), Arc::clone(&writer_schema));
        let reader = build_test_spill_manager(env, Arc::clone(&reader_schema));
        let written_batch = build_writer_batch(Arc::clone(&writer_schema))?;

        let spill_file = writer
            .spill_record_batch_and_finish(
                std::slice::from_ref(&written_batch),
                "writer",
            )?
            .unwrap();

        let stream = reader.read_spill_as_stream(spill_file, None)?;
        let err = collect(stream)
            .await
            .expect_err("schema mismatch should fail fast");
        let err = err.to_string();
        assert!(err.contains("Spill file schema mismatch"));
        assert!(err.contains("expected"));
        assert!(err.contains("got"));

        Ok(())
    }

    #[test]
    fn check_sliced_size_for_string_view_array() -> Result<()> {
        let array_length = 50;
        let short_len = 8;
        let long_len = 25;

        // Build StringViewArray that includes both inline strings and non inlined strings
        let strings: Vec<String> = (0..array_length)
            .map(|i| {
                if i % 2 == 0 {
                    "a".repeat(short_len)
                } else {
                    "b".repeat(long_len)
                }
            })
            .collect();

        let string_array = StringViewArray::from(strings);
        let array_ref: ArrayRef = Arc::new(string_array);
        let batch = RecordBatch::try_new(
            Arc::new(Schema::new(vec![Field::new(
                "strings",
                DataType::Utf8View,
                false,
            )])),
            vec![array_ref],
        )
        .unwrap();

        // We did not slice the batch, so these two memory size should be equal
        assert_eq!(
            batch.get_sliced_size().unwrap(),
            get_record_batch_memory_size(&batch)
        );

        // Slice the batch into half
        let half_batch = batch.slice(0, array_length / 2);
        // Now sliced_size is smaller because the views buffer is sliced
        assert!(
            half_batch.get_sliced_size().unwrap()
                < get_record_batch_memory_size(&half_batch)
        );
        let data = arrow::array::Array::to_data(&half_batch.column(0));
        let views_sliced_size = data.get_slice_memory_size()?;
        // The sliced size should be larger than sliced views buffer size
        assert!(views_sliced_size < half_batch.get_sliced_size().unwrap());

        Ok(())
    }
}