From 4171275a23aa3f3c9e8f8c24d433e4a0eeadc392 Mon Sep 17 00:00:00 2001 From: Matheus Alcantara Date: Wed, 25 Sep 2024 08:28:42 -0300 Subject: [PATCH 01/46] scan: change ErrorKind when table dont have spanshots (#608) --- crates/iceberg/src/scan.rs | 5 +---- 1 file changed, 1 insertion(+), 4 deletions(-) diff --git a/crates/iceberg/src/scan.rs b/crates/iceberg/src/scan.rs index 0d8a4bf0d..1c3361163 100644 --- a/crates/iceberg/src/scan.rs +++ b/crates/iceberg/src/scan.rs @@ -197,10 +197,7 @@ impl<'a> TableScanBuilder<'a> { .metadata() .current_snapshot() .ok_or_else(|| { - Error::new( - ErrorKind::FeatureUnsupported, - "Can't scan table without snapshots", - ) + Error::new(ErrorKind::Unexpected, "Can't scan table without snapshots") })? .clone(), }; From 984c91e2f82cdc2fc43434e400e3d4ae9c3bf000 Mon Sep 17 00:00:00 2001 From: ZENOTME <43447882+ZENOTME@users.noreply.github.com> Date: Thu, 26 Sep 2024 17:56:02 +0800 Subject: [PATCH 02/46] avoid to create memory schema operator every time (#635) Co-authored-by: ZENOTME --- crates/iceberg/src/io/file_io.rs | 19 +++++++++++++++++++ crates/iceberg/src/io/storage.rs | 12 +++++------- 2 files changed, 24 insertions(+), 7 deletions(-) diff --git a/crates/iceberg/src/io/file_io.rs b/crates/iceberg/src/io/file_io.rs index 5e30968f5..e0d83a08b 100644 --- a/crates/iceberg/src/io/file_io.rs +++ b/crates/iceberg/src/io/file_io.rs @@ -368,6 +368,7 @@ mod tests { use std::io::Write; use std::path::Path; + use bytes::Bytes; use futures::io::AllowStdIo; use futures::AsyncReadExt; use tempfile::TempDir; @@ -490,4 +491,22 @@ mod tests { let io = FileIO::from_path("tmp/||c"); assert!(io.is_err()); } + + #[tokio::test] + async fn test_memory_io() { + let io = FileIOBuilder::new("memory").build().unwrap(); + + let path = format!("{}/1.txt", TempDir::new().unwrap().path().to_str().unwrap()); + + let output_file = io.new_output(&path).unwrap(); + output_file.write("test".into()).await.unwrap(); + + assert!(io.is_exist(&path.clone()).await.unwrap()); + let input_file = io.new_input(&path).unwrap(); + let content = input_file.read().await.unwrap(); + assert_eq!(content, Bytes::from("test")); + + io.delete(&path).await.unwrap(); + assert!(!io.is_exist(&path).await.unwrap()); + } } diff --git a/crates/iceberg/src/io/storage.rs b/crates/iceberg/src/io/storage.rs index 890104448..cd9b54d54 100644 --- a/crates/iceberg/src/io/storage.rs +++ b/crates/iceberg/src/io/storage.rs @@ -30,7 +30,7 @@ use crate::{Error, ErrorKind}; #[derive(Debug)] pub(crate) enum Storage { #[cfg(feature = "storage-memory")] - Memory, + Memory(Operator), #[cfg(feature = "storage-fs")] LocalFs, #[cfg(feature = "storage-s3")] @@ -56,7 +56,7 @@ impl Storage { match scheme { #[cfg(feature = "storage-memory")] - Scheme::Memory => Ok(Self::Memory), + Scheme::Memory => Ok(Self::Memory(super::memory_config_build()?)), #[cfg(feature = "storage-fs")] Scheme::Fs => Ok(Self::LocalFs), #[cfg(feature = "storage-s3")] @@ -96,13 +96,11 @@ impl Storage { let path = path.as_ref(); match self { #[cfg(feature = "storage-memory")] - Storage::Memory => { - let op = super::memory_config_build()?; - + Storage::Memory(op) => { if let Some(stripped) = path.strip_prefix("memory:/") { - Ok((op, stripped)) + Ok((op.clone(), stripped)) } else { - Ok((op, &path[1..])) + Ok((op.clone(), &path[1..])) } } #[cfg(feature = "storage-fs")] From 87483b40a284b2775d61b7712e4bee92d24a2a96 Mon Sep 17 00:00:00 2001 From: Alon Agmon <54080741+a-agmon@users.noreply.github.com> Date: Fri, 27 Sep 2024 04:40:08 +0300 Subject: [PATCH 03/46] making table provider pub (#650) Co-authored-by: Alon Agmon --- crates/integrations/datafusion/src/lib.rs | 1 + crates/integrations/datafusion/src/table.rs | 9 ++++++++- 2 files changed, 9 insertions(+), 1 deletion(-) diff --git a/crates/integrations/datafusion/src/lib.rs b/crates/integrations/datafusion/src/lib.rs index c40290116..b64f8fb8e 100644 --- a/crates/integrations/datafusion/src/lib.rs +++ b/crates/integrations/datafusion/src/lib.rs @@ -24,3 +24,4 @@ pub use error::*; mod physical_plan; mod schema; mod table; +pub use table::*; diff --git a/crates/integrations/datafusion/src/table.rs b/crates/integrations/datafusion/src/table.rs index 016c6c00f..f12d41eec 100644 --- a/crates/integrations/datafusion/src/table.rs +++ b/crates/integrations/datafusion/src/table.rs @@ -33,7 +33,7 @@ use crate::physical_plan::scan::IcebergTableScan; /// Represents a [`TableProvider`] for the Iceberg [`Catalog`], /// managing access to a [`Table`]. -pub(crate) struct IcebergTableProvider { +pub struct IcebergTableProvider { /// A table in the catalog. table: Table, /// A reference-counted arrow `Schema`. @@ -56,6 +56,13 @@ impl IcebergTableProvider { Ok(IcebergTableProvider { table, schema }) } + + /// Asynchronously tries to construct a new [`IcebergTableProvider`] + /// using the given table. Can be used to create a table provider from an existing table regardless of the catalog implementation. + pub async fn try_new_from_table(table: Table) -> Result { + let schema = Arc::new(schema_to_arrow_schema(table.metadata().current_schema())?); + Ok(IcebergTableProvider { table, schema }) + } } #[async_trait] From a6a3fd779dfe895e02885803f4f82f6b39407732 Mon Sep 17 00:00:00 2001 From: Alon Agmon <54080741+a-agmon@users.noreply.github.com> Date: Sat, 28 Sep 2024 10:10:08 +0300 Subject: [PATCH 04/46] test (datafusion): add test for table provider creation (#651) * add test for table provider creation * fix formatting * fixing yet another formatting issue * testing schema using data fusion --------- Co-authored-by: Alon Agmon --- crates/integrations/datafusion/src/table.rs | 54 ++++++++ .../tests/test_data/TableMetadataV2Valid.json | 122 ++++++++++++++++++ 2 files changed, 176 insertions(+) create mode 100644 crates/integrations/datafusion/tests/test_data/TableMetadataV2Valid.json diff --git a/crates/integrations/datafusion/src/table.rs b/crates/integrations/datafusion/src/table.rs index f12d41eec..2797e12d6 100644 --- a/crates/integrations/datafusion/src/table.rs +++ b/crates/integrations/datafusion/src/table.rs @@ -110,3 +110,57 @@ impl TableProvider for IcebergTableProvider { Ok(filter_support) } } + +#[cfg(test)] +mod tests { + use datafusion::common::Column; + use datafusion::prelude::SessionContext; + use iceberg::io::FileIO; + use iceberg::table::{StaticTable, Table}; + use iceberg::TableIdent; + + use super::*; + + async fn get_test_table_from_metadata_file() -> Table { + let metadata_file_name = "TableMetadataV2Valid.json"; + let metadata_file_path = format!( + "{}/tests/test_data/{}", + env!("CARGO_MANIFEST_DIR"), + metadata_file_name + ); + let file_io = FileIO::from_path(&metadata_file_path) + .unwrap() + .build() + .unwrap(); + let static_identifier = TableIdent::from_strs(["static_ns", "static_table"]).unwrap(); + let static_table = + StaticTable::from_metadata_file(&metadata_file_path, static_identifier, file_io) + .await + .unwrap(); + static_table.into_table() + } + + #[tokio::test] + async fn test_try_new_from_table() { + let table = get_test_table_from_metadata_file().await; + let table_provider = IcebergTableProvider::try_new_from_table(table.clone()) + .await + .unwrap(); + let ctx = SessionContext::new(); + ctx.register_table("mytable", Arc::new(table_provider)) + .unwrap(); + let df = ctx.sql("SELECT * FROM mytable").await.unwrap(); + let df_schema = df.schema(); + let df_columns = df_schema.fields(); + assert_eq!(df_columns.len(), 3); + let x_column = df_columns.first().unwrap(); + let column_data = format!( + "{:?}:{:?}", + x_column.name(), + x_column.data_type().to_string() + ); + assert_eq!(column_data, "\"x\":\"Int64\""); + let has_column = df_schema.has_column(&Column::from_name("z")); + assert!(has_column); + } +} diff --git a/crates/integrations/datafusion/tests/test_data/TableMetadataV2Valid.json b/crates/integrations/datafusion/tests/test_data/TableMetadataV2Valid.json new file mode 100644 index 000000000..0dc89de58 --- /dev/null +++ b/crates/integrations/datafusion/tests/test_data/TableMetadataV2Valid.json @@ -0,0 +1,122 @@ +{ + "format-version": 2, + "table-uuid": "9c12d441-03fe-4693-9a96-a0705ddf69c1", + "location": "s3://bucket/test/location", + "last-sequence-number": 34, + "last-updated-ms": 1602638573590, + "last-column-id": 3, + "current-schema-id": 1, + "schemas": [ + { + "type": "struct", + "schema-id": 0, + "fields": [ + { + "id": 1, + "name": "x", + "required": true, + "type": "long" + } + ] + }, + { + "type": "struct", + "schema-id": 1, + "identifier-field-ids": [ + 1, + 2 + ], + "fields": [ + { + "id": 1, + "name": "x", + "required": true, + "type": "long" + }, + { + "id": 2, + "name": "y", + "required": true, + "type": "long", + "doc": "comment" + }, + { + "id": 3, + "name": "z", + "required": true, + "type": "long" + } + ] + } + ], + "default-spec-id": 0, + "partition-specs": [ + { + "spec-id": 0, + "fields": [ + { + "name": "x", + "transform": "identity", + "source-id": 1, + "field-id": 1000 + } + ] + } + ], + "last-partition-id": 1000, + "default-sort-order-id": 3, + "sort-orders": [ + { + "order-id": 3, + "fields": [ + { + "transform": "identity", + "source-id": 2, + "direction": "asc", + "null-order": "nulls-first" + }, + { + "transform": "bucket[4]", + "source-id": 3, + "direction": "desc", + "null-order": "nulls-last" + } + ] + } + ], + "properties": {}, + "current-snapshot-id": 3055729675574597004, + "snapshots": [ + { + "snapshot-id": 3051729675574597004, + "timestamp-ms": 1515100955770, + "sequence-number": 0, + "summary": { + "operation": "append" + }, + "manifest-list": "s3://a/b/1.avro" + }, + { + "snapshot-id": 3055729675574597004, + "parent-snapshot-id": 3051729675574597004, + "timestamp-ms": 1555100955770, + "sequence-number": 1, + "summary": { + "operation": "append" + }, + "manifest-list": "s3://a/b/2.avro", + "schema-id": 1 + } + ], + "snapshot-log": [ + { + "snapshot-id": 3051729675574597004, + "timestamp-ms": 1515100955770 + }, + { + "snapshot-id": 3055729675574597004, + "timestamp-ms": 1555100955770 + } + ], + "metadata-log": [] +} \ No newline at end of file From af9609d12010295299868a6b7137441b9b5ad40f Mon Sep 17 00:00:00 2001 From: Scott Donnelly Date: Mon, 30 Sep 2024 04:06:14 +0100 Subject: [PATCH 05/46] fix: page index evaluator min/max args inverted (#648) * fix: page index evaluator min/max args inverted * style: fix clippy lint in test --- .../src/expr/visitors/page_index_evaluator.rs | 16 +- crates/iceberg/src/scan.rs | 226 ++++++++++++++++-- .../testdata/example_table_metadata_v2.json | 6 +- 3 files changed, 221 insertions(+), 27 deletions(-) diff --git a/crates/iceberg/src/expr/visitors/page_index_evaluator.rs b/crates/iceberg/src/expr/visitors/page_index_evaluator.rs index af20be0a4..18aeb5587 100644 --- a/crates/iceberg/src/expr/visitors/page_index_evaluator.rs +++ b/crates/iceberg/src/expr/visitors/page_index_evaluator.rs @@ -246,10 +246,10 @@ impl<'a> PageIndexEvaluator<'a> { .zip(row_counts.iter()) .map(|(item, &row_count)| { predicate( - item.max.map(|val| { + item.min.map(|val| { Datum::new(field_type.clone(), PrimitiveLiteral::Boolean(val)) }), - item.min.map(|val| { + item.max.map(|val| { Datum::new(field_type.clone(), PrimitiveLiteral::Boolean(val)) }), PageNullCount::from_row_and_null_counts(row_count, item.null_count), @@ -262,10 +262,10 @@ impl<'a> PageIndexEvaluator<'a> { .zip(row_counts.iter()) .map(|(item, &row_count)| { predicate( - item.max - .map(|val| Datum::new(field_type.clone(), PrimitiveLiteral::Int(val))), item.min .map(|val| Datum::new(field_type.clone(), PrimitiveLiteral::Int(val))), + item.max + .map(|val| Datum::new(field_type.clone(), PrimitiveLiteral::Int(val))), PageNullCount::from_row_and_null_counts(row_count, item.null_count), ) }) @@ -276,10 +276,10 @@ impl<'a> PageIndexEvaluator<'a> { .zip(row_counts.iter()) .map(|(item, &row_count)| { predicate( - item.max - .map(|val| Datum::new(field_type.clone(), PrimitiveLiteral::Long(val))), item.min .map(|val| Datum::new(field_type.clone(), PrimitiveLiteral::Long(val))), + item.max + .map(|val| Datum::new(field_type.clone(), PrimitiveLiteral::Long(val))), PageNullCount::from_row_and_null_counts(row_count, item.null_count), ) }) @@ -312,13 +312,13 @@ impl<'a> PageIndexEvaluator<'a> { .zip(row_counts.iter()) .map(|(item, &row_count)| { predicate( - item.max.map(|val| { + item.min.map(|val| { Datum::new( field_type.clone(), PrimitiveLiteral::Double(OrderedFloat::from(val)), ) }), - item.min.map(|val| { + item.max.map(|val| { Datum::new( field_type.clone(), PrimitiveLiteral::Double(OrderedFloat::from(val)), diff --git a/crates/iceberg/src/scan.rs b/crates/iceberg/src/scan.rs index 1c3361163..f5cbbcf06 100644 --- a/crates/iceberg/src/scan.rs +++ b/crates/iceberg/src/scan.rs @@ -913,7 +913,9 @@ mod tests { use std::fs::File; use std::sync::Arc; - use arrow_array::{ArrayRef, Int64Array, RecordBatch, StringArray}; + use arrow_array::{ + ArrayRef, BooleanArray, Float64Array, Int32Array, Int64Array, RecordBatch, StringArray, + }; use futures::{stream, TryStreamExt}; use parquet::arrow::{ArrowWriter, PARQUET_FIELD_ID_META_KEY}; use parquet::basic::Compression; @@ -1110,10 +1112,29 @@ mod tests { PARQUET_FIELD_ID_META_KEY.to_string(), "4".to_string(), )])), + arrow_schema::Field::new("dbl", arrow_schema::DataType::Float64, false) + .with_metadata(HashMap::from([( + PARQUET_FIELD_ID_META_KEY.to_string(), + "5".to_string(), + )])), + arrow_schema::Field::new("i32", arrow_schema::DataType::Int32, false) + .with_metadata(HashMap::from([( + PARQUET_FIELD_ID_META_KEY.to_string(), + "6".to_string(), + )])), + arrow_schema::Field::new("i64", arrow_schema::DataType::Int64, false) + .with_metadata(HashMap::from([( + PARQUET_FIELD_ID_META_KEY.to_string(), + "7".to_string(), + )])), + arrow_schema::Field::new("bool", arrow_schema::DataType::Boolean, false) + .with_metadata(HashMap::from([( + PARQUET_FIELD_ID_META_KEY.to_string(), + "8".to_string(), + )])), ]; Arc::new(arrow_schema::Schema::new(fields)) }; - // 4 columns: // x: [1, 1, 1, 1, ...] let col1 = Arc::new(Int64Array::from_iter_values(vec![1; 1024])) as ArrayRef; @@ -1136,8 +1157,34 @@ mod tests { values.append(vec!["Iceberg"; 512].as_mut()); let col4 = Arc::new(StringArray::from_iter_values(values)) as ArrayRef; - let to_write = - RecordBatch::try_new(schema.clone(), vec![col1, col2, col3, col4]).unwrap(); + // dbl: + let mut values = vec![100.0f64; 512]; + values.append(vec![150.0f64; 12].as_mut()); + values.append(vec![200.0f64; 500].as_mut()); + let col5 = Arc::new(Float64Array::from_iter_values(values)) as ArrayRef; + + // i32: + let mut values = vec![100i32; 512]; + values.append(vec![150i32; 12].as_mut()); + values.append(vec![200i32; 500].as_mut()); + let col6 = Arc::new(Int32Array::from_iter_values(values)) as ArrayRef; + + // i64: + let mut values = vec![100i64; 512]; + values.append(vec![150i64; 12].as_mut()); + values.append(vec![200i64; 500].as_mut()); + let col7 = Arc::new(Int64Array::from_iter_values(values)) as ArrayRef; + + // bool: + let mut values = vec![false; 512]; + values.append(vec![true; 512].as_mut()); + let values: BooleanArray = values.into(); + let col8 = Arc::new(values) as ArrayRef; + + let to_write = RecordBatch::try_new(schema.clone(), vec![ + col1, col2, col3, col4, col5, col6, col7, col8, + ]) + .unwrap(); // Write the Parquet files let props = WriterProperties::builder() @@ -1215,6 +1262,7 @@ mod tests { let table_scan = table .scan() .snapshot_id(3051729675574597004) + .with_row_selection_enabled(true) .build() .unwrap(); assert_eq!(table_scan.snapshot().snapshot_id(), 3051729675574597004); @@ -1226,7 +1274,13 @@ mod tests { fixture.setup_manifest_files().await; // Create table scan for current snapshot and plan files - let table_scan = fixture.table.scan().build().unwrap(); + let table_scan = fixture + .table + .scan() + .with_row_selection_enabled(true) + .build() + .unwrap(); + let mut tasks = table_scan .plan_files() .await @@ -1261,7 +1315,12 @@ mod tests { fixture.setup_manifest_files().await; // Create table scan for current snapshot and plan files - let table_scan = fixture.table.scan().build().unwrap(); + let table_scan = fixture + .table + .scan() + .with_row_selection_enabled(true) + .build() + .unwrap(); let batch_stream = table_scan.to_arrow().await.unwrap(); @@ -1279,7 +1338,12 @@ mod tests { fixture.setup_manifest_files().await; // Create table scan for current snapshot and plan files - let table_scan = fixture.table.scan().build().unwrap(); + let table_scan = fixture + .table + .scan() + .with_row_selection_enabled(true) + .build() + .unwrap(); let mut plan_task: Vec<_> = table_scan .plan_files() @@ -1312,7 +1376,13 @@ mod tests { fixture.setup_manifest_files().await; // Create table scan for current snapshot and plan files - let table_scan = fixture.table.scan().select(["x", "z"]).build().unwrap(); + let table_scan = fixture + .table + .scan() + .select(["x", "z"]) + .with_row_selection_enabled(true) + .build() + .unwrap(); let batch_stream = table_scan.to_arrow().await.unwrap(); @@ -1337,7 +1407,9 @@ mod tests { // Filter: y < 3 let mut builder = fixture.table.scan(); let predicate = Reference::new("y").less_than(Datum::long(3)); - builder = builder.with_filter(predicate); + builder = builder + .with_filter(predicate) + .with_row_selection_enabled(true); let table_scan = builder.build().unwrap(); let batch_stream = table_scan.to_arrow().await.unwrap(); @@ -1363,7 +1435,9 @@ mod tests { // Filter: y >= 5 let mut builder = fixture.table.scan(); let predicate = Reference::new("y").greater_than_or_equal_to(Datum::long(5)); - builder = builder.with_filter(predicate); + builder = builder + .with_filter(predicate) + .with_row_selection_enabled(true); let table_scan = builder.build().unwrap(); let batch_stream = table_scan.to_arrow().await.unwrap(); @@ -1381,6 +1455,106 @@ mod tests { assert_eq!(int64_arr.value(0), 5); } + #[tokio::test] + async fn test_filter_double_eq() { + let mut fixture = TableTestFixture::new(); + fixture.setup_manifest_files().await; + + // Filter: dbl == 150.0 + let mut builder = fixture.table.scan(); + let predicate = Reference::new("dbl").equal_to(Datum::double(150.0f64)); + builder = builder + .with_filter(predicate) + .with_row_selection_enabled(true); + let table_scan = builder.build().unwrap(); + + let batch_stream = table_scan.to_arrow().await.unwrap(); + + let batches: Vec<_> = batch_stream.try_collect().await.unwrap(); + + assert_eq!(batches.len(), 2); + assert_eq!(batches[0].num_rows(), 12); + + let col = batches[0].column_by_name("dbl").unwrap(); + let f64_arr = col.as_any().downcast_ref::().unwrap(); + assert_eq!(f64_arr.value(1), 150.0f64); + } + + #[tokio::test] + async fn test_filter_int_eq() { + let mut fixture = TableTestFixture::new(); + fixture.setup_manifest_files().await; + + // Filter: i32 == 150 + let mut builder = fixture.table.scan(); + let predicate = Reference::new("i32").equal_to(Datum::int(150i32)); + builder = builder + .with_filter(predicate) + .with_row_selection_enabled(true); + let table_scan = builder.build().unwrap(); + + let batch_stream = table_scan.to_arrow().await.unwrap(); + + let batches: Vec<_> = batch_stream.try_collect().await.unwrap(); + + assert_eq!(batches.len(), 2); + assert_eq!(batches[0].num_rows(), 12); + + let col = batches[0].column_by_name("i32").unwrap(); + let i32_arr = col.as_any().downcast_ref::().unwrap(); + assert_eq!(i32_arr.value(1), 150i32); + } + + #[tokio::test] + async fn test_filter_long_eq() { + let mut fixture = TableTestFixture::new(); + fixture.setup_manifest_files().await; + + // Filter: i64 == 150 + let mut builder = fixture.table.scan(); + let predicate = Reference::new("i64").equal_to(Datum::long(150i64)); + builder = builder + .with_filter(predicate) + .with_row_selection_enabled(true); + let table_scan = builder.build().unwrap(); + + let batch_stream = table_scan.to_arrow().await.unwrap(); + + let batches: Vec<_> = batch_stream.try_collect().await.unwrap(); + + assert_eq!(batches.len(), 2); + assert_eq!(batches[0].num_rows(), 12); + + let col = batches[0].column_by_name("i64").unwrap(); + let i64_arr = col.as_any().downcast_ref::().unwrap(); + assert_eq!(i64_arr.value(1), 150i64); + } + + #[tokio::test] + async fn test_filter_bool_eq() { + let mut fixture = TableTestFixture::new(); + fixture.setup_manifest_files().await; + + // Filter: bool == true + let mut builder = fixture.table.scan(); + let predicate = Reference::new("bool").equal_to(Datum::bool(true)); + builder = builder + .with_filter(predicate) + .with_row_selection_enabled(true); + let table_scan = builder.build().unwrap(); + + let batch_stream = table_scan.to_arrow().await.unwrap(); + + let batches: Vec<_> = batch_stream.try_collect().await.unwrap(); + + assert_eq!(batches.len(), 2); + assert_eq!(batches[0].num_rows(), 512); + + let col = batches[0].column_by_name("bool").unwrap(); + let bool_arr = col.as_any().downcast_ref::().unwrap(); + assert!(bool_arr.value(1)); + } + #[tokio::test] async fn test_filter_on_arrow_is_null() { let mut fixture = TableTestFixture::new(); @@ -1389,7 +1563,9 @@ mod tests { // Filter: y is null let mut builder = fixture.table.scan(); let predicate = Reference::new("y").is_null(); - builder = builder.with_filter(predicate); + builder = builder + .with_filter(predicate) + .with_row_selection_enabled(true); let table_scan = builder.build().unwrap(); let batch_stream = table_scan.to_arrow().await.unwrap(); @@ -1406,7 +1582,9 @@ mod tests { // Filter: y is not null let mut builder = fixture.table.scan(); let predicate = Reference::new("y").is_not_null(); - builder = builder.with_filter(predicate); + builder = builder + .with_filter(predicate) + .with_row_selection_enabled(true); let table_scan = builder.build().unwrap(); let batch_stream = table_scan.to_arrow().await.unwrap(); @@ -1425,7 +1603,9 @@ mod tests { let predicate = Reference::new("y") .less_than(Datum::long(5)) .and(Reference::new("z").greater_than_or_equal_to(Datum::long(4))); - builder = builder.with_filter(predicate); + builder = builder + .with_filter(predicate) + .with_row_selection_enabled(true); let table_scan = builder.build().unwrap(); let batch_stream = table_scan.to_arrow().await.unwrap(); @@ -1459,7 +1639,9 @@ mod tests { let predicate = Reference::new("y") .less_than(Datum::long(5)) .or(Reference::new("z").greater_than_or_equal_to(Datum::long(4))); - builder = builder.with_filter(predicate); + builder = builder + .with_filter(predicate) + .with_row_selection_enabled(true); let table_scan = builder.build().unwrap(); let batch_stream = table_scan.to_arrow().await.unwrap(); @@ -1494,7 +1676,9 @@ mod tests { // Filter: a STARTSWITH "Ice" let mut builder = fixture.table.scan(); let predicate = Reference::new("a").starts_with(Datum::string("Ice")); - builder = builder.with_filter(predicate); + builder = builder + .with_filter(predicate) + .with_row_selection_enabled(true); let table_scan = builder.build().unwrap(); let batch_stream = table_scan.to_arrow().await.unwrap(); @@ -1516,7 +1700,9 @@ mod tests { // Filter: a NOT STARTSWITH "Ice" let mut builder = fixture.table.scan(); let predicate = Reference::new("a").not_starts_with(Datum::string("Ice")); - builder = builder.with_filter(predicate); + builder = builder + .with_filter(predicate) + .with_row_selection_enabled(true); let table_scan = builder.build().unwrap(); let batch_stream = table_scan.to_arrow().await.unwrap(); @@ -1539,7 +1725,9 @@ mod tests { let mut builder = fixture.table.scan(); let predicate = Reference::new("a").is_in([Datum::string("Sioux"), Datum::string("Iceberg")]); - builder = builder.with_filter(predicate); + builder = builder + .with_filter(predicate) + .with_row_selection_enabled(true); let table_scan = builder.build().unwrap(); let batch_stream = table_scan.to_arrow().await.unwrap(); @@ -1562,7 +1750,9 @@ mod tests { let mut builder = fixture.table.scan(); let predicate = Reference::new("a").is_not_in([Datum::string("Sioux"), Datum::string("Iceberg")]); - builder = builder.with_filter(predicate); + builder = builder + .with_filter(predicate) + .with_row_selection_enabled(true); let table_scan = builder.build().unwrap(); let batch_stream = table_scan.to_arrow().await.unwrap(); diff --git a/crates/iceberg/testdata/example_table_metadata_v2.json b/crates/iceberg/testdata/example_table_metadata_v2.json index cf9fef96d..35230966a 100644 --- a/crates/iceberg/testdata/example_table_metadata_v2.json +++ b/crates/iceberg/testdata/example_table_metadata_v2.json @@ -16,7 +16,11 @@ {"id": 1, "name": "x", "required": true, "type": "long"}, {"id": 2, "name": "y", "required": true, "type": "long", "doc": "comment"}, {"id": 3, "name": "z", "required": true, "type": "long"}, - {"id": 4, "name": "a", "required": true, "type": "string"} + {"id": 4, "name": "a", "required": true, "type": "string"}, + {"id": 5, "name": "dbl", "required": true, "type": "double"}, + {"id": 6, "name": "i32", "required": true, "type": "int"}, + {"id": 7, "name": "i64", "required": true, "type": "long"}, + {"id": 8, "name": "bool", "required": true, "type": "boolean"} ] } ], From cda4a0c595af2606e2f4076e9ef81d79d4428f4b Mon Sep 17 00:00:00 2001 From: congyi wang <58715567+wcy-fdu@users.noreply.github.com> Date: Mon, 30 Sep 2024 11:07:36 +0800 Subject: [PATCH 06/46] chore: fix typo in FileIO Schemes (#653) * fix typo * fix typo --- crates/iceberg/src/io/file_io.rs | 2 +- crates/iceberg/src/io/storage.rs | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/crates/iceberg/src/io/file_io.rs b/crates/iceberg/src/io/file_io.rs index e0d83a08b..1ac2c1cd5 100644 --- a/crates/iceberg/src/io/file_io.rs +++ b/crates/iceberg/src/io/file_io.rs @@ -40,7 +40,7 @@ use crate::{Error, ErrorKind, Result}; /// | Local file system | `storage-fs` | `file` | /// | Memory | `storage-memory` | `memory` | /// | S3 | `storage-s3` | `s3`, `s3a`| -/// | GCS | `storage-gcs` | `gs` | +/// | GCS | `storage-gcs` | `gcs` | #[derive(Clone, Debug)] pub struct FileIO { inner: Arc, diff --git a/crates/iceberg/src/io/storage.rs b/crates/iceberg/src/io/storage.rs index cd9b54d54..b1f799a84 100644 --- a/crates/iceberg/src/io/storage.rs +++ b/crates/iceberg/src/io/storage.rs @@ -164,7 +164,7 @@ impl Storage { "memory" => Ok(Scheme::Memory), "file" | "" => Ok(Scheme::Fs), "s3" | "s3a" => Ok(Scheme::S3), - "gs" => Ok(Scheme::Gcs), + "gcs" => Ok(Scheme::Gcs), s => Ok(s.parse::()?), } } From d3b3ab102630845ede1b5cdfe1b67d52a784950d Mon Sep 17 00:00:00 2001 From: Christian Date: Thu, 3 Oct 2024 06:09:37 +0200 Subject: [PATCH 07/46] fix: TableUpdate Snapshot deserialization for v1 (#656) --- crates/iceberg/src/catalog/mod.rs | 96 +++++++++++++++++++++++++++++++ 1 file changed, 96 insertions(+) diff --git a/crates/iceberg/src/catalog/mod.rs b/crates/iceberg/src/catalog/mod.rs index 54abe8083..854c1269c 100644 --- a/crates/iceberg/src/catalog/mod.rs +++ b/crates/iceberg/src/catalog/mod.rs @@ -22,6 +22,7 @@ use std::fmt::Debug; use std::mem::take; use std::ops::Deref; +use _serde::deserialize_snapshot; use async_trait::async_trait; use serde_derive::{Deserialize, Serialize}; use typed_builder::TypedBuilder; @@ -401,6 +402,7 @@ pub enum TableUpdate { #[serde(rename_all = "kebab-case")] AddSnapshot { /// Snapshot to add. + #[serde(deserialize_with = "deserialize_snapshot")] snapshot: Snapshot, }, /// Set table's snapshot ref. @@ -451,6 +453,65 @@ impl TableUpdate { } } +pub(super) mod _serde { + use serde::{Deserialize as _, Deserializer}; + + use super::*; + use crate::spec::{SchemaId, Summary}; + + pub(super) fn deserialize_snapshot<'de, D>( + deserializer: D, + ) -> std::result::Result + where D: Deserializer<'de> { + let buf = CatalogSnapshot::deserialize(deserializer)?; + Ok(buf.into()) + } + + #[derive(Debug, Deserialize, PartialEq, Eq)] + #[serde(rename_all = "kebab-case")] + /// Defines the structure of a v2 snapshot for the catalog. + /// Main difference to SnapshotV2 is that sequence-number is optional + /// in the rest catalog spec to allow for backwards compatibility with v1. + struct CatalogSnapshot { + snapshot_id: i64, + #[serde(skip_serializing_if = "Option::is_none")] + parent_snapshot_id: Option, + #[serde(default)] + sequence_number: i64, + timestamp_ms: i64, + manifest_list: String, + summary: Summary, + #[serde(skip_serializing_if = "Option::is_none")] + schema_id: Option, + } + + impl From for Snapshot { + fn from(snapshot: CatalogSnapshot) -> Self { + let CatalogSnapshot { + snapshot_id, + parent_snapshot_id, + sequence_number, + timestamp_ms, + manifest_list, + schema_id, + summary, + } = snapshot; + let builder = Snapshot::builder() + .with_snapshot_id(snapshot_id) + .with_parent_snapshot_id(parent_snapshot_id) + .with_sequence_number(sequence_number) + .with_timestamp_ms(timestamp_ms) + .with_manifest_list(manifest_list) + .with_summary(summary); + if let Some(schema_id) = schema_id { + builder.with_schema_id(schema_id).build() + } else { + builder.build() + } + } + } +} + /// ViewCreation represents the creation of a view in the catalog. #[derive(Debug, TypedBuilder)] pub struct ViewCreation { @@ -968,6 +1029,41 @@ mod tests { test_serde_json(json, update); } + #[test] + fn test_add_snapshot_v1() { + let json = r#" +{ + "action": "add-snapshot", + "snapshot": { + "snapshot-id": 3055729675574597000, + "parent-snapshot-id": 3051729675574597000, + "timestamp-ms": 1555100955770, + "summary": { + "operation": "append" + }, + "manifest-list": "s3://a/b/2.avro" + } +} + "#; + + let update = TableUpdate::AddSnapshot { + snapshot: Snapshot::builder() + .with_snapshot_id(3055729675574597000) + .with_parent_snapshot_id(Some(3051729675574597000)) + .with_timestamp_ms(1555100955770) + .with_sequence_number(0) + .with_manifest_list("s3://a/b/2.avro") + .with_summary(Summary { + operation: Operation::Append, + other: HashMap::default(), + }) + .build(), + }; + + let actual: TableUpdate = serde_json::from_str(json).expect("Failed to parse from json"); + assert_eq!(actual, update, "Parsed value is not equal to expected"); + } + #[test] fn test_remove_snapshots() { let json = r#" From 2bc66c43cf0aa7cf79e6c1b82a47c39598616d77 Mon Sep 17 00:00:00 2001 From: Christian Date: Fri, 4 Oct 2024 05:22:10 +0200 Subject: [PATCH 08/46] feat: Reassign field ids for schema (#615) * Reassign field ids for schema * Address comments * Schema ensure unique field ids * Fix tests with duplicate nested field ids * Use Schema::builder() for reassigned ids * Better docs --- crates/iceberg/src/arrow/schema.rs | 20 +- crates/iceberg/src/spec/datatypes.rs | 6 + crates/iceberg/src/spec/schema.rs | 395 ++++++++++++++++++++++++++- crates/iceberg/src/spec/values.rs | 22 +- 4 files changed, 411 insertions(+), 32 deletions(-) diff --git a/crates/iceberg/src/arrow/schema.rs b/crates/iceberg/src/arrow/schema.rs index a32c10a22..e73b409c6 100644 --- a/crates/iceberg/src/arrow/schema.rs +++ b/crates/iceberg/src/arrow/schema.rs @@ -826,8 +826,8 @@ mod tests { fn arrow_schema_for_arrow_schema_to_schema_test() -> ArrowSchema { let fields = Fields::from(vec![ - simple_field("key", DataType::Int32, false, "17"), - simple_field("value", DataType::Utf8, true, "18"), + simple_field("key", DataType::Int32, false, "28"), + simple_field("value", DataType::Utf8, true, "29"), ]); let r#struct = DataType::Struct(fields); @@ -1057,9 +1057,9 @@ mod tests { "required": true, "type": { "type": "map", - "key-id": 17, + "key-id": 28, "key": "int", - "value-id": 18, + "value-id": 29, "value-required": false, "value": "string" } @@ -1110,8 +1110,8 @@ mod tests { fn arrow_schema_for_schema_to_arrow_schema_test() -> ArrowSchema { let fields = Fields::from(vec![ - simple_field("key", DataType::Int32, false, "17"), - simple_field("value", DataType::Utf8, true, "18"), + simple_field("key", DataType::Int32, false, "28"), + simple_field("value", DataType::Utf8, true, "29"), ]); let r#struct = DataType::Struct(fields); @@ -1200,7 +1200,7 @@ mod tests { ), simple_field("map", map, false, "16"), simple_field("struct", r#struct, false, "17"), - simple_field("uuid", DataType::FixedSizeBinary(16), false, "26"), + simple_field("uuid", DataType::FixedSizeBinary(16), false, "30"), ]) } @@ -1344,9 +1344,9 @@ mod tests { "required": true, "type": { "type": "map", - "key-id": 17, + "key-id": 28, "key": "int", - "value-id": 18, + "value-id": 29, "value-required": false, "value": "string" } @@ -1380,7 +1380,7 @@ mod tests { } }, { - "id":26, + "id":30, "name":"uuid", "required":true, "type":"uuid" diff --git a/crates/iceberg/src/spec/datatypes.rs b/crates/iceberg/src/spec/datatypes.rs index d38245960..bce10ad5f 100644 --- a/crates/iceberg/src/spec/datatypes.rs +++ b/crates/iceberg/src/spec/datatypes.rs @@ -668,6 +668,12 @@ impl NestedField { self.write_default = Some(value); self } + + /// Set the id of the field. + pub(crate) fn with_id(mut self, id: i32) -> Self { + self.id = id; + self + } } impl fmt::Display for NestedField { diff --git a/crates/iceberg/src/spec/schema.rs b/crates/iceberg/src/spec/schema.rs index 63a9e3cb4..cf86874dc 100644 --- a/crates/iceberg/src/spec/schema.rs +++ b/crates/iceberg/src/spec/schema.rs @@ -39,7 +39,7 @@ use crate::{ensure_data_valid, Error, ErrorKind}; pub type SchemaId = i32; /// Reference to [`Schema`]. pub type SchemaRef = Arc; -const DEFAULT_SCHEMA_ID: SchemaId = 0; +pub(crate) const DEFAULT_SCHEMA_ID: SchemaId = 0; /// Defines schema in iceberg. #[derive(Debug, Serialize, Deserialize, Clone)] @@ -77,6 +77,7 @@ pub struct SchemaBuilder { fields: Vec, alias_to_id: BiHashMap, identifier_field_ids: HashSet, + reassign_field_ids_from: Option, } impl SchemaBuilder { @@ -86,6 +87,16 @@ impl SchemaBuilder { self } + /// Reassign all field-ids (including nested) on build. + /// Reassignment starts from the field-id specified in `start_from` (inclusive). + /// + /// All specified aliases and identifier fields will be updated to the new field-ids. + #[allow(dead_code)] // Will be needed in TableMetadataBuilder + pub(crate) fn with_reassigned_field_ids(mut self, start_from: u32) -> Self { + self.reassign_field_ids_from = Some(start_from.try_into().unwrap_or(i32::MAX)); + self + } + /// Set schema id. pub fn with_schema_id(mut self, schema_id: i32) -> Self { self.schema_id = schema_id; @@ -130,7 +141,7 @@ impl SchemaBuilder { let highest_field_id = id_to_field.keys().max().cloned().unwrap_or(0); - Ok(Schema { + let mut schema = Schema { r#struct, schema_id: self.schema_id, highest_field_id, @@ -143,7 +154,24 @@ impl SchemaBuilder { id_to_name, field_id_to_accessor, - }) + }; + + if let Some(start_from) = self.reassign_field_ids_from { + let mut id_reassigner = ReassignFieldIds::new(start_from); + let new_fields = id_reassigner.reassign_field_ids(schema.r#struct.fields().to_vec())?; + let new_identifier_field_ids = + id_reassigner.apply_to_identifier_fields(schema.identifier_field_ids)?; + let new_alias_to_id = id_reassigner.apply_to_aliases(schema.alias_to_id.clone())?; + + schema = Schema::builder() + .with_schema_id(schema.schema_id) + .with_fields(new_fields) + .with_identifier_field_ids(new_identifier_field_ids) + .with_alias(new_alias_to_id) + .build()?; + } + + Ok(schema) } fn build_accessors(&self) -> HashMap> { @@ -265,6 +293,7 @@ impl Schema { fields: vec![], identifier_field_ids: HashSet::default(), alias_to_id: BiHashMap::default(), + reassign_field_ids_from: None, } } @@ -275,6 +304,7 @@ impl Schema { fields: self.r#struct.fields().to_vec(), alias_to_id: self.alias_to_id, identifier_field_ids: self.identifier_field_ids, + reassign_field_ids_from: None, } } @@ -475,8 +505,7 @@ pub fn index_by_id(r#struct: &StructType) -> Result } fn field(&mut self, field: &NestedFieldRef, _value: ()) -> Result<()> { - self.0.insert(field.id, field.clone()); - Ok(()) + try_insert_field(&mut self.0, field.id, field.clone()) } fn r#struct(&mut self, _struct: &StructType, _results: Vec) -> Result { @@ -484,15 +513,16 @@ pub fn index_by_id(r#struct: &StructType) -> Result } fn list(&mut self, list: &ListType, _value: Self::T) -> Result { - self.0 - .insert(list.element_field.id, list.element_field.clone()); - Ok(()) + try_insert_field( + &mut self.0, + list.element_field.id, + list.element_field.clone(), + ) } fn map(&mut self, map: &MapType, _key_value: Self::T, _value: Self::T) -> Result { - self.0.insert(map.key_field.id, map.key_field.clone()); - self.0.insert(map.value_field.id, map.value_field.clone()); - Ok(()) + try_insert_field(&mut self.0, map.key_field.id, map.key_field.clone())?; + try_insert_field(&mut self.0, map.value_field.id, map.value_field.clone()) } fn primitive(&mut self, _: &PrimitiveType) -> Result { @@ -943,6 +973,148 @@ impl SchemaVisitor for PruneColumn { } } +struct ReassignFieldIds { + next_field_id: i32, + old_to_new_id: HashMap, +} + +fn try_insert_field(map: &mut HashMap, field_id: i32, value: V) -> Result<()> { + map.insert(field_id, value).map_or_else( + || Ok(()), + |_| { + Err(Error::new( + ErrorKind::DataInvalid, + format!( + "Found duplicate 'field.id' {}. Field ids must be unique.", + field_id + ), + )) + }, + ) +} + +// We are not using the visitor here, as post order traversal is not desired. +// Instead we want to re-assign all fields on one level first before diving deeper. +impl ReassignFieldIds { + fn new(start_from: i32) -> Self { + Self { + next_field_id: start_from, + old_to_new_id: HashMap::new(), + } + } + + fn reassign_field_ids(&mut self, fields: Vec) -> Result> { + // Visit fields on the same level first + let outer_fields = fields + .into_iter() + .map(|field| { + try_insert_field(&mut self.old_to_new_id, field.id, self.next_field_id)?; + let new_field = Arc::unwrap_or_clone(field).with_id(self.next_field_id); + self.increase_next_field_id()?; + Ok(Arc::new(new_field)) + }) + .collect::>>()?; + + // Now visit nested fields + outer_fields + .into_iter() + .map(|field| { + if field.field_type.is_primitive() { + Ok(field) + } else { + let mut new_field = Arc::unwrap_or_clone(field); + *new_field.field_type = self.reassign_ids_visit_type(*new_field.field_type)?; + Ok(Arc::new(new_field)) + } + }) + .collect() + } + + fn reassign_ids_visit_type(&mut self, field_type: Type) -> Result { + match field_type { + Type::Primitive(s) => Ok(Type::Primitive(s)), + Type::Struct(s) => { + let new_fields = self.reassign_field_ids(s.fields().to_vec())?; + Ok(Type::Struct(StructType::new(new_fields))) + } + Type::List(l) => { + self.old_to_new_id + .insert(l.element_field.id, self.next_field_id); + let mut element_field = Arc::unwrap_or_clone(l.element_field); + element_field.id = self.next_field_id; + self.increase_next_field_id()?; + *element_field.field_type = + self.reassign_ids_visit_type(*element_field.field_type)?; + Ok(Type::List(ListType { + element_field: Arc::new(element_field), + })) + } + Type::Map(m) => { + self.old_to_new_id + .insert(m.key_field.id, self.next_field_id); + let mut key_field = Arc::unwrap_or_clone(m.key_field); + key_field.id = self.next_field_id; + self.increase_next_field_id()?; + *key_field.field_type = self.reassign_ids_visit_type(*key_field.field_type)?; + + self.old_to_new_id + .insert(m.value_field.id, self.next_field_id); + let mut value_field = Arc::unwrap_or_clone(m.value_field); + value_field.id = self.next_field_id; + self.increase_next_field_id()?; + *value_field.field_type = self.reassign_ids_visit_type(*value_field.field_type)?; + + Ok(Type::Map(MapType { + key_field: Arc::new(key_field), + value_field: Arc::new(value_field), + })) + } + } + } + + fn increase_next_field_id(&mut self) -> Result<()> { + self.next_field_id = self.next_field_id.checked_add(1).ok_or_else(|| { + Error::new( + ErrorKind::DataInvalid, + "Field ID overflowed, cannot add more fields", + ) + })?; + Ok(()) + } + + fn apply_to_identifier_fields(&self, field_ids: HashSet) -> Result> { + field_ids + .into_iter() + .map(|id| { + self.old_to_new_id.get(&id).copied().ok_or_else(|| { + Error::new( + ErrorKind::DataInvalid, + format!("Identifier Field ID {} not found", id), + ) + }) + }) + .collect() + } + + fn apply_to_aliases(&self, alias: BiHashMap) -> Result> { + alias + .into_iter() + .map(|(name, id)| { + self.old_to_new_id + .get(&id) + .copied() + .ok_or_else(|| { + Error::new( + ErrorKind::DataInvalid, + format!("Field with id {} for alias {} not found", id, name), + ) + }) + .map(|new_id| (name, new_id)) + }) + .collect() + } +} + pub(super) mod _serde { /// This is a helper module that defines types to help with serialization/deserialization. /// For deserialization the input first gets read into either the [SchemaV1] or [SchemaV2] struct @@ -1062,6 +1234,8 @@ pub(super) mod _serde { mod tests { use std::collections::{HashMap, HashSet}; + use bimap::BiHashMap; + use super::DEFAULT_SCHEMA_ID; use crate::spec::datatypes::Type::{List, Map, Primitive, Struct}; use crate::spec::datatypes::{ @@ -2237,4 +2411,203 @@ table { let schema = table_schema_simple().0; assert_eq!(3, schema.highest_field_id()); } + + #[test] + fn test_highest_field_id_no_fields() { + let schema = Schema::builder().with_schema_id(1).build().unwrap(); + assert_eq!(0, schema.highest_field_id()); + } + + #[test] + fn test_reassign_ids() { + let schema = Schema::builder() + .with_schema_id(1) + .with_identifier_field_ids(vec![3]) + .with_alias(BiHashMap::from_iter(vec![("bar_alias".to_string(), 3)])) + .with_fields(vec![ + NestedField::optional(5, "foo", Type::Primitive(PrimitiveType::String)).into(), + NestedField::required(3, "bar", Type::Primitive(PrimitiveType::Int)).into(), + NestedField::optional(4, "baz", Type::Primitive(PrimitiveType::Boolean)).into(), + ]) + .build() + .unwrap(); + + let reassigned_schema = schema + .into_builder() + .with_reassigned_field_ids(0) + .build() + .unwrap(); + + let expected = Schema::builder() + .with_schema_id(1) + .with_identifier_field_ids(vec![1]) + .with_alias(BiHashMap::from_iter(vec![("bar_alias".to_string(), 1)])) + .with_fields(vec![ + NestedField::optional(0, "foo", Type::Primitive(PrimitiveType::String)).into(), + NestedField::required(1, "bar", Type::Primitive(PrimitiveType::Int)).into(), + NestedField::optional(2, "baz", Type::Primitive(PrimitiveType::Boolean)).into(), + ]) + .build() + .unwrap(); + + pretty_assertions::assert_eq!(expected, reassigned_schema); + assert_eq!(reassigned_schema.highest_field_id(), 2); + } + + #[test] + fn test_reassigned_ids_nested() { + let schema = table_schema_nested(); + let reassigned_schema = schema + .into_builder() + .with_alias(BiHashMap::from_iter(vec![("bar_alias".to_string(), 2)])) + .with_reassigned_field_ids(0) + .build() + .unwrap(); + + let expected = Schema::builder() + .with_schema_id(1) + .with_identifier_field_ids(vec![1]) + .with_alias(BiHashMap::from_iter(vec![("bar_alias".to_string(), 1)])) + .with_fields(vec![ + NestedField::optional(0, "foo", Type::Primitive(PrimitiveType::String)).into(), + NestedField::required(1, "bar", Type::Primitive(PrimitiveType::Int)).into(), + NestedField::optional(2, "baz", Type::Primitive(PrimitiveType::Boolean)).into(), + NestedField::required( + 3, + "qux", + Type::List(ListType { + element_field: NestedField::list_element( + 7, + Type::Primitive(PrimitiveType::String), + true, + ) + .into(), + }), + ) + .into(), + NestedField::required( + 4, + "quux", + Type::Map(MapType { + key_field: NestedField::map_key_element( + 8, + Type::Primitive(PrimitiveType::String), + ) + .into(), + value_field: NestedField::map_value_element( + 9, + Type::Map(MapType { + key_field: NestedField::map_key_element( + 10, + Type::Primitive(PrimitiveType::String), + ) + .into(), + value_field: NestedField::map_value_element( + 11, + Type::Primitive(PrimitiveType::Int), + true, + ) + .into(), + }), + true, + ) + .into(), + }), + ) + .into(), + NestedField::required( + 5, + "location", + Type::List(ListType { + element_field: NestedField::list_element( + 12, + Type::Struct(StructType::new(vec![ + NestedField::optional( + 13, + "latitude", + Type::Primitive(PrimitiveType::Float), + ) + .into(), + NestedField::optional( + 14, + "longitude", + Type::Primitive(PrimitiveType::Float), + ) + .into(), + ])), + true, + ) + .into(), + }), + ) + .into(), + NestedField::optional( + 6, + "person", + Type::Struct(StructType::new(vec![ + NestedField::optional(15, "name", Type::Primitive(PrimitiveType::String)) + .into(), + NestedField::required(16, "age", Type::Primitive(PrimitiveType::Int)) + .into(), + ])), + ) + .into(), + ]) + .build() + .unwrap(); + + pretty_assertions::assert_eq!(expected, reassigned_schema); + assert_eq!(reassigned_schema.highest_field_id(), 16); + assert_eq!(reassigned_schema.field_by_id(6).unwrap().name, "person"); + assert_eq!(reassigned_schema.field_by_id(16).unwrap().name, "age"); + } + + #[test] + fn test_reassign_ids_fails_with_duplicate_ids() { + let reassigned_schema = Schema::builder() + .with_schema_id(1) + .with_identifier_field_ids(vec![5]) + .with_alias(BiHashMap::from_iter(vec![("bar_alias".to_string(), 3)])) + .with_fields(vec![ + NestedField::required(5, "foo", Type::Primitive(PrimitiveType::String)).into(), + NestedField::optional(3, "bar", Type::Primitive(PrimitiveType::Int)).into(), + NestedField::optional(3, "baz", Type::Primitive(PrimitiveType::Boolean)).into(), + ]) + .with_reassigned_field_ids(0) + .build() + .unwrap_err(); + + assert!(reassigned_schema.message().contains("'field.id' 3")); + } + + #[test] + fn test_field_ids_must_be_unique() { + let reassigned_schema = Schema::builder() + .with_schema_id(1) + .with_identifier_field_ids(vec![5]) + .with_alias(BiHashMap::from_iter(vec![("bar_alias".to_string(), 3)])) + .with_fields(vec![ + NestedField::required(5, "foo", Type::Primitive(PrimitiveType::String)).into(), + NestedField::optional(3, "bar", Type::Primitive(PrimitiveType::Int)).into(), + NestedField::optional(3, "baz", Type::Primitive(PrimitiveType::Boolean)).into(), + ]) + .build() + .unwrap_err(); + + assert!(reassigned_schema.message().contains("'field.id' 3")); + } + + #[test] + fn test_reassign_ids_empty_schema() { + let schema = Schema::builder().with_schema_id(1).build().unwrap(); + let reassigned_schema = schema + .clone() + .into_builder() + .with_reassigned_field_ids(0) + .build() + .unwrap(); + + assert_eq!(schema, reassigned_schema); + assert_eq!(schema.highest_field_id(), 0); + } } diff --git a/crates/iceberg/src/spec/values.rs b/crates/iceberg/src/spec/values.rs index 3568d3dcd..3c6e2aa68 100644 --- a/crates/iceberg/src/spec/values.rs +++ b/crates/iceberg/src/spec/values.rs @@ -3192,10 +3192,10 @@ mod tests { (Literal::Primitive(PrimitiveLiteral::Int(3)), None), ])), &Type::Map(MapType { - key_field: NestedField::map_key_element(0, Type::Primitive(PrimitiveType::Int)) + key_field: NestedField::map_key_element(2, Type::Primitive(PrimitiveType::Int)) .into(), value_field: NestedField::map_value_element( - 1, + 3, Type::Primitive(PrimitiveType::Long), false, ) @@ -3219,10 +3219,10 @@ mod tests { ), ])), &Type::Map(MapType { - key_field: NestedField::map_key_element(0, Type::Primitive(PrimitiveType::Int)) + key_field: NestedField::map_key_element(2, Type::Primitive(PrimitiveType::Int)) .into(), value_field: NestedField::map_value_element( - 1, + 3, Type::Primitive(PrimitiveType::Long), true, ) @@ -3249,10 +3249,10 @@ mod tests { ), ])), &Type::Map(MapType { - key_field: NestedField::map_key_element(0, Type::Primitive(PrimitiveType::String)) + key_field: NestedField::map_key_element(2, Type::Primitive(PrimitiveType::String)) .into(), value_field: NestedField::map_value_element( - 1, + 3, Type::Primitive(PrimitiveType::Int), false, ) @@ -3276,10 +3276,10 @@ mod tests { ), ])), &Type::Map(MapType { - key_field: NestedField::map_key_element(0, Type::Primitive(PrimitiveType::String)) + key_field: NestedField::map_key_element(2, Type::Primitive(PrimitiveType::String)) .into(), value_field: NestedField::map_value_element( - 1, + 3, Type::Primitive(PrimitiveType::Int), true, ) @@ -3299,9 +3299,9 @@ mod tests { None, ])), &Type::Struct(StructType::new(vec![ - NestedField::required(1, "id", Type::Primitive(PrimitiveType::Int)).into(), - NestedField::optional(2, "name", Type::Primitive(PrimitiveType::String)).into(), - NestedField::optional(3, "address", Type::Primitive(PrimitiveType::String)).into(), + NestedField::required(2, "id", Type::Primitive(PrimitiveType::Int)).into(), + NestedField::optional(3, "name", Type::Primitive(PrimitiveType::String)).into(), + NestedField::optional(4, "address", Type::Primitive(PrimitiveType::String)).into(), ])), ); } From 4c3e47d765fa6a864c8b9e5604b30e06960af541 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Tobias=20P=C3=BCtz?= Date: Wed, 9 Oct 2024 10:51:41 +0200 Subject: [PATCH 09/46] feat: add gcp oauth support (#654) --- crates/iceberg/src/io/storage_gcs.rs | 14 ++++++++++++++ 1 file changed, 14 insertions(+) diff --git a/crates/iceberg/src/io/storage_gcs.rs b/crates/iceberg/src/io/storage_gcs.rs index 0a2410799..de938e552 100644 --- a/crates/iceberg/src/io/storage_gcs.rs +++ b/crates/iceberg/src/io/storage_gcs.rs @@ -34,11 +34,25 @@ pub const GCS_SERVICE_PATH: &str = "gcs.service.path"; pub const GCS_USER_PROJECT: &str = "gcs.user-project"; /// Allow unauthenticated requests pub const GCS_NO_AUTH: &str = "gcs.no-auth"; +/// Google Cloud Storage credentials JSON string, base64 encoded. +/// +/// E.g. base64::prelude::BASE64_STANDARD.encode(serde_json::to_string(credential).as_bytes()) +pub const GCS_CREDENTIALS_JSON: &str = "gcs.credentials-json"; +/// Google Cloud Storage token +pub const GCS_TOKEN: &str = "gcs.oauth2.token"; /// Parse iceberg properties to [`GcsConfig`]. pub(crate) fn gcs_config_parse(mut m: HashMap) -> Result { let mut cfg = GcsConfig::default(); + if let Some(cred) = m.remove(GCS_CREDENTIALS_JSON) { + cfg.credential = Some(cred); + } + + if let Some(token) = m.remove(GCS_TOKEN) { + cfg.token = Some(token); + } + if let Some(endpoint) = m.remove(GCS_SERVICE_PATH) { cfg.endpoint = Some(endpoint); } From 8cf581b2edc574fbd591cb9090dff55efb0f8eb7 Mon Sep 17 00:00:00 2001 From: Xuanwo Date: Thu, 10 Oct 2024 10:11:43 +0800 Subject: [PATCH 10/46] fix(arrow): Use new ParquetMetaDataReader instead (#661) Signed-off-by: Xuanwo --- crates/iceberg/src/arrow/reader.rs | 20 +++++++++++--------- 1 file changed, 11 insertions(+), 9 deletions(-) diff --git a/crates/iceberg/src/arrow/reader.rs b/crates/iceberg/src/arrow/reader.rs index ed422be99..f6680e312 100644 --- a/crates/iceberg/src/arrow/reader.rs +++ b/crates/iceberg/src/arrow/reader.rs @@ -31,11 +31,11 @@ use bytes::Bytes; use fnv::FnvHashSet; use futures::channel::mpsc::{channel, Sender}; use futures::future::BoxFuture; -use futures::{try_join, SinkExt, StreamExt, TryFutureExt, TryStreamExt}; +use futures::{try_join, FutureExt, SinkExt, StreamExt, TryFutureExt, TryStreamExt}; use parquet::arrow::arrow_reader::{ArrowPredicateFn, ArrowReaderOptions, RowFilter, RowSelection}; -use parquet::arrow::async_reader::{AsyncFileReader, MetadataLoader}; +use parquet::arrow::async_reader::AsyncFileReader; use parquet::arrow::{ParquetRecordBatchStreamBuilder, ProjectionMask, PARQUET_FIELD_ID_META_KEY}; -use parquet::file::metadata::ParquetMetaData; +use parquet::file::metadata::{ParquetMetaData, ParquetMetaDataReader}; use parquet::schema::types::{SchemaDescriptor, Type as ParquetType}; use crate::arrow::{arrow_schema_to_schema, get_arrow_datum}; @@ -1078,12 +1078,14 @@ impl AsyncFileReader for ArrowFileReader { } fn get_metadata(&mut self) -> BoxFuture<'_, parquet::errors::Result>> { - Box::pin(async move { - let file_size = self.meta.size; - let mut loader = MetadataLoader::load(self, file_size as usize, None).await?; - loader.load_page_index(false, false).await?; - Ok(Arc::new(loader.finish())) - }) + async move { + let reader = ParquetMetaDataReader::new(); + let size = self.meta.size as usize; + let meta = reader.load_and_finish(self, size).await?; + + Ok(Arc::new(meta)) + } + .boxed() } } From d09e32ee3e64bad2082d27ef21c3a8be7a3274b6 Mon Sep 17 00:00:00 2001 From: Matt Willian Date: Thu, 10 Oct 2024 20:22:47 -0500 Subject: [PATCH 11/46] chore(deps): bump typos crate to 1.25.0 (#662) --- .github/workflows/ci_typos.yml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/.github/workflows/ci_typos.yml b/.github/workflows/ci_typos.yml index 2a12a4ba7..e6031fbd1 100644 --- a/.github/workflows/ci_typos.yml +++ b/.github/workflows/ci_typos.yml @@ -42,4 +42,4 @@ jobs: steps: - uses: actions/checkout@v4 - name: Check typos - uses: crate-ci/typos@v1.24.6 + uses: crate-ci/typos@v1.25.0 From 5c1a9e68da346819072a15327080a498ad91c488 Mon Sep 17 00:00:00 2001 From: Scott Donnelly Date: Fri, 11 Oct 2024 08:27:47 +0100 Subject: [PATCH 12/46] RecordBatchTransformer: Handle schema migration and column re-ordering in table scans (#602) * feat: Add skeleton of RecordBatchEvolutionProcessor * feat: Add initial implementation of RecordBatchEvolutionProcessor * feat: support more column types. Improve error handling. Add more comments * feat(wip): adress issues with reordered / skipped fields * feat: RecordBatchEvolutionProcessor handles skipped fields in projection * chore: add missing license header * chore: remove unneeded comment * refactor: rename to RecordBatchTransformer. Improve passthrough handling * feat: more performant handling of case where only schema transform is required but columns can remain unmodified * refactor: import arrow_cast rather than arrow --- Cargo.toml | 1 + crates/iceberg/Cargo.toml | 1 + crates/iceberg/src/arrow/mod.rs | 2 + crates/iceberg/src/arrow/reader.rs | 11 +- .../src/arrow/record_batch_transformer.rs | 622 ++++++++++++++++++ crates/iceberg/src/arrow/schema.rs | 2 +- crates/iceberg/src/error.rs | 6 + crates/iceberg/src/lib.rs | 1 + crates/iceberg/src/scan.rs | 27 + 9 files changed, 671 insertions(+), 2 deletions(-) create mode 100644 crates/iceberg/src/arrow/record_batch_transformer.rs diff --git a/Cargo.toml b/Cargo.toml index 82f98103e..5e2b89730 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -41,6 +41,7 @@ apache-avro = "0.17" array-init = "2" arrow-arith = { version = "53" } arrow-array = { version = "53" } +arrow-cast = { version = "53" } arrow-ord = { version = "53" } arrow-schema = { version = "53" } arrow-select = { version = "53" } diff --git a/crates/iceberg/Cargo.toml b/crates/iceberg/Cargo.toml index 4d0160949..1307cc6f3 100644 --- a/crates/iceberg/Cargo.toml +++ b/crates/iceberg/Cargo.toml @@ -46,6 +46,7 @@ apache-avro = { workspace = true } array-init = { workspace = true } arrow-arith = { workspace = true } arrow-array = { workspace = true } +arrow-cast = { workspace = true } arrow-ord = { workspace = true } arrow-schema = { workspace = true } arrow-select = { workspace = true } diff --git a/crates/iceberg/src/arrow/mod.rs b/crates/iceberg/src/arrow/mod.rs index 2076a958f..31a892fa8 100644 --- a/crates/iceberg/src/arrow/mod.rs +++ b/crates/iceberg/src/arrow/mod.rs @@ -20,4 +20,6 @@ mod schema; pub use schema::*; mod reader; +pub(crate) mod record_batch_transformer; + pub use reader::*; diff --git a/crates/iceberg/src/arrow/reader.rs b/crates/iceberg/src/arrow/reader.rs index f6680e312..66c233f65 100644 --- a/crates/iceberg/src/arrow/reader.rs +++ b/crates/iceberg/src/arrow/reader.rs @@ -38,6 +38,7 @@ use parquet::arrow::{ParquetRecordBatchStreamBuilder, ProjectionMask, PARQUET_FI use parquet::file::metadata::{ParquetMetaData, ParquetMetaDataReader}; use parquet::schema::types::{SchemaDescriptor, Type as ParquetType}; +use crate::arrow::record_batch_transformer::RecordBatchTransformer; use crate::arrow::{arrow_schema_to_schema, get_arrow_datum}; use crate::error::Result; use crate::expr::visitors::bound_predicate_visitor::{visit, BoundPredicateVisitor}; @@ -209,6 +210,12 @@ impl ArrowReader { )?; record_batch_stream_builder = record_batch_stream_builder.with_projection(projection_mask); + // RecordBatchTransformer performs any required transformations on the RecordBatches + // that come back from the file, such as type promotion, default column insertion + // and column re-ordering + let mut record_batch_transformer = + RecordBatchTransformer::build(task.schema_ref(), task.project_field_ids()); + if let Some(batch_size) = batch_size { record_batch_stream_builder = record_batch_stream_builder.with_batch_size(batch_size); } @@ -261,8 +268,10 @@ impl ArrowReader { // Build the batch stream and send all the RecordBatches that it generates // to the requester. let mut record_batch_stream = record_batch_stream_builder.build()?; + while let Some(batch) = record_batch_stream.try_next().await? { - tx.send(Ok(batch)).await? + tx.send(record_batch_transformer.process_record_batch(batch)) + .await? } Ok(()) diff --git a/crates/iceberg/src/arrow/record_batch_transformer.rs b/crates/iceberg/src/arrow/record_batch_transformer.rs new file mode 100644 index 000000000..01ce9f0a8 --- /dev/null +++ b/crates/iceberg/src/arrow/record_batch_transformer.rs @@ -0,0 +1,622 @@ +// 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. + +use std::collections::HashMap; +use std::sync::Arc; + +use arrow_array::{ + Array as ArrowArray, ArrayRef, BinaryArray, BooleanArray, Float32Array, Float64Array, + Int32Array, Int64Array, NullArray, RecordBatch, StringArray, +}; +use arrow_cast::cast; +use arrow_schema::{ + DataType, FieldRef, Schema as ArrowSchema, SchemaRef as ArrowSchemaRef, SchemaRef, +}; +use parquet::arrow::PARQUET_FIELD_ID_META_KEY; + +use crate::arrow::schema_to_arrow_schema; +use crate::spec::{Literal, PrimitiveLiteral, Schema as IcebergSchema}; +use crate::{Error, ErrorKind, Result}; + +/// Indicates how a particular column in a processed RecordBatch should +/// be sourced. +#[derive(Debug)] +pub(crate) enum ColumnSource { + // signifies that a column should be passed through unmodified + // from the file's RecordBatch + PassThrough { + source_index: usize, + }, + + // signifies that a column from the file's RecordBatch has undergone + // type promotion so the source column with the given index needs + // to be promoted to the specified type + Promote { + target_type: DataType, + source_index: usize, + }, + + // Signifies that a new column has been inserted before the column + // with index `index`. (we choose "before" rather than "after" so + // that we can use usize; if we insert after, then we need to + // be able to store -1 here to signify that a new + // column is to be added at the front of the column list). + // If multiple columns need to be inserted at a given + // location, they should all be given the same index, as the index + // here refers to the original RecordBatch, not the interim state after + // a preceding operation. + Add { + target_type: DataType, + value: Option, + }, + // The iceberg spec refers to other permissible schema evolution actions + // (see https://iceberg.apache.org/spec/#schema-evolution): + // renaming fields, deleting fields and reordering fields. + // Renames only affect the schema of the RecordBatch rather than the + // columns themselves, so a single updated cached schema can + // be re-used and no per-column actions are required. + // Deletion and Reorder can be achieved without needing this + // post-processing step by using the projection mask. +} + +#[derive(Debug)] +enum BatchTransform { + // Indicates that no changes need to be performed to the RecordBatches + // coming in from the stream and that they can be passed through + // unmodified + PassThrough, + + Modify { + // Every transformed RecordBatch will have the same schema. We create the + // target just once and cache it here. Helpfully, Arc is needed in + // the constructor for RecordBatch, so we don't need an expensive copy + // each time we build a new RecordBatch + target_schema: Arc, + + // Indicates how each column in the target schema is derived. + operations: Vec, + }, + + // Sometimes only the schema will need modifying, for example when + // the column names have changed vs the file, but not the column types. + // we can avoid a heap allocation per RecordBach in this case by retaining + // the existing column Vec. + ModifySchema { + target_schema: Arc, + }, +} + +#[derive(Debug)] +enum SchemaComparison { + Equivalent, + NameChangesOnly, + Different, +} + +#[derive(Debug)] +pub(crate) struct RecordBatchTransformer { + snapshot_schema: Arc, + projected_iceberg_field_ids: Vec, + + // BatchTransform gets lazily constructed based on the schema of + // the first RecordBatch we receive from the file + batch_transform: Option, +} + +impl RecordBatchTransformer { + /// Build a RecordBatchTransformer for a given + /// Iceberg snapshot schema and list of projected field ids. + pub(crate) fn build( + snapshot_schema: Arc, + projected_iceberg_field_ids: &[i32], + ) -> Self { + let projected_iceberg_field_ids = if projected_iceberg_field_ids.is_empty() { + // If the list of field ids is empty, this indicates that we + // need to select all fields. + // Project all fields in table schema order + snapshot_schema + .as_struct() + .fields() + .iter() + .map(|field| field.id) + .collect() + } else { + projected_iceberg_field_ids.to_vec() + }; + + Self { + snapshot_schema, + projected_iceberg_field_ids, + batch_transform: None, + } + } + + pub(crate) fn process_record_batch( + &mut self, + record_batch: RecordBatch, + ) -> Result { + Ok(match &self.batch_transform { + Some(BatchTransform::PassThrough) => record_batch, + Some(BatchTransform::Modify { + ref target_schema, + ref operations, + }) => RecordBatch::try_new( + target_schema.clone(), + self.transform_columns(record_batch.columns(), operations)?, + )?, + Some(BatchTransform::ModifySchema { target_schema }) => { + record_batch.with_schema(target_schema.clone())? + } + None => { + self.batch_transform = Some(Self::generate_batch_transform( + record_batch.schema_ref(), + self.snapshot_schema.as_ref(), + &self.projected_iceberg_field_ids, + )?); + + self.process_record_batch(record_batch)? + } + }) + } + + // Compare the schema of the incoming RecordBatches to the schema of + // the Iceberg snapshot to determine what, if any, transformation + // needs to be applied. If the schemas match, we return BatchTransform::PassThrough + // to indicate that no changes need to be made. Otherwise, we return a + // BatchTransform::Modify containing the target RecordBatch schema and + // the list of `ColumnSource`s that indicate how to source each column in + // the resulting RecordBatches. + fn generate_batch_transform( + source_schema: &ArrowSchemaRef, + snapshot_schema: &IcebergSchema, + projected_iceberg_field_ids: &[i32], + ) -> Result { + let mapped_unprojected_arrow_schema = Arc::new(schema_to_arrow_schema(snapshot_schema)?); + let field_id_to_mapped_schema_map = + Self::build_field_id_to_arrow_schema_map(&mapped_unprojected_arrow_schema)?; + + // Create a new arrow schema by selecting fields from mapped_unprojected, + // in the order of the field ids in projected_iceberg_field_ids + let fields: Result> = projected_iceberg_field_ids + .iter() + .map(|field_id| { + Ok(field_id_to_mapped_schema_map + .get(field_id) + .ok_or(Error::new(ErrorKind::Unexpected, "field not found"))? + .0 + .clone()) + }) + .collect(); + + let target_schema = Arc::new(ArrowSchema::new(fields?)); + + match Self::compare_schemas(source_schema, &target_schema) { + SchemaComparison::Equivalent => Ok(BatchTransform::PassThrough), + SchemaComparison::NameChangesOnly => Ok(BatchTransform::ModifySchema { target_schema }), + SchemaComparison::Different => Ok(BatchTransform::Modify { + operations: Self::generate_transform_operations( + source_schema, + snapshot_schema, + projected_iceberg_field_ids, + field_id_to_mapped_schema_map, + )?, + target_schema, + }), + } + } + + /// Compares the source and target schemas + /// Determines if they have changed in any meaningful way: + /// * If they have different numbers of fields, then we need to modify + /// the incoming RecordBatch schema AND columns + /// * If they have the same number of fields, but some of them differ in + /// either data type or nullability, then we need to modify the + /// incoming RecordBatch schema AND columns + /// * If the schemas differ only in the column names, then we need + /// to modify the RecordBatch schema BUT we can keep the + /// original column data unmodified + /// * If the schemas are identical (or differ only in inconsequential + /// ways) then we can pass through the original RecordBatch unmodified + fn compare_schemas( + source_schema: &ArrowSchemaRef, + target_schema: &ArrowSchemaRef, + ) -> SchemaComparison { + if source_schema.fields().len() != target_schema.fields().len() { + return SchemaComparison::Different; + } + + let mut names_changed = false; + + for (source_field, target_field) in source_schema + .fields() + .iter() + .zip(target_schema.fields().iter()) + { + if source_field.data_type() != target_field.data_type() + || source_field.is_nullable() != target_field.is_nullable() + { + return SchemaComparison::Different; + } + + if source_field.name() != target_field.name() { + names_changed = true; + } + } + + if names_changed { + SchemaComparison::NameChangesOnly + } else { + SchemaComparison::Equivalent + } + } + + fn generate_transform_operations( + source_schema: &ArrowSchemaRef, + snapshot_schema: &IcebergSchema, + projected_iceberg_field_ids: &[i32], + field_id_to_mapped_schema_map: HashMap, + ) -> Result> { + let field_id_to_source_schema_map = + Self::build_field_id_to_arrow_schema_map(source_schema)?; + + projected_iceberg_field_ids.iter().map(|field_id|{ + let (target_field, _) = field_id_to_mapped_schema_map.get(field_id).ok_or( + Error::new(ErrorKind::Unexpected, "could not find field in schema") + )?; + let target_type = target_field.data_type(); + + Ok(if let Some((source_field, source_index)) = field_id_to_source_schema_map.get(field_id) { + // column present in source + + if source_field.data_type().equals_datatype(target_type) { + // no promotion required + ColumnSource::PassThrough { + source_index: *source_index + } + } else { + // promotion required + ColumnSource::Promote { + target_type: target_type.clone(), + source_index: *source_index, + } + } + } else { + // column must be added + let iceberg_field = snapshot_schema.field_by_id(*field_id).ok_or( + Error::new(ErrorKind::Unexpected, "Field not found in snapshot schema") + )?; + + let default_value = if let Some(ref iceberg_default_value) = + &iceberg_field.initial_default + { + let Literal::Primitive(primitive_literal) = iceberg_default_value else { + return Err(Error::new( + ErrorKind::Unexpected, + format!("Default value for column must be primitive type, but encountered {:?}", iceberg_default_value) + )); + }; + Some(primitive_literal.clone()) + } else { + None + }; + + ColumnSource::Add { + value: default_value, + target_type: target_type.clone(), + } + }) + }).collect() + } + + fn build_field_id_to_arrow_schema_map( + source_schema: &SchemaRef, + ) -> Result> { + let mut field_id_to_source_schema = HashMap::new(); + for (source_field_idx, source_field) in source_schema.fields.iter().enumerate() { + let this_field_id = source_field + .metadata() + .get(PARQUET_FIELD_ID_META_KEY) + .ok_or_else(|| { + Error::new( + ErrorKind::DataInvalid, + "field ID not present in parquet metadata", + ) + })? + .parse() + .map_err(|e| { + Error::new( + ErrorKind::DataInvalid, + format!("field id not parseable as an i32: {}", e), + ) + })?; + + field_id_to_source_schema + .insert(this_field_id, (source_field.clone(), source_field_idx)); + } + + Ok(field_id_to_source_schema) + } + + fn transform_columns( + &self, + columns: &[Arc], + operations: &[ColumnSource], + ) -> Result>> { + if columns.is_empty() { + return Ok(columns.to_vec()); + } + let num_rows = columns[0].len(); + + operations + .iter() + .map(|op| { + Ok(match op { + ColumnSource::PassThrough { source_index } => columns[*source_index].clone(), + + ColumnSource::Promote { + target_type, + source_index, + } => cast(&*columns[*source_index], target_type)?, + + ColumnSource::Add { target_type, value } => { + Self::create_column(target_type, value, num_rows)? + } + }) + }) + .collect() + } + + fn create_column( + target_type: &DataType, + prim_lit: &Option, + num_rows: usize, + ) -> Result { + Ok(match (target_type, prim_lit) { + (DataType::Boolean, Some(PrimitiveLiteral::Boolean(value))) => { + Arc::new(BooleanArray::from(vec![*value; num_rows])) + } + (DataType::Boolean, None) => { + let vals: Vec> = vec![None; num_rows]; + Arc::new(BooleanArray::from(vals)) + } + (DataType::Int32, Some(PrimitiveLiteral::Int(value))) => { + Arc::new(Int32Array::from(vec![*value; num_rows])) + } + (DataType::Int32, None) => { + let vals: Vec> = vec![None; num_rows]; + Arc::new(Int32Array::from(vals)) + } + (DataType::Int64, Some(PrimitiveLiteral::Long(value))) => { + Arc::new(Int64Array::from(vec![*value; num_rows])) + } + (DataType::Int64, None) => { + let vals: Vec> = vec![None; num_rows]; + Arc::new(Int64Array::from(vals)) + } + (DataType::Float32, Some(PrimitiveLiteral::Float(value))) => { + Arc::new(Float32Array::from(vec![value.0; num_rows])) + } + (DataType::Float32, None) => { + let vals: Vec> = vec![None; num_rows]; + Arc::new(Float32Array::from(vals)) + } + (DataType::Float64, Some(PrimitiveLiteral::Double(value))) => { + Arc::new(Float64Array::from(vec![value.0; num_rows])) + } + (DataType::Float64, None) => { + let vals: Vec> = vec![None; num_rows]; + Arc::new(Float64Array::from(vals)) + } + (DataType::Utf8, Some(PrimitiveLiteral::String(value))) => { + Arc::new(StringArray::from(vec![value.clone(); num_rows])) + } + (DataType::Utf8, None) => { + let vals: Vec> = vec![None; num_rows]; + Arc::new(StringArray::from(vals)) + } + (DataType::Binary, Some(PrimitiveLiteral::Binary(value))) => { + Arc::new(BinaryArray::from_vec(vec![value; num_rows])) + } + (DataType::Binary, None) => { + let vals: Vec> = vec![None; num_rows]; + Arc::new(BinaryArray::from_opt_vec(vals)) + } + (DataType::Null, _) => Arc::new(NullArray::new(num_rows)), + (dt, _) => { + return Err(Error::new( + ErrorKind::Unexpected, + format!("unexpected target column type {}", dt), + )) + } + }) + } +} + +#[cfg(test)] +mod test { + use std::collections::HashMap; + use std::sync::Arc; + + use arrow_array::{ + Float32Array, Float64Array, Int32Array, Int64Array, RecordBatch, StringArray, + }; + use arrow_schema::{DataType, Field, Schema as ArrowSchema}; + use parquet::arrow::PARQUET_FIELD_ID_META_KEY; + + use crate::arrow::record_batch_transformer::RecordBatchTransformer; + use crate::spec::{Literal, NestedField, PrimitiveType, Schema, Type}; + + #[test] + fn build_field_id_to_source_schema_map_works() { + let arrow_schema = arrow_schema_already_same_as_target(); + + let result = + RecordBatchTransformer::build_field_id_to_arrow_schema_map(&arrow_schema).unwrap(); + + let expected = HashMap::from_iter([ + (10, (arrow_schema.fields()[0].clone(), 0)), + (11, (arrow_schema.fields()[1].clone(), 1)), + (12, (arrow_schema.fields()[2].clone(), 2)), + (14, (arrow_schema.fields()[3].clone(), 3)), + (15, (arrow_schema.fields()[4].clone(), 4)), + ]); + + assert!(result.eq(&expected)); + } + + #[test] + fn processor_returns_properly_shaped_record_batch_when_no_schema_migration_required() { + let snapshot_schema = Arc::new(iceberg_table_schema()); + let projected_iceberg_field_ids = [13, 14]; + + let mut inst = RecordBatchTransformer::build(snapshot_schema, &projected_iceberg_field_ids); + + let result = inst + .process_record_batch(source_record_batch_no_migration_required()) + .unwrap(); + + let expected = source_record_batch_no_migration_required(); + + assert_eq!(result, expected); + } + + #[test] + fn processor_returns_properly_shaped_record_batch_when_schema_migration_required() { + let snapshot_schema = Arc::new(iceberg_table_schema()); + let projected_iceberg_field_ids = [10, 11, 12, 14, 15]; // a, b, c, e, f + + let mut inst = RecordBatchTransformer::build(snapshot_schema, &projected_iceberg_field_ids); + + let result = inst.process_record_batch(source_record_batch()).unwrap(); + + let expected = expected_record_batch_migration_required(); + + assert_eq!(result, expected); + } + + pub fn source_record_batch() -> RecordBatch { + RecordBatch::try_new( + arrow_schema_promotion_addition_and_renaming_required(), + vec![ + Arc::new(Int32Array::from(vec![Some(1001), Some(1002), Some(1003)])), // b + Arc::new(Float32Array::from(vec![ + Some(12.125), + Some(23.375), + Some(34.875), + ])), // c + Arc::new(Int32Array::from(vec![Some(2001), Some(2002), Some(2003)])), // d + Arc::new(StringArray::from(vec![ + Some("Apache"), + Some("Iceberg"), + Some("Rocks"), + ])), // e + ], + ) + .unwrap() + } + + pub fn source_record_batch_no_migration_required() -> RecordBatch { + RecordBatch::try_new( + arrow_schema_no_promotion_addition_or_renaming_required(), + vec![ + Arc::new(Int32Array::from(vec![Some(2001), Some(2002), Some(2003)])), // d + Arc::new(StringArray::from(vec![ + Some("Apache"), + Some("Iceberg"), + Some("Rocks"), + ])), // e + ], + ) + .unwrap() + } + + pub fn expected_record_batch_migration_required() -> RecordBatch { + RecordBatch::try_new(arrow_schema_already_same_as_target(), vec![ + Arc::new(StringArray::from(Vec::>::from([ + None, None, None, + ]))), // a + Arc::new(Int64Array::from(vec![Some(1001), Some(1002), Some(1003)])), // b + Arc::new(Float64Array::from(vec![ + Some(12.125), + Some(23.375), + Some(34.875), + ])), // c + Arc::new(StringArray::from(vec![ + Some("Apache"), + Some("Iceberg"), + Some("Rocks"), + ])), // e (d skipped by projection) + Arc::new(StringArray::from(vec![ + Some("(╯°□°)╯"), + Some("(╯°□°)╯"), + Some("(╯°□°)╯"), + ])), // f + ]) + .unwrap() + } + + pub fn iceberg_table_schema() -> Schema { + Schema::builder() + .with_schema_id(2) + .with_fields(vec![ + NestedField::optional(10, "a", Type::Primitive(PrimitiveType::String)).into(), + NestedField::required(11, "b", Type::Primitive(PrimitiveType::Long)).into(), + NestedField::required(12, "c", Type::Primitive(PrimitiveType::Double)).into(), + NestedField::required(13, "d", Type::Primitive(PrimitiveType::Int)).into(), + NestedField::optional(14, "e", Type::Primitive(PrimitiveType::String)).into(), + NestedField::required(15, "f", Type::Primitive(PrimitiveType::String)) + .with_initial_default(Literal::string("(╯°□°)╯")) + .into(), + ]) + .build() + .unwrap() + } + + fn arrow_schema_already_same_as_target() -> Arc { + Arc::new(ArrowSchema::new(vec![ + simple_field("a", DataType::Utf8, true, "10"), + simple_field("b", DataType::Int64, false, "11"), + simple_field("c", DataType::Float64, false, "12"), + simple_field("e", DataType::Utf8, true, "14"), + simple_field("f", DataType::Utf8, false, "15"), + ])) + } + + fn arrow_schema_promotion_addition_and_renaming_required() -> Arc { + Arc::new(ArrowSchema::new(vec![ + simple_field("b", DataType::Int32, false, "11"), + simple_field("c", DataType::Float32, false, "12"), + simple_field("d", DataType::Int32, false, "13"), + simple_field("e_old", DataType::Utf8, true, "14"), + ])) + } + + fn arrow_schema_no_promotion_addition_or_renaming_required() -> Arc { + Arc::new(ArrowSchema::new(vec![ + simple_field("d", DataType::Int32, false, "13"), + simple_field("e", DataType::Utf8, true, "14"), + ])) + } + + /// Create a simple arrow field with metadata. + fn simple_field(name: &str, ty: DataType, nullable: bool, value: &str) -> Field { + Field::new(name, ty, nullable).with_metadata(HashMap::from([( + PARQUET_FIELD_ID_META_KEY.to_string(), + value.to_string(), + )])) + } +} diff --git a/crates/iceberg/src/arrow/schema.rs b/crates/iceberg/src/arrow/schema.rs index e73b409c6..ab30bed8b 100644 --- a/crates/iceberg/src/arrow/schema.rs +++ b/crates/iceberg/src/arrow/schema.rs @@ -207,7 +207,7 @@ fn visit_schema(schema: &ArrowSchema, visitor: &mut V) -> visitor.schema(schema, results) } -/// Convert Arrow schema to ceberg schema. +/// Convert Arrow schema to Iceberg schema. pub fn arrow_schema_to_schema(schema: &ArrowSchema) -> Result { let mut visitor = ArrowSchemaConverter::new(); visit_schema(schema, &mut visitor) diff --git a/crates/iceberg/src/error.rs b/crates/iceberg/src/error.rs index 2b69b4706..3f50acac2 100644 --- a/crates/iceberg/src/error.rs +++ b/crates/iceberg/src/error.rs @@ -337,6 +337,12 @@ define_from_err!( "Failed to send a message to a channel" ); +define_from_err!( + arrow_schema::ArrowError, + ErrorKind::Unexpected, + "Arrow Schema Error" +); + define_from_err!(std::io::Error, ErrorKind::Unexpected, "IO Operation failed"); /// Converts a timestamp in milliseconds to `DateTime`, handling errors. diff --git a/crates/iceberg/src/lib.rs b/crates/iceberg/src/lib.rs index d6c5010d3..72cf18d4b 100644 --- a/crates/iceberg/src/lib.rs +++ b/crates/iceberg/src/lib.rs @@ -55,6 +55,7 @@ #[macro_use] extern crate derive_builder; +extern crate core; mod error; pub use error::{Error, ErrorKind, Result}; diff --git a/crates/iceberg/src/scan.rs b/crates/iceberg/src/scan.rs index f5cbbcf06..ef0e5f542 100644 --- a/crates/iceberg/src/scan.rs +++ b/crates/iceberg/src/scan.rs @@ -906,6 +906,33 @@ pub struct FileScanTask { pub predicate: Option, } +impl FileScanTask { + /// Returns the data file path of this file scan task. + pub fn data_file_path(&self) -> &str { + &self.data_file_path + } + + /// Returns the project field id of this file scan task. + pub fn project_field_ids(&self) -> &[i32] { + &self.project_field_ids + } + + /// Returns the predicate of this file scan task. + pub fn predicate(&self) -> Option<&BoundPredicate> { + self.predicate.as_ref() + } + + /// Returns the schema of this file scan task as a reference + pub fn schema(&self) -> &Schema { + &self.schema + } + + /// Returns the schema of this file scan task as a SchemaRef + pub fn schema_ref(&self) -> SchemaRef { + self.schema.clone() + } +} + #[cfg(test)] mod tests { use std::collections::HashMap; From 35dac589e48af018ae72b56e820a32b676764b18 Mon Sep 17 00:00:00 2001 From: Nishant Sachdeva <32475507+nishant-sachdeva@users.noreply.github.com> Date: Sat, 12 Oct 2024 11:24:10 +0530 Subject: [PATCH 13/46] docs: installation of the new `iceberg_catalog_rest` added to the docs (#355) --- website/src/install.md | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/website/src/install.md b/website/src/install.md index 38b76a972..bcda751a3 100644 --- a/website/src/install.md +++ b/website/src/install.md @@ -23,10 +23,11 @@ Cargo 1.75.0 or later is required to build. -Add `iceberg` into `Cargo.toml` dependencies: +Add `iceberg` and `iceberg-catalog-rest` into `Cargo.toml` dependencies: ```toml iceberg = "0.2.0" +iceberg-catalog-rest = "0.2.0" ``` iceberg is under active development, you may want to use the git version instead: From 026b436ff457b05da1fa05ea60e69260f62bb90c Mon Sep 17 00:00:00 2001 From: FANNG Date: Sat, 12 Oct 2024 13:56:35 +0800 Subject: [PATCH 14/46] feat(datafusion): Support pushdown more datafusion exprs to Iceberg (#649) * suppport more datafusion predict * add it * reverse op --- crates/iceberg/src/expr/predicate.rs | 27 +- .../src/physical_plan/expr_to_predicate.rs | 454 ++++++++++-------- .../datafusion/src/physical_plan/scan.rs | 26 +- crates/integrations/datafusion/src/table.rs | 13 +- .../tests/integration_datafusion_test.rs | 54 ++- 5 files changed, 340 insertions(+), 234 deletions(-) diff --git a/crates/iceberg/src/expr/predicate.rs b/crates/iceberg/src/expr/predicate.rs index acf21a5b1..76befb6d8 100644 --- a/crates/iceberg/src/expr/predicate.rs +++ b/crates/iceberg/src/expr/predicate.rs @@ -132,7 +132,16 @@ impl Bind for UnaryExpression { } impl UnaryExpression { - pub(crate) fn new(op: PredicateOperator, term: T) -> Self { + /// Creates a unary expression with the given operator and term. + /// + /// # Example + /// + /// ```rust + /// use iceberg::expr::{PredicateOperator, Reference, UnaryExpression}; + /// + /// UnaryExpression::new(PredicateOperator::IsNull, Reference::new("c")); + /// ``` + pub fn new(op: PredicateOperator, term: T) -> Self { debug_assert!(op.is_unary()); Self { op, term } } @@ -171,7 +180,21 @@ impl Debug for BinaryExpression { } impl BinaryExpression { - pub(crate) fn new(op: PredicateOperator, term: T, literal: Datum) -> Self { + /// Creates a binary expression with the given operator, term and literal. + /// + /// # Example + /// + /// ```rust + /// use iceberg::expr::{BinaryExpression, PredicateOperator, Reference}; + /// use iceberg::spec::Datum; + /// + /// BinaryExpression::new( + /// PredicateOperator::LessThanOrEq, + /// Reference::new("a"), + /// Datum::int(10), + /// ); + /// ``` + pub fn new(op: PredicateOperator, term: T, literal: Datum) -> Self { debug_assert!(op.is_binary()); Self { op, term, literal } } 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 110e4f7e4..f438308e6 100644 --- a/crates/integrations/datafusion/src/physical_plan/expr_to_predicate.rs +++ b/crates/integrations/datafusion/src/physical_plan/expr_to_predicate.rs @@ -15,103 +15,179 @@ // specific language governing permissions and limitations // under the License. -use std::collections::VecDeque; +use std::vec; -use datafusion::common::tree_node::{TreeNodeRecursion, TreeNodeVisitor}; -use datafusion::common::Column; -use datafusion::error::DataFusionError; use datafusion::logical_expr::{Expr, Operator}; use datafusion::scalar::ScalarValue; -use iceberg::expr::{Predicate, Reference}; +use iceberg::expr::{BinaryExpression, Predicate, PredicateOperator, Reference, UnaryExpression}; use iceberg::spec::Datum; -pub struct ExprToPredicateVisitor { - stack: VecDeque>, +// A datafusion expression could be an Iceberg predicate, column, or literal. +enum TransformedResult { + Predicate(Predicate), + Column(Reference), + Literal(Datum), + NotTransformed, } -impl ExprToPredicateVisitor { - /// Create a new predicate conversion visitor. - pub fn new() -> Self { - Self { - stack: VecDeque::new(), + +enum OpTransformedResult { + Operator(PredicateOperator), + And, + Or, + NotTransformed, +} + +/// Converts DataFusion filters ([`Expr`]) to an iceberg [`Predicate`]. +/// If none of the filters could be converted, return `None` which adds no predicates to the scan operation. +/// If the conversion was successful, return the converted predicates combined with an AND operator. +pub fn convert_filters_to_predicate(filters: &[Expr]) -> Option { + filters + .iter() + .filter_map(convert_filter_to_predicate) + .reduce(Predicate::and) +} + +fn convert_filter_to_predicate(expr: &Expr) -> Option { + match to_iceberg_predicate(expr) { + TransformedResult::Predicate(predicate) => Some(predicate), + TransformedResult::Column(_) | TransformedResult::Literal(_) => { + unreachable!("Not a valid expression: {:?}", expr) } + _ => None, } - /// Get the predicate from the stack. - pub fn get_predicate(&self) -> Option { - self.stack - .iter() - .filter_map(|opt| opt.clone()) - .reduce(Predicate::and) +} + +fn to_iceberg_predicate(expr: &Expr) -> TransformedResult { + match expr { + Expr::BinaryExpr(binary) => { + let left = to_iceberg_predicate(&binary.left); + let right = to_iceberg_predicate(&binary.right); + let op = to_iceberg_operation(binary.op); + match op { + OpTransformedResult::Operator(op) => to_iceberg_binary_predicate(left, right, op), + OpTransformedResult::And => to_iceberg_and_predicate(left, right), + OpTransformedResult::Or => to_iceberg_or_predicate(left, right), + OpTransformedResult::NotTransformed => TransformedResult::NotTransformed, + } + } + Expr::Not(exp) => { + let expr = to_iceberg_predicate(exp); + match expr { + TransformedResult::Predicate(p) => TransformedResult::Predicate(!p), + _ => TransformedResult::NotTransformed, + } + } + Expr::Column(column) => TransformedResult::Column(Reference::new(column.name())), + Expr::Literal(literal) => match scalar_value_to_datum(literal) { + Some(data) => TransformedResult::Literal(data), + None => TransformedResult::NotTransformed, + }, + Expr::InList(inlist) => { + let mut datums = vec![]; + for expr in &inlist.list { + let p = to_iceberg_predicate(expr); + match p { + TransformedResult::Literal(l) => datums.push(l), + _ => return TransformedResult::NotTransformed, + } + } + + let expr = to_iceberg_predicate(&inlist.expr); + match expr { + TransformedResult::Column(r) => match inlist.negated { + false => TransformedResult::Predicate(r.is_in(datums)), + true => TransformedResult::Predicate(r.is_not_in(datums)), + }, + _ => TransformedResult::NotTransformed, + } + } + Expr::IsNull(expr) => { + let p = to_iceberg_predicate(expr); + match p { + TransformedResult::Column(r) => TransformedResult::Predicate(Predicate::Unary( + UnaryExpression::new(PredicateOperator::IsNull, r), + )), + _ => TransformedResult::NotTransformed, + } + } + Expr::IsNotNull(expr) => { + let p = to_iceberg_predicate(expr); + match p { + TransformedResult::Column(r) => TransformedResult::Predicate(Predicate::Unary( + UnaryExpression::new(PredicateOperator::NotNull, r), + )), + _ => TransformedResult::NotTransformed, + } + } + _ => TransformedResult::NotTransformed, } +} - /// Convert a column expression to an iceberg predicate. - fn convert_column_expr( - &self, - col: &Column, - op: &Operator, - lit: &ScalarValue, - ) -> Option { - let reference = Reference::new(col.name.clone()); - let datum = scalar_value_to_datum(lit)?; - Some(binary_op_to_predicate(reference, op, datum)) +fn to_iceberg_operation(op: Operator) -> OpTransformedResult { + match op { + Operator::Eq => OpTransformedResult::Operator(PredicateOperator::Eq), + Operator::NotEq => OpTransformedResult::Operator(PredicateOperator::NotEq), + Operator::Lt => OpTransformedResult::Operator(PredicateOperator::LessThan), + Operator::LtEq => OpTransformedResult::Operator(PredicateOperator::LessThanOrEq), + Operator::Gt => OpTransformedResult::Operator(PredicateOperator::GreaterThan), + Operator::GtEq => OpTransformedResult::Operator(PredicateOperator::GreaterThanOrEq), + // AND OR + Operator::And => OpTransformedResult::And, + Operator::Or => OpTransformedResult::Or, + // Others not supported + _ => OpTransformedResult::NotTransformed, } +} - /// Convert a compound expression to an iceberg predicate. - /// - /// The strategy is to support the following cases: - /// - if its an AND expression then the result will be the valid predicates, whether there are 2 or just 1 - /// - if its an OR expression then a predicate will be returned only if there are 2 valid predicates on both sides - fn convert_compound_expr(&self, valid_preds: &[Predicate], op: &Operator) -> Option { - let valid_preds_count = valid_preds.len(); - match (op, valid_preds_count) { - (Operator::And, 1) => valid_preds.first().cloned(), - (Operator::And, 2) => Some(Predicate::and( - valid_preds[0].clone(), - valid_preds[1].clone(), - )), - (Operator::Or, 2) => Some(Predicate::or( - valid_preds[0].clone(), - valid_preds[1].clone(), - )), - _ => None, +fn to_iceberg_and_predicate( + left: TransformedResult, + right: TransformedResult, +) -> TransformedResult { + match (left, right) { + (TransformedResult::Predicate(left), TransformedResult::Predicate(right)) => { + TransformedResult::Predicate(left.and(right)) } + (TransformedResult::Predicate(left), _) => TransformedResult::Predicate(left), + (_, TransformedResult::Predicate(right)) => TransformedResult::Predicate(right), + _ => TransformedResult::NotTransformed, } } -// Implement TreeNodeVisitor for ExprToPredicateVisitor -impl<'n> TreeNodeVisitor<'n> for ExprToPredicateVisitor { - type Node = Expr; - - fn f_down(&mut self, _node: &'n Expr) -> Result { - Ok(TreeNodeRecursion::Continue) +fn to_iceberg_or_predicate(left: TransformedResult, right: TransformedResult) -> TransformedResult { + match (left, right) { + (TransformedResult::Predicate(left), TransformedResult::Predicate(right)) => { + TransformedResult::Predicate(left.or(right)) + } + _ => TransformedResult::NotTransformed, } +} - fn f_up(&mut self, expr: &'n Expr) -> Result { - if let Expr::BinaryExpr(binary) = expr { - match (&*binary.left, &binary.op, &*binary.right) { - // process simple binary expressions, e.g. col > 1 - (Expr::Column(col), op, Expr::Literal(lit)) => { - let col_pred = self.convert_column_expr(col, op, lit); - self.stack.push_back(col_pred); - } - // // process reversed binary expressions, e.g. 1 < col - (Expr::Literal(lit), op, Expr::Column(col)) => { - let col_pred = op - .swap() - .and_then(|negated_op| self.convert_column_expr(col, &negated_op, lit)); - self.stack.push_back(col_pred); - } - // process compound expressions (involving logical operators. e.g., AND or OR and children) - (_left, op, _right) if op.is_logic_operator() => { - let right_pred = self.stack.pop_back().flatten(); - let left_pred = self.stack.pop_back().flatten(); - let children: Vec<_> = [left_pred, right_pred].into_iter().flatten().collect(); - let compound_pred = self.convert_compound_expr(&children, op); - self.stack.push_back(compound_pred); - } - _ => return Ok(TreeNodeRecursion::Continue), - } +fn to_iceberg_binary_predicate( + left: TransformedResult, + right: TransformedResult, + op: PredicateOperator, +) -> TransformedResult { + let (r, d, op) = match (left, right) { + (TransformedResult::NotTransformed, _) => return TransformedResult::NotTransformed, + (_, TransformedResult::NotTransformed) => return TransformedResult::NotTransformed, + (TransformedResult::Column(r), TransformedResult::Literal(d)) => (r, d, op), + (TransformedResult::Literal(d), TransformedResult::Column(r)) => { + (r, d, reverse_predicate_operator(op)) } - Ok(TreeNodeRecursion::Continue) + _ => return TransformedResult::NotTransformed, + }; + TransformedResult::Predicate(Predicate::Binary(BinaryExpression::new(op, r, d))) +} + +fn reverse_predicate_operator(op: PredicateOperator) -> PredicateOperator { + match op { + PredicateOperator::Eq => PredicateOperator::Eq, + PredicateOperator::NotEq => PredicateOperator::NotEq, + PredicateOperator::GreaterThan => PredicateOperator::LessThan, + PredicateOperator::GreaterThanOrEq => PredicateOperator::LessThanOrEq, + PredicateOperator::LessThan => PredicateOperator::GreaterThan, + PredicateOperator::LessThanOrEq => PredicateOperator::GreaterThanOrEq, + _ => unreachable!("Reverse {}", op), } } @@ -133,93 +209,113 @@ fn scalar_value_to_datum(value: &ScalarValue) -> Option { } } -/// convert the data fusion Exp to an iceberg [`Predicate`] -fn binary_op_to_predicate(reference: Reference, op: &Operator, datum: Datum) -> Predicate { - match op { - Operator::Eq => reference.equal_to(datum), - Operator::NotEq => reference.not_equal_to(datum), - Operator::Lt => reference.less_than(datum), - Operator::LtEq => reference.less_than_or_equal_to(datum), - Operator::Gt => reference.greater_than(datum), - Operator::GtEq => reference.greater_than_or_equal_to(datum), - _ => Predicate::AlwaysTrue, - } -} - #[cfg(test)] mod tests { - use std::collections::VecDeque; - use datafusion::arrow::datatypes::{DataType, Field, Schema}; - use datafusion::common::tree_node::TreeNode; use datafusion::common::DFSchema; - use datafusion::prelude::SessionContext; + use datafusion::logical_expr::utils::split_conjunction; + use datafusion::prelude::{Expr, SessionContext}; use iceberg::expr::{Predicate, Reference}; use iceberg::spec::Datum; - use super::ExprToPredicateVisitor; + use super::convert_filters_to_predicate; fn create_test_schema() -> DFSchema { let arrow_schema = Schema::new(vec![ - Field::new("foo", DataType::Int32, false), - Field::new("bar", DataType::Utf8, false), + Field::new("foo", DataType::Int32, true), + Field::new("bar", DataType::Utf8, true), ]); DFSchema::try_from_qualified_schema("my_table", &arrow_schema).unwrap() } - #[test] - fn test_predicate_conversion_with_single_condition() { - let sql = "foo > 1"; + fn convert_to_iceberg_predicate(sql: &str) -> Option { let df_schema = create_test_schema(); let expr = SessionContext::new() .parse_sql_expr(sql, &df_schema) .unwrap(); - let mut visitor = ExprToPredicateVisitor::new(); - expr.visit(&mut visitor).unwrap(); - let predicate = visitor.get_predicate().unwrap(); + let exprs: Vec = split_conjunction(&expr).into_iter().cloned().collect(); + convert_filters_to_predicate(&exprs[..]) + } + + #[test] + fn test_predicate_conversion_with_single_condition() { + let predicate = convert_to_iceberg_predicate("foo = 1").unwrap(); + assert_eq!(predicate, Reference::new("foo").equal_to(Datum::long(1))); + + let predicate = convert_to_iceberg_predicate("foo != 1").unwrap(); + assert_eq!( + predicate, + Reference::new("foo").not_equal_to(Datum::long(1)) + ); + + let predicate = convert_to_iceberg_predicate("foo > 1").unwrap(); assert_eq!( predicate, Reference::new("foo").greater_than(Datum::long(1)) ); + + let predicate = convert_to_iceberg_predicate("foo >= 1").unwrap(); + assert_eq!( + predicate, + Reference::new("foo").greater_than_or_equal_to(Datum::long(1)) + ); + + let predicate = convert_to_iceberg_predicate("foo < 1").unwrap(); + assert_eq!(predicate, Reference::new("foo").less_than(Datum::long(1))); + + let predicate = convert_to_iceberg_predicate("foo <= 1").unwrap(); + assert_eq!( + predicate, + Reference::new("foo").less_than_or_equal_to(Datum::long(1)) + ); + + let predicate = convert_to_iceberg_predicate("foo is null").unwrap(); + assert_eq!(predicate, Reference::new("foo").is_null()); + + let predicate = convert_to_iceberg_predicate("foo is not null").unwrap(); + assert_eq!(predicate, Reference::new("foo").is_not_null()); + + let predicate = convert_to_iceberg_predicate("foo in (5, 6)").unwrap(); + assert_eq!( + predicate, + Reference::new("foo").is_in([Datum::long(5), Datum::long(6)]) + ); + + let predicate = convert_to_iceberg_predicate("foo not in (5, 6)").unwrap(); + assert_eq!( + predicate, + Reference::new("foo").is_not_in([Datum::long(5), Datum::long(6)]) + ); + + let predicate = convert_to_iceberg_predicate("not foo = 1").unwrap(); + assert_eq!(predicate, !Reference::new("foo").equal_to(Datum::long(1))); } + #[test] fn test_predicate_conversion_with_single_unsupported_condition() { - let sql = "foo is null"; - let df_schema = create_test_schema(); - let expr = SessionContext::new() - .parse_sql_expr(sql, &df_schema) - .unwrap(); - let mut visitor = ExprToPredicateVisitor::new(); - expr.visit(&mut visitor).unwrap(); - let predicate = visitor.get_predicate(); + let predicate = convert_to_iceberg_predicate("foo + 1 = 1"); + assert_eq!(predicate, None); + + let predicate = convert_to_iceberg_predicate("length(bar) = 1"); + assert_eq!(predicate, None); + + let predicate = convert_to_iceberg_predicate("foo in (1, 2, foo)"); assert_eq!(predicate, None); } #[test] fn test_predicate_conversion_with_single_condition_rev() { - let sql = "1 < foo"; - let df_schema = create_test_schema(); - let expr = SessionContext::new() - .parse_sql_expr(sql, &df_schema) - .unwrap(); - let mut visitor = ExprToPredicateVisitor::new(); - expr.visit(&mut visitor).unwrap(); - let predicate = visitor.get_predicate().unwrap(); + let predicate = convert_to_iceberg_predicate("1 < foo").unwrap(); assert_eq!( predicate, Reference::new("foo").greater_than(Datum::long(1)) ); } + #[test] fn test_predicate_conversion_with_and_condition() { let sql = "foo > 1 and bar = 'test'"; - let df_schema = create_test_schema(); - let expr = SessionContext::new() - .parse_sql_expr(sql, &df_schema) - .unwrap(); - let mut visitor = ExprToPredicateVisitor::new(); - expr.visit(&mut visitor).unwrap(); - let predicate = visitor.get_predicate().unwrap(); + let predicate = convert_to_iceberg_predicate(sql).unwrap(); let expected_predicate = Predicate::and( Reference::new("foo").greater_than(Datum::long(1)), Reference::new("bar").equal_to(Datum::string("test")), @@ -229,55 +325,42 @@ mod tests { #[test] fn test_predicate_conversion_with_and_condition_unsupported() { - let sql = "foo > 1 and bar is not null"; - let df_schema = create_test_schema(); - let expr = SessionContext::new() - .parse_sql_expr(sql, &df_schema) - .unwrap(); - let mut visitor = ExprToPredicateVisitor::new(); - expr.visit(&mut visitor).unwrap(); - let predicate = visitor.get_predicate().unwrap(); + let sql = "foo > 1 and length(bar) = 1"; + let predicate = convert_to_iceberg_predicate(sql).unwrap(); let expected_predicate = Reference::new("foo").greater_than(Datum::long(1)); assert_eq!(predicate, expected_predicate); } + #[test] fn test_predicate_conversion_with_and_condition_both_unsupported() { - let sql = "foo in (1, 2, 3) and bar is not null"; - let df_schema = create_test_schema(); - let expr = SessionContext::new() - .parse_sql_expr(sql, &df_schema) - .unwrap(); - let mut visitor = ExprToPredicateVisitor::new(); - expr.visit(&mut visitor).unwrap(); - let predicate = visitor.get_predicate(); - let expected_predicate = None; - assert_eq!(predicate, expected_predicate); + let sql = "foo in (1, 2, foo) and length(bar) = 1"; + let predicate = convert_to_iceberg_predicate(sql); + assert_eq!(predicate, None); } #[test] fn test_predicate_conversion_with_or_condition_unsupported() { - let sql = "foo > 1 or bar is not null"; - let df_schema = create_test_schema(); - let expr = SessionContext::new() - .parse_sql_expr(sql, &df_schema) - .unwrap(); - let mut visitor = ExprToPredicateVisitor::new(); - expr.visit(&mut visitor).unwrap(); - let predicate = visitor.get_predicate(); - let expected_predicate = None; + let sql = "foo > 1 or length(bar) = 1"; + let predicate = convert_to_iceberg_predicate(sql); + assert_eq!(predicate, None); + } + + #[test] + fn test_predicate_conversion_with_or_condition_supported() { + let sql = "foo > 1 or bar = 'test'"; + let predicate = convert_to_iceberg_predicate(sql).unwrap(); + let expected_predicate = Predicate::or( + Reference::new("foo").greater_than(Datum::long(1)), + Reference::new("bar").equal_to(Datum::string("test")), + ); assert_eq!(predicate, expected_predicate); } #[test] fn test_predicate_conversion_with_complex_binary_expr() { let sql = "(foo > 1 and bar = 'test') or foo < 0 "; - let df_schema = create_test_schema(); - let expr = SessionContext::new() - .parse_sql_expr(sql, &df_schema) - .unwrap(); - let mut visitor = ExprToPredicateVisitor::new(); - expr.visit(&mut visitor).unwrap(); - let predicate = visitor.get_predicate().unwrap(); + let predicate = convert_to_iceberg_predicate(sql).unwrap(); + let inner_predicate = Predicate::and( Reference::new("foo").greater_than(Datum::long(1)), Reference::new("bar").equal_to(Datum::string("test")), @@ -290,46 +373,23 @@ mod tests { } #[test] - fn test_predicate_conversion_with_complex_binary_expr_unsupported() { - let sql = "(foo > 1 or bar in ('test', 'test2')) and foo < 0 "; - let df_schema = create_test_schema(); - let expr = SessionContext::new() - .parse_sql_expr(sql, &df_schema) - .unwrap(); - let mut visitor = ExprToPredicateVisitor::new(); - expr.visit(&mut visitor).unwrap(); - let predicate = visitor.get_predicate().unwrap(); - let expected_predicate = Reference::new("foo").less_than(Datum::long(0)); - assert_eq!(predicate, expected_predicate); - } + fn test_predicate_conversion_with_one_and_expr_supported() { + let sql = "(foo > 1 and length(bar) = 1 ) or foo < 0 "; + let predicate = convert_to_iceberg_predicate(sql).unwrap(); - #[test] - // test the get result method - fn test_get_result_multiple() { - let predicates = vec![ - Some(Reference::new("foo").greater_than(Datum::long(1))), - None, - Some(Reference::new("bar").equal_to(Datum::string("test"))), - ]; - let stack = VecDeque::from(predicates); - let visitor = ExprToPredicateVisitor { stack }; - assert_eq!( - visitor.get_predicate(), - Some(Predicate::and( - Reference::new("foo").greater_than(Datum::long(1)), - Reference::new("bar").equal_to(Datum::string("test")), - )) + let inner_predicate = Reference::new("foo").greater_than(Datum::long(1)); + let expected_predicate = Predicate::or( + inner_predicate, + Reference::new("foo").less_than(Datum::long(0)), ); + assert_eq!(predicate, expected_predicate); } #[test] - fn test_get_result_single() { - let predicates = vec![Some(Reference::new("foo").greater_than(Datum::long(1)))]; - let stack = VecDeque::from(predicates); - let visitor = ExprToPredicateVisitor { stack }; - assert_eq!( - visitor.get_predicate(), - Some(Reference::new("foo").greater_than(Datum::long(1))) - ); + fn test_predicate_conversion_with_complex_binary_expr_unsupported() { + let sql = "(foo > 1 or length(bar) = 1 ) and foo < 0 "; + let predicate = convert_to_iceberg_predicate(sql).unwrap(); + let expected_predicate = Reference::new("foo").less_than(Datum::long(0)); + assert_eq!(predicate, expected_predicate); } } diff --git a/crates/integrations/datafusion/src/physical_plan/scan.rs b/crates/integrations/datafusion/src/physical_plan/scan.rs index c53ce76d5..59cf09976 100644 --- a/crates/integrations/datafusion/src/physical_plan/scan.rs +++ b/crates/integrations/datafusion/src/physical_plan/scan.rs @@ -22,7 +22,6 @@ use std::vec; use datafusion::arrow::array::RecordBatch; use datafusion::arrow::datatypes::SchemaRef as ArrowSchemaRef; -use datafusion::common::tree_node::TreeNode; use datafusion::error::Result as DFResult; use datafusion::execution::{SendableRecordBatchStream, TaskContext}; use datafusion::physical_expr::EquivalenceProperties; @@ -35,7 +34,7 @@ use futures::{Stream, TryStreamExt}; use iceberg::expr::Predicate; use iceberg::table::Table; -use crate::physical_plan::expr_to_predicate::ExprToPredicateVisitor; +use super::expr_to_predicate::convert_filters_to_predicate; use crate::to_datafusion_error; /// Manages the scanning process of an Iceberg [`Table`], encapsulating the @@ -140,10 +139,13 @@ impl DisplayAs for IcebergTableScan { ) -> std::fmt::Result { write!( f, - "IcebergTableScan projection:[{}]", + "IcebergTableScan projection:[{}] predicate:[{}]", self.projection .clone() - .map_or(String::new(), |v| v.join(",")) + .map_or(String::new(), |v| v.join(",")), + self.predicates + .clone() + .map_or(String::from(""), |p| format!("{}", p)) ) } } @@ -175,22 +177,6 @@ async fn get_batch_stream( Ok(Box::pin(stream)) } -/// Converts DataFusion filters ([`Expr`]) to an iceberg [`Predicate`]. -/// If none of the filters could be converted, return `None` which adds no predicates to the scan operation. -/// If the conversion was successful, return the converted predicates combined with an AND operator. -fn convert_filters_to_predicate(filters: &[Expr]) -> Option { - filters - .iter() - .filter_map(|expr| { - let mut visitor = ExprToPredicateVisitor::new(); - if expr.visit(&mut visitor).is_ok() { - visitor.get_predicate() - } else { - None - } - }) - .reduce(Predicate::and) -} fn get_column_names( schema: ArrowSchemaRef, projection: Option<&Vec>, diff --git a/crates/integrations/datafusion/src/table.rs b/crates/integrations/datafusion/src/table.rs index 2797e12d6..bb24713aa 100644 --- a/crates/integrations/datafusion/src/table.rs +++ b/crates/integrations/datafusion/src/table.rs @@ -23,7 +23,7 @@ use datafusion::arrow::datatypes::SchemaRef as ArrowSchemaRef; use datafusion::catalog::Session; use datafusion::datasource::{TableProvider, TableType}; use datafusion::error::Result as DFResult; -use datafusion::logical_expr::{BinaryExpr, Expr, TableProviderFilterPushDown}; +use datafusion::logical_expr::{Expr, TableProviderFilterPushDown}; use datafusion::physical_plan::ExecutionPlan; use iceberg::arrow::schema_to_arrow_schema; use iceberg::table::Table; @@ -99,15 +99,8 @@ impl TableProvider for IcebergTableProvider { filters: &[&Expr], ) -> std::result::Result, datafusion::error::DataFusionError> { - let filter_support = filters - .iter() - .map(|e| match e { - Expr::BinaryExpr(BinaryExpr { .. }) => TableProviderFilterPushDown::Inexact, - _ => TableProviderFilterPushDown::Unsupported, - }) - .collect::>(); - - Ok(filter_support) + // Push down all filters, as a single source of truth, the scanner will drop the filters which couldn't be push down + Ok(vec![TableProviderFilterPushDown::Inexact; filters.len()]) } } diff --git a/crates/integrations/datafusion/tests/integration_datafusion_test.rs b/crates/integrations/datafusion/tests/integration_datafusion_test.rs index d6e22d044..d320c8ef0 100644 --- a/crates/integrations/datafusion/tests/integration_datafusion_test.rs +++ b/crates/integrations/datafusion/tests/integration_datafusion_test.rs @@ -204,10 +204,7 @@ async fn test_table_projection() -> Result<()> { .unwrap(); assert_eq!(2, s.len()); // the first row is logical_plan, the second row is physical_plan - assert_eq!( - "IcebergTableScan projection:[foo1,foo2,foo3]", - s.value(1).trim() - ); + assert!(s.value(1).contains("projection:[foo1,foo2,foo3]")); // datafusion doesn't support query foo3.s_foo1, use foo3 instead let records = table_df @@ -226,7 +223,54 @@ async fn test_table_projection() -> Result<()> { .downcast_ref::() .unwrap(); assert_eq!(2, s.len()); - assert_eq!("IcebergTableScan projection:[foo1,foo3]", s.value(1).trim()); + assert!(s + .value(1) + .contains("IcebergTableScan projection:[foo1,foo3]")); Ok(()) } + +#[tokio::test] +async fn test_table_predict_pushdown() -> Result<()> { + let iceberg_catalog = get_iceberg_catalog(); + let namespace = NamespaceIdent::new("ns".to_string()); + set_test_namespace(&iceberg_catalog, &namespace).await?; + + let schema = Schema::builder() + .with_schema_id(0) + .with_fields(vec![ + NestedField::required(1, "foo", Type::Primitive(PrimitiveType::Int)).into(), + NestedField::optional(2, "bar", Type::Primitive(PrimitiveType::String)).into(), + ]) + .build()?; + let creation = get_table_creation(temp_path(), "t1", Some(schema))?; + iceberg_catalog.create_table(&namespace, creation).await?; + + let client = Arc::new(iceberg_catalog); + let catalog = Arc::new(IcebergCatalogProvider::try_new(client).await?); + + let ctx = SessionContext::new(); + ctx.register_catalog("catalog", catalog); + let records = ctx + .sql("select * from catalog.ns.t1 where (foo > 1 and length(bar) = 1 ) or bar is null") + .await + .unwrap() + .explain(false, false) + .unwrap() + .collect() + .await + .unwrap(); + assert_eq!(1, records.len()); + let record = &records[0]; + // the first column is plan_type, the second column plan string. + let s = record + .column(1) + .as_any() + .downcast_ref::() + .unwrap(); + assert_eq!(2, s.len()); + // the first row is logical_plan, the second row is physical_plan + let expected = "predicate:[(foo > 1) OR (bar IS NULL)]"; + assert!(s.value(1).trim().contains(expected)); + Ok(()) +} From 97fc01dcfa7305f0fb0dd9024489eb02c2e59454 Mon Sep 17 00:00:00 2001 From: Xuanwo Date: Sat, 12 Oct 2024 15:29:34 +0800 Subject: [PATCH 15/46] feat: Derive PartialEq for FileScanTask (#660) Signed-off-by: Xuanwo --- crates/iceberg/src/scan.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/crates/iceberg/src/scan.rs b/crates/iceberg/src/scan.rs index ef0e5f542..b7fa62d66 100644 --- a/crates/iceberg/src/scan.rs +++ b/crates/iceberg/src/scan.rs @@ -878,7 +878,7 @@ impl ExpressionEvaluatorCache { } /// A task to scan part of file. -#[derive(Debug, Clone, Serialize, Deserialize)] +#[derive(Debug, Clone, Serialize, Deserialize, PartialEq)] pub struct FileScanTask { /// The start offset of the file to scan. pub start: u64, From ae6d0958843e6d18be301547de31de1610a72304 Mon Sep 17 00:00:00 2001 From: Callum Ryan <19956159+callum-ryan@users.noreply.github.com> Date: Sat, 12 Oct 2024 10:17:07 +0100 Subject: [PATCH 16/46] feat: SQL Catalog - Tables (#610) * feat: add list/exist table + drop namespace Signed-off-by: callum-ryan <19956159+callum-ryan@users.noreply.github.com> * feat: create table Signed-off-by: callum-ryan <19956159+callum-ryan@users.noreply.github.com> * feat: add load table Signed-off-by: callum-ryan <19956159+callum-ryan@users.noreply.github.com> * feat: add the rest of table ops Signed-off-by: callum-ryan <19956159+callum-ryan@users.noreply.github.com> * fix: sort order on test Signed-off-by: callum-ryan <19956159+callum-ryan@users.noreply.github.com> * fix: sort Cargo.toml Signed-off-by: callum-ryan <19956159+callum-ryan@users.noreply.github.com> * fix: Adjust error message for existence of table Signed-off-by: callum-ryan <19956159+callum-ryan@users.noreply.github.com> * fix: update_table throws Unsupported, add catalog filter to drop_nsp Signed-off-by: callum-ryan <19956159+callum-ryan@users.noreply.github.com> --------- Signed-off-by: callum-ryan <19956159+callum-ryan@users.noreply.github.com> --- crates/catalog/sql/Cargo.toml | 2 + crates/catalog/sql/src/catalog.rs | 836 ++++++++++++++++++++++++++++-- crates/catalog/sql/src/error.rs | 16 +- 3 files changed, 824 insertions(+), 30 deletions(-) diff --git a/crates/catalog/sql/Cargo.toml b/crates/catalog/sql/Cargo.toml index 4a88e75b4..a51671650 100644 --- a/crates/catalog/sql/Cargo.toml +++ b/crates/catalog/sql/Cargo.toml @@ -31,8 +31,10 @@ keywords = ["iceberg", "sql", "catalog"] [dependencies] async-trait = { workspace = true } iceberg = { workspace = true } +serde_json = { workspace = true } sqlx = { version = "0.8.1", features = ["any"], default-features = false } typed-builder = { workspace = true } +uuid = { workspace = true, features = ["v4"] } [dev-dependencies] iceberg_test_utils = { path = "../../test_utils", features = ["tests"] } diff --git a/crates/catalog/sql/src/catalog.rs b/crates/catalog/sql/src/catalog.rs index c6a524cea..b7976d9d5 100644 --- a/crates/catalog/sql/src/catalog.rs +++ b/crates/catalog/sql/src/catalog.rs @@ -20,15 +20,20 @@ use std::time::Duration; use async_trait::async_trait; use iceberg::io::FileIO; +use iceberg::spec::{TableMetadata, TableMetadataBuilder}; use iceberg::table::Table; use iceberg::{ - Catalog, Error, Namespace, NamespaceIdent, Result, TableCommit, TableCreation, TableIdent, + Catalog, Error, ErrorKind, Namespace, NamespaceIdent, Result, TableCommit, TableCreation, + TableIdent, }; use sqlx::any::{install_default_drivers, AnyPoolOptions, AnyQueryResult, AnyRow}; use sqlx::{Any, AnyPool, Row, Transaction}; use typed_builder::TypedBuilder; +use uuid::Uuid; -use crate::error::{from_sqlx_error, no_such_namespace_err}; +use crate::error::{ + from_sqlx_error, no_such_namespace_err, no_such_table_err, table_already_exists_err, +}; static CATALOG_TABLE_NAME: &str = "iceberg_tables"; static CATALOG_FIELD_CATALOG_NAME: &str = "catalog_name"; @@ -37,12 +42,15 @@ static CATALOG_FIELD_TABLE_NAMESPACE: &str = "table_namespace"; static CATALOG_FIELD_METADATA_LOCATION_PROP: &str = "metadata_location"; static CATALOG_FIELD_PREVIOUS_METADATA_LOCATION_PROP: &str = "previous_metadata_location"; static CATALOG_FIELD_RECORD_TYPE: &str = "iceberg_type"; +static CATALOG_FIELD_TABLE_RECORD_TYPE: &str = "TABLE"; static NAMESPACE_TABLE_NAME: &str = "iceberg_namespace_properties"; static NAMESPACE_FIELD_NAME: &str = "namespace"; static NAMESPACE_FIELD_PROPERTY_KEY: &str = "property_key"; static NAMESPACE_FIELD_PROPERTY_VALUE: &str = "property_value"; +static NAMESPACE_LOCATION_PROPERTY_KEY: &str = "location"; + static MAX_CONNECTIONS: u32 = 10; // Default the SQL pool to 10 connections if not provided static IDLE_TIMEOUT: u64 = 10; // Default the maximum idle timeout per connection to 10s before it is closed static TEST_BEFORE_ACQUIRE: bool = true; // Default the health-check of each connection to enabled prior to returning @@ -71,8 +79,8 @@ pub struct SqlCatalogConfig { pub struct SqlCatalog { name: String, connection: AnyPool, - _warehouse_location: String, - _fileio: FileIO, + warehouse_location: String, + fileio: FileIO, sql_bind_style: SqlBindStyle, } @@ -142,8 +150,8 @@ impl SqlCatalog { Ok(SqlCatalog { name: config.name.to_owned(), connection: pool, - _warehouse_location: config.warehouse_location, - _fileio: config.file_io, + warehouse_location: config.warehouse_location, + fileio: config.file_io, sql_bind_style: config.sql_bind_style, }) } @@ -472,40 +480,298 @@ impl Catalog for SqlCatalog { } } - async fn drop_namespace(&self, _namespace: &NamespaceIdent) -> Result<()> { - todo!() + async fn drop_namespace(&self, namespace: &NamespaceIdent) -> Result<()> { + let exists = self.namespace_exists(namespace).await?; + if exists { + // if there are tables in the namespace, don't allow drop. + let tables = self.list_tables(namespace).await?; + if !tables.is_empty() { + return Err(Error::new( + iceberg::ErrorKind::Unexpected, + format!( + "Namespace {:?} is not empty. {} tables exist.", + namespace, + tables.len() + ), + )); + } + + self.execute( + &format!( + "DELETE FROM {NAMESPACE_TABLE_NAME} + WHERE {NAMESPACE_FIELD_NAME} = ? + AND {CATALOG_FIELD_CATALOG_NAME} = ?" + ), + vec![Some(&namespace.join(".")), Some(&self.name)], + None, + ) + .await?; + + Ok(()) + } else { + no_such_namespace_err(namespace) + } } - async fn list_tables(&self, _namespace: &NamespaceIdent) -> Result> { - todo!() + async fn list_tables(&self, namespace: &NamespaceIdent) -> Result> { + let exists = self.namespace_exists(namespace).await?; + if exists { + let rows = self + .fetch_rows( + &format!( + "SELECT {CATALOG_FIELD_TABLE_NAME}, + {CATALOG_FIELD_TABLE_NAMESPACE} + FROM {CATALOG_TABLE_NAME} + WHERE {CATALOG_FIELD_TABLE_NAMESPACE} = ? + AND {CATALOG_FIELD_CATALOG_NAME} = ? + AND ( + {CATALOG_FIELD_RECORD_TYPE} = '{CATALOG_FIELD_TABLE_RECORD_TYPE}' + OR {CATALOG_FIELD_RECORD_TYPE} IS NULL + )", + ), + vec![Some(&namespace.join(".")), Some(&self.name)], + ) + .await?; + + let mut tables = HashSet::::with_capacity(rows.len()); + + for row in rows.iter() { + let tbl = row + .try_get::(CATALOG_FIELD_TABLE_NAME) + .map_err(from_sqlx_error)?; + let ns_strs = row + .try_get::(CATALOG_FIELD_TABLE_NAMESPACE) + .map_err(from_sqlx_error)?; + let ns = NamespaceIdent::from_strs(ns_strs.split("."))?; + tables.insert(TableIdent::new(ns, tbl)); + } + + Ok(tables.into_iter().collect::>()) + } else { + no_such_namespace_err(namespace) + } } - async fn table_exists(&self, _identifier: &TableIdent) -> Result { - todo!() + async fn table_exists(&self, identifier: &TableIdent) -> Result { + let namespace = identifier.namespace().join("."); + let table_name = identifier.name(); + let table_counts = self + .fetch_rows( + &format!( + "SELECT 1 + FROM {CATALOG_TABLE_NAME} + WHERE {CATALOG_FIELD_TABLE_NAMESPACE} = ? + AND {CATALOG_FIELD_CATALOG_NAME} = ? + AND {CATALOG_FIELD_TABLE_NAME} = ? + AND ( + {CATALOG_FIELD_RECORD_TYPE} = '{CATALOG_FIELD_TABLE_RECORD_TYPE}' + OR {CATALOG_FIELD_RECORD_TYPE} IS NULL + )" + ), + vec![Some(&namespace), Some(&self.name), Some(table_name)], + ) + .await?; + + if !table_counts.is_empty() { + Ok(true) + } else { + Ok(false) + } } - async fn drop_table(&self, _identifier: &TableIdent) -> Result<()> { - todo!() + async fn drop_table(&self, identifier: &TableIdent) -> Result<()> { + if !self.table_exists(identifier).await? { + return no_such_table_err(identifier); + } + + self.execute( + &format!( + "DELETE FROM {CATALOG_TABLE_NAME} + WHERE {CATALOG_FIELD_CATALOG_NAME} = ? + AND {CATALOG_FIELD_TABLE_NAMESPACE} = ? + AND {CATALOG_FIELD_TABLE_NAME} = ? + AND ( + {CATALOG_FIELD_RECORD_TYPE} = '{CATALOG_FIELD_TABLE_RECORD_TYPE}' + OR {CATALOG_FIELD_RECORD_TYPE} IS NULL + )" + ), + vec![ + Some(&self.name), + Some(identifier.name()), + Some(&identifier.namespace().join(".")), + ], + None, + ) + .await?; + + Ok(()) } - async fn load_table(&self, _identifier: &TableIdent) -> Result { - todo!() + async fn load_table(&self, identifier: &TableIdent) -> Result
{ + if !self.table_exists(identifier).await? { + return no_such_table_err(identifier); + } + + let rows = self + .fetch_rows( + &format!( + "SELECT {CATALOG_FIELD_METADATA_LOCATION_PROP} + FROM {CATALOG_TABLE_NAME} + WHERE {CATALOG_FIELD_CATALOG_NAME} = ? + AND {CATALOG_FIELD_TABLE_NAME} = ? + AND {CATALOG_FIELD_TABLE_NAMESPACE} = ? + AND ( + {CATALOG_FIELD_RECORD_TYPE} = '{CATALOG_FIELD_TABLE_RECORD_TYPE}' + OR {CATALOG_FIELD_RECORD_TYPE} IS NULL + )" + ), + vec![ + Some(&self.name), + Some(identifier.name()), + Some(&identifier.namespace().join(".")), + ], + ) + .await?; + + if rows.is_empty() { + return no_such_table_err(identifier); + } + + let row = &rows[0]; + let tbl_metadata_location = row + .try_get::(CATALOG_FIELD_METADATA_LOCATION_PROP) + .map_err(from_sqlx_error)?; + + let file = self.fileio.new_input(&tbl_metadata_location)?; + let metadata_content = file.read().await?; + let metadata = serde_json::from_slice::(&metadata_content)?; + + Ok(Table::builder() + .file_io(self.fileio.clone()) + .identifier(identifier.clone()) + .metadata_location(tbl_metadata_location) + .metadata(metadata) + .build()?) } async fn create_table( &self, - _namespace: &NamespaceIdent, - _creation: TableCreation, + namespace: &NamespaceIdent, + creation: TableCreation, ) -> Result
{ - todo!() + if !self.namespace_exists(namespace).await? { + return no_such_namespace_err(namespace); + } + + let tbl_name = creation.name.clone(); + let tbl_ident = TableIdent::new(namespace.clone(), tbl_name.clone()); + + if self.table_exists(&tbl_ident).await? { + return table_already_exists_err(&tbl_ident); + } + + let (tbl_creation, location) = match creation.location.clone() { + Some(location) => (creation, location), + None => { + // fall back to namespace-specific location + // and then to warehouse location + let nsp_properties = self.get_namespace(namespace).await?.properties().clone(); + let nsp_location = match nsp_properties.get(NAMESPACE_LOCATION_PROPERTY_KEY) { + Some(location) => location.clone(), + None => { + format!( + "{}/{}", + self.warehouse_location.clone(), + namespace.join("/") + ) + } + }; + + let tbl_location = format!("{}/{}", nsp_location, tbl_ident.name()); + + ( + TableCreation { + location: Some(tbl_location.clone()), + ..creation + }, + tbl_location, + ) + } + }; + + let tbl_metadata = TableMetadataBuilder::from_table_creation(tbl_creation)?.build()?; + let tbl_metadata_location = format!( + "{}/metadata/0-{}.metadata.json", + location.clone(), + Uuid::new_v4() + ); + + let file = self.fileio.new_output(&tbl_metadata_location)?; + file.write(serde_json::to_vec(&tbl_metadata)?.into()) + .await?; + + self.execute(&format!( + "INSERT INTO {CATALOG_TABLE_NAME} + ({CATALOG_FIELD_CATALOG_NAME}, {CATALOG_FIELD_TABLE_NAMESPACE}, {CATALOG_FIELD_TABLE_NAME}, {CATALOG_FIELD_METADATA_LOCATION_PROP}, {CATALOG_FIELD_RECORD_TYPE}) + VALUES (?, ?, ?, ?, ?) + "), vec![Some(&self.name), Some(&namespace.join(".")), Some(&tbl_name.clone()), Some(&tbl_metadata_location), Some(CATALOG_FIELD_TABLE_RECORD_TYPE)], None).await?; + + Ok(Table::builder() + .file_io(self.fileio.clone()) + .metadata_location(tbl_metadata_location) + .identifier(tbl_ident) + .metadata(tbl_metadata) + .build()?) } - async fn rename_table(&self, _src: &TableIdent, _dest: &TableIdent) -> Result<()> { - todo!() + async fn rename_table(&self, src: &TableIdent, dest: &TableIdent) -> Result<()> { + if src == dest { + return Ok(()); + } + + if !self.table_exists(src).await? { + return no_such_table_err(src); + } + + if !self.namespace_exists(dest.namespace()).await? { + return no_such_namespace_err(dest.namespace()); + } + + if self.table_exists(dest).await? { + return table_already_exists_err(dest); + } + + self.execute( + &format!( + "UPDATE {CATALOG_TABLE_NAME} + SET {CATALOG_FIELD_TABLE_NAME} = ?, {CATALOG_FIELD_TABLE_NAMESPACE} = ? + WHERE {CATALOG_FIELD_CATALOG_NAME} = ? + AND {CATALOG_FIELD_TABLE_NAME} = ? + AND {CATALOG_FIELD_TABLE_NAMESPACE} = ? + AND ( + {CATALOG_FIELD_RECORD_TYPE} = '{CATALOG_FIELD_TABLE_RECORD_TYPE}' + OR {CATALOG_FIELD_RECORD_TYPE} IS NULL + )" + ), + vec![ + Some(dest.name()), + Some(&dest.namespace().join(".")), + Some(&self.name), + Some(src.name()), + Some(&src.namespace().join(".")), + ], + None, + ) + .await?; + + Ok(()) } async fn update_table(&self, _commit: TableCommit) -> Result
{ - todo!() + Err(Error::new( + ErrorKind::FeatureUnsupported, + "Updating a table is not supported yet", + )) } } @@ -515,12 +781,19 @@ mod tests { use std::hash::Hash; use iceberg::io::FileIOBuilder; - use iceberg::{Catalog, Namespace, NamespaceIdent}; + use iceberg::spec::{NestedField, PartitionSpec, PrimitiveType, Schema, SortOrder, Type}; + use iceberg::table::Table; + use iceberg::{Catalog, Namespace, NamespaceIdent, TableCreation, TableIdent}; + use itertools::Itertools; + use regex::Regex; use sqlx::migrate::MigrateDatabase; use tempfile::TempDir; + use crate::catalog::NAMESPACE_LOCATION_PROPERTY_KEY; use crate::{SqlBindStyle, SqlCatalog, SqlCatalogConfig}; + const UUID_REGEX_STR: &str = "[0-9a-f]{8}-[0-9a-f]{4}-[0-9a-f]{4}-[0-9a-f]{4}-[0-9a-f]{12}"; + fn temp_path() -> String { let temp_dir = TempDir::new().unwrap(); temp_dir.path().to_str().unwrap().to_string() @@ -562,6 +835,83 @@ mod tests { } } + fn simple_table_schema() -> Schema { + Schema::builder() + .with_fields(vec![NestedField::required( + 1, + "foo", + Type::Primitive(PrimitiveType::Int), + ) + .into()]) + .build() + .unwrap() + } + + async fn create_table(catalog: &C, table_ident: &TableIdent) { + let _ = catalog + .create_table( + &table_ident.namespace, + TableCreation::builder() + .name(table_ident.name().into()) + .schema(simple_table_schema()) + .location(temp_path()) + .build(), + ) + .await + .unwrap(); + } + + async fn create_tables(catalog: &C, table_idents: Vec<&TableIdent>) { + for table_ident in table_idents { + create_table(catalog, table_ident).await; + } + } + + fn assert_table_eq(table: &Table, expected_table_ident: &TableIdent, expected_schema: &Schema) { + assert_eq!(table.identifier(), expected_table_ident); + + let metadata = table.metadata(); + + assert_eq!(metadata.current_schema().as_ref(), expected_schema); + + let expected_partition_spec = PartitionSpec::builder(expected_schema) + .with_spec_id(0) + .build() + .unwrap(); + + assert_eq!( + metadata + .partition_specs_iter() + .map(|p| p.as_ref()) + .collect_vec(), + vec![&expected_partition_spec] + ); + + let expected_sorted_order = SortOrder::builder() + .with_order_id(0) + .with_fields(vec![]) + .build(expected_schema) + .unwrap(); + + assert_eq!( + metadata + .sort_orders_iter() + .map(|s| s.as_ref()) + .collect_vec(), + vec![&expected_sorted_order] + ); + + assert_eq!(metadata.properties(), &HashMap::new()); + + assert!(!table.readonly()); + } + + fn assert_table_metadata_location_matches(table: &Table, regex_str: &str) { + let actual = table.metadata_location().unwrap().to_string(); + let regex = Regex::new(regex_str).unwrap(); + assert!(regex.is_match(&actual)) + } + #[tokio::test] async fn test_initialized() { let warehouse_loc = temp_path(); @@ -810,7 +1160,6 @@ mod tests { } #[tokio::test] - #[ignore = "drop_namespace not implemented"] async fn test_drop_namespace() { let warehouse_loc = temp_path(); let catalog = new_sql_catalog(warehouse_loc).await; @@ -823,7 +1172,6 @@ mod tests { } #[tokio::test] - #[ignore = "drop_namespace not implemented"] async fn test_drop_nested_namespace() { let warehouse_loc = temp_path(); let catalog = new_sql_catalog(warehouse_loc).await; @@ -842,7 +1190,6 @@ mod tests { } #[tokio::test] - #[ignore = "drop_namespace not implemented"] async fn test_drop_deeply_nested_namespace() { let warehouse_loc = temp_path(); let catalog = new_sql_catalog(warehouse_loc).await; @@ -875,7 +1222,6 @@ mod tests { } #[tokio::test] - #[ignore = "drop_namespace not implemented"] async fn test_drop_namespace_throws_error_if_namespace_doesnt_exist() { let warehouse_loc = temp_path(); let catalog = new_sql_catalog(warehouse_loc).await; @@ -895,7 +1241,6 @@ mod tests { } #[tokio::test] - #[ignore = "drop_namespace not implemented"] async fn test_drop_namespace_throws_error_if_nested_namespace_doesnt_exist() { let warehouse_loc = temp_path(); let catalog = new_sql_catalog(warehouse_loc).await; @@ -917,7 +1262,6 @@ mod tests { } #[tokio::test] - #[ignore = "drop_namespace not implemented"] async fn test_dropping_a_namespace_does_not_drop_namespaces_nested_under_that_one() { let warehouse_loc = temp_path(); let catalog = new_sql_catalog(warehouse_loc).await; @@ -934,4 +1278,438 @@ mod tests { .await .unwrap()); } + + #[tokio::test] + async fn test_list_tables_returns_empty_vector() { + let warehouse_loc = temp_path(); + let catalog = new_sql_catalog(warehouse_loc).await; + let namespace_ident = NamespaceIdent::new("a".into()); + create_namespace(&catalog, &namespace_ident).await; + + assert_eq!(catalog.list_tables(&namespace_ident).await.unwrap(), vec![]); + } + + #[tokio::test] + async fn test_list_tables_throws_error_if_namespace_doesnt_exist() { + let warehouse_loc = temp_path(); + let catalog = new_sql_catalog(warehouse_loc).await; + + let non_existent_namespace_ident = NamespaceIdent::new("n1".into()); + + assert_eq!( + catalog + .list_tables(&non_existent_namespace_ident) + .await + .unwrap_err() + .to_string(), + format!( + "Unexpected => No such namespace: {:?}", + non_existent_namespace_ident + ), + ); + } + + #[tokio::test] + async fn test_create_table_with_location() { + let warehouse_loc = temp_path(); + let catalog = new_sql_catalog(warehouse_loc.clone()).await; + let namespace_ident = NamespaceIdent::new("a".into()); + create_namespace(&catalog, &namespace_ident).await; + + let table_name = "abc"; + let location = warehouse_loc.clone(); + let table_creation = TableCreation::builder() + .name(table_name.into()) + .location(location.clone()) + .schema(simple_table_schema()) + .build(); + + let expected_table_ident = TableIdent::new(namespace_ident.clone(), table_name.into()); + + assert_table_eq( + &catalog + .create_table(&namespace_ident, table_creation) + .await + .unwrap(), + &expected_table_ident, + &simple_table_schema(), + ); + + let table = catalog.load_table(&expected_table_ident).await.unwrap(); + + assert_table_eq(&table, &expected_table_ident, &simple_table_schema()); + + assert!(table + .metadata_location() + .unwrap() + .to_string() + .starts_with(&location)) + } + + #[tokio::test] + async fn test_create_table_falls_back_to_namespace_location_if_table_location_is_missing() { + let warehouse_loc = temp_path(); + let catalog = new_sql_catalog(warehouse_loc).await; + + let namespace_ident = NamespaceIdent::new("a".into()); + let mut namespace_properties = HashMap::new(); + let namespace_location = temp_path(); + namespace_properties.insert( + NAMESPACE_LOCATION_PROPERTY_KEY.to_string(), + namespace_location.to_string(), + ); + catalog + .create_namespace(&namespace_ident, namespace_properties) + .await + .unwrap(); + + let table_name = "tbl1"; + let expected_table_ident = TableIdent::new(namespace_ident.clone(), table_name.into()); + let expected_table_metadata_location_regex = format!( + "^{}/tbl1/metadata/0-{}.metadata.json$", + namespace_location, UUID_REGEX_STR, + ); + + let table = catalog + .create_table( + &namespace_ident, + TableCreation::builder() + .name(table_name.into()) + .schema(simple_table_schema()) + // no location specified for table + .build(), + ) + .await + .unwrap(); + assert_table_eq(&table, &expected_table_ident, &simple_table_schema()); + assert_table_metadata_location_matches(&table, &expected_table_metadata_location_regex); + + let table = catalog.load_table(&expected_table_ident).await.unwrap(); + assert_table_eq(&table, &expected_table_ident, &simple_table_schema()); + assert_table_metadata_location_matches(&table, &expected_table_metadata_location_regex); + } + + #[tokio::test] + async fn test_create_table_in_nested_namespace_falls_back_to_nested_namespace_location_if_table_location_is_missing( + ) { + let warehouse_loc = temp_path(); + let catalog = new_sql_catalog(warehouse_loc).await; + + let namespace_ident = NamespaceIdent::new("a".into()); + let mut namespace_properties = HashMap::new(); + let namespace_location = temp_path(); + namespace_properties.insert( + NAMESPACE_LOCATION_PROPERTY_KEY.to_string(), + namespace_location.to_string(), + ); + catalog + .create_namespace(&namespace_ident, namespace_properties) + .await + .unwrap(); + + let nested_namespace_ident = NamespaceIdent::from_strs(vec!["a", "b"]).unwrap(); + let mut nested_namespace_properties = HashMap::new(); + let nested_namespace_location = temp_path(); + nested_namespace_properties.insert( + NAMESPACE_LOCATION_PROPERTY_KEY.to_string(), + nested_namespace_location.to_string(), + ); + catalog + .create_namespace(&nested_namespace_ident, nested_namespace_properties) + .await + .unwrap(); + + let table_name = "tbl1"; + let expected_table_ident = + TableIdent::new(nested_namespace_ident.clone(), table_name.into()); + let expected_table_metadata_location_regex = format!( + "^{}/tbl1/metadata/0-{}.metadata.json$", + nested_namespace_location, UUID_REGEX_STR, + ); + + let table = catalog + .create_table( + &nested_namespace_ident, + TableCreation::builder() + .name(table_name.into()) + .schema(simple_table_schema()) + // no location specified for table + .build(), + ) + .await + .unwrap(); + assert_table_eq(&table, &expected_table_ident, &simple_table_schema()); + assert_table_metadata_location_matches(&table, &expected_table_metadata_location_regex); + + let table = catalog.load_table(&expected_table_ident).await.unwrap(); + assert_table_eq(&table, &expected_table_ident, &simple_table_schema()); + assert_table_metadata_location_matches(&table, &expected_table_metadata_location_regex); + } + + #[tokio::test] + async fn test_create_table_falls_back_to_warehouse_location_if_both_table_location_and_namespace_location_are_missing( + ) { + let warehouse_loc = temp_path(); + let catalog = new_sql_catalog(warehouse_loc.clone()).await; + + let namespace_ident = NamespaceIdent::new("a".into()); + // note: no location specified in namespace_properties + let namespace_properties = HashMap::new(); + catalog + .create_namespace(&namespace_ident, namespace_properties) + .await + .unwrap(); + + let table_name = "tbl1"; + let expected_table_ident = TableIdent::new(namespace_ident.clone(), table_name.into()); + let expected_table_metadata_location_regex = format!( + "^{}/a/tbl1/metadata/0-{}.metadata.json$", + warehouse_loc, UUID_REGEX_STR + ); + + let table = catalog + .create_table( + &namespace_ident, + TableCreation::builder() + .name(table_name.into()) + .schema(simple_table_schema()) + // no location specified for table + .build(), + ) + .await + .unwrap(); + assert_table_eq(&table, &expected_table_ident, &simple_table_schema()); + assert_table_metadata_location_matches(&table, &expected_table_metadata_location_regex); + + let table = catalog.load_table(&expected_table_ident).await.unwrap(); + assert_table_eq(&table, &expected_table_ident, &simple_table_schema()); + assert_table_metadata_location_matches(&table, &expected_table_metadata_location_regex); + } + + #[tokio::test] + async fn test_create_table_in_nested_namespace_falls_back_to_warehouse_location_if_both_table_location_and_namespace_location_are_missing( + ) { + let warehouse_loc = temp_path(); + let catalog = new_sql_catalog(warehouse_loc.clone()).await; + + let namespace_ident = NamespaceIdent::new("a".into()); + create_namespace(&catalog, &namespace_ident).await; + + let nested_namespace_ident = NamespaceIdent::from_strs(vec!["a", "b"]).unwrap(); + create_namespace(&catalog, &nested_namespace_ident).await; + + let table_name = "tbl1"; + let expected_table_ident = + TableIdent::new(nested_namespace_ident.clone(), table_name.into()); + let expected_table_metadata_location_regex = format!( + "^{}/a/b/tbl1/metadata/0-{}.metadata.json$", + warehouse_loc, UUID_REGEX_STR + ); + + let table = catalog + .create_table( + &nested_namespace_ident, + TableCreation::builder() + .name(table_name.into()) + .schema(simple_table_schema()) + // no location specified for table + .build(), + ) + .await + .unwrap(); + assert_table_eq(&table, &expected_table_ident, &simple_table_schema()); + assert_table_metadata_location_matches(&table, &expected_table_metadata_location_regex); + + let table = catalog.load_table(&expected_table_ident).await.unwrap(); + assert_table_eq(&table, &expected_table_ident, &simple_table_schema()); + assert_table_metadata_location_matches(&table, &expected_table_metadata_location_regex); + } + + #[tokio::test] + async fn test_create_table_throws_error_if_table_with_same_name_already_exists() { + let warehouse_loc = temp_path(); + let catalog = new_sql_catalog(warehouse_loc.clone()).await; + let namespace_ident = NamespaceIdent::new("a".into()); + create_namespace(&catalog, &namespace_ident).await; + let table_name = "tbl1"; + let table_ident = TableIdent::new(namespace_ident.clone(), table_name.into()); + create_table(&catalog, &table_ident).await; + + let tmp_dir = TempDir::new().unwrap(); + let location = tmp_dir.path().to_str().unwrap().to_string(); + + assert_eq!( + catalog + .create_table( + &namespace_ident, + TableCreation::builder() + .name(table_name.into()) + .schema(simple_table_schema()) + .location(location) + .build() + ) + .await + .unwrap_err() + .to_string(), + format!("Unexpected => Table {:?} already exists.", &table_ident) + ); + } + + #[tokio::test] + async fn test_rename_table_in_same_namespace() { + let warehouse_loc = temp_path(); + let catalog = new_sql_catalog(warehouse_loc).await; + let namespace_ident = NamespaceIdent::new("n1".into()); + create_namespace(&catalog, &namespace_ident).await; + let src_table_ident = TableIdent::new(namespace_ident.clone(), "tbl1".into()); + let dst_table_ident = TableIdent::new(namespace_ident.clone(), "tbl2".into()); + create_table(&catalog, &src_table_ident).await; + + catalog + .rename_table(&src_table_ident, &dst_table_ident) + .await + .unwrap(); + + assert_eq!(catalog.list_tables(&namespace_ident).await.unwrap(), vec![ + dst_table_ident + ],); + } + + #[tokio::test] + async fn test_rename_table_across_namespaces() { + let warehouse_loc = temp_path(); + let catalog = new_sql_catalog(warehouse_loc).await; + let src_namespace_ident = NamespaceIdent::new("a".into()); + let dst_namespace_ident = NamespaceIdent::new("b".into()); + create_namespaces(&catalog, &vec![&src_namespace_ident, &dst_namespace_ident]).await; + let src_table_ident = TableIdent::new(src_namespace_ident.clone(), "tbl1".into()); + let dst_table_ident = TableIdent::new(dst_namespace_ident.clone(), "tbl2".into()); + create_table(&catalog, &src_table_ident).await; + + catalog + .rename_table(&src_table_ident, &dst_table_ident) + .await + .unwrap(); + + assert_eq!( + catalog.list_tables(&src_namespace_ident).await.unwrap(), + vec![], + ); + + assert_eq!( + catalog.list_tables(&dst_namespace_ident).await.unwrap(), + vec![dst_table_ident], + ); + } + + #[tokio::test] + async fn test_rename_table_src_table_is_same_as_dst_table() { + let warehouse_loc = temp_path(); + let catalog = new_sql_catalog(warehouse_loc).await; + let namespace_ident = NamespaceIdent::new("n1".into()); + create_namespace(&catalog, &namespace_ident).await; + let table_ident = TableIdent::new(namespace_ident.clone(), "tbl".into()); + create_table(&catalog, &table_ident).await; + + catalog + .rename_table(&table_ident, &table_ident) + .await + .unwrap(); + + assert_eq!(catalog.list_tables(&namespace_ident).await.unwrap(), vec![ + table_ident + ],); + } + + #[tokio::test] + async fn test_rename_table_across_nested_namespaces() { + let warehouse_loc = temp_path(); + let catalog = new_sql_catalog(warehouse_loc).await; + let namespace_ident_a = NamespaceIdent::new("a".into()); + let namespace_ident_a_b = NamespaceIdent::from_strs(vec!["a", "b"]).unwrap(); + let namespace_ident_a_b_c = NamespaceIdent::from_strs(vec!["a", "b", "c"]).unwrap(); + create_namespaces(&catalog, &vec![ + &namespace_ident_a, + &namespace_ident_a_b, + &namespace_ident_a_b_c, + ]) + .await; + + let src_table_ident = TableIdent::new(namespace_ident_a_b_c.clone(), "tbl1".into()); + create_tables(&catalog, vec![&src_table_ident]).await; + + let dst_table_ident = TableIdent::new(namespace_ident_a_b.clone(), "tbl1".into()); + catalog + .rename_table(&src_table_ident, &dst_table_ident) + .await + .unwrap(); + + assert!(!catalog.table_exists(&src_table_ident).await.unwrap()); + + assert!(catalog.table_exists(&dst_table_ident).await.unwrap()); + } + + #[tokio::test] + async fn test_rename_table_throws_error_if_dst_namespace_doesnt_exist() { + let warehouse_loc = temp_path(); + let catalog = new_sql_catalog(warehouse_loc).await; + let src_namespace_ident = NamespaceIdent::new("n1".into()); + let src_table_ident = TableIdent::new(src_namespace_ident.clone(), "tbl1".into()); + create_namespace(&catalog, &src_namespace_ident).await; + create_table(&catalog, &src_table_ident).await; + + let non_existent_dst_namespace_ident = NamespaceIdent::new("n2".into()); + let dst_table_ident = + TableIdent::new(non_existent_dst_namespace_ident.clone(), "tbl1".into()); + assert_eq!( + catalog + .rename_table(&src_table_ident, &dst_table_ident) + .await + .unwrap_err() + .to_string(), + format!( + "Unexpected => No such namespace: {:?}", + non_existent_dst_namespace_ident + ), + ); + } + + #[tokio::test] + async fn test_rename_table_throws_error_if_src_table_doesnt_exist() { + let warehouse_loc = temp_path(); + let catalog = new_sql_catalog(warehouse_loc).await; + let namespace_ident = NamespaceIdent::new("n1".into()); + create_namespace(&catalog, &namespace_ident).await; + let src_table_ident = TableIdent::new(namespace_ident.clone(), "tbl1".into()); + let dst_table_ident = TableIdent::new(namespace_ident.clone(), "tbl2".into()); + + assert_eq!( + catalog + .rename_table(&src_table_ident, &dst_table_ident) + .await + .unwrap_err() + .to_string(), + format!("Unexpected => No such table: {:?}", src_table_ident), + ); + } + + #[tokio::test] + async fn test_rename_table_throws_error_if_dst_table_already_exists() { + let warehouse_loc = temp_path(); + let catalog = new_sql_catalog(warehouse_loc).await; + let namespace_ident = NamespaceIdent::new("n1".into()); + create_namespace(&catalog, &namespace_ident).await; + let src_table_ident = TableIdent::new(namespace_ident.clone(), "tbl1".into()); + let dst_table_ident = TableIdent::new(namespace_ident.clone(), "tbl2".into()); + create_tables(&catalog, vec![&src_table_ident, &dst_table_ident]).await; + + assert_eq!( + catalog + .rename_table(&src_table_ident, &dst_table_ident) + .await + .unwrap_err() + .to_string(), + format!("Unexpected => Table {:?} already exists.", &dst_table_ident), + ); + } } diff --git a/crates/catalog/sql/src/error.rs b/crates/catalog/sql/src/error.rs index cfefcc26a..15b56e8e2 100644 --- a/crates/catalog/sql/src/error.rs +++ b/crates/catalog/sql/src/error.rs @@ -15,7 +15,7 @@ // specific language governing permissions and limitations // under the License. -use iceberg::{Error, ErrorKind, NamespaceIdent, Result}; +use iceberg::{Error, ErrorKind, NamespaceIdent, Result, TableIdent}; /// Format an sqlx error into iceberg error. pub fn from_sqlx_error(error: sqlx::Error) -> Error { @@ -32,3 +32,17 @@ pub fn no_such_namespace_err(namespace: &NamespaceIdent) -> Result { format!("No such namespace: {:?}", namespace), )) } + +pub fn no_such_table_err(table_ident: &TableIdent) -> Result { + Err(Error::new( + ErrorKind::Unexpected, + format!("No such table: {:?}", table_ident), + )) +} + +pub fn table_already_exists_err(table_ident: &TableIdent) -> Result { + Err(Error::new( + ErrorKind::Unexpected, + format!("Table {:?} already exists.", table_ident), + )) +} From 8b20ce7a21a3c56cba47a20ad3926b084f8dafde Mon Sep 17 00:00:00 2001 From: Xuanwo Date: Sun, 13 Oct 2024 10:40:55 +0800 Subject: [PATCH 17/46] ci: Allow install a non-debian-packaged Python package (#666) Signed-off-by: Xuanwo --- .github/workflows/bindings_python_ci.yml | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/.github/workflows/bindings_python_ci.yml b/.github/workflows/bindings_python_ci.yml index d4b1aa922..b5fb44ed8 100644 --- a/.github/workflows/bindings_python_ci.yml +++ b/.github/workflows/bindings_python_ci.yml @@ -45,7 +45,7 @@ jobs: - uses: actions/checkout@v4 - name: Install tools run: | - pip install ruff + pip install ruff --break-system-packages - name: Check format working-directory: "bindings/python" run: | @@ -78,6 +78,6 @@ jobs: shell: bash run: | set -e - pip install hatch==1.12.0 + pip install hatch==1.12.0 --break-system-packages hatch run dev:pip install dist/pyiceberg_core-*.whl --force-reinstall hatch run dev:test From de0dde561cde86c47439c37e760edfe56338d571 Mon Sep 17 00:00:00 2001 From: Cancai Cai <77189278+caicancai@users.noreply.github.com> Date: Mon, 14 Oct 2024 10:10:34 +0800 Subject: [PATCH 18/46] docs: README uses iceberg-rust instead of we (#667) --- NOTICE | 2 +- README.md | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/NOTICE b/NOTICE index c2746f155..9340680cb 100644 --- a/NOTICE +++ b/NOTICE @@ -1,5 +1,5 @@ Apache Iceberg Rust -Copyright 2023 The Apache Software Foundation +Copyright 2023-2024 The Apache Software Foundation This product includes software developed at The Apache Software Foundation (http://www.apache.org/). diff --git a/README.md b/README.md index 30168141b..6a65904ac 100644 --- a/README.md +++ b/README.md @@ -81,13 +81,13 @@ The Apache Iceberg Rust project is composed of the following components: Iceberg Rust is built and tested with stable rust, and will keep a rolling MSRV(minimum supported rust version). The current MSRV is 1.77.1. -Also, we use unstable rust to run linters, such as `clippy` and `rustfmt`. But this will not affect downstream users, +Also, iceberg-rust use unstable rust to run linters, such as `clippy` and `rustfmt`. But this will not affect downstream users, and only MSRV is required. ## Contribute -Apache Iceberg is an active open-source project, governed under the Apache Software Foundation (ASF). We are always open to people who want to use or contribute to it. Here are some ways to get involved. +Apache Iceberg is an active open-source project, governed under the Apache Software Foundation (ASF). Iceberg-rust are always open to people who want to use or contribute to it. Here are some ways to get involved. - Start with [Contributing Guide](CONTRIBUTING.md). - Submit [Issues](https://github.com/apache/iceberg-rust/issues/new) for bug report or feature requests. From 7891f2dd21d236fb4d023c6d6f2c06e3a399e700 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Mon, 14 Oct 2024 10:11:53 +0800 Subject: [PATCH 19/46] chore(deps): Bump crate-ci/typos from 1.25.0 to 1.26.0 (#668) Bumps [crate-ci/typos](https://github.com/crate-ci/typos) from 1.25.0 to 1.26.0. - [Release notes](https://github.com/crate-ci/typos/releases) - [Changelog](https://github.com/crate-ci/typos/blob/master/CHANGELOG.md) - [Commits](https://github.com/crate-ci/typos/compare/v1.25.0...v1.26.0) --- updated-dependencies: - dependency-name: crate-ci/typos dependency-type: direct:production update-type: version-update:semver-minor ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- .github/workflows/ci_typos.yml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/.github/workflows/ci_typos.yml b/.github/workflows/ci_typos.yml index e6031fbd1..9d49fe2ee 100644 --- a/.github/workflows/ci_typos.yml +++ b/.github/workflows/ci_typos.yml @@ -42,4 +42,4 @@ jobs: steps: - uses: actions/checkout@v4 - name: Check typos - uses: crate-ci/typos@v1.25.0 + uses: crate-ci/typos@v1.26.0 From ad89eac02712ceac2c3cff6bf0fe5d1b6e289a26 Mon Sep 17 00:00:00 2001 From: Yue Deng <59086724+Dysprosium0626@users.noreply.github.com> Date: Tue, 15 Oct 2024 11:47:11 +0800 Subject: [PATCH 20/46] feat: Add equality delete writer (#372) * feat: add EqualityDeleteWriter * WIP: add test cases * fix: move delete schema out of writer * test: add test case for equality delete writer * fix: refactor projector * fix: fix projector * fix: add result * test: add float and double column test for equality delete writer * fmt * fix: compatibility with #364 * fix: remove unwrap * fix: minor --- .../base_writer/equality_delete_writer.rs | 589 ++++++++++++++++++ crates/iceberg/src/writer/base_writer/mod.rs | 1 + 2 files changed, 590 insertions(+) create mode 100644 crates/iceberg/src/writer/base_writer/equality_delete_writer.rs diff --git a/crates/iceberg/src/writer/base_writer/equality_delete_writer.rs b/crates/iceberg/src/writer/base_writer/equality_delete_writer.rs new file mode 100644 index 000000000..ba198821a --- /dev/null +++ b/crates/iceberg/src/writer/base_writer/equality_delete_writer.rs @@ -0,0 +1,589 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +//! This module provide `EqualityDeleteWriter`. + +use arrow_array::{ArrayRef, RecordBatch, StructArray}; +use arrow_schema::{DataType, FieldRef, Fields, Schema, SchemaRef}; +use itertools::Itertools; + +use crate::spec::{DataFile, Struct}; +use crate::writer::file_writer::FileWriter; +use crate::writer::{file_writer::FileWriterBuilder, IcebergWriter, IcebergWriterBuilder}; +use crate::{Error, ErrorKind, Result}; + +/// Builder for `EqualityDeleteWriter`. +#[derive(Clone)] +pub struct EqualityDeleteFileWriterBuilder { + inner: B, +} + +impl EqualityDeleteFileWriterBuilder { + /// Create a new `EqualityDeleteFileWriterBuilder` using a `FileWriterBuilder`. + pub fn new(inner: B) -> Self { + Self { inner } + } +} + +/// Config for `EqualityDeleteWriter`. +pub struct EqualityDeleteWriterConfig { + equality_ids: Vec, + projector: FieldProjector, + schema: SchemaRef, + partition_value: Struct, +} + +impl EqualityDeleteWriterConfig { + /// Create a new `DataFileWriterConfig` with equality ids. + pub fn new( + equality_ids: Vec, + projector: FieldProjector, + schema: Schema, + partition_value: Option, + ) -> Self { + Self { + equality_ids, + projector, + schema: schema.into(), + partition_value: partition_value.unwrap_or(Struct::empty()), + } + } +} + +#[async_trait::async_trait] +impl IcebergWriterBuilder for EqualityDeleteFileWriterBuilder { + type R = EqualityDeleteFileWriter; + type C = EqualityDeleteWriterConfig; + + async fn build(self, config: Self::C) -> Result { + Ok(EqualityDeleteFileWriter { + inner_writer: Some(self.inner.clone().build().await?), + projector: config.projector, + delete_schema_ref: config.schema, + equality_ids: config.equality_ids, + partition_value: config.partition_value, + }) + } +} + +/// A writer write data +pub struct EqualityDeleteFileWriter { + inner_writer: Option, + projector: FieldProjector, + delete_schema_ref: SchemaRef, + equality_ids: Vec, + partition_value: Struct, +} + +impl EqualityDeleteFileWriter { + fn project_record_batch_columns(&self, batch: RecordBatch) -> Result { + RecordBatch::try_new( + self.delete_schema_ref.clone(), + self.projector.project(batch.columns())?, + ) + .map_err(|err| Error::new(ErrorKind::DataInvalid, format!("{err}"))) + } +} + +#[async_trait::async_trait] +impl IcebergWriter for EqualityDeleteFileWriter { + async fn write(&mut self, batch: RecordBatch) -> Result<()> { + let batch = self.project_record_batch_columns(batch)?; + if let Some(writer) = self.inner_writer.as_mut() { + writer.write(&batch).await + } else { + Err(Error::new( + ErrorKind::Unexpected, + "Equality delete inner writer does not exist", + )) + } + } + + async fn close(&mut self) -> Result> { + if let Some(writer) = self.inner_writer.take() { + Ok(writer + .close() + .await? + .into_iter() + .map(|mut res| { + res.content(crate::spec::DataContentType::EqualityDeletes); + res.equality_ids(self.equality_ids.iter().map(|id| *id as i32).collect_vec()); + res.partition(self.partition_value.clone()); + res.build().expect("msg") + }) + .collect_vec()) + } else { + Err(Error::new( + ErrorKind::Unexpected, + "Equality delete inner writer does not exist", + )) + } + } +} + +/// Help to project specific field from `RecordBatch`` according to the column id. +pub struct FieldProjector { + index_vec_vec: Vec>, +} + +impl FieldProjector { + /// Init FieldProjector + pub fn new( + batch_fields: &Fields, + column_ids: &[usize], + column_id_meta_key: &str, + ) -> Result<(Self, Fields)> { + let mut index_vec_vec = Vec::with_capacity(column_ids.len()); + let mut fields = Vec::with_capacity(column_ids.len()); + for &id in column_ids { + let mut index_vec = vec![]; + if let Ok(field) = Self::fetch_column_index( + batch_fields, + &mut index_vec, + id as i64, + column_id_meta_key, + ) { + fields.push(field.clone()); + index_vec_vec.push(index_vec); + } else { + return Err(Error::new( + ErrorKind::DataInvalid, + format!( + "Can't find source column id or column data type invalid: {}", + id + ), + )); + } + } + Ok((Self { index_vec_vec }, Fields::from_iter(fields))) + } + + fn fetch_column_index( + fields: &Fields, + index_vec: &mut Vec, + col_id: i64, + column_id_meta_key: &str, + ) -> Result { + for (pos, field) in fields.iter().enumerate() { + match field.data_type() { + DataType::Float16 | DataType::Float32 | DataType::Float64 => { + return Err(Error::new( + ErrorKind::DataInvalid, + "Delete column data type cannot be float or double", + )); + } + _ => { + let id: i64 = field + .metadata() + .get(column_id_meta_key) + .ok_or_else(|| Error::new(ErrorKind::DataInvalid, "column_id must be set"))? + .parse::() + .map_err(|_| { + Error::new(ErrorKind::DataInvalid, "column_id must be parsable as i64") + })?; + if col_id == id { + index_vec.push(pos); + return Ok(field.clone()); + } + if let DataType::Struct(inner) = field.data_type() { + let res = + Self::fetch_column_index(inner, index_vec, col_id, column_id_meta_key); + if !index_vec.is_empty() { + index_vec.push(pos); + return res; + } + } + } + } + } + Err(Error::new( + ErrorKind::DataInvalid, + "Column id not found in fields", + )) + } + /// Do projection with batch + pub fn project(&self, batch: &[ArrayRef]) -> Result> { + self.index_vec_vec + .iter() + .map(|index_vec| Self::get_column_by_index_vec(batch, index_vec)) + .collect::>>() + } + + fn get_column_by_index_vec(batch: &[ArrayRef], index_vec: &[usize]) -> Result { + let mut rev_iterator = index_vec.iter().rev(); + let mut array = batch[*rev_iterator.next().unwrap()].clone(); + for idx in rev_iterator { + array = array + .as_any() + .downcast_ref::() + .ok_or(Error::new( + ErrorKind::Unexpected, + "Cannot convert Array to StructArray", + ))? + .column(*idx) + .clone(); + } + Ok(array) + } +} + +#[cfg(test)] +mod test { + use arrow_select::concat::concat_batches; + use itertools::Itertools; + use std::{collections::HashMap, sync::Arc}; + + use arrow_array::{types::Int64Type, ArrayRef, Int64Array, RecordBatch, StructArray}; + use parquet::{ + arrow::{arrow_reader::ParquetRecordBatchReaderBuilder, PARQUET_FIELD_ID_META_KEY}, + file::properties::WriterProperties, + }; + use tempfile::TempDir; + + use crate::{ + io::{FileIO, FileIOBuilder}, + spec::{DataFile, DataFileFormat}, + writer::{ + base_writer::equality_delete_writer::{ + EqualityDeleteFileWriterBuilder, EqualityDeleteWriterConfig, FieldProjector, + }, + file_writer::{ + location_generator::{test::MockLocationGenerator, DefaultFileNameGenerator}, + ParquetWriterBuilder, + }, + IcebergWriter, IcebergWriterBuilder, + }, + }; + + async fn check_parquet_data_file_with_equality_delete_write( + file_io: &FileIO, + data_file: &DataFile, + batch: &RecordBatch, + ) { + assert_eq!(data_file.file_format, DataFileFormat::Parquet); + + // read the written file + let input_file = file_io.new_input(data_file.file_path.clone()).unwrap(); + // read the written file + let input_content = input_file.read().await.unwrap(); + let reader_builder = + ParquetRecordBatchReaderBuilder::try_new(input_content.clone()).unwrap(); + let metadata = reader_builder.metadata().clone(); + + // check data + let reader = reader_builder.build().unwrap(); + let batches = reader.map(|batch| batch.unwrap()).collect::>(); + let res = concat_batches(&batch.schema(), &batches).unwrap(); + assert_eq!(*batch, res); + + // check metadata + let expect_column_num = batch.num_columns(); + + assert_eq!( + data_file.record_count, + metadata + .row_groups() + .iter() + .map(|group| group.num_rows()) + .sum::() as u64 + ); + + assert_eq!(data_file.file_size_in_bytes, input_content.len() as u64); + + assert_eq!(data_file.column_sizes.len(), expect_column_num); + + for (index, id) in data_file.column_sizes().keys().sorted().enumerate() { + metadata + .row_groups() + .iter() + .map(|group| group.columns()) + .for_each(|column| { + assert_eq!( + *data_file.column_sizes.get(id).unwrap() as i64, + column.get(index).unwrap().compressed_size() + ); + }); + } + + assert_eq!(data_file.value_counts.len(), expect_column_num); + data_file.value_counts.iter().for_each(|(_, &v)| { + let expect = metadata + .row_groups() + .iter() + .map(|group| group.num_rows()) + .sum::() as u64; + assert_eq!(v, expect); + }); + + for (index, id) in data_file.null_value_counts().keys().enumerate() { + let expect = batch.column(index).null_count() as u64; + assert_eq!(*data_file.null_value_counts.get(id).unwrap(), expect); + } + + assert_eq!(data_file.split_offsets.len(), metadata.num_row_groups()); + data_file + .split_offsets + .iter() + .enumerate() + .for_each(|(i, &v)| { + let expect = metadata.row_groups()[i].file_offset().unwrap(); + assert_eq!(v, expect); + }); + } + + #[tokio::test] + async fn test_equality_delete_writer() -> Result<(), anyhow::Error> { + 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 + // Int, Struct(Int), String, List(Int), Struct(Struct(Int)) + let schema = { + let fields = vec![ + arrow_schema::Field::new("col0", 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::Struct( + vec![arrow_schema::Field::new( + "sub_col", + arrow_schema::DataType::Int64, + true, + ) + .with_metadata(HashMap::from([( + PARQUET_FIELD_ID_META_KEY.to_string(), + "5".to_string(), + )]))] + .into(), + ), + true, + ) + .with_metadata(HashMap::from([( + PARQUET_FIELD_ID_META_KEY.to_string(), + "1".to_string(), + )])), + arrow_schema::Field::new("col2", arrow_schema::DataType::Utf8, true).with_metadata( + HashMap::from([(PARQUET_FIELD_ID_META_KEY.to_string(), "2".to_string())]), + ), + arrow_schema::Field::new( + "col3", + arrow_schema::DataType::List(Arc::new( + arrow_schema::Field::new("item", arrow_schema::DataType::Int64, true) + .with_metadata(HashMap::from([( + PARQUET_FIELD_ID_META_KEY.to_string(), + "6".to_string(), + )])), + )), + true, + ) + .with_metadata(HashMap::from([( + PARQUET_FIELD_ID_META_KEY.to_string(), + "3".to_string(), + )])), + arrow_schema::Field::new( + "col4", + arrow_schema::DataType::Struct( + vec![arrow_schema::Field::new( + "sub_col", + arrow_schema::DataType::Struct( + vec![arrow_schema::Field::new( + "sub_sub_col", + arrow_schema::DataType::Int64, + true, + ) + .with_metadata(HashMap::from([( + PARQUET_FIELD_ID_META_KEY.to_string(), + "7".to_string(), + )]))] + .into(), + ), + true, + ) + .with_metadata(HashMap::from([( + PARQUET_FIELD_ID_META_KEY.to_string(), + "8".to_string(), + )]))] + .into(), + ), + true, + ) + .with_metadata(HashMap::from([( + PARQUET_FIELD_ID_META_KEY.to_string(), + "4".to_string(), + )])), + ]; + arrow_schema::Schema::new(fields) + }; + let col0 = Arc::new(Int64Array::from_iter_values(vec![1; 1024])) as ArrayRef; + let col1 = Arc::new(StructArray::new( + vec![ + arrow_schema::Field::new("sub_col", arrow_schema::DataType::Int64, true) + .with_metadata(HashMap::from([( + PARQUET_FIELD_ID_META_KEY.to_string(), + "5".to_string(), + )])), + ] + .into(), + vec![Arc::new(Int64Array::from_iter_values(vec![1; 1024]))], + None, + )); + let col2 = Arc::new(arrow_array::StringArray::from_iter_values(vec![ + "test"; + 1024 + ])) as ArrayRef; + let col3 = Arc::new({ + let list_parts = arrow_array::ListArray::from_iter_primitive::(vec![ + Some( + vec![Some(1),] + ); + 1024 + ]) + .into_parts(); + arrow_array::ListArray::new( + Arc::new(list_parts.0.as_ref().clone().with_metadata(HashMap::from([( + PARQUET_FIELD_ID_META_KEY.to_string(), + "6".to_string(), + )]))), + list_parts.1, + list_parts.2, + list_parts.3, + ) + }) as ArrayRef; + let col4 = Arc::new(StructArray::new( + vec![arrow_schema::Field::new( + "sub_col", + arrow_schema::DataType::Struct( + vec![arrow_schema::Field::new( + "sub_sub_col", + arrow_schema::DataType::Int64, + true, + ) + .with_metadata(HashMap::from([( + PARQUET_FIELD_ID_META_KEY.to_string(), + "7".to_string(), + )]))] + .into(), + ), + true, + ) + .with_metadata(HashMap::from([( + PARQUET_FIELD_ID_META_KEY.to_string(), + "8".to_string(), + )]))] + .into(), + vec![Arc::new(StructArray::new( + vec![ + arrow_schema::Field::new("sub_sub_col", arrow_schema::DataType::Int64, true) + .with_metadata(HashMap::from([( + PARQUET_FIELD_ID_META_KEY.to_string(), + "7".to_string(), + )])), + ] + .into(), + vec![Arc::new(Int64Array::from_iter_values(vec![1; 1024]))], + None, + ))], + None, + )); + let columns = vec![col0, col1, col2, col3, col4]; + + let equality_ids = vec![1, 3]; + let (projector, fields) = + FieldProjector::new(schema.fields(), &equality_ids, PARQUET_FIELD_ID_META_KEY)?; + let delete_schema = arrow_schema::Schema::new(fields); + let delete_schema_ref = Arc::new(delete_schema.clone()); + + // prepare writer + let to_write = RecordBatch::try_new(Arc::new(schema.clone()), columns).unwrap(); + let pb = ParquetWriterBuilder::new( + WriterProperties::builder().build(), + delete_schema_ref.clone(), + file_io.clone(), + location_gen, + file_name_gen, + ); + + let mut equality_delete_writer = EqualityDeleteFileWriterBuilder::new(pb) + .build(EqualityDeleteWriterConfig::new( + equality_ids, + projector, + delete_schema.clone(), + None, + )) + .await?; + // write + equality_delete_writer.write(to_write.clone()).await?; + let res = equality_delete_writer.close().await?; + assert_eq!(res.len(), 1); + let data_file = res.into_iter().next().unwrap(); + + // check + let to_write_projected = equality_delete_writer.project_record_batch_columns(to_write)?; + check_parquet_data_file_with_equality_delete_write( + &file_io, + &data_file, + &to_write_projected, + ) + .await; + Ok(()) + } + + #[tokio::test] + async fn test_equality_delete_float_or_double_column() -> Result<(), anyhow::Error> { + // Float32, Float64 + let schema = { + let fields = vec![ + arrow_schema::Field::new("col0", 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::Schema::new(fields) + }; + + let equality_id_float = vec![0]; + let result_float = FieldProjector::new( + schema.fields(), + &equality_id_float, + PARQUET_FIELD_ID_META_KEY, + ); + assert!(result_float.is_err()); + + let equality_ids_double = vec![1]; + let result_double = FieldProjector::new( + schema.fields(), + &equality_ids_double, + PARQUET_FIELD_ID_META_KEY, + ); + assert!(result_double.is_err()); + + Ok(()) + } +} diff --git a/crates/iceberg/src/writer/base_writer/mod.rs b/crates/iceberg/src/writer/base_writer/mod.rs index 37da2ab81..37ab97eb6 100644 --- a/crates/iceberg/src/writer/base_writer/mod.rs +++ b/crates/iceberg/src/writer/base_writer/mod.rs @@ -18,3 +18,4 @@ //! Base writer module contains the basic writer provide by iceberg: `DataFileWriter`, `PositionDeleteFileWriter`, `EqualityDeleteFileWriter`. pub mod data_file_writer; +pub mod equality_delete_writer; From 15b2a7aeea577b84044bda43e37bd7bc07c4a82c Mon Sep 17 00:00:00 2001 From: Xuanwo Date: Tue, 15 Oct 2024 21:57:10 +0800 Subject: [PATCH 21/46] Revert "feat: Add equality delete writer (#372)" (#672) --- .../base_writer/equality_delete_writer.rs | 589 ------------------ crates/iceberg/src/writer/base_writer/mod.rs | 1 - 2 files changed, 590 deletions(-) delete mode 100644 crates/iceberg/src/writer/base_writer/equality_delete_writer.rs diff --git a/crates/iceberg/src/writer/base_writer/equality_delete_writer.rs b/crates/iceberg/src/writer/base_writer/equality_delete_writer.rs deleted file mode 100644 index ba198821a..000000000 --- a/crates/iceberg/src/writer/base_writer/equality_delete_writer.rs +++ /dev/null @@ -1,589 +0,0 @@ -// Licensed to the Apache Software Foundation (ASF) under one -// or more contributor license agreements. See the NOTICE file -// distributed with this work for additional information -// regarding copyright ownership. The ASF licenses this file -// to you under the Apache License, Version 2.0 (the -// "License"); you may not use this file except in compliance -// with the License. You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, -// software distributed under the License is distributed on an -// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -// KIND, either express or implied. See the License for the -// specific language governing permissions and limitations -// under the License. - -//! This module provide `EqualityDeleteWriter`. - -use arrow_array::{ArrayRef, RecordBatch, StructArray}; -use arrow_schema::{DataType, FieldRef, Fields, Schema, SchemaRef}; -use itertools::Itertools; - -use crate::spec::{DataFile, Struct}; -use crate::writer::file_writer::FileWriter; -use crate::writer::{file_writer::FileWriterBuilder, IcebergWriter, IcebergWriterBuilder}; -use crate::{Error, ErrorKind, Result}; - -/// Builder for `EqualityDeleteWriter`. -#[derive(Clone)] -pub struct EqualityDeleteFileWriterBuilder { - inner: B, -} - -impl EqualityDeleteFileWriterBuilder { - /// Create a new `EqualityDeleteFileWriterBuilder` using a `FileWriterBuilder`. - pub fn new(inner: B) -> Self { - Self { inner } - } -} - -/// Config for `EqualityDeleteWriter`. -pub struct EqualityDeleteWriterConfig { - equality_ids: Vec, - projector: FieldProjector, - schema: SchemaRef, - partition_value: Struct, -} - -impl EqualityDeleteWriterConfig { - /// Create a new `DataFileWriterConfig` with equality ids. - pub fn new( - equality_ids: Vec, - projector: FieldProjector, - schema: Schema, - partition_value: Option, - ) -> Self { - Self { - equality_ids, - projector, - schema: schema.into(), - partition_value: partition_value.unwrap_or(Struct::empty()), - } - } -} - -#[async_trait::async_trait] -impl IcebergWriterBuilder for EqualityDeleteFileWriterBuilder { - type R = EqualityDeleteFileWriter; - type C = EqualityDeleteWriterConfig; - - async fn build(self, config: Self::C) -> Result { - Ok(EqualityDeleteFileWriter { - inner_writer: Some(self.inner.clone().build().await?), - projector: config.projector, - delete_schema_ref: config.schema, - equality_ids: config.equality_ids, - partition_value: config.partition_value, - }) - } -} - -/// A writer write data -pub struct EqualityDeleteFileWriter { - inner_writer: Option, - projector: FieldProjector, - delete_schema_ref: SchemaRef, - equality_ids: Vec, - partition_value: Struct, -} - -impl EqualityDeleteFileWriter { - fn project_record_batch_columns(&self, batch: RecordBatch) -> Result { - RecordBatch::try_new( - self.delete_schema_ref.clone(), - self.projector.project(batch.columns())?, - ) - .map_err(|err| Error::new(ErrorKind::DataInvalid, format!("{err}"))) - } -} - -#[async_trait::async_trait] -impl IcebergWriter for EqualityDeleteFileWriter { - async fn write(&mut self, batch: RecordBatch) -> Result<()> { - let batch = self.project_record_batch_columns(batch)?; - if let Some(writer) = self.inner_writer.as_mut() { - writer.write(&batch).await - } else { - Err(Error::new( - ErrorKind::Unexpected, - "Equality delete inner writer does not exist", - )) - } - } - - async fn close(&mut self) -> Result> { - if let Some(writer) = self.inner_writer.take() { - Ok(writer - .close() - .await? - .into_iter() - .map(|mut res| { - res.content(crate::spec::DataContentType::EqualityDeletes); - res.equality_ids(self.equality_ids.iter().map(|id| *id as i32).collect_vec()); - res.partition(self.partition_value.clone()); - res.build().expect("msg") - }) - .collect_vec()) - } else { - Err(Error::new( - ErrorKind::Unexpected, - "Equality delete inner writer does not exist", - )) - } - } -} - -/// Help to project specific field from `RecordBatch`` according to the column id. -pub struct FieldProjector { - index_vec_vec: Vec>, -} - -impl FieldProjector { - /// Init FieldProjector - pub fn new( - batch_fields: &Fields, - column_ids: &[usize], - column_id_meta_key: &str, - ) -> Result<(Self, Fields)> { - let mut index_vec_vec = Vec::with_capacity(column_ids.len()); - let mut fields = Vec::with_capacity(column_ids.len()); - for &id in column_ids { - let mut index_vec = vec![]; - if let Ok(field) = Self::fetch_column_index( - batch_fields, - &mut index_vec, - id as i64, - column_id_meta_key, - ) { - fields.push(field.clone()); - index_vec_vec.push(index_vec); - } else { - return Err(Error::new( - ErrorKind::DataInvalid, - format!( - "Can't find source column id or column data type invalid: {}", - id - ), - )); - } - } - Ok((Self { index_vec_vec }, Fields::from_iter(fields))) - } - - fn fetch_column_index( - fields: &Fields, - index_vec: &mut Vec, - col_id: i64, - column_id_meta_key: &str, - ) -> Result { - for (pos, field) in fields.iter().enumerate() { - match field.data_type() { - DataType::Float16 | DataType::Float32 | DataType::Float64 => { - return Err(Error::new( - ErrorKind::DataInvalid, - "Delete column data type cannot be float or double", - )); - } - _ => { - let id: i64 = field - .metadata() - .get(column_id_meta_key) - .ok_or_else(|| Error::new(ErrorKind::DataInvalid, "column_id must be set"))? - .parse::() - .map_err(|_| { - Error::new(ErrorKind::DataInvalid, "column_id must be parsable as i64") - })?; - if col_id == id { - index_vec.push(pos); - return Ok(field.clone()); - } - if let DataType::Struct(inner) = field.data_type() { - let res = - Self::fetch_column_index(inner, index_vec, col_id, column_id_meta_key); - if !index_vec.is_empty() { - index_vec.push(pos); - return res; - } - } - } - } - } - Err(Error::new( - ErrorKind::DataInvalid, - "Column id not found in fields", - )) - } - /// Do projection with batch - pub fn project(&self, batch: &[ArrayRef]) -> Result> { - self.index_vec_vec - .iter() - .map(|index_vec| Self::get_column_by_index_vec(batch, index_vec)) - .collect::>>() - } - - fn get_column_by_index_vec(batch: &[ArrayRef], index_vec: &[usize]) -> Result { - let mut rev_iterator = index_vec.iter().rev(); - let mut array = batch[*rev_iterator.next().unwrap()].clone(); - for idx in rev_iterator { - array = array - .as_any() - .downcast_ref::() - .ok_or(Error::new( - ErrorKind::Unexpected, - "Cannot convert Array to StructArray", - ))? - .column(*idx) - .clone(); - } - Ok(array) - } -} - -#[cfg(test)] -mod test { - use arrow_select::concat::concat_batches; - use itertools::Itertools; - use std::{collections::HashMap, sync::Arc}; - - use arrow_array::{types::Int64Type, ArrayRef, Int64Array, RecordBatch, StructArray}; - use parquet::{ - arrow::{arrow_reader::ParquetRecordBatchReaderBuilder, PARQUET_FIELD_ID_META_KEY}, - file::properties::WriterProperties, - }; - use tempfile::TempDir; - - use crate::{ - io::{FileIO, FileIOBuilder}, - spec::{DataFile, DataFileFormat}, - writer::{ - base_writer::equality_delete_writer::{ - EqualityDeleteFileWriterBuilder, EqualityDeleteWriterConfig, FieldProjector, - }, - file_writer::{ - location_generator::{test::MockLocationGenerator, DefaultFileNameGenerator}, - ParquetWriterBuilder, - }, - IcebergWriter, IcebergWriterBuilder, - }, - }; - - async fn check_parquet_data_file_with_equality_delete_write( - file_io: &FileIO, - data_file: &DataFile, - batch: &RecordBatch, - ) { - assert_eq!(data_file.file_format, DataFileFormat::Parquet); - - // read the written file - let input_file = file_io.new_input(data_file.file_path.clone()).unwrap(); - // read the written file - let input_content = input_file.read().await.unwrap(); - let reader_builder = - ParquetRecordBatchReaderBuilder::try_new(input_content.clone()).unwrap(); - let metadata = reader_builder.metadata().clone(); - - // check data - let reader = reader_builder.build().unwrap(); - let batches = reader.map(|batch| batch.unwrap()).collect::>(); - let res = concat_batches(&batch.schema(), &batches).unwrap(); - assert_eq!(*batch, res); - - // check metadata - let expect_column_num = batch.num_columns(); - - assert_eq!( - data_file.record_count, - metadata - .row_groups() - .iter() - .map(|group| group.num_rows()) - .sum::() as u64 - ); - - assert_eq!(data_file.file_size_in_bytes, input_content.len() as u64); - - assert_eq!(data_file.column_sizes.len(), expect_column_num); - - for (index, id) in data_file.column_sizes().keys().sorted().enumerate() { - metadata - .row_groups() - .iter() - .map(|group| group.columns()) - .for_each(|column| { - assert_eq!( - *data_file.column_sizes.get(id).unwrap() as i64, - column.get(index).unwrap().compressed_size() - ); - }); - } - - assert_eq!(data_file.value_counts.len(), expect_column_num); - data_file.value_counts.iter().for_each(|(_, &v)| { - let expect = metadata - .row_groups() - .iter() - .map(|group| group.num_rows()) - .sum::() as u64; - assert_eq!(v, expect); - }); - - for (index, id) in data_file.null_value_counts().keys().enumerate() { - let expect = batch.column(index).null_count() as u64; - assert_eq!(*data_file.null_value_counts.get(id).unwrap(), expect); - } - - assert_eq!(data_file.split_offsets.len(), metadata.num_row_groups()); - data_file - .split_offsets - .iter() - .enumerate() - .for_each(|(i, &v)| { - let expect = metadata.row_groups()[i].file_offset().unwrap(); - assert_eq!(v, expect); - }); - } - - #[tokio::test] - async fn test_equality_delete_writer() -> Result<(), anyhow::Error> { - 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 - // Int, Struct(Int), String, List(Int), Struct(Struct(Int)) - let schema = { - let fields = vec![ - arrow_schema::Field::new("col0", 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::Struct( - vec![arrow_schema::Field::new( - "sub_col", - arrow_schema::DataType::Int64, - true, - ) - .with_metadata(HashMap::from([( - PARQUET_FIELD_ID_META_KEY.to_string(), - "5".to_string(), - )]))] - .into(), - ), - true, - ) - .with_metadata(HashMap::from([( - PARQUET_FIELD_ID_META_KEY.to_string(), - "1".to_string(), - )])), - arrow_schema::Field::new("col2", arrow_schema::DataType::Utf8, true).with_metadata( - HashMap::from([(PARQUET_FIELD_ID_META_KEY.to_string(), "2".to_string())]), - ), - arrow_schema::Field::new( - "col3", - arrow_schema::DataType::List(Arc::new( - arrow_schema::Field::new("item", arrow_schema::DataType::Int64, true) - .with_metadata(HashMap::from([( - PARQUET_FIELD_ID_META_KEY.to_string(), - "6".to_string(), - )])), - )), - true, - ) - .with_metadata(HashMap::from([( - PARQUET_FIELD_ID_META_KEY.to_string(), - "3".to_string(), - )])), - arrow_schema::Field::new( - "col4", - arrow_schema::DataType::Struct( - vec![arrow_schema::Field::new( - "sub_col", - arrow_schema::DataType::Struct( - vec![arrow_schema::Field::new( - "sub_sub_col", - arrow_schema::DataType::Int64, - true, - ) - .with_metadata(HashMap::from([( - PARQUET_FIELD_ID_META_KEY.to_string(), - "7".to_string(), - )]))] - .into(), - ), - true, - ) - .with_metadata(HashMap::from([( - PARQUET_FIELD_ID_META_KEY.to_string(), - "8".to_string(), - )]))] - .into(), - ), - true, - ) - .with_metadata(HashMap::from([( - PARQUET_FIELD_ID_META_KEY.to_string(), - "4".to_string(), - )])), - ]; - arrow_schema::Schema::new(fields) - }; - let col0 = Arc::new(Int64Array::from_iter_values(vec![1; 1024])) as ArrayRef; - let col1 = Arc::new(StructArray::new( - vec![ - arrow_schema::Field::new("sub_col", arrow_schema::DataType::Int64, true) - .with_metadata(HashMap::from([( - PARQUET_FIELD_ID_META_KEY.to_string(), - "5".to_string(), - )])), - ] - .into(), - vec![Arc::new(Int64Array::from_iter_values(vec![1; 1024]))], - None, - )); - let col2 = Arc::new(arrow_array::StringArray::from_iter_values(vec![ - "test"; - 1024 - ])) as ArrayRef; - let col3 = Arc::new({ - let list_parts = arrow_array::ListArray::from_iter_primitive::(vec![ - Some( - vec![Some(1),] - ); - 1024 - ]) - .into_parts(); - arrow_array::ListArray::new( - Arc::new(list_parts.0.as_ref().clone().with_metadata(HashMap::from([( - PARQUET_FIELD_ID_META_KEY.to_string(), - "6".to_string(), - )]))), - list_parts.1, - list_parts.2, - list_parts.3, - ) - }) as ArrayRef; - let col4 = Arc::new(StructArray::new( - vec![arrow_schema::Field::new( - "sub_col", - arrow_schema::DataType::Struct( - vec![arrow_schema::Field::new( - "sub_sub_col", - arrow_schema::DataType::Int64, - true, - ) - .with_metadata(HashMap::from([( - PARQUET_FIELD_ID_META_KEY.to_string(), - "7".to_string(), - )]))] - .into(), - ), - true, - ) - .with_metadata(HashMap::from([( - PARQUET_FIELD_ID_META_KEY.to_string(), - "8".to_string(), - )]))] - .into(), - vec![Arc::new(StructArray::new( - vec![ - arrow_schema::Field::new("sub_sub_col", arrow_schema::DataType::Int64, true) - .with_metadata(HashMap::from([( - PARQUET_FIELD_ID_META_KEY.to_string(), - "7".to_string(), - )])), - ] - .into(), - vec![Arc::new(Int64Array::from_iter_values(vec![1; 1024]))], - None, - ))], - None, - )); - let columns = vec![col0, col1, col2, col3, col4]; - - let equality_ids = vec![1, 3]; - let (projector, fields) = - FieldProjector::new(schema.fields(), &equality_ids, PARQUET_FIELD_ID_META_KEY)?; - let delete_schema = arrow_schema::Schema::new(fields); - let delete_schema_ref = Arc::new(delete_schema.clone()); - - // prepare writer - let to_write = RecordBatch::try_new(Arc::new(schema.clone()), columns).unwrap(); - let pb = ParquetWriterBuilder::new( - WriterProperties::builder().build(), - delete_schema_ref.clone(), - file_io.clone(), - location_gen, - file_name_gen, - ); - - let mut equality_delete_writer = EqualityDeleteFileWriterBuilder::new(pb) - .build(EqualityDeleteWriterConfig::new( - equality_ids, - projector, - delete_schema.clone(), - None, - )) - .await?; - // write - equality_delete_writer.write(to_write.clone()).await?; - let res = equality_delete_writer.close().await?; - assert_eq!(res.len(), 1); - let data_file = res.into_iter().next().unwrap(); - - // check - let to_write_projected = equality_delete_writer.project_record_batch_columns(to_write)?; - check_parquet_data_file_with_equality_delete_write( - &file_io, - &data_file, - &to_write_projected, - ) - .await; - Ok(()) - } - - #[tokio::test] - async fn test_equality_delete_float_or_double_column() -> Result<(), anyhow::Error> { - // Float32, Float64 - let schema = { - let fields = vec![ - arrow_schema::Field::new("col0", 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::Schema::new(fields) - }; - - let equality_id_float = vec![0]; - let result_float = FieldProjector::new( - schema.fields(), - &equality_id_float, - PARQUET_FIELD_ID_META_KEY, - ); - assert!(result_float.is_err()); - - let equality_ids_double = vec![1]; - let result_double = FieldProjector::new( - schema.fields(), - &equality_ids_double, - PARQUET_FIELD_ID_META_KEY, - ); - assert!(result_double.is_err()); - - Ok(()) - } -} diff --git a/crates/iceberg/src/writer/base_writer/mod.rs b/crates/iceberg/src/writer/base_writer/mod.rs index 37ab97eb6..37da2ab81 100644 --- a/crates/iceberg/src/writer/base_writer/mod.rs +++ b/crates/iceberg/src/writer/base_writer/mod.rs @@ -18,4 +18,3 @@ //! Base writer module contains the basic writer provide by iceberg: `DataFileWriter`, `PositionDeleteFileWriter`, `EqualityDeleteFileWriter`. pub mod data_file_writer; -pub mod equality_delete_writer; From 6f392e3a525fe1d08c6789e7ea9df904d80f06ba Mon Sep 17 00:00:00 2001 From: Xuanwo Date: Wed, 23 Oct 2024 15:33:03 +0800 Subject: [PATCH 22/46] ci: Fix CI for bindings python (#678) * ci: Fix CI for bindings python * Update bindings_python_ci.yml --- .github/workflows/bindings_python_ci.yml | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/.github/workflows/bindings_python_ci.yml b/.github/workflows/bindings_python_ci.yml index b5fb44ed8..d4b1aa922 100644 --- a/.github/workflows/bindings_python_ci.yml +++ b/.github/workflows/bindings_python_ci.yml @@ -45,7 +45,7 @@ jobs: - uses: actions/checkout@v4 - name: Install tools run: | - pip install ruff --break-system-packages + pip install ruff - name: Check format working-directory: "bindings/python" run: | @@ -78,6 +78,6 @@ jobs: shell: bash run: | set -e - pip install hatch==1.12.0 --break-system-packages + pip install hatch==1.12.0 hatch run dev:pip install dist/pyiceberg_core-*.whl --force-reinstall hatch run dev:test From cafc939d6aa303f7424a74fa8467ac0cb306591d Mon Sep 17 00:00:00 2001 From: Scott Donnelly Date: Fri, 25 Oct 2024 02:36:05 +0100 Subject: [PATCH 23/46] fix: OpenDAL `is_exist` => `exists` (#680) * change usage of deprecated OpenDAL `is_exist` method to its replacement, `exists` * perform similar change to `FileIO::exists` Fixes: https://github.com/apache/iceberg-rust/issues/679 --- Cargo.toml | 2 +- .../catalog/glue/tests/glue_catalog_test.rs | 2 +- crates/catalog/hms/tests/hms_catalog_test.rs | 2 +- crates/iceberg/src/io/file_io.rs | 30 ++++++++----------- crates/iceberg/src/io/mod.rs | 2 +- crates/iceberg/tests/file_io_gcs_test.rs | 9 ++---- crates/iceberg/tests/file_io_s3_test.rs | 10 +++---- 7 files changed, 24 insertions(+), 33 deletions(-) diff --git a/Cargo.toml b/Cargo.toml index 5e2b89730..bce6470d4 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -71,7 +71,7 @@ log = "0.4" mockito = "1" murmur3 = "0.5.2" once_cell = "1" -opendal = "0.50" +opendal = "0.50.1" ordered-float = "4" parquet = "53" paste = "1" diff --git a/crates/catalog/glue/tests/glue_catalog_test.rs b/crates/catalog/glue/tests/glue_catalog_test.rs index d9c5b4e0b..ebfb1f432 100644 --- a/crates/catalog/glue/tests/glue_catalog_test.rs +++ b/crates/catalog/glue/tests/glue_catalog_test.rs @@ -237,7 +237,7 @@ async fn test_create_table() -> Result<()> { assert!( catalog .file_io() - .is_exist("s3a://warehouse/hive/metadata/") + .exists("s3a://warehouse/hive/metadata/") .await? ); diff --git a/crates/catalog/hms/tests/hms_catalog_test.rs b/crates/catalog/hms/tests/hms_catalog_test.rs index 5b8004439..37a95941b 100644 --- a/crates/catalog/hms/tests/hms_catalog_test.rs +++ b/crates/catalog/hms/tests/hms_catalog_test.rs @@ -212,7 +212,7 @@ async fn test_create_table() -> Result<()> { assert!( catalog .file_io() - .is_exist("s3a://warehouse/hive/metadata/") + .exists("s3a://warehouse/hive/metadata/") .await? ); diff --git a/crates/iceberg/src/io/file_io.rs b/crates/iceberg/src/io/file_io.rs index 1ac2c1cd5..8365d622c 100644 --- a/crates/iceberg/src/io/file_io.rs +++ b/crates/iceberg/src/io/file_io.rs @@ -95,9 +95,9 @@ impl FileIO { /// # Arguments /// /// * path: It should be *absolute* path starting with scheme string used to construct [`FileIO`]. - pub async fn is_exist(&self, path: impl AsRef) -> Result { + pub async fn exists(&self, path: impl AsRef) -> Result { let (op, relative_path) = self.inner.create_operator(&path)?; - Ok(op.is_exist(relative_path).await?) + Ok(op.exists(relative_path).await?) } /// Creates input file. @@ -241,10 +241,7 @@ impl InputFile { /// Check if file exists. pub async fn exists(&self) -> crate::Result { - Ok(self - .op - .is_exist(&self.path[self.relative_path_pos..]) - .await?) + Ok(self.op.exists(&self.path[self.relative_path_pos..]).await?) } /// Fetch and returns metadata of file. @@ -323,10 +320,7 @@ impl OutputFile { /// Checks if file exists. pub async fn exists(&self) -> crate::Result { - Ok(self - .op - .is_exist(&self.path[self.relative_path_pos..]) - .await?) + Ok(self.op.exists(&self.path[self.relative_path_pos..]).await?) } /// Converts into [`InputFile`]. @@ -426,15 +420,15 @@ mod tests { write_to_file("Iceberg loves rust.", &c_path); let file_io = create_local_file_io(); - assert!(file_io.is_exist(&a_path).await.unwrap()); + assert!(file_io.exists(&a_path).await.unwrap()); file_io.remove_all(&sub_dir_path).await.unwrap(); - assert!(!file_io.is_exist(&b_path).await.unwrap()); - assert!(!file_io.is_exist(&c_path).await.unwrap()); - assert!(file_io.is_exist(&a_path).await.unwrap()); + assert!(!file_io.exists(&b_path).await.unwrap()); + assert!(!file_io.exists(&c_path).await.unwrap()); + assert!(file_io.exists(&a_path).await.unwrap()); file_io.delete(&a_path).await.unwrap(); - assert!(!file_io.is_exist(&a_path).await.unwrap()); + assert!(!file_io.exists(&a_path).await.unwrap()); } #[tokio::test] @@ -445,7 +439,7 @@ mod tests { let full_path = format!("{}/{}", tmp_dir.path().to_str().unwrap(), file_name); let file_io = create_local_file_io(); - assert!(!file_io.is_exist(&full_path).await.unwrap()); + assert!(!file_io.exists(&full_path).await.unwrap()); assert!(file_io.delete(&full_path).await.is_ok()); assert!(file_io.remove_all(&full_path).await.is_ok()); } @@ -501,12 +495,12 @@ mod tests { let output_file = io.new_output(&path).unwrap(); output_file.write("test".into()).await.unwrap(); - assert!(io.is_exist(&path.clone()).await.unwrap()); + assert!(io.exists(&path.clone()).await.unwrap()); let input_file = io.new_input(&path).unwrap(); let content = input_file.read().await.unwrap(); assert_eq!(content, Bytes::from("test")); io.delete(&path).await.unwrap(); - assert!(!io.is_exist(&path).await.unwrap()); + assert!(!io.exists(&path).await.unwrap()); } } diff --git a/crates/iceberg/src/io/mod.rs b/crates/iceberg/src/io/mod.rs index 52a1da23a..aaac734da 100644 --- a/crates/iceberg/src/io/mod.rs +++ b/crates/iceberg/src/io/mod.rs @@ -62,7 +62,7 @@ //! Currently `FileIO` provides simple methods for file operations: //! //! - `delete`: Delete file. -//! - `is_exist`: Check if file exists. +//! - `exists`: Check if file exists. //! - `new_input`: Create input file for reading. //! - `new_output`: Create output file for writing. diff --git a/crates/iceberg/tests/file_io_gcs_test.rs b/crates/iceberg/tests/file_io_gcs_test.rs index 540cd9d99..f78465571 100644 --- a/crates/iceberg/tests/file_io_gcs_test.rs +++ b/crates/iceberg/tests/file_io_gcs_test.rs @@ -93,10 +93,7 @@ mod tests { #[tokio::test] async fn gcs_exists() { let file_io = get_file_io_gcs().await; - assert!(file_io - .is_exist(format!("{}/", get_gs_path())) - .await - .unwrap()); + assert!(file_io.exists(format!("{}/", get_gs_path())).await.unwrap()); } #[tokio::test] @@ -108,7 +105,7 @@ mod tests { .write(bytes::Bytes::from_static(b"iceberg-gcs!")) .await .expect("Write to test output file"); - assert!(file_io.is_exist(gs_file).await.unwrap()) + assert!(file_io.exists(gs_file).await.unwrap()) } #[tokio::test] @@ -120,7 +117,7 @@ mod tests { .write(bytes::Bytes::from_static(b"iceberg!")) .await .expect("Write to test output file"); - assert!(file_io.is_exist(&gs_file).await.unwrap()); + assert!(file_io.exists(&gs_file).await.unwrap()); let input = file_io.new_input(gs_file).unwrap(); assert_eq!(input.read().await.unwrap(), Bytes::from_static(b"iceberg!")); diff --git a/crates/iceberg/tests/file_io_s3_test.rs b/crates/iceberg/tests/file_io_s3_test.rs index 32e2d12a4..22b798fc0 100644 --- a/crates/iceberg/tests/file_io_s3_test.rs +++ b/crates/iceberg/tests/file_io_s3_test.rs @@ -68,21 +68,21 @@ mod tests { } #[tokio::test] - async fn test_file_io_s3_is_exist() { + async fn test_file_io_s3_exists() { let file_io = get_file_io().await; - assert!(!file_io.is_exist("s3://bucket2/any").await.unwrap()); - assert!(file_io.is_exist("s3://bucket1/").await.unwrap()); + assert!(!file_io.exists("s3://bucket2/any").await.unwrap()); + assert!(file_io.exists("s3://bucket1/").await.unwrap()); } #[tokio::test] async fn test_file_io_s3_output() { let file_io = get_file_io().await; - assert!(!file_io.is_exist("s3://bucket1/test_output").await.unwrap()); + assert!(!file_io.exists("s3://bucket1/test_output").await.unwrap()); let output_file = file_io.new_output("s3://bucket1/test_output").unwrap(); { output_file.write("123".into()).await.unwrap(); } - assert!(file_io.is_exist("s3://bucket1/test_output").await.unwrap()); + assert!(file_io.exists("s3://bucket1/test_output").await.unwrap()); } #[tokio::test] From 0c44e50dc08f44601ea2530311aed5222a810f87 Mon Sep 17 00:00:00 2001 From: Zachary Heilbron Date: Thu, 24 Oct 2024 23:09:41 -1000 Subject: [PATCH 24/46] feat: Expose ManifestEntry status (#681) Co-authored-by: Renjie Liu --- crates/iceberg/src/spec/manifest.rs | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/crates/iceberg/src/spec/manifest.rs b/crates/iceberg/src/spec/manifest.rs index f0dfdf47c..997908d5f 100644 --- a/crates/iceberg/src/spec/manifest.rs +++ b/crates/iceberg/src/spec/manifest.rs @@ -850,6 +850,11 @@ impl ManifestEntry { ) } + /// Status of this manifest entry + pub fn status(&self) -> ManifestStatus { + self.status + } + /// Content type of this manifest entry. #[inline] pub fn content_type(&self) -> DataContentType { From 11e36c0ae635aac57471f82f9e7e0c12e587aa22 Mon Sep 17 00:00:00 2001 From: sundyli <543950155@qq.com> Date: Fri, 25 Oct 2024 17:40:20 +0800 Subject: [PATCH 25/46] feat: allow empty projection in table scan (#677) * fix: allow empty projection in scan * fix: allow empty projection in scan * fix: pub get_manifest_list * Update crates/iceberg/src/scan.rs Co-authored-by: Renjie Liu * chore: remove pub --------- Co-authored-by: Renjie Liu --- .../src/arrow/record_batch_transformer.rs | 29 ++++----- crates/iceberg/src/scan.rs | 63 ++++++++++++++----- 2 files changed, 57 insertions(+), 35 deletions(-) diff --git a/crates/iceberg/src/arrow/record_batch_transformer.rs b/crates/iceberg/src/arrow/record_batch_transformer.rs index 01ce9f0a8..216e68ead 100644 --- a/crates/iceberg/src/arrow/record_batch_transformer.rs +++ b/crates/iceberg/src/arrow/record_batch_transformer.rs @@ -20,7 +20,7 @@ use std::sync::Arc; use arrow_array::{ Array as ArrowArray, ArrayRef, BinaryArray, BooleanArray, Float32Array, Float64Array, - Int32Array, Int64Array, NullArray, RecordBatch, StringArray, + Int32Array, Int64Array, NullArray, RecordBatch, RecordBatchOptions, StringArray, }; use arrow_cast::cast; use arrow_schema::{ @@ -124,19 +124,7 @@ impl RecordBatchTransformer { snapshot_schema: Arc, projected_iceberg_field_ids: &[i32], ) -> Self { - let projected_iceberg_field_ids = if projected_iceberg_field_ids.is_empty() { - // If the list of field ids is empty, this indicates that we - // need to select all fields. - // Project all fields in table schema order - snapshot_schema - .as_struct() - .fields() - .iter() - .map(|field| field.id) - .collect() - } else { - projected_iceberg_field_ids.to_vec() - }; + let projected_iceberg_field_ids = projected_iceberg_field_ids.to_vec(); Self { snapshot_schema, @@ -154,10 +142,15 @@ impl RecordBatchTransformer { Some(BatchTransform::Modify { ref target_schema, ref operations, - }) => RecordBatch::try_new( - target_schema.clone(), - self.transform_columns(record_batch.columns(), operations)?, - )?, + }) => { + let options = + RecordBatchOptions::default().with_row_count(Some(record_batch.num_rows())); + RecordBatch::try_new_with_options( + target_schema.clone(), + self.transform_columns(record_batch.columns(), operations)?, + &options, + )? + } Some(BatchTransform::ModifySchema { target_schema }) => { record_batch.with_schema(target_schema.clone())? } diff --git a/crates/iceberg/src/scan.rs b/crates/iceberg/src/scan.rs index b7fa62d66..d03554541 100644 --- a/crates/iceberg/src/scan.rs +++ b/crates/iceberg/src/scan.rs @@ -51,8 +51,8 @@ pub type ArrowRecordBatchStream = BoxStream<'static, Result>; /// Builder to create table scan. pub struct TableScanBuilder<'a> { table: &'a Table, - // Empty column names means to select all columns - column_names: Vec, + // Defaults to none which means select all columns + column_names: Option>, snapshot_id: Option, batch_size: Option, case_sensitive: bool, @@ -70,7 +70,7 @@ impl<'a> TableScanBuilder<'a> { Self { table, - column_names: vec![], + column_names: None, snapshot_id: None, batch_size: None, case_sensitive: true, @@ -106,16 +106,24 @@ impl<'a> TableScanBuilder<'a> { /// Select all columns. pub fn select_all(mut self) -> Self { - self.column_names.clear(); + self.column_names = None; + self + } + + /// Select empty columns. + pub fn select_empty(mut self) -> Self { + self.column_names = Some(vec![]); self } /// Select some columns of the table. pub fn select(mut self, column_names: impl IntoIterator) -> Self { - self.column_names = column_names - .into_iter() - .map(|item| item.to_string()) - .collect(); + self.column_names = Some( + column_names + .into_iter() + .map(|item| item.to_string()) + .collect(), + ); self } @@ -205,8 +213,8 @@ impl<'a> TableScanBuilder<'a> { let schema = snapshot.schema(self.table.metadata())?; // Check that all column names exist in the schema. - if !self.column_names.is_empty() { - for column_name in &self.column_names { + if let Some(column_names) = self.column_names.as_ref() { + for column_name in column_names { if schema.field_by_name(column_name).is_none() { return Err(Error::new( ErrorKind::DataInvalid, @@ -220,7 +228,16 @@ impl<'a> TableScanBuilder<'a> { } let mut field_ids = vec![]; - for column_name in &self.column_names { + let column_names = self.column_names.clone().unwrap_or_else(|| { + schema + .as_struct() + .fields() + .iter() + .map(|f| f.name.clone()) + .collect() + }); + + for column_name in column_names.iter() { let field_id = schema.field_id_by_name(column_name).ok_or_else(|| { Error::new( ErrorKind::DataInvalid, @@ -297,7 +314,7 @@ pub struct TableScan { plan_context: PlanContext, batch_size: Option, file_io: FileIO, - column_names: Vec, + column_names: Option>, /// The maximum number of manifest files that will be /// retrieved from [`FileIO`] concurrently concurrency_limit_manifest_files: usize, @@ -409,9 +426,10 @@ impl TableScan { } /// Returns a reference to the column names of the table scan. - pub fn column_names(&self) -> &[String] { - &self.column_names + pub fn column_names(&self) -> Option<&[String]> { + self.column_names.as_deref() } + /// Returns a reference to the snapshot of the table scan. pub fn snapshot(&self) -> &SnapshotRef { &self.plan_context.snapshot @@ -1236,7 +1254,10 @@ mod tests { let table = TableTestFixture::new().table; let table_scan = table.scan().select(["x", "y"]).build().unwrap(); - assert_eq!(vec!["x", "y"], table_scan.column_names); + assert_eq!( + Some(vec!["x".to_string(), "y".to_string()]), + table_scan.column_names + ); let table_scan = table .scan() @@ -1244,7 +1265,7 @@ mod tests { .select(["z"]) .build() .unwrap(); - assert_eq!(vec!["z"], table_scan.column_names); + assert_eq!(Some(vec!["z".to_string()]), table_scan.column_names); } #[test] @@ -1252,7 +1273,7 @@ mod tests { let table = TableTestFixture::new().table; let table_scan = table.scan().select_all().build().unwrap(); - assert!(table_scan.column_names.is_empty()); + assert!(table_scan.column_names.is_none()); } #[test] @@ -1424,6 +1445,14 @@ mod tests { let col2 = batches[0].column_by_name("z").unwrap(); let int64_arr = col2.as_any().downcast_ref::().unwrap(); assert_eq!(int64_arr.value(0), 3); + + // test empty scan + let table_scan = fixture.table.scan().select_empty().build().unwrap(); + let batch_stream = table_scan.to_arrow().await.unwrap(); + let batches: Vec<_> = batch_stream.try_collect().await.unwrap(); + + assert_eq!(batches[0].num_columns(), 0); + assert_eq!(batches[0].num_rows(), 1024); } #[tokio::test] From 2f3554b30a05028aef1d46d37209e24e1ed96582 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Mon, 28 Oct 2024 12:59:16 +0800 Subject: [PATCH 26/46] chore(deps): Bump crate-ci/typos from 1.26.0 to 1.26.8 (#683) Bumps [crate-ci/typos](https://github.com/crate-ci/typos) from 1.26.0 to 1.26.8. - [Release notes](https://github.com/crate-ci/typos/releases) - [Changelog](https://github.com/crate-ci/typos/blob/master/CHANGELOG.md) - [Commits](https://github.com/crate-ci/typos/compare/v1.26.0...v1.26.8) --- updated-dependencies: - dependency-name: crate-ci/typos dependency-type: direct:production update-type: version-update:semver-patch ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- .github/workflows/ci_typos.yml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/.github/workflows/ci_typos.yml b/.github/workflows/ci_typos.yml index 9d49fe2ee..c086f0a3b 100644 --- a/.github/workflows/ci_typos.yml +++ b/.github/workflows/ci_typos.yml @@ -42,4 +42,4 @@ jobs: steps: - uses: actions/checkout@v4 - name: Check typos - uses: crate-ci/typos@v1.26.0 + uses: crate-ci/typos@v1.26.8 From 774002d71f816a37b52b828babf3e5face430bc9 Mon Sep 17 00:00:00 2001 From: xxchan Date: Thu, 31 Oct 2024 14:58:35 +0800 Subject: [PATCH 27/46] fix: bump parquet minor version (#684) --- Cargo.toml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/Cargo.toml b/Cargo.toml index bce6470d4..efff593b3 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -73,7 +73,7 @@ murmur3 = "0.5.2" once_cell = "1" opendal = "0.50.1" ordered-float = "4" -parquet = "53" +parquet = "53.1" paste = "1" pilota = "0.11.2" pretty_assertions = "1.4" From 61af83b16b86b37143093ffcfe76d8e775f2363c Mon Sep 17 00:00:00 2001 From: Dylan Date: Fri, 1 Nov 2024 17:42:04 +0800 Subject: [PATCH 28/46] fix type promote (#685) --- crates/iceberg/src/arrow/record_batch_transformer.rs | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/crates/iceberg/src/arrow/record_batch_transformer.rs b/crates/iceberg/src/arrow/record_batch_transformer.rs index 216e68ead..9a4dd1784 100644 --- a/crates/iceberg/src/arrow/record_batch_transformer.rs +++ b/crates/iceberg/src/arrow/record_batch_transformer.rs @@ -143,8 +143,9 @@ impl RecordBatchTransformer { ref target_schema, ref operations, }) => { - let options = - RecordBatchOptions::default().with_row_count(Some(record_batch.num_rows())); + let options = RecordBatchOptions::default() + .with_match_field_names(false) + .with_row_count(Some(record_batch.num_rows())); RecordBatch::try_new_with_options( target_schema.clone(), self.transform_columns(record_batch.columns(), operations)?, From b8f088eed6c79f4ec7e81b4b5f9df2aeb2663473 Mon Sep 17 00:00:00 2001 From: "yujie.zhang (he/him)" Date: Fri, 1 Nov 2024 21:04:41 +0800 Subject: [PATCH 29/46] feat: implement IcebergTableProviderFactory for datafusion (#600) * feat: implement IcebergTableProviderFactory for datafusion * fix comments * add doc&ut * remove print * fix comments --- crates/integrations/datafusion/src/lib.rs | 1 + .../datafusion/src/{table.rs => table/mod.rs} | 5 + .../src/table/table_provider_factory.rs | 300 ++++++++++++++++++ .../table_metadata/TableMetadataV2.json | 121 +++++++ 4 files changed, 427 insertions(+) rename crates/integrations/datafusion/src/{table.rs => table/mod.rs} (97%) create mode 100644 crates/integrations/datafusion/src/table/table_provider_factory.rs create mode 100644 crates/integrations/datafusion/testdata/table_metadata/TableMetadataV2.json diff --git a/crates/integrations/datafusion/src/lib.rs b/crates/integrations/datafusion/src/lib.rs index b64f8fb8e..b7b927fdd 100644 --- a/crates/integrations/datafusion/src/lib.rs +++ b/crates/integrations/datafusion/src/lib.rs @@ -24,4 +24,5 @@ pub use error::*; mod physical_plan; mod schema; mod table; +pub use table::table_provider_factory::IcebergTableProviderFactory; pub use table::*; diff --git a/crates/integrations/datafusion/src/table.rs b/crates/integrations/datafusion/src/table/mod.rs similarity index 97% rename from crates/integrations/datafusion/src/table.rs rename to crates/integrations/datafusion/src/table/mod.rs index bb24713aa..82f29bb52 100644 --- a/crates/integrations/datafusion/src/table.rs +++ b/crates/integrations/datafusion/src/table/mod.rs @@ -15,6 +15,8 @@ // specific language governing permissions and limitations // under the License. +pub mod table_provider_factory; + use std::any::Any; use std::sync::Arc; @@ -41,6 +43,9 @@ pub struct IcebergTableProvider { } impl IcebergTableProvider { + pub(crate) fn new(table: Table, schema: ArrowSchemaRef) -> Self { + IcebergTableProvider { table, schema } + } /// Asynchronously tries to construct a new [`IcebergTableProvider`] /// using the given client and table name to fetch an actual [`Table`] /// in the provided namespace. diff --git a/crates/integrations/datafusion/src/table/table_provider_factory.rs b/crates/integrations/datafusion/src/table/table_provider_factory.rs new file mode 100644 index 000000000..b8e66bd30 --- /dev/null +++ b/crates/integrations/datafusion/src/table/table_provider_factory.rs @@ -0,0 +1,300 @@ +// 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. + +use std::borrow::Cow; +use std::collections::HashMap; +use std::sync::Arc; + +use async_trait::async_trait; +use datafusion::catalog::{Session, TableProvider, TableProviderFactory}; +use datafusion::error::Result as DFResult; +use datafusion::logical_expr::CreateExternalTable; +use datafusion::sql::TableReference; +use iceberg::arrow::schema_to_arrow_schema; +use iceberg::io::FileIO; +use iceberg::table::StaticTable; +use iceberg::{Error, ErrorKind, Result, TableIdent}; + +use super::IcebergTableProvider; +use crate::to_datafusion_error; + +/// A factory that implements DataFusion's `TableProviderFactory` to create `IcebergTableProvider` instances. +/// +/// # Example +/// +/// The following example demonstrates how to create an Iceberg external table using SQL in +/// a DataFusion session with `IcebergTableProviderFactory`: +/// +/// ``` +/// use std::sync::Arc; +/// +/// use datafusion::execution::session_state::SessionStateBuilder; +/// use datafusion::prelude::*; +/// use datafusion::sql::TableReference; +/// use iceberg_datafusion::IcebergTableProviderFactory; +/// +/// #[tokio::main] +/// async fn main() { +/// // Create a new session context +/// let mut state = SessionStateBuilder::new().with_default_features().build(); +/// +/// // Register the IcebergTableProviderFactory in the session +/// state.table_factories_mut().insert( +/// "ICEBERG".to_string(), +/// Arc::new(IcebergTableProviderFactory::new()), +/// ); +/// +/// let ctx = SessionContext::new_with_state(state); +/// +/// // Define the table reference and the location of the Iceberg metadata file +/// let table_ref = TableReference::bare("my_iceberg_table"); +/// // /path/to/iceberg/metadata +/// let metadata_file_path = format!( +/// "{}/testdata/table_metadata/{}", +/// env!("CARGO_MANIFEST_DIR"), +/// "TableMetadataV2.json" +/// ); +/// +/// // SQL command to create the Iceberg external table +/// let sql = format!( +/// "CREATE EXTERNAL TABLE {} STORED AS ICEBERG LOCATION '{}'", +/// table_ref, metadata_file_path +/// ); +/// +/// // Execute the SQL to create the external table +/// ctx.sql(&sql).await.expect("Failed to create table"); +/// +/// // Verify the table was created by retrieving the table provider +/// let table_provider = ctx +/// .table_provider(table_ref) +/// .await +/// .expect("Table not found"); +/// +/// println!("Iceberg external table created successfully."); +/// } +/// ``` +/// +/// # Note +/// This factory is designed to work with the DataFusion query engine, +/// specifically for handling Iceberg tables in external table commands. +/// Currently, this implementation supports only reading Iceberg tables, with +/// the creation of new tables not yet available. +/// +/// # Errors +/// An error will be returned if any unsupported feature, such as partition columns, +/// order expressions, constraints, or column defaults, is detected in the table creation command. +#[derive(Default)] +pub struct IcebergTableProviderFactory {} + +impl IcebergTableProviderFactory { + pub fn new() -> Self { + Self {} + } +} + +#[async_trait] +impl TableProviderFactory for IcebergTableProviderFactory { + async fn create( + &self, + _state: &dyn Session, + cmd: &CreateExternalTable, + ) -> DFResult> { + check_cmd(cmd).map_err(to_datafusion_error)?; + + let table_name = &cmd.name; + let metadata_file_path = &cmd.location; + let options = &cmd.options; + + let table_name_with_ns = complement_namespace_if_necessary(table_name); + + let table = create_static_table(table_name_with_ns, metadata_file_path, options) + .await + .map_err(to_datafusion_error)? + .into_table(); + + let schema = schema_to_arrow_schema(table.metadata().current_schema()) + .map_err(to_datafusion_error)?; + + Ok(Arc::new(IcebergTableProvider::new(table, Arc::new(schema)))) + } +} + +fn check_cmd(cmd: &CreateExternalTable) -> Result<()> { + let CreateExternalTable { + schema, + table_partition_cols, + order_exprs, + constraints, + column_defaults, + .. + } = cmd; + + // Check if any of the fields violate the constraints in a single condition + let is_invalid = !schema.fields().is_empty() + || !table_partition_cols.is_empty() + || !order_exprs.is_empty() + || !constraints.is_empty() + || !column_defaults.is_empty(); + + if is_invalid { + return Err(Error::new(ErrorKind::FeatureUnsupported, "Currently we only support reading existing icebergs tables in external table command. To create new table, please use catalog provider.")); + } + + Ok(()) +} + +/// Complements the namespace of a table name if necessary. +/// +/// # Note +/// If the table name is a bare name, it will be complemented with the 'default' namespace. +/// Otherwise, it will be returned as is. Because Iceberg tables are always namespaced, but DataFusion +/// external table commands maybe not include the namespace, this function ensures that the namespace is always present. +/// +/// # See also +/// - [`iceberg::NamespaceIdent`] +/// - [`datafusion::sql::planner::SqlToRel::external_table_to_plan`] +fn complement_namespace_if_necessary(table_name: &TableReference) -> Cow<'_, TableReference> { + match table_name { + TableReference::Bare { table } => { + Cow::Owned(TableReference::partial("default", table.as_ref())) + } + other => Cow::Borrowed(other), + } +} + +async fn create_static_table( + table_name: Cow<'_, TableReference>, + metadata_file_path: &str, + props: &HashMap, +) -> Result { + let table_ident = TableIdent::from_strs(table_name.to_vec())?; + let file_io = FileIO::from_path(metadata_file_path)? + .with_props(props) + .build()?; + StaticTable::from_metadata_file(metadata_file_path, table_ident, file_io).await +} + +#[cfg(test)] +mod tests { + + use datafusion::arrow::datatypes::{DataType, Field, Schema}; + use datafusion::catalog::TableProviderFactory; + use datafusion::common::{Constraints, DFSchema}; + use datafusion::execution::session_state::SessionStateBuilder; + use datafusion::logical_expr::CreateExternalTable; + use datafusion::parquet::arrow::PARQUET_FIELD_ID_META_KEY; + use datafusion::prelude::SessionContext; + use datafusion::sql::TableReference; + + use super::*; + + fn table_metadata_v2_schema() -> Schema { + Schema::new(vec![ + Field::new("x", DataType::Int64, false).with_metadata(HashMap::from([( + PARQUET_FIELD_ID_META_KEY.to_string(), + "1".to_string(), + )])), + Field::new("y", DataType::Int64, false).with_metadata(HashMap::from([( + PARQUET_FIELD_ID_META_KEY.to_string(), + "2".to_string(), + )])), + Field::new("z", DataType::Int64, false).with_metadata(HashMap::from([( + PARQUET_FIELD_ID_META_KEY.to_string(), + "3".to_string(), + )])), + ]) + } + + fn table_metadata_location() -> String { + format!( + "{}/testdata/table_metadata/{}", + env!("CARGO_MANIFEST_DIR"), + "TableMetadataV2.json" + ) + } + + fn create_external_table_cmd() -> CreateExternalTable { + let metadata_file_path = table_metadata_location(); + + CreateExternalTable { + name: TableReference::partial("static_ns", "static_table"), + location: metadata_file_path, + schema: Arc::new(DFSchema::empty()), + file_type: "iceberg".to_string(), + options: Default::default(), + table_partition_cols: Default::default(), + order_exprs: Default::default(), + constraints: Constraints::empty(), + column_defaults: Default::default(), + if_not_exists: Default::default(), + definition: Default::default(), + unbounded: Default::default(), + } + } + + #[tokio::test] + async fn test_schema_of_created_table() { + let factory = IcebergTableProviderFactory::new(); + + let state = SessionStateBuilder::new().build(); + let cmd = create_external_table_cmd(); + + let table_provider = factory + .create(&state, &cmd) + .await + .expect("create table failed"); + + let expected_schema = table_metadata_v2_schema(); + let actual_schema = table_provider.schema(); + + assert_eq!(actual_schema.as_ref(), &expected_schema); + } + + #[tokio::test] + async fn test_schema_of_created_external_table_sql() { + let mut state = SessionStateBuilder::new().with_default_features().build(); + state.table_factories_mut().insert( + "ICEBERG".to_string(), + Arc::new(IcebergTableProviderFactory::new()), + ); + let ctx = SessionContext::new_with_state(state); + + // All external tables in DataFusion use bare names. + // See https://github.com/apache/datafusion/blob/main/datafusion/sql/src/statement.rs#L1038-#L1039 + let table_ref = TableReference::bare("static_table"); + + // Create the external table + let sql = format!( + "CREATE EXTERNAL TABLE {} STORED AS ICEBERG LOCATION '{}'", + table_ref, + table_metadata_location() + ); + let _df = ctx.sql(&sql).await.expect("create table failed"); + + // Get the created external table + let table_provider = ctx + .table_provider(table_ref) + .await + .expect("table not found"); + + // Check the schema of the created table + let expected_schema = table_metadata_v2_schema(); + let actual_schema = table_provider.schema(); + + assert_eq!(actual_schema.as_ref(), &expected_schema); + } +} diff --git a/crates/integrations/datafusion/testdata/table_metadata/TableMetadataV2.json b/crates/integrations/datafusion/testdata/table_metadata/TableMetadataV2.json new file mode 100644 index 000000000..a7b47217f --- /dev/null +++ b/crates/integrations/datafusion/testdata/table_metadata/TableMetadataV2.json @@ -0,0 +1,121 @@ +{ + "format-version": 2, + "table-uuid": "9c12d441-03fe-4693-9a96-a0705ddf69c1", + "location": "s3://bucket/test/location", + "last-sequence-number": 34, + "last-updated-ms": 1602638573590, + "last-column-id": 3, + "current-schema-id": 1, + "schemas": [ + { + "type": "struct", + "schema-id": 0, + "fields": [ + { + "id": 1, + "name": "x", + "required": true, + "type": "long" + } + ] + }, + { + "type": "struct", + "schema-id": 1, + "identifier-field-ids": [ + 1, + 2 + ], + "fields": [ + { + "id": 1, + "name": "x", + "required": true, + "type": "long" + }, + { + "id": 2, + "name": "y", + "required": true, + "type": "long" + }, + { + "id": 3, + "name": "z", + "required": true, + "type": "long" + } + ] + } + ], + "default-spec-id": 0, + "partition-specs": [ + { + "spec-id": 0, + "fields": [ + { + "name": "x", + "transform": "identity", + "source-id": 1, + "field-id": 1000 + } + ] + } + ], + "last-partition-id": 1000, + "default-sort-order-id": 3, + "sort-orders": [ + { + "order-id": 3, + "fields": [ + { + "transform": "identity", + "source-id": 2, + "direction": "asc", + "null-order": "nulls-first" + }, + { + "transform": "bucket[4]", + "source-id": 3, + "direction": "desc", + "null-order": "nulls-last" + } + ] + } + ], + "properties": {}, + "current-snapshot-id": 3055729675574597004, + "snapshots": [ + { + "snapshot-id": 3051729675574597004, + "timestamp-ms": 1515100955770, + "sequence-number": 0, + "summary": { + "operation": "append" + }, + "manifest-list": "s3://a/b/1.avro" + }, + { + "snapshot-id": 3055729675574597004, + "parent-snapshot-id": 3051729675574597004, + "timestamp-ms": 1555100955770, + "sequence-number": 1, + "summary": { + "operation": "append" + }, + "manifest-list": "s3://a/b/2.avro", + "schema-id": 1 + } + ], + "snapshot-log": [ + { + "snapshot-id": 3051729675574597004, + "timestamp-ms": 1515100955770 + }, + { + "snapshot-id": 3055729675574597004, + "timestamp-ms": 1555100955770 + } + ], + "metadata-log": [] +} \ No newline at end of file From 6e0bcf56028e0d20d5ceeedf87dbb3db7c929ee3 Mon Sep 17 00:00:00 2001 From: Christian Date: Sat, 2 Nov 2024 21:10:03 -0700 Subject: [PATCH 30/46] feat: Safer PartitionSpec & SchemalessPartitionSpec (#645) * SchemalessPartitionSpec * Traits -> Enum * Remove PartitionSpec enum * Address comments --- crates/catalog/memory/src/catalog.rs | 6 +- crates/catalog/sql/src/catalog.rs | 7 +- .../src/expr/visitors/expression_evaluator.rs | 136 +++-- .../visitors/inclusive_metrics_evaluator.rs | 13 +- .../src/expr/visitors/inclusive_projection.rs | 155 ++++-- crates/iceberg/src/io/object_cache.rs | 2 +- crates/iceberg/src/scan.rs | 8 +- crates/iceberg/src/spec/manifest.rs | 516 +++++++++--------- crates/iceberg/src/spec/partition.rs | 433 +++++++++++---- crates/iceberg/src/spec/table_metadata.rs | 315 +++++++---- .../writer/file_writer/location_generator.rs | 5 +- 11 files changed, 969 insertions(+), 627 deletions(-) diff --git a/crates/catalog/memory/src/catalog.rs b/crates/catalog/memory/src/catalog.rs index 1da044821..e4192aae0 100644 --- a/crates/catalog/memory/src/catalog.rs +++ b/crates/catalog/memory/src/catalog.rs @@ -283,7 +283,7 @@ mod tests { use std::iter::FromIterator; use iceberg::io::FileIOBuilder; - use iceberg::spec::{NestedField, PartitionSpec, PrimitiveType, Schema, SortOrder, Type}; + use iceberg::spec::{BoundPartitionSpec, NestedField, PrimitiveType, Schema, SortOrder, Type}; use regex::Regex; use tempfile::TempDir; @@ -355,7 +355,7 @@ mod tests { assert_eq!(metadata.current_schema().as_ref(), expected_schema); - let expected_partition_spec = PartitionSpec::builder(expected_schema) + let expected_partition_spec = BoundPartitionSpec::builder((*expected_schema).clone()) .with_spec_id(0) .build() .unwrap(); @@ -365,7 +365,7 @@ mod tests { .partition_specs_iter() .map(|p| p.as_ref()) .collect_vec(), - vec![&expected_partition_spec] + vec![&expected_partition_spec.into_schemaless()] ); let expected_sorted_order = SortOrder::builder() diff --git a/crates/catalog/sql/src/catalog.rs b/crates/catalog/sql/src/catalog.rs index b7976d9d5..abf22ffde 100644 --- a/crates/catalog/sql/src/catalog.rs +++ b/crates/catalog/sql/src/catalog.rs @@ -781,7 +781,7 @@ mod tests { use std::hash::Hash; use iceberg::io::FileIOBuilder; - use iceberg::spec::{NestedField, PartitionSpec, PrimitiveType, Schema, SortOrder, Type}; + use iceberg::spec::{BoundPartitionSpec, NestedField, PrimitiveType, Schema, SortOrder, Type}; use iceberg::table::Table; use iceberg::{Catalog, Namespace, NamespaceIdent, TableCreation, TableIdent}; use itertools::Itertools; @@ -874,10 +874,11 @@ mod tests { assert_eq!(metadata.current_schema().as_ref(), expected_schema); - let expected_partition_spec = PartitionSpec::builder(expected_schema) + let expected_partition_spec = BoundPartitionSpec::builder(expected_schema.clone()) .with_spec_id(0) .build() - .unwrap(); + .unwrap() + .into_schemaless(); assert_eq!( metadata diff --git a/crates/iceberg/src/expr/visitors/expression_evaluator.rs b/crates/iceberg/src/expr/visitors/expression_evaluator.rs index 8f3c2971c..2add5761f 100644 --- a/crates/iceberg/src/expr/visitors/expression_evaluator.rs +++ b/crates/iceberg/src/expr/visitors/expression_evaluator.rs @@ -258,15 +258,13 @@ mod tests { UnaryExpression, }; use crate::spec::{ - DataContentType, DataFile, DataFileFormat, Datum, Literal, NestedField, PartitionSpec, - PartitionSpecRef, PrimitiveType, Schema, SchemaRef, Struct, Transform, Type, + BoundPartitionSpec, BoundPartitionSpecRef, DataContentType, DataFile, DataFileFormat, + Datum, Literal, NestedField, PrimitiveType, Schema, Struct, Transform, Type, UnboundPartitionField, }; use crate::Result; - fn create_schema_and_partition_spec( - r#type: PrimitiveType, - ) -> Result<(SchemaRef, PartitionSpecRef)> { + fn create_partition_spec(r#type: PrimitiveType) -> Result { let schema = Schema::builder() .with_fields(vec![Arc::new(NestedField::optional( 1, @@ -275,7 +273,7 @@ mod tests { ))]) .build()?; - let spec = PartitionSpec::builder(&schema) + let spec = BoundPartitionSpec::builder(schema.clone()) .with_spec_id(1) .add_unbound_fields(vec![UnboundPartitionField::builder() .source_id(1) @@ -287,16 +285,15 @@ mod tests { .build() .unwrap(); - Ok((Arc::new(schema), Arc::new(spec))) + Ok(Arc::new(spec)) } fn create_partition_filter( - schema: &Schema, - partition_spec: PartitionSpecRef, + partition_spec: BoundPartitionSpecRef, predicate: &BoundPredicate, case_sensitive: bool, ) -> Result { - let partition_type = partition_spec.partition_type(schema)?; + let partition_type = partition_spec.partition_type(); let partition_fields = partition_type.fields().to_owned(); let partition_schema = Schema::builder() @@ -304,7 +301,8 @@ mod tests { .with_fields(partition_fields) .build()?; - let mut inclusive_projection = InclusiveProjection::new(partition_spec); + let mut inclusive_projection = + InclusiveProjection::new((*partition_spec).clone().into_schemaless().into()); let partition_filter = inclusive_projection .project(predicate)? @@ -315,13 +313,11 @@ mod tests { } fn create_expression_evaluator( - schema: &Schema, - partition_spec: PartitionSpecRef, + partition_spec: BoundPartitionSpecRef, predicate: &BoundPredicate, case_sensitive: bool, ) -> Result { - let partition_filter = - create_partition_filter(schema, partition_spec, predicate, case_sensitive)?; + let partition_filter = create_partition_filter(partition_spec, predicate, case_sensitive)?; Ok(ExpressionEvaluator::new(partition_filter)) } @@ -375,7 +371,7 @@ mod tests { #[test] fn test_expr_or() -> Result<()> { let case_sensitive = true; - let (schema, partition_spec) = create_schema_and_partition_spec(PrimitiveType::Float)?; + let partition_spec = create_partition_spec(PrimitiveType::Float)?; let predicate = Predicate::Binary(BinaryExpression::new( PredicateOperator::LessThan, @@ -387,10 +383,10 @@ mod tests { Reference::new("a"), Datum::float(0.4), ))) - .bind(schema.clone(), case_sensitive)?; + .bind(partition_spec.schema_ref().clone(), case_sensitive)?; let expression_evaluator = - create_expression_evaluator(&schema, partition_spec, &predicate, case_sensitive)?; + create_expression_evaluator(partition_spec, &predicate, case_sensitive)?; let data_file = create_data_file_float(); @@ -404,7 +400,7 @@ mod tests { #[test] fn test_expr_and() -> Result<()> { let case_sensitive = true; - let (schema, partition_spec) = create_schema_and_partition_spec(PrimitiveType::Float)?; + let partition_spec = create_partition_spec(PrimitiveType::Float)?; let predicate = Predicate::Binary(BinaryExpression::new( PredicateOperator::LessThan, @@ -416,10 +412,10 @@ mod tests { Reference::new("a"), Datum::float(0.4), ))) - .bind(schema.clone(), case_sensitive)?; + .bind(partition_spec.schema_ref().clone(), case_sensitive)?; let expression_evaluator = - create_expression_evaluator(&schema, partition_spec, &predicate, case_sensitive)?; + create_expression_evaluator(partition_spec, &predicate, case_sensitive)?; let data_file = create_data_file_float(); @@ -433,17 +429,17 @@ mod tests { #[test] fn test_expr_not_in() -> Result<()> { let case_sensitive = true; - let (schema, partition_spec) = create_schema_and_partition_spec(PrimitiveType::Float)?; + let partition_spec = create_partition_spec(PrimitiveType::Float)?; let predicate = Predicate::Set(SetExpression::new( PredicateOperator::NotIn, Reference::new("a"), FnvHashSet::from_iter([Datum::float(0.9), Datum::float(1.2), Datum::float(2.4)]), )) - .bind(schema.clone(), case_sensitive)?; + .bind(partition_spec.schema_ref().clone(), case_sensitive)?; let expression_evaluator = - create_expression_evaluator(&schema, partition_spec, &predicate, case_sensitive)?; + create_expression_evaluator(partition_spec, &predicate, case_sensitive)?; let data_file = create_data_file_float(); @@ -457,17 +453,17 @@ mod tests { #[test] fn test_expr_in() -> Result<()> { let case_sensitive = true; - let (schema, partition_spec) = create_schema_and_partition_spec(PrimitiveType::Float)?; + let partition_spec = create_partition_spec(PrimitiveType::Float)?; let predicate = Predicate::Set(SetExpression::new( PredicateOperator::In, Reference::new("a"), FnvHashSet::from_iter([Datum::float(1.0), Datum::float(1.2), Datum::float(2.4)]), )) - .bind(schema.clone(), case_sensitive)?; + .bind(partition_spec.schema_ref().clone(), case_sensitive)?; let expression_evaluator = - create_expression_evaluator(&schema, partition_spec, &predicate, case_sensitive)?; + create_expression_evaluator(partition_spec, &predicate, case_sensitive)?; let data_file = create_data_file_float(); @@ -481,17 +477,17 @@ mod tests { #[test] fn test_expr_not_starts_with() -> Result<()> { let case_sensitive = true; - let (schema, partition_spec) = create_schema_and_partition_spec(PrimitiveType::String)?; + let partition_spec = create_partition_spec(PrimitiveType::String)?; let predicate = Predicate::Binary(BinaryExpression::new( PredicateOperator::NotStartsWith, Reference::new("a"), Datum::string("not"), )) - .bind(schema.clone(), case_sensitive)?; + .bind(partition_spec.schema_ref().clone(), case_sensitive)?; let expression_evaluator = - create_expression_evaluator(&schema, partition_spec, &predicate, case_sensitive)?; + create_expression_evaluator(partition_spec, &predicate, case_sensitive)?; let data_file = create_data_file_string(); @@ -505,17 +501,17 @@ mod tests { #[test] fn test_expr_starts_with() -> Result<()> { let case_sensitive = true; - let (schema, partition_spec) = create_schema_and_partition_spec(PrimitiveType::String)?; + let partition_spec = create_partition_spec(PrimitiveType::String)?; let predicate = Predicate::Binary(BinaryExpression::new( PredicateOperator::StartsWith, Reference::new("a"), Datum::string("test"), )) - .bind(schema.clone(), case_sensitive)?; + .bind(partition_spec.schema_ref().clone(), case_sensitive)?; let expression_evaluator = - create_expression_evaluator(&schema, partition_spec, &predicate, case_sensitive)?; + create_expression_evaluator(partition_spec, &predicate, case_sensitive)?; let data_file = create_data_file_string(); @@ -529,17 +525,17 @@ mod tests { #[test] fn test_expr_not_eq() -> Result<()> { let case_sensitive = true; - let (schema, partition_spec) = create_schema_and_partition_spec(PrimitiveType::Float)?; + let partition_spec = create_partition_spec(PrimitiveType::Float)?; let predicate = Predicate::Binary(BinaryExpression::new( PredicateOperator::NotEq, Reference::new("a"), Datum::float(0.9), )) - .bind(schema.clone(), case_sensitive)?; + .bind(partition_spec.schema_ref().clone(), case_sensitive)?; let expression_evaluator = - create_expression_evaluator(&schema, partition_spec, &predicate, case_sensitive)?; + create_expression_evaluator(partition_spec, &predicate, case_sensitive)?; let data_file = create_data_file_float(); @@ -553,17 +549,17 @@ mod tests { #[test] fn test_expr_eq() -> Result<()> { let case_sensitive = true; - let (schema, partition_spec) = create_schema_and_partition_spec(PrimitiveType::Float)?; + let partition_spec = create_partition_spec(PrimitiveType::Float)?; let predicate = Predicate::Binary(BinaryExpression::new( PredicateOperator::Eq, Reference::new("a"), Datum::float(1.0), )) - .bind(schema.clone(), case_sensitive)?; + .bind(partition_spec.schema_ref().clone(), case_sensitive)?; let expression_evaluator = - create_expression_evaluator(&schema, partition_spec, &predicate, case_sensitive)?; + create_expression_evaluator(partition_spec, &predicate, case_sensitive)?; let data_file = create_data_file_float(); @@ -577,17 +573,17 @@ mod tests { #[test] fn test_expr_greater_than_or_eq() -> Result<()> { let case_sensitive = true; - let (schema, partition_spec) = create_schema_and_partition_spec(PrimitiveType::Float)?; + let partition_spec = create_partition_spec(PrimitiveType::Float)?; let predicate = Predicate::Binary(BinaryExpression::new( PredicateOperator::GreaterThanOrEq, Reference::new("a"), Datum::float(1.0), )) - .bind(schema.clone(), case_sensitive)?; + .bind(partition_spec.schema_ref().clone(), case_sensitive)?; let expression_evaluator = - create_expression_evaluator(&schema, partition_spec, &predicate, case_sensitive)?; + create_expression_evaluator(partition_spec, &predicate, case_sensitive)?; let data_file = create_data_file_float(); @@ -601,17 +597,17 @@ mod tests { #[test] fn test_expr_greater_than() -> Result<()> { let case_sensitive = true; - let (schema, partition_spec) = create_schema_and_partition_spec(PrimitiveType::Float)?; + let partition_spec = create_partition_spec(PrimitiveType::Float)?; let predicate = Predicate::Binary(BinaryExpression::new( PredicateOperator::GreaterThan, Reference::new("a"), Datum::float(0.9), )) - .bind(schema.clone(), case_sensitive)?; + .bind(partition_spec.schema_ref().clone(), case_sensitive)?; let expression_evaluator = - create_expression_evaluator(&schema, partition_spec, &predicate, case_sensitive)?; + create_expression_evaluator(partition_spec, &predicate, case_sensitive)?; let data_file = create_data_file_float(); @@ -625,17 +621,17 @@ mod tests { #[test] fn test_expr_less_than_or_eq() -> Result<()> { let case_sensitive = true; - let (schema, partition_spec) = create_schema_and_partition_spec(PrimitiveType::Float)?; + let partition_spec = create_partition_spec(PrimitiveType::Float)?; let predicate = Predicate::Binary(BinaryExpression::new( PredicateOperator::LessThanOrEq, Reference::new("a"), Datum::float(1.0), )) - .bind(schema.clone(), case_sensitive)?; + .bind(partition_spec.schema_ref().clone(), case_sensitive)?; let expression_evaluator = - create_expression_evaluator(&schema, partition_spec, &predicate, case_sensitive)?; + create_expression_evaluator(partition_spec, &predicate, case_sensitive)?; let data_file = create_data_file_float(); @@ -649,17 +645,17 @@ mod tests { #[test] fn test_expr_less_than() -> Result<()> { let case_sensitive = true; - let (schema, partition_spec) = create_schema_and_partition_spec(PrimitiveType::Float)?; + let partition_spec = create_partition_spec(PrimitiveType::Float)?; let predicate = Predicate::Binary(BinaryExpression::new( PredicateOperator::LessThan, Reference::new("a"), Datum::float(1.1), )) - .bind(schema.clone(), case_sensitive)?; + .bind(partition_spec.schema_ref().clone(), case_sensitive)?; let expression_evaluator = - create_expression_evaluator(&schema, partition_spec, &predicate, case_sensitive)?; + create_expression_evaluator(partition_spec, &predicate, case_sensitive)?; let data_file = create_data_file_float(); @@ -673,15 +669,15 @@ mod tests { #[test] fn test_expr_is_not_nan() -> Result<()> { let case_sensitive = true; - let (schema, partition_spec) = create_schema_and_partition_spec(PrimitiveType::Float)?; + let partition_spec = create_partition_spec(PrimitiveType::Float)?; let predicate = Predicate::Unary(UnaryExpression::new( PredicateOperator::NotNan, Reference::new("a"), )) - .bind(schema.clone(), case_sensitive)?; + .bind(partition_spec.schema_ref().clone(), case_sensitive)?; let expression_evaluator = - create_expression_evaluator(&schema, partition_spec, &predicate, case_sensitive)?; + create_expression_evaluator(partition_spec, &predicate, case_sensitive)?; let data_file = create_data_file_float(); @@ -695,15 +691,15 @@ mod tests { #[test] fn test_expr_is_nan() -> Result<()> { let case_sensitive = true; - let (schema, partition_spec) = create_schema_and_partition_spec(PrimitiveType::Float)?; + let partition_spec = create_partition_spec(PrimitiveType::Float)?; let predicate = Predicate::Unary(UnaryExpression::new( PredicateOperator::IsNan, Reference::new("a"), )) - .bind(schema.clone(), case_sensitive)?; + .bind(partition_spec.schema_ref().clone(), case_sensitive)?; let expression_evaluator = - create_expression_evaluator(&schema, partition_spec, &predicate, case_sensitive)?; + create_expression_evaluator(partition_spec, &predicate, case_sensitive)?; let data_file = create_data_file_float(); @@ -717,15 +713,15 @@ mod tests { #[test] fn test_expr_is_not_null() -> Result<()> { let case_sensitive = true; - let (schema, partition_spec) = create_schema_and_partition_spec(PrimitiveType::Float)?; + let partition_spec = create_partition_spec(PrimitiveType::Float)?; let predicate = Predicate::Unary(UnaryExpression::new( PredicateOperator::NotNull, Reference::new("a"), )) - .bind(schema.clone(), case_sensitive)?; + .bind(partition_spec.schema_ref().clone(), case_sensitive)?; let expression_evaluator = - create_expression_evaluator(&schema, partition_spec, &predicate, case_sensitive)?; + create_expression_evaluator(partition_spec, &predicate, case_sensitive)?; let data_file = create_data_file_float(); @@ -739,15 +735,15 @@ mod tests { #[test] fn test_expr_is_null() -> Result<()> { let case_sensitive = true; - let (schema, partition_spec) = create_schema_and_partition_spec(PrimitiveType::Float)?; + let partition_spec = create_partition_spec(PrimitiveType::Float)?; let predicate = Predicate::Unary(UnaryExpression::new( PredicateOperator::IsNull, Reference::new("a"), )) - .bind(schema.clone(), case_sensitive)?; + .bind(partition_spec.schema_ref().clone(), case_sensitive)?; let expression_evaluator = - create_expression_evaluator(&schema, partition_spec, &predicate, case_sensitive)?; + create_expression_evaluator(partition_spec, &predicate, case_sensitive)?; let data_file = create_data_file_float(); @@ -761,11 +757,12 @@ mod tests { #[test] fn test_expr_always_false() -> Result<()> { let case_sensitive = true; - let (schema, partition_spec) = create_schema_and_partition_spec(PrimitiveType::Float)?; - let predicate = Predicate::AlwaysFalse.bind(schema.clone(), case_sensitive)?; + let partition_spec = create_partition_spec(PrimitiveType::Float)?; + let predicate = + Predicate::AlwaysFalse.bind(partition_spec.schema_ref().clone(), case_sensitive)?; let expression_evaluator = - create_expression_evaluator(&schema, partition_spec, &predicate, case_sensitive)?; + create_expression_evaluator(partition_spec, &predicate, case_sensitive)?; let data_file = create_data_file_float(); @@ -779,11 +776,12 @@ mod tests { #[test] fn test_expr_always_true() -> Result<()> { let case_sensitive = true; - let (schema, partition_spec) = create_schema_and_partition_spec(PrimitiveType::Float)?; - let predicate = Predicate::AlwaysTrue.bind(schema.clone(), case_sensitive)?; + let partition_spec = create_partition_spec(PrimitiveType::Float)?; + let predicate = + Predicate::AlwaysTrue.bind(partition_spec.schema_ref().clone(), case_sensitive)?; let expression_evaluator = - create_expression_evaluator(&schema, partition_spec, &predicate, case_sensitive)?; + create_expression_evaluator(partition_spec, &predicate, case_sensitive)?; let data_file = create_data_file_float(); diff --git a/crates/iceberg/src/expr/visitors/inclusive_metrics_evaluator.rs b/crates/iceberg/src/expr/visitors/inclusive_metrics_evaluator.rs index a2ee4722f..1cdc75771 100644 --- a/crates/iceberg/src/expr/visitors/inclusive_metrics_evaluator.rs +++ b/crates/iceberg/src/expr/visitors/inclusive_metrics_evaluator.rs @@ -495,7 +495,7 @@ mod test { UnaryExpression, }; use crate::spec::{ - DataContentType, DataFile, DataFileFormat, Datum, NestedField, PartitionSpec, + BoundPartitionSpec, DataContentType, DataFile, DataFileFormat, Datum, NestedField, PrimitiveType, Schema, Struct, Transform, Type, UnboundPartitionField, }; @@ -504,10 +504,10 @@ mod test { #[test] fn test_data_file_no_partitions() { - let (table_schema_ref, _partition_spec_ref) = create_test_schema_and_partition_spec(); + let partition_spec_ref = create_test_partition_spec(); let partition_filter = Predicate::AlwaysTrue - .bind(table_schema_ref.clone(), false) + .bind(partition_spec_ref.schema_ref().clone(), false) .unwrap(); let case_sensitive = false; @@ -1645,7 +1645,7 @@ mod test { assert!(result, "Should read: NotIn on no nulls column"); } - fn create_test_schema_and_partition_spec() -> (Arc, Arc) { + fn create_test_partition_spec() -> Arc { let table_schema = Schema::builder() .with_fields(vec![Arc::new(NestedField::optional( 1, @@ -1656,7 +1656,7 @@ mod test { .unwrap(); let table_schema_ref = Arc::new(table_schema); - let partition_spec = PartitionSpec::builder(&table_schema_ref) + let partition_spec = BoundPartitionSpec::builder(table_schema_ref.clone()) .with_spec_id(1) .add_unbound_fields(vec![UnboundPartitionField::builder() .source_id(1) @@ -1667,8 +1667,7 @@ mod test { .unwrap() .build() .unwrap(); - let partition_spec_ref = Arc::new(partition_spec); - (table_schema_ref, partition_spec_ref) + Arc::new(partition_spec) } fn not_null(reference: &str) -> BoundPredicate { diff --git a/crates/iceberg/src/expr/visitors/inclusive_projection.rs b/crates/iceberg/src/expr/visitors/inclusive_projection.rs index 2087207ea..7c6e0b2d5 100644 --- a/crates/iceberg/src/expr/visitors/inclusive_projection.rs +++ b/crates/iceberg/src/expr/visitors/inclusive_projection.rs @@ -21,16 +21,16 @@ use fnv::FnvHashSet; use crate::expr::visitors::bound_predicate_visitor::{visit, BoundPredicateVisitor}; use crate::expr::{BoundPredicate, BoundReference, Predicate}; -use crate::spec::{Datum, PartitionField, PartitionSpecRef}; +use crate::spec::{Datum, PartitionField, SchemalessPartitionSpecRef}; use crate::Error; pub(crate) struct InclusiveProjection { - partition_spec: PartitionSpecRef, + partition_spec: SchemalessPartitionSpecRef, cached_parts: HashMap>, } impl InclusiveProjection { - pub(crate) fn new(partition_spec: PartitionSpecRef) -> Self { + pub(crate) fn new(partition_spec: SchemalessPartitionSpecRef) -> Self { Self { partition_spec, cached_parts: HashMap::new(), @@ -235,7 +235,7 @@ mod tests { use crate::expr::visitors::inclusive_projection::InclusiveProjection; use crate::expr::{Bind, Predicate, Reference}; use crate::spec::{ - Datum, NestedField, PartitionField, PartitionSpec, PrimitiveType, Schema, Transform, Type, + BoundPartitionSpec, Datum, NestedField, PrimitiveType, Schema, Transform, Type, UnboundPartitionField, }; @@ -265,13 +265,14 @@ mod tests { #[test] fn test_inclusive_projection_logic_ops() { let schema = build_test_schema(); + let arc_schema = Arc::new(schema); - let partition_spec = PartitionSpec::builder(&schema) + let partition_spec = BoundPartitionSpec::builder(arc_schema.clone()) .with_spec_id(1) .build() - .unwrap(); + .unwrap() + .into_schemaless(); - let arc_schema = Arc::new(schema); let arc_partition_spec = Arc::new(partition_spec); // this predicate contains only logic operators, @@ -295,8 +296,9 @@ mod tests { #[test] fn test_inclusive_projection_identity_transform() { let schema = build_test_schema(); + let arc_schema = Arc::new(schema); - let partition_spec = PartitionSpec::builder(&schema) + let partition_spec = BoundPartitionSpec::builder(arc_schema.clone()) .with_spec_id(1) .add_unbound_field( UnboundPartitionField::builder() @@ -308,9 +310,9 @@ mod tests { ) .unwrap() .build() - .unwrap(); + .unwrap() + .into_schemaless(); - let arc_schema = Arc::new(schema); let arc_partition_spec = Arc::new(partition_spec); let unbound_predicate = Reference::new("a").less_than(Datum::int(10)); @@ -321,7 +323,7 @@ mod tests { // should result in the same Predicate as the original // `unbound_predicate`, since we have just a single partition field, // and it has an Identity transform - let mut inclusive_projection = InclusiveProjection::new(arc_partition_spec.clone()); + let mut inclusive_projection = InclusiveProjection::new(arc_partition_spec); let result = inclusive_projection.project(&bound_predicate).unwrap(); let expected = "a < 10".to_string(); @@ -330,34 +332,95 @@ mod tests { } #[test] - fn test_inclusive_projection_date_transforms() { + fn test_inclusive_projection_date_year_transform() { let schema = build_test_schema(); + let arc_schema = Arc::new(schema); + + let partition_spec = BoundPartitionSpec::builder(arc_schema.clone()) + .with_spec_id(1) + .add_unbound_fields(vec![UnboundPartitionField { + source_id: 2, + name: "year".to_string(), + field_id: Some(1000), + transform: Transform::Year, + }]) + .unwrap() + .build() + .unwrap() + .into_schemaless(); + + let arc_partition_spec = Arc::new(partition_spec); + + let unbound_predicate = + Reference::new("date").less_than(Datum::date_from_str("2024-01-01").unwrap()); + + let bound_predicate = unbound_predicate.bind(arc_schema.clone(), false).unwrap(); + + // applying InclusiveProjection to bound_predicate + // should result in a predicate that correctly handles + // year, month and date + let mut inclusive_projection = InclusiveProjection::new(arc_partition_spec); + let result = inclusive_projection.project(&bound_predicate).unwrap(); + + let expected = "year <= 53".to_string(); + + assert_eq!(result.to_string(), expected); + } + + #[test] + fn test_inclusive_projection_date_month_transform() { + let schema = build_test_schema(); + let arc_schema = Arc::new(schema); + + let partition_spec = BoundPartitionSpec::builder(arc_schema.clone()) + .with_spec_id(1) + .add_unbound_fields(vec![UnboundPartitionField { + source_id: 2, + name: "month".to_string(), + field_id: Some(1000), + transform: Transform::Month, + }]) + .unwrap() + .build() + .unwrap() + .into_schemaless(); + + let arc_partition_spec = Arc::new(partition_spec); + + let unbound_predicate = + Reference::new("date").less_than(Datum::date_from_str("2024-01-01").unwrap()); + + let bound_predicate = unbound_predicate.bind(arc_schema.clone(), false).unwrap(); - let partition_spec = PartitionSpec { - spec_id: 1, - fields: vec![ - PartitionField { - source_id: 2, - name: "year".to_string(), - field_id: 1000, - transform: Transform::Year, - }, - PartitionField { - source_id: 2, - name: "month".to_string(), - field_id: 1001, - transform: Transform::Month, - }, - PartitionField { - source_id: 2, - name: "day".to_string(), - field_id: 1002, - transform: Transform::Day, - }, - ], - }; + // applying InclusiveProjection to bound_predicate + // should result in a predicate that correctly handles + // year, month and date + let mut inclusive_projection = InclusiveProjection::new(arc_partition_spec); + let result = inclusive_projection.project(&bound_predicate).unwrap(); + + let expected = "month <= 647".to_string(); + + assert_eq!(result.to_string(), expected); + } + #[test] + fn test_inclusive_projection_date_day_transform() { + let schema = build_test_schema(); let arc_schema = Arc::new(schema); + + let partition_spec = BoundPartitionSpec::builder(arc_schema.clone()) + .with_spec_id(1) + .add_unbound_fields(vec![UnboundPartitionField { + source_id: 2, + name: "day".to_string(), + field_id: Some(1000), + transform: Transform::Day, + }]) + .unwrap() + .build() + .unwrap() + .into_schemaless(); + let arc_partition_spec = Arc::new(partition_spec); let unbound_predicate = @@ -368,10 +431,10 @@ mod tests { // applying InclusiveProjection to bound_predicate // should result in a predicate that correctly handles // year, month and date - let mut inclusive_projection = InclusiveProjection::new(arc_partition_spec.clone()); + let mut inclusive_projection = InclusiveProjection::new(arc_partition_spec); let result = inclusive_projection.project(&bound_predicate).unwrap(); - let expected = "((year <= 53) AND (month <= 647)) AND (day <= 19722)".to_string(); + let expected = "day <= 19722".to_string(); assert_eq!(result.to_string(), expected); } @@ -379,8 +442,9 @@ mod tests { #[test] fn test_inclusive_projection_truncate_transform() { let schema = build_test_schema(); + let arc_schema = Arc::new(schema); - let partition_spec = PartitionSpec::builder(&schema) + let partition_spec = BoundPartitionSpec::builder(arc_schema.clone()) .with_spec_id(1) .add_unbound_field( UnboundPartitionField::builder() @@ -392,9 +456,9 @@ mod tests { ) .unwrap() .build() - .unwrap(); + .unwrap() + .into_schemaless(); - let arc_schema = Arc::new(schema); let arc_partition_spec = Arc::new(partition_spec); let unbound_predicate = Reference::new("name").starts_with(Datum::string("Testy McTest")); @@ -408,7 +472,7 @@ mod tests { // name that start with "Testy McTest" into a partition // for values of name that start with the first four letters // of that, ie "Test". - let mut inclusive_projection = InclusiveProjection::new(arc_partition_spec.clone()); + let mut inclusive_projection = InclusiveProjection::new(arc_partition_spec); let result = inclusive_projection.project(&bound_predicate).unwrap(); let expected = "name_truncate STARTS WITH \"Test\"".to_string(); @@ -419,8 +483,9 @@ mod tests { #[test] fn test_inclusive_projection_bucket_transform() { let schema = build_test_schema(); + let arc_schema = Arc::new(schema); - let partition_spec = PartitionSpec::builder(&schema) + let partition_spec = BoundPartitionSpec::builder(arc_schema.clone()) .with_spec_id(1) .add_unbound_field( UnboundPartitionField::builder() @@ -432,9 +497,9 @@ mod tests { ) .unwrap() .build() - .unwrap(); + .unwrap() + .into_schemaless(); - let arc_schema = Arc::new(schema); let arc_partition_spec = Arc::new(partition_spec); let unbound_predicate = Reference::new("a").equal_to(Datum::int(10)); @@ -445,7 +510,7 @@ mod tests { // should result in the "a = 10" predicate being // transformed into "a = 2", since 10 gets bucketed // to 2 with a Bucket(7) partition - let mut inclusive_projection = InclusiveProjection::new(arc_partition_spec.clone()); + let mut inclusive_projection = InclusiveProjection::new(arc_partition_spec); let result = inclusive_projection.project(&bound_predicate).unwrap(); let expected = "a_bucket[7] = 2".to_string(); diff --git a/crates/iceberg/src/io/object_cache.rs b/crates/iceberg/src/io/object_cache.rs index 731072a5a..35b6a2c94 100644 --- a/crates/iceberg/src/io/object_cache.rs +++ b/crates/iceberg/src/io/object_cache.rs @@ -262,7 +262,7 @@ mod tests { ) .write(Manifest::new( ManifestMetadata::builder() - .schema((*current_schema).clone()) + .schema(current_schema.clone()) .content(ManifestContentType::Data) .format_version(FormatVersion::V2) .partition_spec((**current_partition_spec).clone()) diff --git a/crates/iceberg/src/scan.rs b/crates/iceberg/src/scan.rs index d03554541..89e8846f0 100644 --- a/crates/iceberg/src/scan.rs +++ b/crates/iceberg/src/scan.rs @@ -706,7 +706,7 @@ impl PartitionFilterCache { &self, spec_id: i32, table_metadata: &TableMetadataRef, - schema: &SchemaRef, + schema: &Schema, case_sensitive: bool, filter: BoundPredicate, ) -> Result> { @@ -732,11 +732,11 @@ impl PartitionFilterCache { format!("Could not find partition spec for id {}", spec_id), ))?; - let partition_type = partition_spec.partition_type(schema.as_ref())?; + let partition_type = partition_spec.partition_type(schema)?; let partition_fields = partition_type.fields().to_owned(); let partition_schema = Arc::new( Schema::builder() - .with_schema_id(partition_spec.spec_id) + .with_schema_id(partition_spec.spec_id()) .with_fields(partition_fields) .build()?, ); @@ -1057,7 +1057,7 @@ mod tests { ) .write(Manifest::new( ManifestMetadata::builder() - .schema((*current_schema).clone()) + .schema(current_schema.clone()) .content(ManifestContentType::Data) .format_version(FormatVersion::V2) .partition_spec((**current_partition_spec).clone()) diff --git a/crates/iceberg/src/spec/manifest.rs b/crates/iceberg/src/spec/manifest.rs index 997908d5f..085200b7c 100644 --- a/crates/iceberg/src/spec/manifest.rs +++ b/crates/iceberg/src/spec/manifest.rs @@ -30,8 +30,8 @@ use typed_builder::TypedBuilder; use self::_const_schema::{manifest_schema_v1, manifest_schema_v2}; use super::{ - Datum, FieldSummary, FormatVersion, ManifestContentType, ManifestFile, PartitionSpec, Schema, - SchemaId, Struct, INITIAL_SEQUENCE_NUMBER, UNASSIGNED_SEQUENCE_NUMBER, + BoundPartitionSpec, Datum, FieldSummary, FormatVersion, ManifestContentType, ManifestFile, + Schema, SchemaId, SchemaRef, Struct, INITIAL_SEQUENCE_NUMBER, UNASSIGNED_SEQUENCE_NUMBER, }; use crate::error::Result; use crate::io::OutputFile; @@ -55,7 +55,7 @@ impl Manifest { let metadata = ManifestMetadata::parse(meta)?; // Parse manifest entries - let partition_type = metadata.partition_spec.partition_type(&metadata.schema)?; + let partition_type = metadata.partition_spec.partition_type(); let entries = match metadata.format_version { FormatVersion::V1 => { @@ -65,7 +65,7 @@ impl Manifest { .into_iter() .map(|value| { from_value::<_serde::ManifestEntryV1>(&value?)? - .try_into(&partition_type, &metadata.schema) + .try_into(partition_type, &metadata.schema) }) .collect::>>()? } @@ -76,7 +76,7 @@ impl Manifest { .into_iter() .map(|value| { from_value::<_serde::ManifestEntryV2>(&value?)? - .try_into(&partition_type, &metadata.schema) + .try_into(partition_type, &metadata.schema) }) .collect::>>()? } @@ -206,10 +206,7 @@ impl ManifestWriter { /// Write a manifest. pub async fn write(mut self, manifest: Manifest) -> Result { // Create the avro writer - let partition_type = manifest - .metadata - .partition_spec - .partition_type(&manifest.metadata.schema)?; + let partition_type = manifest.metadata.partition_spec.partition_type(); let table_schema = &manifest.metadata.schema; let avro_schema = match manifest.metadata.format_version { FormatVersion::V1 => manifest_schema_v1(partition_type.clone())?, @@ -284,12 +281,12 @@ impl ManifestWriter { let value = match manifest.metadata.format_version { FormatVersion::V1 => to_value(_serde::ManifestEntryV1::try_from( (*entry).clone(), - &partition_type, + partition_type, )?)? .resolve(&avro_schema)?, FormatVersion::V2 => to_value(_serde::ManifestEntryV2::try_from( (*entry).clone(), - &partition_type, + partition_type, )?)? .resolve(&avro_schema)?, }; @@ -705,11 +702,11 @@ mod _const_schema { pub struct ManifestMetadata { /// The table schema at the time the manifest /// was written - schema: Schema, + schema: SchemaRef, /// ID of the schema used to write the manifest as a string schema_id: SchemaId, /// The partition spec used to write the manifest - partition_spec: PartitionSpec, + partition_spec: BoundPartitionSpec, /// Table format version number of the manifest as a string format_version: FormatVersion, /// Type of content files tracked by the manifest: “data” or “deletes” @@ -719,7 +716,7 @@ pub struct ManifestMetadata { impl ManifestMetadata { /// Parse from metadata in avro file. pub fn parse(meta: &HashMap>) -> Result { - let schema = { + let schema = Arc::new({ let bs = meta.get("schema").ok_or_else(|| { Error::new( ErrorKind::DataInvalid, @@ -733,7 +730,7 @@ impl ManifestMetadata { ) .with_source(err) })? - }; + }); let schema_id: i32 = meta .get("schema-id") .map(|bs| { @@ -776,7 +773,10 @@ impl ManifestMetadata { }) .transpose()? .unwrap_or(0); - PartitionSpec { spec_id, fields } + BoundPartitionSpec::builder(schema.clone()) + .with_spec_id(spec_id) + .add_unbound_fields(fields.into_iter().map(|f| f.into_unbound()))? + .build()? }; let format_version = if let Some(bs) = meta.get("format-version") { serde_json::from_slice::(bs).map_err(|err| { @@ -1519,82 +1519,82 @@ mod tests { #[tokio::test] async fn test_parse_manifest_v2_unpartition() { + let schema = Arc::new( + Schema::builder() + .with_fields(vec![ + // id v_int v_long v_float v_double v_varchar v_bool v_date v_timestamp v_decimal v_ts_ntz + Arc::new(NestedField::optional( + 1, + "id", + Type::Primitive(PrimitiveType::Long), + )), + Arc::new(NestedField::optional( + 2, + "v_int", + Type::Primitive(PrimitiveType::Int), + )), + Arc::new(NestedField::optional( + 3, + "v_long", + Type::Primitive(PrimitiveType::Long), + )), + Arc::new(NestedField::optional( + 4, + "v_float", + Type::Primitive(PrimitiveType::Float), + )), + Arc::new(NestedField::optional( + 5, + "v_double", + Type::Primitive(PrimitiveType::Double), + )), + Arc::new(NestedField::optional( + 6, + "v_varchar", + Type::Primitive(PrimitiveType::String), + )), + Arc::new(NestedField::optional( + 7, + "v_bool", + Type::Primitive(PrimitiveType::Boolean), + )), + Arc::new(NestedField::optional( + 8, + "v_date", + Type::Primitive(PrimitiveType::Date), + )), + Arc::new(NestedField::optional( + 9, + "v_timestamp", + Type::Primitive(PrimitiveType::Timestamptz), + )), + Arc::new(NestedField::optional( + 10, + "v_decimal", + Type::Primitive(PrimitiveType::Decimal { + precision: 36, + scale: 10, + }), + )), + Arc::new(NestedField::optional( + 11, + "v_ts_ntz", + Type::Primitive(PrimitiveType::Timestamp), + )), + Arc::new(NestedField::optional( + 12, + "v_ts_ns_ntz", + Type::Primitive(PrimitiveType::TimestampNs), + )), + ]) + .build() + .unwrap(), + ); let manifest = Manifest { metadata: ManifestMetadata { schema_id: 0, - schema: Schema::builder() - .with_fields(vec![ - // id v_int v_long v_float v_double v_varchar v_bool v_date v_timestamp v_decimal v_ts_ntz - Arc::new(NestedField::optional( - 1, - "id", - Type::Primitive(PrimitiveType::Long), - )), - Arc::new(NestedField::optional( - 2, - "v_int", - Type::Primitive(PrimitiveType::Int), - )), - Arc::new(NestedField::optional( - 3, - "v_long", - Type::Primitive(PrimitiveType::Long), - )), - Arc::new(NestedField::optional( - 4, - "v_float", - Type::Primitive(PrimitiveType::Float), - )), - Arc::new(NestedField::optional( - 5, - "v_double", - Type::Primitive(PrimitiveType::Double), - )), - Arc::new(NestedField::optional( - 6, - "v_varchar", - Type::Primitive(PrimitiveType::String), - )), - Arc::new(NestedField::optional( - 7, - "v_bool", - Type::Primitive(PrimitiveType::Boolean), - )), - Arc::new(NestedField::optional( - 8, - "v_date", - Type::Primitive(PrimitiveType::Date), - )), - Arc::new(NestedField::optional( - 9, - "v_timestamp", - Type::Primitive(PrimitiveType::Timestamptz), - )), - Arc::new(NestedField::optional( - 10, - "v_decimal", - Type::Primitive(PrimitiveType::Decimal { - precision: 36, - scale: 10, - }), - )), - Arc::new(NestedField::optional( - 11, - "v_ts_ntz", - Type::Primitive(PrimitiveType::Timestamp), - )), - Arc::new(NestedField::optional( - 12, - "v_ts_ns_ntz", - Type::Primitive(PrimitiveType::TimestampNs - ))), - ]) - .build() - .unwrap(), - partition_spec: PartitionSpec { - spec_id: 0, - fields: vec![], - }, + schema: schema.clone(), + partition_spec: BoundPartitionSpec::builder(schema).with_spec_id(0).build().unwrap(), content: ManifestContentType::Data, format_version: FormatVersion::V2, }, @@ -1633,94 +1633,83 @@ mod tests { #[tokio::test] async fn test_parse_manifest_v2_partition() { + let schema = Arc::new( + Schema::builder() + .with_fields(vec![ + Arc::new(NestedField::optional( + 1, + "id", + Type::Primitive(PrimitiveType::Long), + )), + Arc::new(NestedField::optional( + 2, + "v_int", + Type::Primitive(PrimitiveType::Int), + )), + Arc::new(NestedField::optional( + 3, + "v_long", + Type::Primitive(PrimitiveType::Long), + )), + Arc::new(NestedField::optional( + 4, + "v_float", + Type::Primitive(PrimitiveType::Float), + )), + Arc::new(NestedField::optional( + 5, + "v_double", + Type::Primitive(PrimitiveType::Double), + )), + Arc::new(NestedField::optional( + 6, + "v_varchar", + Type::Primitive(PrimitiveType::String), + )), + Arc::new(NestedField::optional( + 7, + "v_bool", + Type::Primitive(PrimitiveType::Boolean), + )), + Arc::new(NestedField::optional( + 8, + "v_date", + Type::Primitive(PrimitiveType::Date), + )), + Arc::new(NestedField::optional( + 9, + "v_timestamp", + Type::Primitive(PrimitiveType::Timestamptz), + )), + Arc::new(NestedField::optional( + 10, + "v_decimal", + Type::Primitive(PrimitiveType::Decimal { + precision: 36, + scale: 10, + }), + )), + Arc::new(NestedField::optional( + 11, + "v_ts_ntz", + Type::Primitive(PrimitiveType::Timestamp), + )), + Arc::new(NestedField::optional( + 12, + "v_ts_ns_ntz", + Type::Primitive(PrimitiveType::TimestampNs), + )), + ]) + .build() + .unwrap(), + ); let manifest = Manifest { metadata: ManifestMetadata { schema_id: 0, - schema: Schema::builder() - .with_fields(vec![ - Arc::new(NestedField::optional( - 1, - "id", - Type::Primitive(PrimitiveType::Long), - )), - Arc::new(NestedField::optional( - 2, - "v_int", - Type::Primitive(PrimitiveType::Int), - )), - Arc::new(NestedField::optional( - 3, - "v_long", - Type::Primitive(PrimitiveType::Long), - )), - Arc::new(NestedField::optional( - 4, - "v_float", - Type::Primitive(PrimitiveType::Float), - )), - Arc::new(NestedField::optional( - 5, - "v_double", - Type::Primitive(PrimitiveType::Double), - )), - Arc::new(NestedField::optional( - 6, - "v_varchar", - Type::Primitive(PrimitiveType::String), - )), - Arc::new(NestedField::optional( - 7, - "v_bool", - Type::Primitive(PrimitiveType::Boolean), - )), - Arc::new(NestedField::optional( - 8, - "v_date", - Type::Primitive(PrimitiveType::Date), - )), - Arc::new(NestedField::optional( - 9, - "v_timestamp", - Type::Primitive(PrimitiveType::Timestamptz), - )), - Arc::new(NestedField::optional( - 10, - "v_decimal", - Type::Primitive(PrimitiveType::Decimal { - precision: 36, - scale: 10, - }), - )), - Arc::new(NestedField::optional( - 11, - "v_ts_ntz", - Type::Primitive(PrimitiveType::Timestamp), - )), - Arc::new(NestedField::optional( - 12, - "v_ts_ns_ntz", - Type::Primitive(PrimitiveType::TimestampNs - ))) - ]) - .build() - .unwrap(), - partition_spec: PartitionSpec { - spec_id: 0, - fields: vec![ - PartitionField { - name: "v_int".to_string(), - transform: Transform::Identity, - source_id: 2, - field_id: 1000, - }, - PartitionField { - name: "v_long".to_string(), - transform: Transform::Identity, - source_id: 3, - field_id: 1001, - }, - ], - }, + schema: schema.clone(), + partition_spec: BoundPartitionSpec::builder(schema) + .with_spec_id(0).add_partition_field("v_int", "v_int", Transform::Identity).unwrap() + .add_partition_field("v_long", "v_long", Transform::Identity).unwrap().build().unwrap(), content: ManifestContentType::Data, format_version: FormatVersion::V2, }, @@ -1802,34 +1791,34 @@ mod tests { #[tokio::test] async fn test_parse_manifest_v1_unpartition() { + let schema = Arc::new( + Schema::builder() + .with_schema_id(1) + .with_fields(vec![ + Arc::new(NestedField::optional( + 1, + "id", + Type::Primitive(PrimitiveType::Int), + )), + Arc::new(NestedField::optional( + 2, + "data", + Type::Primitive(PrimitiveType::String), + )), + Arc::new(NestedField::optional( + 3, + "comment", + Type::Primitive(PrimitiveType::String), + )), + ]) + .build() + .unwrap(), + ); let manifest = Manifest { metadata: ManifestMetadata { schema_id: 1, - schema: Schema::builder() - .with_schema_id(1) - .with_fields(vec![ - Arc::new(NestedField::optional( - 1, - "id", - Type::Primitive(PrimitiveType::Int), - )), - Arc::new(NestedField::optional( - 2, - "data", - Type::Primitive(PrimitiveType::String), - )), - Arc::new(NestedField::optional( - 3, - "comment", - Type::Primitive(PrimitiveType::String), - )), - ]) - .build() - .unwrap(), - partition_spec: PartitionSpec { - spec_id: 0, - fields: vec![], - }, + schema: schema.clone(), + partition_spec: BoundPartitionSpec::builder(schema).with_spec_id(0).build().unwrap(), content: ManifestContentType::Data, format_version: FormatVersion::V1, }, @@ -1867,38 +1856,33 @@ mod tests { #[tokio::test] async fn test_parse_manifest_v1_partition() { + let schema = Arc::new( + Schema::builder() + .with_fields(vec![ + Arc::new(NestedField::optional( + 1, + "id", + Type::Primitive(PrimitiveType::Long), + )), + Arc::new(NestedField::optional( + 2, + "data", + Type::Primitive(PrimitiveType::String), + )), + Arc::new(NestedField::optional( + 3, + "category", + Type::Primitive(PrimitiveType::String), + )), + ]) + .build() + .unwrap(), + ); let manifest = Manifest { metadata: ManifestMetadata { schema_id: 0, - schema: Schema::builder() - .with_fields(vec![ - Arc::new(NestedField::optional( - 1, - "id", - Type::Primitive(PrimitiveType::Long), - )), - Arc::new(NestedField::optional( - 2, - "data", - Type::Primitive(PrimitiveType::String), - )), - Arc::new(NestedField::optional( - 3, - "category", - Type::Primitive(PrimitiveType::String), - )), - ]) - .build() - .unwrap(), - partition_spec: PartitionSpec { - spec_id: 0, - fields: vec![PartitionField { - name: "category".to_string(), - transform: Transform::Identity, - source_id: 3, - field_id: 1000, - }], - }, + schema: schema.clone(), + partition_spec: BoundPartitionSpec::builder(schema).add_partition_field("category", "category", Transform::Identity).unwrap().build().unwrap(), content: ManifestContentType::Data, format_version: FormatVersion::V1, }, @@ -1956,28 +1940,28 @@ mod tests { #[tokio::test] async fn test_parse_manifest_with_schema_evolution() { + let schema = Arc::new( + Schema::builder() + .with_fields(vec![ + Arc::new(NestedField::optional( + 1, + "id", + Type::Primitive(PrimitiveType::Long), + )), + Arc::new(NestedField::optional( + 2, + "v_int", + Type::Primitive(PrimitiveType::Int), + )), + ]) + .build() + .unwrap(), + ); let manifest = Manifest { metadata: ManifestMetadata { schema_id: 0, - schema: Schema::builder() - .with_fields(vec![ - Arc::new(NestedField::optional( - 1, - "id", - Type::Primitive(PrimitiveType::Long), - )), - Arc::new(NestedField::optional( - 2, - "v_int", - Type::Primitive(PrimitiveType::Int), - )), - ]) - .build() - .unwrap(), - partition_spec: PartitionSpec { - spec_id: 0, - fields: vec![], - }, + schema: schema.clone(), + partition_spec: BoundPartitionSpec::builder(schema).with_spec_id(0).build().unwrap(), content: ManifestContentType::Data, format_version: FormatVersion::V2, }, @@ -2028,28 +2012,28 @@ mod tests { // Compared with original manifest, the lower_bounds and upper_bounds no longer has data for field 3, and // other parts should be same. + let schema = Arc::new( + Schema::builder() + .with_fields(vec![ + Arc::new(NestedField::optional( + 1, + "id", + Type::Primitive(PrimitiveType::Long), + )), + Arc::new(NestedField::optional( + 2, + "v_int", + Type::Primitive(PrimitiveType::Int), + )), + ]) + .build() + .unwrap(), + ); let expected_manifest = Manifest { metadata: ManifestMetadata { schema_id: 0, - schema: Schema::builder() - .with_fields(vec![ - Arc::new(NestedField::optional( - 1, - "id", - Type::Primitive(PrimitiveType::Long), - )), - Arc::new(NestedField::optional( - 2, - "v_int", - Type::Primitive(PrimitiveType::Int), - )), - ]) - .build() - .unwrap(), - partition_spec: PartitionSpec { - spec_id: 0, - fields: vec![], - }, + schema: schema.clone(), + partition_spec: BoundPartitionSpec::builder(schema).with_spec_id(0).build().unwrap(), content: ManifestContentType::Data, format_version: FormatVersion::V2, }, diff --git a/crates/iceberg/src/spec/partition.rs b/crates/iceberg/src/spec/partition.rs index 36763df7e..75e5d9247 100644 --- a/crates/iceberg/src/spec/partition.rs +++ b/crates/iceberg/src/spec/partition.rs @@ -24,14 +24,15 @@ use serde::{Deserialize, Serialize}; use typed_builder::TypedBuilder; use super::transform::Transform; -use super::{NestedField, Schema, StructType}; +use super::{NestedField, Schema, SchemaRef, StructType}; use crate::{Error, ErrorKind, Result}; pub(crate) const UNPARTITIONED_LAST_ASSIGNED_ID: i32 = 999; pub(crate) const DEFAULT_PARTITION_SPEC_ID: i32 = 0; -/// Reference to [`PartitionSpec`]. -pub type PartitionSpecRef = Arc; +/// Reference to [`BoundPartitionSpec`]. +pub type BoundPartitionSpecRef = Arc; + /// Partition fields capture the transform from table data to partition values. #[derive(Debug, Serialize, Deserialize, PartialEq, Eq, Clone, TypedBuilder)] #[serde(rename_all = "kebab-case")] @@ -54,22 +55,51 @@ impl PartitionField { } } -/// Partition spec that defines how to produce a tuple of partition values from a record. -#[derive(Debug, Serialize, Deserialize, PartialEq, Eq, Clone, Default)] +/// Partition spec that defines how to produce a tuple of partition values from a record. +/// `PartitionSpec` is bound to a specific schema. +#[derive(Debug, PartialEq, Eq, Clone)] +pub struct BoundPartitionSpec { + /// Identifier for PartitionSpec + spec_id: i32, + /// Details of the partition spec + fields: Vec, + /// The schema this partition spec is bound to + schema: SchemaRef, + /// Type of the partition spec + partition_type: StructType, +} + +/// Reference to [`SchemalessPartitionSpec`]. +pub type SchemalessPartitionSpecRef = Arc; +/// Partition spec that defines how to produce a tuple of partition values from a record. +/// Schemaless partition specs are never constructed manually. They occur when a table is mutated +/// and partition spec and schemas are updated. While old partition specs are retained, the bound +/// schema might not be available anymore as part of the table metadata. +#[derive(Debug, Serialize, Deserialize, PartialEq, Eq, Clone)] #[serde(rename_all = "kebab-case")] -pub struct PartitionSpec { +pub struct SchemalessPartitionSpec { /// Identifier for PartitionSpec - pub(crate) spec_id: i32, + spec_id: i32, /// Details of the partition spec - pub(crate) fields: Vec, + fields: Vec, } -impl PartitionSpec { +impl BoundPartitionSpec { /// Create partition spec builder - pub fn builder(schema: &Schema) -> PartitionSpecBuilder { + pub fn builder(schema: impl Into) -> PartitionSpecBuilder { PartitionSpecBuilder::new(schema) } + /// Get a new unpatitioned partition spec + pub fn unpartition_spec(schema: impl Into) -> Self { + Self { + spec_id: DEFAULT_PARTITION_SPEC_ID, + fields: vec![], + schema: schema.into(), + partition_type: StructType::new(vec![]), + } + } + /// Spec id of the partition spec pub fn spec_id(&self) -> i32 { self.spec_id @@ -80,39 +110,21 @@ impl PartitionSpec { &self.fields } + /// The schema this partition spec is bound to + pub fn schema(&self) -> &Schema { + &self.schema + } + + /// The schema ref this partition spec is bound to + pub fn schema_ref(&self) -> &SchemaRef { + &self.schema + } + /// Returns if the partition spec is unpartitioned. /// /// A [`PartitionSpec`] is unpartitioned if it has no fields or all fields are [`Transform::Void`] transform. pub fn is_unpartitioned(&self) -> bool { - self.fields.is_empty() - || self - .fields - .iter() - .all(|f| matches!(f.transform, Transform::Void)) - } - - /// Returns the partition type of this partition spec. - pub fn partition_type(&self, schema: &Schema) -> Result { - let mut fields = Vec::with_capacity(self.fields.len()); - for partition_field in &self.fields { - let field = schema - .field_by_id(partition_field.source_id) - .ok_or_else(|| { - Error::new( - ErrorKind::DataInvalid, - format!( - "No column with source column id {} in schema {:?}", - partition_field.source_id, schema - ), - ) - })?; - let res_type = partition_field.transform.result_type(&field.field_type)?; - let field = - NestedField::optional(partition_field.field_id, &partition_field.name, res_type) - .into(); - fields.push(field); - } - Ok(StructType::new(fields)) + self.fields.is_empty() || self.fields.iter().all(|f| f.transform == Transform::Void) } /// Turn this partition spec into an unbound partition spec. @@ -122,6 +134,29 @@ impl PartitionSpec { self.into() } + /// Turn this partition spec into a preserved partition spec. + pub fn into_schemaless(self) -> SchemalessPartitionSpec { + self.into() + } + + /// Check if this partition spec has sequential partition ids. + /// Sequential ids start from 1000 and increment by 1 for each field. + /// This is required for spec version 1 + pub fn has_sequential_ids(&self) -> bool { + has_sequential_ids(self.fields.iter().map(|f| f.field_id)) + } + + /// Get the highest field id in the partition spec. + /// If the partition spec is unpartitioned, it returns the last unpartitioned last assigned id (999). + pub fn highest_field_id(&self) -> Option { + self.fields.iter().map(|f| f.field_id).max() + } + + /// Returns the partition type of this partition spec. + pub fn partition_type(&self) -> &StructType { + &self.partition_type + } + /// Check if this partition spec is compatible with another partition spec. /// /// Returns true if the partition spec is equal to the other spec with partition field ids ignored and @@ -131,15 +166,15 @@ impl PartitionSpec { /// * Field names /// * Source column ids /// * Transforms - pub fn is_compatible_with(&self, other: &UnboundPartitionSpec) -> bool { + pub fn is_compatible_with_schemaless(&self, other: &SchemalessPartitionSpec) -> bool { if self.fields.len() != other.fields.len() { return false; } - for (this_field, other_field) in self.fields.iter().zip(&other.fields) { + for (this_field, other_field) in self.fields.iter().zip(other.fields.iter()) { if this_field.source_id != other_field.source_id - || this_field.transform != other_field.transform || this_field.name != other_field.name + || this_field.transform != other_field.transform { return false; } @@ -147,33 +182,40 @@ impl PartitionSpec { true } +} - /// Check if this partition spec has sequential partition ids. - /// Sequential ids start from 1000 and increment by 1 for each field. - /// This is required for spec version 1 - pub fn has_sequential_ids(&self) -> bool { - for (index, field) in self.fields.iter().enumerate() { - let expected_id = (UNPARTITIONED_LAST_ASSIGNED_ID as i64) - .checked_add(1) - .and_then(|id| id.checked_add(index as i64)) - .unwrap_or(i64::MAX); +impl SchemalessPartitionSpec { + /// Fields of the partition spec + pub fn fields(&self) -> &[PartitionField] { + &self.fields + } - if field.field_id as i64 != expected_id { - return false; - } + /// Spec id of the partition spec + pub fn spec_id(&self) -> i32 { + self.spec_id + } + + /// Bind this schemaless partition spec to a schema. + pub fn bind(self, schema: impl Into) -> Result { + PartitionSpecBuilder::new_from_unbound(self.into_unbound(), schema)?.build() + } + + /// Get a new unpatitioned partition spec + pub fn unpartition_spec() -> Self { + Self { + spec_id: DEFAULT_PARTITION_SPEC_ID, + fields: vec![], } + } - true + /// Returns the partition type of this partition spec. + pub fn partition_type(&self, schema: &Schema) -> Result { + PartitionSpecBuilder::partition_type(&self.fields, schema) } - /// Get the highest field id in the partition spec. - /// If the partition spec is unpartitioned, it returns the last unpartitioned last assigned id (999). - pub fn highest_field_id(&self) -> i32 { - self.fields - .iter() - .map(|f| f.field_id) - .max() - .unwrap_or(UNPARTITIONED_LAST_ASSIGNED_ID) + /// Convert to unbound partition spec + pub fn into_unbound(self) -> UnboundPartitionSpec { + self.into() } } @@ -212,7 +254,7 @@ impl UnboundPartitionSpec { } /// Bind this unbound partition spec to a schema. - pub fn bind(self, schema: &Schema) -> Result { + pub fn bind(self, schema: impl Into) -> Result { PartitionSpecBuilder::new_from_unbound(self, schema)?.build() } @@ -235,6 +277,21 @@ impl UnboundPartitionSpec { } } +fn has_sequential_ids(field_ids: impl Iterator) -> bool { + for (index, field_id) in field_ids.enumerate() { + let expected_id = (UNPARTITIONED_LAST_ASSIGNED_ID as i64) + .checked_add(1) + .and_then(|id| id.checked_add(index as i64)) + .unwrap_or(i64::MAX); + + if field_id as i64 != expected_id { + return false; + } + } + + true +} + impl From for UnboundPartitionField { fn from(field: PartitionField) -> Self { UnboundPartitionField { @@ -246,8 +303,17 @@ impl From for UnboundPartitionField { } } -impl From for UnboundPartitionSpec { - fn from(spec: PartitionSpec) -> Self { +impl From for UnboundPartitionSpec { + fn from(spec: BoundPartitionSpec) -> Self { + UnboundPartitionSpec { + spec_id: Some(spec.spec_id), + fields: spec.fields.into_iter().map(Into::into).collect(), + } + } +} + +impl From for UnboundPartitionSpec { + fn from(spec: SchemalessPartitionSpec) -> Self { UnboundPartitionSpec { spec_id: Some(spec.spec_id), fields: spec.fields.into_iter().map(Into::into).collect(), @@ -255,6 +321,15 @@ impl From for UnboundPartitionSpec { } } +impl From for SchemalessPartitionSpec { + fn from(spec: BoundPartitionSpec) -> Self { + SchemalessPartitionSpec { + spec_id: spec.spec_id, + fields: spec.fields, + } + } +} + /// Create a new UnboundPartitionSpec #[derive(Debug, Default)] pub struct UnboundPartitionSpecBuilder { @@ -326,26 +401,29 @@ impl UnboundPartitionSpecBuilder { /// Create valid partition specs for a given schema. #[derive(Debug)] -pub struct PartitionSpecBuilder<'a> { +pub struct PartitionSpecBuilder { spec_id: Option, last_assigned_field_id: i32, fields: Vec, - schema: &'a Schema, + schema: SchemaRef, } -impl<'a> PartitionSpecBuilder<'a> { +impl PartitionSpecBuilder { /// Create a new partition spec builder with the given schema. - pub fn new(schema: &'a Schema) -> Self { + pub fn new(schema: impl Into) -> Self { Self { spec_id: None, fields: vec![], last_assigned_field_id: UNPARTITIONED_LAST_ASSIGNED_ID, - schema, + schema: schema.into(), } } /// Create a new partition spec builder from an existing unbound partition spec. - pub fn new_from_unbound(unbound: UnboundPartitionSpec, schema: &'a Schema) -> Result { + pub fn new_from_unbound( + unbound: UnboundPartitionSpec, + schema: impl Into, + ) -> Result { let mut builder = Self::new(schema).with_spec_id(unbound.spec_id.unwrap_or(DEFAULT_PARTITION_SPEC_ID)); @@ -408,8 +486,8 @@ impl<'a> PartitionSpecBuilder<'a> { pub fn add_unbound_field(mut self, field: UnboundPartitionField) -> Result { self.check_name_set_and_unique(&field.name)?; self.check_for_redundant_partitions(field.source_id, &field.transform)?; - Self::check_name_does_not_collide_with_schema(&field, self.schema)?; - Self::check_transform_compatibility(&field, self.schema)?; + Self::check_name_does_not_collide_with_schema(&field, &self.schema)?; + Self::check_transform_compatibility(&field, &self.schema)?; if let Some(partition_field_id) = field.field_id { self.check_partition_id_unique(partition_field_id)?; } @@ -432,11 +510,14 @@ impl<'a> PartitionSpecBuilder<'a> { } /// Build a bound partition spec with the given schema. - pub fn build(self) -> Result { + pub fn build(self) -> Result { let fields = Self::set_field_ids(self.fields, self.last_assigned_field_id)?; - Ok(PartitionSpec { + let partition_type = Self::partition_type(&fields, &self.schema)?; + Ok(BoundPartitionSpec { spec_id: self.spec_id.unwrap_or(DEFAULT_PARTITION_SPEC_ID), fields, + partition_type, + schema: self.schema, }) } @@ -485,6 +566,32 @@ impl<'a> PartitionSpecBuilder<'a> { Ok(bound_fields) } + /// Returns the partition type of this partition spec. + fn partition_type(fields: &Vec, schema: &Schema) -> Result { + let mut struct_fields = Vec::with_capacity(fields.len()); + for partition_field in fields { + let field = schema + .field_by_id(partition_field.source_id) + .ok_or_else(|| { + Error::new( + // This should never occur as check_transform_compatibility + // already ensures that the source field exists in the schema + ErrorKind::Unexpected, + format!( + "No column with source column id {} in schema {:?}", + partition_field.source_id, schema + ), + ) + })?; + let res_type = partition_field.transform.result_type(&field.field_type)?; + let field = + NestedField::optional(partition_field.field_id, &partition_field.name, res_type) + .into(); + struct_fields.push(field); + } + Ok(StructType::new(struct_fields)) + } + /// Ensure that the partition name is unique among columns in the schema. /// Duplicate names are allowed if: /// 1. The column is sourced from the column with the same name. @@ -622,7 +729,7 @@ trait CorePartitionSpecValidator { fn fields(&self) -> &Vec; } -impl CorePartitionSpecValidator for PartitionSpecBuilder<'_> { +impl CorePartitionSpecValidator for PartitionSpecBuilder { fn fields(&self) -> &Vec { &self.fields } @@ -637,7 +744,7 @@ impl CorePartitionSpecValidator for UnboundPartitionSpecBuilder { #[cfg(test)] mod tests { use super::*; - use crate::spec::Type; + use crate::spec::{PrimitiveType, Type}; #[test] fn test_partition_spec() { @@ -663,7 +770,7 @@ mod tests { } "#; - let partition_spec: PartitionSpec = serde_json::from_str(spec).unwrap(); + let partition_spec: SchemalessPartitionSpec = serde_json::from_str(spec).unwrap(); assert_eq!(4, partition_spec.fields[0].source_id); assert_eq!(1000, partition_spec.fields[0].field_id); assert_eq!("ts_day", partition_spec.fields[0].name); @@ -695,7 +802,7 @@ mod tests { ]) .build() .unwrap(); - let partition_spec = PartitionSpec::builder(&schema) + let partition_spec = BoundPartitionSpec::builder(schema.clone()) .with_spec_id(1) .build() .unwrap(); @@ -704,7 +811,7 @@ mod tests { "Empty partition spec should be unpartitioned" ); - let partition_spec = PartitionSpec::builder(&schema) + let partition_spec = BoundPartitionSpec::builder(schema.clone()) .add_unbound_fields(vec![ UnboundPartitionField::builder() .source_id(1) @@ -726,7 +833,7 @@ mod tests { "Partition spec with one non void transform should not be unpartitioned" ); - let partition_spec = PartitionSpec::builder(&schema) + let partition_spec = BoundPartitionSpec::builder(schema.clone()) .with_spec_id(1) .add_unbound_fields(vec![ UnboundPartitionField::builder() @@ -809,6 +916,32 @@ mod tests { assert_eq!(Transform::Day, partition_spec.fields[0].transform); } + #[test] + fn test_new_unpartition() { + let schema = Schema::builder() + .with_fields(vec![ + NestedField::required(1, "id", Type::Primitive(crate::spec::PrimitiveType::Int)) + .into(), + NestedField::required( + 2, + "name", + Type::Primitive(crate::spec::PrimitiveType::String), + ) + .into(), + ]) + .build() + .unwrap(); + let partition_spec = BoundPartitionSpec::builder(schema.clone()) + .with_spec_id(0) + .build() + .unwrap(); + let partition_type = partition_spec.partition_type(); + assert_eq!(0, partition_type.fields().len()); + + let unpartition_spec = BoundPartitionSpec::unpartition_spec(schema); + assert_eq!(partition_spec, unpartition_spec); + } + #[test] fn test_partition_type() { let spec = r#" @@ -833,7 +966,7 @@ mod tests { } "#; - let partition_spec: PartitionSpec = serde_json::from_str(spec).unwrap(); + let partition_spec: SchemalessPartitionSpec = serde_json::from_str(spec).unwrap(); let schema = Schema::builder() .with_fields(vec![ NestedField::required(1, "id", Type::Primitive(crate::spec::PrimitiveType::Int)) @@ -909,7 +1042,7 @@ mod tests { } "#; - let partition_spec: PartitionSpec = serde_json::from_str(spec).unwrap(); + let partition_spec: SchemalessPartitionSpec = serde_json::from_str(spec).unwrap(); let schema = Schema::builder() .with_fields(vec![ NestedField::required(1, "id", Type::Primitive(crate::spec::PrimitiveType::Int)) @@ -976,7 +1109,7 @@ mod tests { } "#; - let partition_spec: PartitionSpec = serde_json::from_str(spec).unwrap(); + let partition_spec: SchemalessPartitionSpec = serde_json::from_str(spec).unwrap(); let schema = Schema::builder() .with_fields(vec![ NestedField::required(1, "id", Type::Primitive(crate::spec::PrimitiveType::Int)) @@ -994,6 +1127,50 @@ mod tests { assert!(partition_spec.partition_type(&schema).is_err()); } + #[test] + fn test_schemaless_bind_schema_keeps_field_ids_and_spec_id() { + let schema: Schema = Schema::builder() + .with_fields(vec![ + NestedField::required(1, "id", Type::Primitive(crate::spec::PrimitiveType::Int)) + .into(), + NestedField::required( + 2, + "name", + Type::Primitive(crate::spec::PrimitiveType::String), + ) + .into(), + ]) + .build() + .unwrap(); + + let partition_spec = BoundPartitionSpec::builder(schema.clone()) + .with_spec_id(99) + .add_unbound_field(UnboundPartitionField { + source_id: 1, + field_id: Some(1010), + name: "id".to_string(), + transform: Transform::Identity, + }) + .unwrap() + .add_unbound_field(UnboundPartitionField { + source_id: 2, + field_id: Some(1001), + name: "name_void".to_string(), + transform: Transform::Void, + }) + .unwrap() + .build() + .unwrap(); + + let schemaless_partition_spec = SchemalessPartitionSpec::from(partition_spec.clone()); + let bound_partition_spec = schemaless_partition_spec.bind(schema).unwrap(); + + assert_eq!(partition_spec, bound_partition_spec); + assert_eq!(partition_spec.fields[0].field_id, 1010); + assert_eq!(partition_spec.fields[1].field_id, 1001); + assert_eq!(bound_partition_spec.spec_id(), 99); + } + #[test] fn test_builder_disallow_duplicate_names() { UnboundPartitionSpec::builder() @@ -1018,7 +1195,7 @@ mod tests { ]) .build() .unwrap(); - PartitionSpec::builder(&schema) + BoundPartitionSpec::builder(schema.clone()) .add_unbound_field(UnboundPartitionField { source_id: 1, field_id: Some(1000), @@ -1056,7 +1233,7 @@ mod tests { ]) .build() .unwrap(); - let spec = PartitionSpec::builder(&schema) + let spec = BoundPartitionSpec::builder(schema.clone()) .with_spec_id(1) .add_unbound_field(UnboundPartitionField { source_id: 1, @@ -1104,26 +1281,33 @@ mod tests { .build() .unwrap(); - PartitionSpec::builder(&schema) + BoundPartitionSpec::builder(schema.clone()) .with_spec_id(1) .build() .unwrap(); - let spec = PartitionSpec::builder(&schema) + let spec = BoundPartitionSpec::builder(schema.clone()) .with_spec_id(1) .add_partition_field("id", "id_bucket[16]", Transform::Bucket(16)) .unwrap() .build() .unwrap(); - assert_eq!(spec, PartitionSpec { + assert_eq!(spec, BoundPartitionSpec { spec_id: 1, + schema: schema.into(), fields: vec![PartitionField { source_id: 1, field_id: 1000, name: "id_bucket[16]".to_string(), transform: Transform::Bucket(16), - }] + }], + partition_type: StructType::new(vec![NestedField::optional( + 1000, + "id_bucket[16]", + Type::Primitive(PrimitiveType::Int) + ) + .into()]) }); } @@ -1139,12 +1323,12 @@ mod tests { .build() .unwrap(); - PartitionSpec::builder(&schema) + BoundPartitionSpec::builder(schema.clone()) .with_spec_id(1) .build() .unwrap(); - let err = PartitionSpec::builder(&schema) + let err = BoundPartitionSpec::builder(schema) .with_spec_id(1) .add_unbound_field(UnboundPartitionField { source_id: 1, @@ -1172,12 +1356,12 @@ mod tests { .build() .unwrap(); - PartitionSpec::builder(&schema) + BoundPartitionSpec::builder(schema.clone()) .with_spec_id(1) .build() .unwrap(); - PartitionSpec::builder(&schema) + BoundPartitionSpec::builder(schema.clone()) .with_spec_id(1) .add_unbound_field(UnboundPartitionField { source_id: 1, @@ -1190,7 +1374,7 @@ mod tests { .unwrap(); // Not OK for different source id - PartitionSpec::builder(&schema) + BoundPartitionSpec::builder(schema) .with_spec_id(1) .add_unbound_field(UnboundPartitionField { source_id: 2, @@ -1224,7 +1408,7 @@ mod tests { .unwrap(); // Valid - PartitionSpec::builder(&schema) + BoundPartitionSpec::builder(schema.clone()) .with_spec_id(1) .add_unbound_fields(vec![ UnboundPartitionField { @@ -1245,7 +1429,7 @@ mod tests { .unwrap(); // Invalid - PartitionSpec::builder(&schema) + BoundPartitionSpec::builder(schema) .with_spec_id(1) .add_unbound_fields(vec![ UnboundPartitionField { @@ -1291,7 +1475,7 @@ mod tests { .build() .unwrap(); - PartitionSpec::builder(&schema) + BoundPartitionSpec::builder(schema) .with_spec_id(1) .add_unbound_field(UnboundPartitionField { source_id: 1, @@ -1342,7 +1526,7 @@ mod tests { .build() .unwrap(); - let partition_spec_1 = PartitionSpec::builder(&schema) + let partition_spec_1 = BoundPartitionSpec::builder(schema.clone()) .with_spec_id(1) .add_unbound_field(UnboundPartitionField { source_id: 1, @@ -1354,7 +1538,7 @@ mod tests { .build() .unwrap(); - let partition_spec_2 = PartitionSpec::builder(&schema) + let partition_spec_2 = BoundPartitionSpec::builder(schema) .with_spec_id(1) .add_unbound_field(UnboundPartitionField { source_id: 1, @@ -1366,7 +1550,7 @@ mod tests { .build() .unwrap(); - assert!(partition_spec_1.is_compatible_with(&partition_spec_2.into_unbound())); + assert!(partition_spec_1.is_compatible_with_schemaless(&partition_spec_2.into_schemaless())); } #[test] @@ -1381,7 +1565,7 @@ mod tests { .build() .unwrap(); - let partition_spec_1 = PartitionSpec::builder(&schema) + let partition_spec_1 = BoundPartitionSpec::builder(schema.clone()) .with_spec_id(1) .add_unbound_field(UnboundPartitionField { source_id: 1, @@ -1393,7 +1577,7 @@ mod tests { .build() .unwrap(); - let partition_spec_2 = PartitionSpec::builder(&schema) + let partition_spec_2 = BoundPartitionSpec::builder(schema) .with_spec_id(1) .add_unbound_field(UnboundPartitionField { source_id: 1, @@ -1405,7 +1589,9 @@ mod tests { .build() .unwrap(); - assert!(!partition_spec_1.is_compatible_with(&partition_spec_2.into_unbound())); + assert!( + !partition_spec_1.is_compatible_with_schemaless(&partition_spec_2.into_schemaless()) + ); } #[test] @@ -1424,7 +1610,7 @@ mod tests { .build() .unwrap(); - let partition_spec_1 = PartitionSpec::builder(&schema) + let partition_spec_1 = BoundPartitionSpec::builder(schema.clone()) .with_spec_id(1) .add_unbound_field(UnboundPartitionField { source_id: 1, @@ -1436,7 +1622,7 @@ mod tests { .build() .unwrap(); - let partition_spec_2 = PartitionSpec::builder(&schema) + let partition_spec_2 = BoundPartitionSpec::builder(schema) .with_spec_id(1) .add_unbound_field(UnboundPartitionField { source_id: 2, @@ -1448,7 +1634,9 @@ mod tests { .build() .unwrap(); - assert!(!partition_spec_1.is_compatible_with(&partition_spec_2.into_unbound())); + assert!( + !partition_spec_1.is_compatible_with_schemaless(&partition_spec_2.into_schemaless()) + ); } #[test] @@ -1467,7 +1655,7 @@ mod tests { .build() .unwrap(); - let partition_spec_1 = PartitionSpec::builder(&schema) + let partition_spec_1 = BoundPartitionSpec::builder(schema.clone()) .with_spec_id(1) .add_unbound_field(UnboundPartitionField { source_id: 1, @@ -1486,7 +1674,7 @@ mod tests { .build() .unwrap(); - let partition_spec_2 = PartitionSpec::builder(&schema) + let partition_spec_2 = BoundPartitionSpec::builder(schema) .with_spec_id(1) .add_unbound_field(UnboundPartitionField { source_id: 2, @@ -1505,17 +1693,20 @@ mod tests { .build() .unwrap(); - assert!(!partition_spec_1.is_compatible_with(&partition_spec_2.into_unbound())); + assert!( + !partition_spec_1.is_compatible_with_schemaless(&partition_spec_2.into_schemaless()) + ); } #[test] fn test_highest_field_id_unpartitioned() { - let spec = PartitionSpec::builder(&Schema::builder().with_fields(vec![]).build().unwrap()) - .with_spec_id(1) - .build() - .unwrap(); + let spec = + BoundPartitionSpec::builder(Schema::builder().with_fields(vec![]).build().unwrap()) + .with_spec_id(1) + .build() + .unwrap(); - assert_eq!(UNPARTITIONED_LAST_ASSIGNED_ID, spec.highest_field_id()); + assert!(spec.highest_field_id().is_none()); } #[test] @@ -1534,7 +1725,7 @@ mod tests { .build() .unwrap(); - let spec = PartitionSpec::builder(&schema) + let spec = BoundPartitionSpec::builder(schema) .with_spec_id(1) .add_unbound_field(UnboundPartitionField { source_id: 1, @@ -1553,7 +1744,7 @@ mod tests { .build() .unwrap(); - assert_eq!(1001, spec.highest_field_id()); + assert_eq!(Some(1001), spec.highest_field_id()); } #[test] @@ -1572,7 +1763,7 @@ mod tests { .build() .unwrap(); - let spec = PartitionSpec::builder(&schema) + let spec = BoundPartitionSpec::builder(schema) .with_spec_id(1) .add_unbound_field(UnboundPartitionField { source_id: 1, @@ -1612,7 +1803,7 @@ mod tests { .build() .unwrap(); - let spec = PartitionSpec::builder(&schema) + let spec = BoundPartitionSpec::builder(schema) .with_spec_id(1) .add_unbound_field(UnboundPartitionField { source_id: 1, @@ -1652,7 +1843,7 @@ mod tests { .build() .unwrap(); - let spec = PartitionSpec::builder(&schema) + let spec = BoundPartitionSpec::builder(schema) .with_spec_id(1) .add_unbound_field(UnboundPartitionField { source_id: 1, diff --git a/crates/iceberg/src/spec/table_metadata.rs b/crates/iceberg/src/spec/table_metadata.rs index cde709375..4a2e3ab73 100644 --- a/crates/iceberg/src/spec/table_metadata.rs +++ b/crates/iceberg/src/spec/table_metadata.rs @@ -31,8 +31,8 @@ use uuid::Uuid; use super::snapshot::SnapshotReference; use super::{ - PartitionSpec, PartitionSpecRef, SchemaId, SchemaRef, Snapshot, SnapshotRef, SnapshotRetention, - SortOrder, SortOrderRef, DEFAULT_PARTITION_SPEC_ID, + BoundPartitionSpec, BoundPartitionSpecRef, SchemaId, SchemaRef, SchemalessPartitionSpecRef, + Snapshot, SnapshotRef, SnapshotRetention, SortOrder, SortOrderRef, DEFAULT_PARTITION_SPEC_ID, }; use crate::error::{timestamp_ms_to_utc, Result}; use crate::{Error, ErrorKind, TableCreation}; @@ -118,9 +118,9 @@ pub struct TableMetadata { /// ID of the table’s current schema. pub(crate) current_schema_id: i32, /// A list of partition specs, stored as full partition spec objects. - pub(crate) partition_specs: HashMap, + pub(crate) partition_specs: HashMap, /// ID of the “current” spec that writers should use by default. - pub(crate) default_spec_id: i32, + pub(crate) default_spec: BoundPartitionSpecRef, /// An integer; the highest assigned partition field ID across all partition specs for the table. pub(crate) last_partition_id: i32, ///A string to string map of table properties. This is used to control settings that @@ -222,21 +222,26 @@ impl TableMetadata { /// Returns all partition specs. #[inline] - pub fn partition_specs_iter(&self) -> impl Iterator { + pub fn partition_specs_iter(&self) -> impl Iterator { self.partition_specs.values() } /// Lookup partition spec by id. #[inline] - pub fn partition_spec_by_id(&self, spec_id: i32) -> Option<&PartitionSpecRef> { + pub fn partition_spec_by_id(&self, spec_id: i32) -> Option<&SchemalessPartitionSpecRef> { self.partition_specs.get(&spec_id) } /// Get default partition spec #[inline] - pub fn default_partition_spec(&self) -> &PartitionSpecRef { - self.partition_spec_by_id(self.default_spec_id) - .expect("Default partition spec id set, but not found in table metadata") + pub fn default_partition_spec(&self) -> &BoundPartitionSpecRef { + &self.default_spec + } + + #[inline] + /// Returns spec id of the "current" partition spec. + pub fn default_partition_spec_id(&self) -> i32 { + self.default_spec.spec_id() } /// Returns all snapshots @@ -352,29 +357,18 @@ impl TableMetadata { Ok(self) } - /// If the default partition spec is specified but the spec is not present in specs, add it + /// If the default partition spec is not present in specs, add it fn try_normalize_partition_spec(&mut self) -> Result<()> { - if self.partition_spec_by_id(self.default_spec_id).is_some() { - return Ok(()); - } - - if self.default_spec_id != DEFAULT_PARTITION_SPEC_ID { - return Err(Error::new( - ErrorKind::DataInvalid, - format!( - "No partition spec exists with the default spec id {}.", - self.default_spec_id - ), - )); + if self + .partition_spec_by_id(self.default_spec.spec_id()) + .is_none() + { + self.partition_specs.insert( + self.default_spec.spec_id(), + Arc::new(Arc::unwrap_or_clone(self.default_spec.clone()).into_schemaless()), + ); } - let partition_spec = PartitionSpec { - spec_id: DEFAULT_PARTITION_SPEC_ID, - fields: vec![], - }; - self.partition_specs - .insert(DEFAULT_PARTITION_SPEC_ID, Arc::new(partition_spec)); - Ok(()) } @@ -565,6 +559,8 @@ impl TableMetadataBuilder { properties, } = table_creation; + let schema: Arc = Arc::new(schema); + let unpartition_spec = BoundPartitionSpec::unpartition_spec(schema.clone()); let partition_specs = match partition_spec { Some(_) => { return Err(Error::new( @@ -573,11 +569,8 @@ impl TableMetadataBuilder { )) } None => HashMap::from([( - DEFAULT_PARTITION_SPEC_ID, - Arc::new(PartitionSpec { - spec_id: DEFAULT_PARTITION_SPEC_ID, - fields: vec![], - }), + unpartition_spec.spec_id(), + Arc::new(unpartition_spec.clone().into_schemaless()), )]), }; @@ -607,9 +600,9 @@ impl TableMetadataBuilder { last_updated_ms: Utc::now().timestamp_millis(), last_column_id: schema.highest_field_id(), current_schema_id: schema.schema_id(), - schemas: HashMap::from([(schema.schema_id(), Arc::new(schema))]), + schemas: HashMap::from([(schema.schema_id(), schema)]), partition_specs, - default_spec_id: DEFAULT_PARTITION_SPEC_ID, + default_spec: BoundPartitionSpecRef::new(unpartition_spec), last_partition_id: 0, properties, current_snapshot_id: None, @@ -661,8 +654,8 @@ pub(super) mod _serde { use crate::spec::schema::_serde::{SchemaV1, SchemaV2}; use crate::spec::snapshot::_serde::{SnapshotV1, SnapshotV2}; use crate::spec::{ - PartitionField, PartitionSpec, Schema, Snapshot, SnapshotReference, SnapshotRetention, - SortOrder, + BoundPartitionSpec, PartitionField, Schema, SchemaRef, SchemalessPartitionSpec, Snapshot, + SnapshotReference, SnapshotRetention, SortOrder, }; use crate::{Error, ErrorKind}; @@ -685,7 +678,7 @@ pub(super) mod _serde { pub last_column_id: i32, pub schemas: Vec, pub current_schema_id: i32, - pub partition_specs: Vec, + pub partition_specs: Vec, pub default_spec_id: i32, pub last_partition_id: i32, #[serde(skip_serializing_if = "Option::is_none")] @@ -721,7 +714,7 @@ pub(super) mod _serde { pub current_schema_id: Option, pub partition_spec: Vec, #[serde(skip_serializing_if = "Option::is_none")] - pub partition_specs: Option>, + pub partition_specs: Option>, #[serde(skip_serializing_if = "Option::is_none")] pub default_spec_id: Option, #[serde(skip_serializing_if = "Option::is_none")] @@ -809,6 +802,44 @@ pub(super) mod _serde { .map(|schema| Ok((schema.schema_id, Arc::new(schema.try_into()?)))) .collect::, Error>>()?, ); + + let current_schema: &SchemaRef = + schemas.get(&value.current_schema_id).ok_or_else(|| { + Error::new( + ErrorKind::DataInvalid, + format!( + "No schema exists with the current schema id {}.", + value.current_schema_id + ), + ) + })?; + let partition_specs = HashMap::from_iter( + value + .partition_specs + .into_iter() + .map(|x| (x.spec_id(), Arc::new(x))), + ); + let default_spec_id = value.default_spec_id; + let default_spec = partition_specs + .get(&value.default_spec_id) + .map(|schemaless_spec| { + (*schemaless_spec.clone()) + .clone() + .bind(current_schema.clone()) + }) + .transpose()? + .or_else(|| { + (DEFAULT_PARTITION_SPEC_ID == default_spec_id) + .then(|| BoundPartitionSpec::unpartition_spec(current_schema.clone())) + }) + .ok_or_else(|| { + Error::new( + ErrorKind::DataInvalid, + format!("Default partition spec {default_spec_id} not found"), + ) + })? + .into(); + let mut metadata = TableMetadata { format_version: FormatVersion::V2, table_uuid: value.table_uuid, @@ -818,13 +849,8 @@ pub(super) mod _serde { last_column_id: value.last_column_id, current_schema_id: value.current_schema_id, schemas, - partition_specs: HashMap::from_iter( - value - .partition_specs - .into_iter() - .map(|x| (x.spec_id(), Arc::new(x))), - ), - default_spec_id: value.default_spec_id, + partition_specs, + default_spec, last_partition_id: value.last_partition_id, properties: value.properties.unwrap_or_default(), current_snapshot_id, @@ -876,6 +902,7 @@ pub(super) mod _serde { } else { value.current_snapshot_id }; + let schemas = value .schemas .map(|schemas| { @@ -900,18 +927,49 @@ pub(super) mod _serde { }) .transpose()? .unwrap_or_default(); - let partition_specs = HashMap::from_iter( - value - .partition_specs - .unwrap_or_else(|| { - vec![PartitionSpec { - spec_id: DEFAULT_PARTITION_SPEC_ID, - fields: value.partition_spec, - }] - }) - .into_iter() - .map(|x| (x.spec_id(), Arc::new(x))), - ); + let current_schema_id = value + .current_schema_id + .unwrap_or_else(|| schemas.keys().copied().max().unwrap_or_default()); + let current_schema = schemas + .get(¤t_schema_id) + .ok_or_else(|| { + Error::new( + ErrorKind::DataInvalid, + format!( + "No schema exists with the current schema id {}.", + current_schema_id + ), + ) + })? + .clone(); + + let partition_specs = match value.partition_specs { + Some(partition_specs) => partition_specs, + None => vec![BoundPartitionSpec::builder(current_schema.clone()) + .with_spec_id(DEFAULT_PARTITION_SPEC_ID) + .add_unbound_fields(value.partition_spec.into_iter().map(|f| f.into_unbound()))? + .build()? + .into_schemaless()], + } + .into_iter() + .map(|x| (x.spec_id(), Arc::new(x))) + .collect::>(); + + let default_spec_id = value + .default_spec_id + .unwrap_or_else(|| partition_specs.keys().copied().max().unwrap_or_default()); + let default_spec = partition_specs + .get(&default_spec_id) + .map(|x| Arc::unwrap_or_clone(x.clone()).bind(current_schema.clone())) + .transpose()? + .ok_or_else(|| { + Error::new( + ErrorKind::DataInvalid, + format!("Default partition spec {default_spec_id} not found"), + ) + })? + .into(); + let mut metadata = TableMetadata { format_version: FormatVersion::V1, table_uuid: value.table_uuid.unwrap_or_default(), @@ -919,12 +977,8 @@ pub(super) mod _serde { last_sequence_number: 0, last_updated_ms: value.last_updated_ms, last_column_id: value.last_column_id, - current_schema_id: value - .current_schema_id - .unwrap_or_else(|| schemas.keys().copied().max().unwrap_or_default()), - default_spec_id: value - .default_spec_id - .unwrap_or_else(|| partition_specs.keys().copied().max().unwrap_or_default()), + current_schema_id, + default_spec, last_partition_id: value .last_partition_id .unwrap_or_else(|| partition_specs.keys().copied().max().unwrap_or_default()), @@ -998,7 +1052,7 @@ pub(super) mod _serde { .into_values() .map(|x| Arc::try_unwrap(x).unwrap_or_else(|s| s.as_ref().clone())) .collect(), - default_spec_id: v.default_spec_id, + default_spec_id: v.default_spec.spec_id(), last_partition_id: v.last_partition_id, properties: Some(v.properties), current_snapshot_id: v.current_snapshot_id.or(Some(-1)), @@ -1067,18 +1121,14 @@ pub(super) mod _serde { .collect(), ), current_schema_id: Some(v.current_schema_id), - partition_spec: v - .partition_specs - .get(&v.default_spec_id) - .map(|x| x.fields().to_vec()) - .unwrap_or_default(), + partition_spec: v.default_spec.fields().to_vec(), partition_specs: Some( v.partition_specs .into_values() .map(|x| Arc::try_unwrap(x).unwrap_or_else(|s| s.as_ref().clone())) .collect(), ), - default_spec_id: Some(v.default_spec_id), + default_spec_id: Some(v.default_spec.spec_id()), last_partition_id: Some(v.last_partition_id), properties: if v.properties.is_empty() { None @@ -1195,9 +1245,9 @@ mod tests { use super::{FormatVersion, MetadataLog, SnapshotLog, TableMetadataBuilder}; use crate::spec::table_metadata::TableMetadata; use crate::spec::{ - NestedField, NullOrder, Operation, PartitionField, PartitionSpec, PrimitiveType, Schema, - Snapshot, SnapshotReference, SnapshotRetention, SortDirection, SortField, SortOrder, - Summary, Transform, Type, UnboundPartitionField, + BoundPartitionSpec, NestedField, NullOrder, Operation, PrimitiveType, Schema, Snapshot, + SnapshotReference, SnapshotRetention, SortDirection, SortField, SortOrder, Summary, + Transform, Type, UnboundPartitionField, }; use crate::TableCreation; @@ -1238,6 +1288,12 @@ mod tests { "name": "struct_name", "required": true, "type": "fixed[1]" + }, + { + "id": 4, + "name": "ts", + "required": true, + "type": "timestamp" } ] } @@ -1279,23 +1335,32 @@ mod tests { let schema = Schema::builder() .with_schema_id(1) - .with_fields(vec![Arc::new(NestedField::required( - 1, - "struct_name", - Type::Primitive(PrimitiveType::Fixed(1)), - ))]) + .with_fields(vec![ + Arc::new(NestedField::required( + 1, + "struct_name", + Type::Primitive(PrimitiveType::Fixed(1)), + )), + Arc::new(NestedField::required( + 4, + "ts", + Type::Primitive(PrimitiveType::Timestamp), + )), + ]) .build() .unwrap(); - let partition_spec = PartitionSpec { - spec_id: 0, - fields: vec![PartitionField { + let partition_spec = BoundPartitionSpec::builder(schema.clone()) + .with_spec_id(0) + .add_unbound_field(UnboundPartitionField { name: "ts_day".to_string(), transform: Transform::Day, source_id: 4, - field_id: 1000, - }], - }; + field_id: Some(1000), + }) + .unwrap() + .build() + .unwrap(); let expected = TableMetadata { format_version: FormatVersion::V2, @@ -1305,8 +1370,11 @@ mod tests { last_column_id: 1, schemas: HashMap::from_iter(vec![(1, Arc::new(schema))]), current_schema_id: 1, - partition_specs: HashMap::from_iter(vec![(0, partition_spec.into())]), - default_spec_id: 0, + partition_specs: HashMap::from_iter(vec![( + 0, + partition_spec.clone().into_schemaless().into(), + )]), + default_spec: partition_spec.into(), last_partition_id: 1000, default_sort_order_id: 0, sort_orders: HashMap::from_iter(vec![(0, SortOrder::unsorted_order().into())]), @@ -1445,7 +1513,8 @@ mod tests { .build() .unwrap(); - let partition_spec = PartitionSpec::builder(&schema) + let schema = Arc::new(schema); + let partition_spec = BoundPartitionSpec::builder(schema.clone()) .with_spec_id(0) .add_partition_field("vendor_id", "vendor_id", Transform::Identity) .unwrap() @@ -1472,10 +1541,10 @@ mod tests { location: "/home/iceberg/warehouse/nyc/taxis".to_string(), last_updated_ms: 1662532818843, last_column_id: 5, - schemas: HashMap::from_iter(vec![(0, Arc::new(schema))]), + schemas: HashMap::from_iter(vec![(0, schema)]), current_schema_id: 0, - partition_specs: HashMap::from_iter(vec![(0, partition_spec.into())]), - default_spec_id: 0, + partition_specs: HashMap::from_iter(vec![(0, partition_spec.clone().into_schemaless().into())]), + default_spec: Arc::new(partition_spec), last_partition_id: 1000, default_sort_order_id: 0, sort_orders: HashMap::from_iter(vec![(0, sort_order.into())]), @@ -1514,6 +1583,12 @@ mod tests { "name": "struct_name", "required": true, "type": "fixed[1]" + }, + { + "id": 4, + "name": "ts", + "required": true, + "type": "timestamp" } ] } @@ -1614,6 +1689,12 @@ mod tests { "name": "struct_name", "required": true, "type": "fixed[1]" + }, + { + "id": 4, + "name": "ts", + "required": true, + "type": "timestamp" } ] } @@ -1699,6 +1780,12 @@ mod tests { "name": "struct_name", "required": true, "type": "fixed[1]" + }, + { + "id": 4, + "name": "ts", + "required": true, + "type": "timestamp" } ] } @@ -1828,7 +1915,7 @@ mod tests { .build() .unwrap(); - let partition_spec = PartitionSpec::builder(&schema1) + let partition_spec = BoundPartitionSpec::builder(schema2.clone()) .with_spec_id(0) .add_unbound_field(UnboundPartitionField { name: "x".to_string(), @@ -1889,8 +1976,11 @@ mod tests { last_column_id: 3, schemas: HashMap::from_iter(vec![(0, Arc::new(schema1)), (1, Arc::new(schema2))]), current_schema_id: 1, - partition_specs: HashMap::from_iter(vec![(0, partition_spec.into())]), - default_spec_id: 0, + partition_specs: HashMap::from_iter(vec![( + 0, + partition_spec.clone().into_schemaless().into(), + )]), + default_spec: Arc::new(partition_spec), last_partition_id: 1000, default_sort_order_id: 3, sort_orders: HashMap::from_iter(vec![(3, sort_order.into())]), @@ -1951,7 +2041,7 @@ mod tests { .build() .unwrap(); - let partition_spec = PartitionSpec::builder(&schema) + let partition_spec = BoundPartitionSpec::builder(schema.clone()) .with_spec_id(0) .add_unbound_field(UnboundPartitionField { name: "x".to_string(), @@ -1988,8 +2078,11 @@ mod tests { last_column_id: 3, schemas: HashMap::from_iter(vec![(0, Arc::new(schema))]), current_schema_id: 0, - partition_specs: HashMap::from_iter(vec![(0, partition_spec.into())]), - default_spec_id: 0, + partition_specs: HashMap::from_iter(vec![( + 0, + partition_spec.clone().into_schemaless().into(), + )]), + default_spec: Arc::new(partition_spec), last_partition_id: 1000, default_sort_order_id: 3, sort_orders: HashMap::from_iter(vec![(3, sort_order.into())]), @@ -2031,7 +2124,7 @@ mod tests { .build() .unwrap(); - let partition_spec = PartitionSpec::builder(&schema) + let partition_spec = BoundPartitionSpec::builder(schema.clone()) .with_spec_id(0) .add_unbound_field(UnboundPartitionField { name: "x".to_string(), @@ -2051,8 +2144,11 @@ mod tests { last_column_id: 3, schemas: HashMap::from_iter(vec![(0, Arc::new(schema))]), current_schema_id: 0, - partition_specs: HashMap::from_iter(vec![(0, partition_spec.into())]), - default_spec_id: 0, + partition_specs: HashMap::from_iter(vec![( + 0, + partition_spec.clone().into_schemaless().into(), + )]), + default_spec: Arc::new(partition_spec), last_partition_id: 0, default_sort_order_id: 0, // Sort order is added during deserialization for V2 compatibility @@ -2165,16 +2261,22 @@ mod tests { fn test_default_partition_spec() { let default_spec_id = 1234; let mut table_meta_data = get_test_table_metadata("TableMetadataV2Valid.json"); - table_meta_data.default_spec_id = default_spec_id; + let partition_spec = + BoundPartitionSpec::unpartition_spec(table_meta_data.current_schema().clone()); + table_meta_data.default_spec = partition_spec.clone().into(); table_meta_data .partition_specs - .insert(default_spec_id, Arc::new(PartitionSpec::default())); + .insert(default_spec_id, Arc::new(partition_spec.into_schemaless())); assert_eq!( - table_meta_data.default_partition_spec(), - table_meta_data + (*table_meta_data.default_partition_spec().clone()) + .clone() + .into_schemaless(), + (*table_meta_data .partition_spec_by_id(default_spec_id) .unwrap() + .clone()) + .clone() ); } #[test] @@ -2225,10 +2327,11 @@ mod tests { HashMap::from([( 0, Arc::new( - PartitionSpec::builder(table_metadata.schemas.get(&0).unwrap()) + BoundPartitionSpec::builder(table_metadata.schemas.get(&0).unwrap().clone()) .with_spec_id(0) .build() .unwrap() + .into_schemaless() ) )]) ); diff --git a/crates/iceberg/src/writer/file_writer/location_generator.rs b/crates/iceberg/src/writer/file_writer/location_generator.rs index 44326190d..def18b580 100644 --- a/crates/iceberg/src/writer/file_writer/location_generator.rs +++ b/crates/iceberg/src/writer/file_writer/location_generator.rs @@ -132,7 +132,7 @@ pub(crate) mod test { use uuid::Uuid; use super::LocationGenerator; - use crate::spec::{FormatVersion, TableMetadata}; + use crate::spec::{BoundPartitionSpec, FormatVersion, TableMetadata}; use crate::writer::file_writer::location_generator::{ FileNameGenerator, WRITE_DATA_LOCATION, WRITE_FOLDER_STORAGE_LOCATION, }; @@ -156,6 +156,7 @@ pub(crate) mod test { #[test] fn test_default_location_generate() { + let schema = crate::spec::Schema::builder().build().unwrap(); let mut table_metadata = TableMetadata { format_version: FormatVersion::V2, table_uuid: Uuid::parse_str("fb072c92-a02b-11e9-ae9c-1bb7bc9eca94").unwrap(), @@ -165,7 +166,7 @@ pub(crate) mod test { schemas: HashMap::new(), current_schema_id: 1, partition_specs: HashMap::new(), - default_spec_id: 1, + default_spec: BoundPartitionSpec::unpartition_spec(schema).into(), last_partition_id: 1000, default_sort_order_id: 0, sort_orders: HashMap::from_iter(vec![]), From 0ef93043963c851361c45aa79b27fd2ed463b355 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Mon, 4 Nov 2024 09:30:24 +0800 Subject: [PATCH 31/46] chore(deps): Bump crate-ci/typos from 1.26.8 to 1.27.0 (#687) Bumps [crate-ci/typos](https://github.com/crate-ci/typos) from 1.26.8 to 1.27.0. - [Release notes](https://github.com/crate-ci/typos/releases) - [Changelog](https://github.com/crate-ci/typos/blob/master/CHANGELOG.md) - [Commits](https://github.com/crate-ci/typos/compare/v1.26.8...v1.27.0) --- updated-dependencies: - dependency-name: crate-ci/typos dependency-type: direct:production update-type: version-update:semver-minor ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- .github/workflows/ci_typos.yml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/.github/workflows/ci_typos.yml b/.github/workflows/ci_typos.yml index c086f0a3b..1e715cba3 100644 --- a/.github/workflows/ci_typos.yml +++ b/.github/workflows/ci_typos.yml @@ -42,4 +42,4 @@ jobs: steps: - uses: actions/checkout@v4 - name: Check typos - uses: crate-ci/typos@v1.26.8 + uses: crate-ci/typos@v1.27.0 From 92cc45cb649f9bcdd7dae1c1deb33e5a6eec0474 Mon Sep 17 00:00:00 2001 From: Christian Date: Fri, 8 Nov 2024 14:01:37 +0100 Subject: [PATCH 32/46] Add TableMetadata Accessors (#688) --- crates/iceberg/src/spec/table_metadata.rs | 18 ++++++++++++++++++ 1 file changed, 18 insertions(+) diff --git a/crates/iceberg/src/spec/table_metadata.rs b/crates/iceberg/src/spec/table_metadata.rs index 4a2e3ab73..74e690077 100644 --- a/crates/iceberg/src/spec/table_metadata.rs +++ b/crates/iceberg/src/spec/table_metadata.rs @@ -220,6 +220,12 @@ impl TableMetadata { .expect("Current schema id set, but not found in table metadata") } + /// Get the id of the current schema + #[inline] + pub fn current_schema_id(&self) -> SchemaId { + self.current_schema_id + } + /// Returns all partition specs. #[inline] pub fn partition_specs_iter(&self) -> impl Iterator { @@ -277,6 +283,12 @@ impl TableMetadata { }) } + /// Get the current snapshot id + #[inline] + pub fn current_snapshot_id(&self) -> Option { + self.current_snapshot_id + } + /// Get the snapshot for a reference /// Returns an option if the `ref_name` is not found #[inline] @@ -307,6 +319,12 @@ impl TableMetadata { .expect("Default order id has been set, but not found in table metadata!") } + /// Returns default sort order id. + #[inline] + pub fn default_sort_order_id(&self) -> i64 { + self.default_sort_order_id + } + /// Returns properties of table. #[inline] pub fn properties(&self) -> &HashMap { From 5692c0d051c9faa273f4cca80e9b49943ada863b Mon Sep 17 00:00:00 2001 From: Marko Grujic Date: Sat, 9 Nov 2024 08:27:26 +0100 Subject: [PATCH 33/46] Upgrade to DataFusion 43 (#691) --- crates/integrations/datafusion/Cargo.toml | 4 ++-- crates/integrations/datafusion/src/catalog.rs | 1 + crates/integrations/datafusion/src/schema.rs | 1 + crates/integrations/datafusion/src/table/mod.rs | 1 + .../datafusion/src/table/table_provider_factory.rs | 3 ++- 5 files changed, 7 insertions(+), 3 deletions(-) diff --git a/crates/integrations/datafusion/Cargo.toml b/crates/integrations/datafusion/Cargo.toml index c4acf1178..d3930cfda 100644 --- a/crates/integrations/datafusion/Cargo.toml +++ b/crates/integrations/datafusion/Cargo.toml @@ -23,7 +23,7 @@ homepage = { workspace = true } rust-version = { workspace = true } categories = ["database"] -description = "Apache Iceberg Datafusion Integration" +description = "Apache Iceberg DataFusion Integration" repository = { workspace = true } license = { workspace = true } keywords = ["iceberg", "integrations", "datafusion"] @@ -31,7 +31,7 @@ keywords = ["iceberg", "integrations", "datafusion"] [dependencies] anyhow = { workspace = true } async-trait = { workspace = true } -datafusion = { version = "42" } +datafusion = { version = "43" } futures = { workspace = true } iceberg = { workspace = true } tokio = { workspace = true } diff --git a/crates/integrations/datafusion/src/catalog.rs b/crates/integrations/datafusion/src/catalog.rs index ab6ebdccc..c3cbcc88b 100644 --- a/crates/integrations/datafusion/src/catalog.rs +++ b/crates/integrations/datafusion/src/catalog.rs @@ -30,6 +30,7 @@ use crate::schema::IcebergSchemaProvider; /// /// Acts as a centralized catalog provider that aggregates /// multiple [`SchemaProvider`], each associated with distinct namespaces. +#[derive(Debug)] pub struct IcebergCatalogProvider { /// A `HashMap` where keys are namespace names /// and values are dynamic references to objects implementing the diff --git a/crates/integrations/datafusion/src/schema.rs b/crates/integrations/datafusion/src/schema.rs index 8133b3746..3be6da426 100644 --- a/crates/integrations/datafusion/src/schema.rs +++ b/crates/integrations/datafusion/src/schema.rs @@ -30,6 +30,7 @@ use crate::table::IcebergTableProvider; /// Represents a [`SchemaProvider`] for the Iceberg [`Catalog`], managing /// access to table providers within a specific namespace. +#[derive(Debug)] pub(crate) struct IcebergSchemaProvider { /// A `HashMap` where keys are table names /// and values are dynamic references to objects implementing the diff --git a/crates/integrations/datafusion/src/table/mod.rs b/crates/integrations/datafusion/src/table/mod.rs index 82f29bb52..f0f6514c6 100644 --- a/crates/integrations/datafusion/src/table/mod.rs +++ b/crates/integrations/datafusion/src/table/mod.rs @@ -35,6 +35,7 @@ use crate::physical_plan::scan::IcebergTableScan; /// Represents a [`TableProvider`] for the Iceberg [`Catalog`], /// managing access to a [`Table`]. +#[derive(Debug)] pub struct IcebergTableProvider { /// A table in the catalog. table: Table, diff --git a/crates/integrations/datafusion/src/table/table_provider_factory.rs b/crates/integrations/datafusion/src/table/table_provider_factory.rs index b8e66bd30..15a3fef68 100644 --- a/crates/integrations/datafusion/src/table/table_provider_factory.rs +++ b/crates/integrations/datafusion/src/table/table_provider_factory.rs @@ -97,7 +97,7 @@ use crate::to_datafusion_error; /// # Errors /// An error will be returned if any unsupported feature, such as partition columns, /// order expressions, constraints, or column defaults, is detected in the table creation command. -#[derive(Default)] +#[derive(Debug, Default)] pub struct IcebergTableProviderFactory {} impl IcebergTableProviderFactory { @@ -241,6 +241,7 @@ mod tests { constraints: Constraints::empty(), column_defaults: Default::default(), if_not_exists: Default::default(), + temporary: false, definition: Default::default(), unbounded: Default::default(), } From 40e3ea4a9b520830dddb32754caecd2840f0794d Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Mon, 11 Nov 2024 14:43:15 +0800 Subject: [PATCH 34/46] chore(deps): Bump crate-ci/typos from 1.27.0 to 1.27.3 (#693) Bumps [crate-ci/typos](https://github.com/crate-ci/typos) from 1.27.0 to 1.27.3. - [Release notes](https://github.com/crate-ci/typos/releases) - [Changelog](https://github.com/crate-ci/typos/blob/master/CHANGELOG.md) - [Commits](https://github.com/crate-ci/typos/compare/v1.27.0...v1.27.3) --- updated-dependencies: - dependency-name: crate-ci/typos dependency-type: direct:production update-type: version-update:semver-patch ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- .github/workflows/ci_typos.yml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/.github/workflows/ci_typos.yml b/.github/workflows/ci_typos.yml index 1e715cba3..38da965a6 100644 --- a/.github/workflows/ci_typos.yml +++ b/.github/workflows/ci_typos.yml @@ -42,4 +42,4 @@ jobs: steps: - uses: actions/checkout@v4 - name: Check typos - uses: crate-ci/typos@v1.27.0 + uses: crate-ci/typos@v1.27.3 From 213f84e97f97d6e7d9f5f3149b522144a9e56d9d Mon Sep 17 00:00:00 2001 From: Christian Date: Mon, 11 Nov 2024 07:46:32 +0100 Subject: [PATCH 35/46] Expose length of iterators via ExactSizeIterator (#692) --- crates/iceberg/src/spec/schema.rs | 2 +- crates/iceberg/src/spec/table_metadata.rs | 10 ++++++---- crates/iceberg/src/spec/values.rs | 2 +- crates/iceberg/src/spec/view_metadata.rs | 4 ++-- 4 files changed, 10 insertions(+), 8 deletions(-) diff --git a/crates/iceberg/src/spec/schema.rs b/crates/iceberg/src/spec/schema.rs index cf86874dc..eaf1fcc85 100644 --- a/crates/iceberg/src/spec/schema.rs +++ b/crates/iceberg/src/spec/schema.rs @@ -358,7 +358,7 @@ impl Schema { /// Returns [`identifier_field_ids`]. #[inline] - pub fn identifier_field_ids(&self) -> impl Iterator + '_ { + pub fn identifier_field_ids(&self) -> impl ExactSizeIterator + '_ { self.identifier_field_ids.iter().copied() } diff --git a/crates/iceberg/src/spec/table_metadata.rs b/crates/iceberg/src/spec/table_metadata.rs index 74e690077..daed758cb 100644 --- a/crates/iceberg/src/spec/table_metadata.rs +++ b/crates/iceberg/src/spec/table_metadata.rs @@ -203,7 +203,7 @@ impl TableMetadata { /// Returns schemas #[inline] - pub fn schemas_iter(&self) -> impl Iterator { + pub fn schemas_iter(&self) -> impl ExactSizeIterator { self.schemas.values() } @@ -228,7 +228,9 @@ impl TableMetadata { /// Returns all partition specs. #[inline] - pub fn partition_specs_iter(&self) -> impl Iterator { + pub fn partition_specs_iter( + &self, + ) -> impl ExactSizeIterator { self.partition_specs.values() } @@ -252,7 +254,7 @@ impl TableMetadata { /// Returns all snapshots #[inline] - pub fn snapshots(&self) -> impl Iterator { + pub fn snapshots(&self) -> impl ExactSizeIterator { self.snapshots.values() } @@ -301,7 +303,7 @@ impl TableMetadata { /// Return all sort orders. #[inline] - pub fn sort_orders_iter(&self) -> impl Iterator { + pub fn sort_orders_iter(&self) -> impl ExactSizeIterator { self.sort_orders.values() } diff --git a/crates/iceberg/src/spec/values.rs b/crates/iceberg/src/spec/values.rs index 3c6e2aa68..552ac497f 100644 --- a/crates/iceberg/src/spec/values.rs +++ b/crates/iceberg/src/spec/values.rs @@ -1560,7 +1560,7 @@ impl Struct { } /// Create a iterator to read the field in order of field_value. - pub fn iter(&self) -> impl Iterator> { + pub fn iter(&self) -> impl ExactSizeIterator> { self.null_bitmap.iter().zip(self.fields.iter()).map( |(null, value)| { if *null { diff --git a/crates/iceberg/src/spec/view_metadata.rs b/crates/iceberg/src/spec/view_metadata.rs index 741e38649..7c247c17d 100644 --- a/crates/iceberg/src/spec/view_metadata.rs +++ b/crates/iceberg/src/spec/view_metadata.rs @@ -94,7 +94,7 @@ impl ViewMetadata { /// Returns all view versions. #[inline] - pub fn versions(&self) -> impl Iterator { + pub fn versions(&self) -> impl ExactSizeIterator { self.versions.values() } @@ -114,7 +114,7 @@ impl ViewMetadata { /// Returns schemas #[inline] - pub fn schemas_iter(&self) -> impl Iterator { + pub fn schemas_iter(&self) -> impl ExactSizeIterator { self.schemas.values() } From 52296eb5e1be88a241903488c133214cb5a9a363 Mon Sep 17 00:00:00 2001 From: Christian Date: Mon, 11 Nov 2024 12:03:34 +0000 Subject: [PATCH 36/46] feat: Implement TableRequirement checks (#689) * Impelment TableRequirement check * Address comments --- crates/iceberg/src/catalog/mod.rs | 309 +++++++++++++++++++++++++++++- crates/iceberg/src/transaction.rs | 2 +- 2 files changed, 303 insertions(+), 8 deletions(-) diff --git a/crates/iceberg/src/catalog/mod.rs b/crates/iceberg/src/catalog/mod.rs index 854c1269c..536726fd3 100644 --- a/crates/iceberg/src/catalog/mod.rs +++ b/crates/iceberg/src/catalog/mod.rs @@ -29,8 +29,8 @@ use typed_builder::TypedBuilder; use uuid::Uuid; use crate::spec::{ - FormatVersion, Schema, Snapshot, SnapshotReference, SortOrder, TableMetadataBuilder, - UnboundPartitionSpec, ViewRepresentations, + FormatVersion, Schema, SchemaId, Snapshot, SnapshotReference, SortOrder, TableMetadata, + TableMetadataBuilder, UnboundPartitionSpec, ViewRepresentations, }; use crate::table::Table; use crate::{Error, ErrorKind, Result}; @@ -312,14 +312,14 @@ pub enum TableRequirement { LastAssignedFieldIdMatch { /// The last assigned field id of the table to assert. #[serde(rename = "last-assigned-field-id")] - last_assigned_field_id: i64, + last_assigned_field_id: i32, }, /// The table's current schema id must match the requirement. #[serde(rename = "assert-current-schema-id")] CurrentSchemaIdMatch { /// Current schema id of the table to assert. #[serde(rename = "current-schema-id")] - current_schema_id: i64, + current_schema_id: SchemaId, }, /// The table's last assigned partition id must match the /// requirement. @@ -327,14 +327,14 @@ pub enum TableRequirement { LastAssignedPartitionIdMatch { /// Last assigned partition id of the table to assert. #[serde(rename = "last-assigned-partition-id")] - last_assigned_partition_id: i64, + last_assigned_partition_id: i32, }, /// The table's default spec id must match the requirement. #[serde(rename = "assert-default-spec-id")] DefaultSpecIdMatch { /// Default spec id of the table to assert. #[serde(rename = "default-spec-id")] - default_spec_id: i64, + default_spec_id: i32, }, /// The table's default sort order id must match the requirement. #[serde(rename = "assert-default-sort-order-id")] @@ -453,6 +453,140 @@ impl TableUpdate { } } +impl TableRequirement { + /// Check that the requirement is met by the table metadata. + /// If the requirement is not met, an appropriate error is returned. + /// + /// Provide metadata as `None` if the table does not exist. + pub fn check(&self, metadata: Option<&TableMetadata>) -> Result<()> { + if let Some(metadata) = metadata { + match self { + TableRequirement::NotExist => { + return Err(Error::new( + ErrorKind::DataInvalid, + format!( + "Requirement failed: Table with id {} already exists", + metadata.uuid() + ), + )); + } + TableRequirement::UuidMatch { uuid } => { + if &metadata.uuid() != uuid { + return Err(Error::new( + ErrorKind::DataInvalid, + "Requirement failed: Table UUID does not match", + ) + .with_context("expected", *uuid) + .with_context("found", metadata.uuid())); + } + } + TableRequirement::CurrentSchemaIdMatch { current_schema_id } => { + // ToDo: Harmonize the types of current_schema_id + if metadata.current_schema_id != *current_schema_id { + return Err(Error::new( + ErrorKind::DataInvalid, + "Requirement failed: Current schema id does not match", + ) + .with_context("expected", current_schema_id.to_string()) + .with_context("found", metadata.current_schema_id.to_string())); + } + } + TableRequirement::DefaultSortOrderIdMatch { + default_sort_order_id, + } => { + if metadata.default_sort_order().order_id != *default_sort_order_id { + return Err(Error::new( + ErrorKind::DataInvalid, + "Requirement failed: Default sort order id does not match", + ) + .with_context("expected", default_sort_order_id.to_string()) + .with_context( + "found", + metadata.default_sort_order().order_id.to_string(), + )); + } + } + TableRequirement::RefSnapshotIdMatch { r#ref, snapshot_id } => { + let snapshot_ref = metadata.snapshot_for_ref(r#ref); + if let Some(snapshot_id) = snapshot_id { + let snapshot_ref = snapshot_ref.ok_or(Error::new( + ErrorKind::DataInvalid, + format!("Requirement failed: Branch or tag `{}` not found", r#ref), + ))?; + if snapshot_ref.snapshot_id() != *snapshot_id { + return Err(Error::new( + ErrorKind::DataInvalid, + format!( + "Requirement failed: Branch or tag `{}`'s snapshot has changed", + r#ref + ), + ) + .with_context("expected", snapshot_id.to_string()) + .with_context("found", snapshot_ref.snapshot_id().to_string())); + } + } else if snapshot_ref.is_some() { + // a null snapshot ID means the ref should not exist already + return Err(Error::new( + ErrorKind::DataInvalid, + format!( + "Requirement failed: Branch or tag `{}` already exists", + r#ref + ), + )); + } + } + TableRequirement::DefaultSpecIdMatch { default_spec_id } => { + // ToDo: Harmonize the types of default_spec_id + if metadata.default_partition_spec_id() != *default_spec_id { + return Err(Error::new( + ErrorKind::DataInvalid, + "Requirement failed: Default partition spec id does not match", + ) + .with_context("expected", default_spec_id.to_string()) + .with_context("found", metadata.default_partition_spec_id().to_string())); + } + } + TableRequirement::LastAssignedPartitionIdMatch { + last_assigned_partition_id, + } => { + if metadata.last_partition_id != *last_assigned_partition_id { + return Err(Error::new( + ErrorKind::DataInvalid, + "Requirement failed: Last assigned partition id does not match", + ) + .with_context("expected", last_assigned_partition_id.to_string()) + .with_context("found", metadata.last_partition_id.to_string())); + } + } + TableRequirement::LastAssignedFieldIdMatch { + last_assigned_field_id, + } => { + if &metadata.last_column_id != last_assigned_field_id { + return Err(Error::new( + ErrorKind::DataInvalid, + "Requirement failed: Last assigned field id does not match", + ) + .with_context("expected", last_assigned_field_id.to_string()) + .with_context("found", metadata.last_column_id.to_string())); + } + } + }; + } else { + match self { + TableRequirement::NotExist => {} + _ => { + return Err(Error::new( + ErrorKind::DataInvalid, + "Requirement failed: Table does not exist", + )); + } + } + } + + Ok(()) + } +} + pub(super) mod _serde { use serde::{Deserialize as _, Deserializer}; @@ -549,7 +683,7 @@ mod tests { use crate::spec::{ FormatVersion, NestedField, NullOrder, Operation, PrimitiveType, Schema, Snapshot, SnapshotReference, SnapshotRetention, SortDirection, SortField, SortOrder, Summary, - TableMetadataBuilder, Transform, Type, UnboundPartitionSpec, + TableMetadata, TableMetadataBuilder, Transform, Type, UnboundPartitionSpec, }; use crate::{NamespaceIdent, TableCreation, TableIdent, TableRequirement, TableUpdate}; @@ -593,6 +727,167 @@ mod tests { ); } + fn metadata() -> TableMetadata { + let tbl_creation = TableCreation::builder() + .name("table".to_string()) + .location("/path/to/table".to_string()) + .schema(Schema::builder().build().unwrap()) + .build(); + + TableMetadataBuilder::from_table_creation(tbl_creation) + .unwrap() + .assign_uuid(uuid::Uuid::nil()) + .unwrap() + .build() + .unwrap() + } + + #[test] + fn test_check_requirement_not_exist() { + let metadata = metadata(); + let requirement = TableRequirement::NotExist; + + assert!(requirement.check(Some(&metadata)).is_err()); + assert!(requirement.check(None).is_ok()); + } + + #[test] + fn test_check_table_uuid() { + let metadata = metadata(); + + let requirement = TableRequirement::UuidMatch { + uuid: uuid::Uuid::now_v7(), + }; + assert!(requirement.check(Some(&metadata)).is_err()); + + let requirement = TableRequirement::UuidMatch { + uuid: uuid::Uuid::nil(), + }; + assert!(requirement.check(Some(&metadata)).is_ok()); + } + + #[test] + fn test_check_ref_snapshot_id() { + let metadata = metadata(); + + // Ref does not exist but should + let requirement = TableRequirement::RefSnapshotIdMatch { + r#ref: "my_branch".to_string(), + snapshot_id: Some(1), + }; + assert!(requirement.check(Some(&metadata)).is_err()); + + // Ref does not exist and should not + let requirement = TableRequirement::RefSnapshotIdMatch { + r#ref: "my_branch".to_string(), + snapshot_id: None, + }; + assert!(requirement.check(Some(&metadata)).is_ok()); + + // Add snapshot + let record = r#" + { + "snapshot-id": 3051729675574597004, + "sequence-number": 10, + "timestamp-ms": 1515100955770, + "summary": { + "operation": "append" + }, + "manifest-list": "s3://b/wh/.../s1.avro", + "schema-id": 0 + } + "#; + + let snapshot = serde_json::from_str::(record).unwrap(); + let mut metadata = metadata; + metadata.append_snapshot(snapshot); + + // Ref exists and should matches + let requirement = TableRequirement::RefSnapshotIdMatch { + r#ref: "main".to_string(), + snapshot_id: Some(3051729675574597004), + }; + assert!(requirement.check(Some(&metadata)).is_ok()); + + // Ref exists but does not match + let requirement = TableRequirement::RefSnapshotIdMatch { + r#ref: "main".to_string(), + snapshot_id: Some(1), + }; + assert!(requirement.check(Some(&metadata)).is_err()); + } + + #[test] + fn test_check_last_assigned_field_id() { + let metadata = metadata(); + + let requirement = TableRequirement::LastAssignedFieldIdMatch { + last_assigned_field_id: 1, + }; + assert!(requirement.check(Some(&metadata)).is_err()); + + let requirement = TableRequirement::LastAssignedFieldIdMatch { + last_assigned_field_id: 0, + }; + assert!(requirement.check(Some(&metadata)).is_ok()); + } + + #[test] + fn test_check_current_schema_id() { + let metadata = metadata(); + + let requirement = TableRequirement::CurrentSchemaIdMatch { + current_schema_id: 1, + }; + assert!(requirement.check(Some(&metadata)).is_err()); + + let requirement = TableRequirement::CurrentSchemaIdMatch { + current_schema_id: 0, + }; + assert!(requirement.check(Some(&metadata)).is_ok()); + } + + #[test] + fn test_check_last_assigned_partition_id() { + let metadata = metadata(); + + let requirement = TableRequirement::LastAssignedPartitionIdMatch { + last_assigned_partition_id: 1, + }; + assert!(requirement.check(Some(&metadata)).is_err()); + + let requirement = TableRequirement::LastAssignedPartitionIdMatch { + last_assigned_partition_id: 0, + }; + assert!(requirement.check(Some(&metadata)).is_ok()); + } + + #[test] + fn test_check_default_spec_id() { + let metadata = metadata(); + + let requirement = TableRequirement::DefaultSpecIdMatch { default_spec_id: 1 }; + assert!(requirement.check(Some(&metadata)).is_err()); + + let requirement = TableRequirement::DefaultSpecIdMatch { default_spec_id: 0 }; + assert!(requirement.check(Some(&metadata)).is_ok()); + } + + #[test] + fn test_check_default_sort_order_id() { + let metadata = metadata(); + + let requirement = TableRequirement::DefaultSortOrderIdMatch { + default_sort_order_id: 1, + }; + assert!(requirement.check(Some(&metadata)).is_err()); + + let requirement = TableRequirement::DefaultSortOrderIdMatch { + default_sort_order_id: 0, + }; + assert!(requirement.check(Some(&metadata)).is_ok()); + } + #[test] fn test_table_uuid() { test_serde_json( diff --git a/crates/iceberg/src/transaction.rs b/crates/iceberg/src/transaction.rs index db7c3f28f..f29cf5122 100644 --- a/crates/iceberg/src/transaction.rs +++ b/crates/iceberg/src/transaction.rs @@ -154,7 +154,7 @@ impl<'a> ReplaceSortOrderAction<'a> { let requirements = vec![ TableRequirement::CurrentSchemaIdMatch { - current_schema_id: self.tx.table.metadata().current_schema().schema_id() as i64, + current_schema_id: self.tx.table.metadata().current_schema().schema_id(), }, TableRequirement::DefaultSortOrderIdMatch { default_sort_order_id: self.tx.table.metadata().default_sort_order().order_id, From 50345196c87b00badc1a6490aef284e84f4c3e9a Mon Sep 17 00:00:00 2001 From: Christian Date: Wed, 13 Nov 2024 09:47:01 +0100 Subject: [PATCH 37/46] Add ViewUpdate to catalog (#690) --- crates/iceberg/src/catalog/mod.rs | 264 +++++++++++++++++++++++++++++- 1 file changed, 261 insertions(+), 3 deletions(-) diff --git a/crates/iceberg/src/catalog/mod.rs b/crates/iceberg/src/catalog/mod.rs index 536726fd3..70a403905 100644 --- a/crates/iceberg/src/catalog/mod.rs +++ b/crates/iceberg/src/catalog/mod.rs @@ -30,7 +30,8 @@ use uuid::Uuid; use crate::spec::{ FormatVersion, Schema, SchemaId, Snapshot, SnapshotReference, SortOrder, TableMetadata, - TableMetadataBuilder, UnboundPartitionSpec, ViewRepresentations, + TableMetadataBuilder, UnboundPartitionSpec, ViewFormatVersion, ViewRepresentations, + ViewVersion, }; use crate::table::Table; use crate::{Error, ErrorKind, Result}; @@ -671,6 +672,64 @@ pub struct ViewCreation { pub summary: HashMap, } +/// ViewUpdate represents an update to a view in the catalog. +#[derive(Clone, Debug, PartialEq, Serialize, Deserialize)] +#[serde(tag = "action", rename_all = "kebab-case")] +pub enum ViewUpdate { + /// Assign a new UUID to the view + #[serde(rename_all = "kebab-case")] + AssignUuid { + /// The new UUID to assign. + uuid: uuid::Uuid, + }, + /// Upgrade view's format version + #[serde(rename_all = "kebab-case")] + UpgradeFormatVersion { + /// Target format upgrade to. + format_version: ViewFormatVersion, + }, + /// Add a new schema to the view + #[serde(rename_all = "kebab-case")] + AddSchema { + /// The schema to add. + schema: Schema, + /// The last column id of the view. + last_column_id: Option, + }, + /// Set view's current schema + #[serde(rename_all = "kebab-case")] + SetLocation { + /// New location for view. + location: String, + }, + /// Set view's properties + /// + /// Matching keys are updated, and non-matching keys are left unchanged. + #[serde(rename_all = "kebab-case")] + SetProperties { + /// Properties to update for view. + updates: HashMap, + }, + /// Remove view's properties + #[serde(rename_all = "kebab-case")] + RemoveProperties { + /// Properties to remove + removals: Vec, + }, + /// Add a new version to the view + #[serde(rename_all = "kebab-case")] + AddViewVersion { + /// The view version to add. + view_version: ViewVersion, + }, + /// Set view's current version + #[serde(rename_all = "kebab-case")] + SetCurrentViewVersion { + /// View version id to set as current, or -1 to set last added version + view_version_id: i32, + }, +} + #[cfg(test)] mod tests { use std::collections::HashMap; @@ -680,10 +739,13 @@ mod tests { use serde::Serialize; use uuid::uuid; + use super::ViewUpdate; use crate::spec::{ FormatVersion, NestedField, NullOrder, Operation, PrimitiveType, Schema, Snapshot, - SnapshotReference, SnapshotRetention, SortDirection, SortField, SortOrder, Summary, - TableMetadata, TableMetadataBuilder, Transform, Type, UnboundPartitionSpec, + SnapshotReference, SnapshotRetention, SortDirection, SortField, SortOrder, + SqlViewRepresentation, Summary, TableMetadata, TableMetadataBuilder, Transform, Type, + UnboundPartitionSpec, ViewFormatVersion, ViewRepresentation, ViewRepresentations, + ViewVersion, }; use crate::{NamespaceIdent, TableCreation, TableIdent, TableRequirement, TableUpdate}; @@ -1528,4 +1590,200 @@ mod tests { .unwrap(); assert_eq!(updated_metadata.uuid(), uuid); } + + #[test] + fn test_view_assign_uuid() { + test_serde_json( + r#" +{ + "action": "assign-uuid", + "uuid": "2cc52516-5e73-41f2-b139-545d41a4e151" +} + "#, + ViewUpdate::AssignUuid { + uuid: uuid!("2cc52516-5e73-41f2-b139-545d41a4e151"), + }, + ); + } + + #[test] + fn test_view_upgrade_format_version() { + test_serde_json( + r#" +{ + "action": "upgrade-format-version", + "format-version": 1 +} + "#, + ViewUpdate::UpgradeFormatVersion { + format_version: ViewFormatVersion::V1, + }, + ); + } + + #[test] + fn test_view_add_schema() { + let test_schema = Schema::builder() + .with_schema_id(1) + .with_identifier_field_ids(vec![2]) + .with_fields(vec![ + NestedField::optional(1, "foo", Type::Primitive(PrimitiveType::String)).into(), + NestedField::required(2, "bar", Type::Primitive(PrimitiveType::Int)).into(), + NestedField::optional(3, "baz", Type::Primitive(PrimitiveType::Boolean)).into(), + ]) + .build() + .unwrap(); + test_serde_json( + r#" +{ + "action": "add-schema", + "schema": { + "type": "struct", + "schema-id": 1, + "fields": [ + { + "id": 1, + "name": "foo", + "required": false, + "type": "string" + }, + { + "id": 2, + "name": "bar", + "required": true, + "type": "int" + }, + { + "id": 3, + "name": "baz", + "required": false, + "type": "boolean" + } + ], + "identifier-field-ids": [ + 2 + ] + }, + "last-column-id": 3 +} + "#, + ViewUpdate::AddSchema { + schema: test_schema.clone(), + last_column_id: Some(3), + }, + ); + } + + #[test] + fn test_view_set_location() { + test_serde_json( + r#" +{ + "action": "set-location", + "location": "s3://db/view" +} + "#, + ViewUpdate::SetLocation { + location: "s3://db/view".to_string(), + }, + ); + } + + #[test] + fn test_view_set_properties() { + test_serde_json( + r#" +{ + "action": "set-properties", + "updates": { + "prop1": "v1", + "prop2": "v2" + } +} + "#, + ViewUpdate::SetProperties { + updates: vec![ + ("prop1".to_string(), "v1".to_string()), + ("prop2".to_string(), "v2".to_string()), + ] + .into_iter() + .collect(), + }, + ); + } + + #[test] + fn test_view_remove_properties() { + test_serde_json( + r#" +{ + "action": "remove-properties", + "removals": [ + "prop1", + "prop2" + ] +} + "#, + ViewUpdate::RemoveProperties { + removals: vec!["prop1".to_string(), "prop2".to_string()], + }, + ); + } + + #[test] + fn test_view_add_view_version() { + test_serde_json( + r#" +{ + "action": "add-view-version", + "view-version": { + "version-id" : 1, + "timestamp-ms" : 1573518431292, + "schema-id" : 1, + "default-catalog" : "prod", + "default-namespace" : [ "default" ], + "summary" : { + "engine-name" : "Spark" + }, + "representations" : [ { + "type" : "sql", + "sql" : "SELECT\n COUNT(1), CAST(event_ts AS DATE)\nFROM events\nGROUP BY 2", + "dialect" : "spark" + } ] + } +} + "#, + ViewUpdate::AddViewVersion { + view_version: ViewVersion::builder() + .with_version_id(1) + .with_timestamp_ms(1573518431292) + .with_schema_id(1) + .with_default_catalog(Some("prod".to_string())) + .with_default_namespace(NamespaceIdent::from_strs(vec!["default"]).unwrap()) + .with_summary( + vec![("engine-name".to_string(), "Spark".to_string())] + .into_iter() + .collect(), + ) + .with_representations(ViewRepresentations(vec![ViewRepresentation::Sql(SqlViewRepresentation { + sql: "SELECT\n COUNT(1), CAST(event_ts AS DATE)\nFROM events\nGROUP BY 2".to_string(), + dialect: "spark".to_string(), + })])) + .build(), + }, + ); + } + + #[test] + fn test_view_set_current_view_version() { + test_serde_json( + r#" +{ + "action": "set-current-view-version", + "view-version-id": 1 +} + "#, + ViewUpdate::SetCurrentViewVersion { view_version_id: 1 }, + ); + } } From b2fb80339089eed658964cb6942979cb24f52dc4 Mon Sep 17 00:00:00 2001 From: Christian Date: Mon, 18 Nov 2024 12:20:41 +0000 Subject: [PATCH 38/46] Update .asf.yaml (#701) As requested by Fokko: Could you raise a PR to remove Xuanwo and Renjie as a collaborator (they are committer now, so the collaborator right are implied), and add you next to JanKaul? --- .asf.yaml | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/.asf.yaml b/.asf.yaml index ce27a54e3..675f474c6 100644 --- a/.asf.yaml +++ b/.asf.yaml @@ -48,9 +48,8 @@ github: issues: true projects: true collaborators: - - Xuanwo - - liurenjie1024 - JanKaul + - c-thiel ghp_branch: gh-pages ghp_path: / From d00f84b3958ccb0d4b4c419b482b966d0c8a62f6 Mon Sep 17 00:00:00 2001 From: Dylan Chen Date: Wed, 30 Oct 2024 15:13:10 +0800 Subject: [PATCH 39/46] pitch feat: Implement Decimal from/to bytes represents #665 --- Cargo.toml | 1 + crates/iceberg/Cargo.toml | 1 + crates/iceberg/src/spec/values.rs | 65 ++++++++++++++++++++++++++++--- 3 files changed, 62 insertions(+), 5 deletions(-) diff --git a/Cargo.toml b/Cargo.toml index efff593b3..7db84f10f 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -70,6 +70,7 @@ itertools = "0.13" log = "0.4" mockito = "1" murmur3 = "0.5.2" +num-bigint = "0.4.6" once_cell = "1" opendal = "0.50.1" ordered-float = "4" diff --git a/crates/iceberg/Cargo.toml b/crates/iceberg/Cargo.toml index 1307cc6f3..d23ba2592 100644 --- a/crates/iceberg/Cargo.toml +++ b/crates/iceberg/Cargo.toml @@ -63,6 +63,7 @@ futures = { workspace = true } itertools = { workspace = true } moka = { version = "0.12.8", features = ["future"] } murmur3 = { workspace = true } +num-bigint = { workspace = true } once_cell = { workspace = true } opendal = { workspace = true } ordered-float = { workspace = true } diff --git a/crates/iceberg/src/spec/values.rs b/crates/iceberg/src/spec/values.rs index 552ac497f..1b10f9b04 100644 --- a/crates/iceberg/src/spec/values.rs +++ b/crates/iceberg/src/spec/values.rs @@ -29,7 +29,9 @@ use std::str::FromStr; pub use _serde::RawLiteral; use bitvec::vec::BitVec; use chrono::{DateTime, NaiveDate, NaiveDateTime, NaiveTime, TimeZone, Utc}; +use num_bigint::BigInt; use ordered_float::OrderedFloat; +use rust_decimal::prelude::ToPrimitive; use rust_decimal::Decimal; use serde::de::{ MapAccess, {self}, @@ -422,10 +424,15 @@ impl Datum { } PrimitiveType::Fixed(_) => PrimitiveLiteral::Binary(Vec::from(bytes)), PrimitiveType::Binary => PrimitiveLiteral::Binary(Vec::from(bytes)), - PrimitiveType::Decimal { - precision: _, - scale: _, - } => todo!(), + PrimitiveType::Decimal { .. } => { + let unscaled_value = BigInt::from_signed_bytes_be(bytes); + PrimitiveLiteral::Int128(unscaled_value.to_i128().ok_or_else(|| { + Error::new( + ErrorKind::DataInvalid, + format!("Can't convert bytes to i128: {:?}", bytes), + ) + })?) + } }; Ok(Datum::new(data_type, literal)) } @@ -449,7 +456,30 @@ impl Datum { PrimitiveLiteral::String(val) => ByteBuf::from(val.as_bytes()), PrimitiveLiteral::UInt128(val) => ByteBuf::from(val.to_be_bytes()), PrimitiveLiteral::Binary(val) => ByteBuf::from(val.as_slice()), - PrimitiveLiteral::Int128(_) => todo!(), + PrimitiveLiteral::Int128(val) => { + let PrimitiveType::Decimal { precision, .. } = self.r#type else { + unreachable!( + "PrimitiveLiteral Int128 must be PrimitiveType Decimal but got {}", + &self.r#type + ) + }; + + // It's required by iceberg spec that we must keep the minimum + // number of bytes for the value + let required_bytes = Type::decimal_required_bytes(precision) + .expect("PrimitiveType must has valid precision") + as usize; + + // The primitive literal is unscaled value. + let unscaled_value = BigInt::from(*val); + // Convert into two's-complement byte representation of the BigInt + // in big-endian byte order. + let mut bytes = unscaled_value.to_signed_bytes_be(); + // Truncate with required bytes to make sure. + bytes.truncate(required_bytes); + + ByteBuf::from(bytes) + } } } @@ -3031,6 +3061,31 @@ mod tests { check_avro_bytes_serde(bytes, Datum::string("iceberg"), &PrimitiveType::String); } + #[test] + fn avro_bytes_decimal() { + let bytes = vec![4u8, 210u8]; + + check_avro_bytes_serde( + bytes, + Datum::decimal(Decimal::new(1234, 2)).unwrap(), + &PrimitiveType::Decimal { + precision: 38, + scale: 2, + }, + ); + + let bytes = vec![251u8, 46u8]; + + check_avro_bytes_serde( + bytes, + Datum::decimal(Decimal::new(-1234, 2)).unwrap(), + &PrimitiveType::Decimal { + precision: 38, + scale: 2, + }, + ); + } + #[test] fn avro_convert_test_int() { check_convert_with_avro( From b687f57b8884ef5b6cabc133cb4ec85d162a0b7d Mon Sep 17 00:00:00 2001 From: Dylan Chen Date: Thu, 5 Sep 2024 22:12:34 +0800 Subject: [PATCH 40/46] support scan with nested type --- crates/iceberg/src/arrow/reader.rs | 15 +++++----- crates/iceberg/src/scan.rs | 44 +++++++++++++++--------------- 2 files changed, 29 insertions(+), 30 deletions(-) diff --git a/crates/iceberg/src/arrow/reader.rs b/crates/iceberg/src/arrow/reader.rs index 66c233f65..347db6cd7 100644 --- a/crates/iceberg/src/arrow/reader.rs +++ b/crates/iceberg/src/arrow/reader.rs @@ -307,35 +307,34 @@ impl ArrowReader { let fields = arrow_schema.fields(); let iceberg_schema = arrow_schema_to_schema(arrow_schema)?; - fields.filter_leaves(|idx, field| { + fields.into_iter().enumerate().for_each(|(idx, field)| { let field_id = field.metadata().get(PARQUET_FIELD_ID_META_KEY); if field_id.is_none() { - return false; + return; } let field_id = i32::from_str(field_id.unwrap()); if field_id.is_err() { - return false; + return; } let field_id = field_id.unwrap(); if !field_ids.contains(&field_id) { - return false; + return; } let iceberg_field = iceberg_schema_of_task.field_by_id(field_id); let parquet_iceberg_field = iceberg_schema.field_by_id(field_id); if iceberg_field.is_none() || parquet_iceberg_field.is_none() { - return false; + return; } if iceberg_field.unwrap().field_type != parquet_iceberg_field.unwrap().field_type { - return false; + return; } column_map.insert(field_id, idx); - true }); if column_map.len() != field_ids.len() { @@ -359,7 +358,7 @@ impl ArrowReader { )); } } - Ok(ProjectionMask::leaves(parquet_schema, indices)) + Ok(ProjectionMask::roots(parquet_schema, indices)) } } diff --git a/crates/iceberg/src/scan.rs b/crates/iceberg/src/scan.rs index 89e8846f0..990cf81aa 100644 --- a/crates/iceberg/src/scan.rs +++ b/crates/iceberg/src/scan.rs @@ -248,28 +248,28 @@ impl<'a> TableScanBuilder<'a> { ) })?; - let field = schema - .as_struct() - .field_by_id(field_id) - .ok_or_else(|| { - Error::new( - ErrorKind::FeatureUnsupported, - format!( - "Column {} is not a direct child of schema but a nested field, which is not supported now. Schema: {}", - column_name, schema - ), - ) - })?; - - if !field.field_type.is_primitive() { - return Err(Error::new( - ErrorKind::FeatureUnsupported, - format!( - "Column {} is not a primitive type. Schema: {}", - column_name, schema - ), - )); - } + // let field = schema + // .as_struct() + // .field_by_id(field_id) + // .ok_or_else(|| { + // Error::new( + // ErrorKind::FeatureUnsupported, + // format!( + // "Column {} is not a direct child of schema but a nested field, which is not supported now. Schema: {}", + // column_name, schema + // ), + // ) + // })?; + + // if !field.field_type.is_primitive() { + // return Err(Error::new( + // ErrorKind::FeatureUnsupported, + // format!( + // "Column {} is not a primitive type. Schema: {}", + // column_name, schema + // ), + // )); + // } field_ids.push(field_id); } From e982780874099c758e6302b3b8a3eba6db2684a6 Mon Sep 17 00:00:00 2001 From: Dylan Chen Date: Thu, 12 Sep 2024 14:17:53 +0800 Subject: [PATCH 41/46] reorder record batch --- crates/iceberg/src/arrow/reader.rs | 24 ++++++++++++++++++++---- 1 file changed, 20 insertions(+), 4 deletions(-) diff --git a/crates/iceberg/src/arrow/reader.rs b/crates/iceberg/src/arrow/reader.rs index 347db6cd7..921a9669f 100644 --- a/crates/iceberg/src/arrow/reader.rs +++ b/crates/iceberg/src/arrow/reader.rs @@ -202,7 +202,8 @@ impl ArrowReader { // Create a projection mask for the batch stream to select which columns in the // Parquet file that we want in the response - let projection_mask = Self::get_arrow_projection_mask( + // Since Parquet projection mask will lose the order of the columns, we need to reorder. + let (projection_mask, reorder) = Self::get_arrow_projection_mask( &task.project_field_ids, &task.schema, record_batch_stream_builder.parquet_schema(), @@ -270,6 +271,11 @@ impl ArrowReader { let mut record_batch_stream = record_batch_stream_builder.build()?; while let Some(batch) = record_batch_stream.try_next().await? { + let batch = if let Some(reorder) = reorder.as_ref() { + batch.project(&reorder).expect("must be able to reorder") + } else { + batch + }; tx.send(record_batch_transformer.process_record_batch(batch)) .await? } @@ -298,9 +304,9 @@ impl ArrowReader { iceberg_schema_of_task: &Schema, parquet_schema: &SchemaDescriptor, arrow_schema: &ArrowSchemaRef, - ) -> Result { + ) -> Result<(ProjectionMask, Option>)> { if field_ids.is_empty() { - Ok(ProjectionMask::all()) + Ok((ProjectionMask::all(), None)) } else { // Build the map between field id and column index in Parquet schema. let mut column_map = HashMap::new(); @@ -358,7 +364,17 @@ impl ArrowReader { )); } } - Ok(ProjectionMask::roots(parquet_schema, indices)) + + // projection mask is order by indices + let mut mask_indices = indices.clone(); + mask_indices.sort_by_key(|&x| x); + // try to reorder the mask_indices to indices + let reorder = indices + .iter() + .map(|idx| mask_indices.iter().position(|&i| i == *idx).unwrap()) + .collect::>(); + + Ok((ProjectionMask::roots(parquet_schema, indices), Some(reorder))) } } From 1cb18c9f60d9b2c322f0ced7d850206a7fd3a219 Mon Sep 17 00:00:00 2001 From: xxhZs <1060434431@qq.com> Date: Tue, 10 Sep 2024 11:30:24 +0800 Subject: [PATCH 42/46] fix scan fix filtered_entries fix ci ut --- crates/iceberg/src/arrow/reader.rs | 5 ++++- crates/iceberg/src/scan.rs | 32 +++++++++++++----------------- 2 files changed, 18 insertions(+), 19 deletions(-) diff --git a/crates/iceberg/src/arrow/reader.rs b/crates/iceberg/src/arrow/reader.rs index 921a9669f..ad0c3104f 100644 --- a/crates/iceberg/src/arrow/reader.rs +++ b/crates/iceberg/src/arrow/reader.rs @@ -374,7 +374,10 @@ impl ArrowReader { .map(|idx| mask_indices.iter().position(|&i| i == *idx).unwrap()) .collect::>(); - Ok((ProjectionMask::roots(parquet_schema, indices), Some(reorder))) + Ok(( + ProjectionMask::roots(parquet_schema, indices), + Some(reorder), + )) } } diff --git a/crates/iceberg/src/scan.rs b/crates/iceberg/src/scan.rs index 990cf81aa..5b5509930 100644 --- a/crates/iceberg/src/scan.rs +++ b/crates/iceberg/src/scan.rs @@ -36,8 +36,8 @@ use crate::io::object_cache::ObjectCache; use crate::io::FileIO; use crate::runtime::spawn; use crate::spec::{ - DataContentType, DataFileFormat, ManifestContentType, ManifestEntryRef, ManifestFile, - ManifestList, Schema, SchemaRef, SnapshotRef, TableMetadataRef, + DataContentType, DataFileFormat, ManifestEntryRef, ManifestFile, ManifestList, Schema, + SchemaRef, SnapshotRef, TableMetadataRef, }; use crate::table::Table; use crate::utils::available_parallelism; @@ -444,15 +444,6 @@ impl TableScan { return Ok(()); } - // abort the plan if we encounter a manifest entry whose data file's - // content type is currently unsupported - if manifest_entry_context.manifest_entry.content_type() != DataContentType::Data { - return Err(Error::new( - ErrorKind::FeatureUnsupported, - "Only Data files currently supported", - )); - } - if let Some(ref bound_predicates) = manifest_entry_context.bound_predicates { let BoundPredicates { ref snapshot_bound_predicate, @@ -581,6 +572,8 @@ impl ManifestEntryContext { predicate: self .bound_predicates .map(|x| x.as_ref().snapshot_bound_predicate.clone()), + sequence_number: self.manifest_entry.sequence_number().unwrap_or(0), + equality_ids: self.manifest_entry.data_file().equality_ids().to_vec(), } } } @@ -619,15 +612,10 @@ impl PlanContext { manifest_list: Arc, sender: Sender, ) -> Result>>> { - let filtered_entries = manifest_list - .entries() - .iter() - .filter(|manifest_file| manifest_file.content == ManifestContentType::Data); - // TODO: Ideally we could ditch this intermediate Vec as we return an iterator. let mut filtered_mfcs = vec![]; if self.predicate.is_some() { - for manifest_file in filtered_entries { + for manifest_file in manifest_list.entries().iter() { let partition_bound_predicate = self.get_partition_filter(manifest_file)?; // evaluate the ManifestFile against the partition filter. Skip @@ -649,7 +637,7 @@ impl PlanContext { } } } else { - for manifest_file in filtered_entries { + for manifest_file in manifest_list.entries().iter() { let mfc = self.create_manifest_file_context(manifest_file, None, sender.clone()); filtered_mfcs.push(Ok(mfc)); } @@ -922,6 +910,10 @@ pub struct FileScanTask { /// The predicate to filter. #[serde(skip_serializing_if = "Option::is_none")] pub predicate: Option, + /// The `sequence_number` of the task. + pub sequence_number: i64, + /// The `equality_ids` of the task. + pub equality_ids: Vec, } impl FileScanTask { @@ -1857,6 +1849,8 @@ mod tests { schema: schema.clone(), record_count: Some(100), data_file_format: DataFileFormat::Parquet, + sequence_number: 0, + equality_ids: vec![], }; test_fn(task); @@ -1871,6 +1865,8 @@ mod tests { schema, record_count: None, data_file_format: DataFileFormat::Avro, + sequence_number: 0, + equality_ids: vec![], }; test_fn(task); } From 5ab0ed5ef0a2277c0a2ab52df5d4dc4a73ee87e4 Mon Sep 17 00:00:00 2001 From: xxhZs <1060434431@qq.com> Date: Thu, 12 Sep 2024 19:27:55 +0800 Subject: [PATCH 43/46] change moka to 0.12.0 --- crates/iceberg/Cargo.toml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/crates/iceberg/Cargo.toml b/crates/iceberg/Cargo.toml index d23ba2592..68a8658b0 100644 --- a/crates/iceberg/Cargo.toml +++ b/crates/iceberg/Cargo.toml @@ -61,7 +61,7 @@ derive_builder = { workspace = true } fnv = { workspace = true } futures = { workspace = true } itertools = { workspace = true } -moka = { version = "0.12.8", features = ["future"] } +moka = { version = "0.12.0", features = ["future"] } murmur3 = { workspace = true } num-bigint = { workspace = true } once_cell = { workspace = true } From 4fd5de38e3c99a7312295037f1e5e599be705506 Mon Sep 17 00:00:00 2001 From: Dylan Chen Date: Thu, 31 Oct 2024 23:39:21 +0800 Subject: [PATCH 44/46] Revert "reorder record batch" This reverts commit 66d7c4ae3bfdaa860588a513723ed7d535d637fc. --- crates/iceberg/src/arrow/reader.rs | 27 ++++----------------------- 1 file changed, 4 insertions(+), 23 deletions(-) diff --git a/crates/iceberg/src/arrow/reader.rs b/crates/iceberg/src/arrow/reader.rs index ad0c3104f..347db6cd7 100644 --- a/crates/iceberg/src/arrow/reader.rs +++ b/crates/iceberg/src/arrow/reader.rs @@ -202,8 +202,7 @@ impl ArrowReader { // Create a projection mask for the batch stream to select which columns in the // Parquet file that we want in the response - // Since Parquet projection mask will lose the order of the columns, we need to reorder. - let (projection_mask, reorder) = Self::get_arrow_projection_mask( + let projection_mask = Self::get_arrow_projection_mask( &task.project_field_ids, &task.schema, record_batch_stream_builder.parquet_schema(), @@ -271,11 +270,6 @@ impl ArrowReader { let mut record_batch_stream = record_batch_stream_builder.build()?; while let Some(batch) = record_batch_stream.try_next().await? { - let batch = if let Some(reorder) = reorder.as_ref() { - batch.project(&reorder).expect("must be able to reorder") - } else { - batch - }; tx.send(record_batch_transformer.process_record_batch(batch)) .await? } @@ -304,9 +298,9 @@ impl ArrowReader { iceberg_schema_of_task: &Schema, parquet_schema: &SchemaDescriptor, arrow_schema: &ArrowSchemaRef, - ) -> Result<(ProjectionMask, Option>)> { + ) -> Result { if field_ids.is_empty() { - Ok((ProjectionMask::all(), None)) + Ok(ProjectionMask::all()) } else { // Build the map between field id and column index in Parquet schema. let mut column_map = HashMap::new(); @@ -364,20 +358,7 @@ impl ArrowReader { )); } } - - // projection mask is order by indices - let mut mask_indices = indices.clone(); - mask_indices.sort_by_key(|&x| x); - // try to reorder the mask_indices to indices - let reorder = indices - .iter() - .map(|idx| mask_indices.iter().position(|&i| i == *idx).unwrap()) - .collect::>(); - - Ok(( - ProjectionMask::roots(parquet_schema, indices), - Some(reorder), - )) + Ok(ProjectionMask::roots(parquet_schema, indices)) } } From a717ef9008f5d222e98e9139d63f1230dea1d11d Mon Sep 17 00:00:00 2001 From: Xinhao Xu <84456268+xxhZs@users.noreply.github.com> Date: Fri, 1 Nov 2024 16:03:41 +0800 Subject: [PATCH 45/46] fix(delete): fix position delete (#8) * fix position * fix comm * fix comm --- crates/iceberg/src/arrow/reader.rs | 27 +++++++++++++++++---------- 1 file changed, 17 insertions(+), 10 deletions(-) diff --git a/crates/iceberg/src/arrow/reader.rs b/crates/iceberg/src/arrow/reader.rs index 347db6cd7..f686c1045 100644 --- a/crates/iceberg/src/arrow/reader.rs +++ b/crates/iceberg/src/arrow/reader.rs @@ -48,7 +48,7 @@ use crate::expr::{BoundPredicate, BoundReference}; use crate::io::{FileIO, FileMetadata, FileRead}; use crate::runtime::spawn; use crate::scan::{ArrowRecordBatchStream, FileScanTask, FileScanTaskStream}; -use crate::spec::{Datum, Schema}; +use crate::spec::{DataContentType, Datum, Schema}; use crate::utils::available_parallelism; use crate::{Error, ErrorKind}; @@ -210,12 +210,6 @@ impl ArrowReader { )?; record_batch_stream_builder = record_batch_stream_builder.with_projection(projection_mask); - // RecordBatchTransformer performs any required transformations on the RecordBatches - // that come back from the file, such as type promotion, default column insertion - // and column re-ordering - let mut record_batch_transformer = - RecordBatchTransformer::build(task.schema_ref(), task.project_field_ids()); - if let Some(batch_size) = batch_size { record_batch_stream_builder = record_batch_stream_builder.with_batch_size(batch_size); } @@ -269,9 +263,22 @@ impl ArrowReader { // to the requester. let mut record_batch_stream = record_batch_stream_builder.build()?; - while let Some(batch) = record_batch_stream.try_next().await? { - tx.send(record_batch_transformer.process_record_batch(batch)) - .await? + // The schema of the xxx file doesn't change, so we don't need to convert the schema. + if matches!(task.data_file_content, DataContentType::PositionDeletes) { + while let Some(batch) = record_batch_stream.try_next().await? { + tx.send(Ok(batch)).await? + } + } else { + // RecordBatchTransformer performs any required transformations on the RecordBatches + // that come back from the file, such as type promotion, default column insertion + // and column re-ordering. + let mut record_batch_transformer = + RecordBatchTransformer::build(task.schema_ref(), task.project_field_ids()); + + while let Some(batch) = record_batch_stream.try_next().await? { + tx.send(record_batch_transformer.process_record_batch(batch)) + .await? + } } Ok(()) From 85076c7ca5ca1937b52eb9004754e7e9f4ee5e09 Mon Sep 17 00:00:00 2001 From: ZENOTME <43447882+ZENOTME@users.noreply.github.com> Date: Wed, 20 Nov 2024 18:16:22 +0800 Subject: [PATCH 46/46] feat: support append data file and add e2e test (#9) * support append data file and add e2e test * fix typos * refine append action * fix cargo sort * add consistent check for partition value * generate unique snapshot id * avoid to set snapshot id for v2 * refine test * fix unit test * export ports * fix None case for parant_snapshot_id * fix parquect schema check * refactor append action of transaction * refine * refine e2e test * refine commit uuid * fix file format field to uppercase in manifest --------- Co-authored-by: ZENOTME --- Cargo.toml | 11 +- crates/catalog/rest/src/catalog.rs | 2 +- crates/e2e_test/Cargo.toml | 37 ++ crates/e2e_test/src/lib.rs | 74 +++ crates/e2e_test/testdata/docker-compose.yaml | 62 ++ .../e2e_test/tests/append_data_file_test.rs | 178 ++++++ crates/e2e_test/tests/conflict_commit_test.rs | 145 +++++ crates/iceberg/src/catalog/mod.rs | 4 +- crates/iceberg/src/io/object_cache.rs | 6 +- crates/iceberg/src/scan.rs | 5 +- crates/iceberg/src/spec/manifest.rs | 8 +- crates/iceberg/src/spec/manifest_list.rs | 50 +- crates/iceberg/src/spec/snapshot.rs | 21 +- crates/iceberg/src/spec/table_metadata.rs | 18 +- crates/iceberg/src/spec/values.rs | 13 + crates/iceberg/src/transaction.rs | 548 +++++++++++++++++- 16 files changed, 1136 insertions(+), 46 deletions(-) create mode 100644 crates/e2e_test/Cargo.toml create mode 100644 crates/e2e_test/src/lib.rs create mode 100644 crates/e2e_test/testdata/docker-compose.yaml create mode 100644 crates/e2e_test/tests/append_data_file_test.rs create mode 100644 crates/e2e_test/tests/conflict_commit_test.rs diff --git a/Cargo.toml b/Cargo.toml index 7db84f10f..83ba945a1 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -18,11 +18,12 @@ [workspace] resolver = "2" members = [ - "crates/catalog/*", - "crates/examples", - "crates/iceberg", - "crates/integrations/*", - "crates/test_utils", + "crates/catalog/*", + "crates/e2e_test", + "crates/examples", + "crates/iceberg", + "crates/integrations/*", + "crates/test_utils", ] exclude = ["bindings/python"] diff --git a/crates/catalog/rest/src/catalog.rs b/crates/catalog/rest/src/catalog.rs index 1181c3cc1..bd73f6d03 100644 --- a/crates/catalog/rest/src/catalog.rs +++ b/crates/catalog/rest/src/catalog.rs @@ -1376,7 +1376,7 @@ mod tests { .with_schema_id(0) .with_summary(Summary { operation: Operation::Append, - other: HashMap::from_iter([ + additional_properties: HashMap::from_iter([ ("spark.app.id", "local-1646787004168"), ("added-data-files", "1"), ("added-records", "1"), diff --git a/crates/e2e_test/Cargo.toml b/crates/e2e_test/Cargo.toml new file mode 100644 index 000000000..2607b099c --- /dev/null +++ b/crates/e2e_test/Cargo.toml @@ -0,0 +1,37 @@ +# 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. + +[package] +name = "iceberg-e2e_test" +version = { workspace = true } +edition = { workspace = true } +homepage = { workspace = true } +repository = { workspace = true } +license = { workspace = true } +rust-version = { workspace = true } + +[dependencies] +arrow-array = { workspace = true } +arrow-schema = { workspace = true } +futures = { workspace = true } +iceberg = { workspace = true } +iceberg-catalog-rest = { workspace = true } +iceberg_test_utils = { path = "../test_utils", features = ["tests"] } +log = { workspace = true } +parquet = { workspace = true } +port_scanner = { workspace = true } +tokio = { workspace = true } diff --git a/crates/e2e_test/src/lib.rs b/crates/e2e_test/src/lib.rs new file mode 100644 index 000000000..5777a4018 --- /dev/null +++ b/crates/e2e_test/src/lib.rs @@ -0,0 +1,74 @@ +// 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. + +use std::collections::HashMap; + +use iceberg::io::{S3_ACCESS_KEY_ID, S3_ENDPOINT, S3_REGION, S3_SECRET_ACCESS_KEY}; +use iceberg_catalog_rest::{RestCatalog, RestCatalogConfig}; +use iceberg_test_utils::docker::DockerCompose; +use iceberg_test_utils::{normalize_test_name, set_up}; +use port_scanner::scan_port_addr; +use tokio::time::sleep; + +const REST_CATALOG_PORT: u16 = 8181; + +pub struct TestFixture { + pub _docker_compose: DockerCompose, + pub rest_catalog: RestCatalog, +} + +pub async fn set_test_fixture(func: &str) -> TestFixture { + set_up(); + let docker_compose = DockerCompose::new( + normalize_test_name(format!("{}_{func}", module_path!())), + format!("{}/testdata", env!("CARGO_MANIFEST_DIR")), + ); + + // Start docker compose + docker_compose.run(); + + let rest_catalog_ip = docker_compose.get_container_ip("rest"); + + let read_port = format!("{}:{}", rest_catalog_ip, REST_CATALOG_PORT); + loop { + if !scan_port_addr(&read_port) { + log::info!("Waiting for 1s rest catalog to ready..."); + sleep(std::time::Duration::from_millis(1000)).await; + } else { + break; + } + } + + let container_ip = docker_compose.get_container_ip("minio"); + let read_port = format!("{}:{}", container_ip, 9000); + + let config = RestCatalogConfig::builder() + .uri(format!("http://{}:{}", rest_catalog_ip, REST_CATALOG_PORT)) + .props(HashMap::from([ + (S3_ENDPOINT.to_string(), format!("http://{}", read_port)), + (S3_ACCESS_KEY_ID.to_string(), "admin".to_string()), + (S3_SECRET_ACCESS_KEY.to_string(), "password".to_string()), + (S3_REGION.to_string(), "us-east-1".to_string()), + ])) + .build(); + let rest_catalog = RestCatalog::new(config); + + TestFixture { + _docker_compose: docker_compose, + rest_catalog, + } +} diff --git a/crates/e2e_test/testdata/docker-compose.yaml b/crates/e2e_test/testdata/docker-compose.yaml new file mode 100644 index 000000000..d461471b5 --- /dev/null +++ b/crates/e2e_test/testdata/docker-compose.yaml @@ -0,0 +1,62 @@ +# 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. + +version: '3.8' + +services: + rest: + image: tabulario/iceberg-rest:0.10.0 + environment: + - AWS_ACCESS_KEY_ID=admin + - AWS_SECRET_ACCESS_KEY=password + - AWS_REGION=us-east-1 + - CATALOG_URI=jdbc:sqlite:file:/tmp/iceberg_rest_mode=memory + - CATALOG_WAREHOUSE=s3://icebergdata/demo + - CATALOG_IO__IMPL=org.apache.iceberg.aws.s3.S3FileIO + - CATALOG_S3_ENDPOINT=http://minio:9000 + depends_on: + - minio + links: + - minio:icebergdata.minio + ports: + - 8181:8181 + expose: + - 8181 + + minio: + image: minio/minio:latest + environment: + - MINIO_ROOT_USER=admin + - MINIO_ROOT_PASSWORD=password + - MINIO_DOMAIN=minio + ports: + - 9001:9001 + expose: + - 9001 + - 9000 + command: [ "server", "/data", "--console-address", ":9001" ] + + mc: + depends_on: + - minio + image: minio/mc:latest + environment: + - AWS_ACCESS_KEY_ID=admin + - AWS_SECRET_ACCESS_KEY=password + - AWS_REGION=us-east-1 + entrypoint: > + /bin/sh -c " until (/usr/bin/mc config host add minio http://minio:9000 admin password) do echo '...waiting...' && sleep 1; done; /usr/bin/mc rm -r --force minio/icebergdata; /usr/bin/mc mb minio/icebergdata; /usr/bin/mc policy set public minio/icebergdata; tail -f /dev/null " diff --git a/crates/e2e_test/tests/append_data_file_test.rs b/crates/e2e_test/tests/append_data_file_test.rs new file mode 100644 index 000000000..1971d4041 --- /dev/null +++ b/crates/e2e_test/tests/append_data_file_test.rs @@ -0,0 +1,178 @@ +// 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. + +//! Integration tests for rest catalog. + +use std::collections::HashMap; +use std::sync::Arc; + +use arrow_array::{ArrayRef, BooleanArray, Int32Array, RecordBatch, StringArray}; +use futures::TryStreamExt; +use iceberg::spec::{NestedField, PrimitiveType, Schema, Type}; +use iceberg::transaction::Transaction; +use iceberg::writer::base_writer::data_file_writer::{DataFileWriterBuilder, DataFileWriterConfig}; +use iceberg::writer::file_writer::location_generator::{ + DefaultFileNameGenerator, DefaultLocationGenerator, +}; +use iceberg::writer::file_writer::ParquetWriterBuilder; +use iceberg::writer::{IcebergWriter, IcebergWriterBuilder}; +use iceberg::{Catalog, Namespace, NamespaceIdent, TableCreation}; +use iceberg_e2e_test::set_test_fixture; +use parquet::arrow::arrow_reader::ArrowReaderOptions; +use parquet::file::properties::WriterProperties; + +#[tokio::test] +async fn test_append_data_file() { + let fixture = set_test_fixture("test_create_table").await; + + let ns = Namespace::with_properties( + NamespaceIdent::from_strs(["apple", "ios"]).unwrap(), + HashMap::from([ + ("owner".to_string(), "ray".to_string()), + ("community".to_string(), "apache".to_string()), + ]), + ); + + fixture + .rest_catalog + .create_namespace(ns.name(), ns.properties().clone()) + .await + .unwrap(); + + let schema = Schema::builder() + .with_schema_id(1) + .with_identifier_field_ids(vec![2]) + .with_fields(vec![ + NestedField::optional(1, "foo", Type::Primitive(PrimitiveType::String)).into(), + NestedField::required(2, "bar", Type::Primitive(PrimitiveType::Int)).into(), + NestedField::optional(3, "baz", Type::Primitive(PrimitiveType::Boolean)).into(), + ]) + .build() + .unwrap(); + + let table_creation = TableCreation::builder() + .name("t1".to_string()) + .schema(schema.clone()) + .build(); + + let table = fixture + .rest_catalog + .create_table(ns.name(), table_creation) + .await + .unwrap(); + + // Create the writer and write the data + let schema: Arc = Arc::new( + table + .metadata() + .current_schema() + .as_ref() + .try_into() + .unwrap(), + ); + let location_generator = DefaultLocationGenerator::new(table.metadata().clone()).unwrap(); + let file_name_generator = DefaultFileNameGenerator::new( + "test".to_string(), + None, + iceberg::spec::DataFileFormat::Parquet, + ); + let parquet_writer_builder = ParquetWriterBuilder::new( + WriterProperties::default(), + table.metadata().current_schema().clone(), + table.file_io().clone(), + location_generator.clone(), + file_name_generator.clone(), + ); + let data_file_writer_builder = DataFileWriterBuilder::new(parquet_writer_builder); + let mut data_file_writer = data_file_writer_builder + .build(DataFileWriterConfig::new(None)) + .await + .unwrap(); + let col1 = StringArray::from(vec![Some("foo"), Some("bar"), None, Some("baz")]); + let col2 = Int32Array::from(vec![Some(1), Some(2), Some(3), Some(4)]); + let col3 = BooleanArray::from(vec![Some(true), Some(false), None, Some(false)]); + let batch = RecordBatch::try_new(schema.clone(), vec![ + Arc::new(col1) as ArrayRef, + Arc::new(col2) as ArrayRef, + Arc::new(col3) as ArrayRef, + ]) + .unwrap(); + data_file_writer.write(batch.clone()).await.unwrap(); + let data_file = data_file_writer.close().await.unwrap(); + + // check parquet file schema + let content = table + .file_io() + .new_input(data_file[0].file_path()) + .unwrap() + .read() + .await + .unwrap(); + let parquet_reader = parquet::arrow::arrow_reader::ArrowReaderMetadata::load( + &content, + ArrowReaderOptions::default(), + ) + .unwrap(); + let field_ids: Vec = parquet_reader + .parquet_schema() + .columns() + .iter() + .map(|col| col.self_type().get_basic_info().id()) + .collect(); + assert_eq!(field_ids, vec![1, 2, 3]); + + // commit result + let tx = Transaction::new(&table); + let mut append_action = tx.fast_append(None, vec![]).unwrap(); + append_action.add_data_files(data_file.clone()).unwrap(); + let tx = append_action.apply().await.unwrap(); + let table = tx.commit(&fixture.rest_catalog).await.unwrap(); + + // check result + let batch_stream = table + .scan() + .select_all() + .build() + .unwrap() + .to_arrow() + .await + .unwrap(); + let batches: Vec<_> = batch_stream.try_collect().await.unwrap(); + assert_eq!(batches.len(), 1); + assert_eq!(batches[0], batch); + + // commit result again + let tx = Transaction::new(&table); + let mut append_action = tx.fast_append(None, vec![]).unwrap(); + append_action.add_data_files(data_file.clone()).unwrap(); + let tx = append_action.apply().await.unwrap(); + let table = tx.commit(&fixture.rest_catalog).await.unwrap(); + + // check result again + let batch_stream = table + .scan() + .select_all() + .build() + .unwrap() + .to_arrow() + .await + .unwrap(); + let batches: Vec<_> = batch_stream.try_collect().await.unwrap(); + assert_eq!(batches.len(), 2); + assert_eq!(batches[0], batch); + assert_eq!(batches[1], batch); +} diff --git a/crates/e2e_test/tests/conflict_commit_test.rs b/crates/e2e_test/tests/conflict_commit_test.rs new file mode 100644 index 000000000..5b71a742e --- /dev/null +++ b/crates/e2e_test/tests/conflict_commit_test.rs @@ -0,0 +1,145 @@ +// 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. + +//! Integration tests for rest catalog. + +use std::collections::HashMap; +use std::sync::Arc; + +use arrow_array::{ArrayRef, BooleanArray, Int32Array, RecordBatch, StringArray}; +use futures::TryStreamExt; +use iceberg::spec::{NestedField, PrimitiveType, Schema, Type}; +use iceberg::transaction::Transaction; +use iceberg::writer::base_writer::data_file_writer::{DataFileWriterBuilder, DataFileWriterConfig}; +use iceberg::writer::file_writer::location_generator::{ + DefaultFileNameGenerator, DefaultLocationGenerator, +}; +use iceberg::writer::file_writer::ParquetWriterBuilder; +use iceberg::writer::{IcebergWriter, IcebergWriterBuilder}; +use iceberg::{Catalog, Namespace, NamespaceIdent, TableCreation}; +use iceberg_e2e_test::set_test_fixture; +use parquet::file::properties::WriterProperties; + +#[tokio::test] +async fn test_append_data_file_conflict() { + let fixture = set_test_fixture("test_create_table").await; + + let ns = Namespace::with_properties( + NamespaceIdent::from_strs(["apple", "ios"]).unwrap(), + HashMap::from([ + ("owner".to_string(), "ray".to_string()), + ("community".to_string(), "apache".to_string()), + ]), + ); + + fixture + .rest_catalog + .create_namespace(ns.name(), ns.properties().clone()) + .await + .unwrap(); + + let schema = Schema::builder() + .with_schema_id(1) + .with_identifier_field_ids(vec![2]) + .with_fields(vec![ + NestedField::optional(1, "foo", Type::Primitive(PrimitiveType::String)).into(), + NestedField::required(2, "bar", Type::Primitive(PrimitiveType::Int)).into(), + NestedField::optional(3, "baz", Type::Primitive(PrimitiveType::Boolean)).into(), + ]) + .build() + .unwrap(); + + let table_creation = TableCreation::builder() + .name("t1".to_string()) + .schema(schema.clone()) + .build(); + + let table = fixture + .rest_catalog + .create_table(ns.name(), table_creation) + .await + .unwrap(); + + // Create the writer and write the data + let schema: Arc = Arc::new( + table + .metadata() + .current_schema() + .as_ref() + .try_into() + .unwrap(), + ); + let location_generator = DefaultLocationGenerator::new(table.metadata().clone()).unwrap(); + let file_name_generator = DefaultFileNameGenerator::new( + "test".to_string(), + None, + iceberg::spec::DataFileFormat::Parquet, + ); + let parquet_writer_builder = ParquetWriterBuilder::new( + WriterProperties::default(), + table.metadata().current_schema().clone(), + table.file_io().clone(), + location_generator.clone(), + file_name_generator.clone(), + ); + let data_file_writer_builder = DataFileWriterBuilder::new(parquet_writer_builder); + let mut data_file_writer = data_file_writer_builder + .build(DataFileWriterConfig::new(None)) + .await + .unwrap(); + let col1 = StringArray::from(vec![Some("foo"), Some("bar"), None, Some("baz")]); + let col2 = Int32Array::from(vec![Some(1), Some(2), Some(3), Some(4)]); + let col3 = BooleanArray::from(vec![Some(true), Some(false), None, Some(false)]); + let batch = RecordBatch::try_new(schema.clone(), vec![ + Arc::new(col1) as ArrayRef, + Arc::new(col2) as ArrayRef, + Arc::new(col3) as ArrayRef, + ]) + .unwrap(); + data_file_writer.write(batch.clone()).await.unwrap(); + let data_file = data_file_writer.close().await.unwrap(); + + // start two transaction and commit one of them + let tx1 = Transaction::new(&table); + let mut append_action = tx1.fast_append(None, vec![]).unwrap(); + append_action.add_data_files(data_file.clone()).unwrap(); + let tx1 = append_action.apply().await.unwrap(); + let tx2 = Transaction::new(&table); + let mut append_action = tx2.fast_append(None, vec![]).unwrap(); + append_action.add_data_files(data_file.clone()).unwrap(); + let tx2 = append_action.apply().await.unwrap(); + let table = tx2 + .commit(&fixture.rest_catalog) + .await + .expect("The first commit should not fail."); + + // check result + let batch_stream = table + .scan() + .select_all() + .build() + .unwrap() + .to_arrow() + .await + .unwrap(); + let batches: Vec<_> = batch_stream.try_collect().await.unwrap(); + assert_eq!(batches.len(), 1); + assert_eq!(batches[0], batch); + + // another commit should fail + assert!(tx1.commit(&fixture.rest_catalog).await.is_err()); +} diff --git a/crates/iceberg/src/catalog/mod.rs b/crates/iceberg/src/catalog/mod.rs index 70a403905..b78184e3c 100644 --- a/crates/iceberg/src/catalog/mod.rs +++ b/crates/iceberg/src/catalog/mod.rs @@ -1378,7 +1378,7 @@ mod tests { .with_schema_id(1) .with_summary(Summary { operation: Operation::Append, - other: HashMap::default(), + additional_properties: HashMap::default(), }) .build(), }; @@ -1412,7 +1412,7 @@ mod tests { .with_manifest_list("s3://a/b/2.avro") .with_summary(Summary { operation: Operation::Append, - other: HashMap::default(), + additional_properties: HashMap::default(), }) .build(), }; diff --git a/crates/iceberg/src/io/object_cache.rs b/crates/iceberg/src/io/object_cache.rs index 35b6a2c94..88e2d0e2d 100644 --- a/crates/iceberg/src/io/object_cache.rs +++ b/crates/iceberg/src/io/object_cache.rs @@ -185,7 +185,7 @@ mod tests { use crate::spec::{ DataContentType, DataFileBuilder, DataFileFormat, FormatVersion, Literal, Manifest, ManifestContentType, ManifestEntry, ManifestListWriter, ManifestMetadata, ManifestStatus, - ManifestWriter, Struct, TableMetadata, EMPTY_SNAPSHOT_ID, + ManifestWriter, Struct, TableMetadata, }; use crate::table::Table; use crate::TableIdent; @@ -293,9 +293,7 @@ mod tests { .new_output(current_snapshot.manifest_list()) .unwrap(), current_snapshot.snapshot_id(), - current_snapshot - .parent_snapshot_id() - .unwrap_or(EMPTY_SNAPSHOT_ID), + current_snapshot.parent_snapshot_id(), current_snapshot.sequence_number(), ); manifest_list_write diff --git a/crates/iceberg/src/scan.rs b/crates/iceberg/src/scan.rs index 5b5509930..e97c44ce5 100644 --- a/crates/iceberg/src/scan.rs +++ b/crates/iceberg/src/scan.rs @@ -969,7 +969,6 @@ mod tests { DataContentType, DataFileBuilder, DataFileFormat, Datum, FormatVersion, Literal, Manifest, ManifestContentType, ManifestEntry, ManifestListWriter, ManifestMetadata, ManifestStatus, ManifestWriter, NestedField, PrimitiveType, Schema, Struct, TableMetadata, Type, - EMPTY_SNAPSHOT_ID, }; use crate::table::Table; use crate::TableIdent; @@ -1116,9 +1115,7 @@ mod tests { .new_output(current_snapshot.manifest_list()) .unwrap(), current_snapshot.snapshot_id(), - current_snapshot - .parent_snapshot_id() - .unwrap_or(EMPTY_SNAPSHOT_ID), + current_snapshot.parent_snapshot_id(), current_snapshot.sequence_number(), ); manifest_list_write diff --git a/crates/iceberg/src/spec/manifest.rs b/crates/iceberg/src/spec/manifest.rs index 085200b7c..10031bd56 100644 --- a/crates/iceberg/src/spec/manifest.rs +++ b/crates/iceberg/src/spec/manifest.rs @@ -900,6 +900,12 @@ impl ManifestEntry { } } + /// Snapshot id + #[inline] + pub fn snapshot_id(&self) -> Option { + self.snapshot_id + } + /// Data sequence number. #[inline] pub fn sequence_number(&self) -> Option { @@ -1328,7 +1334,7 @@ mod _serde { Ok(Self { content: value.content as i32, file_path: value.file_path, - file_format: value.file_format.to_string(), + file_format: value.file_format.to_string().to_ascii_uppercase(), partition: RawLiteral::try_from( Literal::Struct(value.partition), &Type::Struct(partition_type.clone()), diff --git a/crates/iceberg/src/spec/manifest_list.rs b/crates/iceberg/src/spec/manifest_list.rs index 3aaecf12d..7ef6b8c31 100644 --- a/crates/iceberg/src/spec/manifest_list.rs +++ b/crates/iceberg/src/spec/manifest_list.rs @@ -106,15 +106,17 @@ impl std::fmt::Debug for ManifestListWriter { impl ManifestListWriter { /// Construct a v1 [`ManifestListWriter`] that writes to a provided [`OutputFile`]. - pub fn v1(output_file: OutputFile, snapshot_id: i64, parent_snapshot_id: i64) -> Self { - let metadata = HashMap::from_iter([ + pub fn v1(output_file: OutputFile, snapshot_id: i64, parent_snapshot_id: Option) -> Self { + let mut metadata = HashMap::from_iter([ ("snapshot-id".to_string(), snapshot_id.to_string()), - ( - "parent-snapshot-id".to_string(), - parent_snapshot_id.to_string(), - ), ("format-version".to_string(), "1".to_string()), ]); + if let Some(parent_snapshot_id) = parent_snapshot_id { + metadata.insert( + "parent-snapshot-id".to_string(), + parent_snapshot_id.to_string(), + ); + } Self::new(FormatVersion::V1, output_file, metadata, 0, snapshot_id) } @@ -122,18 +124,20 @@ impl ManifestListWriter { pub fn v2( output_file: OutputFile, snapshot_id: i64, - parent_snapshot_id: i64, + parent_snapshot_id: Option, sequence_number: i64, ) -> Self { - let metadata = HashMap::from_iter([ + let mut metadata = HashMap::from_iter([ ("snapshot-id".to_string(), snapshot_id.to_string()), - ( - "parent-snapshot-id".to_string(), - parent_snapshot_id.to_string(), - ), ("sequence-number".to_string(), sequence_number.to_string()), ("format-version".to_string(), "2".to_string()), ]); + if let Some(parent_snapshot_id) = parent_snapshot_id { + metadata.insert( + "parent-snapshot-id".to_string(), + parent_snapshot_id.to_string(), + ); + } Self::new( FormatVersion::V2, output_file, @@ -580,6 +584,18 @@ pub struct ManifestFile { pub key_metadata: Vec, } +impl ManifestFile { + /// Checks if the manifest file has any added files. + pub fn has_added_files(&self) -> bool { + self.added_files_count.is_none() || self.added_files_count.unwrap() > 0 + } + + /// Checks if the manifest file has any existed files. + pub fn has_existing_files(&self) -> bool { + self.existing_files_count.is_none() || self.existing_files_count.unwrap() > 0 + } +} + /// The type of files tracked by the manifest, either data or delete files; Data(0) for all v1 manifests #[derive(Debug, PartialEq, Clone, Eq)] pub enum ManifestContentType { @@ -1146,7 +1162,7 @@ mod test { let mut writer = ManifestListWriter::v1( file_io.new_output(full_path.clone()).unwrap(), 1646658105718557341, - 1646658105718557341, + Some(1646658105718557341), ); writer @@ -1213,7 +1229,7 @@ mod test { let mut writer = ManifestListWriter::v2( file_io.new_output(full_path.clone()).unwrap(), 1646658105718557341, - 1646658105718557341, + Some(1646658105718557341), 1, ); @@ -1335,7 +1351,7 @@ mod test { let io = FileIOBuilder::new_fs_io().build().unwrap(); let output_file = io.new_output(path.to_str().unwrap()).unwrap(); - let mut writer = ManifestListWriter::v1(output_file, 1646658105718557341, 0); + let mut writer = ManifestListWriter::v1(output_file, 1646658105718557341, Some(0)); writer .add_manifests(expected_manifest_list.entries.clone().into_iter()) .unwrap(); @@ -1391,7 +1407,7 @@ mod test { let io = FileIOBuilder::new_fs_io().build().unwrap(); let output_file = io.new_output(path.to_str().unwrap()).unwrap(); - let mut writer = ManifestListWriter::v2(output_file, snapshot_id, 0, seq_num); + let mut writer = ManifestListWriter::v2(output_file, snapshot_id, Some(0), seq_num); writer .add_manifests(expected_manifest_list.entries.clone().into_iter()) .unwrap(); @@ -1445,7 +1461,7 @@ mod test { let io = FileIOBuilder::new_fs_io().build().unwrap(); let output_file = io.new_output(path.to_str().unwrap()).unwrap(); - let mut writer = ManifestListWriter::v2(output_file, 1646658105718557341, 0, 1); + let mut writer = ManifestListWriter::v2(output_file, 1646658105718557341, Some(0), 1); writer .add_manifests(expected_manifest_list.entries.clone().into_iter()) .unwrap(); diff --git a/crates/iceberg/src/spec/snapshot.rs b/crates/iceberg/src/spec/snapshot.rs index f42e736ea..0240afaba 100644 --- a/crates/iceberg/src/spec/snapshot.rs +++ b/crates/iceberg/src/spec/snapshot.rs @@ -59,7 +59,7 @@ pub struct Summary { pub operation: Operation, /// Other summary data. #[serde(flatten)] - pub other: HashMap, + pub additional_properties: HashMap, } impl Default for Operation { @@ -291,7 +291,7 @@ pub(super) mod _serde { }, summary: v1.summary.unwrap_or(Summary { operation: Operation::default(), - other: HashMap::new(), + additional_properties: HashMap::new(), }), schema_id: v1.schema_id, }) @@ -365,6 +365,21 @@ pub enum SnapshotRetention { }, } +impl SnapshotRetention { + /// Create a new branch retention policy + pub fn branch( + min_snapshots_to_keep: Option, + max_snapshot_age_ms: Option, + max_ref_age_ms: Option, + ) -> Self { + SnapshotRetention::Branch { + min_snapshots_to_keep, + max_snapshot_age_ms, + max_ref_age_ms, + } + } +} + #[cfg(test)] mod tests { use std::collections::HashMap; @@ -401,7 +416,7 @@ mod tests { assert_eq!( Summary { operation: Operation::Append, - other: HashMap::new() + additional_properties: HashMap::new() }, *result.summary() ); diff --git a/crates/iceberg/src/spec/table_metadata.rs b/crates/iceberg/src/spec/table_metadata.rs index daed758cb..01f526656 100644 --- a/crates/iceberg/src/spec/table_metadata.rs +++ b/crates/iceberg/src/spec/table_metadata.rs @@ -189,6 +189,18 @@ impl TableMetadata { self.last_sequence_number } + /// Returns the next sequence number for the table. + /// + /// For format version 1, it always returns the initial sequence number. + /// For other versions, it returns the last sequence number incremented by 1. + #[inline] + pub fn next_sequence_number(&self) -> i64 { + match self.format_version { + FormatVersion::V1 => INITIAL_SEQUENCE_NUMBER, + _ => self.last_sequence_number + 1, + } + } + /// Returns last updated time. #[inline] pub fn last_updated_timestamp(&self) -> Result> { @@ -1552,7 +1564,7 @@ mod tests { .with_sequence_number(0) .with_schema_id(0) .with_manifest_list("/home/iceberg/warehouse/nyc/taxis/metadata/snap-638933773299822130-1-7e6760f0-4f6c-4b23-b907-0a5a174e3863.avro") - .with_summary(Summary { operation: Operation::Append, other: HashMap::from_iter(vec![("spark.app.id".to_string(), "local-1662532784305".to_string()), ("added-data-files".to_string(), "4".to_string()), ("added-records".to_string(), "4".to_string()), ("added-files-size".to_string(), "6001".to_string())]) }) + .with_summary(Summary { operation: Operation::Append, additional_properties: HashMap::from_iter(vec![("spark.app.id".to_string(), "local-1662532784305".to_string()), ("added-data-files".to_string(), "4".to_string()), ("added-records".to_string(), "4".to_string()), ("added-files-size".to_string(), "6001".to_string())]) }) .build(); let expected = TableMetadata { @@ -1971,7 +1983,7 @@ mod tests { .with_manifest_list("s3://a/b/1.avro") .with_summary(Summary { operation: Operation::Append, - other: HashMap::new(), + additional_properties: HashMap::new(), }) .build(); @@ -1984,7 +1996,7 @@ mod tests { .with_manifest_list("s3://a/b/2.avro") .with_summary(Summary { operation: Operation::Append, - other: HashMap::new(), + additional_properties: HashMap::new(), }) .build(); diff --git a/crates/iceberg/src/spec/values.rs b/crates/iceberg/src/spec/values.rs index 1b10f9b04..49748df0d 100644 --- a/crates/iceberg/src/spec/values.rs +++ b/crates/iceberg/src/spec/values.rs @@ -1567,6 +1567,14 @@ impl Literal { })?; Ok(Self::decimal(decimal.mantissa())) } + + /// Attempts to convert the Literal to a PrimitiveLiteral + pub fn as_primitive_literal(&self) -> Option { + match self { + Literal::Primitive(primitive) => Some(primitive.clone()), + _ => None, + } + } } /// The partition struct stores the tuple of partition values for each file. @@ -1606,6 +1614,11 @@ impl Struct { pub fn is_null_at_index(&self, index: usize) -> bool { self.null_bitmap[index] } + + /// Return fields in the struct. + pub fn fields(&self) -> &[Literal] { + &self.fields + } } impl Index for Struct { diff --git a/crates/iceberg/src/transaction.rs b/crates/iceberg/src/transaction.rs index f29cf5122..906690614 100644 --- a/crates/iceberg/src/transaction.rs +++ b/crates/iceberg/src/transaction.rs @@ -19,14 +19,27 @@ use std::cmp::Ordering; use std::collections::HashMap; +use std::future::Future; use std::mem::discriminant; +use std::ops::RangeFrom; +use std::sync::Arc; + +use uuid::Uuid; use crate::error::Result; -use crate::spec::{FormatVersion, NullOrder, SortDirection, SortField, SortOrder, Transform}; +use crate::io::OutputFile; +use crate::spec::{ + BoundPartitionSpec, DataFile, DataFileFormat, FormatVersion, Manifest, ManifestEntry, + ManifestFile, ManifestListWriter, ManifestMetadata, ManifestWriter, NullOrder, Operation, + Schema, Snapshot, SnapshotReference, SnapshotRetention, SortDirection, SortField, SortOrder, + Struct, StructType, Summary, Transform, MAIN_BRANCH, +}; use crate::table::Table; use crate::TableUpdate::UpgradeFormatVersion; use crate::{Catalog, Error, ErrorKind, TableCommit, TableRequirement, TableUpdate}; +const META_ROOT_PATH: &str = "metadata"; + /// Table transaction. pub struct Transaction<'a> { table: &'a Table, @@ -96,6 +109,60 @@ impl<'a> Transaction<'a> { Ok(self) } + fn generate_unique_snapshot_id(&self) -> i64 { + let generate_random_id = || -> i64 { + let (lhs, rhs) = Uuid::new_v4().as_u64_pair(); + let snapshot_id = (lhs ^ rhs) as i64; + if snapshot_id < 0 { + -snapshot_id + } else { + snapshot_id + } + }; + let mut snapshot_id = generate_random_id(); + while self + .table + .metadata() + .snapshots() + .any(|s| s.snapshot_id() == snapshot_id) + { + snapshot_id = generate_random_id(); + } + snapshot_id + } + + /// Creates a fast append action. + pub fn fast_append( + self, + commit_uuid: Option, + key_metadata: Vec, + ) -> Result> { + let parent_snapshot_id = self + .table + .metadata() + .current_snapshot() + .map(|s| s.snapshot_id()); + let snapshot_id = self.generate_unique_snapshot_id(); + let schema = self.table.metadata().current_schema().as_ref().clone(); + let schema_id = schema.schema_id(); + let format_version = self.table.metadata().format_version(); + let partition_spec = self.table.metadata().default_partition_spec().clone(); + let commit_uuid = commit_uuid.unwrap_or_else(Uuid::new_v4); + + FastAppendAction::new( + self, + parent_snapshot_id, + snapshot_id, + schema, + schema_id, + format_version, + partition_spec, + key_metadata, + commit_uuid, + HashMap::new(), + ) + } + /// Creates replace sort order action. pub fn replace_sort_order(self) -> ReplaceSortOrderAction<'a> { ReplaceSortOrderAction { @@ -122,6 +189,383 @@ impl<'a> Transaction<'a> { } } +/// FastAppendAction is a transaction action for fast append data files to the table. +pub struct FastAppendAction<'a> { + snapshot_produce_action: SnapshotProduceAction<'a>, +} + +impl<'a> FastAppendAction<'a> { + #[allow(clippy::too_many_arguments)] + pub(crate) fn new( + tx: Transaction<'a>, + parent_snapshot_id: Option, + snapshot_id: i64, + schema: Schema, + schema_id: i32, + format_version: FormatVersion, + partition_spec: Arc, + key_metadata: Vec, + commit_uuid: Uuid, + snapshot_properties: HashMap, + ) -> Result { + Ok(Self { + snapshot_produce_action: SnapshotProduceAction::new( + tx, + snapshot_id, + parent_snapshot_id, + schema_id, + format_version, + partition_spec, + schema, + key_metadata, + commit_uuid, + snapshot_properties, + )?, + }) + } + + /// Add data files to the snapshot. + pub fn add_data_files( + &mut self, + data_files: impl IntoIterator, + ) -> Result<&mut Self> { + self.snapshot_produce_action.add_data_files(data_files)?; + Ok(self) + } + + /// Finished building the action and apply it to the transaction. + pub async fn apply(self) -> Result> { + self.snapshot_produce_action + .apply(FastAppendOperation, DefaultManifestProcess) + .await + } +} + +struct FastAppendOperation; + +impl SnapshotProduceOperation for FastAppendOperation { + fn operation(&self) -> Operation { + Operation::Append + } + + async fn delete_entries( + &self, + _snapshot_produce: &SnapshotProduceAction<'_>, + ) -> Result> { + Ok(vec![]) + } + + async fn existing_manifest( + &self, + snapshot_produce: &SnapshotProduceAction<'_>, + ) -> Result> { + let Some(snapshot) = snapshot_produce + .parent_snapshot_id + .and_then(|id| snapshot_produce.tx.table.metadata().snapshot_by_id(id)) + else { + return Ok(vec![]); + }; + + let manifest_list = snapshot + .load_manifest_list( + snapshot_produce.tx.table.file_io(), + &snapshot_produce.tx.table.metadata_ref(), + ) + .await?; + + Ok(manifest_list + .entries() + .iter() + .filter(|entry| entry.has_added_files() || entry.has_existing_files()) + .cloned() + .collect()) + } +} + +trait SnapshotProduceOperation: Send + Sync { + fn operation(&self) -> Operation; + #[allow(unused)] + fn delete_entries( + &self, + snapshot_produce: &SnapshotProduceAction, + ) -> impl Future>> + Send; + fn existing_manifest( + &self, + snapshot_produce: &SnapshotProduceAction, + ) -> impl Future>> + Send; +} + +struct DefaultManifestProcess; + +impl ManifestProcess for DefaultManifestProcess { + fn process_manifeset(&self, manifests: Vec) -> Vec { + manifests + } +} + +trait ManifestProcess: Send + Sync { + fn process_manifeset(&self, manifests: Vec) -> Vec; +} + +struct SnapshotProduceAction<'a> { + tx: Transaction<'a>, + + parent_snapshot_id: Option, + snapshot_id: i64, + schema_id: i32, + format_version: FormatVersion, + partition_spec: Arc, + schema: Schema, + key_metadata: Vec, + + commit_uuid: Uuid, + + snapshot_properties: HashMap, + added_data_files: Vec, + + // A counter used to generate unique manifest file names. + // It starts from 0 and increments for each new manifest file. + // Note: This counter is limited to the range of (0..u64::MAX). + manifest_counter: RangeFrom, +} + +impl<'a> SnapshotProduceAction<'a> { + #[allow(clippy::too_many_arguments)] + pub(crate) fn new( + tx: Transaction<'a>, + snapshot_id: i64, + parent_snapshot_id: Option, + schema_id: i32, + format_version: FormatVersion, + partition_spec: Arc, + schema: Schema, + key_metadata: Vec, + commit_uuid: Uuid, + snapshot_properties: HashMap, + ) -> Result { + Ok(Self { + tx, + parent_snapshot_id, + snapshot_id, + schema_id, + format_version, + commit_uuid, + snapshot_properties, + added_data_files: vec![], + manifest_counter: (0..), + partition_spec, + schema, + key_metadata, + }) + } + + // Check if the partition value is compatible with the partition type. + fn validate_partition_value( + partition_value: &Struct, + partition_type: &StructType, + ) -> Result<()> { + if partition_value.fields().len() != partition_type.fields().len() { + return Err(Error::new( + ErrorKind::DataInvalid, + "Partition value is not compatitable with partition type", + )); + } + if partition_value + .fields() + .iter() + .zip(partition_type.fields()) + .any(|(field_from_value, field_from_type)| { + !field_from_type + .field_type + .as_primitive_type() + .unwrap() + .compatible(&field_from_value.as_primitive_literal().unwrap()) + }) + { + return Err(Error::new( + ErrorKind::DataInvalid, + "Partition value is not compatitable partition type", + )); + } + Ok(()) + } + + /// Add data files to the snapshot. + pub fn add_data_files( + &mut self, + data_files: impl IntoIterator, + ) -> Result<&mut Self> { + let data_files: Vec = data_files.into_iter().collect(); + for data_file in &data_files { + if data_file.content_type() != crate::spec::DataContentType::Data { + return Err(Error::new( + ErrorKind::DataInvalid, + "Only data content type is allowed for fast append", + )); + } + Self::validate_partition_value( + data_file.partition(), + self.partition_spec.partition_type(), + )?; + } + self.added_data_files.extend(data_files); + Ok(self) + } + + fn new_manifest_output(&mut self) -> Result { + let new_manifest_path = format!( + "{}/{}/{}-m{}.{}", + self.tx.table.metadata().location(), + META_ROOT_PATH, + self.commit_uuid, + self.manifest_counter.next().unwrap(), + DataFileFormat::Avro + ); + self.tx.table.file_io().new_output(new_manifest_path) + } + + // Write manifest file for added data files and return the ManifestFile for ManifestList. + async fn write_added_manifest(&mut self) -> Result { + let added_data_files = std::mem::take(&mut self.added_data_files); + let manifest_entries = added_data_files + .into_iter() + .map(|data_file| { + let builder = ManifestEntry::builder() + .status(crate::spec::ManifestStatus::Added) + .data_file(data_file); + if self.format_version as u8 == 1u8 { + builder.snapshot_id(self.snapshot_id).build() + } else { + // For format version > 1, we set the snapshot id at the inherited time to avoid rewrite the manifest file when + // commit failed. + builder.build() + } + }) + .collect(); + let manifest_meta = ManifestMetadata::builder() + .schema(self.schema.clone().into()) + .schema_id(self.schema_id) + .format_version(self.format_version) + .partition_spec(self.partition_spec.as_ref().clone()) + .content(crate::spec::ManifestContentType::Data) + .build(); + let manifest = Manifest::new(manifest_meta, manifest_entries); + let writer = ManifestWriter::new( + self.new_manifest_output()?, + self.snapshot_id, + self.key_metadata.clone(), + ); + writer.write(manifest).await + } + + async fn manifest_file( + &mut self, + snapshot_produce_operation: &OP, + manifest_process: &MP, + ) -> Result> { + let added_manifest = self.write_added_manifest().await?; + let existing_manifests = snapshot_produce_operation.existing_manifest(self).await?; + + let mut manifest_files = vec![added_manifest]; + manifest_files.extend(existing_manifests); + let manifest_files = manifest_process.process_manifeset(manifest_files); + Ok(manifest_files) + } + + // # TODO + // Fulfill this function + fn summary(&self, snapshot_produce_operation: &OP) -> Summary { + Summary { + operation: snapshot_produce_operation.operation(), + additional_properties: self.snapshot_properties.clone(), + } + } + + fn generate_manifest_list_file_path(&self, attempt: i64) -> String { + format!( + "{}/{}/snap-{}-{}-{}.{}", + self.tx.table.metadata().location(), + META_ROOT_PATH, + self.snapshot_id, + attempt, + self.commit_uuid, + DataFileFormat::Avro + ) + } + + /// Finished building the action and apply it to the transaction. + pub async fn apply( + mut self, + snapshot_produce_operation: OP, + process: MP, + ) -> Result> { + let new_manifests = self + .manifest_file(&snapshot_produce_operation, &process) + .await?; + let next_seq_num = self.tx.table.metadata().next_sequence_number(); + + let summary = self.summary(&snapshot_produce_operation); + + let manifest_list_path = self.generate_manifest_list_file_path(0); + + let mut manifest_list_writer = match self.tx.table.metadata().format_version() { + FormatVersion::V1 => ManifestListWriter::v1( + self.tx + .table + .file_io() + .new_output(manifest_list_path.clone())?, + self.snapshot_id, + self.parent_snapshot_id, + ), + FormatVersion::V2 => ManifestListWriter::v2( + self.tx + .table + .file_io() + .new_output(manifest_list_path.clone())?, + self.snapshot_id, + self.parent_snapshot_id, + next_seq_num, + ), + }; + manifest_list_writer.add_manifests(new_manifests.into_iter())?; + manifest_list_writer.close().await?; + + let commit_ts = chrono::Utc::now().timestamp_millis(); + let new_snapshot = Snapshot::builder() + .with_manifest_list(manifest_list_path) + .with_snapshot_id(self.snapshot_id) + .with_parent_snapshot_id(self.parent_snapshot_id) + .with_sequence_number(next_seq_num) + .with_summary(summary) + .with_schema_id(self.schema_id) + .with_timestamp_ms(commit_ts) + .build(); + + self.tx.append_updates(vec![ + TableUpdate::AddSnapshot { + snapshot: new_snapshot, + }, + TableUpdate::SetSnapshotRef { + ref_name: MAIN_BRANCH.to_string(), + reference: SnapshotReference::new( + self.snapshot_id, + SnapshotRetention::branch(None, None, None), + ), + }, + ])?; + self.tx.append_requirements(vec![ + TableRequirement::UuidMatch { + uuid: self.tx.table.metadata().uuid(), + }, + TableRequirement::RefSnapshotIdMatch { + r#ref: MAIN_BRANCH.to_string(), + snapshot_id: self.parent_snapshot_id, + }, + ])?; + Ok(self.tx) + } +} + /// Transaction action for replacing sort order. pub struct ReplaceSortOrderAction<'a> { tx: Transaction<'a>, @@ -203,10 +647,13 @@ mod tests { use std::fs::File; use std::io::BufReader; - use crate::io::FileIO; - use crate::spec::{FormatVersion, TableMetadata}; + use crate::io::FileIOBuilder; + use crate::spec::{ + DataContentType, DataFileBuilder, DataFileFormat, FormatVersion, Literal, Struct, + TableMetadata, + }; use crate::table::Table; - use crate::transaction::Transaction; + use crate::transaction::{Transaction, MAIN_BRANCH}; use crate::{TableIdent, TableRequirement, TableUpdate}; fn make_v1_table() -> Table { @@ -223,7 +670,7 @@ mod tests { .metadata(resp) .metadata_location("s3://bucket/test/location/metadata/v1.json".to_string()) .identifier(TableIdent::from_strs(["ns1", "test1"]).unwrap()) - .file_io(FileIO::from_path("/tmp").unwrap().build().unwrap()) + .file_io(FileIOBuilder::new("memory").build().unwrap()) .build() .unwrap() } @@ -242,7 +689,26 @@ mod tests { .metadata(resp) .metadata_location("s3://bucket/test/location/metadata/v1.json".to_string()) .identifier(TableIdent::from_strs(["ns1", "test1"]).unwrap()) - .file_io(FileIO::from_path("/tmp").unwrap().build().unwrap()) + .file_io(FileIOBuilder::new("memory").build().unwrap()) + .build() + .unwrap() + } + + fn make_v2_minimal_table() -> Table { + let file = File::open(format!( + "{}/testdata/table_metadata/{}", + env!("CARGO_MANIFEST_DIR"), + "TableMetadataV2ValidMinimal.json" + )) + .unwrap(); + let reader = BufReader::new(file); + let resp = serde_json::from_reader::<_, TableMetadata>(reader).unwrap(); + + Table::builder() + .metadata(resp) + .metadata_location("s3://bucket/test/location/metadata/v1.json".to_string()) + .identifier(TableIdent::from_strs(["ns1", "test1"]).unwrap()) + .file_io(FileIOBuilder::new("memory").build().unwrap()) .build() .unwrap() } @@ -347,6 +813,76 @@ mod tests { ); } + #[tokio::test] + async fn test_fast_append_action() { + let table = make_v2_minimal_table(); + let tx = Transaction::new(&table); + + let data_file = DataFileBuilder::default() + .content(DataContentType::Data) + .file_path("test/3.parquet".to_string()) + .file_format(DataFileFormat::Parquet) + .file_size_in_bytes(100) + .record_count(1) + .partition(Struct::from_iter([Some(Literal::long(300))])) + .build() + .unwrap(); + let mut action = tx.fast_append(None, vec![]).unwrap(); + action.add_data_files(vec![data_file.clone()]).unwrap(); + let tx = action.apply().await.unwrap(); + + // check updates and requirements + assert!( + matches!((&tx.updates[0],&tx.updates[1]), (TableUpdate::AddSnapshot { snapshot },TableUpdate::SetSnapshotRef { reference,ref_name }) if snapshot.snapshot_id() == reference.snapshot_id && ref_name == MAIN_BRANCH) + ); + assert_eq!( + vec![ + TableRequirement::UuidMatch { + uuid: tx.table.metadata().uuid() + }, + TableRequirement::RefSnapshotIdMatch { + r#ref: MAIN_BRANCH.to_string(), + snapshot_id: tx.table.metadata().current_snapshot_id + } + ], + tx.requirements + ); + + // check manifest list + let new_snapshot = if let TableUpdate::AddSnapshot { snapshot } = &tx.updates[0] { + snapshot + } else { + unreachable!() + }; + let manifest_list = new_snapshot + .load_manifest_list(table.file_io(), table.metadata()) + .await + .unwrap(); + assert_eq!(1, manifest_list.entries().len()); + assert_eq!( + manifest_list.entries()[0].sequence_number, + new_snapshot.sequence_number() + ); + + // check manifset + let manifest = manifest_list.entries()[0] + .load_manifest(table.file_io()) + .await + .unwrap(); + assert_eq!(1, manifest.entries().len()); + assert_eq!( + new_snapshot.sequence_number(), + manifest.entries()[0] + .sequence_number() + .expect("Inherit sequence number by load manifest") + ); + assert_eq!( + new_snapshot.snapshot_id(), + manifest.entries()[0].snapshot_id().unwrap() + ); + assert_eq!(data_file, *manifest.entries()[0].data_file()); + } + #[test] fn test_do_same_update_in_same_transaction() { let table = make_v2_table();