datafusion-datasource 53.1.0

datafusion-datasource
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
// 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 enabling
//! dividing input stream into multiple output files at execution time

use std::borrow::Cow;
use std::collections::HashMap;
use std::sync::Arc;

use crate::url::ListingTableUrl;
use crate::write::FileSinkConfig;
use datafusion_common::error::Result;
use datafusion_physical_plan::SendableRecordBatchStream;

use arrow::array::{
    ArrayAccessor, RecordBatch, StringArray, StructArray, builder::UInt64Builder,
    cast::AsArray, downcast_dictionary_array,
};
use arrow::datatypes::{DataType, Schema};
use datafusion_common::cast::{
    as_boolean_array, as_date32_array, as_date64_array, as_float16_array,
    as_float32_array, as_float64_array, as_int8_array, as_int16_array, as_int32_array,
    as_int64_array, as_large_string_array, as_string_array, as_string_view_array,
    as_uint8_array, as_uint16_array, as_uint32_array, as_uint64_array,
};
use datafusion_common::{exec_datafusion_err, internal_datafusion_err, not_impl_err};
use datafusion_common_runtime::SpawnedTask;

use chrono::NaiveDate;
use datafusion_execution::TaskContext;
use futures::StreamExt;
use object_store::path::Path;
use rand::distr::SampleString;
use tokio::sync::mpsc::{self, Receiver, Sender, UnboundedReceiver, UnboundedSender};

type RecordBatchReceiver = Receiver<RecordBatch>;
pub type DemuxedStreamReceiver = UnboundedReceiver<(Path, RecordBatchReceiver)>;

/// Splits a single [SendableRecordBatchStream] into a dynamically determined
/// number of partitions at execution time.
///
/// The partitions are determined by factors known only at execution time, such
/// as total number of rows and partition column values. The demuxer task
/// communicates to the caller by sending channels over a channel. The inner
/// channels send RecordBatches which should be contained within the same output
/// file. The outer channel is used to send a dynamic number of inner channels,
/// representing a dynamic number of total output files.
///
/// The caller is also responsible to monitor the demux task for errors and
/// abort accordingly.
///
/// A path with an extension will force only a single file to
/// be written with the extension from the path. Otherwise the default extension
/// will be used and the output will be split into multiple files.
///
/// Output file guarantees:
///  - Partitioned files: Files are created only for non-empty partitions.
///  - Single-file output: 1 file is always written, even when the stream is empty.
///  - Multi-file output: Depending on the number of record batches, 0 or more files are written.
///
/// Examples of `base_output_path`
///  * `tmp/dataset/` -> is a folder since it ends in `/`
///  * `tmp/dataset` -> is still a folder since it does not end in `/` but has no valid file extension
///  * `tmp/file.parquet` -> is a file since it does not end in `/` and has a valid file extension `.parquet`
///  * `tmp/file.parquet/` -> is a folder since it ends in `/`
///
/// The `partition_by` parameter will additionally split the input based on the
/// unique values of a specific column, see
/// <https://github.com/apache/datafusion/issues/7744>
///
/// ```text
///                                                                              ┌───────────┐               ┌────────────┐    ┌─────────────┐
///                                                                     ┌──────▶ │  batch 1  ├────▶...──────▶│   Batch a  │    │ Output File1│
///                                                                     │        └───────────┘               └────────────┘    └─────────────┘
//////                                                 ┌──────────┐        │        ┌───────────┐               ┌────────────┐    ┌─────────────┐
/// ┌───────────┐               ┌────────────┐      │          │        ├──────▶ │  batch a+1├────▶...──────▶│   Batch b  │    │ Output File2│
/// │  batch 1  ├────▶...──────▶│   Batch N  ├─────▶│  Demux   ├────────┤ ...    └───────────┘               └────────────┘    └─────────────┘
/// └───────────┘               └────────────┘      │          │        │
///                                                 └──────────┘        │        ┌───────────┐               ┌────────────┐    ┌─────────────┐
///                                                                     └──────▶ │  batch d  ├────▶...──────▶│   Batch n  │    │ Output FileN│
///                                                                              └───────────┘               └────────────┘    └─────────────┘
/// ```
pub(crate) fn start_demuxer_task(
    config: &FileSinkConfig,
    data: SendableRecordBatchStream,
    context: &Arc<TaskContext>,
) -> (SpawnedTask<Result<()>>, DemuxedStreamReceiver) {
    let (tx, rx) = mpsc::unbounded_channel();
    let context = Arc::clone(context);
    let file_extension = config.file_extension.clone();
    let base_output_path = config.table_paths[0].clone();
    let task = if config.table_partition_cols.is_empty() {
        let single_file_output = config
            .file_output_mode
            .single_file_output(&base_output_path);
        SpawnedTask::spawn(async move {
            row_count_demuxer(
                tx,
                data,
                context,
                base_output_path,
                file_extension,
                single_file_output,
            )
            .await
        })
    } else {
        // There could be an arbitrarily large number of parallel hive style partitions being written to, so we cannot
        // bound this channel without risking a deadlock.
        let partition_by = config.table_partition_cols.clone();
        let keep_partition_by_columns = config.keep_partition_by_columns;
        SpawnedTask::spawn(async move {
            hive_style_partitions_demuxer(
                tx,
                data,
                context,
                partition_by,
                base_output_path,
                file_extension,
                keep_partition_by_columns,
            )
            .await
        })
    };

    (task, rx)
}

