datafusion_datasource/write/
demux.rs1use 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
54pub(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 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
144async 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 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 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 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
234fn 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
251fn 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
274async 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 let mut value_map: HashMap<Vec<String>, Sender<RecordBatch>> = HashMap::new();
293
294 while let Some(rb) = input.next().await.transpose()? {
295 let all_partition_values = compute_partition_keys_by_row(&rb, &partition_by)?;
297
298 let take_map = compute_take_arrays(&rb, all_partition_values);
300
301 for (part_key, mut builder) in take_map.into_iter() {
303 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 let part_tx = match value_map.get_mut(&part_key) {
313 Some(part_tx) => part_tx,
314 None => {
315 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 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 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 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 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}