datafusion_datasource/write/
demux.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//! Module containing helper methods/traits related to enabling
19//! dividing input stream into multiple output files at execution time
20
21use std::borrow::Cow;
22use std::collections::HashMap;
23use std::sync::Arc;
24
25use crate::url::ListingTableUrl;
26use crate::write::FileSinkConfig;
27use datafusion_common::error::Result;
28use datafusion_physical_plan::SendableRecordBatchStream;
29
30use arrow::array::{
31    builder::UInt64Builder, cast::AsArray, downcast_dictionary_array, ArrayAccessor,
32    RecordBatch, StringArray, StructArray,
33};
34use arrow::datatypes::{DataType, Schema};
35use datafusion_common::cast::{
36    as_boolean_array, as_date32_array, as_date64_array, as_float16_array,
37    as_float32_array, as_float64_array, as_int16_array, as_int32_array, as_int64_array,
38    as_int8_array, as_string_array, as_string_view_array, as_uint16_array,
39    as_uint32_array, as_uint64_array, as_uint8_array,
40};
41use datafusion_common::{exec_datafusion_err, internal_datafusion_err, not_impl_err};
42use datafusion_common_runtime::SpawnedTask;
43
44use chrono::NaiveDate;
45use datafusion_execution::TaskContext;
46use futures::StreamExt;
47use object_store::path::Path;
48use rand::distr::SampleString;
49use tokio::sync::mpsc::{self, Receiver, Sender, UnboundedReceiver, UnboundedSender};
50
51type RecordBatchReceiver = Receiver<RecordBatch>;
52pub type DemuxedStreamReceiver = UnboundedReceiver<(Path, RecordBatchReceiver)>;
53
54/// Splits a single [SendableRecordBatchStream] into a dynamically determined
55/// number of partitions at execution time.
56///
57/// The partitions are determined by factors known only at execution time, such
58/// as total number of rows and partition column values. The demuxer task
59/// communicates to the caller by sending channels over a channel. The inner
60/// channels send RecordBatches which should be contained within the same output
61/// file. The outer channel is used to send a dynamic number of inner channels,
62/// representing a dynamic number of total output files.
63///
64/// The caller is also responsible to monitor the demux task for errors and
65/// abort accordingly.
66///
67/// A path with an extension will force only a single file to
68/// be written with the extension from the path. Otherwise the default extension
69/// will be used and the output will be split into multiple files.
70///
71/// Output file guarantees:
72///  - Partitioned files: Files are created only for non-empty partitions.
73///  - Single-file output: 1 file is always written, even when the stream is empty.
74///  - Multi-file output: Depending on the number of record batches, 0 or more files are written.
75///
76/// Examples of `base_output_path`
77///  * `tmp/dataset/` -> is a folder since it ends in `/`
78///  * `tmp/dataset` -> is still a folder since it does not end in `/` but has no valid file extension
79///  * `tmp/file.parquet` -> is a file since it does not end in `/` and has a valid file extension `.parquet`
80///  * `tmp/file.parquet/` -> is a folder since it ends in `/`
81///
82/// The `partition_by` parameter will additionally split the input based on the
83/// unique values of a specific column, see
84/// <https://github.com/apache/datafusion/issues/7744>
85///
86/// ```text
87///                                                                              ┌───────────┐               ┌────────────┐    ┌─────────────┐
88///                                                                     ┌──────▶ │  batch 1  ├────▶...──────▶│   Batch a  │    │ Output File1│
89///                                                                     │        └───────────┘               └────────────┘    └─────────────┘
90///                                                                     │
91///                                                 ┌──────────┐        │        ┌───────────┐               ┌────────────┐    ┌─────────────┐
92/// ┌───────────┐               ┌────────────┐      │          │        ├──────▶ │  batch a+1├────▶...──────▶│   Batch b  │    │ Output File2│
93/// │  batch 1  ├────▶...──────▶│   Batch N  ├─────▶│  Demux   ├────────┤ ...    └───────────┘               └────────────┘    └─────────────┘
94/// └───────────┘               └────────────┘      │          │        │
95///                                                 └──────────┘        │        ┌───────────┐               ┌────────────┐    ┌─────────────┐
96///                                                                     └──────▶ │  batch d  ├────▶...──────▶│   Batch n  │    │ Output FileN│
97///                                                                              └───────────┘               └────────────┘    └─────────────┘
98/// ```
99pub(crate) fn start_demuxer_task(
100    config: &FileSinkConfig,
101    data: SendableRecordBatchStream,
102    context: &Arc<TaskContext>,
103) -> (SpawnedTask<Result<()>>, DemuxedStreamReceiver) {
104    let (tx, rx) = mpsc::unbounded_channel();
105    let context = Arc::clone(context);
106    let file_extension = config.file_extension.clone();
107    let base_output_path = config.table_paths[0].clone();
108    let task = if config.table_partition_cols.is_empty() {
109        let single_file_output = !base_output_path.is_collection()
110            && base_output_path.file_extension().is_some();
111        SpawnedTask::spawn(async move {
112            row_count_demuxer(
113                tx,
114                data,
115                context,
116                base_output_path,
117                file_extension,
118                single_file_output,
119            )
120            .await
121        })
122    } else {
123        // There could be an arbitrarily large number of parallel hive style partitions being written to, so we cannot
124        // bound this channel without risking a deadlock.
125        let partition_by = config.table_partition_cols.clone();
126        let keep_partition_by_columns = config.keep_partition_by_columns;
127        SpawnedTask::spawn(async move {
128            hive_style_partitions_demuxer(
129                tx,
130                data,
131                context,
132                partition_by,
133                base_output_path,
134                file_extension,
135                keep_partition_by_columns,
136            )
137            .await
138        })
139    };
140
141    (task, rx)
142}
143
144/// Dynamically partitions input stream to achieve desired maximum rows per file
145async fn row_count_demuxer(
146    mut tx: UnboundedSender<(Path, Receiver<RecordBatch>)>,
147    mut input: SendableRecordBatchStream,
148    context: Arc<TaskContext>,
149    base_output_path: ListingTableUrl,
150    file_extension: String,
151    single_file_output: bool,
152) -> Result<()> {
153    let exec_options = &context.session_config().options().execution;
154
155    let max_rows_per_file = exec_options.soft_max_rows_per_output_file;
156    let max_buffered_batches = exec_options.max_buffered_batches_per_output_file;
157    let minimum_parallel_files = exec_options.minimum_parallel_output_files;
158    let mut part_idx = 0;
159    let write_id = rand::distr::Alphanumeric.sample_string(&mut rand::rng(), 16);
160
161    let mut open_file_streams = Vec::with_capacity(minimum_parallel_files);
162
163    let mut next_send_steam = 0;
164    let mut row_counts = Vec::with_capacity(minimum_parallel_files);
165
166    // Overrides if single_file_output is set
167    let minimum_parallel_files = if single_file_output {
168        1
169    } else {
170        minimum_parallel_files
171    };
172
173    let max_rows_per_file = if single_file_output {
174        usize::MAX
175    } else {
176        max_rows_per_file
177    };
178
179    if single_file_output {
180        // ensure we have one file open, even when the input stream is empty
181        open_file_streams.push(create_new_file_stream(
182            &base_output_path,
183            &write_id,
184            part_idx,
185            &file_extension,
186            single_file_output,
187            max_buffered_batches,
188            &mut tx,
189        )?);
190        row_counts.push(0);
191        part_idx += 1;
192    }
193
194    while let Some(rb) = input.next().await.transpose()? {
195        // ensure we have at least minimum_parallel_files open
196        if open_file_streams.len() < minimum_parallel_files {
197            open_file_streams.push(create_new_file_stream(
198                &base_output_path,
199                &write_id,
200                part_idx,
201                &file_extension,
202                single_file_output,
203                max_buffered_batches,
204                &mut tx,
205            )?);
206            row_counts.push(0);
207            part_idx += 1;
208        } else if row_counts[next_send_steam] >= max_rows_per_file {
209            row_counts[next_send_steam] = 0;
210            open_file_streams[next_send_steam] = create_new_file_stream(
211                &base_output_path,
212                &write_id,
213                part_idx,
214                &file_extension,
215                single_file_output,
216                max_buffered_batches,
217                &mut tx,
218            )?;
219            part_idx += 1;
220        }
221        row_counts[next_send_steam] += rb.num_rows();
222        open_file_streams[next_send_steam]
223            .send(rb)
224            .await
225            .map_err(|_| {
226                exec_datafusion_err!("Error sending RecordBatch to file stream!")
227            })?;
228
229        next_send_steam = (next_send_steam + 1) % minimum_parallel_files;
230    }
231    Ok(())
232}
233
234/// Helper for row count demuxer
235fn generate_file_path(
236    base_output_path: &ListingTableUrl,
237    write_id: &str,
238    part_idx: usize,
239    file_extension: &str,
240    single_file_output: bool,
241) -> Path {
242    if !single_file_output {
243        base_output_path
244            .prefix()
245            .child(format!("{write_id}_{part_idx}.{file_extension}"))
246    } else {
247        base_output_path.prefix().to_owned()
248    }
249}
250
251/// Helper for row count demuxer
252fn create_new_file_stream(
253    base_output_path: &ListingTableUrl,
254    write_id: &str,
255    part_idx: usize,
256    file_extension: &str,
257    single_file_output: bool,
258    max_buffered_batches: usize,
259    tx: &mut UnboundedSender<(Path, Receiver<RecordBatch>)>,
260) -> Result<Sender<RecordBatch>> {
261    let file_path = generate_file_path(
262        base_output_path,
263        write_id,
264        part_idx,
265        file_extension,
266        single_file_output,
267    );
268    let (tx_file, rx_file) = mpsc::channel(max_buffered_batches / 2);
269    tx.send((file_path, rx_file))
270        .map_err(|_| exec_datafusion_err!("Error sending RecordBatch to file stream!"))?;
271    Ok(tx_file)
272}
273
274/// Splits an input stream based on the distinct values of a set of columns
275/// Assumes standard hive style partition paths such as
276/// /col1=val1/col2=val2/outputfile.parquet
277async fn hive_style_partitions_demuxer(
278    tx: UnboundedSender<(Path, Receiver<RecordBatch>)>,
279    mut input: SendableRecordBatchStream,
280    context: Arc<TaskContext>,
281    partition_by: Vec<(String, DataType)>,
282    base_output_path: ListingTableUrl,
283    file_extension: String,
284    keep_partition_by_columns: bool,
285) -> Result<()> {
286    let write_id = rand::distr::Alphanumeric.sample_string(&mut rand::rng(), 16);
287
288    let exec_options = &context.session_config().options().execution;
289    let max_buffered_recordbatches = exec_options.max_buffered_batches_per_output_file;
290
291    // To support non string partition col types, cast the type to &str first
292    let mut value_map: HashMap<Vec<String>, Sender<RecordBatch>> = HashMap::new();
293
294    while let Some(rb) = input.next().await.transpose()? {
295        // First compute partition key for each row of batch, e.g. (col1=val1, col2=val2, ...)
296        let all_partition_values = compute_partition_keys_by_row(&rb, &partition_by)?;
297
298        // Next compute how the batch should be split up to take each distinct key to its own batch
299        let take_map = compute_take_arrays(&rb, all_partition_values);
300
301        // Divide up the batch into distinct partition key batches and send each batch
302        for (part_key, mut builder) in take_map.into_iter() {
303            // Take method adapted from https://github.com/lancedb/lance/pull/1337/files
304            // TODO: upstream RecordBatch::take to arrow-rs
305            let take_indices = builder.finish();
306            let struct_array: StructArray = rb.clone().into();
307            let parted_batch = RecordBatch::from(
308                arrow::compute::take(&struct_array, &take_indices, None)?.as_struct(),
309            );
310
311            // Get or create channel for this batch
312            let part_tx = match value_map.get_mut(&part_key) {
313                Some(part_tx) => part_tx,
314                None => {
315                    // Create channel for previously unseen distinct partition key and notify consumer of new file
316                    let (part_tx, part_rx) =
317                        mpsc::channel::<RecordBatch>(max_buffered_recordbatches);
318                    let file_path = compute_hive_style_file_path(
319                        &part_key,
320                        &partition_by,
321                        &write_id,
322                        &file_extension,
323                        &base_output_path,
324                    );
325
326                    tx.send((file_path, part_rx)).map_err(|_| {
327                        exec_datafusion_err!("Error sending new file stream!")
328                    })?;
329
330                    value_map.insert(part_key.clone(), part_tx);
331                    value_map.get_mut(&part_key).ok_or_else(|| {
332                        exec_datafusion_err!("Key must exist since it was just inserted!")
333                    })?
334                }
335            };
336
337            let final_batch_to_send = if keep_partition_by_columns {
338                parted_batch
339            } else {
340                remove_partition_by_columns(&parted_batch, &partition_by)?
341            };
342
343            // Finally send the partial batch partitioned by distinct value!
344            part_tx.send(final_batch_to_send).await.map_err(|_| {
345                internal_datafusion_err!("Unexpected error sending parted batch!")
346            })?;
347        }
348    }
349
350    Ok(())
351}
352
353fn compute_partition_keys_by_row<'a>(
354    rb: &'a RecordBatch,
355    partition_by: &'a [(String, DataType)],
356) -> Result<Vec<Vec<Cow<'a, str>>>> {
357    let mut all_partition_values = vec![];
358
359    const EPOCH_DAYS_FROM_CE: i32 = 719_163;
360
361    // For the purposes of writing partitioned data, we can rely on schema inference
362    // to determine the type of the partition cols in order to provide a more ergonomic
363    // UI which does not require specifying DataTypes manually. So, we ignore the
364    // DataType within the partition_by array and infer the correct type from the
365    // batch schema instead.
366    let schema = rb.schema();
367    for (col, _) in partition_by.iter() {
368        let mut partition_values = vec![];
369
370        let dtype = schema.field_with_name(col)?.data_type();
371        let col_array = rb.column_by_name(col).ok_or(exec_datafusion_err!(
372            "PartitionBy Column {} does not exist in source data! Got schema {schema}.",
373            col
374        ))?;
375
376        match dtype {
377            DataType::Utf8 => {
378                let array = as_string_array(col_array)?;
379                for i in 0..rb.num_rows() {
380                    partition_values.push(Cow::from(array.value(i)));
381                }
382            }
383            DataType::Utf8View => {
384                let array = as_string_view_array(col_array)?;
385                for i in 0..rb.num_rows() {
386                    partition_values.push(Cow::from(array.value(i)));
387                }
388            }
389            DataType::Boolean => {
390                let array = as_boolean_array(col_array)?;
391                for i in 0..rb.num_rows() {
392                    partition_values.push(Cow::from(array.value(i).to_string()));
393                }
394            }
395            DataType::Date32 => {
396                let array = as_date32_array(col_array)?;
397                // ISO-8601/RFC3339 format - yyyy-mm-dd
398                let format = "%Y-%m-%d";
399                for i in 0..rb.num_rows() {
400                    let date = NaiveDate::from_num_days_from_ce_opt(
401                        EPOCH_DAYS_FROM_CE + array.value(i),
402                    )
403                    .unwrap()
404                    .format(format)
405                    .to_string();
406                    partition_values.push(Cow::from(date));
407                }
408            }
409            DataType::Date64 => {
410                let array = as_date64_array(col_array)?;
411                // ISO-8601/RFC3339 format - yyyy-mm-dd
412                let format = "%Y-%m-%d";
413                for i in 0..rb.num_rows() {
414                    let date = NaiveDate::from_num_days_from_ce_opt(
415                        EPOCH_DAYS_FROM_CE + (array.value(i) / 86_400_000) as i32,
416                    )
417                    .unwrap()
418                    .format(format)
419                    .to_string();
420                    partition_values.push(Cow::from(date));
421                }
422            }
423            DataType::Int8 => {
424                let array = as_int8_array(col_array)?;
425                for i in 0..rb.num_rows() {
426                    partition_values.push(Cow::from(array.value(i).to_string()));
427                }
428            }
429            DataType::Int16 => {
430                let array = as_int16_array(col_array)?;
431                for i in 0..rb.num_rows() {
432                    partition_values.push(Cow::from(array.value(i).to_string()));
433                }
434            }
435            DataType::Int32 => {
436                let array = as_int32_array(col_array)?;
437                for i in 0..rb.num_rows() {
438                    partition_values.push(Cow::from(array.value(i).to_string()));
439                }
440            }
441            DataType::Int64 => {
442                let array = as_int64_array(col_array)?;
443                for i in 0..rb.num_rows() {
444                    partition_values.push(Cow::from(array.value(i).to_string()));
445                }
446            }
447            DataType::UInt8 => {
448                let array = as_uint8_array(col_array)?;
449                for i in 0..rb.num_rows() {
450                    partition_values.push(Cow::from(array.value(i).to_string()));
451                }
452            }
453            DataType::UInt16 => {
454                let array = as_uint16_array(col_array)?;
455                for i in 0..rb.num_rows() {
456                    partition_values.push(Cow::from(array.value(i).to_string()));
457                }
458            }
459            DataType::UInt32 => {
460                let array = as_uint32_array(col_array)?;
461                for i in 0..rb.num_rows() {
462                    partition_values.push(Cow::from(array.value(i).to_string()));
463                }
464            }
465            DataType::UInt64 => {
466                let array = as_uint64_array(col_array)?;
467                for i in 0..rb.num_rows() {
468                    partition_values.push(Cow::from(array.value(i).to_string()));
469                }
470            }
471            DataType::Float16 => {
472                let array = as_float16_array(col_array)?;
473                for i in 0..rb.num_rows() {
474                    partition_values.push(Cow::from(array.value(i).to_string()));
475                }
476            }
477            DataType::Float32 => {
478                let array = as_float32_array(col_array)?;
479                for i in 0..rb.num_rows() {
480                    partition_values.push(Cow::from(array.value(i).to_string()));
481                }
482            }
483            DataType::Float64 => {
484                let array = as_float64_array(col_array)?;
485                for i in 0..rb.num_rows() {
486                    partition_values.push(Cow::from(array.value(i).to_string()));
487                }
488            }
489            DataType::Dictionary(_, _) => {
490                downcast_dictionary_array!(
491                    col_array =>  {
492                        let array = col_array.downcast_dict::<StringArray>()
493                            .ok_or(exec_datafusion_err!("it is not yet supported to write to hive partitions with datatype {}",
494                            dtype))?;
495
496                        for i in 0..rb.num_rows() {
497                            partition_values.push(Cow::from(array.value(i)));
498                        }
499                    },
500                    _ => unreachable!(),
501                )
502            }
503            _ => {
504                return not_impl_err!(
505                "it is not yet supported to write to hive partitions with datatype {}",
506                dtype
507            )
508            }
509        }
510
511        all_partition_values.push(partition_values);
512    }
513
514    Ok(all_partition_values)
515}
516
517fn compute_take_arrays(
518    rb: &RecordBatch,
519    all_partition_values: Vec<Vec<Cow<str>>>,
520) -> HashMap<Vec<String>, UInt64Builder> {
521    let mut take_map = HashMap::new();
522    for i in 0..rb.num_rows() {
523        let mut part_key = vec![];
524        for vals in all_partition_values.iter() {
525            part_key.push(vals[i].clone().into());
526        }
527        let builder = take_map.entry(part_key).or_insert_with(UInt64Builder::new);
528        builder.append_value(i as u64);
529    }
530    take_map
531}
532
533fn remove_partition_by_columns(
534    parted_batch: &RecordBatch,
535    partition_by: &[(String, DataType)],
536) -> Result<RecordBatch> {
537    let partition_names: Vec<_> = partition_by.iter().map(|(s, _)| s).collect();
538    let (non_part_cols, non_part_fields): (Vec<_>, Vec<_>) = parted_batch
539        .columns()
540        .iter()
541        .zip(parted_batch.schema().fields())
542        .filter_map(|(a, f)| {
543            if !partition_names.contains(&f.name()) {
544                Some((Arc::clone(a), (**f).clone()))
545            } else {
546                None
547            }
548        })
549        .unzip();
550
551    let non_part_schema = Schema::new(non_part_fields);
552    let final_batch_to_send =
553        RecordBatch::try_new(Arc::new(non_part_schema), non_part_cols)?;
554
555    Ok(final_batch_to_send)
556}
557
558fn compute_hive_style_file_path(
559    part_key: &[String],
560    partition_by: &[(String, DataType)],
561    write_id: &str,
562    file_extension: &str,
563    base_output_path: &ListingTableUrl,
564) -> Path {
565    let mut file_path = base_output_path.prefix().clone();
566    for j in 0..part_key.len() {
567        file_path = file_path.child(format!("{}={}", partition_by[j].0, part_key[j]));
568    }
569
570    file_path.child(format!("{write_id}.{file_extension}"))
571}