/// Dynamically partitions input stream to achieve desired maximum rows per file
async fn row_count_demuxer(
    mut tx: UnboundedSender<(Path, Receiver<RecordBatch>)>,
    mut input: SendableRecordBatchStream,
    context: Arc<TaskContext>,
    base_output_path: ListingTableUrl,
    file_extension: String,
    single_file_output: bool,
) -> Result<()> {
    let exec_options = &context.session_config().options().execution;

    let max_rows_per_file = exec_options.soft_max_rows_per_output_file;
    let max_buffered_batches = exec_options.max_buffered_batches_per_output_file;
    let minimum_parallel_files = exec_options.minimum_parallel_output_files;
    let mut part_idx = 0;
    let write_id = rand::distr::Alphanumeric.sample_string(&mut rand::rng(), 16);

    let mut open_file_streams = Vec::with_capacity(minimum_parallel_files);

    let mut next_send_steam = 0;
    let mut row_counts = Vec::with_capacity(minimum_parallel_files);

    // Overrides if single_file_output is set
    let minimum_parallel_files = if single_file_output {
        1
    } else {
        minimum_parallel_files
    };

    let max_rows_per_file = if single_file_output {
        usize::MAX
    } else {
        max_rows_per_file
    };

    if single_file_output {
        // ensure we have one file open, even when the input stream is empty
        open_file_streams.push(create_new_file_stream(
            &base_output_path,
            &write_id,
            part_idx,
            &file_extension,
            single_file_output,
            max_buffered_batches,
            &mut tx,
        )?);
        row_counts.push(0);
        part_idx += 1;
    }

    let schema = input.schema();
    let mut is_batch_received = false;

    while let Some(rb) = input.next().await.transpose()? {
        is_batch_received = true;
        // ensure we have at least minimum_parallel_files open
        if open_file_streams.len() < minimum_parallel_files {
            open_file_streams.push(create_new_file_stream(
                &base_output_path,
                &write_id,
                part_idx,
                &file_extension,
                single_file_output,
                max_buffered_batches,
                &mut tx,
            )?);
            row_counts.push(0);
            part_idx += 1;
        } else if row_counts[next_send_steam] >= max_rows_per_file {
            row_counts[next_send_steam] = 0;
            open_file_streams[next_send_steam] = create_new_file_stream(
                &base_output_path,
                &write_id,
                part_idx,
                &file_extension,
                single_file_output,
                max_buffered_batches,
                &mut tx,
            )?;
            part_idx += 1;
        }
        row_counts[next_send_steam] += rb.num_rows();
        open_file_streams[next_send_steam]
            .send(rb)
            .await
            .map_err(|_| {
                exec_datafusion_err!("Error sending RecordBatch to file stream!")
            })?;

        next_send_steam = (next_send_steam + 1) % minimum_parallel_files;
    }

    // if there is no batch send but with a single file, send an empty batch
    if single_file_output && !is_batch_received {
        open_file_streams
            .first_mut()
            .ok_or_else(|| internal_datafusion_err!("Expected a single output file"))?
            .send(RecordBatch::new_empty(schema))
            .await
            .map_err(|_| {
                exec_datafusion_err!("Error sending empty RecordBatch to file stream!")
            })?;
    }

    Ok(())
}

