use std::marker::PhantomData;
use crate::Result;
use crate::writer::{DefaultInput, DefaultOutput, IcebergWriter, IcebergWriterBuilder};
pub struct UnpartitionedWriter<B, I = DefaultInput, O = DefaultOutput>
where
B: IcebergWriterBuilder<I, O>,
O: IntoIterator + FromIterator<<O as IntoIterator>::Item>,
<O as IntoIterator>::Item: Clone,
{
inner_builder: B,
writer: Option<B::R>,
output: Vec<<O as IntoIterator>::Item>,
_phantom: PhantomData<I>,
}
impl<B, I, O> UnpartitionedWriter<B, I, O>
where
B: IcebergWriterBuilder<I, O>,
I: Send + 'static,
O: IntoIterator + FromIterator<<O as IntoIterator>::Item>,
<O as IntoIterator>::Item: Send + Clone,
{
pub fn new(inner_builder: B) -> Self {
Self {
inner_builder,
writer: None,
output: Vec::new(),
_phantom: PhantomData,
}
}
pub async fn write(&mut self, input: I) -> Result<()> {
if self.writer.is_none() {
self.writer = Some(self.inner_builder.build(None).await?);
}
self.writer
.as_mut()
.expect("Writer should be initialized")
.write(input)
.await
}
pub async fn close(mut self) -> Result<O> {
if let Some(mut writer) = self.writer.take() {
self.output.extend(writer.close().await?);
}
Ok(O::from_iter(self.output))
}
}
#[cfg(test)]
mod tests {
use std::collections::HashMap;
use std::sync::Arc;
use arrow_array::{Int32Array, RecordBatch, StringArray};
use arrow_schema::{DataType, Field, Schema};
use parquet::arrow::PARQUET_FIELD_ID_META_KEY;
use parquet::file::properties::WriterProperties;
use tempfile::TempDir;
use super::*;
use crate::Result;
use crate::io::FileIO;
use crate::spec::{DataFileFormat, 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, DefaultLocationGenerator,
};
use crate::writer::file_writer::rolling_writer::RollingFileWriterBuilder;
#[tokio::test]
async fn test_unpartitioned_writer() -> Result<()> {
let temp_dir = TempDir::new()?;
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(),
])
.build()?,
);
let arrow_schema = Arc::new(Schema::new(vec![
Field::new("id", DataType::Int32, false).with_metadata(HashMap::from([(
PARQUET_FIELD_ID_META_KEY.to_string(),
"1".to_string(),
)])),
Field::new("name", DataType::Utf8, false).with_metadata(HashMap::from([(
PARQUET_FIELD_ID_META_KEY.to_string(),
"2".to_string(),
)])),
]));
let file_io = FileIO::new_with_fs();
let location_gen = DefaultLocationGenerator::with_data_location(
temp_dir.path().to_str().unwrap().to_string(),
);
let file_name_gen =
DefaultFileNameGenerator::new("test".to_string(), None, DataFileFormat::Parquet);
let parquet_writer_builder =
ParquetWriterBuilder::new(WriterProperties::builder().build(), schema.clone());
let rolling_writer_builder = RollingFileWriterBuilder::new_with_default_file_size(
parquet_writer_builder,
file_io,
location_gen,
file_name_gen,
);
let writer_builder = DataFileWriterBuilder::new(rolling_writer_builder);
let mut writer = UnpartitionedWriter::new(writer_builder);
let batch1 = RecordBatch::try_new(arrow_schema.clone(), vec![
Arc::new(Int32Array::from(vec![1, 2])),
Arc::new(StringArray::from(vec!["Alice", "Bob"])),
])?;
let batch2 = RecordBatch::try_new(arrow_schema, vec![
Arc::new(Int32Array::from(vec![3, 4])),
Arc::new(StringArray::from(vec!["Charlie", "Dave"])),
])?;
writer.write(batch1).await?;
writer.write(batch2).await?;
let data_files = writer.close().await?;
assert!(!data_files.is_empty());
for file in &data_files {
assert_eq!(file.partition, Struct::empty());
assert_eq!(file.file_format, DataFileFormat::Parquet);
assert_eq!(file.record_count, 4);
}
Ok(())
}
}