datafusion 33.0.0

DataFusion is an in-memory query engine that uses Apache Arrow as the memory model
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
// 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.

//! Module containing helper methods/traits related to
//! orchestrating file serialization, streaming to object store,
//! parallelization, and abort handling

use std::sync::Arc;

use crate::datasource::file_format::file_compression_type::FileCompressionType;
use crate::datasource::listing::PartitionedFile;
use crate::datasource::physical_plan::FileSinkConfig;
use crate::error::Result;
use crate::physical_plan::SendableRecordBatchStream;

use arrow_array::RecordBatch;

use datafusion_common::DataFusionError;

use bytes::Bytes;
use datafusion_execution::TaskContext;

use futures::StreamExt;

use object_store::{ObjectMeta, ObjectStore};

use tokio::io::{AsyncWrite, AsyncWriteExt};
use tokio::sync::mpsc::{self, Receiver};
use tokio::task::{JoinHandle, JoinSet};
use tokio::try_join;

use super::demux::start_demuxer_task;
use super::{create_writer, AbortableWrite, BatchSerializer, FileWriterMode};

type WriterType = AbortableWrite<Box<dyn AsyncWrite + Send + Unpin>>;
type SerializerType = Box<dyn BatchSerializer>;

/// Serializes a single data stream in parallel and writes to an ObjectStore
/// concurrently. Data order is preserved. In the event of an error,
/// the ObjectStore writer is returned to the caller in addition to an error,
/// so that the caller may handle aborting failed writes.
pub(crate) async fn serialize_rb_stream_to_object_store(
    mut data_rx: Receiver<RecordBatch>,
    mut serializer: Box<dyn BatchSerializer>,
    mut writer: AbortableWrite<Box<dyn AsyncWrite + Send + Unpin>>,
    unbounded_input: bool,
) -> std::result::Result<(WriterType, u64), (WriterType, DataFusionError)> {
    let (tx, mut rx) =
        mpsc::channel::<JoinHandle<Result<(usize, Bytes), DataFusionError>>>(100);

    let serialize_task = tokio::spawn(async move {
        while let Some(batch) = data_rx.recv().await {
            match serializer.duplicate() {
                Ok(mut serializer_clone) => {
                    let handle = tokio::spawn(async move {
                        let num_rows = batch.num_rows();
                        let bytes = serializer_clone.serialize(batch).await?;
                        Ok((num_rows, bytes))
                    });
                    tx.send(handle).await.map_err(|_| {
                        DataFusionError::Internal(
                            "Unknown error writing to object store".into(),
                        )
                    })?;
                    if unbounded_input {
                        tokio::task::yield_now().await;
                    }
                }
                Err(_) => {
                    return Err(DataFusionError::Internal(
                        "Unknown error writing to object store".into(),
                    ))
                }
            }
        }
        Ok(())
    });

    let mut row_count = 0;
    while let Some(handle) = rx.recv().await {
        match handle.await {
            Ok(Ok((cnt, bytes))) => {
                match writer.write_all(&bytes).await {
                    Ok(_) => (),
                    Err(e) => {
                        return Err((
                            writer,
                            DataFusionError::Execution(format!(
                                "Error writing to object store: {e}"
                            )),
                        ))
                    }
                };
                row_count += cnt;
            }
            Ok(Err(e)) => {
                // Return the writer along with the error
                return Err((writer, e));
            }
            Err(e) => {
                // Handle task panic or cancellation
                return Err((
                    writer,
                    DataFusionError::Execution(format!(
                        "Serialization task panicked or was cancelled: {e}"
                    )),
                ));
            }
        }
    }

    match serialize_task.await {
        Ok(Ok(_)) => (),
        Ok(Err(e)) => return Err((writer, e)),
        Err(_) => {
            return Err((
                writer,
                DataFusionError::Internal("Unknown error writing to object store".into()),
            ))
        }
    };
    Ok((writer, row_count as u64))
}