/// Helper for row count demuxer
fn generate_file_path(
    base_output_path: &ListingTableUrl,
    write_id: &str,
    part_idx: usize,
    file_extension: &str,
    single_file_output: bool,
) -> Path {
    if !single_file_output {
        base_output_path
            .prefix()
            .child(format!("{write_id}_{part_idx}.{file_extension}"))
    } else {
        base_output_path.prefix().to_owned()
    }
}

/// Helper for row count demuxer
fn create_new_file_stream(
    base_output_path: &ListingTableUrl,
    write_id: &str,
    part_idx: usize,
    file_extension: &str,
    single_file_output: bool,
    max_buffered_batches: usize,
    tx: &mut UnboundedSender<(Path, Receiver<RecordBatch>)>,
) -> Result<Sender<RecordBatch>> {
    let file_path = generate_file_path(
        base_output_path,
        write_id,
        part_idx,
        file_extension,
        single_file_output,
    );
    let (tx_file, rx_file) = mpsc::channel(max_buffered_batches / 2);
    tx.send((file_path, rx_file))
        .map_err(|_| exec_datafusion_err!("Error sending RecordBatch to file stream!"))?;
    Ok(tx_file)
}

/// Splits an input stream based on the distinct values of a set of columns
/// Assumes standard hive style partition paths such as
/// /col1=val1/col2=val2/outputfile.parquet
async fn hive_style_partitions_demuxer(
    tx: UnboundedSender<(Path, Receiver<RecordBatch>)>,
    mut input: SendableRecordBatchStream,
    context: Arc<TaskContext>,
    partition_by: Vec<(String, DataType)>,
    base_output_path: ListingTableUrl,
    file_extension: String,
    keep_partition_by_columns: bool,
) -> Result<()> {
    let write_id = rand::distr::Alphanumeric.sample_string(&mut rand::rng(), 16);

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

    // To support non string partition col types, cast the type to &str first
    let mut value_map: HashMap<Vec<String>, Sender<RecordBatch>> = HashMap::new();

    while let Some(rb) = input.next().await.transpose()? {
        // First compute partition key for each row of batch, e.g. (col1=val1, col2=val2, ...)
        let all_partition_values = compute_partition_keys_by_row(&rb, &partition_by)?;

        // Next compute how the batch should be split up to take each distinct key to its own batch
        let take_map = compute_take_arrays(&rb, &all_partition_values);

        // Divide up the batch into distinct partition key batches and send each batch
        for (part_key, mut builder) in take_map.into_iter() {
            // Take method adapted from https://github.com/lancedb/lance/pull/1337/files
            // TODO: upstream RecordBatch::take to arrow-rs
            let take_indices = builder.finish();
            let struct_array: StructArray = rb.clone().into();
            let parted_batch = RecordBatch::from(
                arrow::compute::take(&struct_array, &take_indices, None)?.as_struct(),
            );

            // Get or create channel for this batch
            let part_tx = match value_map.get_mut(&part_key) {
                Some(part_tx) => part_tx,
                None => {
                    // Create channel for previously unseen distinct partition key and notify consumer of new file
                    let (part_tx, part_rx) =
                        mpsc::channel::<RecordBatch>(max_buffered_recordbatches);
                    let file_path = compute_hive_style_file_path(
                        &part_key,
                        &partition_by,
                        &write_id,
                        &file_extension,
                        &base_output_path,
                    );

                    tx.send((file_path, part_rx)).map_err(|_| {
                        exec_datafusion_err!("Error sending new file stream!")
                    })?;

                    value_map.insert(part_key.clone(), part_tx);
                    value_map.get_mut(&part_key).ok_or_else(|| {
                        exec_datafusion_err!("Key must exist since it was just inserted!")
                    })?
                }
            };

            let final_batch_to_send = if keep_partition_by_columns {
                parted_batch
            } else {
                remove_partition_by_columns(&parted_batch, &partition_by)?
            };

            // Finally send the partial batch partitioned by distinct value!
            part_tx.send(final_batch_to_send).await.map_err(|_| {
                internal_datafusion_err!("Unexpected error sending parted batch!")
            })?;
        }
    }

    Ok(())
}

