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, 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
52pub(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 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
137async 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 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 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
215fn 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
232fn 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
256async 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 let mut value_map: HashMap<Vec<String>, Sender<RecordBatch>> = HashMap::new();
276
277 while let Some(rb) = input.next().await.transpose()? {
278 let all_partition_values = compute_partition_keys_by_row(&rb, &partition_by)?;
280
281 let take_map = compute_take_arrays(&rb, all_partition_values);
283
284 for (part_key, mut builder) in take_map.into_iter() {
286 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 let part_tx = match value_map.get_mut(&part_key) {
296 Some(part_tx) => part_tx,
297 None => {
298 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 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 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 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 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}