From ac01a34bf1771b39753f7c45cadaa3eef853a297 Mon Sep 17 00:00:00 2001 From: Shawn Chang Date: Thu, 18 Sep 2025 23:44:01 -0700 Subject: [PATCH 01/13] partitionhead --- crates/iceberg/src/spec/partition.rs | 24 +++ .../writer/base_writer/data_file_writer.rs | 69 ++++--- .../base_writer/equality_delete_writer.rs | 49 ++--- crates/iceberg/src/writer/file_writer/mod.rs | 4 +- .../src/writer/file_writer/parquet_writer.rs | 55 +---- .../src/writer/file_writer/rolling_writer.rs | 191 +++++++++++++----- .../datafusion/src/physical_plan/write.rs | 30 +-- 7 files changed, 268 insertions(+), 154 deletions(-) diff --git a/crates/iceberg/src/spec/partition.rs b/crates/iceberg/src/spec/partition.rs index fff9b62d78..167cf3cf88 100644 --- a/crates/iceberg/src/spec/partition.rs +++ b/crates/iceberg/src/spec/partition.rs @@ -194,6 +194,15 @@ impl PartitionKey { Self { spec, schema, data } } + /// todo do we need this? + pub fn unpartition_key() -> Self { + Self { + spec: PartitionSpec::unpartition_spec(), + schema: Arc::new(Schema::builder().build().unwrap()), + data: Struct::empty(), + } + } + /// Generates a partition path based on the partition values. pub fn to_path(&self) -> String { self.spec.partition_to_path(&self.data, self.schema.clone()) @@ -207,6 +216,21 @@ impl PartitionKey { Some(pk) => pk.spec.is_unpartitioned(), } } + + /// todo doc + pub fn spec(&self) -> &PartitionSpec { + &self.spec + } + + /// todo doc + pub fn schema(&self) -> &SchemaRef { + &self.schema + } + + /// todo doc + pub fn data(&self) -> &Struct { + &self.data + } } /// Reference to [`UnboundPartitionSpec`]. diff --git a/crates/iceberg/src/writer/base_writer/data_file_writer.rs b/crates/iceberg/src/writer/base_writer/data_file_writer.rs index f215673df1..f08b10d4e4 100644 --- a/crates/iceberg/src/writer/base_writer/data_file_writer.rs +++ b/crates/iceberg/src/writer/base_writer/data_file_writer.rs @@ -21,54 +21,65 @@ use arrow_array::RecordBatch; use itertools::Itertools; use crate::Result; -use crate::spec::{DataContentType, DataFile, Struct}; +use crate::spec::{DEFAULT_PARTITION_SPEC_ID, DataContentType, DataFile, PartitionKey, Struct}; +use crate::writer::file_writer::location_generator::{FileNameGenerator, LocationGenerator}; +use crate::writer::file_writer::rolling_writer::RollingFileWriter; use crate::writer::file_writer::{FileWriter, FileWriterBuilder}; use crate::writer::{CurrentFileStatus, IcebergWriter, IcebergWriterBuilder}; /// Builder for `DataFileWriter`. #[derive(Clone, Debug)] -pub struct DataFileWriterBuilder { - inner: B, - partition_value: Option, - partition_spec_id: i32, +pub struct DataFileWriterBuilder { + inner_writer: RollingFileWriter, + partition_key: Option, } -impl DataFileWriterBuilder { +impl + DataFileWriterBuilder +{ /// Create a new `DataFileWriterBuilder` using a `FileWriterBuilder`. - pub fn new(inner: B, partition_value: Option, partition_spec_id: i32) -> Self { + pub fn new( + inner_writer: RollingFileWriter, + partition_key: Option, + ) -> Self { Self { - inner, - partition_value, - partition_spec_id, + inner_writer, + partition_key, } } } #[async_trait::async_trait] -impl IcebergWriterBuilder for DataFileWriterBuilder { - type R = DataFileWriter; +impl IcebergWriterBuilder + for DataFileWriterBuilder +{ + type R = DataFileWriter; async fn build(self) -> Result { Ok(DataFileWriter { - inner_writer: Some(self.inner.clone().build().await?), - partition_value: self.partition_value.unwrap_or(Struct::empty()), - partition_spec_id: self.partition_spec_id, + inner_writer: Some(self.inner_writer), // todo revisit this, probably still need a builder for rolling writer + partition_key: self.partition_key, }) } } /// A writer write data is within one spec/partition. #[derive(Debug)] -pub struct DataFileWriter { - inner_writer: Option, - partition_value: Struct, - partition_spec_id: i32, +pub struct DataFileWriter { + inner_writer: Option>, + partition_key: Option, } #[async_trait::async_trait] -impl IcebergWriter for DataFileWriter { +impl IcebergWriter + for DataFileWriter +{ async fn write(&mut self, batch: RecordBatch) -> Result<()> { - self.inner_writer.as_mut().unwrap().write(&batch).await + self.inner_writer + .as_mut() + .unwrap() + .write(&self.partition_key, &batch) + .await } async fn close(&mut self) -> Result> { @@ -79,15 +90,25 @@ impl IcebergWriter for DataFileWriter { .into_iter() .map(|mut res| { res.content(DataContentType::Data); - res.partition(self.partition_value.clone()); - res.partition_spec_id(self.partition_spec_id); + res.partition( + self.partition_key + .as_ref() + .map_or(Struct::empty(), |pk| pk.data().clone()), + ); + res.partition_spec_id( + self.partition_key + .as_ref() + .map_or(DEFAULT_PARTITION_SPEC_ID, |pk| pk.spec().spec_id()), + ); res.build().expect("Guaranteed to be valid") }) .collect_vec()) } } -impl CurrentFileStatus for DataFileWriter { +impl CurrentFileStatus + for DataFileWriter +{ fn current_file_path(&self) -> String { self.inner_writer.as_ref().unwrap().current_file_path() } diff --git a/crates/iceberg/src/writer/base_writer/equality_delete_writer.rs b/crates/iceberg/src/writer/base_writer/equality_delete_writer.rs index 765ff1cacd..cae452d68c 100644 --- a/crates/iceberg/src/writer/base_writer/equality_delete_writer.rs +++ b/crates/iceberg/src/writer/base_writer/equality_delete_writer.rs @@ -26,21 +26,23 @@ use parquet::arrow::PARQUET_FIELD_ID_META_KEY; use crate::arrow::record_batch_projector::RecordBatchProjector; use crate::arrow::schema_to_arrow_schema; -use crate::spec::{DataFile, SchemaRef, Struct}; +use crate::spec::{DataFile, PartitionKey, SchemaRef, Struct, DEFAULT_PARTITION_SPEC_ID}; use crate::writer::file_writer::{FileWriter, FileWriterBuilder}; use crate::writer::{IcebergWriter, IcebergWriterBuilder}; use crate::{Error, ErrorKind, Result}; +use crate::writer::file_writer::location_generator::{FileNameGenerator, LocationGenerator}; +use crate::writer::file_writer::rolling_writer::RollingFileWriter; /// Builder for `EqualityDeleteWriter`. #[derive(Clone, Debug)] -pub struct EqualityDeleteFileWriterBuilder { - inner: B, +pub struct EqualityDeleteFileWriterBuilder { + inner: RollingFileWriter, config: EqualityDeleteWriterConfig, } -impl EqualityDeleteFileWriterBuilder { +impl EqualityDeleteFileWriterBuilder { /// Create a new `EqualityDeleteFileWriterBuilder` using a `FileWriterBuilder`. - pub fn new(inner: B, config: EqualityDeleteWriterConfig) -> Self { + pub fn new(inner: RollingFileWriter, config: EqualityDeleteWriterConfig) -> Self { Self { inner, config } } } @@ -52,8 +54,7 @@ pub struct EqualityDeleteWriterConfig { equality_ids: Vec, // Projector used to project the data chunk into specific fields. projector: RecordBatchProjector, - partition_value: Struct, - partition_spec_id: i32, + partition_key: Option, } impl EqualityDeleteWriterConfig { @@ -61,8 +62,7 @@ impl EqualityDeleteWriterConfig { pub fn new( equality_ids: Vec, original_schema: SchemaRef, - partition_value: Option, - partition_spec_id: i32, + partition_key: Option ) -> Result { let original_arrow_schema = Arc::new(schema_to_arrow_schema(&original_schema)?); let projector = RecordBatchProjector::new( @@ -98,8 +98,7 @@ impl EqualityDeleteWriterConfig { Ok(Self { equality_ids, projector, - partition_value: partition_value.unwrap_or(Struct::empty()), - partition_spec_id, + partition_key, }) } @@ -110,36 +109,34 @@ impl EqualityDeleteWriterConfig { } #[async_trait::async_trait] -impl IcebergWriterBuilder for EqualityDeleteFileWriterBuilder { - type R = EqualityDeleteFileWriter; +impl IcebergWriterBuilder for EqualityDeleteFileWriterBuilder { + type R = EqualityDeleteFileWriter; async fn build(self) -> Result { Ok(EqualityDeleteFileWriter { - inner_writer: Some(self.inner.clone().build().await?), + inner_writer: Some(self.inner), // todo revisit this, probably still need a builder for rolling writer projector: self.config.projector, equality_ids: self.config.equality_ids, - partition_value: self.config.partition_value, - partition_spec_id: self.config.partition_spec_id, + partition_key: self.config.partition_key, }) } } /// Writer used to write equality delete files. #[derive(Debug)] -pub struct EqualityDeleteFileWriter { - inner_writer: Option, +pub struct EqualityDeleteFileWriter { + inner_writer: Option>, projector: RecordBatchProjector, equality_ids: Vec, - partition_value: Struct, - partition_spec_id: i32, + partition_key: Option, } #[async_trait::async_trait] -impl IcebergWriter for EqualityDeleteFileWriter { +impl IcebergWriter for EqualityDeleteFileWriter { async fn write(&mut self, batch: RecordBatch) -> Result<()> { let batch = self.projector.project_batch(batch)?; if let Some(writer) = self.inner_writer.as_mut() { - writer.write(&batch).await + writer.write(&self.partition_key, &batch).await } else { Err(Error::new( ErrorKind::Unexpected, @@ -157,8 +154,12 @@ impl IcebergWriter for EqualityDeleteFileWriter { .map(|mut res| { res.content(crate::spec::DataContentType::EqualityDeletes); res.equality_ids(Some(self.equality_ids.iter().copied().collect_vec())); - res.partition(self.partition_value.clone()); - res.partition_spec_id(self.partition_spec_id); + res.partition(self.partition_key + .as_ref() + .map_or(Struct::empty(), |pk| pk.data().clone())); + res.partition_spec_id(self.partition_key + .as_ref() + .map_or(DEFAULT_PARTITION_SPEC_ID, |pk| pk.spec().spec_id())); res.build().expect("msg") }) .collect_vec()) diff --git a/crates/iceberg/src/writer/file_writer/mod.rs b/crates/iceberg/src/writer/file_writer/mod.rs index 2a5a735534..2ed6414ce8 100644 --- a/crates/iceberg/src/writer/file_writer/mod.rs +++ b/crates/iceberg/src/writer/file_writer/mod.rs @@ -27,6 +27,8 @@ use crate::spec::DataFileBuilder; mod parquet_writer; pub use parquet_writer::{ParquetWriter, ParquetWriterBuilder}; +use crate::io::OutputFile; + pub mod location_generator; /// Module providing writers that can automatically roll over to new files based on size thresholds. pub mod rolling_writer; @@ -38,7 +40,7 @@ pub trait FileWriterBuilder: Send + Clone + 'static { /// The associated file writer type. type R: FileWriter; /// Build file writer. - fn build(self) -> impl Future> + Send; + fn build(self, output_file: OutputFile) -> impl Future> + Send; } /// File writer focus on writing record batch to different physical file format.(Such as parquet. orc) diff --git a/crates/iceberg/src/writer/file_writer/parquet_writer.rs b/crates/iceberg/src/writer/file_writer/parquet_writer.rs index 0a8a095ea8..c4d1a6c27d 100644 --- a/crates/iceberg/src/writer/file_writer/parquet_writer.rs +++ b/crates/iceberg/src/writer/file_writer/parquet_writer.rs @@ -34,7 +34,6 @@ use parquet::format::FileMetaData; use parquet::thrift::{TCompactOutputProtocol, TSerializable}; use thrift::protocol::TOutputProtocol; -use super::location_generator::{FileNameGenerator, LocationGenerator}; use super::{FileWriter, FileWriterBuilder}; use crate::arrow::{ ArrowFileReader, DEFAULT_MAP_FIELD_NAME, FieldMatchMode, NanValueCountVisitor, @@ -52,78 +51,46 @@ use crate::{Error, ErrorKind, Result}; /// ParquetWriterBuilder is used to builder a [`ParquetWriter`] #[derive(Clone, Debug)] -pub struct ParquetWriterBuilder { +pub struct ParquetWriterBuilder { props: WriterProperties, schema: SchemaRef, - partition_key: Option, match_mode: FieldMatchMode, - - file_io: FileIO, - location_generator: T, - file_name_generator: F, } -impl ParquetWriterBuilder { +impl ParquetWriterBuilder { /// Create a new `ParquetWriterBuilder` /// To construct the write result, the schema should contain the `PARQUET_FIELD_ID_META_KEY` metadata for each field. pub fn new( props: WriterProperties, schema: SchemaRef, - partition_key: Option, - file_io: FileIO, - location_generator: T, - file_name_generator: F, ) -> Self { - Self::new_with_match_mode( - props, - schema, - partition_key, - FieldMatchMode::Id, - file_io, - location_generator, - file_name_generator, - ) + Self::new_with_match_mode(props, schema, FieldMatchMode::Id) } /// Create a new `ParquetWriterBuilder` with custom match mode pub fn new_with_match_mode( props: WriterProperties, schema: SchemaRef, - partition_key: Option, match_mode: FieldMatchMode, - file_io: FileIO, - location_generator: T, - file_name_generator: F, ) -> Self { Self { props, schema, - partition_key, match_mode, - file_io, - location_generator, - file_name_generator, } } } -impl FileWriterBuilder for ParquetWriterBuilder { +impl FileWriterBuilder for ParquetWriterBuilder { type R = ParquetWriter; - async fn build(self) -> Result { - let out_file = self - .file_io - .new_output(self.location_generator.generate_location( - self.partition_key.as_ref(), - &self.file_name_generator.generate_file_name(), - ))?; - + async fn build(self, output_file: OutputFile) -> Result { Ok(ParquetWriter { schema: self.schema.clone(), inner_writer: None, writer_properties: self.props, current_row_num: 0, - out_file, + output_file, nan_value_count_visitor: NanValueCountVisitor::new_with_match_mode(self.match_mode), }) } @@ -250,7 +217,7 @@ impl SchemaVisitor for IndexByParquetPathName { /// `ParquetWriter`` is used to write arrow data into parquet file on storage. pub struct ParquetWriter { schema: SchemaRef, - out_file: OutputFile, + output_file: OutputFile, inner_writer: Option>>>, writer_properties: WriterProperties, current_row_num: usize, @@ -555,7 +522,7 @@ impl FileWriter for ParquetWriter { writer } else { let arrow_schema: ArrowSchemaRef = Arc::new(self.schema.as_ref().try_into()?); - let inner_writer = self.out_file.writer().await?; + let inner_writer = self.output_file.writer().await?; let async_writer = AsyncFileWriter::new(inner_writer); let writer = AsyncArrowWriter::try_new( async_writer, @@ -594,7 +561,7 @@ impl FileWriter for ParquetWriter { let written_size = writer.bytes_written(); if self.current_row_num == 0 { - self.out_file.delete().await.map_err(|err| { + self.output_file.delete().await.map_err(|err| { Error::new( ErrorKind::Unexpected, "Failed to delete empty parquet file.", @@ -616,7 +583,7 @@ impl FileWriter for ParquetWriter { self.schema, parquet_metadata, written_size, - self.out_file.location().to_string(), + self.output_file.location().to_string(), self.nan_value_count_visitor.nan_value_counts, )?]) } @@ -625,7 +592,7 @@ impl FileWriter for ParquetWriter { impl CurrentFileStatus for ParquetWriter { fn current_file_path(&self) -> String { - self.out_file.location().to_string() + self.output_file.location().to_string() } fn current_row_num(&self) -> usize { diff --git a/crates/iceberg/src/writer/file_writer/rolling_writer.rs b/crates/iceberg/src/writer/file_writer/rolling_writer.rs index 181205c371..361b748c03 100644 --- a/crates/iceberg/src/writer/file_writer/rolling_writer.rs +++ b/crates/iceberg/src/writer/file_writer/rolling_writer.rs @@ -15,67 +15,138 @@ // specific language governing permissions and limitations // under the License. +use std::fmt::{Debug, Formatter}; + use arrow_array::RecordBatch; -use crate::spec::DataFileBuilder; +use crate::io::{FileIO, OutputFile}; +use crate::spec::{DataFileBuilder, PartitionKey}; use crate::writer::CurrentFileStatus; +use crate::writer::file_writer::location_generator::{FileNameGenerator, LocationGenerator}; use crate::writer::file_writer::{FileWriter, FileWriterBuilder}; use crate::{Error, ErrorKind, Result}; +// +// /// Builder for creating a `RollingFileWriter` that rolls over to a new file +// /// when the data size exceeds a target threshold. +// #[derive(Clone)] +// pub struct RollingFileWriterBuilder< +// B: FileWriterBuilder, +// L: LocationGenerator, +// F: FileNameGenerator, +// > { +// inner_builder: B, +// target_file_size: usize, +// file_io: FileIO, +// location_generator: L, +// file_name_generator: F, +// } +// +// impl +// RollingFileWriterBuilder +// { +// /// Creates a new `RollingFileWriterBuilder` with the specified inner builder and target size. +// /// +// /// # Arguments +// /// +// /// * `inner_builder` - The builder for the underlying file writer +// /// * `target_file_size` - The target size in bytes before rolling over to a new file +// /// +// /// NOTE: The `target_file_size` does not exactly reflect the final size on physical storage. +// /// This is because the input size is based on the Arrow in-memory format and cannot precisely control rollover behavior. +// /// The actual file size on disk is expected to be slightly larger than `target_file_size`. +// pub fn new( +// inner_builder: B, +// target_file_size: usize, +// file_io: FileIO, +// location_generator: L, +// file_name_generator: F, +// ) -> Self { +// Self { +// inner_builder, +// target_file_size, +// file_io, +// location_generator, +// file_name_generator, +// } +// } +// } + +// impl FileWriterBuilder for RollingFileWriterBuilder { +// type R = RollingFileWriter; +// +// async fn build(self) -> Result { +// Ok(RollingFileWriter { +// inner: None, +// inner_builder: self.inner_builder, +// target_file_size: self.target_file_size, +// data_file_builders: vec![], +// }) +// } +// } -/// Builder for creating a `RollingFileWriter` that rolls over to a new file -/// when the data size exceeds a target threshold. -#[derive(Clone)] -pub struct RollingFileWriterBuilder { +/// A writer that automatically rolls over to a new file when the data size +/// exceeds a target threshold. +/// +/// This writer wraps another file writer that tracks the amount of data written. +/// When the data size exceeds the target size, it closes the current file and +/// starts writing to a new one. +pub struct RollingFileWriter { + inner: Option, inner_builder: B, target_file_size: usize, + data_file_builders: Vec, + file_io: FileIO, + location_generator: L, + file_name_generator: F, } -impl RollingFileWriterBuilder { - /// Creates a new `RollingFileWriterBuilder` with the specified inner builder and target size. - /// - /// # Arguments - /// - /// * `inner_builder` - The builder for the underlying file writer - /// * `target_file_size` - The target size in bytes before rolling over to a new file - /// - /// NOTE: The `target_file_size` does not exactly reflect the final size on physical storage. - /// This is because the input size is based on the Arrow in-memory format and cannot precisely control rollover behavior. - /// The actual file size on disk is expected to be slightly larger than `target_file_size`. - pub fn new(inner_builder: B, target_file_size: usize) -> Self { +impl Clone + for RollingFileWriter +{ + fn clone(&self) -> Self { Self { - inner_builder, - target_file_size, + inner: None, + inner_builder: self.inner_builder.clone(), + target_file_size: self.target_file_size, + data_file_builders: vec![], + file_io: self.file_io.clone(), + location_generator: self.location_generator.clone(), + file_name_generator: self.file_name_generator.clone(), } } } -impl FileWriterBuilder for RollingFileWriterBuilder { - type R = RollingFileWriter; +impl Debug + for RollingFileWriter +{ + fn fmt(&self, f: &mut Formatter<'_>) -> std::fmt::Result { + f.debug_struct("RollingFileWriter") + .field("target_file_size", &self.target_file_size) + .field("file_io", &self.file_io) + .finish() + } +} - async fn build(self) -> Result { - Ok(RollingFileWriter { +impl RollingFileWriter { + /// todo doc + pub fn new( + inner_builder: B, + target_file_size: usize, + file_io: FileIO, + location_generator: L, + file_name_generator: F, + ) -> Self { + Self { inner: None, - inner_builder: self.inner_builder, - target_file_size: self.target_file_size, + inner_builder, + target_file_size, data_file_builders: vec![], - }) + file_io, + location_generator, + file_name_generator, + } } -} - -/// A writer that automatically rolls over to a new file when the data size -/// exceeds a target threshold. -/// -/// This writer wraps another file writer that tracks the amount of data written. -/// When the data size exceeds the target size, it closes the current file and -/// starts writing to a new one. -pub struct RollingFileWriter { - inner: Option, - inner_builder: B, - target_file_size: usize, - data_file_builders: Vec, -} -impl RollingFileWriter { /// Determines if the writer should roll over to a new file. /// /// # Returns @@ -84,13 +155,29 @@ impl RollingFileWriter { fn should_roll(&self) -> bool { self.current_written_size() > self.target_file_size } -} -impl FileWriter for RollingFileWriter { - async fn write(&mut self, input: &RecordBatch) -> Result<()> { + fn new_output_file(&self, partition_key: &Option) -> Result { + self.file_io + .new_output(self.location_generator.generate_location( + partition_key.as_ref(), + &self.file_name_generator.generate_file_name(), + )) + } + + /// todo doc + pub async fn write( + &mut self, + partition_key: &Option, + input: &RecordBatch, + ) -> Result<()> { if self.inner.is_none() { // initialize inner writer - self.inner = Some(self.inner_builder.clone().build().await?); + self.inner = Some( + self.inner_builder + .clone() + .build(self.new_output_file(partition_key)?) + .await?, + ); } if self.should_roll() { @@ -99,7 +186,12 @@ impl FileWriter for RollingFileWriter { self.data_file_builders.extend(inner.close().await?); // start a new writer - self.inner = Some(self.inner_builder.clone().build().await?); + self.inner = Some( + self.inner_builder + .clone() + .build(self.new_output_file(partition_key)?) + .await?, + ); } } @@ -114,7 +206,8 @@ impl FileWriter for RollingFileWriter { } } - async fn close(mut self) -> Result> { + /// todo doc + pub async fn close(mut self) -> Result> { // close the current writer and merge the output if let Some(current_writer) = self.inner { self.data_file_builders @@ -125,7 +218,9 @@ impl FileWriter for RollingFileWriter { } } -impl CurrentFileStatus for RollingFileWriter { +impl CurrentFileStatus + for RollingFileWriter +{ fn current_file_path(&self) -> String { self.inner.as_ref().unwrap().current_file_path() } diff --git a/crates/integrations/datafusion/src/physical_plan/write.rs b/crates/integrations/datafusion/src/physical_plan/write.rs index 625405c95b..db8de2a92d 100644 --- a/crates/integrations/datafusion/src/physical_plan/write.rs +++ b/crates/integrations/datafusion/src/physical_plan/write.rs @@ -38,8 +38,7 @@ use futures::StreamExt; use iceberg::arrow::{FieldMatchMode, schema_to_arrow_schema}; use iceberg::spec::{ DataFileFormat, PROPERTY_DEFAULT_FILE_FORMAT, PROPERTY_DEFAULT_FILE_FORMAT_DEFAULT, - PROPERTY_WRITE_TARGET_FILE_SIZE_BYTES, PROPERTY_WRITE_TARGET_FILE_SIZE_BYTES_DEFAULT, - serialize_data_file_to_json, + PROPERTY_WRITE_TARGET_FILE_SIZE_BYTES, PROPERTY_WRITE_TARGET_FILE_SIZE_BYTES_DEFAULT, serialize_data_file_to_json, }; use iceberg::table::Table; use iceberg::writer::base_writer::data_file_writer::DataFileWriterBuilder; @@ -47,7 +46,7 @@ use iceberg::writer::file_writer::ParquetWriterBuilder; use iceberg::writer::file_writer::location_generator::{ DefaultFileNameGenerator, DefaultLocationGenerator, }; -use iceberg::writer::file_writer::rolling_writer::RollingFileWriterBuilder; +use iceberg::writer::file_writer::rolling_writer::RollingFileWriter; use iceberg::writer::{IcebergWriter, IcebergWriterBuilder}; use iceberg::{Error, ErrorKind}; use parquet::file::properties::WriterProperties; @@ -235,13 +234,7 @@ impl ExecutionPlan for IcebergWriteExec { let parquet_file_writer_builder = ParquetWriterBuilder::new_with_match_mode( WriterProperties::default(), self.table.metadata().current_schema().clone(), - None, FieldMatchMode::Name, - self.table.file_io().clone(), - DefaultLocationGenerator::new(self.table.metadata().clone()) - .map_err(to_datafusion_error)?, - // todo filename prefix/suffix should be configurable - DefaultFileNameGenerator::new(Uuid::now_v7().to_string(), None, file_format), ); let target_file_size = match self .table @@ -261,10 +254,21 @@ impl ExecutionPlan for IcebergWriteExec { .map_err(to_datafusion_error)?, None => PROPERTY_WRITE_TARGET_FILE_SIZE_BYTES_DEFAULT, }; - let rolling_writer_builder = - RollingFileWriterBuilder::new(parquet_file_writer_builder, target_file_size); - let data_file_writer_builder = - DataFileWriterBuilder::new(rolling_writer_builder, None, spec_id); + + let file_io = self.table.file_io().clone(); + let location_generator = DefaultLocationGenerator::new(self.table.metadata().clone()) + .map_err(to_datafusion_error)?; + // todo filename prefix/suffix should be configurable + let file_name_generator = + DefaultFileNameGenerator::new(Uuid::now_v7().to_string(), None, file_format); + let rolling_writer = RollingFileWriter::new( + parquet_file_writer_builder, + target_file_size, + file_io, + location_generator, + file_name_generator, + ); + let data_file_writer_builder = DataFileWriterBuilder::new(rolling_writer, None); // Get input data let data = execute_input_stream( From 0349d640633cc84051acfc5d146ff8019fd51ea9 Mon Sep 17 00:00:00 2001 From: Shawn Chang Date: Sat, 20 Sep 2025 20:25:00 -0700 Subject: [PATCH 02/13] little clean up and add partitioning writer traits --- Cargo.lock | 2 +- .../writer/base_writer/data_file_writer.rs | 2 +- .../base_writer/equality_delete_writer.rs | 50 ++-- .../src/writer/file_writer/parquet_writer.rs | 5 +- crates/iceberg/src/writer/mod.rs | 1 + .../src/writer/partitioning/clustered.rs | 251 ++++++++++++++++++ crates/iceberg/src/writer/partitioning/mod.rs | 40 +++ .../datafusion/src/physical_plan/write.rs | 3 +- 8 files changed, 331 insertions(+), 23 deletions(-) create mode 100644 crates/iceberg/src/writer/partitioning/clustered.rs create mode 100644 crates/iceberg/src/writer/partitioning/mod.rs diff --git a/Cargo.lock b/Cargo.lock index 00d73288f7..97d8e19003 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -3805,7 +3805,7 @@ dependencies = [ "iceberg-catalog-rest", "iceberg-datafusion", "iceberg_test_utils", - "ordered-float 4.6.0", + "ordered-float 2.10.1", "parquet", "tokio", "uuid", diff --git a/crates/iceberg/src/writer/base_writer/data_file_writer.rs b/crates/iceberg/src/writer/base_writer/data_file_writer.rs index f08b10d4e4..d4f88486e8 100644 --- a/crates/iceberg/src/writer/base_writer/data_file_writer.rs +++ b/crates/iceberg/src/writer/base_writer/data_file_writer.rs @@ -22,9 +22,9 @@ use itertools::Itertools; use crate::Result; use crate::spec::{DEFAULT_PARTITION_SPEC_ID, DataContentType, DataFile, PartitionKey, Struct}; +use crate::writer::file_writer::FileWriterBuilder; use crate::writer::file_writer::location_generator::{FileNameGenerator, LocationGenerator}; use crate::writer::file_writer::rolling_writer::RollingFileWriter; -use crate::writer::file_writer::{FileWriter, FileWriterBuilder}; use crate::writer::{CurrentFileStatus, IcebergWriter, IcebergWriterBuilder}; /// Builder for `DataFileWriter`. diff --git a/crates/iceberg/src/writer/base_writer/equality_delete_writer.rs b/crates/iceberg/src/writer/base_writer/equality_delete_writer.rs index cae452d68c..590e5067bc 100644 --- a/crates/iceberg/src/writer/base_writer/equality_delete_writer.rs +++ b/crates/iceberg/src/writer/base_writer/equality_delete_writer.rs @@ -26,21 +26,27 @@ use parquet::arrow::PARQUET_FIELD_ID_META_KEY; use crate::arrow::record_batch_projector::RecordBatchProjector; use crate::arrow::schema_to_arrow_schema; -use crate::spec::{DataFile, PartitionKey, SchemaRef, Struct, DEFAULT_PARTITION_SPEC_ID}; -use crate::writer::file_writer::{FileWriter, FileWriterBuilder}; -use crate::writer::{IcebergWriter, IcebergWriterBuilder}; -use crate::{Error, ErrorKind, Result}; +use crate::spec::{DEFAULT_PARTITION_SPEC_ID, DataFile, PartitionKey, SchemaRef, Struct}; +use crate::writer::file_writer::FileWriterBuilder; use crate::writer::file_writer::location_generator::{FileNameGenerator, LocationGenerator}; use crate::writer::file_writer::rolling_writer::RollingFileWriter; +use crate::writer::{IcebergWriter, IcebergWriterBuilder}; +use crate::{Error, ErrorKind, Result}; /// Builder for `EqualityDeleteWriter`. #[derive(Clone, Debug)] -pub struct EqualityDeleteFileWriterBuilder { +pub struct EqualityDeleteFileWriterBuilder< + B: FileWriterBuilder, + L: LocationGenerator, + F: FileNameGenerator, +> { inner: RollingFileWriter, config: EqualityDeleteWriterConfig, } -impl EqualityDeleteFileWriterBuilder { +impl + EqualityDeleteFileWriterBuilder +{ /// Create a new `EqualityDeleteFileWriterBuilder` using a `FileWriterBuilder`. pub fn new(inner: RollingFileWriter, config: EqualityDeleteWriterConfig) -> Self { Self { inner, config } @@ -62,7 +68,7 @@ impl EqualityDeleteWriterConfig { pub fn new( equality_ids: Vec, original_schema: SchemaRef, - partition_key: Option + partition_key: Option, ) -> Result { let original_arrow_schema = Arc::new(schema_to_arrow_schema(&original_schema)?); let projector = RecordBatchProjector::new( @@ -109,7 +115,9 @@ impl EqualityDeleteWriterConfig { } #[async_trait::async_trait] -impl IcebergWriterBuilder for EqualityDeleteFileWriterBuilder { +impl IcebergWriterBuilder + for EqualityDeleteFileWriterBuilder +{ type R = EqualityDeleteFileWriter; async fn build(self) -> Result { @@ -124,7 +132,11 @@ impl IcebergWr /// Writer used to write equality delete files. #[derive(Debug)] -pub struct EqualityDeleteFileWriter { +pub struct EqualityDeleteFileWriter< + B: FileWriterBuilder, + L: LocationGenerator, + F: FileNameGenerator, +> { inner_writer: Option>, projector: RecordBatchProjector, equality_ids: Vec, @@ -132,7 +144,9 @@ pub struct EqualityDeleteFileWriter IcebergWriter for EqualityDeleteFileWriter { +impl IcebergWriter + for EqualityDeleteFileWriter +{ async fn write(&mut self, batch: RecordBatch) -> Result<()> { let batch = self.projector.project_batch(batch)?; if let Some(writer) = self.inner_writer.as_mut() { @@ -154,12 +168,16 @@ impl IcebergWr .map(|mut res| { res.content(crate::spec::DataContentType::EqualityDeletes); res.equality_ids(Some(self.equality_ids.iter().copied().collect_vec())); - res.partition(self.partition_key - .as_ref() - .map_or(Struct::empty(), |pk| pk.data().clone())); - res.partition_spec_id(self.partition_key - .as_ref() - .map_or(DEFAULT_PARTITION_SPEC_ID, |pk| pk.spec().spec_id())); + res.partition( + self.partition_key + .as_ref() + .map_or(Struct::empty(), |pk| pk.data().clone()), + ); + res.partition_spec_id( + self.partition_key + .as_ref() + .map_or(DEFAULT_PARTITION_SPEC_ID, |pk| pk.spec().spec_id()), + ); res.build().expect("msg") }) .collect_vec()) diff --git a/crates/iceberg/src/writer/file_writer/parquet_writer.rs b/crates/iceberg/src/writer/file_writer/parquet_writer.rs index c4d1a6c27d..ee39084aef 100644 --- a/crates/iceberg/src/writer/file_writer/parquet_writer.rs +++ b/crates/iceberg/src/writer/file_writer/parquet_writer.rs @@ -60,10 +60,7 @@ pub struct ParquetWriterBuilder { impl ParquetWriterBuilder { /// Create a new `ParquetWriterBuilder` /// To construct the write result, the schema should contain the `PARQUET_FIELD_ID_META_KEY` metadata for each field. - pub fn new( - props: WriterProperties, - schema: SchemaRef, - ) -> Self { + pub fn new(props: WriterProperties, schema: SchemaRef) -> Self { Self::new_with_match_mode(props, schema, FieldMatchMode::Id) } diff --git a/crates/iceberg/src/writer/mod.rs b/crates/iceberg/src/writer/mod.rs index 8f17d50e27..374a39f5b7 100644 --- a/crates/iceberg/src/writer/mod.rs +++ b/crates/iceberg/src/writer/mod.rs @@ -225,6 +225,7 @@ pub mod base_writer; pub mod file_writer; +pub mod partitioning; use arrow_array::RecordBatch; diff --git a/crates/iceberg/src/writer/partitioning/clustered.rs b/crates/iceberg/src/writer/partitioning/clustered.rs new file mode 100644 index 0000000000..1ea0d24093 --- /dev/null +++ b/crates/iceberg/src/writer/partitioning/clustered.rs @@ -0,0 +1,251 @@ +// // Licensed to the Apache Software Foundation (ASF) under one +// // or more contributor license agreements. See the NOTICE file +// // distributed with this work for additional information +// // regarding copyright ownership. The ASF licenses this file +// // to you under the Apache License, Version 2.0 (the +// // "License"); you may not use this file except in compliance +// // with the License. You may obtain a copy of the License at +// // +// // http://www.apache.org/licenses/LICENSE-2.0 +// // +// // Unless required by applicable law or agreed to in writing, +// // software distributed under the License is distributed on an +// // "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// // KIND, either express or implied. See the License for the +// // specific language governing permissions and limitations +// // under the License. +// +// //! This module provides the `ClusteredWriter` implementation. +// +// use std::collections::HashSet; +// +// use arrow_array::RecordBatch; +// use async_trait::async_trait; +// +// use crate::spec::{PartitionKey, Struct}; +// use crate::writer::{DefaultInput, DefaultOutput, IcebergWriter, IcebergWriterBuilder}; +// use crate::writer::file_writer::location_generator::{FileNameGenerator, LocationGenerator}; +// use crate::{Error, ErrorKind, Result}; +// use crate::writer::partitioning::PartitioningWriter; +// +// /// A writer that writes data to a single partition at a time. +// /// +// /// When data for a new partition arrives, it closes the current writer +// /// and creates a new one for the new partition. +// /// +// /// Once a partition has been written to and closed, any further attempts +// /// to write to that partition will result in an error. +// pub struct ClusteredWriter +// { +// inner_builder: B, +// current_writer: Option, +// current_partition: Option, +// closed_partitions: HashSet, +// output: O, +// } +// +// impl ClusteredWriter { +// /// todo doc +// pub fn new( +// inner_builder: B +// ) -> Self { +// Self { +// inner_builder, +// current_writer: None, +// current_partition: None, +// closed_partitions: HashSet::new(), +// output: O::default(), +// } +// } +// +// /// Close the current writer if it exists +// async fn close_current_writer(&mut self) -> Result<()> { +// if let Some(mut writer) = self.current_writer.take() { +// self.output.extend(writer.close().await?); +// +// // Add the current partition to the set of closed partitions +// if let Some(current_partition) = self.current_partition.take() { +// self.closed_partitions.insert(current_partition); +// } +// } +// +// Ok(()) +// } +// +// async fn do_write(&mut self, input: I) -> Result<()> { +// if let Some(writer) = &mut self.current_writer { +// writer.write(input).await?; +// Ok(()) +// } else { +// Err(Error::new( +// ErrorKind::Unexpected, +// "Writer is not initialized!", +// )) +// } +// } +// } +// +// #[async_trait] +// impl PartitioningWriter for ClusteredWriter { +// async fn write(&mut self, partition_key: Option, input: I) -> Result<()> { +// if let Some(partition_key) = partition_key { +// let partition_value = partition_key.data(); +// +// // Check if this partition has been closed already +// if self.closed_partitions.contains(partition_value) { +// return Err(Error::new( +// ErrorKind::Unexpected, +// format!( +// "Cannot write to partition that was previously closed: {:?}", +// partition_key +// ), +// )); +// } +// +// // Check if we need to switch to a new partition +// let need_new_writer = match &self.current_partition { +// Some(current) => current != partition_value, +// None => true, +// }; +// +// if need_new_writer { +// self.close_current_writer().await?; +// +// // Create a new writer for the new partition +// self.current_writer = Some(self.inner_builder.clone().build().await?); +// self.current_partition = Some(partition_value.clone()); +// } +// } +// +// self.do_write(input).await +// } +// +// async fn close(&mut self) -> Result { +// self.close_current_writer().await?; +// +// // Return all collected data files +// Ok(std::mem::take(&mut self.output)) +// } +// } +// +// #[cfg(test)] +// mod tests { +// use std::sync::Arc; +// +// use arrow_array::{Int32Array, StringArray}; +// use arrow_schema::{DataType, Field, Schema}; +// use parquet::file::properties::WriterProperties; +// use tempfile::TempDir; +// +// use super::*; +// use crate::arrow::FieldMatchMode; +// use crate::io::FileIOBuilder; +// use crate::spec::{DataFileFormat, Literal, NestedField, PrimitiveType, Struct, Type}; +// use crate::writer::base_writer::data_file_writer::DataFileWriterBuilder; +// use crate::writer::file_writer::ParquetWriterBuilder; +// use crate::writer::file_writer::location_generator::DefaultFileNameGenerator; +// use crate::writer::file_writer::location_generator::test::MockLocationGenerator; +// +// #[tokio::test] +// async fn test_clustered_writer_basic() -> Result<()> { +// let temp_dir = TempDir::new()?; +// let file_io = FileIOBuilder::new_fs_io().build()?; +// let location_gen = +// MockLocationGenerator::new(temp_dir.path().to_str().unwrap().to_string()); +// let file_name_gen = +// DefaultFileNameGenerator::new("test".to_string(), None, DataFileFormat::Parquet); +// +// // Create schema +// let schema = Arc::new( +// crate::spec::Schema::builder() +// .with_schema_id(1) +// .with_fields(vec![ +// NestedField::required(1, "id", Type::Primitive(PrimitiveType::Int)).into(), +// NestedField::required(2, "name", Type::Primitive(PrimitiveType::String)).into(), +// NestedField::required(3, "category", Type::Primitive(PrimitiveType::String)) +// .into(), +// ]) +// .build()?, +// ); +// +// // Create writer builder +// let parquet_writer_builder = ParquetWriterBuilder::new_with_match_mode( +// WriterProperties::builder().build(), +// schema.clone(), +// FieldMatchMode::Name, +// ); +// +// // Create data file writer builder +// let data_file_writer_builder = DataFileWriterBuilder::new(parquet_writer_builder, None, 0); +// +// // Create clustered writer +// let mut writer = ClusteredWriter::new( +// data_file_writer_builder, +// location_gen, +// file_name_gen, +// file_io.clone(), +// 1024 * 1024, // 1MB +// ); +// +// // Create test data for partition A +// let arrow_schema = Schema::new(vec![ +// Field::new("id", DataType::Int32, false), +// Field::new("name", DataType::Utf8, false), +// Field::new("category", DataType::Utf8, false), +// ]); +// +// let batch_a1 = RecordBatch::try_new(Arc::new(arrow_schema.clone()), vec![ +// Arc::new(Int32Array::from(vec![1, 2, 3])), +// Arc::new(StringArray::from(vec!["Alice", "Bob", "Charlie"])), +// Arc::new(StringArray::from(vec!["A", "A", "A"])), +// ])?; +// +// let batch_a2 = RecordBatch::try_new(Arc::new(arrow_schema.clone()), vec![ +// Arc::new(Int32Array::from(vec![4, 5])), +// Arc::new(StringArray::from(vec!["Dave", "Eve"])), +// Arc::new(StringArray::from(vec!["A", "A"])), +// ])?; +// +// // Create test data for partition B +// let batch_b = RecordBatch::try_new(Arc::new(arrow_schema.clone()), vec![ +// Arc::new(Int32Array::from(vec![6, 7, 8])), +// Arc::new(StringArray::from(vec!["Frank", "Grace", "Heidi"])), +// Arc::new(StringArray::from(vec!["B", "B", "B"])), +// ])?; +// +// // Create partition keys +// let partition_a = PartitionKey::new( +// crate::spec::PartitionSpec::unpartition_spec().into(), +// schema.clone(), +// Struct::from_iter([Some(Literal::string("A"))]), +// ); +// +// let partition_b = PartitionKey::new( +// crate::spec::PartitionSpec::unpartition_spec().into(), +// schema.clone(), +// Struct::from_iter([Some(Literal::string("B"))]), +// ); +// +// // Write data to partition A +// writer.write(partition_a.clone(), batch_a1.clone()).await?; +// writer.write(partition_a.clone(), batch_a2.clone()).await?; +// +// // Write data to partition B +// writer.write(partition_b.clone(), batch_b.clone()).await?; +// +// // Try to write to partition A again (should fail) +// let result = writer.write(partition_a.clone(), batch_a1.clone()).await; +// assert!( +// result.is_err(), +// "Expected error when writing to closed partition" +// ); +// +// // Close writer and get data files +// let data_files = writer.close().await?; +// +// // Verify files were created +// assert_eq!(data_files.len(), 2, "Expected two data files to be created"); +// +// Ok(()) +// } +// } diff --git a/crates/iceberg/src/writer/partitioning/mod.rs b/crates/iceberg/src/writer/partitioning/mod.rs new file mode 100644 index 0000000000..1005d37a72 --- /dev/null +++ b/crates/iceberg/src/writer/partitioning/mod.rs @@ -0,0 +1,40 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +mod clustered; + +use crate::Result; +use crate::spec::PartitionKey; +use crate::writer::{DefaultInput, DefaultOutput}; + +#[async_trait::async_trait] +pub trait PartitioningWriterBuilder: + Send + Clone + 'static +{ + /// todo doc + type R: PartitioningWriter; + /// todo doc + async fn build(self) -> Result; +} + +#[async_trait::async_trait] +pub trait PartitioningWriter: Send + 'static { + /// todo doc + async fn write(&mut self, partition_key: Option, input: I) -> Result<()>; + /// todo doc + async fn close(&mut self) -> Result; +} diff --git a/crates/integrations/datafusion/src/physical_plan/write.rs b/crates/integrations/datafusion/src/physical_plan/write.rs index db8de2a92d..f49fbfc334 100644 --- a/crates/integrations/datafusion/src/physical_plan/write.rs +++ b/crates/integrations/datafusion/src/physical_plan/write.rs @@ -38,7 +38,8 @@ use futures::StreamExt; use iceberg::arrow::{FieldMatchMode, schema_to_arrow_schema}; use iceberg::spec::{ DataFileFormat, PROPERTY_DEFAULT_FILE_FORMAT, PROPERTY_DEFAULT_FILE_FORMAT_DEFAULT, - PROPERTY_WRITE_TARGET_FILE_SIZE_BYTES, PROPERTY_WRITE_TARGET_FILE_SIZE_BYTES_DEFAULT, serialize_data_file_to_json, + PROPERTY_WRITE_TARGET_FILE_SIZE_BYTES, PROPERTY_WRITE_TARGET_FILE_SIZE_BYTES_DEFAULT, + serialize_data_file_to_json, }; use iceberg::table::Table; use iceberg::writer::base_writer::data_file_writer::DataFileWriterBuilder; From 91242bd98fcb4577f29e7f1d4ae9bc54436bce27 Mon Sep 17 00:00:00 2001 From: Shawn Chang Date: Mon, 22 Sep 2025 22:01:44 -0700 Subject: [PATCH 03/13] some cleanup --- .../writer/base_writer/data_file_writer.rs | 48 ++++++++------- .../src/writer/file_writer/rolling_writer.rs | 58 ------------------- crates/iceberg/src/writer/partitioning/mod.rs | 10 ---- 3 files changed, 27 insertions(+), 89 deletions(-) diff --git a/crates/iceberg/src/writer/base_writer/data_file_writer.rs b/crates/iceberg/src/writer/base_writer/data_file_writer.rs index d4f88486e8..ebc0f5f610 100644 --- a/crates/iceberg/src/writer/base_writer/data_file_writer.rs +++ b/crates/iceberg/src/writer/base_writer/data_file_writer.rs @@ -20,12 +20,12 @@ use arrow_array::RecordBatch; use itertools::Itertools; -use crate::Result; use crate::spec::{DEFAULT_PARTITION_SPEC_ID, DataContentType, DataFile, PartitionKey, Struct}; use crate::writer::file_writer::FileWriterBuilder; use crate::writer::file_writer::location_generator::{FileNameGenerator, LocationGenerator}; use crate::writer::file_writer::rolling_writer::RollingFileWriter; use crate::writer::{CurrentFileStatus, IcebergWriter, IcebergWriterBuilder}; +use crate::{Error, ErrorKind, Result}; /// Builder for `DataFileWriter`. #[derive(Clone, Debug)] @@ -83,26 +83,32 @@ impl IcebergWr } async fn close(&mut self) -> Result> { - let writer = self.inner_writer.take().unwrap(); - Ok(writer - .close() - .await? - .into_iter() - .map(|mut res| { - res.content(DataContentType::Data); - res.partition( - self.partition_key - .as_ref() - .map_or(Struct::empty(), |pk| pk.data().clone()), - ); - res.partition_spec_id( - self.partition_key - .as_ref() - .map_or(DEFAULT_PARTITION_SPEC_ID, |pk| pk.spec().spec_id()), - ); - res.build().expect("Guaranteed to be valid") - }) - .collect_vec()) + if let Some(writer) = self.inner_writer.take() { + Ok(writer + .close() + .await? + .into_iter() + .map(|mut res| { + res.content(DataContentType::Data); + res.partition( + self.partition_key + .as_ref() + .map_or(Struct::empty(), |pk| pk.data().clone()), + ); + res.partition_spec_id( + self.partition_key + .as_ref() + .map_or(DEFAULT_PARTITION_SPEC_ID, |pk| pk.spec().spec_id()), + ); + res.build().expect("Guaranteed to be valid") + }) + .collect_vec()) + } else { + Err(Error::new( + ErrorKind::Unexpected, + "Data file writer has been closed.", + )) + } } } diff --git a/crates/iceberg/src/writer/file_writer/rolling_writer.rs b/crates/iceberg/src/writer/file_writer/rolling_writer.rs index 361b748c03..647552f24b 100644 --- a/crates/iceberg/src/writer/file_writer/rolling_writer.rs +++ b/crates/iceberg/src/writer/file_writer/rolling_writer.rs @@ -25,64 +25,6 @@ use crate::writer::CurrentFileStatus; use crate::writer::file_writer::location_generator::{FileNameGenerator, LocationGenerator}; use crate::writer::file_writer::{FileWriter, FileWriterBuilder}; use crate::{Error, ErrorKind, Result}; -// -// /// Builder for creating a `RollingFileWriter` that rolls over to a new file -// /// when the data size exceeds a target threshold. -// #[derive(Clone)] -// pub struct RollingFileWriterBuilder< -// B: FileWriterBuilder, -// L: LocationGenerator, -// F: FileNameGenerator, -// > { -// inner_builder: B, -// target_file_size: usize, -// file_io: FileIO, -// location_generator: L, -// file_name_generator: F, -// } -// -// impl -// RollingFileWriterBuilder -// { -// /// Creates a new `RollingFileWriterBuilder` with the specified inner builder and target size. -// /// -// /// # Arguments -// /// -// /// * `inner_builder` - The builder for the underlying file writer -// /// * `target_file_size` - The target size in bytes before rolling over to a new file -// /// -// /// NOTE: The `target_file_size` does not exactly reflect the final size on physical storage. -// /// This is because the input size is based on the Arrow in-memory format and cannot precisely control rollover behavior. -// /// The actual file size on disk is expected to be slightly larger than `target_file_size`. -// pub fn new( -// inner_builder: B, -// target_file_size: usize, -// file_io: FileIO, -// location_generator: L, -// file_name_generator: F, -// ) -> Self { -// Self { -// inner_builder, -// target_file_size, -// file_io, -// location_generator, -// file_name_generator, -// } -// } -// } - -// impl FileWriterBuilder for RollingFileWriterBuilder { -// type R = RollingFileWriter; -// -// async fn build(self) -> Result { -// Ok(RollingFileWriter { -// inner: None, -// inner_builder: self.inner_builder, -// target_file_size: self.target_file_size, -// data_file_builders: vec![], -// }) -// } -// } /// A writer that automatically rolls over to a new file when the data size /// exceeds a target threshold. diff --git a/crates/iceberg/src/writer/partitioning/mod.rs b/crates/iceberg/src/writer/partitioning/mod.rs index 1005d37a72..6639a511d3 100644 --- a/crates/iceberg/src/writer/partitioning/mod.rs +++ b/crates/iceberg/src/writer/partitioning/mod.rs @@ -21,16 +21,6 @@ use crate::Result; use crate::spec::PartitionKey; use crate::writer::{DefaultInput, DefaultOutput}; -#[async_trait::async_trait] -pub trait PartitioningWriterBuilder: - Send + Clone + 'static -{ - /// todo doc - type R: PartitioningWriter; - /// todo doc - async fn build(self) -> Result; -} - #[async_trait::async_trait] pub trait PartitioningWriter: Send + 'static { /// todo doc From 4d6e48cc28e3a67aae716a2dfb0efec03395aa9d Mon Sep 17 00:00:00 2001 From: Shawn Chang Date: Wed, 24 Sep 2025 16:05:00 -0700 Subject: [PATCH 04/13] fix compile issues --- crates/iceberg/src/spec/partition.rs | 15 +- .../writer/base_writer/data_file_writer.rs | 34 ++- .../base_writer/equality_delete_writer.rs | 60 ++--- .../src/writer/file_writer/parquet_writer.rs | 185 ++++++------- .../src/writer/file_writer/rolling_writer.rs | 57 ++-- crates/iceberg/src/writer/mod.rs | 27 +- .../src/writer/partitioning/clustered.rs | 251 ------------------ crates/iceberg/src/writer/partitioning/mod.rs | 30 --- .../datafusion/src/physical_plan/write.rs | 2 +- 9 files changed, 192 insertions(+), 469 deletions(-) delete mode 100644 crates/iceberg/src/writer/partitioning/clustered.rs delete mode 100644 crates/iceberg/src/writer/partitioning/mod.rs diff --git a/crates/iceberg/src/spec/partition.rs b/crates/iceberg/src/spec/partition.rs index 167cf3cf88..fe8767266e 100644 --- a/crates/iceberg/src/spec/partition.rs +++ b/crates/iceberg/src/spec/partition.rs @@ -194,15 +194,6 @@ impl PartitionKey { Self { spec, schema, data } } - /// todo do we need this? - pub fn unpartition_key() -> Self { - Self { - spec: PartitionSpec::unpartition_spec(), - schema: Arc::new(Schema::builder().build().unwrap()), - data: Struct::empty(), - } - } - /// Generates a partition path based on the partition values. pub fn to_path(&self) -> String { self.spec.partition_to_path(&self.data, self.schema.clone()) @@ -217,17 +208,17 @@ impl PartitionKey { } } - /// todo doc + /// Returns the associated [`PartitionSpec`]. pub fn spec(&self) -> &PartitionSpec { &self.spec } - /// todo doc + /// Returns the associated [`SchemaRef`]. pub fn schema(&self) -> &SchemaRef { &self.schema } - /// todo doc + /// Returns the associated [`Struct`]. pub fn data(&self) -> &Struct { &self.data } diff --git a/crates/iceberg/src/writer/base_writer/data_file_writer.rs b/crates/iceberg/src/writer/base_writer/data_file_writer.rs index ebc0f5f610..b56655e06c 100644 --- a/crates/iceberg/src/writer/base_writer/data_file_writer.rs +++ b/crates/iceberg/src/writer/base_writer/data_file_writer.rs @@ -57,7 +57,7 @@ impl IcebergWr async fn build(self) -> Result { Ok(DataFileWriter { - inner_writer: Some(self.inner_writer), // todo revisit this, probably still need a builder for rolling writer + inner_writer: Some(self.inner_writer), partition_key: self.partition_key, }) } @@ -143,13 +143,16 @@ mod test { use crate::Result; use crate::io::FileIOBuilder; use crate::spec::{ - DataContentType, DataFileFormat, Literal, NestedField, PrimitiveType, Schema, Struct, Type, + DataContentType, DataFileFormat, Literal, NestedField, + PROPERTY_WRITE_TARGET_FILE_SIZE_BYTES_DEFAULT, PartitionKey, PartitionSpec, PrimitiveType, + Schema, Struct, Type, }; use crate::writer::base_writer::data_file_writer::DataFileWriterBuilder; use crate::writer::file_writer::ParquetWriterBuilder; use crate::writer::file_writer::location_generator::{ DefaultFileNameGenerator, DefaultLocationGenerator, }; + use crate::writer::file_writer::rolling_writer::RollingFileWriter; use crate::writer::{IcebergWriter, IcebergWriterBuilder, RecordBatch}; #[tokio::test] @@ -170,16 +173,16 @@ mod test { ]) .build()?; - let pw = ParquetWriterBuilder::new( - WriterProperties::builder().build(), - Arc::new(schema), - None, + let pw = ParquetWriterBuilder::new(WriterProperties::builder().build(), Arc::new(schema)); + + let rolling_file_writer = RollingFileWriter::new_with_default_file_size( + pw, file_io.clone(), location_gen, file_name_gen, ); - let mut data_file_writer = DataFileWriterBuilder::new(pw, None, 0) + let mut data_file_writer = DataFileWriterBuilder::new(rolling_file_writer, None) .build() .await .unwrap(); @@ -246,20 +249,27 @@ mod test { NestedField::required(6, "name", Type::Primitive(PrimitiveType::String)).into(), ]) .build()?; + let schema_ref = Arc::new(schema); let partition_value = Struct::from_iter([Some(Literal::int(1))]); + let partition_key = PartitionKey::new( + PartitionSpec::builder(schema_ref.clone()).build()?, + schema_ref.clone(), + partition_value, + ); - let parquet_writer_builder = ParquetWriterBuilder::new( - WriterProperties::builder().build(), - Arc::new(schema.clone()), - None, + let parquet_writer_builder = + ParquetWriterBuilder::new(WriterProperties::builder().build(), schema_ref.clone()); + + let rolling_file_writer = RollingFileWriter::new_with_default_file_size( + parquet_writer_builder, file_io.clone(), location_gen, file_name_gen, ); let mut data_file_writer = - DataFileWriterBuilder::new(parquet_writer_builder, Some(partition_value.clone()), 0) + DataFileWriterBuilder::new(rolling_file_writer, Some(partition_key)) .build() .await?; diff --git a/crates/iceberg/src/writer/base_writer/equality_delete_writer.rs b/crates/iceberg/src/writer/base_writer/equality_delete_writer.rs index 590e5067bc..3b0d43e2c1 100644 --- a/crates/iceberg/src/writer/base_writer/equality_delete_writer.rs +++ b/crates/iceberg/src/writer/base_writer/equality_delete_writer.rs @@ -220,6 +220,7 @@ mod test { use crate::writer::file_writer::location_generator::{ DefaultFileNameGenerator, DefaultLocationGenerator, }; + use crate::writer::file_writer::rolling_writer::RollingFileWriter; use crate::writer::{IcebergWriter, IcebergWriterBuilder}; async fn check_parquet_data_file_with_equality_delete_write( @@ -416,23 +417,20 @@ mod test { let equality_ids = vec![0_i32, 8]; let equality_config = - EqualityDeleteWriterConfig::new(equality_ids, Arc::new(schema), None, 0).unwrap(); + EqualityDeleteWriterConfig::new(equality_ids, Arc::new(schema), None).unwrap(); let delete_schema = arrow_schema_to_schema(equality_config.projected_arrow_schema_ref()).unwrap(); let projector = equality_config.projector.clone(); // prepare writer - let pb = ParquetWriterBuilder::new( - WriterProperties::builder().build(), - Arc::new(delete_schema), - None, - file_io.clone(), - location_gen, - file_name_gen, - ); - let mut equality_delete_writer = EqualityDeleteFileWriterBuilder::new(pb, equality_config) - .build() - .await?; + let pb = + ParquetWriterBuilder::new(WriterProperties::builder().build(), Arc::new(delete_schema)); + let rolling_writer = + RollingFileWriter::new_with_default_file_size(pb, file_io, location_gen, file_name_gen); + let mut equality_delete_writer = + EqualityDeleteFileWriterBuilder::new(rolling_writer, equality_config) + .build() + .await?; // write equality_delete_writer.write(to_write.clone()).await?; @@ -518,19 +516,19 @@ mod test { .unwrap(), ); // Float and Double are not allowed to be used for equality delete - assert!(EqualityDeleteWriterConfig::new(vec![0], schema.clone(), None, 0).is_err()); - assert!(EqualityDeleteWriterConfig::new(vec![1], schema.clone(), None, 0).is_err()); + assert!(EqualityDeleteWriterConfig::new(vec![0], schema.clone(), None).is_err()); + assert!(EqualityDeleteWriterConfig::new(vec![1], schema.clone(), None).is_err()); // Struct is not allowed to be used for equality delete - assert!(EqualityDeleteWriterConfig::new(vec![3], schema.clone(), None, 0).is_err()); + assert!(EqualityDeleteWriterConfig::new(vec![3], schema.clone(), None).is_err()); // Nested field of struct is allowed to be used for equality delete - assert!(EqualityDeleteWriterConfig::new(vec![4], schema.clone(), None, 0).is_ok()); + assert!(EqualityDeleteWriterConfig::new(vec![4], schema.clone(), None).is_ok()); // Nested field of map is not allowed to be used for equality delete - assert!(EqualityDeleteWriterConfig::new(vec![7], schema.clone(), None, 0).is_err()); - assert!(EqualityDeleteWriterConfig::new(vec![8], schema.clone(), None, 0).is_err()); - assert!(EqualityDeleteWriterConfig::new(vec![9], schema.clone(), None, 0).is_err()); + assert!(EqualityDeleteWriterConfig::new(vec![7], schema.clone(), None).is_err()); + assert!(EqualityDeleteWriterConfig::new(vec![8], schema.clone(), None).is_err()); + assert!(EqualityDeleteWriterConfig::new(vec![9], schema.clone(), None).is_err()); // Nested field of list is not allowed to be used for equality delete - assert!(EqualityDeleteWriterConfig::new(vec![10], schema.clone(), None, 0).is_err()); - assert!(EqualityDeleteWriterConfig::new(vec![11], schema.clone(), None, 0).is_err()); + assert!(EqualityDeleteWriterConfig::new(vec![10], schema.clone(), None).is_err()); + assert!(EqualityDeleteWriterConfig::new(vec![11], schema.clone(), None).is_err()); Ok(()) } @@ -584,22 +582,22 @@ mod test { .unwrap(), ); let equality_ids = vec![0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13]; - let config = - EqualityDeleteWriterConfig::new(equality_ids, schema.clone(), None, 0).unwrap(); + let config = EqualityDeleteWriterConfig::new(equality_ids, schema.clone(), None).unwrap(); let delete_arrow_schema = config.projected_arrow_schema_ref().clone(); let delete_schema = arrow_schema_to_schema(&delete_arrow_schema).unwrap(); - let pb = ParquetWriterBuilder::new( - WriterProperties::builder().build(), - Arc::new(delete_schema), - None, + let pb = + ParquetWriterBuilder::new(WriterProperties::builder().build(), Arc::new(delete_schema)); + let rolling_writer = RollingFileWriter::new_with_default_file_size( + pb, file_io.clone(), location_gen, file_name_gen, ); - let mut equality_delete_writer = EqualityDeleteFileWriterBuilder::new(pb, config) - .build() - .await?; + let mut equality_delete_writer = + EqualityDeleteFileWriterBuilder::new(rolling_writer, config) + .build() + .await?; // prepare data let col0 = Arc::new(BooleanArray::from(vec![ @@ -782,7 +780,7 @@ mod test { let to_write = RecordBatch::try_new(arrow_schema.clone(), columns).unwrap(); let equality_ids = vec![0_i32, 2, 5]; let equality_config = - EqualityDeleteWriterConfig::new(equality_ids, Arc::new(schema), None, 0).unwrap(); + EqualityDeleteWriterConfig::new(equality_ids, Arc::new(schema), None).unwrap(); let projector = equality_config.projector.clone(); // check diff --git a/crates/iceberg/src/writer/file_writer/parquet_writer.rs b/crates/iceberg/src/writer/file_writer/parquet_writer.rs index ee39084aef..2d7f7ec090 100644 --- a/crates/iceberg/src/writer/file_writer/parquet_writer.rs +++ b/crates/iceberg/src/writer/file_writer/parquet_writer.rs @@ -42,8 +42,8 @@ use crate::arrow::{ use crate::io::{FileIO, FileWrite, OutputFile}; use crate::spec::{ DataContentType, DataFileBuilder, DataFileFormat, Datum, ListType, Literal, MapType, - NestedFieldRef, PartitionKey, PartitionSpec, PrimitiveType, Schema, SchemaRef, SchemaVisitor, - Struct, StructType, TableMetadata, Type, visit_schema, + NestedFieldRef, PartitionSpec, PrimitiveType, Schema, SchemaRef, SchemaVisitor, Struct, + StructType, TableMetadata, Type, visit_schema, }; use crate::transform::create_transform_function; use crate::writer::{CurrentFileStatus, DataFile}; @@ -667,7 +667,7 @@ mod tests { use crate::io::FileIOBuilder; use crate::spec::{PrimitiveLiteral, Struct, *}; use crate::writer::file_writer::location_generator::{ - DefaultFileNameGenerator, DefaultLocationGenerator, + DefaultFileNameGenerator, DefaultLocationGenerator, FileNameGenerator, LocationGenerator, }; use crate::writer::tests::check_parquet_data_file; @@ -835,11 +835,13 @@ mod tests { // prepare data let schema = { - let fields = vec![ - arrow_schema::Field::new("col", arrow_schema::DataType::Int64, true).with_metadata( - HashMap::from([(PARQUET_FIELD_ID_META_KEY.to_string(), "0".to_string())]), - ), - ]; + let fields = + vec![ + Field::new("col", DataType::Int64, true).with_metadata(HashMap::from([( + PARQUET_FIELD_ID_META_KEY.to_string(), + "0".to_string(), + )])), + ]; Arc::new(arrow_schema::Schema::new(fields)) }; let col = Arc::new(Int64Array::from_iter_values(0..1024)) as ArrayRef; @@ -847,18 +849,18 @@ mod tests { let to_write = RecordBatch::try_new(schema.clone(), vec![col]).unwrap(); let to_write_null = RecordBatch::try_new(schema.clone(), vec![null_col]).unwrap(); + let output_file = file_io.new_output( + location_gen.generate_location(None, &file_name_gen.generate_file_name()), + )?; + // write data let mut pw = ParquetWriterBuilder::new( WriterProperties::builder() .set_max_row_group_size(128) .build(), Arc::new(to_write.schema().as_ref().try_into().unwrap()), - None, - file_io.clone(), - location_gen, - file_name_gen, ) - .build() + .build(output_file) .await?; pw.write(&to_write).await?; pw.write(&to_write_null).await?; @@ -869,7 +871,7 @@ mod tests { .next() .unwrap() // Put dummy field for build successfully. - .content(crate::spec::DataContentType::Data) + .content(DataContentType::Data) .partition(Struct::empty()) .partition_spec_id(0) .build() @@ -970,7 +972,7 @@ mod tests { None, )); let col5 = Arc::new({ - let mut map_array_builder = arrow_array::builder::MapBuilder::new( + let mut map_array_builder = MapBuilder::new( None, arrow_array::builder::StringBuilder::new(), arrow_array::builder::ListBuilder::new(arrow_array::builder::PrimitiveBuilder::< @@ -1047,18 +1049,15 @@ mod tests { col0, col1, col2, col3, col4, col5, ]) .unwrap(); + let output_file = file_io.new_output( + location_gen.generate_location(None, &file_name_gen.generate_file_name()), + )?; // write data - let mut pw = ParquetWriterBuilder::new( - WriterProperties::builder().build(), - Arc::new(schema), - None, - file_io.clone(), - location_gen, - file_name_gen, - ) - .build() - .await?; + let mut pw = + ParquetWriterBuilder::new(WriterProperties::builder().build(), Arc::new(schema)) + .build(output_file) + .await?; pw.write(&to_write).await?; let res = pw.close().await?; assert_eq!(res.len(), 1); @@ -1125,7 +1124,7 @@ mod tests { async fn test_all_type_for_write() -> Result<()> { let temp_dir = TempDir::new().unwrap(); let file_io = FileIOBuilder::new_fs_io().build().unwrap(); - let loccation_gen = DefaultLocationGenerator::with_data_location( + let location_gen = DefaultLocationGenerator::with_data_location( temp_dir.path().to_str().unwrap().to_string(), ); let file_name_gen = @@ -1240,18 +1239,15 @@ mod tests { col14, col15, col16, ]) .unwrap(); + let output_file = file_io.new_output( + location_gen.generate_location(None, &file_name_gen.generate_file_name()), + )?; // write data - let mut pw = ParquetWriterBuilder::new( - WriterProperties::builder().build(), - Arc::new(schema), - None, - file_io.clone(), - loccation_gen, - file_name_gen, - ) - .build() - .await?; + let mut pw = + ParquetWriterBuilder::new(WriterProperties::builder().build(), Arc::new(schema)) + .build(output_file) + .await?; pw.write(&to_write).await?; let res = pw.close().await?; assert_eq!(res.len(), 1); @@ -1369,7 +1365,7 @@ mod tests { async fn test_decimal_bound() -> Result<()> { let temp_dir = TempDir::new().unwrap(); let file_io = FileIOBuilder::new_fs_io().build().unwrap(); - let loccation_gen = DefaultLocationGenerator::with_data_location( + let location_gen = DefaultLocationGenerator::with_data_location( temp_dir.path().to_str().unwrap().to_string(), ); let file_name_gen = @@ -1393,16 +1389,12 @@ mod tests { .unwrap(), ); let arrow_schema: ArrowSchemaRef = Arc::new(schema_to_arrow_schema(&schema).unwrap()); - let mut pw = ParquetWriterBuilder::new( - WriterProperties::builder().build(), - schema.clone(), - None, - file_io.clone(), - loccation_gen.clone(), - file_name_gen.clone(), - ) - .build() - .await?; + let output_file = file_io.new_output( + location_gen.generate_location(None, &file_name_gen.generate_file_name()), + )?; + let mut pw = ParquetWriterBuilder::new(WriterProperties::builder().build(), schema.clone()) + .build(output_file) + .await?; let col0 = Arc::new( Decimal128Array::from(vec![Some(22000000000), Some(11000000000)]) .with_data_type(DataType::Decimal128(28, 10)), @@ -1449,16 +1441,12 @@ mod tests { .unwrap(), ); let arrow_schema: ArrowSchemaRef = Arc::new(schema_to_arrow_schema(&schema).unwrap()); - let mut pw = ParquetWriterBuilder::new( - WriterProperties::builder().build(), - schema.clone(), - None, - file_io.clone(), - loccation_gen.clone(), - file_name_gen.clone(), - ) - .build() - .await?; + let output_file = file_io.new_output( + location_gen.generate_location(None, &file_name_gen.generate_file_name()), + )?; + let mut pw = ParquetWriterBuilder::new(WriterProperties::builder().build(), schema.clone()) + .build(output_file) + .await?; let col0 = Arc::new( Decimal128Array::from(vec![Some(-22000000000), Some(-11000000000)]) .with_data_type(DataType::Decimal128(28, 10)), @@ -1508,16 +1496,12 @@ mod tests { .unwrap(), ); let arrow_schema: ArrowSchemaRef = Arc::new(schema_to_arrow_schema(&schema).unwrap()); - let mut pw = ParquetWriterBuilder::new( - WriterProperties::builder().build(), - schema, - None, - file_io.clone(), - loccation_gen, - file_name_gen, - ) - .build() - .await?; + let output_file = file_io.new_output( + location_gen.generate_location(None, &file_name_gen.generate_file_name()), + )?; + let mut pw = ParquetWriterBuilder::new(WriterProperties::builder().build(), schema) + .build(output_file) + .await?; let col0 = Arc::new( Decimal128Array::from(vec![ Some(decimal_max.mantissa()), @@ -1640,35 +1624,33 @@ mod tests { }; let col = Arc::new(Int64Array::from_iter_values(0..1024)) as ArrayRef; let to_write = RecordBatch::try_new(schema.clone(), vec![col]).unwrap(); + let output_file = file_io.new_output( + location_gen.generate_location(None, &file_name_gen.generate_file_name()), + )?; let mut pw = ParquetWriterBuilder::new( WriterProperties::builder().build(), Arc::new(to_write.schema().as_ref().try_into().unwrap()), - None, - file_io.clone(), - location_gen.clone(), - file_name_gen, ) - .build() + .build(output_file) .await?; pw.write(&to_write).await?; - let file_path = pw.out_file.location().to_string(); + let file_path = output_file.location().to_string(); pw.close().await.unwrap(); assert!(file_io.exists(file_path).await.unwrap()); // Test that file will not create if no data to write let file_name_gen = DefaultFileNameGenerator::new("test_empty".to_string(), None, DataFileFormat::Parquet); + let output_file = file_io.new_output( + location_gen.generate_location(None, &file_name_gen.generate_file_name()), + )?; let pw = ParquetWriterBuilder::new( WriterProperties::builder().build(), Arc::new(to_write.schema().as_ref().try_into().unwrap()), - None, - file_io.clone(), - location_gen, - file_name_gen, ) - .build() + .build(output_file) .await?; - let file_path = pw.out_file.location().to_string(); + let file_path = output_file.location().to_string(); pw.close().await.unwrap(); assert!(!file_io.exists(file_path).await.unwrap()); @@ -1710,17 +1692,16 @@ mod tests { let to_write = RecordBatch::try_new(arrow_schema.clone(), vec![float_32_col, float_64_col]).unwrap(); + let output_file = file_io.new_output( + location_gen.generate_location(None, &file_name_gen.generate_file_name()), + )?; // write data let mut pw = ParquetWriterBuilder::new( WriterProperties::builder().build(), Arc::new(to_write.schema().as_ref().try_into().unwrap()), - None, - file_io.clone(), - location_gen, - file_name_gen, ) - .build() + .build(output_file) .await?; pw.write(&to_write).await?; @@ -1851,17 +1832,16 @@ mod tests { struct_nested_float_field_col, ]) .unwrap(); + let output_file = file_io.new_output( + location_gen.generate_location(None, &file_name_gen.generate_file_name()), + )?; // write data let mut pw = ParquetWriterBuilder::new( WriterProperties::builder().build(), Arc::new(to_write.schema().as_ref().try_into().unwrap()), - None, - file_io.clone(), - location_gen, - file_name_gen, ) - .build() + .build(output_file) .await?; pw.write(&to_write).await?; @@ -2011,6 +1991,9 @@ mod tests { // large_list_float_field_col, ]) .expect("Could not form record batch"); + let output_file = file_io.new_output( + location_gen.generate_location(None, &file_name_gen.generate_file_name()), + )?; // write data let mut pw = ParquetWriterBuilder::new( @@ -2022,12 +2005,8 @@ mod tests { .try_into() .expect("Could not convert iceberg schema"), ), - None, - file_io.clone(), - location_gen, - file_name_gen, ) - .build() + .build(output_file) .await?; pw.write(&to_write).await?; @@ -2193,6 +2172,9 @@ mod tests { struct_list_float_field_col, ]) .expect("Could not form record batch"); + let output_file = file_io.new_output( + location_gen.generate_location(None, &file_name_gen.generate_file_name()), + )?; // write data let mut pw = ParquetWriterBuilder::new( @@ -2204,12 +2186,8 @@ mod tests { .try_into() .expect("Could not convert iceberg schema"), ), - None, - file_io.clone(), - location_gen, - file_name_gen, ) - .build() + .build(output_file) .await?; pw.write(&to_write).await?; @@ -2274,6 +2252,9 @@ mod tests { ); let file_name_gen = DefaultFileNameGenerator::new("test".to_string(), None, DataFileFormat::Parquet); + let output_file = file_io + .new_output(location_gen.generate_location(None, &file_name_gen.generate_file_name())) + .unwrap(); // write data let pw = ParquetWriterBuilder::new( @@ -2289,12 +2270,8 @@ mod tests { .build() .expect("Failed to create schema"), ), - None, - file_io.clone(), - location_gen, - file_name_gen, ) - .build() + .build(output_file) .await .unwrap(); diff --git a/crates/iceberg/src/writer/file_writer/rolling_writer.rs b/crates/iceberg/src/writer/file_writer/rolling_writer.rs index 647552f24b..c05ffd69e9 100644 --- a/crates/iceberg/src/writer/file_writer/rolling_writer.rs +++ b/crates/iceberg/src/writer/file_writer/rolling_writer.rs @@ -20,7 +20,7 @@ use std::fmt::{Debug, Formatter}; use arrow_array::RecordBatch; use crate::io::{FileIO, OutputFile}; -use crate::spec::{DataFileBuilder, PartitionKey}; +use crate::spec::{DataFileBuilder, PROPERTY_WRITE_TARGET_FILE_SIZE_BYTES_DEFAULT, PartitionKey}; use crate::writer::CurrentFileStatus; use crate::writer::file_writer::location_generator::{FileNameGenerator, LocationGenerator}; use crate::writer::file_writer::{FileWriter, FileWriterBuilder}; @@ -89,6 +89,24 @@ impl RollingFi } } + /// todo doc + pub fn new_with_default_file_size( + inner_builder: B, + file_io: FileIO, + location_generator: L, + file_name_generator: F, + ) -> Self { + Self { + inner: None, + inner_builder, + target_file_size: PROPERTY_WRITE_TARGET_FILE_SIZE_BYTES_DEFAULT, + data_file_builders: vec![], + file_io, + location_generator, + file_name_generator, + } + } + /// Determines if the writer should roll over to a new file. /// /// # Returns @@ -236,22 +254,19 @@ mod tests { let schema = make_test_schema()?; // Create writer builders - let parquet_writer_builder = ParquetWriterBuilder::new( - WriterProperties::builder().build(), - Arc::new(schema), - None, - file_io.clone(), - location_gen, - file_name_gen, - ); + let parquet_writer_builder = + ParquetWriterBuilder::new(WriterProperties::builder().build(), Arc::new(schema)); // Set a large target size so no rolling occurs - let rolling_writer_builder = RollingFileWriterBuilder::new( + let rolling_file_writer = RollingFileWriter::new( parquet_writer_builder, - 1024 * 1024, // 1MB, large enough to not trigger rolling + 1024 * 1024, + file_io.clone(), + location_gen, + file_name_gen, ); - let data_file_writer_builder = DataFileWriterBuilder::new(rolling_writer_builder, None, 0); + let data_file_writer_builder = DataFileWriterBuilder::new(rolling_file_writer, None); // Create writer let mut writer = data_file_writer_builder.build().await?; @@ -297,19 +312,19 @@ mod tests { let schema = make_test_schema()?; // Create writer builders - let parquet_writer_builder = ParquetWriterBuilder::new( - WriterProperties::builder().build(), - Arc::new(schema), - None, - file_io.clone(), + let parquet_writer_builder = + ParquetWriterBuilder::new(WriterProperties::builder().build(), Arc::new(schema)); + + // Set a very small target size to trigger rolling + let rolling_writer_builder = RollingFileWriter::new( + parquet_writer_builder, + 1024, + file_io, location_gen, file_name_gen, ); - // Set a very small target size to trigger rolling - let rolling_writer_builder = RollingFileWriterBuilder::new(parquet_writer_builder, 1024); - - let data_file_writer_builder = DataFileWriterBuilder::new(rolling_writer_builder, None, 0); + let data_file_writer_builder = DataFileWriterBuilder::new(rolling_writer_builder, None); // Create writer let mut writer = data_file_writer_builder.build().await?; diff --git a/crates/iceberg/src/writer/mod.rs b/crates/iceberg/src/writer/mod.rs index 374a39f5b7..5e1b91bfed 100644 --- a/crates/iceberg/src/writer/mod.rs +++ b/crates/iceberg/src/writer/mod.rs @@ -60,6 +60,7 @@ //! async fn main() -> Result<()> { //! // Connect to a catalog. //! use iceberg::memory::{MEMORY_CATALOG_WAREHOUSE, MemoryCatalogBuilder}; +//! use iceberg::writer::file_writer::rolling_writer::RollingFileWriter; //! let catalog = MemoryCatalogBuilder::default() //! .load( //! "memory", @@ -86,15 +87,20 @@ //! let parquet_writer_builder = ParquetWriterBuilder::new( //! WriterProperties::default(), //! table.metadata().current_schema().clone(), -//! None, +//! ); +//! +//! // Create a rolling file writer using parquet file writer builder. +//! let rolling_file_writer = RollingFileWriter::new_with_default_file_size( +//! parquet_writer_builder, //! table.file_io().clone(), //! location_generator.clone(), //! file_name_generator.clone(), //! ); +//! //! // Create a data file writer using parquet file writer builder. -//! let data_file_writer_builder = DataFileWriterBuilder::new(parquet_writer_builder, None, 0); +//! let data_file_writer_builder = DataFileWriterBuilder::new(rolling_file_writer, None); //! // Build the data file writer -//! let mut data_file_writer = data_file_writer_builder.build().await.unwrap(); +//! let mut data_file_writer = data_file_writer_builder.build().await?; //! //! // Write the data using data_file_writer... //! @@ -174,6 +180,7 @@ //! // Connect to a catalog. //! use iceberg::memory::MEMORY_CATALOG_WAREHOUSE; //! use iceberg::spec::{Literal, PartitionKey, Struct}; +//! use iceberg::writer::file_writer::rolling_writer::RollingFileWriter; //! //! let catalog = MemoryCatalogBuilder::default() //! .load( @@ -207,13 +214,20 @@ //! let parquet_writer_builder = ParquetWriterBuilder::new( //! WriterProperties::default(), //! table.metadata().current_schema().clone(), -//! Some(partition_key), +//! ); +//! +//! // Create a rolling file writer +//! let rolling_file_writer = RollingFileWriter::new( +//! parquet_writer_builder, +//! 512 * 1024 * 1024, //! table.file_io().clone(), //! location_generator.clone(), //! file_name_generator.clone(), //! ); -//! // Create a data file writer builder using parquet file writer builder. -//! let data_file_writer_builder = DataFileWriterBuilder::new(parquet_writer_builder, None, 0); +//! +//! // Create a data file writer builder using rolling file writer. +//! let data_file_writer_builder = +//! DataFileWriterBuilder::new(rolling_file_writer, Some(partition_key)); //! // Create latency record writer using data file writer builder. //! let latency_record_builder = LatencyRecordWriterBuilder::new(data_file_writer_builder); //! // Build the final writer @@ -225,7 +239,6 @@ pub mod base_writer; pub mod file_writer; -pub mod partitioning; use arrow_array::RecordBatch; diff --git a/crates/iceberg/src/writer/partitioning/clustered.rs b/crates/iceberg/src/writer/partitioning/clustered.rs deleted file mode 100644 index 1ea0d24093..0000000000 --- a/crates/iceberg/src/writer/partitioning/clustered.rs +++ /dev/null @@ -1,251 +0,0 @@ -// // Licensed to the Apache Software Foundation (ASF) under one -// // or more contributor license agreements. See the NOTICE file -// // distributed with this work for additional information -// // regarding copyright ownership. The ASF licenses this file -// // to you under the Apache License, Version 2.0 (the -// // "License"); you may not use this file except in compliance -// // with the License. You may obtain a copy of the License at -// // -// // http://www.apache.org/licenses/LICENSE-2.0 -// // -// // Unless required by applicable law or agreed to in writing, -// // software distributed under the License is distributed on an -// // "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -// // KIND, either express or implied. See the License for the -// // specific language governing permissions and limitations -// // under the License. -// -// //! This module provides the `ClusteredWriter` implementation. -// -// use std::collections::HashSet; -// -// use arrow_array::RecordBatch; -// use async_trait::async_trait; -// -// use crate::spec::{PartitionKey, Struct}; -// use crate::writer::{DefaultInput, DefaultOutput, IcebergWriter, IcebergWriterBuilder}; -// use crate::writer::file_writer::location_generator::{FileNameGenerator, LocationGenerator}; -// use crate::{Error, ErrorKind, Result}; -// use crate::writer::partitioning::PartitioningWriter; -// -// /// A writer that writes data to a single partition at a time. -// /// -// /// When data for a new partition arrives, it closes the current writer -// /// and creates a new one for the new partition. -// /// -// /// Once a partition has been written to and closed, any further attempts -// /// to write to that partition will result in an error. -// pub struct ClusteredWriter -// { -// inner_builder: B, -// current_writer: Option, -// current_partition: Option, -// closed_partitions: HashSet, -// output: O, -// } -// -// impl ClusteredWriter { -// /// todo doc -// pub fn new( -// inner_builder: B -// ) -> Self { -// Self { -// inner_builder, -// current_writer: None, -// current_partition: None, -// closed_partitions: HashSet::new(), -// output: O::default(), -// } -// } -// -// /// Close the current writer if it exists -// async fn close_current_writer(&mut self) -> Result<()> { -// if let Some(mut writer) = self.current_writer.take() { -// self.output.extend(writer.close().await?); -// -// // Add the current partition to the set of closed partitions -// if let Some(current_partition) = self.current_partition.take() { -// self.closed_partitions.insert(current_partition); -// } -// } -// -// Ok(()) -// } -// -// async fn do_write(&mut self, input: I) -> Result<()> { -// if let Some(writer) = &mut self.current_writer { -// writer.write(input).await?; -// Ok(()) -// } else { -// Err(Error::new( -// ErrorKind::Unexpected, -// "Writer is not initialized!", -// )) -// } -// } -// } -// -// #[async_trait] -// impl PartitioningWriter for ClusteredWriter { -// async fn write(&mut self, partition_key: Option, input: I) -> Result<()> { -// if let Some(partition_key) = partition_key { -// let partition_value = partition_key.data(); -// -// // Check if this partition has been closed already -// if self.closed_partitions.contains(partition_value) { -// return Err(Error::new( -// ErrorKind::Unexpected, -// format!( -// "Cannot write to partition that was previously closed: {:?}", -// partition_key -// ), -// )); -// } -// -// // Check if we need to switch to a new partition -// let need_new_writer = match &self.current_partition { -// Some(current) => current != partition_value, -// None => true, -// }; -// -// if need_new_writer { -// self.close_current_writer().await?; -// -// // Create a new writer for the new partition -// self.current_writer = Some(self.inner_builder.clone().build().await?); -// self.current_partition = Some(partition_value.clone()); -// } -// } -// -// self.do_write(input).await -// } -// -// async fn close(&mut self) -> Result { -// self.close_current_writer().await?; -// -// // Return all collected data files -// Ok(std::mem::take(&mut self.output)) -// } -// } -// -// #[cfg(test)] -// mod tests { -// use std::sync::Arc; -// -// use arrow_array::{Int32Array, StringArray}; -// use arrow_schema::{DataType, Field, Schema}; -// use parquet::file::properties::WriterProperties; -// use tempfile::TempDir; -// -// use super::*; -// use crate::arrow::FieldMatchMode; -// use crate::io::FileIOBuilder; -// use crate::spec::{DataFileFormat, Literal, NestedField, PrimitiveType, Struct, Type}; -// use crate::writer::base_writer::data_file_writer::DataFileWriterBuilder; -// use crate::writer::file_writer::ParquetWriterBuilder; -// use crate::writer::file_writer::location_generator::DefaultFileNameGenerator; -// use crate::writer::file_writer::location_generator::test::MockLocationGenerator; -// -// #[tokio::test] -// async fn test_clustered_writer_basic() -> Result<()> { -// let temp_dir = TempDir::new()?; -// let file_io = FileIOBuilder::new_fs_io().build()?; -// let location_gen = -// MockLocationGenerator::new(temp_dir.path().to_str().unwrap().to_string()); -// let file_name_gen = -// DefaultFileNameGenerator::new("test".to_string(), None, DataFileFormat::Parquet); -// -// // Create schema -// let schema = Arc::new( -// crate::spec::Schema::builder() -// .with_schema_id(1) -// .with_fields(vec![ -// NestedField::required(1, "id", Type::Primitive(PrimitiveType::Int)).into(), -// NestedField::required(2, "name", Type::Primitive(PrimitiveType::String)).into(), -// NestedField::required(3, "category", Type::Primitive(PrimitiveType::String)) -// .into(), -// ]) -// .build()?, -// ); -// -// // Create writer builder -// let parquet_writer_builder = ParquetWriterBuilder::new_with_match_mode( -// WriterProperties::builder().build(), -// schema.clone(), -// FieldMatchMode::Name, -// ); -// -// // Create data file writer builder -// let data_file_writer_builder = DataFileWriterBuilder::new(parquet_writer_builder, None, 0); -// -// // Create clustered writer -// let mut writer = ClusteredWriter::new( -// data_file_writer_builder, -// location_gen, -// file_name_gen, -// file_io.clone(), -// 1024 * 1024, // 1MB -// ); -// -// // Create test data for partition A -// let arrow_schema = Schema::new(vec![ -// Field::new("id", DataType::Int32, false), -// Field::new("name", DataType::Utf8, false), -// Field::new("category", DataType::Utf8, false), -// ]); -// -// let batch_a1 = RecordBatch::try_new(Arc::new(arrow_schema.clone()), vec![ -// Arc::new(Int32Array::from(vec![1, 2, 3])), -// Arc::new(StringArray::from(vec!["Alice", "Bob", "Charlie"])), -// Arc::new(StringArray::from(vec!["A", "A", "A"])), -// ])?; -// -// let batch_a2 = RecordBatch::try_new(Arc::new(arrow_schema.clone()), vec![ -// Arc::new(Int32Array::from(vec![4, 5])), -// Arc::new(StringArray::from(vec!["Dave", "Eve"])), -// Arc::new(StringArray::from(vec!["A", "A"])), -// ])?; -// -// // Create test data for partition B -// let batch_b = RecordBatch::try_new(Arc::new(arrow_schema.clone()), vec![ -// Arc::new(Int32Array::from(vec![6, 7, 8])), -// Arc::new(StringArray::from(vec!["Frank", "Grace", "Heidi"])), -// Arc::new(StringArray::from(vec!["B", "B", "B"])), -// ])?; -// -// // Create partition keys -// let partition_a = PartitionKey::new( -// crate::spec::PartitionSpec::unpartition_spec().into(), -// schema.clone(), -// Struct::from_iter([Some(Literal::string("A"))]), -// ); -// -// let partition_b = PartitionKey::new( -// crate::spec::PartitionSpec::unpartition_spec().into(), -// schema.clone(), -// Struct::from_iter([Some(Literal::string("B"))]), -// ); -// -// // Write data to partition A -// writer.write(partition_a.clone(), batch_a1.clone()).await?; -// writer.write(partition_a.clone(), batch_a2.clone()).await?; -// -// // Write data to partition B -// writer.write(partition_b.clone(), batch_b.clone()).await?; -// -// // Try to write to partition A again (should fail) -// let result = writer.write(partition_a.clone(), batch_a1.clone()).await; -// assert!( -// result.is_err(), -// "Expected error when writing to closed partition" -// ); -// -// // Close writer and get data files -// let data_files = writer.close().await?; -// -// // Verify files were created -// assert_eq!(data_files.len(), 2, "Expected two data files to be created"); -// -// Ok(()) -// } -// } diff --git a/crates/iceberg/src/writer/partitioning/mod.rs b/crates/iceberg/src/writer/partitioning/mod.rs deleted file mode 100644 index 6639a511d3..0000000000 --- a/crates/iceberg/src/writer/partitioning/mod.rs +++ /dev/null @@ -1,30 +0,0 @@ -// Licensed to the Apache Software Foundation (ASF) under one -// or more contributor license agreements. See the NOTICE file -// distributed with this work for additional information -// regarding copyright ownership. The ASF licenses this file -// to you under the Apache License, Version 2.0 (the -// "License"); you may not use this file except in compliance -// with the License. You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, -// software distributed under the License is distributed on an -// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -// KIND, either express or implied. See the License for the -// specific language governing permissions and limitations -// under the License. - -mod clustered; - -use crate::Result; -use crate::spec::PartitionKey; -use crate::writer::{DefaultInput, DefaultOutput}; - -#[async_trait::async_trait] -pub trait PartitioningWriter: Send + 'static { - /// todo doc - async fn write(&mut self, partition_key: Option, input: I) -> Result<()>; - /// todo doc - async fn close(&mut self) -> Result; -} diff --git a/crates/integrations/datafusion/src/physical_plan/write.rs b/crates/integrations/datafusion/src/physical_plan/write.rs index f49fbfc334..e095060bdc 100644 --- a/crates/integrations/datafusion/src/physical_plan/write.rs +++ b/crates/integrations/datafusion/src/physical_plan/write.rs @@ -208,7 +208,6 @@ impl ExecutionPlan for IcebergWriteExec { )); } - let spec_id = self.table.metadata().default_partition_spec_id(); let partition_type = self.table.metadata().default_partition_type().clone(); let format_version = self.table.metadata().format_version(); @@ -269,6 +268,7 @@ impl ExecutionPlan for IcebergWriteExec { location_generator, file_name_generator, ); + // todo specify partition key when partitioning writer is supported let data_file_writer_builder = DataFileWriterBuilder::new(rolling_writer, None); // Get input data From 4532f1eeda6004ec11094a4b71e086e0d21ff69b Mon Sep 17 00:00:00 2001 From: Shawn Chang Date: Wed, 24 Sep 2025 16:55:57 -0700 Subject: [PATCH 05/13] fix test compilation, rebase --- crates/iceberg/src/spec/partition.rs | 9 ++ .../writer/base_writer/data_file_writer.rs | 7 +- .../base_writer/equality_delete_writer.rs | 8 +- .../src/writer/file_writer/parquet_writer.rs | 16 ++-- .../shared_tests/append_data_file_test.rs | 7 +- .../append_partition_data_file_test.rs | 84 +++++++++---------- .../shared_tests/conflict_commit_test.rs | 7 +- .../tests/shared_tests/scan_all_type.rs | 7 +- 8 files changed, 80 insertions(+), 65 deletions(-) diff --git a/crates/iceberg/src/spec/partition.rs b/crates/iceberg/src/spec/partition.rs index fe8767266e..128db338af 100644 --- a/crates/iceberg/src/spec/partition.rs +++ b/crates/iceberg/src/spec/partition.rs @@ -194,6 +194,15 @@ impl PartitionKey { Self { spec, schema, data } } + /// Creates a new partition key from another partition key, with a new data field. + pub fn copy_with_data(&self, data: Struct) -> Self { + Self { + spec: self.spec.clone(), + schema: self.schema.clone(), + data, + } + } + /// Generates a partition path based on the partition values. pub fn to_path(&self) -> String { self.spec.partition_to_path(&self.data, self.schema.clone()) diff --git a/crates/iceberg/src/writer/base_writer/data_file_writer.rs b/crates/iceberg/src/writer/base_writer/data_file_writer.rs index b56655e06c..797225eb2a 100644 --- a/crates/iceberg/src/writer/base_writer/data_file_writer.rs +++ b/crates/iceberg/src/writer/base_writer/data_file_writer.rs @@ -143,9 +143,8 @@ mod test { use crate::Result; use crate::io::FileIOBuilder; use crate::spec::{ - DataContentType, DataFileFormat, Literal, NestedField, - PROPERTY_WRITE_TARGET_FILE_SIZE_BYTES_DEFAULT, PartitionKey, PartitionSpec, PrimitiveType, - Schema, Struct, Type, + DataContentType, DataFileFormat, Literal, NestedField, PartitionKey, PartitionSpec, + PrimitiveType, Schema, Struct, Type, }; use crate::writer::base_writer::data_file_writer::DataFileWriterBuilder; use crate::writer::file_writer::ParquetWriterBuilder; @@ -255,7 +254,7 @@ mod test { let partition_key = PartitionKey::new( PartitionSpec::builder(schema_ref.clone()).build()?, schema_ref.clone(), - partition_value, + partition_value.clone(), ); let parquet_writer_builder = diff --git a/crates/iceberg/src/writer/base_writer/equality_delete_writer.rs b/crates/iceberg/src/writer/base_writer/equality_delete_writer.rs index 3b0d43e2c1..88663f4cde 100644 --- a/crates/iceberg/src/writer/base_writer/equality_delete_writer.rs +++ b/crates/iceberg/src/writer/base_writer/equality_delete_writer.rs @@ -425,8 +425,12 @@ mod test { // prepare writer let pb = ParquetWriterBuilder::new(WriterProperties::builder().build(), Arc::new(delete_schema)); - let rolling_writer = - RollingFileWriter::new_with_default_file_size(pb, file_io, location_gen, file_name_gen); + let rolling_writer = RollingFileWriter::new_with_default_file_size( + pb, + file_io.clone(), + location_gen, + file_name_gen, + ); let mut equality_delete_writer = EqualityDeleteFileWriterBuilder::new(rolling_writer, equality_config) .build() diff --git a/crates/iceberg/src/writer/file_writer/parquet_writer.rs b/crates/iceberg/src/writer/file_writer/parquet_writer.rs index 2d7f7ec090..620f27df33 100644 --- a/crates/iceberg/src/writer/file_writer/parquet_writer.rs +++ b/crates/iceberg/src/writer/file_writer/parquet_writer.rs @@ -1624,9 +1624,8 @@ mod tests { }; let col = Arc::new(Int64Array::from_iter_values(0..1024)) as ArrayRef; let to_write = RecordBatch::try_new(schema.clone(), vec![col]).unwrap(); - let output_file = file_io.new_output( - location_gen.generate_location(None, &file_name_gen.generate_file_name()), - )?; + let file_path = location_gen.generate_location(None, &file_name_gen.generate_file_name()); + let output_file = file_io.new_output(&file_path)?; let mut pw = ParquetWriterBuilder::new( WriterProperties::builder().build(), Arc::new(to_write.schema().as_ref().try_into().unwrap()), @@ -1634,25 +1633,22 @@ mod tests { .build(output_file) .await?; pw.write(&to_write).await?; - let file_path = output_file.location().to_string(); pw.close().await.unwrap(); - assert!(file_io.exists(file_path).await.unwrap()); + assert!(file_io.exists(&file_path).await.unwrap()); // Test that file will not create if no data to write let file_name_gen = DefaultFileNameGenerator::new("test_empty".to_string(), None, DataFileFormat::Parquet); - let output_file = file_io.new_output( - location_gen.generate_location(None, &file_name_gen.generate_file_name()), - )?; + let file_path = location_gen.generate_location(None, &file_name_gen.generate_file_name()); + let output_file = file_io.new_output(&file_path)?; let pw = ParquetWriterBuilder::new( WriterProperties::builder().build(), Arc::new(to_write.schema().as_ref().try_into().unwrap()), ) .build(output_file) .await?; - let file_path = output_file.location().to_string(); pw.close().await.unwrap(); - assert!(!file_io.exists(file_path).await.unwrap()); + assert!(!file_io.exists(&file_path).await.unwrap()); Ok(()) } diff --git a/crates/integration_tests/tests/shared_tests/append_data_file_test.rs b/crates/integration_tests/tests/shared_tests/append_data_file_test.rs index 9a3d62137e..6c7b06740b 100644 --- a/crates/integration_tests/tests/shared_tests/append_data_file_test.rs +++ b/crates/integration_tests/tests/shared_tests/append_data_file_test.rs @@ -27,6 +27,7 @@ use iceberg::writer::file_writer::ParquetWriterBuilder; use iceberg::writer::file_writer::location_generator::{ DefaultFileNameGenerator, DefaultLocationGenerator, }; +use iceberg::writer::file_writer::rolling_writer::RollingFileWriter; use iceberg::writer::{IcebergWriter, IcebergWriterBuilder}; use iceberg::{Catalog, CatalogBuilder, TableCreation}; use iceberg_catalog_rest::RestCatalogBuilder; @@ -74,12 +75,14 @@ async fn test_append_data_file() { let parquet_writer_builder = ParquetWriterBuilder::new( WriterProperties::default(), table.metadata().current_schema().clone(), - None, + ); + let rolling_file_writer = RollingFileWriter::new_with_default_file_size( + parquet_writer_builder, table.file_io().clone(), location_generator.clone(), file_name_generator.clone(), ); - let data_file_writer_builder = DataFileWriterBuilder::new(parquet_writer_builder, None, 0); + let data_file_writer_builder = DataFileWriterBuilder::new(rolling_file_writer, None); let mut data_file_writer = data_file_writer_builder.build().await.unwrap(); let col1 = StringArray::from(vec![Some("foo"), Some("bar"), None, Some("baz")]); let col2 = Int32Array::from(vec![Some(1), Some(2), Some(3), Some(4)]); diff --git a/crates/integration_tests/tests/shared_tests/append_partition_data_file_test.rs b/crates/integration_tests/tests/shared_tests/append_partition_data_file_test.rs index c737357ee9..204b6d6e39 100644 --- a/crates/integration_tests/tests/shared_tests/append_partition_data_file_test.rs +++ b/crates/integration_tests/tests/shared_tests/append_partition_data_file_test.rs @@ -31,6 +31,7 @@ use iceberg::writer::file_writer::ParquetWriterBuilder; use iceberg::writer::file_writer::location_generator::{ DefaultFileNameGenerator, DefaultLocationGenerator, }; +use iceberg::writer::file_writer::rolling_writer::RollingFileWriter; use iceberg::writer::{IcebergWriter, IcebergWriterBuilder}; use iceberg::{Catalog, CatalogBuilder, TableCreation}; use iceberg_catalog_rest::RestCatalogBuilder; @@ -96,22 +97,20 @@ async fn test_append_partition_data_file() { let parquet_writer_builder = ParquetWriterBuilder::new( WriterProperties::default(), table.metadata().current_schema().clone(), - Some(partition_key), + ); + + let rolling_file_writer = RollingFileWriter::new_with_default_file_size( + parquet_writer_builder.clone(), table.file_io().clone(), location_generator.clone(), file_name_generator.clone(), ); - let mut data_file_writer_valid = DataFileWriterBuilder::new( - parquet_writer_builder.clone(), - Some(Struct::from_iter([Some(Literal::Primitive( - PrimitiveLiteral::Int(first_partition_id_value), - ))])), - 0, - ) - .build() - .await - .unwrap(); + let mut data_file_writer_valid = + DataFileWriterBuilder::new(rolling_file_writer.clone(), Some(partition_key.clone())) + .build() + .await + .unwrap(); let col1 = StringArray::from(vec![Some("foo1"), Some("foo2")]); let col2 = Int32Array::from(vec![ @@ -148,44 +147,53 @@ async fn test_append_partition_data_file() { assert_eq!(batches.len(), 1); assert_eq!(batches[0], batch); + let partition_key = partition_key.copy_with_data(Struct::from_iter([Some( + Literal::Primitive(PrimitiveLiteral::Boolean(true)), + )])); test_schema_incompatible_partition_type( - parquet_writer_builder.clone(), + rolling_file_writer.clone(), batch.clone(), + partition_key.clone(), table.clone(), &rest_catalog, ) .await; + let partition_key = partition_key.copy_with_data(Struct::from_iter([ + Some(Literal::Primitive(PrimitiveLiteral::Int( + first_partition_id_value, + ))), + Some(Literal::Primitive(PrimitiveLiteral::Int( + first_partition_id_value, + ))), + ])); test_schema_incompatible_partition_fields( - parquet_writer_builder, + rolling_file_writer.clone(), batch, + partition_key, table, &rest_catalog, - first_partition_id_value, ) .await; } async fn test_schema_incompatible_partition_type( - parquet_writer_builder: ParquetWriterBuilder< + rolling_file_writer: RollingFileWriter< + ParquetWriterBuilder, DefaultLocationGenerator, DefaultFileNameGenerator, >, batch: RecordBatch, + partition_key: PartitionKey, table: Table, catalog: &dyn Catalog, ) { // test writing different "type" of partition than mentioned in schema - let mut data_file_writer_invalid = DataFileWriterBuilder::new( - parquet_writer_builder.clone(), - Some(Struct::from_iter([Some(Literal::Primitive( - PrimitiveLiteral::Boolean(true), - ))])), - 0, - ) - .build() - .await - .unwrap(); + let mut data_file_writer_invalid = + DataFileWriterBuilder::new(rolling_file_writer.clone(), Some(partition_key)) + .build() + .await + .unwrap(); data_file_writer_invalid.write(batch.clone()).await.unwrap(); let data_file_invalid = data_file_writer_invalid.close().await.unwrap(); @@ -200,32 +208,22 @@ async fn test_schema_incompatible_partition_type( } async fn test_schema_incompatible_partition_fields( - parquet_writer_builder: ParquetWriterBuilder< + rolling_file_writer: RollingFileWriter< + ParquetWriterBuilder, DefaultLocationGenerator, DefaultFileNameGenerator, >, batch: RecordBatch, + partition_key: PartitionKey, table: Table, catalog: &dyn Catalog, - first_partition_id_value: i32, ) { // test writing different number of partition fields than mentioned in schema - - let mut data_file_writer_invalid = DataFileWriterBuilder::new( - parquet_writer_builder, - Some(Struct::from_iter([ - Some(Literal::Primitive(PrimitiveLiteral::Int( - first_partition_id_value, - ))), - Some(Literal::Primitive(PrimitiveLiteral::Int( - first_partition_id_value, - ))), - ])), - 0, - ) - .build() - .await - .unwrap(); + let mut data_file_writer_invalid = + DataFileWriterBuilder::new(rolling_file_writer, Some(partition_key)) + .build() + .await + .unwrap(); data_file_writer_invalid.write(batch.clone()).await.unwrap(); let data_file_invalid = data_file_writer_invalid.close().await.unwrap(); diff --git a/crates/integration_tests/tests/shared_tests/conflict_commit_test.rs b/crates/integration_tests/tests/shared_tests/conflict_commit_test.rs index 4b85612ff5..8afaaac76f 100644 --- a/crates/integration_tests/tests/shared_tests/conflict_commit_test.rs +++ b/crates/integration_tests/tests/shared_tests/conflict_commit_test.rs @@ -27,6 +27,7 @@ use iceberg::writer::file_writer::ParquetWriterBuilder; use iceberg::writer::file_writer::location_generator::{ DefaultFileNameGenerator, DefaultLocationGenerator, }; +use iceberg::writer::file_writer::rolling_writer::RollingFileWriter; use iceberg::writer::{IcebergWriter, IcebergWriterBuilder}; use iceberg::{Catalog, CatalogBuilder, TableCreation}; use iceberg_catalog_rest::RestCatalogBuilder; @@ -73,12 +74,14 @@ async fn test_append_data_file_conflict() { let parquet_writer_builder = ParquetWriterBuilder::new( WriterProperties::default(), table.metadata().current_schema().clone(), - None, + ); + let rolling_file_writer = RollingFileWriter::new_with_default_file_size( + parquet_writer_builder, table.file_io().clone(), location_generator.clone(), file_name_generator.clone(), ); - let data_file_writer_builder = DataFileWriterBuilder::new(parquet_writer_builder, None, 0); + let data_file_writer_builder = DataFileWriterBuilder::new(rolling_file_writer, None); let mut data_file_writer = data_file_writer_builder.build().await.unwrap(); let col1 = StringArray::from(vec![Some("foo"), Some("bar"), None, Some("baz")]); let col2 = Int32Array::from(vec![Some(1), Some(2), Some(3), Some(4)]); diff --git a/crates/integration_tests/tests/shared_tests/scan_all_type.rs b/crates/integration_tests/tests/shared_tests/scan_all_type.rs index b8e7c5941e..36e7367fa4 100644 --- a/crates/integration_tests/tests/shared_tests/scan_all_type.rs +++ b/crates/integration_tests/tests/shared_tests/scan_all_type.rs @@ -39,6 +39,7 @@ use iceberg::writer::file_writer::ParquetWriterBuilder; use iceberg::writer::file_writer::location_generator::{ DefaultFileNameGenerator, DefaultLocationGenerator, }; +use iceberg::writer::file_writer::rolling_writer::RollingFileWriter; use iceberg::writer::{IcebergWriter, IcebergWriterBuilder}; use iceberg::{Catalog, CatalogBuilder, TableCreation}; use iceberg_catalog_rest::RestCatalogBuilder; @@ -155,12 +156,14 @@ async fn test_scan_all_type() { let parquet_writer_builder = ParquetWriterBuilder::new( WriterProperties::default(), table.metadata().current_schema().clone(), - None, + ); + let rolling_file_writer = RollingFileWriter::new_with_default_file_size( + parquet_writer_builder, table.file_io().clone(), location_generator.clone(), file_name_generator.clone(), ); - let data_file_writer_builder = DataFileWriterBuilder::new(parquet_writer_builder, None, 0); + let data_file_writer_builder = DataFileWriterBuilder::new(rolling_file_writer, None); let mut data_file_writer = data_file_writer_builder.build().await.unwrap(); // Prepare data From 917c25930238b07e2553f3c564aa4eb52ae9a766 Mon Sep 17 00:00:00 2001 From: Shawn Chang Date: Wed, 24 Sep 2025 17:36:44 -0700 Subject: [PATCH 06/13] CI From ebe92b517dcb0423c9888531bfd51f47217cc7fc Mon Sep 17 00:00:00 2001 From: Shawn Chang Date: Thu, 25 Sep 2025 08:56:57 -0700 Subject: [PATCH 07/13] fix doc --- .../src/writer/file_writer/rolling_writer.rs | 49 +++++++++++++++++-- 1 file changed, 45 insertions(+), 4 deletions(-) diff --git a/crates/iceberg/src/writer/file_writer/rolling_writer.rs b/crates/iceberg/src/writer/file_writer/rolling_writer.rs index c05ffd69e9..863f0feb36 100644 --- a/crates/iceberg/src/writer/file_writer/rolling_writer.rs +++ b/crates/iceberg/src/writer/file_writer/rolling_writer.rs @@ -70,7 +70,19 @@ impl Debug } impl RollingFileWriter { - /// todo doc + /// Creates a new `RollingFileWriter` with the specified target file size. + /// + /// # Parameters + /// + /// * `inner_builder` - The builder for the underlying file writer + /// * `target_file_size` - The target file size in bytes that triggers rollover + /// * `file_io` - The file IO interface for creating output files + /// * `location_generator` - Generator for file locations + /// * `file_name_generator` - Generator for file names + /// + /// # Returns + /// + /// A new `RollingFileWriter` instance pub fn new( inner_builder: B, target_file_size: usize, @@ -89,7 +101,18 @@ impl RollingFi } } - /// todo doc + /// Creates a new `RollingFileWriter` with the default target file size. + /// + /// # Parameters + /// + /// * `inner_builder` - The builder for the underlying file writer + /// * `file_io` - The file IO interface for creating output files + /// * `location_generator` - Generator for file locations + /// * `file_name_generator` - Generator for file names + /// + /// # Returns + /// + /// A new `RollingFileWriter` instance with default target file size pub fn new_with_default_file_size( inner_builder: B, file_io: FileIO, @@ -124,7 +147,20 @@ impl RollingFi )) } - /// todo doc + /// Writes a record batch to the current file, rolling over to a new file if necessary. + /// + /// # Parameters + /// + /// * `partition_key` - Optional partition key for the data + /// * `input` - The record batch to write + /// + /// # Returns + /// + /// A `Result` indicating success or failure + /// + /// # Errors + /// + /// Returns an error if the writer is not initialized or if writing fails pub async fn write( &mut self, partition_key: &Option, @@ -166,7 +202,12 @@ impl RollingFi } } - /// todo doc + /// Closes the writer and returns all data file builders. + /// + /// # Returns + /// + /// A `Result` containing a vector of `DataFileBuilder` instances representing + /// all files that were written, including any that were created due to rollover pub async fn close(mut self) -> Result> { // close the current writer and merge the output if let Some(current_writer) = self.inner { From 9cef787e848241a7290474994c27836adc43f47a Mon Sep 17 00:00:00 2001 From: Shawn Chang Date: Thu, 25 Sep 2025 16:17:01 -0700 Subject: [PATCH 08/13] ci flaky? From ef0aafb0daeff75499b816ee358bd2043e67bc64 Mon Sep 17 00:00:00 2001 From: Shawn Chang Date: Fri, 26 Sep 2025 16:12:57 -0700 Subject: [PATCH 09/13] handSome(writer) --- .../src/writer/base_writer/data_file_writer.rs | 13 ++++++++----- 1 file changed, 8 insertions(+), 5 deletions(-) diff --git a/crates/iceberg/src/writer/base_writer/data_file_writer.rs b/crates/iceberg/src/writer/base_writer/data_file_writer.rs index 797225eb2a..aaa1ce51b5 100644 --- a/crates/iceberg/src/writer/base_writer/data_file_writer.rs +++ b/crates/iceberg/src/writer/base_writer/data_file_writer.rs @@ -75,11 +75,14 @@ impl IcebergWr for DataFileWriter { async fn write(&mut self, batch: RecordBatch) -> Result<()> { - self.inner_writer - .as_mut() - .unwrap() - .write(&self.partition_key, &batch) - .await + if let Some(writer) = self.inner_writer.as_mut() { + writer.write(&self.partition_key, &batch).await + } else { + Err(Error::new( + ErrorKind::Unexpected, + "Writer is not initialized!", + )) + } } async fn close(&mut self) -> Result> { From 80c34b505579f789559cd0336805eaf3d9d55b96 Mon Sep 17 00:00:00 2001 From: Shawn Chang Date: Fri, 26 Sep 2025 21:15:54 -0700 Subject: [PATCH 10/13] give default values to datafile partition and spec_id --- crates/iceberg/src/spec/manifest/data_file.rs | 4 +++- .../writer/base_writer/data_file_writer.rs | 19 +++++++------------ .../base_writer/equality_delete_writer.rs | 17 ++++++----------- 3 files changed, 16 insertions(+), 24 deletions(-) diff --git a/crates/iceberg/src/spec/manifest/data_file.rs b/crates/iceberg/src/spec/manifest/data_file.rs index 931f9441e9..d7455b56fe 100644 --- a/crates/iceberg/src/spec/manifest/data_file.rs +++ b/crates/iceberg/src/spec/manifest/data_file.rs @@ -26,7 +26,7 @@ use serde_with::{DeserializeFromStr, SerializeDisplay}; use super::_serde::DataFileSerde; use super::{Datum, FormatVersion, Schema, data_file_schema_v1, data_file_schema_v2}; use crate::error::Result; -use crate::spec::{Struct, StructType}; +use crate::spec::{DEFAULT_PARTITION_SPEC_ID, Struct, StructType}; use crate::{Error, ErrorKind}; /// Data file carries data file path, partition tuple, metrics, … @@ -49,6 +49,7 @@ pub struct DataFile { /// /// Partition data tuple, schema based on the partition spec output using /// partition field ids for the struct field ids + #[builder(default = "Struct::empty()")] pub(crate) partition: Struct, /// field id: 103 /// @@ -156,6 +157,7 @@ pub struct DataFile { pub(crate) first_row_id: Option, /// This field is not included in spec. It is just store in memory representation used /// in process. + #[builder(default = "DEFAULT_PARTITION_SPEC_ID")] pub(crate) partition_spec_id: i32, /// field id: 143 /// diff --git a/crates/iceberg/src/writer/base_writer/data_file_writer.rs b/crates/iceberg/src/writer/base_writer/data_file_writer.rs index aaa1ce51b5..7d944007a6 100644 --- a/crates/iceberg/src/writer/base_writer/data_file_writer.rs +++ b/crates/iceberg/src/writer/base_writer/data_file_writer.rs @@ -20,7 +20,7 @@ use arrow_array::RecordBatch; use itertools::Itertools; -use crate::spec::{DEFAULT_PARTITION_SPEC_ID, DataContentType, DataFile, PartitionKey, Struct}; +use crate::spec::{DataContentType, DataFile, PartitionKey}; use crate::writer::file_writer::FileWriterBuilder; use crate::writer::file_writer::location_generator::{FileNameGenerator, LocationGenerator}; use crate::writer::file_writer::rolling_writer::RollingFileWriter; @@ -93,17 +93,12 @@ impl IcebergWr .into_iter() .map(|mut res| { res.content(DataContentType::Data); - res.partition( - self.partition_key - .as_ref() - .map_or(Struct::empty(), |pk| pk.data().clone()), - ); - res.partition_spec_id( - self.partition_key - .as_ref() - .map_or(DEFAULT_PARTITION_SPEC_ID, |pk| pk.spec().spec_id()), - ); - res.build().expect("Guaranteed to be valid") + if let Some(pk) = self.partition_key.as_ref() { + res.partition(pk.data().clone()); + res.partition_spec_id(pk.spec().spec_id()); + } + res.build() + .expect("DataFileBuilder is guaranteed to be valid") }) .collect_vec()) } else { diff --git a/crates/iceberg/src/writer/base_writer/equality_delete_writer.rs b/crates/iceberg/src/writer/base_writer/equality_delete_writer.rs index 88663f4cde..21263ba1a6 100644 --- a/crates/iceberg/src/writer/base_writer/equality_delete_writer.rs +++ b/crates/iceberg/src/writer/base_writer/equality_delete_writer.rs @@ -168,17 +168,12 @@ impl IcebergWr .map(|mut res| { res.content(crate::spec::DataContentType::EqualityDeletes); res.equality_ids(Some(self.equality_ids.iter().copied().collect_vec())); - res.partition( - self.partition_key - .as_ref() - .map_or(Struct::empty(), |pk| pk.data().clone()), - ); - res.partition_spec_id( - self.partition_key - .as_ref() - .map_or(DEFAULT_PARTITION_SPEC_ID, |pk| pk.spec().spec_id()), - ); - res.build().expect("msg") + if let Some(pk) = self.partition_key.as_ref() { + res.partition(pk.data().clone()); + res.partition_spec_id(pk.spec().spec_id()); + } + res.build() + .expect("DataFileBuilder is guaranteed to be valid") }) .collect_vec()) } else { From 02ed7f6d70a82bb1abf78d3d984cb2479541659e Mon Sep 17 00:00:00 2001 From: Shawn Chang Date: Fri, 26 Sep 2025 21:35:05 -0700 Subject: [PATCH 11/13] daily clippy fix --- crates/iceberg/src/writer/base_writer/equality_delete_writer.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/crates/iceberg/src/writer/base_writer/equality_delete_writer.rs b/crates/iceberg/src/writer/base_writer/equality_delete_writer.rs index 21263ba1a6..edd5e2c735 100644 --- a/crates/iceberg/src/writer/base_writer/equality_delete_writer.rs +++ b/crates/iceberg/src/writer/base_writer/equality_delete_writer.rs @@ -26,7 +26,7 @@ use parquet::arrow::PARQUET_FIELD_ID_META_KEY; use crate::arrow::record_batch_projector::RecordBatchProjector; use crate::arrow::schema_to_arrow_schema; -use crate::spec::{DEFAULT_PARTITION_SPEC_ID, DataFile, PartitionKey, SchemaRef, Struct}; +use crate::spec::{DataFile, PartitionKey, SchemaRef}; use crate::writer::file_writer::FileWriterBuilder; use crate::writer::file_writer::location_generator::{FileNameGenerator, LocationGenerator}; use crate::writer::file_writer::rolling_writer::RollingFileWriter; From b87354e9762908e8c91b82cb8649ba2c6e60d0f9 Mon Sep 17 00:00:00 2001 From: Shawn Chang Date: Mon, 29 Sep 2025 12:38:34 -0700 Subject: [PATCH 12/13] expose datafile builder error --- .../src/writer/base_writer/data_file_writer.rs | 13 ++++++++----- .../writer/base_writer/equality_delete_writer.rs | 12 ++++++++---- 2 files changed, 16 insertions(+), 9 deletions(-) diff --git a/crates/iceberg/src/writer/base_writer/data_file_writer.rs b/crates/iceberg/src/writer/base_writer/data_file_writer.rs index 7d944007a6..f6bc2b605f 100644 --- a/crates/iceberg/src/writer/base_writer/data_file_writer.rs +++ b/crates/iceberg/src/writer/base_writer/data_file_writer.rs @@ -18,7 +18,6 @@ //! This module provide `DataFileWriter`. use arrow_array::RecordBatch; -use itertools::Itertools; use crate::spec::{DataContentType, DataFile, PartitionKey}; use crate::writer::file_writer::FileWriterBuilder; @@ -87,7 +86,7 @@ impl IcebergWr async fn close(&mut self) -> Result> { if let Some(writer) = self.inner_writer.take() { - Ok(writer + writer .close() .await? .into_iter() @@ -97,10 +96,14 @@ impl IcebergWr res.partition(pk.data().clone()); res.partition_spec_id(pk.spec().spec_id()); } - res.build() - .expect("DataFileBuilder is guaranteed to be valid") + res.build().map_err(|e| { + Error::new( + ErrorKind::DataInvalid, + format!("Failed to build data file: {}", e), + ) + }) }) - .collect_vec()) + .collect() } else { Err(Error::new( ErrorKind::Unexpected, diff --git a/crates/iceberg/src/writer/base_writer/equality_delete_writer.rs b/crates/iceberg/src/writer/base_writer/equality_delete_writer.rs index edd5e2c735..67d5710078 100644 --- a/crates/iceberg/src/writer/base_writer/equality_delete_writer.rs +++ b/crates/iceberg/src/writer/base_writer/equality_delete_writer.rs @@ -161,7 +161,7 @@ impl IcebergWr async fn close(&mut self) -> Result> { if let Some(writer) = self.inner_writer.take() { - Ok(writer + writer .close() .await? .into_iter() @@ -172,10 +172,14 @@ impl IcebergWr res.partition(pk.data().clone()); res.partition_spec_id(pk.spec().spec_id()); } - res.build() - .expect("DataFileBuilder is guaranteed to be valid") + res.build().map_err(|e| { + Error::new( + ErrorKind::DataInvalid, + format!("Failed to build data file: {}", e), + ) + }) }) - .collect_vec()) + .collect() } else { Err(Error::new( ErrorKind::Unexpected, From 99ea41b1ec5bf33e764b8006e6e489e2c9874fc7 Mon Sep 17 00:00:00 2001 From: Shawn Chang Date: Tue, 30 Sep 2025 13:03:28 -0700 Subject: [PATCH 13/13] Add rolling writer builder back, remove clone for rolling writer --- .../writer/base_writer/data_file_writer.rs | 62 ++++++---- .../base_writer/equality_delete_writer.rs | 50 +++++--- .../src/writer/file_writer/rolling_writer.rs | 117 +++++++++++------- crates/iceberg/src/writer/mod.rs | 17 ++- .../shared_tests/append_data_file_test.rs | 6 +- .../append_partition_data_file_test.rs | 28 +++-- .../shared_tests/conflict_commit_test.rs | 6 +- .../tests/shared_tests/scan_all_type.rs | 6 +- .../datafusion/src/physical_plan/write.rs | 6 +- 9 files changed, 176 insertions(+), 122 deletions(-) diff --git a/crates/iceberg/src/writer/base_writer/data_file_writer.rs b/crates/iceberg/src/writer/base_writer/data_file_writer.rs index f6bc2b605f..a950547d33 100644 --- a/crates/iceberg/src/writer/base_writer/data_file_writer.rs +++ b/crates/iceberg/src/writer/base_writer/data_file_writer.rs @@ -22,41 +22,47 @@ use arrow_array::RecordBatch; use crate::spec::{DataContentType, DataFile, PartitionKey}; use crate::writer::file_writer::FileWriterBuilder; use crate::writer::file_writer::location_generator::{FileNameGenerator, LocationGenerator}; -use crate::writer::file_writer::rolling_writer::RollingFileWriter; +use crate::writer::file_writer::rolling_writer::{RollingFileWriter, RollingFileWriterBuilder}; use crate::writer::{CurrentFileStatus, IcebergWriter, IcebergWriterBuilder}; use crate::{Error, ErrorKind, Result}; /// Builder for `DataFileWriter`. #[derive(Clone, Debug)] pub struct DataFileWriterBuilder { - inner_writer: RollingFileWriter, + inner: RollingFileWriterBuilder, partition_key: Option, } -impl - DataFileWriterBuilder +impl DataFileWriterBuilder +where + B: FileWriterBuilder, + L: LocationGenerator, + F: FileNameGenerator, { - /// Create a new `DataFileWriterBuilder` using a `FileWriterBuilder`. + /// Create a new `DataFileWriterBuilder` using a `RollingFileWriterBuilder`. pub fn new( - inner_writer: RollingFileWriter, + inner_builder: RollingFileWriterBuilder, partition_key: Option, ) -> Self { Self { - inner_writer, + inner: inner_builder, partition_key, } } } #[async_trait::async_trait] -impl IcebergWriterBuilder - for DataFileWriterBuilder +impl IcebergWriterBuilder for DataFileWriterBuilder +where + B: FileWriterBuilder, + L: LocationGenerator, + F: FileNameGenerator, { type R = DataFileWriter; async fn build(self) -> Result { Ok(DataFileWriter { - inner_writer: Some(self.inner_writer), + inner: Some(self.inner.clone().build()), partition_key: self.partition_key, }) } @@ -65,16 +71,19 @@ impl IcebergWr /// A writer write data is within one spec/partition. #[derive(Debug)] pub struct DataFileWriter { - inner_writer: Option>, + inner: Option>, partition_key: Option, } #[async_trait::async_trait] -impl IcebergWriter - for DataFileWriter +impl IcebergWriter for DataFileWriter +where + B: FileWriterBuilder, + L: LocationGenerator, + F: FileNameGenerator, { async fn write(&mut self, batch: RecordBatch) -> Result<()> { - if let Some(writer) = self.inner_writer.as_mut() { + if let Some(writer) = self.inner.as_mut() { writer.write(&self.partition_key, &batch).await } else { Err(Error::new( @@ -85,7 +94,7 @@ impl IcebergWr } async fn close(&mut self) -> Result> { - if let Some(writer) = self.inner_writer.take() { + if let Some(writer) = self.inner.take() { writer .close() .await? @@ -113,19 +122,22 @@ impl IcebergWr } } -impl CurrentFileStatus - for DataFileWriter +impl CurrentFileStatus for DataFileWriter +where + B: FileWriterBuilder, + L: LocationGenerator, + F: FileNameGenerator, { fn current_file_path(&self) -> String { - self.inner_writer.as_ref().unwrap().current_file_path() + self.inner.as_ref().unwrap().current_file_path() } fn current_row_num(&self) -> usize { - self.inner_writer.as_ref().unwrap().current_row_num() + self.inner.as_ref().unwrap().current_row_num() } fn current_written_size(&self) -> usize { - self.inner_writer.as_ref().unwrap().current_written_size() + self.inner.as_ref().unwrap().current_written_size() } } @@ -152,7 +164,7 @@ mod test { use crate::writer::file_writer::location_generator::{ DefaultFileNameGenerator, DefaultLocationGenerator, }; - use crate::writer::file_writer::rolling_writer::RollingFileWriter; + use crate::writer::file_writer::rolling_writer::RollingFileWriterBuilder; use crate::writer::{IcebergWriter, IcebergWriterBuilder, RecordBatch}; #[tokio::test] @@ -175,14 +187,14 @@ mod test { let pw = ParquetWriterBuilder::new(WriterProperties::builder().build(), Arc::new(schema)); - let rolling_file_writer = RollingFileWriter::new_with_default_file_size( + let rolling_file_writer_builder = RollingFileWriterBuilder::new_with_default_file_size( pw, file_io.clone(), location_gen, file_name_gen, ); - let mut data_file_writer = DataFileWriterBuilder::new(rolling_file_writer, None) + let mut data_file_writer = DataFileWriterBuilder::new(rolling_file_writer_builder, None) .build() .await .unwrap(); @@ -261,7 +273,7 @@ mod test { let parquet_writer_builder = ParquetWriterBuilder::new(WriterProperties::builder().build(), schema_ref.clone()); - let rolling_file_writer = RollingFileWriter::new_with_default_file_size( + let rolling_file_writer_builder = RollingFileWriterBuilder::new_with_default_file_size( parquet_writer_builder, file_io.clone(), location_gen, @@ -269,7 +281,7 @@ mod test { ); let mut data_file_writer = - DataFileWriterBuilder::new(rolling_file_writer, Some(partition_key)) + DataFileWriterBuilder::new(rolling_file_writer_builder, Some(partition_key)) .build() .await?; diff --git a/crates/iceberg/src/writer/base_writer/equality_delete_writer.rs b/crates/iceberg/src/writer/base_writer/equality_delete_writer.rs index 67d5710078..6740ed435c 100644 --- a/crates/iceberg/src/writer/base_writer/equality_delete_writer.rs +++ b/crates/iceberg/src/writer/base_writer/equality_delete_writer.rs @@ -29,7 +29,7 @@ use crate::arrow::schema_to_arrow_schema; use crate::spec::{DataFile, PartitionKey, SchemaRef}; use crate::writer::file_writer::FileWriterBuilder; use crate::writer::file_writer::location_generator::{FileNameGenerator, LocationGenerator}; -use crate::writer::file_writer::rolling_writer::RollingFileWriter; +use crate::writer::file_writer::rolling_writer::{RollingFileWriter, RollingFileWriterBuilder}; use crate::writer::{IcebergWriter, IcebergWriterBuilder}; use crate::{Error, ErrorKind, Result}; @@ -40,15 +40,21 @@ pub struct EqualityDeleteFileWriterBuilder< L: LocationGenerator, F: FileNameGenerator, > { - inner: RollingFileWriter, + inner: RollingFileWriterBuilder, config: EqualityDeleteWriterConfig, } -impl - EqualityDeleteFileWriterBuilder +impl EqualityDeleteFileWriterBuilder +where + B: FileWriterBuilder, + L: LocationGenerator, + F: FileNameGenerator, { - /// Create a new `EqualityDeleteFileWriterBuilder` using a `FileWriterBuilder`. - pub fn new(inner: RollingFileWriter, config: EqualityDeleteWriterConfig) -> Self { + /// Create a new `EqualityDeleteFileWriterBuilder` using a `RollingFileWriterBuilder`. + pub fn new( + inner: RollingFileWriterBuilder, + config: EqualityDeleteWriterConfig, + ) -> Self { Self { inner, config } } } @@ -115,14 +121,17 @@ impl EqualityDeleteWriterConfig { } #[async_trait::async_trait] -impl IcebergWriterBuilder - for EqualityDeleteFileWriterBuilder +impl IcebergWriterBuilder for EqualityDeleteFileWriterBuilder +where + B: FileWriterBuilder, + L: LocationGenerator, + F: FileNameGenerator, { type R = EqualityDeleteFileWriter; async fn build(self) -> Result { Ok(EqualityDeleteFileWriter { - inner_writer: Some(self.inner), // todo revisit this, probably still need a builder for rolling writer + inner: Some(self.inner.clone().build()), // todo revisit this, probably still need a builder for rolling writer projector: self.config.projector, equality_ids: self.config.equality_ids, partition_key: self.config.partition_key, @@ -137,19 +146,22 @@ pub struct EqualityDeleteFileWriter< L: LocationGenerator, F: FileNameGenerator, > { - inner_writer: Option>, + inner: Option>, projector: RecordBatchProjector, equality_ids: Vec, partition_key: Option, } #[async_trait::async_trait] -impl IcebergWriter - for EqualityDeleteFileWriter +impl IcebergWriter for EqualityDeleteFileWriter +where + B: FileWriterBuilder, + L: LocationGenerator, + F: FileNameGenerator, { async fn write(&mut self, batch: RecordBatch) -> Result<()> { let batch = self.projector.project_batch(batch)?; - if let Some(writer) = self.inner_writer.as_mut() { + if let Some(writer) = self.inner.as_mut() { writer.write(&self.partition_key, &batch).await } else { Err(Error::new( @@ -160,7 +172,7 @@ impl IcebergWr } async fn close(&mut self) -> Result> { - if let Some(writer) = self.inner_writer.take() { + if let Some(writer) = self.inner.take() { writer .close() .await? @@ -219,7 +231,7 @@ mod test { use crate::writer::file_writer::location_generator::{ DefaultFileNameGenerator, DefaultLocationGenerator, }; - use crate::writer::file_writer::rolling_writer::RollingFileWriter; + use crate::writer::file_writer::rolling_writer::RollingFileWriterBuilder; use crate::writer::{IcebergWriter, IcebergWriterBuilder}; async fn check_parquet_data_file_with_equality_delete_write( @@ -424,14 +436,14 @@ mod test { // prepare writer let pb = ParquetWriterBuilder::new(WriterProperties::builder().build(), Arc::new(delete_schema)); - let rolling_writer = RollingFileWriter::new_with_default_file_size( + let rolling_writer_builder = RollingFileWriterBuilder::new_with_default_file_size( pb, file_io.clone(), location_gen, file_name_gen, ); let mut equality_delete_writer = - EqualityDeleteFileWriterBuilder::new(rolling_writer, equality_config) + EqualityDeleteFileWriterBuilder::new(rolling_writer_builder, equality_config) .build() .await?; @@ -591,14 +603,14 @@ mod test { let pb = ParquetWriterBuilder::new(WriterProperties::builder().build(), Arc::new(delete_schema)); - let rolling_writer = RollingFileWriter::new_with_default_file_size( + let rolling_writer_builder = RollingFileWriterBuilder::new_with_default_file_size( pb, file_io.clone(), location_gen, file_name_gen, ); let mut equality_delete_writer = - EqualityDeleteFileWriterBuilder::new(rolling_writer, config) + EqualityDeleteFileWriterBuilder::new(rolling_writer_builder, config) .build() .await?; diff --git a/crates/iceberg/src/writer/file_writer/rolling_writer.rs b/crates/iceberg/src/writer/file_writer/rolling_writer.rs index 863f0feb36..0b9b105c5e 100644 --- a/crates/iceberg/src/writer/file_writer/rolling_writer.rs +++ b/crates/iceberg/src/writer/file_writer/rolling_writer.rs @@ -26,51 +26,27 @@ use crate::writer::file_writer::location_generator::{FileNameGenerator, Location use crate::writer::file_writer::{FileWriter, FileWriterBuilder}; use crate::{Error, ErrorKind, Result}; -/// A writer that automatically rolls over to a new file when the data size -/// exceeds a target threshold. -/// -/// This writer wraps another file writer that tracks the amount of data written. -/// When the data size exceeds the target size, it closes the current file and -/// starts writing to a new one. -pub struct RollingFileWriter { - inner: Option, +/// Builder for [`RollingFileWriter`]. +#[derive(Clone, Debug)] +pub struct RollingFileWriterBuilder< + B: FileWriterBuilder, + L: LocationGenerator, + F: FileNameGenerator, +> { inner_builder: B, target_file_size: usize, - data_file_builders: Vec, file_io: FileIO, location_generator: L, file_name_generator: F, } -impl Clone - for RollingFileWriter +impl RollingFileWriterBuilder +where + B: FileWriterBuilder, + L: LocationGenerator, + F: FileNameGenerator, { - fn clone(&self) -> Self { - Self { - inner: None, - inner_builder: self.inner_builder.clone(), - target_file_size: self.target_file_size, - data_file_builders: vec![], - file_io: self.file_io.clone(), - location_generator: self.location_generator.clone(), - file_name_generator: self.file_name_generator.clone(), - } - } -} - -impl Debug - for RollingFileWriter -{ - fn fmt(&self, f: &mut Formatter<'_>) -> std::fmt::Result { - f.debug_struct("RollingFileWriter") - .field("target_file_size", &self.target_file_size) - .field("file_io", &self.file_io) - .finish() - } -} - -impl RollingFileWriter { - /// Creates a new `RollingFileWriter` with the specified target file size. + /// Creates a new `RollingFileWriterBuilder` with the specified target file size. /// /// # Parameters /// @@ -82,7 +58,7 @@ impl RollingFi /// /// # Returns /// - /// A new `RollingFileWriter` instance + /// A new `RollingFileWriterBuilder` instance pub fn new( inner_builder: B, target_file_size: usize, @@ -91,17 +67,15 @@ impl RollingFi file_name_generator: F, ) -> Self { Self { - inner: None, inner_builder, target_file_size, - data_file_builders: vec![], file_io, location_generator, file_name_generator, } } - /// Creates a new `RollingFileWriter` with the default target file size. + /// Creates a new `RollingFileWriterBuilder` with the default target file size. /// /// # Parameters /// @@ -112,7 +86,7 @@ impl RollingFi /// /// # Returns /// - /// A new `RollingFileWriter` instance with default target file size + /// A new `RollingFileWriterBuilder` instance with default target file size pub fn new_with_default_file_size( inner_builder: B, file_io: FileIO, @@ -120,16 +94,64 @@ impl RollingFi file_name_generator: F, ) -> Self { Self { - inner: None, inner_builder, target_file_size: PROPERTY_WRITE_TARGET_FILE_SIZE_BYTES_DEFAULT, - data_file_builders: vec![], file_io, location_generator, file_name_generator, } } + /// Build a new [`RollingFileWriter`]. + pub fn build(self) -> RollingFileWriter { + RollingFileWriter { + inner: None, + inner_builder: self.inner_builder, + target_file_size: self.target_file_size, + data_file_builders: vec![], + file_io: self.file_io, + location_generator: self.location_generator, + file_name_generator: self.file_name_generator, + } + } +} + +/// A writer that automatically rolls over to a new file when the data size +/// exceeds a target threshold. +/// +/// This writer wraps another file writer that tracks the amount of data written. +/// When the data size exceeds the target size, it closes the current file and +/// starts writing to a new one. +pub struct RollingFileWriter { + inner: Option, + inner_builder: B, + target_file_size: usize, + data_file_builders: Vec, + file_io: FileIO, + location_generator: L, + file_name_generator: F, +} + +impl Debug for RollingFileWriter +where + B: FileWriterBuilder, + L: LocationGenerator, + F: FileNameGenerator, +{ + fn fmt(&self, f: &mut Formatter<'_>) -> std::fmt::Result { + f.debug_struct("RollingFileWriter") + .field("target_file_size", &self.target_file_size) + .field("file_io", &self.file_io) + .finish() + } +} + +impl RollingFileWriter +where + B: FileWriterBuilder, + L: LocationGenerator, + F: FileNameGenerator, +{ /// Determines if the writer should roll over to a new file. /// /// # Returns @@ -299,7 +321,7 @@ mod tests { ParquetWriterBuilder::new(WriterProperties::builder().build(), Arc::new(schema)); // Set a large target size so no rolling occurs - let rolling_file_writer = RollingFileWriter::new( + let rolling_file_writer_builder = RollingFileWriterBuilder::new( parquet_writer_builder, 1024 * 1024, file_io.clone(), @@ -307,7 +329,8 @@ mod tests { file_name_gen, ); - let data_file_writer_builder = DataFileWriterBuilder::new(rolling_file_writer, None); + let data_file_writer_builder = + DataFileWriterBuilder::new(rolling_file_writer_builder, None); // Create writer let mut writer = data_file_writer_builder.build().await?; @@ -357,7 +380,7 @@ mod tests { ParquetWriterBuilder::new(WriterProperties::builder().build(), Arc::new(schema)); // Set a very small target size to trigger rolling - let rolling_writer_builder = RollingFileWriter::new( + let rolling_writer_builder = RollingFileWriterBuilder::new( parquet_writer_builder, 1024, file_io, diff --git a/crates/iceberg/src/writer/mod.rs b/crates/iceberg/src/writer/mod.rs index 5e1b91bfed..d5a8a66861 100644 --- a/crates/iceberg/src/writer/mod.rs +++ b/crates/iceberg/src/writer/mod.rs @@ -60,7 +60,9 @@ //! async fn main() -> Result<()> { //! // Connect to a catalog. //! use iceberg::memory::{MEMORY_CATALOG_WAREHOUSE, MemoryCatalogBuilder}; -//! use iceberg::writer::file_writer::rolling_writer::RollingFileWriter; +//! use iceberg::writer::file_writer::rolling_writer::{ +//! RollingFileWriter, RollingFileWriterBuilder, +//! }; //! let catalog = MemoryCatalogBuilder::default() //! .load( //! "memory", @@ -90,7 +92,7 @@ //! ); //! //! // Create a rolling file writer using parquet file writer builder. -//! let rolling_file_writer = RollingFileWriter::new_with_default_file_size( +//! let rolling_file_writer_builder = RollingFileWriterBuilder::new_with_default_file_size( //! parquet_writer_builder, //! table.file_io().clone(), //! location_generator.clone(), @@ -98,7 +100,8 @@ //! ); //! //! // Create a data file writer using parquet file writer builder. -//! let data_file_writer_builder = DataFileWriterBuilder::new(rolling_file_writer, None); +//! let data_file_writer_builder = +//! DataFileWriterBuilder::new(rolling_file_writer_builder, None); //! // Build the data file writer //! let mut data_file_writer = data_file_writer_builder.build().await?; //! @@ -180,7 +183,9 @@ //! // Connect to a catalog. //! use iceberg::memory::MEMORY_CATALOG_WAREHOUSE; //! use iceberg::spec::{Literal, PartitionKey, Struct}; -//! use iceberg::writer::file_writer::rolling_writer::RollingFileWriter; +//! use iceberg::writer::file_writer::rolling_writer::{ +//! RollingFileWriter, RollingFileWriterBuilder, +//! }; //! //! let catalog = MemoryCatalogBuilder::default() //! .load( @@ -217,7 +222,7 @@ //! ); //! //! // Create a rolling file writer -//! let rolling_file_writer = RollingFileWriter::new( +//! let rolling_file_writer_builder = RollingFileWriterBuilder::new( //! parquet_writer_builder, //! 512 * 1024 * 1024, //! table.file_io().clone(), @@ -227,7 +232,7 @@ //! //! // Create a data file writer builder using rolling file writer. //! let data_file_writer_builder = -//! DataFileWriterBuilder::new(rolling_file_writer, Some(partition_key)); +//! DataFileWriterBuilder::new(rolling_file_writer_builder, Some(partition_key)); //! // Create latency record writer using data file writer builder. //! let latency_record_builder = LatencyRecordWriterBuilder::new(data_file_writer_builder); //! // Build the final writer diff --git a/crates/integration_tests/tests/shared_tests/append_data_file_test.rs b/crates/integration_tests/tests/shared_tests/append_data_file_test.rs index 6c7b06740b..f4cba959ee 100644 --- a/crates/integration_tests/tests/shared_tests/append_data_file_test.rs +++ b/crates/integration_tests/tests/shared_tests/append_data_file_test.rs @@ -27,7 +27,7 @@ use iceberg::writer::file_writer::ParquetWriterBuilder; use iceberg::writer::file_writer::location_generator::{ DefaultFileNameGenerator, DefaultLocationGenerator, }; -use iceberg::writer::file_writer::rolling_writer::RollingFileWriter; +use iceberg::writer::file_writer::rolling_writer::RollingFileWriterBuilder; use iceberg::writer::{IcebergWriter, IcebergWriterBuilder}; use iceberg::{Catalog, CatalogBuilder, TableCreation}; use iceberg_catalog_rest::RestCatalogBuilder; @@ -76,13 +76,13 @@ async fn test_append_data_file() { WriterProperties::default(), table.metadata().current_schema().clone(), ); - let rolling_file_writer = RollingFileWriter::new_with_default_file_size( + let rolling_file_writer_builder = RollingFileWriterBuilder::new_with_default_file_size( parquet_writer_builder, table.file_io().clone(), location_generator.clone(), file_name_generator.clone(), ); - let data_file_writer_builder = DataFileWriterBuilder::new(rolling_file_writer, None); + let data_file_writer_builder = DataFileWriterBuilder::new(rolling_file_writer_builder, None); let mut data_file_writer = data_file_writer_builder.build().await.unwrap(); let col1 = StringArray::from(vec![Some("foo"), Some("bar"), None, Some("baz")]); let col2 = Int32Array::from(vec![Some(1), Some(2), Some(3), Some(4)]); diff --git a/crates/integration_tests/tests/shared_tests/append_partition_data_file_test.rs b/crates/integration_tests/tests/shared_tests/append_partition_data_file_test.rs index 204b6d6e39..0da88f1a09 100644 --- a/crates/integration_tests/tests/shared_tests/append_partition_data_file_test.rs +++ b/crates/integration_tests/tests/shared_tests/append_partition_data_file_test.rs @@ -31,7 +31,7 @@ use iceberg::writer::file_writer::ParquetWriterBuilder; use iceberg::writer::file_writer::location_generator::{ DefaultFileNameGenerator, DefaultLocationGenerator, }; -use iceberg::writer::file_writer::rolling_writer::RollingFileWriter; +use iceberg::writer::file_writer::rolling_writer::RollingFileWriterBuilder; use iceberg::writer::{IcebergWriter, IcebergWriterBuilder}; use iceberg::{Catalog, CatalogBuilder, TableCreation}; use iceberg_catalog_rest::RestCatalogBuilder; @@ -99,18 +99,20 @@ async fn test_append_partition_data_file() { table.metadata().current_schema().clone(), ); - let rolling_file_writer = RollingFileWriter::new_with_default_file_size( + let rolling_file_writer_builder = RollingFileWriterBuilder::new_with_default_file_size( parquet_writer_builder.clone(), table.file_io().clone(), location_generator.clone(), file_name_generator.clone(), ); - let mut data_file_writer_valid = - DataFileWriterBuilder::new(rolling_file_writer.clone(), Some(partition_key.clone())) - .build() - .await - .unwrap(); + let mut data_file_writer_valid = DataFileWriterBuilder::new( + rolling_file_writer_builder.clone(), + Some(partition_key.clone()), + ) + .build() + .await + .unwrap(); let col1 = StringArray::from(vec![Some("foo1"), Some("foo2")]); let col2 = Int32Array::from(vec![ @@ -151,7 +153,7 @@ async fn test_append_partition_data_file() { Literal::Primitive(PrimitiveLiteral::Boolean(true)), )])); test_schema_incompatible_partition_type( - rolling_file_writer.clone(), + rolling_file_writer_builder.clone(), batch.clone(), partition_key.clone(), table.clone(), @@ -168,7 +170,7 @@ async fn test_append_partition_data_file() { ))), ])); test_schema_incompatible_partition_fields( - rolling_file_writer.clone(), + rolling_file_writer_builder.clone(), batch, partition_key, table, @@ -178,7 +180,7 @@ async fn test_append_partition_data_file() { } async fn test_schema_incompatible_partition_type( - rolling_file_writer: RollingFileWriter< + rolling_file_writer_builder: RollingFileWriterBuilder< ParquetWriterBuilder, DefaultLocationGenerator, DefaultFileNameGenerator, @@ -190,7 +192,7 @@ async fn test_schema_incompatible_partition_type( ) { // test writing different "type" of partition than mentioned in schema let mut data_file_writer_invalid = - DataFileWriterBuilder::new(rolling_file_writer.clone(), Some(partition_key)) + DataFileWriterBuilder::new(rolling_file_writer_builder, Some(partition_key)) .build() .await .unwrap(); @@ -208,7 +210,7 @@ async fn test_schema_incompatible_partition_type( } async fn test_schema_incompatible_partition_fields( - rolling_file_writer: RollingFileWriter< + rolling_file_writer_builder: RollingFileWriterBuilder< ParquetWriterBuilder, DefaultLocationGenerator, DefaultFileNameGenerator, @@ -220,7 +222,7 @@ async fn test_schema_incompatible_partition_fields( ) { // test writing different number of partition fields than mentioned in schema let mut data_file_writer_invalid = - DataFileWriterBuilder::new(rolling_file_writer, Some(partition_key)) + DataFileWriterBuilder::new(rolling_file_writer_builder, Some(partition_key)) .build() .await .unwrap(); diff --git a/crates/integration_tests/tests/shared_tests/conflict_commit_test.rs b/crates/integration_tests/tests/shared_tests/conflict_commit_test.rs index 8afaaac76f..a248fa707a 100644 --- a/crates/integration_tests/tests/shared_tests/conflict_commit_test.rs +++ b/crates/integration_tests/tests/shared_tests/conflict_commit_test.rs @@ -27,7 +27,7 @@ use iceberg::writer::file_writer::ParquetWriterBuilder; use iceberg::writer::file_writer::location_generator::{ DefaultFileNameGenerator, DefaultLocationGenerator, }; -use iceberg::writer::file_writer::rolling_writer::RollingFileWriter; +use iceberg::writer::file_writer::rolling_writer::RollingFileWriterBuilder; use iceberg::writer::{IcebergWriter, IcebergWriterBuilder}; use iceberg::{Catalog, CatalogBuilder, TableCreation}; use iceberg_catalog_rest::RestCatalogBuilder; @@ -75,13 +75,13 @@ async fn test_append_data_file_conflict() { WriterProperties::default(), table.metadata().current_schema().clone(), ); - let rolling_file_writer = RollingFileWriter::new_with_default_file_size( + let rolling_file_writer_builder = RollingFileWriterBuilder::new_with_default_file_size( parquet_writer_builder, table.file_io().clone(), location_generator.clone(), file_name_generator.clone(), ); - let data_file_writer_builder = DataFileWriterBuilder::new(rolling_file_writer, None); + let data_file_writer_builder = DataFileWriterBuilder::new(rolling_file_writer_builder, None); let mut data_file_writer = data_file_writer_builder.build().await.unwrap(); let col1 = StringArray::from(vec![Some("foo"), Some("bar"), None, Some("baz")]); let col2 = Int32Array::from(vec![Some(1), Some(2), Some(3), Some(4)]); diff --git a/crates/integration_tests/tests/shared_tests/scan_all_type.rs b/crates/integration_tests/tests/shared_tests/scan_all_type.rs index 36e7367fa4..1125de11a5 100644 --- a/crates/integration_tests/tests/shared_tests/scan_all_type.rs +++ b/crates/integration_tests/tests/shared_tests/scan_all_type.rs @@ -39,7 +39,7 @@ use iceberg::writer::file_writer::ParquetWriterBuilder; use iceberg::writer::file_writer::location_generator::{ DefaultFileNameGenerator, DefaultLocationGenerator, }; -use iceberg::writer::file_writer::rolling_writer::RollingFileWriter; +use iceberg::writer::file_writer::rolling_writer::RollingFileWriterBuilder; use iceberg::writer::{IcebergWriter, IcebergWriterBuilder}; use iceberg::{Catalog, CatalogBuilder, TableCreation}; use iceberg_catalog_rest::RestCatalogBuilder; @@ -157,13 +157,13 @@ async fn test_scan_all_type() { WriterProperties::default(), table.metadata().current_schema().clone(), ); - let rolling_file_writer = RollingFileWriter::new_with_default_file_size( + let rolling_file_writer_builder = RollingFileWriterBuilder::new_with_default_file_size( parquet_writer_builder, table.file_io().clone(), location_generator.clone(), file_name_generator.clone(), ); - let data_file_writer_builder = DataFileWriterBuilder::new(rolling_file_writer, None); + let data_file_writer_builder = DataFileWriterBuilder::new(rolling_file_writer_builder, None); let mut data_file_writer = data_file_writer_builder.build().await.unwrap(); // Prepare data diff --git a/crates/integrations/datafusion/src/physical_plan/write.rs b/crates/integrations/datafusion/src/physical_plan/write.rs index e095060bdc..712da92b21 100644 --- a/crates/integrations/datafusion/src/physical_plan/write.rs +++ b/crates/integrations/datafusion/src/physical_plan/write.rs @@ -47,7 +47,7 @@ use iceberg::writer::file_writer::ParquetWriterBuilder; use iceberg::writer::file_writer::location_generator::{ DefaultFileNameGenerator, DefaultLocationGenerator, }; -use iceberg::writer::file_writer::rolling_writer::RollingFileWriter; +use iceberg::writer::file_writer::rolling_writer::RollingFileWriterBuilder; use iceberg::writer::{IcebergWriter, IcebergWriterBuilder}; use iceberg::{Error, ErrorKind}; use parquet::file::properties::WriterProperties; @@ -261,7 +261,7 @@ impl ExecutionPlan for IcebergWriteExec { // todo filename prefix/suffix should be configurable let file_name_generator = DefaultFileNameGenerator::new(Uuid::now_v7().to_string(), None, file_format); - let rolling_writer = RollingFileWriter::new( + let rolling_writer_builder = RollingFileWriterBuilder::new( parquet_file_writer_builder, target_file_size, file_io, @@ -269,7 +269,7 @@ impl ExecutionPlan for IcebergWriteExec { file_name_generator, ); // todo specify partition key when partitioning writer is supported - let data_file_writer_builder = DataFileWriterBuilder::new(rolling_writer, None); + let data_file_writer_builder = DataFileWriterBuilder::new(rolling_writer_builder, None); // Get input data let data = execute_input_stream(