fn compute_partition_keys_by_row<'a>(
    rb: &'a RecordBatch,
    partition_by: &'a [(String, DataType)],
) -> Result<Vec<Vec<Cow<'a, str>>>> {
    let mut all_partition_values = vec![];

    const EPOCH_DAYS_FROM_CE: i32 = 719_163;

    // For the purposes of writing partitioned data, we can rely on schema inference
    // to determine the type of the partition cols in order to provide a more ergonomic
    // UI which does not require specifying DataTypes manually. So, we ignore the
    // DataType within the partition_by array and infer the correct type from the
    // batch schema instead.
    let schema = rb.schema();
    for (col, _) in partition_by.iter() {
        let mut partition_values = vec![];

        let dtype = schema.field_with_name(col)?.data_type();
        let col_array = rb.column_by_name(col).ok_or(exec_datafusion_err!(
            "PartitionBy Column {} does not exist in source data! Got schema {schema}.",
            col
        ))?;

        match dtype {
            DataType::Utf8 => {
                let array = as_string_array(col_array)?;
                for i in 0..rb.num_rows() {
                    partition_values.push(Cow::from(array.value(i)));
                }
            }
            DataType::LargeUtf8 => {
                let array = as_large_string_array(col_array)?;
                for i in 0..rb.num_rows() {
                    partition_values.push(Cow::from(array.value(i)));
                }
            }
            DataType::Utf8View => {
                let array = as_string_view_array(col_array)?;
                for i in 0..rb.num_rows() {
                    partition_values.push(Cow::from(array.value(i)));
                }
            }
            DataType::Boolean => {
                let array = as_boolean_array(col_array)?;
                for i in 0..rb.num_rows() {
                    partition_values.push(Cow::from(array.value(i).to_string()));
                }
            }
            DataType::Date32 => {
                let array = as_date32_array(col_array)?;
                // ISO-8601/RFC3339 format - yyyy-mm-dd
                let format = "%Y-%m-%d";
                for i in 0..rb.num_rows() {
                    let date = NaiveDate::from_num_days_from_ce_opt(
                        EPOCH_DAYS_FROM_CE + array.value(i),
                    )
                    .unwrap()
                    .format(format)
                    .to_string();
                    partition_values.push(Cow::from(date));
                }
            }
            DataType::Date64 => {
                let array = as_date64_array(col_array)?;
                // ISO-8601/RFC3339 format - yyyy-mm-dd
                let format = "%Y-%m-%d";
                for i in 0..rb.num_rows() {
                    let date = NaiveDate::from_num_days_from_ce_opt(
                        EPOCH_DAYS_FROM_CE + (array.value(i) / 86_400_000) as i32,
                    )
                    .unwrap()
                    .format(format)
                    .to_string();
                    partition_values.push(Cow::from(date));
                }
            }
            DataType::Int8 => {
                let array = as_int8_array(col_array)?;
                for i in 0..rb.num_rows() {
                    partition_values.push(Cow::from(array.value(i).to_string()));
                }
            }
            DataType::Int16 => {
                let array = as_int16_array(col_array)?;
                for i in 0..rb.num_rows() {
                    partition_values.push(Cow::from(array.value(i).to_string()));
                }
            }
            DataType::Int32 => {
                let array = as_int32_array(col_array)?;
                for i in 0..rb.num_rows() {
                    partition_values.push(Cow::from(array.value(i).to_string()));
                }
            }
            DataType::Int64 => {
                let array = as_int64_array(col_array)?;
                for i in 0..rb.num_rows() {
                    partition_values.push(Cow::from(array.value(i).to_string()));
                }
            }
            DataType::UInt8 => {
                let array = as_uint8_array(col_array)?;
                for i in 0..rb.num_rows() {
                    partition_values.push(Cow::from(array.value(i).to_string()));
                }
            }
            DataType::UInt16 => {
                let array = as_uint16_array(col_array)?;
                for i in 0..rb.num_rows() {
                    partition_values.push(Cow::from(array.value(i).to_string()));
                }
            }
            DataType::UInt32 => {
                let array = as_uint32_array(col_array)?;
                for i in 0..rb.num_rows() {
                    partition_values.push(Cow::from(array.value(i).to_string()));
                }
            }
            DataType::UInt64 => {
                let array = as_uint64_array(col_array)?;
                for i in 0..rb.num_rows() {
                    partition_values.push(Cow::from(array.value(i).to_string()));
                }
            }
            DataType::Float16 => {
                let array = as_float16_array(col_array)?;
                for i in 0..rb.num_rows() {
                    partition_values.push(Cow::from(array.value(i).to_string()));
                }
            }
            DataType::Float32 => {
                let array = as_float32_array(col_array)?;
                for i in 0..rb.num_rows() {
                    partition_values.push(Cow::from(array.value(i).to_string()));
                }
            }
            DataType::Float64 => {
                let array = as_float64_array(col_array)?;
                for i in 0..rb.num_rows() {
                    partition_values.push(Cow::from(array.value(i).to_string()));
                }
            }
            DataType::Dictionary(_, _) => {
                downcast_dictionary_array!(
                    col_array =>  {
                        let array = col_array.downcast_dict::<StringArray>()
                            .ok_or(exec_datafusion_err!("it is not yet supported to write to hive partitions with datatype {}",
                            dtype))?;

                        for i in 0..rb.num_rows() {
                            partition_values.push(Cow::from(array.value(i)));
                        }
                    },
                    _ => unreachable!(),
                )
            }
            _ => {
                return not_impl_err!(
                    "it is not yet supported to write to hive partitions with datatype {}",
                    dtype
                );
            }
        }

        all_partition_values.push(partition_values);
    }

    Ok(all_partition_values)
}

