From 4d5af1a06a59c7467a9c24499710a2eabda4612e Mon Sep 17 00:00:00 2001 From: Colin Ho Date: Thu, 3 Oct 2024 11:05:19 -0700 Subject: [PATCH 01/15] physical --- Cargo.lock | 1 + Cargo.toml | 2 + daft/io/writer.py | 135 ++++++++ src/daft-local-execution/src/buffer.rs | 89 +++++ .../src/intermediate_ops/buffer.rs | 77 ----- .../src/intermediate_ops/intermediate_op.rs | 19 +- .../src/intermediate_ops/mod.rs | 1 - src/daft-local-execution/src/lib.rs | 11 +- src/daft-local-execution/src/pipeline.rs | 88 ++++- src/daft-local-execution/src/run.rs | 2 +- src/daft-local-execution/src/writes/mod.rs | 16 + .../src/writes/partitioned_write.rs | 311 ++++++++++++++++++ .../src/writes/physical_write.rs | 37 +++ .../src/writes/unpartitioned_write.rs | 224 +++++++++++++ src/daft-micropartition/src/lib.rs | 40 ++- src/daft-micropartition/src/py_writers.rs | 113 +++++++ src/daft-physical-plan/Cargo.toml | 3 + src/daft-physical-plan/src/local_plan.rs | 27 +- src/daft-physical-plan/src/translate.rs | 15 + src/daft-plan/src/builder.rs | 8 + src/daft-plan/src/lib.rs | 2 +- src/daft-plan/src/sink_info.rs | 4 +- tests/benchmarks/conftest.py | 80 +++++ tests/benchmarks/test_local_tpch.py | 83 +---- tests/benchmarks/test_streaming_writes.py | 68 ++++ tests/cookbook/test_write.py | 18 +- tests/dataframe/test_decimals.py | 6 - tests/dataframe/test_temporals.py | 7 +- tests/io/test_s3_credentials_refresh.py | 6 - 29 files changed, 1272 insertions(+), 221 deletions(-) create mode 100644 daft/io/writer.py create mode 100644 src/daft-local-execution/src/buffer.rs delete mode 100644 src/daft-local-execution/src/intermediate_ops/buffer.rs create mode 100644 src/daft-local-execution/src/writes/mod.rs create mode 100644 src/daft-local-execution/src/writes/partitioned_write.rs create mode 100644 src/daft-local-execution/src/writes/physical_write.rs create mode 100644 src/daft-local-execution/src/writes/unpartitioned_write.rs create mode 100644 src/daft-micropartition/src/py_writers.rs create mode 100644 tests/benchmarks/test_streaming_writes.py diff --git a/Cargo.lock b/Cargo.lock index 8d60981bd3..7bdd87958d 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -1688,6 +1688,7 @@ dependencies = [ "daft-micropartition", "daft-minhash", "daft-parquet", + "daft-physical-plan", "daft-plan", "daft-scan", "daft-scheduler", diff --git a/Cargo.toml b/Cargo.toml index 1d1065f026..eae7570216 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -20,6 +20,7 @@ daft-local-execution = {path = "src/daft-local-execution", default-features = fa daft-micropartition = {path = "src/daft-micropartition", default-features = false} daft-minhash = {path = "src/daft-minhash", default-features = false} daft-parquet = {path = "src/daft-parquet", default-features = false} +daft-physical-plan = {path = "src/daft-physical-plan", default-features = false} daft-plan = {path = "src/daft-plan", default-features = false} daft-scan = {path = "src/daft-scan", default-features = false} daft-scheduler = {path = "src/daft-scheduler", default-features = false} @@ -47,6 +48,7 @@ python = [ "daft-json/python", "daft-micropartition/python", "daft-parquet/python", + "daft-physical-plan/python", "daft-plan/python", "daft-scan/python", "daft-scheduler/python", diff --git a/daft/io/writer.py b/daft/io/writer.py new file mode 100644 index 0000000000..e507d168e7 --- /dev/null +++ b/daft/io/writer.py @@ -0,0 +1,135 @@ +import uuid +from typing import Dict, Optional, Union + +from daft.daft import IOConfig, PyTable +from daft.dependencies import pa, pacsv, pq +from daft.filesystem import ( + _resolve_paths_and_filesystem, + canonicalize_protocol, + get_protocol_from_path, +) +from daft.series import Series +from daft.table.micropartition import MicroPartition +from daft.table.table import Table + + +def partition_values_to_str_mapping( + partition_values: Table, +) -> Dict[str, str]: + null_part = Series.from_pylist([None]) + pkey_names = partition_values.column_names() + + partition_strings = {} + + for c in pkey_names: + column = partition_values.get_column(c) + string_names = column._to_str_values() + null_filled = column.is_null().if_else(null_part, string_names) + partition_strings[c] = null_filled.to_pylist()[0] + + return partition_strings + + +def partition_string_mapping_to_postfix( + partition_strings: Dict[str, str], + default_partition_fallback: str, +) -> str: + postfix = "/".join( + f"{k}={v if v is not None else default_partition_fallback}" for k, v in partition_strings.items() + ) + return postfix + + +class FileWriterBase: + def __init__( + self, + root_dir: str, + file_idx: int, + file_format: str, + partition_values: Optional[PyTable] = None, + compression: Optional[str] = None, + io_config: Optional[IOConfig] = None, + default_partition_fallback: str = "__HIVE_DEFAULT_PARTITION__", + ): + [self.resolved_path], self.fs = _resolve_paths_and_filesystem(root_dir, io_config=io_config) + protocol = get_protocol_from_path(root_dir) + canonicalized_protocol = canonicalize_protocol(protocol) + is_local_fs = canonicalized_protocol == "file" + + self.file_name = f"{uuid.uuid4()}-{file_idx}.{file_format}" + self.partition_values = Table._from_pytable(partition_values) if partition_values is not None else None + if self.partition_values is not None: + partition_strings = partition_values_to_str_mapping(self.partition_values) + postfix = partition_string_mapping_to_postfix(partition_strings, default_partition_fallback) + self.dir_path = f"{self.resolved_path}/{postfix}" + else: + self.dir_path = f"{self.resolved_path}" + + self.full_path = f"{self.dir_path}/{self.file_name}" + if is_local_fs: + self.fs.create_dir(self.dir_path, recursive=True) + + self.compression = compression if compression is not None else "none" + self.current_writer: Optional[Union[pq.ParquetWriter, pacsv.CSVWriter]] = None + + def _create_writer(self, schema: pa.Schema): + raise NotImplementedError("Subclasses must implement this method.") + + def write(self, table: MicroPartition): + if self.current_writer is None: + self.current_writer = self._create_writer(table.schema().to_pyarrow_schema()) + self.current_writer.write_table(table.to_arrow()) + + def close(self) -> PyTable: + if self.current_writer is not None: + self.current_writer.close() + + metadata = {"path": Series.from_pylist([self.full_path])} + if self.partition_values is not None: + for col_name in self.partition_values.column_names(): + metadata[col_name] = self.partition_values.get_column(col_name) + return Table.from_pydict(metadata)._table + + +class ParquetFileWriter(FileWriterBase): + def __init__( + self, + root_dir: str, + file_idx: int, + partition_values: Optional[PyTable] = None, + compression: str = "none", + io_config: Optional[IOConfig] = None, + ): + super().__init__(root_dir, file_idx, "parquet", partition_values, compression, io_config) + + def _create_writer(self, schema: pa.Schema) -> pq.ParquetWriter: + return pq.ParquetWriter( + self.full_path, + schema, + compression=self.compression, + use_compliant_nested_type=False, + filesystem=self.fs, + ) + + +class CSVFileWriter(FileWriterBase): + def __init__( + self, + root_dir: str, + file_idx: int, + partition_values: Optional[PyTable] = None, + io_config: Optional[IOConfig] = None, + ): + super().__init__( + root_dir, + file_idx, + "csv", + partition_values=partition_values, + io_config=io_config, + ) + + def _create_writer(self, schema: pa.Schema) -> pacsv.CSVWriter: + return pacsv.CSVWriter( + self.full_path, + schema, + ) diff --git a/src/daft-local-execution/src/buffer.rs b/src/daft-local-execution/src/buffer.rs new file mode 100644 index 0000000000..391e4ab604 --- /dev/null +++ b/src/daft-local-execution/src/buffer.rs @@ -0,0 +1,89 @@ +use std::{cmp::Ordering::*, collections::VecDeque, sync::Arc}; + +use common_error::DaftResult; +use daft_micropartition::MicroPartition; + +pub struct RowBasedBuffer { + pub buffer: VecDeque>, + pub curr_len: usize, + pub threshold: usize, +} + +impl RowBasedBuffer { + pub fn new(threshold: usize) -> Self { + assert!(threshold > 0); + Self { + buffer: VecDeque::new(), + curr_len: 0, + threshold, + } + } + + pub fn push(&mut self, part: Arc) { + self.curr_len += part.len(); + self.buffer.push_back(part); + } + + pub fn pop_enough(&mut self) -> DaftResult>>> { + match self.curr_len.cmp(&self.threshold) { + Less => Ok(None), + Equal => { + if self.buffer.len() == 1 { + let part = self.buffer.pop_front().unwrap(); + self.curr_len = 0; + Ok(Some(vec![part])) + } else { + let chunk = MicroPartition::concat( + &std::mem::take(&mut self.buffer) + .iter() + .map(|x| x.as_ref()) + .collect::>(), + )?; + self.curr_len = 0; + Ok(Some(vec![Arc::new(chunk)])) + } + } + Greater => { + let num_ready_chunks = self.curr_len / self.threshold; + let concated = MicroPartition::concat( + &std::mem::take(&mut self.buffer) + .iter() + .map(|x| x.as_ref()) + .collect::>(), + )?; + let mut start = 0; + let mut parts_to_return = Vec::with_capacity(num_ready_chunks); + for _ in 0..num_ready_chunks { + let end = start + self.threshold; + let part = Arc::new(concated.slice(start, end)?); + parts_to_return.push(part); + start = end; + } + if start < concated.len() { + let part = Arc::new(concated.slice(start, concated.len())?); + self.curr_len = part.len(); + self.buffer.push_back(part); + } else { + self.curr_len = 0; + } + Ok(Some(parts_to_return)) + } + } + } + + pub fn pop_all(&mut self) -> DaftResult>> { + assert!(self.curr_len < self.threshold); + if self.buffer.is_empty() { + Ok(None) + } else { + let concated = MicroPartition::concat( + &std::mem::take(&mut self.buffer) + .iter() + .map(|x| x.as_ref()) + .collect::>(), + )?; + self.curr_len = 0; + Ok(Some(Arc::new(concated))) + } + } +} diff --git a/src/daft-local-execution/src/intermediate_ops/buffer.rs b/src/daft-local-execution/src/intermediate_ops/buffer.rs deleted file mode 100644 index 67b17c5380..0000000000 --- a/src/daft-local-execution/src/intermediate_ops/buffer.rs +++ /dev/null @@ -1,77 +0,0 @@ -use std::{cmp::Ordering::*, collections::VecDeque, sync::Arc}; - -use common_error::DaftResult; -use daft_micropartition::MicroPartition; - -pub struct OperatorBuffer { - pub buffer: VecDeque>, - pub curr_len: usize, - pub threshold: usize, -} - -impl OperatorBuffer { - pub fn new(threshold: usize) -> Self { - assert!(threshold > 0); - Self { - buffer: VecDeque::new(), - curr_len: 0, - threshold, - } - } - - pub fn push(&mut self, part: Arc) { - self.curr_len += part.len(); - self.buffer.push_back(part); - } - - pub fn try_clear(&mut self) -> Option>> { - match self.curr_len.cmp(&self.threshold) { - Less => None, - Equal => self.clear_all(), - Greater => Some(self.clear_enough()), - } - } - - fn clear_enough(&mut self) -> DaftResult> { - assert!(self.curr_len > self.threshold); - - let mut to_concat = Vec::with_capacity(self.buffer.len()); - let mut remaining = self.threshold; - - while remaining > 0 { - let part = self.buffer.pop_front().expect("Buffer should not be empty"); - let part_len = part.len(); - if part_len <= remaining { - remaining -= part_len; - to_concat.push(part); - } else { - let (head, tail) = part.split_at(remaining)?; - remaining = 0; - to_concat.push(Arc::new(head)); - self.buffer.push_front(Arc::new(tail)); - break; - } - } - assert_eq!(remaining, 0); - - self.curr_len -= self.threshold; - match to_concat.len() { - 1 => Ok(to_concat.pop().unwrap()), - _ => MicroPartition::concat(&to_concat.iter().map(|x| x.as_ref()).collect::>()) - .map(Arc::new), - } - } - - pub fn clear_all(&mut self) -> Option>> { - if self.buffer.is_empty() { - return None; - } - - let concated = - MicroPartition::concat(&self.buffer.iter().map(|x| x.as_ref()).collect::>()) - .map(Arc::new); - self.buffer.clear(); - self.curr_len = 0; - Some(concated) - } -} diff --git a/src/daft-local-execution/src/intermediate_ops/intermediate_op.rs b/src/daft-local-execution/src/intermediate_ops/intermediate_op.rs index abb5c5388b..da81b364d8 100644 --- a/src/daft-local-execution/src/intermediate_ops/intermediate_op.rs +++ b/src/daft-local-execution/src/intermediate_ops/intermediate_op.rs @@ -5,8 +5,8 @@ use common_error::DaftResult; use daft_micropartition::MicroPartition; use tracing::{info_span, instrument}; -use super::buffer::OperatorBuffer; use crate::{ + buffer::RowBasedBuffer, channel::{create_channel, PipelineChannel, Receiver, Sender}, pipeline::{PipelineNode, PipelineResultType}, runtime_stats::{CountingReceiver, CountingSender, RuntimeStatsContext}, @@ -135,7 +135,7 @@ impl IntermediateNode { }; for (idx, mut receiver) in receivers.into_iter().enumerate() { - let mut buffer = OperatorBuffer::new(morsel_size); + let mut buffer = RowBasedBuffer::new(morsel_size); while let Some(morsel) = receiver.recv().await { if morsel.should_broadcast() { for worker_sender in worker_senders.iter() { @@ -143,18 +143,15 @@ impl IntermediateNode { } } else { buffer.push(morsel.as_data().clone()); - if let Some(ready) = buffer.try_clear() { - let _ = send_to_next_worker(idx, ready?.into()).await; + if let Some(ready) = buffer.pop_enough()? { + for part in ready { + let _ = send_to_next_worker(idx, part.into()).await; + } } } } - // Buffer may still have some morsels left above the threshold - while let Some(ready) = buffer.try_clear() { - let _ = send_to_next_worker(idx, ready?.into()).await; - } - // Clear all remaining morsels - if let Some(last_morsel) = buffer.clear_all() { - let _ = send_to_next_worker(idx, last_morsel?.into()).await; + if let Some(ready) = buffer.pop_all()? { + let _ = send_to_next_worker(idx, ready.into()).await; } } Ok(()) diff --git a/src/daft-local-execution/src/intermediate_ops/mod.rs b/src/daft-local-execution/src/intermediate_ops/mod.rs index 593f9ef5ed..7dafadebba 100644 --- a/src/daft-local-execution/src/intermediate_ops/mod.rs +++ b/src/daft-local-execution/src/intermediate_ops/mod.rs @@ -1,6 +1,5 @@ pub mod aggregate; pub mod anti_semi_hash_join_probe; -pub mod buffer; pub mod filter; pub mod hash_join_probe; pub mod intermediate_op; diff --git a/src/daft-local-execution/src/lib.rs b/src/daft-local-execution/src/lib.rs index 2968c2b04b..42f3610f25 100644 --- a/src/daft-local-execution/src/lib.rs +++ b/src/daft-local-execution/src/lib.rs @@ -1,4 +1,5 @@ #![feature(let_chains)] +mod buffer; mod channel; mod intermediate_ops; mod pipeline; @@ -6,6 +7,7 @@ mod run; mod runtime_stats; mod sinks; mod sources; +mod writes; use common_error::{DaftError, DaftResult}; use lazy_static::lazy_static; pub use run::NativeExecutor; @@ -14,15 +16,20 @@ lazy_static! { pub static ref NUM_CPUS: usize = std::thread::available_parallelism().unwrap().get(); } +pub(crate) type TaskSet = tokio::task::JoinSet; +pub(crate) fn create_task_set() -> TaskSet { + TaskSet::new() +} + pub struct ExecutionRuntimeHandle { - worker_set: tokio::task::JoinSet>, + worker_set: TaskSet>, default_morsel_size: usize, } impl ExecutionRuntimeHandle { pub fn new(default_morsel_size: usize) -> Self { Self { - worker_set: tokio::task::JoinSet::new(), + worker_set: create_task_set(), default_morsel_size, } } diff --git a/src/daft-local-execution/src/pipeline.rs b/src/daft-local-execution/src/pipeline.rs index 0f84ac2636..9349ca0429 100644 --- a/src/daft-local-execution/src/pipeline.rs +++ b/src/daft-local-execution/src/pipeline.rs @@ -1,7 +1,9 @@ -use std::{collections::HashMap, sync::Arc}; +use std::{cmp::min, collections::HashMap, sync::Arc}; +use common_daft_config::DaftExecutionConfig; use common_display::{mermaid::MermaidDisplayVisitor, tree::TreeDisplay}; use common_error::DaftResult; +use common_file_formats::FileFormat; use daft_core::{ datatypes::Field, prelude::{Schema, SchemaRef}, @@ -10,8 +12,8 @@ use daft_core::{ use daft_dsl::{col, join::get_common_join_keys, Expr}; use daft_micropartition::MicroPartition; use daft_physical_plan::{ - Filter, HashAggregate, HashJoin, InMemoryScan, Limit, LocalPhysicalPlan, Project, Sort, - UnGroupedAggregate, + Filter, HashAggregate, HashJoin, InMemoryScan, Limit, LocalPhysicalPlan, PhysicalWrite, + Project, Sort, UnGroupedAggregate, }; use daft_plan::{populate_aggregation_stages, JoinType}; use daft_table::{Probeable, Table}; @@ -31,6 +33,10 @@ use crate::{ streaming_sink::StreamingSinkNode, }, sources::in_memory::InMemorySource, + writes::{ + partitioned_write::PartitionedWriteNode, physical_write::PhysicalWriteOperator, + unpartitioned_write::UnpartitionedWriteNode, + }, ExecutionRuntimeHandle, PipelineCreationSnafu, }; @@ -99,6 +105,7 @@ pub(crate) fn viz_pipeline(root: &dyn PipelineNode) -> String { pub fn physical_plan_to_pipeline( physical_plan: &LocalPhysicalPlan, psets: &HashMap>>, + cfg: &Arc, ) -> crate::Result> { use daft_physical_plan::PhysicalScan; @@ -118,21 +125,21 @@ pub fn physical_plan_to_pipeline( input, projection, .. }) => { let proj_op = ProjectOperator::new(projection.clone()); - let child_node = physical_plan_to_pipeline(input, psets)?; + let child_node = physical_plan_to_pipeline(input, psets, cfg)?; IntermediateNode::new(Arc::new(proj_op), vec![child_node]).boxed() } LocalPhysicalPlan::Filter(Filter { input, predicate, .. }) => { let filter_op = FilterOperator::new(predicate.clone()); - let child_node = physical_plan_to_pipeline(input, psets)?; + let child_node = physical_plan_to_pipeline(input, psets, cfg)?; IntermediateNode::new(Arc::new(filter_op), vec![child_node]).boxed() } LocalPhysicalPlan::Limit(Limit { input, num_rows, .. }) => { let sink = LimitSink::new(*num_rows as usize); - let child_node = physical_plan_to_pipeline(input, psets)?; + let child_node = physical_plan_to_pipeline(input, psets, cfg)?; StreamingSinkNode::new(sink.boxed(), vec![child_node]).boxed() } LocalPhysicalPlan::Concat(_) => { @@ -158,7 +165,7 @@ pub fn physical_plan_to_pipeline( .collect(), vec![], ); - let child_node = physical_plan_to_pipeline(input, psets)?; + let child_node = physical_plan_to_pipeline(input, psets, cfg)?; let post_first_agg_node = IntermediateNode::new(Arc::new(first_stage_agg_op), vec![child_node]).boxed(); @@ -186,7 +193,7 @@ pub fn physical_plan_to_pipeline( }) => { let (first_stage_aggs, second_stage_aggs, final_exprs) = populate_aggregation_stages(aggregations, schema, group_by); - let child_node = physical_plan_to_pipeline(input, psets)?; + let child_node = physical_plan_to_pipeline(input, psets, cfg)?; let (post_first_agg_node, group_by) = if !first_stage_aggs.is_empty() { let agg_op = AggregateOperator::new( first_stage_aggs @@ -226,7 +233,7 @@ pub fn physical_plan_to_pipeline( .. }) => { let sort_sink = SortSink::new(sort_by.clone(), descending.clone()); - let child_node = physical_plan_to_pipeline(input, psets)?; + let child_node = physical_plan_to_pipeline(input, psets, cfg)?; BlockingSinkNode::new(sort_sink.boxed(), child_node).boxed() } LocalPhysicalPlan::HashJoin(HashJoin { @@ -298,11 +305,11 @@ pub fn physical_plan_to_pipeline( // we should move to a builder pattern let build_sink = HashJoinBuildSink::new(key_schema.clone(), casted_build_on, join_type)?; - let build_child_node = physical_plan_to_pipeline(build_child, psets)?; + let build_child_node = physical_plan_to_pipeline(build_child, psets, cfg)?; let build_node = BlockingSinkNode::new(build_sink.boxed(), build_child_node).boxed(); - let probe_child_node = physical_plan_to_pipeline(probe_child, psets)?; + let probe_child_node = physical_plan_to_pipeline(probe_child, psets, cfg)?; match join_type { JoinType::Anti | JoinType::Semi => DaftResult::Ok(IntermediateNode::new( @@ -332,8 +339,63 @@ pub fn physical_plan_to_pipeline( })?; probe_node.boxed() } - _ => { - unimplemented!("Physical plan not supported: {}", physical_plan.name()); + LocalPhysicalPlan::PhysicalWrite(PhysicalWrite { + input, + file_info, + data_schema, + file_schema, + .. + }) => { + let child_node = physical_plan_to_pipeline(input, psets, cfg)?; + let estimated_row_size_bytes = data_schema.estimate_row_size_bytes(); + let (inflation_factor, target_file_size, target_chunk_size) = + match file_info.file_format { + FileFormat::Parquet => ( + cfg.parquet_inflation_factor, + cfg.parquet_target_filesize, + cfg.parquet_target_row_group_size, + ), + FileFormat::Csv => ( + cfg.csv_inflation_factor, + cfg.csv_target_filesize, + cfg.parquet_target_row_group_size, // Just assume same chunk size for CSV and Parquet for now + ), + _ => unreachable!("Physical write should only support Parquet and CSV"), + }; + let file_size = target_file_size as f64 * inflation_factor; + let target_file_rows = if estimated_row_size_bytes > 0.0 { + file_size / estimated_row_size_bytes + } else { + file_size + } as usize; + let target_chunk_rows = min( + target_file_rows, + if estimated_row_size_bytes > 0.0 { + target_chunk_size as f64 / estimated_row_size_bytes + } else { + target_chunk_size as f64 + } as usize, + ); + let write_op = PhysicalWriteOperator::new(file_info.clone()); + match &file_info.partition_cols { + Some(part_cols) => PartitionedWriteNode::new( + child_node, + Arc::new(write_op), + part_cols.clone(), + target_file_rows, + target_chunk_rows, + file_schema.clone(), + ) + .boxed(), + None => UnpartitionedWriteNode::new( + child_node, + Arc::new(write_op), + target_file_rows, + target_chunk_rows, + file_schema.clone(), + ) + .boxed(), + } } }; diff --git a/src/daft-local-execution/src/run.rs b/src/daft-local-execution/src/run.rs index 38d7c3e479..48d8d9b40b 100644 --- a/src/daft-local-execution/src/run.rs +++ b/src/daft-local-execution/src/run.rs @@ -121,7 +121,7 @@ pub fn run_local( results_buffer_size: Option, ) -> DaftResult>> + Send>> { refresh_chrome_trace(); - let mut pipeline = physical_plan_to_pipeline(physical_plan, &psets)?; + let mut pipeline = physical_plan_to_pipeline(physical_plan, &psets, &cfg)?; let (tx, rx) = create_channel(results_buffer_size.unwrap_or(1)); let handle = std::thread::spawn(move || { let runtime = tokio::runtime::Builder::new_multi_thread() diff --git a/src/daft-local-execution/src/writes/mod.rs b/src/daft-local-execution/src/writes/mod.rs new file mode 100644 index 0000000000..fc9ce58553 --- /dev/null +++ b/src/daft-local-execution/src/writes/mod.rs @@ -0,0 +1,16 @@ +use common_error::DaftResult; +use daft_micropartition::FileWriter; +use daft_table::Table; + +pub mod partitioned_write; +pub mod physical_write; +pub mod unpartitioned_write; + +pub trait WriteOperator: Send + Sync { + fn name(&self) -> &'static str; + fn create_writer( + &self, + file_idx: usize, + partition_values: Option<&Table>, + ) -> DaftResult>; +} diff --git a/src/daft-local-execution/src/writes/partitioned_write.rs b/src/daft-local-execution/src/writes/partitioned_write.rs new file mode 100644 index 0000000000..c05e4d5f48 --- /dev/null +++ b/src/daft-local-execution/src/writes/partitioned_write.rs @@ -0,0 +1,311 @@ +use std::{collections::HashMap, sync::Arc}; + +use common_display::tree::TreeDisplay; +use common_error::DaftResult; +use daft_core::prelude::{AsArrow as _, SchemaRef}; +use daft_dsl::ExprRef; +use daft_io::IOStatsContext; +use daft_micropartition::{FileWriter, MicroPartition}; +use daft_table::Table; +use snafu::ResultExt; + +use super::WriteOperator; +use crate::{ + buffer::RowBasedBuffer, + channel::{create_channel, PipelineChannel, Receiver, Sender}, + create_task_set, + pipeline::PipelineNode, + runtime_stats::{CountingReceiver, RuntimeStatsContext}, + ExecutionRuntimeHandle, JoinSnafu, NUM_CPUS, +}; + +struct PerPartitionWriter { + writer: Box, + write_operator: Arc, + partition_values: Table, + buffer: RowBasedBuffer, + target_file_rows: usize, + written_rows_so_far: usize, + results: Vec, +} + +impl PerPartitionWriter { + fn new( + write_operator: Arc, + partition_values: Table, + target_file_rows: usize, + target_chunk_rows: usize, + ) -> DaftResult { + Ok(Self { + writer: write_operator.create_writer(0, Some(&partition_values))?, + write_operator, + partition_values, + buffer: RowBasedBuffer::new(target_chunk_rows), + target_file_rows, + written_rows_so_far: 0, + results: vec![], + }) + } + + fn submit(&mut self, data: &Arc) -> DaftResult<()> { + self.buffer.push(data.clone()); + if let Some(ready) = self.buffer.pop_enough()? { + for part in ready { + self.write(&part)?; + } + } + Ok(()) + } + + fn write(&mut self, data: &Arc) -> DaftResult<()> { + let len = data.len(); + self.writer.write(data)?; + self.written_rows_so_far += len; + + // Check if the file is full, close and start a new file if necessary + if self.written_rows_so_far >= self.target_file_rows { + let result = self.writer.close()?; + if let Some(result) = result { + self.results.push(result); + } + self.written_rows_so_far = 0; + self.writer = self + .write_operator + .create_writer(self.results.len(), Some(&self.partition_values))? + } + Ok(()) + } + + fn finalize(&mut self) -> DaftResult> { + // Write any remaining data from the buffer + let remaining = self.buffer.pop_all()?; + if let Some(part) = remaining { + self.write(&part)?; + } + + // Finalize the current file and collect results + if let Some(result) = self.writer.close()? { + self.results.push(result); + } + Ok(std::mem::take(&mut self.results)) + } +} + +pub(crate) struct PartitionedWriteNode { + child: Box, + runtime_stats: Arc, + write_operator: Arc, + partition_cols: Vec, + target_file_rows: usize, + target_chunk_rows: usize, + file_schema: SchemaRef, +} + +impl PartitionedWriteNode { + pub(crate) fn new( + child: Box, + write_operator: Arc, + partition_cols: Vec, + target_file_rows: usize, + target_chunk_rows: usize, + file_schema: SchemaRef, + ) -> Self { + Self { + child, + runtime_stats: RuntimeStatsContext::new(), + partition_cols, + write_operator, + target_file_rows, + target_chunk_rows, + file_schema, + } + } + + pub(crate) fn boxed(self) -> Box { + Box::new(self) + } + + fn partition( + partition_cols: &[ExprRef], + data: &Arc, + ) -> DaftResult<(Vec
, Table)> { + let data = data.concat_or_get(IOStatsContext::new("MicroPartition::partition_by_value"))?; + let table = data.first().unwrap(); + let (split_tables, partition_values) = table.partition_by_value(partition_cols)?; + Ok((split_tables, partition_values)) + } + + async fn run_writer( + mut input_receiver: Receiver<(Table, Table)>, + write_operator: Arc, + target_chunk_rows: usize, + target_file_rows: usize, + ) -> DaftResult> { + let mut per_partition_writers = HashMap::new(); + while let Some((data, partition_values)) = input_receiver.recv().await { + let partition_values_str = partition_values.to_string(); // TODO (Colin): Figure out how to map a partition to a writer without using String as key + let per_partition_writer = if !per_partition_writers.contains_key(&partition_values_str) + { + per_partition_writers.insert( + partition_values_str.clone(), + PerPartitionWriter::new( + write_operator.clone(), + partition_values, + target_file_rows, + target_chunk_rows, + )?, + ); + per_partition_writers + .get_mut(&partition_values_str) + .unwrap() + } else { + per_partition_writers + .get_mut(&partition_values_str) + .unwrap() + }; + + per_partition_writer.submit(&Arc::new(MicroPartition::new_loaded( + data.schema.clone(), + vec![data].into(), + None, + )))? + } + + let mut results = vec![]; + for writer in per_partition_writers.values_mut() { + let res = writer.finalize()?; + results.extend(res); + } + Ok(results) + } + + fn spawn_writers( + num_writers: usize, + task_set: &mut tokio::task::JoinSet>>, + write_operator: Arc, + target_chunk_rows: usize, + target_file_rows: usize, + ) -> Vec> { + let mut writer_senders = Vec::with_capacity(num_writers); + for _ in 0..num_writers { + let (writer_sender, writer_receiver) = create_channel(1); + task_set.spawn(Self::run_writer( + writer_receiver, + write_operator.clone(), + target_chunk_rows, + target_file_rows, + )); + writer_senders.push(writer_sender); + } + writer_senders + } + + async fn dispatch( + mut input_receiver: CountingReceiver, + senders: Vec>, + partition_cols: Vec, + ) -> DaftResult<()> { + while let Some(data) = input_receiver.recv().await { + let data = data.as_data(); + let (split_tables, partition_values) = Self::partition(&partition_cols, data)?; + let hashes = partition_values.hash_rows()?; + for (idx, (partition, hash)) in split_tables + .into_iter() + .zip(hashes.as_arrow().values_iter()) + .enumerate() + { + let send_to = *hash as usize % senders.len(); + let partition_value_row = partition_values.slice(idx, idx + 1)?; + let _ = senders[send_to] + .send((partition, partition_value_row)) + .await; + } + } + Ok(()) + } +} + +impl TreeDisplay for PartitionedWriteNode { + fn display_as(&self, level: common_display::DisplayLevel) -> String { + use std::fmt::Write; + let mut display = String::new(); + writeln!(display, "{}", self.name()).unwrap(); + use common_display::DisplayLevel::*; + match level { + Compact => {} + _ => { + let rt_result = self.runtime_stats.result(); + rt_result.display(&mut display, true, true, true).unwrap(); + } + } + display + } + + fn get_children(&self) -> Vec<&dyn TreeDisplay> { + vec![self.child.as_tree_display()] + } +} + +impl PipelineNode for PartitionedWriteNode { + fn children(&self) -> Vec<&dyn PipelineNode> { + vec![self.child.as_ref()] + } + + fn name(&self) -> &'static str { + self.write_operator.name() + } + + fn start( + &mut self, + maintain_order: bool, + runtime_handle: &mut ExecutionRuntimeHandle, + ) -> crate::Result { + // Start children + let child = self.child.as_mut(); + let child_results_receiver = child + .start(false, runtime_handle)? + .get_receiver_with_stats(&self.runtime_stats); + + // Initialize destination channels + let mut destination_channel = PipelineChannel::new(1, maintain_order); + let destination_sender = + destination_channel.get_next_sender_with_stats(&self.runtime_stats); + + // Start writers + let mut task_set = create_task_set(); + let writer_senders = Self::spawn_writers( + *NUM_CPUS, + &mut task_set, + self.write_operator.clone(), + self.target_chunk_rows, + self.target_file_rows, + ); + + // Start dispatch + let partition_cols = self.partition_cols.clone(); + runtime_handle.spawn( + Self::dispatch(child_results_receiver, writer_senders, partition_cols), + self.name(), + ); + + // Join writers, receive results, and send to destination + let schema = self.file_schema.clone(); + runtime_handle.spawn( + async move { + let mut results = vec![]; + while let Some(result) = task_set.join_next().await { + results.extend(result.context(JoinSnafu)??); + } + let result_mp = + Arc::new(MicroPartition::new_loaded(schema, Arc::new(results), None)); + let _ = destination_sender.send(result_mp.into()).await; + Ok(()) + }, + self.name(), + ); + Ok(destination_channel) + } + fn as_tree_display(&self) -> &dyn TreeDisplay { + self + } +} diff --git a/src/daft-local-execution/src/writes/physical_write.rs b/src/daft-local-execution/src/writes/physical_write.rs new file mode 100644 index 0000000000..0c869a971a --- /dev/null +++ b/src/daft-local-execution/src/writes/physical_write.rs @@ -0,0 +1,37 @@ +use common_error::DaftResult; +use daft_micropartition::{create_file_writer, FileWriter}; +use daft_plan::OutputFileInfo; +use daft_table::Table; + +use super::WriteOperator; + +pub(crate) struct PhysicalWriteOperator { + output_file_info: OutputFileInfo, +} + +impl PhysicalWriteOperator { + pub(crate) fn new(output_file_info: OutputFileInfo) -> Self { + Self { output_file_info } + } +} + +impl WriteOperator for PhysicalWriteOperator { + fn name(&self) -> &'static str { + "PhysicalWriteOperator" + } + fn create_writer( + &self, + file_idx: usize, + partition_values: Option<&Table>, + ) -> DaftResult> { + let writer = create_file_writer( + &self.output_file_info.root_dir, + file_idx, + &self.output_file_info.compression, + &self.output_file_info.io_config, + self.output_file_info.file_format, + partition_values, + )?; + Ok(writer) + } +} diff --git a/src/daft-local-execution/src/writes/unpartitioned_write.rs b/src/daft-local-execution/src/writes/unpartitioned_write.rs new file mode 100644 index 0000000000..ee221855ae --- /dev/null +++ b/src/daft-local-execution/src/writes/unpartitioned_write.rs @@ -0,0 +1,224 @@ +use std::sync::Arc; + +use common_display::tree::TreeDisplay; +use common_error::DaftResult; +use daft_core::prelude::SchemaRef; +use daft_micropartition::{FileWriter, MicroPartition}; +use daft_table::Table; +use snafu::ResultExt; + +use super::WriteOperator; +use crate::{ + buffer::RowBasedBuffer, + channel::{create_channel, PipelineChannel, Receiver, Sender}, + create_task_set, + pipeline::PipelineNode, + runtime_stats::{CountingReceiver, RuntimeStatsContext}, + ExecutionRuntimeHandle, JoinSnafu, TaskSet, NUM_CPUS, +}; + +pub(crate) struct UnpartitionedWriteNode { + child: Box, + runtime_stats: Arc, + write_operator: Arc, + target_in_memory_file_rows: usize, + target_in_memory_chunk_rows: usize, + file_schema: SchemaRef, +} + +impl UnpartitionedWriteNode { + pub(crate) fn new( + child: Box, + write_operator: Arc, + target_in_memory_file_rows: usize, + target_in_memory_chunk_rows: usize, + file_schema: SchemaRef, + ) -> Self { + Self { + child, + runtime_stats: RuntimeStatsContext::new(), + write_operator, + target_in_memory_file_rows, + target_in_memory_chunk_rows, + file_schema, + } + } + + pub(crate) fn boxed(self) -> Box { + Box::new(self) + } + + async fn run_writer( + mut input_receiver: Receiver<(Arc, usize)>, + write_operator: Arc, + ) -> DaftResult> { + let mut written_file_paths = vec![]; + let mut current_writer: Option> = None; + let mut current_file_idx = None; + while let Some((data, file_idx)) = input_receiver.recv().await { + if current_file_idx.is_none() || current_file_idx.unwrap() != file_idx { + if let Some(writer) = current_writer.take() { + if let Some(path) = writer.close()? { + written_file_paths.push(path); + } + } + current_file_idx = Some(file_idx); + current_writer = Some(write_operator.create_writer(file_idx, None)?); + } + if let Some(writer) = current_writer.as_mut() { + writer.write(&data)?; + } + } + if let Some(writer) = current_writer { + if let Some(path) = writer.close()? { + written_file_paths.push(path); + } + } + Ok(written_file_paths) + } + + fn spawn_writers( + num_writers: usize, + task_set: &mut TaskSet>>, + write_operator: &Arc, + channel_size: usize, + ) -> Vec, usize)>> { + let mut writer_senders = Vec::with_capacity(num_writers); + for _ in 0..num_writers { + let (writer_sender, writer_receiver) = create_channel(channel_size); + task_set.spawn(Self::run_writer(writer_receiver, write_operator.clone())); + writer_senders.push(writer_sender); + } + writer_senders + } + + async fn dispatch( + mut input_receiver: CountingReceiver, + target_chunk_rows: usize, + target_file_rows: usize, + senders: Vec, usize)>>, + ) -> DaftResult<()> { + let mut curr_sent_rows = 0; + let mut curr_file_idx = 0; + let mut curr_sender_idx = 0; + let mut buffer = RowBasedBuffer::new(target_chunk_rows); + while let Some(data) = input_receiver.recv().await { + let data = data.as_data(); + if data.is_empty() { + continue; + } + + buffer.push(data.clone()); + if let Some(ready) = buffer.pop_enough()? { + for part in ready { + curr_sent_rows += part.len(); + let _ = senders[curr_sender_idx].send((part, curr_file_idx)).await; + if curr_sent_rows >= target_file_rows { + curr_sent_rows = 0; + curr_file_idx += 1; + curr_sender_idx = (curr_sender_idx + 1) % senders.len(); + } + } + } + } + if let Some(leftover) = buffer.pop_all()? { + let _ = senders[curr_sender_idx] + .send((leftover, curr_file_idx)) + .await; + } + Ok(()) + } +} + +impl TreeDisplay for UnpartitionedWriteNode { + fn display_as(&self, level: common_display::DisplayLevel) -> String { + use std::fmt::Write; + let mut display = String::new(); + writeln!(display, "{}", self.name()).unwrap(); + use common_display::DisplayLevel::*; + match level { + Compact => {} + _ => { + let rt_result = self.runtime_stats.result(); + rt_result.display(&mut display, true, true, true).unwrap(); + } + } + display + } + + fn get_children(&self) -> Vec<&dyn TreeDisplay> { + vec![self.child.as_tree_display()] + } +} + +impl PipelineNode for UnpartitionedWriteNode { + fn children(&self) -> Vec<&dyn PipelineNode> { + vec![self.child.as_ref()] + } + + fn name(&self) -> &'static str { + self.write_operator.name() + } + + fn start( + &mut self, + maintain_order: bool, + runtime_handle: &mut ExecutionRuntimeHandle, + ) -> crate::Result { + // Start children + let child = self.child.as_mut(); + let child_results_receiver = child + .start(false, runtime_handle)? + .get_receiver_with_stats(&self.runtime_stats); + + // Initialize destination channel + let mut destination_channel = PipelineChannel::new(1, maintain_order); + let destination_sender = + destination_channel.get_next_sender_with_stats(&self.runtime_stats); + + // Start writers + let write_operator = self.write_operator.clone(); + let mut task_set = create_task_set(); + let writer_senders = Self::spawn_writers( + *NUM_CPUS, + &mut task_set, + &write_operator, + (self.target_in_memory_file_rows + self.target_in_memory_chunk_rows + 1) + / self.target_in_memory_chunk_rows, + ); + + // Start dispatch + let (target_file_rows, target_chunk_rows) = ( + self.target_in_memory_file_rows, + self.target_in_memory_chunk_rows, + ); + runtime_handle.spawn( + Self::dispatch( + child_results_receiver, + target_chunk_rows, + target_file_rows, + writer_senders, + ), + self.name(), + ); + + // Join writers, receive results, and send to destination + let schema = self.file_schema.clone(); + runtime_handle.spawn( + async move { + let mut results = vec![]; + while let Some(result) = task_set.join_next().await { + results.extend(result.context(JoinSnafu)??); + } + let mp = MicroPartition::new_loaded(schema, results.into(), None); + let _ = destination_sender.send(Arc::new(mp).into()).await; + Ok(()) + }, + self.name(), + ); + Ok(destination_channel) + } + fn as_tree_display(&self) -> &dyn TreeDisplay { + self + } +} diff --git a/src/daft-micropartition/src/lib.rs b/src/daft-micropartition/src/lib.rs index 1a01f4e933..8f47448e96 100644 --- a/src/daft-micropartition/src/lib.rs +++ b/src/daft-micropartition/src/lib.rs @@ -1,7 +1,11 @@ #![feature(let_chains)] #![feature(iterator_try_reduce)] -use common_error::DaftError; +use std::sync::Arc; + +use common_error::{DaftError, DaftResult}; +use common_file_formats::FileFormat; +use daft_table::Table; use snafu::Snafu; mod micropartition; mod ops; @@ -13,6 +17,8 @@ pub mod python; #[cfg(feature = "python")] use pyo3::PyErr; #[cfg(feature = "python")] +pub mod py_writers; +#[cfg(feature = "python")] pub use python::register_modules; #[derive(Debug, Snafu)] @@ -59,3 +65,35 @@ impl From for pyo3::PyErr { daft_error.into() } } + +pub trait FileWriter: Send + Sync { + fn write(&self, data: &Arc) -> DaftResult<()>; + fn close(&self) -> DaftResult>; +} + +pub fn create_file_writer( + root_dir: &str, + file_idx: usize, + compression: &Option, + io_config: &Option, + format: FileFormat, + partition: Option<&Table>, +) -> DaftResult> { + match format { + #[cfg(feature = "python")] + FileFormat::Parquet => Ok(Box::new(py_writers::PyArrowParquetWriter::new( + root_dir, + file_idx, + compression, + io_config, + partition, + )?)), + #[cfg(feature = "python")] + FileFormat::Csv => Ok(Box::new(py_writers::PyArrowCSVWriter::new( + root_dir, file_idx, io_config, partition, + )?)), + _ => Err(DaftError::ComputeError( + "Unsupported file format for physical write".to_string(), + )), + } +} diff --git a/src/daft-micropartition/src/py_writers.rs b/src/daft-micropartition/src/py_writers.rs new file mode 100644 index 0000000000..7c1e972de5 --- /dev/null +++ b/src/daft-micropartition/src/py_writers.rs @@ -0,0 +1,113 @@ +use std::sync::Arc; + +use common_error::DaftResult; +use daft_table::{python::PyTable, Table}; +use pyo3::{types::PyAnyMethods, PyObject, Python}; + +use crate::{python::PyMicroPartition, FileWriter, MicroPartition}; + +pub struct PyArrowParquetWriter { + py_writer: PyObject, +} + +impl PyArrowParquetWriter { + pub fn new( + root_dir: &str, + file_idx: usize, + compression: &Option, + io_config: &Option, + partition_values: Option<&Table>, + ) -> DaftResult { + Python::with_gil(|py| { + let file_writer_module = py.import_bound(pyo3::intern!(py, "daft.io.writer"))?; + let file_writer_class = file_writer_module.getattr("ParquetFileWriter")?; + + let py_writer = file_writer_class.call1(( + root_dir, + file_idx, + partition_values.map(|pv| PyTable::from(pv.clone())), + compression.as_ref().map(|c| c.as_str()), + io_config.as_ref().map(|cfg| daft_io::python::IOConfig { + config: cfg.clone(), + }), + ))?; + Ok(Self { + py_writer: py_writer.into(), + }) + }) + } +} + +impl FileWriter for PyArrowParquetWriter { + fn write(&self, data: &Arc) -> DaftResult<()> { + Python::with_gil(|py| { + let py_micropartition = py + .import_bound(pyo3::intern!(py, "daft.table"))? + .getattr(pyo3::intern!(py, "MicroPartition"))? + .getattr(pyo3::intern!(py, "_from_pymicropartition"))? + .call1((PyMicroPartition::from(data.clone()),))?; + self.py_writer + .call_method1(py, "write", (py_micropartition,))?; + Ok(()) + }) + } + + fn close(&self) -> DaftResult> { + Python::with_gil(|py| { + let result = self.py_writer.call_method0(py, "close")?; + Ok(Some(result.extract::(py)?.into())) + }) + } +} + +pub struct PyArrowCSVWriter { + py_writer: PyObject, +} + +impl PyArrowCSVWriter { + pub fn new( + root_dir: &str, + file_idx: usize, + io_config: &Option, + partition_values: Option<&Table>, + ) -> DaftResult { + Python::with_gil(|py| { + let file_writer_module = py.import_bound(pyo3::intern!(py, "daft.io.writer"))?; + let file_writer_class = file_writer_module.getattr("CSVFileWriter")?; + + let py_writer = file_writer_class.call1(( + root_dir, + file_idx, + partition_values.map(|pv| PyTable::from(pv.clone())), + io_config.as_ref().map(|cfg| daft_io::python::IOConfig { + config: cfg.clone(), + }), + ))?; + Ok(Self { + py_writer: py_writer.into(), + }) + }) + } +} + +impl FileWriter for PyArrowCSVWriter { + fn write(&self, data: &Arc) -> DaftResult<()> { + Python::with_gil(|py| { + let py_micropartition = py + .import_bound(pyo3::intern!(py, "daft.table"))? + .getattr(pyo3::intern!(py, "MicroPartition"))? + .getattr(pyo3::intern!(py, "_from_pymicropartition"))? + .call1((PyMicroPartition::from(data.clone()),))?; + self.py_writer + .call_method1(py, "write", (py_micropartition,))?; + Ok(()) + }) + } + + fn close(&self) -> DaftResult> { + Python::with_gil(|py| { + let result = self.py_writer.call_method0(py, "close")?; + Ok(Some(result.extract::(py)?.into())) + }) + } +} diff --git a/src/daft-physical-plan/Cargo.toml b/src/daft-physical-plan/Cargo.toml index 778b8b8560..17d419e63f 100644 --- a/src/daft-physical-plan/Cargo.toml +++ b/src/daft-physical-plan/Cargo.toml @@ -8,6 +8,9 @@ daft-scan = {path = "../daft-scan", default-features = false} log = {workspace = true} strum = {version = "0.26", features = ["derive"]} +[features] +python = ["common-error/python", "common-resource-request/python", "daft-core/python", "daft-dsl/python", "daft-plan/python", "daft-scan/python"] + [lints] workspace = true diff --git a/src/daft-physical-plan/src/local_plan.rs b/src/daft-physical-plan/src/local_plan.rs index 548e6505d8..d1a161ebf8 100644 --- a/src/daft-physical-plan/src/local_plan.rs +++ b/src/daft-physical-plan/src/local_plan.rs @@ -3,7 +3,7 @@ use std::sync::Arc; use common_resource_request::ResourceRequest; use daft_core::prelude::*; use daft_dsl::{AggExpr, ExprRef}; -use daft_plan::InMemoryInfo; +use daft_plan::{InMemoryInfo, OutputFileInfo}; use daft_scan::{ScanTask, ScanTaskRef}; pub type LocalPhysicalPlanRef = Arc; @@ -190,6 +190,22 @@ impl LocalPhysicalPlan { .arced() } + pub(crate) fn physical_write( + input: LocalPhysicalPlanRef, + data_schema: SchemaRef, + file_schema: SchemaRef, + file_info: OutputFileInfo, + ) -> LocalPhysicalPlanRef { + Self::PhysicalWrite(PhysicalWrite { + input, + data_schema, + file_schema, + file_info, + plan_stats: PlanStats {}, + }) + .arced() + } + pub fn schema(&self) -> &SchemaRef { match self { Self::PhysicalScan(PhysicalScan { schema, .. }) @@ -293,7 +309,14 @@ pub struct Concat { #[derive(Debug)] -pub struct PhysicalWrite {} +pub struct PhysicalWrite { + pub input: LocalPhysicalPlanRef, + pub data_schema: SchemaRef, + pub file_schema: SchemaRef, + pub file_info: OutputFileInfo, + pub plan_stats: PlanStats, +} + #[derive(Debug)] pub struct PlanStats {} diff --git a/src/daft-physical-plan/src/translate.rs b/src/daft-physical-plan/src/translate.rs index 2d6be9a40b..b691d673a2 100644 --- a/src/daft-physical-plan/src/translate.rs +++ b/src/daft-physical-plan/src/translate.rs @@ -105,6 +105,21 @@ pub fn translate(plan: &LogicalPlanRef) -> DaftResult { log::warn!("Repartition Not supported for Local Executor!; This will be a No-Op"); translate(&repartition.input) } + LogicalPlan::Sink(sink) => { + use daft_plan::SinkInfo; + let input = translate(&sink.input)?; + let data_schema = input.schema().clone(); + match sink.sink_info.as_ref() { + SinkInfo::OutputFileInfo(info) => Ok(LocalPhysicalPlan::physical_write( + input, + data_schema, + sink.schema.clone(), + info.clone(), + )), + #[cfg(feature = "python")] + SinkInfo::CatalogInfo(_) => todo!("CatalogInfo not yet implemented"), + } + } _ => todo!("{} not yet implemented", plan.name()), } } diff --git a/src/daft-plan/src/builder.rs b/src/daft-plan/src/builder.rs index a9e05ec6cb..e60517f3ea 100644 --- a/src/daft-plan/src/builder.rs +++ b/src/daft-plan/src/builder.rs @@ -479,7 +479,9 @@ impl LogicalPlanBuilder { table_name: String, table_location: String, partition_spec: PyObject, + partition_cols: Vec, iceberg_schema: PyObject, + daft_iceberg_schema: SchemaRef, iceberg_properties: PyObject, io_config: Option, catalog_columns: Vec, @@ -489,7 +491,9 @@ impl LogicalPlanBuilder { table_name, table_location, partition_spec, + partition_cols, iceberg_schema, + daft_iceberg_schema, iceberg_properties, io_config, }), @@ -894,7 +898,9 @@ impl PyLogicalPlanBuilder { table_name: String, table_location: String, partition_spec: PyObject, + partition_cols: Vec, iceberg_schema: PyObject, + daft_iceberg_schema: PySchema, iceberg_properties: PyObject, catalog_columns: Vec, io_config: Option, @@ -905,7 +911,9 @@ impl PyLogicalPlanBuilder { table_name, table_location, partition_spec, + pyexprs_to_exprs(partition_cols), iceberg_schema, + daft_iceberg_schema.into(), iceberg_properties, io_config.map(|cfg| cfg.config), catalog_columns, diff --git a/src/daft-plan/src/lib.rs b/src/daft-plan/src/lib.rs index 2541a143db..25d8fd4b62 100644 --- a/src/daft-plan/src/lib.rs +++ b/src/daft-plan/src/lib.rs @@ -31,7 +31,7 @@ pub use physical_planner::{ #[cfg(feature = "python")] use pyo3::prelude::*; #[cfg(feature = "python")] -pub use sink_info::{DeltaLakeCatalogInfo, IcebergCatalogInfo, LanceCatalogInfo}; +pub use sink_info::{CatalogType, DeltaLakeCatalogInfo, IcebergCatalogInfo, LanceCatalogInfo}; pub use sink_info::{OutputFileInfo, SinkInfo}; pub use source_info::{FileInfo, FileInfos, InMemoryInfo, SourceInfo}; #[cfg(feature = "python")] diff --git a/src/daft-plan/src/sink_info.rs b/src/daft-plan/src/sink_info.rs index 02c8e05273..f9050d201b 100644 --- a/src/daft-plan/src/sink_info.rs +++ b/src/daft-plan/src/sink_info.rs @@ -4,6 +4,7 @@ use common_file_formats::FileFormat; use common_io_config::IOConfig; #[cfg(feature = "python")] use common_py_serde::{deserialize_py_object, serialize_py_object}; +use daft_core::prelude::SchemaRef; use daft_dsl::ExprRef; use derivative::Derivative; use itertools::Itertools; @@ -56,6 +57,7 @@ pub struct IcebergCatalogInfo { #[derivative(PartialEq = "ignore")] #[derivative(Hash = "ignore")] pub partition_spec: PyObject, + pub partition_cols: Vec, #[serde( serialize_with = "serialize_py_object", deserialize_with = "deserialize_py_object" @@ -63,7 +65,7 @@ pub struct IcebergCatalogInfo { #[derivative(PartialEq = "ignore")] #[derivative(Hash = "ignore")] pub iceberg_schema: PyObject, - + pub daft_iceberg_schema: SchemaRef, #[serde( serialize_with = "serialize_py_object", deserialize_with = "deserialize_py_object" diff --git a/tests/benchmarks/conftest.py b/tests/benchmarks/conftest.py index 48e4cd43d9..9107225b94 100644 --- a/tests/benchmarks/conftest.py +++ b/tests/benchmarks/conftest.py @@ -6,6 +6,17 @@ import memray import pytest +from fsspec.implementations.local import LocalFileSystem + +import daft +from benchmarking.tpch import data_generation +from tests.assets import TPCH_DBGEN_DIR + +IS_CI = True if os.getenv("CI") else False + +SCALE_FACTOR = 0.2 +NUM_PARTS = [1] if IS_CI else [1, 2] +SOURCE_TYPES = ["in-memory"] if IS_CI else ["parquet", "in-memory"] memray_stats = defaultdict(dict) @@ -48,3 +59,72 @@ def benchmark_wrapper(func, group): return track_mem(func, group) return benchmark_wrapper + + +@pytest.fixture(scope="session", params=NUM_PARTS) +def gen_tpch(request): + # Parametrize the number of parts for each file so that we run tests on single-partition files and multi-partition files + num_parts = request.param + + csv_files_location = data_generation.gen_csv_files(TPCH_DBGEN_DIR, num_parts, SCALE_FACTOR) + + # Disable native executor to generate parquet files, remove once native executor supports writing parquet files + with daft.context.execution_config_ctx(enable_native_executor=False): + parquet_files_location = data_generation.gen_parquet(csv_files_location) + + in_memory_tables = {} + for tbl_name in data_generation.SCHEMA.keys(): + arrow_table = daft.read_parquet(f"{parquet_files_location}/{tbl_name}/*").to_arrow() + in_memory_tables[tbl_name] = daft.from_arrow(arrow_table) + + sqlite_path = data_generation.gen_sqlite_db( + csv_filepath=csv_files_location, + num_parts=num_parts, + ) + + return ( + csv_files_location, + parquet_files_location, + in_memory_tables, + num_parts, + ), sqlite_path + + +@pytest.fixture(scope="module", params=SOURCE_TYPES) +def get_df(gen_tpch, request): + (csv_files_location, parquet_files_location, in_memory_tables, num_parts), _ = gen_tpch + source_type = request.param + print(f"Source Type: {source_type}") + + def _get_df(tbl_name: str): + print(f"Table Name: {tbl_name}, Source Type: {source_type}") + if source_type == "csv": + local_fs = LocalFileSystem() + nonchunked_filepath = f"{csv_files_location}/{tbl_name}.tbl" + chunked_filepath = nonchunked_filepath + ".*" + try: + local_fs.expand_path(chunked_filepath) + fp = chunked_filepath + except FileNotFoundError: + fp = nonchunked_filepath + + df = daft.read_csv( + fp, + has_headers=False, + delimiter="|", + ) + df = df.select( + *[ + daft.col(autoname).alias(colname) + for autoname, colname in zip(df.column_names, data_generation.SCHEMA[tbl_name]) + ] + ) + elif source_type == "parquet": + fp = f"{parquet_files_location}/{tbl_name}/*" + df = daft.read_parquet(fp) + elif source_type == "in-memory": + df = in_memory_tables[tbl_name] + + return df + + return _get_df, num_parts diff --git a/tests/benchmarks/test_local_tpch.py b/tests/benchmarks/test_local_tpch.py index 07165d9ebc..eed56f4b0e 100644 --- a/tests/benchmarks/test_local_tpch.py +++ b/tests/benchmarks/test_local_tpch.py @@ -1,13 +1,12 @@ from __future__ import annotations -import os import sys import pytest -from fsspec.implementations.local import LocalFileSystem import daft -from benchmarking.tpch import answers, data_generation +from benchmarking.tpch import answers +from tests.benchmarks.conftest import IS_CI if sys.platform == "win32": pytest.skip(allow_module_level=True) @@ -15,83 +14,9 @@ import itertools import daft.context -from tests.assets import TPCH_DBGEN_DIR -from tests.integration.conftest import * # noqa: F403 +from tests.integration.conftest import check_answer # noqa F401 -IS_CI = True if os.getenv("CI") else False - -SCALE_FACTOR = 0.2 ENGINES = ["native"] if IS_CI else ["native", "python"] -NUM_PARTS = [1] if IS_CI else [1, 2] -SOURCE_TYPES = ["in-memory"] if IS_CI else ["parquet", "in-memory"] - - -@pytest.fixture(scope="session", params=NUM_PARTS) -def gen_tpch(request): - # Parametrize the number of parts for each file so that we run tests on single-partition files and multi-partition files - num_parts = request.param - - csv_files_location = data_generation.gen_csv_files(TPCH_DBGEN_DIR, num_parts, SCALE_FACTOR) - - # Disable native executor to generate parquet files, remove once native executor supports writing parquet files - with daft.context.execution_config_ctx(enable_native_executor=False): - parquet_files_location = data_generation.gen_parquet(csv_files_location) - - in_memory_tables = {} - for tbl_name in data_generation.SCHEMA.keys(): - arrow_table = daft.read_parquet(f"{parquet_files_location}/{tbl_name}/*").to_arrow() - in_memory_tables[tbl_name] = daft.from_arrow(arrow_table) - - sqlite_path = data_generation.gen_sqlite_db( - csv_filepath=csv_files_location, - num_parts=num_parts, - ) - - return ( - csv_files_location, - parquet_files_location, - in_memory_tables, - num_parts, - ), sqlite_path - - -@pytest.fixture(scope="module", params=SOURCE_TYPES) # TODO: Enable CSV after improving the CSV reader -def get_df(gen_tpch, request): - (csv_files_location, parquet_files_location, in_memory_tables, num_parts), _ = gen_tpch - source_type = request.param - - def _get_df(tbl_name: str): - if source_type == "csv": - local_fs = LocalFileSystem() - nonchunked_filepath = f"{csv_files_location}/{tbl_name}.tbl" - chunked_filepath = nonchunked_filepath + ".*" - try: - local_fs.expand_path(chunked_filepath) - fp = chunked_filepath - except FileNotFoundError: - fp = nonchunked_filepath - - df = daft.read_csv( - fp, - has_headers=False, - delimiter="|", - ) - df = df.select( - *[ - daft.col(autoname).alias(colname) - for autoname, colname in zip(df.column_names, data_generation.SCHEMA[tbl_name]) - ] - ) - elif source_type == "parquet": - fp = f"{parquet_files_location}/{tbl_name}/*" - df = daft.read_parquet(fp) - elif source_type == "in-memory": - df = in_memory_tables[tbl_name] - - return df - - return _get_df, num_parts - TPCH_QUESTIONS = list(range(1, 11)) @@ -102,7 +27,7 @@ def _get_df(tbl_name: str): ) @pytest.mark.benchmark(group="tpch") @pytest.mark.parametrize("engine, q", itertools.product(ENGINES, TPCH_QUESTIONS)) -def test_tpch(tmp_path, check_answer, get_df, benchmark_with_memray, engine, q): +def test_tpch(tmp_path, check_answer, get_df, benchmark_with_memray, engine, q): # noqa F811 get_df, num_parts = get_df def f(): diff --git a/tests/benchmarks/test_streaming_writes.py b/tests/benchmarks/test_streaming_writes.py new file mode 100644 index 0000000000..eaab068101 --- /dev/null +++ b/tests/benchmarks/test_streaming_writes.py @@ -0,0 +1,68 @@ +import pytest + +import daft +from tests.benchmarks.conftest import IS_CI + +ENGINES = ["native", "python"] + + +@pytest.mark.skipif(IS_CI, reason="Write benchmarks are not run in CI") +@pytest.mark.benchmark(group="write") +@pytest.mark.parametrize("engine", ENGINES) +@pytest.mark.parametrize( + "file_type, target_file_size, target_row_group_size", + [ + ("parquet", None, None), + ( + "parquet", + 5 * 1024 * 1024, + 1024 * 1024, + ), # 5MB target file size, 1MB target row group size + ("csv", None, None), + ], +) +@pytest.mark.parametrize("partition_cols", [None, ["L_SHIPMODE"]]) +@pytest.mark.parametrize("get_df", ["in-memory"], indirect=True) +def test_streaming_write( + tmp_path, + get_df, + benchmark_with_memray, + engine, + file_type, + target_file_size, + target_row_group_size, + partition_cols, +): + get_df, num_parts = get_df + daft_df = get_df("lineitem") + + def f(): + if engine == "native": + ctx = daft.context.execution_config_ctx( + enable_native_executor=True, + parquet_target_filesize=target_file_size, + parquet_target_row_group_size=target_row_group_size, + csv_target_filesize=target_file_size, + ) + elif engine == "python": + ctx = daft.context.execution_config_ctx( + enable_native_executor=False, + parquet_target_filesize=target_file_size, + parquet_target_row_group_size=target_row_group_size, + csv_target_filesize=target_file_size, + ) + else: + raise ValueError(f"{engine} unsupported") + + with ctx: + if file_type == "parquet": + return daft_df.write_parquet(tmp_path, partition_cols=partition_cols) + elif file_type == "csv": + return daft_df.write_csv(tmp_path, partition_cols=partition_cols) + else: + raise ValueError(f"{file_type} unsupported") + + benchmark_group = f"parts-{num_parts}-partition-cols-{partition_cols}-file-type-{file_type}-target-file-size-{target_file_size}-target-row-group-size-{target_row_group_size}" + result_files = benchmark_with_memray(f, benchmark_group).to_pydict()["path"] + read_back = daft.read_parquet(result_files) if file_type == "parquet" else daft.read_csv(result_files) + assert read_back.count_rows() == daft_df.count_rows() diff --git a/tests/cookbook/test_write.py b/tests/cookbook/test_write.py index ddd2c9b040..c00e00f1ac 100644 --- a/tests/cookbook/test_write.py +++ b/tests/cookbook/test_write.py @@ -7,14 +7,9 @@ from pyarrow import dataset as pads import daft -from daft import context from tests.conftest import assert_df_equals from tests.cookbook.assets import COOKBOOK_DATA_CSV -pytestmark = pytest.mark.skipif( - context.get_context().daft_execution_config.enable_native_executor is True, - reason="Native executor fails for these tests", -) PYARROW_GE_7_0_0 = tuple(int(s) for s in pa.__version__.split(".") if s.isnumeric()) >= (7, 0, 0) @@ -178,9 +173,8 @@ def test_parquet_write_multifile(tmp_path, smaller_parquet_target_filesize): df = daft.from_pydict(data) df2 = df.write_parquet(tmp_path) assert len(df2) > 1 - ds = pads.dataset(tmp_path, format="parquet") - readback = ds.to_table() - assert readback.to_pydict() == data + read_back = daft.read_parquet(tmp_path.as_posix() + "/*.parquet").sort(by="x").to_pydict() + assert read_back == data @pytest.mark.skipif( @@ -201,9 +195,7 @@ def test_parquet_write_multifile_with_partitioning(tmp_path, smaller_parquet_tar def test_parquet_write_with_some_empty_partitions(tmp_path): data = {"x": [1, 2, 3], "y": ["a", "b", "c"]} - output_files = daft.from_pydict(data).into_partitions(4).write_parquet(tmp_path) - - assert len(output_files) == 3 + daft.from_pydict(data).into_partitions(4).write_parquet(tmp_path) read_back = daft.read_parquet(tmp_path.as_posix() + "/**/*.parquet").sort("x").to_pydict() assert read_back == data @@ -286,9 +278,7 @@ def test_empty_csv_write_with_partitioning(tmp_path): def test_csv_write_with_some_empty_partitions(tmp_path): data = {"x": [1, 2, 3], "y": ["a", "b", "c"]} - output_files = daft.from_pydict(data).into_partitions(4).write_csv(tmp_path) - - assert len(output_files) == 3 + daft.from_pydict(data).into_partitions(4).write_csv(tmp_path) read_back = daft.read_csv(tmp_path.as_posix() + "/**/*.csv").sort("x").to_pydict() assert read_back == data diff --git a/tests/dataframe/test_decimals.py b/tests/dataframe/test_decimals.py index daafec29f0..3a2d11babe 100644 --- a/tests/dataframe/test_decimals.py +++ b/tests/dataframe/test_decimals.py @@ -7,12 +7,6 @@ import pytest import daft -from daft import context - -pytestmark = pytest.mark.skipif( - context.get_context().daft_execution_config.enable_native_executor is True, - reason="Native executor fails for these tests", -) PYARROW_GE_7_0_0 = tuple(int(s) for s in pa.__version__.split(".") if s.isnumeric()) >= (7, 0, 0) diff --git a/tests/dataframe/test_temporals.py b/tests/dataframe/test_temporals.py index 8843028b01..aff8f540b5 100644 --- a/tests/dataframe/test_temporals.py +++ b/tests/dataframe/test_temporals.py @@ -9,12 +9,7 @@ import pytz import daft -from daft import DataType, col, context - -pytestmark = pytest.mark.skipif( - context.get_context().daft_execution_config.enable_native_executor is True, - reason="Native executor fails for these tests", -) +from daft import DataType, col PYARROW_GE_7_0_0 = tuple(int(s) for s in pa.__version__.split(".") if s.isnumeric()) >= (7, 0, 0) diff --git a/tests/io/test_s3_credentials_refresh.py b/tests/io/test_s3_credentials_refresh.py index 25c5c0cd8c..16a98fadf0 100644 --- a/tests/io/test_s3_credentials_refresh.py +++ b/tests/io/test_s3_credentials_refresh.py @@ -10,14 +10,8 @@ import pytest import daft -from daft import context from tests.io.mock_aws_server import start_service, stop_process -pytestmark = pytest.mark.skipif( - context.get_context().daft_execution_config.enable_native_executor is True, - reason="Native executor fails for these tests", -) - @pytest.fixture(scope="session") def aws_log_file(tmp_path_factory: pytest.TempPathFactory) -> Iterator[io.IOBase]: From 823e8893bf2cfed2e5cf64300bca407dd7340bdd Mon Sep 17 00:00:00 2001 From: Colin Ho Date: Thu, 3 Oct 2024 11:26:20 -0700 Subject: [PATCH 02/15] iceberg fix --- src/daft-plan/src/builder.rs | 8 -------- src/daft-plan/src/sink_info.rs | 3 --- 2 files changed, 11 deletions(-) diff --git a/src/daft-plan/src/builder.rs b/src/daft-plan/src/builder.rs index e60517f3ea..a9e05ec6cb 100644 --- a/src/daft-plan/src/builder.rs +++ b/src/daft-plan/src/builder.rs @@ -479,9 +479,7 @@ impl LogicalPlanBuilder { table_name: String, table_location: String, partition_spec: PyObject, - partition_cols: Vec, iceberg_schema: PyObject, - daft_iceberg_schema: SchemaRef, iceberg_properties: PyObject, io_config: Option, catalog_columns: Vec, @@ -491,9 +489,7 @@ impl LogicalPlanBuilder { table_name, table_location, partition_spec, - partition_cols, iceberg_schema, - daft_iceberg_schema, iceberg_properties, io_config, }), @@ -898,9 +894,7 @@ impl PyLogicalPlanBuilder { table_name: String, table_location: String, partition_spec: PyObject, - partition_cols: Vec, iceberg_schema: PyObject, - daft_iceberg_schema: PySchema, iceberg_properties: PyObject, catalog_columns: Vec, io_config: Option, @@ -911,9 +905,7 @@ impl PyLogicalPlanBuilder { table_name, table_location, partition_spec, - pyexprs_to_exprs(partition_cols), iceberg_schema, - daft_iceberg_schema.into(), iceberg_properties, io_config.map(|cfg| cfg.config), catalog_columns, diff --git a/src/daft-plan/src/sink_info.rs b/src/daft-plan/src/sink_info.rs index f9050d201b..d083a1f53d 100644 --- a/src/daft-plan/src/sink_info.rs +++ b/src/daft-plan/src/sink_info.rs @@ -4,7 +4,6 @@ use common_file_formats::FileFormat; use common_io_config::IOConfig; #[cfg(feature = "python")] use common_py_serde::{deserialize_py_object, serialize_py_object}; -use daft_core::prelude::SchemaRef; use daft_dsl::ExprRef; use derivative::Derivative; use itertools::Itertools; @@ -57,7 +56,6 @@ pub struct IcebergCatalogInfo { #[derivative(PartialEq = "ignore")] #[derivative(Hash = "ignore")] pub partition_spec: PyObject, - pub partition_cols: Vec, #[serde( serialize_with = "serialize_py_object", deserialize_with = "deserialize_py_object" @@ -65,7 +63,6 @@ pub struct IcebergCatalogInfo { #[derivative(PartialEq = "ignore")] #[derivative(Hash = "ignore")] pub iceberg_schema: PyObject, - pub daft_iceberg_schema: SchemaRef, #[serde( serialize_with = "serialize_py_object", deserialize_with = "deserialize_py_object" From 3854cd22efcda5a7ab2c5e4c1e54dc54c57fdcce Mon Sep 17 00:00:00 2001 From: Colin Ho Date: Thu, 3 Oct 2024 14:17:45 -0700 Subject: [PATCH 03/15] rename to factory --- src/daft-local-execution/src/pipeline.rs | 17 +++++++--- src/daft-local-execution/src/writes/mod.rs | 3 +- .../src/writes/partitioned_write.rs | 33 ++++++++++--------- .../src/writes/physical_write.rs | 11 +++---- .../src/writes/unpartitioned_write.rs | 33 ++++++++++++------- tests/benchmarks/conftest.py | 5 +-- 6 files changed, 59 insertions(+), 43 deletions(-) diff --git a/src/daft-local-execution/src/pipeline.rs b/src/daft-local-execution/src/pipeline.rs index 9349ca0429..680e7416e1 100644 --- a/src/daft-local-execution/src/pipeline.rs +++ b/src/daft-local-execution/src/pipeline.rs @@ -34,7 +34,7 @@ use crate::{ }, sources::in_memory::InMemorySource, writes::{ - partitioned_write::PartitionedWriteNode, physical_write::PhysicalWriteOperator, + partitioned_write::PartitionedWriteNode, physical_write::PhysicalWriterFactory, unpartitioned_write::UnpartitionedWriteNode, }, ExecutionRuntimeHandle, PipelineCreationSnafu, @@ -376,11 +376,19 @@ pub fn physical_plan_to_pipeline( target_chunk_size as f64 } as usize, ); - let write_op = PhysicalWriteOperator::new(file_info.clone()); + let write_factory = PhysicalWriterFactory::new(file_info.clone()); + let name = match (&file_info.partition_cols.is_some(), &file_info.file_format) { + (true, FileFormat::Parquet) => "PartitionedParquetWrite", + (true, FileFormat::Csv) => "PartitionedCSVWrite", + (false, FileFormat::Parquet) => "UnpartitionedParquetWrite", + (false, FileFormat::Csv) => "UnpartitionedCSVWrite", + _ => unreachable!("Physical write should only support Parquet and CSV"), + }; match &file_info.partition_cols { Some(part_cols) => PartitionedWriteNode::new( + name, child_node, - Arc::new(write_op), + Arc::new(write_factory), part_cols.clone(), target_file_rows, target_chunk_rows, @@ -388,8 +396,9 @@ pub fn physical_plan_to_pipeline( ) .boxed(), None => UnpartitionedWriteNode::new( + name, child_node, - Arc::new(write_op), + Arc::new(write_factory), target_file_rows, target_chunk_rows, file_schema.clone(), diff --git a/src/daft-local-execution/src/writes/mod.rs b/src/daft-local-execution/src/writes/mod.rs index fc9ce58553..9ada93017d 100644 --- a/src/daft-local-execution/src/writes/mod.rs +++ b/src/daft-local-execution/src/writes/mod.rs @@ -6,8 +6,7 @@ pub mod partitioned_write; pub mod physical_write; pub mod unpartitioned_write; -pub trait WriteOperator: Send + Sync { - fn name(&self) -> &'static str; +pub trait WriterFactory: Send + Sync { fn create_writer( &self, file_idx: usize, diff --git a/src/daft-local-execution/src/writes/partitioned_write.rs b/src/daft-local-execution/src/writes/partitioned_write.rs index c05e4d5f48..2288525530 100644 --- a/src/daft-local-execution/src/writes/partitioned_write.rs +++ b/src/daft-local-execution/src/writes/partitioned_write.rs @@ -9,7 +9,7 @@ use daft_micropartition::{FileWriter, MicroPartition}; use daft_table::Table; use snafu::ResultExt; -use super::WriteOperator; +use super::WriterFactory; use crate::{ buffer::RowBasedBuffer, channel::{create_channel, PipelineChannel, Receiver, Sender}, @@ -21,7 +21,7 @@ use crate::{ struct PerPartitionWriter { writer: Box, - write_operator: Arc, + writer_factory: Arc, partition_values: Table, buffer: RowBasedBuffer, target_file_rows: usize, @@ -31,14 +31,14 @@ struct PerPartitionWriter { impl PerPartitionWriter { fn new( - write_operator: Arc, + writer_factory: Arc, partition_values: Table, target_file_rows: usize, target_chunk_rows: usize, ) -> DaftResult { Ok(Self { - writer: write_operator.create_writer(0, Some(&partition_values))?, - write_operator, + writer: writer_factory.create_writer(0, Some(&partition_values))?, + writer_factory, partition_values, buffer: RowBasedBuffer::new(target_chunk_rows), target_file_rows, @@ -70,7 +70,7 @@ impl PerPartitionWriter { } self.written_rows_so_far = 0; self.writer = self - .write_operator + .writer_factory .create_writer(self.results.len(), Some(&self.partition_values))? } Ok(()) @@ -92,9 +92,10 @@ impl PerPartitionWriter { } pub(crate) struct PartitionedWriteNode { + name: &'static str, child: Box, runtime_stats: Arc, - write_operator: Arc, + writer_factory: Arc, partition_cols: Vec, target_file_rows: usize, target_chunk_rows: usize, @@ -103,18 +104,20 @@ pub(crate) struct PartitionedWriteNode { impl PartitionedWriteNode { pub(crate) fn new( + name: &'static str, child: Box, - write_operator: Arc, + writer_factory: Arc, partition_cols: Vec, target_file_rows: usize, target_chunk_rows: usize, file_schema: SchemaRef, ) -> Self { Self { + name, child, runtime_stats: RuntimeStatsContext::new(), partition_cols, - write_operator, + writer_factory, target_file_rows, target_chunk_rows, file_schema, @@ -137,7 +140,7 @@ impl PartitionedWriteNode { async fn run_writer( mut input_receiver: Receiver<(Table, Table)>, - write_operator: Arc, + writer_factory: Arc, target_chunk_rows: usize, target_file_rows: usize, ) -> DaftResult> { @@ -149,7 +152,7 @@ impl PartitionedWriteNode { per_partition_writers.insert( partition_values_str.clone(), PerPartitionWriter::new( - write_operator.clone(), + writer_factory.clone(), partition_values, target_file_rows, target_chunk_rows, @@ -182,7 +185,7 @@ impl PartitionedWriteNode { fn spawn_writers( num_writers: usize, task_set: &mut tokio::task::JoinSet>>, - write_operator: Arc, + writer_factory: Arc, target_chunk_rows: usize, target_file_rows: usize, ) -> Vec> { @@ -191,7 +194,7 @@ impl PartitionedWriteNode { let (writer_sender, writer_receiver) = create_channel(1); task_set.spawn(Self::run_writer( writer_receiver, - write_operator.clone(), + writer_factory.clone(), target_chunk_rows, target_file_rows, )); @@ -252,7 +255,7 @@ impl PipelineNode for PartitionedWriteNode { } fn name(&self) -> &'static str { - self.write_operator.name() + self.name } fn start( @@ -276,7 +279,7 @@ impl PipelineNode for PartitionedWriteNode { let writer_senders = Self::spawn_writers( *NUM_CPUS, &mut task_set, - self.write_operator.clone(), + self.writer_factory.clone(), self.target_chunk_rows, self.target_file_rows, ); diff --git a/src/daft-local-execution/src/writes/physical_write.rs b/src/daft-local-execution/src/writes/physical_write.rs index 0c869a971a..54ec6501e1 100644 --- a/src/daft-local-execution/src/writes/physical_write.rs +++ b/src/daft-local-execution/src/writes/physical_write.rs @@ -3,22 +3,19 @@ use daft_micropartition::{create_file_writer, FileWriter}; use daft_plan::OutputFileInfo; use daft_table::Table; -use super::WriteOperator; +use super::WriterFactory; -pub(crate) struct PhysicalWriteOperator { +pub(crate) struct PhysicalWriterFactory { output_file_info: OutputFileInfo, } -impl PhysicalWriteOperator { +impl PhysicalWriterFactory { pub(crate) fn new(output_file_info: OutputFileInfo) -> Self { Self { output_file_info } } } -impl WriteOperator for PhysicalWriteOperator { - fn name(&self) -> &'static str { - "PhysicalWriteOperator" - } +impl WriterFactory for PhysicalWriterFactory { fn create_writer( &self, file_idx: usize, diff --git a/src/daft-local-execution/src/writes/unpartitioned_write.rs b/src/daft-local-execution/src/writes/unpartitioned_write.rs index ee221855ae..0788f83208 100644 --- a/src/daft-local-execution/src/writes/unpartitioned_write.rs +++ b/src/daft-local-execution/src/writes/unpartitioned_write.rs @@ -7,7 +7,7 @@ use daft_micropartition::{FileWriter, MicroPartition}; use daft_table::Table; use snafu::ResultExt; -use super::WriteOperator; +use super::WriterFactory; use crate::{ buffer::RowBasedBuffer, channel::{create_channel, PipelineChannel, Receiver, Sender}, @@ -18,9 +18,10 @@ use crate::{ }; pub(crate) struct UnpartitionedWriteNode { + name: &'static str, child: Box, runtime_stats: Arc, - write_operator: Arc, + writer_factory: Arc, target_in_memory_file_rows: usize, target_in_memory_chunk_rows: usize, file_schema: SchemaRef, @@ -28,16 +29,18 @@ pub(crate) struct UnpartitionedWriteNode { impl UnpartitionedWriteNode { pub(crate) fn new( + name: &'static str, child: Box, - write_operator: Arc, + writer_factory: Arc, target_in_memory_file_rows: usize, target_in_memory_chunk_rows: usize, file_schema: SchemaRef, ) -> Self { Self { + name, child, runtime_stats: RuntimeStatsContext::new(), - write_operator, + writer_factory, target_in_memory_file_rows, target_in_memory_chunk_rows, file_schema, @@ -48,9 +51,12 @@ impl UnpartitionedWriteNode { Box::new(self) } + // Receives data from the dispatcher and writes it. + // If the received file idx is different from the current file idx, this means that the current file is full and needs to be closed. + // Once input is exhausted, the current writer is closed and all written file paths are returned. async fn run_writer( mut input_receiver: Receiver<(Arc, usize)>, - write_operator: Arc, + writer_factory: Arc, ) -> DaftResult> { let mut written_file_paths = vec![]; let mut current_writer: Option> = None; @@ -63,7 +69,7 @@ impl UnpartitionedWriteNode { } } current_file_idx = Some(file_idx); - current_writer = Some(write_operator.create_writer(file_idx, None)?); + current_writer = Some(writer_factory.create_writer(file_idx, None)?); } if let Some(writer) = current_writer.as_mut() { writer.write(&data)?; @@ -80,18 +86,22 @@ impl UnpartitionedWriteNode { fn spawn_writers( num_writers: usize, task_set: &mut TaskSet>>, - write_operator: &Arc, + writer_factory: &Arc, channel_size: usize, ) -> Vec, usize)>> { let mut writer_senders = Vec::with_capacity(num_writers); for _ in 0..num_writers { let (writer_sender, writer_receiver) = create_channel(channel_size); - task_set.spawn(Self::run_writer(writer_receiver, write_operator.clone())); + task_set.spawn(Self::run_writer(writer_receiver, writer_factory.clone())); writer_senders.push(writer_sender); } writer_senders } + // Dispatches data received from the child to the writers + // As data is received, it is buffered until enough data is available to fill a chunk + // Once a chunk is filled, it is sent to a writer + // If the writer has written enough rows for a file, increment the file index and switch to the next writer async fn dispatch( mut input_receiver: CountingReceiver, target_chunk_rows: usize, @@ -157,7 +167,7 @@ impl PipelineNode for UnpartitionedWriteNode { } fn name(&self) -> &'static str { - self.write_operator.name() + self.name } fn start( @@ -177,12 +187,13 @@ impl PipelineNode for UnpartitionedWriteNode { destination_channel.get_next_sender_with_stats(&self.runtime_stats); // Start writers - let write_operator = self.write_operator.clone(); + let writer_factory = self.writer_factory.clone(); let mut task_set = create_task_set(); let writer_senders = Self::spawn_writers( *NUM_CPUS, &mut task_set, - &write_operator, + &writer_factory, + // The channel size is set to the number of chunks per file such that writes can be parallelized (self.target_in_memory_file_rows + self.target_in_memory_chunk_rows + 1) / self.target_in_memory_chunk_rows, ); diff --git a/tests/benchmarks/conftest.py b/tests/benchmarks/conftest.py index 9107225b94..3c49a2733f 100644 --- a/tests/benchmarks/conftest.py +++ b/tests/benchmarks/conftest.py @@ -67,10 +67,7 @@ def gen_tpch(request): num_parts = request.param csv_files_location = data_generation.gen_csv_files(TPCH_DBGEN_DIR, num_parts, SCALE_FACTOR) - - # Disable native executor to generate parquet files, remove once native executor supports writing parquet files - with daft.context.execution_config_ctx(enable_native_executor=False): - parquet_files_location = data_generation.gen_parquet(csv_files_location) + parquet_files_location = data_generation.gen_parquet(csv_files_location) in_memory_tables = {} for tbl_name in data_generation.SCHEMA.keys(): From 56c8416e20620004ed88214e9df31d034c6eebec Mon Sep 17 00:00:00 2001 From: Colin Ho Date: Mon, 7 Oct 2024 12:33:49 -0500 Subject: [PATCH 04/15] dont use table str for hash --- src/daft-local-execution/src/lib.rs | 1 + .../src/writes/partitioned_write.rs | 92 +++++++++++++------ src/daft-table/src/lib.rs | 17 ++++ 3 files changed, 84 insertions(+), 26 deletions(-) diff --git a/src/daft-local-execution/src/lib.rs b/src/daft-local-execution/src/lib.rs index 42f3610f25..05c7f669de 100644 --- a/src/daft-local-execution/src/lib.rs +++ b/src/daft-local-execution/src/lib.rs @@ -1,4 +1,5 @@ #![feature(let_chains)] +#![feature(hash_raw_entry)] mod buffer; mod channel; mod intermediate_ops; diff --git a/src/daft-local-execution/src/writes/partitioned_write.rs b/src/daft-local-execution/src/writes/partitioned_write.rs index 2288525530..8b1cdd5e07 100644 --- a/src/daft-local-execution/src/writes/partitioned_write.rs +++ b/src/daft-local-execution/src/writes/partitioned_write.rs @@ -1,8 +1,15 @@ -use std::{collections::HashMap, sync::Arc}; +use std::{ + collections::{hash_map::RawEntryMut, HashMap}, + hash::{Hash, Hasher}, + sync::Arc, +}; use common_display::tree::TreeDisplay; use common_error::DaftResult; -use daft_core::prelude::{AsArrow as _, SchemaRef}; +use daft_core::{ + prelude::{AsArrow as _, SchemaRef}, + utils::identity_hash_set::IdentityBuildHasher, +}; use daft_dsl::ExprRef; use daft_io::IOStatsContext; use daft_micropartition::{FileWriter, MicroPartition}; @@ -19,6 +26,17 @@ use crate::{ ExecutionRuntimeHandle, JoinSnafu, NUM_CPUS, }; +pub struct IndexHash { + pub idx: u64, + pub hash: u64, +} + +impl Hash for IndexHash { + fn hash(&self, state: &mut H) { + state.write_u64(self.hash) + } +} + struct PerPartitionWriter { writer: Box, writer_factory: Arc, @@ -144,34 +162,56 @@ impl PartitionedWriteNode { target_chunk_rows: usize, target_file_rows: usize, ) -> DaftResult> { - let mut per_partition_writers = HashMap::new(); + let mut per_partition_writers = + HashMap::::with_capacity_and_hasher( + 20, + Default::default(), + ); + let mut saved_partition_values = vec![]; while let Some((data, partition_values)) = input_receiver.recv().await { - let partition_values_str = partition_values.to_string(); // TODO (Colin): Figure out how to map a partition to a writer without using String as key - let per_partition_writer = if !per_partition_writers.contains_key(&partition_values_str) - { - per_partition_writers.insert( - partition_values_str.clone(), - PerPartitionWriter::new( + assert!(partition_values.len() == 1); + let hash = partition_values.hash_rows()?.get(0).unwrap(); + let entry = per_partition_writers + .raw_entry_mut() + .from_hash(hash, |other| { + (hash == other.hash) && { + let other_table = saved_partition_values.get(other.idx as usize).unwrap(); + other_table == &partition_values + } + }); + match entry { + RawEntryMut::Vacant(entry) => { + let mut new_partition_writer = PerPartitionWriter::new( writer_factory.clone(), - partition_values, + partition_values.clone(), target_file_rows, target_chunk_rows, - )?, - ); - per_partition_writers - .get_mut(&partition_values_str) - .unwrap() - } else { - per_partition_writers - .get_mut(&partition_values_str) - .unwrap() - }; - - per_partition_writer.submit(&Arc::new(MicroPartition::new_loaded( - data.schema.clone(), - vec![data].into(), - None, - )))? + )?; + new_partition_writer.submit(&Arc::new(MicroPartition::new_loaded( + data.schema.clone(), + vec![data].into(), + None, + )))?; + entry.insert_hashed_nocheck( + hash, + IndexHash { + idx: saved_partition_values.len() as u64, + hash, + }, + new_partition_writer, + ); + saved_partition_values.push(partition_values); + } + RawEntryMut::Occupied(mut entry) => { + entry + .get_mut() + .submit(&Arc::new(MicroPartition::new_loaded( + data.schema.clone(), + vec![data].into(), + None, + )))?; + } + } } let mut results = vec![]; diff --git a/src/daft-table/src/lib.rs b/src/daft-table/src/lib.rs index 3669fda3f5..e557beb8a3 100644 --- a/src/daft-table/src/lib.rs +++ b/src/daft-table/src/lib.rs @@ -773,6 +773,23 @@ impl Table { } } +impl PartialEq for Table { + fn eq(&self, other: &Self) -> bool { + if self.schema != other.schema { + return false; + } + if self.len() != other.len() { + return false; + } + for (lhs, rhs) in self.columns.iter().zip(other.columns.iter()) { + if lhs != rhs { + return false; + } + } + true + } +} + impl Display for Table { // `f` is a buffer, and this method must write the formatted string into it fn fmt(&self, f: &mut Formatter) -> Result { From b662ad98413630c27927529bef30067196ff9113 Mon Sep 17 00:00:00 2001 From: Colin Ho Date: Wed, 9 Oct 2024 17:10:35 -0700 Subject: [PATCH 05/15] cleanup --- Cargo.lock | 1 + src/daft-core/src/utils/identity_hash_set.rs | 13 ++- src/daft-local-execution/Cargo.toml | 1 + src/daft-local-execution/src/buffer.rs | 8 ++ .../src/writes/partitioned_write.rs | 92 +++++++++---------- src/daft-table/src/ops/hash.rs | 18 +--- src/daft-table/src/probeable/probe_set.rs | 4 +- src/daft-table/src/probeable/probe_table.rs | 4 +- 8 files changed, 71 insertions(+), 70 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index 7bdd87958d..fc44711692 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -1969,6 +1969,7 @@ dependencies = [ "daft-table", "futures", "indexmap 2.5.0", + "itertools 0.11.0", "lazy_static", "log", "num-format", diff --git a/src/daft-core/src/utils/identity_hash_set.rs b/src/daft-core/src/utils/identity_hash_set.rs index cadff5f774..2258c31ff2 100644 --- a/src/daft-core/src/utils/identity_hash_set.rs +++ b/src/daft-core/src/utils/identity_hash_set.rs @@ -1,4 +1,4 @@ -use std::hash::{BuildHasherDefault, Hasher}; +use std::hash::{BuildHasherDefault, Hash, Hasher}; pub type IdentityBuildHasher = BuildHasherDefault; @@ -27,3 +27,14 @@ impl Hasher for IdentityHasher { self.hash = i; } } + +pub struct IndexHash { + pub idx: u64, + pub hash: u64, +} + +impl Hash for IndexHash { + fn hash(&self, state: &mut H) { + state.write_u64(self.hash) + } +} diff --git a/src/daft-local-execution/Cargo.toml b/src/daft-local-execution/Cargo.toml index cd061c1c35..074c20bd59 100644 --- a/src/daft-local-execution/Cargo.toml +++ b/src/daft-local-execution/Cargo.toml @@ -17,6 +17,7 @@ daft-scan = {path = "../daft-scan", default-features = false} daft-table = {path = "../daft-table", default-features = false} futures = {workspace = true} indexmap = {workspace = true} +itertools = {workspace = true} lazy_static = {workspace = true} log = {workspace = true} num-format = "0.4.4" diff --git a/src/daft-local-execution/src/buffer.rs b/src/daft-local-execution/src/buffer.rs index 391e4ab604..5b3cc60f3b 100644 --- a/src/daft-local-execution/src/buffer.rs +++ b/src/daft-local-execution/src/buffer.rs @@ -3,6 +3,7 @@ use std::{cmp::Ordering::*, collections::VecDeque, sync::Arc}; use common_error::DaftResult; use daft_micropartition::MicroPartition; +// A buffer that accumulates morsels until a threshold is reached pub struct RowBasedBuffer { pub buffer: VecDeque>, pub curr_len: usize, @@ -19,11 +20,17 @@ impl RowBasedBuffer { } } + // Push a morsel to the buffer pub fn push(&mut self, part: Arc) { self.curr_len += part.len(); self.buffer.push_back(part); } + // Pop enough morsels that reach the threshold + // - If the buffer currently has not enough morsels, return None + // - If the buffer has exactly enough morsels, return the morsels + // - If the buffer has more than enough morsels, return a vec of morsels, each correctly sized to the threshold. + // The remaining morsels will be pushed back to the buffer pub fn pop_enough(&mut self) -> DaftResult>>> { match self.curr_len.cmp(&self.threshold) { Less => Ok(None), @@ -71,6 +78,7 @@ impl RowBasedBuffer { } } + // Pop all morsels in the buffer regardless of the threshold pub fn pop_all(&mut self) -> DaftResult>> { assert!(self.curr_len < self.threshold); if self.buffer.is_empty() { diff --git a/src/daft-local-execution/src/writes/partitioned_write.rs b/src/daft-local-execution/src/writes/partitioned_write.rs index 8b1cdd5e07..3589c8b25d 100644 --- a/src/daft-local-execution/src/writes/partitioned_write.rs +++ b/src/daft-local-execution/src/writes/partitioned_write.rs @@ -1,6 +1,5 @@ use std::{ collections::{hash_map::RawEntryMut, HashMap}, - hash::{Hash, Hasher}, sync::Arc, }; @@ -8,12 +7,13 @@ use common_display::tree::TreeDisplay; use common_error::DaftResult; use daft_core::{ prelude::{AsArrow as _, SchemaRef}, - utils::identity_hash_set::IdentityBuildHasher, + utils::identity_hash_set::{IdentityBuildHasher, IndexHash}, }; use daft_dsl::ExprRef; use daft_io::IOStatsContext; use daft_micropartition::{FileWriter, MicroPartition}; use daft_table::Table; +use itertools::Itertools; use snafu::ResultExt; use super::WriterFactory; @@ -26,19 +26,8 @@ use crate::{ ExecutionRuntimeHandle, JoinSnafu, NUM_CPUS, }; -pub struct IndexHash { - pub idx: u64, - pub hash: u64, -} - -impl Hash for IndexHash { - fn hash(&self, state: &mut H) { - state.write_u64(self.hash) - } -} - -struct PerPartitionWriter { - writer: Box, +struct PerPartitionWriteHandler { + current_writer: Box, writer_factory: Arc, partition_values: Table, buffer: RowBasedBuffer, @@ -47,7 +36,7 @@ struct PerPartitionWriter { results: Vec
, } -impl PerPartitionWriter { +impl PerPartitionWriteHandler { fn new( writer_factory: Arc, partition_values: Table, @@ -55,7 +44,7 @@ impl PerPartitionWriter { target_chunk_rows: usize, ) -> DaftResult { Ok(Self { - writer: writer_factory.create_writer(0, Some(&partition_values))?, + current_writer: writer_factory.create_writer(0, Some(&partition_values))?, writer_factory, partition_values, buffer: RowBasedBuffer::new(target_chunk_rows), @@ -69,25 +58,25 @@ impl PerPartitionWriter { self.buffer.push(data.clone()); if let Some(ready) = self.buffer.pop_enough()? { for part in ready { - self.write(&part)?; + self.handle_write(&part)?; } } Ok(()) } - fn write(&mut self, data: &Arc) -> DaftResult<()> { + fn handle_write(&mut self, data: &Arc) -> DaftResult<()> { let len = data.len(); - self.writer.write(data)?; + self.current_writer.write(data)?; self.written_rows_so_far += len; // Check if the file is full, close and start a new file if necessary if self.written_rows_so_far >= self.target_file_rows { - let result = self.writer.close()?; + let result = self.current_writer.close()?; if let Some(result) = result { self.results.push(result); } self.written_rows_so_far = 0; - self.writer = self + self.current_writer = self .writer_factory .create_writer(self.results.len(), Some(&self.partition_values))? } @@ -98,11 +87,11 @@ impl PerPartitionWriter { // Write any remaining data from the buffer let remaining = self.buffer.pop_all()?; if let Some(part) = remaining { - self.write(&part)?; + self.handle_write(&part)?; } // Finalize the current file and collect results - if let Some(result) = self.writer.close()? { + if let Some(result) = self.current_writer.close()? { self.results.push(result); } Ok(std::mem::take(&mut self.results)) @@ -146,27 +135,20 @@ impl PartitionedWriteNode { Box::new(self) } - fn partition( - partition_cols: &[ExprRef], - data: &Arc, - ) -> DaftResult<(Vec
, Table)> { - let data = data.concat_or_get(IOStatsContext::new("MicroPartition::partition_by_value"))?; - let table = data.first().unwrap(); - let (split_tables, partition_values) = table.partition_by_value(partition_cols)?; - Ok((split_tables, partition_values)) - } - - async fn run_writer( + // Receives data from the dispatcher and writes it. + // Each partition value is mapped to a PerPartitionWriteHandler, which handles the write. + // Each worker is responsible for a subset of partition values. + async fn run_worker( mut input_receiver: Receiver<(Table, Table)>, writer_factory: Arc, target_chunk_rows: usize, target_file_rows: usize, ) -> DaftResult> { - let mut per_partition_writers = - HashMap::::with_capacity_and_hasher( - 20, - Default::default(), - ); + let mut per_partition_writers = HashMap::< + IndexHash, + PerPartitionWriteHandler, + IdentityBuildHasher, + >::with_capacity_and_hasher(20, Default::default()); let mut saved_partition_values = vec![]; while let Some((data, partition_values)) = input_receiver.recv().await { assert!(partition_values.len() == 1); @@ -181,13 +163,13 @@ impl PartitionedWriteNode { }); match entry { RawEntryMut::Vacant(entry) => { - let mut new_partition_writer = PerPartitionWriter::new( + let mut new_partition_write_handler = PerPartitionWriteHandler::new( writer_factory.clone(), partition_values.clone(), target_file_rows, target_chunk_rows, )?; - new_partition_writer.submit(&Arc::new(MicroPartition::new_loaded( + new_partition_write_handler.submit(&Arc::new(MicroPartition::new_loaded( data.schema.clone(), vec![data].into(), None, @@ -198,7 +180,7 @@ impl PartitionedWriteNode { idx: saved_partition_values.len() as u64, hash, }, - new_partition_writer, + new_partition_write_handler, ); saved_partition_values.push(partition_values); } @@ -214,11 +196,11 @@ impl PartitionedWriteNode { } } - let mut results = vec![]; - for writer in per_partition_writers.values_mut() { - let res = writer.finalize()?; - results.extend(res); - } + let results = per_partition_writers + .values_mut() + .map(|writer| writer.finalize()) + .flatten_ok() + .collect::>>()?; Ok(results) } @@ -232,7 +214,7 @@ impl PartitionedWriteNode { let mut writer_senders = Vec::with_capacity(num_writers); for _ in 0..num_writers { let (writer_sender, writer_receiver) = create_channel(1); - task_set.spawn(Self::run_writer( + task_set.spawn(Self::run_worker( writer_receiver, writer_factory.clone(), target_chunk_rows, @@ -243,6 +225,18 @@ impl PartitionedWriteNode { writer_senders } + fn partition( + partition_cols: &[ExprRef], + data: &Arc, + ) -> DaftResult<(Vec
, Table)> { + let data = data.concat_or_get(IOStatsContext::new("MicroPartition::partition_by_value"))?; + let table = data.first().unwrap(); + let (split_tables, partition_values) = table.partition_by_value(partition_cols)?; + Ok((split_tables, partition_values)) + } + + // Dispatches data received from the child to the writers + // Partition the data by the partition columns, hash the partition values, and send to the appropriate writer based on the hash async fn dispatch( mut input_receiver: CountingReceiver, senders: Vec>, diff --git a/src/daft-table/src/ops/hash.rs b/src/daft-table/src/ops/hash.rs index 0abdcb8867..4b4d120da0 100644 --- a/src/daft-table/src/ops/hash.rs +++ b/src/daft-table/src/ops/hash.rs @@ -1,28 +1,14 @@ -use std::{ - collections::{hash_map::RawEntryMut, HashMap}, - hash::{Hash, Hasher}, -}; +use std::collections::{hash_map::RawEntryMut, HashMap}; use common_error::{DaftError, DaftResult}; use daft_core::{ array::ops::{arrow2::comparison::build_multi_array_is_equal, as_arrow::AsArrow}, datatypes::UInt64Array, - utils::identity_hash_set::IdentityBuildHasher, + utils::identity_hash_set::{IdentityBuildHasher, IndexHash}, }; use crate::Table; -pub struct IndexHash { - pub idx: u64, - pub hash: u64, -} - -impl Hash for IndexHash { - fn hash(&self, state: &mut H) { - state.write_u64(self.hash) - } -} - impl Table { pub fn hash_rows(&self) -> DaftResult { if self.num_columns() == 0 { diff --git a/src/daft-table/src/probeable/probe_set.rs b/src/daft-table/src/probeable/probe_set.rs index 0fdff1e0fc..b29cc89b2c 100644 --- a/src/daft-table/src/probeable/probe_set.rs +++ b/src/daft-table/src/probeable/probe_set.rs @@ -9,12 +9,12 @@ use daft_core::{ prelude::SchemaRef, utils::{ dyn_compare::{build_dyn_multi_array_compare, MultiDynArrayComparator}, - identity_hash_set::IdentityBuildHasher, + identity_hash_set::{IdentityBuildHasher, IndexHash}, }, }; use super::{ArrowTableEntry, IndicesMapper, Probeable, ProbeableBuilder}; -use crate::{ops::hash::IndexHash, Table}; +use crate::Table; pub(crate) struct ProbeSet { schema: SchemaRef, hash_table: HashMap, diff --git a/src/daft-table/src/probeable/probe_table.rs b/src/daft-table/src/probeable/probe_table.rs index c8e401084f..65487671d9 100644 --- a/src/daft-table/src/probeable/probe_table.rs +++ b/src/daft-table/src/probeable/probe_table.rs @@ -9,12 +9,12 @@ use daft_core::{ prelude::SchemaRef, utils::{ dyn_compare::{build_dyn_multi_array_compare, MultiDynArrayComparator}, - identity_hash_set::IdentityBuildHasher, + identity_hash_set::{IdentityBuildHasher, IndexHash}, }, }; use super::{ArrowTableEntry, IndicesMapper, Probeable, ProbeableBuilder}; -use crate::{ops::hash::IndexHash, Table}; +use crate::Table; pub(crate) struct ProbeTable { schema: SchemaRef, From 9ed8e84e6e9a0298bd1bba877fd6668df3e9abc9 Mon Sep 17 00:00:00 2001 From: Colin Ho Date: Mon, 28 Oct 2024 18:16:07 -0700 Subject: [PATCH 06/15] feedback --- Cargo.lock | 39 ++ Cargo.toml | 2 + daft/io/writer.py | 61 ++- src/daft-io/src/lib.rs | 1 + src/daft-local-execution/Cargo.toml | 2 + src/daft-local-execution/src/lib.rs | 1 - src/daft-local-execution/src/pipeline.rs | 78 +--- src/daft-local-execution/src/sinks/mod.rs | 1 + src/daft-local-execution/src/sinks/write.rs | 439 ++++++++++++++++++ src/daft-local-execution/src/writes/mod.rs | 15 - .../src/writes/partitioned_write.rs | 347 -------------- .../src/writes/physical_write.rs | 34 -- .../src/writes/unpartitioned_write.rs | 234 ---------- src/daft-micropartition/src/lib.rs | 40 +- src/daft-micropartition/src/py_writers.rs | 113 ----- src/daft-table/src/lib.rs | 4 +- src/daft-writers/Cargo.toml | 42 ++ src/daft-writers/src/lib.rs | 32 ++ src/daft-writers/src/physical.rs | 56 +++ src/daft-writers/src/python.rs | 141 ++++++ tests/io/test_parquet_roundtrip.py | 7 - 21 files changed, 816 insertions(+), 873 deletions(-) create mode 100644 src/daft-local-execution/src/sinks/write.rs delete mode 100644 src/daft-local-execution/src/writes/mod.rs delete mode 100644 src/daft-local-execution/src/writes/partitioned_write.rs delete mode 100644 src/daft-local-execution/src/writes/physical_write.rs delete mode 100644 src/daft-local-execution/src/writes/unpartitioned_write.rs delete mode 100644 src/daft-micropartition/src/py_writers.rs create mode 100644 src/daft-writers/Cargo.toml create mode 100644 src/daft-writers/src/lib.rs create mode 100644 src/daft-writers/src/physical.rs create mode 100644 src/daft-writers/src/python.rs diff --git a/Cargo.lock b/Cargo.lock index c341162c2c..a275ca49e3 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -1710,6 +1710,7 @@ dependencies = [ "daft-sql", "daft-stats", "daft-table", + "daft-writers", "lazy_static", "libc", "log", @@ -1972,6 +1973,7 @@ dependencies = [ name = "daft-local-execution" version = "0.3.0-dev0" dependencies = [ + "async-trait", "common-daft-config", "common-display", "common-error", @@ -1990,6 +1992,7 @@ dependencies = [ "daft-plan", "daft-scan", "daft-table", + "daft-writers", "futures", "indexmap 2.5.0", "itertools 0.11.0", @@ -2246,6 +2249,42 @@ dependencies = [ "serde", ] +[[package]] +name = "daft-writers" +version = "0.3.0-dev0" +dependencies = [ + "async-trait", + "common-daft-config", + "common-display", + "common-error", + "common-file-formats", + "common-runtime", + "common-tracing", + "daft-core", + "daft-csv", + "daft-dsl", + "daft-functions", + "daft-io", + "daft-json", + "daft-micropartition", + "daft-parquet", + "daft-physical-plan", + "daft-plan", + "daft-scan", + "daft-table", + "futures", + "indexmap 2.5.0", + "itertools 0.11.0", + "lazy_static", + "log", + "num-format", + "pyo3", + "snafu", + "tokio", + "tokio-stream", + "tracing", +] + [[package]] name = "deflate64" version = "0.1.9" diff --git a/Cargo.toml b/Cargo.toml index 897ac46dd8..cac465eb08 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -28,6 +28,7 @@ daft-scheduler = {path = "src/daft-scheduler", default-features = false} daft-sql = {path = "src/daft-sql", default-features = false} daft-stats = {path = "src/daft-stats", default-features = false} daft-table = {path = "src/daft-table", default-features = false} +daft-writers = {path = "src/daft-writers", default-features = false} lazy_static = {workspace = true} log = {workspace = true} lzma-sys = {version = "*", features = ["static"]} @@ -140,6 +141,7 @@ members = [ "src/daft-functions", "src/daft-functions-json", "src/daft-sql", + "src/daft-writers", "src/hyperloglog" ] diff --git a/daft/io/writer.py b/daft/io/writer.py index e507d168e7..c9b7667805 100644 --- a/daft/io/writer.py +++ b/daft/io/writer.py @@ -1,7 +1,8 @@ import uuid +from abc import ABC, abstractmethod from typing import Dict, Optional, Union -from daft.daft import IOConfig, PyTable +from daft.daft import IOConfig from daft.dependencies import pa, pacsv, pq from daft.filesystem import ( _resolve_paths_and_filesystem, @@ -16,7 +17,9 @@ def partition_values_to_str_mapping( partition_values: Table, ) -> Dict[str, str]: - null_part = Series.from_pylist([None]) + null_part = Series.from_pylist( + [None] + ) # This is to ensure that the null values are replaced with the default_partition_fallback value pkey_names = partition_values.column_names() partition_strings = {} @@ -40,13 +43,13 @@ def partition_string_mapping_to_postfix( return postfix -class FileWriterBase: +class FileWriterBase(ABC): def __init__( self, root_dir: str, file_idx: int, file_format: str, - partition_values: Optional[PyTable] = None, + partition_values: Optional[Table] = None, compression: Optional[str] = None, io_config: Optional[IOConfig] = None, default_partition_fallback: str = "__HIVE_DEFAULT_PARTITION__", @@ -57,7 +60,7 @@ def __init__( is_local_fs = canonicalized_protocol == "file" self.file_name = f"{uuid.uuid4()}-{file_idx}.{file_format}" - self.partition_values = Table._from_pytable(partition_values) if partition_values is not None else None + self.partition_values = partition_values if self.partition_values is not None: partition_strings = partition_values_to_str_mapping(self.partition_values) postfix = partition_string_mapping_to_postfix(partition_strings, default_partition_fallback) @@ -72,15 +75,34 @@ def __init__( self.compression = compression if compression is not None else "none" self.current_writer: Optional[Union[pq.ParquetWriter, pacsv.CSVWriter]] = None - def _create_writer(self, schema: pa.Schema): - raise NotImplementedError("Subclasses must implement this method.") + @abstractmethod + def _create_writer(self, schema: pa.Schema) -> Union[pq.ParquetWriter, pacsv.CSVWriter]: + """Create a writer instance for the specific file format. - def write(self, table: MicroPartition): + Args: + schema: PyArrow schema defining the structure of the data to be written. + + Returns: + A writer instance specific to the file format (Parquet or CSV). + """ + pass + + def write(self, table: MicroPartition) -> None: + """Write data to the file using the appropriate writer. + + Args: + table: MicroPartition containing the data to be written. + """ if self.current_writer is None: self.current_writer = self._create_writer(table.schema().to_pyarrow_schema()) self.current_writer.write_table(table.to_arrow()) - def close(self) -> PyTable: + def close(self) -> Table: + """Close the writer and return metadata about the written file. + + Returns: + Table containing metadata about the written file, including path and partition values. + """ if self.current_writer is not None: self.current_writer.close() @@ -88,7 +110,7 @@ def close(self) -> PyTable: if self.partition_values is not None: for col_name in self.partition_values.column_names(): metadata[col_name] = self.partition_values.get_column(col_name) - return Table.from_pydict(metadata)._table + return Table.from_pydict(metadata) class ParquetFileWriter(FileWriterBase): @@ -96,11 +118,18 @@ def __init__( self, root_dir: str, file_idx: int, - partition_values: Optional[PyTable] = None, + partition_values: Optional[Table] = None, compression: str = "none", io_config: Optional[IOConfig] = None, ): - super().__init__(root_dir, file_idx, "parquet", partition_values, compression, io_config) + super().__init__( + root_dir=root_dir, + file_idx=file_idx, + file_format="parquet", + partition_values=partition_values, + compression=compression, + io_config=io_config, + ) def _create_writer(self, schema: pa.Schema) -> pq.ParquetWriter: return pq.ParquetWriter( @@ -117,13 +146,13 @@ def __init__( self, root_dir: str, file_idx: int, - partition_values: Optional[PyTable] = None, + partition_values: Optional[Table] = None, io_config: Optional[IOConfig] = None, ): super().__init__( - root_dir, - file_idx, - "csv", + root_dir=root_dir, + file_idx=file_idx, + file_format="csv", partition_values=partition_values, io_config=io_config, ) diff --git a/src/daft-io/src/lib.rs b/src/daft-io/src/lib.rs index 63d1d08485..923be570ba 100644 --- a/src/daft-io/src/lib.rs +++ b/src/daft-io/src/lib.rs @@ -12,6 +12,7 @@ mod object_store_glob; mod s3_like; mod stats; mod stream_utils; + use azure_blob::AzureBlobSource; use common_file_formats::FileFormat; use google_cloud::GCSSource; diff --git a/src/daft-local-execution/Cargo.toml b/src/daft-local-execution/Cargo.toml index 27c78bd3c0..877cec27dd 100644 --- a/src/daft-local-execution/Cargo.toml +++ b/src/daft-local-execution/Cargo.toml @@ -1,4 +1,5 @@ [dependencies] +async-trait = {workspace = true} common-daft-config = {path = "../common/daft-config", default-features = false} common-display = {path = "../common/display", default-features = false} common-error = {path = "../common/error", default-features = false} @@ -17,6 +18,7 @@ daft-physical-plan = {path = "../daft-physical-plan", default-features = false} daft-plan = {path = "../daft-plan", default-features = false} daft-scan = {path = "../daft-scan", default-features = false} daft-table = {path = "../daft-table", default-features = false} +daft-writers = {path = "../daft-writers", default-features = false} futures = {workspace = true} indexmap = {workspace = true} itertools = {workspace = true} diff --git a/src/daft-local-execution/src/lib.rs b/src/daft-local-execution/src/lib.rs index 957adac380..3f91c58955 100644 --- a/src/daft-local-execution/src/lib.rs +++ b/src/daft-local-execution/src/lib.rs @@ -8,7 +8,6 @@ mod run; mod runtime_stats; mod sinks; mod sources; -mod writes; use common_error::{DaftError, DaftResult}; use lazy_static::lazy_static; diff --git a/src/daft-local-execution/src/pipeline.rs b/src/daft-local-execution/src/pipeline.rs index e4b3c9cf0d..49fab6c385 100644 --- a/src/daft-local-execution/src/pipeline.rs +++ b/src/daft-local-execution/src/pipeline.rs @@ -1,9 +1,8 @@ -use std::{cmp::min, collections::HashMap, sync::Arc}; +use std::{collections::HashMap, sync::Arc}; use common_daft_config::DaftExecutionConfig; use common_display::{mermaid::MermaidDisplayVisitor, tree::TreeDisplay}; use common_error::DaftResult; -use common_file_formats::FileFormat; use daft_core::{ datatypes::Field, prelude::{Schema, SchemaRef}, @@ -33,13 +32,9 @@ use crate::{ aggregate::AggregateSink, blocking_sink::BlockingSinkNode, concat::ConcatSink, hash_join_build::HashJoinBuildSink, limit::LimitSink, outer_hash_join_probe::OuterHashJoinProbeSink, sort::SortSink, - streaming_sink::StreamingSinkNode, + streaming_sink::StreamingSinkNode, write::WriteSink, }, sources::{empty_scan::EmptyScanSource, in_memory::InMemorySource}, - writes::{ - partitioned_write::PartitionedWriteNode, physical_write::PhysicalWriterFactory, - unpartitioned_write::UnpartitionedWriteNode, - }, ExecutionRuntimeHandle, PipelineCreationSnafu, }; @@ -417,64 +412,17 @@ pub fn physical_plan_to_pipeline( .. }) => { let child_node = physical_plan_to_pipeline(input, psets, cfg)?; - let estimated_row_size_bytes = data_schema.estimate_row_size_bytes(); - let (inflation_factor, target_file_size, target_chunk_size) = - match file_info.file_format { - FileFormat::Parquet => ( - cfg.parquet_inflation_factor, - cfg.parquet_target_filesize, - cfg.parquet_target_row_group_size, - ), - FileFormat::Csv => ( - cfg.csv_inflation_factor, - cfg.csv_target_filesize, - cfg.parquet_target_row_group_size, // Just assume same chunk size for CSV and Parquet for now - ), - _ => unreachable!("Physical write should only support Parquet and CSV"), - }; - let file_size = target_file_size as f64 * inflation_factor; - let target_file_rows = if estimated_row_size_bytes > 0.0 { - file_size / estimated_row_size_bytes - } else { - file_size - } as usize; - let target_chunk_rows = min( - target_file_rows, - if estimated_row_size_bytes > 0.0 { - target_chunk_size as f64 / estimated_row_size_bytes - } else { - target_chunk_size as f64 - } as usize, - ); - let write_factory = PhysicalWriterFactory::new(file_info.clone()); - let name = match (&file_info.partition_cols.is_some(), &file_info.file_format) { - (true, FileFormat::Parquet) => "PartitionedParquetWrite", - (true, FileFormat::Csv) => "PartitionedCSVWrite", - (false, FileFormat::Parquet) => "UnpartitionedParquetWrite", - (false, FileFormat::Csv) => "UnpartitionedCSVWrite", - _ => unreachable!("Physical write should only support Parquet and CSV"), - }; - match &file_info.partition_cols { - Some(part_cols) => PartitionedWriteNode::new( - name, - child_node, - Arc::new(write_factory), - part_cols.clone(), - target_file_rows, - target_chunk_rows, - file_schema.clone(), - ) - .boxed(), - None => UnpartitionedWriteNode::new( - name, - child_node, - Arc::new(write_factory), - target_file_rows, - target_chunk_rows, - file_schema.clone(), - ) - .boxed(), - } + let write_sink = WriteSink::new( + file_info.clone(), + data_schema.clone(), + file_schema.clone(), + cfg, + ) + .context(PipelineCreationSnafu { + plan_name: physical_plan.name(), + })? + .boxed(); + BlockingSinkNode::new(write_sink, child_node).boxed() } }; diff --git a/src/daft-local-execution/src/sinks/mod.rs b/src/daft-local-execution/src/sinks/mod.rs index 7960e55a7c..64366385c3 100644 --- a/src/daft-local-execution/src/sinks/mod.rs +++ b/src/daft-local-execution/src/sinks/mod.rs @@ -6,3 +6,4 @@ pub mod limit; pub mod outer_hash_join_probe; pub mod sort; pub mod streaming_sink; +pub mod write; diff --git a/src/daft-local-execution/src/sinks/write.rs b/src/daft-local-execution/src/sinks/write.rs new file mode 100644 index 0000000000..296e80ad3e --- /dev/null +++ b/src/daft-local-execution/src/sinks/write.rs @@ -0,0 +1,439 @@ +use std::{ + cmp::min, + collections::{hash_map::RawEntryMut, HashMap}, + sync::Arc, +}; + +use common_daft_config::DaftExecutionConfig; +use common_error::DaftResult; +use common_file_formats::FileFormat; +use daft_core::{ + prelude::{AsArrow, SchemaRef}, + utils::identity_hash_set::IndexHash, +}; +use daft_dsl::ExprRef; +use daft_io::IOStatsContext; +use daft_micropartition::MicroPartition; +use daft_plan::OutputFileInfo; +use daft_table::Table; +use daft_writers::{FileWriter, PhysicalWriterFactory, WriterFactory}; +use tracing::instrument; + +use super::blocking_sink::{BlockingSink, BlockingSinkStatus}; +use crate::{buffer::RowBasedBuffer, pipeline::PipelineResultType}; + +struct TargetBatchWriter { + buffer: RowBasedBuffer, + writer: Box, Result = Option
>>, +} + +impl TargetBatchWriter { + pub fn new( + target_in_memory_chunk_rows: usize, + writer: Box, Result = Option
>>, + ) -> Self { + Self { + buffer: RowBasedBuffer::new(target_in_memory_chunk_rows), + writer, + } + } +} + +impl FileWriter for TargetBatchWriter { + type Input = Arc; + type Result = Option
; + + fn write(&mut self, input: &Arc) -> DaftResult<()> { + self.buffer.push(input.clone()); + if let Some(ready) = self.buffer.pop_enough()? { + for r in ready { + self.writer.write(&r)?; + } + } + Ok(()) + } + + fn close(&mut self) -> DaftResult { + if let Some(ready) = self.buffer.pop_all()? { + self.writer.write(&ready)?; + } + self.writer.close() + } +} + +struct TargetBatchWriterFactory { + writer_factory: Arc>, + target_in_memory_chunk_rows: usize, +} + +impl WriterFactory for TargetBatchWriterFactory, Option
> { + type Input = Arc; + type Result = Option
; + + fn create_writer( + &self, + file_idx: usize, + partition_values: Option<&Table>, + ) -> DaftResult>> { + let writer = self + .writer_factory + .create_writer(file_idx, partition_values)?; + Ok(Box::new(TargetBatchWriter::new( + self.target_in_memory_chunk_rows, + writer, + )) + as Box< + dyn FileWriter, + >) + } +} + +struct TargetFileSizeWriter { + current_file_rows: usize, + current_writer: Box, Result = Option
>>, + writer_factory: Arc, Result = Option
>>, + target_in_memory_file_rows: usize, + results: Vec
, + partition_values: Option
, +} + +impl TargetFileSizeWriter { + pub fn new( + target_in_memory_file_rows: usize, + writer_factory: Arc, Result = Option
>>, + partition_values: Option
, + ) -> DaftResult { + let writer: Box, Result = Option
>> = + writer_factory.create_writer(0, partition_values.as_ref())?; + Ok(Self { + current_file_rows: 0, + current_writer: writer, + writer_factory, + target_in_memory_file_rows, + results: vec![], + partition_values, + }) + } + + fn rotate_writer(&mut self) -> DaftResult<()> { + if let Some(result) = self.current_writer.close()? { + self.results.push(result); + } + self.current_file_rows = 0; + self.current_writer = self + .writer_factory + .create_writer(self.results.len(), self.partition_values.as_ref())?; + Ok(()) + } +} + +impl FileWriter for TargetFileSizeWriter { + type Input = Arc; + type Result = Vec
; + + fn write(&mut self, input: &Arc) -> DaftResult<()> { + use std::cmp::Ordering; + match (input.len() + self.current_file_rows).cmp(&self.target_in_memory_file_rows) { + Ordering::Equal => { + self.current_writer.write(input)?; + self.rotate_writer()?; + } + Ordering::Greater => { + // Finish up the current writer first + let remaining_rows = self.target_in_memory_file_rows - self.current_file_rows; + let (to_write, mut remaining) = input.split_at(remaining_rows)?; + self.current_writer.write(&to_write.into())?; + self.rotate_writer()?; + + // Write as many full files as possible + let num_full_files = remaining.len() / self.target_in_memory_file_rows; + for _ in 0..num_full_files { + let (to_write, new_remaining) = + remaining.split_at(self.target_in_memory_file_rows)?; + self.current_writer.write(&to_write.into())?; + self.rotate_writer()?; + remaining = new_remaining; + } + + // Write the remaining rows + if !remaining.is_empty() { + self.current_file_rows = remaining.len(); + self.current_writer.write(&remaining.into())?; + } else { + self.current_file_rows = 0; + } + } + Ordering::Less => { + self.current_writer.write(input)?; + self.current_file_rows += input.len(); + } + } + Ok(()) + } + + fn close(&mut self) -> DaftResult { + if self.current_file_rows > 0 { + if let Some(result) = self.current_writer.close()? { + self.results.push(result); + } + } + Ok(std::mem::take(&mut self.results)) + } +} + +struct TargetFileSizeWriterFactory { + writer_factory: Arc, Result = Option
>>, + target_in_memory_file_rows: usize, +} + +impl WriterFactory for TargetFileSizeWriterFactory { + type Input = Arc; + type Result = Vec
; + + fn create_writer( + &self, + _file_idx: usize, + partition_values: Option<&Table>, + ) -> DaftResult>> { + Ok(Box::new(TargetFileSizeWriter::new( + self.target_in_memory_file_rows, + self.writer_factory.clone(), + partition_values.cloned(), + )?) + as Box< + dyn FileWriter, + >) + } +} + +struct PartitionedWriter { + per_partition_writers: + HashMap, Result = Vec
>>>, + saved_partition_values: Vec
, + writer_factory: Arc, Result = Vec
>>, + partition_by: Vec, +} + +impl PartitionedWriter { + pub fn new( + writer_factory: Arc, Result = Vec
>>, + partition_by: Vec, + ) -> Self { + Self { + per_partition_writers: HashMap::new(), + saved_partition_values: vec![], + writer_factory, + partition_by, + } + } + + fn partition( + partition_cols: &[ExprRef], + data: &Arc, + ) -> DaftResult<(Vec
, Table)> { + let data = data.concat_or_get(IOStatsContext::new("MicroPartition::partition_by_value"))?; + let table = data.first().unwrap(); + let (split_tables, partition_values) = table.partition_by_value(partition_cols)?; + Ok((split_tables, partition_values)) + } +} + +impl FileWriter for PartitionedWriter { + type Input = Arc; + type Result = Vec
; + + fn write(&mut self, input: &Arc) -> DaftResult<()> { + let (split_tables, partition_values) = + Self::partition(self.partition_by.as_slice(), input)?; + let partition_values_hash = partition_values.hash_rows()?; + for (idx, (table, partition_value_hash)) in split_tables + .into_iter() + .zip(partition_values_hash.as_arrow().values_iter()) + .enumerate() + { + let partition_value_row = partition_values.slice(idx, idx + 1)?; + let entry = self.per_partition_writers.raw_entry_mut().from_hash( + *partition_value_hash, + |other| { + (*partition_value_hash == other.hash) && { + let other_table = + self.saved_partition_values.get(other.idx as usize).unwrap(); + other_table == &partition_value_row + } + }, + ); + match entry { + RawEntryMut::Vacant(entry) => { + let mut writer = self + .writer_factory + .create_writer(0, Some(partition_value_row.as_ref()))?; + writer.write(&Arc::new(MicroPartition::new_loaded( + table.schema.clone(), + vec![table].into(), + None, + )))?; + entry.insert_hashed_nocheck( + *partition_value_hash, + IndexHash { + idx: self.saved_partition_values.len() as u64, + hash: *partition_value_hash, + }, + writer, + ); + self.saved_partition_values.push(partition_value_row); + } + RawEntryMut::Occupied(mut entry) => { + let writer = entry.get_mut(); + writer.write(&Arc::new(MicroPartition::new_loaded( + table.schema.clone(), + vec![table].into(), + None, + )))?; + } + } + } + Ok(()) + } + + fn close(&mut self) -> DaftResult { + let mut results = vec![]; + for (_, mut writer) in self.per_partition_writers.drain() { + results.extend(writer.close()?); + } + Ok(results) + } +} + +pub struct WriteSink { + name: &'static str, + writer: Box, Result = Vec
>>, + file_schema: SchemaRef, +} + +impl WriteSink { + pub fn new( + file_info: OutputFileInfo, + data_schema: SchemaRef, + file_schema: SchemaRef, + cfg: &DaftExecutionConfig, + ) -> DaftResult { + let estimated_row_size_bytes = data_schema.estimate_row_size_bytes(); + let base_writer_factory = PhysicalWriterFactory::new(file_info.clone()); + let (writer, name) = match file_info.file_format { + FileFormat::Parquet => { + let target_in_memory_file_size = + cfg.parquet_target_filesize as f64 * cfg.parquet_inflation_factor; + let target_in_memory_row_group_size = + cfg.parquet_target_row_group_size as f64 * cfg.parquet_inflation_factor; + + let target_file_rows = if estimated_row_size_bytes > 0.0 { + target_in_memory_file_size / estimated_row_size_bytes + } else { + target_in_memory_file_size + } as usize; + + let target_row_group_rows = min( + target_file_rows, + if estimated_row_size_bytes > 0.0 { + target_in_memory_row_group_size / estimated_row_size_bytes + } else { + target_in_memory_row_group_size + } as usize, + ); + + let row_group_writer_factory = TargetBatchWriterFactory { + writer_factory: Arc::new(base_writer_factory), + target_in_memory_chunk_rows: target_row_group_rows, + }; + + if let Some(partition_cols) = &file_info.partition_cols { + let file_writer_factory = TargetFileSizeWriterFactory { + writer_factory: Arc::new(row_group_writer_factory), + target_in_memory_file_rows: target_file_rows, + }; + let partitioned_writer = Box::new(PartitionedWriter::new( + Arc::new(file_writer_factory), + partition_cols.clone(), + )); + ( + partitioned_writer as Box>, + "PartitionedParquetWrite", + ) + } else { + ( + Box::new(TargetFileSizeWriter::new( + target_file_rows, + Arc::new(row_group_writer_factory), + None, + )?) as Box>, + "UnpartitionedParquetWrite", + ) + } + } + FileFormat::Csv => { + let target_in_memory_file_size = + cfg.csv_target_filesize as f64 * cfg.csv_inflation_factor; + let target_file_rows = if estimated_row_size_bytes > 0.0 { + target_in_memory_file_size / estimated_row_size_bytes + } else { + target_in_memory_file_size + } as usize; + + if let Some(partition_cols) = &file_info.partition_cols { + let file_writer_factory = TargetFileSizeWriterFactory { + writer_factory: Arc::new(base_writer_factory), + target_in_memory_file_rows: target_file_rows, + }; + let partitioned_writer = Box::new(PartitionedWriter::new( + Arc::new(file_writer_factory), + partition_cols.clone(), + )); + ( + partitioned_writer as Box>, + "PartitionedCsvWrite", + ) + } else { + ( + Box::new(TargetFileSizeWriter::new( + target_file_rows, + Arc::new(base_writer_factory), + None, + )?) as Box>, + "UnpartitionedCsvWrite", + ) + } + } + _ => unreachable!("Physical write should only support Parquet and CSV"), + }; + Ok(Self { + name, + writer, + file_schema, + }) + } + pub fn boxed(self) -> Box { + Box::new(self) + } +} + +impl BlockingSink for WriteSink { + #[instrument(skip_all, name = "WriteSink::sink")] + fn sink(&mut self, input: &Arc) -> DaftResult { + self.writer.write(input)?; + Ok(BlockingSinkStatus::NeedMoreInput) + } + + #[instrument(skip_all, name = "WriteSink::finalize")] + fn finalize(&mut self) -> DaftResult> { + let result = self.writer.close()?; + let mp = Arc::new(MicroPartition::new_loaded( + self.file_schema.clone(), + result.into(), + None, + )); + Ok(Some(mp.into())) + } + fn name(&self) -> &'static str { + self.name + } +} diff --git a/src/daft-local-execution/src/writes/mod.rs b/src/daft-local-execution/src/writes/mod.rs deleted file mode 100644 index 9ada93017d..0000000000 --- a/src/daft-local-execution/src/writes/mod.rs +++ /dev/null @@ -1,15 +0,0 @@ -use common_error::DaftResult; -use daft_micropartition::FileWriter; -use daft_table::Table; - -pub mod partitioned_write; -pub mod physical_write; -pub mod unpartitioned_write; - -pub trait WriterFactory: Send + Sync { - fn create_writer( - &self, - file_idx: usize, - partition_values: Option<&Table>, - ) -> DaftResult>; -} diff --git a/src/daft-local-execution/src/writes/partitioned_write.rs b/src/daft-local-execution/src/writes/partitioned_write.rs deleted file mode 100644 index fc75bf9501..0000000000 --- a/src/daft-local-execution/src/writes/partitioned_write.rs +++ /dev/null @@ -1,347 +0,0 @@ -use std::{ - collections::{hash_map::RawEntryMut, HashMap}, - sync::Arc, -}; - -use common_display::tree::TreeDisplay; -use common_error::DaftResult; -use daft_core::{ - prelude::{AsArrow as _, SchemaRef}, - utils::identity_hash_set::{IdentityBuildHasher, IndexHash}, -}; -use daft_dsl::ExprRef; -use daft_io::IOStatsContext; -use daft_micropartition::{FileWriter, MicroPartition}; -use daft_table::Table; -use itertools::Itertools; -use snafu::ResultExt; - -use super::WriterFactory; -use crate::{ - buffer::RowBasedBuffer, - channel::{create_channel, PipelineChannel, Receiver, Sender}, - pipeline::PipelineNode, - runtime_stats::{CountingReceiver, RuntimeStatsContext}, - ExecutionRuntimeHandle, JoinSnafu, TaskSet, NUM_CPUS, -}; - -struct PerPartitionWriteHandler { - current_writer: Box, - writer_factory: Arc, - partition_values: Table, - buffer: RowBasedBuffer, - target_file_rows: usize, - written_rows_so_far: usize, - results: Vec
, -} - -impl PerPartitionWriteHandler { - fn new( - writer_factory: Arc, - partition_values: Table, - target_file_rows: usize, - target_chunk_rows: usize, - ) -> DaftResult { - Ok(Self { - current_writer: writer_factory.create_writer(0, Some(&partition_values))?, - writer_factory, - partition_values, - buffer: RowBasedBuffer::new(target_chunk_rows), - target_file_rows, - written_rows_so_far: 0, - results: vec![], - }) - } - - fn submit(&mut self, data: &Arc) -> DaftResult<()> { - self.buffer.push(data.clone()); - if let Some(ready) = self.buffer.pop_enough()? { - for part in ready { - self.handle_write(&part)?; - } - } - Ok(()) - } - - fn handle_write(&mut self, data: &Arc) -> DaftResult<()> { - let len = data.len(); - self.current_writer.write(data)?; - self.written_rows_so_far += len; - - // Check if the file is full, close and start a new file if necessary - if self.written_rows_so_far >= self.target_file_rows { - let result = self.current_writer.close()?; - if let Some(result) = result { - self.results.push(result); - } - self.written_rows_so_far = 0; - self.current_writer = self - .writer_factory - .create_writer(self.results.len(), Some(&self.partition_values))?; - } - Ok(()) - } - - fn finalize(&mut self) -> DaftResult> { - // Write any remaining data from the buffer - let remaining = self.buffer.pop_all()?; - if let Some(part) = remaining { - self.handle_write(&part)?; - } - - // Finalize the current file and collect results - if let Some(result) = self.current_writer.close()? { - self.results.push(result); - } - Ok(std::mem::take(&mut self.results)) - } -} - -pub(crate) struct PartitionedWriteNode { - name: &'static str, - child: Box, - runtime_stats: Arc, - writer_factory: Arc, - partition_cols: Vec, - target_file_rows: usize, - target_chunk_rows: usize, - file_schema: SchemaRef, -} - -impl PartitionedWriteNode { - pub(crate) fn new( - name: &'static str, - child: Box, - writer_factory: Arc, - partition_cols: Vec, - target_file_rows: usize, - target_chunk_rows: usize, - file_schema: SchemaRef, - ) -> Self { - Self { - name, - child, - runtime_stats: RuntimeStatsContext::new(), - partition_cols, - writer_factory, - target_file_rows, - target_chunk_rows, - file_schema, - } - } - - pub(crate) fn boxed(self) -> Box { - Box::new(self) - } - - // Receives data from the dispatcher and writes it. - // Each partition value is mapped to a PerPartitionWriteHandler, which handles the write. - // Each worker is responsible for a subset of partition values. - async fn run_worker( - mut input_receiver: Receiver<(Table, Table)>, - writer_factory: Arc, - target_chunk_rows: usize, - target_file_rows: usize, - ) -> DaftResult> { - let mut per_partition_writers = HashMap::< - IndexHash, - PerPartitionWriteHandler, - IdentityBuildHasher, - >::with_capacity_and_hasher(20, Default::default()); - let mut saved_partition_values = vec![]; - while let Some((data, partition_values)) = input_receiver.recv().await { - assert!(partition_values.len() == 1); - let hash = partition_values.hash_rows()?.get(0).unwrap(); - let entry = per_partition_writers - .raw_entry_mut() - .from_hash(hash, |other| { - (hash == other.hash) && { - let other_table = saved_partition_values.get(other.idx as usize).unwrap(); - other_table == &partition_values - } - }); - match entry { - RawEntryMut::Vacant(entry) => { - let mut new_partition_write_handler = PerPartitionWriteHandler::new( - writer_factory.clone(), - partition_values.clone(), - target_file_rows, - target_chunk_rows, - )?; - new_partition_write_handler.submit(&Arc::new(MicroPartition::new_loaded( - data.schema.clone(), - vec![data].into(), - None, - )))?; - entry.insert_hashed_nocheck( - hash, - IndexHash { - idx: saved_partition_values.len() as u64, - hash, - }, - new_partition_write_handler, - ); - saved_partition_values.push(partition_values); - } - RawEntryMut::Occupied(mut entry) => { - entry - .get_mut() - .submit(&Arc::new(MicroPartition::new_loaded( - data.schema.clone(), - vec![data].into(), - None, - )))?; - } - } - } - - let results = per_partition_writers - .values_mut() - .map(|writer| writer.finalize()) - .flatten_ok() - .collect::>>()?; - Ok(results) - } - - fn spawn_writers( - num_writers: usize, - task_set: &mut TaskSet>>, - writer_factory: Arc, - target_chunk_rows: usize, - target_file_rows: usize, - ) -> Vec> { - let mut writer_senders = Vec::with_capacity(num_writers); - for _ in 0..num_writers { - let (writer_sender, writer_receiver) = create_channel(1); - task_set.spawn(Self::run_worker( - writer_receiver, - writer_factory.clone(), - target_chunk_rows, - target_file_rows, - )); - writer_senders.push(writer_sender); - } - writer_senders - } - - fn partition( - partition_cols: &[ExprRef], - data: &Arc, - ) -> DaftResult<(Vec
, Table)> { - let data = data.concat_or_get(IOStatsContext::new("MicroPartition::partition_by_value"))?; - let table = data.first().unwrap(); - let (split_tables, partition_values) = table.partition_by_value(partition_cols)?; - Ok((split_tables, partition_values)) - } - - // Dispatches data received from the child to the writers - // Partition the data by the partition columns, hash the partition values, and send to the appropriate writer based on the hash - async fn dispatch( - mut input_receiver: CountingReceiver, - senders: Vec>, - partition_cols: Vec, - ) -> DaftResult<()> { - while let Some(data) = input_receiver.recv().await { - let data = data.as_data(); - let (split_tables, partition_values) = Self::partition(&partition_cols, data)?; - let hashes = partition_values.hash_rows()?; - for (idx, (partition, hash)) in split_tables - .into_iter() - .zip(hashes.as_arrow().values_iter()) - .enumerate() - { - let send_to = *hash as usize % senders.len(); - let partition_value_row = partition_values.slice(idx, idx + 1)?; - let _ = senders[send_to] - .send((partition, partition_value_row)) - .await; - } - } - Ok(()) - } -} - -impl TreeDisplay for PartitionedWriteNode { - fn display_as(&self, level: common_display::DisplayLevel) -> String { - use std::fmt::Write; - let mut display = String::new(); - writeln!(display, "{}", self.name()).unwrap(); - use common_display::DisplayLevel::*; - match level { - Compact => {} - _ => { - let rt_result = self.runtime_stats.result(); - rt_result.display(&mut display, true, true, true).unwrap(); - } - } - display - } - - fn get_children(&self) -> Vec<&dyn TreeDisplay> { - vec![self.child.as_tree_display()] - } -} - -impl PipelineNode for PartitionedWriteNode { - fn children(&self) -> Vec<&dyn PipelineNode> { - vec![self.child.as_ref()] - } - - fn name(&self) -> &'static str { - self.name - } - - fn start( - &mut self, - maintain_order: bool, - runtime_handle: &mut ExecutionRuntimeHandle, - ) -> crate::Result { - // Start children - let child = self.child.as_mut(); - let child_results_receiver = child - .start(false, runtime_handle)? - .get_receiver_with_stats(&self.runtime_stats); - - // Initialize destination channels - let mut destination_channel = PipelineChannel::new(1, maintain_order); - let destination_sender = - destination_channel.get_next_sender_with_stats(&self.runtime_stats); - - // Start writers - let mut task_set = TaskSet::new(); - let writer_senders = Self::spawn_writers( - *NUM_CPUS, - &mut task_set, - self.writer_factory.clone(), - self.target_chunk_rows, - self.target_file_rows, - ); - - // Start dispatch - let partition_cols = self.partition_cols.clone(); - runtime_handle.spawn( - Self::dispatch(child_results_receiver, writer_senders, partition_cols), - self.name(), - ); - - // Join writers, receive results, and send to destination - let schema = self.file_schema.clone(); - runtime_handle.spawn( - async move { - let mut results = vec![]; - while let Some(result) = task_set.join_next().await { - results.extend(result.context(JoinSnafu)??); - } - let result_mp = - Arc::new(MicroPartition::new_loaded(schema, Arc::new(results), None)); - let _ = destination_sender.send(result_mp.into()).await; - Ok(()) - }, - self.name(), - ); - Ok(destination_channel) - } - fn as_tree_display(&self) -> &dyn TreeDisplay { - self - } -} diff --git a/src/daft-local-execution/src/writes/physical_write.rs b/src/daft-local-execution/src/writes/physical_write.rs deleted file mode 100644 index 54ec6501e1..0000000000 --- a/src/daft-local-execution/src/writes/physical_write.rs +++ /dev/null @@ -1,34 +0,0 @@ -use common_error::DaftResult; -use daft_micropartition::{create_file_writer, FileWriter}; -use daft_plan::OutputFileInfo; -use daft_table::Table; - -use super::WriterFactory; - -pub(crate) struct PhysicalWriterFactory { - output_file_info: OutputFileInfo, -} - -impl PhysicalWriterFactory { - pub(crate) fn new(output_file_info: OutputFileInfo) -> Self { - Self { output_file_info } - } -} - -impl WriterFactory for PhysicalWriterFactory { - fn create_writer( - &self, - file_idx: usize, - partition_values: Option<&Table>, - ) -> DaftResult> { - let writer = create_file_writer( - &self.output_file_info.root_dir, - file_idx, - &self.output_file_info.compression, - &self.output_file_info.io_config, - self.output_file_info.file_format, - partition_values, - )?; - Ok(writer) - } -} diff --git a/src/daft-local-execution/src/writes/unpartitioned_write.rs b/src/daft-local-execution/src/writes/unpartitioned_write.rs deleted file mode 100644 index 958ef09466..0000000000 --- a/src/daft-local-execution/src/writes/unpartitioned_write.rs +++ /dev/null @@ -1,234 +0,0 @@ -use std::sync::Arc; - -use common_display::tree::TreeDisplay; -use common_error::DaftResult; -use daft_core::prelude::SchemaRef; -use daft_micropartition::{FileWriter, MicroPartition}; -use daft_table::Table; -use snafu::ResultExt; - -use super::WriterFactory; -use crate::{ - buffer::RowBasedBuffer, - channel::{create_channel, PipelineChannel, Receiver, Sender}, - pipeline::PipelineNode, - runtime_stats::{CountingReceiver, RuntimeStatsContext}, - ExecutionRuntimeHandle, JoinSnafu, TaskSet, NUM_CPUS, -}; - -pub(crate) struct UnpartitionedWriteNode { - name: &'static str, - child: Box, - runtime_stats: Arc, - writer_factory: Arc, - target_in_memory_file_rows: usize, - target_in_memory_chunk_rows: usize, - file_schema: SchemaRef, -} - -impl UnpartitionedWriteNode { - pub(crate) fn new( - name: &'static str, - child: Box, - writer_factory: Arc, - target_in_memory_file_rows: usize, - target_in_memory_chunk_rows: usize, - file_schema: SchemaRef, - ) -> Self { - Self { - name, - child, - runtime_stats: RuntimeStatsContext::new(), - writer_factory, - target_in_memory_file_rows, - target_in_memory_chunk_rows, - file_schema, - } - } - - pub(crate) fn boxed(self) -> Box { - Box::new(self) - } - - // Receives data from the dispatcher and writes it. - // If the received file idx is different from the current file idx, this means that the current file is full and needs to be closed. - // Once input is exhausted, the current writer is closed and all written file paths are returned. - async fn run_writer( - mut input_receiver: Receiver<(Arc, usize)>, - writer_factory: Arc, - ) -> DaftResult> { - let mut written_file_paths = vec![]; - let mut current_writer: Option> = None; - let mut current_file_idx = None; - while let Some((data, file_idx)) = input_receiver.recv().await { - if current_file_idx.is_none() || current_file_idx.unwrap() != file_idx { - if let Some(writer) = current_writer.take() { - if let Some(path) = writer.close()? { - written_file_paths.push(path); - } - } - current_file_idx = Some(file_idx); - current_writer = Some(writer_factory.create_writer(file_idx, None)?); - } - if let Some(writer) = current_writer.as_mut() { - writer.write(&data)?; - } - } - if let Some(writer) = current_writer { - if let Some(path) = writer.close()? { - written_file_paths.push(path); - } - } - Ok(written_file_paths) - } - - fn spawn_writers( - num_writers: usize, - task_set: &mut TaskSet>>, - writer_factory: &Arc, - channel_size: usize, - ) -> Vec, usize)>> { - let mut writer_senders = Vec::with_capacity(num_writers); - for _ in 0..num_writers { - let (writer_sender, writer_receiver) = create_channel(channel_size); - task_set.spawn(Self::run_writer(writer_receiver, writer_factory.clone())); - writer_senders.push(writer_sender); - } - writer_senders - } - - // Dispatches data received from the child to the writers - // As data is received, it is buffered until enough data is available to fill a chunk - // Once a chunk is filled, it is sent to a writer - // If the writer has written enough rows for a file, increment the file index and switch to the next writer - async fn dispatch( - mut input_receiver: CountingReceiver, - target_chunk_rows: usize, - target_file_rows: usize, - senders: Vec, usize)>>, - ) -> DaftResult<()> { - let mut curr_sent_rows = 0; - let mut curr_file_idx = 0; - let mut curr_sender_idx = 0; - let mut buffer = RowBasedBuffer::new(target_chunk_rows); - while let Some(data) = input_receiver.recv().await { - let data = data.as_data(); - if data.is_empty() { - continue; - } - - buffer.push(data.clone()); - if let Some(ready) = buffer.pop_enough()? { - for part in ready { - curr_sent_rows += part.len(); - let _ = senders[curr_sender_idx].send((part, curr_file_idx)).await; - if curr_sent_rows >= target_file_rows { - curr_sent_rows = 0; - curr_file_idx += 1; - curr_sender_idx = (curr_sender_idx + 1) % senders.len(); - } - } - } - } - if let Some(leftover) = buffer.pop_all()? { - let _ = senders[curr_sender_idx] - .send((leftover, curr_file_idx)) - .await; - } - Ok(()) - } -} - -impl TreeDisplay for UnpartitionedWriteNode { - fn display_as(&self, level: common_display::DisplayLevel) -> String { - use std::fmt::Write; - let mut display = String::new(); - writeln!(display, "{}", self.name()).unwrap(); - use common_display::DisplayLevel::*; - match level { - Compact => {} - _ => { - let rt_result = self.runtime_stats.result(); - rt_result.display(&mut display, true, true, true).unwrap(); - } - } - display - } - - fn get_children(&self) -> Vec<&dyn TreeDisplay> { - vec![self.child.as_tree_display()] - } -} - -impl PipelineNode for UnpartitionedWriteNode { - fn children(&self) -> Vec<&dyn PipelineNode> { - vec![self.child.as_ref()] - } - - fn name(&self) -> &'static str { - self.name - } - - fn start( - &mut self, - maintain_order: bool, - runtime_handle: &mut ExecutionRuntimeHandle, - ) -> crate::Result { - // Start children - let child = self.child.as_mut(); - let child_results_receiver = child - .start(false, runtime_handle)? - .get_receiver_with_stats(&self.runtime_stats); - - // Initialize destination channel - let mut destination_channel = PipelineChannel::new(1, maintain_order); - let destination_sender = - destination_channel.get_next_sender_with_stats(&self.runtime_stats); - - // Start writers - let writer_factory = self.writer_factory.clone(); - let mut task_set = TaskSet::new(); - let writer_senders = Self::spawn_writers( - *NUM_CPUS, - &mut task_set, - &writer_factory, - // The channel size is set to the number of chunks per file such that writes can be parallelized - (self.target_in_memory_file_rows + self.target_in_memory_chunk_rows + 1) - / self.target_in_memory_chunk_rows, - ); - - // Start dispatch - let (target_file_rows, target_chunk_rows) = ( - self.target_in_memory_file_rows, - self.target_in_memory_chunk_rows, - ); - runtime_handle.spawn( - Self::dispatch( - child_results_receiver, - target_chunk_rows, - target_file_rows, - writer_senders, - ), - self.name(), - ); - - // Join writers, receive results, and send to destination - let schema = self.file_schema.clone(); - runtime_handle.spawn( - async move { - let mut results = vec![]; - while let Some(result) = task_set.join_next().await { - results.extend(result.context(JoinSnafu)??); - } - let mp = MicroPartition::new_loaded(schema, results.into(), None); - let _ = destination_sender.send(Arc::new(mp).into()).await; - Ok(()) - }, - self.name(), - ); - Ok(destination_channel) - } - fn as_tree_display(&self) -> &dyn TreeDisplay { - self - } -} diff --git a/src/daft-micropartition/src/lib.rs b/src/daft-micropartition/src/lib.rs index c906354036..c677a0fd96 100644 --- a/src/daft-micropartition/src/lib.rs +++ b/src/daft-micropartition/src/lib.rs @@ -2,11 +2,7 @@ #![feature(iterator_try_reduce)] #![feature(iterator_try_collect)] -use std::sync::Arc; - -use common_error::{DaftError, DaftResult}; -use common_file_formats::FileFormat; -use daft_table::Table; +use common_error::DaftError; use snafu::Snafu; mod micropartition; mod ops; @@ -18,8 +14,6 @@ pub mod python; #[cfg(feature = "python")] use pyo3::PyErr; #[cfg(feature = "python")] -pub mod py_writers; -#[cfg(feature = "python")] pub use python::register_modules; #[derive(Debug, Snafu)] @@ -66,35 +60,3 @@ impl From for pyo3::PyErr { daft_error.into() } } - -pub trait FileWriter: Send + Sync { - fn write(&self, data: &Arc) -> DaftResult<()>; - fn close(&self) -> DaftResult>; -} - -pub fn create_file_writer( - root_dir: &str, - file_idx: usize, - compression: &Option, - io_config: &Option, - format: FileFormat, - partition: Option<&Table>, -) -> DaftResult> { - match format { - #[cfg(feature = "python")] - FileFormat::Parquet => Ok(Box::new(py_writers::PyArrowParquetWriter::new( - root_dir, - file_idx, - compression, - io_config, - partition, - )?)), - #[cfg(feature = "python")] - FileFormat::Csv => Ok(Box::new(py_writers::PyArrowCSVWriter::new( - root_dir, file_idx, io_config, partition, - )?)), - _ => Err(DaftError::ComputeError( - "Unsupported file format for physical write".to_string(), - )), - } -} diff --git a/src/daft-micropartition/src/py_writers.rs b/src/daft-micropartition/src/py_writers.rs deleted file mode 100644 index 7c1e972de5..0000000000 --- a/src/daft-micropartition/src/py_writers.rs +++ /dev/null @@ -1,113 +0,0 @@ -use std::sync::Arc; - -use common_error::DaftResult; -use daft_table::{python::PyTable, Table}; -use pyo3::{types::PyAnyMethods, PyObject, Python}; - -use crate::{python::PyMicroPartition, FileWriter, MicroPartition}; - -pub struct PyArrowParquetWriter { - py_writer: PyObject, -} - -impl PyArrowParquetWriter { - pub fn new( - root_dir: &str, - file_idx: usize, - compression: &Option, - io_config: &Option, - partition_values: Option<&Table>, - ) -> DaftResult { - Python::with_gil(|py| { - let file_writer_module = py.import_bound(pyo3::intern!(py, "daft.io.writer"))?; - let file_writer_class = file_writer_module.getattr("ParquetFileWriter")?; - - let py_writer = file_writer_class.call1(( - root_dir, - file_idx, - partition_values.map(|pv| PyTable::from(pv.clone())), - compression.as_ref().map(|c| c.as_str()), - io_config.as_ref().map(|cfg| daft_io::python::IOConfig { - config: cfg.clone(), - }), - ))?; - Ok(Self { - py_writer: py_writer.into(), - }) - }) - } -} - -impl FileWriter for PyArrowParquetWriter { - fn write(&self, data: &Arc) -> DaftResult<()> { - Python::with_gil(|py| { - let py_micropartition = py - .import_bound(pyo3::intern!(py, "daft.table"))? - .getattr(pyo3::intern!(py, "MicroPartition"))? - .getattr(pyo3::intern!(py, "_from_pymicropartition"))? - .call1((PyMicroPartition::from(data.clone()),))?; - self.py_writer - .call_method1(py, "write", (py_micropartition,))?; - Ok(()) - }) - } - - fn close(&self) -> DaftResult> { - Python::with_gil(|py| { - let result = self.py_writer.call_method0(py, "close")?; - Ok(Some(result.extract::(py)?.into())) - }) - } -} - -pub struct PyArrowCSVWriter { - py_writer: PyObject, -} - -impl PyArrowCSVWriter { - pub fn new( - root_dir: &str, - file_idx: usize, - io_config: &Option, - partition_values: Option<&Table>, - ) -> DaftResult { - Python::with_gil(|py| { - let file_writer_module = py.import_bound(pyo3::intern!(py, "daft.io.writer"))?; - let file_writer_class = file_writer_module.getattr("CSVFileWriter")?; - - let py_writer = file_writer_class.call1(( - root_dir, - file_idx, - partition_values.map(|pv| PyTable::from(pv.clone())), - io_config.as_ref().map(|cfg| daft_io::python::IOConfig { - config: cfg.clone(), - }), - ))?; - Ok(Self { - py_writer: py_writer.into(), - }) - }) - } -} - -impl FileWriter for PyArrowCSVWriter { - fn write(&self, data: &Arc) -> DaftResult<()> { - Python::with_gil(|py| { - let py_micropartition = py - .import_bound(pyo3::intern!(py, "daft.table"))? - .getattr(pyo3::intern!(py, "MicroPartition"))? - .getattr(pyo3::intern!(py, "_from_pymicropartition"))? - .call1((PyMicroPartition::from(data.clone()),))?; - self.py_writer - .call_method1(py, "write", (py_micropartition,))?; - Ok(()) - }) - } - - fn close(&self) -> DaftResult> { - Python::with_gil(|py| { - let result = self.py_writer.call_method0(py, "close")?; - Ok(Some(result.extract::(py)?.into())) - }) - } -} diff --git a/src/daft-table/src/lib.rs b/src/daft-table/src/lib.rs index f33df1119b..0a450ace70 100644 --- a/src/daft-table/src/lib.rs +++ b/src/daft-table/src/lib.rs @@ -787,10 +787,10 @@ impl Table { impl PartialEq for Table { fn eq(&self, other: &Self) -> bool { - if self.schema != other.schema { + if self.len() != other.len() { return false; } - if self.len() != other.len() { + if self.schema != other.schema { return false; } for (lhs, rhs) in self.columns.iter().zip(other.columns.iter()) { diff --git a/src/daft-writers/Cargo.toml b/src/daft-writers/Cargo.toml new file mode 100644 index 0000000000..c0694aeb81 --- /dev/null +++ b/src/daft-writers/Cargo.toml @@ -0,0 +1,42 @@ +[dependencies] +async-trait = {workspace = true} +common-daft-config = {path = "../common/daft-config", default-features = false} +common-display = {path = "../common/display", default-features = false} +common-error = {path = "../common/error", default-features = false} +common-file-formats = {path = "../common/file-formats", default-features = false} +common-runtime = {path = "../common/runtime", default-features = false} +common-tracing = {path = "../common/tracing", default-features = false} +daft-core = {path = "../daft-core", default-features = false} +daft-csv = {path = "../daft-csv", default-features = false} +daft-dsl = {path = "../daft-dsl", default-features = false} +daft-functions = {path = "../daft-functions", default-features = false} +daft-io = {path = "../daft-io", default-features = false} +daft-json = {path = "../daft-json", default-features = false} +daft-micropartition = {path = "../daft-micropartition", default-features = false} +daft-parquet = {path = "../daft-parquet", default-features = false} +daft-physical-plan = {path = "../daft-physical-plan", default-features = false} +daft-plan = {path = "../daft-plan", default-features = false} +daft-scan = {path = "../daft-scan", default-features = false} +daft-table = {path = "../daft-table", default-features = false} +futures = {workspace = true} +indexmap = {workspace = true} +itertools = {workspace = true} +lazy_static = {workspace = true} +log = {workspace = true} +num-format = "0.4.4" +pyo3 = {workspace = true, optional = true} +snafu = {workspace = true} +tokio = {workspace = true} +tokio-stream = {workspace = true} +tracing = {workspace = true} + +[features] +python = ["dep:pyo3", "common-daft-config/python", "common-file-formats/python", "common-error/python", "daft-dsl/python", "daft-io/python", "daft-micropartition/python", "daft-plan/python", "daft-scan/python", "common-display/python"] + +[lints] +workspace = true + +[package] +edition = {workspace = true} +name = "daft-writers" +version = {workspace = true} diff --git a/src/daft-writers/src/lib.rs b/src/daft-writers/src/lib.rs new file mode 100644 index 0000000000..f5a5155b36 --- /dev/null +++ b/src/daft-writers/src/lib.rs @@ -0,0 +1,32 @@ +mod physical; +#[cfg(feature = "python")] +mod python; + +use common_error::DaftResult; +use daft_table::Table; +pub use physical::PhysicalWriterFactory; + +/// This trait is used to abstract the writing of data to a file. +/// The `Input` type is the type of data that will be written to the file. +/// The `Result` type is the type of the result that will be returned when the file is closed. +pub trait FileWriter: Send + Sync { + type Input; + type Result; + + fn write(&mut self, data: &Self::Input) -> DaftResult<()>; + fn close(&mut self) -> DaftResult; +} + +/// This trait is used to abstract the creation of a `FileWriter` +/// The `create_writer` method is used to create a new `FileWriter`. +/// `file_idx` is the index of the file that will be written to. +/// `partition_values` is the partition values of the data that will be written to the file. +pub trait WriterFactory: Send + Sync { + type Input; + type Result; + fn create_writer( + &self, + file_idx: usize, + partition_values: Option<&Table>, + ) -> DaftResult>>; +} diff --git a/src/daft-writers/src/physical.rs b/src/daft-writers/src/physical.rs new file mode 100644 index 0000000000..87990211ba --- /dev/null +++ b/src/daft-writers/src/physical.rs @@ -0,0 +1,56 @@ +use std::sync::Arc; + +use common_error::DaftResult; +use daft_micropartition::MicroPartition; +use daft_plan::OutputFileInfo; +use daft_table::Table; + +use crate::{FileWriter, WriterFactory}; + +pub struct PhysicalWriterFactory { + output_file_info: OutputFileInfo, + native: bool, // TODO: Implement native writer +} + +impl PhysicalWriterFactory { + pub fn new(output_file_info: OutputFileInfo) -> Self { + Self { + output_file_info, + native: false, + } + } +} + +impl WriterFactory for PhysicalWriterFactory { + type Input = Arc; + type Result = Option
; + + fn create_writer( + &self, + file_idx: usize, + partition_values: Option<&Table>, + ) -> DaftResult>> { + match self.native { + true => unimplemented!(), + false => { + #[cfg(feature = "python")] + { + let writer = crate::python::create_pyarrow_file_writer( + &self.output_file_info.root_dir, + file_idx, + &self.output_file_info.compression, + &self.output_file_info.io_config, + self.output_file_info.file_format, + partition_values, + )?; + Ok(writer) + } + + #[cfg(not(feature = "python"))] + { + unimplemented!() + } + } + } + } +} diff --git a/src/daft-writers/src/python.rs b/src/daft-writers/src/python.rs new file mode 100644 index 0000000000..5b0321f60a --- /dev/null +++ b/src/daft-writers/src/python.rs @@ -0,0 +1,141 @@ +use std::sync::Arc; + +use common_error::{DaftError, DaftResult}; +use common_file_formats::FileFormat; +use daft_micropartition::{python::PyMicroPartition, MicroPartition}; +use daft_table::{python::PyTable, Table}; +use pyo3::{types::PyAnyMethods, PyObject, Python}; + +use crate::FileWriter; + +pub fn create_pyarrow_file_writer( + root_dir: &str, + file_idx: usize, + compression: &Option, + io_config: &Option, + format: FileFormat, + partition: Option<&Table>, +) -> DaftResult, Result = Option
>>> { + match format { + #[cfg(feature = "python")] + FileFormat::Parquet => Ok(Box::new(PyArrowWriter::new_parquet_writer( + root_dir, + file_idx, + compression, + io_config, + partition, + )?)), + #[cfg(feature = "python")] + FileFormat::Csv => Ok(Box::new(PyArrowWriter::new_csv_writer( + root_dir, file_idx, io_config, partition, + )?)), + _ => Err(DaftError::ComputeError( + "Unsupported file format for physical write".to_string(), + )), + } +} + +pub struct PyArrowWriter { + py_writer: PyObject, +} + +impl PyArrowWriter { + pub fn new_parquet_writer( + root_dir: &str, + file_idx: usize, + compression: &Option, + io_config: &Option, + partition_values: Option<&Table>, + ) -> DaftResult { + Python::with_gil(|py| { + let file_writer_module = py.import_bound(pyo3::intern!(py, "daft.io.writer"))?; + let file_writer_class = file_writer_module.getattr("ParquetFileWriter")?; + let _from_pytable = py + .import_bound(pyo3::intern!(py, "daft.table"))? + .getattr(pyo3::intern!(py, "Table"))? + .getattr(pyo3::intern!(py, "_from_pytable"))?; + let partition_values = match partition_values { + Some(pv) => { + let py_table = _from_pytable.call1((PyTable::from(pv.clone()),))?; + Some(py_table) + } + None => None, + }; + + let py_writer = file_writer_class.call1(( + root_dir, + file_idx, + partition_values, + compression.as_ref().map(|c| c.as_str()), + io_config.as_ref().map(|cfg| daft_io::python::IOConfig { + config: cfg.clone(), + }), + ))?; + Ok(Self { + py_writer: py_writer.into(), + }) + }) + } + + pub fn new_csv_writer( + root_dir: &str, + file_idx: usize, + io_config: &Option, + partition_values: Option<&Table>, + ) -> DaftResult { + Python::with_gil(|py| { + let file_writer_module = py.import_bound(pyo3::intern!(py, "daft.io.writer"))?; + let file_writer_class = file_writer_module.getattr("CSVFileWriter")?; + let _from_pytable = py + .import_bound(pyo3::intern!(py, "daft.table"))? + .getattr(pyo3::intern!(py, "Table"))? + .getattr(pyo3::intern!(py, "_from_pytable"))?; + let partition_values = match partition_values { + Some(pv) => { + let py_table = _from_pytable.call1((PyTable::from(pv.clone()),))?; + Some(py_table) + } + None => None, + }; + let py_writer = file_writer_class.call1(( + root_dir, + file_idx, + partition_values, + io_config.as_ref().map(|cfg| daft_io::python::IOConfig { + config: cfg.clone(), + }), + ))?; + Ok(Self { + py_writer: py_writer.into(), + }) + }) + } +} + +impl FileWriter for PyArrowWriter { + type Input = Arc; + type Result = Option
; + + fn write(&mut self, data: &Self::Input) -> DaftResult<()> { + Python::with_gil(|py| { + let py_micropartition = py + .import_bound(pyo3::intern!(py, "daft.table"))? + .getattr(pyo3::intern!(py, "MicroPartition"))? + .getattr(pyo3::intern!(py, "_from_pymicropartition"))? + .call1((PyMicroPartition::from(data.clone()),))?; + self.py_writer + .call_method1(py, pyo3::intern!(py, "write"), (py_micropartition,))?; + Ok(()) + }) + } + + fn close(&mut self) -> DaftResult { + Python::with_gil(|py| { + let result = self + .py_writer + .call_method0(py, pyo3::intern!(py, "close"))? + .getattr(py, pyo3::intern!(py, "_table"))?; + Ok(Some(result.extract::(py)?.into())) + }) + } +} diff --git a/tests/io/test_parquet_roundtrip.py b/tests/io/test_parquet_roundtrip.py index 292c5b98e1..8804ef6d33 100644 --- a/tests/io/test_parquet_roundtrip.py +++ b/tests/io/test_parquet_roundtrip.py @@ -12,13 +12,6 @@ PYARROW_GE_8_0_0 = tuple(int(s) for s in pa.__version__.split(".") if s.isnumeric()) >= (8, 0, 0) -from daft import context - -pytestmark = pytest.mark.skipif( - context.get_context().daft_execution_config.enable_native_executor is True, - reason="Native executor fails for these tests", -) - @pytest.mark.skipif( not PYARROW_GE_8_0_0, From 16f3f7d426408599038c396f2be59499d48cd895 Mon Sep 17 00:00:00 2001 From: Colin Ho Date: Mon, 28 Oct 2024 18:20:06 -0700 Subject: [PATCH 07/15] add comments --- src/daft-local-execution/src/sinks/write.rs | 12 +++++++++--- src/daft-writers/src/physical.rs | 1 + 2 files changed, 10 insertions(+), 3 deletions(-) diff --git a/src/daft-local-execution/src/sinks/write.rs b/src/daft-local-execution/src/sinks/write.rs index 296e80ad3e..cb67354ae4 100644 --- a/src/daft-local-execution/src/sinks/write.rs +++ b/src/daft-local-execution/src/sinks/write.rs @@ -22,6 +22,8 @@ use tracing::instrument; use super::blocking_sink::{BlockingSink, BlockingSinkStatus}; use crate::{buffer::RowBasedBuffer, pipeline::PipelineResultType}; +// TargetBatchWriter is a writer that writes in batches of rows, i.e. for Parquet where we want to write +// a row group at a time. It uses a buffer to accumulate rows until it has enough to write a batch. struct TargetBatchWriter { buffer: RowBasedBuffer, writer: Box, Result = Option
>>, @@ -61,12 +63,12 @@ impl FileWriter for TargetBatchWriter { } } -struct TargetBatchWriterFactory { - writer_factory: Arc>, +struct TargetBatchWriterFactory { + writer_factory: Arc, Result = Option
>>, target_in_memory_chunk_rows: usize, } -impl WriterFactory for TargetBatchWriterFactory, Option
> { +impl WriterFactory for TargetBatchWriterFactory { type Input = Arc; type Result = Option
; @@ -88,6 +90,8 @@ impl WriterFactory for TargetBatchWriterFactory, Option, Result = Option
>>, @@ -206,6 +210,8 @@ impl WriterFactory for TargetFileSizeWriterFactory { } } +/// PartitionedWriter is a writer that partitions the input data by a set of columns, and writes each partition +/// to a separate file. It uses a map to keep track of the writers for each partition. struct PartitionedWriter { per_partition_writers: HashMap, Result = Vec
>>>, diff --git a/src/daft-writers/src/physical.rs b/src/daft-writers/src/physical.rs index 87990211ba..f94828d734 100644 --- a/src/daft-writers/src/physical.rs +++ b/src/daft-writers/src/physical.rs @@ -7,6 +7,7 @@ use daft_table::Table; use crate::{FileWriter, WriterFactory}; +/// PhysicalWriterFactory is a factory for creating physical writers, i.e. parquet, csv writers. pub struct PhysicalWriterFactory { output_file_info: OutputFileInfo, native: bool, // TODO: Implement native writer From 11d9244f5da2337a1cba2934dd54670a7d3fc841 Mon Sep 17 00:00:00 2001 From: Colin Ho Date: Mon, 28 Oct 2024 18:25:35 -0700 Subject: [PATCH 08/15] make write and close abstract --- daft/io/writer.py | 45 ++++++++++++++++++++++++++++++++++----------- 1 file changed, 34 insertions(+), 11 deletions(-) diff --git a/daft/io/writer.py b/daft/io/writer.py index c9b7667805..141c4dd521 100644 --- a/daft/io/writer.py +++ b/daft/io/writer.py @@ -87,30 +87,23 @@ def _create_writer(self, schema: pa.Schema) -> Union[pq.ParquetWriter, pacsv.CSV """ pass + @abstractmethod def write(self, table: MicroPartition) -> None: """Write data to the file using the appropriate writer. Args: table: MicroPartition containing the data to be written. """ - if self.current_writer is None: - self.current_writer = self._create_writer(table.schema().to_pyarrow_schema()) - self.current_writer.write_table(table.to_arrow()) + pass + @abstractmethod def close(self) -> Table: """Close the writer and return metadata about the written file. Returns: Table containing metadata about the written file, including path and partition values. """ - if self.current_writer is not None: - self.current_writer.close() - - metadata = {"path": Series.from_pylist([self.full_path])} - if self.partition_values is not None: - for col_name in self.partition_values.column_names(): - metadata[col_name] = self.partition_values.get_column(col_name) - return Table.from_pydict(metadata) + pass class ParquetFileWriter(FileWriterBase): @@ -140,6 +133,21 @@ def _create_writer(self, schema: pa.Schema) -> pq.ParquetWriter: filesystem=self.fs, ) + def write(self, table: MicroPartition) -> None: + if self.current_writer is None: + self.current_writer = self._create_writer(table.schema().to_pyarrow_schema()) + self.current_writer.write_table(table.to_arrow()) + + def close(self) -> Table: + if self.current_writer is not None: + self.current_writer.close() + + metadata = {"path": Series.from_pylist([self.full_path])} + if self.partition_values is not None: + for col_name in self.partition_values.column_names(): + metadata[col_name] = self.partition_values.get_column(col_name) + return Table.from_pydict(metadata) + class CSVFileWriter(FileWriterBase): def __init__( @@ -162,3 +170,18 @@ def _create_writer(self, schema: pa.Schema) -> pacsv.CSVWriter: self.full_path, schema, ) + + def write(self, table: MicroPartition) -> None: + if self.current_writer is None: + self.current_writer = self._create_writer(table.schema().to_pyarrow_schema()) + self.current_writer.write_table(table.to_arrow()) + + def close(self) -> Table: + if self.current_writer is not None: + self.current_writer.close() + + metadata = {"path": Series.from_pylist([self.full_path])} + if self.partition_values is not None: + for col_name in self.partition_values.column_names(): + metadata[col_name] = self.partition_values.get_column(col_name) + return Table.from_pydict(metadata) From 685416b8da28b5c722d8344cfeb9716f9e77691a Mon Sep 17 00:00:00 2001 From: Colin Ho Date: Mon, 28 Oct 2024 21:57:25 -0700 Subject: [PATCH 09/15] dispatcher --- Cargo.toml | 1 + src/daft-local-execution/Cargo.toml | 2 +- src/daft-local-execution/src/dispatcher.rs | 102 +++++++++++ src/daft-local-execution/src/lib.rs | 1 + src/daft-local-execution/src/pipeline.rs | 13 +- .../src/sinks/aggregate.rs | 104 +++++++---- .../src/sinks/blocking_sink.rs | 163 +++++++++++++---- .../src/sinks/hash_join_build.rs | 61 +++++-- src/daft-local-execution/src/sinks/sort.rs | 113 ++++++++---- src/daft-local-execution/src/sinks/write.rs | 167 ++++++++++++++---- src/daft-writers/src/physical.rs | 56 ++++-- src/daft-writers/src/python.rs | 30 +--- 12 files changed, 605 insertions(+), 208 deletions(-) create mode 100644 src/daft-local-execution/src/dispatcher.rs diff --git a/Cargo.toml b/Cargo.toml index cac465eb08..27d5f98309 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -59,6 +59,7 @@ python = [ "daft-table/python", "daft-functions/python", "daft-functions-json/python", + "daft-writers/python", "common-daft-config/python", "common-system-info/python", "common-display/python", diff --git a/src/daft-local-execution/Cargo.toml b/src/daft-local-execution/Cargo.toml index 877cec27dd..c6b14c12b6 100644 --- a/src/daft-local-execution/Cargo.toml +++ b/src/daft-local-execution/Cargo.toml @@ -32,7 +32,7 @@ tokio-stream = {workspace = true} tracing = {workspace = true} [features] -python = ["dep:pyo3", "common-daft-config/python", "common-file-formats/python", "common-error/python", "daft-dsl/python", "daft-io/python", "daft-micropartition/python", "daft-plan/python", "daft-scan/python", "common-display/python"] +python = ["dep:pyo3", "common-daft-config/python", "common-file-formats/python", "common-error/python", "daft-dsl/python", "daft-io/python", "daft-micropartition/python", "daft-plan/python", "daft-scan/python", "daft-writers/python", "common-display/python"] [lints] workspace = true diff --git a/src/daft-local-execution/src/dispatcher.rs b/src/daft-local-execution/src/dispatcher.rs new file mode 100644 index 0000000000..161b7de3bb --- /dev/null +++ b/src/daft-local-execution/src/dispatcher.rs @@ -0,0 +1,102 @@ +use std::sync::Arc; + +use async_trait::async_trait; +use common_error::DaftResult; +use daft_dsl::ExprRef; + +use crate::{ + buffer::RowBasedBuffer, channel::Sender, pipeline::PipelineResultType, + runtime_stats::CountingReceiver, +}; + +#[async_trait] +pub(crate) trait Dispatcher { + async fn dispatch( + &self, + receiver: CountingReceiver, + worker_senders: Vec>, + ) -> DaftResult<()>; +} + +pub(crate) struct RoundRobinBufferedDispatcher { + morsel_size: usize, +} + +impl RoundRobinBufferedDispatcher { + pub(crate) fn new(morsel_size: usize) -> Self { + Self { morsel_size } + } +} + +#[async_trait] +impl Dispatcher for RoundRobinBufferedDispatcher { + async fn dispatch( + &self, + mut receiver: CountingReceiver, + worker_senders: Vec>, + ) -> DaftResult<()> { + let mut next_worker_idx = 0; + let mut send_to_next_worker = |data: PipelineResultType| { + let next_worker_sender = worker_senders.get(next_worker_idx).unwrap(); + next_worker_idx = (next_worker_idx + 1) % worker_senders.len(); + next_worker_sender.send(data) + }; + + let mut buffer = RowBasedBuffer::new(self.morsel_size); + while let Some(morsel) = receiver.recv().await { + if morsel.should_broadcast() { + for worker_sender in &worker_senders { + let _ = worker_sender.send(morsel.clone()).await; + } + } else { + buffer.push(morsel.as_data().clone()); + if let Some(ready) = buffer.pop_enough()? { + for r in ready { + let _ = send_to_next_worker(r.into()).await; + } + } + } + } + // Clear all remaining morsels + if let Some(last_morsel) = buffer.pop_all()? { + let _ = send_to_next_worker(last_morsel.into()).await; + } + Ok(()) + } +} + +pub(crate) struct PartitionedDispatcher { + partition_by: Vec, +} + +impl PartitionedDispatcher { + pub(crate) fn new(partition_by: Vec) -> Self { + Self { partition_by } + } +} + +#[async_trait] +impl Dispatcher for PartitionedDispatcher { + async fn dispatch( + &self, + mut receiver: CountingReceiver, + worker_senders: Vec>, + ) -> DaftResult<()> { + while let Some(morsel) = receiver.recv().await { + if morsel.should_broadcast() { + for worker_sender in &worker_senders { + let _ = worker_sender.send(morsel.clone()).await; + } + } else { + let partitions = morsel + .as_data() + .partition_by_hash(&self.partition_by, worker_senders.len())?; + for (partition, worker_sender) in partitions.into_iter().zip(worker_senders.iter()) + { + let _ = worker_sender.send(Arc::new(partition).into()).await; + } + } + } + Ok(()) + } +} diff --git a/src/daft-local-execution/src/lib.rs b/src/daft-local-execution/src/lib.rs index 3f91c58955..448d8408ad 100644 --- a/src/daft-local-execution/src/lib.rs +++ b/src/daft-local-execution/src/lib.rs @@ -2,6 +2,7 @@ #![feature(hash_raw_entry)] mod buffer; mod channel; +mod dispatcher; mod intermediate_ops; mod pipeline; mod run; diff --git a/src/daft-local-execution/src/pipeline.rs b/src/daft-local-execution/src/pipeline.rs index 49fab6c385..a46f40871a 100644 --- a/src/daft-local-execution/src/pipeline.rs +++ b/src/daft-local-execution/src/pipeline.rs @@ -197,7 +197,7 @@ pub fn physical_plan_to_pipeline( vec![], ); let second_stage_node = - BlockingSinkNode::new(second_stage_agg_sink.boxed(), post_first_agg_node).boxed(); + BlockingSinkNode::new(Arc::new(second_stage_agg_sink), post_first_agg_node).boxed(); let final_stage_project = ProjectOperator::new(final_exprs); @@ -239,7 +239,7 @@ pub fn physical_plan_to_pipeline( group_by.clone(), ); let second_stage_node = - BlockingSinkNode::new(second_stage_agg_sink.boxed(), post_first_agg_node).boxed(); + BlockingSinkNode::new(Arc::new(second_stage_agg_sink), post_first_agg_node).boxed(); let final_stage_project = ProjectOperator::new(final_exprs); @@ -287,7 +287,7 @@ pub fn physical_plan_to_pipeline( }) => { let sort_sink = SortSink::new(sort_by.clone(), descending.clone()); let child_node = physical_plan_to_pipeline(input, psets, cfg)?; - BlockingSinkNode::new(sort_sink.boxed(), child_node).boxed() + BlockingSinkNode::new(Arc::new(sort_sink), child_node).boxed() } LocalPhysicalPlan::HashJoin(HashJoin { @@ -358,7 +358,7 @@ pub fn physical_plan_to_pipeline( let build_sink = HashJoinBuildSink::new(key_schema, casted_build_on, join_type)?; let build_child_node = physical_plan_to_pipeline(build_child, psets, cfg)?; let build_node = - BlockingSinkNode::new(build_sink.boxed(), build_child_node).boxed(); + BlockingSinkNode::new(Arc::new(build_sink), build_child_node).boxed(); let probe_child_node = physical_plan_to_pipeline(probe_child, psets, cfg)?; @@ -420,9 +420,8 @@ pub fn physical_plan_to_pipeline( ) .context(PipelineCreationSnafu { plan_name: physical_plan.name(), - })? - .boxed(); - BlockingSinkNode::new(write_sink, child_node).boxed() + })?; + BlockingSinkNode::new(Arc::new(write_sink), child_node).boxed() } }; diff --git a/src/daft-local-execution/src/sinks/aggregate.rs b/src/daft-local-execution/src/sinks/aggregate.rs index e94ff7c68b..7fc95dd607 100644 --- a/src/daft-local-execution/src/sinks/aggregate.rs +++ b/src/daft-local-execution/src/sinks/aggregate.rs @@ -5,19 +5,45 @@ use daft_dsl::ExprRef; use daft_micropartition::MicroPartition; use tracing::instrument; -use super::blocking_sink::{BlockingSink, BlockingSinkStatus}; -use crate::pipeline::PipelineResultType; +use super::blocking_sink::{ + BlockingSink, BlockingSinkState, BlockingSinkStatus, DynBlockingSinkState, +}; +use crate::{pipeline::PipelineResultType, NUM_CPUS}; enum AggregateState { Accumulating(Vec>), - #[allow(dead_code)] - Done(Arc), + Done, +} + +impl AggregateState { + fn push(&mut self, part: Arc) { + if let Self::Accumulating(ref mut parts) = self { + parts.push(part); + } else { + panic!("AggregateSink should be in Accumulating state"); + } + } + + fn finalize(&mut self) -> DaftResult>> { + let res = if let Self::Accumulating(ref mut parts) = self { + std::mem::take(parts) + } else { + panic!("AggregateSink should be in Accumulating state"); + }; + *self = Self::Done; + Ok(res) + } +} + +impl DynBlockingSinkState for AggregateState { + fn as_any_mut(&mut self) -> &mut dyn std::any::Any { + self + } } pub struct AggregateSink { agg_exprs: Vec, group_by: Vec, - state: AggregateState, } impl AggregateSink { @@ -25,47 +51,59 @@ impl AggregateSink { Self { agg_exprs, group_by, - state: AggregateState::Accumulating(vec![]), } } - - pub fn boxed(self) -> Box { - Box::new(self) - } } impl BlockingSink for AggregateSink { #[instrument(skip_all, name = "AggregateSink::sink")] - fn sink(&mut self, input: &Arc) -> DaftResult { - if let AggregateState::Accumulating(parts) = &mut self.state { - parts.push(input.clone()); + fn sink( + &self, + input: &Arc, + state_handle: &BlockingSinkState, + ) -> DaftResult { + state_handle.with_state_mut::(|state| { + state.push(input.clone()); Ok(BlockingSinkStatus::NeedMoreInput) - } else { - panic!("AggregateSink should be in Accumulating state"); - } + }) } #[instrument(skip_all, name = "AggregateSink::finalize")] - fn finalize(&mut self) -> DaftResult> { - if let AggregateState::Accumulating(parts) = &mut self.state { - assert!( - !parts.is_empty(), - "We can not finalize AggregateSink with no data" - ); - let concated = MicroPartition::concat( - &parts - .iter() - .map(std::convert::AsRef::as_ref) - .collect::>(), - )?; - let agged = Arc::new(concated.agg(&self.agg_exprs, &self.group_by)?); - self.state = AggregateState::Done(agged.clone()); - Ok(Some(agged.into())) - } else { - panic!("AggregateSink should be in Accumulating state"); + fn finalize( + &self, + states: Vec>, + ) -> DaftResult> { + let mut all_parts = vec![]; + for mut state in states { + let state = state + .as_any_mut() + .downcast_mut::() + .expect("State type mismatch"); + all_parts.extend(state.finalize()?); } + assert!( + !all_parts.is_empty(), + "We can not finalize AggregateSink with no data" + ); + let concated = MicroPartition::concat( + &all_parts + .iter() + .map(std::convert::AsRef::as_ref) + .collect::>(), + )?; + let agged = Arc::new(concated.agg(&self.group_by, &self.agg_exprs)?); + Ok(Some(agged.into())) } + fn name(&self) -> &'static str { "AggregateSink" } + + fn max_concurrency(&self) -> usize { + *NUM_CPUS + } + + fn make_state(&self) -> DaftResult> { + Ok(Box::new(AggregateState::Accumulating(vec![]))) + } } diff --git a/src/daft-local-execution/src/sinks/blocking_sink.rs b/src/daft-local-execution/src/sinks/blocking_sink.rs index 3fcbf8d660..7dcad48dc0 100644 --- a/src/daft-local-execution/src/sinks/blocking_sink.rs +++ b/src/daft-local-execution/src/sinks/blocking_sink.rs @@ -1,17 +1,47 @@ -use std::sync::Arc; +use std::sync::{Arc, Mutex}; use common_display::tree::TreeDisplay; use common_error::DaftResult; use common_runtime::get_compute_runtime; use daft_micropartition::MicroPartition; -use tracing::info_span; +use snafu::ResultExt; +use tracing::{info_span, instrument}; use crate::{ - channel::PipelineChannel, + channel::{create_channel, PipelineChannel, Receiver}, + dispatcher::{Dispatcher, RoundRobinBufferedDispatcher}, pipeline::{PipelineNode, PipelineResultType}, runtime_stats::RuntimeStatsContext, - ExecutionRuntimeHandle, + ExecutionRuntimeHandle, JoinSnafu, TaskSet, }; +pub trait DynBlockingSinkState: Send + Sync { + fn as_any_mut(&mut self) -> &mut dyn std::any::Any; +} + +pub(crate) struct BlockingSinkState { + inner: Mutex>, +} + +impl BlockingSinkState { + fn new(inner: Box) -> Arc { + Arc::new(Self { + inner: Mutex::new(inner), + }) + } + + pub(crate) fn with_state_mut(&self, f: F) -> R + where + F: FnOnce(&mut T) -> R, + { + let mut guard = self.inner.lock().unwrap(); + let state = guard + .as_any_mut() + .downcast_mut::() + .expect("State type mismatch"); + f(state) + } +} + pub enum BlockingSinkStatus { NeedMoreInput, #[allow(dead_code)] @@ -19,24 +49,37 @@ pub enum BlockingSinkStatus { } pub trait BlockingSink: Send + Sync { - fn sink(&mut self, input: &Arc) -> DaftResult; - fn finalize(&mut self) -> DaftResult>; + fn sink( + &self, + input: &Arc, + state_handle: &BlockingSinkState, + ) -> DaftResult; + fn finalize( + &self, + states: Vec>, + ) -> DaftResult>; fn name(&self) -> &'static str; + fn make_state(&self) -> DaftResult>; + fn make_dispatcher(&self, runtime_handle: &ExecutionRuntimeHandle) -> Arc { + Arc::new(RoundRobinBufferedDispatcher::new( + runtime_handle.default_morsel_size(), + )) + } + fn max_concurrency(&self) -> usize; } pub struct BlockingSinkNode { - // use a RW lock - op: Arc>>, + op: Arc, name: &'static str, child: Box, runtime_stats: Arc, } impl BlockingSinkNode { - pub(crate) fn new(op: Box, child: Box) -> Self { + pub(crate) fn new(op: Arc, child: Box) -> Self { let name = op.name(); Self { - op: Arc::new(tokio::sync::Mutex::new(op)), + op, name, child, runtime_stats: RuntimeStatsContext::new(), @@ -45,6 +88,53 @@ impl BlockingSinkNode { pub(crate) fn boxed(self) -> Box { Box::new(self) } + + #[instrument(level = "info", skip_all, name = "BlockingSink::run_worker")] + async fn run_worker( + op: Arc, + mut input_receiver: Receiver, + rt_context: Arc, + ) -> DaftResult> { + let span = info_span!("BlockingSink::Sink"); + let compute_runtime = get_compute_runtime(); + let state_wrapper = BlockingSinkState::new(op.make_state()?); + while let Some(morsel) = input_receiver.recv().await { + let op = op.clone(); + let morsel = morsel.clone(); + let span = span.clone(); + let rt_context = rt_context.clone(); + let state_wrapper = state_wrapper.clone(); + let fut = async move { + rt_context.in_span(&span, || op.sink(morsel.as_data(), &state_wrapper)) + }; + let result = compute_runtime.await_on(fut).await??; + match result { + BlockingSinkStatus::NeedMoreInput => {} + BlockingSinkStatus::Finished => { + break; + } + } + } + + // Take the state out of the Arc and Mutex because we need to return it. + // It should be guaranteed that the ONLY holder of state at this point is this function. + Ok(Arc::into_inner(state_wrapper) + .expect("Completed worker should have exclusive access to state wrapper") + .inner + .into_inner() + .expect("Completed worker should have exclusive access to inner state")) + } + + fn spawn_workers( + op: Arc, + input_receivers: Vec>, + task_set: &mut TaskSet>>, + stats: Arc, + ) { + for input_receiver in input_receivers { + task_set.spawn(Self::run_worker(op.clone(), input_receiver, stats.clone())); + } + } } impl TreeDisplay for BlockingSinkNode { @@ -81,7 +171,7 @@ impl PipelineNode for BlockingSinkNode { runtime_handle: &mut ExecutionRuntimeHandle, ) -> crate::Result { let child = self.child.as_mut(); - let mut child_results_receiver = child + let child_results_receiver = child .start(false, runtime_handle)? .get_receiver_with_stats(&self.runtime_stats); @@ -89,34 +179,45 @@ impl PipelineNode for BlockingSinkNode { let destination_sender = destination_channel.get_next_sender_with_stats(&self.runtime_stats); let op = self.op.clone(); - let rt_context = self.runtime_stats.clone(); + let runtime_stats = self.runtime_stats.clone(); + let num_workers = op.max_concurrency(); + let (input_senders, input_receivers) = (0..num_workers).map(|_| create_channel(1)).unzip(); + let dispatcher = op.make_dispatcher(runtime_handle); runtime_handle.spawn( async move { - let span = info_span!("BlockingSinkNode::execute"); - let compute_runtime = get_compute_runtime(); - while let Some(val) = child_results_receiver.recv().await { - let op = op.clone(); - let span = span.clone(); - let rt_context = rt_context.clone(); - let fut = async move { - let mut guard = op.lock().await; - rt_context.in_span(&span, || guard.sink(val.as_data())) - }; - let result = compute_runtime.await_on(fut).await??; - if matches!(result, BlockingSinkStatus::Finished) { - break; - } + dispatcher + .dispatch(child_results_receiver, input_senders) + .await + }, + self.name(), + ); + + runtime_handle.spawn( + async move { + let mut task_set = TaskSet::new(); + Self::spawn_workers( + op.clone(), + input_receivers, + &mut task_set, + runtime_stats.clone(), + ); + + let mut finished_states = Vec::with_capacity(num_workers); + while let Some(result) = task_set.join_next().await { + let state = result.context(JoinSnafu)??; + finished_states.push(state); } + + let compute_runtime = get_compute_runtime(); let finalized_result = compute_runtime .await_on(async move { - let mut guard = op.lock().await; - rt_context.in_span(&info_span!("BlockingSinkNode::finalize"), || { - guard.finalize() + runtime_stats.in_span(&info_span!("BlockingSinkNode::finalize"), || { + op.finalize(finished_states) }) }) .await??; - if let Some(part) = finalized_result { - let _ = destination_sender.send(part).await; + if let Some(res) = finalized_result { + let _ = destination_sender.send(res).await; } Ok(()) }, diff --git a/src/daft-local-execution/src/sinks/hash_join_build.rs b/src/daft-local-execution/src/sinks/hash_join_build.rs index c8258e281a..55440c3f5f 100644 --- a/src/daft-local-execution/src/sinks/hash_join_build.rs +++ b/src/daft-local-execution/src/sinks/hash_join_build.rs @@ -7,7 +7,9 @@ use daft_micropartition::MicroPartition; use daft_plan::JoinType; use daft_table::{make_probeable_builder, ProbeState, ProbeableBuilder, Table}; -use super::blocking_sink::{BlockingSink, BlockingSinkStatus}; +use super::blocking_sink::{ + BlockingSink, BlockingSinkState, BlockingSinkStatus, DynBlockingSinkState, +}; use crate::pipeline::PipelineResultType; enum ProbeTableState { @@ -74,8 +76,16 @@ impl ProbeTableState { } } +impl DynBlockingSinkState for ProbeTableState { + fn as_any_mut(&mut self) -> &mut dyn std::any::Any { + self + } +} + pub struct HashJoinBuildSink { - probe_table_state: ProbeTableState, + key_schema: SchemaRef, + projection: Vec, + join_type: JoinType, } impl HashJoinBuildSink { @@ -85,13 +95,11 @@ impl HashJoinBuildSink { join_type: &JoinType, ) -> DaftResult { Ok(Self { - probe_table_state: ProbeTableState::new(&key_schema, projection, join_type)?, + key_schema, + projection, + join_type: *join_type, }) } - - pub(crate) fn boxed(self) -> Box { - Box::new(self) - } } impl BlockingSink for HashJoinBuildSink { @@ -99,17 +107,44 @@ impl BlockingSink for HashJoinBuildSink { "HashJoinBuildSink" } - fn sink(&mut self, input: &Arc) -> DaftResult { - self.probe_table_state.add_tables(input)?; - Ok(BlockingSinkStatus::NeedMoreInput) + fn sink( + &self, + input: &Arc, + state_handle: &BlockingSinkState, + ) -> DaftResult { + state_handle.with_state_mut::(|state| { + state.add_tables(input)?; + Ok(BlockingSinkStatus::NeedMoreInput) + }) } - fn finalize(&mut self) -> DaftResult> { - self.probe_table_state.finalize()?; - if let ProbeTableState::Done { probe_state } = &self.probe_table_state { + fn finalize( + &self, + states: Vec>, + ) -> DaftResult> { + assert_eq!(states.len(), 1); + let mut state = states.into_iter().next().unwrap(); + let probe_table_state = state + .as_any_mut() + .downcast_mut::() + .expect("State type mismatch"); + probe_table_state.finalize()?; + if let ProbeTableState::Done { probe_state } = probe_table_state { Ok(Some(probe_state.clone().into())) } else { panic!("finalize should only be called after the probe table is built") } } + + fn max_concurrency(&self) -> usize { + 1 + } + + fn make_state(&self) -> DaftResult> { + Ok(Box::new(ProbeTableState::new( + &self.key_schema, + self.projection.clone(), + &self.join_type, + )?)) + } } diff --git a/src/daft-local-execution/src/sinks/sort.rs b/src/daft-local-execution/src/sinks/sort.rs index 169ea9e55d..12ca9cb66e 100644 --- a/src/daft-local-execution/src/sinks/sort.rs +++ b/src/daft-local-execution/src/sinks/sort.rs @@ -5,18 +5,44 @@ use daft_dsl::ExprRef; use daft_micropartition::MicroPartition; use tracing::instrument; -use super::blocking_sink::{BlockingSink, BlockingSinkStatus}; -use crate::pipeline::PipelineResultType; -pub struct SortSink { - sort_by: Vec, - descending: Vec, - state: SortState, -} +use super::blocking_sink::{ + BlockingSink, BlockingSinkState, BlockingSinkStatus, DynBlockingSinkState, +}; +use crate::{pipeline::PipelineResultType, NUM_CPUS}; enum SortState { Building(Vec>), - #[allow(dead_code)] - Done(Arc), + Done, +} + +impl SortState { + fn push(&mut self, part: Arc) { + if let Self::Building(ref mut parts) = self { + parts.push(part); + } else { + panic!("SortSink should be in Building state"); + } + } + + fn finalize(&mut self) -> DaftResult>> { + let res = if let Self::Building(ref mut parts) = self { + std::mem::take(parts) + } else { + panic!("SortSink should be in Building state"); + }; + *self = Self::Done; + Ok(res) + } +} + +impl DynBlockingSinkState for SortState { + fn as_any_mut(&mut self) -> &mut dyn std::any::Any { + self + } +} +pub struct SortSink { + sort_by: Vec, + descending: Vec, } impl SortSink { @@ -24,46 +50,59 @@ impl SortSink { Self { sort_by, descending, - state: SortState::Building(vec![]), } } - pub fn boxed(self) -> Box { - Box::new(self) - } } impl BlockingSink for SortSink { #[instrument(skip_all, name = "SortSink::sink")] - fn sink(&mut self, input: &Arc) -> DaftResult { - if let SortState::Building(parts) = &mut self.state { - parts.push(input.clone()); - } else { - panic!("SortSink should be in Building state"); - } - Ok(BlockingSinkStatus::NeedMoreInput) + fn sink( + &self, + input: &Arc, + state_handle: &BlockingSinkState, + ) -> DaftResult { + state_handle.with_state_mut::(|state| { + state.push(input.clone()); + Ok(BlockingSinkStatus::NeedMoreInput) + }) } #[instrument(skip_all, name = "SortSink::finalize")] - fn finalize(&mut self) -> DaftResult> { - if let SortState::Building(parts) = &mut self.state { - assert!( - !parts.is_empty(), - "We can not finalize SortSink with no data" - ); - let concated = MicroPartition::concat( - &parts - .iter() - .map(std::convert::AsRef::as_ref) - .collect::>(), - )?; - let sorted = Arc::new(concated.sort(&self.sort_by, &self.descending)?); - self.state = SortState::Done(sorted.clone()); - Ok(Some(sorted.into())) - } else { - panic!("SortSink should be in Building state"); + fn finalize( + &self, + states: Vec>, + ) -> DaftResult> { + let mut parts = Vec::new(); + for mut state in states { + let state = state + .as_any_mut() + .downcast_mut::() + .expect("State type mismatch"); + parts.extend(state.finalize()?); } + assert!( + !parts.is_empty(), + "We can not finalize SortSink with no data" + ); + let concated = MicroPartition::concat( + &parts + .iter() + .map(std::convert::AsRef::as_ref) + .collect::>(), + )?; + let sorted = Arc::new(concated.sort(&self.sort_by, &self.descending)?); + Ok(Some(sorted.into())) } + fn name(&self) -> &'static str { "SortResult" } + + fn make_state(&self) -> DaftResult> { + Ok(Box::new(SortState::Building(Vec::new()))) + } + + fn max_concurrency(&self) -> usize { + *NUM_CPUS + } } diff --git a/src/daft-local-execution/src/sinks/write.rs b/src/daft-local-execution/src/sinks/write.rs index cb67354ae4..015cae68bf 100644 --- a/src/daft-local-execution/src/sinks/write.rs +++ b/src/daft-local-execution/src/sinks/write.rs @@ -19,8 +19,14 @@ use daft_table::Table; use daft_writers::{FileWriter, PhysicalWriterFactory, WriterFactory}; use tracing::instrument; -use super::blocking_sink::{BlockingSink, BlockingSinkStatus}; -use crate::{buffer::RowBasedBuffer, pipeline::PipelineResultType}; +use super::blocking_sink::{ + BlockingSink, BlockingSinkState, BlockingSinkStatus, DynBlockingSinkState, +}; +use crate::{ + buffer::RowBasedBuffer, + dispatcher::{Dispatcher, RoundRobinBufferedDispatcher}, + pipeline::PipelineResultType, +}; // TargetBatchWriter is a writer that writes in batches of rows, i.e. for Parquet where we want to write // a row group at a time. It uses a buffer to accumulate rows until it has enough to write a batch. @@ -310,10 +316,53 @@ impl FileWriter for PartitionedWriter { } } +struct PartitionedWriterFactory { + writer_factory: Arc, Result = Vec
>>, + partition_cols: Vec, +} + +impl WriterFactory for PartitionedWriterFactory { + type Input = Arc; + type Result = Vec
; + + fn create_writer( + &self, + _file_idx: usize, + _partition_values: Option<&Table>, + ) -> DaftResult>> { + Ok(Box::new(PartitionedWriter::new( + self.writer_factory.clone(), + self.partition_cols.clone(), + )) + as Box< + dyn FileWriter, + >) + } +} + +struct WriteState { + writer: Box, Result = Vec
>>, +} + +impl WriteState { + pub fn new( + writer: Box, Result = Vec
>>, + ) -> Self { + Self { writer } + } +} + +impl DynBlockingSinkState for WriteState { + fn as_any_mut(&mut self) -> &mut dyn std::any::Any { + self + } +} + pub struct WriteSink { name: &'static str, - writer: Box, Result = Vec
>>, + writer_factory: Arc, Result = Vec
>>, file_schema: SchemaRef, + partition_cols: Option>, } impl WriteSink { @@ -325,7 +374,7 @@ impl WriteSink { ) -> DaftResult { let estimated_row_size_bytes = data_schema.estimate_row_size_bytes(); let base_writer_factory = PhysicalWriterFactory::new(file_info.clone()); - let (writer, name) = match file_info.file_format { + let (writer_factory, name) = match file_info.file_format { FileFormat::Parquet => { let target_in_memory_file_size = cfg.parquet_target_filesize as f64 * cfg.parquet_inflation_factor; @@ -352,26 +401,23 @@ impl WriteSink { target_in_memory_chunk_rows: target_row_group_rows, }; + let file_writer_factory = Arc::new(TargetFileSizeWriterFactory { + writer_factory: Arc::new(row_group_writer_factory), + target_in_memory_file_rows: target_file_rows, + }); + if let Some(partition_cols) = &file_info.partition_cols { - let file_writer_factory = TargetFileSizeWriterFactory { - writer_factory: Arc::new(row_group_writer_factory), - target_in_memory_file_rows: target_file_rows, - }; - let partitioned_writer = Box::new(PartitionedWriter::new( - Arc::new(file_writer_factory), - partition_cols.clone(), - )); + let partitioned_writer_factory = Arc::new(PartitionedWriterFactory { + writer_factory: file_writer_factory, + partition_cols: partition_cols.clone(), + }); ( - partitioned_writer as Box>, + partitioned_writer_factory as Arc>, "PartitionedParquetWrite", ) } else { ( - Box::new(TargetFileSizeWriter::new( - target_file_rows, - Arc::new(row_group_writer_factory), - None, - )?) as Box>, + file_writer_factory as Arc>, "UnpartitionedParquetWrite", ) } @@ -385,26 +431,23 @@ impl WriteSink { target_in_memory_file_size } as usize; + let file_writer_factory = Arc::new(TargetFileSizeWriterFactory { + writer_factory: Arc::new(base_writer_factory), + target_in_memory_file_rows: target_file_rows, + }); + if let Some(partition_cols) = &file_info.partition_cols { - let file_writer_factory = TargetFileSizeWriterFactory { - writer_factory: Arc::new(base_writer_factory), - target_in_memory_file_rows: target_file_rows, - }; - let partitioned_writer = Box::new(PartitionedWriter::new( - Arc::new(file_writer_factory), - partition_cols.clone(), - )); + let partitioned_writer_factory = Arc::new(PartitionedWriterFactory { + writer_factory: file_writer_factory, + partition_cols: partition_cols.clone(), + }); ( - partitioned_writer as Box>, + partitioned_writer_factory as Arc>, "PartitionedCsvWrite", ) } else { ( - Box::new(TargetFileSizeWriter::new( - target_file_rows, - Arc::new(base_writer_factory), - None, - )?) as Box>, + file_writer_factory as Arc>, "UnpartitionedCsvWrite", ) } @@ -413,8 +456,9 @@ impl WriteSink { }; Ok(Self { name, - writer, + writer_factory, file_schema, + partition_cols: file_info.partition_cols, }) } pub fn boxed(self) -> Box { @@ -424,22 +468,67 @@ impl WriteSink { impl BlockingSink for WriteSink { #[instrument(skip_all, name = "WriteSink::sink")] - fn sink(&mut self, input: &Arc) -> DaftResult { - self.writer.write(input)?; - Ok(BlockingSinkStatus::NeedMoreInput) + fn sink( + &self, + input: &Arc, + state_handle: &BlockingSinkState, + ) -> DaftResult { + state_handle.with_state_mut::(|state| { + state.writer.write(input)?; + Ok(BlockingSinkStatus::NeedMoreInput) + }) } #[instrument(skip_all, name = "WriteSink::finalize")] - fn finalize(&mut self) -> DaftResult> { - let result = self.writer.close()?; + fn finalize( + &self, + states: Vec>, + ) -> DaftResult> { + let mut results = vec![]; + for mut state in states { + let state = state + .as_any_mut() + .downcast_mut::() + .expect("State type mismatch"); + results.extend(state.writer.close()?); + } let mp = Arc::new(MicroPartition::new_loaded( self.file_schema.clone(), - result.into(), + results.into(), None, )); Ok(Some(mp.into())) } + fn name(&self) -> &'static str { self.name } + + fn make_state(&self) -> DaftResult> { + let writer = self.writer_factory.create_writer(0, None)?; + Ok(Box::new(WriteState::new(writer)) as Box) + } + + fn max_concurrency(&self) -> usize { + if self.partition_cols.is_some() { + *crate::NUM_CPUS + } else { + 1 + } + } + + fn make_dispatcher( + &self, + runtime_handle: &crate::ExecutionRuntimeHandle, + ) -> Arc { + if let Some(partition_cols) = &self.partition_cols { + Arc::new(crate::dispatcher::PartitionedDispatcher::new( + partition_cols.clone(), + )) + } else { + Arc::new(RoundRobinBufferedDispatcher::new( + runtime_handle.default_morsel_size(), + )) + } + } } diff --git a/src/daft-writers/src/physical.rs b/src/daft-writers/src/physical.rs index f94828d734..81cb3dfbfe 100644 --- a/src/daft-writers/src/physical.rs +++ b/src/daft-writers/src/physical.rs @@ -1,6 +1,7 @@ use std::sync::Arc; -use common_error::DaftResult; +use common_error::{DaftError, DaftResult}; +use common_file_formats::FileFormat; use daft_micropartition::MicroPartition; use daft_plan::OutputFileInfo; use daft_table::Table; @@ -34,24 +35,43 @@ impl WriterFactory for PhysicalWriterFactory { match self.native { true => unimplemented!(), false => { - #[cfg(feature = "python")] - { - let writer = crate::python::create_pyarrow_file_writer( - &self.output_file_info.root_dir, - file_idx, - &self.output_file_info.compression, - &self.output_file_info.io_config, - self.output_file_info.file_format, - partition_values, - )?; - Ok(writer) - } - - #[cfg(not(feature = "python"))] - { - unimplemented!() - } + let writer = create_pyarrow_file_writer( + &self.output_file_info.root_dir, + file_idx, + &self.output_file_info.compression, + &self.output_file_info.io_config, + self.output_file_info.file_format, + partition_values, + )?; + Ok(writer) } } } } + +pub fn create_pyarrow_file_writer( + root_dir: &str, + file_idx: usize, + compression: &Option, + io_config: &Option, + format: FileFormat, + partition: Option<&Table>, +) -> DaftResult, Result = Option
>>> { + match format { + #[cfg(feature = "python")] + FileFormat::Parquet => Ok(Box::new(crate::python::PyArrowWriter::new_parquet_writer( + root_dir, + file_idx, + compression, + io_config, + partition, + )?)), + #[cfg(feature = "python")] + FileFormat::Csv => Ok(Box::new(crate::python::PyArrowWriter::new_csv_writer( + root_dir, file_idx, io_config, partition, + )?)), + _ => Err(DaftError::ComputeError( + "Unsupported file format for physical write".to_string(), + )), + } +} diff --git a/src/daft-writers/src/python.rs b/src/daft-writers/src/python.rs index 5b0321f60a..4047d54848 100644 --- a/src/daft-writers/src/python.rs +++ b/src/daft-writers/src/python.rs @@ -1,40 +1,12 @@ use std::sync::Arc; -use common_error::{DaftError, DaftResult}; -use common_file_formats::FileFormat; +use common_error::DaftResult; use daft_micropartition::{python::PyMicroPartition, MicroPartition}; use daft_table::{python::PyTable, Table}; use pyo3::{types::PyAnyMethods, PyObject, Python}; use crate::FileWriter; -pub fn create_pyarrow_file_writer( - root_dir: &str, - file_idx: usize, - compression: &Option, - io_config: &Option, - format: FileFormat, - partition: Option<&Table>, -) -> DaftResult, Result = Option
>>> { - match format { - #[cfg(feature = "python")] - FileFormat::Parquet => Ok(Box::new(PyArrowWriter::new_parquet_writer( - root_dir, - file_idx, - compression, - io_config, - partition, - )?)), - #[cfg(feature = "python")] - FileFormat::Csv => Ok(Box::new(PyArrowWriter::new_csv_writer( - root_dir, file_idx, io_config, partition, - )?)), - _ => Err(DaftError::ComputeError( - "Unsupported file format for physical write".to_string(), - )), - } -} - pub struct PyArrowWriter { py_writer: PyObject, } From 85ab44c4b8502edeac64e65be7832dd017c5e07f Mon Sep 17 00:00:00 2001 From: Colin Ho Date: Mon, 28 Oct 2024 22:08:38 -0700 Subject: [PATCH 10/15] agg fix --- daft/io/writer.py | 50 +++---------------- daft/table/partitioning.py | 38 +++++++++----- .../src/sinks/aggregate.rs | 2 +- 3 files changed, 32 insertions(+), 58 deletions(-) diff --git a/daft/io/writer.py b/daft/io/writer.py index 141c4dd521..d76dc6122e 100644 --- a/daft/io/writer.py +++ b/daft/io/writer.py @@ -1,6 +1,6 @@ import uuid from abc import ABC, abstractmethod -from typing import Dict, Optional, Union +from typing import Optional, Union from daft.daft import IOConfig from daft.dependencies import pa, pacsv, pq @@ -11,38 +11,13 @@ ) from daft.series import Series from daft.table.micropartition import MicroPartition +from daft.table.partitioning import ( + partition_strings_to_path, + partition_values_to_str_mapping, +) from daft.table.table import Table -def partition_values_to_str_mapping( - partition_values: Table, -) -> Dict[str, str]: - null_part = Series.from_pylist( - [None] - ) # This is to ensure that the null values are replaced with the default_partition_fallback value - pkey_names = partition_values.column_names() - - partition_strings = {} - - for c in pkey_names: - column = partition_values.get_column(c) - string_names = column._to_str_values() - null_filled = column.is_null().if_else(null_part, string_names) - partition_strings[c] = null_filled.to_pylist()[0] - - return partition_strings - - -def partition_string_mapping_to_postfix( - partition_strings: Dict[str, str], - default_partition_fallback: str, -) -> str: - postfix = "/".join( - f"{k}={v if v is not None else default_partition_fallback}" for k, v in partition_strings.items() - ) - return postfix - - class FileWriterBase(ABC): def __init__( self, @@ -63,8 +38,7 @@ def __init__( self.partition_values = partition_values if self.partition_values is not None: partition_strings = partition_values_to_str_mapping(self.partition_values) - postfix = partition_string_mapping_to_postfix(partition_strings, default_partition_fallback) - self.dir_path = f"{self.resolved_path}/{postfix}" + self.dir_path = partition_strings_to_path(self.resolved_path, partition_strings, default_partition_fallback) else: self.dir_path = f"{self.resolved_path}" @@ -75,18 +49,6 @@ def __init__( self.compression = compression if compression is not None else "none" self.current_writer: Optional[Union[pq.ParquetWriter, pacsv.CSVWriter]] = None - @abstractmethod - def _create_writer(self, schema: pa.Schema) -> Union[pq.ParquetWriter, pacsv.CSVWriter]: - """Create a writer instance for the specific file format. - - Args: - schema: PyArrow schema defining the structure of the data to be written. - - Returns: - A writer instance specific to the file format (Parquet or CSV). - """ - pass - @abstractmethod def write(self, table: MicroPartition) -> None: """Write data to the file using the appropriate writer. diff --git a/daft/table/partitioning.py b/daft/table/partitioning.py index 70a590cb45..8876b9a90e 100644 --- a/daft/table/partitioning.py +++ b/daft/table/partitioning.py @@ -1,13 +1,16 @@ -from typing import Dict, List, Optional +from typing import Dict, List, Optional, Union from daft import Series from daft.expressions import ExpressionsProjection +from daft.table.table import Table from .micropartition import MicroPartition def partition_strings_to_path( - root_path: str, parts: Dict[str, str], partition_null_fallback: str = "__HIVE_DEFAULT_PARTITION__" + root_path: str, + parts: Dict[str, str], + partition_null_fallback: str = "__HIVE_DEFAULT_PARTITION__", ) -> str: keys = parts.keys() values = [partition_null_fallback if value is None else value for value in parts.values()] @@ -15,6 +18,25 @@ def partition_strings_to_path( return f"{root_path}/{postfix}" +def partition_values_to_str_mapping( + partition_values: Union[MicroPartition, Table], +) -> Dict[str, str]: + null_part = Series.from_pylist( + [None] + ) # This is to ensure that the null values are replaced with the default_partition_fallback value + pkey_names = partition_values.column_names() + + partition_strings = {} + + for c in pkey_names: + column = partition_values.get_column(c) + string_names = column._to_str_values() + null_filled = column.is_null().if_else(null_part, string_names) + partition_strings[c] = null_filled.to_pylist()[0] + + return partition_strings + + class PartitionedTable: def __init__(self, table: MicroPartition, partition_keys: Optional[ExpressionsProjection]): self.table = table @@ -56,20 +78,10 @@ def partition_values_str(self) -> Optional[MicroPartition]: If the table is not partitioned, returns None. """ - null_part = Series.from_pylist([None]) partition_values = self.partition_values() if partition_values is None: return None else: - pkey_names = partition_values.column_names() - - partition_strings = {} - - for c in pkey_names: - column = partition_values.get_column(c) - string_names = column._to_str_values() - null_filled = column.is_null().if_else(null_part, string_names) - partition_strings[c] = null_filled - + partition_strings = partition_values_to_str_mapping(partition_values) return MicroPartition.from_pydict(partition_strings) diff --git a/src/daft-local-execution/src/sinks/aggregate.rs b/src/daft-local-execution/src/sinks/aggregate.rs index 7fc95dd607..7b5102ffc3 100644 --- a/src/daft-local-execution/src/sinks/aggregate.rs +++ b/src/daft-local-execution/src/sinks/aggregate.rs @@ -91,7 +91,7 @@ impl BlockingSink for AggregateSink { .map(std::convert::AsRef::as_ref) .collect::>(), )?; - let agged = Arc::new(concated.agg(&self.group_by, &self.agg_exprs)?); + let agged = Arc::new(concated.agg(&self.agg_exprs, &self.group_by)?); Ok(Some(agged.into())) } From 0ce9dcc303f16fb9fdfcfe121fa90bbe9a639d5c Mon Sep 17 00:00:00 2001 From: Colin Ho Date: Mon, 28 Oct 2024 23:11:27 -0700 Subject: [PATCH 11/15] move buffer --- Cargo.lock | 17 +- Cargo.toml | 1 + daft/io/writer.py | 5 +- daft/table/partitioning.py | 4 +- src/common/buffer/Cargo.toml | 15 + .../buffer.rs => common/buffer/src/lib.rs} | 76 ++- src/daft-local-execution/Cargo.toml | 2 +- src/daft-local-execution/src/dispatcher.rs | 6 +- .../src/intermediate_ops/intermediate_op.rs | 4 +- src/daft-local-execution/src/lib.rs | 1 - src/daft-local-execution/src/pipeline.rs | 121 +++- src/daft-local-execution/src/sinks/mod.rs | 3 +- .../src/sinks/partitioned_write.rs | 258 +++++++++ .../src/sinks/unpartitioned_write.rs | 234 ++++++++ src/daft-local-execution/src/sinks/write.rs | 534 ------------------ src/daft-micropartition/Cargo.toml | 1 + src/daft-micropartition/src/micropartition.rs | 23 + src/daft-writers/Cargo.toml | 1 + src/daft-writers/src/batch.rs | 84 +++ src/daft-writers/src/lib.rs | 2 + 20 files changed, 801 insertions(+), 591 deletions(-) create mode 100644 src/common/buffer/Cargo.toml rename src/{daft-local-execution/src/buffer.rs => common/buffer/src/lib.rs} (57%) create mode 100644 src/daft-local-execution/src/sinks/partitioned_write.rs create mode 100644 src/daft-local-execution/src/sinks/unpartitioned_write.rs delete mode 100644 src/daft-local-execution/src/sinks/write.rs create mode 100644 src/daft-writers/src/batch.rs diff --git a/Cargo.lock b/Cargo.lock index a275ca49e3..47c2a6feda 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -1298,6 +1298,18 @@ dependencies = [ "pyo3", ] +[[package]] +name = "common-buffer" +version = "0.3.0-dev0" +dependencies = [ + "common-error", + "futures", + "lazy_static", + "log", + "oneshot", + "tokio", +] + [[package]] name = "common-daft-config" version = "0.3.0-dev0" @@ -1681,6 +1693,7 @@ dependencies = [ name = "daft" version = "0.3.0-dev0" dependencies = [ + "common-buffer", "common-daft-config", "common-display", "common-file-formats", @@ -1974,6 +1987,7 @@ name = "daft-local-execution" version = "0.3.0-dev0" dependencies = [ "async-trait", + "common-buffer", "common-daft-config", "common-display", "common-error", @@ -1995,7 +2009,6 @@ dependencies = [ "daft-writers", "futures", "indexmap 2.5.0", - "itertools 0.11.0", "lazy_static", "log", "num-format", @@ -2012,6 +2025,7 @@ version = "0.3.0-dev0" dependencies = [ "arrow2", "bincode", + "common-buffer", "common-error", "common-file-formats", "common-runtime", @@ -2254,6 +2268,7 @@ name = "daft-writers" version = "0.3.0-dev0" dependencies = [ "async-trait", + "common-buffer", "common-daft-config", "common-display", "common-error", diff --git a/Cargo.toml b/Cargo.toml index 27d5f98309..fdc7ef9fd0 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -1,4 +1,5 @@ [dependencies] +common-buffer = {path = "src/common/buffer", default-features = false} common-daft-config = {path = "src/common/daft-config", default-features = false} common-display = {path = "src/common/display", default-features = false} common-file-formats = {path = "src/common/file-formats", default-features = false} diff --git a/daft/io/writer.py b/daft/io/writer.py index d76dc6122e..bf3f8a3370 100644 --- a/daft/io/writer.py +++ b/daft/io/writer.py @@ -37,7 +37,10 @@ def __init__( self.file_name = f"{uuid.uuid4()}-{file_idx}.{file_format}" self.partition_values = partition_values if self.partition_values is not None: - partition_strings = partition_values_to_str_mapping(self.partition_values) + partition_strings = { + key: values.to_pylist()[0] + for key, values in partition_values_to_str_mapping(self.partition_values).items() + } self.dir_path = partition_strings_to_path(self.resolved_path, partition_strings, default_partition_fallback) else: self.dir_path = f"{self.resolved_path}" diff --git a/daft/table/partitioning.py b/daft/table/partitioning.py index 8876b9a90e..2333a198e5 100644 --- a/daft/table/partitioning.py +++ b/daft/table/partitioning.py @@ -20,7 +20,7 @@ def partition_strings_to_path( def partition_values_to_str_mapping( partition_values: Union[MicroPartition, Table], -) -> Dict[str, str]: +) -> Dict[str, Series]: null_part = Series.from_pylist( [None] ) # This is to ensure that the null values are replaced with the default_partition_fallback value @@ -32,7 +32,7 @@ def partition_values_to_str_mapping( column = partition_values.get_column(c) string_names = column._to_str_values() null_filled = column.is_null().if_else(null_part, string_names) - partition_strings[c] = null_filled.to_pylist()[0] + partition_strings[c] = null_filled return partition_strings diff --git a/src/common/buffer/Cargo.toml b/src/common/buffer/Cargo.toml new file mode 100644 index 0000000000..f3f41bbce6 --- /dev/null +++ b/src/common/buffer/Cargo.toml @@ -0,0 +1,15 @@ +[dependencies] +common-error = {path = "../error", default-features = false} +futures = {workspace = true} +lazy_static = {workspace = true} +log = {workspace = true} +oneshot = "0.1.8" +tokio = {workspace = true} + +[lints] +workspace = true + +[package] +edition = {workspace = true} +name = "common-buffer" +version = {workspace = true} diff --git a/src/daft-local-execution/src/buffer.rs b/src/common/buffer/src/lib.rs similarity index 57% rename from src/daft-local-execution/src/buffer.rs rename to src/common/buffer/src/lib.rs index 5b3cc60f3b..4028940eef 100644 --- a/src/daft-local-execution/src/buffer.rs +++ b/src/common/buffer/src/lib.rs @@ -1,16 +1,47 @@ use std::{cmp::Ordering::*, collections::VecDeque, sync::Arc}; use common_error::DaftResult; -use daft_micropartition::MicroPartition; + +#[allow(clippy::len_without_is_empty)] +pub trait Bufferable { + fn len(&self) -> usize; + fn slice(&self, start: usize, end: usize) -> DaftResult + where + Self: Sized; + fn concat(parts: &[&Self]) -> DaftResult + where + Self: Sized; +} + +impl Bufferable for Arc +where + T: Bufferable, +{ + fn len(&self) -> usize { + (**self).len() + } + + fn concat(parts: &[&Self]) -> DaftResult { + // Deref twice: once for the reference, once for the Arc + let inner_parts: Vec<&T> = parts.iter().map(|p| p.as_ref()).collect(); + let concated = T::concat(&inner_parts)?; + Ok(Self::new(concated)) + } + + fn slice(&self, start: usize, end: usize) -> DaftResult { + let sliced = (**self).slice(start, end)?; + Ok(Self::new(sliced)) + } +} // A buffer that accumulates morsels until a threshold is reached -pub struct RowBasedBuffer { - pub buffer: VecDeque>, +pub struct RowBasedBuffer { + pub buffer: VecDeque, pub curr_len: usize, pub threshold: usize, } -impl RowBasedBuffer { +impl RowBasedBuffer { pub fn new(threshold: usize) -> Self { assert!(threshold > 0); Self { @@ -21,7 +52,7 @@ impl RowBasedBuffer { } // Push a morsel to the buffer - pub fn push(&mut self, part: Arc) { + pub fn push(&mut self, part: B) { self.curr_len += part.len(); self.buffer.push_back(part); } @@ -31,7 +62,7 @@ impl RowBasedBuffer { // - If the buffer has exactly enough morsels, return the morsels // - If the buffer has more than enough morsels, return a vec of morsels, each correctly sized to the threshold. // The remaining morsels will be pushed back to the buffer - pub fn pop_enough(&mut self) -> DaftResult>>> { + pub fn pop_enough(&mut self) -> DaftResult>> { match self.curr_len.cmp(&self.threshold) { Less => Ok(None), Equal => { @@ -40,34 +71,26 @@ impl RowBasedBuffer { self.curr_len = 0; Ok(Some(vec![part])) } else { - let chunk = MicroPartition::concat( - &std::mem::take(&mut self.buffer) - .iter() - .map(|x| x.as_ref()) - .collect::>(), - )?; + let chunk = + B::concat(&std::mem::take(&mut self.buffer).iter().collect::>())?; self.curr_len = 0; - Ok(Some(vec![Arc::new(chunk)])) + Ok(Some(vec![chunk])) } } Greater => { let num_ready_chunks = self.curr_len / self.threshold; - let concated = MicroPartition::concat( - &std::mem::take(&mut self.buffer) - .iter() - .map(|x| x.as_ref()) - .collect::>(), - )?; + let concated = + B::concat(&std::mem::take(&mut self.buffer).iter().collect::>())?; let mut start = 0; let mut parts_to_return = Vec::with_capacity(num_ready_chunks); for _ in 0..num_ready_chunks { let end = start + self.threshold; - let part = Arc::new(concated.slice(start, end)?); + let part = concated.slice(start, end)?; parts_to_return.push(part); start = end; } if start < concated.len() { - let part = Arc::new(concated.slice(start, concated.len())?); + let part = concated.slice(start, concated.len())?; self.curr_len = part.len(); self.buffer.push_back(part); } else { @@ -79,19 +102,14 @@ impl RowBasedBuffer { } // Pop all morsels in the buffer regardless of the threshold - pub fn pop_all(&mut self) -> DaftResult>> { + pub fn pop_all(&mut self) -> DaftResult> { assert!(self.curr_len < self.threshold); if self.buffer.is_empty() { Ok(None) } else { - let concated = MicroPartition::concat( - &std::mem::take(&mut self.buffer) - .iter() - .map(|x| x.as_ref()) - .collect::>(), - )?; + let concated = B::concat(&self.buffer.iter().collect::>())?; self.curr_len = 0; - Ok(Some(Arc::new(concated))) + Ok(Some(concated)) } } } diff --git a/src/daft-local-execution/Cargo.toml b/src/daft-local-execution/Cargo.toml index c6b14c12b6..9757487993 100644 --- a/src/daft-local-execution/Cargo.toml +++ b/src/daft-local-execution/Cargo.toml @@ -1,5 +1,6 @@ [dependencies] async-trait = {workspace = true} +common-buffer = {path = "../common/buffer", default-features = false} common-daft-config = {path = "../common/daft-config", default-features = false} common-display = {path = "../common/display", default-features = false} common-error = {path = "../common/error", default-features = false} @@ -21,7 +22,6 @@ daft-table = {path = "../daft-table", default-features = false} daft-writers = {path = "../daft-writers", default-features = false} futures = {workspace = true} indexmap = {workspace = true} -itertools = {workspace = true} lazy_static = {workspace = true} log = {workspace = true} num-format = "0.4.4" diff --git a/src/daft-local-execution/src/dispatcher.rs b/src/daft-local-execution/src/dispatcher.rs index 161b7de3bb..5c50b368b1 100644 --- a/src/daft-local-execution/src/dispatcher.rs +++ b/src/daft-local-execution/src/dispatcher.rs @@ -1,13 +1,11 @@ use std::sync::Arc; use async_trait::async_trait; +use common_buffer::RowBasedBuffer; use common_error::DaftResult; use daft_dsl::ExprRef; -use crate::{ - buffer::RowBasedBuffer, channel::Sender, pipeline::PipelineResultType, - runtime_stats::CountingReceiver, -}; +use crate::{channel::Sender, pipeline::PipelineResultType, runtime_stats::CountingReceiver}; #[async_trait] pub(crate) trait Dispatcher { diff --git a/src/daft-local-execution/src/intermediate_ops/intermediate_op.rs b/src/daft-local-execution/src/intermediate_ops/intermediate_op.rs index 82cdeb2151..5e9fd71239 100644 --- a/src/daft-local-execution/src/intermediate_ops/intermediate_op.rs +++ b/src/daft-local-execution/src/intermediate_ops/intermediate_op.rs @@ -1,5 +1,6 @@ use std::sync::{Arc, Mutex}; +use common_buffer::RowBasedBuffer; use common_display::tree::TreeDisplay; use common_error::DaftResult; use common_runtime::get_compute_runtime; @@ -7,7 +8,6 @@ use daft_micropartition::MicroPartition; use tracing::{info_span, instrument}; use crate::{ - buffer::RowBasedBuffer, channel::{create_channel, PipelineChannel, Receiver, Sender}, pipeline::{PipelineNode, PipelineResultType}, runtime_stats::{CountingReceiver, CountingSender, RuntimeStatsContext}, @@ -175,7 +175,7 @@ impl IntermediateNode { }; for (idx, mut receiver) in receivers.into_iter().enumerate() { - let mut buffer = RowBasedBuffer::new(morsel_size); + let mut buffer = RowBasedBuffer::>::new(morsel_size); while let Some(morsel) = receiver.recv().await { if morsel.should_broadcast() { for worker_sender in &worker_senders { diff --git a/src/daft-local-execution/src/lib.rs b/src/daft-local-execution/src/lib.rs index 448d8408ad..4a8a2cddca 100644 --- a/src/daft-local-execution/src/lib.rs +++ b/src/daft-local-execution/src/lib.rs @@ -1,6 +1,5 @@ #![feature(let_chains)] #![feature(hash_raw_entry)] -mod buffer; mod channel; mod dispatcher; mod intermediate_ops; diff --git a/src/daft-local-execution/src/pipeline.rs b/src/daft-local-execution/src/pipeline.rs index a46f40871a..c07089fd74 100644 --- a/src/daft-local-execution/src/pipeline.rs +++ b/src/daft-local-execution/src/pipeline.rs @@ -1,8 +1,9 @@ -use std::{collections::HashMap, sync::Arc}; +use std::{cmp::min, collections::HashMap, sync::Arc}; use common_daft_config::DaftExecutionConfig; use common_display::{mermaid::MermaidDisplayVisitor, tree::TreeDisplay}; use common_error::DaftResult; +use common_file_formats::FileFormat; use daft_core::{ datatypes::Field, prelude::{Schema, SchemaRef}, @@ -16,6 +17,7 @@ use daft_physical_plan::{ }; use daft_plan::{populate_aggregation_stages, JoinType}; use daft_table::ProbeState; +use daft_writers::{PhysicalWriterFactory, TargetBatchWriterFactory}; use indexmap::IndexSet; use snafu::ResultExt; @@ -29,10 +31,16 @@ use crate::{ unpivot::UnpivotOperator, }, sinks::{ - aggregate::AggregateSink, blocking_sink::BlockingSinkNode, concat::ConcatSink, - hash_join_build::HashJoinBuildSink, limit::LimitSink, - outer_hash_join_probe::OuterHashJoinProbeSink, sort::SortSink, - streaming_sink::StreamingSinkNode, write::WriteSink, + aggregate::AggregateSink, + blocking_sink::BlockingSinkNode, + concat::ConcatSink, + hash_join_build::HashJoinBuildSink, + limit::LimitSink, + outer_hash_join_probe::OuterHashJoinProbeSink, + partitioned_write::{PartitionedWriteSink, PartitionedWriterFactory}, + sort::SortSink, + streaming_sink::StreamingSinkNode, + unpartitioned_write::{TargetFileSizeWriterFactory, UnpartitionedWriteSink}, }, sources::{empty_scan::EmptyScanSource, in_memory::InMemorySource}, ExecutionRuntimeHandle, PipelineCreationSnafu, @@ -412,16 +420,99 @@ pub fn physical_plan_to_pipeline( .. }) => { let child_node = physical_plan_to_pipeline(input, psets, cfg)?; - let write_sink = WriteSink::new( - file_info.clone(), - data_schema.clone(), - file_schema.clone(), - cfg, - ) - .context(PipelineCreationSnafu { - plan_name: physical_plan.name(), - })?; - BlockingSinkNode::new(Arc::new(write_sink), child_node).boxed() + let estimated_row_size_bytes = data_schema.estimate_row_size_bytes(); + let base_writer_factory = PhysicalWriterFactory::new(file_info.clone()); + let write_sink = match file_info.file_format { + FileFormat::Parquet => { + let target_in_memory_file_size = + cfg.parquet_target_filesize as f64 * cfg.parquet_inflation_factor; + let target_in_memory_row_group_size = + cfg.parquet_target_row_group_size as f64 * cfg.parquet_inflation_factor; + + let target_file_rows = if estimated_row_size_bytes > 0.0 { + target_in_memory_file_size / estimated_row_size_bytes + } else { + target_in_memory_file_size + } as usize; + + let target_row_group_rows = min( + target_file_rows, + if estimated_row_size_bytes > 0.0 { + target_in_memory_row_group_size / estimated_row_size_bytes + } else { + target_in_memory_row_group_size + } as usize, + ); + + let row_group_writer_factory = TargetBatchWriterFactory::new( + Arc::new(base_writer_factory), + target_row_group_rows, + ); + + let file_writer_factory = TargetFileSizeWriterFactory::new( + Arc::new(row_group_writer_factory), + target_file_rows, + ); + + if let Some(partition_cols) = &file_info.partition_cols { + let partitioned_writer_factory = PartitionedWriterFactory::new( + Arc::new(file_writer_factory), + partition_cols.clone(), + ); + PartitionedWriteSink::new( + "PartitionedParquetWrite", + partition_cols.clone(), + partitioned_writer_factory, + file_schema.clone(), + ) + .arced() + } else { + UnpartitionedWriteSink::new( + "UnpartitionedParquetWrite", + file_writer_factory, + file_schema.clone(), + ) + .arced() + } + } + FileFormat::Csv => { + let target_in_memory_file_size = + cfg.csv_target_filesize as f64 * cfg.csv_inflation_factor; + let target_file_rows = if estimated_row_size_bytes > 0.0 { + target_in_memory_file_size / estimated_row_size_bytes + } else { + target_in_memory_file_size + } as usize; + + let file_writer_factory = TargetFileSizeWriterFactory::new( + Arc::new(base_writer_factory), + target_file_rows, + ); + + if let Some(partition_cols) = &file_info.partition_cols { + let partitioned_writer_factory = PartitionedWriterFactory::new( + Arc::new(file_writer_factory), + partition_cols.clone(), + ); + PartitionedWriteSink::new( + "PartitionedCsvWrite", + partition_cols.clone(), + partitioned_writer_factory, + file_schema.clone(), + ) + .arced() + } else { + UnpartitionedWriteSink::new( + "UnpartitionedCsvWrite", + file_writer_factory, + file_schema.clone(), + ) + .arced() + } + } + _ => unreachable!("Physical write should only support Parquet and CSV"), + }; + BlockingSinkNode::new(write_sink, child_node).boxed() } }; diff --git a/src/daft-local-execution/src/sinks/mod.rs b/src/daft-local-execution/src/sinks/mod.rs index 64366385c3..64f0c5b09d 100644 --- a/src/daft-local-execution/src/sinks/mod.rs +++ b/src/daft-local-execution/src/sinks/mod.rs @@ -4,6 +4,7 @@ pub mod concat; pub mod hash_join_build; pub mod limit; pub mod outer_hash_join_probe; +pub mod partitioned_write; pub mod sort; pub mod streaming_sink; -pub mod write; +pub mod unpartitioned_write; diff --git a/src/daft-local-execution/src/sinks/partitioned_write.rs b/src/daft-local-execution/src/sinks/partitioned_write.rs new file mode 100644 index 0000000000..2b0dce1213 --- /dev/null +++ b/src/daft-local-execution/src/sinks/partitioned_write.rs @@ -0,0 +1,258 @@ +use std::{ + collections::{hash_map::RawEntryMut, HashMap}, + sync::Arc, +}; + +use common_error::DaftResult; +use daft_core::{ + prelude::{AsArrow, SchemaRef}, + utils::identity_hash_set::IndexHash, +}; +use daft_dsl::ExprRef; +use daft_io::IOStatsContext; +use daft_micropartition::MicroPartition; +use daft_table::Table; +use daft_writers::{FileWriter, WriterFactory}; +use tracing::instrument; + +use super::blocking_sink::{ + BlockingSink, BlockingSinkState, BlockingSinkStatus, DynBlockingSinkState, +}; +use crate::{ + dispatcher::{Dispatcher, PartitionedDispatcher}, + pipeline::PipelineResultType, +}; + +/// PartitionedWriter is a writer that partitions the input data by a set of columns, and writes each partition +/// to a separate file. It uses a map to keep track of the writers for each partition. +struct PartitionedWriter { + per_partition_writers: + HashMap, Result = Vec
>>>, + saved_partition_values: Vec
, + writer_factory: Arc, Result = Vec
>>, + partition_by: Vec, +} + +impl PartitionedWriter { + pub fn new( + writer_factory: Arc, Result = Vec
>>, + partition_by: Vec, + ) -> Self { + Self { + per_partition_writers: HashMap::new(), + saved_partition_values: vec![], + writer_factory, + partition_by, + } + } + + fn partition( + partition_cols: &[ExprRef], + data: &Arc, + ) -> DaftResult<(Vec
, Table)> { + let data = data.concat_or_get(IOStatsContext::new("MicroPartition::partition_by_value"))?; + let table = data.first().unwrap(); + let (split_tables, partition_values) = table.partition_by_value(partition_cols)?; + Ok((split_tables, partition_values)) + } +} + +impl FileWriter for PartitionedWriter { + type Input = Arc; + type Result = Vec
; + + fn write(&mut self, input: &Arc) -> DaftResult<()> { + let (split_tables, partition_values) = + Self::partition(self.partition_by.as_slice(), input)?; + let partition_values_hash = partition_values.hash_rows()?; + for (idx, (table, partition_value_hash)) in split_tables + .into_iter() + .zip(partition_values_hash.as_arrow().values_iter()) + .enumerate() + { + let partition_value_row = partition_values.slice(idx, idx + 1)?; + let entry = self.per_partition_writers.raw_entry_mut().from_hash( + *partition_value_hash, + |other| { + (*partition_value_hash == other.hash) && { + let other_table = + self.saved_partition_values.get(other.idx as usize).unwrap(); + other_table == &partition_value_row + } + }, + ); + match entry { + RawEntryMut::Vacant(entry) => { + let mut writer = self + .writer_factory + .create_writer(0, Some(partition_value_row.as_ref()))?; + writer.write(&Arc::new(MicroPartition::new_loaded( + table.schema.clone(), + vec![table].into(), + None, + )))?; + entry.insert_hashed_nocheck( + *partition_value_hash, + IndexHash { + idx: self.saved_partition_values.len() as u64, + hash: *partition_value_hash, + }, + writer, + ); + self.saved_partition_values.push(partition_value_row); + } + RawEntryMut::Occupied(mut entry) => { + let writer = entry.get_mut(); + writer.write(&Arc::new(MicroPartition::new_loaded( + table.schema.clone(), + vec![table].into(), + None, + )))?; + } + } + } + Ok(()) + } + + fn close(&mut self) -> DaftResult { + let mut results = vec![]; + for (_, mut writer) in self.per_partition_writers.drain() { + results.extend(writer.close()?); + } + Ok(results) + } +} + +pub(crate) struct PartitionedWriterFactory { + writer_factory: Arc, Result = Vec
>>, + partition_cols: Vec, +} + +impl PartitionedWriterFactory { + pub(crate) fn new( + writer_factory: Arc, Result = Vec
>>, + partition_cols: Vec, + ) -> Self { + Self { + writer_factory, + partition_cols, + } + } +} +impl WriterFactory for PartitionedWriterFactory { + type Input = Arc; + type Result = Vec
; + + fn create_writer( + &self, + _file_idx: usize, + _partition_values: Option<&Table>, + ) -> DaftResult>> { + Ok(Box::new(PartitionedWriter::new( + self.writer_factory.clone(), + self.partition_cols.clone(), + )) + as Box< + dyn FileWriter, + >) + } +} + +struct PartitionedWriteState { + writer: Box, Result = Vec
>>, +} + +impl PartitionedWriteState { + pub fn new( + writer: Box, Result = Vec
>>, + ) -> Self { + Self { writer } + } +} + +impl DynBlockingSinkState for PartitionedWriteState { + fn as_any_mut(&mut self) -> &mut dyn std::any::Any { + self + } +} + +pub struct PartitionedWriteSink { + name: &'static str, + writer_factory: PartitionedWriterFactory, + file_schema: SchemaRef, + partition_cols: Vec, +} + +impl PartitionedWriteSink { + pub fn new( + name: &'static str, + partition_cols: Vec, + writer_factory: PartitionedWriterFactory, + file_schema: SchemaRef, + ) -> Self { + Self { + name, + writer_factory, + file_schema, + partition_cols, + } + } + pub fn arced(self) -> Arc { + Arc::new(self) + } +} + +impl BlockingSink for PartitionedWriteSink { + #[instrument(skip_all, name = "PartitionedWriteSink::sink")] + fn sink( + &self, + input: &Arc, + state_handle: &BlockingSinkState, + ) -> DaftResult { + state_handle.with_state_mut::(|state| { + state.writer.write(input)?; + Ok(BlockingSinkStatus::NeedMoreInput) + }) + } + + #[instrument(skip_all, name = "PartitionedWriteSink::finalize")] + fn finalize( + &self, + states: Vec>, + ) -> DaftResult> { + let mut results = vec![]; + for mut state in states { + let state = state + .as_any_mut() + .downcast_mut::() + .expect("State type mismatch"); + results.extend(state.writer.close()?); + } + let mp = Arc::new(MicroPartition::new_loaded( + self.file_schema.clone(), + results.into(), + None, + )); + Ok(Some(mp.into())) + } + + fn name(&self) -> &'static str { + self.name + } + + fn make_state(&self) -> DaftResult> { + let writer = self.writer_factory.create_writer(0, None)?; + Ok(Box::new(PartitionedWriteState::new(writer)) as Box) + } + + fn max_concurrency(&self) -> usize { + *crate::NUM_CPUS + } + + fn make_dispatcher( + &self, + _runtime_handle: &crate::ExecutionRuntimeHandle, + ) -> Arc { + Arc::new(PartitionedDispatcher::new(self.partition_cols.clone())) + } +} diff --git a/src/daft-local-execution/src/sinks/unpartitioned_write.rs b/src/daft-local-execution/src/sinks/unpartitioned_write.rs new file mode 100644 index 0000000000..c2af193f9c --- /dev/null +++ b/src/daft-local-execution/src/sinks/unpartitioned_write.rs @@ -0,0 +1,234 @@ +use std::sync::Arc; + +use common_error::DaftResult; +use daft_core::prelude::SchemaRef; +use daft_micropartition::MicroPartition; +use daft_table::Table; +use daft_writers::{FileWriter, WriterFactory}; +use tracing::instrument; + +use super::blocking_sink::{ + BlockingSink, BlockingSinkState, BlockingSinkStatus, DynBlockingSinkState, +}; +use crate::pipeline::PipelineResultType; + +// TargetFileSizeWriter is a writer that writes in files of a target size. +// It rotates the writer when the current file reaches the target size. +struct TargetFileSizeWriter { + current_file_rows: usize, + current_writer: Box, Result = Option
>>, + writer_factory: Arc, Result = Option
>>, + target_in_memory_file_rows: usize, + results: Vec
, + partition_values: Option
, +} + +impl TargetFileSizeWriter { + fn new( + target_in_memory_file_rows: usize, + writer_factory: Arc, Result = Option
>>, + partition_values: Option
, + ) -> DaftResult { + let writer: Box, Result = Option
>> = + writer_factory.create_writer(0, partition_values.as_ref())?; + Ok(Self { + current_file_rows: 0, + current_writer: writer, + writer_factory, + target_in_memory_file_rows, + results: vec![], + partition_values, + }) + } + + fn rotate_writer(&mut self) -> DaftResult<()> { + if let Some(result) = self.current_writer.close()? { + self.results.push(result); + } + self.current_file_rows = 0; + self.current_writer = self + .writer_factory + .create_writer(self.results.len(), self.partition_values.as_ref())?; + Ok(()) + } +} + +impl FileWriter for TargetFileSizeWriter { + type Input = Arc; + type Result = Vec
; + + fn write(&mut self, input: &Arc) -> DaftResult<()> { + use std::cmp::Ordering; + match (input.len() + self.current_file_rows).cmp(&self.target_in_memory_file_rows) { + Ordering::Equal => { + self.current_writer.write(input)?; + self.rotate_writer()?; + } + Ordering::Greater => { + // Finish up the current writer first + let remaining_rows = self.target_in_memory_file_rows - self.current_file_rows; + let (to_write, mut remaining) = input.split_at(remaining_rows)?; + self.current_writer.write(&to_write.into())?; + self.rotate_writer()?; + + // Write as many full files as possible + let num_full_files = remaining.len() / self.target_in_memory_file_rows; + for _ in 0..num_full_files { + let (to_write, new_remaining) = + remaining.split_at(self.target_in_memory_file_rows)?; + self.current_writer.write(&to_write.into())?; + self.rotate_writer()?; + remaining = new_remaining; + } + + // Write the remaining rows + if !remaining.is_empty() { + self.current_file_rows = remaining.len(); + self.current_writer.write(&remaining.into())?; + } else { + self.current_file_rows = 0; + } + } + Ordering::Less => { + self.current_writer.write(input)?; + self.current_file_rows += input.len(); + } + } + Ok(()) + } + + fn close(&mut self) -> DaftResult { + if self.current_file_rows > 0 { + if let Some(result) = self.current_writer.close()? { + self.results.push(result); + } + } + Ok(std::mem::take(&mut self.results)) + } +} + +pub(crate) struct TargetFileSizeWriterFactory { + writer_factory: Arc, Result = Option
>>, + target_in_memory_file_rows: usize, +} + +impl TargetFileSizeWriterFactory { + pub(crate) fn new( + writer_factory: Arc, Result = Option
>>, + target_in_memory_file_rows: usize, + ) -> Self { + Self { + writer_factory, + target_in_memory_file_rows, + } + } +} + +impl WriterFactory for TargetFileSizeWriterFactory { + type Input = Arc; + type Result = Vec
; + + fn create_writer( + &self, + _file_idx: usize, + partition_values: Option<&Table>, + ) -> DaftResult>> { + Ok(Box::new(TargetFileSizeWriter::new( + self.target_in_memory_file_rows, + self.writer_factory.clone(), + partition_values.cloned(), + )?) + as Box< + dyn FileWriter, + >) + } +} + +struct UnpartitionedWriteState { + writer: Box, Result = Vec
>>, +} + +impl UnpartitionedWriteState { + pub fn new( + writer: Box, Result = Vec
>>, + ) -> Self { + Self { writer } + } +} + +impl DynBlockingSinkState for UnpartitionedWriteState { + fn as_any_mut(&mut self) -> &mut dyn std::any::Any { + self + } +} + +pub(crate) struct UnpartitionedWriteSink { + name: &'static str, + writer_factory: TargetFileSizeWriterFactory, + file_schema: SchemaRef, +} + +impl UnpartitionedWriteSink { + pub(crate) fn new( + name: &'static str, + writer_factory: TargetFileSizeWriterFactory, + file_schema: SchemaRef, + ) -> Self { + Self { + name, + writer_factory, + file_schema, + } + } + pub(crate) fn arced(self) -> Arc { + Arc::new(self) + } +} + +impl BlockingSink for UnpartitionedWriteSink { + #[instrument(skip_all, name = "WriteSink::sink")] + fn sink( + &self, + input: &Arc, + state_handle: &BlockingSinkState, + ) -> DaftResult { + state_handle.with_state_mut::(|state| { + state.writer.write(input)?; + Ok(BlockingSinkStatus::NeedMoreInput) + }) + } + + #[instrument(skip_all, name = "WriteSink::finalize")] + fn finalize( + &self, + states: Vec>, + ) -> DaftResult> { + let mut results = vec![]; + for mut state in states { + let state = state + .as_any_mut() + .downcast_mut::() + .expect("State type mismatch"); + results.extend(state.writer.close()?); + } + let mp = Arc::new(MicroPartition::new_loaded( + self.file_schema.clone(), + results.into(), + None, + )); + Ok(Some(mp.into())) + } + + fn name(&self) -> &'static str { + self.name + } + + fn make_state(&self) -> DaftResult> { + let writer = self.writer_factory.create_writer(0, None)?; + Ok(Box::new(UnpartitionedWriteState::new(writer)) as Box) + } + + fn max_concurrency(&self) -> usize { + 1 + } +} diff --git a/src/daft-local-execution/src/sinks/write.rs b/src/daft-local-execution/src/sinks/write.rs deleted file mode 100644 index 015cae68bf..0000000000 --- a/src/daft-local-execution/src/sinks/write.rs +++ /dev/null @@ -1,534 +0,0 @@ -use std::{ - cmp::min, - collections::{hash_map::RawEntryMut, HashMap}, - sync::Arc, -}; - -use common_daft_config::DaftExecutionConfig; -use common_error::DaftResult; -use common_file_formats::FileFormat; -use daft_core::{ - prelude::{AsArrow, SchemaRef}, - utils::identity_hash_set::IndexHash, -}; -use daft_dsl::ExprRef; -use daft_io::IOStatsContext; -use daft_micropartition::MicroPartition; -use daft_plan::OutputFileInfo; -use daft_table::Table; -use daft_writers::{FileWriter, PhysicalWriterFactory, WriterFactory}; -use tracing::instrument; - -use super::blocking_sink::{ - BlockingSink, BlockingSinkState, BlockingSinkStatus, DynBlockingSinkState, -}; -use crate::{ - buffer::RowBasedBuffer, - dispatcher::{Dispatcher, RoundRobinBufferedDispatcher}, - pipeline::PipelineResultType, -}; - -// TargetBatchWriter is a writer that writes in batches of rows, i.e. for Parquet where we want to write -// a row group at a time. It uses a buffer to accumulate rows until it has enough to write a batch. -struct TargetBatchWriter { - buffer: RowBasedBuffer, - writer: Box, Result = Option
>>, -} - -impl TargetBatchWriter { - pub fn new( - target_in_memory_chunk_rows: usize, - writer: Box, Result = Option
>>, - ) -> Self { - Self { - buffer: RowBasedBuffer::new(target_in_memory_chunk_rows), - writer, - } - } -} - -impl FileWriter for TargetBatchWriter { - type Input = Arc; - type Result = Option
; - - fn write(&mut self, input: &Arc) -> DaftResult<()> { - self.buffer.push(input.clone()); - if let Some(ready) = self.buffer.pop_enough()? { - for r in ready { - self.writer.write(&r)?; - } - } - Ok(()) - } - - fn close(&mut self) -> DaftResult { - if let Some(ready) = self.buffer.pop_all()? { - self.writer.write(&ready)?; - } - self.writer.close() - } -} - -struct TargetBatchWriterFactory { - writer_factory: Arc, Result = Option
>>, - target_in_memory_chunk_rows: usize, -} - -impl WriterFactory for TargetBatchWriterFactory { - type Input = Arc; - type Result = Option
; - - fn create_writer( - &self, - file_idx: usize, - partition_values: Option<&Table>, - ) -> DaftResult>> { - let writer = self - .writer_factory - .create_writer(file_idx, partition_values)?; - Ok(Box::new(TargetBatchWriter::new( - self.target_in_memory_chunk_rows, - writer, - )) - as Box< - dyn FileWriter, - >) - } -} - -// TargetFileSizeWriter is a writer that writes in files of a target size. -// It rotates the writer when the current file reaches the target size. -struct TargetFileSizeWriter { - current_file_rows: usize, - current_writer: Box, Result = Option
>>, - writer_factory: Arc, Result = Option
>>, - target_in_memory_file_rows: usize, - results: Vec
, - partition_values: Option
, -} - -impl TargetFileSizeWriter { - pub fn new( - target_in_memory_file_rows: usize, - writer_factory: Arc, Result = Option
>>, - partition_values: Option
, - ) -> DaftResult { - let writer: Box, Result = Option
>> = - writer_factory.create_writer(0, partition_values.as_ref())?; - Ok(Self { - current_file_rows: 0, - current_writer: writer, - writer_factory, - target_in_memory_file_rows, - results: vec![], - partition_values, - }) - } - - fn rotate_writer(&mut self) -> DaftResult<()> { - if let Some(result) = self.current_writer.close()? { - self.results.push(result); - } - self.current_file_rows = 0; - self.current_writer = self - .writer_factory - .create_writer(self.results.len(), self.partition_values.as_ref())?; - Ok(()) - } -} - -impl FileWriter for TargetFileSizeWriter { - type Input = Arc; - type Result = Vec
; - - fn write(&mut self, input: &Arc) -> DaftResult<()> { - use std::cmp::Ordering; - match (input.len() + self.current_file_rows).cmp(&self.target_in_memory_file_rows) { - Ordering::Equal => { - self.current_writer.write(input)?; - self.rotate_writer()?; - } - Ordering::Greater => { - // Finish up the current writer first - let remaining_rows = self.target_in_memory_file_rows - self.current_file_rows; - let (to_write, mut remaining) = input.split_at(remaining_rows)?; - self.current_writer.write(&to_write.into())?; - self.rotate_writer()?; - - // Write as many full files as possible - let num_full_files = remaining.len() / self.target_in_memory_file_rows; - for _ in 0..num_full_files { - let (to_write, new_remaining) = - remaining.split_at(self.target_in_memory_file_rows)?; - self.current_writer.write(&to_write.into())?; - self.rotate_writer()?; - remaining = new_remaining; - } - - // Write the remaining rows - if !remaining.is_empty() { - self.current_file_rows = remaining.len(); - self.current_writer.write(&remaining.into())?; - } else { - self.current_file_rows = 0; - } - } - Ordering::Less => { - self.current_writer.write(input)?; - self.current_file_rows += input.len(); - } - } - Ok(()) - } - - fn close(&mut self) -> DaftResult { - if self.current_file_rows > 0 { - if let Some(result) = self.current_writer.close()? { - self.results.push(result); - } - } - Ok(std::mem::take(&mut self.results)) - } -} - -struct TargetFileSizeWriterFactory { - writer_factory: Arc, Result = Option
>>, - target_in_memory_file_rows: usize, -} - -impl WriterFactory for TargetFileSizeWriterFactory { - type Input = Arc; - type Result = Vec
; - - fn create_writer( - &self, - _file_idx: usize, - partition_values: Option<&Table>, - ) -> DaftResult>> { - Ok(Box::new(TargetFileSizeWriter::new( - self.target_in_memory_file_rows, - self.writer_factory.clone(), - partition_values.cloned(), - )?) - as Box< - dyn FileWriter, - >) - } -} - -/// PartitionedWriter is a writer that partitions the input data by a set of columns, and writes each partition -/// to a separate file. It uses a map to keep track of the writers for each partition. -struct PartitionedWriter { - per_partition_writers: - HashMap, Result = Vec
>>>, - saved_partition_values: Vec
, - writer_factory: Arc, Result = Vec
>>, - partition_by: Vec, -} - -impl PartitionedWriter { - pub fn new( - writer_factory: Arc, Result = Vec
>>, - partition_by: Vec, - ) -> Self { - Self { - per_partition_writers: HashMap::new(), - saved_partition_values: vec![], - writer_factory, - partition_by, - } - } - - fn partition( - partition_cols: &[ExprRef], - data: &Arc, - ) -> DaftResult<(Vec
, Table)> { - let data = data.concat_or_get(IOStatsContext::new("MicroPartition::partition_by_value"))?; - let table = data.first().unwrap(); - let (split_tables, partition_values) = table.partition_by_value(partition_cols)?; - Ok((split_tables, partition_values)) - } -} - -impl FileWriter for PartitionedWriter { - type Input = Arc; - type Result = Vec
; - - fn write(&mut self, input: &Arc) -> DaftResult<()> { - let (split_tables, partition_values) = - Self::partition(self.partition_by.as_slice(), input)?; - let partition_values_hash = partition_values.hash_rows()?; - for (idx, (table, partition_value_hash)) in split_tables - .into_iter() - .zip(partition_values_hash.as_arrow().values_iter()) - .enumerate() - { - let partition_value_row = partition_values.slice(idx, idx + 1)?; - let entry = self.per_partition_writers.raw_entry_mut().from_hash( - *partition_value_hash, - |other| { - (*partition_value_hash == other.hash) && { - let other_table = - self.saved_partition_values.get(other.idx as usize).unwrap(); - other_table == &partition_value_row - } - }, - ); - match entry { - RawEntryMut::Vacant(entry) => { - let mut writer = self - .writer_factory - .create_writer(0, Some(partition_value_row.as_ref()))?; - writer.write(&Arc::new(MicroPartition::new_loaded( - table.schema.clone(), - vec![table].into(), - None, - )))?; - entry.insert_hashed_nocheck( - *partition_value_hash, - IndexHash { - idx: self.saved_partition_values.len() as u64, - hash: *partition_value_hash, - }, - writer, - ); - self.saved_partition_values.push(partition_value_row); - } - RawEntryMut::Occupied(mut entry) => { - let writer = entry.get_mut(); - writer.write(&Arc::new(MicroPartition::new_loaded( - table.schema.clone(), - vec![table].into(), - None, - )))?; - } - } - } - Ok(()) - } - - fn close(&mut self) -> DaftResult { - let mut results = vec![]; - for (_, mut writer) in self.per_partition_writers.drain() { - results.extend(writer.close()?); - } - Ok(results) - } -} - -struct PartitionedWriterFactory { - writer_factory: Arc, Result = Vec
>>, - partition_cols: Vec, -} - -impl WriterFactory for PartitionedWriterFactory { - type Input = Arc; - type Result = Vec
; - - fn create_writer( - &self, - _file_idx: usize, - _partition_values: Option<&Table>, - ) -> DaftResult>> { - Ok(Box::new(PartitionedWriter::new( - self.writer_factory.clone(), - self.partition_cols.clone(), - )) - as Box< - dyn FileWriter, - >) - } -} - -struct WriteState { - writer: Box, Result = Vec
>>, -} - -impl WriteState { - pub fn new( - writer: Box, Result = Vec
>>, - ) -> Self { - Self { writer } - } -} - -impl DynBlockingSinkState for WriteState { - fn as_any_mut(&mut self) -> &mut dyn std::any::Any { - self - } -} - -pub struct WriteSink { - name: &'static str, - writer_factory: Arc, Result = Vec
>>, - file_schema: SchemaRef, - partition_cols: Option>, -} - -impl WriteSink { - pub fn new( - file_info: OutputFileInfo, - data_schema: SchemaRef, - file_schema: SchemaRef, - cfg: &DaftExecutionConfig, - ) -> DaftResult { - let estimated_row_size_bytes = data_schema.estimate_row_size_bytes(); - let base_writer_factory = PhysicalWriterFactory::new(file_info.clone()); - let (writer_factory, name) = match file_info.file_format { - FileFormat::Parquet => { - let target_in_memory_file_size = - cfg.parquet_target_filesize as f64 * cfg.parquet_inflation_factor; - let target_in_memory_row_group_size = - cfg.parquet_target_row_group_size as f64 * cfg.parquet_inflation_factor; - - let target_file_rows = if estimated_row_size_bytes > 0.0 { - target_in_memory_file_size / estimated_row_size_bytes - } else { - target_in_memory_file_size - } as usize; - - let target_row_group_rows = min( - target_file_rows, - if estimated_row_size_bytes > 0.0 { - target_in_memory_row_group_size / estimated_row_size_bytes - } else { - target_in_memory_row_group_size - } as usize, - ); - - let row_group_writer_factory = TargetBatchWriterFactory { - writer_factory: Arc::new(base_writer_factory), - target_in_memory_chunk_rows: target_row_group_rows, - }; - - let file_writer_factory = Arc::new(TargetFileSizeWriterFactory { - writer_factory: Arc::new(row_group_writer_factory), - target_in_memory_file_rows: target_file_rows, - }); - - if let Some(partition_cols) = &file_info.partition_cols { - let partitioned_writer_factory = Arc::new(PartitionedWriterFactory { - writer_factory: file_writer_factory, - partition_cols: partition_cols.clone(), - }); - ( - partitioned_writer_factory as Arc>, - "PartitionedParquetWrite", - ) - } else { - ( - file_writer_factory as Arc>, - "UnpartitionedParquetWrite", - ) - } - } - FileFormat::Csv => { - let target_in_memory_file_size = - cfg.csv_target_filesize as f64 * cfg.csv_inflation_factor; - let target_file_rows = if estimated_row_size_bytes > 0.0 { - target_in_memory_file_size / estimated_row_size_bytes - } else { - target_in_memory_file_size - } as usize; - - let file_writer_factory = Arc::new(TargetFileSizeWriterFactory { - writer_factory: Arc::new(base_writer_factory), - target_in_memory_file_rows: target_file_rows, - }); - - if let Some(partition_cols) = &file_info.partition_cols { - let partitioned_writer_factory = Arc::new(PartitionedWriterFactory { - writer_factory: file_writer_factory, - partition_cols: partition_cols.clone(), - }); - ( - partitioned_writer_factory as Arc>, - "PartitionedCsvWrite", - ) - } else { - ( - file_writer_factory as Arc>, - "UnpartitionedCsvWrite", - ) - } - } - _ => unreachable!("Physical write should only support Parquet and CSV"), - }; - Ok(Self { - name, - writer_factory, - file_schema, - partition_cols: file_info.partition_cols, - }) - } - pub fn boxed(self) -> Box { - Box::new(self) - } -} - -impl BlockingSink for WriteSink { - #[instrument(skip_all, name = "WriteSink::sink")] - fn sink( - &self, - input: &Arc, - state_handle: &BlockingSinkState, - ) -> DaftResult { - state_handle.with_state_mut::(|state| { - state.writer.write(input)?; - Ok(BlockingSinkStatus::NeedMoreInput) - }) - } - - #[instrument(skip_all, name = "WriteSink::finalize")] - fn finalize( - &self, - states: Vec>, - ) -> DaftResult> { - let mut results = vec![]; - for mut state in states { - let state = state - .as_any_mut() - .downcast_mut::() - .expect("State type mismatch"); - results.extend(state.writer.close()?); - } - let mp = Arc::new(MicroPartition::new_loaded( - self.file_schema.clone(), - results.into(), - None, - )); - Ok(Some(mp.into())) - } - - fn name(&self) -> &'static str { - self.name - } - - fn make_state(&self) -> DaftResult> { - let writer = self.writer_factory.create_writer(0, None)?; - Ok(Box::new(WriteState::new(writer)) as Box) - } - - fn max_concurrency(&self) -> usize { - if self.partition_cols.is_some() { - *crate::NUM_CPUS - } else { - 1 - } - } - - fn make_dispatcher( - &self, - runtime_handle: &crate::ExecutionRuntimeHandle, - ) -> Arc { - if let Some(partition_cols) = &self.partition_cols { - Arc::new(crate::dispatcher::PartitionedDispatcher::new( - partition_cols.clone(), - )) - } else { - Arc::new(RoundRobinBufferedDispatcher::new( - runtime_handle.default_morsel_size(), - )) - } - } -} diff --git a/src/daft-micropartition/Cargo.toml b/src/daft-micropartition/Cargo.toml index b03935d6a6..f9aef047d6 100644 --- a/src/daft-micropartition/Cargo.toml +++ b/src/daft-micropartition/Cargo.toml @@ -1,6 +1,7 @@ [dependencies] arrow2 = {workspace = true} bincode = {workspace = true} +common-buffer = {path = "../common/buffer", default-features = false} common-error = {path = "../common/error", default-features = false} common-file-formats = {path = "../common/file-formats", default-features = false} common-runtime = {path = "../common/runtime", default-features = false} diff --git a/src/daft-micropartition/src/micropartition.rs b/src/daft-micropartition/src/micropartition.rs index af78186cd8..f674ed1655 100644 --- a/src/daft-micropartition/src/micropartition.rs +++ b/src/daft-micropartition/src/micropartition.rs @@ -5,6 +5,7 @@ use std::{ }; use arrow2::io::parquet::read::schema::infer_schema_with_options; +use common_buffer::Bufferable; use common_error::DaftResult; use common_file_formats::{CsvSourceConfig, FileFormatConfig, ParquetSourceConfig}; use common_runtime::get_io_runtime; @@ -1276,5 +1277,27 @@ impl Display for MicroPartition { } } +impl Bufferable for MicroPartition { + fn len(&self) -> usize { + self.len() + } + + fn concat(parts: &[&Self]) -> DaftResult + where + Self: Sized, + { + let concated = Self::concat(parts)?; + Ok(concated) + } + + fn slice(&self, start: usize, end: usize) -> DaftResult + where + Self: Sized, + { + let sliced = self.slice(start, end)?; + Ok(sliced) + } +} + #[cfg(test)] mod test {} diff --git a/src/daft-writers/Cargo.toml b/src/daft-writers/Cargo.toml index c0694aeb81..91b49c3353 100644 --- a/src/daft-writers/Cargo.toml +++ b/src/daft-writers/Cargo.toml @@ -1,5 +1,6 @@ [dependencies] async-trait = {workspace = true} +common-buffer = {path = "../common/buffer", default-features = false} common-daft-config = {path = "../common/daft-config", default-features = false} common-display = {path = "../common/display", default-features = false} common-error = {path = "../common/error", default-features = false} diff --git a/src/daft-writers/src/batch.rs b/src/daft-writers/src/batch.rs new file mode 100644 index 0000000000..aba5741b49 --- /dev/null +++ b/src/daft-writers/src/batch.rs @@ -0,0 +1,84 @@ +use std::sync::Arc; + +use common_buffer::{Bufferable, RowBasedBuffer}; +use common_error::DaftResult; +use daft_table::Table; + +use crate::{FileWriter, WriterFactory}; + +// TargetBatchWriter is a writer that writes in batches of rows, i.e. for Parquet where we want to write +// a row group at a time. It uses a buffer to accumulate rows until it has enough to write a batch. +pub struct TargetBatchWriter { + buffer: RowBasedBuffer, + writer: Box>>, +} + +impl TargetBatchWriter { + pub fn new( + target_in_memory_chunk_rows: usize, + writer: Box>>, + ) -> Self { + Self { + buffer: RowBasedBuffer::new(target_in_memory_chunk_rows), + writer, + } + } +} + +impl FileWriter for TargetBatchWriter { + type Input = B; + type Result = Option
; + + fn write(&mut self, input: &B) -> DaftResult<()> { + self.buffer.push(input.clone()); + if let Some(ready) = self.buffer.pop_enough()? { + for r in ready { + self.writer.write(&r)?; + } + } + Ok(()) + } + + fn close(&mut self) -> DaftResult { + if let Some(ready) = self.buffer.pop_all()? { + self.writer.write(&ready)?; + } + self.writer.close() + } +} + +pub struct TargetBatchWriterFactory { + writer_factory: Arc>>, + target_in_memory_chunk_rows: usize, +} + +impl TargetBatchWriterFactory { + pub fn new( + writer_factory: Arc>>, + target_in_memory_chunk_rows: usize, + ) -> Self { + Self { + writer_factory, + target_in_memory_chunk_rows, + } + } +} + +impl WriterFactory for TargetBatchWriterFactory { + type Input = B; + type Result = Option
; + + fn create_writer( + &self, + file_idx: usize, + partition_values: Option<&Table>, + ) -> DaftResult>> { + let writer = self + .writer_factory + .create_writer(file_idx, partition_values)?; + Ok(Box::new(TargetBatchWriter::new( + self.target_in_memory_chunk_rows, + writer, + ))) + } +} diff --git a/src/daft-writers/src/lib.rs b/src/daft-writers/src/lib.rs index f5a5155b36..d08f3888a0 100644 --- a/src/daft-writers/src/lib.rs +++ b/src/daft-writers/src/lib.rs @@ -1,7 +1,9 @@ +mod batch; mod physical; #[cfg(feature = "python")] mod python; +pub use batch::{TargetBatchWriter, TargetBatchWriterFactory}; use common_error::DaftResult; use daft_table::Table; pub use physical::PhysicalWriterFactory; From 03dcfcb785db868e64971ebec515219d71a9717a Mon Sep 17 00:00:00 2001 From: Colin Ho Date: Mon, 28 Oct 2024 23:28:23 -0700 Subject: [PATCH 12/15] remove unused deps --- Cargo.lock | 28 ---------------------------- src/common/buffer/Cargo.toml | 5 ----- src/daft-writers/Cargo.toml | 25 +------------------------ tests/io/test_csv_roundtrip.py | 6 +----- 4 files changed, 2 insertions(+), 62 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index 47c2a6feda..b2ffbcf954 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -1303,11 +1303,6 @@ name = "common-buffer" version = "0.3.0-dev0" dependencies = [ "common-error", - "futures", - "lazy_static", - "log", - "oneshot", - "tokio", ] [[package]] @@ -2267,37 +2262,14 @@ dependencies = [ name = "daft-writers" version = "0.3.0-dev0" dependencies = [ - "async-trait", "common-buffer", - "common-daft-config", - "common-display", "common-error", "common-file-formats", - "common-runtime", - "common-tracing", - "daft-core", - "daft-csv", - "daft-dsl", - "daft-functions", "daft-io", - "daft-json", "daft-micropartition", - "daft-parquet", - "daft-physical-plan", "daft-plan", - "daft-scan", "daft-table", - "futures", - "indexmap 2.5.0", - "itertools 0.11.0", - "lazy_static", - "log", - "num-format", "pyo3", - "snafu", - "tokio", - "tokio-stream", - "tracing", ] [[package]] diff --git a/src/common/buffer/Cargo.toml b/src/common/buffer/Cargo.toml index f3f41bbce6..b9ba3a62f8 100644 --- a/src/common/buffer/Cargo.toml +++ b/src/common/buffer/Cargo.toml @@ -1,10 +1,5 @@ [dependencies] common-error = {path = "../error", default-features = false} -futures = {workspace = true} -lazy_static = {workspace = true} -log = {workspace = true} -oneshot = "0.1.8" -tokio = {workspace = true} [lints] workspace = true diff --git a/src/daft-writers/Cargo.toml b/src/daft-writers/Cargo.toml index 91b49c3353..7f7e45ac9a 100644 --- a/src/daft-writers/Cargo.toml +++ b/src/daft-writers/Cargo.toml @@ -1,38 +1,15 @@ [dependencies] -async-trait = {workspace = true} common-buffer = {path = "../common/buffer", default-features = false} -common-daft-config = {path = "../common/daft-config", default-features = false} -common-display = {path = "../common/display", default-features = false} common-error = {path = "../common/error", default-features = false} common-file-formats = {path = "../common/file-formats", default-features = false} -common-runtime = {path = "../common/runtime", default-features = false} -common-tracing = {path = "../common/tracing", default-features = false} -daft-core = {path = "../daft-core", default-features = false} -daft-csv = {path = "../daft-csv", default-features = false} -daft-dsl = {path = "../daft-dsl", default-features = false} -daft-functions = {path = "../daft-functions", default-features = false} daft-io = {path = "../daft-io", default-features = false} -daft-json = {path = "../daft-json", default-features = false} daft-micropartition = {path = "../daft-micropartition", default-features = false} -daft-parquet = {path = "../daft-parquet", default-features = false} -daft-physical-plan = {path = "../daft-physical-plan", default-features = false} daft-plan = {path = "../daft-plan", default-features = false} -daft-scan = {path = "../daft-scan", default-features = false} daft-table = {path = "../daft-table", default-features = false} -futures = {workspace = true} -indexmap = {workspace = true} -itertools = {workspace = true} -lazy_static = {workspace = true} -log = {workspace = true} -num-format = "0.4.4" pyo3 = {workspace = true, optional = true} -snafu = {workspace = true} -tokio = {workspace = true} -tokio-stream = {workspace = true} -tracing = {workspace = true} [features] -python = ["dep:pyo3", "common-daft-config/python", "common-file-formats/python", "common-error/python", "daft-dsl/python", "daft-io/python", "daft-micropartition/python", "daft-plan/python", "daft-scan/python", "common-display/python"] +python = ["dep:pyo3", "common-file-formats/python", "common-error/python", "daft-io/python", "daft-micropartition/python", "daft-plan/python"] [lints] workspace = true diff --git a/tests/io/test_csv_roundtrip.py b/tests/io/test_csv_roundtrip.py index b9e8ccc9b8..dd288e6806 100644 --- a/tests/io/test_csv_roundtrip.py +++ b/tests/io/test_csv_roundtrip.py @@ -7,12 +7,8 @@ import pytest import daft -from daft import DataType, TimeUnit, context +from daft import DataType, TimeUnit -pytestmark = pytest.mark.skipif( - context.get_context().daft_execution_config.enable_native_executor is True, - reason="Native executor fails for these tests", -) PYARROW_GE_11_0_0 = tuple(int(s) for s in pa.__version__.split(".") if s.isnumeric()) >= (11, 0, 0) From bc9c7fa777852b2c9e1daaf9749c798fb152a613 Mon Sep 17 00:00:00 2001 From: Colin Ho Date: Tue, 29 Oct 2024 10:06:10 -0700 Subject: [PATCH 13/15] if we move all file writer stuff to daft writer then the ray runner can use it too --- Cargo.lock | 3 + src/daft-local-execution/src/pipeline.rs | 124 +++--------------- src/daft-local-execution/src/sinks/mod.rs | 3 +- src/daft-local-execution/src/sinks/write.rs | 124 ++++++++++++++++++ src/daft-writers/Cargo.toml | 5 +- src/daft-writers/src/batch.rs | 31 ++--- .../src/file.rs} | 97 +------------- src/daft-writers/src/lib.rs | 90 ++++++++++++- .../src/partition.rs} | 114 +--------------- 9 files changed, 259 insertions(+), 332 deletions(-) create mode 100644 src/daft-local-execution/src/sinks/write.rs rename src/{daft-local-execution/src/sinks/unpartitioned_write.rs => daft-writers/src/file.rs} (64%) rename src/{daft-local-execution/src/sinks/partitioned_write.rs => daft-writers/src/partition.rs} (62%) diff --git a/Cargo.lock b/Cargo.lock index b2ffbcf954..46cf959d70 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -2263,8 +2263,11 @@ name = "daft-writers" version = "0.3.0-dev0" dependencies = [ "common-buffer", + "common-daft-config", "common-error", "common-file-formats", + "daft-core", + "daft-dsl", "daft-io", "daft-micropartition", "daft-plan", diff --git a/src/daft-local-execution/src/pipeline.rs b/src/daft-local-execution/src/pipeline.rs index c07089fd74..1cab11508d 100644 --- a/src/daft-local-execution/src/pipeline.rs +++ b/src/daft-local-execution/src/pipeline.rs @@ -1,4 +1,4 @@ -use std::{cmp::min, collections::HashMap, sync::Arc}; +use std::{collections::HashMap, sync::Arc}; use common_daft_config::DaftExecutionConfig; use common_display::{mermaid::MermaidDisplayVisitor, tree::TreeDisplay}; @@ -17,7 +17,7 @@ use daft_physical_plan::{ }; use daft_plan::{populate_aggregation_stages, JoinType}; use daft_table::ProbeState; -use daft_writers::{PhysicalWriterFactory, TargetBatchWriterFactory}; +use daft_writers::make_writer_factory; use indexmap::IndexSet; use snafu::ResultExt; @@ -31,16 +31,10 @@ use crate::{ unpivot::UnpivotOperator, }, sinks::{ - aggregate::AggregateSink, - blocking_sink::BlockingSinkNode, - concat::ConcatSink, - hash_join_build::HashJoinBuildSink, - limit::LimitSink, - outer_hash_join_probe::OuterHashJoinProbeSink, - partitioned_write::{PartitionedWriteSink, PartitionedWriterFactory}, - sort::SortSink, - streaming_sink::StreamingSinkNode, - unpartitioned_write::{TargetFileSizeWriterFactory, UnpartitionedWriteSink}, + aggregate::AggregateSink, blocking_sink::BlockingSinkNode, concat::ConcatSink, + hash_join_build::HashJoinBuildSink, limit::LimitSink, + outer_hash_join_probe::OuterHashJoinProbeSink, sort::SortSink, + streaming_sink::StreamingSinkNode, write::WriteSink, }, sources::{empty_scan::EmptyScanSource, in_memory::InMemorySource}, ExecutionRuntimeHandle, PipelineCreationSnafu, @@ -420,99 +414,21 @@ pub fn physical_plan_to_pipeline( .. }) => { let child_node = physical_plan_to_pipeline(input, psets, cfg)?; - let estimated_row_size_bytes = data_schema.estimate_row_size_bytes(); - let base_writer_factory = PhysicalWriterFactory::new(file_info.clone()); - let write_sink = match file_info.file_format { - FileFormat::Parquet => { - let target_in_memory_file_size = - cfg.parquet_target_filesize as f64 * cfg.parquet_inflation_factor; - let target_in_memory_row_group_size = - cfg.parquet_target_row_group_size as f64 * cfg.parquet_inflation_factor; - - let target_file_rows = if estimated_row_size_bytes > 0.0 { - target_in_memory_file_size / estimated_row_size_bytes - } else { - target_in_memory_file_size - } as usize; - - let target_row_group_rows = min( - target_file_rows, - if estimated_row_size_bytes > 0.0 { - target_in_memory_row_group_size / estimated_row_size_bytes - } else { - target_in_memory_row_group_size - } as usize, - ); - - let row_group_writer_factory = TargetBatchWriterFactory::new( - Arc::new(base_writer_factory), - target_row_group_rows, - ); - - let file_writer_factory = TargetFileSizeWriterFactory::new( - Arc::new(row_group_writer_factory), - target_file_rows, - ); - - if let Some(partition_cols) = &file_info.partition_cols { - let partitioned_writer_factory = PartitionedWriterFactory::new( - Arc::new(file_writer_factory), - partition_cols.clone(), - ); - PartitionedWriteSink::new( - "PartitionedParquetWrite", - partition_cols.clone(), - partitioned_writer_factory, - file_schema.clone(), - ) - .arced() - } else { - UnpartitionedWriteSink::new( - "UnpartitionedParquetWrite", - file_writer_factory, - file_schema.clone(), - ) - .arced() - } - } - FileFormat::Csv => { - let target_in_memory_file_size = - cfg.csv_target_filesize as f64 * cfg.csv_inflation_factor; - let target_file_rows = if estimated_row_size_bytes > 0.0 { - target_in_memory_file_size / estimated_row_size_bytes - } else { - target_in_memory_file_size - } as usize; - - let file_writer_factory = TargetFileSizeWriterFactory::new( - Arc::new(base_writer_factory), - target_file_rows, - ); - - if let Some(partition_cols) = &file_info.partition_cols { - let partitioned_writer_factory = PartitionedWriterFactory::new( - Arc::new(file_writer_factory), - partition_cols.clone(), - ); - PartitionedWriteSink::new( - "PartitionedCsvWrite", - partition_cols.clone(), - partitioned_writer_factory, - file_schema.clone(), - ) - .arced() - } else { - UnpartitionedWriteSink::new( - "UnpartitionedCsvWrite", - file_writer_factory, - file_schema.clone(), - ) - .arced() - } - } - _ => unreachable!("Physical write should only support Parquet and CSV"), + let writer_factory = make_writer_factory(file_info, data_schema, cfg); + let name = match (file_info.file_format, file_info.partition_cols.is_some()) { + (FileFormat::Parquet, true) => "PartitionedParquet", + (FileFormat::Parquet, false) => "Parquet", + (FileFormat::Csv, true) => "PartitionedCsv", + (FileFormat::Csv, false) => "Csv", + (_, _) => panic!("Unsupported file format"), }; - BlockingSinkNode::new(write_sink, child_node).boxed() + let write_sink = WriteSink::new( + name, + writer_factory, + file_info.partition_cols.clone(), + file_schema.clone(), + ); + BlockingSinkNode::new(Arc::new(write_sink), child_node).boxed() } }; diff --git a/src/daft-local-execution/src/sinks/mod.rs b/src/daft-local-execution/src/sinks/mod.rs index 64f0c5b09d..64366385c3 100644 --- a/src/daft-local-execution/src/sinks/mod.rs +++ b/src/daft-local-execution/src/sinks/mod.rs @@ -4,7 +4,6 @@ pub mod concat; pub mod hash_join_build; pub mod limit; pub mod outer_hash_join_probe; -pub mod partitioned_write; pub mod sort; pub mod streaming_sink; -pub mod unpartitioned_write; +pub mod write; diff --git a/src/daft-local-execution/src/sinks/write.rs b/src/daft-local-execution/src/sinks/write.rs new file mode 100644 index 0000000000..e1c839e328 --- /dev/null +++ b/src/daft-local-execution/src/sinks/write.rs @@ -0,0 +1,124 @@ +use std::sync::Arc; + +use common_error::DaftResult; +use daft_core::prelude::SchemaRef; +use daft_dsl::ExprRef; +use daft_micropartition::MicroPartition; +use daft_table::Table; +use daft_writers::{FileWriter, WriterFactory}; +use tracing::instrument; + +use super::blocking_sink::{ + BlockingSink, BlockingSinkState, BlockingSinkStatus, DynBlockingSinkState, +}; +use crate::{ + dispatcher::{Dispatcher, PartitionedDispatcher, RoundRobinBufferedDispatcher}, + pipeline::PipelineResultType, + NUM_CPUS, +}; + +struct WriteState { + writer: Box, Result = Vec
>>, +} + +impl WriteState { + pub fn new( + writer: Box, Result = Vec
>>, + ) -> Self { + Self { writer } + } +} + +impl DynBlockingSinkState for WriteState { + fn as_any_mut(&mut self) -> &mut dyn std::any::Any { + self + } +} + +pub(crate) struct WriteSink { + name: &'static str, + writer_factory: Arc, Result = Vec
>>, + partition_by: Option>, + file_schema: SchemaRef, +} + +impl WriteSink { + pub(crate) fn new( + name: &'static str, + writer_factory: Arc, Result = Vec
>>, + partition_by: Option>, + file_schema: SchemaRef, + ) -> Self { + Self { + name, + writer_factory, + partition_by, + file_schema, + } + } +} + +impl BlockingSink for WriteSink { + #[instrument(skip_all, name = "WriteSink::sink")] + fn sink( + &self, + input: &Arc, + state_handle: &BlockingSinkState, + ) -> DaftResult { + state_handle.with_state_mut::(|state| { + state.writer.write(input)?; + Ok(BlockingSinkStatus::NeedMoreInput) + }) + } + + #[instrument(skip_all, name = "WriteSink::finalize")] + fn finalize( + &self, + states: Vec>, + ) -> DaftResult> { + let mut results = vec![]; + for mut state in states { + let state = state + .as_any_mut() + .downcast_mut::() + .expect("State type mismatch"); + results.extend(state.writer.close()?); + } + let mp = Arc::new(MicroPartition::new_loaded( + self.file_schema.clone(), + results.into(), + None, + )); + Ok(Some(mp.into())) + } + + fn name(&self) -> &'static str { + self.name + } + + fn make_state(&self) -> DaftResult> { + let writer = self.writer_factory.create_writer(0, None)?; + Ok(Box::new(WriteState::new(writer)) as Box) + } + + fn make_dispatcher( + &self, + runtime_handle: &crate::ExecutionRuntimeHandle, + ) -> Arc { + if let Some(partition_by) = &self.partition_by { + Arc::new(PartitionedDispatcher::new(partition_by.clone())) + } else { + Arc::new(RoundRobinBufferedDispatcher::new( + runtime_handle.default_morsel_size(), + )) + } + } + + fn max_concurrency(&self) -> usize { + if self.partition_by.is_some() { + *NUM_CPUS + } else { + 1 + } + } +} diff --git a/src/daft-writers/Cargo.toml b/src/daft-writers/Cargo.toml index 7f7e45ac9a..e818fddb9a 100644 --- a/src/daft-writers/Cargo.toml +++ b/src/daft-writers/Cargo.toml @@ -1,7 +1,10 @@ [dependencies] common-buffer = {path = "../common/buffer", default-features = false} +common-daft-config = {path = "../common/daft-config", default-features = false} common-error = {path = "../common/error", default-features = false} common-file-formats = {path = "../common/file-formats", default-features = false} +daft-core = {path = "../daft-core", default-features = false} +daft-dsl = {path = "../daft-dsl", default-features = false} daft-io = {path = "../daft-io", default-features = false} daft-micropartition = {path = "../daft-micropartition", default-features = false} daft-plan = {path = "../daft-plan", default-features = false} @@ -9,7 +12,7 @@ daft-table = {path = "../daft-table", default-features = false} pyo3 = {workspace = true, optional = true} [features] -python = ["dep:pyo3", "common-file-formats/python", "common-error/python", "daft-io/python", "daft-micropartition/python", "daft-plan/python"] +python = ["dep:pyo3", "common-file-formats/python", "common-error/python", "daft-dsl/python", "daft-io/python", "daft-micropartition/python", "daft-plan/python"] [lints] workspace = true diff --git a/src/daft-writers/src/batch.rs b/src/daft-writers/src/batch.rs index aba5741b49..d56ae6fdf2 100644 --- a/src/daft-writers/src/batch.rs +++ b/src/daft-writers/src/batch.rs @@ -1,22 +1,23 @@ use std::sync::Arc; -use common_buffer::{Bufferable, RowBasedBuffer}; +use common_buffer::RowBasedBuffer; use common_error::DaftResult; +use daft_micropartition::MicroPartition; use daft_table::Table; use crate::{FileWriter, WriterFactory}; // TargetBatchWriter is a writer that writes in batches of rows, i.e. for Parquet where we want to write // a row group at a time. It uses a buffer to accumulate rows until it has enough to write a batch. -pub struct TargetBatchWriter { - buffer: RowBasedBuffer, - writer: Box>>, +pub struct TargetBatchWriter { + buffer: RowBasedBuffer>, + writer: Box, Result = Option
>>, } -impl TargetBatchWriter { +impl TargetBatchWriter { pub fn new( target_in_memory_chunk_rows: usize, - writer: Box>>, + writer: Box, Result = Option
>>, ) -> Self { Self { buffer: RowBasedBuffer::new(target_in_memory_chunk_rows), @@ -25,11 +26,11 @@ impl TargetBatchWriter { } } -impl FileWriter for TargetBatchWriter { - type Input = B; +impl FileWriter for TargetBatchWriter { + type Input = Arc; type Result = Option
; - fn write(&mut self, input: &B) -> DaftResult<()> { + fn write(&mut self, input: &Arc) -> DaftResult<()> { self.buffer.push(input.clone()); if let Some(ready) = self.buffer.pop_enough()? { for r in ready { @@ -47,14 +48,14 @@ impl FileWriter for TargetBatchWriter { } } -pub struct TargetBatchWriterFactory { - writer_factory: Arc>>, +pub struct TargetBatchWriterFactory { + writer_factory: Arc, Result = Option
>>, target_in_memory_chunk_rows: usize, } -impl TargetBatchWriterFactory { +impl TargetBatchWriterFactory { pub fn new( - writer_factory: Arc>>, + writer_factory: Arc, Result = Option
>>, target_in_memory_chunk_rows: usize, ) -> Self { Self { @@ -64,8 +65,8 @@ impl TargetBatchWriterFactory { } } -impl WriterFactory for TargetBatchWriterFactory { - type Input = B; +impl WriterFactory for TargetBatchWriterFactory { + type Input = Arc; type Result = Option
; fn create_writer( diff --git a/src/daft-local-execution/src/sinks/unpartitioned_write.rs b/src/daft-writers/src/file.rs similarity index 64% rename from src/daft-local-execution/src/sinks/unpartitioned_write.rs rename to src/daft-writers/src/file.rs index c2af193f9c..a71ec189b4 100644 --- a/src/daft-local-execution/src/sinks/unpartitioned_write.rs +++ b/src/daft-writers/src/file.rs @@ -1,16 +1,10 @@ use std::sync::Arc; use common_error::DaftResult; -use daft_core::prelude::SchemaRef; use daft_micropartition::MicroPartition; use daft_table::Table; -use daft_writers::{FileWriter, WriterFactory}; -use tracing::instrument; -use super::blocking_sink::{ - BlockingSink, BlockingSinkState, BlockingSinkStatus, DynBlockingSinkState, -}; -use crate::pipeline::PipelineResultType; +use crate::{FileWriter, WriterFactory}; // TargetFileSizeWriter is a writer that writes in files of a target size. // It rotates the writer when the current file reaches the target size. @@ -143,92 +137,3 @@ impl WriterFactory for TargetFileSizeWriterFactory { >) } } - -struct UnpartitionedWriteState { - writer: Box, Result = Vec
>>, -} - -impl UnpartitionedWriteState { - pub fn new( - writer: Box, Result = Vec
>>, - ) -> Self { - Self { writer } - } -} - -impl DynBlockingSinkState for UnpartitionedWriteState { - fn as_any_mut(&mut self) -> &mut dyn std::any::Any { - self - } -} - -pub(crate) struct UnpartitionedWriteSink { - name: &'static str, - writer_factory: TargetFileSizeWriterFactory, - file_schema: SchemaRef, -} - -impl UnpartitionedWriteSink { - pub(crate) fn new( - name: &'static str, - writer_factory: TargetFileSizeWriterFactory, - file_schema: SchemaRef, - ) -> Self { - Self { - name, - writer_factory, - file_schema, - } - } - pub(crate) fn arced(self) -> Arc { - Arc::new(self) - } -} - -impl BlockingSink for UnpartitionedWriteSink { - #[instrument(skip_all, name = "WriteSink::sink")] - fn sink( - &self, - input: &Arc, - state_handle: &BlockingSinkState, - ) -> DaftResult { - state_handle.with_state_mut::(|state| { - state.writer.write(input)?; - Ok(BlockingSinkStatus::NeedMoreInput) - }) - } - - #[instrument(skip_all, name = "WriteSink::finalize")] - fn finalize( - &self, - states: Vec>, - ) -> DaftResult> { - let mut results = vec![]; - for mut state in states { - let state = state - .as_any_mut() - .downcast_mut::() - .expect("State type mismatch"); - results.extend(state.writer.close()?); - } - let mp = Arc::new(MicroPartition::new_loaded( - self.file_schema.clone(), - results.into(), - None, - )); - Ok(Some(mp.into())) - } - - fn name(&self) -> &'static str { - self.name - } - - fn make_state(&self) -> DaftResult> { - let writer = self.writer_factory.create_writer(0, None)?; - Ok(Box::new(UnpartitionedWriteState::new(writer)) as Box) - } - - fn max_concurrency(&self) -> usize { - 1 - } -} diff --git a/src/daft-writers/src/lib.rs b/src/daft-writers/src/lib.rs index d08f3888a0..fd065abfa2 100644 --- a/src/daft-writers/src/lib.rs +++ b/src/daft-writers/src/lib.rs @@ -1,12 +1,25 @@ +#![feature(hash_raw_entry)] mod batch; +mod file; +mod partition; mod physical; + #[cfg(feature = "python")] mod python; -pub use batch::{TargetBatchWriter, TargetBatchWriterFactory}; +use std::{cmp::min, sync::Arc}; + +use batch::TargetBatchWriterFactory; +use common_daft_config::DaftExecutionConfig; use common_error::DaftResult; +use common_file_formats::FileFormat; +use daft_core::prelude::SchemaRef; +use daft_micropartition::MicroPartition; +use daft_plan::OutputFileInfo; use daft_table::Table; -pub use physical::PhysicalWriterFactory; +use file::TargetFileSizeWriterFactory; +use partition::PartitionedWriterFactory; +use physical::PhysicalWriterFactory; /// This trait is used to abstract the writing of data to a file. /// The `Input` type is the type of data that will be written to the file. @@ -32,3 +45,76 @@ pub trait WriterFactory: Send + Sync { partition_values: Option<&Table>, ) -> DaftResult>>; } + +pub fn make_writer_factory( + file_info: &OutputFileInfo, + schema: &SchemaRef, + cfg: &DaftExecutionConfig, +) -> Arc, Result = Vec
>> { + let estimated_row_size_bytes = schema.estimate_row_size_bytes(); + let base_writer_factory = PhysicalWriterFactory::new(file_info.clone()); + match file_info.file_format { + FileFormat::Parquet => { + let target_in_memory_file_size = + cfg.parquet_target_filesize as f64 * cfg.parquet_inflation_factor; + let target_in_memory_row_group_size = + cfg.parquet_target_row_group_size as f64 * cfg.parquet_inflation_factor; + + let target_file_rows = if estimated_row_size_bytes > 0.0 { + target_in_memory_file_size / estimated_row_size_bytes + } else { + target_in_memory_file_size + } as usize; + + let target_row_group_rows = min( + target_file_rows, + if estimated_row_size_bytes > 0.0 { + target_in_memory_row_group_size / estimated_row_size_bytes + } else { + target_in_memory_row_group_size + } as usize, + ); + + let row_group_writer_factory = + TargetBatchWriterFactory::new(Arc::new(base_writer_factory), target_row_group_rows); + + let file_writer_factory = TargetFileSizeWriterFactory::new( + Arc::new(row_group_writer_factory), + target_file_rows, + ); + + if let Some(partition_cols) = &file_info.partition_cols { + let partitioned_writer_factory = PartitionedWriterFactory::new( + Arc::new(file_writer_factory), + partition_cols.clone(), + ); + Arc::new(partitioned_writer_factory) + } else { + Arc::new(file_writer_factory) + } + } + FileFormat::Csv => { + let target_in_memory_file_size = + cfg.csv_target_filesize as f64 * cfg.csv_inflation_factor; + let target_file_rows = if estimated_row_size_bytes > 0.0 { + target_in_memory_file_size / estimated_row_size_bytes + } else { + target_in_memory_file_size + } as usize; + + let file_writer_factory = + TargetFileSizeWriterFactory::new(Arc::new(base_writer_factory), target_file_rows); + + if let Some(partition_cols) = &file_info.partition_cols { + let partitioned_writer_factory = PartitionedWriterFactory::new( + Arc::new(file_writer_factory), + partition_cols.clone(), + ); + Arc::new(partitioned_writer_factory) + } else { + Arc::new(file_writer_factory) + } + } + _ => unreachable!("Physical write should only support Parquet and CSV"), + } +} diff --git a/src/daft-local-execution/src/sinks/partitioned_write.rs b/src/daft-writers/src/partition.rs similarity index 62% rename from src/daft-local-execution/src/sinks/partitioned_write.rs rename to src/daft-writers/src/partition.rs index 2b0dce1213..24eff1f171 100644 --- a/src/daft-local-execution/src/sinks/partitioned_write.rs +++ b/src/daft-writers/src/partition.rs @@ -4,24 +4,13 @@ use std::{ }; use common_error::DaftResult; -use daft_core::{ - prelude::{AsArrow, SchemaRef}, - utils::identity_hash_set::IndexHash, -}; +use daft_core::{array::ops::as_arrow::AsArrow, utils::identity_hash_set::IndexHash}; use daft_dsl::ExprRef; use daft_io::IOStatsContext; use daft_micropartition::MicroPartition; use daft_table::Table; -use daft_writers::{FileWriter, WriterFactory}; -use tracing::instrument; -use super::blocking_sink::{ - BlockingSink, BlockingSinkState, BlockingSinkStatus, DynBlockingSinkState, -}; -use crate::{ - dispatcher::{Dispatcher, PartitionedDispatcher}, - pipeline::PipelineResultType, -}; +use crate::{FileWriter, WriterFactory}; /// PartitionedWriter is a writer that partitions the input data by a set of columns, and writes each partition /// to a separate file. It uses a map to keep track of the writers for each partition. @@ -157,102 +146,3 @@ impl WriterFactory for PartitionedWriterFactory { >) } } - -struct PartitionedWriteState { - writer: Box, Result = Vec
>>, -} - -impl PartitionedWriteState { - pub fn new( - writer: Box, Result = Vec
>>, - ) -> Self { - Self { writer } - } -} - -impl DynBlockingSinkState for PartitionedWriteState { - fn as_any_mut(&mut self) -> &mut dyn std::any::Any { - self - } -} - -pub struct PartitionedWriteSink { - name: &'static str, - writer_factory: PartitionedWriterFactory, - file_schema: SchemaRef, - partition_cols: Vec, -} - -impl PartitionedWriteSink { - pub fn new( - name: &'static str, - partition_cols: Vec, - writer_factory: PartitionedWriterFactory, - file_schema: SchemaRef, - ) -> Self { - Self { - name, - writer_factory, - file_schema, - partition_cols, - } - } - pub fn arced(self) -> Arc { - Arc::new(self) - } -} - -impl BlockingSink for PartitionedWriteSink { - #[instrument(skip_all, name = "PartitionedWriteSink::sink")] - fn sink( - &self, - input: &Arc, - state_handle: &BlockingSinkState, - ) -> DaftResult { - state_handle.with_state_mut::(|state| { - state.writer.write(input)?; - Ok(BlockingSinkStatus::NeedMoreInput) - }) - } - - #[instrument(skip_all, name = "PartitionedWriteSink::finalize")] - fn finalize( - &self, - states: Vec>, - ) -> DaftResult> { - let mut results = vec![]; - for mut state in states { - let state = state - .as_any_mut() - .downcast_mut::() - .expect("State type mismatch"); - results.extend(state.writer.close()?); - } - let mp = Arc::new(MicroPartition::new_loaded( - self.file_schema.clone(), - results.into(), - None, - )); - Ok(Some(mp.into())) - } - - fn name(&self) -> &'static str { - self.name - } - - fn make_state(&self) -> DaftResult> { - let writer = self.writer_factory.create_writer(0, None)?; - Ok(Box::new(PartitionedWriteState::new(writer)) as Box) - } - - fn max_concurrency(&self) -> usize { - *crate::NUM_CPUS - } - - fn make_dispatcher( - &self, - _runtime_handle: &crate::ExecutionRuntimeHandle, - ) -> Arc { - Arc::new(PartitionedDispatcher::new(self.partition_cols.clone())) - } -} From 072ae6e959ff952cd1cd82e316e04535d7ada14a Mon Sep 17 00:00:00 2001 From: Colin Ho Date: Wed, 30 Oct 2024 23:35:34 -0700 Subject: [PATCH 14/15] address feedback --- Cargo.lock | 11 --- Cargo.toml | 1 - daft/io/writer.py | 13 +++- src/common/buffer/Cargo.toml | 10 --- src/daft-local-execution/Cargo.toml | 1 - .../src/buffer.rs} | 63 +++++------------ src/daft-local-execution/src/dispatcher.rs | 8 ++- .../src/intermediate_ops/intermediate_op.rs | 6 +- src/daft-local-execution/src/lib.rs | 2 +- src/daft-local-execution/src/pipeline.rs | 25 ++++--- .../src/sinks/aggregate.rs | 48 ++++++------- .../src/sinks/blocking_sink.rs | 65 +++++------------- .../src/sinks/hash_join_build.rs | 22 +++--- src/daft-local-execution/src/sinks/sort.rs | 44 +++++------- src/daft-local-execution/src/sinks/write.rs | 45 ++++++++----- src/daft-micropartition/Cargo.toml | 1 - src/daft-micropartition/src/micropartition.rs | 23 ------- src/daft-micropartition/src/ops/concat.rs | 24 ++++--- src/daft-micropartition/src/python.rs | 4 +- src/daft-writers/Cargo.toml | 1 - src/daft-writers/src/batch.rs | 57 +++++++++++++--- src/daft-writers/src/file.rs | 67 ++++++++++--------- src/daft-writers/src/lib.rs | 4 ++ src/daft-writers/src/partition.rs | 8 +++ src/daft-writers/src/python.rs | 5 ++ 25 files changed, 260 insertions(+), 298 deletions(-) delete mode 100644 src/common/buffer/Cargo.toml rename src/{common/buffer/src/lib.rs => daft-local-execution/src/buffer.rs} (57%) diff --git a/Cargo.lock b/Cargo.lock index 0e084cdd1b..f5a1ef9c25 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -1380,13 +1380,6 @@ dependencies = [ "pyo3", ] -[[package]] -name = "common-buffer" -version = "0.3.0-dev0" -dependencies = [ - "common-error", -] - [[package]] name = "common-daft-config" version = "0.3.0-dev0" @@ -1787,7 +1780,6 @@ dependencies = [ name = "daft" version = "0.3.0-dev0" dependencies = [ - "common-buffer", "common-daft-config", "common-display", "common-file-formats", @@ -2095,7 +2087,6 @@ name = "daft-local-execution" version = "0.3.0-dev0" dependencies = [ "async-trait", - "common-buffer", "common-daft-config", "common-display", "common-error", @@ -2133,7 +2124,6 @@ version = "0.3.0-dev0" dependencies = [ "arrow2", "bincode", - "common-buffer", "common-error", "common-file-formats", "common-runtime", @@ -2381,7 +2371,6 @@ dependencies = [ name = "daft-writers" version = "0.3.0-dev0" dependencies = [ - "common-buffer", "common-daft-config", "common-error", "common-file-formats", diff --git a/Cargo.toml b/Cargo.toml index e62f68195e..6b0b24dbe0 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -1,5 +1,4 @@ [dependencies] -common-buffer = {path = "src/common/buffer", default-features = false} common-daft-config = {path = "src/common/daft-config", default-features = false} common-display = {path = "src/common/display", default-features = false} common-file-formats = {path = "src/common/file-formats", default-features = false} diff --git a/daft/io/writer.py b/daft/io/writer.py index bf3f8a3370..a3e99046a8 100644 --- a/daft/io/writer.py +++ b/daft/io/writer.py @@ -1,6 +1,6 @@ import uuid from abc import ABC, abstractmethod -from typing import Optional, Union +from typing import Optional from daft.daft import IOConfig from daft.dependencies import pa, pacsv, pq @@ -50,7 +50,6 @@ def __init__( self.fs.create_dir(self.dir_path, recursive=True) self.compression = compression if compression is not None else "none" - self.current_writer: Optional[Union[pq.ParquetWriter, pacsv.CSVWriter]] = None @abstractmethod def write(self, table: MicroPartition) -> None: @@ -63,7 +62,7 @@ def write(self, table: MicroPartition) -> None: @abstractmethod def close(self) -> Table: - """Close the writer and return metadata about the written file. + """Close the writer and return metadata about the written file. Write should not be called after close. Returns: Table containing metadata about the written file, including path and partition values. @@ -88,6 +87,8 @@ def __init__( compression=compression, io_config=io_config, ) + self.is_closed = False + self.current_writer: Optional[pq.ParquetWriter] = None def _create_writer(self, schema: pa.Schema) -> pq.ParquetWriter: return pq.ParquetWriter( @@ -99,6 +100,7 @@ def _create_writer(self, schema: pa.Schema) -> pq.ParquetWriter: ) def write(self, table: MicroPartition) -> None: + assert not self.is_closed, "Cannot write to a closed ParquetFileWriter" if self.current_writer is None: self.current_writer = self._create_writer(table.schema().to_pyarrow_schema()) self.current_writer.write_table(table.to_arrow()) @@ -107,6 +109,7 @@ def close(self) -> Table: if self.current_writer is not None: self.current_writer.close() + self.is_closed = True metadata = {"path": Series.from_pylist([self.full_path])} if self.partition_values is not None: for col_name in self.partition_values.column_names(): @@ -129,6 +132,8 @@ def __init__( partition_values=partition_values, io_config=io_config, ) + self.current_writer: Optional[pacsv.CSVWriter] = None + self.is_closed = False def _create_writer(self, schema: pa.Schema) -> pacsv.CSVWriter: return pacsv.CSVWriter( @@ -137,6 +142,7 @@ def _create_writer(self, schema: pa.Schema) -> pacsv.CSVWriter: ) def write(self, table: MicroPartition) -> None: + assert not self.is_closed, "Cannot write to a closed CSVFileWriter" if self.current_writer is None: self.current_writer = self._create_writer(table.schema().to_pyarrow_schema()) self.current_writer.write_table(table.to_arrow()) @@ -145,6 +151,7 @@ def close(self) -> Table: if self.current_writer is not None: self.current_writer.close() + self.is_closed = True metadata = {"path": Series.from_pylist([self.full_path])} if self.partition_values is not None: for col_name in self.partition_values.column_names(): diff --git a/src/common/buffer/Cargo.toml b/src/common/buffer/Cargo.toml deleted file mode 100644 index b9ba3a62f8..0000000000 --- a/src/common/buffer/Cargo.toml +++ /dev/null @@ -1,10 +0,0 @@ -[dependencies] -common-error = {path = "../error", default-features = false} - -[lints] -workspace = true - -[package] -edition = {workspace = true} -name = "common-buffer" -version = {workspace = true} diff --git a/src/daft-local-execution/Cargo.toml b/src/daft-local-execution/Cargo.toml index 9757487993..f0be341b6a 100644 --- a/src/daft-local-execution/Cargo.toml +++ b/src/daft-local-execution/Cargo.toml @@ -1,6 +1,5 @@ [dependencies] async-trait = {workspace = true} -common-buffer = {path = "../common/buffer", default-features = false} common-daft-config = {path = "../common/daft-config", default-features = false} common-display = {path = "../common/display", default-features = false} common-error = {path = "../common/error", default-features = false} diff --git a/src/common/buffer/src/lib.rs b/src/daft-local-execution/src/buffer.rs similarity index 57% rename from src/common/buffer/src/lib.rs rename to src/daft-local-execution/src/buffer.rs index 4028940eef..4211200182 100644 --- a/src/common/buffer/src/lib.rs +++ b/src/daft-local-execution/src/buffer.rs @@ -1,47 +1,16 @@ use std::{cmp::Ordering::*, collections::VecDeque, sync::Arc}; use common_error::DaftResult; - -#[allow(clippy::len_without_is_empty)] -pub trait Bufferable { - fn len(&self) -> usize; - fn slice(&self, start: usize, end: usize) -> DaftResult - where - Self: Sized; - fn concat(parts: &[&Self]) -> DaftResult - where - Self: Sized; -} - -impl Bufferable for Arc -where - T: Bufferable, -{ - fn len(&self) -> usize { - (**self).len() - } - - fn concat(parts: &[&Self]) -> DaftResult { - // Deref twice: once for the reference, once for the Arc - let inner_parts: Vec<&T> = parts.iter().map(|p| p.as_ref()).collect(); - let concated = T::concat(&inner_parts)?; - Ok(Self::new(concated)) - } - - fn slice(&self, start: usize, end: usize) -> DaftResult { - let sliced = (**self).slice(start, end)?; - Ok(Self::new(sliced)) - } -} +use daft_micropartition::MicroPartition; // A buffer that accumulates morsels until a threshold is reached -pub struct RowBasedBuffer { - pub buffer: VecDeque, +pub struct RowBasedBuffer { + pub buffer: VecDeque>, pub curr_len: usize, pub threshold: usize, } -impl RowBasedBuffer { +impl RowBasedBuffer { pub fn new(threshold: usize) -> Self { assert!(threshold > 0); Self { @@ -52,9 +21,9 @@ impl RowBasedBuffer { } // Push a morsel to the buffer - pub fn push(&mut self, part: B) { + pub fn push(&mut self, part: &Arc) { self.curr_len += part.len(); - self.buffer.push_back(part); + self.buffer.push_back(part.clone()); } // Pop enough morsels that reach the threshold @@ -62,7 +31,7 @@ impl RowBasedBuffer { // - If the buffer has exactly enough morsels, return the morsels // - If the buffer has more than enough morsels, return a vec of morsels, each correctly sized to the threshold. // The remaining morsels will be pushed back to the buffer - pub fn pop_enough(&mut self) -> DaftResult>> { + pub fn pop_enough(&mut self) -> DaftResult>>> { match self.curr_len.cmp(&self.threshold) { Less => Ok(None), Equal => { @@ -71,28 +40,26 @@ impl RowBasedBuffer { self.curr_len = 0; Ok(Some(vec![part])) } else { - let chunk = - B::concat(&std::mem::take(&mut self.buffer).iter().collect::>())?; + let chunk = MicroPartition::concat(std::mem::take(&mut self.buffer))?; self.curr_len = 0; - Ok(Some(vec![chunk])) + Ok(Some(vec![chunk.into()])) } } Greater => { let num_ready_chunks = self.curr_len / self.threshold; - let concated = - B::concat(&std::mem::take(&mut self.buffer).iter().collect::>())?; + let concated = MicroPartition::concat(std::mem::take(&mut self.buffer))?; let mut start = 0; let mut parts_to_return = Vec::with_capacity(num_ready_chunks); for _ in 0..num_ready_chunks { let end = start + self.threshold; let part = concated.slice(start, end)?; - parts_to_return.push(part); + parts_to_return.push(part.into()); start = end; } if start < concated.len() { let part = concated.slice(start, concated.len())?; self.curr_len = part.len(); - self.buffer.push_back(part); + self.buffer.push_back(part.into()); } else { self.curr_len = 0; } @@ -102,14 +69,14 @@ impl RowBasedBuffer { } // Pop all morsels in the buffer regardless of the threshold - pub fn pop_all(&mut self) -> DaftResult> { + pub fn pop_all(&mut self) -> DaftResult>> { assert!(self.curr_len < self.threshold); if self.buffer.is_empty() { Ok(None) } else { - let concated = B::concat(&self.buffer.iter().collect::>())?; + let concated = MicroPartition::concat(std::mem::take(&mut self.buffer))?; self.curr_len = 0; - Ok(Some(concated)) + Ok(Some(concated.into())) } } } diff --git a/src/daft-local-execution/src/dispatcher.rs b/src/daft-local-execution/src/dispatcher.rs index 5c50b368b1..d21fc306b6 100644 --- a/src/daft-local-execution/src/dispatcher.rs +++ b/src/daft-local-execution/src/dispatcher.rs @@ -1,11 +1,13 @@ use std::sync::Arc; use async_trait::async_trait; -use common_buffer::RowBasedBuffer; use common_error::DaftResult; use daft_dsl::ExprRef; -use crate::{channel::Sender, pipeline::PipelineResultType, runtime_stats::CountingReceiver}; +use crate::{ + buffer::RowBasedBuffer, channel::Sender, pipeline::PipelineResultType, + runtime_stats::CountingReceiver, +}; #[async_trait] pub(crate) trait Dispatcher { @@ -47,7 +49,7 @@ impl Dispatcher for RoundRobinBufferedDispatcher { let _ = worker_sender.send(morsel.clone()).await; } } else { - buffer.push(morsel.as_data().clone()); + buffer.push(morsel.as_data()); if let Some(ready) = buffer.pop_enough()? { for r in ready { let _ = send_to_next_worker(r.into()).await; diff --git a/src/daft-local-execution/src/intermediate_ops/intermediate_op.rs b/src/daft-local-execution/src/intermediate_ops/intermediate_op.rs index 21d3e16728..d7503fa33c 100644 --- a/src/daft-local-execution/src/intermediate_ops/intermediate_op.rs +++ b/src/daft-local-execution/src/intermediate_ops/intermediate_op.rs @@ -1,6 +1,5 @@ use std::sync::{Arc, Mutex}; -use common_buffer::RowBasedBuffer; use common_display::tree::TreeDisplay; use common_error::DaftResult; use common_runtime::get_compute_runtime; @@ -8,6 +7,7 @@ use daft_micropartition::MicroPartition; use tracing::{info_span, instrument}; use crate::{ + buffer::RowBasedBuffer, channel::{create_channel, PipelineChannel, Receiver, Sender}, pipeline::{PipelineNode, PipelineResultType}, runtime_stats::{CountingReceiver, CountingSender, RuntimeStatsContext}, @@ -185,14 +185,14 @@ impl IntermediateNode { }; for (idx, mut receiver) in receivers.into_iter().enumerate() { - let mut buffer = RowBasedBuffer::>::new(morsel_size); + let mut buffer = RowBasedBuffer::new(morsel_size); while let Some(morsel) = receiver.recv().await { if morsel.should_broadcast() { for worker_sender in &worker_senders { let _ = worker_sender.send((idx, morsel.clone())).await; } } else { - buffer.push(morsel.as_data().clone()); + buffer.push(morsel.as_data()); if let Some(ready) = buffer.pop_enough()? { for part in ready { let _ = send_to_next_worker(idx, part.into()).await; diff --git a/src/daft-local-execution/src/lib.rs b/src/daft-local-execution/src/lib.rs index 4a8a2cddca..553ad18b40 100644 --- a/src/daft-local-execution/src/lib.rs +++ b/src/daft-local-execution/src/lib.rs @@ -1,5 +1,5 @@ #![feature(let_chains)] -#![feature(hash_raw_entry)] +mod buffer; mod channel; mod dispatcher; mod intermediate_ops; diff --git a/src/daft-local-execution/src/pipeline.rs b/src/daft-local-execution/src/pipeline.rs index 391f03d118..b371af0c8a 100644 --- a/src/daft-local-execution/src/pipeline.rs +++ b/src/daft-local-execution/src/pipeline.rs @@ -32,10 +32,15 @@ use crate::{ sample::SampleOperator, unpivot::UnpivotOperator, }, sinks::{ - aggregate::AggregateSink, blocking_sink::BlockingSinkNode, concat::ConcatSink, - hash_join_build::HashJoinBuildSink, limit::LimitSink, - outer_hash_join_probe::OuterHashJoinProbeSink, sort::SortSink, - streaming_sink::StreamingSinkNode, write::WriteSink, + aggregate::AggregateSink, + blocking_sink::BlockingSinkNode, + concat::ConcatSink, + hash_join_build::HashJoinBuildSink, + limit::LimitSink, + outer_hash_join_probe::OuterHashJoinProbeSink, + sort::SortSink, + streaming_sink::StreamingSinkNode, + write::{WriteFormat, WriteSink}, }, sources::{empty_scan::EmptyScanSource, in_memory::InMemorySource}, ExecutionRuntimeHandle, PipelineCreationSnafu, @@ -423,15 +428,15 @@ pub fn physical_plan_to_pipeline( }) => { let child_node = physical_plan_to_pipeline(input, psets, cfg)?; let writer_factory = make_writer_factory(file_info, data_schema, cfg); - let name = match (file_info.file_format, file_info.partition_cols.is_some()) { - (FileFormat::Parquet, true) => "PartitionedParquet", - (FileFormat::Parquet, false) => "Parquet", - (FileFormat::Csv, true) => "PartitionedCsv", - (FileFormat::Csv, false) => "Csv", + let write_format = match (file_info.file_format, file_info.partition_cols.is_some()) { + (FileFormat::Parquet, true) => WriteFormat::PartitionedParquet, + (FileFormat::Parquet, false) => WriteFormat::Parquet, + (FileFormat::Csv, true) => WriteFormat::PartitionedCsv, + (FileFormat::Csv, false) => WriteFormat::Csv, (_, _) => panic!("Unsupported file format"), }; let write_sink = WriteSink::new( - name, + write_format, writer_factory, file_info.partition_cols.clone(), file_schema.clone(), diff --git a/src/daft-local-execution/src/sinks/aggregate.rs b/src/daft-local-execution/src/sinks/aggregate.rs index 7b5102ffc3..abc8acce4c 100644 --- a/src/daft-local-execution/src/sinks/aggregate.rs +++ b/src/daft-local-execution/src/sinks/aggregate.rs @@ -5,9 +5,7 @@ use daft_dsl::ExprRef; use daft_micropartition::MicroPartition; use tracing::instrument; -use super::blocking_sink::{ - BlockingSink, BlockingSinkState, BlockingSinkStatus, DynBlockingSinkState, -}; +use super::blocking_sink::{BlockingSink, BlockingSinkState, BlockingSinkStatus}; use crate::{pipeline::PipelineResultType, NUM_CPUS}; enum AggregateState { @@ -24,18 +22,18 @@ impl AggregateState { } } - fn finalize(&mut self) -> DaftResult>> { + fn finalize(&mut self) -> Vec> { let res = if let Self::Accumulating(ref mut parts) = self { std::mem::take(parts) } else { panic!("AggregateSink should be in Accumulating state"); }; *self = Self::Done; - Ok(res) + res } } -impl DynBlockingSinkState for AggregateState { +impl BlockingSinkState for AggregateState { fn as_any_mut(&mut self) -> &mut dyn std::any::Any { self } @@ -60,37 +58,29 @@ impl BlockingSink for AggregateSink { fn sink( &self, input: &Arc, - state_handle: &BlockingSinkState, + mut state: Box, ) -> DaftResult { - state_handle.with_state_mut::(|state| { - state.push(input.clone()); - Ok(BlockingSinkStatus::NeedMoreInput) - }) + state + .as_any_mut() + .downcast_mut::() + .expect("AggregateSink should have AggregateState") + .push(input.clone()); + Ok(BlockingSinkStatus::NeedMoreInput(state)) } #[instrument(skip_all, name = "AggregateSink::finalize")] fn finalize( &self, - states: Vec>, + states: Vec>, ) -> DaftResult> { - let mut all_parts = vec![]; - for mut state in states { - let state = state + let all_parts = states.into_iter().flat_map(|mut state| { + state .as_any_mut() .downcast_mut::() - .expect("State type mismatch"); - all_parts.extend(state.finalize()?); - } - assert!( - !all_parts.is_empty(), - "We can not finalize AggregateSink with no data" - ); - let concated = MicroPartition::concat( - &all_parts - .iter() - .map(std::convert::AsRef::as_ref) - .collect::>(), - )?; + .expect("AggregateSink should have AggregateState") + .finalize() + }); + let concated = MicroPartition::concat(all_parts)?; let agged = Arc::new(concated.agg(&self.agg_exprs, &self.group_by)?); Ok(Some(agged.into())) } @@ -103,7 +93,7 @@ impl BlockingSink for AggregateSink { *NUM_CPUS } - fn make_state(&self) -> DaftResult> { + fn make_state(&self) -> DaftResult> { Ok(Box::new(AggregateState::Accumulating(vec![]))) } } diff --git a/src/daft-local-execution/src/sinks/blocking_sink.rs b/src/daft-local-execution/src/sinks/blocking_sink.rs index 7dcad48dc0..7835fbe138 100644 --- a/src/daft-local-execution/src/sinks/blocking_sink.rs +++ b/src/daft-local-execution/src/sinks/blocking_sink.rs @@ -1,4 +1,4 @@ -use std::sync::{Arc, Mutex}; +use std::sync::Arc; use common_display::tree::TreeDisplay; use common_error::DaftResult; @@ -14,52 +14,28 @@ use crate::{ runtime_stats::RuntimeStatsContext, ExecutionRuntimeHandle, JoinSnafu, TaskSet, }; -pub trait DynBlockingSinkState: Send + Sync { +pub trait BlockingSinkState: Send + Sync { fn as_any_mut(&mut self) -> &mut dyn std::any::Any; } -pub(crate) struct BlockingSinkState { - inner: Mutex>, -} - -impl BlockingSinkState { - fn new(inner: Box) -> Arc { - Arc::new(Self { - inner: Mutex::new(inner), - }) - } - - pub(crate) fn with_state_mut(&self, f: F) -> R - where - F: FnOnce(&mut T) -> R, - { - let mut guard = self.inner.lock().unwrap(); - let state = guard - .as_any_mut() - .downcast_mut::() - .expect("State type mismatch"); - f(state) - } -} - pub enum BlockingSinkStatus { - NeedMoreInput, + NeedMoreInput(Box), #[allow(dead_code)] - Finished, + Finished(Box), } pub trait BlockingSink: Send + Sync { fn sink( &self, input: &Arc, - state_handle: &BlockingSinkState, + state: Box, ) -> DaftResult; fn finalize( &self, - states: Vec>, + states: Vec>, ) -> DaftResult>; fn name(&self) -> &'static str; - fn make_state(&self) -> DaftResult>; + fn make_state(&self) -> DaftResult>; fn make_dispatcher(&self, runtime_handle: &ExecutionRuntimeHandle) -> Arc { Arc::new(RoundRobinBufferedDispatcher::new( runtime_handle.default_morsel_size(), @@ -94,41 +70,34 @@ impl BlockingSinkNode { op: Arc, mut input_receiver: Receiver, rt_context: Arc, - ) -> DaftResult> { + ) -> DaftResult> { let span = info_span!("BlockingSink::Sink"); let compute_runtime = get_compute_runtime(); - let state_wrapper = BlockingSinkState::new(op.make_state()?); + let mut state = op.make_state()?; while let Some(morsel) = input_receiver.recv().await { let op = op.clone(); let morsel = morsel.clone(); let span = span.clone(); let rt_context = rt_context.clone(); - let state_wrapper = state_wrapper.clone(); - let fut = async move { - rt_context.in_span(&span, || op.sink(morsel.as_data(), &state_wrapper)) - }; + let fut = async move { rt_context.in_span(&span, || op.sink(morsel.as_data(), state)) }; let result = compute_runtime.await_on(fut).await??; match result { - BlockingSinkStatus::NeedMoreInput => {} - BlockingSinkStatus::Finished => { - break; + BlockingSinkStatus::NeedMoreInput(new_state) => { + state = new_state; + } + BlockingSinkStatus::Finished(new_state) => { + return Ok(new_state); } } } - // Take the state out of the Arc and Mutex because we need to return it. - // It should be guaranteed that the ONLY holder of state at this point is this function. - Ok(Arc::into_inner(state_wrapper) - .expect("Completed worker should have exclusive access to state wrapper") - .inner - .into_inner() - .expect("Completed worker should have exclusive access to inner state")) + Ok(state) } fn spawn_workers( op: Arc, input_receivers: Vec>, - task_set: &mut TaskSet>>, + task_set: &mut TaskSet>>, stats: Arc, ) { for input_receiver in input_receivers { diff --git a/src/daft-local-execution/src/sinks/hash_join_build.rs b/src/daft-local-execution/src/sinks/hash_join_build.rs index 55440c3f5f..677f63279d 100644 --- a/src/daft-local-execution/src/sinks/hash_join_build.rs +++ b/src/daft-local-execution/src/sinks/hash_join_build.rs @@ -7,9 +7,7 @@ use daft_micropartition::MicroPartition; use daft_plan::JoinType; use daft_table::{make_probeable_builder, ProbeState, ProbeableBuilder, Table}; -use super::blocking_sink::{ - BlockingSink, BlockingSinkState, BlockingSinkStatus, DynBlockingSinkState, -}; +use super::blocking_sink::{BlockingSink, BlockingSinkState, BlockingSinkStatus}; use crate::pipeline::PipelineResultType; enum ProbeTableState { @@ -76,7 +74,7 @@ impl ProbeTableState { } } -impl DynBlockingSinkState for ProbeTableState { +impl BlockingSinkState for ProbeTableState { fn as_any_mut(&mut self) -> &mut dyn std::any::Any { self } @@ -110,17 +108,19 @@ impl BlockingSink for HashJoinBuildSink { fn sink( &self, input: &Arc, - state_handle: &BlockingSinkState, + mut state: Box, ) -> DaftResult { - state_handle.with_state_mut::(|state| { - state.add_tables(input)?; - Ok(BlockingSinkStatus::NeedMoreInput) - }) + state + .as_any_mut() + .downcast_mut::() + .expect("HashJoinBuildSink should have ProbeTableState") + .add_tables(input)?; + Ok(BlockingSinkStatus::NeedMoreInput(state)) } fn finalize( &self, - states: Vec>, + states: Vec>, ) -> DaftResult> { assert_eq!(states.len(), 1); let mut state = states.into_iter().next().unwrap(); @@ -140,7 +140,7 @@ impl BlockingSink for HashJoinBuildSink { 1 } - fn make_state(&self) -> DaftResult> { + fn make_state(&self) -> DaftResult> { Ok(Box::new(ProbeTableState::new( &self.key_schema, self.projection.clone(), diff --git a/src/daft-local-execution/src/sinks/sort.rs b/src/daft-local-execution/src/sinks/sort.rs index 12ca9cb66e..83c933d1ec 100644 --- a/src/daft-local-execution/src/sinks/sort.rs +++ b/src/daft-local-execution/src/sinks/sort.rs @@ -5,9 +5,7 @@ use daft_dsl::ExprRef; use daft_micropartition::MicroPartition; use tracing::instrument; -use super::blocking_sink::{ - BlockingSink, BlockingSinkState, BlockingSinkStatus, DynBlockingSinkState, -}; +use super::blocking_sink::{BlockingSink, BlockingSinkState, BlockingSinkStatus}; use crate::{pipeline::PipelineResultType, NUM_CPUS}; enum SortState { @@ -24,18 +22,18 @@ impl SortState { } } - fn finalize(&mut self) -> DaftResult>> { + fn finalize(&mut self) -> Vec> { let res = if let Self::Building(ref mut parts) = self { std::mem::take(parts) } else { panic!("SortSink should be in Building state"); }; *self = Self::Done; - Ok(res) + res } } -impl DynBlockingSinkState for SortState { +impl BlockingSinkState for SortState { fn as_any_mut(&mut self) -> &mut dyn std::any::Any { self } @@ -59,37 +57,29 @@ impl BlockingSink for SortSink { fn sink( &self, input: &Arc, - state_handle: &BlockingSinkState, + mut state: Box, ) -> DaftResult { - state_handle.with_state_mut::(|state| { - state.push(input.clone()); - Ok(BlockingSinkStatus::NeedMoreInput) - }) + state + .as_any_mut() + .downcast_mut::() + .expect("SortSink should have sort state") + .push(input.clone()); + Ok(BlockingSinkStatus::NeedMoreInput(state)) } #[instrument(skip_all, name = "SortSink::finalize")] fn finalize( &self, - states: Vec>, + states: Vec>, ) -> DaftResult> { - let mut parts = Vec::new(); - for mut state in states { + let parts = states.into_iter().flat_map(|mut state| { let state = state .as_any_mut() .downcast_mut::() .expect("State type mismatch"); - parts.extend(state.finalize()?); - } - assert!( - !parts.is_empty(), - "We can not finalize SortSink with no data" - ); - let concated = MicroPartition::concat( - &parts - .iter() - .map(std::convert::AsRef::as_ref) - .collect::>(), - )?; + state.finalize() + }); + let concated = MicroPartition::concat(parts)?; let sorted = Arc::new(concated.sort(&self.sort_by, &self.descending)?); Ok(Some(sorted.into())) } @@ -98,7 +88,7 @@ impl BlockingSink for SortSink { "SortResult" } - fn make_state(&self) -> DaftResult> { + fn make_state(&self) -> DaftResult> { Ok(Box::new(SortState::Building(Vec::new()))) } diff --git a/src/daft-local-execution/src/sinks/write.rs b/src/daft-local-execution/src/sinks/write.rs index e1c839e328..002f32a25a 100644 --- a/src/daft-local-execution/src/sinks/write.rs +++ b/src/daft-local-execution/src/sinks/write.rs @@ -8,15 +8,20 @@ use daft_table::Table; use daft_writers::{FileWriter, WriterFactory}; use tracing::instrument; -use super::blocking_sink::{ - BlockingSink, BlockingSinkState, BlockingSinkStatus, DynBlockingSinkState, -}; +use super::blocking_sink::{BlockingSink, BlockingSinkState, BlockingSinkStatus}; use crate::{ dispatcher::{Dispatcher, PartitionedDispatcher, RoundRobinBufferedDispatcher}, pipeline::PipelineResultType, NUM_CPUS, }; +pub enum WriteFormat { + Parquet, + PartitionedParquet, + Csv, + PartitionedCsv, +} + struct WriteState { writer: Box, Result = Vec
>>, } @@ -29,14 +34,14 @@ impl WriteState { } } -impl DynBlockingSinkState for WriteState { +impl BlockingSinkState for WriteState { fn as_any_mut(&mut self) -> &mut dyn std::any::Any { self } } pub(crate) struct WriteSink { - name: &'static str, + write_format: WriteFormat, writer_factory: Arc, Result = Vec
>>, partition_by: Option>, file_schema: SchemaRef, @@ -44,13 +49,13 @@ pub(crate) struct WriteSink { impl WriteSink { pub(crate) fn new( - name: &'static str, + write_format: WriteFormat, writer_factory: Arc, Result = Vec
>>, partition_by: Option>, file_schema: SchemaRef, ) -> Self { Self { - name, + write_format, writer_factory, partition_by, file_schema, @@ -63,18 +68,21 @@ impl BlockingSink for WriteSink { fn sink( &self, input: &Arc, - state_handle: &BlockingSinkState, + mut state: Box, ) -> DaftResult { - state_handle.with_state_mut::(|state| { - state.writer.write(input)?; - Ok(BlockingSinkStatus::NeedMoreInput) - }) + state + .as_any_mut() + .downcast_mut::() + .expect("WriteSink should have WriteState") + .writer + .write(input)?; + Ok(BlockingSinkStatus::NeedMoreInput(state)) } #[instrument(skip_all, name = "WriteSink::finalize")] fn finalize( &self, - states: Vec>, + states: Vec>, ) -> DaftResult> { let mut results = vec![]; for mut state in states { @@ -93,12 +101,17 @@ impl BlockingSink for WriteSink { } fn name(&self) -> &'static str { - self.name + match self.write_format { + WriteFormat::Parquet => "ParquetSink", + WriteFormat::PartitionedParquet => "PartitionedParquetSink", + WriteFormat::Csv => "CsvSink", + WriteFormat::PartitionedCsv => "PartitionedCsvSink", + } } - fn make_state(&self) -> DaftResult> { + fn make_state(&self) -> DaftResult> { let writer = self.writer_factory.create_writer(0, None)?; - Ok(Box::new(WriteState::new(writer)) as Box) + Ok(Box::new(WriteState::new(writer)) as Box) } fn make_dispatcher( diff --git a/src/daft-micropartition/Cargo.toml b/src/daft-micropartition/Cargo.toml index f9aef047d6..b03935d6a6 100644 --- a/src/daft-micropartition/Cargo.toml +++ b/src/daft-micropartition/Cargo.toml @@ -1,7 +1,6 @@ [dependencies] arrow2 = {workspace = true} bincode = {workspace = true} -common-buffer = {path = "../common/buffer", default-features = false} common-error = {path = "../common/error", default-features = false} common-file-formats = {path = "../common/file-formats", default-features = false} common-runtime = {path = "../common/runtime", default-features = false} diff --git a/src/daft-micropartition/src/micropartition.rs b/src/daft-micropartition/src/micropartition.rs index f674ed1655..af78186cd8 100644 --- a/src/daft-micropartition/src/micropartition.rs +++ b/src/daft-micropartition/src/micropartition.rs @@ -5,7 +5,6 @@ use std::{ }; use arrow2::io::parquet::read::schema::infer_schema_with_options; -use common_buffer::Bufferable; use common_error::DaftResult; use common_file_formats::{CsvSourceConfig, FileFormatConfig, ParquetSourceConfig}; use common_runtime::get_io_runtime; @@ -1277,27 +1276,5 @@ impl Display for MicroPartition { } } -impl Bufferable for MicroPartition { - fn len(&self) -> usize { - self.len() - } - - fn concat(parts: &[&Self]) -> DaftResult - where - Self: Sized, - { - let concated = Self::concat(parts)?; - Ok(concated) - } - - fn slice(&self, start: usize, end: usize) -> DaftResult - where - Self: Sized, - { - let sliced = self.slice(start, end)?; - Ok(sliced) - } -} - #[cfg(test)] mod test {} diff --git a/src/daft-micropartition/src/ops/concat.rs b/src/daft-micropartition/src/ops/concat.rs index 2108cc01e3..2ca6160178 100644 --- a/src/daft-micropartition/src/ops/concat.rs +++ b/src/daft-micropartition/src/ops/concat.rs @@ -1,4 +1,4 @@ -use std::sync::Mutex; +use std::{borrow::Borrow, ops::Deref, sync::Mutex}; use common_error::{DaftError, DaftResult}; use daft_io::IOStatsContext; @@ -7,18 +7,25 @@ use daft_stats::TableMetadata; use crate::micropartition::{MicroPartition, TableState}; impl MicroPartition { - pub fn concat(mps: &[&Self]) -> DaftResult { + pub fn concat(mps: I) -> DaftResult + where + I: IntoIterator, + T: Deref, + T::Target: Borrow, + { + let mps: Vec<_> = mps.into_iter().collect(); if mps.is_empty() { return Err(DaftError::ValueError( "Need at least 1 MicroPartition to perform concat".to_string(), )); } - let first_table = mps.first().unwrap(); + let first_table = mps.first().unwrap().deref().borrow(); - let first_schema = first_table.schema.as_ref(); + let first_schema = &first_table.schema; for tab in mps.iter().skip(1) { - if tab.schema.as_ref() != first_schema { + let tab = tab.deref().borrow(); + if &tab.schema != first_schema { return Err(DaftError::SchemaMismatch(format!( "MicroPartition concat requires all schemas to match, {} vs {}", first_schema, tab.schema @@ -30,13 +37,14 @@ impl MicroPartition { let mut all_tables = vec![]; - for m in mps { + for m in &mps { + let m = m.deref().borrow(); let tables = m.tables_or_read(io_stats.clone())?; all_tables.extend_from_slice(tables.as_slice()); } let mut all_stats = None; - for stats in mps.iter().flat_map(|m| &m.statistics) { + for stats in mps.iter().flat_map(|m| &m.deref().borrow().statistics) { if all_stats.is_none() { all_stats = Some(stats.clone()); } @@ -48,7 +56,7 @@ impl MicroPartition { let new_len = all_tables.iter().map(daft_table::Table::len).sum(); Ok(Self { - schema: mps.first().unwrap().schema.clone(), + schema: first_schema.clone(), state: Mutex::new(TableState::Loaded(all_tables.into())), metadata: TableMetadata { length: new_len }, statistics: all_stats, diff --git a/src/daft-micropartition/src/python.rs b/src/daft-micropartition/src/python.rs index 8875c29517..ab9b4a7db1 100644 --- a/src/daft-micropartition/src/python.rs +++ b/src/daft-micropartition/src/python.rs @@ -146,8 +146,8 @@ impl PyMicroPartition { #[staticmethod] pub fn concat(py: Python, to_concat: Vec) -> PyResult { - let mps: Vec<_> = to_concat.iter().map(|t| t.inner.as_ref()).collect(); - py.allow_threads(|| Ok(MicroPartition::concat(mps.as_slice())?.into())) + let mps_iter = to_concat.iter().map(|t| t.inner.as_ref()); + py.allow_threads(|| Ok(MicroPartition::concat(mps_iter)?.into())) } pub fn slice(&self, py: Python, start: i64, end: i64) -> PyResult { diff --git a/src/daft-writers/Cargo.toml b/src/daft-writers/Cargo.toml index e818fddb9a..19cff1807d 100644 --- a/src/daft-writers/Cargo.toml +++ b/src/daft-writers/Cargo.toml @@ -1,5 +1,4 @@ [dependencies] -common-buffer = {path = "../common/buffer", default-features = false} common-daft-config = {path = "../common/daft-config", default-features = false} common-error = {path = "../common/error", default-features = false} common-file-formats = {path = "../common/file-formats", default-features = false} diff --git a/src/daft-writers/src/batch.rs b/src/daft-writers/src/batch.rs index d56ae6fdf2..d6e2f57fa2 100644 --- a/src/daft-writers/src/batch.rs +++ b/src/daft-writers/src/batch.rs @@ -1,6 +1,5 @@ use std::sync::Arc; -use common_buffer::RowBasedBuffer; use common_error::DaftResult; use daft_micropartition::MicroPartition; use daft_table::Table; @@ -8,10 +7,12 @@ use daft_table::Table; use crate::{FileWriter, WriterFactory}; // TargetBatchWriter is a writer that writes in batches of rows, i.e. for Parquet where we want to write -// a row group at a time. It uses a buffer to accumulate rows until it has enough to write a batch. +// a row group at a time. pub struct TargetBatchWriter { - buffer: RowBasedBuffer>, + target_in_memory_chunk_rows: usize, writer: Box, Result = Option
>>, + leftovers: Option>, + is_closed: bool, } impl TargetBatchWriter { @@ -20,8 +21,10 @@ impl TargetBatchWriter { writer: Box, Result = Option
>>, ) -> Self { Self { - buffer: RowBasedBuffer::new(target_in_memory_chunk_rows), + target_in_memory_chunk_rows, writer, + leftovers: None, + is_closed: false, } } } @@ -31,19 +34,51 @@ impl FileWriter for TargetBatchWriter { type Result = Option
; fn write(&mut self, input: &Arc) -> DaftResult<()> { - self.buffer.push(input.clone()); - if let Some(ready) = self.buffer.pop_enough()? { - for r in ready { - self.writer.write(&r)?; + assert!( + !self.is_closed, + "Cannot write to a closed TargetBatchWriter" + ); + let input = if let Some(leftovers) = self.leftovers.take() { + MicroPartition::concat([&leftovers, input])?.into() + } else { + input.clone() + }; + + let mut local_offset = 0; + loop { + let remaining_rows = input.len() - local_offset; + + use std::cmp::Ordering; + match remaining_rows.cmp(&self.target_in_memory_chunk_rows) { + Ordering::Equal => { + // Write exactly one chunk + let chunk = input.slice(local_offset, local_offset + remaining_rows)?; + return self.writer.write(&chunk.into()); + } + Ordering::Less => { + // Store remaining rows as leftovers + let remainder = input.slice(local_offset, local_offset + remaining_rows)?; + self.leftovers = Some(remainder.into()); + return Ok(()); + } + Ordering::Greater => { + // Write a complete chunk and continue + let chunk = input.slice( + local_offset, + local_offset + self.target_in_memory_chunk_rows, + )?; + self.writer.write(&chunk.into())?; + local_offset += self.target_in_memory_chunk_rows; + } } } - Ok(()) } fn close(&mut self) -> DaftResult { - if let Some(ready) = self.buffer.pop_all()? { - self.writer.write(&ready)?; + if let Some(leftovers) = self.leftovers.take() { + self.writer.write(&leftovers)?; } + self.is_closed = true; self.writer.close() } } diff --git a/src/daft-writers/src/file.rs b/src/daft-writers/src/file.rs index a71ec189b4..6d06aa3a41 100644 --- a/src/daft-writers/src/file.rs +++ b/src/daft-writers/src/file.rs @@ -15,6 +15,7 @@ struct TargetFileSizeWriter { target_in_memory_file_rows: usize, results: Vec
, partition_values: Option
, + is_closed: bool, } impl TargetFileSizeWriter { @@ -32,6 +33,7 @@ impl TargetFileSizeWriter { target_in_memory_file_rows, results: vec![], partition_values, + is_closed: false, }) } @@ -39,7 +41,6 @@ impl TargetFileSizeWriter { if let Some(result) = self.current_writer.close()? { self.results.push(result); } - self.current_file_rows = 0; self.current_writer = self .writer_factory .create_writer(self.results.len(), self.partition_values.as_ref())?; @@ -52,43 +53,48 @@ impl FileWriter for TargetFileSizeWriter { type Result = Vec
; fn write(&mut self, input: &Arc) -> DaftResult<()> { + assert!( + !self.is_closed, + "Cannot write to a closed TargetFileSizeWriter" + ); use std::cmp::Ordering; - match (input.len() + self.current_file_rows).cmp(&self.target_in_memory_file_rows) { - Ordering::Equal => { - self.current_writer.write(input)?; - self.rotate_writer()?; - } - Ordering::Greater => { - // Finish up the current writer first - let remaining_rows = self.target_in_memory_file_rows - self.current_file_rows; - let (to_write, mut remaining) = input.split_at(remaining_rows)?; - self.current_writer.write(&to_write.into())?; - self.rotate_writer()?; - - // Write as many full files as possible - let num_full_files = remaining.len() / self.target_in_memory_file_rows; - for _ in 0..num_full_files { - let (to_write, new_remaining) = - remaining.split_at(self.target_in_memory_file_rows)?; + + let mut local_offset = 0; + + loop { + let remaining_input_rows = input.len() - local_offset; + let rows_until_target = self.target_in_memory_file_rows - self.current_file_rows; + + match remaining_input_rows.cmp(&rows_until_target) { + Ordering::Equal => { + // Write exactly what's needed to fill the current file + let to_write = + input.slice(local_offset, local_offset + remaining_input_rows)?; self.current_writer.write(&to_write.into())?; self.rotate_writer()?; - remaining = new_remaining; + self.current_file_rows = 0; + return Ok(()); } - - // Write the remaining rows - if !remaining.is_empty() { - self.current_file_rows = remaining.len(); - self.current_writer.write(&remaining.into())?; - } else { + Ordering::Less => { + // Write remaining input and update counter + let to_write = + input.slice(local_offset, local_offset + remaining_input_rows)?; + self.current_writer.write(&to_write.into())?; + self.current_file_rows += remaining_input_rows; + return Ok(()); + } + Ordering::Greater => { + // Write what fits in current file + let to_write = input.slice(local_offset, local_offset + rows_until_target)?; + self.current_writer.write(&to_write.into())?; + self.rotate_writer()?; self.current_file_rows = 0; + + // Update offset and continue loop + local_offset += rows_until_target; } } - Ordering::Less => { - self.current_writer.write(input)?; - self.current_file_rows += input.len(); - } } - Ok(()) } fn close(&mut self) -> DaftResult { @@ -97,6 +103,7 @@ impl FileWriter for TargetFileSizeWriter { self.results.push(result); } } + self.is_closed = true; Ok(std::mem::take(&mut self.results)) } } diff --git a/src/daft-writers/src/lib.rs b/src/daft-writers/src/lib.rs index fd065abfa2..0ba902c084 100644 --- a/src/daft-writers/src/lib.rs +++ b/src/daft-writers/src/lib.rs @@ -1,4 +1,5 @@ #![feature(hash_raw_entry)] +#![feature(let_chains)] mod batch; mod file; mod partition; @@ -28,7 +29,10 @@ pub trait FileWriter: Send + Sync { type Input; type Result; + /// Write data to the file. fn write(&mut self, data: &Self::Input) -> DaftResult<()>; + + /// Close the file and return the result. The caller should NOT write to the file after calling this method. fn close(&mut self) -> DaftResult; } diff --git a/src/daft-writers/src/partition.rs b/src/daft-writers/src/partition.rs index 24eff1f171..87519ad027 100644 --- a/src/daft-writers/src/partition.rs +++ b/src/daft-writers/src/partition.rs @@ -20,6 +20,7 @@ struct PartitionedWriter { saved_partition_values: Vec
, writer_factory: Arc, Result = Vec
>>, partition_by: Vec, + is_closed: bool, } impl PartitionedWriter { @@ -32,6 +33,7 @@ impl PartitionedWriter { saved_partition_values: vec![], writer_factory, partition_by, + is_closed: false, } } @@ -51,6 +53,11 @@ impl FileWriter for PartitionedWriter { type Result = Vec
; fn write(&mut self, input: &Arc) -> DaftResult<()> { + assert!( + !self.is_closed, + "Cannot write to a closed PartitionedWriter" + ); + let (split_tables, partition_values) = Self::partition(self.partition_by.as_slice(), input)?; let partition_values_hash = partition_values.hash_rows()?; @@ -108,6 +115,7 @@ impl FileWriter for PartitionedWriter { for (_, mut writer) in self.per_partition_writers.drain() { results.extend(writer.close()?); } + self.is_closed = true; Ok(results) } } diff --git a/src/daft-writers/src/python.rs b/src/daft-writers/src/python.rs index 4047d54848..7bcecb2b03 100644 --- a/src/daft-writers/src/python.rs +++ b/src/daft-writers/src/python.rs @@ -9,6 +9,7 @@ use crate::FileWriter; pub struct PyArrowWriter { py_writer: PyObject, + is_closed: bool, } impl PyArrowWriter { @@ -45,6 +46,7 @@ impl PyArrowWriter { ))?; Ok(Self { py_writer: py_writer.into(), + is_closed: false, }) }) } @@ -79,6 +81,7 @@ impl PyArrowWriter { ))?; Ok(Self { py_writer: py_writer.into(), + is_closed: false, }) }) } @@ -89,6 +92,7 @@ impl FileWriter for PyArrowWriter { type Result = Option
; fn write(&mut self, data: &Self::Input) -> DaftResult<()> { + assert!(!self.is_closed, "Cannot write to a closed PyArrowWriter"); Python::with_gil(|py| { let py_micropartition = py .import_bound(pyo3::intern!(py, "daft.table"))? @@ -102,6 +106,7 @@ impl FileWriter for PyArrowWriter { } fn close(&mut self) -> DaftResult { + self.is_closed = true; Python::with_gil(|py| { let result = self .py_writer From b67499e18490af8409d0f5d0f65f5f3b62604da3 Mon Sep 17 00:00:00 2001 From: Colin Ho Date: Thu, 31 Oct 2024 11:20:20 -0700 Subject: [PATCH 15/15] add unit tests for batch and file writers --- src/daft-writers/src/batch.rs | 75 +++++++++++++++++++++++++++ src/daft-writers/src/file.rs | 69 +++++++++++++++++++++++++ src/daft-writers/src/lib.rs | 3 ++ src/daft-writers/src/partition.rs | 1 + src/daft-writers/src/test.rs | 84 +++++++++++++++++++++++++++++++ 5 files changed, 232 insertions(+) create mode 100644 src/daft-writers/src/test.rs diff --git a/src/daft-writers/src/batch.rs b/src/daft-writers/src/batch.rs index d6e2f57fa2..d8af2f94fb 100644 --- a/src/daft-writers/src/batch.rs +++ b/src/daft-writers/src/batch.rs @@ -118,3 +118,78 @@ impl WriterFactory for TargetBatchWriterFactory { ))) } } + +#[cfg(test)] +mod tests { + + use super::*; + use crate::test::{make_dummy_mp, DummyWriterFactory}; + + #[test] + fn test_target_batch_writer_exact_batch() { + let dummy_writer_factory = DummyWriterFactory; + let mut writer = + TargetBatchWriter::new(1, dummy_writer_factory.create_writer(0, None).unwrap()); + + let mp = make_dummy_mp(1); + writer.write(&mp).unwrap(); + let res = writer.close().unwrap(); + + assert!(res.is_some()); + let write_count = res + .unwrap() + .get_column("write_count") + .unwrap() + .u64() + .unwrap() + .get(0) + .unwrap(); + assert_eq!(write_count, 1); + } + + #[test] + fn test_target_batch_writer_small_batches() { + let dummy_writer_factory = DummyWriterFactory; + let mut writer = + TargetBatchWriter::new(3, dummy_writer_factory.create_writer(0, None).unwrap()); + + for _ in 0..8 { + let mp = make_dummy_mp(1); + writer.write(&mp).unwrap(); + } + let res = writer.close().unwrap(); + + assert!(res.is_some()); + let write_count = res + .unwrap() + .get_column("write_count") + .unwrap() + .u64() + .unwrap() + .get(0) + .unwrap(); + assert_eq!(write_count, 3); + } + + #[test] + fn test_target_batch_writer_big_batch() { + let dummy_writer_factory = DummyWriterFactory; + let mut writer = + TargetBatchWriter::new(3, dummy_writer_factory.create_writer(0, None).unwrap()); + + let mp = make_dummy_mp(10); + writer.write(&mp).unwrap(); + let res = writer.close().unwrap(); + + assert!(res.is_some()); + let write_count = res + .unwrap() + .get_column("write_count") + .unwrap() + .u64() + .unwrap() + .get(0) + .unwrap(); + assert_eq!(write_count, 4); + } +} diff --git a/src/daft-writers/src/file.rs b/src/daft-writers/src/file.rs index 6d06aa3a41..639b720d15 100644 --- a/src/daft-writers/src/file.rs +++ b/src/daft-writers/src/file.rs @@ -144,3 +144,72 @@ impl WriterFactory for TargetFileSizeWriterFactory { >) } } + +#[cfg(test)] +mod tests { + + use super::*; + use crate::test::{make_dummy_mp, DummyWriterFactory}; + + #[test] + fn test_target_file_writer_exact_file() { + let dummy_writer_factory = DummyWriterFactory; + let mut writer = + TargetFileSizeWriter::new(1, Arc::new(dummy_writer_factory), None).unwrap(); + + let mp = make_dummy_mp(1); + writer.write(&mp).unwrap(); + let res = writer.close().unwrap(); + assert_eq!(res.len(), 1); + } + + #[test] + fn test_target_file_writer_less_rows_for_one_file() { + let dummy_writer_factory = DummyWriterFactory; + let mut writer = + TargetFileSizeWriter::new(3, Arc::new(dummy_writer_factory), None).unwrap(); + + let mp = make_dummy_mp(2); + writer.write(&mp).unwrap(); + let res = writer.close().unwrap(); + assert_eq!(res.len(), 1); + } + + #[test] + fn test_target_file_writer_more_rows_for_one_file() { + let dummy_writer_factory = DummyWriterFactory; + let mut writer = + TargetFileSizeWriter::new(3, Arc::new(dummy_writer_factory), None).unwrap(); + + let mp = make_dummy_mp(4); + writer.write(&mp).unwrap(); + let res = writer.close().unwrap(); + assert_eq!(res.len(), 2); + } + + #[test] + fn test_target_file_writer_multiple_files() { + let dummy_writer_factory = DummyWriterFactory; + let mut writer = + TargetFileSizeWriter::new(3, Arc::new(dummy_writer_factory), None).unwrap(); + + let mp = make_dummy_mp(10); + writer.write(&mp).unwrap(); + let res = writer.close().unwrap(); + assert_eq!(res.len(), 4); + } + + #[test] + fn test_target_file_writer_many_writes_many_files() { + let dummy_writer_factory = DummyWriterFactory; + let mut writer = + TargetFileSizeWriter::new(3, Arc::new(dummy_writer_factory), None).unwrap(); + + for _ in 0..10 { + let mp = make_dummy_mp(1); + writer.write(&mp).unwrap(); + } + let res = writer.close().unwrap(); + assert_eq!(res.len(), 4); + } +} diff --git a/src/daft-writers/src/lib.rs b/src/daft-writers/src/lib.rs index 0ba902c084..405596485b 100644 --- a/src/daft-writers/src/lib.rs +++ b/src/daft-writers/src/lib.rs @@ -5,6 +5,9 @@ mod file; mod partition; mod physical; +#[cfg(test)] +mod test; + #[cfg(feature = "python")] mod python; diff --git a/src/daft-writers/src/partition.rs b/src/daft-writers/src/partition.rs index 87519ad027..73b0ab0216 100644 --- a/src/daft-writers/src/partition.rs +++ b/src/daft-writers/src/partition.rs @@ -15,6 +15,7 @@ use crate::{FileWriter, WriterFactory}; /// PartitionedWriter is a writer that partitions the input data by a set of columns, and writes each partition /// to a separate file. It uses a map to keep track of the writers for each partition. struct PartitionedWriter { + // TODO: Figure out a way to NOT use the IndexHash + RawEntryMut pattern here. Ideally we want to store ScalarValues, aka. single Rows of the partition values as keys for the hashmap. per_partition_writers: HashMap, Result = Vec
>>>, saved_partition_values: Vec
, diff --git a/src/daft-writers/src/test.rs b/src/daft-writers/src/test.rs new file mode 100644 index 0000000000..f862930d3f --- /dev/null +++ b/src/daft-writers/src/test.rs @@ -0,0 +1,84 @@ +use std::sync::Arc; + +use common_error::DaftResult; +use daft_core::{ + prelude::{Int64Array, Schema, UInt64Array, Utf8Array}, + series::IntoSeries, +}; +use daft_micropartition::MicroPartition; +use daft_table::Table; + +use crate::{FileWriter, WriterFactory}; + +pub(crate) struct DummyWriterFactory; + +impl WriterFactory for DummyWriterFactory { + type Input = Arc; + type Result = Option
; + + fn create_writer( + &self, + file_idx: usize, + partition_values: Option<&Table>, + ) -> DaftResult>> { + Ok(Box::new(DummyWriter { + file_idx: file_idx.to_string(), + partition_values: partition_values.cloned(), + write_count: 0, + }) + as Box< + dyn FileWriter, + >) + } +} + +pub(crate) struct DummyWriter { + file_idx: String, + partition_values: Option
, + write_count: usize, +} + +impl FileWriter for DummyWriter { + type Input = Arc; + type Result = Option
; + + fn write(&mut self, _input: &Self::Input) -> DaftResult<()> { + self.write_count += 1; + Ok(()) + } + + fn close(&mut self) -> DaftResult { + let path_series = + Utf8Array::from_values("path", std::iter::once(self.file_idx.clone())).into_series(); + let write_count_series = + UInt64Array::from_values("write_count", std::iter::once(self.write_count as u64)) + .into_series(); + let path_table = Table::new_unchecked( + Schema::new(vec![ + path_series.field().clone(), + write_count_series.field().clone(), + ]) + .unwrap(), + vec![path_series.into(), write_count_series.into()], + 1, + ); + if let Some(partition_values) = self.partition_values.take() { + let unioned = path_table.union(&partition_values)?; + Ok(Some(unioned)) + } else { + Ok(Some(path_table)) + } + } +} + +pub(crate) fn make_dummy_mp(num_rows: usize) -> Arc { + let series = + Int64Array::from_values("ints", std::iter::repeat(42).take(num_rows)).into_series(); + let schema = Arc::new(Schema::new(vec![series.field().clone()]).unwrap()); + let table = Table::new_unchecked(schema.clone(), vec![series.into()], num_rows); + Arc::new(MicroPartition::new_loaded( + schema.into(), + vec![table].into(), + None, + )) +}