type FileWriteBundle = (Receiver<RecordBatch>, SerializerType, WriterType);
/// Contains the common logic for serializing RecordBatches and
/// writing the resulting bytes to an ObjectStore.
/// Serialization is assumed to be stateless, i.e.
/// each RecordBatch can be serialized without any
/// dependency on the RecordBatches before or after.
pub(crate) async fn stateless_serialize_and_write_files(
    mut rx: Receiver<FileWriteBundle>,
    tx: tokio::sync::oneshot::Sender<u64>,
    unbounded_input: bool,
) -> Result<()> {
    let mut row_count = 0;
    // tracks if any writers encountered an error triggering the need to abort
    let mut any_errors = false;
    // tracks the specific error triggering abort
    let mut triggering_error = None;
    // tracks if any errors were encountered in the process of aborting writers.
    // if true, we may not have a guarentee that all written data was cleaned up.
    let mut any_abort_errors = false;
    let mut join_set = JoinSet::new();
    while let Some((data_rx, serializer, writer)) = rx.recv().await {
        join_set.spawn(async move {
            serialize_rb_stream_to_object_store(
                data_rx,
                serializer,
                writer,
                unbounded_input,
            )
            .await
        });
    }
    let mut finished_writers = Vec::new();
    while let Some(result) = join_set.join_next().await {
        match result {
            Ok(res) => match res {
                Ok((writer, cnt)) => {
                    finished_writers.push(writer);
                    row_count += cnt;
                }
                Err((writer, e)) => {
                    finished_writers.push(writer);
                    any_errors = true;
                    triggering_error = Some(e);
                }
            },
            Err(e) => {
                // Don't panic, instead try to clean up as many writers as possible.
                // If we hit this code, ownership of a writer was not joined back to
                // this thread, so we cannot clean it up (hence any_abort_errors is true)
                any_errors = true;
                any_abort_errors = true;
                triggering_error = Some(DataFusionError::Internal(format!(
                    "Unexpected join error while serializing file {e}"
                )));
            }
        }
    }

    // Finalize or abort writers as appropriate
    for mut writer in finished_writers.into_iter() {
        match any_errors {
            true => {
                let abort_result = writer.abort_writer();
                if abort_result.is_err() {
                    any_abort_errors = true;
                }
            }
            false => {
                writer.shutdown()
                    .await
                    .map_err(|_| DataFusionError::Internal("Error encountered while finalizing writes! Partial results may have been written to ObjectStore!".into()))?;
            }
        }
    }

    if any_errors {
        match any_abort_errors{
            true => return Err(DataFusionError::Internal("Error encountered during writing to ObjectStore and failed to abort all writers. Partial result may have been written.".into())),
            false => match triggering_error {
                Some(e) => return Err(e),
                None => return Err(DataFusionError::Internal("Unknown Error encountered during writing to ObjectStore. All writers succesfully aborted.".into()))
            }
        }
    }

    tx.send(row_count).map_err(|_| {
        DataFusionError::Internal(
            "Error encountered while sending row count back to file sink!".into(),
        )
    })?;
    Ok(())
}