fn compute_take_arrays(
    rb: &RecordBatch,
    all_partition_values: &[Vec<Cow<str>>],
) -> HashMap<Vec<String>, UInt64Builder> {
    let mut take_map = HashMap::new();
    for i in 0..rb.num_rows() {
        let mut part_key = vec![];
        for vals in all_partition_values.iter() {
            part_key.push(vals[i].clone().into());
        }
        let builder = take_map.entry(part_key).or_insert_with(UInt64Builder::new);
        builder.append_value(i as u64);
    }
    take_map
}

fn remove_partition_by_columns(
    parted_batch: &RecordBatch,
    partition_by: &[(String, DataType)],
) -> Result<RecordBatch> {
    let partition_names: Vec<_> = partition_by.iter().map(|(s, _)| s).collect();
    let (non_part_cols, non_part_fields): (Vec<_>, Vec<_>) = parted_batch
        .columns()
        .iter()
        .zip(parted_batch.schema().fields())
        .filter_map(|(a, f)| {
            if !partition_names.contains(&f.name()) {
                Some((Arc::clone(a), (**f).clone()))
            } else {
                None
            }
        })
        .unzip();

    let non_part_schema = Schema::new(non_part_fields);
    let final_batch_to_send =
        RecordBatch::try_new(Arc::new(non_part_schema), non_part_cols)?;

    Ok(final_batch_to_send)
}

fn compute_hive_style_file_path(
    part_key: &[String],
    partition_by: &[(String, DataType)],
    write_id: &str,
    file_extension: &str,
    base_output_path: &ListingTableUrl,
) -> Path {
    let mut file_path = base_output_path.prefix().clone();
    for j in 0..part_key.len() {
        file_path = file_path.child(format!("{}={}", partition_by[j].0, part_key[j]));
    }

    file_path.child(format!("{write_id}.{file_extension}"))
}