From 570085576aa27d2a594bb1a66ffb8ffd541c1407 Mon Sep 17 00:00:00 2001 From: feniljain Date: Thu, 23 Jan 2025 00:45:42 +0530 Subject: [PATCH 01/14] feat: nan_value_counts support --- .../src/writer/file_writer/parquet_writer.rs | 83 ++++++++++++++++--- 1 file changed, 73 insertions(+), 10 deletions(-) diff --git a/crates/iceberg/src/writer/file_writer/parquet_writer.rs b/crates/iceberg/src/writer/file_writer/parquet_writer.rs index bed9cc3ddc..181d203b6f 100644 --- a/crates/iceberg/src/writer/file_writer/parquet_writer.rs +++ b/crates/iceberg/src/writer/file_writer/parquet_writer.rs @@ -17,11 +17,13 @@ //! The module contains the file writer for parquet file format. +use std::collections::hash_map::Entry; use std::collections::HashMap; use std::sync::atomic::AtomicI64; use std::sync::Arc; -use arrow_schema::SchemaRef as ArrowSchemaRef; +use arrow_array::Float32Array; +use arrow_schema::{DataType, SchemaRef as ArrowSchemaRef}; use bytes::Bytes; use futures::future::BoxFuture; use itertools::Itertools; @@ -97,6 +99,7 @@ impl FileWriterBuilder for ParquetWr written_size, current_row_num: 0, out_file, + nan_value_counts: HashMap::new(), }) } } @@ -222,6 +225,7 @@ pub struct ParquetWriter { writer_properties: WriterProperties, written_size: Arc, current_row_num: usize, + nan_value_counts: HashMap, } /// Used to aggregate min and max value of each column. @@ -357,6 +361,7 @@ impl ParquetWriter { metadata: FileMetaData, written_size: usize, file_path: String, + nan_value_counts: HashMap, ) -> Result { let index_by_parquet_path = { let mut visitor = IndexByParquetPathName::new(); @@ -423,8 +428,8 @@ impl ParquetWriter { .null_value_counts(null_value_counts) .lower_bounds(lower_bounds) .upper_bounds(upper_bounds) + .nan_value_counts(nan_value_counts) // # TODO(#417) - // - nan_value_counts // - distinct_counts .key_metadata(metadata.footer_signing_key_metadata) .split_offsets( @@ -541,6 +546,37 @@ impl FileWriter for ParquetWriter { self.inner_writer.as_mut().unwrap() }; + + for (col, field) in batch + .columns() + .iter() + .zip(self.schema.as_struct().fields().iter()) + { + let dt = col.data_type(); + + let nan_val_cnt: u64 = match dt { + DataType::Float32 => { + let float_array = col.as_any().downcast_ref::().unwrap(); + + float_array + .iter() + .filter(|value| value.map_or(false, |v| v.is_nan())) + .count() as u64 + } + _ => 0, + }; + + match self.nan_value_counts.entry(field.id) { + Entry::Occupied(mut ele) => { + let total_nan_val_cnt = ele.get() + nan_val_cnt; + ele.insert(total_nan_val_cnt); + } + Entry::Vacant(v) => { + v.insert(nan_val_cnt); + } + } + } + writer.write(batch).await.map_err(|err| { Error::new( ErrorKind::Unexpected, @@ -548,6 +584,7 @@ impl FileWriter for ParquetWriter { ) .with_source(err) })?; + Ok(()) } @@ -566,6 +603,7 @@ impl FileWriter for ParquetWriter { metadata, written_size as usize, self.out_file.location().to_string(), + self.nan_value_counts, )?]) } } @@ -626,8 +664,8 @@ mod tests { use anyhow::Result; use arrow_array::types::Int64Type; use arrow_array::{ - Array, ArrayRef, BooleanArray, Decimal128Array, Int32Array, Int64Array, ListArray, - RecordBatch, StructArray, + Array, ArrayRef, BooleanArray, Decimal128Array, Float32Array, Int32Array, Int64Array, + ListArray, RecordBatch, StructArray, }; use arrow_schema::{DataType, SchemaRef as ArrowSchemaRef}; use arrow_select::concat::concat_batches; @@ -807,13 +845,27 @@ mod tests { arrow_schema::Field::new("col", arrow_schema::DataType::Int64, true).with_metadata( HashMap::from([(PARQUET_FIELD_ID_META_KEY.to_string(), "0".to_string())]), ), + arrow_schema::Field::new("col1", arrow_schema::DataType::Float32, true) + .with_metadata(HashMap::from([( + PARQUET_FIELD_ID_META_KEY.to_string(), + "1".to_string(), + )])), ]; Arc::new(arrow_schema::Schema::new(fields)) }; let col = Arc::new(Int64Array::from_iter_values(0..1024)) as ArrayRef; let null_col = Arc::new(Int64Array::new_null(1024)) as ArrayRef; - 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 float_col = Arc::new(Float32Array::from_iter_values((0..1024).map(|x| { + if x % 100 == 0 { + // There will be 11 NANs as there are 1024 entries + f32::NAN + } else { + x as f32 + } + }))) as ArrayRef; + let to_write = RecordBatch::try_new(schema.clone(), vec![col, float_col.clone()]).unwrap(); + let to_write_null = + RecordBatch::try_new(schema.clone(), vec![null_col, float_col]).unwrap(); // write data let mut pw = ParquetWriterBuilder::new( @@ -825,6 +877,7 @@ mod tests { ) .build() .await?; + pw.write(&to_write).await?; pw.write(&to_write_null).await?; let res = pw.close().await?; @@ -841,16 +894,26 @@ mod tests { // check data file assert_eq!(data_file.record_count(), 2048); - assert_eq!(*data_file.value_counts(), HashMap::from([(0, 2048)])); + assert_eq!( + *data_file.value_counts(), + HashMap::from([(0, 2048), (1, 2048)]) + ); assert_eq!( *data_file.lower_bounds(), - HashMap::from([(0, Datum::long(0))]) + HashMap::from([(0, Datum::long(0)), (1, Datum::float(1.0))]) ); assert_eq!( *data_file.upper_bounds(), - HashMap::from([(0, Datum::long(1023))]) + HashMap::from([(0, Datum::long(1023)), (1, Datum::float(1023.0))]) + ); + assert_eq!( + *data_file.null_value_counts(), + HashMap::from([(0, 1024), (1, 0)]) + ); + assert_eq!( + *data_file.nan_value_counts(), + HashMap::from([(0, 0), (1, 22)]) // 22, cause we wrote float column twice ); - assert_eq!(*data_file.null_value_counts(), HashMap::from([(0, 1024)])); // check the written file let expect_batch = concat_batches(&schema, vec![&to_write, &to_write_null]).unwrap(); From c232b83aba4cd62e4da344a485cac9dbb072c095 Mon Sep 17 00:00:00 2001 From: feniljain Date: Mon, 27 Jan 2025 19:26:44 +0000 Subject: [PATCH 02/14] feat: add float64 nan value counts support --- .../src/writer/file_writer/parquet_writer.rs | 107 +++++++++++++++++- 1 file changed, 106 insertions(+), 1 deletion(-) diff --git a/crates/iceberg/src/writer/file_writer/parquet_writer.rs b/crates/iceberg/src/writer/file_writer/parquet_writer.rs index 181d203b6f..d1f1e5a23d 100644 --- a/crates/iceberg/src/writer/file_writer/parquet_writer.rs +++ b/crates/iceberg/src/writer/file_writer/parquet_writer.rs @@ -22,7 +22,7 @@ use std::collections::HashMap; use std::sync::atomic::AtomicI64; use std::sync::Arc; -use arrow_array::Float32Array; +use arrow_array::{Float32Array, Float64Array}; use arrow_schema::{DataType, SchemaRef as ArrowSchemaRef}; use bytes::Bytes; use futures::future::BoxFuture; @@ -563,6 +563,14 @@ impl FileWriter for ParquetWriter { .filter(|value| value.map_or(false, |v| v.is_nan())) .count() as u64 } + DataType::Float64 => { + let float_array = col.as_any().downcast_ref::().unwrap(); + + float_array + .iter() + .filter(|value| value.map_or(false, |v| v.is_nan())) + .count() as u64 + } _ => 0, }; @@ -830,6 +838,7 @@ mod tests { assert_eq!(visitor.name_to_id, expect); } + // TODO(feniljain): Remove nan value count test from here #[tokio::test] async fn test_parquet_writer() -> Result<()> { let temp_dir = TempDir::new().unwrap(); @@ -922,6 +931,102 @@ mod tests { Ok(()) } + #[tokio::test] + async fn test_parquet_writer_for_nan_value_counts() -> Result<()> { + let temp_dir = TempDir::new().unwrap(); + let file_io = FileIOBuilder::new_fs_io().build().unwrap(); + 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); + + // prepare data + let schema = { + let fields = vec![ + // TODO(feniljain): + // Types: + // [X] Primitive + // [ ] Struct + // [ ] List + // [ ] Map + arrow_schema::Field::new("col", arrow_schema::DataType::Float32, true) + .with_metadata(HashMap::from([( + PARQUET_FIELD_ID_META_KEY.to_string(), + "0".to_string(), + )])), + arrow_schema::Field::new("col1", arrow_schema::DataType::Float64, true) + .with_metadata(HashMap::from([( + PARQUET_FIELD_ID_META_KEY.to_string(), + "1".to_string(), + )])), + ]; + Arc::new(arrow_schema::Schema::new(fields)) + }; + + let float_32_col = Arc::new(Float32Array::from_iter_values_with_nulls( + [1.0_f32, f32::NAN, 2.0, 2.0].into_iter(), + None, + )) as ArrayRef; + + let float_64_col = Arc::new(Float64Array::from_iter_values_with_nulls( + [1.0_f64, f64::NAN, 2.0, 2.0].into_iter(), + None, + )) as ArrayRef; + + let to_write = + RecordBatch::try_new(schema.clone(), vec![float_32_col, float_64_col]).unwrap(); + + // write data + let mut pw = ParquetWriterBuilder::new( + WriterProperties::builder().build(), + Arc::new(to_write.schema().as_ref().try_into().unwrap()), + file_io.clone(), + location_gen, + file_name_gen, + ) + .build() + .await?; + + pw.write(&to_write).await?; + let res = pw.close().await?; + assert_eq!(res.len(), 1); + let data_file = res + .into_iter() + .next() + .unwrap() + // Put dummy field for build successfully. + .content(crate::spec::DataContentType::Data) + .partition(Struct::empty()) + .build() + .unwrap(); + + // check data file + assert_eq!(data_file.record_count(), 4); + assert_eq!(*data_file.value_counts(), HashMap::from([(0, 4), (1, 4)])); + assert_eq!( + *data_file.lower_bounds(), + HashMap::from([(0, Datum::float(1.0)), (1, Datum::double(1.0))]) + ); + assert_eq!( + *data_file.upper_bounds(), + HashMap::from([(0, Datum::float(2.0)), (1, Datum::double(2.0))]) + ); + assert_eq!( + *data_file.null_value_counts(), + HashMap::from([(0, 0), (1, 0)]) + ); + assert_eq!( + *data_file.nan_value_counts(), + HashMap::from([(0, 1), (1, 1)]) + ); + + // check the written file + let expect_batch = concat_batches(&schema, vec![&to_write]).unwrap(); + check_parquet_data_file(&file_io, &data_file, &expect_batch).await; + + Ok(()) + } + #[tokio::test] async fn test_parquet_writer_with_complex_schema() -> Result<()> { let temp_dir = TempDir::new().unwrap(); From 0a6367810d41e436d76e7b66ab5f10aa3fa44c52 Mon Sep 17 00:00:00 2001 From: feniljain Date: Sat, 8 Mar 2025 12:11:45 +0000 Subject: [PATCH 03/14] feat: nan val cnt for nested types --- .../src/writer/file_writer/parquet_writer.rs | 803 +++++++++++++++--- 1 file changed, 706 insertions(+), 97 deletions(-) diff --git a/crates/iceberg/src/writer/file_writer/parquet_writer.rs b/crates/iceberg/src/writer/file_writer/parquet_writer.rs index d1f1e5a23d..6b3a732346 100644 --- a/crates/iceberg/src/writer/file_writer/parquet_writer.rs +++ b/crates/iceberg/src/writer/file_writer/parquet_writer.rs @@ -21,8 +21,9 @@ use std::collections::hash_map::Entry; use std::collections::HashMap; use std::sync::atomic::AtomicI64; use std::sync::Arc; +use std::ops::Deref; -use arrow_array::{Float32Array, Float64Array}; +use arrow_array::{Float32Array, Float64Array, ArrayRef, StructArray, ListArray, MapArray}; use arrow_schema::{DataType, SchemaRef as ArrowSchemaRef}; use bytes::Bytes; use futures::future::BoxFuture; @@ -319,6 +320,75 @@ impl MinMaxColAggregator { } } +macro_rules! count_float_nans { + ($t:ty, $col:ident, $self:ident, $field_id:ident) => { + let nan_val_cnt = $col + .as_any() + .downcast_ref::<$t>() + .unwrap() + .iter() + .filter(|value| value.map_or(false, |v| v.is_nan())) + .count() as u64; + + match $self.nan_value_counts.entry($field_id) { + Entry::Occupied(mut ele) => { + let total_nan_val_cnt = ele.get() + nan_val_cnt; + ele.insert(total_nan_val_cnt); + } + Entry::Vacant(v) => { + v.insert(nan_val_cnt); + } + }; + }; +} + +macro_rules! handle_list_type { + ($t:ty, $col:ident, $self:ident, $field:ident, $arrow_field:ident) => { + let list_arr = $col.as_any().downcast_ref::<$t>().unwrap(); + let field_data_typ = $arrow_field.data_type(); + + let n_vals = list_arr.offsets().len() - 1; + + let field = match $field.clone().field_type.deref() { + Type::List(list_typ) => list_typ.element_field.clone(), + _ => unreachable!(), + }; + let field_id = field.id; + + match field_data_typ { + DataType::Float32 => { + for idx in 0..n_vals { + let arr_ref = list_arr.value(idx); + count_float_nans!(Float32Array, arr_ref, $self, field_id); + } + } + DataType::Float64 => { + for idx in 0..n_vals { + let arr_ref = list_arr.value(idx); + count_float_nans!(Float64Array, arr_ref, $self, field_id); + } + } + DataType::List(_) + | DataType::LargeList(_) + | DataType::FixedSizeList(_, _) + | DataType::Struct(_) + | DataType::Map(_, _) => { + for idx in 0..n_vals { + let arr_ref = list_arr.value(idx); + + let field = match field.clone().field_type.deref() { + Type::List(list_ty) => list_ty.element_field.clone(), + _ => unreachable!(), + }; + + $self.transverse_batch(&arr_ref, &field); + } + } + _ => {} + }; + }; +} + impl ParquetWriter { /// Converts parquet files to data files #[allow(dead_code)] @@ -514,6 +584,89 @@ impl ParquetWriter { Ok(builder) } + + fn transverse_batch(&mut self, col: &ArrayRef, field: &NestedFieldRef) { + let dt = col.data_type(); + + match dt { + DataType::Float32 => { + let field_id: i32 = field.id; + count_float_nans!(Float32Array, col, self, field_id); + } + DataType::Float64 => { + let field_id: i32 = field.id; + count_float_nans!(Float64Array, col, self, field_id); + } + DataType::Struct(fields) => { + let struct_arr = col.as_any().downcast_ref::().unwrap(); + for (idx, arrow_field) in fields.iter().enumerate() { + match arrow_field.data_type() { + DataType::Float32 => { + let float_arr_ref = struct_arr.column(idx); + + let field_id = match field.clone().field_type.deref() { + Type::Struct(struct_ty) => struct_ty.fields()[idx].id, + _ => unreachable!(), + }; + + count_float_nans!(Float32Array, float_arr_ref, self, field_id); + } + DataType::Float64 => { + let float_arr_ref = struct_arr.column(idx); + + let field_id = match field.clone().field_type.deref() { + Type::Struct(struct_ty) => struct_ty.fields()[idx].id, + _ => unreachable!(), + }; + + count_float_nans!(Float64Array, float_arr_ref, self, field_id); + } + DataType::List(_) + | DataType::LargeList(_) + | DataType::FixedSizeList(_, _) + | DataType::Struct(_) + | DataType::Map(_, _) => { + let arr_ref = struct_arr.column(idx); + + let field = match field.clone().field_type.deref() { + Type::Struct(struct_ty) => struct_ty.fields()[idx].clone(), + _ => unreachable!(), + }; + + self.transverse_batch(arr_ref, &field); + } + _ => {} + }; + } + } + DataType::List(arrow_field) => { + handle_list_type!(ListArray, col, self, field, arrow_field); + } + // NOTE: iceberg to arrow schema conversion does not form these types, + // meaning these branches never get called right now. + DataType::LargeList(_) => { + // handle_list_type!(LargeListArray, col, self, field, arrow_field); + } + DataType::FixedSizeList(_, _) => { + // handle_list_type!(FixedSizeList, col, self, field, arrow_field); + } + DataType::Map(_, _) => { + let map_arr = col.as_any().downcast_ref::().unwrap(); + + let map_ty = match field.clone().field_type.deref() { + Type::Map(map_ty) => map_ty.clone(), + _ => unreachable!(), + }; + + let keys_col = map_arr.keys(); + self.transverse_batch(keys_col, &map_ty.key_field); + + let values_col = map_arr.values(); + self.transverse_batch(values_col, &map_ty.value_field); + } + _ => {} + }; + } } impl FileWriter for ParquetWriter { @@ -525,6 +678,13 @@ impl FileWriter for ParquetWriter { self.current_row_num += batch.num_rows(); + let schema_c = self.schema.clone(); + let fields = schema_c.as_struct().fields(); + + for (col, field) in batch.columns().iter().zip(fields) { + self.transverse_batch(col, field); + } + // Lazy initialize the writer let writer = if let Some(writer) = &mut self.inner_writer { writer @@ -546,45 +706,6 @@ impl FileWriter for ParquetWriter { self.inner_writer.as_mut().unwrap() }; - - for (col, field) in batch - .columns() - .iter() - .zip(self.schema.as_struct().fields().iter()) - { - let dt = col.data_type(); - - let nan_val_cnt: u64 = match dt { - DataType::Float32 => { - let float_array = col.as_any().downcast_ref::().unwrap(); - - float_array - .iter() - .filter(|value| value.map_or(false, |v| v.is_nan())) - .count() as u64 - } - DataType::Float64 => { - let float_array = col.as_any().downcast_ref::().unwrap(); - - float_array - .iter() - .filter(|value| value.map_or(false, |v| v.is_nan())) - .count() as u64 - } - _ => 0, - }; - - match self.nan_value_counts.entry(field.id) { - Entry::Occupied(mut ele) => { - let total_nan_val_cnt = ele.get() + nan_val_cnt; - ele.insert(total_nan_val_cnt); - } - Entry::Vacant(v) => { - v.insert(nan_val_cnt); - } - } - } - writer.write(batch).await.map_err(|err| { Error::new( ErrorKind::Unexpected, @@ -670,12 +791,13 @@ mod tests { use std::sync::Arc; use anyhow::Result; - use arrow_array::types::Int64Type; + use arrow_array::types::{Int64Type, Float32Type}; use arrow_array::{ Array, ArrayRef, BooleanArray, Decimal128Array, Float32Array, Int32Array, Int64Array, ListArray, RecordBatch, StructArray, }; - use arrow_schema::{DataType, SchemaRef as ArrowSchemaRef}; + use arrow_array::builder::{MapBuilder, Int32Builder, Float32Builder}; + use arrow_schema::{DataType, Field, Fields, SchemaRef as ArrowSchemaRef}; use arrow_select::concat::concat_batches; use parquet::arrow::PARQUET_FIELD_ID_META_KEY; use rust_decimal::Decimal; @@ -838,9 +960,8 @@ mod tests { assert_eq!(visitor.name_to_id, expect); } - // TODO(feniljain): Remove nan value count test from here #[tokio::test] - async fn test_parquet_writer() -> Result<()> { + async fn test_parquet_writer_for_nan_value_counts() -> Result<()> { let temp_dir = TempDir::new().unwrap(); let file_io = FileIOBuilder::new_fs_io().build().unwrap(); let location_gen = @@ -851,10 +972,18 @@ 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())]), - ), - arrow_schema::Field::new("col1", arrow_schema::DataType::Float32, true) + // TODO(feniljain): + // Types: + // [X] Primitive + // [ ] Struct + // [ ] List + // [ ] Map + arrow_schema::Field::new("col", arrow_schema::DataType::Float32, true) + .with_metadata(HashMap::from([( + PARQUET_FIELD_ID_META_KEY.to_string(), + "0".to_string(), + )])), + arrow_schema::Field::new("col1", arrow_schema::DataType::Float64, true) .with_metadata(HashMap::from([( PARQUET_FIELD_ID_META_KEY.to_string(), "1".to_string(), @@ -862,19 +991,19 @@ mod tests { ]; Arc::new(arrow_schema::Schema::new(fields)) }; - let col = Arc::new(Int64Array::from_iter_values(0..1024)) as ArrayRef; - let null_col = Arc::new(Int64Array::new_null(1024)) as ArrayRef; - let float_col = Arc::new(Float32Array::from_iter_values((0..1024).map(|x| { - if x % 100 == 0 { - // There will be 11 NANs as there are 1024 entries - f32::NAN - } else { - x as f32 - } - }))) as ArrayRef; - let to_write = RecordBatch::try_new(schema.clone(), vec![col, float_col.clone()]).unwrap(); - let to_write_null = - RecordBatch::try_new(schema.clone(), vec![null_col, float_col]).unwrap(); + + let float_32_col = Arc::new(Float32Array::from_iter_values_with_nulls( + [1.0_f32, f32::NAN, 2.0, 2.0].into_iter(), + None, + )) as ArrayRef; + + let float_64_col = Arc::new(Float64Array::from_iter_values_with_nulls( + [1.0_f64, f64::NAN, 2.0, 2.0].into_iter(), + None, + )) as ArrayRef; + + let to_write = + RecordBatch::try_new(schema.clone(), vec![float_32_col, float_64_col]).unwrap(); // write data let mut pw = ParquetWriterBuilder::new( @@ -888,7 +1017,6 @@ mod tests { .await?; pw.write(&to_write).await?; - pw.write(&to_write_null).await?; let res = pw.close().await?; assert_eq!(res.len(), 1); let data_file = res @@ -902,63 +1030,50 @@ mod tests { .unwrap(); // check data file - assert_eq!(data_file.record_count(), 2048); - assert_eq!( - *data_file.value_counts(), - HashMap::from([(0, 2048), (1, 2048)]) - ); + assert_eq!(data_file.record_count(), 4); + assert_eq!(*data_file.value_counts(), HashMap::from([(0, 4), (1, 4)])); assert_eq!( *data_file.lower_bounds(), - HashMap::from([(0, Datum::long(0)), (1, Datum::float(1.0))]) + HashMap::from([(0, Datum::float(1.0)), (1, Datum::double(1.0))]) ); assert_eq!( *data_file.upper_bounds(), - HashMap::from([(0, Datum::long(1023)), (1, Datum::float(1023.0))]) + HashMap::from([(0, Datum::float(2.0)), (1, Datum::double(2.0))]) ); assert_eq!( *data_file.null_value_counts(), - HashMap::from([(0, 1024), (1, 0)]) + HashMap::from([(0, 0), (1, 0)]) ); assert_eq!( *data_file.nan_value_counts(), - HashMap::from([(0, 0), (1, 22)]) // 22, cause we wrote float column twice + HashMap::from([(0, 1), (1, 1)]) ); // check the written file - let expect_batch = concat_batches(&schema, vec![&to_write, &to_write_null]).unwrap(); + let expect_batch = concat_batches(&schema, vec![&to_write]).unwrap(); check_parquet_data_file(&file_io, &data_file, &expect_batch).await; Ok(()) } #[tokio::test] - async fn test_parquet_writer_for_nan_value_counts() -> Result<()> { + async fn test_nan_val_cnts_primitive_type() -> Result<()> { let temp_dir = TempDir::new().unwrap(); let file_io = FileIOBuilder::new_fs_io().build().unwrap(); 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); - + // // prepare data - let schema = { + let arrow_schema = { let fields = vec![ - // TODO(feniljain): - // Types: - // [X] Primitive - // [ ] Struct - // [ ] List - // [ ] Map - arrow_schema::Field::new("col", arrow_schema::DataType::Float32, true) - .with_metadata(HashMap::from([( - PARQUET_FIELD_ID_META_KEY.to_string(), - "0".to_string(), - )])), - arrow_schema::Field::new("col1", arrow_schema::DataType::Float64, true) - .with_metadata(HashMap::from([( - PARQUET_FIELD_ID_META_KEY.to_string(), - "1".to_string(), - )])), + Field::new("col", arrow_schema::DataType::Float32, false).with_metadata( + HashMap::from([(PARQUET_FIELD_ID_META_KEY.to_string(), "0".to_string())]), + ), + Field::new("col2", arrow_schema::DataType::Float64, false).with_metadata( + HashMap::from([(PARQUET_FIELD_ID_META_KEY.to_string(), "1".to_string())]), + ), ]; Arc::new(arrow_schema::Schema::new(fields)) }; @@ -973,8 +1088,11 @@ mod tests { None, )) as ArrayRef; - let to_write = - RecordBatch::try_new(schema.clone(), vec![float_32_col, float_64_col]).unwrap(); + let to_write = RecordBatch::try_new(arrow_schema.clone(), vec![ + float_32_col, + float_64_col, + ]) + .unwrap(); // write data let mut pw = ParquetWriterBuilder::new( @@ -1002,14 +1120,23 @@ mod tests { // check data file assert_eq!(data_file.record_count(), 4); - assert_eq!(*data_file.value_counts(), HashMap::from([(0, 4), (1, 4)])); + assert_eq!( + *data_file.value_counts(), + HashMap::from([(0, 4), (1, 4)]) + ); assert_eq!( *data_file.lower_bounds(), - HashMap::from([(0, Datum::float(1.0)), (1, Datum::double(1.0))]) + HashMap::from([ + (0, Datum::float(1.0)), + (1, Datum::double(1.0)), + ]) ); assert_eq!( *data_file.upper_bounds(), - HashMap::from([(0, Datum::float(2.0)), (1, Datum::double(2.0))]) + HashMap::from([ + (0, Datum::float(2.0)), + (1, Datum::double(2.0)), + ]) ); assert_eq!( *data_file.null_value_counts(), @@ -1021,7 +1148,489 @@ mod tests { ); // check the written file - let expect_batch = concat_batches(&schema, vec![&to_write]).unwrap(); + let expect_batch = concat_batches(&arrow_schema, vec![&to_write]).unwrap(); + check_parquet_data_file(&file_io, &data_file, &expect_batch).await; + + Ok(()) + } + + #[tokio::test] + async fn test_nan_val_cnts_struct_type() -> Result<()> { + let temp_dir = TempDir::new().unwrap(); + let file_io = FileIOBuilder::new_fs_io().build().unwrap(); + 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); + + let schema_struct_float_fields = + Fields::from(vec![Field::new("col4", DataType::Float32, false) + .with_metadata(HashMap::from([( + PARQUET_FIELD_ID_META_KEY.to_string(), + "4".to_string(), + )]))]); + + let schema_struct_nested_float_fields = + Fields::from(vec![Field::new("col7", DataType::Float32, false) + .with_metadata(HashMap::from([( + PARQUET_FIELD_ID_META_KEY.to_string(), + "7".to_string(), + )]))]); + + let schema_struct_nested_fields = Fields::from(vec![Field::new( + "col6", + arrow_schema::DataType::Struct(schema_struct_nested_float_fields.clone()), + false, + ) + .with_metadata(HashMap::from([( + PARQUET_FIELD_ID_META_KEY.to_string(), + "6".to_string(), + )]))]); + + // prepare data + let arrow_schema = { + let fields = vec![ + Field::new( + "col3", + arrow_schema::DataType::Struct(schema_struct_float_fields.clone()), + false, + ) + .with_metadata(HashMap::from([( + PARQUET_FIELD_ID_META_KEY.to_string(), + "3".to_string(), + )])), + Field::new( + "col5", + arrow_schema::DataType::Struct(schema_struct_nested_fields.clone()), + false, + ) + .with_metadata(HashMap::from([( + PARQUET_FIELD_ID_META_KEY.to_string(), + "5".to_string(), + )])), + ]; + Arc::new(arrow_schema::Schema::new(fields)) + }; + + let float_32_col = Arc::new(Float32Array::from_iter_values_with_nulls( + [1.0_f32, f32::NAN, 2.0, 2.0].into_iter(), + None, + )) as ArrayRef; + + let struct_float_field_col = Arc::new(StructArray::new( + schema_struct_float_fields, + vec![float_32_col.clone()], + None, + )) as ArrayRef; + + let struct_nested_float_field_col = Arc::new(StructArray::new( + schema_struct_nested_fields, + vec![Arc::new(StructArray::new( + schema_struct_nested_float_fields, + vec![float_32_col.clone()], + None, + )) as ArrayRef], + None, + )) as ArrayRef; + + let to_write = RecordBatch::try_new(arrow_schema.clone(), vec![ + struct_float_field_col, + struct_nested_float_field_col, + ]) + .unwrap(); + + // write data + let mut pw = ParquetWriterBuilder::new( + WriterProperties::builder().build(), + Arc::new(to_write.schema().as_ref().try_into().unwrap()), + file_io.clone(), + location_gen, + file_name_gen, + ) + .build() + .await?; + + pw.write(&to_write).await?; + let res = pw.close().await?; + assert_eq!(res.len(), 1); + let data_file = res + .into_iter() + .next() + .unwrap() + // Put dummy field for build successfully. + .content(crate::spec::DataContentType::Data) + .partition(Struct::empty()) + .build() + .unwrap(); + + // check data file + assert_eq!(data_file.record_count(), 4); + assert_eq!( + *data_file.value_counts(), + HashMap::from([(4, 4), (7, 4)]) + ); + assert_eq!( + *data_file.lower_bounds(), + HashMap::from([ + (4, Datum::float(1.0)), + (7, Datum::float(1.0)), + ]) + ); + assert_eq!( + *data_file.upper_bounds(), + HashMap::from([ + (4, Datum::float(2.0)), + (7, Datum::float(2.0)), + ]) + ); + assert_eq!( + *data_file.null_value_counts(), + HashMap::from([(4, 0), (7, 0)]) + ); + assert_eq!( + *data_file.nan_value_counts(), + HashMap::from([(4, 1), (7, 1)]) + ); + + // check the written file + let expect_batch = concat_batches(&arrow_schema, vec![&to_write]).unwrap(); + check_parquet_data_file(&file_io, &data_file, &expect_batch).await; + + Ok(()) + } + + #[tokio::test] + async fn test_nan_val_cnts_list_type() -> Result<()> { + let temp_dir = TempDir::new().unwrap(); + let file_io = FileIOBuilder::new_fs_io().build().unwrap(); + 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); + + let schema_list_float_field = Field::new("element", DataType::Float32, true).with_metadata( + HashMap::from([(PARQUET_FIELD_ID_META_KEY.to_string(), "1".to_string())]), + ); + + let schema_struct_list_float_field = Field::new("element", DataType::Float32, true) + .with_metadata(HashMap::from([( + PARQUET_FIELD_ID_META_KEY.to_string(), + "4".to_string(), + )])); + + let schema_struct_list_field = Fields::from(vec![Field::new_list( + "col2", + schema_struct_list_float_field.clone(), + true, + ) + .with_metadata(HashMap::from([( + PARQUET_FIELD_ID_META_KEY.to_string(), + "3".to_string(), + )]))]); + + let arrow_schema = { + let fields = vec![ + Field::new_list("col0", schema_list_float_field.clone(), true).with_metadata( + HashMap::from([(PARQUET_FIELD_ID_META_KEY.to_string(), "0".to_string())]), + ), + Field::new_struct("col1", schema_struct_list_field.clone(), true) + .with_metadata(HashMap::from([( + PARQUET_FIELD_ID_META_KEY.to_string(), + "2".to_string(), + )])) + .clone(), + // Field::new_large_list("col3", schema_large_list_float_field.clone(), true).with_metadata( + // HashMap::from([(PARQUET_FIELD_ID_META_KEY.to_string(), "5".to_string())]), + // ).clone(), + ]; + Arc::new(arrow_schema::Schema::new(fields)) + }; + + let list_parts = ListArray::from_iter_primitive::(vec![Some(vec![ + Some(1.0_f32), + Some(f32::NAN), + Some(2.0), + Some(2.0), + ])]) + .into_parts(); + + let list_float_field_col = Arc::new({ + let list_parts = list_parts.clone(); + ListArray::new( + { + if let DataType::List(field) = arrow_schema.field(0).data_type() { + field.clone() + } else { + unreachable!() + } + }, + list_parts.1, + list_parts.2, + list_parts.3, + ) + }) as ArrayRef; + + let struct_list_fields_schema = + if let DataType::Struct(fields) = arrow_schema.field(1).data_type() { + fields.clone() + } else { + unreachable!() + }; + + let struct_list_float_field_col = Arc::new({ + ListArray::new( + { + if let DataType::List(field) = struct_list_fields_schema + .get(0) + .expect("could not find first list field") + .data_type() + { + field.clone() + } else { + unreachable!() + } + }, + list_parts.1, + list_parts.2, + list_parts.3, + ) + }) as ArrayRef; + + let struct_list_float_field_col = Arc::new(StructArray::new( + struct_list_fields_schema, + vec![struct_list_float_field_col.clone()], + None, + )) as ArrayRef; + + let to_write = RecordBatch::try_new(arrow_schema.clone(), vec![ + list_float_field_col, + struct_list_float_field_col, + // large_list_float_field_col, + ]) + .expect("Could not form record batch"); + + // write data + let mut pw = ParquetWriterBuilder::new( + WriterProperties::builder().build(), + Arc::new( + to_write + .schema() + .as_ref() + .try_into() + .expect("Could not convert iceberg schema"), + ), + file_io.clone(), + location_gen, + file_name_gen, + ) + .build() + .await?; + + pw.write(&to_write).await?; + let res = pw.close().await?; + assert_eq!(res.len(), 1); + let data_file = res + .into_iter() + .next() + .unwrap() + .content(crate::spec::DataContentType::Data) + .partition(Struct::empty()) + .build() + .unwrap(); + + // check data file + assert_eq!(data_file.record_count(), 1); + assert_eq!(*data_file.value_counts(), HashMap::from([(1, 4), (4, 4)])); + assert_eq!( + *data_file.lower_bounds(), + HashMap::from([(1, Datum::float(1.0)), (4, Datum::float(1.0))]) + ); + assert_eq!( + *data_file.upper_bounds(), + HashMap::from([(1, Datum::float(2.0)), (4, Datum::float(2.0))]) + ); + assert_eq!( + *data_file.null_value_counts(), + HashMap::from([(1, 0), (4, 0)]) + ); + assert_eq!( + *data_file.nan_value_counts(), + HashMap::from([(1, 1), (4, 1)]) + ); + + // check the written file + let expect_batch = concat_batches(&arrow_schema, vec![&to_write]).unwrap(); + check_parquet_data_file(&file_io, &data_file, &expect_batch).await; + + Ok(()) + } + + macro_rules! construct_map_arr { + ($map_key_field_schema:ident, $map_value_field_schema:ident) => {{ + let int_builder = Int32Builder::new(); + let float_builder = Float32Builder::with_capacity(4); + let mut builder = MapBuilder::new(None, int_builder, float_builder); + builder.keys().append_value(1); + builder.values().append_value(1.0_f32); + builder.append(true).unwrap(); + builder.keys().append_value(2); + builder.values().append_value(f32::NAN); + builder.append(true).unwrap(); + builder.keys().append_value(3); + builder.values().append_value(2.0); + builder.append(true).unwrap(); + builder.keys().append_value(4); + builder.values().append_value(2.0); + builder.append(true).unwrap(); + let array = builder.finish(); + + let (_field, offsets, entries, nulls, ordered) = array.into_parts(); + let new_struct_fields_schema = + Fields::from(vec![$map_key_field_schema, $map_value_field_schema]); + + let entries = { + let (_, arrays, nulls) = entries.into_parts(); + StructArray::new(new_struct_fields_schema.clone(), arrays, nulls) + }; + + let field = Arc::new(Field::new( + DEFAULT_MAP_FIELD_NAME, + DataType::Struct(new_struct_fields_schema), + false, + )); + + Arc::new(MapArray::new(field, offsets, entries, nulls, ordered)) + }}; + } + + #[tokio::test] + async fn test_nan_val_cnts_map_type() -> Result<()> { + let temp_dir = TempDir::new().unwrap(); + let file_io = FileIOBuilder::new_fs_io().build().unwrap(); + 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); + + let map_key_field_schema = + Field::new(MAP_KEY_FIELD_NAME, DataType::Int32, false).with_metadata(HashMap::from([ + (PARQUET_FIELD_ID_META_KEY.to_string(), "1".to_string()), + ])); + + let map_value_field_schema = + Field::new(MAP_VALUE_FIELD_NAME, DataType::Float32, true).with_metadata(HashMap::from( + [(PARQUET_FIELD_ID_META_KEY.to_string(), "2".to_string())], + )); + + let struct_map_key_field_schema = + Field::new(MAP_KEY_FIELD_NAME, DataType::Int32, false).with_metadata(HashMap::from([ + (PARQUET_FIELD_ID_META_KEY.to_string(), "6".to_string()), + ])); + + let struct_map_value_field_schema = + Field::new(MAP_VALUE_FIELD_NAME, DataType::Float32, true).with_metadata(HashMap::from( + [(PARQUET_FIELD_ID_META_KEY.to_string(), "7".to_string())], + )); + + let schema_struct_map_field = Fields::from(vec![Field::new_map( + "col3", + DEFAULT_MAP_FIELD_NAME, + struct_map_key_field_schema.clone(), + struct_map_value_field_schema.clone(), + false, + false, + ) + .with_metadata(HashMap::from([( + PARQUET_FIELD_ID_META_KEY.to_string(), + "5".to_string(), + )]))]); + + let arrow_schema = { + let fields = vec![ + Field::new_map( + "col0", + DEFAULT_MAP_FIELD_NAME, + map_key_field_schema.clone(), + map_value_field_schema.clone(), + false, + false, + ) + .with_metadata(HashMap::from([( + PARQUET_FIELD_ID_META_KEY.to_string(), + "0".to_string(), + )])), + Field::new_struct("col1", schema_struct_map_field.clone(), true) + .with_metadata(HashMap::from([( + PARQUET_FIELD_ID_META_KEY.to_string(), + "3".to_string(), + )])) + .clone(), + ]; + Arc::new(arrow_schema::Schema::new(fields)) + }; + + let map_array = construct_map_arr!(map_key_field_schema, map_value_field_schema); + + let struct_map_arr = + construct_map_arr!(struct_map_key_field_schema, struct_map_value_field_schema); + + let struct_list_float_field_col = Arc::new(StructArray::new( + schema_struct_map_field, + vec![struct_map_arr], + None, + )) as ArrayRef; + + let to_write = RecordBatch::try_new(arrow_schema.clone(), vec![map_array, struct_list_float_field_col]) + .expect("Could not form record batch"); + + // write data + let mut pw = ParquetWriterBuilder::new( + WriterProperties::builder().build(), + Arc::new( + to_write + .schema() + .as_ref() + .try_into() + .expect("Could not convert iceberg schema"), + ), + file_io.clone(), + location_gen, + file_name_gen, + ) + .build() + .await?; + + pw.write(&to_write).await?; + let res = pw.close().await?; + assert_eq!(res.len(), 1); + let data_file = res + .into_iter() + .next() + .unwrap() + .content(crate::spec::DataContentType::Data) + .partition(Struct::empty()) + .build() + .unwrap(); + + // check data file + assert_eq!(data_file.record_count(), 4); + assert_eq!(*data_file.value_counts(), HashMap::from([(1, 4), (2, 4), (6, 4), (7, 4)])); + assert_eq!( + *data_file.lower_bounds(), + HashMap::from([(1, Datum::int(1)), (2, Datum::float(1.0)), (6, Datum::int(1)), (7, Datum::float(1.0))]) + ); + assert_eq!( + *data_file.upper_bounds(), + HashMap::from([(1, Datum::int(4)), (2, Datum::float(2.0)), (6, Datum::int(4)), (7, Datum::float(2.0))]) + ); + assert_eq!( + *data_file.null_value_counts(), + HashMap::from([(1, 0), (2, 0), (6, 0), (7, 0)]) + ); + assert_eq!(*data_file.nan_value_counts(), HashMap::from([(2, 1), (7, 1)])); + + // check the written file + let expect_batch = concat_batches(&arrow_schema, vec![&to_write]).unwrap(); check_parquet_data_file(&file_io, &data_file, &expect_batch).await; Ok(()) From 45d64c25339994c3affd3d57ad248db7abd1a5d5 Mon Sep 17 00:00:00 2001 From: feniljain Date: Sat, 8 Mar 2025 12:14:27 +0000 Subject: [PATCH 04/14] chore: fix typos --- crates/iceberg/src/writer/file_writer/parquet_writer.rs | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/crates/iceberg/src/writer/file_writer/parquet_writer.rs b/crates/iceberg/src/writer/file_writer/parquet_writer.rs index 6b3a732346..f4decddc6d 100644 --- a/crates/iceberg/src/writer/file_writer/parquet_writer.rs +++ b/crates/iceberg/src/writer/file_writer/parquet_writer.rs @@ -345,17 +345,17 @@ macro_rules! count_float_nans { macro_rules! handle_list_type { ($t:ty, $col:ident, $self:ident, $field:ident, $arrow_field:ident) => { let list_arr = $col.as_any().downcast_ref::<$t>().unwrap(); - let field_data_typ = $arrow_field.data_type(); + let field_data_type = $arrow_field.data_type(); let n_vals = list_arr.offsets().len() - 1; let field = match $field.clone().field_type.deref() { - Type::List(list_typ) => list_typ.element_field.clone(), + Type::List(list_type) => list_type.element_field.clone(), _ => unreachable!(), }; let field_id = field.id; - match field_data_typ { + match field_data_type { DataType::Float32 => { for idx in 0..n_vals { let arr_ref = list_arr.value(idx); From e9ce0ae5cde72f048e764f5fecc95be5a00e642f Mon Sep 17 00:00:00 2001 From: feniljain Date: Sat, 8 Mar 2025 12:16:10 +0000 Subject: [PATCH 05/14] chore: cargo fmt --- .../src/writer/file_writer/parquet_writer.rs | 76 +++++++++---------- 1 file changed, 37 insertions(+), 39 deletions(-) diff --git a/crates/iceberg/src/writer/file_writer/parquet_writer.rs b/crates/iceberg/src/writer/file_writer/parquet_writer.rs index f4decddc6d..d967d0b07b 100644 --- a/crates/iceberg/src/writer/file_writer/parquet_writer.rs +++ b/crates/iceberg/src/writer/file_writer/parquet_writer.rs @@ -19,11 +19,11 @@ use std::collections::hash_map::Entry; use std::collections::HashMap; +use std::ops::Deref; use std::sync::atomic::AtomicI64; use std::sync::Arc; -use std::ops::Deref; -use arrow_array::{Float32Array, Float64Array, ArrayRef, StructArray, ListArray, MapArray}; +use arrow_array::{ArrayRef, Float32Array, Float64Array, ListArray, MapArray, StructArray}; use arrow_schema::{DataType, SchemaRef as ArrowSchemaRef}; use bytes::Bytes; use futures::future::BoxFuture; @@ -791,12 +791,12 @@ mod tests { use std::sync::Arc; use anyhow::Result; - use arrow_array::types::{Int64Type, Float32Type}; + use arrow_array::builder::{Float32Builder, Int32Builder, MapBuilder}; + use arrow_array::types::{Float32Type, Int64Type}; use arrow_array::{ Array, ArrayRef, BooleanArray, Decimal128Array, Float32Array, Int32Array, Int64Array, ListArray, RecordBatch, StructArray, }; - use arrow_array::builder::{MapBuilder, Int32Builder, Float32Builder}; use arrow_schema::{DataType, Field, Fields, SchemaRef as ArrowSchemaRef}; use arrow_select::concat::concat_batches; use parquet::arrow::PARQUET_FIELD_ID_META_KEY; @@ -1088,11 +1088,8 @@ mod tests { None, )) as ArrayRef; - let to_write = RecordBatch::try_new(arrow_schema.clone(), vec![ - float_32_col, - float_64_col, - ]) - .unwrap(); + let to_write = + RecordBatch::try_new(arrow_schema.clone(), vec![float_32_col, float_64_col]).unwrap(); // write data let mut pw = ParquetWriterBuilder::new( @@ -1120,23 +1117,14 @@ mod tests { // check data file assert_eq!(data_file.record_count(), 4); - assert_eq!( - *data_file.value_counts(), - HashMap::from([(0, 4), (1, 4)]) - ); + assert_eq!(*data_file.value_counts(), HashMap::from([(0, 4), (1, 4)])); assert_eq!( *data_file.lower_bounds(), - HashMap::from([ - (0, Datum::float(1.0)), - (1, Datum::double(1.0)), - ]) + HashMap::from([(0, Datum::float(1.0)), (1, Datum::double(1.0)),]) ); assert_eq!( *data_file.upper_bounds(), - HashMap::from([ - (0, Datum::float(2.0)), - (1, Datum::double(2.0)), - ]) + HashMap::from([(0, Datum::float(2.0)), (1, Datum::double(2.0)),]) ); assert_eq!( *data_file.null_value_counts(), @@ -1265,23 +1253,14 @@ mod tests { // check data file assert_eq!(data_file.record_count(), 4); - assert_eq!( - *data_file.value_counts(), - HashMap::from([(4, 4), (7, 4)]) - ); + assert_eq!(*data_file.value_counts(), HashMap::from([(4, 4), (7, 4)])); assert_eq!( *data_file.lower_bounds(), - HashMap::from([ - (4, Datum::float(1.0)), - (7, Datum::float(1.0)), - ]) + HashMap::from([(4, Datum::float(1.0)), (7, Datum::float(1.0)),]) ); assert_eq!( *data_file.upper_bounds(), - HashMap::from([ - (4, Datum::float(2.0)), - (7, Datum::float(2.0)), - ]) + HashMap::from([(4, Datum::float(2.0)), (7, Datum::float(2.0)),]) ); assert_eq!( *data_file.null_value_counts(), @@ -1580,8 +1559,11 @@ mod tests { None, )) as ArrayRef; - let to_write = RecordBatch::try_new(arrow_schema.clone(), vec![map_array, struct_list_float_field_col]) - .expect("Could not form record batch"); + let to_write = RecordBatch::try_new(arrow_schema.clone(), vec![ + map_array, + struct_list_float_field_col, + ]) + .expect("Could not form record batch"); // write data let mut pw = ParquetWriterBuilder::new( @@ -1614,20 +1596,36 @@ mod tests { // check data file assert_eq!(data_file.record_count(), 4); - assert_eq!(*data_file.value_counts(), HashMap::from([(1, 4), (2, 4), (6, 4), (7, 4)])); + assert_eq!( + *data_file.value_counts(), + HashMap::from([(1, 4), (2, 4), (6, 4), (7, 4)]) + ); assert_eq!( *data_file.lower_bounds(), - HashMap::from([(1, Datum::int(1)), (2, Datum::float(1.0)), (6, Datum::int(1)), (7, Datum::float(1.0))]) + HashMap::from([ + (1, Datum::int(1)), + (2, Datum::float(1.0)), + (6, Datum::int(1)), + (7, Datum::float(1.0)) + ]) ); assert_eq!( *data_file.upper_bounds(), - HashMap::from([(1, Datum::int(4)), (2, Datum::float(2.0)), (6, Datum::int(4)), (7, Datum::float(2.0))]) + HashMap::from([ + (1, Datum::int(4)), + (2, Datum::float(2.0)), + (6, Datum::int(4)), + (7, Datum::float(2.0)) + ]) ); assert_eq!( *data_file.null_value_counts(), HashMap::from([(1, 0), (2, 0), (6, 0), (7, 0)]) ); - assert_eq!(*data_file.nan_value_counts(), HashMap::from([(2, 1), (7, 1)])); + assert_eq!( + *data_file.nan_value_counts(), + HashMap::from([(2, 1), (7, 1)]) + ); // check the written file let expect_batch = concat_batches(&arrow_schema, vec![&to_write]).unwrap(); From 18a55516f61b1c52cabc9e671d529691cefca573 Mon Sep 17 00:00:00 2001 From: feniljain Date: Sat, 8 Mar 2025 12:25:29 +0000 Subject: [PATCH 06/14] fix: bring back accidentally deleted test --- .../src/writer/file_writer/parquet_writer.rs | 99 +++++++++++++++---- 1 file changed, 80 insertions(+), 19 deletions(-) diff --git a/crates/iceberg/src/writer/file_writer/parquet_writer.rs b/crates/iceberg/src/writer/file_writer/parquet_writer.rs index d967d0b07b..b7027b9b57 100644 --- a/crates/iceberg/src/writer/file_writer/parquet_writer.rs +++ b/crates/iceberg/src/writer/file_writer/parquet_writer.rs @@ -499,8 +499,8 @@ impl ParquetWriter { .lower_bounds(lower_bounds) .upper_bounds(upper_bounds) .nan_value_counts(nan_value_counts) - // # TODO(#417) - // - distinct_counts + // # NOTE: + // - We can ignore implementing distinct_counts due to this: https://lists.apache.org/thread/j52tsojv0x4bopxyzsp7m7bqt23n5fnd .key_metadata(metadata.footer_signing_key_metadata) .split_offsets( metadata @@ -639,17 +639,6 @@ impl ParquetWriter { }; } } - DataType::List(arrow_field) => { - handle_list_type!(ListArray, col, self, field, arrow_field); - } - // NOTE: iceberg to arrow schema conversion does not form these types, - // meaning these branches never get called right now. - DataType::LargeList(_) => { - // handle_list_type!(LargeListArray, col, self, field, arrow_field); - } - DataType::FixedSizeList(_, _) => { - // handle_list_type!(FixedSizeList, col, self, field, arrow_field); - } DataType::Map(_, _) => { let map_arr = col.as_any().downcast_ref::().unwrap(); @@ -664,6 +653,17 @@ impl ParquetWriter { let values_col = map_arr.values(); self.transverse_batch(values_col, &map_ty.value_field); } + DataType::List(arrow_field) => { + handle_list_type!(ListArray, col, self, field, arrow_field); + } + // NOTE: iceberg to arrow schema conversion does not form these types, + // meaning these branches never get called right now. + DataType::LargeList(_) => { + // handle_list_type!(LargeListArray, col, self, field, arrow_field); + } + DataType::FixedSizeList(_, _) => { + // handle_list_type!(FixedSizeList, col, self, field, arrow_field); + } _ => {} }; } @@ -972,12 +972,6 @@ mod tests { // prepare data let schema = { let fields = vec![ - // TODO(feniljain): - // Types: - // [X] Primitive - // [ ] Struct - // [ ] List - // [ ] Map arrow_schema::Field::new("col", arrow_schema::DataType::Float32, true) .with_metadata(HashMap::from([( PARQUET_FIELD_ID_META_KEY.to_string(), @@ -1634,6 +1628,73 @@ mod tests { Ok(()) } + #[tokio::test] + async fn test_parquet_writer() -> Result<()> { + let temp_dir = TempDir::new().unwrap(); + let file_io = FileIOBuilder::new_fs_io().build().unwrap(); + 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); + + // 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())]), + ), + ]; + Arc::new(arrow_schema::Schema::new(fields)) + }; + let col = Arc::new(Int64Array::from_iter_values(0..1024)) as ArrayRef; + let null_col = Arc::new(Int64Array::new_null(1024)) as ArrayRef; + let to_write = RecordBatch::try_new(schema.clone(), vec![col]).unwrap(); + let to_write_null = RecordBatch::try_new(schema.clone(), vec![null_col]).unwrap(); + + // write data + let mut pw = ParquetWriterBuilder::new( + WriterProperties::builder().build(), + Arc::new(to_write.schema().as_ref().try_into().unwrap()), + file_io.clone(), + location_gen, + file_name_gen, + ) + .build() + .await?; + pw.write(&to_write).await?; + pw.write(&to_write_null).await?; + let res = pw.close().await?; + assert_eq!(res.len(), 1); + let data_file = res + .into_iter() + .next() + .unwrap() + // Put dummy field for build successfully. + .content(crate::spec::DataContentType::Data) + .partition(Struct::empty()) + .build() + .unwrap(); + + // check data file + assert_eq!(data_file.record_count(), 2048); + assert_eq!(*data_file.value_counts(), HashMap::from([(0, 2048)])); + assert_eq!( + *data_file.lower_bounds(), + HashMap::from([(0, Datum::long(0))]) + ); + assert_eq!( + *data_file.upper_bounds(), + HashMap::from([(0, Datum::long(1023))]) + ); + assert_eq!(*data_file.null_value_counts(), HashMap::from([(0, 1024)])); + + // check the written file + let expect_batch = concat_batches(&schema, vec![&to_write, &to_write_null]).unwrap(); + check_parquet_data_file(&file_io, &data_file, &expect_batch).await; + + Ok(()) + } + #[tokio::test] async fn test_parquet_writer_with_complex_schema() -> Result<()> { let temp_dir = TempDir::new().unwrap(); From 736c5ade718ddb943d191ad84e2829b7d48fb69f Mon Sep 17 00:00:00 2001 From: feniljain Date: Sat, 8 Mar 2025 12:27:04 +0000 Subject: [PATCH 07/14] chore: cargo clippy fix --- crates/iceberg/src/writer/file_writer/parquet_writer.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/crates/iceberg/src/writer/file_writer/parquet_writer.rs b/crates/iceberg/src/writer/file_writer/parquet_writer.rs index b7027b9b57..26aefd9d0d 100644 --- a/crates/iceberg/src/writer/file_writer/parquet_writer.rs +++ b/crates/iceberg/src/writer/file_writer/parquet_writer.rs @@ -1354,7 +1354,7 @@ mod tests { ListArray::new( { if let DataType::List(field) = struct_list_fields_schema - .get(0) + .first() .expect("could not find first list field") .data_type() { From 23eb3431842d710173fe53d26f9707d85f882da3 Mon Sep 17 00:00:00 2001 From: feniljain Date: Mon, 10 Mar 2025 19:55:15 +0000 Subject: [PATCH 08/14] feat: shift to use standard visitors --- crates/iceberg/src/arrow/mod.rs | 2 + .../iceberg/src/arrow/nan_val_cnt_visitor.rs | 123 + crates/iceberg/src/arrow/value.rs | 3 +- .../src/writer/file_writer/parquet_writer.rs | 2342 ++++++++--------- 4 files changed, 1174 insertions(+), 1296 deletions(-) create mode 100644 crates/iceberg/src/arrow/nan_val_cnt_visitor.rs diff --git a/crates/iceberg/src/arrow/mod.rs b/crates/iceberg/src/arrow/mod.rs index 0c885e65f4..5ac6817f7e 100644 --- a/crates/iceberg/src/arrow/mod.rs +++ b/crates/iceberg/src/arrow/mod.rs @@ -19,6 +19,8 @@ mod schema; pub use schema::*; +mod nan_val_cnt_visitor; +pub use nan_val_cnt_visitor::*; mod reader; pub(crate) mod record_batch_projector; pub(crate) mod record_batch_transformer; diff --git a/crates/iceberg/src/arrow/nan_val_cnt_visitor.rs b/crates/iceberg/src/arrow/nan_val_cnt_visitor.rs new file mode 100644 index 0000000000..1036e9b8f5 --- /dev/null +++ b/crates/iceberg/src/arrow/nan_val_cnt_visitor.rs @@ -0,0 +1,123 @@ +use std::collections::hash_map::Entry; +use std::collections::HashMap; + +use arrow_array::{ArrayRef, Float32Array, Float64Array, RecordBatch}; + +use crate::arrow::ArrowArrayAccessor; +use crate::spec::{ + visit_schema_with_partner, ListType, MapType, NestedFieldRef, SchemaRef, + PrimitiveType, Schema, SchemaWithPartnerVisitor, StructType, +}; +use crate::Result; + +macro_rules! count_float_nans { + ($t:ty, $col:ident, $self:ident, $field_id:ident) => { + let nan_val_cnt = $col + .as_any() + .downcast_ref::<$t>() + .unwrap() + .iter() + .filter(|value| value.map_or(false, |v| v.is_nan())) + .count() as u64; + + match $self.nan_value_counts.entry($field_id) { + Entry::Occupied(mut ele) => { + let total_nan_val_cnt = ele.get() + nan_val_cnt; + ele.insert(total_nan_val_cnt); + } + Entry::Vacant(v) => { + v.insert(nan_val_cnt); + } + }; + }; +} + +/// TODO(feniljain) +pub struct NanValueCountVisitor { + /// Stores field ID to NaN value count mapping + pub nan_value_counts: HashMap, +} + +impl SchemaWithPartnerVisitor for NanValueCountVisitor { + type T = (); + + fn schema( + &mut self, + _schema: &Schema, + _partner: &ArrayRef, + _value: Self::T, + ) -> Result { + Ok(()) + } + + fn field( + &mut self, + _field: &NestedFieldRef, + _partner: &ArrayRef, + _value: Self::T, + ) -> Result { + Ok(()) + } + + fn r#struct( + &mut self, + _struct: &StructType, + _partner: &ArrayRef, + _results: Vec, + ) -> Result { + Ok(()) + } + + fn list(&mut self, _list: &ListType, _list_arr: &ArrayRef, _value: Self::T) -> Result { + Ok(()) + } + + fn map( + &mut self, + _map: &MapType, + _partner: &ArrayRef, + _key_value: Self::T, + _value: Self::T, + ) -> Result { + Ok(()) + } + + fn primitive(&mut self, p: &PrimitiveType, col: &ArrayRef) -> Result { + match p { + PrimitiveType::Float => { + // let field_id = p.id; + // TODO(feniljain): fix this + let field_id = 1; + count_float_nans!(Float32Array, col, self, field_id); + } + PrimitiveType::Double => { + let field_id = 1; + count_float_nans!(Float64Array, col, self, field_id); + } + _ => {} + } + + Ok(()) + } +} + +impl NanValueCountVisitor { + /// Creates new instance of NanValueCountVisitor + pub fn new() -> Self { + Self { + nan_value_counts: HashMap::new(), + } + } + + /// Compute nan value counts in given schema and record batch + pub fn compute(&mut self, schema: SchemaRef, batch: &RecordBatch) -> Result<()> { + let arrow_arr_partner_accessor = ArrowArrayAccessor{}; + + for arr_ref in batch.columns() { + visit_schema_with_partner(&schema, arr_ref, self, &arrow_arr_partner_accessor)?; + } + + Ok(()) + } +} + diff --git a/crates/iceberg/src/arrow/value.rs b/crates/iceberg/src/arrow/value.rs index d78c4f4400..0b8f0d1e13 100644 --- a/crates/iceberg/src/arrow/value.rs +++ b/crates/iceberg/src/arrow/value.rs @@ -425,7 +425,8 @@ impl SchemaWithPartnerVisitor for ArrowArrayToIcebergStructConverter { } } -struct ArrowArrayAccessor; +/// TODO(feniljain) +pub struct ArrowArrayAccessor; impl PartnerAccessor for ArrowArrayAccessor { fn struct_parner<'a>(&self, schema_partner: &'a ArrayRef) -> Result<&'a ArrayRef> { diff --git a/crates/iceberg/src/writer/file_writer/parquet_writer.rs b/crates/iceberg/src/writer/file_writer/parquet_writer.rs index 26aefd9d0d..f18529015f 100644 --- a/crates/iceberg/src/writer/file_writer/parquet_writer.rs +++ b/crates/iceberg/src/writer/file_writer/parquet_writer.rs @@ -17,14 +17,11 @@ //! The module contains the file writer for parquet file format. -use std::collections::hash_map::Entry; use std::collections::HashMap; -use std::ops::Deref; use std::sync::atomic::AtomicI64; use std::sync::Arc; -use arrow_array::{ArrayRef, Float32Array, Float64Array, ListArray, MapArray, StructArray}; -use arrow_schema::{DataType, SchemaRef as ArrowSchemaRef}; +use arrow_schema::SchemaRef as ArrowSchemaRef; use bytes::Bytes; use futures::future::BoxFuture; use itertools::Itertools; @@ -41,7 +38,7 @@ use super::track_writer::TrackWriter; use super::{FileWriter, FileWriterBuilder}; use crate::arrow::{ get_parquet_stat_max_as_datum, get_parquet_stat_min_as_datum, ArrowFileReader, - DEFAULT_MAP_FIELD_NAME, + DEFAULT_MAP_FIELD_NAME, NanValueCountVisitor, }; use crate::io::{FileIO, FileWrite, OutputFile}; use crate::spec::{ @@ -100,7 +97,7 @@ impl FileWriterBuilder for ParquetWr written_size, current_row_num: 0, out_file, - nan_value_counts: HashMap::new(), + nan_value_count_visitor: NanValueCountVisitor::new(), }) } } @@ -226,7 +223,7 @@ pub struct ParquetWriter { writer_properties: WriterProperties, written_size: Arc, current_row_num: usize, - nan_value_counts: HashMap, + nan_value_count_visitor: NanValueCountVisitor, } /// Used to aggregate min and max value of each column. @@ -320,75 +317,8 @@ impl MinMaxColAggregator { } } -macro_rules! count_float_nans { - ($t:ty, $col:ident, $self:ident, $field_id:ident) => { - let nan_val_cnt = $col - .as_any() - .downcast_ref::<$t>() - .unwrap() - .iter() - .filter(|value| value.map_or(false, |v| v.is_nan())) - .count() as u64; - - match $self.nan_value_counts.entry($field_id) { - Entry::Occupied(mut ele) => { - let total_nan_val_cnt = ele.get() + nan_val_cnt; - ele.insert(total_nan_val_cnt); - } - Entry::Vacant(v) => { - v.insert(nan_val_cnt); - } - }; - }; -} - -macro_rules! handle_list_type { - ($t:ty, $col:ident, $self:ident, $field:ident, $arrow_field:ident) => { - let list_arr = $col.as_any().downcast_ref::<$t>().unwrap(); - let field_data_type = $arrow_field.data_type(); - - let n_vals = list_arr.offsets().len() - 1; - - let field = match $field.clone().field_type.deref() { - Type::List(list_type) => list_type.element_field.clone(), - _ => unreachable!(), - }; - let field_id = field.id; - - match field_data_type { - DataType::Float32 => { - for idx in 0..n_vals { - let arr_ref = list_arr.value(idx); - count_float_nans!(Float32Array, arr_ref, $self, field_id); - } - } - DataType::Float64 => { - for idx in 0..n_vals { - let arr_ref = list_arr.value(idx); - count_float_nans!(Float64Array, arr_ref, $self, field_id); - } - } - DataType::List(_) - | DataType::LargeList(_) - | DataType::FixedSizeList(_, _) - | DataType::Struct(_) - | DataType::Map(_, _) => { - for idx in 0..n_vals { - let arr_ref = list_arr.value(idx); - - let field = match field.clone().field_type.deref() { - Type::List(list_ty) => list_ty.element_field.clone(), - _ => unreachable!(), - }; - - $self.transverse_batch(&arr_ref, &field); - } - } - _ => {} - }; - }; -} - +// TODO(feniljain): Think do we need to add nan_value_counts to `parquet_files_to_data_files` and +// `parquet_to_data_file_builder` too? impl ParquetWriter { /// Converts parquet files to data files #[allow(dead_code)] @@ -584,89 +514,6 @@ impl ParquetWriter { Ok(builder) } - - fn transverse_batch(&mut self, col: &ArrayRef, field: &NestedFieldRef) { - let dt = col.data_type(); - - match dt { - DataType::Float32 => { - let field_id: i32 = field.id; - count_float_nans!(Float32Array, col, self, field_id); - } - DataType::Float64 => { - let field_id: i32 = field.id; - count_float_nans!(Float64Array, col, self, field_id); - } - DataType::Struct(fields) => { - let struct_arr = col.as_any().downcast_ref::().unwrap(); - for (idx, arrow_field) in fields.iter().enumerate() { - match arrow_field.data_type() { - DataType::Float32 => { - let float_arr_ref = struct_arr.column(idx); - - let field_id = match field.clone().field_type.deref() { - Type::Struct(struct_ty) => struct_ty.fields()[idx].id, - _ => unreachable!(), - }; - - count_float_nans!(Float32Array, float_arr_ref, self, field_id); - } - DataType::Float64 => { - let float_arr_ref = struct_arr.column(idx); - - let field_id = match field.clone().field_type.deref() { - Type::Struct(struct_ty) => struct_ty.fields()[idx].id, - _ => unreachable!(), - }; - - count_float_nans!(Float64Array, float_arr_ref, self, field_id); - } - DataType::List(_) - | DataType::LargeList(_) - | DataType::FixedSizeList(_, _) - | DataType::Struct(_) - | DataType::Map(_, _) => { - let arr_ref = struct_arr.column(idx); - - let field = match field.clone().field_type.deref() { - Type::Struct(struct_ty) => struct_ty.fields()[idx].clone(), - _ => unreachable!(), - }; - - self.transverse_batch(arr_ref, &field); - } - _ => {} - }; - } - } - DataType::Map(_, _) => { - let map_arr = col.as_any().downcast_ref::().unwrap(); - - let map_ty = match field.clone().field_type.deref() { - Type::Map(map_ty) => map_ty.clone(), - _ => unreachable!(), - }; - - let keys_col = map_arr.keys(); - self.transverse_batch(keys_col, &map_ty.key_field); - - let values_col = map_arr.values(); - self.transverse_batch(values_col, &map_ty.value_field); - } - DataType::List(arrow_field) => { - handle_list_type!(ListArray, col, self, field, arrow_field); - } - // NOTE: iceberg to arrow schema conversion does not form these types, - // meaning these branches never get called right now. - DataType::LargeList(_) => { - // handle_list_type!(LargeListArray, col, self, field, arrow_field); - } - DataType::FixedSizeList(_, _) => { - // handle_list_type!(FixedSizeList, col, self, field, arrow_field); - } - _ => {} - }; - } } impl FileWriter for ParquetWriter { @@ -678,12 +525,7 @@ impl FileWriter for ParquetWriter { self.current_row_num += batch.num_rows(); - let schema_c = self.schema.clone(); - let fields = schema_c.as_struct().fields(); - - for (col, field) in batch.columns().iter().zip(fields) { - self.transverse_batch(col, field); - } + self.nan_value_count_visitor.compute(self.schema.clone(), batch)?; // Lazy initialize the writer let writer = if let Some(writer) = &mut self.inner_writer { @@ -732,7 +574,7 @@ impl FileWriter for ParquetWriter { metadata, written_size as usize, self.out_file.location().to_string(), - self.nan_value_counts, + self.nan_value_count_visitor.nan_value_counts, )?]) } } @@ -795,7 +637,7 @@ mod tests { use arrow_array::types::{Float32Type, Int64Type}; use arrow_array::{ Array, ArrayRef, BooleanArray, Decimal128Array, Float32Array, Int32Array, Int64Array, - ListArray, RecordBatch, StructArray, + ListArray, RecordBatch, StructArray, MapArray, Float64Array, }; use arrow_schema::{DataType, Field, Fields, SchemaRef as ArrowSchemaRef}; use arrow_select::concat::concat_batches; @@ -961,7 +803,7 @@ mod tests { } #[tokio::test] - async fn test_parquet_writer_for_nan_value_counts() -> Result<()> { + async fn test_parquet_writer() -> Result<()> { let temp_dir = TempDir::new().unwrap(); let file_io = FileIOBuilder::new_fs_io().build().unwrap(); let location_gen = @@ -972,32 +814,16 @@ mod tests { // prepare data let schema = { let fields = vec![ - arrow_schema::Field::new("col", arrow_schema::DataType::Float32, true) - .with_metadata(HashMap::from([( - PARQUET_FIELD_ID_META_KEY.to_string(), - "0".to_string(), - )])), - arrow_schema::Field::new("col1", arrow_schema::DataType::Float64, true) - .with_metadata(HashMap::from([( - PARQUET_FIELD_ID_META_KEY.to_string(), - "1".to_string(), - )])), + arrow_schema::Field::new("col", arrow_schema::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 float_32_col = Arc::new(Float32Array::from_iter_values_with_nulls( - [1.0_f32, f32::NAN, 2.0, 2.0].into_iter(), - None, - )) as ArrayRef; - - let float_64_col = Arc::new(Float64Array::from_iter_values_with_nulls( - [1.0_f64, f64::NAN, 2.0, 2.0].into_iter(), - None, - )) as ArrayRef; - - let to_write = - RecordBatch::try_new(schema.clone(), vec![float_32_col, float_64_col]).unwrap(); + let col = Arc::new(Int64Array::from_iter_values(0..1024)) as ArrayRef; + let null_col = Arc::new(Int64Array::new_null(1024)) as ArrayRef; + let to_write = RecordBatch::try_new(schema.clone(), vec![col]).unwrap(); + let to_write_null = RecordBatch::try_new(schema.clone(), vec![null_col]).unwrap(); // write data let mut pw = ParquetWriterBuilder::new( @@ -1009,8 +835,8 @@ mod tests { ) .build() .await?; - pw.write(&to_write).await?; + pw.write(&to_write_null).await?; let res = pw.close().await?; assert_eq!(res.len(), 1); let data_file = res @@ -1024,78 +850,187 @@ mod tests { .unwrap(); // check data file - assert_eq!(data_file.record_count(), 4); - assert_eq!(*data_file.value_counts(), HashMap::from([(0, 4), (1, 4)])); + assert_eq!(data_file.record_count(), 2048); + assert_eq!(*data_file.value_counts(), HashMap::from([(0, 2048)])); assert_eq!( *data_file.lower_bounds(), - HashMap::from([(0, Datum::float(1.0)), (1, Datum::double(1.0))]) + HashMap::from([(0, Datum::long(0))]) ); assert_eq!( *data_file.upper_bounds(), - HashMap::from([(0, Datum::float(2.0)), (1, Datum::double(2.0))]) - ); - assert_eq!( - *data_file.null_value_counts(), - HashMap::from([(0, 0), (1, 0)]) - ); - assert_eq!( - *data_file.nan_value_counts(), - HashMap::from([(0, 1), (1, 1)]) + HashMap::from([(0, Datum::long(1023))]) ); + assert_eq!(*data_file.null_value_counts(), HashMap::from([(0, 1024)])); // check the written file - let expect_batch = concat_batches(&schema, vec![&to_write]).unwrap(); + let expect_batch = concat_batches(&schema, vec![&to_write, &to_write_null]).unwrap(); check_parquet_data_file(&file_io, &data_file, &expect_batch).await; Ok(()) } #[tokio::test] - async fn test_nan_val_cnts_primitive_type() -> Result<()> { + async fn test_parquet_writer_with_complex_schema() -> Result<()> { let temp_dir = TempDir::new().unwrap(); let file_io = FileIOBuilder::new_fs_io().build().unwrap(); 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); - // - // prepare data - let arrow_schema = { - let fields = vec![ - Field::new("col", arrow_schema::DataType::Float32, false).with_metadata( - HashMap::from([(PARQUET_FIELD_ID_META_KEY.to_string(), "0".to_string())]), - ), - Field::new("col2", arrow_schema::DataType::Float64, false).with_metadata( - HashMap::from([(PARQUET_FIELD_ID_META_KEY.to_string(), "1".to_string())]), - ), - ]; - Arc::new(arrow_schema::Schema::new(fields)) - }; - let float_32_col = Arc::new(Float32Array::from_iter_values_with_nulls( - [1.0_f32, f32::NAN, 2.0, 2.0].into_iter(), + // prepare data + let schema = nested_schema_for_test(); + let arrow_schema: ArrowSchemaRef = Arc::new((&schema).try_into().unwrap()); + let col0 = Arc::new(Int64Array::from_iter_values(0..1024)) as ArrayRef; + let col1 = Arc::new(StructArray::new( + { + if let DataType::Struct(fields) = arrow_schema.field(1).data_type() { + fields.clone() + } else { + unreachable!() + } + }, + vec![ + Arc::new(Int64Array::from_iter_values(0..1024)), + Arc::new(Int64Array::from_iter_values(0..1024)), + ], None, + )); + let col2 = Arc::new(arrow_array::StringArray::from_iter_values( + (0..1024).map(|n| n.to_string()), )) as ArrayRef; - - let float_64_col = Arc::new(Float64Array::from_iter_values_with_nulls( - [1.0_f64, f64::NAN, 2.0, 2.0].into_iter(), + let col3 = Arc::new({ + let list_parts = arrow_array::ListArray::from_iter_primitive::( + (0..1024).map(|n| Some(vec![Some(n)])), + ) + .into_parts(); + arrow_array::ListArray::new( + { + if let DataType::List(field) = arrow_schema.field(3).data_type() { + field.clone() + } else { + unreachable!() + } + }, + list_parts.1, + list_parts.2, + list_parts.3, + ) + }) as ArrayRef; + let col4 = Arc::new(StructArray::new( + { + if let DataType::Struct(fields) = arrow_schema.field(4).data_type() { + fields.clone() + } else { + unreachable!() + } + }, + vec![Arc::new(StructArray::new( + { + if let DataType::Struct(fields) = arrow_schema.field(4).data_type() { + if let DataType::Struct(fields) = fields[0].data_type() { + fields.clone() + } else { + unreachable!() + } + } else { + unreachable!() + } + }, + vec![Arc::new(Int64Array::from_iter_values(0..1024))], + None, + ))], None, - )) as ArrayRef; - - let to_write = - RecordBatch::try_new(arrow_schema.clone(), vec![float_32_col, float_64_col]).unwrap(); + )); + let col5 = Arc::new({ + let mut map_array_builder = arrow_array::builder::MapBuilder::new( + None, + arrow_array::builder::StringBuilder::new(), + arrow_array::builder::ListBuilder::new(arrow_array::builder::PrimitiveBuilder::< + Int64Type, + >::new()), + ); + for i in 0..1024 { + map_array_builder.keys().append_value(i.to_string()); + map_array_builder + .values() + .append_value(vec![Some(i as i64); i + 1]); + map_array_builder.append(true)?; + } + let (_, offset_buffer, struct_array, null_buffer, ordered) = + map_array_builder.finish().into_parts(); + let struct_array = { + let (_, mut arrays, nulls) = struct_array.into_parts(); + let list_array = { + let list_array = arrays[1] + .as_any() + .downcast_ref::() + .unwrap() + .clone(); + let (_, offsets, array, nulls) = list_array.into_parts(); + let list_field = { + if let DataType::Map(map_field, _) = arrow_schema.field(5).data_type() { + if let DataType::Struct(fields) = map_field.data_type() { + if let DataType::List(list_field) = fields[1].data_type() { + list_field.clone() + } else { + unreachable!() + } + } else { + unreachable!() + } + } else { + unreachable!() + } + }; + ListArray::new(list_field, offsets, array, nulls) + }; + arrays[1] = Arc::new(list_array) as ArrayRef; + StructArray::new( + { + if let DataType::Map(map_field, _) = arrow_schema.field(5).data_type() { + if let DataType::Struct(fields) = map_field.data_type() { + fields.clone() + } else { + unreachable!() + } + } else { + unreachable!() + } + }, + arrays, + nulls, + ) + }; + arrow_array::MapArray::new( + { + if let DataType::Map(map_field, _) = arrow_schema.field(5).data_type() { + map_field.clone() + } else { + unreachable!() + } + }, + offset_buffer, + struct_array, + null_buffer, + ordered, + ) + }) as ArrayRef; + let to_write = RecordBatch::try_new(arrow_schema.clone(), vec![ + col0, col1, col2, col3, col4, col5, + ]) + .unwrap(); // write data let mut pw = ParquetWriterBuilder::new( WriterProperties::builder().build(), - Arc::new(to_write.schema().as_ref().try_into().unwrap()), + Arc::new(schema), file_io.clone(), location_gen, file_name_gen, ) .build() .await?; - pw.write(&to_write).await?; let res = pw.close().await?; assert_eq!(res.len(), 1); @@ -1110,128 +1045,182 @@ mod tests { .unwrap(); // check data file - assert_eq!(data_file.record_count(), 4); - assert_eq!(*data_file.value_counts(), HashMap::from([(0, 4), (1, 4)])); - assert_eq!( - *data_file.lower_bounds(), - HashMap::from([(0, Datum::float(1.0)), (1, Datum::double(1.0)),]) - ); + assert_eq!(data_file.record_count(), 1024); assert_eq!( - *data_file.upper_bounds(), - HashMap::from([(0, Datum::float(2.0)), (1, Datum::double(2.0)),]) + *data_file.value_counts(), + HashMap::from([ + (0, 1024), + (5, 1024), + (6, 1024), + (2, 1024), + (7, 1024), + (9, 1024), + (11, 1024), + (13, (1..1025).sum()), + ]) ); assert_eq!( - *data_file.null_value_counts(), - HashMap::from([(0, 0), (1, 0)]) + *data_file.lower_bounds(), + HashMap::from([ + (0, Datum::long(0)), + (5, Datum::long(0)), + (6, Datum::long(0)), + (2, Datum::string("0")), + (7, Datum::long(0)), + (9, Datum::long(0)), + (11, Datum::string("0")), + (13, Datum::long(0)) + ]) ); assert_eq!( - *data_file.nan_value_counts(), - HashMap::from([(0, 1), (1, 1)]) + *data_file.upper_bounds(), + HashMap::from([ + (0, Datum::long(1023)), + (5, Datum::long(1023)), + (6, Datum::long(1023)), + (2, Datum::string("999")), + (7, Datum::long(1023)), + (9, Datum::long(1023)), + (11, Datum::string("999")), + (13, Datum::long(1023)) + ]) ); // check the written file - let expect_batch = concat_batches(&arrow_schema, vec![&to_write]).unwrap(); - check_parquet_data_file(&file_io, &data_file, &expect_batch).await; + check_parquet_data_file(&file_io, &data_file, &to_write).await; Ok(()) } #[tokio::test] - async fn test_nan_val_cnts_struct_type() -> Result<()> { + async fn test_all_type_for_write() -> Result<()> { let temp_dir = TempDir::new().unwrap(); let file_io = FileIOBuilder::new_fs_io().build().unwrap(); - let location_gen = + let loccation_gen = MockLocationGenerator::new(temp_dir.path().to_str().unwrap().to_string()); let file_name_gen = DefaultFileNameGenerator::new("test".to_string(), None, DataFileFormat::Parquet); - let schema_struct_float_fields = - Fields::from(vec![Field::new("col4", DataType::Float32, false) - .with_metadata(HashMap::from([( - PARQUET_FIELD_ID_META_KEY.to_string(), - "4".to_string(), - )]))]); - - let schema_struct_nested_float_fields = - Fields::from(vec![Field::new("col7", DataType::Float32, false) - .with_metadata(HashMap::from([( - PARQUET_FIELD_ID_META_KEY.to_string(), - "7".to_string(), - )]))]); - - let schema_struct_nested_fields = Fields::from(vec![Field::new( - "col6", - arrow_schema::DataType::Struct(schema_struct_nested_float_fields.clone()), - false, - ) - .with_metadata(HashMap::from([( - PARQUET_FIELD_ID_META_KEY.to_string(), - "6".to_string(), - )]))]); - // prepare data - let arrow_schema = { - let fields = vec![ - Field::new( - "col3", - arrow_schema::DataType::Struct(schema_struct_float_fields.clone()), - false, - ) - .with_metadata(HashMap::from([( - PARQUET_FIELD_ID_META_KEY.to_string(), - "3".to_string(), - )])), - Field::new( - "col5", - arrow_schema::DataType::Struct(schema_struct_nested_fields.clone()), - false, - ) - .with_metadata(HashMap::from([( - PARQUET_FIELD_ID_META_KEY.to_string(), - "5".to_string(), - )])), - ]; - Arc::new(arrow_schema::Schema::new(fields)) - }; - - let float_32_col = Arc::new(Float32Array::from_iter_values_with_nulls( - [1.0_f32, f32::NAN, 2.0, 2.0].into_iter(), + // generate iceberg schema for all type + let schema = schema_for_all_type(); + let arrow_schema: ArrowSchemaRef = Arc::new((&schema).try_into().unwrap()); + let col0 = Arc::new(BooleanArray::from(vec![ + Some(true), + Some(false), None, - )) as ArrayRef; - - let struct_float_field_col = Arc::new(StructArray::new( - schema_struct_float_fields, - vec![float_32_col.clone()], + Some(true), + ])) as ArrayRef; + let col1 = Arc::new(Int32Array::from(vec![Some(1), Some(2), None, Some(4)])) as ArrayRef; + let col2 = Arc::new(Int64Array::from(vec![Some(1), Some(2), None, Some(4)])) as ArrayRef; + let col3 = Arc::new(arrow_array::Float32Array::from(vec![ + Some(0.5), + Some(2.0), None, - )) as ArrayRef; - - let struct_nested_float_field_col = Arc::new(StructArray::new( - schema_struct_nested_fields, - vec![Arc::new(StructArray::new( - schema_struct_nested_float_fields, - vec![float_32_col.clone()], - None, - )) as ArrayRef], + Some(3.5), + ])) as ArrayRef; + let col4 = Arc::new(arrow_array::Float64Array::from(vec![ + Some(0.5), + Some(2.0), None, - )) as ArrayRef; - + Some(3.5), + ])) as ArrayRef; + let col5 = Arc::new(arrow_array::StringArray::from(vec![ + Some("a"), + Some("b"), + None, + Some("d"), + ])) as ArrayRef; + let col6 = Arc::new(arrow_array::LargeBinaryArray::from_opt_vec(vec![ + Some(b"one"), + None, + Some(b""), + Some(b"zzzz"), + ])) as ArrayRef; + let col7 = Arc::new(arrow_array::Date32Array::from(vec![ + Some(0), + Some(1), + None, + Some(3), + ])) as ArrayRef; + let col8 = Arc::new(arrow_array::Time64MicrosecondArray::from(vec![ + Some(0), + Some(1), + None, + Some(3), + ])) as ArrayRef; + let col9 = Arc::new(arrow_array::TimestampMicrosecondArray::from(vec![ + Some(0), + Some(1), + None, + Some(3), + ])) as ArrayRef; + let col10 = Arc::new( + arrow_array::TimestampMicrosecondArray::from(vec![Some(0), Some(1), None, Some(3)]) + .with_timezone_utc(), + ) as ArrayRef; + let col11 = Arc::new(arrow_array::TimestampNanosecondArray::from(vec![ + Some(0), + Some(1), + None, + Some(3), + ])) as ArrayRef; + let col12 = Arc::new( + arrow_array::TimestampNanosecondArray::from(vec![Some(0), Some(1), None, Some(3)]) + .with_timezone_utc(), + ) as ArrayRef; + let col13 = Arc::new( + arrow_array::Decimal128Array::from(vec![Some(1), Some(2), None, Some(100)]) + .with_precision_and_scale(10, 5) + .unwrap(), + ) as ArrayRef; + let col14 = Arc::new( + arrow_array::FixedSizeBinaryArray::try_from_sparse_iter_with_size( + vec![ + Some(Uuid::from_u128(0).as_bytes().to_vec()), + Some(Uuid::from_u128(1).as_bytes().to_vec()), + None, + Some(Uuid::from_u128(3).as_bytes().to_vec()), + ] + .into_iter(), + 16, + ) + .unwrap(), + ) as ArrayRef; + let col15 = Arc::new( + arrow_array::FixedSizeBinaryArray::try_from_sparse_iter_with_size( + vec![ + Some(vec![1, 2, 3, 4, 5, 6, 7, 8, 9, 10]), + Some(vec![11, 12, 13, 14, 15, 16, 17, 18, 19, 20]), + None, + Some(vec![21, 22, 23, 24, 25, 26, 27, 28, 29, 30]), + ] + .into_iter(), + 10, + ) + .unwrap(), + ) as ArrayRef; + let col16 = Arc::new( + arrow_array::Decimal128Array::from(vec![Some(1), Some(2), None, Some(100)]) + .with_precision_and_scale(38, 5) + .unwrap(), + ) as ArrayRef; let to_write = RecordBatch::try_new(arrow_schema.clone(), vec![ - struct_float_field_col, - struct_nested_float_field_col, + col0, col1, col2, col3, col4, col5, col6, col7, col8, col9, col10, col11, col12, col13, + col14, col15, col16, ]) .unwrap(); // write data let mut pw = ParquetWriterBuilder::new( WriterProperties::builder().build(), - Arc::new(to_write.schema().as_ref().try_into().unwrap()), + Arc::new(schema), file_io.clone(), - location_gen, + loccation_gen, file_name_gen, ) .build() .await?; - pw.write(&to_write).await?; let res = pw.close().await?; assert_eq!(res.len(), 1); @@ -1247,158 +1236,140 @@ mod tests { // check data file assert_eq!(data_file.record_count(), 4); - assert_eq!(*data_file.value_counts(), HashMap::from([(4, 4), (7, 4)])); + assert!(data_file.value_counts().iter().all(|(_, &v)| { v == 4 })); + assert!(data_file + .null_value_counts() + .iter() + .all(|(_, &v)| { v == 1 })); assert_eq!( *data_file.lower_bounds(), - HashMap::from([(4, Datum::float(1.0)), (7, Datum::float(1.0)),]) - ); - assert_eq!( - *data_file.upper_bounds(), - HashMap::from([(4, Datum::float(2.0)), (7, Datum::float(2.0)),]) - ); - assert_eq!( - *data_file.null_value_counts(), - HashMap::from([(4, 0), (7, 0)]) - ); - assert_eq!( - *data_file.nan_value_counts(), - HashMap::from([(4, 1), (7, 1)]) - ); - - // check the written file - let expect_batch = concat_batches(&arrow_schema, vec![&to_write]).unwrap(); - check_parquet_data_file(&file_io, &data_file, &expect_batch).await; - - Ok(()) - } + HashMap::from([ + (0, Datum::bool(false)), + (1, Datum::int(1)), + (2, Datum::long(1)), + (3, Datum::float(0.5)), + (4, Datum::double(0.5)), + (5, Datum::string("a")), + (6, Datum::binary(vec![])), + (7, Datum::date(0)), + (8, Datum::time_micros(0).unwrap()), + (9, Datum::timestamp_micros(0)), + (10, Datum::timestamptz_micros(0)), + (11, Datum::timestamp_nanos(0)), + (12, Datum::timestamptz_nanos(0)), + ( + 13, + Datum::new( + PrimitiveType::Decimal { + precision: 10, + scale: 5 + }, + PrimitiveLiteral::Int128(1) + ) + ), + (14, Datum::uuid(Uuid::from_u128(0))), + (15, Datum::fixed(vec![1, 2, 3, 4, 5, 6, 7, 8, 9, 10])), + ( + 16, + Datum::new( + PrimitiveType::Decimal { + precision: 38, + scale: 5 + }, + PrimitiveLiteral::Int128(1) + ) + ), + ]) + ); + assert_eq!( + *data_file.upper_bounds(), + HashMap::from([ + (0, Datum::bool(true)), + (1, Datum::int(4)), + (2, Datum::long(4)), + (3, Datum::float(3.5)), + (4, Datum::double(3.5)), + (5, Datum::string("d")), + (6, Datum::binary(vec![122, 122, 122, 122])), + (7, Datum::date(3)), + (8, Datum::time_micros(3).unwrap()), + (9, Datum::timestamp_micros(3)), + (10, Datum::timestamptz_micros(3)), + (11, Datum::timestamp_nanos(3)), + (12, Datum::timestamptz_nanos(3)), + ( + 13, + Datum::new( + PrimitiveType::Decimal { + precision: 10, + scale: 5 + }, + PrimitiveLiteral::Int128(100) + ) + ), + (14, Datum::uuid(Uuid::from_u128(3))), + ( + 15, + Datum::fixed(vec![21, 22, 23, 24, 25, 26, 27, 28, 29, 30]) + ), + ( + 16, + Datum::new( + PrimitiveType::Decimal { + precision: 38, + scale: 5 + }, + PrimitiveLiteral::Int128(100) + ) + ), + ]) + ); + + // check the written file + check_parquet_data_file(&file_io, &data_file, &to_write).await; + + Ok(()) + } #[tokio::test] - async fn test_nan_val_cnts_list_type() -> Result<()> { + async fn test_decimal_bound() -> Result<()> { let temp_dir = TempDir::new().unwrap(); let file_io = FileIOBuilder::new_fs_io().build().unwrap(); - let location_gen = + let loccation_gen = MockLocationGenerator::new(temp_dir.path().to_str().unwrap().to_string()); let file_name_gen = DefaultFileNameGenerator::new("test".to_string(), None, DataFileFormat::Parquet); - let schema_list_float_field = Field::new("element", DataType::Float32, true).with_metadata( - HashMap::from([(PARQUET_FIELD_ID_META_KEY.to_string(), "1".to_string())]), + // test 1.1 and 2.2 + let schema = Arc::new( + Schema::builder() + .with_fields(vec![NestedField::optional( + 0, + "decimal", + Type::Primitive(PrimitiveType::Decimal { + precision: 28, + scale: 10, + }), + ) + .into()]) + .build() + .unwrap(), ); - - let schema_struct_list_float_field = Field::new("element", DataType::Float32, true) - .with_metadata(HashMap::from([( - PARQUET_FIELD_ID_META_KEY.to_string(), - "4".to_string(), - )])); - - let schema_struct_list_field = Fields::from(vec![Field::new_list( - "col2", - schema_struct_list_float_field.clone(), - true, - ) - .with_metadata(HashMap::from([( - PARQUET_FIELD_ID_META_KEY.to_string(), - "3".to_string(), - )]))]); - - let arrow_schema = { - let fields = vec![ - Field::new_list("col0", schema_list_float_field.clone(), true).with_metadata( - HashMap::from([(PARQUET_FIELD_ID_META_KEY.to_string(), "0".to_string())]), - ), - Field::new_struct("col1", schema_struct_list_field.clone(), true) - .with_metadata(HashMap::from([( - PARQUET_FIELD_ID_META_KEY.to_string(), - "2".to_string(), - )])) - .clone(), - // Field::new_large_list("col3", schema_large_list_float_field.clone(), true).with_metadata( - // HashMap::from([(PARQUET_FIELD_ID_META_KEY.to_string(), "5".to_string())]), - // ).clone(), - ]; - Arc::new(arrow_schema::Schema::new(fields)) - }; - - let list_parts = ListArray::from_iter_primitive::(vec![Some(vec![ - Some(1.0_f32), - Some(f32::NAN), - Some(2.0), - Some(2.0), - ])]) - .into_parts(); - - let list_float_field_col = Arc::new({ - let list_parts = list_parts.clone(); - ListArray::new( - { - if let DataType::List(field) = arrow_schema.field(0).data_type() { - field.clone() - } else { - unreachable!() - } - }, - list_parts.1, - list_parts.2, - list_parts.3, - ) - }) as ArrayRef; - - let struct_list_fields_schema = - if let DataType::Struct(fields) = arrow_schema.field(1).data_type() { - fields.clone() - } else { - unreachable!() - }; - - let struct_list_float_field_col = Arc::new({ - ListArray::new( - { - if let DataType::List(field) = struct_list_fields_schema - .first() - .expect("could not find first list field") - .data_type() - { - field.clone() - } else { - unreachable!() - } - }, - list_parts.1, - list_parts.2, - list_parts.3, - ) - }) as ArrayRef; - - let struct_list_float_field_col = Arc::new(StructArray::new( - struct_list_fields_schema, - vec![struct_list_float_field_col.clone()], - None, - )) as ArrayRef; - - let to_write = RecordBatch::try_new(arrow_schema.clone(), vec![ - list_float_field_col, - struct_list_float_field_col, - // large_list_float_field_col, - ]) - .expect("Could not form record batch"); - - // write data + let arrow_schema: ArrowSchemaRef = Arc::new(schema_to_arrow_schema(&schema).unwrap()); let mut pw = ParquetWriterBuilder::new( WriterProperties::builder().build(), - Arc::new( - to_write - .schema() - .as_ref() - .try_into() - .expect("Could not convert iceberg schema"), - ), + schema.clone(), file_io.clone(), - location_gen, - file_name_gen, + loccation_gen.clone(), + file_name_gen.clone(), ) .build() .await?; - + let col0 = Arc::new( + Decimal128Array::from(vec![Some(22000000000), Some(11000000000)]) + .with_data_type(DataType::Decimal128(28, 10)), + ) as ArrayRef; + let to_write = RecordBatch::try_new(arrow_schema.clone(), vec![col0]).unwrap(); pw.write(&to_write).await?; let res = pw.close().await?; assert_eq!(res.len(), 1); @@ -1410,172 +1381,47 @@ mod tests { .partition(Struct::empty()) .build() .unwrap(); - - // check data file - assert_eq!(data_file.record_count(), 1); - assert_eq!(*data_file.value_counts(), HashMap::from([(1, 4), (4, 4)])); - assert_eq!( - *data_file.lower_bounds(), - HashMap::from([(1, Datum::float(1.0)), (4, Datum::float(1.0))]) - ); - assert_eq!( - *data_file.upper_bounds(), - HashMap::from([(1, Datum::float(2.0)), (4, Datum::float(2.0))]) - ); assert_eq!( - *data_file.null_value_counts(), - HashMap::from([(1, 0), (4, 0)]) + data_file.upper_bounds().get(&0), + Some(Datum::decimal_with_precision(Decimal::new(22000000000_i64, 10), 28).unwrap()) + .as_ref() ); assert_eq!( - *data_file.nan_value_counts(), - HashMap::from([(1, 1), (4, 1)]) + data_file.lower_bounds().get(&0), + Some(Datum::decimal_with_precision(Decimal::new(11000000000_i64, 10), 28).unwrap()) + .as_ref() ); - // check the written file - let expect_batch = concat_batches(&arrow_schema, vec![&to_write]).unwrap(); - check_parquet_data_file(&file_io, &data_file, &expect_batch).await; - - Ok(()) - } - - macro_rules! construct_map_arr { - ($map_key_field_schema:ident, $map_value_field_schema:ident) => {{ - let int_builder = Int32Builder::new(); - let float_builder = Float32Builder::with_capacity(4); - let mut builder = MapBuilder::new(None, int_builder, float_builder); - builder.keys().append_value(1); - builder.values().append_value(1.0_f32); - builder.append(true).unwrap(); - builder.keys().append_value(2); - builder.values().append_value(f32::NAN); - builder.append(true).unwrap(); - builder.keys().append_value(3); - builder.values().append_value(2.0); - builder.append(true).unwrap(); - builder.keys().append_value(4); - builder.values().append_value(2.0); - builder.append(true).unwrap(); - let array = builder.finish(); - - let (_field, offsets, entries, nulls, ordered) = array.into_parts(); - let new_struct_fields_schema = - Fields::from(vec![$map_key_field_schema, $map_value_field_schema]); - - let entries = { - let (_, arrays, nulls) = entries.into_parts(); - StructArray::new(new_struct_fields_schema.clone(), arrays, nulls) - }; - - let field = Arc::new(Field::new( - DEFAULT_MAP_FIELD_NAME, - DataType::Struct(new_struct_fields_schema), - false, - )); - - Arc::new(MapArray::new(field, offsets, entries, nulls, ordered)) - }}; - } - - #[tokio::test] - async fn test_nan_val_cnts_map_type() -> Result<()> { - let temp_dir = TempDir::new().unwrap(); - let file_io = FileIOBuilder::new_fs_io().build().unwrap(); - 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); - - let map_key_field_schema = - Field::new(MAP_KEY_FIELD_NAME, DataType::Int32, false).with_metadata(HashMap::from([ - (PARQUET_FIELD_ID_META_KEY.to_string(), "1".to_string()), - ])); - - let map_value_field_schema = - Field::new(MAP_VALUE_FIELD_NAME, DataType::Float32, true).with_metadata(HashMap::from( - [(PARQUET_FIELD_ID_META_KEY.to_string(), "2".to_string())], - )); - - let struct_map_key_field_schema = - Field::new(MAP_KEY_FIELD_NAME, DataType::Int32, false).with_metadata(HashMap::from([ - (PARQUET_FIELD_ID_META_KEY.to_string(), "6".to_string()), - ])); - - let struct_map_value_field_schema = - Field::new(MAP_VALUE_FIELD_NAME, DataType::Float32, true).with_metadata(HashMap::from( - [(PARQUET_FIELD_ID_META_KEY.to_string(), "7".to_string())], - )); - - let schema_struct_map_field = Fields::from(vec![Field::new_map( - "col3", - DEFAULT_MAP_FIELD_NAME, - struct_map_key_field_schema.clone(), - struct_map_value_field_schema.clone(), - false, - false, - ) - .with_metadata(HashMap::from([( - PARQUET_FIELD_ID_META_KEY.to_string(), - "5".to_string(), - )]))]); - - let arrow_schema = { - let fields = vec![ - Field::new_map( - "col0", - DEFAULT_MAP_FIELD_NAME, - map_key_field_schema.clone(), - map_value_field_schema.clone(), - false, - false, + // test -1.1 and -2.2 + let schema = Arc::new( + Schema::builder() + .with_fields(vec![NestedField::optional( + 0, + "decimal", + Type::Primitive(PrimitiveType::Decimal { + precision: 28, + scale: 10, + }), ) - .with_metadata(HashMap::from([( - PARQUET_FIELD_ID_META_KEY.to_string(), - "0".to_string(), - )])), - Field::new_struct("col1", schema_struct_map_field.clone(), true) - .with_metadata(HashMap::from([( - PARQUET_FIELD_ID_META_KEY.to_string(), - "3".to_string(), - )])) - .clone(), - ]; - Arc::new(arrow_schema::Schema::new(fields)) - }; - - let map_array = construct_map_arr!(map_key_field_schema, map_value_field_schema); - - let struct_map_arr = - construct_map_arr!(struct_map_key_field_schema, struct_map_value_field_schema); - - let struct_list_float_field_col = Arc::new(StructArray::new( - schema_struct_map_field, - vec![struct_map_arr], - None, - )) as ArrayRef; - - let to_write = RecordBatch::try_new(arrow_schema.clone(), vec![ - map_array, - struct_list_float_field_col, - ]) - .expect("Could not form record batch"); - - // write data + .into()]) + .build() + .unwrap(), + ); + let arrow_schema: ArrowSchemaRef = Arc::new(schema_to_arrow_schema(&schema).unwrap()); let mut pw = ParquetWriterBuilder::new( WriterProperties::builder().build(), - Arc::new( - to_write - .schema() - .as_ref() - .try_into() - .expect("Could not convert iceberg schema"), - ), + schema.clone(), file_io.clone(), - location_gen, - file_name_gen, + loccation_gen.clone(), + file_name_gen.clone(), ) .build() .await?; - + let col0 = Arc::new( + Decimal128Array::from(vec![Some(-22000000000), Some(-11000000000)]) + .with_data_type(DataType::Decimal128(28, 10)), + ) as ArrayRef; + let to_write = RecordBatch::try_new(arrow_schema.clone(), vec![col0]).unwrap(); pw.write(&to_write).await?; let res = pw.close().await?; assert_eq!(res.len(), 1); @@ -1587,116 +1433,286 @@ mod tests { .partition(Struct::empty()) .build() .unwrap(); - - // check data file - assert_eq!(data_file.record_count(), 4); - assert_eq!( - *data_file.value_counts(), - HashMap::from([(1, 4), (2, 4), (6, 4), (7, 4)]) - ); - assert_eq!( - *data_file.lower_bounds(), - HashMap::from([ - (1, Datum::int(1)), - (2, Datum::float(1.0)), - (6, Datum::int(1)), - (7, Datum::float(1.0)) - ]) - ); - assert_eq!( - *data_file.upper_bounds(), - HashMap::from([ - (1, Datum::int(4)), - (2, Datum::float(2.0)), - (6, Datum::int(4)), - (7, Datum::float(2.0)) - ]) - ); assert_eq!( - *data_file.null_value_counts(), - HashMap::from([(1, 0), (2, 0), (6, 0), (7, 0)]) + data_file.upper_bounds().get(&0), + Some(Datum::decimal_with_precision(Decimal::new(-11000000000_i64, 10), 28).unwrap()) + .as_ref() ); assert_eq!( - *data_file.nan_value_counts(), - HashMap::from([(2, 1), (7, 1)]) + data_file.lower_bounds().get(&0), + Some(Datum::decimal_with_precision(Decimal::new(-22000000000_i64, 10), 28).unwrap()) + .as_ref() ); - // check the written file - let expect_batch = concat_batches(&arrow_schema, vec![&to_write]).unwrap(); - check_parquet_data_file(&file_io, &data_file, &expect_batch).await; - - Ok(()) - } - - #[tokio::test] - async fn test_parquet_writer() -> Result<()> { - let temp_dir = TempDir::new().unwrap(); - let file_io = FileIOBuilder::new_fs_io().build().unwrap(); - 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); - - // 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())]), - ), - ]; - Arc::new(arrow_schema::Schema::new(fields)) - }; - let col = Arc::new(Int64Array::from_iter_values(0..1024)) as ArrayRef; - let null_col = Arc::new(Int64Array::new_null(1024)) as ArrayRef; - let to_write = RecordBatch::try_new(schema.clone(), vec![col]).unwrap(); - let to_write_null = RecordBatch::try_new(schema.clone(), vec![null_col]).unwrap(); - - // write data + // test max and min of rust_decimal + let decimal_max = Decimal::MAX; + let decimal_min = Decimal::MIN; + assert_eq!(decimal_max.scale(), decimal_min.scale()); + let schema = Arc::new( + Schema::builder() + .with_fields(vec![NestedField::optional( + 0, + "decimal", + Type::Primitive(PrimitiveType::Decimal { + precision: 38, + scale: decimal_max.scale(), + }), + ) + .into()]) + .build() + .unwrap(), + ); + let arrow_schema: ArrowSchemaRef = Arc::new(schema_to_arrow_schema(&schema).unwrap()); let mut pw = ParquetWriterBuilder::new( WriterProperties::builder().build(), - Arc::new(to_write.schema().as_ref().try_into().unwrap()), + schema, file_io.clone(), - location_gen, + loccation_gen, file_name_gen, ) .build() .await?; + let col0 = Arc::new( + Decimal128Array::from(vec![ + Some(decimal_max.mantissa()), + Some(decimal_min.mantissa()), + ]) + .with_data_type(DataType::Decimal128(38, 0)), + ) as ArrayRef; + let to_write = RecordBatch::try_new(arrow_schema.clone(), vec![col0]).unwrap(); pw.write(&to_write).await?; - pw.write(&to_write_null).await?; let res = pw.close().await?; assert_eq!(res.len(), 1); let data_file = res .into_iter() .next() .unwrap() - // Put dummy field for build successfully. .content(crate::spec::DataContentType::Data) .partition(Struct::empty()) .build() .unwrap(); - - // check data file - assert_eq!(data_file.record_count(), 2048); - assert_eq!(*data_file.value_counts(), HashMap::from([(0, 2048)])); assert_eq!( - *data_file.lower_bounds(), - HashMap::from([(0, Datum::long(0))]) + data_file.upper_bounds().get(&0), + Some(Datum::decimal(decimal_max).unwrap()).as_ref() + ); + assert_eq!( + data_file.lower_bounds().get(&0), + Some(Datum::decimal(decimal_min).unwrap()).as_ref() + ); + + // test max and min for scale 38 + // # TODO + // Readd this case after resolve https://github.com/apache/iceberg-rust/issues/669 + // let schema = Arc::new( + // Schema::builder() + // .with_fields(vec![NestedField::optional( + // 0, + // "decimal", + // Type::Primitive(PrimitiveType::Decimal { + // precision: 38, + // scale: 0, + // }), + // ) + // .into()]) + // .build() + // .unwrap(), + // ); + // let arrow_schema: ArrowSchemaRef = Arc::new(schema_to_arrow_schema(&schema).unwrap()); + // let mut pw = ParquetWriterBuilder::new( + // WriterProperties::builder().build(), + // schema, + // file_io.clone(), + // loccation_gen, + // file_name_gen, + // ) + // .build() + // .await?; + // let col0 = Arc::new( + // Decimal128Array::from(vec![ + // Some(99999999999999999999999999999999999999_i128), + // Some(-99999999999999999999999999999999999999_i128), + // ]) + // .with_data_type(DataType::Decimal128(38, 0)), + // ) as ArrayRef; + // let to_write = RecordBatch::try_new(arrow_schema.clone(), vec![col0]).unwrap(); + // pw.write(&to_write).await?; + // let res = pw.close().await?; + // assert_eq!(res.len(), 1); + // let data_file = res + // .into_iter() + // .next() + // .unwrap() + // .content(crate::spec::DataContentType::Data) + // .partition(Struct::empty()) + // .build() + // .unwrap(); + // assert_eq!( + // data_file.upper_bounds().get(&0), + // Some(Datum::new( + // PrimitiveType::Decimal { + // precision: 38, + // scale: 0 + // }, + // PrimitiveLiteral::Int128(99999999999999999999999999999999999999_i128) + // )) + // .as_ref() + // ); + // assert_eq!( + // data_file.lower_bounds().get(&0), + // Some(Datum::new( + // PrimitiveType::Decimal { + // precision: 38, + // scale: 0 + // }, + // PrimitiveLiteral::Int128(-99999999999999999999999999999999999999_i128) + // )) + // .as_ref() + // ); + + Ok(()) + } + + #[tokio::test] + async fn test_empty_write() -> Result<()> { + let temp_dir = TempDir::new().unwrap(); + let file_io = FileIOBuilder::new_fs_io().build().unwrap(); + 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); + + // Test that file will create if data to write + 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())]), + ), + ]; + Arc::new(arrow_schema::Schema::new(fields)) + }; + let col = Arc::new(Int64Array::from_iter_values(0..1024)) as ArrayRef; + let to_write = RecordBatch::try_new(schema.clone(), vec![col]).unwrap(); + let mut pw = ParquetWriterBuilder::new( + WriterProperties::builder().build(), + Arc::new(to_write.schema().as_ref().try_into().unwrap()), + file_io.clone(), + location_gen.clone(), + file_name_gen, + ) + .build() + .await?; + pw.write(&to_write).await?; + let file_path = pw.out_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 pw = ParquetWriterBuilder::new( + WriterProperties::builder().build(), + Arc::new(to_write.schema().as_ref().try_into().unwrap()), + file_io.clone(), + location_gen, + file_name_gen, + ) + .build() + .await?; + let file_path = pw.out_file.location().to_string(); + pw.close().await.unwrap(); + assert!(!file_io.exists(file_path).await.unwrap()); + + Ok(()) + } + + #[tokio::test] + async fn test_nan_val_cnts_primitive_type() -> Result<()> { + let temp_dir = TempDir::new().unwrap(); + let file_io = FileIOBuilder::new_fs_io().build().unwrap(); + 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); + // + // prepare data + let arrow_schema = { + let fields = vec![ + Field::new("col", arrow_schema::DataType::Float32, false).with_metadata( + HashMap::from([(PARQUET_FIELD_ID_META_KEY.to_string(), "0".to_string())]), + ), + Field::new("col2", arrow_schema::DataType::Float64, false).with_metadata( + HashMap::from([(PARQUET_FIELD_ID_META_KEY.to_string(), "1".to_string())]), + ), + ]; + Arc::new(arrow_schema::Schema::new(fields)) + }; + + let float_32_col = Arc::new(Float32Array::from_iter_values_with_nulls( + [1.0_f32, f32::NAN, 2.0, 2.0].into_iter(), + None, + )) as ArrayRef; + + let float_64_col = Arc::new(Float64Array::from_iter_values_with_nulls( + [1.0_f64, f64::NAN, 2.0, 2.0].into_iter(), + None, + )) as ArrayRef; + + let to_write = + RecordBatch::try_new(arrow_schema.clone(), vec![float_32_col, float_64_col]).unwrap(); + + // write data + let mut pw = ParquetWriterBuilder::new( + WriterProperties::builder().build(), + Arc::new(to_write.schema().as_ref().try_into().unwrap()), + file_io.clone(), + location_gen, + file_name_gen, + ) + .build() + .await?; + + pw.write(&to_write).await?; + let res = pw.close().await?; + assert_eq!(res.len(), 1); + let data_file = res + .into_iter() + .next() + .unwrap() + // Put dummy field for build successfully. + .content(crate::spec::DataContentType::Data) + .partition(Struct::empty()) + .build() + .unwrap(); + + // check data file + assert_eq!(data_file.record_count(), 4); + assert_eq!(*data_file.value_counts(), HashMap::from([(0, 4), (1, 4)])); + assert_eq!( + *data_file.lower_bounds(), + HashMap::from([(0, Datum::float(1.0)), (1, Datum::double(1.0)),]) ); assert_eq!( *data_file.upper_bounds(), - HashMap::from([(0, Datum::long(1023))]) + HashMap::from([(0, Datum::float(2.0)), (1, Datum::double(2.0)),]) + ); + assert_eq!( + *data_file.null_value_counts(), + HashMap::from([(0, 0), (1, 0)]) + ); + assert_eq!( + *data_file.nan_value_counts(), + HashMap::from([(0, 1), (1, 1)]) ); - assert_eq!(*data_file.null_value_counts(), HashMap::from([(0, 1024)])); // check the written file - let expect_batch = concat_batches(&schema, vec![&to_write, &to_write_null]).unwrap(); + let expect_batch = concat_batches(&arrow_schema, vec![&to_write]).unwrap(); check_parquet_data_file(&file_io, &data_file, &expect_batch).await; Ok(()) } #[tokio::test] - async fn test_parquet_writer_with_complex_schema() -> Result<()> { + async fn test_nan_val_cnts_struct_type() -> Result<()> { let temp_dir = TempDir::new().unwrap(); let file_io = FileIOBuilder::new_fs_io().build().unwrap(); let location_gen = @@ -1704,159 +1720,93 @@ mod tests { let file_name_gen = DefaultFileNameGenerator::new("test".to_string(), None, DataFileFormat::Parquet); + let schema_struct_float_fields = + Fields::from(vec![Field::new("col4", DataType::Float32, false) + .with_metadata(HashMap::from([( + PARQUET_FIELD_ID_META_KEY.to_string(), + "4".to_string(), + )]))]); + + let schema_struct_nested_float_fields = + Fields::from(vec![Field::new("col7", DataType::Float32, false) + .with_metadata(HashMap::from([( + PARQUET_FIELD_ID_META_KEY.to_string(), + "7".to_string(), + )]))]); + + let schema_struct_nested_fields = Fields::from(vec![Field::new( + "col6", + arrow_schema::DataType::Struct(schema_struct_nested_float_fields.clone()), + false, + ) + .with_metadata(HashMap::from([( + PARQUET_FIELD_ID_META_KEY.to_string(), + "6".to_string(), + )]))]); + // prepare data - let schema = nested_schema_for_test(); - let arrow_schema: ArrowSchemaRef = Arc::new((&schema).try_into().unwrap()); - let col0 = Arc::new(Int64Array::from_iter_values(0..1024)) as ArrayRef; - let col1 = Arc::new(StructArray::new( - { - if let DataType::Struct(fields) = arrow_schema.field(1).data_type() { - fields.clone() - } else { - unreachable!() - } - }, - vec![ - Arc::new(Int64Array::from_iter_values(0..1024)), - Arc::new(Int64Array::from_iter_values(0..1024)), - ], + let arrow_schema = { + let fields = vec![ + Field::new( + "col3", + arrow_schema::DataType::Struct(schema_struct_float_fields.clone()), + false, + ) + .with_metadata(HashMap::from([( + PARQUET_FIELD_ID_META_KEY.to_string(), + "3".to_string(), + )])), + Field::new( + "col5", + arrow_schema::DataType::Struct(schema_struct_nested_fields.clone()), + false, + ) + .with_metadata(HashMap::from([( + PARQUET_FIELD_ID_META_KEY.to_string(), + "5".to_string(), + )])), + ]; + Arc::new(arrow_schema::Schema::new(fields)) + }; + + let float_32_col = Arc::new(Float32Array::from_iter_values_with_nulls( + [1.0_f32, f32::NAN, 2.0, 2.0].into_iter(), None, - )); - let col2 = Arc::new(arrow_array::StringArray::from_iter_values( - (0..1024).map(|n| n.to_string()), )) as ArrayRef; - let col3 = Arc::new({ - let list_parts = arrow_array::ListArray::from_iter_primitive::( - (0..1024).map(|n| Some(vec![Some(n)])), - ) - .into_parts(); - arrow_array::ListArray::new( - { - if let DataType::List(field) = arrow_schema.field(3).data_type() { - field.clone() - } else { - unreachable!() - } - }, - list_parts.1, - list_parts.2, - list_parts.3, - ) - }) as ArrayRef; - let col4 = Arc::new(StructArray::new( - { - if let DataType::Struct(fields) = arrow_schema.field(4).data_type() { - fields.clone() - } else { - unreachable!() - } - }, + + let struct_float_field_col = Arc::new(StructArray::new( + schema_struct_float_fields, + vec![float_32_col.clone()], + None, + )) as ArrayRef; + + let struct_nested_float_field_col = Arc::new(StructArray::new( + schema_struct_nested_fields, vec![Arc::new(StructArray::new( - { - if let DataType::Struct(fields) = arrow_schema.field(4).data_type() { - if let DataType::Struct(fields) = fields[0].data_type() { - fields.clone() - } else { - unreachable!() - } - } else { - unreachable!() - } - }, - vec![Arc::new(Int64Array::from_iter_values(0..1024))], + schema_struct_nested_float_fields, + vec![float_32_col.clone()], None, - ))], + )) as ArrayRef], None, - )); - let col5 = Arc::new({ - let mut map_array_builder = arrow_array::builder::MapBuilder::new( - None, - arrow_array::builder::StringBuilder::new(), - arrow_array::builder::ListBuilder::new(arrow_array::builder::PrimitiveBuilder::< - Int64Type, - >::new()), - ); - for i in 0..1024 { - map_array_builder.keys().append_value(i.to_string()); - map_array_builder - .values() - .append_value(vec![Some(i as i64); i + 1]); - map_array_builder.append(true)?; - } - let (_, offset_buffer, struct_array, null_buffer, ordered) = - map_array_builder.finish().into_parts(); - let struct_array = { - let (_, mut arrays, nulls) = struct_array.into_parts(); - let list_array = { - let list_array = arrays[1] - .as_any() - .downcast_ref::() - .unwrap() - .clone(); - let (_, offsets, array, nulls) = list_array.into_parts(); - let list_field = { - if let DataType::Map(map_field, _) = arrow_schema.field(5).data_type() { - if let DataType::Struct(fields) = map_field.data_type() { - if let DataType::List(list_field) = fields[1].data_type() { - list_field.clone() - } else { - unreachable!() - } - } else { - unreachable!() - } - } else { - unreachable!() - } - }; - ListArray::new(list_field, offsets, array, nulls) - }; - arrays[1] = Arc::new(list_array) as ArrayRef; - StructArray::new( - { - if let DataType::Map(map_field, _) = arrow_schema.field(5).data_type() { - if let DataType::Struct(fields) = map_field.data_type() { - fields.clone() - } else { - unreachable!() - } - } else { - unreachable!() - } - }, - arrays, - nulls, - ) - }; - arrow_array::MapArray::new( - { - if let DataType::Map(map_field, _) = arrow_schema.field(5).data_type() { - map_field.clone() - } else { - unreachable!() - } - }, - offset_buffer, - struct_array, - null_buffer, - ordered, - ) - }) as ArrayRef; + )) as ArrayRef; + let to_write = RecordBatch::try_new(arrow_schema.clone(), vec![ - col0, col1, col2, col3, col4, col5, + struct_float_field_col, + struct_nested_float_field_col, ]) .unwrap(); // write data let mut pw = ParquetWriterBuilder::new( WriterProperties::builder().build(), - Arc::new(schema), + Arc::new(to_write.schema().as_ref().try_into().unwrap()), file_io.clone(), location_gen, file_name_gen, ) .build() .await?; + pw.write(&to_write).await?; let res = pw.close().await?; assert_eq!(res.len(), 1); @@ -1871,182 +1821,159 @@ mod tests { .unwrap(); // check data file - assert_eq!(data_file.record_count(), 1024); - assert_eq!( - *data_file.value_counts(), - HashMap::from([ - (0, 1024), - (5, 1024), - (6, 1024), - (2, 1024), - (7, 1024), - (9, 1024), - (11, 1024), - (13, (1..1025).sum()), - ]) - ); + assert_eq!(data_file.record_count(), 4); + assert_eq!(*data_file.value_counts(), HashMap::from([(4, 4), (7, 4)])); assert_eq!( *data_file.lower_bounds(), - HashMap::from([ - (0, Datum::long(0)), - (5, Datum::long(0)), - (6, Datum::long(0)), - (2, Datum::string("0")), - (7, Datum::long(0)), - (9, Datum::long(0)), - (11, Datum::string("0")), - (13, Datum::long(0)) - ]) + HashMap::from([(4, Datum::float(1.0)), (7, Datum::float(1.0)),]) ); assert_eq!( *data_file.upper_bounds(), - HashMap::from([ - (0, Datum::long(1023)), - (5, Datum::long(1023)), - (6, Datum::long(1023)), - (2, Datum::string("999")), - (7, Datum::long(1023)), - (9, Datum::long(1023)), - (11, Datum::string("999")), - (13, Datum::long(1023)) - ]) + HashMap::from([(4, Datum::float(2.0)), (7, Datum::float(2.0)),]) + ); + assert_eq!( + *data_file.null_value_counts(), + HashMap::from([(4, 0), (7, 0)]) + ); + assert_eq!( + *data_file.nan_value_counts(), + HashMap::from([(4, 1), (7, 1)]) ); // check the written file - check_parquet_data_file(&file_io, &data_file, &to_write).await; + let expect_batch = concat_batches(&arrow_schema, vec![&to_write]).unwrap(); + check_parquet_data_file(&file_io, &data_file, &expect_batch).await; Ok(()) } #[tokio::test] - async fn test_all_type_for_write() -> Result<()> { + async fn test_nan_val_cnts_list_type() -> Result<()> { let temp_dir = TempDir::new().unwrap(); let file_io = FileIOBuilder::new_fs_io().build().unwrap(); - let loccation_gen = + 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); - // prepare data - // generate iceberg schema for all type - let schema = schema_for_all_type(); - let arrow_schema: ArrowSchemaRef = Arc::new((&schema).try_into().unwrap()); - let col0 = Arc::new(BooleanArray::from(vec![ - Some(true), - Some(false), - None, - Some(true), - ])) as ArrayRef; - let col1 = Arc::new(Int32Array::from(vec![Some(1), Some(2), None, Some(4)])) as ArrayRef; - let col2 = Arc::new(Int64Array::from(vec![Some(1), Some(2), None, Some(4)])) as ArrayRef; - let col3 = Arc::new(arrow_array::Float32Array::from(vec![ - Some(0.5), + let schema_list_float_field = Field::new("element", DataType::Float32, true).with_metadata( + HashMap::from([(PARQUET_FIELD_ID_META_KEY.to_string(), "1".to_string())]), + ); + + let schema_struct_list_float_field = Field::new("element", DataType::Float32, true) + .with_metadata(HashMap::from([( + PARQUET_FIELD_ID_META_KEY.to_string(), + "4".to_string(), + )])); + + let schema_struct_list_field = Fields::from(vec![Field::new_list( + "col2", + schema_struct_list_float_field.clone(), + true, + ) + .with_metadata(HashMap::from([( + PARQUET_FIELD_ID_META_KEY.to_string(), + "3".to_string(), + )]))]); + + let arrow_schema = { + let fields = vec![ + Field::new_list("col0", schema_list_float_field.clone(), true).with_metadata( + HashMap::from([(PARQUET_FIELD_ID_META_KEY.to_string(), "0".to_string())]), + ), + Field::new_struct("col1", schema_struct_list_field.clone(), true) + .with_metadata(HashMap::from([( + PARQUET_FIELD_ID_META_KEY.to_string(), + "2".to_string(), + )])) + .clone(), + // Field::new_large_list("col3", schema_large_list_float_field.clone(), true).with_metadata( + // HashMap::from([(PARQUET_FIELD_ID_META_KEY.to_string(), "5".to_string())]), + // ).clone(), + ]; + Arc::new(arrow_schema::Schema::new(fields)) + }; + + let list_parts = ListArray::from_iter_primitive::(vec![Some(vec![ + Some(1.0_f32), + Some(f32::NAN), Some(2.0), - None, - Some(3.5), - ])) as ArrayRef; - let col4 = Arc::new(arrow_array::Float64Array::from(vec![ - Some(0.5), Some(2.0), - None, - Some(3.5), - ])) as ArrayRef; - let col5 = Arc::new(arrow_array::StringArray::from(vec![ - Some("a"), - Some("b"), - None, - Some("d"), - ])) as ArrayRef; - let col6 = Arc::new(arrow_array::LargeBinaryArray::from_opt_vec(vec![ - Some(b"one"), - None, - Some(b""), - Some(b"zzzz"), - ])) as ArrayRef; - let col7 = Arc::new(arrow_array::Date32Array::from(vec![ - Some(0), - Some(1), - None, - Some(3), - ])) as ArrayRef; - let col8 = Arc::new(arrow_array::Time64MicrosecondArray::from(vec![ - Some(0), - Some(1), - None, - Some(3), - ])) as ArrayRef; - let col9 = Arc::new(arrow_array::TimestampMicrosecondArray::from(vec![ - Some(0), - Some(1), - None, - Some(3), - ])) as ArrayRef; - let col10 = Arc::new( - arrow_array::TimestampMicrosecondArray::from(vec![Some(0), Some(1), None, Some(3)]) - .with_timezone_utc(), - ) as ArrayRef; - let col11 = Arc::new(arrow_array::TimestampNanosecondArray::from(vec![ - Some(0), - Some(1), - None, - Some(3), - ])) as ArrayRef; - let col12 = Arc::new( - arrow_array::TimestampNanosecondArray::from(vec![Some(0), Some(1), None, Some(3)]) - .with_timezone_utc(), - ) as ArrayRef; - let col13 = Arc::new( - arrow_array::Decimal128Array::from(vec![Some(1), Some(2), None, Some(100)]) - .with_precision_and_scale(10, 5) - .unwrap(), - ) as ArrayRef; - let col14 = Arc::new( - arrow_array::FixedSizeBinaryArray::try_from_sparse_iter_with_size( - vec![ - Some(Uuid::from_u128(0).as_bytes().to_vec()), - Some(Uuid::from_u128(1).as_bytes().to_vec()), - None, - Some(Uuid::from_u128(3).as_bytes().to_vec()), - ] - .into_iter(), - 16, + ])]) + .into_parts(); + + let list_float_field_col = Arc::new({ + let list_parts = list_parts.clone(); + ListArray::new( + { + if let DataType::List(field) = arrow_schema.field(0).data_type() { + field.clone() + } else { + unreachable!() + } + }, + list_parts.1, + list_parts.2, + list_parts.3, ) - .unwrap(), - ) as ArrayRef; - let col15 = Arc::new( - arrow_array::FixedSizeBinaryArray::try_from_sparse_iter_with_size( - vec![ - Some(vec![1, 2, 3, 4, 5, 6, 7, 8, 9, 10]), - Some(vec![11, 12, 13, 14, 15, 16, 17, 18, 19, 20]), - None, - Some(vec![21, 22, 23, 24, 25, 26, 27, 28, 29, 30]), - ] - .into_iter(), - 10, + }) as ArrayRef; + + let struct_list_fields_schema = + if let DataType::Struct(fields) = arrow_schema.field(1).data_type() { + fields.clone() + } else { + unreachable!() + }; + + let struct_list_float_field_col = Arc::new({ + ListArray::new( + { + if let DataType::List(field) = struct_list_fields_schema + .first() + .expect("could not find first list field") + .data_type() + { + field.clone() + } else { + unreachable!() + } + }, + list_parts.1, + list_parts.2, + list_parts.3, ) - .unwrap(), - ) as ArrayRef; - let col16 = Arc::new( - arrow_array::Decimal128Array::from(vec![Some(1), Some(2), None, Some(100)]) - .with_precision_and_scale(38, 5) - .unwrap(), - ) as ArrayRef; + }) as ArrayRef; + + let struct_list_float_field_col = Arc::new(StructArray::new( + struct_list_fields_schema, + vec![struct_list_float_field_col.clone()], + None, + )) as ArrayRef; + let to_write = RecordBatch::try_new(arrow_schema.clone(), vec![ - col0, col1, col2, col3, col4, col5, col6, col7, col8, col9, col10, col11, col12, col13, - col14, col15, col16, + list_float_field_col, + struct_list_float_field_col, + // large_list_float_field_col, ]) - .unwrap(); + .expect("Could not form record batch"); // write data let mut pw = ParquetWriterBuilder::new( WriterProperties::builder().build(), - Arc::new(schema), + Arc::new( + to_write + .schema() + .as_ref() + .try_into() + .expect("Could not convert iceberg schema"), + ), file_io.clone(), - loccation_gen, + location_gen, file_name_gen, ) .build() .await?; + pw.write(&to_write).await?; let res = pw.close().await?; assert_eq!(res.len(), 1); @@ -2054,200 +1981,176 @@ mod tests { .into_iter() .next() .unwrap() - // Put dummy field for build successfully. .content(crate::spec::DataContentType::Data) .partition(Struct::empty()) .build() .unwrap(); // check data file - assert_eq!(data_file.record_count(), 4); - assert!(data_file.value_counts().iter().all(|(_, &v)| { v == 4 })); - assert!(data_file - .null_value_counts() - .iter() - .all(|(_, &v)| { v == 1 })); + assert_eq!(data_file.record_count(), 1); + assert_eq!(*data_file.value_counts(), HashMap::from([(1, 4), (4, 4)])); assert_eq!( *data_file.lower_bounds(), - HashMap::from([ - (0, Datum::bool(false)), - (1, Datum::int(1)), - (2, Datum::long(1)), - (3, Datum::float(0.5)), - (4, Datum::double(0.5)), - (5, Datum::string("a")), - (6, Datum::binary(vec![])), - (7, Datum::date(0)), - (8, Datum::time_micros(0).unwrap()), - (9, Datum::timestamp_micros(0)), - (10, Datum::timestamptz_micros(0)), - (11, Datum::timestamp_nanos(0)), - (12, Datum::timestamptz_nanos(0)), - ( - 13, - Datum::new( - PrimitiveType::Decimal { - precision: 10, - scale: 5 - }, - PrimitiveLiteral::Int128(1) - ) - ), - (14, Datum::uuid(Uuid::from_u128(0))), - (15, Datum::fixed(vec![1, 2, 3, 4, 5, 6, 7, 8, 9, 10])), - ( - 16, - Datum::new( - PrimitiveType::Decimal { - precision: 38, - scale: 5 - }, - PrimitiveLiteral::Int128(1) - ) - ), - ]) + HashMap::from([(1, Datum::float(1.0)), (4, Datum::float(1.0))]) ); assert_eq!( *data_file.upper_bounds(), - HashMap::from([ - (0, Datum::bool(true)), - (1, Datum::int(4)), - (2, Datum::long(4)), - (3, Datum::float(3.5)), - (4, Datum::double(3.5)), - (5, Datum::string("d")), - (6, Datum::binary(vec![122, 122, 122, 122])), - (7, Datum::date(3)), - (8, Datum::time_micros(3).unwrap()), - (9, Datum::timestamp_micros(3)), - (10, Datum::timestamptz_micros(3)), - (11, Datum::timestamp_nanos(3)), - (12, Datum::timestamptz_nanos(3)), - ( - 13, - Datum::new( - PrimitiveType::Decimal { - precision: 10, - scale: 5 - }, - PrimitiveLiteral::Int128(100) - ) - ), - (14, Datum::uuid(Uuid::from_u128(3))), - ( - 15, - Datum::fixed(vec![21, 22, 23, 24, 25, 26, 27, 28, 29, 30]) - ), - ( - 16, - Datum::new( - PrimitiveType::Decimal { - precision: 38, - scale: 5 - }, - PrimitiveLiteral::Int128(100) - ) - ), - ]) + HashMap::from([(1, Datum::float(2.0)), (4, Datum::float(2.0))]) + ); + assert_eq!( + *data_file.null_value_counts(), + HashMap::from([(1, 0), (4, 0)]) + ); + assert_eq!( + *data_file.nan_value_counts(), + HashMap::from([(1, 1), (4, 1)]) ); // check the written file - check_parquet_data_file(&file_io, &data_file, &to_write).await; + let expect_batch = concat_batches(&arrow_schema, vec![&to_write]).unwrap(); + check_parquet_data_file(&file_io, &data_file, &expect_batch).await; Ok(()) } + macro_rules! construct_map_arr { + ($map_key_field_schema:ident, $map_value_field_schema:ident) => {{ + let int_builder = Int32Builder::new(); + let float_builder = Float32Builder::with_capacity(4); + let mut builder = MapBuilder::new(None, int_builder, float_builder); + builder.keys().append_value(1); + builder.values().append_value(1.0_f32); + builder.append(true).unwrap(); + builder.keys().append_value(2); + builder.values().append_value(f32::NAN); + builder.append(true).unwrap(); + builder.keys().append_value(3); + builder.values().append_value(2.0); + builder.append(true).unwrap(); + builder.keys().append_value(4); + builder.values().append_value(2.0); + builder.append(true).unwrap(); + let array = builder.finish(); + + let (_field, offsets, entries, nulls, ordered) = array.into_parts(); + let new_struct_fields_schema = + Fields::from(vec![$map_key_field_schema, $map_value_field_schema]); + + let entries = { + let (_, arrays, nulls) = entries.into_parts(); + StructArray::new(new_struct_fields_schema.clone(), arrays, nulls) + }; + + let field = Arc::new(Field::new( + DEFAULT_MAP_FIELD_NAME, + DataType::Struct(new_struct_fields_schema), + false, + )); + + Arc::new(MapArray::new(field, offsets, entries, nulls, ordered)) + }}; + } + #[tokio::test] - async fn test_decimal_bound() -> Result<()> { + async fn test_nan_val_cnts_map_type() -> Result<()> { let temp_dir = TempDir::new().unwrap(); let file_io = FileIOBuilder::new_fs_io().build().unwrap(); - let loccation_gen = + 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); - // test 1.1 and 2.2 - let schema = Arc::new( - Schema::builder() - .with_fields(vec![NestedField::optional( - 0, - "decimal", - Type::Primitive(PrimitiveType::Decimal { - precision: 28, - scale: 10, - }), - ) - .into()]) - .build() - .unwrap(), - ); - let arrow_schema: ArrowSchemaRef = Arc::new(schema_to_arrow_schema(&schema).unwrap()); - let mut pw = ParquetWriterBuilder::new( - WriterProperties::builder().build(), - schema.clone(), - file_io.clone(), - loccation_gen.clone(), - file_name_gen.clone(), + let map_key_field_schema = + Field::new(MAP_KEY_FIELD_NAME, DataType::Int32, false).with_metadata(HashMap::from([ + (PARQUET_FIELD_ID_META_KEY.to_string(), "1".to_string()), + ])); + + let map_value_field_schema = + Field::new(MAP_VALUE_FIELD_NAME, DataType::Float32, true).with_metadata(HashMap::from( + [(PARQUET_FIELD_ID_META_KEY.to_string(), "2".to_string())], + )); + + let struct_map_key_field_schema = + Field::new(MAP_KEY_FIELD_NAME, DataType::Int32, false).with_metadata(HashMap::from([ + (PARQUET_FIELD_ID_META_KEY.to_string(), "6".to_string()), + ])); + + let struct_map_value_field_schema = + Field::new(MAP_VALUE_FIELD_NAME, DataType::Float32, true).with_metadata(HashMap::from( + [(PARQUET_FIELD_ID_META_KEY.to_string(), "7".to_string())], + )); + + let schema_struct_map_field = Fields::from(vec![Field::new_map( + "col3", + DEFAULT_MAP_FIELD_NAME, + struct_map_key_field_schema.clone(), + struct_map_value_field_schema.clone(), + false, + false, ) - .build() - .await?; - let col0 = Arc::new( - Decimal128Array::from(vec![Some(22000000000), Some(11000000000)]) - .with_data_type(DataType::Decimal128(28, 10)), - ) as ArrayRef; - let to_write = RecordBatch::try_new(arrow_schema.clone(), vec![col0]).unwrap(); - pw.write(&to_write).await?; - let res = pw.close().await?; - assert_eq!(res.len(), 1); - let data_file = res - .into_iter() - .next() - .unwrap() - .content(crate::spec::DataContentType::Data) - .partition(Struct::empty()) - .build() - .unwrap(); - assert_eq!( - data_file.upper_bounds().get(&0), - Some(Datum::decimal_with_precision(Decimal::new(22000000000_i64, 10), 28).unwrap()) - .as_ref() - ); - assert_eq!( - data_file.lower_bounds().get(&0), - Some(Datum::decimal_with_precision(Decimal::new(11000000000_i64, 10), 28).unwrap()) - .as_ref() - ); + .with_metadata(HashMap::from([( + PARQUET_FIELD_ID_META_KEY.to_string(), + "5".to_string(), + )]))]); + + let arrow_schema = { + let fields = vec![ + Field::new_map( + "col0", + DEFAULT_MAP_FIELD_NAME, + map_key_field_schema.clone(), + map_value_field_schema.clone(), + false, + false, + ) + .with_metadata(HashMap::from([( + PARQUET_FIELD_ID_META_KEY.to_string(), + "0".to_string(), + )])), + Field::new_struct("col1", schema_struct_map_field.clone(), true) + .with_metadata(HashMap::from([( + PARQUET_FIELD_ID_META_KEY.to_string(), + "3".to_string(), + )])) + .clone(), + ]; + Arc::new(arrow_schema::Schema::new(fields)) + }; + + let map_array = construct_map_arr!(map_key_field_schema, map_value_field_schema); - // test -1.1 and -2.2 - let schema = Arc::new( - Schema::builder() - .with_fields(vec![NestedField::optional( - 0, - "decimal", - Type::Primitive(PrimitiveType::Decimal { - precision: 28, - scale: 10, - }), - ) - .into()]) - .build() - .unwrap(), - ); - let arrow_schema: ArrowSchemaRef = Arc::new(schema_to_arrow_schema(&schema).unwrap()); + let struct_map_arr = + construct_map_arr!(struct_map_key_field_schema, struct_map_value_field_schema); + + let struct_list_float_field_col = Arc::new(StructArray::new( + schema_struct_map_field, + vec![struct_map_arr], + None, + )) as ArrayRef; + + let to_write = RecordBatch::try_new(arrow_schema.clone(), vec![ + map_array, + struct_list_float_field_col, + ]) + .expect("Could not form record batch"); + + // write data let mut pw = ParquetWriterBuilder::new( WriterProperties::builder().build(), - schema.clone(), + Arc::new( + to_write + .schema() + .as_ref() + .try_into() + .expect("Could not convert iceberg schema"), + ), file_io.clone(), - loccation_gen.clone(), - file_name_gen.clone(), + location_gen, + file_name_gen, ) .build() .await?; - let col0 = Arc::new( - Decimal128Array::from(vec![Some(-22000000000), Some(-11000000000)]) - .with_data_type(DataType::Decimal128(28, 10)), - ) as ArrayRef; - let to_write = RecordBatch::try_new(arrow_schema.clone(), vec![col0]).unwrap(); + pw.write(&to_write).await?; let res = pw.close().await?; assert_eq!(res.len(), 1); @@ -2259,194 +2162,43 @@ mod tests { .partition(Struct::empty()) .build() .unwrap(); + + // check data file + assert_eq!(data_file.record_count(), 4); assert_eq!( - data_file.upper_bounds().get(&0), - Some(Datum::decimal_with_precision(Decimal::new(-11000000000_i64, 10), 28).unwrap()) - .as_ref() + *data_file.value_counts(), + HashMap::from([(1, 4), (2, 4), (6, 4), (7, 4)]) ); assert_eq!( - data_file.lower_bounds().get(&0), - Some(Datum::decimal_with_precision(Decimal::new(-22000000000_i64, 10), 28).unwrap()) - .as_ref() - ); - - // test max and min of rust_decimal - let decimal_max = Decimal::MAX; - let decimal_min = Decimal::MIN; - assert_eq!(decimal_max.scale(), decimal_min.scale()); - let schema = Arc::new( - Schema::builder() - .with_fields(vec![NestedField::optional( - 0, - "decimal", - Type::Primitive(PrimitiveType::Decimal { - precision: 38, - scale: decimal_max.scale(), - }), - ) - .into()]) - .build() - .unwrap(), + *data_file.lower_bounds(), + HashMap::from([ + (1, Datum::int(1)), + (2, Datum::float(1.0)), + (6, Datum::int(1)), + (7, Datum::float(1.0)) + ]) ); - let arrow_schema: ArrowSchemaRef = Arc::new(schema_to_arrow_schema(&schema).unwrap()); - let mut pw = ParquetWriterBuilder::new( - WriterProperties::builder().build(), - schema, - file_io.clone(), - loccation_gen, - file_name_gen, - ) - .build() - .await?; - let col0 = Arc::new( - Decimal128Array::from(vec![ - Some(decimal_max.mantissa()), - Some(decimal_min.mantissa()), + assert_eq!( + *data_file.upper_bounds(), + HashMap::from([ + (1, Datum::int(4)), + (2, Datum::float(2.0)), + (6, Datum::int(4)), + (7, Datum::float(2.0)) ]) - .with_data_type(DataType::Decimal128(38, 0)), - ) as ArrayRef; - let to_write = RecordBatch::try_new(arrow_schema.clone(), vec![col0]).unwrap(); - pw.write(&to_write).await?; - let res = pw.close().await?; - assert_eq!(res.len(), 1); - let data_file = res - .into_iter() - .next() - .unwrap() - .content(crate::spec::DataContentType::Data) - .partition(Struct::empty()) - .build() - .unwrap(); + ); assert_eq!( - data_file.upper_bounds().get(&0), - Some(Datum::decimal(decimal_max).unwrap()).as_ref() + *data_file.null_value_counts(), + HashMap::from([(1, 0), (2, 0), (6, 0), (7, 0)]) ); assert_eq!( - data_file.lower_bounds().get(&0), - Some(Datum::decimal(decimal_min).unwrap()).as_ref() + *data_file.nan_value_counts(), + HashMap::from([(2, 1), (7, 1)]) ); - // test max and min for scale 38 - // # TODO - // Readd this case after resolve https://github.com/apache/iceberg-rust/issues/669 - // let schema = Arc::new( - // Schema::builder() - // .with_fields(vec![NestedField::optional( - // 0, - // "decimal", - // Type::Primitive(PrimitiveType::Decimal { - // precision: 38, - // scale: 0, - // }), - // ) - // .into()]) - // .build() - // .unwrap(), - // ); - // let arrow_schema: ArrowSchemaRef = Arc::new(schema_to_arrow_schema(&schema).unwrap()); - // let mut pw = ParquetWriterBuilder::new( - // WriterProperties::builder().build(), - // schema, - // file_io.clone(), - // loccation_gen, - // file_name_gen, - // ) - // .build() - // .await?; - // let col0 = Arc::new( - // Decimal128Array::from(vec![ - // Some(99999999999999999999999999999999999999_i128), - // Some(-99999999999999999999999999999999999999_i128), - // ]) - // .with_data_type(DataType::Decimal128(38, 0)), - // ) as ArrayRef; - // let to_write = RecordBatch::try_new(arrow_schema.clone(), vec![col0]).unwrap(); - // pw.write(&to_write).await?; - // let res = pw.close().await?; - // assert_eq!(res.len(), 1); - // let data_file = res - // .into_iter() - // .next() - // .unwrap() - // .content(crate::spec::DataContentType::Data) - // .partition(Struct::empty()) - // .build() - // .unwrap(); - // assert_eq!( - // data_file.upper_bounds().get(&0), - // Some(Datum::new( - // PrimitiveType::Decimal { - // precision: 38, - // scale: 0 - // }, - // PrimitiveLiteral::Int128(99999999999999999999999999999999999999_i128) - // )) - // .as_ref() - // ); - // assert_eq!( - // data_file.lower_bounds().get(&0), - // Some(Datum::new( - // PrimitiveType::Decimal { - // precision: 38, - // scale: 0 - // }, - // PrimitiveLiteral::Int128(-99999999999999999999999999999999999999_i128) - // )) - // .as_ref() - // ); - - Ok(()) - } - - #[tokio::test] - async fn test_empty_write() -> Result<()> { - let temp_dir = TempDir::new().unwrap(); - let file_io = FileIOBuilder::new_fs_io().build().unwrap(); - 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); - - // Test that file will create if data to write - 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())]), - ), - ]; - Arc::new(arrow_schema::Schema::new(fields)) - }; - let col = Arc::new(Int64Array::from_iter_values(0..1024)) as ArrayRef; - let to_write = RecordBatch::try_new(schema.clone(), vec![col]).unwrap(); - let mut pw = ParquetWriterBuilder::new( - WriterProperties::builder().build(), - Arc::new(to_write.schema().as_ref().try_into().unwrap()), - file_io.clone(), - location_gen.clone(), - file_name_gen, - ) - .build() - .await?; - pw.write(&to_write).await?; - let file_path = pw.out_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 pw = ParquetWriterBuilder::new( - WriterProperties::builder().build(), - Arc::new(to_write.schema().as_ref().try_into().unwrap()), - file_io.clone(), - location_gen, - file_name_gen, - ) - .build() - .await?; - let file_path = pw.out_file.location().to_string(); - pw.close().await.unwrap(); - assert!(!file_io.exists(file_path).await.unwrap()); + // check the written file + let expect_batch = concat_batches(&arrow_schema, vec![&to_write]).unwrap(); + check_parquet_data_file(&file_io, &data_file, &expect_batch).await; Ok(()) } From 78070b7d1e20867d5cf82ec1a4ea3dc2e498995a Mon Sep 17 00:00:00 2001 From: feniljain Date: Wed, 12 Mar 2025 17:10:07 +0530 Subject: [PATCH 09/14] fix: correct usage of visitor --- .../iceberg/src/arrow/nan_val_cnt_visitor.rs | 79 +++++++++++++------ .../src/writer/file_writer/parquet_writer.rs | 6 +- 2 files changed, 58 insertions(+), 27 deletions(-) diff --git a/crates/iceberg/src/arrow/nan_val_cnt_visitor.rs b/crates/iceberg/src/arrow/nan_val_cnt_visitor.rs index 1036e9b8f5..6ef9cff923 100644 --- a/crates/iceberg/src/arrow/nan_val_cnt_visitor.rs +++ b/crates/iceberg/src/arrow/nan_val_cnt_visitor.rs @@ -1,16 +1,18 @@ use std::collections::hash_map::Entry; use std::collections::HashMap; +use std::sync::Arc; -use arrow_array::{ArrayRef, Float32Array, Float64Array, RecordBatch}; +use arrow_array::{ArrayRef, Float32Array, Float64Array, RecordBatch, StructArray}; +use arrow_schema::DataType; use crate::arrow::ArrowArrayAccessor; use crate::spec::{ - visit_schema_with_partner, ListType, MapType, NestedFieldRef, SchemaRef, - PrimitiveType, Schema, SchemaWithPartnerVisitor, StructType, + visit_struct_with_partner, ListType, MapType, NestedFieldRef, PrimitiveType, Schema, SchemaRef, + SchemaWithPartnerVisitor, StructType, }; use crate::Result; -macro_rules! count_float_nans { +macro_rules! cast_and_update_cnt_map { ($t:ty, $col:ident, $self:ident, $field_id:ident) => { let nan_val_cnt = $col .as_any() @@ -29,10 +31,24 @@ macro_rules! count_float_nans { v.insert(nan_val_cnt); } }; + } +} + +macro_rules! count_float_nans { + ($col:ident, $self:ident, $field_id:ident) => { + match $col.data_type() { + DataType::Float32 => { + cast_and_update_cnt_map!(Float32Array, $col, $self, $field_id); + } + DataType::Float64 => { + cast_and_update_cnt_map!(Float64Array, $col, $self, $field_id); + } + _ => {} + } }; } -/// TODO(feniljain) +/// Visitor which counts and keeps track of NaN value counts in given record batch(s) pub struct NanValueCountVisitor { /// Stores field ID to NaN value count mapping pub nan_value_counts: HashMap, @@ -82,21 +98,31 @@ impl SchemaWithPartnerVisitor for NanValueCountVisitor { Ok(()) } - fn primitive(&mut self, p: &PrimitiveType, col: &ArrayRef) -> Result { - match p { - PrimitiveType::Float => { - // let field_id = p.id; - // TODO(feniljain): fix this - let field_id = 1; - count_float_nans!(Float32Array, col, self, field_id); - } - PrimitiveType::Double => { - let field_id = 1; - count_float_nans!(Float64Array, col, self, field_id); - } - _ => {} - } + fn primitive(&mut self, _p: &PrimitiveType, _col: &ArrayRef) -> Result { + Ok(()) + } + fn after_struct_field(&mut self, field: &NestedFieldRef, partner: &ArrayRef) -> Result<()> { + let field_id = field.id; + count_float_nans!(partner, self, field_id); + Ok(()) + } + + fn after_list_element(&mut self, field: &NestedFieldRef, partner: &ArrayRef) -> Result<()> { + let field_id = field.id; + count_float_nans!(partner, self, field_id); + Ok(()) + } + + fn after_map_key(&mut self, field: &NestedFieldRef, partner: &ArrayRef) -> Result<()> { + let field_id = field.id; + count_float_nans!(partner, self, field_id); + Ok(()) + } + + fn after_map_value(&mut self, field: &NestedFieldRef, partner: &ArrayRef) -> Result<()> { + let field_id = field.id; + count_float_nans!(partner, self, field_id); Ok(()) } } @@ -110,14 +136,17 @@ impl NanValueCountVisitor { } /// Compute nan value counts in given schema and record batch - pub fn compute(&mut self, schema: SchemaRef, batch: &RecordBatch) -> Result<()> { - let arrow_arr_partner_accessor = ArrowArrayAccessor{}; + pub fn compute(&mut self, schema: SchemaRef, batch: RecordBatch) -> Result<()> { + let arrow_arr_partner_accessor = ArrowArrayAccessor {}; - for arr_ref in batch.columns() { - visit_schema_with_partner(&schema, arr_ref, self, &arrow_arr_partner_accessor)?; - } + let struct_arr = Arc::new(StructArray::from(batch)) as ArrayRef; + visit_struct_with_partner( + &schema.as_struct(), + &struct_arr, + self, + &arrow_arr_partner_accessor, + )?; Ok(()) } } - diff --git a/crates/iceberg/src/writer/file_writer/parquet_writer.rs b/crates/iceberg/src/writer/file_writer/parquet_writer.rs index f18529015f..07b8b00067 100644 --- a/crates/iceberg/src/writer/file_writer/parquet_writer.rs +++ b/crates/iceberg/src/writer/file_writer/parquet_writer.rs @@ -525,7 +525,9 @@ impl FileWriter for ParquetWriter { self.current_row_num += batch.num_rows(); - self.nan_value_count_visitor.compute(self.schema.clone(), batch)?; + // TODO(feniljain): Confirm if this `clone` is okay to perform + let batch_c = batch.clone(); + self.nan_value_count_visitor.compute(self.schema.clone(), batch_c)?; // Lazy initialize the writer let writer = if let Some(writer) = &mut self.inner_writer { @@ -1633,7 +1635,7 @@ mod tests { MockLocationGenerator::new(temp_dir.path().to_str().unwrap().to_string()); let file_name_gen = DefaultFileNameGenerator::new("test".to_string(), None, DataFileFormat::Parquet); - // + // prepare data let arrow_schema = { let fields = vec![ From 2726e8d0fbb5a8abf8f21ce201c529c1564c50ff Mon Sep 17 00:00:00 2001 From: feniljain Date: Thu, 13 Mar 2025 17:50:17 +0000 Subject: [PATCH 10/14] chore: remove `TODO`s --- crates/iceberg/src/arrow/value.rs | 2 +- crates/iceberg/src/writer/file_writer/parquet_writer.rs | 3 --- 2 files changed, 1 insertion(+), 4 deletions(-) diff --git a/crates/iceberg/src/arrow/value.rs b/crates/iceberg/src/arrow/value.rs index 0b8f0d1e13..920e069bd6 100644 --- a/crates/iceberg/src/arrow/value.rs +++ b/crates/iceberg/src/arrow/value.rs @@ -425,7 +425,7 @@ impl SchemaWithPartnerVisitor for ArrowArrayToIcebergStructConverter { } } -/// TODO(feniljain) +/// Partner type representing accessing and walking arrow arrays alongside iceberg schema pub struct ArrowArrayAccessor; impl PartnerAccessor for ArrowArrayAccessor { diff --git a/crates/iceberg/src/writer/file_writer/parquet_writer.rs b/crates/iceberg/src/writer/file_writer/parquet_writer.rs index 07b8b00067..0096282a42 100644 --- a/crates/iceberg/src/writer/file_writer/parquet_writer.rs +++ b/crates/iceberg/src/writer/file_writer/parquet_writer.rs @@ -317,8 +317,6 @@ impl MinMaxColAggregator { } } -// TODO(feniljain): Think do we need to add nan_value_counts to `parquet_files_to_data_files` and -// `parquet_to_data_file_builder` too? impl ParquetWriter { /// Converts parquet files to data files #[allow(dead_code)] @@ -525,7 +523,6 @@ impl FileWriter for ParquetWriter { self.current_row_num += batch.num_rows(); - // TODO(feniljain): Confirm if this `clone` is okay to perform let batch_c = batch.clone(); self.nan_value_count_visitor.compute(self.schema.clone(), batch_c)?; From 086f40362b909141022b6ed7b1164887c82bdd4d Mon Sep 17 00:00:00 2001 From: feniljain Date: Thu, 13 Mar 2025 17:57:22 +0000 Subject: [PATCH 11/14] chore: cargo fmt and clippy --- crates/iceberg/src/arrow/nan_val_cnt_visitor.rs | 10 ++++++++-- .../iceberg/src/writer/file_writer/parquet_writer.rs | 9 +++++---- 2 files changed, 13 insertions(+), 6 deletions(-) diff --git a/crates/iceberg/src/arrow/nan_val_cnt_visitor.rs b/crates/iceberg/src/arrow/nan_val_cnt_visitor.rs index 6ef9cff923..db46f5e545 100644 --- a/crates/iceberg/src/arrow/nan_val_cnt_visitor.rs +++ b/crates/iceberg/src/arrow/nan_val_cnt_visitor.rs @@ -31,7 +31,7 @@ macro_rules! cast_and_update_cnt_map { v.insert(nan_val_cnt); } }; - } + }; } macro_rules! count_float_nans { @@ -141,7 +141,7 @@ impl NanValueCountVisitor { let struct_arr = Arc::new(StructArray::from(batch)) as ArrayRef; visit_struct_with_partner( - &schema.as_struct(), + schema.as_struct(), &struct_arr, self, &arrow_arr_partner_accessor, @@ -150,3 +150,9 @@ impl NanValueCountVisitor { Ok(()) } } + +impl Default for NanValueCountVisitor { + fn default() -> Self { + Self::new() + } +} diff --git a/crates/iceberg/src/writer/file_writer/parquet_writer.rs b/crates/iceberg/src/writer/file_writer/parquet_writer.rs index 0096282a42..f80ff88e31 100644 --- a/crates/iceberg/src/writer/file_writer/parquet_writer.rs +++ b/crates/iceberg/src/writer/file_writer/parquet_writer.rs @@ -38,7 +38,7 @@ use super::track_writer::TrackWriter; use super::{FileWriter, FileWriterBuilder}; use crate::arrow::{ get_parquet_stat_max_as_datum, get_parquet_stat_min_as_datum, ArrowFileReader, - DEFAULT_MAP_FIELD_NAME, NanValueCountVisitor, + NanValueCountVisitor, DEFAULT_MAP_FIELD_NAME, }; use crate::io::{FileIO, FileWrite, OutputFile}; use crate::spec::{ @@ -524,7 +524,8 @@ impl FileWriter for ParquetWriter { self.current_row_num += batch.num_rows(); let batch_c = batch.clone(); - self.nan_value_count_visitor.compute(self.schema.clone(), batch_c)?; + self.nan_value_count_visitor + .compute(self.schema.clone(), batch_c)?; // Lazy initialize the writer let writer = if let Some(writer) = &mut self.inner_writer { @@ -635,8 +636,8 @@ mod tests { use arrow_array::builder::{Float32Builder, Int32Builder, MapBuilder}; use arrow_array::types::{Float32Type, Int64Type}; use arrow_array::{ - Array, ArrayRef, BooleanArray, Decimal128Array, Float32Array, Int32Array, Int64Array, - ListArray, RecordBatch, StructArray, MapArray, Float64Array, + Array, ArrayRef, BooleanArray, Decimal128Array, Float32Array, Float64Array, Int32Array, + Int64Array, ListArray, MapArray, RecordBatch, StructArray, }; use arrow_schema::{DataType, Field, Fields, SchemaRef as ArrowSchemaRef}; use arrow_select::concat::concat_batches; From e24406b657ba178177ac37a856f31323b00f3d45 Mon Sep 17 00:00:00 2001 From: feniljain Date: Thu, 13 Mar 2025 18:12:09 +0000 Subject: [PATCH 12/14] fix: add license header --- .../iceberg/src/arrow/nan_val_cnt_visitor.rs | 19 +++++++++++++++++++ 1 file changed, 19 insertions(+) diff --git a/crates/iceberg/src/arrow/nan_val_cnt_visitor.rs b/crates/iceberg/src/arrow/nan_val_cnt_visitor.rs index db46f5e545..db6279d9ca 100644 --- a/crates/iceberg/src/arrow/nan_val_cnt_visitor.rs +++ b/crates/iceberg/src/arrow/nan_val_cnt_visitor.rs @@ -1,3 +1,22 @@ +// 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. + +//! The module contains the visitor for calculating NaN values in give arrow record batch. + use std::collections::hash_map::Entry; use std::collections::HashMap; use std::sync::Arc; From 3b1d33d71034b9621ae1c5380e43883837e732f8 Mon Sep 17 00:00:00 2001 From: feniljain Date: Thu, 13 Mar 2025 18:54:16 +0000 Subject: [PATCH 13/14] fix: arrow schema not constructed with metadata in unrelated test --- Cargo.lock | 1 + crates/iceberg/src/arrow/value.rs | 3 +++ .../writer/base_writer/data_file_writer.rs | 22 +++++++++++++++---- crates/integrations/datafusion/Cargo.toml | 1 + .../src/physical_plan/expr_to_predicate.rs | 17 +++++++++++--- 5 files changed, 37 insertions(+), 7 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index cf4a23df9a..4c85f916d4 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -3109,6 +3109,7 @@ dependencies = [ "futures", "iceberg", "iceberg-catalog-memory", + "parquet", "tempfile", "tokio", ] diff --git a/crates/iceberg/src/arrow/value.rs b/crates/iceberg/src/arrow/value.rs index 920e069bd6..84b33d3ff6 100644 --- a/crates/iceberg/src/arrow/value.rs +++ b/crates/iceberg/src/arrow/value.rs @@ -436,6 +436,7 @@ impl PartnerAccessor for ArrowArrayAccessor { "The schema partner is not a struct type", )); } + Ok(schema_partner) } @@ -453,6 +454,7 @@ impl PartnerAccessor for ArrowArrayAccessor { "The struct partner is not a struct array", ) })?; + let field_pos = struct_array .fields() .iter() @@ -467,6 +469,7 @@ impl PartnerAccessor for ArrowArrayAccessor { format!("Field id {} not found in struct array", field.id), ) })?; + Ok(struct_array.column(field_pos)) } 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 e193b4a94c..87005dd290 100644 --- a/crates/iceberg/src/writer/base_writer/data_file_writer.rs +++ b/crates/iceberg/src/writer/base_writer/data_file_writer.rs @@ -98,11 +98,13 @@ impl CurrentFileStatus for DataFileWriter { #[cfg(test)] mod test { + use std::collections::HashMap; use std::sync::Arc; use arrow_array::{Int32Array, StringArray}; use arrow_schema::{DataType, Field}; use parquet::arrow::arrow_reader::{ArrowReaderMetadata, ArrowReaderOptions}; + use parquet::arrow::PARQUET_FIELD_ID_META_KEY; use parquet::file::properties::WriterProperties; use tempfile::TempDir; @@ -145,8 +147,14 @@ mod test { let mut data_file_writer = DataFileWriterBuilder::new(pw, None).build().await.unwrap(); let arrow_schema = arrow_schema::Schema::new(vec![ - Field::new("foo", DataType::Int32, false), - Field::new("bar", DataType::Utf8, false), + Field::new("foo", DataType::Int32, false).with_metadata(HashMap::from([( + PARQUET_FIELD_ID_META_KEY.to_string(), + 3.to_string(), + )])), + Field::new("bar", DataType::Utf8, false).with_metadata(HashMap::from([( + PARQUET_FIELD_ID_META_KEY.to_string(), + 4.to_string(), + )])), ]); let batch = RecordBatch::try_new(Arc::new(arrow_schema.clone()), vec![ Arc::new(Int32Array::from(vec![1, 2, 3])), @@ -216,8 +224,14 @@ mod test { .await?; let arrow_schema = arrow_schema::Schema::new(vec![ - Field::new("id", DataType::Int32, false), - Field::new("name", DataType::Utf8, false), + Field::new("id", DataType::Int32, false).with_metadata(HashMap::from([( + PARQUET_FIELD_ID_META_KEY.to_string(), + 5.to_string(), + )])), + Field::new("name", DataType::Utf8, false).with_metadata(HashMap::from([( + PARQUET_FIELD_ID_META_KEY.to_string(), + 6.to_string(), + )])), ]); let batch = RecordBatch::try_new(Arc::new(arrow_schema.clone()), vec![ Arc::new(Int32Array::from(vec![1, 2, 3])), diff --git a/crates/integrations/datafusion/Cargo.toml b/crates/integrations/datafusion/Cargo.toml index ccb9ca175b..c6c564574c 100644 --- a/crates/integrations/datafusion/Cargo.toml +++ b/crates/integrations/datafusion/Cargo.toml @@ -41,4 +41,5 @@ tokio = { workspace = true } [dev-dependencies] iceberg-catalog-memory = { workspace = true } +parquet = { workspace = true } tempfile = { workspace = true } diff --git a/crates/integrations/datafusion/src/physical_plan/expr_to_predicate.rs b/crates/integrations/datafusion/src/physical_plan/expr_to_predicate.rs index a45e21a586..10b92d54b1 100644 --- a/crates/integrations/datafusion/src/physical_plan/expr_to_predicate.rs +++ b/crates/integrations/datafusion/src/physical_plan/expr_to_predicate.rs @@ -220,20 +220,31 @@ fn scalar_value_to_datum(value: &ScalarValue) -> Option { #[cfg(test)] mod tests { + use std::collections::HashMap; + use datafusion::arrow::datatypes::{DataType, Field, Schema, TimeUnit}; use datafusion::common::DFSchema; use datafusion::logical_expr::utils::split_conjunction; use datafusion::prelude::{Expr, SessionContext}; use iceberg::expr::{Predicate, Reference}; use iceberg::spec::Datum; + use parquet::arrow::PARQUET_FIELD_ID_META_KEY; use super::convert_filters_to_predicate; fn create_test_schema() -> DFSchema { let arrow_schema = Schema::new(vec![ - Field::new("foo", DataType::Int32, true), - Field::new("bar", DataType::Utf8, true), - Field::new("ts", DataType::Timestamp(TimeUnit::Second, None), true), + Field::new("foo", DataType::Int32, true).with_metadata(HashMap::from([( + PARQUET_FIELD_ID_META_KEY.to_string(), + "1".to_string(), + )])), + Field::new("bar", DataType::Utf8, true).with_metadata(HashMap::from([( + PARQUET_FIELD_ID_META_KEY.to_string(), + "2".to_string(), + )])), + Field::new("ts", DataType::Timestamp(TimeUnit::Second, None), true).with_metadata( + HashMap::from([(PARQUET_FIELD_ID_META_KEY.to_string(), "3".to_string())]), + ), ]); DFSchema::try_from_qualified_schema("my_table", &arrow_schema).unwrap() } From 8c3280fbfce5e09c41adaa4273e2be76c1fb5e2f Mon Sep 17 00:00:00 2001 From: Renjie Liu Date: Tue, 25 Mar 2025 14:18:25 +0800 Subject: [PATCH 14/14] Update mod.rs --- crates/iceberg/src/arrow/mod.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/crates/iceberg/src/arrow/mod.rs b/crates/iceberg/src/arrow/mod.rs index dfa569544b..56caeaf559 100644 --- a/crates/iceberg/src/arrow/mod.rs +++ b/crates/iceberg/src/arrow/mod.rs @@ -21,7 +21,7 @@ mod schema; pub use schema::*; mod nan_val_cnt_visitor; -pub use nan_val_cnt_visitor::*; +pub(crate) use nan_val_cnt_visitor::*; pub(crate) mod delete_file_manager;