/// Orchestrates multipart put of a dynamic number of output files from a single input stream
/// for any statelessly serialized file type. That is, any file type for which each [RecordBatch]
/// can be serialized independently of all other [RecordBatch]s.
pub(crate) async fn stateless_multipart_put(
    data: SendableRecordBatchStream,
    context: &Arc<TaskContext>,
    file_extension: String,
    get_serializer: Box<dyn Fn() -> Box<dyn BatchSerializer> + Send>,
    config: &FileSinkConfig,
    compression: FileCompressionType,
) -> Result<u64> {
    let object_store = context
        .runtime_env()
        .object_store(&config.object_store_url)?;

    let single_file_output = config.single_file_output;
    let base_output_path = &config.table_paths[0];
    let unbounded_input = config.unbounded_input;
    let part_cols = if !config.table_partition_cols.is_empty() {
        Some(config.table_partition_cols.clone())
    } else {
        None
    };

    let (demux_task, mut file_stream_rx) = start_demuxer_task(
        data,
        context,
        part_cols,
        base_output_path.clone(),
        file_extension,
        single_file_output,
    );

    let rb_buffer_size = &context
        .session_config()
        .options()
        .execution
        .max_buffered_batches_per_output_file;

    let (tx_file_bundle, rx_file_bundle) = tokio::sync::mpsc::channel(rb_buffer_size / 2);
    let (tx_row_cnt, rx_row_cnt) = tokio::sync::oneshot::channel();
    let write_coordinater_task = tokio::spawn(async move {
        stateless_serialize_and_write_files(rx_file_bundle, tx_row_cnt, unbounded_input)
            .await
    });
    while let Some((output_location, rb_stream)) = file_stream_rx.recv().await {
        let serializer = get_serializer();
        let object_meta = ObjectMeta {
            location: output_location,
            last_modified: chrono::offset::Utc::now(),
            size: 0,
            e_tag: None,
        };
        let writer = create_writer(
            FileWriterMode::PutMultipart,
            compression,
            object_meta.into(),
            object_store.clone(),
        )
        .await?;

        tx_file_bundle
            .send((rb_stream, serializer, writer))
            .await
            .map_err(|_| {
                DataFusionError::Internal(
                    "Writer receive file bundle channel closed unexpectedly!".into(),
                )
            })?;
    }

    // Signal to the write coordinater that no more files are coming
    drop(tx_file_bundle);

    match try_join!(write_coordinater_task, demux_task) {
        Ok((r1, r2)) => {
            r1?;
            r2?;
        }
        Err(e) => {
            if e.is_panic() {
                std::panic::resume_unwind(e.into_panic());
            } else {
                unreachable!();
            }
        }
    }

    let total_count = rx_row_cnt.await.map_err(|_| {
        DataFusionError::Internal(
            "Did not receieve row count from write coordinater".into(),
        )
    })?;

    Ok(total_count)
}

/// Orchestrates append_all for any statelessly serialized file type. Appends to all files provided
/// in a round robin fashion.
pub(crate) async fn stateless_append_all(
    mut data: SendableRecordBatchStream,
    context: &Arc<TaskContext>,
    object_store: Arc<dyn ObjectStore>,
    file_groups: &Vec<PartitionedFile>,
    unbounded_input: bool,
    compression: FileCompressionType,
    get_serializer: Box<dyn Fn(usize) -> Box<dyn BatchSerializer> + Send>,
) -> Result<u64> {
    let rb_buffer_size = &context
        .session_config()
        .options()
        .execution
        .max_buffered_batches_per_output_file;

    let (tx_file_bundle, rx_file_bundle) = tokio::sync::mpsc::channel(file_groups.len());
    let mut send_channels = vec![];
    for file_group in file_groups {
        let serializer = get_serializer(file_group.object_meta.size);

        let file = file_group.clone();
        let writer = create_writer(
            FileWriterMode::Append,
            compression,
            file.object_meta.clone().into(),
            object_store.clone(),
        )
        .await?;

        let (tx, rx) = tokio::sync::mpsc::channel(rb_buffer_size / 2);
        send_channels.push(tx);
        tx_file_bundle
            .send((rx, serializer, writer))
            .await
            .map_err(|_| {
                DataFusionError::Internal(
                    "Writer receive file bundle channel closed unexpectedly!".into(),
                )
            })?;
    }

    let (tx_row_cnt, rx_row_cnt) = tokio::sync::oneshot::channel();
    let write_coordinater_task = tokio::spawn(async move {
        stateless_serialize_and_write_files(rx_file_bundle, tx_row_cnt, unbounded_input)
            .await
    });

    // Append to file groups in round robin
    let mut next_file_idx = 0;
    while let Some(rb) = data.next().await.transpose()? {
        send_channels[next_file_idx].send(rb).await.map_err(|_| {
            DataFusionError::Internal(
                "Recordbatch file append stream closed unexpectedly!".into(),
            )
        })?;
        next_file_idx = (next_file_idx + 1) % send_channels.len();
        if unbounded_input {
            tokio::task::yield_now().await;
        }
    }
    // Signal to the write coordinater that no more files are coming
    drop(tx_file_bundle);
    drop(send_channels);

    let total_count = rx_row_cnt.await.map_err(|_| {
        DataFusionError::Internal(
            "Did not receieve row count from write coordinater".into(),
        )
    })?;

    match try_join!(write_coordinater_task) {
        Ok(r1) => {
            r1.0?;
        }
        Err(e) => {
            if e.is_panic() {
                std::panic::resume_unwind(e.into_panic());
            } else {
                unreachable!();
            }
        }
    }

    Ok(total_count)
}