diff --git a/Cargo.lock b/Cargo.lock index fae8965677..f9e26a1b82 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -5614,6 +5614,7 @@ dependencies = [ "sail-common", "sail-common-datafusion", "sail-delta-lake", + "sail-iceberg", "serde", "serde_yaml", "syn", @@ -5731,6 +5732,31 @@ dependencies = [ "serde_json", ] +[[package]] +name = "sail-iceberg" +version = "0.3.7" +dependencies = [ + "apache-avro", + "arrow-schema", + "async-trait", + "chrono", + "datafusion", + "datafusion-common", + "futures", + "log", + "num-bigint", + "num-traits", + "object_store", + "once_cell", + "ordered-float 5.1.0", + "percent-encoding", + "sail-common-datafusion", + "serde", + "serde_json", + "url", + "uuid", +] + [[package]] name = "sail-logical-optimizer" version = "0.3.7" diff --git a/Cargo.toml b/Cargo.toml index 61e875d45f..80e078f01b 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -104,6 +104,7 @@ bytes = "1.10.1" indexmap = "2.11.4" pin-project-lite = "0.2.16" ordered-float = { version = "5.1.0", features = ["serde"] } +apache-avro = { version = "0.20.0" } ###### # The versions of the following dependencies are managed manually. diff --git a/crates/sail-data-source/Cargo.toml b/crates/sail-data-source/Cargo.toml index 95d35ddb38..64e7f57280 100644 --- a/crates/sail-data-source/Cargo.toml +++ b/crates/sail-data-source/Cargo.toml @@ -10,6 +10,7 @@ workspace = true sail-common = { path = "../sail-common" } sail-common-datafusion = { path = "../sail-common-datafusion" } sail-delta-lake = { path = "../sail-delta-lake" } +sail-iceberg = { path = "../sail-iceberg" } async-trait = { workspace = true } serde = { workspace = true } diff --git a/crates/sail-data-source/src/formats/iceberg.rs b/crates/sail-data-source/src/formats/iceberg.rs new file mode 100644 index 0000000000..4131e2cbab --- /dev/null +++ b/crates/sail-data-source/src/formats/iceberg.rs @@ -0,0 +1,45 @@ +use std::sync::Arc; + +use async_trait::async_trait; +use datafusion::catalog::{Session, TableProvider}; +use datafusion::common::Result; +use datafusion::physical_plan::ExecutionPlan; +use sail_common_datafusion::datasource::{SinkInfo, SourceInfo, TableFormat}; +use sail_iceberg::IcebergTableFormat; + +/// Iceberg table format implementation that delegates to sail-iceberg +#[derive(Debug)] +pub struct IcebergDataSourceFormat { + inner: IcebergTableFormat, +} + +impl Default for IcebergDataSourceFormat { + fn default() -> Self { + Self { + inner: IcebergTableFormat, + } + } +} + +#[async_trait] +impl TableFormat for IcebergDataSourceFormat { + fn name(&self) -> &str { + self.inner.name() + } + + async fn create_provider( + &self, + ctx: &dyn Session, + info: SourceInfo, + ) -> Result> { + self.inner.create_provider(ctx, info).await + } + + async fn create_writer( + &self, + ctx: &dyn Session, + info: SinkInfo, + ) -> Result> { + self.inner.create_writer(ctx, info).await + } +} diff --git a/crates/sail-data-source/src/formats/mod.rs b/crates/sail-data-source/src/formats/mod.rs index 1b02dec6e1..2998e8a1bd 100644 --- a/crates/sail-data-source/src/formats/mod.rs +++ b/crates/sail-data-source/src/formats/mod.rs @@ -4,6 +4,7 @@ pub mod binary; pub mod console; pub mod csv; pub mod delta; +pub mod iceberg; pub mod json; pub mod listing; pub mod parquet; diff --git a/crates/sail-data-source/src/options/data/iceberg_read.yaml b/crates/sail-data-source/src/options/data/iceberg_read.yaml new file mode 100644 index 0000000000..55c067f0a1 --- /dev/null +++ b/crates/sail-data-source/src/options/data/iceberg_read.yaml @@ -0,0 +1,32 @@ +# Options for reading from an Apache Iceberg table. + +- key: use_ref + aliases: + - ref + - branch + - tag + - iceberg.ref + description: | + Select a snapshot reference (tag or branch) to time-travel when reading. + If unset, the current snapshot is used. + supported: true + rust_type: String + +- key: snapshot_id + aliases: + - snapshot-id + - snapshotId + description: | + Select a specific snapshot id to time-travel when reading. + supported: true + rust_type: i64 + rust_deserialize_with: crate::options::serde::deserialize_i64 + +- key: timestamp_as_of + aliases: + - timestampAsOf + description: | + Select snapshot as of the given timestamp. Accepts RFC3339 or 'yyyy-MM-dd HH:mm:ss.SSS'. + supported: true + rust_type: String + diff --git a/crates/sail-data-source/src/options/data/iceberg_write.yaml b/crates/sail-data-source/src/options/data/iceberg_write.yaml new file mode 100644 index 0000000000..5dcc95fcb5 --- /dev/null +++ b/crates/sail-data-source/src/options/data/iceberg_write.yaml @@ -0,0 +1,22 @@ +# Options for writing to an Apache Iceberg table. + +- key: overwrite_schema + aliases: + - overwriteSchema + description: | + If `true`, allows overwriting the schema of the table when using overwrite mode. + default: "false" + supported: true + rust_type: bool + rust_deserialize_with: crate::options::serde::deserialize_bool + +- key: merge_schema + aliases: + - mergeSchema + description: | + If `true`, allows automatic schema merging during an append or overwrite operation. + default: "false" + supported: true + rust_type: bool + rust_deserialize_with: crate::options::serde::deserialize_bool + diff --git a/crates/sail-data-source/src/registry.rs b/crates/sail-data-source/src/registry.rs index c45cd66651..ced13c386b 100644 --- a/crates/sail-data-source/src/registry.rs +++ b/crates/sail-data-source/src/registry.rs @@ -11,6 +11,7 @@ use crate::formats::binary::BinaryTableFormat; use crate::formats::console::ConsoleTableFormat; use crate::formats::csv::CsvTableFormat; use crate::formats::delta::DeltaTableFormat; +use crate::formats::iceberg::IcebergDataSourceFormat; use crate::formats::json::JsonTableFormat; use crate::formats::parquet::ParquetTableFormat; use crate::formats::rate::RateTableFormat; @@ -42,6 +43,7 @@ impl TableFormatRegistry { registry.register_format(Arc::new(BinaryTableFormat::default())); registry.register_format(Arc::new(CsvTableFormat::default())); registry.register_format(Arc::new(DeltaTableFormat)); + registry.register_format(Arc::new(IcebergDataSourceFormat::default())); registry.register_format(Arc::new(JsonTableFormat::default())); registry.register_format(Arc::new(ParquetTableFormat::default())); registry.register_format(Arc::new(TextTableFormat::default())); diff --git a/crates/sail-iceberg/Cargo.toml b/crates/sail-iceberg/Cargo.toml new file mode 100644 index 0000000000..f632c24a02 --- /dev/null +++ b/crates/sail-iceberg/Cargo.toml @@ -0,0 +1,39 @@ +[package] +name = "sail-iceberg" +version = { workspace = true } +edition = { workspace = true } + +[dependencies] +sail-common-datafusion = { path = "../sail-common-datafusion" } + +# DataFusion dependencies +datafusion = { workspace = true } +datafusion-common = { workspace = true } + +# Arrow dependencies +arrow-schema = { workspace = true } + +# Essential utilities +async-trait = { workspace = true } +object_store = { workspace = true } +chrono = { workspace = true } +serde_json = { workspace = true } +url = { workspace = true } +futures = { workspace = true } +serde = { workspace = true, features = ["derive"] } +uuid = { workspace = true } +# parquet = { workspace = true } +# bytes = { workspace = true } +# indexmap = { workspace = true } +log = { workspace = true } +# itertools = { workspace = true } +percent-encoding = { workspace = true } +once_cell = { workspace = true } +ordered-float = { workspace = true } +apache-avro = { workspace = true } +num-bigint = { workspace = true } +num-traits = { workspace = true } +# base64 = { workspace = true } + +[lints] +workspace = true diff --git a/crates/sail-iceberg/src/arrow_conversion.rs b/crates/sail-iceberg/src/arrow_conversion.rs new file mode 100644 index 0000000000..7936f85311 --- /dev/null +++ b/crates/sail-iceberg/src/arrow_conversion.rs @@ -0,0 +1,195 @@ +use std::sync::Arc; + +use arrow_schema::{ + DataType as ArrowDataType, Field as ArrowField, Schema as ArrowSchema, TimeUnit, +}; +use datafusion_common::Result; + +use crate::spec::{NestedField, PrimitiveType, Schema, StructType, Type}; + +/// Convert Iceberg schema to Arrow schema +pub fn iceberg_schema_to_arrow(schema: &Schema) -> Result { + let fields = schema + .fields() + .iter() + .map(|field| iceberg_field_to_arrow(field)) + .collect::>>()?; + + Ok(ArrowSchema::new(fields)) +} + +/// Convert Iceberg field to Arrow field +pub fn iceberg_field_to_arrow(field: &NestedField) -> Result { + let arrow_type = iceberg_type_to_arrow(&field.field_type)?; + let nullable = !field.required; + + Ok(ArrowField::new(&field.name, arrow_type, nullable)) +} + +/// Convert Iceberg type to Arrow data type +pub fn iceberg_type_to_arrow(iceberg_type: &Type) -> Result { + match iceberg_type { + Type::Primitive(primitive) => iceberg_primitive_to_arrow(primitive), + Type::Struct(struct_type) => iceberg_struct_to_arrow(struct_type), + Type::List(list_type) => { + let element_field = iceberg_field_to_arrow(&list_type.element_field)?; + Ok(ArrowDataType::List(Arc::new(element_field))) + } + Type::Map(map_type) => { + let key_field = iceberg_field_to_arrow(&map_type.key_field)?; + let value_field = iceberg_field_to_arrow(&map_type.value_field)?; + + // Arrow Map type expects a struct with key and value fields + let entries_field = ArrowField::new( + "entries", + ArrowDataType::Struct(vec![key_field, value_field].into()), + false, // entries field itself is not nullable + ); + + Ok(ArrowDataType::Map(Arc::new(entries_field), false)) + } + } +} + +/// Convert Iceberg primitive type to Arrow data type +pub fn iceberg_primitive_to_arrow(primitive: &PrimitiveType) -> Result { + let arrow_type = match primitive { + PrimitiveType::Boolean => ArrowDataType::Boolean, + PrimitiveType::Int => ArrowDataType::Int32, + PrimitiveType::Long => ArrowDataType::Int64, + PrimitiveType::Float => ArrowDataType::Float32, + PrimitiveType::Double => ArrowDataType::Float64, + PrimitiveType::Decimal { precision, scale } => { + ArrowDataType::Decimal128(*precision as u8, *scale as i8) + } + PrimitiveType::Date => ArrowDataType::Date32, + PrimitiveType::Time => ArrowDataType::Time64(TimeUnit::Microsecond), + PrimitiveType::Timestamp => ArrowDataType::Timestamp(TimeUnit::Microsecond, None), + PrimitiveType::Timestamptz => { + ArrowDataType::Timestamp(TimeUnit::Microsecond, Some("UTC".into())) + } + PrimitiveType::TimestampNs => ArrowDataType::Timestamp(TimeUnit::Nanosecond, None), + PrimitiveType::TimestamptzNs => { + ArrowDataType::Timestamp(TimeUnit::Nanosecond, Some("UTC".into())) + } + PrimitiveType::String => ArrowDataType::Utf8, + PrimitiveType::Uuid => ArrowDataType::FixedSizeBinary(16), + PrimitiveType::Fixed(size) => ArrowDataType::FixedSizeBinary(*size as i32), + PrimitiveType::Binary => ArrowDataType::Binary, + }; + + Ok(arrow_type) +} + +/// Convert Iceberg struct type to Arrow struct data type +pub fn iceberg_struct_to_arrow(struct_type: &StructType) -> Result { + let fields = struct_type + .fields() + .iter() + .map(|field| iceberg_field_to_arrow(field)) + .collect::>>()?; + + Ok(ArrowDataType::Struct(fields.into())) +} + +#[cfg(test)] +#[allow(clippy::unwrap_used, clippy::expect_used)] +mod tests { + use std::sync::Arc; + + use super::*; + use crate::spec::{NestedField, PrimitiveType, Schema, Type}; + + #[test] + fn test_primitive_type_conversion() { + let test_cases = vec![ + (PrimitiveType::Boolean, ArrowDataType::Boolean), + (PrimitiveType::Int, ArrowDataType::Int32), + (PrimitiveType::Long, ArrowDataType::Int64), + (PrimitiveType::Float, ArrowDataType::Float32), + (PrimitiveType::Double, ArrowDataType::Float64), + (PrimitiveType::String, ArrowDataType::Utf8), + (PrimitiveType::Binary, ArrowDataType::Binary), + (PrimitiveType::Date, ArrowDataType::Date32), + ( + PrimitiveType::Time, + ArrowDataType::Time64(TimeUnit::Microsecond), + ), + ( + PrimitiveType::Timestamp, + ArrowDataType::Timestamp(TimeUnit::Microsecond, None), + ), + ( + PrimitiveType::Timestamptz, + ArrowDataType::Timestamp(TimeUnit::Microsecond, Some("UTC".into())), + ), + (PrimitiveType::Uuid, ArrowDataType::FixedSizeBinary(16)), + (PrimitiveType::Fixed(10), ArrowDataType::FixedSizeBinary(10)), + ]; + + for (iceberg_type, expected_arrow_type) in test_cases { + let result = iceberg_primitive_to_arrow(&iceberg_type) + .expect("Failed to convert iceberg type to arrow"); + assert_eq!(result, expected_arrow_type); + } + } + + #[test] + fn test_decimal_type_conversion() { + let decimal_type = PrimitiveType::Decimal { + precision: 10, + scale: 2, + }; + let result = iceberg_primitive_to_arrow(&decimal_type) + .expect("Failed to convert decimal type to arrow"); + assert_eq!(result, ArrowDataType::Decimal128(10, 2)); + } + + #[test] + fn test_schema_conversion() { + let schema = Schema::builder() + .with_schema_id(1) + .with_fields(vec![ + Arc::new(NestedField::required( + 1, + "id", + Type::Primitive(PrimitiveType::Long), + )), + Arc::new(NestedField::optional( + 2, + "name", + Type::Primitive(PrimitiveType::String), + )), + Arc::new(NestedField::required( + 3, + "price", + Type::Primitive(PrimitiveType::Decimal { + precision: 10, + scale: 2, + }), + )), + ]) + .build() + .expect("Failed to build schema"); + + let arrow_schema = + iceberg_schema_to_arrow(&schema).expect("Failed to convert schema to arrow"); + + assert_eq!(arrow_schema.fields().len(), 3); + + let id_field = arrow_schema.field(0); + assert_eq!(id_field.name(), "id"); + assert_eq!(id_field.data_type(), &ArrowDataType::Int64); + assert!(!id_field.is_nullable()); + + let name_field = arrow_schema.field(1); + assert_eq!(name_field.name(), "name"); + assert_eq!(name_field.data_type(), &ArrowDataType::Utf8); + assert!(name_field.is_nullable()); + + let price_field = arrow_schema.field(2); + assert_eq!(price_field.name(), "price"); + assert_eq!(price_field.data_type(), &ArrowDataType::Decimal128(10, 2)); + assert!(!price_field.is_nullable()); + } +} diff --git a/crates/sail-iceberg/src/datasource/expr_adapter.rs b/crates/sail-iceberg/src/datasource/expr_adapter.rs new file mode 100644 index 0000000000..0423c749c3 --- /dev/null +++ b/crates/sail-iceberg/src/datasource/expr_adapter.rs @@ -0,0 +1,283 @@ +use std::sync::Arc; + +use datafusion::arrow::compute::can_cast_types; +use datafusion::arrow::datatypes::{DataType, Field, FieldRef, Schema as ArrowSchema, SchemaRef}; +use datafusion::common::tree_node::{Transformed, TransformedResult, TreeNode}; +use datafusion::common::{exec_err, Result, ScalarValue}; +use datafusion::physical_expr::expressions::{Column, Literal}; +use datafusion::physical_expr::PhysicalExpr; +use datafusion::physical_expr_adapter::{PhysicalExprAdapter, PhysicalExprAdapterFactory}; + +#[derive(Debug)] +pub struct IcebergPhysicalExprAdapterFactory {} + +impl PhysicalExprAdapterFactory for IcebergPhysicalExprAdapterFactory { + fn create( + &self, + logical_file_schema: SchemaRef, + physical_file_schema: SchemaRef, + ) -> Arc { + let (column_mapping, default_values) = + create_column_mapping(&logical_file_schema, &physical_file_schema); + + Arc::new(IcebergPhysicalExprAdapter { + logical_file_schema, + physical_file_schema, + partition_values: Vec::new(), + column_mapping, + default_values, + }) + } +} + +fn create_column_mapping( + logical_schema: &ArrowSchema, + physical_schema: &ArrowSchema, +) -> (Vec>, Vec>) { + let mut column_mapping = Vec::with_capacity(logical_schema.fields().len()); + let mut default_values = Vec::with_capacity(logical_schema.fields().len()); + + for logical_field in logical_schema.fields() { + match physical_schema.index_of(logical_field.name()) { + Ok(physical_index) => { + column_mapping.push(Some(physical_index)); + default_values.push(None); + } + Err(_) => { + column_mapping.push(None); + let default_value = if logical_field.is_nullable() { + Some( + ScalarValue::try_from(logical_field.data_type()) + .unwrap_or(ScalarValue::Null), + ) + } else { + Some(create_default_value(logical_field.data_type())) + }; + default_values.push(default_value); + } + } + } + + (column_mapping, default_values) +} + +fn create_default_value(data_type: &DataType) -> ScalarValue { + match data_type { + DataType::Boolean => ScalarValue::Boolean(Some(false)), + DataType::Int8 => ScalarValue::Int8(Some(0)), + DataType::Int16 => ScalarValue::Int16(Some(0)), + DataType::Int32 => ScalarValue::Int32(Some(0)), + DataType::Int64 => ScalarValue::Int64(Some(0)), + DataType::UInt8 => ScalarValue::UInt8(Some(0)), + DataType::UInt16 => ScalarValue::UInt16(Some(0)), + DataType::UInt32 => ScalarValue::UInt32(Some(0)), + DataType::UInt64 => ScalarValue::UInt64(Some(0)), + DataType::Float16 => ScalarValue::Float32(Some(0.0)), + DataType::Float32 => ScalarValue::Float32(Some(0.0)), + DataType::Float64 => ScalarValue::Float64(Some(0.0)), + DataType::Utf8 => ScalarValue::Utf8(Some(String::new())), + DataType::LargeUtf8 => ScalarValue::LargeUtf8(Some(String::new())), + DataType::Binary => ScalarValue::Binary(Some(Vec::new())), + DataType::LargeBinary => ScalarValue::LargeBinary(Some(Vec::new())), + DataType::Date32 => ScalarValue::Date32(Some(0)), + DataType::Date64 => ScalarValue::Date64(Some(0)), + DataType::Time32(_) => ScalarValue::Time32Second(Some(0)), + DataType::Time64(_) => ScalarValue::Time64Nanosecond(Some(0)), + DataType::Timestamp(unit, tz) => match unit { + datafusion::arrow::datatypes::TimeUnit::Second => { + ScalarValue::TimestampSecond(Some(0), tz.clone()) + } + datafusion::arrow::datatypes::TimeUnit::Millisecond => { + ScalarValue::TimestampMillisecond(Some(0), tz.clone()) + } + datafusion::arrow::datatypes::TimeUnit::Microsecond => { + ScalarValue::TimestampMicrosecond(Some(0), tz.clone()) + } + datafusion::arrow::datatypes::TimeUnit::Nanosecond => { + ScalarValue::TimestampNanosecond(Some(0), tz.clone()) + } + }, + _ => ScalarValue::Null, + } +} + +#[derive(Debug)] +struct IcebergPhysicalExprAdapter { + logical_file_schema: SchemaRef, + physical_file_schema: SchemaRef, + partition_values: Vec<(FieldRef, ScalarValue)>, + column_mapping: Vec>, + default_values: Vec>, +} + +impl PhysicalExprAdapter for IcebergPhysicalExprAdapter { + fn rewrite(&self, expr: Arc) -> Result> { + let rewriter = IcebergPhysicalExprRewriter { + logical_file_schema: &self.logical_file_schema, + physical_file_schema: &self.physical_file_schema, + partition_values: &self.partition_values, + column_mapping: &self.column_mapping, + default_values: &self.default_values, + }; + expr.transform(|expr| rewriter.rewrite_expr(Arc::clone(&expr))) + .data() + } + + fn with_partition_values( + &self, + partition_values: Vec<(FieldRef, ScalarValue)>, + ) -> Arc { + Arc::new(IcebergPhysicalExprAdapter { + logical_file_schema: Arc::clone(&self.logical_file_schema), + physical_file_schema: Arc::clone(&self.physical_file_schema), + partition_values, + column_mapping: self.column_mapping.clone(), + default_values: self.default_values.clone(), + }) + } +} + +impl Clone for IcebergPhysicalExprAdapter { + fn clone(&self) -> Self { + Self { + logical_file_schema: Arc::clone(&self.logical_file_schema), + physical_file_schema: Arc::clone(&self.physical_file_schema), + partition_values: self.partition_values.clone(), + column_mapping: self.column_mapping.clone(), + default_values: self.default_values.clone(), + } + } +} + +struct IcebergPhysicalExprRewriter<'a> { + logical_file_schema: &'a ArrowSchema, + physical_file_schema: &'a ArrowSchema, + partition_values: &'a [(FieldRef, ScalarValue)], + column_mapping: &'a [Option], + default_values: &'a [Option], +} + +impl<'a> IcebergPhysicalExprRewriter<'a> { + fn rewrite_expr( + &self, + expr: Arc, + ) -> Result>> { + if let Some(column) = expr.as_any().downcast_ref::() { + return self.rewrite_column(Arc::clone(&expr), column); + } + Ok(Transformed::no(expr)) + } + + fn rewrite_column( + &self, + expr: Arc, + column: &Column, + ) -> Result>> { + if let Some(partition_value) = self.get_partition_value(column.name()) { + return Ok(Transformed::yes(Arc::new(Literal::new(partition_value)))); + } + + let logical_field_index = match self.logical_file_schema.index_of(column.name()) { + Ok(index) => index, + Err(_) => { + if let Ok(_physical_field) = + self.physical_file_schema.field_with_name(column.name()) + { + return Ok(Transformed::no(expr)); + } else { + return exec_err!( + "Column '{}' not found in either logical or physical schema", + column.name() + ); + } + } + }; + + let logical_field = self.logical_file_schema.field(logical_field_index); + + match self.column_mapping.get(logical_field_index) { + Some(Some(physical_index)) => { + let physical_field = self.physical_file_schema.field(*physical_index); + self.handle_existing_column( + expr, + column, + logical_field, + physical_field, + *physical_index, + ) + } + Some(None) => { + if let Some(Some(default_value)) = self.default_values.get(logical_field_index) { + Ok(Transformed::yes(Arc::new(Literal::new( + default_value.clone(), + )))) + } else if logical_field.is_nullable() { + let null_value = ScalarValue::Null.cast_to(logical_field.data_type())?; + Ok(Transformed::yes(Arc::new(Literal::new(null_value)))) + } else { + exec_err!("Non-nullable column '{}' is missing from physical schema and no default value provided", column.name()) + } + } + None => exec_err!( + "Column mapping not found for logical field index {}", + logical_field_index + ), + } + } + + fn handle_existing_column( + &self, + expr: Arc, + column: &Column, + logical_field: &Field, + physical_field: &Field, + physical_index: usize, + ) -> Result>> { + let needs_index_update = column.index() != physical_index; + let needs_type_cast = logical_field.data_type() != physical_field.data_type(); + + match (needs_index_update, needs_type_cast) { + (false, false) => Ok(Transformed::no(expr)), + (true, false) => { + let new_column = + Column::new_with_schema(logical_field.name(), self.physical_file_schema)?; + Ok(Transformed::yes(Arc::new(new_column))) + } + (false, true) => self.apply_type_cast(expr, logical_field, physical_field), + (true, true) => { + let new_column = + Column::new_with_schema(logical_field.name(), self.physical_file_schema)?; + self.apply_type_cast(Arc::new(new_column), logical_field, physical_field) + } + } + } + + fn apply_type_cast( + &self, + column_expr: Arc, + logical_field: &Field, + physical_field: &Field, + ) -> Result>> { + if !can_cast_types(physical_field.data_type(), logical_field.data_type()) { + return exec_err!( + "Cannot cast column '{}' from '{}' (physical) to '{}' (logical)", + logical_field.name(), + physical_field.data_type(), + logical_field.data_type() + ); + } + let cast_expr = datafusion::physical_expr::expressions::CastExpr::new( + column_expr, + logical_field.data_type().clone(), + None, + ); + Ok(Transformed::yes(Arc::new(cast_expr))) + } + + fn get_partition_value(&self, column_name: &str) -> Option { + self.partition_values + .iter() + .find(|(field, _)| field.name() == column_name) + .map(|(_, value)| value.clone()) + } +} diff --git a/crates/sail-iceberg/src/datasource/expressions.rs b/crates/sail-iceberg/src/datasource/expressions.rs new file mode 100644 index 0000000000..e342b86005 --- /dev/null +++ b/crates/sail-iceberg/src/datasource/expressions.rs @@ -0,0 +1,50 @@ +use std::sync::Arc; + +use datafusion::catalog::Session; +use datafusion::common::DFSchema; +use datafusion::logical_expr::execution_props::ExecutionProps; +use datafusion::logical_expr::simplify::SimplifyContext; +use datafusion::logical_expr::{Expr, TableProviderFilterPushDown}; +use datafusion::optimizer::simplify_expressions::ExprSimplifier; +use datafusion::physical_expr::PhysicalExpr; + +pub fn simplify_expr( + session: &dyn Session, + df_schema: &DFSchema, + expr: Expr, +) -> Arc { + let props = ExecutionProps::new(); + let simplify_context = SimplifyContext::new(&props).with_schema(df_schema.clone().into()); + let simplifier = ExprSimplifier::new(simplify_context).with_max_cycles(10); + #[allow(clippy::expect_used)] + let simplified = simplifier + .simplify(expr) + .expect("Failed to simplify expression"); + #[allow(clippy::expect_used)] + session + .create_physical_expr(simplified, df_schema) + .expect("Failed to create physical expression") +} + +pub fn get_pushdown_filters( + filter: &[&Expr], + _partition_cols: &[String], +) -> Vec { + filter + .iter() + .map(|expr| match expr { + Expr::BinaryExpr(be) => match be.op { + datafusion::logical_expr::Operator::Eq + | datafusion::logical_expr::Operator::Lt + | datafusion::logical_expr::Operator::LtEq + | datafusion::logical_expr::Operator::Gt + | datafusion::logical_expr::Operator::GtEq + | datafusion::logical_expr::Operator::And + | datafusion::logical_expr::Operator::Or => TableProviderFilterPushDown::Inexact, + _ => TableProviderFilterPushDown::Unsupported, + }, + Expr::InList(_) => TableProviderFilterPushDown::Inexact, + _ => TableProviderFilterPushDown::Unsupported, + }) + .collect() +} diff --git a/crates/sail-iceberg/src/datasource/mod.rs b/crates/sail-iceberg/src/datasource/mod.rs new file mode 100644 index 0000000000..4508e8a8d2 --- /dev/null +++ b/crates/sail-iceberg/src/datasource/mod.rs @@ -0,0 +1,43 @@ +pub mod expr_adapter; +pub mod expressions; +pub mod provider; +pub mod pruning; + +use datafusion::common::scalar::ScalarValue; +pub use provider::*; + +use crate::spec::types::values::{Literal, PrimitiveLiteral}; + +pub(crate) fn literal_to_scalar_value(literal: &Literal) -> ScalarValue { + match literal { + Literal::Primitive(primitive) => match primitive { + PrimitiveLiteral::Boolean(v) => ScalarValue::Boolean(Some(*v)), + PrimitiveLiteral::Int(v) => ScalarValue::Int32(Some(*v)), + PrimitiveLiteral::Long(v) => ScalarValue::Int64(Some(*v)), + PrimitiveLiteral::Float(v) => ScalarValue::Float32(Some(v.into_inner())), + PrimitiveLiteral::Double(v) => ScalarValue::Float64(Some(v.into_inner())), + PrimitiveLiteral::String(v) => ScalarValue::Utf8(Some(v.clone())), + PrimitiveLiteral::Binary(v) => ScalarValue::Binary(Some(v.clone())), + PrimitiveLiteral::Int128(v) => ScalarValue::Decimal128(Some(*v), 38, 0), + PrimitiveLiteral::UInt128(v) => { + if *v <= i128::MAX as u128 { + ScalarValue::Decimal128(Some(*v as i128), 38, 0) + } else { + ScalarValue::Utf8(Some(v.to_string())) + } + } + }, + Literal::Struct(fields) => { + let json_repr = serde_json::to_string(fields).unwrap_or_default(); + ScalarValue::Utf8(Some(json_repr)) + } + Literal::List(items) => { + let json_repr = serde_json::to_string(items).unwrap_or_default(); + ScalarValue::Utf8(Some(json_repr)) + } + Literal::Map(pairs) => { + let json_repr = serde_json::to_string(pairs).unwrap_or_default(); + ScalarValue::Utf8(Some(json_repr)) + } + } +} diff --git a/crates/sail-iceberg/src/datasource/provider.rs b/crates/sail-iceberg/src/datasource/provider.rs new file mode 100644 index 0000000000..d705cab0c7 --- /dev/null +++ b/crates/sail-iceberg/src/datasource/provider.rs @@ -0,0 +1,852 @@ +use std::any::Any; +use std::borrow::Cow; +use std::collections::HashMap; +use std::sync::Arc; + +use arrow_schema::Schema as ArrowSchema; +use async_trait::async_trait; +use datafusion::catalog::memory::DataSourceExec; +use datafusion::catalog::Session; +use datafusion::common::scalar::ScalarValue; +use datafusion::common::stats::{ColumnStatistics, Precision, Statistics}; +use datafusion::common::{Result as DataFusionResult, ToDFSchema}; +use datafusion::config::TableParquetOptions; +use datafusion::datasource::listing::PartitionedFile; +use datafusion::datasource::physical_plan::{FileGroup, FileScanConfigBuilder, ParquetSource}; +use datafusion::datasource::{TableProvider, TableType}; +use datafusion::execution::object_store::ObjectStoreUrl; +use datafusion::logical_expr::utils::conjunction; +use datafusion::logical_expr::{ + BinaryExpr, Expr, LogicalPlan, Operator, TableProviderFilterPushDown, +}; +use datafusion::physical_expr::PhysicalExpr; +use datafusion::physical_expr_adapter::PhysicalExprAdapterFactory; +use datafusion::physical_plan::ExecutionPlan; +use object_store::path::Path as ObjectPath; +use object_store::ObjectMeta; +use url::Url; + +use crate::arrow_conversion::iceberg_schema_to_arrow; +use crate::datasource::expr_adapter::IcebergPhysicalExprAdapterFactory; +use crate::datasource::expressions::simplify_expr; +use crate::datasource::literal_to_scalar_value; +use crate::datasource::pruning::{prune_files, prune_manifests_by_partition_summaries}; +use crate::spec::manifest::DataContentType; +use crate::spec::types::values::Literal; +use crate::spec::{ + DataFile, FormatVersion, Manifest, ManifestContentType, ManifestList, ManifestStatus, + PartitionSpec, Schema, Snapshot, +}; + +#[derive(Debug, Clone)] +struct IcebergDeleteAttachment { + eq_delete_count: usize, + pos_delete_count: usize, +} + +/// Iceberg table provider for DataFusion +#[derive(Debug)] +pub struct IcebergTableProvider { + /// The table location (URI) + table_uri: String, + /// The current schema of the table + schema: Schema, + /// The current snapshot of the table + snapshot: Snapshot, + /// All partition specs referenced by the table + #[allow(unused)] + partition_specs: Vec, + /// Arrow schema for DataFusion + arrow_schema: Arc, +} + +impl IcebergTableProvider { + /// Create a new Iceberg table provider + pub fn new( + table_uri: impl ToString, + schema: Schema, + snapshot: Snapshot, + partition_specs: Vec, + ) -> DataFusionResult { + let table_uri_str = table_uri.to_string(); + log::trace!("Creating table provider for: {}", table_uri_str); + + let arrow_schema = Arc::new(iceberg_schema_to_arrow(&schema).map_err(|e| { + log::trace!("Failed to convert schema to Arrow: {:?}", e); + e + })?); + + log::trace!( + "Converted schema to Arrow with {} fields", + arrow_schema.fields().len() + ); + + Ok(Self { + table_uri: table_uri_str, + schema, + snapshot, + partition_specs, + arrow_schema, + }) + } + + /// Get the table URI + pub fn table_uri(&self) -> &str { + &self.table_uri + } + + /// Get the Iceberg schema + pub fn iceberg_schema(&self) -> &Schema { + &self.schema + } + + /// Get the current snapshot + pub fn current_snapshot(&self) -> &Snapshot { + &self.snapshot + } + + /// Get object store from DataFusion session + fn get_object_store( + &self, + session: &dyn Session, + ) -> DataFusionResult> { + let table_url = Url::parse(&self.table_uri) + .map_err(|e| datafusion::common::DataFusionError::External(Box::new(e)))?; + + session + .runtime_env() + .object_store_registry + .get_store(&table_url) + .map_err(|e| datafusion::common::DataFusionError::External(Box::new(e))) + } + + /// Load manifest list from snapshot + async fn load_manifest_list( + &self, + object_store: &Arc, + ) -> DataFusionResult { + let manifest_list_str = self.snapshot.manifest_list(); + log::trace!("Manifest list path: {}", manifest_list_str); + + let manifest_list_path = if let Ok(url) = Url::parse(manifest_list_str) { + log::trace!("Parsed manifest list as URL, path: {}", url.path()); + ObjectPath::from(url.path()) + } else { + ObjectPath::from(manifest_list_str) + }; + + let manifest_list_data = object_store + .get(&manifest_list_path) + .await + .map_err(|e| { + log::trace!("Failed to get manifest list: {:?}", e); + datafusion::common::DataFusionError::External(Box::new(e)) + })? + .bytes() + .await + .map_err(|e| datafusion::common::DataFusionError::External(Box::new(e)))?; + + log::trace!("Read {} bytes from manifest list", manifest_list_data.len()); + + ManifestList::parse_with_version(&manifest_list_data, FormatVersion::V2) + .map_err(datafusion::common::DataFusionError::Execution) + } + + /// Load data files from manifests + async fn load_data_files( + &self, + session: &dyn Session, + filters: &[Expr], + object_store: &Arc, + manifest_list: &ManifestList, + ) -> DataFusionResult> { + let mut data_files = Vec::new(); + + let spec_map: HashMap = self + .partition_specs + .iter() + .map(|s| (s.spec_id(), s.clone())) + .collect(); + let manifest_files = + prune_manifests_by_partition_summaries(manifest_list, &self.schema, &spec_map, filters); + + for manifest_file in manifest_files { + // TODO: Add support for delete manifests + if manifest_file.content != ManifestContentType::Data { + continue; + } + + let manifest_path_str = manifest_file.manifest_path.as_str(); + log::trace!("Loading manifest: {}", manifest_path_str); + + let manifest_path = if let Ok(url) = Url::parse(manifest_path_str) { + log::trace!("Parsed manifest as URL, path: {}", url.path()); + ObjectPath::from(url.path()) + } else { + ObjectPath::from(manifest_path_str) + }; + + let manifest_data = object_store + .get(&manifest_path) + .await + .map_err(|e| { + log::trace!("Failed to get manifest: {:?}", e); + datafusion::common::DataFusionError::External(Box::new(e)) + })? + .bytes() + .await + .map_err(|e| datafusion::common::DataFusionError::External(Box::new(e)))?; + + log::trace!("Read {} bytes from manifest", manifest_data.len()); + + let manifest = Manifest::parse_avro(&manifest_data) + .map_err(datafusion::common::DataFusionError::Execution)?; + + // Get partition_spec_id from manifest file + let partition_spec_id = manifest_file.partition_spec_id; + + // Collect data files for this manifest + let manifest_data_files: Vec = manifest + .entries() + .iter() + .filter_map(|entry_ref| { + let entry = entry_ref.as_ref(); + if matches!( + entry.status, + ManifestStatus::Added | ManifestStatus::Existing + ) { + let mut df = entry.data_file.clone(); + df.partition_spec_id = partition_spec_id; + Some(df) + } else { + None + } + }) + .collect(); + + // Early prune at manifest entry level using DataFusion predicate over metrics + if !filters.is_empty() { + let (kept, _mask) = crate::datasource::pruning::prune_files( + session, + filters, + None, + self.arrow_schema.clone(), + manifest_data_files, + &self.schema, + )?; + data_files.extend(kept); + } else { + data_files.extend(manifest_data_files); + } + } + + Ok(data_files) + } + + fn partition_key_for(&self, partition: &[Option]) -> String { + serde_json::to_string(partition).unwrap_or_default() + } + + async fn load_delete_index( + &self, + object_store: &Arc, + manifest_list: &ManifestList, + ) -> DataFusionResult> { + let mut index: std::collections::HashMap = + std::collections::HashMap::new(); + + for manifest_file in manifest_list + .entries() + .iter() + .filter(|mf| mf.content == ManifestContentType::Deletes) + { + let manifest_path_str = manifest_file.manifest_path.as_str(); + let manifest_path = if let Ok(url) = Url::parse(manifest_path_str) { + ObjectPath::from(url.path()) + } else { + ObjectPath::from(manifest_path_str) + }; + + let manifest_data = object_store + .get(&manifest_path) + .await + .map_err(|e| datafusion::common::DataFusionError::External(Box::new(e)))? + .bytes() + .await + .map_err(|e| datafusion::common::DataFusionError::External(Box::new(e)))?; + + let manifest = Manifest::parse_avro(&manifest_data) + .map_err(datafusion::common::DataFusionError::Execution)?; + + for entry_ref in manifest.entries().iter() { + let entry = entry_ref.as_ref(); + if !matches!( + entry.status, + ManifestStatus::Added | ManifestStatus::Existing + ) { + continue; + } + let df = &entry.data_file; + let key = self.partition_key_for(df.partition()); + let att = index.entry(key).or_insert(IcebergDeleteAttachment { + eq_delete_count: 0, + pos_delete_count: 0, + }); + match df.content_type() { + DataContentType::EqualityDeletes => { + att.eq_delete_count = att.eq_delete_count.saturating_add(1); + } + DataContentType::PositionDeletes => { + att.pos_delete_count = att.pos_delete_count.saturating_add(1); + } + _ => {} + } + } + } + + Ok(index) + } + + // Attention: take care of percent-encoding when resolving the data file object path + fn resolve_data_file_object_path(&self, table_base_path: &str, raw_path: &str) -> ObjectPath { + // If the data file path is a full URL, strip scheme/authority and use its path + if let Ok(url) = Url::parse(raw_path) { + let encoded_path = url.path(); + let path_no_leading = encoded_path.strip_prefix('/').unwrap_or(encoded_path); + if let Ok(p) = ObjectPath::parse(path_no_leading) { + return p; + } + return ObjectPath::from(path_no_leading); + } + + // If the data file path already starts with the table base path, use it as-is + if raw_path.starts_with(table_base_path) { + return ObjectPath::from(raw_path); + } + + // If it is an absolute filesystem path, use it directly + if raw_path.starts_with(object_store::path::DELIMITER) { + return ObjectPath::from(raw_path); + } + + // Otherwise, treat as a path relative to the table base path (preserve encoding) + let joined = format!( + "{}{}{}", + table_base_path, + object_store::path::DELIMITER, + raw_path + ); + ObjectPath::from(joined) + } + + /// Create partitioned files for DataFusion from Iceberg data files + fn create_partitioned_files( + &self, + data_files: Vec, + delete_index: &std::collections::HashMap, + ) -> DataFusionResult> { + let mut partitioned_files = Vec::new(); + + let table_url = Url::parse(&self.table_uri) + .map_err(|e| datafusion::common::DataFusionError::External(Box::new(e)))?; + let table_base_path = table_url.path(); + + for data_file in data_files { + let raw_path = data_file.file_path(); + let file_path = self.resolve_data_file_object_path(table_base_path, raw_path); + log::trace!("Processing data file: {}", file_path); + + log::trace!("Final ObjectPath: {}", file_path); + + let object_meta = ObjectMeta { + location: file_path, + last_modified: chrono::Utc::now(), + size: data_file.file_size_in_bytes(), + e_tag: None, + version: None, + }; + + // Convert partition values to ScalarValues + let partition_values = data_file + .partition() + .iter() + .map(|literal_opt| match literal_opt { + Some(literal) => literal_to_scalar_value(literal), + None => ScalarValue::Null, + }) + .collect(); + + let key = self.partition_key_for(data_file.partition()); + let extensions: Option> = delete_index + .get(&key) + .map(|att| Arc::new(att.clone()) as Arc); + + let partitioned_file = PartitionedFile { + object_meta, + partition_values, + range: None, + statistics: Some(Arc::new(self.create_file_statistics(&data_file))), + extensions, + metadata_size_hint: None, + }; + + partitioned_files.push(partitioned_file); + } + + Ok(partitioned_files) + } + + /// Create file groups from partitioned files + fn create_file_groups(&self, partitioned_files: Vec) -> Vec { + // Group files by partition values + let mut file_groups: HashMap, Vec> = HashMap::new(); + + for file in partitioned_files { + file_groups + .entry(file.partition_values.clone()) + .or_default() + .push(file); + } + + file_groups.into_values().map(FileGroup::from).collect() + } + + /// Aggregate table-level statistics from a list of Iceberg data files + fn aggregate_statistics(&self, data_files: &[DataFile]) -> Statistics { + if data_files.is_empty() { + return Statistics::new_unknown(&self.arrow_schema); + } + + let mut total_rows: usize = 0; + let mut total_bytes: usize = 0; + + // Pre-compute field id per column index + let field_ids: Vec = self.schema.fields().iter().map(|f| f.id).collect(); + + // Initialize accumulators per column + let mut min_scalars: Vec> = + vec![None; self.arrow_schema.fields().len()]; + let mut max_scalars: Vec> = + vec![None; self.arrow_schema.fields().len()]; + let mut null_counts: Vec = vec![0; self.arrow_schema.fields().len()]; + + for df in data_files { + total_rows = total_rows.saturating_add(df.record_count() as usize); + total_bytes = total_bytes.saturating_add(df.file_size_in_bytes() as usize); + + for (col_idx, field_id) in field_ids.iter().enumerate() { + // null counts + if let Some(c) = df.null_value_counts().get(field_id) { + null_counts[col_idx] = null_counts[col_idx].saturating_add(*c as usize); + } + + // min + if let Some(d) = df.lower_bounds().get(field_id) { + let v = Literal::Primitive(d.literal.clone()); + let sv = literal_to_scalar_value(&v); + min_scalars[col_idx] = match (&min_scalars[col_idx], &sv) { + (None, s) => Some(s.clone()), + (Some(existing), s) => Some(if s < existing { + s.clone() + } else { + existing.clone() + }), + }; + } + + // max + if let Some(d) = df.upper_bounds().get(field_id) { + let v = Literal::Primitive(d.literal.clone()); + let sv = literal_to_scalar_value(&v); + max_scalars[col_idx] = match (&max_scalars[col_idx], &sv) { + (None, s) => Some(s.clone()), + (Some(existing), s) => Some(if s > existing { + s.clone() + } else { + existing.clone() + }), + }; + } + } + } + + let column_statistics = (0..self.arrow_schema.fields().len()) + .map(|i| ColumnStatistics { + null_count: Precision::Exact(null_counts[i]), + max_value: max_scalars[i] + .clone() + .map(Precision::Exact) + .unwrap_or(Precision::Absent), + min_value: min_scalars[i] + .clone() + .map(Precision::Exact) + .unwrap_or(Precision::Absent), + distinct_count: Precision::Absent, + sum_value: Precision::Absent, + }) + .collect(); + + Statistics { + num_rows: Precision::Exact(total_rows), + total_byte_size: Precision::Exact(total_bytes), + column_statistics, + } + } + + /// Create file statistics from Iceberg data file metadata + fn create_file_statistics(&self, data_file: &DataFile) -> Statistics { + let num_rows = Precision::Exact(data_file.record_count() as usize); + let total_byte_size = Precision::Exact(data_file.file_size_in_bytes() as usize); + + // Create column statistics from Iceberg metadata + let column_statistics = self + .arrow_schema + .fields() + .iter() + .enumerate() + .map(|(i, _field)| { + let field_id = self + .schema + .fields() + .get(i) + .map(|f| f.id) + .unwrap_or(i as i32 + 1); + + let null_count = data_file + .null_value_counts() + .get(&field_id) + .map(|&count| Precision::Exact(count as usize)) + .unwrap_or(Precision::Absent); + + let distinct_count = Precision::Absent; + + let min_value = data_file + .lower_bounds() + .get(&field_id) + .map(|datum| { + // convert Datum -> Literal for existing scalar conversion + let lit = Literal::Primitive(datum.literal.clone()); + literal_to_scalar_value(&lit) + }) + .map(Precision::Exact) + .unwrap_or(Precision::Absent); + + let max_value = data_file + .upper_bounds() + .get(&field_id) + .map(|datum| { + let lit = Literal::Primitive(datum.literal.clone()); + literal_to_scalar_value(&lit) + }) + .map(Precision::Exact) + .unwrap_or(Precision::Absent); + + ColumnStatistics { + null_count, + max_value, + min_value, + distinct_count, + sum_value: Precision::Absent, + } + }) + .collect(); + + Statistics { + num_rows, + total_byte_size, + column_statistics, + } + } +} + +#[async_trait] +impl TableProvider for IcebergTableProvider { + fn as_any(&self) -> &dyn Any { + self + } + + fn schema(&self) -> Arc { + self.arrow_schema.clone() + } + + fn table_type(&self) -> TableType { + TableType::Base + } + + fn get_table_definition(&self) -> Option<&str> { + None + } + + fn get_logical_plan(&self) -> Option> { + None + } + + async fn scan( + &self, + session: &dyn Session, + projection: Option<&Vec>, + filters: &[Expr], + limit: Option, + ) -> DataFusionResult> { + log::trace!("Starting scan for table: {}", self.table_uri); + + let object_store = self.get_object_store(session)?; + log::trace!("Got object store"); + + log::trace!( + "Loading manifest list from: {}", + self.snapshot.manifest_list() + ); + let manifest_list = self.load_manifest_list(&object_store).await?; + log::trace!("Loaded {} manifest files", manifest_list.entries().len()); + + // Classify & split filters for pruning vs parquet pushdown + let (pruning_filters, parquet_pushdown_filters) = self.separate_filters(filters); + + log::trace!("Loading data files from manifests..."); + let mut data_files = self + .load_data_files(session, &pruning_filters, &object_store, &manifest_list) + .await?; + log::trace!("Loaded {} data files", data_files.len()); + + // Build filter conjunction and run DataFusion-based pruning on Iceberg metrics + let filter_expr = conjunction(pruning_filters.iter().cloned()); + let mut _pruning_mask: Option> = None; + if filter_expr.is_some() || limit.is_some() { + let (kept, mask) = prune_files( + session, + &pruning_filters, + limit, + self.rebuild_logical_schema_for_filters(projection, filters), + data_files, + &self.schema, + )?; + _pruning_mask = mask; + data_files = kept; + log::trace!("Pruned data files, remaining: {}", data_files.len()); + } + + log::trace!("Loading delete manifests..."); + let delete_index = self + .load_delete_index(&object_store, &manifest_list) + .await?; + + log::trace!("Creating partitioned files..."); + let partitioned_files = self.create_partitioned_files(data_files.clone(), &delete_index)?; + log::trace!("Created {} partitioned files", partitioned_files.len()); + + // Step 4: Create file groups + let file_groups = self.create_file_groups(partitioned_files); + + // Step 5: Create file scan configuration + let file_schema = self.arrow_schema.clone(); + let table_url = Url::parse(&self.table_uri) + .map_err(|e| datafusion::common::DataFusionError::External(Box::new(e)))?; + + let base_url = format!("{}://{}", table_url.scheme(), table_url.authority()); + let base_url_parsed = Url::parse(&base_url) + .map_err(|e| datafusion::common::DataFusionError::External(Box::new(e)))?; + let object_store_url = ObjectStoreUrl::parse(base_url_parsed) + .map_err(|e| datafusion::common::DataFusionError::External(Box::new(e)))?; + + let parquet_options = TableParquetOptions { + global: session.config().options().execution.parquet.clone(), + ..Default::default() + }; + + let mut parquet_source = ParquetSource::new(parquet_options); + // Prepare pushdown filter for Parquet + let pushdown_filter: Option> = if !parquet_pushdown_filters.is_empty() + { + let logical_schema = self.rebuild_logical_schema_for_filters(projection, filters); + let df_schema = logical_schema.to_dfschema()?; + let pushdown_expr = conjunction(parquet_pushdown_filters); + pushdown_expr.map(|expr| simplify_expr(session, &df_schema, expr)) + } else { + None + }; + if let Some(pred) = pushdown_filter { + // TODO: Consider expression adapter for Parquet pushdown + parquet_source = parquet_source.with_predicate(pred); + } + let parquet_source = Arc::new(parquet_source); + + // Build table statistics from pruned files + let table_stats = self.aggregate_statistics(&data_files); + + let expanded_projection: Option> = if let Some(used) = projection { + let mut cols: Vec = used.clone(); + if let Some(expr) = conjunction(filters.iter().cloned()) { + for c in expr.column_refs() { + if let Ok(idx) = self.arrow_schema.index_of(c.name.as_str()) { + if !cols.contains(&idx) { + cols.push(idx); + } + } + } + } + Some(cols) + } else { + None + }; + + let file_scan_config = + FileScanConfigBuilder::new(object_store_url, file_schema, parquet_source) + .with_file_groups(if file_groups.is_empty() { + vec![FileGroup::from(vec![])] + } else { + file_groups + }) + .with_statistics(table_stats) + .with_projection(expanded_projection) + .with_limit(limit) + .with_expr_adapter(Some(Arc::new(IcebergPhysicalExprAdapterFactory {}) + as Arc)) + .build(); + + Ok(DataSourceExec::from_data_source(file_scan_config)) + } + + fn supports_filters_pushdown( + &self, + filter: &[&Expr], + ) -> DataFusionResult> { + Ok(filter + .iter() + .map(|e| self.classify_pushdown_for_expr(e)) + .collect()) + } +} + +impl IcebergTableProvider { + fn classify_pushdown_for_expr(&self, expr: &Expr) -> TableProviderFilterPushDown { + use TableProviderFilterPushDown as FP; + // Identity partition columns get Exact (Eq/IN) or Inexact (ranges) + match expr { + Expr::BinaryExpr(BinaryExpr { left, op, right }) => { + let (l, r) = (Self::strip_expr(left), Self::strip_expr(right)); + match op { + Operator::Eq => { + if let (Some(col), true) = + (self.expr_as_column_name(l), self.expr_is_literal(r)) + { + if self.is_identity_partition_col(&col) { + return FP::Exact; + } + } + if let (Some(col), true) = + (self.expr_as_column_name(r), self.expr_is_literal(l)) + { + if self.is_identity_partition_col(&col) { + return FP::Exact; + } + } + FP::Unsupported + } + Operator::Gt | Operator::GtEq | Operator::Lt | Operator::LtEq => { + if let Some(col) = self.expr_as_column_name(l) { + if self.expr_is_literal(r) && self.is_identity_partition_col(&col) { + return FP::Inexact; + } + } + FP::Unsupported + } + _ => FP::Unsupported, + } + } + Expr::InList(in_list) if !in_list.negated => { + let e = Self::strip_expr(&in_list.expr); + if let Some(col) = self.expr_as_column_name(e) { + let all_literals = in_list.list.iter().all(|it| self.expr_is_literal(it)); + if all_literals && self.is_identity_partition_col(&col) { + TableProviderFilterPushDown::Exact + } else { + TableProviderFilterPushDown::Unsupported + } + } else { + TableProviderFilterPushDown::Unsupported + } + } + _ => TableProviderFilterPushDown::Unsupported, + } + } + + fn separate_filters(&self, filters: &[Expr]) -> (Vec, Vec) { + let mut pruning_filters = Vec::new(); + let mut parquet_pushdown_filters = Vec::new(); + for f in filters.iter() { + match self.classify_pushdown_for_expr(f) { + TableProviderFilterPushDown::Exact => { + pruning_filters.push(f.clone()); + } + TableProviderFilterPushDown::Inexact => { + pruning_filters.push(f.clone()); + parquet_pushdown_filters.push(f.clone()); + } + TableProviderFilterPushDown::Unsupported => {} + } + } + (pruning_filters, parquet_pushdown_filters) + } + + fn strip_expr(expr: &Expr) -> &Expr { + match expr { + Expr::Cast(c) => Self::strip_expr(&c.expr), + Expr::Alias(a) => Self::strip_expr(&a.expr), + _ => expr, + } + } + + fn expr_as_column_name(&self, expr: &Expr) -> Option { + if let Expr::Column(c) = expr { + return Some(c.name.clone()); + } + None + } + + fn expr_is_literal(&self, expr: &Expr) -> bool { + matches!(expr, Expr::Literal(_, _)) + } + + fn is_identity_partition_col(&self, col_name: &str) -> bool { + // Map identity partition source_id to schema field names + let mut names = std::collections::HashSet::new(); + for spec in &self.partition_specs { + for pf in spec.fields().iter() { + if matches!(pf.transform, crate::spec::transform::Transform::Identity) { + if let Some(field) = self.schema.field_by_id(pf.source_id) { + names.insert(field.name.clone()); + } + } + } + } + names.contains(col_name) + } + + fn rebuild_logical_schema_for_filters( + &self, + projection: Option<&Vec>, + filters: &[Expr], + ) -> Arc { + if let Some(used) = projection { + let mut fields: Vec = Vec::new(); + for idx in used { + fields.push(Arc::new(self.arrow_schema.field(*idx).clone())); + } + if let Some(expr) = conjunction(filters.iter().cloned()) { + for c in expr.column_refs() { + if let Ok(idx) = self.arrow_schema.index_of(c.name.as_str()) { + if !used.contains(&idx) + && !fields.iter().any(|f| f.name() == c.name.as_str()) + { + fields.push(Arc::new(self.arrow_schema.field(idx).clone())); + } + } + } + } + Arc::new(ArrowSchema::new(fields)) + } else { + self.arrow_schema.clone() + } + } +} diff --git a/crates/sail-iceberg/src/datasource/pruning.rs b/crates/sail-iceberg/src/datasource/pruning.rs new file mode 100644 index 0000000000..a4a012f018 --- /dev/null +++ b/crates/sail-iceberg/src/datasource/pruning.rs @@ -0,0 +1,746 @@ +use std::cell::RefCell; +use std::collections::HashMap; +use std::sync::Arc; + +use datafusion::arrow::array::{ArrayRef, BooleanArray, UInt64Array}; +use datafusion::arrow::datatypes::Schema as ArrowSchema; +use datafusion::catalog::Session; +use datafusion::common::pruning::PruningStatistics; +use datafusion::common::{Column, Result, ToDFSchema}; +use datafusion::logical_expr::utils::conjunction; +use datafusion::logical_expr::{BinaryExpr, Expr, Operator}; +use datafusion::physical_optimizer::pruning::PruningPredicate; + +use crate::datasource::literal_to_scalar_value; +use crate::spec::partition::PartitionSpec; +use crate::spec::types::values::{Datum, Literal}; +use crate::spec::types::{PrimitiveType, Type}; +use crate::spec::{DataFile, Manifest, ManifestContentType, ManifestList, Schema}; +// TODO: Implement robust logical expression parsing for summary pruning + +/// Pruning statistics over Iceberg DataFiles +pub struct IcebergPruningStats { + files: Vec, + #[allow(unused)] + arrow_schema: Arc, + /// Arrow field name -> Iceberg field id + name_to_field_id: HashMap, + /// Iceberg field id -> Iceberg primitive type (for proper ScalarValue typing) + field_id_to_type: HashMap, + min_cache: RefCell>, + max_cache: RefCell>, + nulls_cache: RefCell>, + rows_cache: RefCell>, +} + +impl IcebergPruningStats { + pub fn new( + files: Vec, + arrow_schema: Arc, + iceberg_schema: &Schema, + ) -> Self { + let mut name_to_field_id = HashMap::new(); + let mut field_id_to_type = HashMap::new(); + for f in iceberg_schema.fields().iter() { + name_to_field_id.insert(f.name.clone(), f.id); + if let crate::spec::types::Type::Primitive(p) = f.field_type.as_ref() { + field_id_to_type.insert(f.id, p.clone()); + } + } + Self { + files, + arrow_schema, + name_to_field_id, + field_id_to_type, + min_cache: RefCell::new(HashMap::new()), + max_cache: RefCell::new(HashMap::new()), + nulls_cache: RefCell::new(HashMap::new()), + rows_cache: RefCell::new(None), + } + } + + fn field_id_for(&self, column: &Column) -> Option { + self.name_to_field_id.get(&column.name).copied() + } + + fn datum_to_scalar_for_field( + &self, + field_id: i32, + datum: &Datum, + ) -> datafusion::common::scalar::ScalarValue { + use datafusion::common::scalar::ScalarValue as SV; + match self.field_id_to_type.get(&field_id) { + Some(PrimitiveType::Date) => { + if let crate::spec::types::values::PrimitiveLiteral::Int(v) = datum.literal { + SV::Date32(Some(v)) + } else { + literal_to_scalar_value(&Literal::Primitive(datum.literal.clone())) + } + } + Some(PrimitiveType::Timestamp | PrimitiveType::TimestampNs) => { + if let crate::spec::types::values::PrimitiveLiteral::Long(v) = datum.literal { + SV::TimestampMicrosecond(Some(v), None) + } else { + literal_to_scalar_value(&Literal::Primitive(datum.literal.clone())) + } + } + Some(PrimitiveType::Timestamptz | PrimitiveType::TimestamptzNs) => { + if let crate::spec::types::values::PrimitiveLiteral::Long(v) = datum.literal { + SV::TimestampMicrosecond(Some(v), Some(std::sync::Arc::from("UTC"))) + } else { + literal_to_scalar_value(&Literal::Primitive(datum.literal.clone())) + } + } + Some(PrimitiveType::Time) => { + if let crate::spec::types::values::PrimitiveLiteral::Long(v) = datum.literal { + SV::Time64Microsecond(Some(v)) + } else { + literal_to_scalar_value(&Literal::Primitive(datum.literal.clone())) + } + } + _ => literal_to_scalar_value(&Literal::Primitive(datum.literal.clone())), + } + } +} + +impl PruningStatistics for IcebergPruningStats { + fn min_values(&self, column: &Column) -> Option { + // TODO: Materialize arrays only for columns referenced by the predicate + let field_id = self.field_id_for(column)?; + if let Some(arr) = self.min_cache.borrow().get(&field_id) { + return Some(arr.clone()); + } + let scalars = self.files.iter().map(|f| { + f.lower_bounds() + .get(&field_id) + .map(|d| self.datum_to_scalar_for_field(field_id, d)) + }); + let values = + scalars.map(|opt| opt.unwrap_or(datafusion::common::scalar::ScalarValue::Null)); + let arr = datafusion::common::scalar::ScalarValue::iter_to_array(values).ok()?; + self.min_cache.borrow_mut().insert(field_id, arr.clone()); + Some(arr) + } + + fn max_values(&self, column: &Column) -> Option { + let field_id = self.field_id_for(column)?; + if let Some(arr) = self.max_cache.borrow().get(&field_id) { + return Some(arr.clone()); + } + let scalars = self.files.iter().map(|f| { + f.upper_bounds() + .get(&field_id) + .map(|d| self.datum_to_scalar_for_field(field_id, d)) + }); + let values = + scalars.map(|opt| opt.unwrap_or(datafusion::common::scalar::ScalarValue::Null)); + let arr = datafusion::common::scalar::ScalarValue::iter_to_array(values).ok()?; + self.max_cache.borrow_mut().insert(field_id, arr.clone()); + Some(arr) + } + + fn num_containers(&self) -> usize { + self.files.len() + } + + fn null_counts(&self, column: &Column) -> Option { + let field_id = self.field_id_for(column)?; + if let Some(arr) = self.nulls_cache.borrow().get(&field_id) { + return Some(arr.clone()); + } + let counts: Vec = self + .files + .iter() + .map(|f| f.null_value_counts().get(&field_id).copied().unwrap_or(0)) + .collect(); + let arr: ArrayRef = Arc::new(UInt64Array::from(counts)); + self.nulls_cache.borrow_mut().insert(field_id, arr.clone()); + Some(arr) + } + + fn row_counts(&self, _column: &Column) -> Option { + if let Some(arr) = self.rows_cache.borrow().as_ref() { + return Some(arr.clone()); + } + let rows: Vec = self.files.iter().map(|f| f.record_count()).collect(); + let arr: ArrayRef = Arc::new(UInt64Array::from(rows)); + *self.rows_cache.borrow_mut() = Some(arr.clone()); + Some(arr) + } + + fn contained( + &self, + _column: &Column, + _value: &std::collections::HashSet, + ) -> Option { + let field_id = self.field_id_for(_column)?; + let mut result = Vec::with_capacity(self.files.len()); + for f in &self.files { + let lower = f.lower_bounds().get(&field_id); + let upper = f.upper_bounds().get(&field_id); + if let (Some(lb), Some(ub)) = (lower, upper) { + let lb_sv = self.datum_to_scalar_for_field(field_id, lb); + let ub_sv = self.datum_to_scalar_for_field(field_id, ub); + let mut any_match = false; + for v in _value.iter() { + if &lb_sv == v && &ub_sv == v { + any_match = true; + break; + } + } + result.push(any_match); + } else { + result.push(false); + } + } + Some(BooleanArray::from(result)) + } +} + +/// Prune Iceberg data files using DataFusion PruningPredicate over IcebergPruningStats +pub fn prune_files( + session: &dyn Session, + filters: &[Expr], + limit: Option, + logical_schema: Arc, + files: Vec, + iceberg_schema: &Schema, +) -> Result<(Vec, Option>)> { + let filter_expr = conjunction(filters.iter().cloned()); + + if filter_expr.is_none() && limit.is_none() { + return Ok((files, None)); + } + + let stats = IcebergPruningStats::new(files, logical_schema.clone(), iceberg_schema); + + let files_to_keep = if let Some(predicate) = &filter_expr { + let df_schema = logical_schema.clone().to_dfschema()?; + let physical_predicate = session.create_physical_expr(predicate.clone(), &df_schema)?; + let pruning_predicate = PruningPredicate::try_new(physical_predicate, logical_schema)?; + pruning_predicate.prune(&stats)? + } else { + vec![true; stats.num_containers()] + }; + + let mut kept = Vec::new(); + let mut rows_collected: u64 = 0; + for (file, keep) in stats.files.into_iter().zip(files_to_keep.iter()) { + if *keep { + if let Some(lim) = limit { + if rows_collected <= lim as u64 { + rows_collected += file.record_count(); + kept.push(file); + if rows_collected > lim as u64 { + break; + } + } else { + break; + } + } else { + kept.push(file); + } + } + } + + Ok((kept, Some(files_to_keep))) +} + +/// Manifest-level pruning using partition summaries from ManifestList +pub fn prune_manifests_by_partition_summaries<'a>( + manifest_list: &'a ManifestList, + table_schema: &Schema, + partition_specs: &std::collections::HashMap, + filters: &[Expr], +) -> Vec<&'a crate::spec::manifest_list::ManifestFile> { + // TODO: Add support for non-identity transforms (day/month/hour/bucket/truncate) + let eq_filters = collect_identity_eq_filters(table_schema, filters); + let in_filters = collect_identity_in_filters(table_schema, filters); + let range_filters = collect_identity_range_filters(table_schema, filters); + manifest_list + .entries() + .iter() + .filter(|mf| mf.content == ManifestContentType::Data) + .filter(|mf| { + if eq_filters.is_empty() { + return true; + } + let Some(spec) = partition_specs.get(&mf.partition_spec_id) else { + return true; + }; + let Some(part_summaries) = mf.partitions.as_ref() else { + return true; + }; + let part_type = match spec.partition_type(table_schema) { + Ok(t) => t, + Err(_) => return true, + }; + for (source_id, lit) in &eq_filters { + if let Some((idx, _pf)) = spec.fields().iter().enumerate().find(|(_, pf)| { + pf.source_id == *source_id + && matches!(pf.transform, crate::spec::transform::Transform::Identity) + }) { + if let Some(summary) = part_summaries.get(idx) { + let field_ty = part_type.fields().get(idx).map(|nf| nf.field_type.as_ref()); + if let Some(Type::Primitive(prim_ty)) = field_ty { + // TODO: Handle contains_null/contains_nan from FieldSummary + let lower = summary + .lower_bound_bytes + .as_ref() + .and_then(|b| decode_bound_bytes(prim_ty, b).ok()); + let upper = summary + .upper_bound_bytes + .as_ref() + .and_then(|b| decode_bound_bytes(prim_ty, b).ok()); + + if let (Some(lb), Some(ub)) = (lower.as_ref(), upper.as_ref()) { + if lt_prim(lit, lb) || gt_prim(lit, ub) { + return false; + } + } else if let Some(lb) = lower.as_ref() { + if lt_prim(lit, lb) { + return false; + } + } else if let Some(ub) = upper.as_ref() { + if gt_prim(lit, ub) { + return false; + } + } + } + } + } + } + + for (source_id, lits) in &in_filters { + if let Some((idx, _pf)) = spec.fields().iter().enumerate().find(|(_, pf)| { + pf.source_id == *source_id + && matches!(pf.transform, crate::spec::transform::Transform::Identity) + }) { + if let Some(summary) = part_summaries.get(idx) { + let field_ty = part_type.fields().get(idx).map(|nf| nf.field_type.as_ref()); + if let Some(Type::Primitive(prim_ty)) = field_ty { + let lower = summary + .lower_bound_bytes + .as_ref() + .and_then(|b| decode_bound_bytes(prim_ty, b).ok()); + let upper = summary + .upper_bound_bytes + .as_ref() + .and_then(|b| decode_bound_bytes(prim_ty, b).ok()); + if let (Some(lb), Some(ub)) = (lower.as_ref(), upper.as_ref()) { + let mut any_in = false; + for lit in lits { + if !(lt_prim(lit, lb) || gt_prim(lit, ub)) { + any_in = true; + break; + } + } + if !any_in { + return false; + } + } else if let Some(lb) = lower.as_ref() { + let any_in = lits.iter().any(|v| !lt_prim(v, lb)); + if !any_in { + return false; + } + } else if let Some(ub) = upper.as_ref() { + let any_in = lits.iter().any(|v| !gt_prim(v, ub)); + if !any_in { + return false; + } + } + } + } + } + } + + for (source_id, range) in &range_filters { + if let Some((idx, _pf)) = spec.fields().iter().enumerate().find(|(_, pf)| { + pf.source_id == *source_id + && matches!(pf.transform, crate::spec::transform::Transform::Identity) + }) { + if let Some(summary) = part_summaries.get(idx) { + let field_ty = part_type.fields().get(idx).map(|nf| nf.field_type.as_ref()); + if let Some(Type::Primitive(prim_ty)) = field_ty { + let lower = summary + .lower_bound_bytes + .as_ref() + .and_then(|b| decode_bound_bytes(prim_ty, b).ok()); + let upper = summary + .upper_bound_bytes + .as_ref() + .and_then(|b| decode_bound_bytes(prim_ty, b).ok()); + if let (Some(lb), Some(ub)) = (lower.as_ref(), upper.as_ref()) { + if let Some((ref qmin, _incl_min)) = range.min { + if gt_prim(qmin, ub) { + return false; + } + } + if let Some((ref qmax, _incl_max)) = range.max { + if lt_prim(qmax, lb) { + return false; + } + } + } + } + } + } + } + true + }) + .collect() +} + +/// Load a manifest and prune entries by partition+metrics +pub fn prune_manifest_entries( + manifest: &Manifest, + _schema: &Schema, + _filters: &[Expr], +) -> Vec { + // TODO: Add partition-transform awareness and metrics-only pruning at manifest entry granularity + manifest + .entries() + .iter() + .filter(|e| { + matches!( + e.status, + crate::spec::manifest::ManifestStatus::Added + | crate::spec::manifest::ManifestStatus::Existing + ) + }) + .map(|e| e.data_file.clone()) + .collect() +} + +fn collect_identity_eq_filters( + schema: &Schema, + filters: &[Expr], +) -> Vec<(i32, crate::spec::types::values::PrimitiveLiteral)> { + fn strip(expr: &Expr) -> &Expr { + match expr { + Expr::Cast(c) => strip(&c.expr), + Expr::Alias(a) => strip(&a.expr), + _ => expr, + } + } + + let mut result = Vec::new(); + + fn visit_expr( + acc: &mut Vec<(i32, crate::spec::types::values::PrimitiveLiteral)>, + schema: &Schema, + e: &Expr, + ) { + match e { + Expr::BinaryExpr(BinaryExpr { left, op, right }) if *op == Operator::Eq => { + let l = strip(left); + let r = strip(right); + + // col = lit + if let Expr::Column(c) = l { + if let Expr::Literal(sv, _) = r { + let col_name = c.name.clone(); + if let Some(field) = schema.field_by_name(&col_name) { + if let Some(pl) = scalar_to_primitive_literal(sv) { + acc.push((field.id, pl)); + return; + } + } + } + } + + // lit = col + if let Expr::Literal(sv, _) = l { + if let Expr::Column(c) = r { + let col_name = c.name.clone(); + if let Some(field) = schema.field_by_name(&col_name) { + if let Some(pl) = scalar_to_primitive_literal(sv) { + acc.push((field.id, pl)); + } + } + } + } + } + Expr::BinaryExpr(BinaryExpr { left, op, right }) if *op == Operator::And => { + visit_expr(acc, schema, left); + visit_expr(acc, schema, right); + } + _ => {} + } + } + + for expr in filters { + visit_expr(&mut result, schema, expr); + } + result +} + +fn collect_identity_in_filters( + schema: &Schema, + filters: &[Expr], +) -> std::collections::HashMap> { + fn strip(expr: &Expr) -> &Expr { + match expr { + Expr::Cast(c) => strip(&c.expr), + Expr::Alias(a) => strip(&a.expr), + _ => expr, + } + } + + let mut result: std::collections::HashMap<_, Vec<_>> = std::collections::HashMap::new(); + + fn visit_expr( + acc: &mut std::collections::HashMap>, + schema: &Schema, + e: &Expr, + ) { + match e { + Expr::InList(in_list) if !in_list.negated => { + let e = strip(&in_list.expr); + if let Expr::Column(c) = e { + if let Some(field) = schema.field_by_name(&c.name) { + let mut vals = Vec::new(); + for item in &in_list.list { + if let Expr::Literal(ref sv, _) = item { + if let Some(pl) = scalar_to_primitive_literal(sv) { + vals.push(pl); + } + } + } + if !vals.is_empty() { + acc.entry(field.id).or_default().extend(vals); + } + } + } + } + Expr::BinaryExpr(BinaryExpr { left, op, right }) if *op == Operator::And => { + visit_expr(acc, schema, left); + visit_expr(acc, schema, right); + } + _ => {} + } + } + + for expr in filters { + visit_expr(&mut result, schema, expr); + } + result +} + +#[derive(Clone, Default)] +struct RangeConstraint { + min: Option<(crate::spec::types::values::PrimitiveLiteral, bool)>, + max: Option<(crate::spec::types::values::PrimitiveLiteral, bool)>, +} + +fn collect_identity_range_filters( + schema: &Schema, + filters: &[Expr], +) -> std::collections::HashMap { + fn strip(expr: &Expr) -> &Expr { + match expr { + Expr::Cast(c) => strip(&c.expr), + Expr::Alias(a) => strip(&a.expr), + _ => expr, + } + } + + fn tighten_min( + cur: &mut Option<(crate::spec::types::values::PrimitiveLiteral, bool)>, + cand: (crate::spec::types::values::PrimitiveLiteral, bool), + ) { + match cur { + None => *cur = Some(cand), + Some((ref mut v, ref mut incl)) => { + if gt_prim(&cand.0, v) || (eq_prim(&cand.0, v) && cand.1 && !*incl) { + *v = cand.0; + *incl = cand.1; + } + } + } + } + fn tighten_max( + cur: &mut Option<(crate::spec::types::values::PrimitiveLiteral, bool)>, + cand: (crate::spec::types::values::PrimitiveLiteral, bool), + ) { + match cur { + None => *cur = Some(cand), + Some((ref mut v, ref mut incl)) => { + if lt_prim(&cand.0, v) || (eq_prim(&cand.0, v) && cand.1 && !*incl) { + *v = cand.0; + *incl = cand.1; + } + } + } + } + + let mut result: std::collections::HashMap = + std::collections::HashMap::new(); + + fn visit_expr( + acc: &mut std::collections::HashMap, + schema: &Schema, + e: &Expr, + ) { + if let Expr::BinaryExpr(BinaryExpr { left, op, right }) = e { + let l = strip(left); + let r = strip(right); + match op { + Operator::Gt | Operator::GtEq => { + if let (Expr::Column(c), Expr::Literal(sv, _)) = (l, r) { + if let Some(field) = schema.field_by_name(&c.name) { + if let Some(pl) = scalar_to_primitive_literal(sv) { + let entry = acc.entry(field.id).or_default(); + tighten_min(&mut entry.min, (pl, *op == Operator::GtEq)); + } + } + } + } + Operator::Lt | Operator::LtEq => { + if let (Expr::Column(c), Expr::Literal(sv, _)) = (l, r) { + if let Some(field) = schema.field_by_name(&c.name) { + if let Some(pl) = scalar_to_primitive_literal(sv) { + let entry = acc.entry(field.id).or_default(); + tighten_max(&mut entry.max, (pl, *op == Operator::LtEq)); + } + } + } + } + Operator::And => { + visit_expr(acc, schema, l); + visit_expr(acc, schema, r); + } + _ => {} + } + } + } + + for expr in filters { + visit_expr(&mut result, schema, expr); + } + result +} + +fn scalar_to_primitive_literal( + sv: &datafusion::common::scalar::ScalarValue, +) -> Option { + use crate::spec::types::values::PrimitiveLiteral::*; + match sv { + datafusion::common::scalar::ScalarValue::Boolean(Some(v)) => Some(Boolean(*v)), + datafusion::common::scalar::ScalarValue::Int32(Some(v)) => Some(Int(*v)), + datafusion::common::scalar::ScalarValue::Int64(Some(v)) => Some(Long(*v)), + datafusion::common::scalar::ScalarValue::Float32(Some(v)) => { + Some(Float(ordered_float::OrderedFloat(*v))) + } + datafusion::common::scalar::ScalarValue::Float64(Some(v)) => { + Some(Double(ordered_float::OrderedFloat(*v))) + } + datafusion::common::scalar::ScalarValue::Utf8(Some(s)) => Some(String(s.clone())), + datafusion::common::scalar::ScalarValue::LargeUtf8(Some(s)) => Some(String(s.clone())), + datafusion::common::scalar::ScalarValue::Binary(Some(b)) => Some(Binary(b.clone())), + _ => None, + } +} + +fn decode_bound_bytes( + ty: &PrimitiveType, + bytes: &[u8], +) -> Result { + use crate::spec::types::values::PrimitiveLiteral as PL; + let pl = match ty { + PrimitiveType::Boolean => { + let val = !(bytes.len() == 1 && bytes[0] == 0u8); + PL::Boolean(val) + } + PrimitiveType::Int | PrimitiveType::Date => { + let val = i32::from_le_bytes(bytes.try_into().map_err(|_| "Invalid i32 bytes")?); + PL::Int(val) + } + PrimitiveType::Long + | PrimitiveType::Time + | PrimitiveType::Timestamp + | PrimitiveType::Timestamptz + | PrimitiveType::TimestampNs + | PrimitiveType::TimestamptzNs => { + let val = if bytes.len() == 4 { + i32::from_le_bytes(bytes.try_into().map_err(|_| "Invalid i32 bytes")?) as i64 + } else { + i64::from_le_bytes(bytes.try_into().map_err(|_| "Invalid i64 bytes")?) + }; + PL::Long(val) + } + PrimitiveType::Float => { + let val = f32::from_le_bytes(bytes.try_into().map_err(|_| "Invalid f32 bytes")?); + PL::Float(ordered_float::OrderedFloat(val)) + } + PrimitiveType::Double => { + let val = if bytes.len() == 4 { + f32::from_le_bytes(bytes.try_into().map_err(|_| "Invalid f32 bytes")?) as f64 + } else { + f64::from_le_bytes(bytes.try_into().map_err(|_| "Invalid f64 bytes")?) + }; + PL::Double(ordered_float::OrderedFloat(val)) + } + PrimitiveType::String => { + let val = std::str::from_utf8(bytes) + .map_err(|_| "Invalid UTF-8")? + .to_string(); + PL::String(val) + } + PrimitiveType::Uuid => { + return Err("uuid bound decoding not supported".to_string()); + } + PrimitiveType::Fixed(_) | PrimitiveType::Binary => PL::Binary(bytes.to_vec()), + PrimitiveType::Decimal { .. } => { + return Err("decimal bound decoding not supported".to_string()); + } + }; + Ok(pl) +} + +fn lt_prim( + a: &crate::spec::types::values::PrimitiveLiteral, + b: &crate::spec::types::values::PrimitiveLiteral, +) -> bool { + use crate::spec::types::values::PrimitiveLiteral as PL; + match (a, b) { + (PL::Int(x), PL::Int(y)) => x < y, + (PL::Long(x), PL::Long(y)) => x < y, + (PL::Float(x), PL::Float(y)) => x < y, + (PL::Double(x), PL::Double(y)) => x < y, + (PL::String(x), PL::String(y)) => x < y, + _ => false, + } +} + +fn gt_prim( + a: &crate::spec::types::values::PrimitiveLiteral, + b: &crate::spec::types::values::PrimitiveLiteral, +) -> bool { + use crate::spec::types::values::PrimitiveLiteral as PL; + match (a, b) { + (PL::Int(x), PL::Int(y)) => x > y, + (PL::Long(x), PL::Long(y)) => x > y, + (PL::Float(x), PL::Float(y)) => x > y, + (PL::Double(x), PL::Double(y)) => x > y, + (PL::String(x), PL::String(y)) => x > y, + _ => false, + } +} + +fn eq_prim( + a: &crate::spec::types::values::PrimitiveLiteral, + b: &crate::spec::types::values::PrimitiveLiteral, +) -> bool { + use crate::spec::types::values::PrimitiveLiteral as PL; + match (a, b) { + (PL::Int(x), PL::Int(y)) => x == y, + (PL::Long(x), PL::Long(y)) => x == y, + (PL::Float(x), PL::Float(y)) => x == y, + (PL::Double(x), PL::Double(y)) => x == y, + (PL::String(x), PL::String(y)) => x == y, + (PL::Binary(x), PL::Binary(y)) => x == y, + (PL::Boolean(x), PL::Boolean(y)) => x == y, + _ => false, + } +} diff --git a/crates/sail-iceberg/src/lib.rs b/crates/sail-iceberg/src/lib.rs new file mode 100644 index 0000000000..9c91a585b2 --- /dev/null +++ b/crates/sail-iceberg/src/lib.rs @@ -0,0 +1,9 @@ +pub mod arrow_conversion; +pub mod datasource; +pub mod spec; +pub mod table_format; + +pub use arrow_conversion::*; +pub use datasource::*; +pub use spec::*; +pub use table_format::*; diff --git a/crates/sail-iceberg/src/options.rs b/crates/sail-iceberg/src/options.rs new file mode 100644 index 0000000000..0e35d8da4c --- /dev/null +++ b/crates/sail-iceberg/src/options.rs @@ -0,0 +1,9 @@ +use serde::{Deserialize, Serialize}; + +/// Options that control the behavior of Iceberg table reads (time-travel, refs). +#[derive(Debug, Clone, PartialEq, Default, Serialize, Deserialize)] +pub struct TableIcebergOptions { + pub use_ref: Option, + pub snapshot_id: Option, + pub timestamp_as_of: Option, +} diff --git a/crates/sail-iceberg/src/spec/catalog/metadata_location.rs b/crates/sail-iceberg/src/spec/catalog/metadata_location.rs new file mode 100644 index 0000000000..cd6c69dc58 --- /dev/null +++ b/crates/sail-iceberg/src/spec/catalog/metadata_location.rs @@ -0,0 +1,92 @@ +// 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. + +// [CREDIT]: https://raw.githubusercontent.com/apache/iceberg-rust/dc349284a4204c1a56af47fb3177ace6f9e899a0/crates/iceberg/src/catalog/metadata_location.rs + +use std::fmt::Display; +use std::str::FromStr; + +use uuid::Uuid; + +#[derive(Clone, Debug, PartialEq)] +pub struct MetadataLocation { + table_location: String, + version: i32, + id: Uuid, +} + +impl MetadataLocation { + pub fn new_with_table_location(table_location: impl ToString) -> Self { + Self { + table_location: table_location.to_string(), + version: 0, + id: Uuid::new_v4(), + } + } + pub fn with_next_version(&self) -> Self { + Self { + table_location: self.table_location.clone(), + version: self.version + 1, + id: Uuid::new_v4(), + } + } + fn parse_metadata_path_prefix(path: &str) -> Result { + let prefix = path.strip_suffix("/metadata").ok_or_else(|| { + format!( + "Metadata location not under \"/metadata\" subdirectory: {}", + path + ) + })?; + Ok(prefix.to_string()) + } + fn parse_file_name(file_name: &str) -> Result<(i32, Uuid), String> { + let (version, id) = file_name + .strip_suffix(".metadata.json") + .ok_or_else(|| format!("Invalid metadata file ending: {}", file_name))? + .split_once('-') + .ok_or_else(|| format!("Invalid metadata file name format: {}", file_name))?; + let v = version.parse::().map_err(|e| e.to_string())?; + let u = Uuid::parse_str(id).map_err(|e| e.to_string())?; + Ok((v, u)) + } +} + +impl Display for MetadataLocation { + fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + write!( + f, + "{}/metadata/{:0>5}-{}.metadata.json", + self.table_location, self.version, self.id + ) + } +} + +impl FromStr for MetadataLocation { + type Err = String; + fn from_str(s: &str) -> Result { + let (path, file_name) = s + .rsplit_once('/') + .ok_or_else(|| format!("Invalid metadata location: {}", s))?; + let prefix = Self::parse_metadata_path_prefix(path)?; + let (version, id) = Self::parse_file_name(file_name)?; + Ok(Self { + table_location: prefix, + version, + id, + }) + } +} diff --git a/crates/sail-iceberg/src/spec/catalog/mod.rs b/crates/sail-iceberg/src/spec/catalog/mod.rs new file mode 100644 index 0000000000..4ed243c179 --- /dev/null +++ b/crates/sail-iceberg/src/spec/catalog/mod.rs @@ -0,0 +1,371 @@ +// 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. + +// [CREDIT]: https://raw.githubusercontent.com/apache/iceberg-rust/dc349284a4204c1a56af47fb3177ace6f9e899a0/crates/iceberg/src/catalog/mod.rs + +use std::collections::HashMap; +use std::fmt::Display; + +use serde::{Deserialize, Serialize}; +use uuid::Uuid; + +use crate::spec::partition::UnboundPartitionSpec; +use crate::spec::views::ViewVersion; +use crate::spec::{ + FormatVersion, PartitionStatisticsFile, Schema, SchemaId, Snapshot, SnapshotReference, + SortOrder, StatisticsFile, +}; + +pub mod metadata_location; + +#[derive(Serialize, Deserialize, Debug, Clone, PartialEq, Eq, Hash, PartialOrd, Ord)] +pub struct NamespaceIdent(Vec); + +impl NamespaceIdent { + pub fn new(name: String) -> Self { + Self(vec![name]) + } + pub fn from_vec(names: Vec) -> Self { + Self(names) + } + pub fn inner(self) -> Vec { + self.0 + } +} + +impl AsRef> for NamespaceIdent { + fn as_ref(&self) -> &Vec { + &self.0 + } +} + +impl Display for NamespaceIdent { + fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + write!(f, "{}", self.0.join(".")) + } +} + +#[derive(Debug, Clone, PartialEq, Eq)] +pub struct Namespace { + name: NamespaceIdent, + properties: HashMap, +} + +impl Namespace { + pub fn new(name: NamespaceIdent) -> Self { + Self { + name, + properties: HashMap::new(), + } + } + pub fn with_properties(name: NamespaceIdent, properties: HashMap) -> Self { + Self { name, properties } + } + pub fn name(&self) -> &NamespaceIdent { + &self.name + } + pub fn properties(&self) -> &HashMap { + &self.properties + } +} + +#[derive(Serialize, Deserialize, Debug, Clone, PartialEq, Eq, PartialOrd, Ord, Hash)] +pub struct TableIdent { + pub namespace: NamespaceIdent, + pub name: String, +} + +impl TableIdent { + pub fn new(namespace: NamespaceIdent, name: String) -> Self { + Self { namespace, name } + } + pub fn namespace(&self) -> &NamespaceIdent { + &self.namespace + } + pub fn name(&self) -> &str { + &self.name + } +} + +impl Display for TableIdent { + fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + write!(f, "{}.{}", self.namespace, self.name) + } +} + +#[derive(Debug, Clone, Serialize, Deserialize)] +#[serde(rename_all = "kebab-case")] +pub struct TableCreation { + pub name: String, + #[serde(default, skip_serializing_if = "Option::is_none")] + pub location: Option, + pub schema: Schema, + #[serde(default, skip_serializing_if = "Option::is_none")] + pub partition_spec: Option, + #[serde(default, skip_serializing_if = "Option::is_none")] + pub sort_order: Option, + #[serde(default)] + pub properties: HashMap, +} + +#[derive(Clone, Debug, Serialize, Deserialize, PartialEq)] +#[serde(tag = "type")] +pub enum TableRequirement { + #[serde(rename = "assert-create")] + NotExist, + #[serde(rename = "assert-table-uuid")] + UuidMatch { uuid: Uuid }, + #[serde(rename = "assert-ref-snapshot-id")] + RefSnapshotIdMatch { + r#ref: String, + #[serde(rename = "snapshot-id")] + snapshot_id: Option, + }, + #[serde(rename = "assert-last-assigned-field-id")] + LastAssignedFieldIdMatch { + #[serde(rename = "last-assigned-field-id")] + last_assigned_field_id: i32, + }, + #[serde(rename = "assert-current-schema-id")] + CurrentSchemaIdMatch { + #[serde(rename = "current-schema-id")] + current_schema_id: SchemaId, + }, + #[serde(rename = "assert-last-assigned-partition-id")] + LastAssignedPartitionIdMatch { + #[serde(rename = "last-assigned-partition-id")] + last_assigned_partition_id: i32, + }, + #[serde(rename = "assert-default-spec-id")] + DefaultSpecIdMatch { + #[serde(rename = "default-spec-id")] + default_spec_id: i32, + }, + #[serde(rename = "assert-default-sort-order-id")] + DefaultSortOrderIdMatch { + #[serde(rename = "default-sort-order-id")] + default_sort_order_id: i64, + }, +} + +#[derive(Debug, Serialize, Deserialize, PartialEq, Clone)] +#[serde(tag = "action", rename_all = "kebab-case")] +pub enum TableUpdate { + UpgradeFormatVersion { + format_version: FormatVersion, + }, + AssignUuid { + uuid: Uuid, + }, + AddSchema { + schema: Box, + }, + SetCurrentSchema { + #[serde(rename = "schema-id")] + schema_id: i32, + }, + AddSpec { + spec: UnboundPartitionSpec, + }, + SetDefaultSpec { + #[serde(rename = "spec-id")] + spec_id: i32, + }, + AddSortOrder { + sort_order: SortOrder, + }, + SetDefaultSortOrder { + #[serde(rename = "sort-order-id")] + sort_order_id: i64, + }, + AddSnapshot { + #[serde(deserialize_with = "crate::spec::catalog::_serde::deserialize_snapshot")] + snapshot: Snapshot, + }, + SetSnapshotRef { + #[serde(rename = "ref-name")] + ref_name: String, + #[serde(flatten)] + reference: SnapshotReference, + }, + RemoveSnapshots { + #[serde(rename = "snapshot-ids")] + snapshot_ids: Vec, + }, + RemoveSnapshotRef { + #[serde(rename = "ref-name")] + ref_name: String, + }, + SetLocation { + location: String, + }, + SetProperties { + updates: HashMap, + }, + RemoveProperties { + removals: Vec, + }, + RemovePartitionSpecs { + #[serde(rename = "spec-ids")] + spec_ids: Vec, + }, + #[serde(with = "_serde_set_statistics")] + SetStatistics { + statistics: StatisticsFile, + }, + RemoveStatistics { + #[serde(rename = "snapshot-id")] + snapshot_id: i64, + }, + SetPartitionStatistics { + partition_statistics: PartitionStatisticsFile, + }, + RemovePartitionStatistics { + #[serde(rename = "snapshot-id")] + snapshot_id: i64, + }, + RemoveSchemas { + #[serde(rename = "schema-ids")] + schema_ids: Vec, + }, +} + +pub(super) mod _serde { + use serde::{Deserialize as _, Deserializer}; + + use super::*; + pub(super) fn deserialize_snapshot<'de, D>( + deserializer: D, + ) -> std::result::Result + where + D: Deserializer<'de>, + { + let buf = CatalogSnapshot::deserialize(deserializer)?; + Snapshot::try_from(buf).map_err(serde::de::Error::custom) + } + #[derive(Debug, Deserialize, PartialEq, Eq)] + #[serde(rename_all = "kebab-case")] + 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: super::super::snapshot::Summary, + #[serde(skip_serializing_if = "Option::is_none")] + schema_id: Option, + } + impl TryFrom for Snapshot { + type Error = String; + fn try_from(snapshot: CatalogSnapshot) -> Result { + let mut builder = Snapshot::builder() + .with_snapshot_id(snapshot.snapshot_id) + .with_sequence_number(snapshot.sequence_number) + .with_timestamp_ms(snapshot.timestamp_ms) + .with_manifest_list(snapshot.manifest_list) + .with_summary(snapshot.summary); + if let Some(parent) = snapshot.parent_snapshot_id { + builder = builder.with_parent_snapshot_id(parent); + } + if let Some(schema_id) = snapshot.schema_id { + builder = builder.with_schema_id(schema_id); + } + builder.build() + } + } +} + +#[derive(Debug, Clone, Copy, PartialEq, Eq, Serialize, Deserialize)] +pub enum ViewFormatVersion { + #[serde(rename = "1")] + V1 = 1, +} + +#[derive(Clone, Debug, PartialEq, Serialize, Deserialize)] +#[serde(tag = "action", rename_all = "kebab-case")] +pub enum ViewUpdate { + AssignUuid { + uuid: Uuid, + }, + UpgradeFormatVersion { + format_version: ViewFormatVersion, + }, + AddSchema { + schema: Box, + #[serde(rename = "last-column-id", skip_serializing_if = "Option::is_none")] + last_column_id: Option, + }, + SetLocation { + location: String, + }, + SetProperties { + updates: HashMap, + }, + RemoveProperties { + removals: Vec, + }, + AddViewVersion { + #[serde(rename = "view-version")] + view_version: ViewVersion, + }, + SetCurrentViewVersion { + #[serde(rename = "view-version-id")] + view_version_id: i32, + }, +} + +mod _serde_set_statistics { + use serde::{Deserialize, Deserializer, Serialize, Serializer}; + + use super::*; + #[derive(Debug, Serialize, Deserialize)] + #[serde(rename_all = "kebab-case")] + struct SetStatistics { + snapshot_id: Option, + statistics: StatisticsFile, + } + pub fn serialize( + value: &StatisticsFile, + serializer: S, + ) -> std::result::Result + where + S: Serializer, + { + SetStatistics { + snapshot_id: Some(value.snapshot_id), + statistics: value.clone(), + } + .serialize(serializer) + } + pub fn deserialize<'de, D>(deserializer: D) -> std::result::Result + where + D: Deserializer<'de>, + { + let SetStatistics { + snapshot_id, + statistics, + } = SetStatistics::deserialize(deserializer)?; + if let Some(snapshot_id) = snapshot_id { + if snapshot_id != statistics.snapshot_id { + return Err(serde::de::Error::custom(format!("Snapshot id to set {snapshot_id} does not match the statistics file snapshot id {}", statistics.snapshot_id))); + } + } + Ok(statistics) + } +} diff --git a/crates/sail-iceberg/src/spec/encrypted_key.rs b/crates/sail-iceberg/src/spec/encrypted_key.rs new file mode 100644 index 0000000000..3e8d25721d --- /dev/null +++ b/crates/sail-iceberg/src/spec/encrypted_key.rs @@ -0,0 +1,23 @@ +// 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. + +// [CREDIT]: https://raw.githubusercontent.com/apache/iceberg-rust/dc349284a4204c1a56af47fb3177ace6f9e899a0/crates/iceberg/src/spec/encrypted_key.rs + +// Awareness stub for non-read path +#[allow(dead_code)] +#[derive(Debug, Clone)] +pub struct EncryptedKey; diff --git a/crates/sail-iceberg/src/spec/manifest/_serde.rs b/crates/sail-iceberg/src/spec/manifest/_serde.rs new file mode 100644 index 0000000000..deab8caa74 --- /dev/null +++ b/crates/sail-iceberg/src/spec/manifest/_serde.rs @@ -0,0 +1,152 @@ +// 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. + +// [CREDIT]: https://raw.githubusercontent.com/apache/iceberg-rust/dc349284a4204c1a56af47fb3177ace6f9e899a0/crates/iceberg/src/spec/manifest/_serde.rs + +use apache_avro::types::Value as AvroValue; +use serde::{Deserialize, Serialize}; + +use super::{DataContentType, DataFile, DataFileFormat}; +use crate::spec::Schema; + +#[derive(Debug, Serialize, Deserialize)] +pub(super) struct ManifestEntryAvro { + #[serde(rename = "status")] + pub status: i32, + #[serde(rename = "snapshot_id")] + pub snapshot_id: Option, + #[serde(rename = "sequence_number")] + pub sequence_number: Option, + #[serde(rename = "file_sequence_number")] + pub file_sequence_number: Option, + #[serde(rename = "data_file")] + pub data_file: DataFileAvro, +} + +#[derive(Debug, Serialize, Deserialize)] +pub(super) struct DataFileAvro { + #[serde(rename = "content", default)] + pub content: i32, + #[serde(rename = "file_path")] + pub file_path: String, + #[serde(rename = "file_format")] + pub file_format: String, + #[serde(rename = "partition")] + pub partition: serde_json::Value, + #[serde(rename = "record_count")] + pub record_count: i64, + #[serde(rename = "file_size_in_bytes")] + pub file_size_in_bytes: i64, + #[serde(skip)] + pub column_sizes: Option, + #[serde(skip)] + pub value_counts: Option, + #[serde(skip)] + pub null_value_counts: Option, + #[serde(skip)] + pub nan_value_counts: Option, + #[serde(skip)] + pub lower_bounds: Option, + #[serde(skip)] + pub upper_bounds: Option, + #[serde(rename = "key_metadata")] + pub key_metadata: Option>, + #[serde(rename = "split_offsets")] + pub split_offsets: Option>, + #[serde(rename = "equality_ids")] + pub equality_ids: Option>, + #[serde(rename = "sort_order_id")] + pub sort_order_id: Option, +} + +impl DataFileAvro { + pub fn into_data_file( + self, + schema: &Schema, + _partition_type_len: i32, + partition_spec_id: i32, + ) -> DataFile { + let content = match self.content { + 0 => DataContentType::Data, + 1 => DataContentType::PositionDeletes, + 2 => DataContentType::EqualityDeletes, + _ => DataContentType::Data, + }; + + let file_format = match self.file_format.to_uppercase().as_str() { + "PARQUET" => DataFileFormat::Parquet, + "AVRO" => DataFileFormat::Avro, + "ORC" => DataFileFormat::Orc, + _ => DataFileFormat::Parquet, + }; + + let partition = super::super::manifest::parse_partition_values(Some(&self.partition)); + + let column_sizes = super::super::manifest::parse_i64_map_from_avro(&self.column_sizes) + .into_iter() + .map(|(k, v)| (k, v as u64)) + .collect(); + let value_counts = super::super::manifest::parse_i64_map_from_avro(&self.value_counts) + .into_iter() + .map(|(k, v)| (k, v as u64)) + .collect(); + let null_value_counts = + super::super::manifest::parse_i64_map_from_avro(&self.null_value_counts) + .into_iter() + .map(|(k, v)| (k, v as u64)) + .collect(); + let nan_value_counts = + super::super::manifest::parse_i64_map_from_avro(&self.nan_value_counts) + .into_iter() + .map(|(k, v)| (k, v as u64)) + .collect(); + + let lower_bounds_raw = + super::super::manifest::parse_bytes_map_from_avro(&self.lower_bounds); + let upper_bounds_raw = + super::super::manifest::parse_bytes_map_from_avro(&self.upper_bounds); + let lower_bounds = + super::super::manifest::parse_bounds_from_binary(lower_bounds_raw.as_ref(), schema); + let upper_bounds = + super::super::manifest::parse_bounds_from_binary(upper_bounds_raw.as_ref(), schema); + + DataFile { + content, + file_path: self.file_path, + file_format, + partition, + record_count: self.record_count as u64, + file_size_in_bytes: self.file_size_in_bytes as u64, + column_sizes, + value_counts, + null_value_counts, + nan_value_counts, + lower_bounds, + upper_bounds, + block_size_in_bytes: None, + key_metadata: self.key_metadata, + split_offsets: self.split_offsets.unwrap_or_default(), + equality_ids: self.equality_ids.unwrap_or_default().into_iter().collect(), + sort_order_id: self.sort_order_id, + first_row_id: None, + partition_spec_id, + referenced_data_file: None, + content_offset: None, + content_size_in_bytes: None, + } + } +} diff --git a/crates/sail-iceberg/src/spec/manifest/data_file.rs b/crates/sail-iceberg/src/spec/manifest/data_file.rs new file mode 100644 index 0000000000..d660dd1d53 --- /dev/null +++ b/crates/sail-iceberg/src/spec/manifest/data_file.rs @@ -0,0 +1,125 @@ +// 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. + +// [CREDIT]: https://raw.githubusercontent.com/apache/iceberg-rust/dc349284a4204c1a56af47fb3177ace6f9e899a0/crates/iceberg/src/spec/manifest/data_file.rs + +use std::collections::HashMap; + +use serde::{Deserialize, Serialize}; + +use crate::spec::types::values::Literal; +use crate::spec::Datum; + +/// Content type of a data file. +#[derive(Debug, PartialEq, Eq, Clone, Copy, Serialize, Deserialize)] +#[serde(rename_all = "UPPERCASE")] +pub enum DataContentType { + Data, + PositionDeletes, + EqualityDeletes, +} + +/// File format of a data file. +#[derive(Debug, PartialEq, Eq, Clone, Copy, Serialize, Deserialize)] +#[serde(rename_all = "UPPERCASE")] +pub enum DataFileFormat { + Avro, + Orc, + Parquet, + Puffin, +} + +/// A data file in Iceberg. +#[derive(Debug, PartialEq, Eq, Clone, Serialize, Deserialize)] +pub struct DataFile { + pub content: DataContentType, + pub file_path: String, + pub file_format: DataFileFormat, + pub partition: Vec>, + pub record_count: u64, + pub file_size_in_bytes: u64, + #[serde(skip_serializing_if = "HashMap::is_empty")] + pub column_sizes: HashMap, + #[serde(skip_serializing_if = "HashMap::is_empty")] + pub value_counts: HashMap, + #[serde(skip_serializing_if = "HashMap::is_empty")] + pub null_value_counts: HashMap, + #[serde(skip_serializing_if = "HashMap::is_empty")] + pub nan_value_counts: HashMap, + #[serde(skip_serializing_if = "HashMap::is_empty")] + pub lower_bounds: HashMap, + #[serde(skip_serializing_if = "HashMap::is_empty")] + pub upper_bounds: HashMap, + #[serde(skip_serializing_if = "Option::is_none")] + pub block_size_in_bytes: Option, + #[serde(skip_serializing_if = "Option::is_none")] + pub key_metadata: Option>, + #[serde(skip_serializing_if = "Vec::is_empty")] + pub split_offsets: Vec, + #[serde(skip_serializing_if = "Vec::is_empty")] + pub equality_ids: Vec, + #[serde(skip_serializing_if = "Option::is_none")] + pub sort_order_id: Option, + #[serde(skip_serializing_if = "Option::is_none")] + pub first_row_id: Option, + pub partition_spec_id: i32, + #[serde(skip_serializing_if = "Option::is_none")] + pub referenced_data_file: Option, + #[serde(skip_serializing_if = "Option::is_none")] + pub content_offset: Option, + #[serde(skip_serializing_if = "Option::is_none")] + pub content_size_in_bytes: Option, +} + +impl DataFile { + pub fn content_type(&self) -> DataContentType { + self.content + } + pub fn file_path(&self) -> &str { + &self.file_path + } + pub fn file_format(&self) -> DataFileFormat { + self.file_format + } + pub fn partition(&self) -> &[Option] { + &self.partition + } + pub fn record_count(&self) -> u64 { + self.record_count + } + pub fn file_size_in_bytes(&self) -> u64 { + self.file_size_in_bytes + } + pub fn column_sizes(&self) -> &HashMap { + &self.column_sizes + } + pub fn value_counts(&self) -> &HashMap { + &self.value_counts + } + pub fn null_value_counts(&self) -> &HashMap { + &self.null_value_counts + } + pub fn nan_value_counts(&self) -> &HashMap { + &self.nan_value_counts + } + pub fn lower_bounds(&self) -> &HashMap { + &self.lower_bounds + } + pub fn upper_bounds(&self) -> &HashMap { + &self.upper_bounds + } +} diff --git a/crates/sail-iceberg/src/spec/manifest/entry.rs b/crates/sail-iceberg/src/spec/manifest/entry.rs new file mode 100644 index 0000000000..449b7111d4 --- /dev/null +++ b/crates/sail-iceberg/src/spec/manifest/entry.rs @@ -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. + +// [CREDIT]: https://raw.githubusercontent.com/apache/iceberg-rust/dc349284a4204c1a56af47fb3177ace6f9e899a0/crates/iceberg/src/spec/manifest/entry.rs + +use serde::{Deserialize, Serialize}; + +use super::DataFile; + +/// Status of a manifest entry. +#[derive(Debug, PartialEq, Eq, Clone, Copy, Serialize, Deserialize)] +#[serde(rename_all = "UPPERCASE")] +pub enum ManifestStatus { + Added, + Existing, + Deleted, +} + +/// A manifest entry represents a data file in a manifest. +#[derive(Debug, PartialEq, Eq, Clone, Serialize, Deserialize)] +pub struct ManifestEntry { + pub status: ManifestStatus, + #[serde(skip_serializing_if = "Option::is_none")] + pub snapshot_id: Option, + #[serde(skip_serializing_if = "Option::is_none")] + pub sequence_number: Option, + #[serde(skip_serializing_if = "Option::is_none")] + pub file_sequence_number: Option, + pub data_file: DataFile, +} + +impl ManifestEntry { + pub fn new( + status: ManifestStatus, + snapshot_id: Option, + sequence_number: Option, + file_sequence_number: Option, + data_file: DataFile, + ) -> Self { + Self { + status, + snapshot_id, + sequence_number, + file_sequence_number, + data_file, + } + } +} diff --git a/crates/sail-iceberg/src/spec/manifest/metadata.rs b/crates/sail-iceberg/src/spec/manifest/metadata.rs new file mode 100644 index 0000000000..3f24f526d5 --- /dev/null +++ b/crates/sail-iceberg/src/spec/manifest/metadata.rs @@ -0,0 +1,113 @@ +// 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. + +// [CREDIT]: https://raw.githubusercontent.com/apache/iceberg-rust/dc349284a4204c1a56af47fb3177ace6f9e899a0/crates/iceberg/src/spec/manifest/metadata.rs + +use serde::{Deserialize, Serialize}; + +use crate::spec::{ + FormatVersion, ManifestContentType, PartitionSpec, Schema as IcebergSchema, SchemaId, SchemaRef, +}; + +/// Metadata about a manifest file. +#[derive(Debug, PartialEq, Eq, Clone, Serialize, Deserialize)] +pub struct ManifestMetadata { + pub schema: SchemaRef, + pub schema_id: SchemaId, + pub partition_spec: PartitionSpec, + pub format_version: FormatVersion, + pub content: ManifestContentType, +} + +impl ManifestMetadata { + pub fn new( + schema: SchemaRef, + schema_id: SchemaId, + partition_spec: PartitionSpec, + format_version: FormatVersion, + content: ManifestContentType, + ) -> Self { + Self { + schema, + schema_id, + partition_spec, + format_version, + content, + } + } + + pub(crate) fn parse_from_avro_meta( + meta: &std::collections::HashMap>, + ) -> Result { + // schema + let schema_bs = meta + .get("schema") + .ok_or_else(|| "schema is required in manifest metadata but not found".to_string())?; + let schema: IcebergSchema = serde_json::from_slice(schema_bs) + .map_err(|e| format!("Fail to parse schema in manifest metadata: {e}"))?; + let schema_ref = std::sync::Arc::new(schema); + + // schema-id (optional) + let schema_id: i32 = meta + .get("schema-id") + .and_then(|bs| String::from_utf8(bs.clone()).ok()) + .and_then(|s| s.parse::().ok()) + .unwrap_or(0); + + // partition-spec and id + let part_fields_bs = meta.get("partition-spec").ok_or_else(|| { + "partition-spec is required in manifest metadata but not found".to_string() + })?; + let part_fields: Vec = + serde_json::from_slice(part_fields_bs) + .map_err(|e| format!("Fail to parse partition spec in manifest metadata: {e}"))?; + let spec_id: i32 = meta + .get("partition-spec-id") + .and_then(|bs| String::from_utf8(bs.clone()).ok()) + .and_then(|s| s.parse::().ok()) + .unwrap_or(0); + let mut builder = crate::spec::partition::PartitionSpec::builder().with_spec_id(spec_id); + for f in part_fields { + builder = builder.add_field_with_id(f.source_id, f.field_id, f.name, f.transform); + } + let partition_spec = builder.build(); + + // format-version + let format_version = meta + .get("format-version") + .and_then(|bs| serde_json::from_slice::(bs).ok()) + .unwrap_or(crate::spec::FormatVersion::V1); + + // content + let content = meta + .get("content") + .and_then(|bs| String::from_utf8(bs.clone()).ok()) + .map(|s| match s.to_ascii_lowercase().as_str() { + "deletes" => crate::spec::manifest_list::ManifestContentType::Deletes, + _ => crate::spec::manifest_list::ManifestContentType::Data, + }) + .unwrap_or(crate::spec::manifest_list::ManifestContentType::Data); + + Ok(ManifestMetadata::new( + schema_ref, + schema_id, + partition_spec, + format_version, + content, + )) + } +} diff --git a/crates/sail-iceberg/src/spec/manifest/mod.rs b/crates/sail-iceberg/src/spec/manifest/mod.rs new file mode 100644 index 0000000000..b9ec7faf03 --- /dev/null +++ b/crates/sail-iceberg/src/spec/manifest/mod.rs @@ -0,0 +1,350 @@ +// 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. + +// [CREDIT]: https://raw.githubusercontent.com/apache/iceberg-rust/dc349284a4204c1a56af47fb3177ace6f9e899a0/crates/iceberg/src/spec/manifest/mod.rs + +use std::sync::Arc; + +use apache_avro::{from_value as avro_from_value, Reader as AvroReader}; + +use super::Schema; + +mod _serde; +mod data_file; +mod entry; +mod metadata; + +pub use data_file::*; +pub use entry::*; +pub use metadata::*; + +/// Reference to [`ManifestEntry`]. +pub type ManifestEntryRef = Arc; + +/// A manifest contains metadata and a list of entries. +#[derive(Debug, PartialEq, Eq, Clone)] +pub struct Manifest { + /// Metadata about the manifest. + pub metadata: ManifestMetadata, + /// Entries in the manifest. + pub entries: Vec, +} + +impl Manifest { + /// Create a new manifest. + pub fn new(metadata: ManifestMetadata, entries: Vec) -> Self { + Self { + metadata, + entries: entries.into_iter().map(Arc::new).collect(), + } + } + + /// Get the entries in the manifest. + pub fn entries(&self) -> &[ManifestEntryRef] { + &self.entries + } + + /// Get the metadata of the manifest. + pub fn metadata(&self) -> &ManifestMetadata { + &self.metadata + } + + /// Consume this Manifest, returning its constituent parts + pub fn into_parts(self) -> (Vec, ManifestMetadata) { + let Self { entries, metadata } = self; + (entries, metadata) + } + + /// Parse manifest metadata and entries from bytes of avro file. + pub(crate) fn try_from_avro_bytes( + bs: &[u8], + ) -> Result<(ManifestMetadata, Vec), String> { + let reader = AvroReader::new(bs).map_err(|e| format!("Avro read error: {e}"))?; + + // Parse manifest metadata from avro user metadata + let meta = reader.user_metadata(); + let metadata = ManifestMetadata::parse_from_avro_meta(meta)?; + + // Determine partition type to guide value decoding when needed + let partition_type = metadata + .partition_spec + .partition_type(&metadata.schema) + .map_err(|e| format!("Partition type error: {e}"))?; + + // For entries, reuse the embedded schema in the Avro file and deserialize per record + let reader = AvroReader::new(bs).map_err(|e| format!("Avro read error: {e}"))?; + let mut entries = Vec::new(); + for value in reader { + let value = value.map_err(|e| format!("Avro read value error: {e}"))?; + let entry_avro: _serde::ManifestEntryAvro = + avro_from_value(&value).map_err(|e| format!("Avro decode entry error: {e}"))?; + let data_file = entry_avro.data_file.into_data_file( + &metadata.schema, + partition_type.fields().len() as i32, + metadata.partition_spec.spec_id(), + ); + let status = match entry_avro.status { + 1 => ManifestStatus::Added, + 2 => ManifestStatus::Deleted, + _ => ManifestStatus::Existing, + }; + let entry = ManifestEntry::new( + status, + entry_avro.snapshot_id, + entry_avro.sequence_number, + entry_avro.file_sequence_number, + data_file, + ); + entries.push(entry); + } + + Ok((metadata, entries)) + } + + /// Parse a manifest from bytes of avro file. + pub fn parse_avro(bs: &[u8]) -> Result { + let (metadata, entries) = Self::try_from_avro_bytes(bs)?; + Ok(Manifest::new(metadata, entries)) + } +} + +// Helper functions used by Avro serde to parse partition values and bounds +use crate::spec::types::values::{Literal, PrimitiveLiteral}; +use crate::spec::types::Type; +use crate::spec::{Datum, PrimitiveType}; + +pub(super) fn parse_partition_values(json: Option<&serde_json::Value>) -> Vec> { + match json { + Some(serde_json::Value::Array(arr)) => arr + .iter() + .map(|v| match v { + serde_json::Value::Null => None, + serde_json::Value::Bool(b) => { + Some(Literal::Primitive(PrimitiveLiteral::Boolean(*b))) + } + serde_json::Value::Number(n) => { + if let Some(i) = n.as_i64() { + if i >= i32::MIN as i64 && i <= i32::MAX as i64 { + Some(Literal::Primitive(PrimitiveLiteral::Int(i as i32))) + } else { + Some(Literal::Primitive(PrimitiveLiteral::Long(i))) + } + } else { + n.as_f64().map(|f| { + Literal::Primitive(PrimitiveLiteral::Double( + ordered_float::OrderedFloat(f), + )) + }) + } + } + serde_json::Value::String(s) => { + Some(Literal::Primitive(PrimitiveLiteral::String(s.clone()))) + } + _ => None, + }) + .collect(), + _ => Vec::new(), + } +} + +pub(super) fn parse_i64_map_from_avro( + values: &Option, +) -> std::collections::HashMap { + use apache_avro::types::Value; + let mut map = std::collections::HashMap::new(); + if let Some(Value::Map(obj)) = values { + for (k, v) in obj { + if let Value::Long(i) = v { + map.insert(k.parse::().unwrap_or(0), *i); + } + } + return map; + } + if let Some(Value::Array(vec)) = values { + for item in vec { + if let Value::Record(fields) = item { + let mut key_opt = None; + let mut value_opt = None; + for (name, val) in fields { + match name.as_str() { + "key" => { + if let Value::Int(k) = val { + key_opt = Some(*k); + } + } + "value" => { + if let Value::Long(vl) = val { + value_opt = Some(*vl); + } + } + _ => {} + } + } + if let (Some(k), Some(v)) = (key_opt, value_opt) { + map.insert(k, v); + } + } + } + } + map +} + +pub(super) fn parse_bytes_map_from_avro( + values: &Option, +) -> Option>> { + use apache_avro::types::Value; + if let Some(Value::Map(obj)) = values { + let mut map = std::collections::HashMap::new(); + for (k, v) in obj { + if let Value::Bytes(b) = v { + map.insert(k.parse::().unwrap_or(0), b.clone()); + } + } + return Some(map); + } + if let Some(Value::Array(vec)) = values { + let mut map = std::collections::HashMap::new(); + for item in vec { + if let Value::Record(fields) = item { + let mut key_opt = None; + let mut value_opt = None; + for (name, val) in fields { + match name.as_str() { + "key" => { + if let Value::Int(k) = val { + key_opt = Some(*k); + } + } + "value" => { + if let Value::Bytes(b) = val { + value_opt = Some(b.clone()); + } + } + _ => {} + } + } + if let (Some(k), Some(v)) = (key_opt, value_opt) { + map.insert(k, v); + } + } + } + return Some(map); + } + None +} + +pub(super) fn parse_bounds_from_binary( + bounds_data: Option<&std::collections::HashMap>>, + schema: &Schema, +) -> std::collections::HashMap { + let mut bounds = std::collections::HashMap::new(); + if let Some(data) = bounds_data { + for (field_id, binary_data) in data { + if let Some(field) = schema.field_by_id(*field_id) { + let field_type = field.field_type.as_ref(); + let datum = match field_type { + Type::Primitive(prim_type) => { + parse_primitive_bound(binary_data, prim_type).ok() + } + _ => None, + }; + if let Some(d) = datum { + bounds.insert(*field_id, d); + } + } else if let Ok(string_value) = String::from_utf8(binary_data.clone()) { + bounds.insert( + *field_id, + Datum::new( + PrimitiveType::String, + PrimitiveLiteral::String(string_value), + ), + ); + } else { + bounds.insert( + *field_id, + Datum::new( + PrimitiveType::Binary, + PrimitiveLiteral::Binary(binary_data.clone()), + ), + ); + } + } + } + bounds +} + +fn parse_primitive_bound(bytes: &[u8], prim_type: &PrimitiveType) -> Result { + use num_bigint::BigInt; + use num_traits::ToPrimitive; + + let literal = match prim_type { + PrimitiveType::Boolean => { + let val = !(bytes.len() == 1 && bytes[0] == 0u8); + PrimitiveLiteral::Boolean(val) + } + PrimitiveType::Int | PrimitiveType::Date => { + let val = i32::from_le_bytes(bytes.try_into().map_err(|_| "Invalid i32 bytes")?); + PrimitiveLiteral::Int(val) + } + PrimitiveType::Long + | PrimitiveType::Time + | PrimitiveType::Timestamp + | PrimitiveType::Timestamptz + | PrimitiveType::TimestampNs + | PrimitiveType::TimestamptzNs => { + let val = if bytes.len() == 4 { + i32::from_le_bytes(bytes.try_into().map_err(|_| "Invalid i32 bytes")?) as i64 + } else { + i64::from_le_bytes(bytes.try_into().map_err(|_| "Invalid i64 bytes")?) + }; + PrimitiveLiteral::Long(val) + } + PrimitiveType::Float => { + let val = f32::from_le_bytes(bytes.try_into().map_err(|_| "Invalid f32 bytes")?); + PrimitiveLiteral::Float(ordered_float::OrderedFloat(val)) + } + PrimitiveType::Double => { + let val = if bytes.len() == 4 { + f32::from_le_bytes(bytes.try_into().map_err(|_| "Invalid f32 bytes")?) as f64 + } else { + f64::from_le_bytes(bytes.try_into().map_err(|_| "Invalid f64 bytes")?) + }; + PrimitiveLiteral::Double(ordered_float::OrderedFloat(val)) + } + PrimitiveType::String => { + let val = std::str::from_utf8(bytes) + .map_err(|_| "Invalid UTF-8")? + .to_string(); + PrimitiveLiteral::String(val) + } + PrimitiveType::Uuid => { + let val = u128::from_be_bytes(bytes.try_into().map_err(|_| "Invalid UUID bytes")?); + PrimitiveLiteral::UInt128(val) + } + PrimitiveType::Fixed(_) | PrimitiveType::Binary => { + PrimitiveLiteral::Binary(Vec::from(bytes)) + } + PrimitiveType::Decimal { .. } => { + let unscaled_value = BigInt::from_signed_bytes_be(bytes); + let val = unscaled_value + .to_i128() + .ok_or_else(|| format!("Can't convert bytes to i128: {:?}", bytes))?; + PrimitiveLiteral::Int128(val) + } + }; + Ok(Datum::new(prim_type.clone(), literal)) +} diff --git a/crates/sail-iceberg/src/spec/manifest/writer.rs b/crates/sail-iceberg/src/spec/manifest/writer.rs new file mode 100644 index 0000000000..00e99901ae --- /dev/null +++ b/crates/sail-iceberg/src/spec/manifest/writer.rs @@ -0,0 +1,23 @@ +// 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. + +// [CREDIT]: https://raw.githubusercontent.com/apache/iceberg-rust/dc349284a4204c1a56af47fb3177ace6f9e899a0/crates/iceberg/src/spec/manifest/writer.rs + +// TODO: Implement manifest writer +#[allow(dead_code)] +#[derive(Debug, Clone)] +pub struct ManifestWriter; diff --git a/crates/sail-iceberg/src/spec/manifest_list.rs b/crates/sail-iceberg/src/spec/manifest_list.rs new file mode 100644 index 0000000000..80253da145 --- /dev/null +++ b/crates/sail-iceberg/src/spec/manifest_list.rs @@ -0,0 +1,690 @@ +// 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. + +// [CREDIT]: https://raw.githubusercontent.com/apache/iceberg-rust/dc349284a4204c1a56af47fb3177ace6f9e899a0/crates/iceberg/src/spec/manifest_list.rs + +use apache_avro::types::Value as AvroValue; +use apache_avro::{from_value as avro_from_value, Reader as AvroReader}; +use serde::{Deserialize, Serialize}; + +use crate::spec::FormatVersion; + +pub const UNASSIGNED_SEQUENCE_NUMBER: i64 = -1; + +/// Snapshots are embedded in table metadata, but the list of manifests for a +/// snapshot are stored in a separate manifest list file. +/// +/// A new manifest list is written for each attempt to commit a snapshot +/// because the list of manifests always changes to produce a new snapshot. +/// When a manifest list is written, the (optimistic) sequence number of the +/// snapshot is written for all new manifest files tracked by the list. +/// +/// A manifest list includes summary metadata that can be used to avoid +/// scanning all of the manifests in a snapshot when planning a table scan. +/// This includes the number of added, existing, and deleted files, and a +/// summary of values for each field of the partition spec used to write the +/// manifest. +#[derive(Debug, Clone, PartialEq, Serialize, Deserialize)] +pub struct ManifestList { + /// Entries in a manifest list. + pub entries: Vec, +} + +impl ManifestList { + /// Create a new manifest list. + pub fn new(entries: Vec) -> Self { + Self { entries } + } + + /// Get the entries in the manifest list. + pub fn entries(&self) -> &[ManifestFile] { + &self.entries + } + + /// Take ownership of the entries in the manifest list, consuming it + pub fn into_entries(self) -> Vec { + self.entries + } + + /// Parse manifest list from bytes with a specified version. + pub fn parse_with_version(bs: &[u8], version: FormatVersion) -> Result { + match version { + FormatVersion::V1 => { + let reader = AvroReader::new(bs).map_err(|e| format!("Avro read error: {e}"))?; + let mut entries = Vec::new(); + for value in reader { + let value = value.map_err(|e| format!("Avro read value error: {e}"))?; + let v1: _serde::ManifestFileV1 = + avro_from_value(&value).map_err(|e| format!("Avro decode error: {e}"))?; + entries.push(ManifestFile::from(v1)); + } + Ok(ManifestList::new(entries)) + } + FormatVersion::V2 => { + let reader = AvroReader::new(bs).map_err(|e| format!("Avro read error: {e}"))?; + let mut entries = Vec::new(); + for value in reader { + let value = value.map_err(|e| format!("Avro read value error: {e}"))?; + match avro_from_value::<_serde::ManifestFileV2>(&value) { + Ok(v2) => entries.push(ManifestFile::from(v2)), + Err(_) => { + if let Ok(mf) = Self::parse_manifest_v2_fallback(&value) { + entries.push(mf); + } else { + let err = format!("Avro decode error: Failed to deserialize Avro value into value: {value:?}"); + return Err(err); + } + } + } + } + Ok(ManifestList::new(entries)) + } + } + } +} + +// removed duplicate early _serde block; see single _serde module below + +#[derive(Debug, Serialize, Deserialize, Clone)] +#[serde(rename_all = "kebab-case")] +pub struct FieldSummaryAvro { + #[serde(rename = "contains_null")] + contains_null: bool, + #[serde(rename = "contains_nan")] + contains_nan: Option, + #[serde(rename = "lower_bound")] + lower_bound: Option>, + #[serde(rename = "upper_bound")] + upper_bound: Option>, +} + +impl From for FieldSummary { + fn from(summary: FieldSummaryAvro) -> Self { + let mut field_summary = FieldSummary::new(summary.contains_null); + if let Some(contains_nan) = summary.contains_nan { + field_summary = field_summary.with_contains_nan(contains_nan); + } + field_summary.lower_bound_bytes = summary.lower_bound; + field_summary.upper_bound_bytes = summary.upper_bound; + field_summary + } +} + +impl ManifestList { + fn parse_manifest_v2_fallback(value: &AvroValue) -> Result { + match value { + AvroValue::Record(fields) => { + let get = |name: &str| -> Option<&AvroValue> { + fields.iter().find(|(k, _)| k == name).map(|(_, v)| v) + }; + + let string = |v: &AvroValue| -> Result { + if let AvroValue::String(s) = v { + Ok(s.clone()) + } else { + Err("string".into()) + } + }; + let long = |v: &AvroValue| -> Result { + if let AvroValue::Long(x) = v { + Ok(*x) + } else { + Err("long".into()) + } + }; + let int = |v: &AvroValue| -> Result { + if let AvroValue::Int(x) = v { + Ok(*x) + } else { + Err("int".into()) + } + }; + + let manifest_path = string(get("manifest_path").ok_or("manifest_path")?)?; + let manifest_length = long(get("manifest_length").ok_or("manifest_length")?)?; + let partition_spec_id = int(get("partition_spec_id").ok_or("partition_spec_id")?)?; + let content = int(get("content").unwrap_or(&AvroValue::Int(0)))?; + let sequence_number = long(get("sequence_number").ok_or("sequence_number")?)?; + let min_sequence_number = + long(get("min_sequence_number").ok_or("min_sequence_number")?)?; + let added_snapshot_id = long(get("added_snapshot_id").ok_or("added_snapshot_id")?)?; + let added_files_count = get("added_files_count") + .or_else(|| get("added_data_files_count")) + .and_then(|v| { + if let AvroValue::Int(x) = v { + Some(*x) + } else { + None + } + }) + .unwrap_or(0); + let existing_files_count = get("existing_files_count") + .or_else(|| get("existing_data_files_count")) + .and_then(|v| { + if let AvroValue::Int(x) = v { + Some(*x) + } else { + None + } + }) + .unwrap_or(0); + let deleted_files_count = get("deleted_files_count") + .or_else(|| get("deleted_data_files_count")) + .and_then(|v| { + if let AvroValue::Int(x) = v { + Some(*x) + } else { + None + } + }) + .unwrap_or(0); + let added_rows_count = long(get("added_rows_count").ok_or("added_rows_count")?)?; + let existing_rows_count = + long(get("existing_rows_count").ok_or("existing_rows_count")?)?; + let deleted_rows_count = + long(get("deleted_rows_count").ok_or("deleted_rows_count")?)?; + + let partitions = match get("partitions") { + Some(AvroValue::Union(_, inner)) => match inner.as_ref() { + AvroValue::Array(items) => { + let mut out = Vec::new(); + for it in items { + if let AvroValue::Record(fs) = it { + let getf = + |n: &str| fs.iter().find(|(k, _)| k == n).map(|(_, v)| v); + let contains_null = matches!(getf("contains_null"), Some(AvroValue::Boolean(b)) if *b); + let contains_nan = match getf("contains_nan") { + Some(AvroValue::Boolean(b)) => Some(*b), + _ => None, + }; + let lower_bound_bytes = match getf("lower_bound") { + Some(AvroValue::Bytes(b)) => Some(b.clone()), + _ => None, + }; + let upper_bound_bytes = match getf("upper_bound") { + Some(AvroValue::Bytes(b)) => Some(b.clone()), + _ => None, + }; + let mut fs = FieldSummary::new(contains_null); + if let Some(b) = contains_nan { + fs = fs.with_contains_nan(b); + } + fs.lower_bound_bytes = lower_bound_bytes; + fs.upper_bound_bytes = upper_bound_bytes; + out.push(fs); + } + } + Some(out) + } + AvroValue::Null => None, + _ => None, + }, + // Some writers may encode unexpected types; ignore invalid values + _ => None, + }; + + let key_metadata = match get("key_metadata") { + Some(AvroValue::Union(_, inner)) => match inner.as_ref() { + AvroValue::Bytes(b) => Some(b.clone()), + AvroValue::Null => None, + _ => None, + }, + Some(AvroValue::Bytes(b)) => Some(b.clone()), + _ => None, + }; + + let content = match content { + 0 => ManifestContentType::Data, + 1 => ManifestContentType::Deletes, + _ => ManifestContentType::Data, + }; + + Ok(ManifestFile { + manifest_path, + manifest_length, + partition_spec_id, + content, + sequence_number, + min_sequence_number, + added_snapshot_id, + added_files_count: Some(added_files_count), + existing_files_count: Some(existing_files_count), + deleted_files_count: Some(deleted_files_count), + added_rows_count: Some(added_rows_count), + existing_rows_count: Some(existing_rows_count), + deleted_rows_count: Some(deleted_rows_count), + partitions, + key_metadata, + }) + } + _ => Err("not a record".into()), + } + } +} + +impl From<_serde::ManifestFileV2> for ManifestFile { + fn from(avro: _serde::ManifestFileV2) -> Self { + let content = match avro.content { + 0 => ManifestContentType::Data, + 1 => ManifestContentType::Deletes, + _ => ManifestContentType::Data, + }; + + let partitions = avro + .partitions + .map(|summaries| summaries.into_iter().map(FieldSummary::from).collect()); + + ManifestFile { + manifest_path: avro.manifest_path, + manifest_length: avro.manifest_length, + partition_spec_id: avro.partition_spec_id, + content, + sequence_number: avro.sequence_number, + min_sequence_number: avro.min_sequence_number, + added_snapshot_id: avro.added_snapshot_id, + added_files_count: Some(avro.added_files_count), + existing_files_count: Some(avro.existing_files_count), + deleted_files_count: Some(avro.deleted_files_count), + added_rows_count: Some(avro.added_rows_count), + existing_rows_count: Some(avro.existing_rows_count), + deleted_rows_count: Some(avro.deleted_rows_count), + partitions, + key_metadata: avro.key_metadata, + } + } +} + +/// Status of a manifest file in a manifest list. +#[derive(Debug, Clone, PartialEq, Eq, Serialize, Deserialize)] +#[serde(rename_all = "lowercase")] +pub enum ManifestFileStatus { + /// The manifest file was added in this snapshot. + Added, + /// The manifest file was inherited from the parent snapshot. + Existing, + /// The manifest file was deleted in this snapshot. + Deleted, +} + +/// Content type of a manifest file. +#[derive(Debug, Clone, PartialEq, Eq, Serialize, Deserialize)] +#[serde(rename_all = "lowercase")] +pub enum ManifestContentType { + /// The manifest contains data files. + Data, + /// The manifest contains delete files. + Deletes, +} + +/// A manifest file in a manifest list. +#[derive(Debug, Clone, PartialEq, Serialize, Deserialize)] +#[serde(rename_all = "kebab-case")] +pub struct ManifestFile { + /// The path to the manifest file. + pub manifest_path: String, + /// The length of the manifest file in bytes. + pub manifest_length: i64, + /// The ID of the partition spec used to write the manifest. + pub partition_spec_id: i32, + /// The content type of the manifest file. + pub content: ManifestContentType, + /// The sequence number when the manifest was added to the table. + pub sequence_number: i64, + /// The minimum sequence number of all data files in the manifest. + pub min_sequence_number: i64, + /// The snapshot ID when the manifest was added to the table. + pub added_snapshot_id: i64, + /// The number of files added in this manifest. + #[serde(skip_serializing_if = "Option::is_none")] + pub added_files_count: Option, + /// The number of existing files in this manifest. + #[serde(skip_serializing_if = "Option::is_none")] + pub existing_files_count: Option, + /// The number of deleted files in this manifest. + #[serde(skip_serializing_if = "Option::is_none")] + pub deleted_files_count: Option, + /// The number of rows added in this manifest. + #[serde(skip_serializing_if = "Option::is_none")] + pub added_rows_count: Option, + /// The number of existing rows in this manifest. + #[serde(skip_serializing_if = "Option::is_none")] + pub existing_rows_count: Option, + /// The number of deleted rows in this manifest. + #[serde(skip_serializing_if = "Option::is_none")] + pub deleted_rows_count: Option, + /// A list of field summaries for each partition field in the spec. + /// Each field in the list corresponds to a field in the manifest file's partition spec. + #[serde(skip_serializing_if = "Option::is_none")] + pub partitions: Option>, + /// Implementation-specific key metadata for encryption. + #[serde(skip_serializing_if = "Option::is_none")] + pub key_metadata: Option>, +} + +impl ManifestFile { + /// Create a new manifest file builder. + pub fn builder() -> ManifestFileBuilder { + ManifestFileBuilder::new() + } + + /// Get the total number of files in this manifest. + pub fn total_files_count(&self) -> i32 { + self.added_files_count.unwrap_or(0) + + self.existing_files_count.unwrap_or(0) + + self.deleted_files_count.unwrap_or(0) + } + + /// Get the total number of rows in this manifest. + pub fn total_rows_count(&self) -> i64 { + self.added_rows_count.unwrap_or(0) + + self.existing_rows_count.unwrap_or(0) + + self.deleted_rows_count.unwrap_or(0) + } + + /// Whether the manifest contains any added files + pub fn has_added_files(&self) -> bool { + self.added_files_count.unwrap_or(0) > 0 + } + + /// Whether the manifest contains any deleted files + pub fn has_deleted_files(&self) -> bool { + self.deleted_files_count.unwrap_or(0) > 0 + } +} + +/// Field summary for partition fields in a manifest file. +#[derive(Debug, Clone, PartialEq, Serialize, Deserialize)] +#[serde(rename_all = "kebab-case")] +pub struct FieldSummary { + /// Whether the partition field contains null values. + pub contains_null: bool, + /// Whether the partition field contains NaN values (only for float and double). + #[serde(skip_serializing_if = "Option::is_none")] + pub contains_nan: Option, + /// The minimum value of the partition field (binary encoded per spec). + #[serde(skip_serializing_if = "Option::is_none")] + pub lower_bound_bytes: Option>, + /// The maximum value of the partition field (binary encoded per spec). + #[serde(skip_serializing_if = "Option::is_none")] + pub upper_bound_bytes: Option>, +} + +impl FieldSummary { + /// Create a new field summary. + pub fn new(contains_null: bool) -> Self { + Self { + contains_null, + contains_nan: None, + lower_bound_bytes: None, + upper_bound_bytes: None, + } + } + + /// Set whether the field contains NaN values. + pub fn with_contains_nan(mut self, contains_nan: bool) -> Self { + self.contains_nan = Some(contains_nan); + self + } + + /// Set the lower bound of the field. + pub fn with_lower_bound_bytes(mut self, lower: Vec) -> Self { + self.lower_bound_bytes = Some(lower); + self + } + + /// Set the upper bound of the field. + pub fn with_upper_bound_bytes(mut self, upper: Vec) -> Self { + self.upper_bound_bytes = Some(upper); + self + } +} + +/// Builder for creating manifest files. +#[derive(Debug)] +pub struct ManifestFileBuilder { + manifest_path: Option, + manifest_length: i64, + partition_spec_id: i32, + content: ManifestContentType, + sequence_number: i64, + min_sequence_number: i64, + added_snapshot_id: i64, + added_files_count: Option, + existing_files_count: Option, + deleted_files_count: Option, + added_rows_count: Option, + existing_rows_count: Option, + deleted_rows_count: Option, + partitions: Option>, + key_metadata: Option>, +} + +impl ManifestFileBuilder { + /// Create a new manifest file builder. + pub fn new() -> Self { + Self { + manifest_path: None, + manifest_length: 0, + partition_spec_id: 0, + content: ManifestContentType::Data, + sequence_number: UNASSIGNED_SEQUENCE_NUMBER, + min_sequence_number: UNASSIGNED_SEQUENCE_NUMBER, + added_snapshot_id: 0, + added_files_count: None, + existing_files_count: None, + deleted_files_count: None, + added_rows_count: None, + existing_rows_count: None, + deleted_rows_count: None, + partitions: None, + key_metadata: None, + } + } + + /// Set the manifest path. + pub fn with_manifest_path(mut self, manifest_path: impl ToString) -> Self { + self.manifest_path = Some(manifest_path.to_string()); + self + } + + /// Set the manifest length. + pub fn with_manifest_length(mut self, manifest_length: i64) -> Self { + self.manifest_length = manifest_length; + self + } + + /// Set the partition spec id. + pub fn with_partition_spec_id(mut self, partition_spec_id: i32) -> Self { + self.partition_spec_id = partition_spec_id; + self + } + + /// Set the content type. + pub fn with_content(mut self, content: ManifestContentType) -> Self { + self.content = content; + self + } + + /// Set the sequence number. + pub fn with_sequence_number(mut self, sequence_number: i64) -> Self { + self.sequence_number = sequence_number; + self + } + + /// Set the minimum sequence number. + pub fn with_min_sequence_number(mut self, min_sequence_number: i64) -> Self { + self.min_sequence_number = min_sequence_number; + self + } + + /// Set the added snapshot id. + pub fn with_added_snapshot_id(mut self, added_snapshot_id: i64) -> Self { + self.added_snapshot_id = added_snapshot_id; + self + } + + /// Set the file counts. + pub fn with_file_counts(mut self, added: i32, existing: i32, deleted: i32) -> Self { + self.added_files_count = Some(added); + self.existing_files_count = Some(existing); + self.deleted_files_count = Some(deleted); + self + } + + /// Set the row counts. + pub fn with_row_counts(mut self, added: i64, existing: i64, deleted: i64) -> Self { + self.added_rows_count = Some(added); + self.existing_rows_count = Some(existing); + self.deleted_rows_count = Some(deleted); + self + } + + /// Set the partitions. + pub fn with_partitions(mut self, partitions: Vec) -> Self { + self.partitions = Some(partitions); + self + } + + /// Set the key metadata. + pub fn with_key_metadata(mut self, key_metadata: Vec) -> Self { + self.key_metadata = Some(key_metadata); + self + } + + /// Build the manifest file. + pub fn build(self) -> Result { + let manifest_path = self.manifest_path.ok_or("manifest_path is required")?; + + Ok(ManifestFile { + manifest_path, + manifest_length: self.manifest_length, + partition_spec_id: self.partition_spec_id, + content: self.content, + sequence_number: self.sequence_number, + min_sequence_number: self.min_sequence_number, + added_snapshot_id: self.added_snapshot_id, + added_files_count: self.added_files_count, + existing_files_count: self.existing_files_count, + deleted_files_count: self.deleted_files_count, + added_rows_count: self.added_rows_count, + existing_rows_count: self.existing_rows_count, + deleted_rows_count: self.deleted_rows_count, + partitions: self.partitions, + key_metadata: self.key_metadata, + }) + } +} + +impl Default for ManifestFileBuilder { + fn default() -> Self { + Self::new() + } +} + +pub(super) mod _serde { + use serde::{Deserialize, Serialize}; + + use super::*; + + #[derive(Debug, Serialize, Deserialize)] + #[serde(rename_all = "kebab-case")] + pub struct ManifestFileV1 { + #[serde(rename = "manifest_path")] + pub manifest_path: String, + #[serde(rename = "manifest_length")] + pub manifest_length: i64, + #[serde(rename = "partition_spec_id")] + pub partition_spec_id: i32, + #[serde(rename = "added_snapshot_id")] + pub added_snapshot_id: i64, + #[serde(rename = "added_data_files_count")] + pub added_data_files_count: Option, + #[serde(rename = "existing_data_files_count")] + pub existing_data_files_count: Option, + #[serde(rename = "deleted_data_files_count")] + pub deleted_data_files_count: Option, + #[serde(rename = "added_rows_count")] + pub added_rows_count: Option, + #[serde(rename = "existing_rows_count")] + pub existing_rows_count: Option, + #[serde(rename = "deleted_rows_count")] + pub deleted_rows_count: Option, + #[serde(rename = "partitions")] + pub partitions: Option>, // V1 uses same summary encoding + #[serde(rename = "key_metadata")] + pub key_metadata: Option>, + } + + #[derive(Debug, Serialize, Deserialize)] + #[serde(rename_all = "kebab-case")] + pub struct ManifestFileV2 { + #[serde(rename = "manifest_path")] + pub manifest_path: String, + #[serde(rename = "manifest_length")] + pub manifest_length: i64, + #[serde(rename = "partition_spec_id")] + pub partition_spec_id: i32, + #[serde(rename = "content")] + pub content: i32, + #[serde(rename = "sequence_number")] + pub sequence_number: i64, + #[serde(rename = "min_sequence_number")] + pub min_sequence_number: i64, + #[serde(rename = "added_snapshot_id")] + pub added_snapshot_id: i64, + #[serde(rename = "added_files_count", alias = "added_data_files_count")] + pub added_files_count: i32, + #[serde(rename = "existing_files_count", alias = "existing_data_files_count")] + pub existing_files_count: i32, + #[serde(rename = "deleted_files_count", alias = "deleted_data_files_count")] + pub deleted_files_count: i32, + #[serde(rename = "added_rows_count")] + pub added_rows_count: i64, + #[serde(rename = "existing_rows_count")] + pub existing_rows_count: i64, + #[serde(rename = "deleted_rows_count")] + pub deleted_rows_count: i64, + #[serde(rename = "partitions")] + pub partitions: Option>, + #[serde(rename = "key_metadata")] + pub key_metadata: Option>, + } +} + +impl From<_serde::ManifestFileV1> for ManifestFile { + fn from(v1: _serde::ManifestFileV1) -> Self { + ManifestFile { + manifest_path: v1.manifest_path, + manifest_length: v1.manifest_length, + partition_spec_id: v1.partition_spec_id, + content: ManifestContentType::Data, + sequence_number: 0, + min_sequence_number: 0, + added_snapshot_id: v1.added_snapshot_id, + added_files_count: v1.added_data_files_count, + existing_files_count: v1.existing_data_files_count, + deleted_files_count: v1.deleted_data_files_count, + added_rows_count: v1.added_rows_count, + existing_rows_count: v1.existing_rows_count, + deleted_rows_count: v1.deleted_rows_count, + partitions: v1 + .partitions + .map(|v| v.into_iter().map(FieldSummary::from).collect()), + key_metadata: v1.key_metadata, + } + } +} diff --git a/crates/sail-iceberg/src/spec/metadata/format.rs b/crates/sail-iceberg/src/spec/metadata/format.rs new file mode 100644 index 0000000000..3ed4848284 --- /dev/null +++ b/crates/sail-iceberg/src/spec/metadata/format.rs @@ -0,0 +1,59 @@ +// 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. + +// [CREDIT]: https://raw.githubusercontent.com/apache/iceberg-rust/dc349284a4204c1a56af47fb3177ace6f9e899a0/crates/iceberg/src/spec/table_metadata.rs + +/// Format version of Iceberg. +#[derive(Debug, PartialEq, Eq, Clone, Copy)] +pub enum FormatVersion { + /// Version 1 + V1 = 1, + /// Version 2 + V2 = 2, +} + +impl serde::Serialize for FormatVersion { + fn serialize(&self, serializer: S) -> Result + where + S: serde::Serializer, + { + serializer.serialize_i32(*self as i32) + } +} + +impl<'de> serde::Deserialize<'de> for FormatVersion { + fn deserialize(deserializer: D) -> Result + where + D: serde::Deserializer<'de>, + { + let value = i32::deserialize(deserializer)?; + match value { + 1 => Ok(FormatVersion::V1), + 2 => Ok(FormatVersion::V2), + _ => Err(serde::de::Error::custom(format!( + "Invalid format version: {}", + value + ))), + } + } +} + +impl Default for FormatVersion { + fn default() -> Self { + Self::V2 + } +} diff --git a/crates/sail-iceberg/src/spec/metadata/mod.rs b/crates/sail-iceberg/src/spec/metadata/mod.rs new file mode 100644 index 0000000000..5f76b8a2e8 --- /dev/null +++ b/crates/sail-iceberg/src/spec/metadata/mod.rs @@ -0,0 +1,26 @@ +// 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. + +pub mod format; +pub mod statistic_file; +pub mod table_metadata; +pub mod table_metadata_builder; + +pub use format::*; +pub use statistic_file::*; +pub use table_metadata::*; +pub use table_metadata_builder::*; diff --git a/crates/sail-iceberg/src/spec/metadata/statistic_file.rs b/crates/sail-iceberg/src/spec/metadata/statistic_file.rs new file mode 100644 index 0000000000..84f4eb7d84 --- /dev/null +++ b/crates/sail-iceberg/src/spec/metadata/statistic_file.rs @@ -0,0 +1,65 @@ +// 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. + +// [CREDIT]: https://raw.githubusercontent.com/apache/iceberg-rust/dc349284a4204c1a56af47fb3177ace6f9e899a0/crates/iceberg/src/spec/statistic_file.rs + +use serde::{Deserialize, Serialize}; + +#[derive(Debug, Clone, Serialize, Deserialize, PartialEq, Eq)] +#[serde(rename_all = "kebab-case")] +pub struct StatisticsFile { + /// The snapshot id of the statistics file. + pub snapshot_id: i64, + /// Path of the statistics file + pub statistics_path: String, + /// File size in bytes + pub file_size_in_bytes: i64, + /// File footer size in bytes + pub file_footer_size_in_bytes: i64, + /// Base64-encoded implementation-specific key metadata for encryption. + #[serde(default, skip_serializing_if = "Option::is_none")] + pub key_metadata: Option, + /// Blob metadata + pub blob_metadata: Vec, +} + +#[derive(Debug, Clone, Serialize, Deserialize, PartialEq, Eq)] +#[serde(rename_all = "kebab-case")] +pub struct BlobMetadata { + /// Type of the blob. + pub r#type: String, + /// Snapshot id of the blob. + pub snapshot_id: i64, + /// Sequence number of the blob. + pub sequence_number: i64, + /// Fields of the blob. + pub fields: Vec, + /// Properties of the blob. + #[serde(default, skip_serializing_if = "std::collections::HashMap::is_empty")] + pub properties: std::collections::HashMap, +} + +#[derive(Debug, Clone, Serialize, Deserialize, PartialEq, Eq)] +#[serde(rename_all = "kebab-case")] +pub struct PartitionStatisticsFile { + /// The snapshot id of the statistics file. + pub snapshot_id: i64, + /// Path of the statistics file + pub statistics_path: String, + /// File size in bytes + pub file_size_in_bytes: i64, +} diff --git a/crates/sail-iceberg/src/spec/metadata/table_metadata.rs b/crates/sail-iceberg/src/spec/metadata/table_metadata.rs new file mode 100644 index 0000000000..f07bbffe23 --- /dev/null +++ b/crates/sail-iceberg/src/spec/metadata/table_metadata.rs @@ -0,0 +1,189 @@ +// 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. + +// [CREDIT]: https://raw.githubusercontent.com/apache/iceberg-rust/dc349284a4204c1a56af47fb3177ace6f9e899a0/crates/iceberg/src/spec/table_metadata.rs + +use std::collections::HashMap; + +use serde::{Deserialize, Serialize}; +use uuid::Uuid; + +use crate::spec::metadata::format::FormatVersion; +use crate::spec::metadata::statistic_file::{PartitionStatisticsFile, StatisticsFile}; +use crate::spec::partition::PartitionSpec; +use crate::spec::schema::Schema; +use crate::spec::snapshots::{Snapshot, SnapshotReference}; +use crate::spec::sort::SortOrder; + +/// Iceberg table metadata +#[derive(Debug, Clone, Serialize, Deserialize)] +#[serde(rename_all = "kebab-case")] +pub struct TableMetadata { + /// Integer Version for the format + pub format_version: FormatVersion, + /// A UUID that identifies the table + pub table_uuid: Option, + /// Location tables base location + pub location: String, + /// The tables highest sequence number + #[serde(default)] + pub last_sequence_number: i64, + /// Timestamp in milliseconds from the unix epoch when the table was last updated + pub last_updated_ms: i64, + /// An integer; the highest assigned column ID for the table + pub last_column_id: i32, + /// A list of schemas, stored as objects with schema-id + pub schemas: Vec, + /// ID of the table's current schema + pub current_schema_id: i32, + /// A list of partition specs, stored as full partition spec objects + #[serde(default)] + pub partition_specs: Vec, + /// ID of the "current" spec that writers should use by default + #[serde(default)] + pub default_spec_id: i32, + /// An integer; the highest assigned partition field ID across all partition specs for the table + #[serde(default)] + pub last_partition_id: i32, + /// A string to string map of table properties + #[serde(default)] + pub properties: HashMap, + /// long ID of the current table snapshot + pub current_snapshot_id: Option, + /// A list of valid snapshots + #[serde(default)] + pub snapshots: Vec, + /// A list of timestamp and snapshot ID pairs that encodes changes to the current snapshot for the table + #[serde(default)] + pub snapshot_log: Vec, + /// A list of timestamp and metadata file location pairs that encodes changes to the previous metadata files for the table + #[serde(default)] + pub metadata_log: Vec, + /// Sort orders for the table + #[serde(default, skip_serializing_if = "Vec::is_empty")] + pub sort_orders: Vec, + /// Default sort order ID + #[serde(default, skip_serializing_if = "Option::is_none")] + pub default_sort_order_id: Option, + /// Named references to snapshots + #[serde(default, skip_serializing_if = "HashMap::is_empty")] + pub refs: HashMap, + /// Statistics files + #[serde(default, skip_serializing_if = "Vec::is_empty")] + pub statistics: Vec, + /// Partition statistics files + #[serde(default, skip_serializing_if = "Vec::is_empty")] + pub partition_statistics: Vec, +} + +/// Snapshot log entry +#[derive(Debug, Clone, Serialize, Deserialize)] +#[serde(rename_all = "kebab-case")] +pub struct SnapshotLog { + /// Timestamp when the snapshot became current + pub timestamp_ms: i64, + /// Snapshot ID + pub snapshot_id: i64, +} + +/// Metadata log entry +#[derive(Debug, Clone, Serialize, Deserialize)] +#[serde(rename_all = "kebab-case")] +pub struct MetadataLog { + /// Timestamp when the metadata file was created + pub timestamp_ms: i64, + /// Location of the metadata file + pub metadata_file: String, +} + +#[derive(Deserialize)] +#[serde(untagged)] +enum TableMetadataEnum { + V1(TableMetadata), + V2(TableMetadata), +} + +impl TableMetadata { + /// Get the current schema + pub fn current_schema(&self) -> Option<&Schema> { + self.schemas + .iter() + .find(|schema| schema.schema_id() == self.current_schema_id) + } + + /// Get the current snapshot + pub fn current_snapshot(&self) -> Option<&Snapshot> { + if let Some(snapshot_id) = self.current_snapshot_id { + self.snapshots + .iter() + .find(|snapshot| snapshot.snapshot_id() == snapshot_id) + } else { + None + } + } + + /// Get the default partition spec + pub fn default_partition_spec(&self) -> Option<&PartitionSpec> { + self.partition_specs + .iter() + .find(|spec| spec.spec_id() == self.default_spec_id) + } + + pub fn from_json(data: &[u8]) -> Result { + log::trace!("Attempting to parse table metadata JSON"); + + match serde_json::from_slice::(data) { + Ok(json_value) => { + if let Some(obj) = json_value.as_object() { + log::trace!("JSON fields present: {:?}", obj.keys().collect::>()); + + if let Some(refs) = obj.get("refs") { + log::trace!("refs field: {:?}", refs); + } + if let Some(sort_orders) = obj.get("sort-orders") { + log::trace!("sort-orders field: {:?}", sort_orders); + } + if let Some(stats) = obj.get("statistics") { + log::trace!("statistics field: {:?}", stats); + } + if let Some(partition_stats) = obj.get("partition-statistics") { + log::trace!("partition-statistics field: {:?}", partition_stats); + } + } + + log::trace!("Deserializing to TableMetadata struct"); + serde_json::from_value::(json_value) + .map_err(|e| { + log::trace!("Failed to deserialize TableMetadata: {:?}", e); + e + }) + .map(|tm| match tm { + TableMetadataEnum::V1(t) | TableMetadataEnum::V2(t) => t, + }) + } + Err(e) => { + log::trace!("Failed to parse as JSON: {:?}", e); + Err(e) + } + } + } + + /// Serialize table metadata to JSON bytes + pub fn to_json(&self) -> Result, serde_json::Error> { + serde_json::to_vec(self) + } +} diff --git a/crates/sail-iceberg/src/spec/metadata/table_metadata_builder.rs b/crates/sail-iceberg/src/spec/metadata/table_metadata_builder.rs new file mode 100644 index 0000000000..8c9e880449 --- /dev/null +++ b/crates/sail-iceberg/src/spec/metadata/table_metadata_builder.rs @@ -0,0 +1,22 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +// [CREDIT]: https://raw.githubusercontent.com/apache/iceberg-rust/dc349284a4204c1a56af47fb3177ace6f9e899a0/crates/iceberg/src/spec/table_metadata_builder.rs + +#[allow(dead_code)] +#[derive(Debug, Clone)] +pub struct TableMetadataBuilder; diff --git a/crates/sail-iceberg/src/spec/mod.rs b/crates/sail-iceberg/src/spec/mod.rs new file mode 100644 index 0000000000..4d95e97f84 --- /dev/null +++ b/crates/sail-iceberg/src/spec/mod.rs @@ -0,0 +1,46 @@ +// 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. + +// [CREDIT]: https://raw.githubusercontent.com/apache/iceberg-rust/dc349284a4204c1a56af47fb3177ace6f9e899a0/crates/iceberg/src/spec/mod.rs + +pub mod catalog; +pub mod encrypted_key; +pub mod manifest; +pub mod manifest_list; +pub mod metadata; +pub mod name_mapping; +pub mod partition; +pub mod schema; +pub mod snapshots; +pub mod sort; +pub mod transform; +pub mod types; +pub mod views; + +pub use catalog::*; +pub use encrypted_key::*; +pub use manifest::*; +pub use manifest_list::*; +pub use metadata::*; +pub use name_mapping::*; +pub use partition::*; +pub use schema::*; +pub use snapshots::*; +pub use sort::*; +pub use transform::*; +pub use types::*; +pub use views::*; diff --git a/crates/sail-iceberg/src/spec/name_mapping/mod.rs b/crates/sail-iceberg/src/spec/name_mapping/mod.rs new file mode 100644 index 0000000000..b9bd9258a3 --- /dev/null +++ b/crates/sail-iceberg/src/spec/name_mapping/mod.rs @@ -0,0 +1,76 @@ +// 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. + +// [CREDIT]: https://raw.githubusercontent.com/apache/iceberg-rust/dc349284a4204c1a56af47fb3177ace6f9e899a0/crates/iceberg/src/spec/name_mapping/mod.rs + +use std::sync::Arc; + +use serde::{Deserialize, Serialize}; + +/// Default schema name mapping property key +pub const DEFAULT_SCHEMA_NAME_MAPPING: &str = "schema.name-mapping.default"; + +#[derive(Debug, Serialize, Deserialize, PartialEq, Eq, Clone)] +#[serde(transparent)] +pub struct NameMapping(Vec); + +impl NameMapping { + /// Create a new `NameMapping` given mapped fields. + pub fn new(fields: Vec) -> Self { + Self(fields) + } + + /// Returns mapped fields + pub fn fields(&self) -> &[MappedField] { + &self.0 + } +} + +#[derive(Debug, Serialize, Deserialize, PartialEq, Eq, Clone)] +#[serde(rename_all = "kebab-case")] +pub struct MappedField { + #[serde(skip_serializing_if = "Option::is_none")] + field_id: Option, + names: Vec, + #[serde(default)] + #[serde(skip_serializing_if = "Vec::is_empty")] + fields: Vec>, +} + +impl MappedField { + /// Create a new `MappedField`. + pub fn new(field_id: Option, names: Vec, fields: Vec) -> Self { + Self { + field_id, + names, + fields: fields.into_iter().map(Arc::new).collect(), + } + } + + /// Optional field id + pub fn field_id(&self) -> Option { + self.field_id + } + /// All names for this field + pub fn names(&self) -> &[String] { + &self.names + } + /// Child mapped fields + pub fn fields(&self) -> &[Arc] { + &self.fields + } +} diff --git a/crates/sail-iceberg/src/spec/partition/mod.rs b/crates/sail-iceberg/src/spec/partition/mod.rs new file mode 100644 index 0000000000..457e09ea3c --- /dev/null +++ b/crates/sail-iceberg/src/spec/partition/mod.rs @@ -0,0 +1,22 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +pub mod spec; +pub mod unbound; + +pub use spec::*; +pub use unbound::*; diff --git a/crates/sail-iceberg/src/spec/partition/spec.rs b/crates/sail-iceberg/src/spec/partition/spec.rs new file mode 100644 index 0000000000..d55f373271 --- /dev/null +++ b/crates/sail-iceberg/src/spec/partition/spec.rs @@ -0,0 +1,243 @@ +// 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. + +// [CREDIT]: https://raw.githubusercontent.com/apache/iceberg-rust/dc349284a4204c1a56af47fb3177ace6f9e899a0/crates/iceberg/src/spec/partition.rs + +use std::sync::Arc; + +use serde::{Deserialize, Serialize}; + +use crate::spec::schema::Schema; +use crate::spec::transform::Transform; +use crate::spec::types::{NestedField, StructType}; + +#[allow(unused)] +pub(crate) const UNPARTITIONED_LAST_ASSIGNED_ID: i32 = 999; +pub(crate) const DEFAULT_PARTITION_SPEC_ID: i32 = 0; + +/// Partition fields capture the transform from table data to partition values. +#[derive(Debug, Serialize, Deserialize, PartialEq, Eq, Clone)] +#[serde(rename_all = "kebab-case")] +pub struct PartitionField { + /// A source column id from the table's schema + pub source_id: i32, + /// A partition field id that is used to identify a partition field and is unique within a partition spec. + /// In v2 table metadata, it is unique across all partition specs. + pub field_id: i32, + /// A partition name. + pub name: String, + /// A transform that is applied to the source column to produce a partition value. + pub transform: Transform, +} + +impl PartitionField { + /// Create a new partition field. + pub fn new(source_id: i32, field_id: i32, name: impl ToString, transform: Transform) -> Self { + Self { + source_id, + field_id, + name: name.to_string(), + transform, + } + } +} + +/// Reference to [`PartitionSpec`]. +pub type PartitionSpecRef = Arc; + +/// Partition spec that defines how to produce a tuple of partition values from a record. +#[derive(Debug, Serialize, Deserialize, PartialEq, Eq, Clone)] +#[serde(rename_all = "kebab-case")] +pub struct PartitionSpec { + /// Identifier for PartitionSpec + spec_id: i32, + /// Details of the partition spec + fields: Vec, +} + +impl PartitionSpec { + /// Create a new partition spec builder. + pub fn builder() -> PartitionSpecBuilder { + PartitionSpecBuilder::new() + } + + /// Fields of the partition spec + pub fn fields(&self) -> &[PartitionField] { + &self.fields + } + + /// Spec id of the partition spec + pub fn spec_id(&self) -> i32 { + self.spec_id + } + + /// Get a new unpartitioned partition spec + pub fn unpartitioned_spec() -> Self { + Self { + spec_id: DEFAULT_PARTITION_SPEC_ID, + fields: vec![], + } + } + + /// 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| f.transform == Transform::Void) + } + + /// Returns the partition type of this partition spec. + pub fn partition_type(&self, schema: &Schema) -> Result { + let mut partition_fields = Vec::new(); + + for partition_field in self.fields.iter() { + let source_field = schema + .field_by_id(partition_field.source_id) + .ok_or_else(|| { + format!( + "Cannot find source field with id {}", + partition_field.source_id + ) + })?; + + let result_type = partition_field + .transform + .result_type(&source_field.field_type)?; + + let nested_field = NestedField::new( + partition_field.field_id, + &partition_field.name, + result_type, + false, // Partition fields are typically optional + ); + + partition_fields.push(Arc::new(nested_field)); + } + + Ok(StructType::new(partition_fields)) + } + + /// Change the spec id of the partition spec + pub fn with_spec_id(self, spec_id: i32) -> Self { + Self { spec_id, ..self } + } + + /// Get the highest field id in the partition spec. + pub fn highest_field_id(&self) -> Option { + self.fields.iter().map(|f| f.field_id).max() + } + + /// Check if the partition spec has sequential field ids starting from 1000. + /// Required for spec version 1 in the reference implementation. + pub fn has_sequential_ids(&self) -> bool { + let mut expected = 1000; + for field in &self.fields { + if field.field_id != expected { + return false; + } + expected += 1; + } + true + } + + /// 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 + /// spec_id ignored. The following must be identical: + /// * The number of fields + /// * Field order + /// * Field names + /// * Source column ids + /// * Transforms + pub fn is_compatible_with(&self, other: &PartitionSpec) -> bool { + if self.fields.len() != other.fields.len() { + return false; + } + + for (this_field, other_field) in self.fields.iter().zip(other.fields.iter()) { + if this_field.source_id != other_field.source_id + || this_field.name != other_field.name + || this_field.transform != other_field.transform + { + return false; + } + } + + true + } +} + +/// Builder for partition spec. +#[derive(Debug)] +pub struct PartitionSpecBuilder { + spec_id: i32, + fields: Vec, + next_field_id: i32, +} + +impl PartitionSpecBuilder { + /// Create a new partition spec builder. + pub fn new() -> Self { + Self { + spec_id: DEFAULT_PARTITION_SPEC_ID, + fields: Vec::new(), + next_field_id: 1000, // Partition field IDs typically start from 1000 + } + } + + /// Set the spec id. + pub fn with_spec_id(mut self, spec_id: i32) -> Self { + self.spec_id = spec_id; + self + } + + /// Add a partition field. + pub fn add_field(mut self, source_id: i32, name: impl ToString, transform: Transform) -> Self { + let field = PartitionField::new(source_id, self.next_field_id, name, transform); + self.fields.push(field); + self.next_field_id += 1; + self + } + + /// Add a partition field with explicit field id. + pub fn add_field_with_id( + mut self, + source_id: i32, + field_id: i32, + name: impl ToString, + transform: Transform, + ) -> Self { + let field = PartitionField::new(source_id, field_id, name, transform); + self.fields.push(field); + self.next_field_id = self.next_field_id.max(field_id + 1); + self + } + + /// Build the partition spec. + pub fn build(self) -> PartitionSpec { + PartitionSpec { + spec_id: self.spec_id, + fields: self.fields, + } + } +} + +impl Default for PartitionSpecBuilder { + fn default() -> Self { + Self::new() + } +} diff --git a/crates/sail-iceberg/src/spec/partition/unbound.rs b/crates/sail-iceberg/src/spec/partition/unbound.rs new file mode 100644 index 0000000000..884da089c2 --- /dev/null +++ b/crates/sail-iceberg/src/spec/partition/unbound.rs @@ -0,0 +1,36 @@ +// 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. + +// [CREDIT]: https://raw.githubusercontent.com/apache/iceberg-rust/dc349284a4204c1a56af47fb3177ace6f9e899a0/crates/iceberg/src/spec/partition.rs + +use serde::{Deserialize, Serialize}; + +use crate::spec::transform::Transform; + +#[derive(Debug, Clone, Serialize, Deserialize, PartialEq, Eq)] +#[serde(rename_all = "kebab-case")] +pub struct UnboundPartitionField { + pub source_id: i32, + pub name: String, + pub transform: Transform, +} + +#[derive(Debug, Clone, Serialize, Deserialize, PartialEq, Eq)] +#[serde(rename_all = "kebab-case")] +pub struct UnboundPartitionSpec { + pub fields: Vec, +} diff --git a/crates/sail-iceberg/src/spec/schema/mod.rs b/crates/sail-iceberg/src/spec/schema/mod.rs new file mode 100644 index 0000000000..6557a6c67c --- /dev/null +++ b/crates/sail-iceberg/src/spec/schema/mod.rs @@ -0,0 +1,409 @@ +// 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. + +// [CREDIT]: https://raw.githubusercontent.com/apache/iceberg-rust/dc349284a4204c1a56af47fb3177ace6f9e899a0/crates/iceberg/src/spec/schema/mod.rs + +use std::collections::{HashMap, HashSet}; +use std::fmt::{Display, Formatter}; +use std::sync::Arc; + +use serde::{Deserialize, Serialize}; + +use crate::spec::types::{NestedFieldRef, PrimitiveType, StructType, Type}; + +pub mod utils; + +pub use utils::*; + +/// Type alias for schema id. +pub type SchemaId = i32; +/// Reference to [`Schema`]. +pub type SchemaRef = Arc; +/// Default schema id. +pub const DEFAULT_SCHEMA_ID: SchemaId = 0; + +/// Defines schema in iceberg. +#[derive(Debug, Serialize, Clone)] +pub struct Schema { + #[serde(rename = "type")] + schema_type: String, + #[serde(rename = "schema-id")] + schema_id: SchemaId, + #[serde(rename = "fields")] + fields: Vec, + #[serde( + rename = "identifier-field-ids", + skip_serializing_if = "Option::is_none" + )] + identifier_field_ids: Option>, + + // Internal indexes (not serialized) + #[serde(skip)] + struct_type: StructType, + #[serde(skip)] + highest_field_id: i32, + #[serde(skip)] + id_to_field: HashMap, + #[serde(skip)] + name_to_id: HashMap, + #[serde(skip)] + id_to_name: HashMap, +} + +impl PartialEq for Schema { + fn eq(&self, other: &Self) -> bool { + self.schema_id == other.schema_id + && self.fields == other.fields + && self.identifier_field_ids == other.identifier_field_ids + } +} + +impl Eq for Schema {} + +#[derive(Deserialize)] +struct SchemaData { + #[serde(rename = "type")] + schema_type: String, + #[serde(rename = "schema-id")] + schema_id: SchemaId, + #[serde(rename = "fields")] + fields: Vec, + #[serde(rename = "identifier-field-ids")] + identifier_field_ids: Option>, +} + +#[derive(Deserialize)] +#[serde(untagged)] +enum SchemaEnum { + V1(SchemaData), + V2(SchemaData), +} + +impl<'de> Deserialize<'de> for Schema { + fn deserialize(deserializer: D) -> Result + where + D: serde::Deserializer<'de>, + { + let data = match SchemaEnum::deserialize(deserializer)? { + SchemaEnum::V1(d) | SchemaEnum::V2(d) => d, + }; + + let struct_type = StructType::new(data.fields.clone()); + let mut id_to_field = HashMap::new(); + SchemaBuilder::index_fields_recursive(struct_type.fields(), &mut id_to_field); + + let mut name_to_id = HashMap::new(); + let mut id_to_name = HashMap::new(); + SchemaBuilder::index_names_recursive( + struct_type.fields(), + "", + &mut name_to_id, + &mut id_to_name, + ); + + let highest_field_id = id_to_field.keys().max().cloned().unwrap_or(0); + + Ok(Schema { + schema_type: data.schema_type, + schema_id: data.schema_id, + fields: data.fields, + identifier_field_ids: data.identifier_field_ids, + struct_type, + highest_field_id, + id_to_field, + name_to_id, + id_to_name, + }) + } +} + +/// Schema builder. +#[derive(Debug)] +pub struct SchemaBuilder { + schema_id: i32, + fields: Vec, + identifier_field_ids: HashSet, +} + +impl SchemaBuilder { + /// Add fields to schema builder. + pub fn with_fields(mut self, fields: impl IntoIterator) -> Self { + self.fields.extend(fields); + self + } + + /// Set schema id. + pub fn with_schema_id(mut self, schema_id: i32) -> Self { + self.schema_id = schema_id; + self + } + + /// Set identifier field ids. + pub fn with_identifier_field_ids(mut self, ids: impl IntoIterator) -> Self { + self.identifier_field_ids.extend(ids); + self + } + + /// Builds the schema. + pub fn build(self) -> Result { + let struct_type = StructType::new(self.fields.clone()); + let id_to_field = self.build_id_to_field_index(&struct_type); + + self.validate_identifier_ids(&id_to_field)?; + + let (name_to_id, id_to_name) = self.build_name_indexes(&struct_type); + let highest_field_id = id_to_field.keys().max().cloned().unwrap_or(0); + + let identifier_field_ids = if self.identifier_field_ids.is_empty() { + None + } else { + Some(self.identifier_field_ids.into_iter().collect()) + }; + + Ok(Schema { + schema_type: "struct".to_string(), + schema_id: self.schema_id, + fields: self.fields, + identifier_field_ids, + struct_type, + highest_field_id, + id_to_field, + name_to_id, + id_to_name, + }) + } + + fn build_id_to_field_index(&self, struct_type: &StructType) -> HashMap { + let mut id_to_field = HashMap::new(); + Self::index_fields_recursive(struct_type.fields(), &mut id_to_field); + id_to_field + } + + fn index_fields_recursive( + fields: &[NestedFieldRef], + id_to_field: &mut HashMap, + ) { + for field in fields { + id_to_field.insert(field.id, field.clone()); + + match field.field_type.as_ref() { + Type::Struct(struct_type) => { + Self::index_fields_recursive(struct_type.fields(), id_to_field); + } + Type::List(list_type) => { + id_to_field.insert(list_type.element_field.id, list_type.element_field.clone()); + if let Type::Struct(struct_type) = list_type.element_field.field_type.as_ref() { + Self::index_fields_recursive(struct_type.fields(), id_to_field); + } + } + Type::Map(map_type) => { + id_to_field.insert(map_type.key_field.id, map_type.key_field.clone()); + id_to_field.insert(map_type.value_field.id, map_type.value_field.clone()); + if let Type::Struct(struct_type) = map_type.key_field.field_type.as_ref() { + Self::index_fields_recursive(struct_type.fields(), id_to_field); + } + if let Type::Struct(struct_type) = map_type.value_field.field_type.as_ref() { + Self::index_fields_recursive(struct_type.fields(), id_to_field); + } + } + _ => {} + } + } + } + + fn build_name_indexes( + &self, + struct_type: &StructType, + ) -> (HashMap, HashMap) { + let mut name_to_id = HashMap::new(); + let mut id_to_name = HashMap::new(); + Self::index_names_recursive(struct_type.fields(), "", &mut name_to_id, &mut id_to_name); + (name_to_id, id_to_name) + } + + fn index_names_recursive( + fields: &[NestedFieldRef], + prefix: &str, + name_to_id: &mut HashMap, + id_to_name: &mut HashMap, + ) { + for field in fields { + let full_name = if prefix.is_empty() { + field.name.clone() + } else { + format!("{}.{}", prefix, field.name) + }; + + name_to_id.insert(full_name.clone(), field.id); + id_to_name.insert(field.id, full_name.clone()); + + match field.field_type.as_ref() { + Type::Struct(struct_type) => { + Self::index_names_recursive( + struct_type.fields(), + &full_name, + name_to_id, + id_to_name, + ); + } + Type::List(list_type) => { + let element_name = format!("{}.element", full_name); + name_to_id.insert(element_name.clone(), list_type.element_field.id); + id_to_name.insert(list_type.element_field.id, element_name); + + if let Type::Struct(struct_type) = list_type.element_field.field_type.as_ref() { + Self::index_names_recursive( + struct_type.fields(), + &full_name, + name_to_id, + id_to_name, + ); + } + } + Type::Map(map_type) => { + let key_name = format!("{}.key", full_name); + let value_name = format!("{}.value", full_name); + + name_to_id.insert(key_name.clone(), map_type.key_field.id); + id_to_name.insert(map_type.key_field.id, key_name); + + name_to_id.insert(value_name.clone(), map_type.value_field.id); + id_to_name.insert(map_type.value_field.id, value_name.clone()); + + if let Type::Struct(struct_type) = map_type.value_field.field_type.as_ref() { + Self::index_names_recursive( + struct_type.fields(), + &value_name, + name_to_id, + id_to_name, + ); + } + } + _ => {} + } + } + } + + fn validate_identifier_ids( + &self, + id_to_field: &HashMap, + ) -> Result<(), String> { + for identifier_field_id in &self.identifier_field_ids { + let field = id_to_field.get(identifier_field_id).ok_or_else(|| { + format!("Cannot add identifier field {identifier_field_id}: field does not exist") + })?; + + if !field.required { + return Err(format!( + "Cannot add identifier field: {} is an optional field", + field.name + )); + } + + if let Type::Primitive(p) = field.field_type.as_ref() { + if matches!(p, PrimitiveType::Double | PrimitiveType::Float) { + return Err(format!( + "Cannot add identifier field {}: cannot be a float or double type", + field.name + )); + } + } else { + return Err(format!( + "Cannot add field {} as an identifier field: not a primitive type field", + field.name + )); + } + } + + Ok(()) + } +} + +impl Schema { + /// Create a schema builder. + pub fn builder() -> SchemaBuilder { + SchemaBuilder { + schema_id: DEFAULT_SCHEMA_ID, + fields: vec![], + identifier_field_ids: HashSet::default(), + } + } + + /// Get field by field id. + pub fn field_by_id(&self, field_id: i32) -> Option<&NestedFieldRef> { + self.id_to_field.get(&field_id) + } + + /// Get field by field name. + pub fn field_by_name(&self, field_name: &str) -> Option<&NestedFieldRef> { + self.name_to_id + .get(field_name) + .and_then(|id| self.field_by_id(*id)) + } + + /// Returns [`highest_field_id`]. + #[inline] + pub fn highest_field_id(&self) -> i32 { + self.highest_field_id + } + + /// Returns [`schema_id`]. + #[inline] + pub fn schema_id(&self) -> SchemaId { + self.schema_id + } + + /// Returns the struct type representation of this schema. + pub fn as_struct(&self) -> &StructType { + &self.struct_type + } + + /// Returns [`identifier_field_ids`]. + pub fn identifier_field_ids(&self) -> impl ExactSizeIterator + '_ { + self.identifier_field_ids + .as_ref() + .map(|ids| ids.iter().copied()) + .unwrap_or_else(|| [].iter().copied()) + .collect::>() + .into_iter() + } + + /// Get field id by full name. + pub fn field_id_by_name(&self, name: &str) -> Option { + self.name_to_id.get(name).copied() + } + + /// Get full name by field id. + pub fn name_by_field_id(&self, field_id: i32) -> Option<&str> { + self.id_to_name.get(&field_id).map(String::as_str) + } + + /// Get all fields in the schema. + pub fn fields(&self) -> &[NestedFieldRef] { + &self.fields + } +} + +impl Display for Schema { + fn fmt(&self, f: &mut Formatter<'_>) -> std::fmt::Result { + writeln!(f, "table {{")?; + for field in &self.fields { + writeln!(f, " {}", field)?; + } + writeln!(f, "}}") + } +} diff --git a/crates/sail-iceberg/src/spec/schema/utils.rs b/crates/sail-iceberg/src/spec/schema/utils.rs new file mode 100644 index 0000000000..dd6421577e --- /dev/null +++ b/crates/sail-iceberg/src/spec/schema/utils.rs @@ -0,0 +1,68 @@ +// 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. + +// [CREDIT]: https://raw.githubusercontent.com/apache/iceberg-rust/dc349284a4204c1a56af47fb3177ace6f9e899a0/crates/iceberg/src/spec/schema/utils.rs + +use std::collections::{HashMap, HashSet, VecDeque}; + +use super::Schema; +use crate::spec::types::{NestedFieldRef, Type}; + +/// Visit all fields in a schema in breadth-first order, calling the callback for each field id. +pub fn visit_fields_bfs(schema: &Schema, mut f: F) { + let mut queue: VecDeque = VecDeque::new(); + for field in schema.fields() { + queue.push_back(field.clone()); + } + + while let Some(field) = queue.pop_front() { + f(field.id, &field); + match field.field_type.as_ref() { + Type::Struct(s) => { + for child in s.fields() { + queue.push_back(child.clone()); + } + } + Type::List(l) => queue.push_back(l.element_field.clone()), + Type::Map(m) => { + queue.push_back(m.key_field.clone()); + queue.push_back(m.value_field.clone()); + } + _ => {} + } + } +} + +/// Prune a schema by keeping only fields whose ids are included. +/// Ancestor container fields are preserved automatically. +pub fn prune_schema_by_field_ids(schema: &Schema, keep_ids: &HashSet) -> Vec { + let mut kept: HashMap = HashMap::new(); + + visit_fields_bfs(schema, |id, field| { + if keep_ids.contains(&id) { + kept.insert(id, field.clone()); + } + }); + + // Return only top-level fields that are kept; children are implicitly reachable by id + schema + .fields() + .iter() + .filter(|f| kept.contains_key(&f.id)) + .cloned() + .collect() +} diff --git a/crates/sail-iceberg/src/spec/snapshots/mod.rs b/crates/sail-iceberg/src/spec/snapshots/mod.rs new file mode 100644 index 0000000000..ba291d4127 --- /dev/null +++ b/crates/sail-iceberg/src/spec/snapshots/mod.rs @@ -0,0 +1,22 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +pub mod snapshot; +pub mod summary; + +pub use snapshot::*; +pub use summary::*; diff --git a/crates/sail-iceberg/src/spec/snapshots/snapshot.rs b/crates/sail-iceberg/src/spec/snapshots/snapshot.rs new file mode 100644 index 0000000000..c9d764abf2 --- /dev/null +++ b/crates/sail-iceberg/src/spec/snapshots/snapshot.rs @@ -0,0 +1,322 @@ +// 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. + +// [CREDIT]: https://raw.githubusercontent.com/apache/iceberg-rust/dc349284a4204c1a56af47fb3177ace6f9e899a0/crates/iceberg/src/spec/snapshot.rs + +use std::collections::HashMap; +use std::sync::Arc; + +use chrono::{DateTime, Utc}; +use serde::{Deserialize, Serialize}; + +use crate::spec::schema::SchemaId; + +/// The ref name of the main branch of the table. +pub const MAIN_BRANCH: &str = "main"; +pub const UNASSIGNED_SNAPSHOT_ID: i64 = -1; + +/// Reference to [`Snapshot`]. +pub type SnapshotRef = Arc; + +#[derive(Debug, Serialize, Deserialize, PartialEq, Eq, Clone)] +#[serde(rename_all = "lowercase")] +/// The operation field is used by some operations, like snapshot expiration, to skip processing certain snapshots. +pub enum Operation { + /// Only data files were added and no files were removed. + Append, + /// Data and delete files were added and removed without changing table data; + /// i.e., compaction, changing the data file format, or relocating data files. + Replace, + /// Data and delete files were added and removed in a logical overwrite operation. + Overwrite, + /// Data files were removed and their contents logically deleted and/or delete files were added to delete rows. + Delete, +} + +impl Operation { + /// Returns the string representation (lowercase) of the operation. + pub fn as_str(&self) -> &str { + match self { + Operation::Append => "append", + Operation::Replace => "replace", + Operation::Overwrite => "overwrite", + Operation::Delete => "delete", + } + } +} + +impl Default for Operation { + fn default() -> Operation { + Self::Append + } +} + +#[derive(Debug, Serialize, Deserialize, PartialEq, Eq, Clone)] +#[serde(rename_all = "kebab-case")] +/// A snapshot represents the state of a table at some time and is used to access the complete set of data files in the table. +pub struct Snapshot { + /// A unique long ID + pub snapshot_id: i64, + /// The snapshot ID of the snapshot's parent. + /// Omitted for any snapshot with no parent + #[serde(skip_serializing_if = "Option::is_none")] + pub parent_snapshot_id: Option, + /// A monotonically increasing long that tracks the order of + /// changes to a table. + pub sequence_number: i64, + /// A timestamp when the snapshot was created, used for garbage + /// collection and table inspection + pub timestamp_ms: i64, + /// The location of a manifest list for this snapshot that + /// tracks manifest files with additional metadata. + #[serde(default)] + pub manifest_list: String, + /// V1 snapshots list manifests directly instead of a manifest list file. + #[serde(skip_serializing_if = "Option::is_none")] + pub manifests: Option>, + /// A string map that summarizes the snapshot changes, including operation. + pub summary: Summary, + /// ID of the table's current schema when the snapshot was created. + #[serde(skip_serializing_if = "Option::is_none")] + pub schema_id: Option, +} + +#[derive(Debug, Clone, Serialize, Deserialize, PartialEq, Eq)] +#[serde(rename_all = "kebab-case")] +/// A reference’s snapshot and retention policy +pub struct SnapshotReference { + /// A reference’s snapshot ID. The tagged snapshot or latest snapshot of a branch. + pub snapshot_id: i64, + #[serde(flatten)] + /// Snapshot retention policy + pub retention: SnapshotRetention, +} + +impl SnapshotReference { + /// Returns true if the snapshot reference is a branch. + pub fn is_branch(&self) -> bool { + matches!(self.retention, SnapshotRetention::Branch { .. }) + } +} + +#[derive(Debug, Clone, Serialize, Deserialize, PartialEq, Eq)] +#[serde(rename_all = "kebab-case", tag = "type")] +/// Snapshot retention policy +pub enum SnapshotRetention { + /// Branches are mutable named references that can be updated by committing a new snapshot + Branch { + /// Minimum number of snapshots to keep in a branch while expiring snapshots. + #[serde(skip_serializing_if = "Option::is_none")] + min_snapshots_to_keep: Option, + /// Max age of snapshots to keep when expiring, including the latest snapshot. + #[serde(skip_serializing_if = "Option::is_none")] + max_snapshot_age_ms: Option, + /// Max age of the snapshot reference to keep while expiring snapshots. + #[serde(skip_serializing_if = "Option::is_none")] + max_ref_age_ms: Option, + }, + /// Tags are labels for individual snapshots. + Tag { + /// Max age of the snapshot reference to keep while expiring snapshots. + #[serde(skip_serializing_if = "Option::is_none")] + max_ref_age_ms: Option, + }, +} + +#[derive(Debug, Serialize, Deserialize, PartialEq, Eq, Clone)] +/// Summarises the changes in the snapshot. +pub struct Summary { + /// The type of operation in the snapshot + pub operation: Operation, + /// Other summary data. + #[serde(flatten)] + pub additional_properties: HashMap, +} + +impl Summary { + /// Create a new summary with the given operation. + pub fn new(operation: Operation) -> Self { + Self { + operation, + additional_properties: HashMap::new(), + } + } + + /// Add additional property to the summary. + pub fn with_property(mut self, key: impl ToString, value: impl ToString) -> Self { + self.additional_properties + .insert(key.to_string(), value.to_string()); + self + } +} + +impl Snapshot { + /// Create a new snapshot builder. + pub fn builder() -> SnapshotBuilder { + SnapshotBuilder::new() + } + + /// Get the id of the snapshot + #[inline] + pub fn snapshot_id(&self) -> i64 { + self.snapshot_id + } + + /// Get parent snapshot id. + #[inline] + pub fn parent_snapshot_id(&self) -> Option { + self.parent_snapshot_id + } + + /// Get sequence_number of the snapshot. Is 0 for Iceberg V1 tables. + #[inline] + pub fn sequence_number(&self) -> i64 { + self.sequence_number + } + + /// Get location of manifest_list file + #[inline] + pub fn manifest_list(&self) -> &str { + &self.manifest_list + } + + /// Get V1 manifests list if present + #[inline] + pub fn manifests(&self) -> Option<&[String]> { + self.manifests.as_deref() + } + + /// Get summary of the snapshot + #[inline] + pub fn summary(&self) -> &Summary { + &self.summary + } + + /// Get the timestamp of when the snapshot was created + #[inline] + pub fn timestamp(&self) -> Result, String> { + DateTime::from_timestamp_millis(self.timestamp_ms) + .ok_or_else(|| format!("Invalid timestamp: {}", self.timestamp_ms)) + } + + /// Get the timestamp of when the snapshot was created in milliseconds + #[inline] + pub fn timestamp_ms(&self) -> i64 { + self.timestamp_ms + } + + /// Get the schema id of this snapshot. + #[inline] + pub fn schema_id(&self) -> Option { + self.schema_id + } +} + +/// Builder for creating snapshots. +#[derive(Debug)] +pub struct SnapshotBuilder { + snapshot_id: i64, + parent_snapshot_id: Option, + sequence_number: i64, + timestamp_ms: i64, + manifest_list: Option, + summary: Option, + schema_id: Option, +} + +impl SnapshotBuilder { + /// Create a new snapshot builder. + pub fn new() -> Self { + Self { + snapshot_id: UNASSIGNED_SNAPSHOT_ID, + parent_snapshot_id: None, + sequence_number: 0, + timestamp_ms: chrono::Utc::now().timestamp_millis(), + manifest_list: None, + summary: None, + schema_id: None, + } + } + + /// Set the snapshot id. + pub fn with_snapshot_id(mut self, snapshot_id: i64) -> Self { + self.snapshot_id = snapshot_id; + self + } + + /// Set the parent snapshot id. + pub fn with_parent_snapshot_id(mut self, parent_snapshot_id: i64) -> Self { + self.parent_snapshot_id = Some(parent_snapshot_id); + self + } + + /// Set the sequence number. + pub fn with_sequence_number(mut self, sequence_number: i64) -> Self { + self.sequence_number = sequence_number; + self + } + + /// Set the timestamp in milliseconds. + pub fn with_timestamp_ms(mut self, timestamp_ms: i64) -> Self { + self.timestamp_ms = timestamp_ms; + self + } + + /// Set the manifest list location. + pub fn with_manifest_list(mut self, manifest_list: impl ToString) -> Self { + self.manifest_list = Some(manifest_list.to_string()); + self + } + + /// Set the summary. + pub fn with_summary(mut self, summary: Summary) -> Self { + self.summary = Some(summary); + self + } + + /// Set the schema id. + pub fn with_schema_id(mut self, schema_id: SchemaId) -> Self { + self.schema_id = Some(schema_id); + self + } + + /// Build the snapshot. + pub fn build(self) -> Result { + // For V1 compatibility allow manifest_list to be missing when manifests provided + let manifest_list = self.manifest_list.unwrap_or_default(); + let summary = self + .summary + .unwrap_or_else(|| Summary::new(Operation::Append)); + + Ok(Snapshot { + snapshot_id: self.snapshot_id, + parent_snapshot_id: self.parent_snapshot_id, + sequence_number: self.sequence_number, + timestamp_ms: self.timestamp_ms, + manifest_list, + manifests: None, + summary, + schema_id: self.schema_id, + }) + } +} + +impl Default for SnapshotBuilder { + fn default() -> Self { + Self::new() + } +} diff --git a/crates/sail-iceberg/src/spec/snapshots/summary.rs b/crates/sail-iceberg/src/spec/snapshots/summary.rs new file mode 100644 index 0000000000..86b0960ed5 --- /dev/null +++ b/crates/sail-iceberg/src/spec/snapshots/summary.rs @@ -0,0 +1,21 @@ +// 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. + +// Awareness stub for read-path. Extend later if needed. +#[allow(dead_code)] +#[derive(Debug, Clone)] +pub struct SnapshotSummary; diff --git a/crates/sail-iceberg/src/spec/sort.rs b/crates/sail-iceberg/src/spec/sort.rs new file mode 100644 index 0000000000..d507d8ae7f --- /dev/null +++ b/crates/sail-iceberg/src/spec/sort.rs @@ -0,0 +1,110 @@ +// 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. + +// [CREDIT]: https://raw.githubusercontent.com/apache/iceberg-rust/dc349284a4204c1a56af47fb3177ace6f9e899a0/crates/iceberg/src/spec/sort.rs + +use std::fmt::{Display, Formatter}; + +use serde::{Deserialize, Serialize}; + +use super::transform::Transform; + +#[derive(Debug, Serialize, Deserialize, PartialEq, Eq, Copy, Clone)] +/// Sort direction in a partition, either ascending or descending +pub enum SortDirection { + /// Ascending + #[serde(rename = "asc")] + Ascending, + /// Descending + #[serde(rename = "desc")] + Descending, +} + +impl Display for SortDirection { + fn fmt(&self, f: &mut Formatter<'_>) -> std::fmt::Result { + match self { + SortDirection::Ascending => write!(f, "asc"), + SortDirection::Descending => write!(f, "desc"), + } + } +} + +#[derive(Debug, Serialize, Deserialize, PartialEq, Eq, Copy, Clone)] +pub enum NullOrder { + #[serde(rename = "nulls-first")] + /// Nulls are stored first + First, + #[serde(rename = "nulls-last")] + /// Nulls are stored last + Last, +} + +impl Display for NullOrder { + fn fmt(&self, f: &mut Formatter<'_>) -> std::fmt::Result { + match self { + NullOrder::First => write!(f, "nulls-first"), + NullOrder::Last => write!(f, "nulls-last"), + } + } +} + +#[derive(Debug, Serialize, Deserialize, PartialEq, Eq, Clone)] +#[serde(rename_all = "kebab-case")] +pub struct SortField { + /// A source column id from the table’s schema + pub source_id: i32, + /// A transform that is used to produce values to be sorted on from the source column. + pub transform: Transform, + /// A sort direction, that can only be either asc or desc + pub direction: SortDirection, + /// A null order that describes the order of null values when sorted. + pub null_order: NullOrder, +} + +impl Display for SortField { + fn fmt(&self, f: &mut Formatter<'_>) -> std::fmt::Result { + write!( + f, + "{} {} {} {}", + self.source_id, self.transform, self.direction, self.null_order + ) + } +} + +#[derive(Debug, Serialize, Deserialize, PartialEq, Eq, Clone)] +#[serde(rename_all = "kebab-case")] +pub struct SortOrder { + /// Identifier for SortOrder, order_id `0` is no sort order. + #[serde(default)] + pub order_id: i64, + /// Details of the sort + #[serde(default)] + pub fields: Vec, +} + +impl SortOrder { + pub fn unsorted_order() -> SortOrder { + SortOrder { + order_id: 0, + fields: vec![], + } + } + + pub fn is_unsorted(&self) -> bool { + self.order_id == 0 || self.fields.is_empty() + } +} diff --git a/crates/sail-iceberg/src/spec/transform.rs b/crates/sail-iceberg/src/spec/transform.rs new file mode 100644 index 0000000000..49d4a1b165 --- /dev/null +++ b/crates/sail-iceberg/src/spec/transform.rs @@ -0,0 +1,316 @@ +// 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. + +// [CREDIT]: https://raw.githubusercontent.com/apache/iceberg-rust/dc349284a4204c1a56af47fb3177ace6f9e899a0/crates/iceberg/src/spec/transform.rs + +use std::fmt::{Display, Formatter}; +use std::str::FromStr; + +use serde::{Deserialize, Deserializer, Serialize, Serializer}; + +use super::types::values::Literal; +use super::types::{PrimitiveType, Type}; + +/// Transform is used to transform predicates to partition predicates, +/// in addition to transforming data values. +#[derive(Debug, PartialEq, Eq, Clone, Copy)] +pub enum Transform { + /// Source value, unmodified + Identity, + /// Hash of value, mod `N`. + Bucket(u32), + /// Value truncated to width `W` + Truncate(u32), + /// Extract a date or timestamp year, as years from 1970 + Year, + /// Extract a date or timestamp month, as months from 1970-01-01 + Month, + /// Extract a date or timestamp day, as days from 1970-01-01 + Day, + /// Extract a timestamp hour, as hours from 1970-01-01 00:00:00 + Hour, + /// Always produces `null` + Void, + /// Used to represent some customized transform that can't be recognized or supported now. + Unknown, +} + +impl Transform { + pub fn to_human_string(self, field_type: &Type, value: Option<&Literal>) -> String { + fn bytes_to_hex(bytes: &[u8]) -> String { + let mut s = String::with_capacity(bytes.len() * 2); + for b in bytes { + use std::fmt::Write as _; + let _ = write!(&mut s, "{:02x}", b); + } + s + } + + fn lit_str(l: &Literal) -> String { + match l { + Literal::Primitive(p) => match p { + super::types::values::PrimitiveLiteral::Boolean(v) => v.to_string(), + super::types::values::PrimitiveLiteral::Int(v) => v.to_string(), + super::types::values::PrimitiveLiteral::Long(v) => v.to_string(), + super::types::values::PrimitiveLiteral::Float(v) => v.0.to_string(), + super::types::values::PrimitiveLiteral::Double(v) => v.0.to_string(), + super::types::values::PrimitiveLiteral::Int128(v) => v.to_string(), + super::types::values::PrimitiveLiteral::String(v) => v.clone(), + super::types::values::PrimitiveLiteral::UInt128(v) => v.to_string(), + super::types::values::PrimitiveLiteral::Binary(b) => { + format!("0x{}", bytes_to_hex(b)) + } + }, + Literal::Struct(_) | Literal::List(_) | Literal::Map(_) => format!("{:?}", l), + } + } + + match value { + None => "null".to_string(), + Some(val) => match self { + Transform::Identity => lit_str(val), + Transform::Void => "null".to_string(), + Transform::Truncate(w) => match (field_type, val) { + ( + Type::Primitive(PrimitiveType::String), + Literal::Primitive(super::types::values::PrimitiveLiteral::String(s)), + ) => s.chars().take(w as usize).collect::(), + ( + Type::Primitive(PrimitiveType::Int), + Literal::Primitive(super::types::values::PrimitiveLiteral::Int(v)), + ) => { + let w = w as i32; + let rem = v.rem_euclid(w); + (v - rem).to_string() + } + ( + Type::Primitive(PrimitiveType::Long), + Literal::Primitive(super::types::values::PrimitiveLiteral::Long(v)), + ) => { + let w = w as i64; + let rem = v.rem_euclid(w); + (v - rem).to_string() + } + _ => lit_str(val), + }, + Transform::Bucket(n) => format!("bucket[{n}]({})", lit_str(val)), + Transform::Year | Transform::Month | Transform::Day | Transform::Hour => { + lit_str(val) + } + Transform::Unknown => lit_str(val), + }, + } + } + + /// Get the return type of transform given the input type. + /// Returns `None` if it can't be transformed. + pub fn result_type(&self, input_type: &Type) -> Result { + match self { + Transform::Identity => { + if matches!(input_type, Type::Primitive(_)) { + Ok(input_type.clone()) + } else { + Err(format!( + "{input_type} is not a valid input type of identity transform" + )) + } + } + Transform::Void => Ok(input_type.clone()), + Transform::Unknown => Ok(Type::Primitive(PrimitiveType::String)), + Transform::Bucket(_) => { + if let Type::Primitive(p) = input_type { + match p { + PrimitiveType::Int + | PrimitiveType::Long + | PrimitiveType::Decimal { .. } + | PrimitiveType::Date + | PrimitiveType::Time + | PrimitiveType::Timestamp + | PrimitiveType::Timestamptz + | PrimitiveType::TimestampNs + | PrimitiveType::TimestamptzNs + | PrimitiveType::String + | PrimitiveType::Uuid + | PrimitiveType::Fixed(_) + | PrimitiveType::Binary => Ok(Type::Primitive(PrimitiveType::Int)), + _ => Err(format!( + "{input_type} is not a valid input type of bucket transform" + )), + } + } else { + Err(format!( + "{input_type} is not a valid input type of bucket transform" + )) + } + } + Transform::Truncate(_) => { + if let Type::Primitive(p) = input_type { + match p { + PrimitiveType::Int + | PrimitiveType::Long + | PrimitiveType::Decimal { .. } + | PrimitiveType::String => Ok(input_type.clone()), + _ => Err(format!( + "{input_type} is not a valid input type of truncate transform" + )), + } + } else { + Err(format!( + "{input_type} is not a valid input type of truncate transform" + )) + } + } + Transform::Year | Transform::Month | Transform::Day => { + if let Type::Primitive(p) = input_type { + match p { + PrimitiveType::Date + | PrimitiveType::Timestamp + | PrimitiveType::Timestamptz + | PrimitiveType::TimestampNs + | PrimitiveType::TimestamptzNs => Ok(Type::Primitive(PrimitiveType::Int)), + _ => Err(format!( + "{input_type} is not a valid input type of date transform" + )), + } + } else { + Err(format!( + "{input_type} is not a valid input type of date transform" + )) + } + } + Transform::Hour => { + if let Type::Primitive(p) = input_type { + match p { + PrimitiveType::Timestamp + | PrimitiveType::Timestamptz + | PrimitiveType::TimestampNs + | PrimitiveType::TimestamptzNs => Ok(Type::Primitive(PrimitiveType::Int)), + _ => Err(format!( + "{input_type} is not a valid input type of hour transform" + )), + } + } else { + Err(format!( + "{input_type} is not a valid input type of hour transform" + )) + } + } + } + } + + /// Whether the transform preserves the order of values. + pub fn preserves_order(&self) -> bool { + !matches!( + self, + Transform::Void | Transform::Bucket(_) | Transform::Unknown + ) + } + + /// Unique transform name to deduplicate equivalent transforms in a builder. + pub fn dedup_name(&self) -> String { + match self { + Transform::Year | Transform::Month | Transform::Day | Transform::Hour => { + "time".to_string() + } + _ => format!("{self}"), + } + } + + /// Whether ordering by this transform satisfies the ordering of another transform. + pub fn satisfies_order_of(&self, other: &Self) -> bool { + match self { + Transform::Identity => other.preserves_order(), + Transform::Hour => matches!( + other, + Transform::Hour | Transform::Day | Transform::Month | Transform::Year + ), + Transform::Day => matches!(other, Transform::Day | Transform::Month | Transform::Year), + Transform::Month => matches!(other, Transform::Month | Transform::Year), + _ => self == other, + } + } +} + +impl Display for Transform { + fn fmt(&self, f: &mut Formatter<'_>) -> std::fmt::Result { + match self { + Transform::Identity => write!(f, "identity"), + Transform::Bucket(n) => write!(f, "bucket[{}]", n), + Transform::Truncate(w) => write!(f, "truncate[{}]", w), + Transform::Year => write!(f, "year"), + Transform::Month => write!(f, "month"), + Transform::Day => write!(f, "day"), + Transform::Hour => write!(f, "hour"), + Transform::Void => write!(f, "void"), + Transform::Unknown => write!(f, "unknown"), + } + } +} + +impl FromStr for Transform { + type Err = String; + + fn from_str(s: &str) -> Result { + match s { + "identity" => Ok(Transform::Identity), + "year" => Ok(Transform::Year), + "month" => Ok(Transform::Month), + "day" => Ok(Transform::Day), + "hour" => Ok(Transform::Hour), + "void" => Ok(Transform::Void), + _ => { + if let Some(bucket_str) = + s.strip_prefix("bucket[").and_then(|s| s.strip_suffix(']')) + { + let n: u32 = bucket_str + .parse() + .map_err(|_| format!("Invalid bucket parameter: {}", bucket_str))?; + Ok(Transform::Bucket(n)) + } else if let Some(truncate_str) = s + .strip_prefix("truncate[") + .and_then(|s| s.strip_suffix(']')) + { + let w: u32 = truncate_str + .parse() + .map_err(|_| format!("Invalid truncate parameter: {}", truncate_str))?; + Ok(Transform::Truncate(w)) + } else { + Ok(Transform::Unknown) + } + } + } + } +} + +impl Serialize for Transform { + fn serialize(&self, serializer: S) -> Result + where + S: Serializer, + { + serializer.serialize_str(&self.to_string()) + } +} + +impl<'de> Deserialize<'de> for Transform { + fn deserialize(deserializer: D) -> Result + where + D: Deserializer<'de>, + { + let s = String::deserialize(deserializer)?; + Transform::from_str(&s).map_err(serde::de::Error::custom) + } +} diff --git a/crates/sail-iceberg/src/spec/types/mod.rs b/crates/sail-iceberg/src/spec/types/mod.rs new file mode 100644 index 0000000000..4be7e1b9b7 --- /dev/null +++ b/crates/sail-iceberg/src/spec/types/mod.rs @@ -0,0 +1,840 @@ +// 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. + +// [CREDIT]: https://raw.githubusercontent.com/apache/iceberg-rust/dc349284a4204c1a56af47fb3177ace6f9e899a0/crates/iceberg/src/spec/datatypes.rs + +pub mod values; + +use std::collections::HashMap; +use std::fmt; +use std::ops::Index; +use std::sync::{Arc, OnceLock}; + +use serde::de::{Error, IntoDeserializer, MapAccess, Visitor}; +use serde::{de, Deserialize, Deserializer, Serialize, Serializer}; +use serde_json::Value as JsonValue; +pub use values::*; + +/// Field name for list type. +pub const LIST_FIELD_NAME: &str = "element"; +/// Field name for map type's key. +pub const MAP_KEY_FIELD_NAME: &str = "key"; +/// Field name for map type's value. +pub const MAP_VALUE_FIELD_NAME: &str = "value"; + +pub(crate) const MAX_DECIMAL_BYTES: u32 = 24; +pub(crate) const MAX_DECIMAL_PRECISION: u32 = 38; + +mod _decimal { + use once_cell::sync::Lazy; + + use super::{MAX_DECIMAL_BYTES, MAX_DECIMAL_PRECISION}; + + // Max precision of bytes, starts from 1 + pub(super) static MAX_PRECISION: Lazy<[u32; MAX_DECIMAL_BYTES as usize]> = Lazy::new(|| { + let mut ret: [u32; 24] = [0; 24]; + for (i, prec) in ret.iter_mut().enumerate() { + *prec = 2f64.powi((8 * (i + 1) - 1) as i32).log10().floor() as u32; + } + + ret + }); + + // Required bytes of precision, starts from 1 + pub(super) static REQUIRED_LENGTH: Lazy<[u32; MAX_DECIMAL_PRECISION as usize]> = + Lazy::new(|| { + let mut ret: [u32; MAX_DECIMAL_PRECISION as usize] = + [0; MAX_DECIMAL_PRECISION as usize]; + + for (i, required_len) in ret.iter_mut().enumerate() { + for j in 0..MAX_PRECISION.len() { + if MAX_PRECISION[j] >= ((i + 1) as u32) { + *required_len = (j + 1) as u32; + break; + } + } + } + + ret + }); +} + +#[derive(Debug, PartialEq, Eq, Clone)] +/// All data types are either primitives or nested types, which are maps, lists, or structs. +pub enum Type { + /// Primitive types + Primitive(PrimitiveType), + /// Struct type + Struct(StructType), + /// List type. + List(ListType), + /// Map type + Map(MapType), +} + +impl fmt::Display for Type { + fn fmt(&self, f: &mut fmt::Formatter) -> fmt::Result { + match self { + Type::Primitive(primitive) => write!(f, "{}", primitive), + Type::Struct(s) => write!(f, "{}", s), + Type::List(_) => write!(f, "list"), + Type::Map(_) => write!(f, "map"), + } + } +} + +impl Type { + /// Whether the type is primitive type. + #[inline(always)] + pub fn is_primitive(&self) -> bool { + matches!(self, Type::Primitive(_)) + } + + /// Whether the type is struct type. + #[inline(always)] + pub fn is_struct(&self) -> bool { + matches!(self, Type::Struct(_)) + } + + /// Whether the type is nested type. + #[inline(always)] + pub fn is_nested(&self) -> bool { + matches!(self, Type::Struct(_) | Type::List(_) | Type::Map(_)) + } + + /// Convert Type to reference of PrimitiveType + pub fn as_primitive_type(&self) -> Option<&PrimitiveType> { + if let Type::Primitive(primitive_type) = self { + Some(primitive_type) + } else { + None + } + } + + /// Convert Type to StructType + pub fn into_struct_type(self) -> Option { + if let Type::Struct(struct_type) = self { + Some(struct_type) + } else { + None + } + } + + /// Return max precision for decimal given [`num_bytes`] bytes. + #[inline(always)] + pub fn decimal_max_precision(num_bytes: u32) -> Result { + if num_bytes == 0 || num_bytes > MAX_DECIMAL_BYTES { + return Err(format!( + "Decimal length larger than {MAX_DECIMAL_BYTES} is not supported: {num_bytes}" + )); + } + Ok(_decimal::MAX_PRECISION[num_bytes as usize - 1]) + } + + /// Returns minimum bytes required for decimal with [`precision`]. + #[inline(always)] + pub fn decimal_required_bytes(precision: u32) -> Result { + if precision == 0 || precision > MAX_DECIMAL_PRECISION { + return Err(format!( + "Decimals with precision larger than {MAX_DECIMAL_PRECISION} are not supported: {precision}" + )); + } + Ok(_decimal::REQUIRED_LENGTH[precision as usize - 1]) + } + + /// Creates decimal type. + #[inline(always)] + pub fn decimal(precision: u32, scale: u32) -> Result { + if precision == 0 || precision > MAX_DECIMAL_PRECISION { + return Err(format!( + "Decimals with precision larger than {MAX_DECIMAL_PRECISION} are not supported: {precision}" + )); + } + Ok(Type::Primitive(PrimitiveType::Decimal { precision, scale })) + } + + /// Check if it's float or double type. + #[inline(always)] + pub fn is_floating_type(&self) -> bool { + matches!( + self, + Type::Primitive(PrimitiveType::Float) | Type::Primitive(PrimitiveType::Double) + ) + } +} + +impl From for Type { + fn from(value: PrimitiveType) -> Self { + Self::Primitive(value) + } +} + +impl From for Type { + fn from(value: StructType) -> Self { + Type::Struct(value) + } +} + +impl From for Type { + fn from(value: ListType) -> Self { + Type::List(value) + } +} + +impl From for Type { + fn from(value: MapType) -> Self { + Type::Map(value) + } +} + +/// Primitive data types +#[derive(Debug, Serialize, Deserialize, PartialEq, Eq, Clone, Hash)] +#[serde(rename_all = "lowercase", remote = "Self")] +pub enum PrimitiveType { + /// True or False + Boolean, + /// 32-bit signed integer + Int, + /// 64-bit signed integer + Long, + /// 32-bit IEEE 754 floating point. + Float, + /// 64-bit IEEE 754 floating point. + Double, + /// Fixed point decimal + Decimal { + /// Precision, must be 38 or less + precision: u32, + /// Scale + scale: u32, + }, + /// Calendar date without timezone or time. + Date, + /// Time of day in microsecond precision, without date or timezone. + Time, + /// Timestamp in microsecond precision, without timezone + Timestamp, + /// Timestamp in microsecond precision, with timezone + Timestamptz, + /// Timestamp in nanosecond precision, without timezone + #[serde(rename = "timestamp_ns")] + TimestampNs, + /// Timestamp in nanosecond precision with timezone + #[serde(rename = "timestamptz_ns")] + TimestamptzNs, + /// Arbitrary-length character sequences encoded in utf-8 + String, + /// Universally Unique Identifiers, should use 16-byte fixed + Uuid, + /// Fixed length byte array + Fixed(u64), + /// Arbitrary-length byte array. + Binary, +} + +impl PrimitiveType { + /// Check whether literal is compatible with the type. + pub fn compatible(&self, literal: &PrimitiveLiteral) -> bool { + matches!( + (self, literal), + (PrimitiveType::Boolean, PrimitiveLiteral::Boolean(_)) + | (PrimitiveType::Int, PrimitiveLiteral::Int(_)) + | (PrimitiveType::Long, PrimitiveLiteral::Long(_)) + | (PrimitiveType::Float, PrimitiveLiteral::Float(_)) + | (PrimitiveType::Double, PrimitiveLiteral::Double(_)) + | (PrimitiveType::Decimal { .. }, PrimitiveLiteral::Int128(_)) + | (PrimitiveType::Date, PrimitiveLiteral::Int(_)) + | (PrimitiveType::Time, PrimitiveLiteral::Long(_)) + | (PrimitiveType::Timestamp, PrimitiveLiteral::Long(_)) + | (PrimitiveType::Timestamptz, PrimitiveLiteral::Long(_)) + | (PrimitiveType::TimestampNs, PrimitiveLiteral::Long(_)) + | (PrimitiveType::TimestamptzNs, PrimitiveLiteral::Long(_)) + | (PrimitiveType::String, PrimitiveLiteral::String(_)) + | (PrimitiveType::Uuid, PrimitiveLiteral::UInt128(_)) + | (PrimitiveType::Fixed(_), PrimitiveLiteral::Binary(_)) + | (PrimitiveType::Binary, PrimitiveLiteral::Binary(_)) + ) + } +} + +impl Serialize for Type { + fn serialize(&self, serializer: S) -> std::result::Result + where + S: Serializer, + { + let type_serde = _serde::SerdeType::from(self); + type_serde.serialize(serializer) + } +} + +impl<'de> Deserialize<'de> for Type { + fn deserialize(deserializer: D) -> std::result::Result + where + D: Deserializer<'de>, + { + let type_serde = _serde::SerdeType::deserialize(deserializer)?; + Ok(Type::from(type_serde)) + } +} + +impl<'de> Deserialize<'de> for PrimitiveType { + fn deserialize(deserializer: D) -> std::result::Result + where + D: Deserializer<'de>, + { + let s = String::deserialize(deserializer)?; + if s.starts_with("decimal") { + deserialize_decimal(s.into_deserializer()) + } else if s.starts_with("fixed") { + deserialize_fixed(s.into_deserializer()) + } else { + PrimitiveType::deserialize(s.into_deserializer()) + } + } +} + +impl Serialize for PrimitiveType { + fn serialize(&self, serializer: S) -> std::result::Result + where + S: Serializer, + { + match self { + PrimitiveType::Decimal { precision, scale } => { + serialize_decimal(precision, scale, serializer) + } + PrimitiveType::Fixed(l) => serialize_fixed(l, serializer), + _ => PrimitiveType::serialize(self, serializer), + } + } +} + +fn deserialize_decimal<'de, D>(deserializer: D) -> std::result::Result +where + D: Deserializer<'de>, +{ + let s = String::deserialize(deserializer)?; + let (precision, scale) = s + .trim_start_matches(r"decimal(") + .trim_end_matches(')') + .split_once(',') + .ok_or_else(|| D::Error::custom(format!("Decimal requires precision and scale: {s}")))?; + + Ok(PrimitiveType::Decimal { + precision: precision.trim().parse().map_err(D::Error::custom)?, + scale: scale.trim().parse().map_err(D::Error::custom)?, + }) +} + +fn serialize_decimal( + precision: &u32, + scale: &u32, + serializer: S, +) -> std::result::Result +where + S: Serializer, +{ + serializer.serialize_str(&format!("decimal({precision},{scale})")) +} + +fn deserialize_fixed<'de, D>(deserializer: D) -> std::result::Result +where + D: Deserializer<'de>, +{ + let fixed = String::deserialize(deserializer)? + .trim_start_matches(r"fixed[") + .trim_end_matches(']') + .to_owned(); + + fixed + .parse() + .map(PrimitiveType::Fixed) + .map_err(D::Error::custom) +} + +fn serialize_fixed(value: &u64, serializer: S) -> std::result::Result +where + S: Serializer, +{ + serializer.serialize_str(&format!("fixed[{value}]")) +} + +impl fmt::Display for PrimitiveType { + fn fmt(&self, f: &mut fmt::Formatter) -> fmt::Result { + match self { + PrimitiveType::Boolean => write!(f, "boolean"), + PrimitiveType::Int => write!(f, "int"), + PrimitiveType::Long => write!(f, "long"), + PrimitiveType::Float => write!(f, "float"), + PrimitiveType::Double => write!(f, "double"), + PrimitiveType::Decimal { precision, scale } => { + write!(f, "decimal({},{})", precision, scale) + } + PrimitiveType::Date => write!(f, "date"), + PrimitiveType::Time => write!(f, "time"), + PrimitiveType::Timestamp => write!(f, "timestamp"), + PrimitiveType::Timestamptz => write!(f, "timestamptz"), + PrimitiveType::TimestampNs => write!(f, "timestamp_ns"), + PrimitiveType::TimestamptzNs => write!(f, "timestamptz_ns"), + PrimitiveType::String => write!(f, "string"), + PrimitiveType::Uuid => write!(f, "uuid"), + PrimitiveType::Fixed(size) => write!(f, "fixed({})", size), + PrimitiveType::Binary => write!(f, "binary"), + } + } +} + +/// DataType for a specific struct +#[derive(Debug, Serialize, Clone, Default)] +#[serde(rename = "struct", tag = "type")] +pub struct StructType { + /// Struct fields + fields: Vec, + /// Lookup for index by field id + #[serde(skip_serializing)] + id_lookup: OnceLock>, + #[serde(skip_serializing)] + name_lookup: OnceLock>, +} + +impl<'de> Deserialize<'de> for StructType { + fn deserialize(deserializer: D) -> std::result::Result + where + D: Deserializer<'de>, + { + #[derive(Deserialize)] + #[serde(field_identifier, rename_all = "lowercase")] + enum Field { + Type, + Fields, + } + + struct StructTypeVisitor; + + impl<'de> Visitor<'de> for StructTypeVisitor { + type Value = StructType; + + fn expecting(&self, formatter: &mut fmt::Formatter) -> fmt::Result { + formatter.write_str("struct") + } + + fn visit_map(self, mut map: V) -> std::result::Result + where + V: MapAccess<'de>, + { + let mut fields = None; + while let Some(key) = map.next_key()? { + match key { + Field::Type => (), + Field::Fields => { + if fields.is_some() { + return Err(serde::de::Error::duplicate_field("fields")); + } + fields = Some(map.next_value()?); + } + } + } + let fields: Vec = + fields.ok_or_else(|| de::Error::missing_field("fields"))?; + + Ok(StructType::new(fields)) + } + } + + const FIELDS: &[&str] = &["type", "fields"]; + deserializer.deserialize_struct("struct", FIELDS, StructTypeVisitor) + } +} + +impl StructType { + /// Creates a struct type with the given fields. + pub fn new(fields: Vec) -> Self { + Self { + fields, + id_lookup: OnceLock::new(), + name_lookup: OnceLock::new(), + } + } + + /// Get struct field with certain id + pub fn field_by_id(&self, id: i32) -> Option<&NestedFieldRef> { + self.field_id_to_index(id).map(|idx| &self.fields[idx]) + } + + fn field_id_to_index(&self, field_id: i32) -> Option { + self.id_lookup + .get_or_init(|| { + HashMap::from_iter(self.fields.iter().enumerate().map(|(i, x)| (x.id, i))) + }) + .get(&field_id) + .copied() + } + + /// Get struct field with certain field name + pub fn field_by_name(&self, name: &str) -> Option<&NestedFieldRef> { + self.field_name_to_index(name).map(|idx| &self.fields[idx]) + } + + fn field_name_to_index(&self, name: &str) -> Option { + self.name_lookup + .get_or_init(|| { + HashMap::from_iter( + self.fields + .iter() + .enumerate() + .map(|(i, x)| (x.name.clone(), i)), + ) + }) + .get(name) + .copied() + } + + /// Get fields. + pub fn fields(&self) -> &[NestedFieldRef] { + &self.fields + } +} + +impl PartialEq for StructType { + fn eq(&self, other: &Self) -> bool { + self.fields == other.fields + } +} + +impl Eq for StructType {} + +impl Index for StructType { + type Output = NestedField; + + fn index(&self, index: usize) -> &Self::Output { + &self.fields[index] + } +} + +impl fmt::Display for StructType { + fn fmt(&self, f: &mut fmt::Formatter) -> fmt::Result { + write!(f, "struct<")?; + for field in &self.fields { + write!(f, "{}", field.field_type)?; + } + write!(f, ">") + } +} + +#[derive(Debug, PartialEq, Serialize, Deserialize, Eq, Clone)] +#[serde(from = "SerdeNestedField", into = "SerdeNestedField")] +/// A struct is a tuple of typed values. Each field in the tuple is named and has an integer id that is unique in the table schema. +/// Each field can be either optional or required, meaning that values can (or cannot) be null. Fields may be any type. +/// Fields may have an optional comment or doc string. Fields can have default values. +pub struct NestedField { + /// Id unique in table schema + pub id: i32, + /// Field Name + pub name: String, + /// Optional or required + pub required: bool, + /// Datatype + pub field_type: Box, + /// Fields may have an optional comment or doc string. + pub doc: Option, + /// Used to populate the field's value for all records that were written before the field was added to the schema + pub initial_default: Option, + /// Used to populate the field's value for any records written after the field was added to the schema, if the writer does not supply the field's value + pub write_default: Option, +} + +#[derive(Debug, Serialize, Deserialize, PartialEq, Eq, Clone)] +#[serde(rename_all = "kebab-case")] +struct SerdeNestedField { + pub id: i32, + pub name: String, + pub required: bool, + #[serde(rename = "type")] + pub field_type: Box, + #[serde(skip_serializing_if = "Option::is_none")] + pub doc: Option, + #[serde(skip_serializing_if = "Option::is_none")] + pub initial_default: Option, + #[serde(skip_serializing_if = "Option::is_none")] + pub write_default: Option, +} + +impl From for NestedField { + fn from(value: SerdeNestedField) -> Self { + NestedField { + id: value.id, + name: value.name, + required: value.required, + initial_default: value.initial_default.and_then(|x| { + Literal::try_from_json(x, &value.field_type) + .ok() + .and_then(|x| x) + }), + write_default: value.write_default.and_then(|x| { + Literal::try_from_json(x, &value.field_type) + .ok() + .and_then(|x| x) + }), + field_type: value.field_type, + doc: value.doc, + } + } +} + +impl From for SerdeNestedField { + fn from(value: NestedField) -> Self { + let initial_default = value + .initial_default + .and_then(|x| x.try_into_json(&value.field_type).ok()); + let write_default = value + .write_default + .and_then(|x| x.try_into_json(&value.field_type).ok()); + SerdeNestedField { + id: value.id, + name: value.name, + required: value.required, + field_type: value.field_type, + doc: value.doc, + initial_default, + write_default, + } + } +} + +/// Reference to nested field. +pub type NestedFieldRef = Arc; + +impl NestedField { + /// Construct a new field. + pub fn new(id: i32, name: impl ToString, field_type: Type, required: bool) -> Self { + Self { + id, + name: name.to_string(), + required, + field_type: Box::new(field_type), + doc: None, + initial_default: None, + write_default: None, + } + } + + /// Construct a required field. + pub fn required(id: i32, name: impl ToString, field_type: Type) -> Self { + Self::new(id, name, field_type, true) + } + + /// Construct an optional field. + pub fn optional(id: i32, name: impl ToString, field_type: Type) -> Self { + Self::new(id, name, field_type, false) + } + + /// Construct list type's element field. + pub fn list_element(id: i32, field_type: Type, required: bool) -> Self { + Self::new(id, LIST_FIELD_NAME, field_type, required) + } + + /// Construct map type's key field. + pub fn map_key_element(id: i32, field_type: Type) -> Self { + Self::required(id, MAP_KEY_FIELD_NAME, field_type) + } + + /// Construct map type's value field. + pub fn map_value_element(id: i32, field_type: Type, required: bool) -> Self { + Self::new(id, MAP_VALUE_FIELD_NAME, field_type, required) + } + + /// Set the field's doc. + pub fn with_doc(mut self, doc: impl ToString) -> Self { + self.doc = Some(doc.to_string()); + self + } + + /// Set the field's initial default value. + pub fn with_initial_default(mut self, value: Literal) -> Self { + self.initial_default = Some(value); + self + } + + /// Set the field's initial default value. + pub fn with_write_default(mut self, value: Literal) -> Self { + self.write_default = Some(value); + self + } + + /// Set the id of the field. + #[allow(unused)] + pub(crate) fn with_id(mut self, id: i32) -> Self { + self.id = id; + self + } +} + +impl fmt::Display for NestedField { + fn fmt(&self, f: &mut fmt::Formatter) -> fmt::Result { + write!(f, "{}: ", self.id)?; + write!(f, "{}: ", self.name)?; + if self.required { + write!(f, "required ")?; + } else { + write!(f, "optional ")?; + } + write!(f, "{} ", self.field_type)?; + if let Some(doc) = &self.doc { + write!(f, "{}", doc)?; + } + Ok(()) + } +} + +#[derive(Debug, PartialEq, Eq, Clone)] +/// A list is a collection of values with some element type. The element field has an integer id that is unique in the table schema. +/// Elements can be either optional or required. Element types may be any type. +pub struct ListType { + /// Element field of list type. + pub element_field: NestedFieldRef, +} + +impl ListType { + /// Construct a list type with the given element field. + pub fn new(element_field: NestedFieldRef) -> Self { + Self { element_field } + } +} + +/// Module for type serialization/deserialization. +pub(super) mod _serde { + use std::borrow::Cow; + + use serde::{Deserialize, Serialize}; + + use super::Type::Map; + use super::{ListType, MapType, NestedField, NestedFieldRef, PrimitiveType, StructType, Type}; + + /// List type for serialization and deserialization + #[derive(Serialize, Deserialize)] + #[serde(untagged)] + pub(super) enum SerdeType<'a> { + #[serde(rename_all = "kebab-case")] + List { + r#type: String, + element_id: i32, + element_required: bool, + element: Cow<'a, Type>, + }, + Struct { + r#type: String, + fields: Cow<'a, [NestedFieldRef]>, + }, + #[serde(rename_all = "kebab-case")] + Map { + r#type: String, + key_id: i32, + key: Cow<'a, Type>, + value_id: i32, + value_required: bool, + value: Cow<'a, Type>, + }, + Primitive(PrimitiveType), + } + + impl From> for Type { + fn from(value: SerdeType) -> Self { + match value { + SerdeType::List { + r#type: _, + element_id, + element_required, + element, + } => Self::List(ListType { + element_field: NestedField::list_element( + element_id, + element.into_owned(), + element_required, + ) + .into(), + }), + SerdeType::Map { + r#type: _, + key_id, + key, + value_id, + value_required, + value, + } => Map(MapType { + key_field: NestedField::map_key_element(key_id, key.into_owned()).into(), + value_field: NestedField::map_value_element( + value_id, + value.into_owned(), + value_required, + ) + .into(), + }), + SerdeType::Struct { r#type: _, fields } => { + Self::Struct(StructType::new(fields.into_owned())) + } + SerdeType::Primitive(p) => Self::Primitive(p.clone()), + } + } + } + + impl<'a> From<&'a Type> for SerdeType<'a> { + fn from(value: &'a Type) -> Self { + match value { + Type::List(list) => SerdeType::List { + r#type: "list".to_string(), + element_id: list.element_field.id, + element_required: list.element_field.required, + element: Cow::Borrowed(&list.element_field.field_type), + }, + Type::Map(map) => SerdeType::Map { + r#type: "map".to_string(), + key_id: map.key_field.id, + key: Cow::Borrowed(&map.key_field.field_type), + value_id: map.value_field.id, + value_required: map.value_field.required, + value: Cow::Borrowed(&map.value_field.field_type), + }, + Type::Struct(s) => SerdeType::Struct { + r#type: "struct".to_string(), + fields: Cow::Borrowed(&s.fields), + }, + Type::Primitive(p) => SerdeType::Primitive(p.clone()), + } + } + } +} + +#[derive(Debug, PartialEq, Eq, Clone)] +/// A map is a collection of key-value pairs with a key type and a value type. +/// Both the key field and value field each have an integer id that is unique in the table schema. +/// Map keys are required and map values can be either optional or required. +/// Both map keys and map values may be any type, including nested types. +pub struct MapType { + /// Field for key. + pub key_field: NestedFieldRef, + /// Field for value. + pub value_field: NestedFieldRef, +} + +impl MapType { + /// Construct a map type with the given key and value fields. + pub fn new(key_field: NestedFieldRef, value_field: NestedFieldRef) -> Self { + Self { + key_field, + value_field, + } + } +} diff --git a/crates/sail-iceberg/src/spec/types/values.rs b/crates/sail-iceberg/src/spec/types/values.rs new file mode 100644 index 0000000000..443b006813 --- /dev/null +++ b/crates/sail-iceberg/src/spec/types/values.rs @@ -0,0 +1,463 @@ +// 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. + +// [CREDIT]: https://raw.githubusercontent.com/apache/iceberg-rust/dc349284a4204c1a56af47fb3177ace6f9e899a0/crates/iceberg/src/spec/values.rs + +use ordered_float::OrderedFloat; +use serde::{Deserialize, Serialize}; +use serde_json::Value as JsonValue; + +/// Literal values used in Iceberg +#[derive(Debug, Clone, PartialEq, Eq, Hash, Serialize, Deserialize)] +#[serde(untagged)] +pub enum Literal { + Primitive(PrimitiveLiteral), + Struct(Vec<(String, Option)>), + List(Vec>), + Map(Vec<(Literal, Option)>), +} + +/// Primitive literal values +#[derive(Debug, Clone, PartialEq, Eq, Hash, Serialize, Deserialize)] +#[serde(untagged)] +pub enum PrimitiveLiteral { + Boolean(bool), + Int(i32), + Long(i64), + Float(OrderedFloat), + Double(OrderedFloat), + Int128(i128), + String(String), + UInt128(u128), + Binary(Vec), +} + +#[derive(Debug, Clone, PartialEq, Eq, Hash, Serialize, Deserialize)] +#[serde(rename_all = "kebab-case")] +/// Typed single-value used for lower/upper bounds +pub struct Datum { + /// Primitive data type of the datum + pub r#type: crate::spec::types::PrimitiveType, + /// Primitive literal value + pub literal: PrimitiveLiteral, +} + +impl Datum { + pub fn new(r#type: crate::spec::types::PrimitiveType, literal: PrimitiveLiteral) -> Self { + Self { r#type, literal } + } +} + +impl Literal { + pub fn try_from_json( + value: JsonValue, + data_type: &crate::spec::types::Type, + ) -> Result, String> { + use chrono::{NaiveDate, NaiveTime, Timelike}; + use serde_json::Number; + + use crate::spec::types::{PrimitiveType, Type}; + + fn number_to_i32(n: &Number) -> Result { + n.as_i64() + .ok_or_else(|| "Failed to convert json number to i32".to_string()) + .and_then(|v| i32::try_from(v).map_err(|e| e.to_string())) + } + fn number_to_i64(n: &Number) -> Result { + n.as_i64() + .ok_or_else(|| "Failed to convert json number to i64".to_string()) + } + fn number_to_f32(n: &Number) -> Result { + n.as_f64() + .ok_or_else(|| "Failed to convert json number to f32".to_string()) + .map(|v| v as f32) + } + fn number_to_f64(n: &Number) -> Result { + n.as_f64() + .ok_or_else(|| "Failed to convert json number to f64".to_string()) + } + + fn parse_date_to_days(s: &str) -> Result { + let d = NaiveDate::parse_from_str(s, "%Y-%m-%d") + .map_err(|e| format!("Invalid date format: {}", e))?; + let epoch = NaiveDate::from_ymd_opt(1970, 1, 1).ok_or("Bad epoch")?; + Ok((d - epoch).num_days() as i32) + } + fn parse_time_to_micros(s: &str) -> Result { + // Accept up to nanosecond precision, truncate to microseconds + let fmt_candidates = ["%H:%M:%S", "%H:%M:%S%.f"]; + let mut last_err: Option = None; + for fmt in &fmt_candidates { + match NaiveTime::parse_from_str(s, fmt) { + Ok(t) => { + let nanos = t.num_seconds_from_midnight() as i64 * 1_000_000_000 + + (t.nanosecond() as i64); + return Ok(nanos / 1_000); + } + Err(e) => last_err = Some(e.to_string()), + } + } + Err(last_err.unwrap_or_else(|| "Invalid time".to_string())) + } + fn parse_ts_to_micros(s: &str) -> Result { + // Accept naive timestamp like 2020-01-01T12:34:56[.ffffff] + let fmt_candidates = [ + "%Y-%m-%dT%H:%M:%S", + "%Y-%m-%dT%H:%M:%S%.f", + "%Y-%m-%d %H:%M:%S", + "%Y-%m-%d %H:%M:%S%.f", + ]; + let mut last_err: Option = None; + for fmt in &fmt_candidates { + match chrono::NaiveDateTime::parse_from_str(s, fmt) { + Ok(dt) => { + let epoch = NaiveDate::from_ymd_opt(1970, 1, 1) + .ok_or("Bad epoch")? + .and_hms_nano_opt(0, 0, 0, 0) + .ok_or("Bad epoch")?; + let micros = (dt - epoch).num_microseconds().ok_or("overflow")?; + return Ok(micros); + } + Err(e) => last_err = Some(e.to_string()), + } + } + // Try with timezone (treated as UTC) + match chrono::DateTime::parse_from_rfc3339(s) + .or_else(|_| chrono::DateTime::parse_from_str(s, "%Y-%m-%dT%H:%M:%S%.f%:z")) + { + Ok(dt) => Ok(dt.timestamp_micros()), + Err(_) => Err(last_err.unwrap_or_else(|| "Invalid timestamp".to_string())), + } + } + + fn parse_uuid_to_u128(s: &str) -> Result { + let u = uuid::Uuid::parse_str(s).map_err(|e| e.to_string())?; + let bytes = u.as_bytes(); + let mut acc: u128 = 0; + for b in bytes.iter() { + acc = (acc << 8) | (*b as u128); + } + Ok(acc) + } + + fn parse_decimal_to_i128(s: &str, scale: u32) -> Result { + let s = s.trim(); + if s.is_empty() { + return Err("empty decimal".to_string()); + } + let negative = s.starts_with('-'); + let s = if negative || s.starts_with('+') { + &s[1..] + } else { + s + }; + let mut int_part: i128 = 0; + let mut frac_part: i128 = 0; + let mut frac_len: u32 = 0; + let mut seen_dot = false; + for ch in s.chars() { + if ch == '.' { + if seen_dot { + return Err("multiple decimal points".to_string()); + } + seen_dot = true; + continue; + } + if !ch.is_ascii_digit() { + return Err("invalid decimal".to_string()); + } + let d = (ch as u8 - b'0') as i128; + if !seen_dot { + int_part = int_part + .checked_mul(10) + .and_then(|v| v.checked_add(d)) + .ok_or("overflow")?; + } else if frac_len < scale { + frac_part = frac_part + .checked_mul(10) + .and_then(|v| v.checked_add(d)) + .ok_or("overflow")?; + frac_len += 1; + } else { + // truncate extra fractional digits beyond scale (rounding not applied) + } + } + let pow10 = 10i128.pow(scale); + let scaled = int_part + .checked_mul(pow10) + .and_then(|v| v.checked_add(frac_part * 10i128.pow(scale - frac_len))) + .ok_or("overflow")?; + Ok(if negative { -scaled } else { scaled }) + } + + Ok(match (data_type, value) { + (_, JsonValue::Null) => None, + (Type::Primitive(PrimitiveType::Boolean), JsonValue::Bool(v)) => { + Some(Literal::Primitive(PrimitiveLiteral::Boolean(v))) + } + (Type::Primitive(PrimitiveType::Int), JsonValue::Number(n)) => Some( + Literal::Primitive(PrimitiveLiteral::Int(number_to_i32(&n)?)), + ), + (Type::Primitive(PrimitiveType::Long), JsonValue::Number(n)) => Some( + Literal::Primitive(PrimitiveLiteral::Long(number_to_i64(&n)?)), + ), + (Type::Primitive(PrimitiveType::Float), JsonValue::Number(n)) => Some( + Literal::Primitive(PrimitiveLiteral::Float(OrderedFloat(number_to_f32(&n)?))), + ), + (Type::Primitive(PrimitiveType::Double), JsonValue::Number(n)) => Some( + Literal::Primitive(PrimitiveLiteral::Double(OrderedFloat(number_to_f64(&n)?))), + ), + (Type::Primitive(PrimitiveType::Date), JsonValue::String(s)) => Some( + Literal::Primitive(PrimitiveLiteral::Int(parse_date_to_days(&s)?)), + ), + (Type::Primitive(PrimitiveType::Time), JsonValue::String(s)) => Some( + Literal::Primitive(PrimitiveLiteral::Long(parse_time_to_micros(&s)?)), + ), + (Type::Primitive(PrimitiveType::Timestamp), JsonValue::String(s)) => Some( + Literal::Primitive(PrimitiveLiteral::Long(parse_ts_to_micros(&s)?)), + ), + (Type::Primitive(PrimitiveType::Timestamptz), JsonValue::String(s)) => Some( + Literal::Primitive(PrimitiveLiteral::Long(parse_ts_to_micros(&s)?)), + ), + (Type::Primitive(PrimitiveType::TimestampNs), JsonValue::String(s)) => Some( + Literal::Primitive(PrimitiveLiteral::Long(parse_ts_to_micros(&s)? * 1000)), + ), + (Type::Primitive(PrimitiveType::TimestamptzNs), JsonValue::String(s)) => Some( + Literal::Primitive(PrimitiveLiteral::Long(parse_ts_to_micros(&s)? * 1000)), + ), + (Type::Primitive(PrimitiveType::String), JsonValue::String(s)) => { + Some(Literal::Primitive(PrimitiveLiteral::String(s))) + } + (Type::Primitive(PrimitiveType::Uuid), JsonValue::String(s)) => Some( + Literal::Primitive(PrimitiveLiteral::UInt128(parse_uuid_to_u128(&s)?)), + ), + (Type::Primitive(PrimitiveType::Binary), JsonValue::String(s)) => { + Some(Literal::Primitive(PrimitiveLiteral::Binary(s.into_bytes()))) + } + (Type::Primitive(PrimitiveType::Fixed(_)), JsonValue::String(s)) => { + Some(Literal::Primitive(PrimitiveLiteral::Binary(s.into_bytes()))) + } + (Type::Primitive(PrimitiveType::Decimal { scale, .. }), JsonValue::String(s)) => Some( + Literal::Primitive(PrimitiveLiteral::Int128(parse_decimal_to_i128(&s, *scale)?)), + ), + (Type::Struct(struct_ty), JsonValue::Object(mut map)) => { + let mut out = Vec::with_capacity(struct_ty.fields().len()); + for field in struct_ty.fields() { + let key = field.id.to_string(); + let v = map.remove(&key); + let val = match v { + Some(json) => Literal::try_from_json(json, &field.field_type) + .and_then(|opt| { + opt.ok_or_else(|| "Key of map cannot be null".to_string()) + }) + .ok(), + None => None, + }; + out.push((key, val)); + } + Some(Literal::Struct(out)) + } + (Type::List(list_ty), JsonValue::Array(arr)) => { + let mut out = Vec::with_capacity(arr.len()); + for item in arr.into_iter() { + let elem = Literal::try_from_json(item, &list_ty.element_field.field_type)?; + out.push(elem); + } + Some(Literal::List(out)) + } + (Type::Map(map_ty), JsonValue::Object(mut obj)) => { + let keys = obj.remove("keys").unwrap_or(JsonValue::Array(vec![])); + let vals = obj.remove("values").unwrap_or(JsonValue::Array(vec![])); + let (JsonValue::Array(keys), JsonValue::Array(vals)) = (keys, vals) else { + return Err("Invalid map JSON".to_string()); + }; + if keys.len() != vals.len() { + return Err("Keys and values length mismatch".to_string()); + } + let mut out = Vec::with_capacity(keys.len()); + for (k, v) in keys.into_iter().zip(vals.into_iter()) { + let key = Literal::try_from_json(k, &map_ty.key_field.field_type) + .and_then(|opt| opt.ok_or_else(|| "Map key cannot be null".to_string()))?; + let val = Literal::try_from_json(v, &map_ty.value_field.field_type)?; + out.push((key, val)); + } + Some(Literal::Map(out)) + } + // Fallback: store as string for unsupported combinations + (_, other) => Some(Literal::Primitive(PrimitiveLiteral::String( + other.to_string(), + ))), + }) + } + + pub fn try_into_json(&self, data_type: &crate::spec::types::Type) -> Result { + use chrono::{NaiveDate, NaiveTime}; + use serde_json::Number; + + use crate::spec::types::{PrimitiveType, Type}; + + fn days_to_date_str(days: i32) -> String { + #[allow(clippy::unwrap_used)] + let epoch = NaiveDate::from_ymd_opt(1970, 1, 1).unwrap(); + let d = epoch + chrono::Days::new(days as u64); + d.to_string() + } + fn micros_to_time_str(us: i64) -> String { + let secs = us.div_euclid(1_000_000); + let rem = (us.rem_euclid(1_000_000)) as u32; + #[allow(clippy::unwrap_used)] + let t = NaiveTime::from_num_seconds_from_midnight_opt(secs as u32, rem * 1000) + .unwrap_or(NaiveTime::from_hms_opt(0, 0, 0).unwrap()); + t.format("%H:%M:%S%.f").to_string() + } + fn micros_to_datetime_str(us: i64) -> String { + let secs = us.div_euclid(1_000_000); + let rem = (us.rem_euclid(1_000_000)) as u32; + #[allow(clippy::unwrap_used)] + let base = NaiveDate::from_ymd_opt(1970, 1, 1) + .unwrap() + .and_hms_nano_opt(0, 0, 0, 0) + .unwrap(); + let dt = base + .checked_add_signed(chrono::Duration::seconds(secs)) + .and_then(|d| { + d.checked_add_signed(chrono::Duration::nanoseconds((rem as i64) * 1000)) + }) + .unwrap_or(base); + dt.format("%Y-%m-%dT%H:%M:%S%.f").to_string() + } + + match (self, data_type) { + (Literal::Primitive(prim), Type::Primitive(prim_ty)) => match (prim_ty, prim) { + (PrimitiveType::Boolean, PrimitiveLiteral::Boolean(v)) => Ok(JsonValue::Bool(*v)), + (PrimitiveType::Int, PrimitiveLiteral::Int(v)) => { + Ok(JsonValue::Number((*v).into())) + } + (PrimitiveType::Long, PrimitiveLiteral::Long(v)) => { + Ok(JsonValue::Number((*v).into())) + } + (PrimitiveType::Float, PrimitiveLiteral::Float(v)) => Number::from_f64(v.0 as f64) + .map(JsonValue::Number) + .ok_or_else(|| "Invalid float".to_string()), + (PrimitiveType::Double, PrimitiveLiteral::Double(v)) => Number::from_f64(v.0) + .map(JsonValue::Number) + .ok_or_else(|| "Invalid double".to_string()), + (PrimitiveType::Date, PrimitiveLiteral::Int(v)) => { + Ok(JsonValue::String(days_to_date_str(*v))) + } + (PrimitiveType::Time, PrimitiveLiteral::Long(v)) => { + Ok(JsonValue::String(micros_to_time_str(*v))) + } + (PrimitiveType::Timestamp, PrimitiveLiteral::Long(v)) => { + Ok(JsonValue::String(micros_to_datetime_str(*v))) + } + (PrimitiveType::Timestamptz, PrimitiveLiteral::Long(v)) => { + Ok(JsonValue::String(micros_to_datetime_str(*v))) + } + (PrimitiveType::TimestampNs, PrimitiveLiteral::Long(v)) => { + Ok(JsonValue::String(micros_to_datetime_str(*v / 1000))) + } + (PrimitiveType::TimestamptzNs, PrimitiveLiteral::Long(v)) => { + Ok(JsonValue::String(micros_to_datetime_str(*v / 1000))) + } + (PrimitiveType::String, PrimitiveLiteral::String(s)) => { + Ok(JsonValue::String(s.clone())) + } + (PrimitiveType::Uuid, PrimitiveLiteral::UInt128(u)) => { + let mut bytes = [0u8; 16]; + let mut tmp = *u; + for i in (0..16).rev() { + bytes[i] = (tmp & 0xFF) as u8; + tmp >>= 8; + } + let u = uuid::Uuid::from_bytes(bytes); + Ok(JsonValue::String(u.to_string())) + } + (PrimitiveType::Decimal { scale, .. }, PrimitiveLiteral::Int128(v)) => { + // render scaled decimal as string + let neg = *v < 0; + let x = v.abs(); + let scale = *scale as usize; + let mut s = if scale == 0 { + x.to_string() + } else { + let mut frac = String::with_capacity(scale); + let mut tmp = x; + for _ in 0..scale { + frac.insert(0, char::from(b'0' + (tmp % 10) as u8)); + tmp /= 10; + } + let int_part = tmp.to_string(); + format!("{}.{frac}", int_part) + }; + if neg { + s.insert(0, '-'); + } + Ok(JsonValue::String(s)) + } + (PrimitiveType::Binary, PrimitiveLiteral::Binary(b)) => { + // store as UTF-8 string of bytes if valid; otherwise hex-ish + Ok(JsonValue::String(String::from_utf8_lossy(b).into_owned())) + } + (PrimitiveType::Fixed(_), PrimitiveLiteral::Binary(b)) => { + Ok(JsonValue::String(String::from_utf8_lossy(b).into_owned())) + } + // Fallback for mismatched pairs + _ => Ok(JsonValue::Null), + }, + (Literal::Struct(s), Type::Struct(struct_ty)) => { + let mut map = serde_json::Map::with_capacity(struct_ty.fields().len()); + for ((id_str, val_opt), field) in s.iter().zip(struct_ty.fields()) { + let key = id_str.clone(); + let json = match val_opt { + Some(l) => l.try_into_json(&field.field_type)?, + None => JsonValue::Null, + }; + map.insert(key, json); + } + Ok(JsonValue::Object(map)) + } + (Literal::List(list), Type::List(list_ty)) => { + let mut arr = Vec::with_capacity(list.len()); + for opt in list.iter() { + match opt { + Some(l) => arr.push(l.try_into_json(&list_ty.element_field.field_type)?), + None => arr.push(JsonValue::Null), + } + } + Ok(JsonValue::Array(arr)) + } + (Literal::Map(map), Type::Map(map_ty)) => { + let mut keys = Vec::with_capacity(map.len()); + let mut vals = Vec::with_capacity(map.len()); + for (k, v) in map.iter() { + keys.push(k.try_into_json(&map_ty.key_field.field_type)?); + vals.push(match v { + Some(l) => l.try_into_json(&map_ty.value_field.field_type)?, + None => JsonValue::Null, + }); + } + let mut obj = serde_json::Map::new(); + obj.insert("keys".to_string(), JsonValue::Array(keys)); + obj.insert("values".to_string(), JsonValue::Array(vals)); + Ok(JsonValue::Object(obj)) + } + // Fallback + _ => Ok(JsonValue::Null), + } + } +} diff --git a/crates/sail-iceberg/src/spec/views/mod.rs b/crates/sail-iceberg/src/spec/views/mod.rs new file mode 100644 index 0000000000..d58a3264f1 --- /dev/null +++ b/crates/sail-iceberg/src/spec/views/mod.rs @@ -0,0 +1,24 @@ +// 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. + +pub mod view_metadata; +pub mod view_metadata_builder; +pub mod view_version; + +pub use view_metadata::*; +pub use view_metadata_builder::*; +pub use view_version::*; diff --git a/crates/sail-iceberg/src/spec/views/view_metadata.rs b/crates/sail-iceberg/src/spec/views/view_metadata.rs new file mode 100644 index 0000000000..c14058fc72 --- /dev/null +++ b/crates/sail-iceberg/src/spec/views/view_metadata.rs @@ -0,0 +1,21 @@ +// 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. + +// Awareness stub +#[allow(dead_code)] +#[derive(Debug, Clone)] +pub struct ViewMetadata; diff --git a/crates/sail-iceberg/src/spec/views/view_metadata_builder.rs b/crates/sail-iceberg/src/spec/views/view_metadata_builder.rs new file mode 100644 index 0000000000..249350e367 --- /dev/null +++ b/crates/sail-iceberg/src/spec/views/view_metadata_builder.rs @@ -0,0 +1,21 @@ +// 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. + +// Awareness stub +#[allow(dead_code)] +#[derive(Debug, Clone)] +pub struct ViewMetadataBuilder; diff --git a/crates/sail-iceberg/src/spec/views/view_version.rs b/crates/sail-iceberg/src/spec/views/view_version.rs new file mode 100644 index 0000000000..17826713bc --- /dev/null +++ b/crates/sail-iceberg/src/spec/views/view_version.rs @@ -0,0 +1,55 @@ +// 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 serde::{Deserialize, Serialize}; + +use crate::spec::catalog::NamespaceIdent; +use crate::spec::SchemaId; + +#[derive(Clone, Debug, PartialEq, Serialize, Deserialize)] +#[serde(rename_all = "kebab-case")] +pub struct ViewVersion { + #[serde(rename = "version-id")] + pub version_id: i32, + #[serde(rename = "timestamp-ms")] + pub timestamp_ms: i64, + #[serde(rename = "schema-id")] + pub schema_id: SchemaId, + #[serde(default, rename = "default-catalog")] + pub default_catalog: Option, + #[serde(rename = "default-namespace")] + pub default_namespace: NamespaceIdent, + #[serde(default)] + pub summary: HashMap, + pub representations: ViewRepresentations, +} + +#[derive(Clone, Debug, PartialEq, Serialize, Deserialize)] +pub struct ViewRepresentations(pub Vec); + +#[derive(Clone, Debug, PartialEq, Serialize, Deserialize)] +#[serde(tag = "type", rename_all = "kebab-case")] +pub enum ViewRepresentation { + #[serde(rename_all = "kebab-case")] + Sql { + sql: String, + #[serde(default)] + dialect: String, + }, +} diff --git a/crates/sail-iceberg/src/table_format.rs b/crates/sail-iceberg/src/table_format.rs new file mode 100644 index 0000000000..5c892215d1 --- /dev/null +++ b/crates/sail-iceberg/src/table_format.rs @@ -0,0 +1,210 @@ +use std::sync::Arc; + +use async_trait::async_trait; +use datafusion::catalog::{Session, TableProvider}; +use datafusion::common::{not_impl_err, plan_err, DataFusionError, Result}; +use datafusion::physical_plan::ExecutionPlan; +use sail_common_datafusion::datasource::{SinkInfo, SourceInfo, TableFormat}; +use url::Url; + +use crate::datasource::provider::IcebergTableProvider; +use crate::spec::{PartitionSpec, Schema, Snapshot, TableMetadata}; + +#[derive(Debug)] +pub struct IcebergTableFormat; + +#[async_trait] +impl TableFormat for IcebergTableFormat { + fn name(&self) -> &str { + "iceberg" + } + + async fn create_provider( + &self, + ctx: &dyn Session, + info: SourceInfo, + ) -> Result> { + let SourceInfo { + paths, + schema: _schema, + constraints: _, + partition_by: _, + bucket_by: _, + sort_order: _, + options: _options, + } = info; + + log::trace!("Creating table provider for paths: {:?}", paths); + let table_url = Self::parse_table_url(ctx, paths).await?; + log::trace!("Parsed table URL: {}", table_url); + + let (iceberg_schema, snapshot, partition_specs) = + load_table_metadata(ctx, &table_url).await?; + log::trace!("Loaded metadata, snapshot_id: {}", snapshot.snapshot_id()); + + let provider = IcebergTableProvider::new( + table_url.to_string(), + iceberg_schema, + snapshot, + partition_specs, + )?; + Ok(Arc::new(provider)) + } + + async fn create_writer( + &self, + _ctx: &dyn Session, + _info: SinkInfo, + ) -> Result> { + not_impl_err!("Writing to Iceberg tables is not yet implemented") + } +} + +impl IcebergTableFormat { + async fn parse_table_url(ctx: &dyn Session, paths: Vec) -> Result { + if paths.len() != 1 { + return plan_err!( + "Iceberg table requires exactly one path, got {}", + paths.len() + ); + } + + let path = &paths[0]; + let mut table_url = Url::parse(path).map_err(|e| DataFusionError::External(Box::new(e)))?; + + if !table_url.path().ends_with('/') { + table_url.set_path(&format!("{}/", table_url.path())); + } + + // Validate that we can access the object store + let _object_store = ctx + .runtime_env() + .object_store_registry + .get_store(&table_url) + .map_err(|e| DataFusionError::External(Box::new(e)))?; + + Ok(table_url) + } +} + +/// Load Iceberg table metadata from the table location +async fn load_table_metadata( + ctx: &dyn Session, + table_url: &Url, +) -> Result<(Schema, Snapshot, Vec)> { + log::trace!("Loading table metadata from: {}", table_url); + let object_store = ctx + .runtime_env() + .object_store_registry + .get_store(table_url) + .map_err(|e| DataFusionError::External(Box::new(e)))?; + + let metadata_location = find_latest_metadata_file(&object_store, table_url).await?; + log::trace!("Found metadata file: {}", metadata_location); + + let metadata_path = object_store::path::Path::from(metadata_location.as_str()); + let metadata_data = object_store + .get(&metadata_path) + .await + .map_err(|e| DataFusionError::External(Box::new(e)))? + .bytes() + .await + .map_err(|e| DataFusionError::External(Box::new(e)))?; + + log::trace!("Read {} bytes from metadata file", metadata_data.len()); + + let table_metadata = TableMetadata::from_json(&metadata_data).map_err(|e| { + log::trace!("Failed to parse table metadata: {:?}", e); + DataFusionError::External(Box::new(e)) + })?; + + // Get the current schema + let schema = table_metadata + .current_schema() + .ok_or_else(|| { + DataFusionError::Plan("No current schema found in table metadata".to_string()) + })? + .clone(); + + // Get the current snapshot + let snapshot = table_metadata + .current_snapshot() + .ok_or_else(|| { + DataFusionError::Plan("No current snapshot found in table metadata".to_string()) + })? + .clone(); + + let partition_specs = table_metadata.partition_specs.clone(); + Ok((schema, snapshot, partition_specs)) +} + +/// Find the latest metadata file in the table location +async fn find_latest_metadata_file( + object_store: &Arc, + table_url: &Url, +) -> Result { + use futures::TryStreamExt; + use object_store::path::Path as ObjectPath; + + log::trace!("Finding latest metadata file"); + let version_hint_path = + ObjectPath::from(format!("{}metadata/version-hint.text", table_url.path()).as_str()); + + if let Ok(version_hint_data) = object_store.get(&version_hint_path).await { + if let Ok(version_hint_bytes) = version_hint_data.bytes().await { + if let Ok(version_hint) = String::from_utf8(version_hint_bytes.to_vec()) { + let version = version_hint.trim().parse::().unwrap_or(0); + let metadata_file = + format!("{}/metadata/v{}.metadata.json", table_url.path(), version); + log::trace!("Using version hint: {}", version); + return Ok(metadata_file); + } + } + } + + log::trace!("No version hint, listing metadata directory"); + let metadata_prefix = ObjectPath::from(format!("{}metadata/", table_url.path()).as_str()); + let objects = object_store.list(Some(&metadata_prefix)); + + let metadata_files: Result, _> = objects + .try_filter_map(|obj| async move { + let path_str = obj.location.to_string(); + if path_str.ends_with(".metadata.json") { + if let Some(filename) = path_str.split('/').next_back() { + // Try new format first: 00001-uuid.metadata.json + if let Some(version_part) = filename.split('-').next() { + if let Ok(version) = version_part.parse::() { + return Ok(Some((version, path_str, obj.last_modified))); + } + } + // Try old format: v123.metadata.json + if let Some(version_str) = filename + .strip_prefix('v') + .and_then(|s| s.strip_suffix(".metadata.json")) + { + if let Ok(version) = version_str.parse::() { + return Ok(Some((version, path_str, obj.last_modified))); + } + } + } + } + Ok(None) + }) + .try_collect() + .await; + + match metadata_files { + Ok(mut files) => { + files.sort_by_key(|(version, _, _)| *version); + + if let Some((_, latest_file, _)) = files.last() { + Ok(latest_file.clone()) + } else { + plan_err!("No metadata files found in table location: {}", table_url) + } + } + Err(e) => { + plan_err!("Failed to list metadata directory: {}", e) + } + } +} diff --git a/pyproject.toml b/pyproject.toml index ea4c322d33..4b8972d2fe 100644 --- a/pyproject.toml +++ b/pyproject.toml @@ -36,6 +36,8 @@ test = [ "duckdb>=1.0,<2", "pytest>=8.4,<9", "pillow>=10.3.0", + "pyiceberg[sql-sqlite,pyiceberg-core]==0.10.0", + "pydantic>=2.11,<2.12", ] mcp = [ "mcp>=1.0.0,<2", @@ -76,6 +78,8 @@ dependencies = [ "mcp>=1.0,<2", "boto3>=1.38,<2", "pillow>=10.3.0", + "pyiceberg[sql-sqlite,pyiceberg-core]==0.10.0", + "pydantic>=2.11,<2.12", ] path = ".venvs/default" @@ -114,6 +118,8 @@ dependencies = [ "pytest>=8.4,<9", "duckdb>=1.1,<2", "pillow>=10.3.0", + "pyiceberg[sql-sqlite,pyiceberg-core]==0.10.0", + "pydantic>=2.11,<2.12", ] [[tool.hatch.envs.test.matrix]] @@ -140,6 +146,8 @@ dependencies = [ "pytest-xdist>=3.7,<4", "pytest-timeout>=2.4,<3", "pytest-reportlog>=0.4,<0.5", + "pyiceberg[sql-sqlite,pyiceberg-core]==0.10.0", + "pydantic>=2.11,<2.12", ] [[tool.hatch.envs.test-spark.matrix]] diff --git a/python/pysail/tests/spark/iceberg/__init__.py b/python/pysail/tests/spark/iceberg/__init__.py new file mode 100644 index 0000000000..e69de29bb2 diff --git a/python/pysail/tests/spark/iceberg/conftest.py b/python/pysail/tests/spark/iceberg/conftest.py new file mode 100644 index 0000000000..396117cbe5 --- /dev/null +++ b/python/pysail/tests/spark/iceberg/conftest.py @@ -0,0 +1,27 @@ +from typing import TYPE_CHECKING + +import pytest +from pyiceberg.table import Table + +from pysail.tests.spark.iceberg.utils import create_sql_catalog + +if TYPE_CHECKING: + from pyiceberg.schema import Schema + + +@pytest.fixture +def sql_catalog(tmp_path): + return create_sql_catalog(tmp_path) + + +@pytest.fixture +def iceberg_table(sql_catalog, request) -> Table: + params = request.param + schema: Schema = params["schema"] + identifier: str = params["identifier"] + partition_spec = params.get("partition_spec") + table = sql_catalog.create_table(identifier=identifier, schema=schema, partition_spec=partition_spec) + try: + yield table + finally: + sql_catalog.drop_table(identifier) diff --git a/python/pysail/tests/spark/iceberg/test_iceberg_io.py b/python/pysail/tests/spark/iceberg/test_iceberg_io.py new file mode 100644 index 0000000000..6a27e930cd --- /dev/null +++ b/python/pysail/tests/spark/iceberg/test_iceberg_io.py @@ -0,0 +1,132 @@ +import pandas as pd +import pyarrow as pa +import pytest +from pandas.testing import assert_frame_equal +from pyiceberg.schema import Schema +from pyiceberg.types import DoubleType, LongType, NestedField, StringType + +from pysail.tests.spark.utils import escape_sql_string_literal + + +@pytest.fixture +def iceberg_test_data(): + return [ + {"id": 10, "event": "A", "score": 0.98}, + {"id": 11, "event": "B", "score": 0.54}, + {"id": 12, "event": "A", "score": 0.76}, + ] + + +@pytest.fixture +def expected_pandas_df(): + return ( + pd.DataFrame({"id": [10, 11, 12], "event": ["A", "B", "A"], "score": [0.98, 0.54, 0.76]}) + .astype({"id": "int64", "score": "float64"}) + .assign(event=lambda df: df["event"].astype("object")) + ) + + +def test_iceberg_io_basic_read(spark, iceberg_test_data, expected_pandas_df, sql_catalog): + table_name = "test_table" + + schema = Schema( + NestedField(field_id=1, name="id", field_type=LongType(), required=False), + NestedField(field_id=2, name="event", field_type=StringType(), required=False), + NestedField(field_id=3, name="score", field_type=DoubleType(), required=False), + ) + + table = sql_catalog.create_table( + identifier=f"default.{table_name}", + schema=schema, + ) + + try: + df = pd.DataFrame(iceberg_test_data) + arrow_table = pa.Table.from_pandas(df) + table.append(arrow_table) + + table_path = table.location() + + result_df = spark.read.format("iceberg").load(table_path).sort("id") + + assert_frame_equal( + result_df.toPandas(), expected_pandas_df.sort_values("id").reset_index(drop=True), check_dtype=True + ) + finally: + sql_catalog.drop_table(f"default.{table_name}") + + +def test_iceberg_io_read_with_sql(spark, iceberg_test_data, expected_pandas_df, sql_catalog): + table_name = "test_table_sql" + + schema = Schema( + NestedField(field_id=1, name="id", field_type=LongType(), required=False), + NestedField(field_id=2, name="event", field_type=StringType(), required=False), + NestedField(field_id=3, name="score", field_type=DoubleType(), required=False), + ) + + table = sql_catalog.create_table( + identifier=f"default.{table_name}", + schema=schema, + ) + + try: + df = pd.DataFrame(iceberg_test_data) + arrow_table = pa.Table.from_pandas(df) + table.append(arrow_table) + + table_path = table.location() + + spark.sql(f"CREATE TABLE my_iceberg USING iceberg LOCATION '{escape_sql_string_literal(table_path)}'") + + try: + result_df = spark.sql("SELECT * FROM my_iceberg").sort("id") + + assert_frame_equal( + result_df.toPandas(), expected_pandas_df.sort_values("id").reset_index(drop=True), check_dtype=True + ) + finally: + spark.sql("DROP TABLE IF EXISTS my_iceberg") + finally: + sql_catalog.drop_table(f"default.{table_name}") + + +def test_iceberg_io_multiple_files(spark, sql_catalog): + table_name = "test_table_multiple" + + schema = Schema( + NestedField(field_id=1, name="id", field_type=LongType(), required=False), + NestedField(field_id=2, name="value", field_type=StringType(), required=False), + ) + + table = sql_catalog.create_table( + identifier=f"default.{table_name}", + schema=schema, + ) + + try: + df1 = pd.DataFrame([{"id": 1, "value": "a"}, {"id": 2, "value": "b"}]) + arrow_table1 = pa.Table.from_pandas(df1) + table.append(arrow_table1) + + df2 = pd.DataFrame([{"id": 3, "value": "c"}, {"id": 4, "value": "d"}]) + arrow_table2 = pa.Table.from_pandas(df2) + table.append(arrow_table2) + + table_path = table.location() + + result_df = spark.read.format("iceberg").load(table_path).sort("id") + + expected_data = ( + pd.DataFrame({"id": [1, 2, 3, 4], "value": ["a", "b", "c", "d"]}) + .astype({"id": "int64"}) + .assign(value=lambda df: df["value"].astype("object")) + ) + + assert_frame_equal( + result_df.toPandas(), expected_data.sort_values("id").reset_index(drop=True), check_dtype=True + ) + + assert result_df.count() == 4 # noqa: PLR2004 + finally: + sql_catalog.drop_table(f"default.{table_name}") diff --git a/python/pysail/tests/spark/iceberg/test_iceberg_partitioned_reads.py b/python/pysail/tests/spark/iceberg/test_iceberg_partitioned_reads.py new file mode 100644 index 0000000000..911ff6bca0 --- /dev/null +++ b/python/pysail/tests/spark/iceberg/test_iceberg_partitioned_reads.py @@ -0,0 +1,148 @@ +import datetime + +import pyarrow as pa +import pytest +from pyiceberg.partitioning import ( + BucketTransform, + DayTransform, + HourTransform, + IdentityTransform, + MonthTransform, + PartitionField, + PartitionSpec, + TruncateTransform, + YearTransform, +) +from pyiceberg.schema import Schema +from pyiceberg.types import DateType, IntegerType, NestedField, StringType, TimestampType + +from .utils import create_sql_catalog # noqa: TID252 + + +def _make_common_schema(): + return Schema( + NestedField(1, "number", IntegerType(), required=True), + NestedField(2, "letter", StringType(), required=False), + NestedField(3, "ts", TimestampType(), required=False), + NestedField(4, "dt", DateType(), required=False), + ) + + +def _append_sample_data(table): + start_date = datetime.date(2023, 3, 1) + start_dt = datetime.datetime(2023, 3, 1, 0, 0, 0, tzinfo=datetime.timezone.utc) + letters = [ + "a", + "b", + "c", + "d", + "e", + "f", + "g", + "h", + "i", + "j", + "k", + "l", + ] + numbers = list(range(1, 13)) + dts = [start_date + datetime.timedelta(days=i) for i in range(12)] + tss = [start_dt + datetime.timedelta(days=i) for i in range(12)] + + arrow_schema = pa.schema( + [ + pa.field("number", pa.int32(), nullable=False), + pa.field("letter", pa.string(), nullable=True), + pa.field("ts", pa.timestamp("us"), nullable=True), + pa.field("dt", pa.date32(), nullable=True), + ] + ) + + for i in range(0, 12, 4): + arrays = [ + pa.array(numbers[i : i + 4], type=pa.int32()), + pa.array(letters[i : i + 4], type=pa.string()), + pa.array(tss[i : i + 4], type=pa.timestamp("us")), + pa.array(dts[i : i + 4], type=pa.date32()), + ] + batch = pa.Table.from_arrays(arrays, schema=arrow_schema) + table.append(batch) + + +@pytest.mark.parametrize( + ("table_name", "spec", "predicate_column", "predicate_value", "expected_numbers"), + [ + pytest.param( + "default.test_partitioned_by_identity", + PartitionSpec(PartitionField(3, 1001, IdentityTransform(), "ts")), + "ts", + "'2023-03-05T00:00:00+00:00'", + {5, 6, 7, 8, 9, 10, 11, 12}, + id="partition-by-identity", + ), + pytest.param( + "default.test_partitioned_by_years", + PartitionSpec(PartitionField(4, 1002, YearTransform(), "dt_year")), + "dt", + "'2023-03-05'", + {5, 6, 7, 8, 9, 10, 11, 12}, + id="partition-by-years", + ), + pytest.param( + "default.test_partitioned_by_months", + PartitionSpec(PartitionField(4, 1003, MonthTransform(), "dt_month")), + "dt", + "'2023-03-05'", + {5, 6, 7, 8, 9, 10, 11, 12}, + id="partition-by-months", + ), + pytest.param( + "default.test_partitioned_by_days", + PartitionSpec(PartitionField(3, 1004, DayTransform(), "ts_day")), + "ts", + "'2023-03-05T00:00:00+00:00'", + {5, 6, 7, 8, 9, 10, 11, 12}, + id="partition-by-days", + ), + pytest.param( + "default.test_partitioned_by_hours", + PartitionSpec(PartitionField(3, 1005, HourTransform(), "ts_hour")), + "ts", + "'2023-03-05T00:00:00+00:00'", + {5, 6, 7, 8, 9, 10, 11, 12}, + id="partition-by-hours", + ), + pytest.param( + "default.test_partitioned_by_truncate", + PartitionSpec(PartitionField(2, 1006, TruncateTransform(1), "letter_trunc")), + "letter", + "'e'", + {5, 6, 7, 8, 9, 10, 11, 12}, + id="partition-by-truncate", + ), + pytest.param( + "default.test_partitioned_by_bucket", + PartitionSpec(PartitionField(1, 1007, BucketTransform(8), "number_bucket")), + "number", + "5", + {5, 6, 7, 8, 9, 10, 11, 12}, + id="partition-by-bucket", + ), + ], +) +def test_partition_transform_pruning( + spark, tmp_path, table_name, spec, predicate_column, predicate_value, expected_numbers +): + catalog = create_sql_catalog(tmp_path) + schema = _make_common_schema() + table = catalog.create_table(identifier=table_name, schema=schema, partition_spec=spec) + try: + _append_sample_data(table) + path = table.location() + + predicate = f"{predicate_column} >= {predicate_value}" + df = spark.read.format("iceberg").load(path).filter(predicate).select("number") + result = {r[0] for r in df.collect()} + assert result == expected_numbers + finally: + catalog.drop_table(table_name) diff --git a/python/pysail/tests/spark/iceberg/test_iceberg_projection.py b/python/pysail/tests/spark/iceberg/test_iceberg_projection.py new file mode 100644 index 0000000000..6d5047ae9a --- /dev/null +++ b/python/pysail/tests/spark/iceberg/test_iceberg_projection.py @@ -0,0 +1,71 @@ +import pyarrow as pa +from pyiceberg.schema import Schema +from pyiceberg.types import IntegerType, NestedField, StringType, StructType + +from .utils import create_sql_catalog # noqa: TID252 + + +def test_column_projection_subset_and_order(spark, tmp_path): + catalog = create_sql_catalog(tmp_path) + identifier = "default.test_projection_subset" + schema = Schema( + NestedField(1, "a", IntegerType(), required=False), + NestedField(2, "b", StringType(), required=False), + NestedField(3, "c", IntegerType(), required=False), + ) + table = catalog.create_table(identifier=identifier, schema=schema) + try: + table.append( + pa.table( + { + "a": pa.array([1, 2], type=pa.int32()), + "b": pa.array(["x", "y"], type=pa.string()), + "c": pa.array([3, 4], type=pa.int32()), + } + ) + ) + path = table.location() + + df = spark.read.format("iceberg").load(path).select("c", "a") + rows = sorted([(r[0], r[1]) for r in df.collect()]) + assert rows == [(3, 1), (4, 2)] + finally: + catalog.drop_table(identifier) + + +def test_nested_struct_projection_and_nulls(spark, tmp_path): + catalog = create_sql_catalog(tmp_path) + identifier = "default.test_projection_nested" + inner = StructType( + NestedField(10, "x", IntegerType(), required=False), NestedField(11, "y", StringType(), required=False) + ) + schema = Schema( + NestedField(1, "id", IntegerType(), required=False), + NestedField(2, "s", inner, required=False), + ) + table = catalog.create_table(identifier=identifier, schema=schema) + try: + struct_type = pa.struct([("x", pa.int32()), ("y", pa.string())]) + t1 = pa.Table.from_arrays( + [ + pa.array([1], type=pa.int32()), + pa.array([None], type=struct_type), + ], + names=["id", "s"], + ) + t2 = pa.Table.from_arrays( + [ + pa.array([2], type=pa.int32()), + pa.array([{"x": 7, "y": "z"}], type=struct_type), + ], + names=["id", "s"], + ) + table.append(t1) + table.append(t2) + path = table.location() + + df = spark.read.format("iceberg").load(path).select("id", "s.x") + result = sorted([(r[0], r[1]) for r in df.collect()]) + assert result == [(1, None), (2, 7)] + finally: + catalog.drop_table(identifier) diff --git a/python/pysail/tests/spark/iceberg/test_iceberg_pruning.py b/python/pysail/tests/spark/iceberg/test_iceberg_pruning.py new file mode 100644 index 0000000000..cf1ed5b0e4 --- /dev/null +++ b/python/pysail/tests/spark/iceberg/test_iceberg_pruning.py @@ -0,0 +1,271 @@ +# ruff: noqa +import pandas as pd +import pyarrow as pa +from pyiceberg.schema import Schema +from pyiceberg.types import BooleanType, DoubleType, IntegerType, LongType, NestedField, StringType +from .utils import create_sql_catalog + + +def _make_eq_in_table(catalog, ident): + table = catalog.create_table( + identifier=ident, + schema=Schema( + NestedField(field_id=1, name="id", field_type=LongType(), required=False), + NestedField(field_id=2, name="year", field_type=IntegerType(), required=False), + NestedField(field_id=3, name="month", field_type=IntegerType(), required=False), + NestedField(field_id=4, name="value", field_type=StringType(), required=False), + ), + ) + batches = [ + pd.DataFrame({"id": [1, 2], "year": [2023, 2023], "month": [1, 1], "value": ["a", "b"]}), + pd.DataFrame({"id": [3, 4], "year": [2023, 2023], "month": [2, 2], "value": ["c", "d"]}), + pd.DataFrame({"id": [5, 6], "year": [2024, 2024], "month": [1, 1], "value": ["e", "f"]}), + pd.DataFrame({"id": [7, 8], "year": [2024, 2024], "month": [2, 2], "value": ["g", "h"]}), + ] + for df in batches: + df = df.astype({"id": "int64", "year": "int32", "month": "int32"}) + table.append(pa.Table.from_pandas(df)) + return table + + +def test_pruning_equality_filters(spark, tmp_path): + catalog = create_sql_catalog(tmp_path) + ident = "default.prune_eq_only" + table = _make_eq_in_table(catalog, ident) + try: + tp = table.location() + df = spark.read.format("iceberg").load(tp).filter("year = 2023") + assert df.count() == 4 + df = spark.read.format("iceberg").load(tp).filter("year = 2023 AND month = 1") + assert df.count() == 2 + finally: + catalog.drop_table(ident) + + +def test_pruning_in_clause(spark, tmp_path): + catalog = create_sql_catalog(tmp_path) + ident = "default.prune_in_only" + table = _make_eq_in_table(catalog, ident) + try: + tp = table.location() + df = spark.read.format("iceberg").load(tp).filter("month IN (2)") + assert df.count() == 4 + finally: + catalog.drop_table(ident) + + +def test_comparison_and_between(spark, tmp_path): + catalog = create_sql_catalog(tmp_path) + table = catalog.create_table( + identifier="default.prune_cmp", + schema=Schema( + NestedField(field_id=1, name="id", field_type=LongType(), required=False), + NestedField(field_id=2, name="year", field_type=IntegerType(), required=False), + NestedField(field_id=3, name="month", field_type=IntegerType(), required=False), + ), + ) + try: + data = [] + for year in [2021, 2022, 2023, 2024]: + for month in [1, 6, 12]: + data.append({"id": len(data) + 1, "year": year, "month": month}) + for i in range(0, len(data), 6): + batch = pd.DataFrame(data[i : i + 6]).astype({"id": "int64", "year": "int32", "month": "int32"}) + table.append(pa.Table.from_pandas(batch)) + + tp = table.location() + + df = spark.read.format("iceberg").load(tp).filter("year > 2022") + assert df.count() == 6 + + df = spark.read.format("iceberg").load(tp).filter("year BETWEEN 2022 AND 2023") + assert df.count() == 6 + + df = spark.read.format("iceberg").load(tp).filter("year >= 2023 AND month >= 6") + assert df.count() == 4 + finally: + catalog.drop_table("default.prune_cmp") + + +def test_null_and_boolean(spark, tmp_path): + catalog = create_sql_catalog(tmp_path) + table = catalog.create_table( + identifier="default.prune_null_bool", + schema=Schema( + NestedField(field_id=1, name="id", field_type=LongType(), required=False), + NestedField(field_id=2, name="region", field_type=StringType(), required=False), + NestedField(field_id=3, name="active", field_type=BooleanType(), required=False), + ), + ) + try: + table.append( + pa.Table.from_pandas( + pd.DataFrame([{"id": 1, "region": None, "active": True}, {"id": 2, "region": None, "active": True}]) + ) + ) + table.append( + pa.Table.from_pandas( + pd.DataFrame([{"id": 3, "region": "US", "active": False}, {"id": 4, "region": "EU", "active": False}]) + ) + ) + + tp = table.location() + + df = spark.read.format("iceberg").load(tp).filter("region IS NULL") + assert df.count() == 2 + + df = spark.read.format("iceberg").load(tp).filter("active = true") + assert df.count() == 2 + finally: + catalog.drop_table("default.prune_null_bool") + + +def test_correctness_small(spark, tmp_path): + catalog = create_sql_catalog(tmp_path) + table = catalog.create_table( + identifier="default.prune_correct", + schema=Schema( + NestedField(field_id=1, name="id", field_type=LongType(), required=False), + NestedField(field_id=2, name="year", field_type=IntegerType(), required=False), + NestedField(field_id=3, name="month", field_type=IntegerType(), required=False), + NestedField(field_id=4, name="val", field_type=DoubleType(), required=False), + ), + ) + try: + records = [] + for year in [2022, 2023]: + for month in [1, 2, 3]: + for i in range(5): + records.append({"id": len(records) + 1, "year": year, "month": month, "val": float(i)}) + for i in range(0, len(records), 10): + batch = pd.DataFrame(records[i : i + 10]).astype( + {"id": "int64", "year": "int32", "month": "int32", "val": "float64"} + ) + table.append(pa.Table.from_pandas(batch)) + + tp = table.location() + + df = spark.read.format("iceberg").load(tp).filter("year = 2023") + assert df.count() == 15 + + df = spark.read.format("iceberg").load(tp).filter("year = 2022 AND month = 2") + assert df.count() == 5 + + df = spark.read.format("iceberg").load(tp).filter("(year = 2022 AND month = 1) OR (year = 2023 AND month = 3)") + assert df.count() == 10 + finally: + catalog.drop_table("default.prune_correct") + + +def test_or_and_not_pruning(spark, tmp_path): + catalog = create_sql_catalog(tmp_path) + table = catalog.create_table( + identifier="default.prune_or_and_not", + schema=Schema( + NestedField(field_id=1, name="id", field_type=LongType(), required=False), + NestedField(field_id=2, name="year", field_type=IntegerType(), required=False), + NestedField(field_id=3, name="region", field_type=StringType(), required=False), + ), + ) + try: + records = [] + for year in [2022, 2023, 2024]: + for region in ["US", "EU", "ASIA"]: + for i in range(5): + records.append({"id": len(records) + 1, "year": year, "region": region}) + for i in range(0, len(records), 15): + batch = pd.DataFrame(records[i : i + 15]).astype({"id": "int64", "year": "int32"}) + table.append(pa.Table.from_pandas(batch)) + + tp = table.location() + + df = spark.read.format("iceberg").load(tp).filter("year = 2022 OR year = 2024") + assert df.count() == 30 + + df = spark.read.format("iceberg").load(tp).filter("year = 2023 AND region != 'ASIA'") + assert df.count() == 10 + + df = spark.read.format("iceberg").load(tp).filter("NOT (year = 2023 AND region = 'US')") + assert df.count() == 40 + finally: + catalog.drop_table("default.prune_or_and_not") + + +def test_string_in_and_range_pruning(spark, tmp_path): + catalog = create_sql_catalog(tmp_path) + table = catalog.create_table( + identifier="default.prune_string_in_range", + schema=Schema( + NestedField(field_id=1, name="id", field_type=LongType(), required=False), + NestedField(field_id=2, name="dept", field_type=StringType(), required=False), + NestedField(field_id=3, name="team", field_type=StringType(), required=False), + ), + ) + try: + rows = [ + {"id": 1, "dept": "engineering", "team": "backend"}, + {"id": 2, "dept": "engineering", "team": "frontend"}, + {"id": 3, "dept": "marketing", "team": "growth"}, + {"id": 4, "dept": "sales", "team": "enterprise"}, + ] + table.append(pa.Table.from_pandas(pd.DataFrame(rows))) + + tp = table.location() + + df = spark.read.format("iceberg").load(tp).filter("team IN ('backend','frontend')") + assert df.count() == 2 + + df = spark.read.format("iceberg").load(tp).filter("dept > 'engineering'") + assert df.count() == 2 + finally: + catalog.drop_table("default.prune_string_in_range") + + +def test_metrics_based_pruning_numeric(spark, tmp_path): + catalog = create_sql_catalog(tmp_path) + table = catalog.create_table( + identifier="default.prune_metrics_numeric", + schema=Schema( + NestedField(field_id=1, name="id", field_type=LongType(), required=False), + NestedField(field_id=2, name="val", field_type=DoubleType(), required=False), + ), + ) + try: + data = [] + for chunk in range(4): + for i in range(10): + data.append({"id": chunk * 10 + i, "val": float(i)}) + for i in range(0, len(data), 10): + table.append(pa.Table.from_pandas(pd.DataFrame(data[i : i + 10]))) + + tp = table.location() + + df = spark.read.format("iceberg").load(tp).filter("val >= 8.0") + assert df.count() == 8 + + df = spark.read.format("iceberg").load(tp).filter("val BETWEEN 3.0 AND 4.0") + assert df.count() == 8 + finally: + catalog.drop_table("default.prune_metrics_numeric") + + +def test_limit_pushdown_behavior(spark, tmp_path): + catalog = create_sql_catalog(tmp_path) + table = catalog.create_table( + identifier="default.prune_limit", + schema=Schema( + NestedField(field_id=1, name="id", field_type=LongType(), required=False), + NestedField(field_id=2, name="flag", field_type=BooleanType(), required=False), + ), + ) + try: + rows = [{"id": i, "flag": i % 2 == 0} for i in range(100)] + for i in range(0, len(rows), 20): + table.append(pa.Table.from_pandas(pd.DataFrame(rows[i : i + 20]).astype({"id": "int64"}))) + + tp = table.location() + + df = spark.read.format("iceberg").load(tp).filter("flag = true").limit(7) + assert df.count() == 7 + finally: + catalog.drop_table("default.prune_limit") diff --git a/python/pysail/tests/spark/iceberg/test_iceberg_reads.py b/python/pysail/tests/spark/iceberg/test_iceberg_reads.py new file mode 100644 index 0000000000..2c0a465a28 --- /dev/null +++ b/python/pysail/tests/spark/iceberg/test_iceberg_reads.py @@ -0,0 +1,121 @@ +import math + +import pyarrow as pa +from pyiceberg.schema import Schema +from pyiceberg.types import BooleanType, DoubleType, NestedField, StringType, TimestampType + +from .utils import create_sql_catalog # noqa: TID252 + + +def test_nan_reads(spark, tmp_path): + catalog = create_sql_catalog(tmp_path) + identifier = "default.test_nan_reads" + table = catalog.create_table( + identifier=identifier, + schema=Schema( + NestedField(1, "idx", DoubleType(), required=False), + NestedField(2, "col_numeric", DoubleType(), required=False), + ), + ) + try: + tbl = pa.table({"idx": [1.0, 2.0, 3.0], "col_numeric": [float("nan"), 2.0, 3.0]}) + table.append(tbl) + path = table.location() + df = spark.read.format("iceberg").load(path).select("idx", "col_numeric").filter("isnan(col_numeric)") + rows = df.collect() + assert len(rows) == 1 + assert int(rows[0][0]) == 1 + assert math.isnan(rows[0][1]) + finally: + catalog.drop_table(identifier) + + +def test_datetime_filter_reads(spark, tmp_path): + from datetime import datetime, timedelta + + catalog = create_sql_catalog(tmp_path) + identifier = "default.test_datetime_filter_reads" + table = catalog.create_table( + identifier=identifier, + schema=Schema( + NestedField(1, "str", StringType(), required=False), + NestedField(2, "datetime", TimestampType(), required=False), + ), + ) + try: + yesterday = datetime.now() - timedelta(days=1) # noqa: DTZ005 + tbl = pa.table({"str": ["foo"], "datetime": [yesterday]}) + table.append(tbl) + path = table.location() + iso_ts = yesterday.isoformat() + df_ge = spark.read.format("iceberg").load(path).filter(f"datetime >= '{iso_ts}'") + assert df_ge.count() == 1 + df_lt = spark.read.format("iceberg").load(path).filter(f"datetime < '{iso_ts}'") + assert df_lt.count() == 0 + finally: + catalog.drop_table(identifier) + + +def test_struct_null_filters(spark, tmp_path): + catalog = create_sql_catalog(tmp_path) + identifier = "default.test_struct_null_filters" + struct_field = pa.struct([("test", pa.int32())]) + arrow_schema = pa.schema([pa.field("col_struct", struct_field)]) + table = catalog.create_table(identifier=identifier, schema=arrow_schema) + try: + t1 = pa.Table.from_arrays([pa.array([None], type=struct_field)], schema=arrow_schema) + t2 = pa.Table.from_arrays([pa.array([{"test": 1}], type=struct_field)], schema=arrow_schema) + table.append(t1) + table.append(t2) + path = table.location() + df_all = spark.read.format("iceberg").load(path) + assert df_all.count() == 2 # noqa: PLR2004 + df_not_null = df_all.filter("col_struct.test is not null") + assert df_not_null.count() == 1 + df_null = df_all.filter("col_struct.test is null") + assert df_null.count() == 1 + finally: + catalog.drop_table(identifier) + + +def test_limit_with_multiple_files(spark, tmp_path): + catalog = create_sql_catalog(tmp_path) + identifier = "default.test_limit_with_multiple_files" + table = catalog.create_table( + identifier=identifier, + schema=Schema( + NestedField(1, "id", StringType(), required=False), + ), + ) + try: + tbl1 = pa.table({"id": ["a", "b", "c", "d", "e"]}) + tbl2 = pa.table({"id": ["f", "g", "h", "i", "j"]}) + table.append(tbl1) + table.append(tbl2) + path = table.location() + df = spark.read.format("iceberg").load(path).limit(3) + assert df.count() == 3 # noqa: PLR2004 + finally: + catalog.drop_table(identifier) + + +def test_limit_with_filter(spark, tmp_path): + catalog = create_sql_catalog(tmp_path) + identifier = "default.test_limit_with_filter" + table = catalog.create_table( + identifier=identifier, + schema=Schema( + NestedField(1, "id", StringType(), required=False), + NestedField(2, "flag", BooleanType(), required=False), + ), + ) + try: + tbl1 = pa.table({"id": ["a", "b", "c", "d", "e"], "flag": [True, False, True, True, False]}) + tbl2 = pa.table({"id": ["f", "g", "h", "i", "j"], "flag": [False, True, False, True, True]}) + table.append(tbl1) + table.append(tbl2) + path = table.location() + df = spark.read.format("iceberg").load(path).filter("flag = true").limit(3) + assert df.count() == 3 # noqa: PLR2004 + finally: + catalog.drop_table(identifier) diff --git a/python/pysail/tests/spark/iceberg/utils.py b/python/pysail/tests/spark/iceberg/utils.py new file mode 100644 index 0000000000..6713cc446f --- /dev/null +++ b/python/pysail/tests/spark/iceberg/utils.py @@ -0,0 +1,19 @@ +from pathlib import Path + +from pyiceberg.catalog import load_catalog + + +def create_sql_catalog(tmp_path: Path): + warehouse_path = tmp_path / "warehouse" + warehouse_path.mkdir(parents=True, exist_ok=True) + catalog = load_catalog( + "test_catalog", + type="sql", + uri=f"sqlite:///{tmp_path}/pyiceberg_catalog.db", + warehouse=f"file://{warehouse_path}", + ) + try: # noqa: SIM105 + catalog.create_namespace("default") + except Exception: # noqa: S110, BLE001 + pass + return catalog