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