From 2e0b64646fcfbd909788236a251a3a374a193542 Mon Sep 17 00:00:00 2001 From: Marko Grujic Date: Fri, 13 Dec 2024 11:47:11 +0100 Subject: [PATCH 01/23] feat: eagerly project the arrow schema to scope out non-selected fields (#785) --- crates/iceberg/src/arrow/reader.rs | 133 +++++++++++++++++++++++++---- 1 file changed, 116 insertions(+), 17 deletions(-) diff --git a/crates/iceberg/src/arrow/reader.rs b/crates/iceberg/src/arrow/reader.rs index 32df1fad7..16b9468c1 100644 --- a/crates/iceberg/src/arrow/reader.rs +++ b/crates/iceberg/src/arrow/reader.rs @@ -25,7 +25,9 @@ use std::sync::Arc; use arrow_arith::boolean::{and, is_not_null, is_null, not, or}; use arrow_array::{Array, ArrayRef, BooleanArray, RecordBatch}; use arrow_ord::cmp::{eq, gt, gt_eq, lt, lt_eq, neq}; -use arrow_schema::{ArrowError, DataType, SchemaRef as ArrowSchemaRef}; +use arrow_schema::{ + ArrowError, DataType, FieldRef, Schema as ArrowSchema, SchemaRef as ArrowSchemaRef, +}; use arrow_string::like::starts_with; use bytes::Bytes; use fnv::FnvHashSet; @@ -328,22 +330,27 @@ impl ArrowReader { let mut column_map = HashMap::new(); let fields = arrow_schema.fields(); - let iceberg_schema = arrow_schema_to_schema(arrow_schema)?; - fields.filter_leaves(|idx, field| { - let field_id = field.metadata().get(PARQUET_FIELD_ID_META_KEY); - if field_id.is_none() { - return false; - } - - let field_id = i32::from_str(field_id.unwrap()); - if field_id.is_err() { - return false; - } - let field_id = field_id.unwrap(); + // Pre-project only the fields that have been selected, possibly avoiding converting + // some Arrow types that are not yet supported. + let mut projected_fields: HashMap = HashMap::new(); + let projected_arrow_schema = ArrowSchema::new_with_metadata( + fields.filter_leaves(|_, f| { + f.metadata() + .get(PARQUET_FIELD_ID_META_KEY) + .and_then(|field_id| i32::from_str(field_id).ok()) + .map_or(false, |field_id| { + projected_fields.insert((*f).clone(), field_id); + field_ids.contains(&field_id) + }) + }), + arrow_schema.metadata().clone(), + ); + let iceberg_schema = arrow_schema_to_schema(&projected_arrow_schema)?; - if !field_ids.contains(&field_id) { + fields.filter_leaves(|idx, field| { + let Some(field_id) = projected_fields.get(field).cloned() else { return false; - } + }; let iceberg_field = iceberg_schema_of_task.field_by_id(field_id); let parquet_iceberg_field = iceberg_schema.field_by_id(field_id); @@ -1128,13 +1135,20 @@ impl AsyncFileReader for ArrowFileReader { #[cfg(test)] mod tests { - use std::collections::HashSet; + use std::collections::{HashMap, HashSet}; use std::sync::Arc; - use crate::arrow::reader::CollectFieldIdVisitor; + use arrow_schema::{DataType, Field, Schema as ArrowSchema, TimeUnit}; + use parquet::arrow::ProjectionMask; + use parquet::schema::parser::parse_message_type; + use parquet::schema::types::SchemaDescriptor; + + use crate::arrow::reader::{CollectFieldIdVisitor, PARQUET_FIELD_ID_META_KEY}; + use crate::arrow::ArrowReader; use crate::expr::visitors::bound_predicate_visitor::visit; use crate::expr::{Bind, Reference}; use crate::spec::{NestedField, PrimitiveType, Schema, SchemaRef, Type}; + use crate::ErrorKind; fn table_schema_simple() -> SchemaRef { Arc::new( @@ -1208,4 +1222,89 @@ mod tests { assert_eq!(visitor.field_ids, expected); } + + #[test] + fn test_arrow_projection_mask() { + let schema = Arc::new( + Schema::builder() + .with_schema_id(1) + .with_identifier_field_ids(vec![1]) + .with_fields(vec![ + NestedField::required(1, "c1", Type::Primitive(PrimitiveType::String)).into(), + NestedField::optional(2, "c2", Type::Primitive(PrimitiveType::Int)).into(), + NestedField::optional( + 3, + "c3", + Type::Primitive(PrimitiveType::Decimal { + precision: 38, + scale: 3, + }), + ) + .into(), + ]) + .build() + .unwrap(), + ); + let arrow_schema = Arc::new(ArrowSchema::new(vec![ + Field::new("c1", DataType::Utf8, false).with_metadata(HashMap::from([( + PARQUET_FIELD_ID_META_KEY.to_string(), + "1".to_string(), + )])), + // Type not supported + Field::new("c2", DataType::Duration(TimeUnit::Microsecond), true).with_metadata( + HashMap::from([(PARQUET_FIELD_ID_META_KEY.to_string(), "2".to_string())]), + ), + // Precision is beyond the supported range + Field::new("c3", DataType::Decimal128(39, 3), true).with_metadata(HashMap::from([( + PARQUET_FIELD_ID_META_KEY.to_string(), + "3".to_string(), + )])), + ])); + + let message_type = " +message schema { + required binary c1 (STRING) = 1; + optional int32 c2 (INTEGER(8,true)) = 2; + optional fixed_len_byte_array(17) c3 (DECIMAL(39,3)) = 3; +} + "; + let parquet_type = parse_message_type(message_type).expect("should parse schema"); + let parquet_schema = SchemaDescriptor::new(Arc::new(parquet_type)); + + // Try projecting the fields c2 and c3 with the unsupported data types + let err = ArrowReader::get_arrow_projection_mask( + &[1, 2, 3], + &schema, + &parquet_schema, + &arrow_schema, + ) + .unwrap_err(); + + assert_eq!(err.kind(), ErrorKind::DataInvalid); + assert_eq!( + err.to_string(), + "DataInvalid => Unsupported Arrow data type: Duration(Microsecond)".to_string() + ); + + // Omitting field c2, we still get an error due to c3 being selected + let err = ArrowReader::get_arrow_projection_mask( + &[1, 3], + &schema, + &parquet_schema, + &arrow_schema, + ) + .unwrap_err(); + + assert_eq!(err.kind(), ErrorKind::DataInvalid); + assert_eq!( + err.to_string(), + "DataInvalid => Failed to create decimal type, source: DataInvalid => Decimals with precision larger than 38 are not supported: 39".to_string() + ); + + // Finally avoid selecting fields with unsupported data types + let mask = + ArrowReader::get_arrow_projection_mask(&[1], &schema, &parquet_schema, &arrow_schema) + .expect("Some ProjectionMask"); + assert_eq!(mask, ProjectionMask::leaves(&parquet_schema, vec![0])); + } } From d1decdbb67042d2cf61d0c0b0f5aeae6923d110f Mon Sep 17 00:00:00 2001 From: ZENOTME <43447882+ZENOTME@users.noreply.github.com> Date: Sat, 14 Dec 2024 15:51:25 +0800 Subject: [PATCH 02/23] fix: wrong compute of partitions in manifest (#794) * fix partitions of manifest * refine code --------- Co-authored-by: ZENOTME --- crates/iceberg/src/spec/manifest.rs | 332 +++++++++++++++++++---- crates/iceberg/src/spec/manifest_list.rs | 12 +- 2 files changed, 285 insertions(+), 59 deletions(-) diff --git a/crates/iceberg/src/spec/manifest.rs b/crates/iceberg/src/spec/manifest.rs index 13ecdc2e6..a4b4d7cdb 100644 --- a/crates/iceberg/src/spec/manifest.rs +++ b/crates/iceberg/src/spec/manifest.rs @@ -23,6 +23,7 @@ use std::sync::Arc; use apache_avro::{from_value, to_value, Reader as AvroReader, Writer as AvroWriter}; use bytes::Bytes; +use itertools::Itertools; use serde_derive::{Deserialize, Serialize}; use serde_json::to_vec; use serde_with::{DeserializeFromStr, SerializeDisplay}; @@ -31,7 +32,8 @@ use typed_builder::TypedBuilder; use self::_const_schema::{manifest_schema_v1, manifest_schema_v2}; use super::{ BoundPartitionSpec, Datum, FieldSummary, FormatVersion, ManifestContentType, ManifestFile, - Schema, SchemaId, SchemaRef, Struct, INITIAL_SEQUENCE_NUMBER, UNASSIGNED_SEQUENCE_NUMBER, + PrimitiveLiteral, PrimitiveType, Schema, SchemaId, SchemaRef, Struct, INITIAL_SEQUENCE_NUMBER, + UNASSIGNED_SEQUENCE_NUMBER, }; use crate::error::Result; use crate::io::OutputFile; @@ -128,7 +130,69 @@ pub struct ManifestWriter { key_metadata: Vec, - field_summary: HashMap, + partitions: Vec, +} + +struct PartitionFieldStats { + partition_type: PrimitiveType, + summary: FieldSummary, +} + +impl PartitionFieldStats { + pub(crate) fn new(partition_type: PrimitiveType) -> Self { + Self { + partition_type, + summary: FieldSummary::default(), + } + } + + pub(crate) fn update(&mut self, value: Option) -> Result<()> { + let Some(value) = value else { + self.summary.contains_null = true; + return Ok(()); + }; + if !self.partition_type.compatible(&value) { + return Err(Error::new( + ErrorKind::DataInvalid, + "value is not compatitable with type", + )); + } + let value = Datum::new(self.partition_type.clone(), value); + + if value.is_nan() { + self.summary.contains_nan = Some(true); + return Ok(()); + } + + self.summary.lower_bound = Some(self.summary.lower_bound.take().map_or( + value.clone(), + |original| { + if value < original { + value.clone() + } else { + original + } + }, + )); + self.summary.upper_bound = Some(self.summary.upper_bound.take().map_or( + value.clone(), + |original| { + if value > original { + value + } else { + original + } + }, + )); + + Ok(()) + } + + pub(crate) fn finish(mut self) -> FieldSummary { + // Always set contains_nan + self.summary.contains_nan = self.summary.contains_nan.or(Some(false)); + self.summary + } } impl ManifestWriter { @@ -145,62 +209,28 @@ impl ManifestWriter { deleted_rows: 0, min_seq_num: None, key_metadata, - field_summary: HashMap::new(), + partitions: vec![], } } - fn update_field_summary(&mut self, entry: &ManifestEntry) { - // Update field summary - for (&k, &v) in &entry.data_file.null_value_counts { - let field_summary = self.field_summary.entry(k).or_default(); - if v > 0 { - field_summary.contains_null = true; - } - } - - for (&k, &v) in &entry.data_file.nan_value_counts { - let field_summary = self.field_summary.entry(k).or_default(); - if v > 0 { - field_summary.contains_nan = Some(true); - } - if v == 0 { - field_summary.contains_nan = Some(false); - } - } - - for (&k, v) in &entry.data_file.lower_bounds { - let field_summary = self.field_summary.entry(k).or_default(); - if let Some(cur) = &field_summary.lower_bound { - if v < cur { - field_summary.lower_bound = Some(v.clone()); - } - } else { - field_summary.lower_bound = Some(v.clone()); - } - } - - for (&k, v) in &entry.data_file.upper_bounds { - let field_summary = self.field_summary.entry(k).or_default(); - if let Some(cur) = &field_summary.upper_bound { - if v > cur { - field_summary.upper_bound = Some(v.clone()); - } - } else { - field_summary.upper_bound = Some(v.clone()); + fn construct_partition_summaries( + &mut self, + partition_spec: &BoundPartitionSpec, + ) -> Result> { + let partitions = std::mem::take(&mut self.partitions); + let mut field_stats: Vec<_> = partition_spec + .partition_type() + .fields() + .iter() + .map(|f| PartitionFieldStats::new(f.field_type.as_primitive_type().unwrap().clone())) + .collect(); + for partition in partitions { + for (literal, stat) in partition.into_iter().zip_eq(field_stats.iter_mut()) { + let primitive_literal = literal.map(|v| v.as_primitive_literal().unwrap()); + stat.update(primitive_literal)?; } } - } - - fn get_field_summary_vec(&mut self, spec_fields: &[PartitionField]) -> Vec { - let mut partition_summary = Vec::with_capacity(self.field_summary.len()); - for field in spec_fields { - let entry = self - .field_summary - .remove(&field.source_id) - .unwrap_or_default(); - partition_summary.push(entry); - } - partition_summary + Ok(field_stats.into_iter().map(|stat| stat.finish()).collect()) } /// Write a manifest. @@ -276,7 +306,7 @@ impl ManifestWriter { } } - self.update_field_summary(&entry); + self.partitions.push(entry.data_file.partition.clone()); let value = match manifest.metadata.format_version { FormatVersion::V1 => to_value(_serde::ManifestEntryV1::try_from( @@ -299,7 +329,7 @@ impl ManifestWriter { self.output.write(Bytes::from(content)).await?; let partition_summary = - self.get_field_summary_vec(manifest.metadata.partition_spec.fields()); + self.construct_partition_summaries(&manifest.metadata.partition_spec)?; Ok(ManifestFile { manifest_path: self.output.location().to_string(), @@ -2086,6 +2116,198 @@ mod tests { assert_eq!(actual_manifest, expected_manifest); } + #[tokio::test] + async fn test_manifest_summary() { + let schema = Arc::new( + Schema::builder() + .with_fields(vec![ + Arc::new(NestedField::optional( + 1, + "time", + Type::Primitive(PrimitiveType::Date), + )), + Arc::new(NestedField::optional( + 2, + "v_float", + Type::Primitive(PrimitiveType::Float), + )), + Arc::new(NestedField::optional( + 3, + "v_double", + Type::Primitive(PrimitiveType::Double), + )), + ]) + .build() + .unwrap(), + ); + let partition_spec = BoundPartitionSpec::builder(schema.clone()) + .with_spec_id(0) + .add_partition_field("time", "year_of_time", Transform::Year) + .unwrap() + .add_partition_field("v_float", "f", Transform::Identity) + .unwrap() + .add_partition_field("v_double", "d", Transform::Identity) + .unwrap() + .build() + .unwrap(); + let manifest = Manifest { + metadata: ManifestMetadata { + schema_id: 0, + schema, + partition_spec, + content: ManifestContentType::Data, + format_version: FormatVersion::V2, + }, + entries: vec![ + Arc::new(ManifestEntry { + status: ManifestStatus::Added, + snapshot_id: None, + sequence_number: None, + file_sequence_number: None, + data_file: DataFile { + content: DataContentType::Data, + file_path: "s3a://icebergdata/demo/s1/t1/data/00000-0-ba56fbfa-f2ff-40c9-bb27-565ad6dc2be8-00000.parquet".to_string(), + file_format: DataFileFormat::Parquet, + partition: Struct::from_iter( + vec![ + Some(Literal::int(2021)), + Some(Literal::float(1.0)), + Some(Literal::double(2.0)), + ] + ), + record_count: 1, + file_size_in_bytes: 5442, + column_sizes: HashMap::from([(0,73),(6,34),(2,73),(7,61),(3,61),(5,62),(9,79),(10,73),(1,61),(4,73),(8,73)]), + value_counts: HashMap::from([(4,1),(5,1),(2,1),(0,1),(3,1),(6,1),(8,1),(1,1),(10,1),(7,1),(9,1)]), + null_value_counts: HashMap::from([(1,0),(6,0),(2,0),(8,0),(0,0),(3,0),(5,0),(9,0),(7,0),(4,0),(10,0)]), + nan_value_counts: HashMap::new(), + lower_bounds: HashMap::new(), + upper_bounds: HashMap::new(), + key_metadata: Vec::new(), + split_offsets: vec![4], + equality_ids: Vec::new(), + sort_order_id: None, + } + }), + Arc::new( + ManifestEntry { + status: ManifestStatus::Added, + snapshot_id: None, + sequence_number: None, + file_sequence_number: None, + data_file: DataFile { + content: DataContentType::Data, + file_path: "s3a://icebergdata/demo/s1/t1/data/00000-0-ba56fbfa-f2ff-40c9-bb27-565ad6dc2be8-00000.parquet".to_string(), + file_format: DataFileFormat::Parquet, + partition: Struct::from_iter( + vec![ + Some(Literal::int(1111)), + Some(Literal::float(15.5)), + Some(Literal::double(25.5)), + ] + ), + record_count: 1, + file_size_in_bytes: 5442, + column_sizes: HashMap::from([(0,73),(6,34),(2,73),(7,61),(3,61),(5,62),(9,79),(10,73),(1,61),(4,73),(8,73)]), + value_counts: HashMap::from([(4,1),(5,1),(2,1),(0,1),(3,1),(6,1),(8,1),(1,1),(10,1),(7,1),(9,1)]), + null_value_counts: HashMap::from([(1,0),(6,0),(2,0),(8,0),(0,0),(3,0),(5,0),(9,0),(7,0),(4,0),(10,0)]), + nan_value_counts: HashMap::new(), + lower_bounds: HashMap::new(), + upper_bounds: HashMap::new(), + key_metadata: Vec::new(), + split_offsets: vec![4], + equality_ids: Vec::new(), + sort_order_id: None, + } + } + ), + Arc::new( + ManifestEntry { + status: ManifestStatus::Added, + snapshot_id: None, + sequence_number: None, + file_sequence_number: None, + data_file: DataFile { + content: DataContentType::Data, + file_path: "s3a://icebergdata/demo/s1/t1/data/00000-0-ba56fbfa-f2ff-40c9-bb27-565ad6dc2be8-00000.parquet".to_string(), + file_format: DataFileFormat::Parquet, + partition: Struct::from_iter( + vec![ + Some(Literal::int(1211)), + Some(Literal::float(f32::NAN)), + Some(Literal::double(1.0)), + ] + ), + record_count: 1, + file_size_in_bytes: 5442, + column_sizes: HashMap::from([(0,73),(6,34),(2,73),(7,61),(3,61),(5,62),(9,79),(10,73),(1,61),(4,73),(8,73)]), + value_counts: HashMap::from([(4,1),(5,1),(2,1),(0,1),(3,1),(6,1),(8,1),(1,1),(10,1),(7,1),(9,1)]), + null_value_counts: HashMap::from([(1,0),(6,0),(2,0),(8,0),(0,0),(3,0),(5,0),(9,0),(7,0),(4,0),(10,0)]), + nan_value_counts: HashMap::new(), + lower_bounds: HashMap::new(), + upper_bounds: HashMap::new(), + key_metadata: Vec::new(), + split_offsets: vec![4], + equality_ids: Vec::new(), + sort_order_id: None, + } + } + ), + Arc::new( + ManifestEntry { + status: ManifestStatus::Added, + snapshot_id: None, + sequence_number: None, + file_sequence_number: None, + data_file: DataFile { + content: DataContentType::Data, + file_path: "s3a://icebergdata/demo/s1/t1/data/00000-0-ba56fbfa-f2ff-40c9-bb27-565ad6dc2be8-00000.parquet".to_string(), + file_format: DataFileFormat::Parquet, + partition: Struct::from_iter( + vec![ + Some(Literal::int(1111)), + None, + Some(Literal::double(11.0)), + ] + ), + record_count: 1, + file_size_in_bytes: 5442, + column_sizes: HashMap::from([(0,73),(6,34),(2,73),(7,61),(3,61),(5,62),(9,79),(10,73),(1,61),(4,73),(8,73)]), + value_counts: HashMap::from([(4,1),(5,1),(2,1),(0,1),(3,1),(6,1),(8,1),(1,1),(10,1),(7,1),(9,1)]), + null_value_counts: HashMap::from([(1,0),(6,0),(2,0),(8,0),(0,0),(3,0),(5,0),(9,0),(7,0),(4,0),(10,0)]), + nan_value_counts: HashMap::new(), + lower_bounds: HashMap::new(), + upper_bounds: HashMap::new(), + key_metadata: Vec::new(), + split_offsets: vec![4], + equality_ids: Vec::new(), + sort_order_id: None, + } + } + ), + ] + }; + + let writer = |output_file: OutputFile| ManifestWriter::new(output_file, 1, vec![]); + + let res = test_manifest_read_write(manifest, writer).await; + assert!(res.partitions.len() == 3); + assert!(res.partitions[0].lower_bound == Some(Datum::int(1111))); + assert!(res.partitions[0].upper_bound == Some(Datum::int(2021))); + assert!(!res.partitions[0].contains_null); + assert!(res.partitions[0].contains_nan == Some(false)); + + assert!(res.partitions[1].lower_bound == Some(Datum::float(1.0))); + assert!(res.partitions[1].upper_bound == Some(Datum::float(15.5))); + assert!(res.partitions[1].contains_null); + assert!(res.partitions[1].contains_nan == Some(true)); + + assert!(res.partitions[2].lower_bound == Some(Datum::double(1.0))); + assert!(res.partitions[2].upper_bound == Some(Datum::double(25.5))); + assert!(!res.partitions[2].contains_null); + assert!(res.partitions[2].contains_nan == Some(false)); + } + async fn test_manifest_read_write( manifest: Manifest, writer_builder: impl FnOnce(OutputFile) -> ManifestWriter, diff --git a/crates/iceberg/src/spec/manifest_list.rs b/crates/iceberg/src/spec/manifest_list.rs index 6c898f89d..97d259ad3 100644 --- a/crates/iceberg/src/spec/manifest_list.rs +++ b/crates/iceberg/src/spec/manifest_list.rs @@ -857,12 +857,16 @@ pub(super) mod _serde { contains_nan: self.contains_nan, lower_bound: self .lower_bound - .map(|v| Datum::try_from_bytes(&v, r#type.clone())) - .transpose()?, + .as_ref() + .map(|v| Datum::try_from_bytes(v, r#type.clone())) + .transpose() + .map_err(|err| err.with_context("type", format!("{:?}", r#type)))?, upper_bound: self .upper_bound - .map(|v| Datum::try_from_bytes(&v, r#type.clone())) - .transpose()?, + .as_ref() + .map(|v| Datum::try_from_bytes(v, r#type.clone())) + .transpose() + .map_err(|err| err.with_context("type", format!("{:?}", r#type)))?, }) } } From 0ba444fc06241123f78eabca4965aa72fbeb469e Mon Sep 17 00:00:00 2001 From: feniljain <49019259+feniljain@users.noreply.github.com> Date: Sat, 14 Dec 2024 15:57:17 +0530 Subject: [PATCH 03/23] fix: set key_metadata to Null by default (#800) * fix: set key_metadata to Null by default * fix: return Option<&[u8]> instead of &Option> for key_metadata * test: use `None` instead of `Some` for key_metadata fields * refactor: use as_deref instead of explicit ref/deref using map --- .../src/expr/visitors/expression_evaluator.rs | 4 +-- .../visitors/inclusive_metrics_evaluator.rs | 12 +++---- crates/iceberg/src/io/object_cache.rs | 1 + crates/iceberg/src/scan.rs | 1 + crates/iceberg/src/spec/manifest.rs | 31 ++++++++++--------- .../src/writer/file_writer/parquet_writer.rs | 2 +- 6 files changed, 27 insertions(+), 24 deletions(-) diff --git a/crates/iceberg/src/expr/visitors/expression_evaluator.rs b/crates/iceberg/src/expr/visitors/expression_evaluator.rs index 2add5761f..d451401c1 100644 --- a/crates/iceberg/src/expr/visitors/expression_evaluator.rs +++ b/crates/iceberg/src/expr/visitors/expression_evaluator.rs @@ -338,7 +338,7 @@ mod tests { nan_value_counts: HashMap::new(), lower_bounds: HashMap::new(), upper_bounds: HashMap::new(), - key_metadata: vec![], + key_metadata: None, split_offsets: vec![], equality_ids: vec![], sort_order_id: None, @@ -361,7 +361,7 @@ mod tests { nan_value_counts: HashMap::new(), lower_bounds: HashMap::new(), upper_bounds: HashMap::new(), - key_metadata: vec![], + key_metadata: None, split_offsets: vec![], equality_ids: vec![], sort_order_id: None, diff --git a/crates/iceberg/src/expr/visitors/inclusive_metrics_evaluator.rs b/crates/iceberg/src/expr/visitors/inclusive_metrics_evaluator.rs index 1cdc75771..7b04fae3a 100644 --- a/crates/iceberg/src/expr/visitors/inclusive_metrics_evaluator.rs +++ b/crates/iceberg/src/expr/visitors/inclusive_metrics_evaluator.rs @@ -1991,7 +1991,7 @@ mod test { nan_value_counts: Default::default(), lower_bounds: Default::default(), upper_bounds: Default::default(), - key_metadata: vec![], + key_metadata: None, split_offsets: vec![], equality_ids: vec![], sort_order_id: None, @@ -2012,7 +2012,7 @@ mod test { nan_value_counts: Default::default(), lower_bounds: Default::default(), upper_bounds: Default::default(), - key_metadata: vec![], + key_metadata: None, split_offsets: vec![], equality_ids: vec![], sort_order_id: None, @@ -2069,7 +2069,7 @@ mod test { ]), column_sizes: Default::default(), - key_metadata: vec![], + key_metadata: None, split_offsets: vec![], equality_ids: vec![], sort_order_id: None, @@ -2095,7 +2095,7 @@ mod test { upper_bounds: HashMap::from([(3, Datum::string("dC"))]), column_sizes: Default::default(), - key_metadata: vec![], + key_metadata: None, split_offsets: vec![], equality_ids: vec![], sort_order_id: None, @@ -2122,7 +2122,7 @@ mod test { upper_bounds: HashMap::from([(3, Datum::string("3str3"))]), column_sizes: Default::default(), - key_metadata: vec![], + key_metadata: None, split_offsets: vec![], equality_ids: vec![], sort_order_id: None, @@ -2149,7 +2149,7 @@ mod test { upper_bounds: HashMap::from([(3, Datum::string("イロハニホヘト"))]), column_sizes: Default::default(), - key_metadata: vec![], + key_metadata: None, split_offsets: vec![], equality_ids: vec![], sort_order_id: None, diff --git a/crates/iceberg/src/io/object_cache.rs b/crates/iceberg/src/io/object_cache.rs index 88e2d0e2d..809db33f5 100644 --- a/crates/iceberg/src/io/object_cache.rs +++ b/crates/iceberg/src/io/object_cache.rs @@ -278,6 +278,7 @@ mod tests { .file_size_in_bytes(100) .record_count(1) .partition(Struct::from_iter([Some(Literal::long(100))])) + .key_metadata(None) .build() .unwrap(), ) diff --git a/crates/iceberg/src/scan.rs b/crates/iceberg/src/scan.rs index 8f0bc38f6..89cc21bbf 100644 --- a/crates/iceberg/src/scan.rs +++ b/crates/iceberg/src/scan.rs @@ -1073,6 +1073,7 @@ mod tests { .file_size_in_bytes(100) .record_count(1) .partition(Struct::from_iter([Some(Literal::long(100))])) + .key_metadata(None) .build() .unwrap(), ) diff --git a/crates/iceberg/src/spec/manifest.rs b/crates/iceberg/src/spec/manifest.rs index a4b4d7cdb..60f5469cd 100644 --- a/crates/iceberg/src/spec/manifest.rs +++ b/crates/iceberg/src/spec/manifest.rs @@ -1074,7 +1074,7 @@ pub struct DataFile { /// /// Implementation-specific key metadata for encryption #[builder(default)] - pub(crate) key_metadata: Vec, + pub(crate) key_metadata: Option>, /// field id: 132 /// element field id: 133 /// @@ -1164,8 +1164,8 @@ impl DataFile { &self.upper_bounds } /// Get the Implementation-specific key metadata for the data file. - pub fn key_metadata(&self) -> &[u8] { - &self.key_metadata + pub fn key_metadata(&self) -> Option<&[u8]> { + self.key_metadata.as_deref() } /// Get the split offsets of the data file. /// For example, all row group offsets in a Parquet file. @@ -1378,12 +1378,13 @@ mod _serde { nan_value_counts: Some(to_i64_entry(value.nan_value_counts)?), lower_bounds: Some(to_bytes_entry(value.lower_bounds)?), upper_bounds: Some(to_bytes_entry(value.upper_bounds)?), - key_metadata: Some(serde_bytes::ByteBuf::from(value.key_metadata)), + key_metadata: value.key_metadata.map(serde_bytes::ByteBuf::from), split_offsets: Some(value.split_offsets), equality_ids: Some(value.equality_ids), sort_order_id: value.sort_order_id, }) } + pub fn try_into( self, partition_type: &StructType, @@ -1441,7 +1442,7 @@ mod _serde { .map(|v| parse_bytes_entry(v, schema)) .transpose()? .unwrap_or_default(), - key_metadata: self.key_metadata.map(|v| v.to_vec()).unwrap_or_default(), + key_metadata: self.key_metadata.map(|v| v.to_vec()), split_offsets: self.split_offsets.unwrap_or_default(), equality_ids: self.equality_ids.unwrap_or_default(), sort_order_id: self.sort_order_id, @@ -1657,7 +1658,7 @@ mod tests { nan_value_counts: HashMap::new(), lower_bounds: HashMap::new(), upper_bounds: HashMap::new(), - key_metadata: Vec::new(), + key_metadata: None, split_offsets: vec![4], equality_ids: Vec::new(), sort_order_id: None, @@ -1813,7 +1814,7 @@ mod tests { nan_value_counts: HashMap::new(), lower_bounds: HashMap::new(), upper_bounds: HashMap::new(), - key_metadata: vec![], + key_metadata: None, split_offsets: vec![4], equality_ids: vec![], sort_order_id: None, @@ -1880,7 +1881,7 @@ mod tests { nan_value_counts: HashMap::new(), lower_bounds: HashMap::from([(1,Datum::int(1)),(2,Datum::string("a")),(3,Datum::string("AC/DC"))]), upper_bounds: HashMap::from([(1,Datum::int(1)),(2,Datum::string("a")),(3,Datum::string("AC/DC"))]), - key_metadata: vec![], + key_metadata: None, split_offsets: vec![4], equality_ids: vec![], sort_order_id: Some(0), @@ -1960,7 +1961,7 @@ mod tests { (2, Datum::string("a")), (3, Datum::string("x")) ]), - key_metadata: vec![], + key_metadata: None, split_offsets: vec![4], equality_ids: vec![], sort_order_id: Some(0), @@ -2035,7 +2036,7 @@ mod tests { (2, Datum::int(2)), (3, Datum::string("x")) ]), - key_metadata: vec![], + key_metadata: None, split_offsets: vec![4], equality_ids: vec![], sort_order_id: None, @@ -2105,7 +2106,7 @@ mod tests { (1, Datum::long(1)), (2, Datum::int(2)), ]), - key_metadata: vec![], + key_metadata: None, split_offsets: vec![4], equality_ids: vec![], sort_order_id: None, @@ -2183,7 +2184,7 @@ mod tests { nan_value_counts: HashMap::new(), lower_bounds: HashMap::new(), upper_bounds: HashMap::new(), - key_metadata: Vec::new(), + key_metadata: None, split_offsets: vec![4], equality_ids: Vec::new(), sort_order_id: None, @@ -2214,7 +2215,7 @@ mod tests { nan_value_counts: HashMap::new(), lower_bounds: HashMap::new(), upper_bounds: HashMap::new(), - key_metadata: Vec::new(), + key_metadata: None, split_offsets: vec![4], equality_ids: Vec::new(), sort_order_id: None, @@ -2246,7 +2247,7 @@ mod tests { nan_value_counts: HashMap::new(), lower_bounds: HashMap::new(), upper_bounds: HashMap::new(), - key_metadata: Vec::new(), + key_metadata: None, split_offsets: vec![4], equality_ids: Vec::new(), sort_order_id: None, @@ -2278,7 +2279,7 @@ mod tests { nan_value_counts: HashMap::new(), lower_bounds: HashMap::new(), upper_bounds: HashMap::new(), - key_metadata: Vec::new(), + key_metadata: None, split_offsets: vec![4], equality_ids: Vec::new(), sort_order_id: None, diff --git a/crates/iceberg/src/writer/file_writer/parquet_writer.rs b/crates/iceberg/src/writer/file_writer/parquet_writer.rs index 09f9a7057..596228f7c 100644 --- a/crates/iceberg/src/writer/file_writer/parquet_writer.rs +++ b/crates/iceberg/src/writer/file_writer/parquet_writer.rs @@ -381,7 +381,7 @@ impl ParquetWriter { // # TODO(#417) // - nan_value_counts // - distinct_counts - .key_metadata(metadata.footer_signing_key_metadata.unwrap_or_default()) + .key_metadata(metadata.footer_signing_key_metadata) .split_offsets( metadata .row_groups From 7981def302774ad01ae65fd649a02f1686653b83 Mon Sep 17 00:00:00 2001 From: feniljain <49019259+feniljain@users.noreply.github.com> Date: Sat, 14 Dec 2024 17:01:22 +0530 Subject: [PATCH 04/23] test: append partition data file (#742) * test: append partition data file * chore: fix `compatible` spell mistake --- crates/iceberg/src/transaction.rs | 6 +- .../tests/append_partition_data_file_test.rs | 251 ++++++++++++++++++ 2 files changed, 254 insertions(+), 3 deletions(-) create mode 100644 crates/integration_tests/tests/append_partition_data_file_test.rs diff --git a/crates/iceberg/src/transaction.rs b/crates/iceberg/src/transaction.rs index edf1a8596..f58536a2c 100644 --- a/crates/iceberg/src/transaction.rs +++ b/crates/iceberg/src/transaction.rs @@ -317,7 +317,7 @@ impl<'a> SnapshotProduceAction<'a> { if partition_value.fields().len() != partition_type.fields().len() { return Err(Error::new( ErrorKind::DataInvalid, - "Partition value is not compatitable with partition type", + "Partition value is not compatible with partition type", )); } for (value, field) in partition_value.fields().iter().zip(partition_type.fields()) { @@ -334,7 +334,7 @@ impl<'a> SnapshotProduceAction<'a> { { return Err(Error::new( ErrorKind::DataInvalid, - "Partition value is not compatitable partition type", + "Partition value is not compatible partition type", )); } } @@ -784,7 +784,7 @@ mod tests { let tx = Transaction::new(&table); let mut action = tx.fast_append(None, vec![]).unwrap(); - // check add data file with uncompatitable partition value + // check add data file with incompatible partition value let data_file = DataFileBuilder::default() .content(DataContentType::Data) .file_path("test/3.parquet".to_string()) diff --git a/crates/integration_tests/tests/append_partition_data_file_test.rs b/crates/integration_tests/tests/append_partition_data_file_test.rs new file mode 100644 index 000000000..103021532 --- /dev/null +++ b/crates/integration_tests/tests/append_partition_data_file_test.rs @@ -0,0 +1,251 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +//! Integration test for partition data file + +use std::collections::HashMap; +use std::sync::Arc; + +use arrow_array::{ArrayRef, BooleanArray, Int32Array, RecordBatch, StringArray}; +use futures::TryStreamExt; +use iceberg::spec::{ + Literal, NestedField, PrimitiveLiteral, PrimitiveType, Schema, Struct, Transform, Type, + UnboundPartitionSpec, +}; +use iceberg::table::Table; +use iceberg::transaction::Transaction; +use iceberg::writer::base_writer::data_file_writer::DataFileWriterBuilder; +use iceberg::writer::file_writer::location_generator::{ + DefaultFileNameGenerator, DefaultLocationGenerator, +}; +use iceberg::writer::file_writer::ParquetWriterBuilder; +use iceberg::writer::{IcebergWriter, IcebergWriterBuilder}; +use iceberg::{Catalog, Namespace, NamespaceIdent, TableCreation}; +use iceberg_integration_tests::set_test_fixture; +use parquet::file::properties::WriterProperties; + +#[tokio::test] +async fn test_append_partition_data_file() { + let fixture = set_test_fixture("test_partition_data_file").await; + + let ns = Namespace::with_properties( + NamespaceIdent::from_strs(["iceberg", "rust"]).unwrap(), + HashMap::from([ + ("owner".to_string(), "ray".to_string()), + ("community".to_string(), "apache".to_string()), + ]), + ); + + fixture + .rest_catalog + .create_namespace(ns.name(), ns.properties().clone()) + .await + .unwrap(); + + let schema = Schema::builder() + .with_schema_id(1) + .with_identifier_field_ids(vec![2]) + .with_fields(vec![ + NestedField::optional(1, "foo", Type::Primitive(PrimitiveType::String)).into(), + NestedField::required(2, "bar", Type::Primitive(PrimitiveType::Int)).into(), + NestedField::optional(3, "baz", Type::Primitive(PrimitiveType::Boolean)).into(), + ]) + .build() + .unwrap(); + + let unbound_partition_spec = UnboundPartitionSpec::builder() + .add_partition_field(2, "id", Transform::Identity) + .expect("could not add partition field") + .build(); + + let partition_spec = unbound_partition_spec + .bind(schema.clone()) + .expect("could not bind to schema"); + + let table_creation = TableCreation::builder() + .name("t1".to_string()) + .schema(schema.clone()) + .partition_spec(partition_spec) + .build(); + + let table = fixture + .rest_catalog + .create_table(ns.name(), table_creation) + .await + .unwrap(); + + // Create the writer and write the data + let schema: Arc = Arc::new( + table + .metadata() + .current_schema() + .as_ref() + .try_into() + .unwrap(), + ); + + let first_partition_id_value = 100; + + let location_generator = DefaultLocationGenerator::new(table.metadata().clone()).unwrap(); + let file_name_generator = DefaultFileNameGenerator::new( + "test".to_string(), + None, + iceberg::spec::DataFileFormat::Parquet, + ); + + let parquet_writer_builder = ParquetWriterBuilder::new( + WriterProperties::default(), + table.metadata().current_schema().clone(), + table.file_io().clone(), + location_generator.clone(), + file_name_generator.clone(), + ); + + let mut data_file_writer_valid = DataFileWriterBuilder::new( + parquet_writer_builder.clone(), + Some(Struct::from_iter([Some(Literal::Primitive( + PrimitiveLiteral::Int(first_partition_id_value), + ))])), + ) + .build() + .await + .unwrap(); + + let col1 = StringArray::from(vec![Some("foo1"), Some("foo2")]); + let col2 = Int32Array::from(vec![ + Some(first_partition_id_value), + Some(first_partition_id_value), + ]); + let col3 = BooleanArray::from(vec![Some(true), Some(false)]); + let batch = RecordBatch::try_new(schema.clone(), vec![ + Arc::new(col1) as ArrayRef, + Arc::new(col2) as ArrayRef, + Arc::new(col3) as ArrayRef, + ]) + .unwrap(); + + data_file_writer_valid.write(batch.clone()).await.unwrap(); + let data_file_valid = data_file_writer_valid.close().await.unwrap(); + + // commit result + let tx = Transaction::new(&table); + let mut append_action = tx.fast_append(None, vec![]).unwrap(); + append_action + .add_data_files(data_file_valid.clone()) + .unwrap(); + let tx = append_action.apply().await.unwrap(); + let table = tx.commit(&fixture.rest_catalog).await.unwrap(); + + // check result + let batch_stream = table + .scan() + .select_all() + .build() + .unwrap() + .to_arrow() + .await + .unwrap(); + let batches: Vec<_> = batch_stream.try_collect().await.unwrap(); + assert_eq!(batches.len(), 1); + assert_eq!(batches[0], batch); + + test_schema_incompatible_partition_type( + parquet_writer_builder.clone(), + batch.clone(), + table.clone(), + ) + .await; + + test_schema_incompatible_partition_fields( + parquet_writer_builder, + batch, + table, + first_partition_id_value, + ) + .await; +} + +async fn test_schema_incompatible_partition_type( + parquet_writer_builder: ParquetWriterBuilder< + DefaultLocationGenerator, + DefaultFileNameGenerator, + >, + batch: RecordBatch, + table: Table, +) { + // test writing different "type" of partition than mentioned in schema + let mut data_file_writer_invalid = DataFileWriterBuilder::new( + parquet_writer_builder.clone(), + Some(Struct::from_iter([Some(Literal::Primitive( + PrimitiveLiteral::Boolean(true), + ))])), + ) + .build() + .await + .unwrap(); + + data_file_writer_invalid.write(batch.clone()).await.unwrap(); + let data_file_invalid = data_file_writer_invalid.close().await.unwrap(); + + let tx = Transaction::new(&table); + let mut append_action = tx.fast_append(None, vec![]).unwrap(); + if append_action + .add_data_files(data_file_invalid.clone()) + .is_ok() + { + panic!("diverging partition info should have returned error"); + } +} + +async fn test_schema_incompatible_partition_fields( + parquet_writer_builder: ParquetWriterBuilder< + DefaultLocationGenerator, + DefaultFileNameGenerator, + >, + batch: RecordBatch, + table: Table, + first_partition_id_value: i32, +) { + // test writing different number of partition fields than mentioned in schema + + let mut data_file_writer_invalid = DataFileWriterBuilder::new( + parquet_writer_builder, + Some(Struct::from_iter([ + Some(Literal::Primitive(PrimitiveLiteral::Int( + first_partition_id_value, + ))), + Some(Literal::Primitive(PrimitiveLiteral::Int( + first_partition_id_value, + ))), + ])), + ) + .build() + .await + .unwrap(); + + data_file_writer_invalid.write(batch.clone()).await.unwrap(); + let data_file_invalid = data_file_writer_invalid.close().await.unwrap(); + + let tx = Transaction::new(&table); + let mut append_action = tx.fast_append(None, vec![]).unwrap(); + if append_action + .add_data_files(data_file_invalid.clone()) + .is_ok() + { + panic!("passing different number of partition fields should have returned error"); + } +} From b9f1849e6719101c4e4da17b970cdbe7a0cd9412 Mon Sep 17 00:00:00 2001 From: Xuanwo Date: Sat, 14 Dec 2024 20:02:32 +0800 Subject: [PATCH 05/23] chore: Add more debug message inside error (#793) Signed-off-by: Xuanwo --- crates/catalog/rest/src/client.rs | 58 ++++++++++++++++++++++++++----- 1 file changed, 49 insertions(+), 9 deletions(-) diff --git a/crates/catalog/rest/src/client.rs b/crates/catalog/rest/src/client.rs index 53dcd4cee..7027edef8 100644 --- a/crates/catalog/rest/src/client.rs +++ b/crates/catalog/rest/src/client.rs @@ -134,28 +134,39 @@ impl HttpClient { .request(Method::POST, &self.token_endpoint) .form(¶ms) .build()?; + let auth_url = auth_req.url().clone(); let auth_resp = self.client.execute(auth_req).await?; let auth_res: TokenResponse = if auth_resp.status().as_u16() == OK { - let text = auth_resp.bytes().await?; + let text = auth_resp + .bytes() + .await + .map_err(|err| err.with_url(auth_url.clone()))?; Ok(serde_json::from_slice(&text).map_err(|e| { Error::new( ErrorKind::Unexpected, "Failed to parse response from rest catalog server!", ) + .with_context("operation", "auth") + .with_context("url", auth_url.to_string()) .with_context("json", String::from_utf8_lossy(&text)) .with_source(e) })?) } else { let code = auth_resp.status(); - let text = auth_resp.bytes().await?; + let text = auth_resp + .bytes() + .await + .map_err(|err| err.with_url(auth_url.clone()))?; let e: ErrorResponse = serde_json::from_slice(&text).map_err(|e| { Error::new( ErrorKind::Unexpected, "Failed to parse response from rest catalog server!", ) - .with_context("json", String::from_utf8_lossy(&text)) .with_context("code", code.to_string()) + .with_context("operation", "auth") + .with_context("url", auth_url.to_string()) + .with_context("json", String::from_utf8_lossy(&text)) .with_source(e) })?; Err(Error::from(e)) @@ -193,28 +204,41 @@ impl HttpClient { ) -> Result { self.authenticate(&mut request).await?; + let method = request.method().clone(); + let url = request.url().clone(); + let resp = self.client.execute(request).await?; if resp.status().as_u16() == SUCCESS_CODE { - let text = resp.bytes().await?; + let text = resp + .bytes() + .await + .map_err(|err| err.with_url(url.clone()))?; Ok(serde_json::from_slice::(&text).map_err(|e| { Error::new( ErrorKind::Unexpected, "Failed to parse response from rest catalog server!", ) + .with_context("method", method.to_string()) + .with_context("url", url.to_string()) .with_context("json", String::from_utf8_lossy(&text)) .with_source(e) })?) } else { let code = resp.status(); - let text = resp.bytes().await?; + let text = resp + .bytes() + .await + .map_err(|err| err.with_url(url.clone()))?; let e = serde_json::from_slice::(&text).map_err(|e| { Error::new( ErrorKind::Unexpected, "Failed to parse response from rest catalog server!", ) - .with_context("json", String::from_utf8_lossy(&text)) .with_context("code", code.to_string()) + .with_context("method", method.to_string()) + .with_context("url", url.to_string()) + .with_context("json", String::from_utf8_lossy(&text)) .with_source(e) })?; Err(e.into()) @@ -227,20 +251,28 @@ impl HttpClient { ) -> Result<()> { self.authenticate(&mut request).await?; + let method = request.method().clone(); + let url = request.url().clone(); + let resp = self.client.execute(request).await?; if resp.status().as_u16() == SUCCESS_CODE { Ok(()) } else { let code = resp.status(); - let text = resp.bytes().await?; + let text = resp + .bytes() + .await + .map_err(|err| err.with_url(url.clone()))?; let e = serde_json::from_slice::(&text).map_err(|e| { Error::new( ErrorKind::Unexpected, "Failed to parse response from rest catalog server!", ) - .with_context("json", String::from_utf8_lossy(&text)) .with_context("code", code.to_string()) + .with_context("method", method.to_string()) + .with_context("url", url.to_string()) + .with_context("json", String::from_utf8_lossy(&text)) .with_source(e) })?; Err(e.into()) @@ -255,19 +287,27 @@ impl HttpClient { ) -> Result { self.authenticate(&mut request).await?; + let method = request.method().clone(); + let url = request.url().clone(); + let resp = self.client.execute(request).await?; if let Some(ret) = handler(&resp) { Ok(ret) } else { let code = resp.status(); - let text = resp.bytes().await?; + let text = resp + .bytes() + .await + .map_err(|err| err.with_url(url.clone()))?; let e = serde_json::from_slice::(&text).map_err(|e| { Error::new( ErrorKind::Unexpected, "Failed to parse response from rest catalog server!", ) .with_context("code", code.to_string()) + .with_context("method", method.to_string()) + .with_context("url", url.to_string()) .with_context("json", String::from_utf8_lossy(&text)) .with_source(e) })?; From 748d37c8d552d3d001fe884699307f1522602961 Mon Sep 17 00:00:00 2001 From: Xuanwo Date: Sat, 14 Dec 2024 20:03:27 +0800 Subject: [PATCH 06/23] fix: Error source from cache has been shadowed (#792) * feat: Print debug source error instead Signed-off-by: Xuanwo * Don't hide internal source error Signed-off-by: Xuanwo * revert error fmt changes, not related Signed-off-by: Xuanwo --------- Signed-off-by: Xuanwo --- crates/iceberg/src/io/object_cache.rs | 10 +++++++++- 1 file changed, 9 insertions(+), 1 deletion(-) diff --git a/crates/iceberg/src/io/object_cache.rs b/crates/iceberg/src/io/object_cache.rs index 809db33f5..6ea7594ba 100644 --- a/crates/iceberg/src/io/object_cache.rs +++ b/crates/iceberg/src/io/object_cache.rs @@ -141,7 +141,15 @@ impl ObjectCache { .entry_by_ref(&key) .or_try_insert_with(self.fetch_and_parse_manifest_list(snapshot, table_metadata)) .await - .map_err(|err| Error::new(ErrorKind::Unexpected, err.as_ref().message()))? + .map_err(|err| { + Arc::try_unwrap(err).unwrap_or_else(|err| { + Error::new( + ErrorKind::Unexpected, + "Failed to load manifest list in cache", + ) + .with_source(err) + }) + })? .into_value(); match cache_entry { From 54926a2ded1b9747ffcded2ced5ce64d4f900c56 Mon Sep 17 00:00:00 2001 From: Xuanwo Date: Sat, 14 Dec 2024 20:25:55 +0800 Subject: [PATCH 07/23] fix(catalog/rest): Ensure token been reused correctly (#801) * fix(catalog/rest): Ensure token been reused correctly Signed-off-by: Xuanwo * Fix oauth test Signed-off-by: Xuanwo * Fix tests Signed-off-by: Xuanwo --------- Signed-off-by: Xuanwo --- crates/catalog/rest/src/catalog.rs | 17 ++++++++++------- crates/catalog/rest/src/client.rs | 27 +++++++++++++++++++++++++++ 2 files changed, 37 insertions(+), 7 deletions(-) diff --git a/crates/catalog/rest/src/catalog.rs b/crates/catalog/rest/src/catalog.rs index fce5fe2be..96da5dc95 100644 --- a/crates/catalog/rest/src/catalog.rs +++ b/crates/catalog/rest/src/catalog.rs @@ -256,9 +256,10 @@ impl RestCatalog { async fn context(&self) -> Result<&RestContext> { self.ctx .get_or_try_init(|| async { - let catalog_config = RestCatalog::load_config(&self.user_config).await?; + let client = HttpClient::new(&self.user_config)?; + let catalog_config = RestCatalog::load_config(&client, &self.user_config).await?; let config = self.user_config.clone().merge_with_config(catalog_config); - let client = HttpClient::new(&config)?; + let client = client.update_with(&config)?; Ok(RestContext { config, client }) }) @@ -268,9 +269,10 @@ impl RestCatalog { /// Load the runtime config from the server by user_config. /// /// It's required for a rest catalog to update it's config after creation. - async fn load_config(user_config: &RestCatalogConfig) -> Result { - let client = HttpClient::new(user_config)?; - + async fn load_config( + client: &HttpClient, + user_config: &RestCatalogConfig, + ) -> Result { let mut request = client.request(Method::GET, user_config.config_endpoint()); if let Some(warehouse_location) = &user_config.warehouse { @@ -280,6 +282,7 @@ impl RestCatalog { let config = client .query::(request.build()?) .await?; + Ok(config) } @@ -777,7 +780,7 @@ mod tests { "expires_in": 86400 }"#, ) - .expect(2) + .expect(1) .create_async() .await } @@ -831,7 +834,7 @@ mod tests { "expires_in": 86400 }"#, ) - .expect(2) + .expect(1) .create_async() .await; diff --git a/crates/catalog/rest/src/client.rs b/crates/catalog/rest/src/client.rs index 7027edef8..e06090134 100644 --- a/crates/catalog/rest/src/client.rs +++ b/crates/catalog/rest/src/client.rs @@ -54,6 +54,7 @@ impl Debug for HttpClient { } impl HttpClient { + /// Create a new http client. pub fn new(cfg: &RestCatalogConfig) -> Result { Ok(HttpClient { client: Client::new(), @@ -66,6 +67,32 @@ impl HttpClient { }) } + /// Update the http client with new configuration. + /// + /// If cfg carries new value, we will use cfg instead. + /// Otherwise, we will keep the old value. + pub fn update_with(self, cfg: &RestCatalogConfig) -> Result { + Ok(HttpClient { + client: self.client, + + token: Mutex::new( + cfg.token() + .or_else(|| self.token.into_inner().ok().flatten()), + ), + token_endpoint: (!cfg.get_token_endpoint().is_empty()) + .then(|| cfg.get_token_endpoint()) + .unwrap_or(self.token_endpoint), + credential: cfg.credential().or(self.credential), + extra_headers: (!cfg.extra_headers()?.is_empty()) + .then(|| cfg.extra_headers()) + .transpose()? + .unwrap_or(self.extra_headers), + extra_oauth_params: (!cfg.extra_oauth_params().is_empty()) + .then(|| cfg.extra_oauth_params()) + .unwrap_or(self.extra_oauth_params), + }) + } + /// This API is testing only to assert the token. #[cfg(test)] pub(crate) async fn token(&self) -> Option { From 821f8dda3c4ebb635994e6acb8ed7914452f235f Mon Sep 17 00:00:00 2001 From: Christian Date: Sun, 15 Dec 2024 16:26:22 +0100 Subject: [PATCH 08/23] feat!: Remove `BoundPartitionSpec` (#771) * Remove bound partition spec * Fix UnboundPartitionSpec name * Update crates/iceberg/src/expr/visitors/expression_evaluator.rs Co-authored-by: Fokko Driesprong * Update crates/iceberg/src/spec/table_metadata.rs Co-authored-by: Fokko Driesprong * Fix syntax * Address comments --------- Co-authored-by: Fokko Driesprong --- crates/catalog/memory/src/catalog.rs | 6 +- crates/catalog/sql/src/catalog.rs | 7 +- .../src/expr/visitors/expression_evaluator.rs | 148 ++++----- .../visitors/inclusive_metrics_evaluator.rs | 15 +- .../src/expr/visitors/inclusive_projection.rs | 43 ++- crates/iceberg/src/spec/manifest.rs | 47 +-- crates/iceberg/src/spec/partition.rs | 305 +++++------------- crates/iceberg/src/spec/table_metadata.rs | 122 ++++--- .../src/spec/table_metadata_builder.rs | 49 ++- crates/iceberg/src/transaction.rs | 6 +- .../writer/file_writer/location_generator.rs | 6 +- 11 files changed, 302 insertions(+), 452 deletions(-) diff --git a/crates/catalog/memory/src/catalog.rs b/crates/catalog/memory/src/catalog.rs index 3a4a4e3b4..c5a98391f 100644 --- a/crates/catalog/memory/src/catalog.rs +++ b/crates/catalog/memory/src/catalog.rs @@ -285,7 +285,7 @@ mod tests { use std::iter::FromIterator; use iceberg::io::FileIOBuilder; - use iceberg::spec::{BoundPartitionSpec, NestedField, PrimitiveType, Schema, SortOrder, Type}; + use iceberg::spec::{NestedField, PartitionSpec, PrimitiveType, Schema, SortOrder, Type}; use regex::Regex; use tempfile::TempDir; @@ -357,7 +357,7 @@ mod tests { assert_eq!(metadata.current_schema().as_ref(), expected_schema); - let expected_partition_spec = BoundPartitionSpec::builder((*expected_schema).clone()) + let expected_partition_spec = PartitionSpec::builder((*expected_schema).clone()) .with_spec_id(0) .build() .unwrap(); @@ -367,7 +367,7 @@ mod tests { .partition_specs_iter() .map(|p| p.as_ref()) .collect_vec(), - vec![&expected_partition_spec.into_schemaless()] + vec![&expected_partition_spec] ); let expected_sorted_order = SortOrder::builder() diff --git a/crates/catalog/sql/src/catalog.rs b/crates/catalog/sql/src/catalog.rs index 1556614dd..51e2904f7 100644 --- a/crates/catalog/sql/src/catalog.rs +++ b/crates/catalog/sql/src/catalog.rs @@ -783,7 +783,7 @@ mod tests { use std::hash::Hash; use iceberg::io::FileIOBuilder; - use iceberg::spec::{BoundPartitionSpec, NestedField, PrimitiveType, Schema, SortOrder, Type}; + use iceberg::spec::{NestedField, PartitionSpec, PrimitiveType, Schema, SortOrder, Type}; use iceberg::table::Table; use iceberg::{Catalog, Namespace, NamespaceIdent, TableCreation, TableIdent}; use itertools::Itertools; @@ -876,11 +876,10 @@ mod tests { assert_eq!(metadata.current_schema().as_ref(), expected_schema); - let expected_partition_spec = BoundPartitionSpec::builder(expected_schema.clone()) + let expected_partition_spec = PartitionSpec::builder(expected_schema.clone()) .with_spec_id(0) .build() - .unwrap() - .into_schemaless(); + .unwrap(); assert_eq!( metadata diff --git a/crates/iceberg/src/expr/visitors/expression_evaluator.rs b/crates/iceberg/src/expr/visitors/expression_evaluator.rs index d451401c1..eae321272 100644 --- a/crates/iceberg/src/expr/visitors/expression_evaluator.rs +++ b/crates/iceberg/src/expr/visitors/expression_evaluator.rs @@ -258,13 +258,13 @@ mod tests { UnaryExpression, }; use crate::spec::{ - BoundPartitionSpec, BoundPartitionSpecRef, DataContentType, DataFile, DataFileFormat, - Datum, Literal, NestedField, PrimitiveType, Schema, Struct, Transform, Type, + DataContentType, DataFile, DataFileFormat, Datum, Literal, NestedField, PartitionSpec, + PartitionSpecRef, PrimitiveType, Schema, SchemaRef, Struct, Transform, Type, UnboundPartitionField, }; use crate::Result; - fn create_partition_spec(r#type: PrimitiveType) -> Result { + fn create_partition_spec(r#type: PrimitiveType) -> Result<(PartitionSpecRef, SchemaRef)> { let schema = Schema::builder() .with_fields(vec![Arc::new(NestedField::optional( 1, @@ -273,27 +273,30 @@ mod tests { ))]) .build()?; - let spec = BoundPartitionSpec::builder(schema.clone()) + let spec = PartitionSpec::builder(schema.clone()) .with_spec_id(1) - .add_unbound_fields(vec![UnboundPartitionField::builder() - .source_id(1) - .name("a".to_string()) - .field_id(1) - .transform(Transform::Identity) - .build()]) + .add_unbound_field( + UnboundPartitionField::builder() + .source_id(1) + .name("a".to_string()) + .field_id(1) + .transform(Transform::Identity) + .build(), + ) .unwrap() .build() .unwrap(); - Ok(Arc::new(spec)) + Ok((Arc::new(spec), schema.into())) } fn create_partition_filter( - partition_spec: BoundPartitionSpecRef, + partition_spec: PartitionSpecRef, + schema: &Schema, predicate: &BoundPredicate, case_sensitive: bool, ) -> Result { - let partition_type = partition_spec.partition_type(); + let partition_type = partition_spec.partition_type(schema).unwrap(); let partition_fields = partition_type.fields().to_owned(); let partition_schema = Schema::builder() @@ -301,8 +304,7 @@ mod tests { .with_fields(partition_fields) .build()?; - let mut inclusive_projection = - InclusiveProjection::new((*partition_spec).clone().into_schemaless().into()); + let mut inclusive_projection = InclusiveProjection::new((*partition_spec).clone().into()); let partition_filter = inclusive_projection .project(predicate)? @@ -313,11 +315,13 @@ mod tests { } fn create_expression_evaluator( - partition_spec: BoundPartitionSpecRef, + partition_spec: PartitionSpecRef, + schema: &Schema, predicate: &BoundPredicate, case_sensitive: bool, ) -> Result { - let partition_filter = create_partition_filter(partition_spec, predicate, case_sensitive)?; + let partition_filter = + create_partition_filter(partition_spec, schema, predicate, case_sensitive)?; Ok(ExpressionEvaluator::new(partition_filter)) } @@ -371,7 +375,7 @@ mod tests { #[test] fn test_expr_or() -> Result<()> { let case_sensitive = true; - let partition_spec = create_partition_spec(PrimitiveType::Float)?; + let (partition_spec, schema) = create_partition_spec(PrimitiveType::Float)?; let predicate = Predicate::Binary(BinaryExpression::new( PredicateOperator::LessThan, @@ -383,10 +387,10 @@ mod tests { Reference::new("a"), Datum::float(0.4), ))) - .bind(partition_spec.schema_ref().clone(), case_sensitive)?; + .bind(schema.clone(), case_sensitive)?; let expression_evaluator = - create_expression_evaluator(partition_spec, &predicate, case_sensitive)?; + create_expression_evaluator(partition_spec, &schema, &predicate, case_sensitive)?; let data_file = create_data_file_float(); @@ -400,7 +404,7 @@ mod tests { #[test] fn test_expr_and() -> Result<()> { let case_sensitive = true; - let partition_spec = create_partition_spec(PrimitiveType::Float)?; + let (partition_spec, schema) = create_partition_spec(PrimitiveType::Float)?; let predicate = Predicate::Binary(BinaryExpression::new( PredicateOperator::LessThan, @@ -412,10 +416,10 @@ mod tests { Reference::new("a"), Datum::float(0.4), ))) - .bind(partition_spec.schema_ref().clone(), case_sensitive)?; + .bind(schema.clone(), case_sensitive)?; let expression_evaluator = - create_expression_evaluator(partition_spec, &predicate, case_sensitive)?; + create_expression_evaluator(partition_spec, &schema, &predicate, case_sensitive)?; let data_file = create_data_file_float(); @@ -429,17 +433,17 @@ mod tests { #[test] fn test_expr_not_in() -> Result<()> { let case_sensitive = true; - let partition_spec = create_partition_spec(PrimitiveType::Float)?; + let (partition_spec, schema) = create_partition_spec(PrimitiveType::Float)?; let predicate = Predicate::Set(SetExpression::new( PredicateOperator::NotIn, Reference::new("a"), FnvHashSet::from_iter([Datum::float(0.9), Datum::float(1.2), Datum::float(2.4)]), )) - .bind(partition_spec.schema_ref().clone(), case_sensitive)?; + .bind(schema.clone(), case_sensitive)?; let expression_evaluator = - create_expression_evaluator(partition_spec, &predicate, case_sensitive)?; + create_expression_evaluator(partition_spec, &schema, &predicate, case_sensitive)?; let data_file = create_data_file_float(); @@ -453,17 +457,17 @@ mod tests { #[test] fn test_expr_in() -> Result<()> { let case_sensitive = true; - let partition_spec = create_partition_spec(PrimitiveType::Float)?; + let (partition_spec, schema) = create_partition_spec(PrimitiveType::Float)?; let predicate = Predicate::Set(SetExpression::new( PredicateOperator::In, Reference::new("a"), FnvHashSet::from_iter([Datum::float(1.0), Datum::float(1.2), Datum::float(2.4)]), )) - .bind(partition_spec.schema_ref().clone(), case_sensitive)?; + .bind(schema.clone(), case_sensitive)?; let expression_evaluator = - create_expression_evaluator(partition_spec, &predicate, case_sensitive)?; + create_expression_evaluator(partition_spec, &schema, &predicate, case_sensitive)?; let data_file = create_data_file_float(); @@ -477,17 +481,17 @@ mod tests { #[test] fn test_expr_not_starts_with() -> Result<()> { let case_sensitive = true; - let partition_spec = create_partition_spec(PrimitiveType::String)?; + let (partition_spec, schema) = create_partition_spec(PrimitiveType::String)?; let predicate = Predicate::Binary(BinaryExpression::new( PredicateOperator::NotStartsWith, Reference::new("a"), Datum::string("not"), )) - .bind(partition_spec.schema_ref().clone(), case_sensitive)?; + .bind(schema.clone(), case_sensitive)?; let expression_evaluator = - create_expression_evaluator(partition_spec, &predicate, case_sensitive)?; + create_expression_evaluator(partition_spec, &schema, &predicate, case_sensitive)?; let data_file = create_data_file_string(); @@ -501,17 +505,17 @@ mod tests { #[test] fn test_expr_starts_with() -> Result<()> { let case_sensitive = true; - let partition_spec = create_partition_spec(PrimitiveType::String)?; + let (partition_spec, schema) = create_partition_spec(PrimitiveType::String)?; let predicate = Predicate::Binary(BinaryExpression::new( PredicateOperator::StartsWith, Reference::new("a"), Datum::string("test"), )) - .bind(partition_spec.schema_ref().clone(), case_sensitive)?; + .bind(schema.clone(), case_sensitive)?; let expression_evaluator = - create_expression_evaluator(partition_spec, &predicate, case_sensitive)?; + create_expression_evaluator(partition_spec, &schema, &predicate, case_sensitive)?; let data_file = create_data_file_string(); @@ -525,17 +529,17 @@ mod tests { #[test] fn test_expr_not_eq() -> Result<()> { let case_sensitive = true; - let partition_spec = create_partition_spec(PrimitiveType::Float)?; + let (partition_spec, schema) = create_partition_spec(PrimitiveType::Float)?; let predicate = Predicate::Binary(BinaryExpression::new( PredicateOperator::NotEq, Reference::new("a"), Datum::float(0.9), )) - .bind(partition_spec.schema_ref().clone(), case_sensitive)?; + .bind(schema.clone(), case_sensitive)?; let expression_evaluator = - create_expression_evaluator(partition_spec, &predicate, case_sensitive)?; + create_expression_evaluator(partition_spec, &schema, &predicate, case_sensitive)?; let data_file = create_data_file_float(); @@ -549,17 +553,17 @@ mod tests { #[test] fn test_expr_eq() -> Result<()> { let case_sensitive = true; - let partition_spec = create_partition_spec(PrimitiveType::Float)?; + let (partition_spec, schema) = create_partition_spec(PrimitiveType::Float)?; let predicate = Predicate::Binary(BinaryExpression::new( PredicateOperator::Eq, Reference::new("a"), Datum::float(1.0), )) - .bind(partition_spec.schema_ref().clone(), case_sensitive)?; + .bind(schema.clone(), case_sensitive)?; let expression_evaluator = - create_expression_evaluator(partition_spec, &predicate, case_sensitive)?; + create_expression_evaluator(partition_spec, &schema, &predicate, case_sensitive)?; let data_file = create_data_file_float(); @@ -573,17 +577,17 @@ mod tests { #[test] fn test_expr_greater_than_or_eq() -> Result<()> { let case_sensitive = true; - let partition_spec = create_partition_spec(PrimitiveType::Float)?; + let (partition_spec, schema) = create_partition_spec(PrimitiveType::Float)?; let predicate = Predicate::Binary(BinaryExpression::new( PredicateOperator::GreaterThanOrEq, Reference::new("a"), Datum::float(1.0), )) - .bind(partition_spec.schema_ref().clone(), case_sensitive)?; + .bind(schema.clone(), case_sensitive)?; let expression_evaluator = - create_expression_evaluator(partition_spec, &predicate, case_sensitive)?; + create_expression_evaluator(partition_spec, &schema, &predicate, case_sensitive)?; let data_file = create_data_file_float(); @@ -597,17 +601,17 @@ mod tests { #[test] fn test_expr_greater_than() -> Result<()> { let case_sensitive = true; - let partition_spec = create_partition_spec(PrimitiveType::Float)?; + let (partition_spec, schema) = create_partition_spec(PrimitiveType::Float)?; let predicate = Predicate::Binary(BinaryExpression::new( PredicateOperator::GreaterThan, Reference::new("a"), Datum::float(0.9), )) - .bind(partition_spec.schema_ref().clone(), case_sensitive)?; + .bind(schema.clone(), case_sensitive)?; let expression_evaluator = - create_expression_evaluator(partition_spec, &predicate, case_sensitive)?; + create_expression_evaluator(partition_spec, &schema, &predicate, case_sensitive)?; let data_file = create_data_file_float(); @@ -621,17 +625,17 @@ mod tests { #[test] fn test_expr_less_than_or_eq() -> Result<()> { let case_sensitive = true; - let partition_spec = create_partition_spec(PrimitiveType::Float)?; + let (partition_spec, schema) = create_partition_spec(PrimitiveType::Float)?; let predicate = Predicate::Binary(BinaryExpression::new( PredicateOperator::LessThanOrEq, Reference::new("a"), Datum::float(1.0), )) - .bind(partition_spec.schema_ref().clone(), case_sensitive)?; + .bind(schema.clone(), case_sensitive)?; let expression_evaluator = - create_expression_evaluator(partition_spec, &predicate, case_sensitive)?; + create_expression_evaluator(partition_spec, &schema, &predicate, case_sensitive)?; let data_file = create_data_file_float(); @@ -645,17 +649,17 @@ mod tests { #[test] fn test_expr_less_than() -> Result<()> { let case_sensitive = true; - let partition_spec = create_partition_spec(PrimitiveType::Float)?; + let (partition_spec, schema) = create_partition_spec(PrimitiveType::Float)?; let predicate = Predicate::Binary(BinaryExpression::new( PredicateOperator::LessThan, Reference::new("a"), Datum::float(1.1), )) - .bind(partition_spec.schema_ref().clone(), case_sensitive)?; + .bind(schema.clone(), case_sensitive)?; let expression_evaluator = - create_expression_evaluator(partition_spec, &predicate, case_sensitive)?; + create_expression_evaluator(partition_spec, &schema, &predicate, case_sensitive)?; let data_file = create_data_file_float(); @@ -669,15 +673,15 @@ mod tests { #[test] fn test_expr_is_not_nan() -> Result<()> { let case_sensitive = true; - let partition_spec = create_partition_spec(PrimitiveType::Float)?; + let (partition_spec, schema) = create_partition_spec(PrimitiveType::Float)?; let predicate = Predicate::Unary(UnaryExpression::new( PredicateOperator::NotNan, Reference::new("a"), )) - .bind(partition_spec.schema_ref().clone(), case_sensitive)?; + .bind(schema.clone(), case_sensitive)?; let expression_evaluator = - create_expression_evaluator(partition_spec, &predicate, case_sensitive)?; + create_expression_evaluator(partition_spec, &schema, &predicate, case_sensitive)?; let data_file = create_data_file_float(); @@ -691,15 +695,15 @@ mod tests { #[test] fn test_expr_is_nan() -> Result<()> { let case_sensitive = true; - let partition_spec = create_partition_spec(PrimitiveType::Float)?; + let (partition_spec, schema) = create_partition_spec(PrimitiveType::Float)?; let predicate = Predicate::Unary(UnaryExpression::new( PredicateOperator::IsNan, Reference::new("a"), )) - .bind(partition_spec.schema_ref().clone(), case_sensitive)?; + .bind(schema.clone(), case_sensitive)?; let expression_evaluator = - create_expression_evaluator(partition_spec, &predicate, case_sensitive)?; + create_expression_evaluator(partition_spec, &schema, &predicate, case_sensitive)?; let data_file = create_data_file_float(); @@ -713,15 +717,15 @@ mod tests { #[test] fn test_expr_is_not_null() -> Result<()> { let case_sensitive = true; - let partition_spec = create_partition_spec(PrimitiveType::Float)?; + let (partition_spec, schema) = create_partition_spec(PrimitiveType::Float)?; let predicate = Predicate::Unary(UnaryExpression::new( PredicateOperator::NotNull, Reference::new("a"), )) - .bind(partition_spec.schema_ref().clone(), case_sensitive)?; + .bind(schema.clone(), case_sensitive)?; let expression_evaluator = - create_expression_evaluator(partition_spec, &predicate, case_sensitive)?; + create_expression_evaluator(partition_spec, &schema, &predicate, case_sensitive)?; let data_file = create_data_file_float(); @@ -735,15 +739,15 @@ mod tests { #[test] fn test_expr_is_null() -> Result<()> { let case_sensitive = true; - let partition_spec = create_partition_spec(PrimitiveType::Float)?; + let (partition_spec, schema) = create_partition_spec(PrimitiveType::Float)?; let predicate = Predicate::Unary(UnaryExpression::new( PredicateOperator::IsNull, Reference::new("a"), )) - .bind(partition_spec.schema_ref().clone(), case_sensitive)?; + .bind(schema.clone(), case_sensitive)?; let expression_evaluator = - create_expression_evaluator(partition_spec, &predicate, case_sensitive)?; + create_expression_evaluator(partition_spec, &schema, &predicate, case_sensitive)?; let data_file = create_data_file_float(); @@ -757,12 +761,11 @@ mod tests { #[test] fn test_expr_always_false() -> Result<()> { let case_sensitive = true; - let partition_spec = create_partition_spec(PrimitiveType::Float)?; - let predicate = - Predicate::AlwaysFalse.bind(partition_spec.schema_ref().clone(), case_sensitive)?; + let (partition_spec, schema) = create_partition_spec(PrimitiveType::Float)?; + let predicate = Predicate::AlwaysFalse.bind(schema.clone(), case_sensitive)?; let expression_evaluator = - create_expression_evaluator(partition_spec, &predicate, case_sensitive)?; + create_expression_evaluator(partition_spec, &schema, &predicate, case_sensitive)?; let data_file = create_data_file_float(); @@ -776,12 +779,11 @@ mod tests { #[test] fn test_expr_always_true() -> Result<()> { let case_sensitive = true; - let partition_spec = create_partition_spec(PrimitiveType::Float)?; - let predicate = - Predicate::AlwaysTrue.bind(partition_spec.schema_ref().clone(), case_sensitive)?; + let (partition_spec, schema) = create_partition_spec(PrimitiveType::Float)?; + let predicate = Predicate::AlwaysTrue.bind(schema.clone(), case_sensitive)?; let expression_evaluator = - create_expression_evaluator(partition_spec, &predicate, case_sensitive)?; + create_expression_evaluator(partition_spec, &schema, &predicate, case_sensitive)?; let data_file = create_data_file_float(); diff --git a/crates/iceberg/src/expr/visitors/inclusive_metrics_evaluator.rs b/crates/iceberg/src/expr/visitors/inclusive_metrics_evaluator.rs index 7b04fae3a..6ae1fcb89 100644 --- a/crates/iceberg/src/expr/visitors/inclusive_metrics_evaluator.rs +++ b/crates/iceberg/src/expr/visitors/inclusive_metrics_evaluator.rs @@ -495,8 +495,9 @@ mod test { UnaryExpression, }; use crate::spec::{ - BoundPartitionSpec, DataContentType, DataFile, DataFileFormat, Datum, NestedField, - PrimitiveType, Schema, Struct, Transform, Type, UnboundPartitionField, + DataContentType, DataFile, DataFileFormat, Datum, NestedField, PartitionSpec, + PartitionSpecRef, PrimitiveType, Schema, SchemaRef, Struct, Transform, Type, + UnboundPartitionField, }; const INT_MIN_VALUE: i32 = 30; @@ -504,10 +505,10 @@ mod test { #[test] fn test_data_file_no_partitions() { - let partition_spec_ref = create_test_partition_spec(); + let (_partition_spec_ref, schema_ref) = create_test_partition_spec(); let partition_filter = Predicate::AlwaysTrue - .bind(partition_spec_ref.schema_ref().clone(), false) + .bind(schema_ref.clone(), false) .unwrap(); let case_sensitive = false; @@ -1645,7 +1646,7 @@ mod test { assert!(result, "Should read: NotIn on no nulls column"); } - fn create_test_partition_spec() -> Arc { + fn create_test_partition_spec() -> (PartitionSpecRef, SchemaRef) { let table_schema = Schema::builder() .with_fields(vec![Arc::new(NestedField::optional( 1, @@ -1656,7 +1657,7 @@ mod test { .unwrap(); let table_schema_ref = Arc::new(table_schema); - let partition_spec = BoundPartitionSpec::builder(table_schema_ref.clone()) + let partition_spec = PartitionSpec::builder(table_schema_ref.clone()) .with_spec_id(1) .add_unbound_fields(vec![UnboundPartitionField::builder() .source_id(1) @@ -1667,7 +1668,7 @@ mod test { .unwrap() .build() .unwrap(); - Arc::new(partition_spec) + (Arc::new(partition_spec), table_schema_ref) } fn not_null(reference: &str) -> BoundPredicate { diff --git a/crates/iceberg/src/expr/visitors/inclusive_projection.rs b/crates/iceberg/src/expr/visitors/inclusive_projection.rs index 7c6e0b2d5..d075fdd81 100644 --- a/crates/iceberg/src/expr/visitors/inclusive_projection.rs +++ b/crates/iceberg/src/expr/visitors/inclusive_projection.rs @@ -21,16 +21,16 @@ use fnv::FnvHashSet; use crate::expr::visitors::bound_predicate_visitor::{visit, BoundPredicateVisitor}; use crate::expr::{BoundPredicate, BoundReference, Predicate}; -use crate::spec::{Datum, PartitionField, SchemalessPartitionSpecRef}; +use crate::spec::{Datum, PartitionField, PartitionSpecRef}; use crate::Error; pub(crate) struct InclusiveProjection { - partition_spec: SchemalessPartitionSpecRef, + partition_spec: PartitionSpecRef, cached_parts: HashMap>, } impl InclusiveProjection { - pub(crate) fn new(partition_spec: SchemalessPartitionSpecRef) -> Self { + pub(crate) fn new(partition_spec: PartitionSpecRef) -> Self { Self { partition_spec, cached_parts: HashMap::new(), @@ -235,7 +235,7 @@ mod tests { use crate::expr::visitors::inclusive_projection::InclusiveProjection; use crate::expr::{Bind, Predicate, Reference}; use crate::spec::{ - BoundPartitionSpec, Datum, NestedField, PrimitiveType, Schema, Transform, Type, + Datum, NestedField, PartitionSpec, PrimitiveType, Schema, Transform, Type, UnboundPartitionField, }; @@ -267,11 +267,10 @@ mod tests { let schema = build_test_schema(); let arc_schema = Arc::new(schema); - let partition_spec = BoundPartitionSpec::builder(arc_schema.clone()) + let partition_spec = PartitionSpec::builder(arc_schema.clone()) .with_spec_id(1) .build() - .unwrap() - .into_schemaless(); + .unwrap(); let arc_partition_spec = Arc::new(partition_spec); @@ -298,7 +297,7 @@ mod tests { let schema = build_test_schema(); let arc_schema = Arc::new(schema); - let partition_spec = BoundPartitionSpec::builder(arc_schema.clone()) + let partition_spec = PartitionSpec::builder(arc_schema.clone()) .with_spec_id(1) .add_unbound_field( UnboundPartitionField::builder() @@ -310,8 +309,7 @@ mod tests { ) .unwrap() .build() - .unwrap() - .into_schemaless(); + .unwrap(); let arc_partition_spec = Arc::new(partition_spec); @@ -336,7 +334,7 @@ mod tests { let schema = build_test_schema(); let arc_schema = Arc::new(schema); - let partition_spec = BoundPartitionSpec::builder(arc_schema.clone()) + let partition_spec = PartitionSpec::builder(arc_schema.clone()) .with_spec_id(1) .add_unbound_fields(vec![UnboundPartitionField { source_id: 2, @@ -346,8 +344,7 @@ mod tests { }]) .unwrap() .build() - .unwrap() - .into_schemaless(); + .unwrap(); let arc_partition_spec = Arc::new(partition_spec); @@ -372,7 +369,7 @@ mod tests { let schema = build_test_schema(); let arc_schema = Arc::new(schema); - let partition_spec = BoundPartitionSpec::builder(arc_schema.clone()) + let partition_spec = PartitionSpec::builder(arc_schema.clone()) .with_spec_id(1) .add_unbound_fields(vec![UnboundPartitionField { source_id: 2, @@ -382,8 +379,7 @@ mod tests { }]) .unwrap() .build() - .unwrap() - .into_schemaless(); + .unwrap(); let arc_partition_spec = Arc::new(partition_spec); @@ -408,7 +404,7 @@ mod tests { let schema = build_test_schema(); let arc_schema = Arc::new(schema); - let partition_spec = BoundPartitionSpec::builder(arc_schema.clone()) + let partition_spec = PartitionSpec::builder(arc_schema.clone()) .with_spec_id(1) .add_unbound_fields(vec![UnboundPartitionField { source_id: 2, @@ -418,8 +414,7 @@ mod tests { }]) .unwrap() .build() - .unwrap() - .into_schemaless(); + .unwrap(); let arc_partition_spec = Arc::new(partition_spec); @@ -444,7 +439,7 @@ mod tests { let schema = build_test_schema(); let arc_schema = Arc::new(schema); - let partition_spec = BoundPartitionSpec::builder(arc_schema.clone()) + let partition_spec = PartitionSpec::builder(arc_schema.clone()) .with_spec_id(1) .add_unbound_field( UnboundPartitionField::builder() @@ -456,8 +451,7 @@ mod tests { ) .unwrap() .build() - .unwrap() - .into_schemaless(); + .unwrap(); let arc_partition_spec = Arc::new(partition_spec); @@ -485,7 +479,7 @@ mod tests { let schema = build_test_schema(); let arc_schema = Arc::new(schema); - let partition_spec = BoundPartitionSpec::builder(arc_schema.clone()) + let partition_spec = PartitionSpec::builder(arc_schema.clone()) .with_spec_id(1) .add_unbound_field( UnboundPartitionField::builder() @@ -497,8 +491,7 @@ mod tests { ) .unwrap() .build() - .unwrap() - .into_schemaless(); + .unwrap(); let arc_partition_spec = Arc::new(partition_spec); diff --git a/crates/iceberg/src/spec/manifest.rs b/crates/iceberg/src/spec/manifest.rs index 60f5469cd..086c63080 100644 --- a/crates/iceberg/src/spec/manifest.rs +++ b/crates/iceberg/src/spec/manifest.rs @@ -31,9 +31,9 @@ use typed_builder::TypedBuilder; use self::_const_schema::{manifest_schema_v1, manifest_schema_v2}; use super::{ - BoundPartitionSpec, Datum, FieldSummary, FormatVersion, ManifestContentType, ManifestFile, - PrimitiveLiteral, PrimitiveType, Schema, SchemaId, SchemaRef, Struct, INITIAL_SEQUENCE_NUMBER, - UNASSIGNED_SEQUENCE_NUMBER, + Datum, FieldSummary, FormatVersion, ManifestContentType, ManifestFile, PartitionSpec, + PrimitiveLiteral, PrimitiveType, Schema, SchemaId, SchemaRef, Struct, StructType, + INITIAL_SEQUENCE_NUMBER, UNASSIGNED_SEQUENCE_NUMBER, }; use crate::error::Result; use crate::io::OutputFile; @@ -57,7 +57,7 @@ impl Manifest { let metadata = ManifestMetadata::parse(meta)?; // Parse manifest entries - let partition_type = metadata.partition_spec.partition_type(); + let partition_type = metadata.partition_spec.partition_type(&metadata.schema)?; let entries = match metadata.format_version { FormatVersion::V1 => { @@ -67,7 +67,7 @@ impl Manifest { .into_iter() .map(|value| { from_value::<_serde::ManifestEntryV1>(&value?)? - .try_into(partition_type, &metadata.schema) + .try_into(&partition_type, &metadata.schema) }) .collect::>>()? } @@ -78,7 +78,7 @@ impl Manifest { .into_iter() .map(|value| { from_value::<_serde::ManifestEntryV2>(&value?)? - .try_into(partition_type, &metadata.schema) + .try_into(&partition_type, &metadata.schema) }) .collect::>>()? } @@ -215,11 +215,10 @@ impl ManifestWriter { fn construct_partition_summaries( &mut self, - partition_spec: &BoundPartitionSpec, + partition_type: &StructType, ) -> Result> { let partitions = std::mem::take(&mut self.partitions); - let mut field_stats: Vec<_> = partition_spec - .partition_type() + let mut field_stats: Vec<_> = partition_type .fields() .iter() .map(|f| PartitionFieldStats::new(f.field_type.as_primitive_type().unwrap().clone())) @@ -236,7 +235,10 @@ impl ManifestWriter { /// Write a manifest. pub async fn write(mut self, manifest: Manifest) -> Result { // Create the avro writer - let partition_type = manifest.metadata.partition_spec.partition_type(); + let partition_type = manifest + .metadata + .partition_spec + .partition_type(&manifest.metadata.schema)?; let table_schema = &manifest.metadata.schema; let avro_schema = match manifest.metadata.format_version { FormatVersion::V1 => manifest_schema_v1(partition_type.clone())?, @@ -311,12 +313,12 @@ impl ManifestWriter { let value = match manifest.metadata.format_version { FormatVersion::V1 => to_value(_serde::ManifestEntryV1::try_from( (*entry).clone(), - partition_type, + &partition_type, )?)? .resolve(&avro_schema)?, FormatVersion::V2 => to_value(_serde::ManifestEntryV2::try_from( (*entry).clone(), - partition_type, + &partition_type, )?)? .resolve(&avro_schema)?, }; @@ -328,8 +330,7 @@ impl ManifestWriter { let length = content.len(); self.output.write(Bytes::from(content)).await?; - let partition_summary = - self.construct_partition_summaries(&manifest.metadata.partition_spec)?; + let partition_summary = self.construct_partition_summaries(&partition_type)?; Ok(ManifestFile { manifest_path: self.output.location().to_string(), @@ -736,7 +737,7 @@ pub struct ManifestMetadata { /// ID of the schema used to write the manifest as a string schema_id: SchemaId, /// The partition spec used to write the manifest - partition_spec: BoundPartitionSpec, + partition_spec: PartitionSpec, /// Table format version number of the manifest as a string format_version: FormatVersion, /// Type of content files tracked by the manifest: “data” or “deletes” @@ -803,7 +804,7 @@ impl ManifestMetadata { }) .transpose()? .unwrap_or(0); - BoundPartitionSpec::builder(schema.clone()) + PartitionSpec::builder(schema.clone()) .with_spec_id(spec_id) .add_unbound_fields(fields.into_iter().map(|f| f.into_unbound()))? .build()? @@ -1635,7 +1636,7 @@ mod tests { metadata: ManifestMetadata { schema_id: 0, schema: schema.clone(), - partition_spec: BoundPartitionSpec::builder(schema).with_spec_id(0).build().unwrap(), + partition_spec: PartitionSpec::builder(schema).with_spec_id(0).build().unwrap(), content: ManifestContentType::Data, format_version: FormatVersion::V2, }, @@ -1748,7 +1749,7 @@ mod tests { metadata: ManifestMetadata { schema_id: 0, schema: schema.clone(), - partition_spec: BoundPartitionSpec::builder(schema) + partition_spec: PartitionSpec::builder(schema) .with_spec_id(0).add_partition_field("v_int", "v_int", Transform::Identity).unwrap() .add_partition_field("v_long", "v_long", Transform::Identity).unwrap().build().unwrap(), content: ManifestContentType::Data, @@ -1859,7 +1860,7 @@ mod tests { metadata: ManifestMetadata { schema_id: 1, schema: schema.clone(), - partition_spec: BoundPartitionSpec::builder(schema).with_spec_id(0).build().unwrap(), + partition_spec: PartitionSpec::builder(schema).with_spec_id(0).build().unwrap(), content: ManifestContentType::Data, format_version: FormatVersion::V1, }, @@ -1923,7 +1924,7 @@ mod tests { metadata: ManifestMetadata { schema_id: 0, schema: schema.clone(), - partition_spec: BoundPartitionSpec::builder(schema).add_partition_field("category", "category", Transform::Identity).unwrap().build().unwrap(), + partition_spec: PartitionSpec::builder(schema).add_partition_field("category", "category", Transform::Identity).unwrap().build().unwrap(), content: ManifestContentType::Data, format_version: FormatVersion::V1, }, @@ -2002,7 +2003,7 @@ mod tests { metadata: ManifestMetadata { schema_id: 0, schema: schema.clone(), - partition_spec: BoundPartitionSpec::builder(schema).with_spec_id(0).build().unwrap(), + partition_spec: PartitionSpec::builder(schema).with_spec_id(0).build().unwrap(), content: ManifestContentType::Data, format_version: FormatVersion::V2, }, @@ -2074,7 +2075,7 @@ mod tests { metadata: ManifestMetadata { schema_id: 0, schema: schema.clone(), - partition_spec: BoundPartitionSpec::builder(schema).with_spec_id(0).build().unwrap(), + partition_spec: PartitionSpec::builder(schema).with_spec_id(0).build().unwrap(), content: ManifestContentType::Data, format_version: FormatVersion::V2, }, @@ -2141,7 +2142,7 @@ mod tests { .build() .unwrap(), ); - let partition_spec = BoundPartitionSpec::builder(schema.clone()) + let partition_spec = PartitionSpec::builder(schema.clone()) .with_spec_id(0) .add_partition_field("time", "year_of_time", Transform::Year) .unwrap() diff --git a/crates/iceberg/src/spec/partition.rs b/crates/iceberg/src/spec/partition.rs index 445e7d441..e6405be4c 100644 --- a/crates/iceberg/src/spec/partition.rs +++ b/crates/iceberg/src/spec/partition.rs @@ -30,9 +30,6 @@ use crate::{Error, ErrorKind, Result}; pub(crate) const UNPARTITIONED_LAST_ASSIGNED_ID: i32 = 999; pub(crate) const DEFAULT_PARTITION_SPEC_ID: i32 = 0; -/// Reference to [`BoundPartitionSpec`]. -pub type BoundPartitionSpecRef = Arc; - /// Partition fields capture the transform from table data to partition values. #[derive(Debug, Serialize, Deserialize, PartialEq, Eq, Clone, TypedBuilder)] #[serde(rename_all = "kebab-case")] @@ -55,69 +52,45 @@ impl PartitionField { } } +/// Reference to [`PartitionSpec`]. +pub type PartitionSpecRef = Arc; /// Partition spec that defines how to produce a tuple of partition values from a record. -/// `PartitionSpec` is bound to a specific schema. -#[derive(Debug, PartialEq, Eq, Clone)] -pub struct BoundPartitionSpec { - /// Identifier for PartitionSpec - spec_id: i32, - /// Details of the partition spec - fields: Vec, - /// The schema this partition spec is bound to - schema: SchemaRef, - /// Type of the partition spec - partition_type: StructType, -} - -/// Reference to [`SchemalessPartitionSpec`]. -pub type SchemalessPartitionSpecRef = Arc; -/// Partition spec that defines how to produce a tuple of partition values from a record. -/// Schemaless partition specs are never constructed manually. They occur when a table is mutated -/// and partition spec and schemas are updated. While old partition specs are retained, the bound -/// schema might not be available anymore as part of the table metadata. +/// +/// A [`PartitionSpec`] is originally obtained by binding an [`UnboundPartitionSpec`] to a schema and is +/// only guaranteed to be valid for that schema. The main difference between [`PartitionSpec`] and +/// [`UnboundPartitionSpec`] is that the former has field ids assigned, +/// while field ids are optional for [`UnboundPartitionSpec`]. #[derive(Debug, Serialize, Deserialize, PartialEq, Eq, Clone)] #[serde(rename_all = "kebab-case")] -pub struct SchemalessPartitionSpec { +pub struct PartitionSpec { /// Identifier for PartitionSpec spec_id: i32, /// Details of the partition spec fields: Vec, } -impl BoundPartitionSpec { - /// Create partition spec builder +impl PartitionSpec { + /// Create a new partition spec builder with the given schema. pub fn builder(schema: impl Into) -> PartitionSpecBuilder { PartitionSpecBuilder::new(schema) } - /// Get a new unpatitioned partition spec - pub fn unpartition_spec(schema: impl Into) -> Self { - Self { - spec_id: DEFAULT_PARTITION_SPEC_ID, - fields: vec![], - schema: schema.into(), - partition_type: StructType::new(vec![]), - } - } - - /// Spec id of the partition spec - pub fn spec_id(&self) -> i32 { - self.spec_id - } - /// Fields of the partition spec pub fn fields(&self) -> &[PartitionField] { &self.fields } - /// The schema this partition spec is bound to - pub fn schema(&self) -> &Schema { - &self.schema + /// Spec id of the partition spec + pub fn spec_id(&self) -> i32 { + self.spec_id } - /// The schema ref this partition spec is bound to - pub fn schema_ref(&self) -> &SchemaRef { - &self.schema + /// Get a new unpartitioned partition spec + pub fn unpartition_spec() -> Self { + Self { + spec_id: DEFAULT_PARTITION_SPEC_ID, + fields: vec![], + } } /// Returns if the partition spec is unpartitioned. @@ -127,16 +100,19 @@ impl BoundPartitionSpec { self.fields.is_empty() || self.fields.iter().all(|f| f.transform == Transform::Void) } - /// Turn this partition spec into an unbound partition spec. - /// - /// The `field_id` is retained as `partition_id` in the unbound partition spec. + /// Returns the partition type of this partition spec. + pub fn partition_type(&self, schema: &Schema) -> Result { + PartitionSpecBuilder::partition_type(&self.fields, schema) + } + + /// Convert to unbound partition spec pub fn into_unbound(self) -> UnboundPartitionSpec { self.into() } - /// Turn this partition spec into a preserved partition spec. - pub fn into_schemaless(self) -> SchemalessPartitionSpec { - self.into() + /// Change the spec id of the partition spec + pub fn with_spec_id(self, spec_id: i32) -> Self { + Self { spec_id, ..self } } /// Check if this partition spec has sequential partition ids. @@ -151,11 +127,6 @@ impl BoundPartitionSpec { self.fields.iter().map(|f| f.field_id).max() } - /// Returns the partition type of this partition spec. - pub fn partition_type(&self) -> &StructType { - &self.partition_type - } - /// Check if this partition spec is compatible with another partition spec. /// /// Returns true if the partition spec is equal to the other spec with partition field ids ignored and @@ -165,7 +136,7 @@ impl BoundPartitionSpec { /// * Field names /// * Source column ids /// * Transforms - pub fn is_compatible_with_schemaless(&self, other: &SchemalessPartitionSpec) -> bool { + pub fn is_compatible_with(&self, other: &PartitionSpec) -> bool { if self.fields.len() != other.fields.len() { return false; } @@ -181,46 +152,6 @@ impl BoundPartitionSpec { true } - - /// Change the spec id of the partition spec - pub fn with_spec_id(self, spec_id: i32) -> Self { - Self { spec_id, ..self } - } -} - -impl SchemalessPartitionSpec { - /// 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 - } - - /// Bind this schemaless partition spec to a schema. - pub fn bind(self, schema: impl Into) -> Result { - PartitionSpecBuilder::new_from_unbound(self.into_unbound(), schema)?.build() - } - - /// Get a new unpatitioned partition spec - pub fn unpartition_spec() -> Self { - Self { - spec_id: DEFAULT_PARTITION_SPEC_ID, - fields: vec![], - } - } - - /// Returns the partition type of this partition spec. - pub fn partition_type(&self, schema: &Schema) -> Result { - PartitionSpecBuilder::partition_type(&self.fields, schema) - } - - /// Convert to unbound partition spec - pub fn into_unbound(self) -> UnboundPartitionSpec { - self.into() - } } /// Reference to [`UnboundPartitionSpec`]. @@ -242,6 +173,8 @@ pub struct UnboundPartitionField { } /// Unbound partition spec can be built without a schema and later bound to a schema. +/// They are used to transport schema information as part of the REST specification. +/// The main difference to [`PartitionSpec`] is that the field ids are optional. #[derive(Debug, Serialize, Deserialize, PartialEq, Eq, Clone, Default)] #[serde(rename_all = "kebab-case")] pub struct UnboundPartitionSpec { @@ -258,7 +191,7 @@ impl UnboundPartitionSpec { } /// Bind this unbound partition spec to a schema. - pub fn bind(self, schema: impl Into) -> Result { + pub fn bind(self, schema: impl Into) -> Result { PartitionSpecBuilder::new_from_unbound(self, schema)?.build() } @@ -307,17 +240,8 @@ impl From for UnboundPartitionField { } } -impl From for UnboundPartitionSpec { - fn from(spec: BoundPartitionSpec) -> Self { - UnboundPartitionSpec { - spec_id: Some(spec.spec_id), - fields: spec.fields.into_iter().map(Into::into).collect(), - } - } -} - -impl From for UnboundPartitionSpec { - fn from(spec: SchemalessPartitionSpec) -> Self { +impl From for UnboundPartitionSpec { + fn from(spec: PartitionSpec) -> Self { UnboundPartitionSpec { spec_id: Some(spec.spec_id), fields: spec.fields.into_iter().map(Into::into).collect(), @@ -325,15 +249,6 @@ impl From for UnboundPartitionSpec { } } -impl From for SchemalessPartitionSpec { - fn from(spec: BoundPartitionSpec) -> Self { - SchemalessPartitionSpec { - spec_id: spec.spec_id, - fields: spec.fields, - } - } -} - /// Create a new UnboundPartitionSpec #[derive(Debug, Default)] pub struct UnboundPartitionSpecBuilder { @@ -514,14 +429,11 @@ impl PartitionSpecBuilder { } /// Build a bound partition spec with the given schema. - pub fn build(self) -> Result { + pub fn build(self) -> Result { let fields = Self::set_field_ids(self.fields, self.last_assigned_field_id)?; - let partition_type = Self::partition_type(&fields, &self.schema)?; - Ok(BoundPartitionSpec { + Ok(PartitionSpec { spec_id: self.spec_id.unwrap_or(DEFAULT_PARTITION_SPEC_ID), fields, - partition_type, - schema: self.schema, }) } @@ -774,7 +686,7 @@ mod tests { } "#; - let partition_spec: SchemalessPartitionSpec = serde_json::from_str(spec).unwrap(); + let partition_spec: PartitionSpec = serde_json::from_str(spec).unwrap(); assert_eq!(4, partition_spec.fields[0].source_id); assert_eq!(1000, partition_spec.fields[0].field_id); assert_eq!("ts_day", partition_spec.fields[0].name); @@ -806,7 +718,7 @@ mod tests { ]) .build() .unwrap(); - let partition_spec = BoundPartitionSpec::builder(schema.clone()) + let partition_spec = PartitionSpec::builder(schema.clone()) .with_spec_id(1) .build() .unwrap(); @@ -815,7 +727,7 @@ mod tests { "Empty partition spec should be unpartitioned" ); - let partition_spec = BoundPartitionSpec::builder(schema.clone()) + let partition_spec = PartitionSpec::builder(schema.clone()) .add_unbound_fields(vec![ UnboundPartitionField::builder() .source_id(1) @@ -837,7 +749,7 @@ mod tests { "Partition spec with one non void transform should not be unpartitioned" ); - let partition_spec = BoundPartitionSpec::builder(schema.clone()) + let partition_spec = PartitionSpec::builder(schema.clone()) .with_spec_id(1) .add_unbound_fields(vec![ UnboundPartitionField::builder() @@ -935,14 +847,14 @@ mod tests { ]) .build() .unwrap(); - let partition_spec = BoundPartitionSpec::builder(schema.clone()) + let partition_spec = PartitionSpec::builder(schema.clone()) .with_spec_id(0) .build() .unwrap(); - let partition_type = partition_spec.partition_type(); + let partition_type = partition_spec.partition_type(&schema).unwrap(); assert_eq!(0, partition_type.fields().len()); - let unpartition_spec = BoundPartitionSpec::unpartition_spec(schema); + let unpartition_spec = PartitionSpec::unpartition_spec(); assert_eq!(partition_spec, unpartition_spec); } @@ -970,7 +882,7 @@ mod tests { } "#; - let partition_spec: SchemalessPartitionSpec = serde_json::from_str(spec).unwrap(); + let partition_spec: PartitionSpec = serde_json::from_str(spec).unwrap(); let schema = Schema::builder() .with_fields(vec![ NestedField::required(1, "id", Type::Primitive(crate::spec::PrimitiveType::Int)) @@ -1046,7 +958,7 @@ mod tests { } "#; - let partition_spec: SchemalessPartitionSpec = serde_json::from_str(spec).unwrap(); + let partition_spec: PartitionSpec = serde_json::from_str(spec).unwrap(); let schema = Schema::builder() .with_fields(vec![ NestedField::required(1, "id", Type::Primitive(crate::spec::PrimitiveType::Int)) @@ -1113,7 +1025,7 @@ mod tests { } "#; - let partition_spec: SchemalessPartitionSpec = serde_json::from_str(spec).unwrap(); + let partition_spec: PartitionSpec = serde_json::from_str(spec).unwrap(); let schema = Schema::builder() .with_fields(vec![ NestedField::required(1, "id", Type::Primitive(crate::spec::PrimitiveType::Int)) @@ -1131,50 +1043,6 @@ mod tests { assert!(partition_spec.partition_type(&schema).is_err()); } - #[test] - fn test_schemaless_bind_schema_keeps_field_ids_and_spec_id() { - let schema: Schema = Schema::builder() - .with_fields(vec![ - NestedField::required(1, "id", Type::Primitive(crate::spec::PrimitiveType::Int)) - .into(), - NestedField::required( - 2, - "name", - Type::Primitive(crate::spec::PrimitiveType::String), - ) - .into(), - ]) - .build() - .unwrap(); - - let partition_spec = BoundPartitionSpec::builder(schema.clone()) - .with_spec_id(99) - .add_unbound_field(UnboundPartitionField { - source_id: 1, - field_id: Some(1010), - name: "id".to_string(), - transform: Transform::Identity, - }) - .unwrap() - .add_unbound_field(UnboundPartitionField { - source_id: 2, - field_id: Some(1001), - name: "name_void".to_string(), - transform: Transform::Void, - }) - .unwrap() - .build() - .unwrap(); - - let schemaless_partition_spec = SchemalessPartitionSpec::from(partition_spec.clone()); - let bound_partition_spec = schemaless_partition_spec.bind(schema).unwrap(); - - assert_eq!(partition_spec, bound_partition_spec); - assert_eq!(partition_spec.fields[0].field_id, 1010); - assert_eq!(partition_spec.fields[1].field_id, 1001); - assert_eq!(bound_partition_spec.spec_id(), 99); - } - #[test] fn test_builder_disallow_duplicate_names() { UnboundPartitionSpec::builder() @@ -1199,7 +1067,7 @@ mod tests { ]) .build() .unwrap(); - BoundPartitionSpec::builder(schema.clone()) + PartitionSpec::builder(schema.clone()) .add_unbound_field(UnboundPartitionField { source_id: 1, field_id: Some(1000), @@ -1237,7 +1105,7 @@ mod tests { ]) .build() .unwrap(); - let spec = BoundPartitionSpec::builder(schema.clone()) + let spec = PartitionSpec::builder(schema.clone()) .with_spec_id(1) .add_unbound_field(UnboundPartitionField { source_id: 1, @@ -1285,34 +1153,36 @@ mod tests { .build() .unwrap(); - BoundPartitionSpec::builder(schema.clone()) + PartitionSpec::builder(schema.clone()) .with_spec_id(1) .build() .unwrap(); - let spec = BoundPartitionSpec::builder(schema.clone()) + let spec = PartitionSpec::builder(schema.clone()) .with_spec_id(1) .add_partition_field("id", "id_bucket[16]", Transform::Bucket(16)) .unwrap() .build() .unwrap(); - assert_eq!(spec, BoundPartitionSpec { + assert_eq!(spec, PartitionSpec { spec_id: 1, - schema: schema.into(), fields: vec![PartitionField { source_id: 1, field_id: 1000, name: "id_bucket[16]".to_string(), transform: Transform::Bucket(16), }], - partition_type: StructType::new(vec![NestedField::optional( + }); + assert_eq!( + spec.partition_type(&schema).unwrap(), + StructType::new(vec![NestedField::optional( 1000, "id_bucket[16]", Type::Primitive(PrimitiveType::Int) ) .into()]) - }); + ) } #[test] @@ -1327,12 +1197,12 @@ mod tests { .build() .unwrap(); - BoundPartitionSpec::builder(schema.clone()) + PartitionSpec::builder(schema.clone()) .with_spec_id(1) .build() .unwrap(); - let err = BoundPartitionSpec::builder(schema) + let err = PartitionSpec::builder(schema) .with_spec_id(1) .add_unbound_field(UnboundPartitionField { source_id: 1, @@ -1360,12 +1230,12 @@ mod tests { .build() .unwrap(); - BoundPartitionSpec::builder(schema.clone()) + PartitionSpec::builder(schema.clone()) .with_spec_id(1) .build() .unwrap(); - BoundPartitionSpec::builder(schema.clone()) + PartitionSpec::builder(schema.clone()) .with_spec_id(1) .add_unbound_field(UnboundPartitionField { source_id: 1, @@ -1378,7 +1248,7 @@ mod tests { .unwrap(); // Not OK for different source id - BoundPartitionSpec::builder(schema) + PartitionSpec::builder(schema) .with_spec_id(1) .add_unbound_field(UnboundPartitionField { source_id: 2, @@ -1412,7 +1282,7 @@ mod tests { .unwrap(); // Valid - BoundPartitionSpec::builder(schema.clone()) + PartitionSpec::builder(schema.clone()) .with_spec_id(1) .add_unbound_fields(vec![ UnboundPartitionField { @@ -1433,7 +1303,7 @@ mod tests { .unwrap(); // Invalid - BoundPartitionSpec::builder(schema) + PartitionSpec::builder(schema) .with_spec_id(1) .add_unbound_fields(vec![ UnboundPartitionField { @@ -1479,7 +1349,7 @@ mod tests { .build() .unwrap(); - BoundPartitionSpec::builder(schema) + PartitionSpec::builder(schema) .with_spec_id(1) .add_unbound_field(UnboundPartitionField { source_id: 1, @@ -1530,7 +1400,7 @@ mod tests { .build() .unwrap(); - let partition_spec_1 = BoundPartitionSpec::builder(schema.clone()) + let partition_spec_1 = PartitionSpec::builder(schema.clone()) .with_spec_id(1) .add_unbound_field(UnboundPartitionField { source_id: 1, @@ -1542,7 +1412,7 @@ mod tests { .build() .unwrap(); - let partition_spec_2 = BoundPartitionSpec::builder(schema) + let partition_spec_2 = PartitionSpec::builder(schema) .with_spec_id(1) .add_unbound_field(UnboundPartitionField { source_id: 1, @@ -1554,7 +1424,7 @@ mod tests { .build() .unwrap(); - assert!(partition_spec_1.is_compatible_with_schemaless(&partition_spec_2.into_schemaless())); + assert!(partition_spec_1.is_compatible_with(&partition_spec_2)); } #[test] @@ -1569,7 +1439,7 @@ mod tests { .build() .unwrap(); - let partition_spec_1 = BoundPartitionSpec::builder(schema.clone()) + let partition_spec_1 = PartitionSpec::builder(schema.clone()) .with_spec_id(1) .add_unbound_field(UnboundPartitionField { source_id: 1, @@ -1581,7 +1451,7 @@ mod tests { .build() .unwrap(); - let partition_spec_2 = BoundPartitionSpec::builder(schema) + let partition_spec_2 = PartitionSpec::builder(schema) .with_spec_id(1) .add_unbound_field(UnboundPartitionField { source_id: 1, @@ -1593,9 +1463,7 @@ mod tests { .build() .unwrap(); - assert!( - !partition_spec_1.is_compatible_with_schemaless(&partition_spec_2.into_schemaless()) - ); + assert!(!partition_spec_1.is_compatible_with(&partition_spec_2)); } #[test] @@ -1614,7 +1482,7 @@ mod tests { .build() .unwrap(); - let partition_spec_1 = BoundPartitionSpec::builder(schema.clone()) + let partition_spec_1 = PartitionSpec::builder(schema.clone()) .with_spec_id(1) .add_unbound_field(UnboundPartitionField { source_id: 1, @@ -1626,7 +1494,7 @@ mod tests { .build() .unwrap(); - let partition_spec_2 = BoundPartitionSpec::builder(schema) + let partition_spec_2 = PartitionSpec::builder(schema) .with_spec_id(1) .add_unbound_field(UnboundPartitionField { source_id: 2, @@ -1638,9 +1506,7 @@ mod tests { .build() .unwrap(); - assert!( - !partition_spec_1.is_compatible_with_schemaless(&partition_spec_2.into_schemaless()) - ); + assert!(!partition_spec_1.is_compatible_with(&partition_spec_2)); } #[test] @@ -1659,7 +1525,7 @@ mod tests { .build() .unwrap(); - let partition_spec_1 = BoundPartitionSpec::builder(schema.clone()) + let partition_spec_1 = PartitionSpec::builder(schema.clone()) .with_spec_id(1) .add_unbound_field(UnboundPartitionField { source_id: 1, @@ -1678,7 +1544,7 @@ mod tests { .build() .unwrap(); - let partition_spec_2 = BoundPartitionSpec::builder(schema) + let partition_spec_2 = PartitionSpec::builder(schema) .with_spec_id(1) .add_unbound_field(UnboundPartitionField { source_id: 2, @@ -1697,18 +1563,15 @@ mod tests { .build() .unwrap(); - assert!( - !partition_spec_1.is_compatible_with_schemaless(&partition_spec_2.into_schemaless()) - ); + assert!(!partition_spec_1.is_compatible_with(&partition_spec_2)); } #[test] fn test_highest_field_id_unpartitioned() { - let spec = - BoundPartitionSpec::builder(Schema::builder().with_fields(vec![]).build().unwrap()) - .with_spec_id(1) - .build() - .unwrap(); + let spec = PartitionSpec::builder(Schema::builder().with_fields(vec![]).build().unwrap()) + .with_spec_id(1) + .build() + .unwrap(); assert!(spec.highest_field_id().is_none()); } @@ -1729,7 +1592,7 @@ mod tests { .build() .unwrap(); - let spec = BoundPartitionSpec::builder(schema) + let spec = PartitionSpec::builder(schema) .with_spec_id(1) .add_unbound_field(UnboundPartitionField { source_id: 1, @@ -1767,7 +1630,7 @@ mod tests { .build() .unwrap(); - let spec = BoundPartitionSpec::builder(schema) + let spec = PartitionSpec::builder(schema) .with_spec_id(1) .add_unbound_field(UnboundPartitionField { source_id: 1, @@ -1807,7 +1670,7 @@ mod tests { .build() .unwrap(); - let spec = BoundPartitionSpec::builder(schema) + let spec = PartitionSpec::builder(schema) .with_spec_id(1) .add_unbound_field(UnboundPartitionField { source_id: 1, @@ -1847,7 +1710,7 @@ mod tests { .build() .unwrap(); - let spec = BoundPartitionSpec::builder(schema) + let spec = PartitionSpec::builder(schema) .with_spec_id(1) .add_unbound_field(UnboundPartitionField { source_id: 1, diff --git a/crates/iceberg/src/spec/table_metadata.rs b/crates/iceberg/src/spec/table_metadata.rs index bc1fe17c1..bb1cc917c 100644 --- a/crates/iceberg/src/spec/table_metadata.rs +++ b/crates/iceberg/src/spec/table_metadata.rs @@ -32,8 +32,8 @@ use uuid::Uuid; use super::snapshot::SnapshotReference; pub use super::table_metadata_builder::{TableMetadataBuildResult, TableMetadataBuilder}; use super::{ - BoundPartitionSpecRef, SchemaId, SchemaRef, SchemalessPartitionSpecRef, Snapshot, SnapshotRef, - SnapshotRetention, SortOrder, SortOrderRef, DEFAULT_PARTITION_SPEC_ID, + PartitionSpecRef, SchemaId, SchemaRef, Snapshot, SnapshotRef, SnapshotRetention, SortOrder, + SortOrderRef, StructType, DEFAULT_PARTITION_SPEC_ID, }; use crate::error::{timestamp_ms_to_utc, Result}; use crate::{Error, ErrorKind}; @@ -119,9 +119,11 @@ pub struct TableMetadata { /// ID of the table’s current schema. pub(crate) current_schema_id: i32, /// A list of partition specs, stored as full partition spec objects. - pub(crate) partition_specs: HashMap, + pub(crate) partition_specs: HashMap, /// ID of the “current” spec that writers should use by default. - pub(crate) default_spec: BoundPartitionSpecRef, + pub(crate) default_spec: PartitionSpecRef, + /// Partition type of the default partition spec. + pub(crate) default_partition_type: StructType, /// An integer; the highest assigned partition field ID across all partition specs for the table. pub(crate) last_partition_id: i32, ///A string to string map of table properties. This is used to control settings that @@ -252,24 +254,28 @@ impl TableMetadata { /// Returns all partition specs. #[inline] - pub fn partition_specs_iter( - &self, - ) -> impl ExactSizeIterator { + pub fn partition_specs_iter(&self) -> impl ExactSizeIterator { self.partition_specs.values() } /// Lookup partition spec by id. #[inline] - pub fn partition_spec_by_id(&self, spec_id: i32) -> Option<&SchemalessPartitionSpecRef> { + pub fn partition_spec_by_id(&self, spec_id: i32) -> Option<&PartitionSpecRef> { self.partition_specs.get(&spec_id) } /// Get default partition spec #[inline] - pub fn default_partition_spec(&self) -> &BoundPartitionSpecRef { + pub fn default_partition_spec(&self) -> &PartitionSpecRef { &self.default_spec } + /// Return the partition type of the default partition spec. + #[inline] + pub fn default_partition_type(&self) -> &StructType { + &self.default_partition_type + } + #[inline] /// Returns spec id of the "current" partition spec. pub fn default_partition_spec_id(&self) -> i32 { @@ -413,7 +419,7 @@ impl TableMetadata { { self.partition_specs.insert( self.default_spec.spec_id(), - Arc::new(Arc::unwrap_or_clone(self.default_spec.clone()).into_schemaless()), + Arc::new(Arc::unwrap_or_clone(self.default_spec.clone())), ); } @@ -610,7 +616,7 @@ pub(super) mod _serde { use crate::spec::schema::_serde::{SchemaV1, SchemaV2}; use crate::spec::snapshot::_serde::{SnapshotV1, SnapshotV2}; use crate::spec::{ - BoundPartitionSpec, PartitionField, Schema, SchemaRef, SchemalessPartitionSpec, Snapshot, + PartitionField, PartitionSpec, PartitionSpecRef, Schema, SchemaRef, Snapshot, SnapshotReference, SnapshotRetention, SortOrder, }; use crate::{Error, ErrorKind}; @@ -634,7 +640,7 @@ pub(super) mod _serde { pub last_column_id: i32, pub schemas: Vec, pub current_schema_id: i32, - pub partition_specs: Vec, + pub partition_specs: Vec, pub default_spec_id: i32, pub last_partition_id: i32, #[serde(skip_serializing_if = "Option::is_none")] @@ -670,7 +676,7 @@ pub(super) mod _serde { pub current_schema_id: Option, pub partition_spec: Vec, #[serde(skip_serializing_if = "Option::is_none")] - pub partition_specs: Option>, + pub partition_specs: Option>, #[serde(skip_serializing_if = "Option::is_none")] pub default_spec_id: Option, #[serde(skip_serializing_if = "Option::is_none")] @@ -776,17 +782,12 @@ pub(super) mod _serde { .map(|x| (x.spec_id(), Arc::new(x))), ); let default_spec_id = value.default_spec_id; - let default_spec = partition_specs + let default_spec: PartitionSpecRef = partition_specs .get(&value.default_spec_id) - .map(|schemaless_spec| { - (*schemaless_spec.clone()) - .clone() - .bind(current_schema.clone()) - }) - .transpose()? + .map(|spec| (**spec).clone()) .or_else(|| { (DEFAULT_PARTITION_SPEC_ID == default_spec_id) - .then(|| BoundPartitionSpec::unpartition_spec(current_schema.clone())) + .then(PartitionSpec::unpartition_spec) }) .ok_or_else(|| { Error::new( @@ -795,6 +796,7 @@ pub(super) mod _serde { ) })? .into(); + let default_partition_type = default_spec.partition_type(current_schema)?; let mut metadata = TableMetadata { format_version: FormatVersion::V2, @@ -806,6 +808,7 @@ pub(super) mod _serde { current_schema_id: value.current_schema_id, schemas, partition_specs, + default_partition_type, default_spec, last_partition_id: value.last_partition_id, properties: value.properties.unwrap_or_default(), @@ -901,11 +904,10 @@ pub(super) mod _serde { let partition_specs = match value.partition_specs { Some(partition_specs) => partition_specs, - None => vec![BoundPartitionSpec::builder(current_schema.clone()) + None => vec![PartitionSpec::builder(current_schema.clone()) .with_spec_id(DEFAULT_PARTITION_SPEC_ID) .add_unbound_fields(value.partition_spec.into_iter().map(|f| f.into_unbound()))? - .build()? - .into_schemaless()], + .build()?], } .into_iter() .map(|x| (x.spec_id(), Arc::new(x))) @@ -914,10 +916,9 @@ pub(super) mod _serde { let default_spec_id = value .default_spec_id .unwrap_or_else(|| partition_specs.keys().copied().max().unwrap_or_default()); - let default_spec = partition_specs + let default_spec: PartitionSpecRef = partition_specs .get(&default_spec_id) - .map(|x| Arc::unwrap_or_clone(x.clone()).bind(current_schema.clone())) - .transpose()? + .map(|x| Arc::unwrap_or_clone(x.clone())) .ok_or_else(|| { Error::new( ErrorKind::DataInvalid, @@ -925,6 +926,7 @@ pub(super) mod _serde { ) })? .into(); + let default_partition_type = default_spec.partition_type(¤t_schema)?; let mut metadata = TableMetadata { format_version: FormatVersion::V1, @@ -935,6 +937,7 @@ pub(super) mod _serde { last_column_id: value.last_column_id, current_schema_id, default_spec, + default_partition_type, last_partition_id: value .last_partition_id .unwrap_or_else(|| partition_specs.keys().copied().max().unwrap_or_default()), @@ -1205,7 +1208,7 @@ mod tests { use super::{FormatVersion, MetadataLog, SnapshotLog, TableMetadataBuilder}; use crate::spec::table_metadata::TableMetadata; use crate::spec::{ - BoundPartitionSpec, NestedField, NullOrder, Operation, PrimitiveType, Schema, Snapshot, + NestedField, NullOrder, Operation, PartitionSpec, PrimitiveType, Schema, Snapshot, SnapshotReference, SnapshotRetention, SortDirection, SortField, SortOrder, Summary, Transform, Type, UnboundPartitionField, }; @@ -1311,7 +1314,7 @@ mod tests { .build() .unwrap(); - let partition_spec = BoundPartitionSpec::builder(schema.clone()) + let partition_spec = PartitionSpec::builder(schema.clone()) .with_spec_id(0) .add_unbound_field(UnboundPartitionField { name: "ts_day".to_string(), @@ -1323,6 +1326,7 @@ mod tests { .build() .unwrap(); + let default_partition_type = partition_spec.partition_type(&schema).unwrap(); let expected = TableMetadata { format_version: FormatVersion::V2, table_uuid: Uuid::parse_str("fb072c92-a02b-11e9-ae9c-1bb7bc9eca94").unwrap(), @@ -1331,10 +1335,8 @@ mod tests { last_column_id: 1, schemas: HashMap::from_iter(vec![(1, Arc::new(schema))]), current_schema_id: 1, - partition_specs: HashMap::from_iter(vec![( - 0, - partition_spec.clone().into_schemaless().into(), - )]), + partition_specs: HashMap::from_iter(vec![(0, partition_spec.clone().into())]), + default_partition_type, default_spec: partition_spec.into(), last_partition_id: 1000, default_sort_order_id: 0, @@ -1479,7 +1481,7 @@ mod tests { .unwrap(); let schema = Arc::new(schema); - let partition_spec = BoundPartitionSpec::builder(schema.clone()) + let partition_spec = PartitionSpec::builder(schema.clone()) .with_spec_id(0) .add_partition_field("vendor_id", "vendor_id", Transform::Identity) .unwrap() @@ -1500,6 +1502,7 @@ mod tests { .with_summary(Summary { operation: Operation::Append, additional_properties: HashMap::from_iter(vec![("spark.app.id".to_string(), "local-1662532784305".to_string()), ("added-data-files".to_string(), "4".to_string()), ("added-records".to_string(), "4".to_string()), ("added-files-size".to_string(), "6001".to_string())]) }) .build(); + let default_partition_type = partition_spec.partition_type(&schema).unwrap(); let expected = TableMetadata { format_version: FormatVersion::V1, table_uuid: Uuid::parse_str("df838b92-0b32-465d-a44e-d39936e538b7").unwrap(), @@ -1508,7 +1511,8 @@ mod tests { last_column_id: 5, schemas: HashMap::from_iter(vec![(0, schema)]), current_schema_id: 0, - partition_specs: HashMap::from_iter(vec![(0, partition_spec.clone().into_schemaless().into())]), + partition_specs: HashMap::from_iter(vec![(0, partition_spec.clone().into())]), + default_partition_type, default_spec: Arc::new(partition_spec), last_partition_id: 1000, default_sort_order_id: 0, @@ -1588,11 +1592,12 @@ mod tests { .build() .unwrap(); - let partition_spec = BoundPartitionSpec::builder(schema.clone()) + let partition_spec = PartitionSpec::builder(schema.clone()) .with_spec_id(0) .build() .unwrap(); + let default_partition_type = partition_spec.partition_type(&schema).unwrap(); let expected = TableMetadata { format_version: FormatVersion::V2, table_uuid: Uuid::parse_str("fb072c92-a02b-11e9-ae9c-1bb7bc9eca94").unwrap(), @@ -1601,10 +1606,8 @@ mod tests { last_column_id: 1, schemas: HashMap::from_iter(vec![(1, Arc::new(schema))]), current_schema_id: 1, - partition_specs: HashMap::from_iter(vec![( - 0, - partition_spec.clone().into_schemaless().into(), - )]), + partition_specs: HashMap::from_iter(vec![(0, partition_spec.clone().into())]), + default_partition_type, default_spec: partition_spec.into(), last_partition_id: 1000, default_sort_order_id: 0, @@ -1980,7 +1983,7 @@ mod tests { .build() .unwrap(); - let partition_spec = BoundPartitionSpec::builder(schema2.clone()) + let partition_spec = PartitionSpec::builder(schema2.clone()) .with_spec_id(0) .add_unbound_field(UnboundPartitionField { name: "x".to_string(), @@ -2033,6 +2036,7 @@ mod tests { }) .build(); + let default_partition_type = partition_spec.partition_type(&schema2).unwrap(); let expected = TableMetadata { format_version: FormatVersion::V2, table_uuid: Uuid::parse_str("9c12d441-03fe-4693-9a96-a0705ddf69c1").unwrap(), @@ -2041,11 +2045,9 @@ mod tests { last_column_id: 3, schemas: HashMap::from_iter(vec![(0, Arc::new(schema1)), (1, Arc::new(schema2))]), current_schema_id: 1, - partition_specs: HashMap::from_iter(vec![( - 0, - partition_spec.clone().into_schemaless().into(), - )]), + partition_specs: HashMap::from_iter(vec![(0, partition_spec.clone().into())]), default_spec: Arc::new(partition_spec), + default_partition_type, last_partition_id: 1000, default_sort_order_id: 3, sort_orders: HashMap::from_iter(vec![(3, sort_order.into())]), @@ -2106,7 +2108,7 @@ mod tests { .build() .unwrap(); - let partition_spec = BoundPartitionSpec::builder(schema.clone()) + let partition_spec = PartitionSpec::builder(schema.clone()) .with_spec_id(0) .add_unbound_field(UnboundPartitionField { name: "x".to_string(), @@ -2135,6 +2137,7 @@ mod tests { .build_unbound() .unwrap(); + let default_partition_type = partition_spec.partition_type(&schema).unwrap(); let expected = TableMetadata { format_version: FormatVersion::V2, table_uuid: Uuid::parse_str("9c12d441-03fe-4693-9a96-a0705ddf69c1").unwrap(), @@ -2143,10 +2146,8 @@ mod tests { last_column_id: 3, schemas: HashMap::from_iter(vec![(0, Arc::new(schema))]), current_schema_id: 0, - partition_specs: HashMap::from_iter(vec![( - 0, - partition_spec.clone().into_schemaless().into(), - )]), + partition_specs: HashMap::from_iter(vec![(0, partition_spec.clone().into())]), + default_partition_type, default_spec: Arc::new(partition_spec), last_partition_id: 1000, default_sort_order_id: 3, @@ -2189,7 +2190,7 @@ mod tests { .build() .unwrap(); - let partition_spec = BoundPartitionSpec::builder(schema.clone()) + let partition_spec = PartitionSpec::builder(schema.clone()) .with_spec_id(0) .add_unbound_field(UnboundPartitionField { name: "x".to_string(), @@ -2201,6 +2202,7 @@ mod tests { .build() .unwrap(); + let default_partition_type = partition_spec.partition_type(&schema).unwrap(); let expected = TableMetadata { format_version: FormatVersion::V1, table_uuid: Uuid::parse_str("d20125c8-7284-442c-9aea-15fee620737c").unwrap(), @@ -2209,11 +2211,9 @@ mod tests { last_column_id: 3, schemas: HashMap::from_iter(vec![(0, Arc::new(schema))]), current_schema_id: 0, - partition_specs: HashMap::from_iter(vec![( - 0, - partition_spec.clone().into_schemaless().into(), - )]), + partition_specs: HashMap::from_iter(vec![(0, partition_spec.clone().into())]), default_spec: Arc::new(partition_spec), + default_partition_type, last_partition_id: 0, default_sort_order_id: 0, // Sort order is added during deserialization for V2 compatibility @@ -2326,17 +2326,14 @@ mod tests { fn test_default_partition_spec() { let default_spec_id = 1234; let mut table_meta_data = get_test_table_metadata("TableMetadataV2Valid.json"); - let partition_spec = - BoundPartitionSpec::unpartition_spec(table_meta_data.current_schema().clone()); + let partition_spec = PartitionSpec::unpartition_spec(); table_meta_data.default_spec = partition_spec.clone().into(); table_meta_data .partition_specs - .insert(default_spec_id, Arc::new(partition_spec.into_schemaless())); + .insert(default_spec_id, Arc::new(partition_spec)); assert_eq!( - (*table_meta_data.default_partition_spec().clone()) - .clone() - .into_schemaless(), + (*table_meta_data.default_partition_spec().clone()).clone(), (*table_meta_data .partition_spec_by_id(default_spec_id) .unwrap() @@ -2393,11 +2390,10 @@ mod tests { HashMap::from([( 0, Arc::new( - BoundPartitionSpec::builder(table_metadata.schemas.get(&0).unwrap().clone()) + PartitionSpec::builder(table_metadata.schemas.get(&0).unwrap().clone()) .with_spec_id(0) .build() .unwrap() - .into_schemaless() ) )]) ); diff --git a/crates/iceberg/src/spec/table_metadata_builder.rs b/crates/iceberg/src/spec/table_metadata_builder.rs index 966abb156..d6ea2c91f 100644 --- a/crates/iceberg/src/spec/table_metadata_builder.rs +++ b/crates/iceberg/src/spec/table_metadata_builder.rs @@ -21,8 +21,8 @@ use std::sync::Arc; use uuid::Uuid; use super::{ - BoundPartitionSpec, FormatVersion, MetadataLog, PartitionSpecBuilder, Schema, SchemaRef, - Snapshot, SnapshotLog, SnapshotReference, SnapshotRetention, SortOrder, SortOrderRef, + FormatVersion, MetadataLog, PartitionSpec, PartitionSpecBuilder, Schema, SchemaRef, Snapshot, + SnapshotLog, SnapshotReference, SnapshotRetention, SortOrder, SortOrderRef, StructType, TableMetadata, UnboundPartitionSpec, DEFAULT_PARTITION_SPEC_ID, DEFAULT_SCHEMA_ID, MAIN_BRANCH, ONE_MINUTE_MS, PROPERTY_METADATA_PREVIOUS_VERSIONS_MAX, PROPERTY_METADATA_PREVIOUS_VERSIONS_MAX_DEFAULT, RESERVED_PROPERTIES, @@ -105,8 +105,9 @@ impl TableMetadataBuilder { // also unpartitioned. // The `default_spec` value is always replaced at the end of this method by he `add_default_partition_spec` // method. - BoundPartitionSpec::unpartition_spec(fresh_schema.clone()).with_spec_id(-1), + PartitionSpec::unpartition_spec().with_spec_id(-1), ), // Overwritten immediately by add_default_partition_spec + default_partition_type: StructType::new(vec![]), last_partition_id: UNPARTITIONED_LAST_ASSIGNED_ID, properties: HashMap::new(), current_snapshot_id: None, @@ -669,7 +670,7 @@ impl TableMetadataBuilder { .unwrap_or(UNPARTITIONED_LAST_ASSIGNED_ID); self.metadata .partition_specs - .insert(new_spec_id, Arc::new(spec.into())); + .insert(new_spec_id, Arc::new(spec)); self.changes .push(TableUpdate::AddSpec { spec: unbound_spec }); @@ -717,9 +718,10 @@ impl TableMetadataBuilder { ) })? .clone(); - let spec = - Arc::unwrap_or_clone(schemaless_spec).bind(self.get_current_schema()?.clone())?; + let spec = Arc::unwrap_or_clone(schemaless_spec); + let spec_type = spec.partition_type(self.get_current_schema()?)?; self.metadata.default_spec = Arc::new(spec); + self.metadata.default_partition_type = spec_type; if self.last_added_spec_id == Some(spec_id) { self.changes.push(TableUpdate::SetDefaultSpec { @@ -850,6 +852,8 @@ impl TableMetadataBuilder { .into_unbound() .bind(schema.clone())?, ); + self.metadata.default_partition_type = + self.metadata.default_spec.partition_type(&schema)?; SortOrder::builder() .with_fields(sort_order.fields) .build(&schema)?; @@ -978,7 +982,7 @@ impl TableMetadataBuilder { schema: Schema, spec: UnboundPartitionSpec, sort_order: SortOrder, - ) -> Result<(Schema, BoundPartitionSpec, SortOrder)> { + ) -> Result<(Schema, PartitionSpec, SortOrder)> { // Re-assign field ids and schema ids for a new table. let previous_id_to_name = schema.field_id_to_name_map().clone(); let fresh_schema = schema @@ -1084,15 +1088,11 @@ impl TableMetadataBuilder { } /// If a compatible spec already exists, use the same ID. Otherwise, use 1 more than the highest ID. - fn reuse_or_create_new_spec_id(&self, new_spec: &BoundPartitionSpec) -> i32 { + fn reuse_or_create_new_spec_id(&self, new_spec: &PartitionSpec) -> i32 { self.metadata .partition_specs .iter() - .find_map(|(id, old_spec)| { - new_spec - .is_compatible_with_schemaless(old_spec) - .then_some(*id) - }) + .find_map(|(id, old_spec)| new_spec.is_compatible_with(old_spec).then_some(*id)) .unwrap_or_else(|| { self.get_highest_spec_id() .map(|id| id + 1) @@ -1138,9 +1138,8 @@ impl From for TableMetadata { mod tests { use super::*; use crate::spec::{ - NestedField, NullOrder, Operation, PrimitiveType, Schema, SchemalessPartitionSpec, - SnapshotRetention, SortDirection, SortField, StructType, Summary, Transform, Type, - UnboundPartitionField, + NestedField, NullOrder, Operation, PartitionSpec, PrimitiveType, Schema, SnapshotRetention, + SortDirection, SortField, StructType, Summary, Transform, Type, UnboundPartitionField, }; const TEST_LOCATION: &str = "s3://bucket/test/location"; @@ -1250,7 +1249,7 @@ mod tests { let schema = Schema::builder().build().unwrap(); let metadata = TableMetadataBuilder::new( schema.clone(), - SchemalessPartitionSpec::unpartition_spec(), + PartitionSpec::unpartition_spec(), SortOrder::unsorted_order(), TEST_LOCATION.to_string(), FormatVersion::V2, @@ -1298,7 +1297,7 @@ mod tests { ]) .build() .unwrap(); - let spec = BoundPartitionSpec::builder(schema.clone()) + let spec = PartitionSpec::builder(schema.clone()) .with_spec_id(20) .add_partition_field("a", "a", Transform::Identity) .unwrap() @@ -1340,7 +1339,7 @@ mod tests { .build() .unwrap(); - let expected_spec = BoundPartitionSpec::builder(expected_schema.clone()) + let expected_spec = PartitionSpec::builder(expected_schema.clone()) .with_spec_id(0) .add_partition_field("a", "a", Transform::Identity) .unwrap() @@ -1413,7 +1412,7 @@ mod tests { TableUpdate::AddSpec { // Because this is a new tables, field-ids are assigned // partition_spec() has None set for field-id - spec: BoundPartitionSpec::builder(schema()) + spec: PartitionSpec::builder(schema()) .with_spec_id(0) .add_unbound_field(UnboundPartitionField { name: "y".to_string(), @@ -1445,7 +1444,7 @@ mod tests { let schema = Schema::builder().build().unwrap(); let changes = TableMetadataBuilder::new( schema.clone(), - SchemalessPartitionSpec::unpartition_spec().into_unbound(), + PartitionSpec::unpartition_spec().into_unbound(), SortOrder::unsorted_order(), TEST_LOCATION.to_string(), FormatVersion::V1, @@ -1468,7 +1467,7 @@ mod tests { TableUpdate::AddSpec { // Because this is a new tables, field-ids are assigned // partition_spec() has None set for field-id - spec: BoundPartitionSpec::builder(schema) + spec: PartitionSpec::builder(schema) .with_spec_id(0) .build() .unwrap() @@ -1515,7 +1514,7 @@ mod tests { // Spec id should be re-assigned let expected_change = added_spec.with_spec_id(1); - let expected_spec = BoundPartitionSpec::builder(schema()) + let expected_spec = PartitionSpec::builder(schema()) .with_spec_id(1) .add_unbound_field(UnboundPartitionField { name: "y".to_string(), @@ -1537,7 +1536,7 @@ mod tests { assert_eq!(build_result.changes.len(), 1); assert_eq!( build_result.metadata.partition_spec_by_id(1), - Some(&Arc::new(expected_spec.into_schemaless())) + Some(&Arc::new(expected_spec)) ); assert_eq!(build_result.metadata.default_spec.spec_id(), 0); assert_eq!(build_result.metadata.last_partition_id, 1001); @@ -1564,7 +1563,7 @@ mod tests { .build() .unwrap(); - let expected_spec = BoundPartitionSpec::builder(schema) + let expected_spec = PartitionSpec::builder(schema) .with_spec_id(1) .add_unbound_field(UnboundPartitionField { name: "y_bucket[2]".to_string(), diff --git a/crates/iceberg/src/transaction.rs b/crates/iceberg/src/transaction.rs index f58536a2c..cfd6a8381 100644 --- a/crates/iceberg/src/transaction.rs +++ b/crates/iceberg/src/transaction.rs @@ -356,11 +356,7 @@ impl<'a> SnapshotProduceAction<'a> { } Self::validate_partition_value( data_file.partition(), - self.tx - .table - .metadata() - .default_partition_spec() - .partition_type(), + self.tx.table.metadata().default_partition_type(), )?; } self.added_data_files.extend(data_files); diff --git a/crates/iceberg/src/writer/file_writer/location_generator.rs b/crates/iceberg/src/writer/file_writer/location_generator.rs index eb255fc67..abf0de4d4 100644 --- a/crates/iceberg/src/writer/file_writer/location_generator.rs +++ b/crates/iceberg/src/writer/file_writer/location_generator.rs @@ -132,7 +132,7 @@ pub(crate) mod test { use uuid::Uuid; use super::LocationGenerator; - use crate::spec::{BoundPartitionSpec, FormatVersion, TableMetadata}; + use crate::spec::{FormatVersion, PartitionSpec, StructType, TableMetadata}; use crate::writer::file_writer::location_generator::{ FileNameGenerator, WRITE_DATA_LOCATION, WRITE_FOLDER_STORAGE_LOCATION, }; @@ -156,7 +156,6 @@ pub(crate) mod test { #[test] fn test_default_location_generate() { - let schema = crate::spec::Schema::builder().build().unwrap(); let mut table_metadata = TableMetadata { format_version: FormatVersion::V2, table_uuid: Uuid::parse_str("fb072c92-a02b-11e9-ae9c-1bb7bc9eca94").unwrap(), @@ -166,7 +165,8 @@ pub(crate) mod test { schemas: HashMap::new(), current_schema_id: 1, partition_specs: HashMap::new(), - default_spec: BoundPartitionSpec::unpartition_spec(schema).into(), + default_spec: PartitionSpec::unpartition_spec().into(), + default_partition_type: StructType::new(vec![]), last_partition_id: 1000, default_sort_order_id: 0, sort_orders: HashMap::from_iter(vec![]), From 95732a13f7529bcdc1f79f2ccdaef4b2c3d6b7a2 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Mon, 16 Dec 2024 11:34:27 +0800 Subject: [PATCH 09/23] chore(deps): Bump crate-ci/typos from 1.28.2 to 1.28.3 (#805) Bumps [crate-ci/typos](https://github.com/crate-ci/typos) from 1.28.2 to 1.28.3. - [Release notes](https://github.com/crate-ci/typos/releases) - [Changelog](https://github.com/crate-ci/typos/blob/master/CHANGELOG.md) - [Commits](https://github.com/crate-ci/typos/compare/v1.28.2...v1.28.3) --- updated-dependencies: - dependency-name: crate-ci/typos dependency-type: direct:production update-type: version-update:semver-patch ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> Co-authored-by: Renjie Liu --- .github/workflows/ci_typos.yml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/.github/workflows/ci_typos.yml b/.github/workflows/ci_typos.yml index d2424c999..e630b4056 100644 --- a/.github/workflows/ci_typos.yml +++ b/.github/workflows/ci_typos.yml @@ -42,4 +42,4 @@ jobs: steps: - uses: actions/checkout@v4 - name: Check typos - uses: crate-ci/typos@v1.28.2 + uses: crate-ci/typos@v1.28.3 From 97f8a792b66afbce805b1c5a4733d16ce88e602a Mon Sep 17 00:00:00 2001 From: Jonathan Chen Date: Mon, 16 Dec 2024 01:26:51 -0500 Subject: [PATCH 10/23] feat: add `DataFileWriter` tests for schema and partition (#768) * feat: add writer test for schema check and * fixes --- .../writer/base_writer/data_file_writer.rs | 127 ++++++++++++++++-- 1 file changed, 118 insertions(+), 9 deletions(-) diff --git a/crates/iceberg/src/writer/base_writer/data_file_writer.rs b/crates/iceberg/src/writer/base_writer/data_file_writer.rs index 6f9c0a892..940aa1584 100644 --- a/crates/iceberg/src/writer/base_writer/data_file_writer.rs +++ b/crates/iceberg/src/writer/base_writer/data_file_writer.rs @@ -100,16 +100,21 @@ impl CurrentFileStatus for DataFileWriter { mod test { use std::sync::Arc; + use arrow_array::{Int32Array, StringArray}; + use arrow_schema::{DataType, Field}; + use parquet::arrow::arrow_reader::{ArrowReaderMetadata, ArrowReaderOptions}; use parquet::file::properties::WriterProperties; use tempfile::TempDir; use crate::io::FileIOBuilder; - use crate::spec::{DataContentType, DataFileFormat, Schema, Struct}; + use crate::spec::{ + DataContentType, DataFileFormat, Literal, NestedField, PrimitiveType, Schema, Struct, Type, + }; use crate::writer::base_writer::data_file_writer::DataFileWriterBuilder; use crate::writer::file_writer::location_generator::test::MockLocationGenerator; use crate::writer::file_writer::location_generator::DefaultFileNameGenerator; use crate::writer::file_writer::ParquetWriterBuilder; - use crate::writer::{IcebergWriter, IcebergWriterBuilder}; + use crate::writer::{IcebergWriter, IcebergWriterBuilder, RecordBatch}; use crate::Result; #[tokio::test] @@ -121,20 +126,124 @@ mod test { let file_name_gen = DefaultFileNameGenerator::new("test".to_string(), None, DataFileFormat::Parquet); + let schema = Schema::builder() + .with_schema_id(3) + .with_fields(vec![ + NestedField::required(3, "foo", Type::Primitive(PrimitiveType::Int)).into(), + NestedField::required(4, "bar", Type::Primitive(PrimitiveType::String)).into(), + ]) + .build()?; + let pw = ParquetWriterBuilder::new( WriterProperties::builder().build(), - Arc::new(Schema::builder().build().unwrap()), + Arc::new(schema), + file_io.clone(), + location_gen, + file_name_gen, + ); + + let mut data_file_writer = DataFileWriterBuilder::new(pw, None).build().await.unwrap(); + + let data_files = data_file_writer.close().await.unwrap(); + assert_eq!(data_files.len(), 1); + + let data_file = &data_files[0]; + assert_eq!(data_file.file_format, DataFileFormat::Parquet); + assert_eq!(data_file.content, DataContentType::Data); + assert_eq!(data_file.partition, Struct::empty()); + + let input_file = file_io.new_input(data_file.file_path.clone())?; + let input_content = input_file.read().await?; + + let parquet_reader = + ArrowReaderMetadata::load(&input_content, ArrowReaderOptions::default()) + .expect("Failed to load Parquet metadata"); + + let field_ids: Vec = parquet_reader + .parquet_schema() + .columns() + .iter() + .map(|col| col.self_type().get_basic_info().id()) + .collect(); + + assert_eq!(field_ids, vec![3, 4]); + Ok(()) + } + + #[tokio::test] + async fn test_parquet_writer_with_partition() -> Result<()> { + let temp_dir = TempDir::new().unwrap(); + let file_io = FileIOBuilder::new_fs_io().build().unwrap(); + let location_gen = + MockLocationGenerator::new(temp_dir.path().to_str().unwrap().to_string()); + let file_name_gen = DefaultFileNameGenerator::new( + "test_partitioned".to_string(), + None, + DataFileFormat::Parquet, + ); + + let schema = Schema::builder() + .with_schema_id(5) + .with_fields(vec![ + NestedField::required(5, "id", Type::Primitive(PrimitiveType::Int)).into(), + NestedField::required(6, "name", Type::Primitive(PrimitiveType::String)).into(), + ]) + .build()?; + + let partition_value = Struct::from_iter([Some(Literal::int(1))]); + + let parquet_writer_builder = ParquetWriterBuilder::new( + WriterProperties::builder().build(), + Arc::new(schema.clone()), file_io.clone(), location_gen, file_name_gen, ); - let mut data_file_writer = DataFileWriterBuilder::new(pw, None).build().await?; - let data_file = data_file_writer.close().await.unwrap(); - assert_eq!(data_file.len(), 1); - assert_eq!(data_file[0].file_format, DataFileFormat::Parquet); - assert_eq!(data_file[0].content, DataContentType::Data); - assert_eq!(data_file[0].partition, Struct::empty()); + let mut data_file_writer = + DataFileWriterBuilder::new(parquet_writer_builder, Some(partition_value.clone())) + .build() + .await?; + + let arrow_schema = arrow_schema::Schema::new(vec![ + Field::new("id", DataType::Int32, false), + Field::new("name", DataType::Utf8, false), + ]); + let batch = RecordBatch::try_new(Arc::new(arrow_schema.clone()), vec![ + Arc::new(Int32Array::from(vec![1, 2, 3])), + Arc::new(StringArray::from(vec!["Alice", "Bob", "Charlie"])), + ])?; + data_file_writer.write(batch).await?; + + let data_files = data_file_writer.close().await.unwrap(); + assert_eq!(data_files.len(), 1); + + let data_file = &data_files[0]; + assert_eq!(data_file.file_format, DataFileFormat::Parquet); + assert_eq!(data_file.content, DataContentType::Data); + assert_eq!(data_file.partition, partition_value); + + let input_file = file_io.new_input(data_file.file_path.clone())?; + let input_content = input_file.read().await?; + + let parquet_reader = + ArrowReaderMetadata::load(&input_content, ArrowReaderOptions::default())?; + + let field_ids: Vec = parquet_reader + .parquet_schema() + .columns() + .iter() + .map(|col| col.self_type().get_basic_info().id()) + .collect(); + assert_eq!(field_ids, vec![5, 6]); + + let field_names: Vec<&str> = parquet_reader + .parquet_schema() + .columns() + .iter() + .map(|col| col.name()) + .collect(); + assert_eq!(field_names, vec!["id", "name"]); Ok(()) } From dff4e5a1c6a45d86d5f3ee1173763b8508b7bd41 Mon Sep 17 00:00:00 2001 From: ZENOTME <43447882+ZENOTME@users.noreply.github.com> Date: Mon, 16 Dec 2024 19:43:50 +0800 Subject: [PATCH 11/23] fix: day transform compute (#796) * fix day transform compute * fix python test --------- Co-authored-by: ZENOTME --- bindings/python/tests/test_transform.py | 2 +- .../src/expr/visitors/inclusive_projection.rs | 2 +- crates/iceberg/src/spec/transform.rs | 20 +++- crates/iceberg/src/transform/temporal.rs | 108 +++++++++--------- 4 files changed, 74 insertions(+), 58 deletions(-) diff --git a/bindings/python/tests/test_transform.py b/bindings/python/tests/test_transform.py index 4180b6902..9df1f8cbd 100644 --- a/bindings/python/tests/test_transform.py +++ b/bindings/python/tests/test_transform.py @@ -73,7 +73,7 @@ def test_month_transform(): def test_day_transform(): arr = pa.array([date(1970, 1, 1), date(2000, 4, 1)]) result = transform.day(arr) - expected = pa.array([0, 11048], type=pa.int32()) + expected = pa.array([0, 11048], type=pa.date32()) assert result == expected diff --git a/crates/iceberg/src/expr/visitors/inclusive_projection.rs b/crates/iceberg/src/expr/visitors/inclusive_projection.rs index d075fdd81..3e49a7a10 100644 --- a/crates/iceberg/src/expr/visitors/inclusive_projection.rs +++ b/crates/iceberg/src/expr/visitors/inclusive_projection.rs @@ -429,7 +429,7 @@ mod tests { let mut inclusive_projection = InclusiveProjection::new(arc_partition_spec); let result = inclusive_projection.project(&bound_predicate).unwrap(); - let expected = "day <= 19722".to_string(); + let expected = "day <= 2023-12-31".to_string(); assert_eq!(result.to_string(), expected); } diff --git a/crates/iceberg/src/spec/transform.rs b/crates/iceberg/src/spec/transform.rs index 29401f174..8e9783a6c 100644 --- a/crates/iceberg/src/spec/transform.rs +++ b/crates/iceberg/src/spec/transform.rs @@ -598,14 +598,30 @@ impl Transform { | PredicateOperator::LessThanOrEq | PredicateOperator::In => { if v < 0 { - return Some(AdjustedProjection::Single(Datum::int(v + 1))); + // # TODO + // An ugly hack to fix. Refine the increment and decrement logic later. + match self { + Transform::Day => { + return Some(AdjustedProjection::Single(Datum::date(v + 1))) + } + _ => { + return Some(AdjustedProjection::Single(Datum::int(v + 1))); + } + } }; } PredicateOperator::Eq => { if v < 0 { let new_set = FnvHashSet::from_iter(vec![ transformed.to_owned(), - Datum::int(v + 1), + // # TODO + // An ugly hack to fix. Refine the increment and decrement logic later. + { + match self { + Transform::Day => Datum::date(v + 1), + _ => Datum::int(v + 1), + } + }, ]); return Some(AdjustedProjection::Set(new_set)); } diff --git a/crates/iceberg/src/transform/temporal.rs b/crates/iceberg/src/transform/temporal.rs index c5721dada..5dd4ab063 100644 --- a/crates/iceberg/src/transform/temporal.rs +++ b/crates/iceberg/src/transform/temporal.rs @@ -272,7 +272,7 @@ impl Day { impl TransformFunction for Day { fn transform(&self, input: ArrayRef) -> Result { - let res: Int32Array = match input.data_type() { + let res: Date32Array = match input.data_type() { DataType::Timestamp(TimeUnit::Microsecond, _) => input .as_any() .downcast_ref::() @@ -324,7 +324,7 @@ impl TransformFunction for Day { )) } }; - Ok(Some(Datum::int(val))) + Ok(Some(Datum::date(val))) } } @@ -1214,7 +1214,7 @@ mod test { PredicateOperator::LessThan, Datum::timestamp_from_str(value)?, ), - Some("name <= 0"), + Some("name <= 1970-01-01"), )?; fixture.assert_projection( @@ -1222,7 +1222,7 @@ mod test { PredicateOperator::LessThanOrEq, Datum::timestamp_from_str(value)?, ), - Some("name <= 0"), + Some("name <= 1970-01-01"), )?; fixture.assert_projection( @@ -1230,7 +1230,7 @@ mod test { PredicateOperator::GreaterThan, Datum::timestamp_from_str(value)?, ), - Some("name >= 0"), + Some("name >= 1970-01-01"), )?; fixture.assert_projection( @@ -1238,12 +1238,12 @@ mod test { PredicateOperator::GreaterThanOrEq, Datum::timestamp_from_str(value)?, ), - Some("name >= -1"), + Some("name >= 1969-12-31"), )?; fixture.assert_projection( &fixture.binary_predicate(PredicateOperator::Eq, Datum::timestamp_from_str(value)?), - Some("name IN (-1, 0)"), + Some("name IN (1969-12-31, 1970-01-01)"), )?; fixture.assert_projection( @@ -1256,7 +1256,7 @@ mod test { Datum::timestamp_from_str(value)?, Datum::timestamp_from_str(another)?, ]), - Some("name IN (0, -1)"), + Some("name IN (1970-01-01, 1969-12-31)"), )?; fixture.assert_projection( @@ -1288,7 +1288,7 @@ mod test { PredicateOperator::LessThan, Datum::timestamp_from_str(value)?, ), - Some("name <= 17501"), + Some("name <= 2017-12-01"), )?; fixture.assert_projection( @@ -1296,7 +1296,7 @@ mod test { PredicateOperator::LessThanOrEq, Datum::timestamp_from_str(value)?, ), - Some("name <= 17501"), + Some("name <= 2017-12-01"), )?; fixture.assert_projection( @@ -1304,7 +1304,7 @@ mod test { PredicateOperator::GreaterThan, Datum::timestamp_from_str(value)?, ), - Some("name >= 17502"), + Some("name >= 2017-12-02"), )?; fixture.assert_projection( @@ -1312,12 +1312,12 @@ mod test { PredicateOperator::GreaterThanOrEq, Datum::timestamp_from_str(value)?, ), - Some("name >= 17501"), + Some("name >= 2017-12-01"), )?; fixture.assert_projection( &fixture.binary_predicate(PredicateOperator::Eq, Datum::timestamp_from_str(value)?), - Some("name = 17501"), + Some("name = 2017-12-01"), )?; fixture.assert_projection( @@ -1330,7 +1330,7 @@ mod test { Datum::timestamp_from_str(value)?, Datum::timestamp_from_str(another)?, ]), - Some("name IN (17501, 17502)"), + Some("name IN (2017-12-02, 2017-12-01)"), )?; fixture.assert_projection( @@ -1362,7 +1362,7 @@ mod test { PredicateOperator::LessThan, Datum::timestamp_from_str(value)?, ), - Some("name <= -365"), + Some("name <= 1969-01-01"), )?; fixture.assert_projection( @@ -1370,7 +1370,7 @@ mod test { PredicateOperator::LessThanOrEq, Datum::timestamp_from_str(value)?, ), - Some("name <= -364"), + Some("name <= 1969-01-02"), )?; fixture.assert_projection( @@ -1378,7 +1378,7 @@ mod test { PredicateOperator::GreaterThan, Datum::timestamp_from_str(value)?, ), - Some("name >= -365"), + Some("name >= 1969-01-01"), )?; fixture.assert_projection( @@ -1386,12 +1386,12 @@ mod test { PredicateOperator::GreaterThanOrEq, Datum::timestamp_from_str(value)?, ), - Some("name >= -365"), + Some("name >= 1969-01-01"), )?; fixture.assert_projection( &fixture.binary_predicate(PredicateOperator::Eq, Datum::timestamp_from_str(value)?), - Some("name IN (-364, -365)"), + Some("name IN (1969-01-01, 1969-01-02)"), )?; fixture.assert_projection( @@ -1404,7 +1404,7 @@ mod test { Datum::timestamp_from_str(value)?, Datum::timestamp_from_str(another)?, ]), - Some("name IN (-363, -365, -364)"), + Some("name IN (1969-01-02, 1969-01-01, 1969-01-03)"), )?; fixture.assert_projection( @@ -1436,7 +1436,7 @@ mod test { PredicateOperator::LessThan, Datum::timestamp_from_str(value)?, ), - Some("name <= 17500"), + Some("name <= 2017-11-30"), )?; fixture.assert_projection( @@ -1444,7 +1444,7 @@ mod test { PredicateOperator::LessThanOrEq, Datum::timestamp_from_str(value)?, ), - Some("name <= 17501"), + Some("name <= 2017-12-01"), )?; fixture.assert_projection( @@ -1452,7 +1452,7 @@ mod test { PredicateOperator::GreaterThan, Datum::timestamp_from_str(value)?, ), - Some("name >= 17501"), + Some("name >= 2017-12-01"), )?; fixture.assert_projection( @@ -1460,12 +1460,12 @@ mod test { PredicateOperator::GreaterThanOrEq, Datum::timestamp_from_str(value)?, ), - Some("name >= 17501"), + Some("name >= 2017-12-01"), )?; fixture.assert_projection( &fixture.binary_predicate(PredicateOperator::Eq, Datum::timestamp_from_str(value)?), - Some("name = 17501"), + Some("name = 2017-12-01"), )?; fixture.assert_projection( @@ -1478,7 +1478,7 @@ mod test { Datum::timestamp_from_str(value)?, Datum::timestamp_from_str(another)?, ]), - Some("name IN (17501, 17502)"), + Some("name IN (2017-12-02, 2017-12-01)"), )?; fixture.assert_projection( @@ -1510,7 +1510,7 @@ mod test { PredicateOperator::LessThan, Datum::timestamp_from_str(value)?, ), - Some("name <= 0"), + Some("name <= 1970-01-01"), )?; fixture.assert_projection( @@ -1518,7 +1518,7 @@ mod test { PredicateOperator::LessThanOrEq, Datum::timestamp_from_str(value)?, ), - Some("name <= 0"), + Some("name <= 1970-01-01"), )?; fixture.assert_projection( @@ -1526,7 +1526,7 @@ mod test { PredicateOperator::GreaterThan, Datum::timestamp_from_str(value)?, ), - Some("name >= 0"), + Some("name >= 1970-01-01"), )?; fixture.assert_projection( @@ -1534,12 +1534,12 @@ mod test { PredicateOperator::GreaterThanOrEq, Datum::timestamp_from_str(value)?, ), - Some("name >= 0"), + Some("name >= 1970-01-01"), )?; fixture.assert_projection( &fixture.binary_predicate(PredicateOperator::Eq, Datum::timestamp_from_str(value)?), - Some("name = 0"), + Some("name = 1970-01-01"), )?; fixture.assert_projection( @@ -1552,7 +1552,7 @@ mod test { Datum::timestamp_from_str(value)?, Datum::timestamp_from_str(another)?, ]), - Some("name IN (1, 0)"), + Some("name IN (1970-01-01, 1970-01-02)"), )?; fixture.assert_projection( @@ -1581,7 +1581,7 @@ mod test { fixture.assert_projection( &fixture.binary_predicate(PredicateOperator::LessThan, Datum::date_from_str(value)?), - Some("name <= -3"), + Some("name <= 1969-12-29"), )?; fixture.assert_projection( @@ -1589,12 +1589,12 @@ mod test { PredicateOperator::LessThanOrEq, Datum::date_from_str(value)?, ), - Some("name <= -2"), + Some("name <= 1969-12-30"), )?; fixture.assert_projection( &fixture.binary_predicate(PredicateOperator::GreaterThan, Datum::date_from_str(value)?), - Some("name >= -1"), + Some("name >= 1969-12-31"), )?; fixture.assert_projection( @@ -1602,12 +1602,12 @@ mod test { PredicateOperator::GreaterThanOrEq, Datum::date_from_str(value)?, ), - Some("name >= -2"), + Some("name >= 1969-12-30"), )?; fixture.assert_projection( &fixture.binary_predicate(PredicateOperator::Eq, Datum::date_from_str(value)?), - Some("name = -2"), + Some("name = 1969-12-30"), )?; fixture.assert_projection( @@ -1620,7 +1620,7 @@ mod test { Datum::date_from_str(value)?, Datum::date_from_str(another)?, ]), - Some("name IN (-2, -4)"), + Some("name IN (1969-12-28, 1969-12-30)"), )?; fixture.assert_projection( @@ -1649,7 +1649,7 @@ mod test { fixture.assert_projection( &fixture.binary_predicate(PredicateOperator::LessThan, Datum::date_from_str(value)?), - Some("name <= 17166"), + Some("name <= 2016-12-31"), )?; fixture.assert_projection( @@ -1657,12 +1657,12 @@ mod test { PredicateOperator::LessThanOrEq, Datum::date_from_str(value)?, ), - Some("name <= 17167"), + Some("name <= 2017-01-01"), )?; fixture.assert_projection( &fixture.binary_predicate(PredicateOperator::GreaterThan, Datum::date_from_str(value)?), - Some("name >= 17168"), + Some("name >= 2017-01-02"), )?; fixture.assert_projection( @@ -1670,12 +1670,12 @@ mod test { PredicateOperator::GreaterThanOrEq, Datum::date_from_str(value)?, ), - Some("name >= 17167"), + Some("name >= 2017-01-01"), )?; fixture.assert_projection( &fixture.binary_predicate(PredicateOperator::Eq, Datum::date_from_str(value)?), - Some("name = 17167"), + Some("name = 2017-01-01"), )?; fixture.assert_projection( @@ -1688,7 +1688,7 @@ mod test { Datum::date_from_str(value)?, Datum::date_from_str(another)?, ]), - Some("name IN (17531, 17167)"), + Some("name IN (2017-01-01, 2017-12-31)"), )?; fixture.assert_projection( @@ -2629,7 +2629,7 @@ mod test { .collect::>(), )); let res = day.transform(date_array).unwrap(); - let res = res.as_any().downcast_ref::().unwrap(); + let res = res.as_any().downcast_ref::().unwrap(); assert_eq!(res.len(), 5); assert_eq!(res.value(0), expect_day[0]); assert_eq!(res.value(1), expect_day[1]); @@ -2668,7 +2668,7 @@ mod test { .collect::>(), )); let res = day.transform(date_array).unwrap(); - let res = res.as_any().downcast_ref::().unwrap(); + let res = res.as_any().downcast_ref::().unwrap(); assert_eq!(res.len(), 5); assert_eq!(res.value(0), expect_day[0]); assert_eq!(res.value(1), expect_day[1]); @@ -2681,18 +2681,18 @@ mod test { fn test_transform_days_literal() { let day = Box::new(super::Day) as BoxedTransformFunction; // Test Date32 - test_date(18628, &day, Datum::int(18628)); - test_date(-31, &day, Datum::int(-31)); + test_date(18628, &day, Datum::date(18628)); + test_date(-31, &day, Datum::date(-31)); // Test TimestampMicrosecond - test_timestamp_and_tz_transform_using_i64(1512151975038194, &day, Datum::int(17501)); - test_timestamp_and_tz_transform_using_i64(-115200000000, &day, Datum::int(-2)); - test_timestamp_and_tz_transform("2017-12-01 10:30:42.123", &day, Datum::int(17501)); + test_timestamp_and_tz_transform_using_i64(1512151975038194, &day, Datum::date(17501)); + test_timestamp_and_tz_transform_using_i64(-115200000000, &day, Datum::date(-2)); + test_timestamp_and_tz_transform("2017-12-01 10:30:42.123", &day, Datum::date(17501)); // Test TimestampNanosecond - test_timestamp_ns_and_tz_transform_using_i64(1512151975038194, &day, Datum::int(17)); - test_timestamp_ns_and_tz_transform_using_i64(-115200000000, &day, Datum::int(-1)); - test_timestamp_ns_and_tz_transform("2017-12-01 10:30:42.123", &day, Datum::int(17501)); + test_timestamp_ns_and_tz_transform_using_i64(1512151975038194, &day, Datum::date(17)); + test_timestamp_ns_and_tz_transform_using_i64(-115200000000, &day, Datum::date(-1)); + test_timestamp_ns_and_tz_transform("2017-12-01 10:30:42.123", &day, Datum::date(17501)); } #[test] From f00d89b92af3e20bc22e3af4661a087a61902276 Mon Sep 17 00:00:00 2001 From: Christian Date: Mon, 16 Dec 2024 16:48:36 +0100 Subject: [PATCH 12/23] feat: TableMetadata Statistic Files (#799) * Statistics * License header, rename statistics module * TableUpdate RemoveStatistics * fix iter names --- crates/iceberg/src/catalog/mod.rs | 199 ++++++++- crates/iceberg/src/spec/mod.rs | 2 + crates/iceberg/src/spec/statistic_file.rs | 196 +++++++++ crates/iceberg/src/spec/table_metadata.rs | 382 +++++++++++++++++- .../src/spec/table_metadata_builder.rs | 162 +++++++- .../writer/file_writer/location_generator.rs | 2 + 6 files changed, 919 insertions(+), 24 deletions(-) create mode 100644 crates/iceberg/src/spec/statistic_file.rs diff --git a/crates/iceberg/src/catalog/mod.rs b/crates/iceberg/src/catalog/mod.rs index b897d1574..deb4d2f34 100644 --- a/crates/iceberg/src/catalog/mod.rs +++ b/crates/iceberg/src/catalog/mod.rs @@ -29,9 +29,9 @@ use typed_builder::TypedBuilder; use uuid::Uuid; use crate::spec::{ - FormatVersion, Schema, SchemaId, Snapshot, SnapshotReference, SortOrder, TableMetadata, - TableMetadataBuilder, UnboundPartitionSpec, ViewFormatVersion, ViewRepresentations, - ViewVersion, + FormatVersion, PartitionStatisticsFile, Schema, SchemaId, Snapshot, SnapshotReference, + SortOrder, StatisticsFile, TableMetadata, TableMetadataBuilder, UnboundPartitionSpec, + ViewFormatVersion, ViewRepresentations, ViewVersion, }; use crate::table::Table; use crate::{Error, ErrorKind, Result}; @@ -446,6 +446,30 @@ pub enum TableUpdate { /// Properties to remove removals: Vec, }, + /// Set statistics for a snapshot + #[serde(with = "_serde_set_statistics")] + SetStatistics { + /// File containing the statistics + statistics: StatisticsFile, + }, + /// Remove statistics for a snapshot + #[serde(rename_all = "kebab-case")] + RemoveStatistics { + /// Snapshot id to remove statistics for. + snapshot_id: i64, + }, + /// Set partition statistics for a snapshot + #[serde(rename_all = "kebab-case")] + SetPartitionStatistics { + /// File containing the partition statistics + partition_statistics: PartitionStatisticsFile, + }, + /// Remove partition statistics for a snapshot + #[serde(rename_all = "kebab-case")] + RemovePartitionStatistics { + /// Snapshot id to remove partition statistics for. + snapshot_id: i64, + }, } impl TableUpdate { @@ -476,6 +500,16 @@ impl TableUpdate { TableUpdate::UpgradeFormatVersion { format_version } => { builder.upgrade_format_version(format_version) } + TableUpdate::SetStatistics { statistics } => Ok(builder.set_statistics(statistics)), + TableUpdate::RemoveStatistics { snapshot_id } => { + Ok(builder.remove_statistics(snapshot_id)) + } + TableUpdate::SetPartitionStatistics { + partition_statistics, + } => Ok(builder.set_partition_statistics(partition_statistics)), + TableUpdate::RemovePartitionStatistics { snapshot_id } => { + Ok(builder.remove_partition_statistics(snapshot_id)) + } } } } @@ -756,6 +790,53 @@ pub enum ViewUpdate { }, } +mod _serde_set_statistics { + // The rest spec requires an additional field `snapshot-id` + // that is redundant with the `snapshot_id` field in the statistics file. + 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) + } +} + #[cfg(test)] mod tests { use std::collections::HashMap; @@ -767,11 +848,11 @@ mod tests { use super::ViewUpdate; use crate::spec::{ - FormatVersion, NestedField, NullOrder, Operation, PrimitiveType, Schema, Snapshot, - SnapshotReference, SnapshotRetention, SortDirection, SortField, SortOrder, - SqlViewRepresentation, Summary, TableMetadata, TableMetadataBuilder, Transform, Type, - UnboundPartitionSpec, ViewFormatVersion, ViewRepresentation, ViewRepresentations, - ViewVersion, MAIN_BRANCH, + BlobMetadata, FormatVersion, NestedField, NullOrder, Operation, PartitionStatisticsFile, + PrimitiveType, Schema, Snapshot, SnapshotReference, SnapshotRetention, SortDirection, + SortField, SortOrder, SqlViewRepresentation, StatisticsFile, Summary, TableMetadata, + TableMetadataBuilder, Transform, Type, UnboundPartitionSpec, ViewFormatVersion, + ViewRepresentation, ViewRepresentations, ViewVersion, MAIN_BRANCH, }; use crate::{NamespaceIdent, TableCreation, TableIdent, TableRequirement, TableUpdate}; @@ -1836,4 +1917,106 @@ mod tests { ViewUpdate::SetCurrentViewVersion { view_version_id: 1 }, ); } + + #[test] + fn test_set_statistics_file() { + test_serde_json( + r#" + { + "action": "set-statistics", + "snapshot-id": 1940541653261589030, + "statistics": { + "snapshot-id": 1940541653261589030, + "statistics-path": "s3://bucket/warehouse/stats.puffin", + "file-size-in-bytes": 124, + "file-footer-size-in-bytes": 27, + "blob-metadata": [ + { + "type": "boring-type", + "snapshot-id": 1940541653261589030, + "sequence-number": 2, + "fields": [ + 1 + ], + "properties": { + "prop-key": "prop-value" + } + } + ] + } + } + "#, + TableUpdate::SetStatistics { + statistics: StatisticsFile { + snapshot_id: 1940541653261589030, + statistics_path: "s3://bucket/warehouse/stats.puffin".to_string(), + file_size_in_bytes: 124, + file_footer_size_in_bytes: 27, + key_metadata: None, + blob_metadata: vec![BlobMetadata { + r#type: "boring-type".to_string(), + snapshot_id: 1940541653261589030, + sequence_number: 2, + fields: vec![1], + properties: vec![("prop-key".to_string(), "prop-value".to_string())] + .into_iter() + .collect(), + }], + }, + }, + ); + } + + #[test] + fn test_remove_statistics_file() { + test_serde_json( + r#" + { + "action": "remove-statistics", + "snapshot-id": 1940541653261589030 + } + "#, + TableUpdate::RemoveStatistics { + snapshot_id: 1940541653261589030, + }, + ); + } + + #[test] + fn test_set_partition_statistics_file() { + test_serde_json( + r#" + { + "action": "set-partition-statistics", + "partition-statistics": { + "snapshot-id": 1940541653261589030, + "statistics-path": "s3://bucket/warehouse/stats1.parquet", + "file-size-in-bytes": 43 + } + } + "#, + TableUpdate::SetPartitionStatistics { + partition_statistics: PartitionStatisticsFile { + snapshot_id: 1940541653261589030, + statistics_path: "s3://bucket/warehouse/stats1.parquet".to_string(), + file_size_in_bytes: 43, + }, + }, + ) + } + + #[test] + fn test_remove_partition_statistics_file() { + test_serde_json( + r#" + { + "action": "remove-partition-statistics", + "snapshot-id": 1940541653261589030 + } + "#, + TableUpdate::RemovePartitionStatistics { + snapshot_id: 1940541653261589030, + }, + ) + } } diff --git a/crates/iceberg/src/spec/mod.rs b/crates/iceberg/src/spec/mod.rs index b0c53a6d6..b3f13b8d9 100644 --- a/crates/iceberg/src/spec/mod.rs +++ b/crates/iceberg/src/spec/mod.rs @@ -25,6 +25,7 @@ mod partition; mod schema; mod snapshot; mod sort; +mod statistic_file; mod table_metadata; mod table_metadata_builder; mod transform; @@ -39,6 +40,7 @@ pub use partition::*; pub use schema::*; pub use snapshot::*; pub use sort::*; +pub use statistic_file::*; pub use table_metadata::*; pub use transform::*; pub use values::*; diff --git a/crates/iceberg/src/spec/statistic_file.rs b/crates/iceberg/src/spec/statistic_file.rs new file mode 100644 index 000000000..4d806f6e4 --- /dev/null +++ b/crates/iceberg/src/spec/statistic_file.rs @@ -0,0 +1,196 @@ +// 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. + +//! Statistic Files for TableMetadata + +use std::collections::HashMap; + +use serde::{Deserialize, Serialize}; + +#[derive(Clone, Debug, PartialEq, Eq, Serialize, Deserialize)] +#[serde(rename_all = "kebab-case")] +/// Represents a statistics file +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(Clone, Debug, PartialEq, Eq, Serialize, Deserialize)] +#[serde(rename_all = "kebab-case")] +/// Represents a blob of metadata, which is a part of a statistics file +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 = "HashMap::is_empty")] + pub properties: HashMap, +} + +#[derive(Clone, Debug, PartialEq, Eq, Serialize, Deserialize)] +#[serde(rename_all = "kebab-case")] +/// Statistics file for a partition +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, +} + +#[cfg(test)] +mod test { + use std::fmt::Debug; + + use serde::de::DeserializeOwned; + use serde_json::json; + + use super::*; + + fn test_serde_json( + json: serde_json::Value, + expected: T, + ) { + let json_str = json.to_string(); + let actual: T = serde_json::from_str(&json_str).expect("Failed to parse from json"); + assert_eq!(actual, expected, "Parsed value is not equal to expected"); + + let restored: T = serde_json::from_str( + &serde_json::to_string(&actual).expect("Failed to serialize to json"), + ) + .expect("Failed to parse from serialized json"); + + assert_eq!( + restored, expected, + "Parsed restored value is not equal to expected" + ); + } + + #[test] + fn test_blob_metadata_serde() { + test_serde_json( + json!({ + "type": "boring-type", + "snapshot-id": 1940541653261589030i64, + "sequence-number": 2, + "fields": [ + 1 + ], + "properties": { + "prop-key": "prop-value" + } + }), + BlobMetadata { + r#type: "boring-type".to_string(), + snapshot_id: 1940541653261589030, + sequence_number: 2, + fields: vec![1], + properties: vec![("prop-key".to_string(), "prop-value".to_string())] + .into_iter() + .collect(), + }, + ); + } + + #[test] + fn test_blob_metadata_serde_no_properties() { + test_serde_json( + json!({ + "type": "boring-type", + "snapshot-id": 1940541653261589030i64, + "sequence-number": 2, + "fields": [ + 1 + ] + }), + BlobMetadata { + r#type: "boring-type".to_string(), + snapshot_id: 1940541653261589030, + sequence_number: 2, + fields: vec![1], + properties: HashMap::new(), + }, + ); + } + + #[test] + fn test_statistics_file_serde() { + test_serde_json( + json!({ + "snapshot-id": 3055729675574597004i64, + "statistics-path": "s3://a/b/stats.puffin", + "file-size-in-bytes": 413, + "file-footer-size-in-bytes": 42, + "blob-metadata": [ + { + "type": "ndv", + "snapshot-id": 3055729675574597004i64, + "sequence-number": 1, + "fields": [1] + } + ] + }), + StatisticsFile { + snapshot_id: 3055729675574597004i64, + statistics_path: "s3://a/b/stats.puffin".to_string(), + file_size_in_bytes: 413, + file_footer_size_in_bytes: 42, + key_metadata: None, + blob_metadata: vec![BlobMetadata { + r#type: "ndv".to_string(), + snapshot_id: 3055729675574597004i64, + sequence_number: 1, + fields: vec![1], + properties: HashMap::new(), + }], + }, + ); + } + + #[test] + fn test_partition_statistics_serde() { + test_serde_json( + json!({ + "snapshot-id": 3055729675574597004i64, + "statistics-path": "s3://a/b/partition-stats.parquet", + "file-size-in-bytes": 43 + }), + PartitionStatisticsFile { + snapshot_id: 3055729675574597004, + statistics_path: "s3://a/b/partition-stats.parquet".to_string(), + file_size_in_bytes: 43, + }, + ); + } +} diff --git a/crates/iceberg/src/spec/table_metadata.rs b/crates/iceberg/src/spec/table_metadata.rs index bb1cc917c..38204fc15 100644 --- a/crates/iceberg/src/spec/table_metadata.rs +++ b/crates/iceberg/src/spec/table_metadata.rs @@ -21,6 +21,7 @@ use std::cmp::Ordering; use std::collections::HashMap; use std::fmt::{Display, Formatter}; +use std::hash::Hash; use std::sync::Arc; use _serde::TableMetadataEnum; @@ -32,8 +33,9 @@ use uuid::Uuid; use super::snapshot::SnapshotReference; pub use super::table_metadata_builder::{TableMetadataBuildResult, TableMetadataBuilder}; use super::{ - PartitionSpecRef, SchemaId, SchemaRef, Snapshot, SnapshotRef, SnapshotRetention, SortOrder, - SortOrderRef, StructType, DEFAULT_PARTITION_SPEC_ID, + PartitionSpecRef, PartitionStatisticsFile, SchemaId, SchemaRef, Snapshot, SnapshotRef, + SnapshotRetention, SortOrder, SortOrderRef, StatisticsFile, StructType, + DEFAULT_PARTITION_SPEC_ID, }; use crate::error::{timestamp_ms_to_utc, Result}; use crate::{Error, ErrorKind}; @@ -160,11 +162,15 @@ pub struct TableMetadata { /// writers, but is not used when reading because reads use the specs /// stored in manifest files. pub(crate) default_sort_order_id: i64, - ///A map of snapshot references. The map keys are the unique snapshot reference + /// A map of snapshot references. The map keys are the unique snapshot reference /// names in the table, and the map values are snapshot reference objects. /// There is always a main branch reference pointing to the current-snapshot-id /// even if the refs map is null. pub(crate) refs: HashMap, + /// Mapping of snapshot ids to statistics files. + pub(crate) statistics: HashMap, + /// Mapping of snapshot ids to partition statistics files. + pub(crate) partition_statistics: HashMap, } impl TableMetadata { @@ -363,6 +369,35 @@ impl TableMetadata { &self.properties } + /// Return location of statistics files. + #[inline] + pub fn statistics_iter(&self) -> impl ExactSizeIterator { + self.statistics.values() + } + + /// Return location of partition statistics files. + #[inline] + pub fn partition_statistics_iter( + &self, + ) -> impl ExactSizeIterator { + self.partition_statistics.values() + } + + /// Get a statistics file for a snapshot id. + #[inline] + pub fn statistics_for_snapshot(&self, snapshot_id: i64) -> Option<&StatisticsFile> { + self.statistics.get(&snapshot_id) + } + + /// Get a partition statistics file for a snapshot id. + #[inline] + pub fn partition_statistics_for_snapshot( + &self, + snapshot_id: i64, + ) -> Option<&PartitionStatisticsFile> { + self.partition_statistics.get(&snapshot_id) + } + /// Append snapshot to table #[deprecated( since = "0.4.0", @@ -616,8 +651,8 @@ pub(super) mod _serde { use crate::spec::schema::_serde::{SchemaV1, SchemaV2}; use crate::spec::snapshot::_serde::{SnapshotV1, SnapshotV2}; use crate::spec::{ - PartitionField, PartitionSpec, PartitionSpecRef, Schema, SchemaRef, Snapshot, - SnapshotReference, SnapshotRetention, SortOrder, + PartitionField, PartitionSpec, PartitionSpecRef, PartitionStatisticsFile, Schema, + SchemaRef, Snapshot, SnapshotReference, SnapshotRetention, SortOrder, StatisticsFile, }; use crate::{Error, ErrorKind}; @@ -657,6 +692,10 @@ pub(super) mod _serde { pub default_sort_order_id: i64, #[serde(skip_serializing_if = "Option::is_none")] pub refs: Option>, + #[serde(default, skip_serializing_if = "Vec::is_empty")] + pub statistics: Vec, + #[serde(default, skip_serializing_if = "Vec::is_empty")] + pub partition_statistics: Vec, } #[derive(Debug, Serialize, Deserialize, PartialEq, Eq)] @@ -693,6 +732,10 @@ pub(super) mod _serde { pub metadata_log: Option>, pub sort_orders: Option>, pub default_sort_order_id: Option, + #[serde(default, skip_serializing_if = "Vec::is_empty")] + pub statistics: Vec, + #[serde(default, skip_serializing_if = "Vec::is_empty")] + pub partition_statistics: Vec, } /// Helper to serialize and deserialize the format version. @@ -846,6 +889,8 @@ pub(super) mod _serde { HashMap::new() } }), + statistics: index_statistics(value.statistics), + partition_statistics: index_partition_statistics(value.partition_statistics), }; metadata.borrow_mut().try_normalize()?; @@ -980,6 +1025,8 @@ pub(super) mod _serde { } else { HashMap::new() }, + statistics: index_statistics(value.statistics), + partition_statistics: index_partition_statistics(value.partition_statistics), }; metadata.borrow_mut().try_normalize()?; @@ -1050,6 +1097,8 @@ pub(super) mod _serde { .collect(), default_sort_order_id: v.default_sort_order_id, refs: Some(v.refs), + statistics: v.statistics.into_values().collect(), + partition_statistics: v.partition_statistics.into_values().collect(), } } } @@ -1126,9 +1175,29 @@ pub(super) mod _serde { .collect(), ), default_sort_order_id: Some(v.default_sort_order_id), + statistics: v.statistics.into_values().collect(), + partition_statistics: v.partition_statistics.into_values().collect(), }) } } + + fn index_statistics(statistics: Vec) -> HashMap { + statistics + .into_iter() + .rev() + .map(|s| (s.snapshot_id, s)) + .collect() + } + + fn index_partition_statistics( + statistics: Vec, + ) -> HashMap { + statistics + .into_iter() + .rev() + .map(|s| (s.snapshot_id, s)) + .collect() + } } #[derive(Debug, Serialize_repr, Deserialize_repr, PartialEq, Eq, Clone, Copy, Hash)] @@ -1208,9 +1277,9 @@ mod tests { use super::{FormatVersion, MetadataLog, SnapshotLog, TableMetadataBuilder}; use crate::spec::table_metadata::TableMetadata; use crate::spec::{ - NestedField, NullOrder, Operation, PartitionSpec, PrimitiveType, Schema, Snapshot, - SnapshotReference, SnapshotRetention, SortDirection, SortField, SortOrder, Summary, - Transform, Type, UnboundPartitionField, + BlobMetadata, NestedField, NullOrder, Operation, PartitionSpec, PartitionStatisticsFile, + PrimitiveType, Schema, Snapshot, SnapshotReference, SnapshotRetention, SortDirection, + SortField, SortOrder, StatisticsFile, Summary, Transform, Type, UnboundPartitionField, }; use crate::TableCreation; @@ -1354,6 +1423,8 @@ mod tests { timestamp_ms: 1515100, }], refs: HashMap::new(), + statistics: HashMap::new(), + partition_statistics: HashMap::new(), }; let expected_json_value = serde_json::to_value(&expected).unwrap(); @@ -1527,6 +1598,8 @@ mod tests { }], metadata_log: vec![MetadataLog { metadata_file: "/home/iceberg/warehouse/nyc/taxis/metadata/00000-8a62c37d-4573-4021-952a-c0baef7d21d0.metadata.json".to_string(), timestamp_ms: 1662532805245 }], refs: HashMap::from_iter(vec![("main".to_string(), SnapshotReference { snapshot_id: 638933773299822130, retention: SnapshotRetention::Branch { min_snapshots_to_keep: None, max_snapshot_age_ms: None, max_ref_age_ms: None } })]), + statistics: HashMap::new(), + partition_statistics: HashMap::new(), }; check_table_metadata_serde(data, expected); @@ -1622,6 +1695,8 @@ mod tests { timestamp_ms: 1515100, }], refs: HashMap::new(), + statistics: HashMap::new(), + partition_statistics: HashMap::new(), }; let expected_json_value = serde_json::to_value(&expected).unwrap(); @@ -1923,6 +1998,291 @@ mod tests { .contains("Snapshot for reference foo does not exist in the existing snapshots list")); } + #[test] + fn test_statistic_files() { + let data = r#" + { + "format-version": 2, + "table-uuid": "9c12d441-03fe-4693-9a96-a0705ddf69c1", + "location": "s3://bucket/test/location", + "last-sequence-number": 34, + "last-updated-ms": 1602638573590, + "last-column-id": 3, + "current-schema-id": 0, + "schemas": [ + { + "type": "struct", + "schema-id": 0, + "fields": [ + { + "id": 1, + "name": "x", + "required": true, + "type": "long" + } + ] + } + ], + "default-spec-id": 0, + "partition-specs": [ + { + "spec-id": 0, + "fields": [] + } + ], + "last-partition-id": 1000, + "default-sort-order-id": 0, + "sort-orders": [ + { + "order-id": 0, + "fields": [] + } + ], + "properties": {}, + "current-snapshot-id": 3055729675574597004, + "snapshots": [ + { + "snapshot-id": 3055729675574597004, + "timestamp-ms": 1555100955770, + "sequence-number": 1, + "summary": { + "operation": "append" + }, + "manifest-list": "s3://a/b/2.avro", + "schema-id": 0 + } + ], + "statistics": [ + { + "snapshot-id": 3055729675574597004, + "statistics-path": "s3://a/b/stats.puffin", + "file-size-in-bytes": 413, + "file-footer-size-in-bytes": 42, + "blob-metadata": [ + { + "type": "ndv", + "snapshot-id": 3055729675574597004, + "sequence-number": 1, + "fields": [ + 1 + ] + } + ] + } + ], + "snapshot-log": [], + "metadata-log": [] + } + "#; + + let schema = Schema::builder() + .with_schema_id(0) + .with_fields(vec![Arc::new(NestedField::required( + 1, + "x", + Type::Primitive(PrimitiveType::Long), + ))]) + .build() + .unwrap(); + let partition_spec = PartitionSpec::builder(schema.clone()) + .with_spec_id(0) + .build() + .unwrap(); + let snapshot = Snapshot::builder() + .with_snapshot_id(3055729675574597004) + .with_timestamp_ms(1555100955770) + .with_sequence_number(1) + .with_manifest_list("s3://a/b/2.avro") + .with_schema_id(0) + .with_summary(Summary { + operation: Operation::Append, + additional_properties: HashMap::new(), + }) + .build(); + + let default_partition_type = partition_spec.partition_type(&schema).unwrap(); + let expected = TableMetadata { + format_version: FormatVersion::V2, + table_uuid: Uuid::parse_str("9c12d441-03fe-4693-9a96-a0705ddf69c1").unwrap(), + location: "s3://bucket/test/location".to_string(), + last_updated_ms: 1602638573590, + last_column_id: 3, + schemas: HashMap::from_iter(vec![(0, Arc::new(schema))]), + current_schema_id: 0, + partition_specs: HashMap::from_iter(vec![(0, partition_spec.clone().into())]), + default_partition_type, + default_spec: Arc::new(partition_spec), + last_partition_id: 1000, + default_sort_order_id: 0, + sort_orders: HashMap::from_iter(vec![(0, SortOrder::unsorted_order().into())]), + snapshots: HashMap::from_iter(vec![(3055729675574597004, Arc::new(snapshot))]), + current_snapshot_id: Some(3055729675574597004), + last_sequence_number: 34, + properties: HashMap::new(), + snapshot_log: Vec::new(), + metadata_log: Vec::new(), + statistics: HashMap::from_iter(vec![(3055729675574597004, StatisticsFile { + snapshot_id: 3055729675574597004, + statistics_path: "s3://a/b/stats.puffin".to_string(), + file_size_in_bytes: 413, + file_footer_size_in_bytes: 42, + key_metadata: None, + blob_metadata: vec![BlobMetadata { + snapshot_id: 3055729675574597004, + sequence_number: 1, + fields: vec![1], + r#type: "ndv".to_string(), + properties: HashMap::new(), + }], + })]), + partition_statistics: HashMap::new(), + refs: HashMap::from_iter(vec![("main".to_string(), SnapshotReference { + snapshot_id: 3055729675574597004, + retention: SnapshotRetention::Branch { + min_snapshots_to_keep: None, + max_snapshot_age_ms: None, + max_ref_age_ms: None, + }, + })]), + }; + + check_table_metadata_serde(data, expected); + } + + #[test] + fn test_partition_statistics_file() { + let data = r#" + { + "format-version": 2, + "table-uuid": "9c12d441-03fe-4693-9a96-a0705ddf69c1", + "location": "s3://bucket/test/location", + "last-sequence-number": 34, + "last-updated-ms": 1602638573590, + "last-column-id": 3, + "current-schema-id": 0, + "schemas": [ + { + "type": "struct", + "schema-id": 0, + "fields": [ + { + "id": 1, + "name": "x", + "required": true, + "type": "long" + } + ] + } + ], + "default-spec-id": 0, + "partition-specs": [ + { + "spec-id": 0, + "fields": [] + } + ], + "last-partition-id": 1000, + "default-sort-order-id": 0, + "sort-orders": [ + { + "order-id": 0, + "fields": [] + } + ], + "properties": {}, + "current-snapshot-id": 3055729675574597004, + "snapshots": [ + { + "snapshot-id": 3055729675574597004, + "timestamp-ms": 1555100955770, + "sequence-number": 1, + "summary": { + "operation": "append" + }, + "manifest-list": "s3://a/b/2.avro", + "schema-id": 0 + } + ], + "partition-statistics": [ + { + "snapshot-id": 3055729675574597004, + "statistics-path": "s3://a/b/partition-stats.parquet", + "file-size-in-bytes": 43 + } + ], + "snapshot-log": [], + "metadata-log": [] + } + "#; + + let schema = Schema::builder() + .with_schema_id(0) + .with_fields(vec![Arc::new(NestedField::required( + 1, + "x", + Type::Primitive(PrimitiveType::Long), + ))]) + .build() + .unwrap(); + let partition_spec = PartitionSpec::builder(schema.clone()) + .with_spec_id(0) + .build() + .unwrap(); + let snapshot = Snapshot::builder() + .with_snapshot_id(3055729675574597004) + .with_timestamp_ms(1555100955770) + .with_sequence_number(1) + .with_manifest_list("s3://a/b/2.avro") + .with_schema_id(0) + .with_summary(Summary { + operation: Operation::Append, + additional_properties: HashMap::new(), + }) + .build(); + + let default_partition_type = partition_spec.partition_type(&schema).unwrap(); + let expected = TableMetadata { + format_version: FormatVersion::V2, + table_uuid: Uuid::parse_str("9c12d441-03fe-4693-9a96-a0705ddf69c1").unwrap(), + location: "s3://bucket/test/location".to_string(), + last_updated_ms: 1602638573590, + last_column_id: 3, + schemas: HashMap::from_iter(vec![(0, Arc::new(schema))]), + current_schema_id: 0, + partition_specs: HashMap::from_iter(vec![(0, partition_spec.clone().into())]), + default_spec: Arc::new(partition_spec), + default_partition_type, + last_partition_id: 1000, + default_sort_order_id: 0, + sort_orders: HashMap::from_iter(vec![(0, SortOrder::unsorted_order().into())]), + snapshots: HashMap::from_iter(vec![(3055729675574597004, Arc::new(snapshot))]), + current_snapshot_id: Some(3055729675574597004), + last_sequence_number: 34, + properties: HashMap::new(), + snapshot_log: Vec::new(), + metadata_log: Vec::new(), + statistics: HashMap::new(), + partition_statistics: HashMap::from_iter(vec![( + 3055729675574597004, + PartitionStatisticsFile { + snapshot_id: 3055729675574597004, + statistics_path: "s3://a/b/partition-stats.parquet".to_string(), + file_size_in_bytes: 43, + }, + )]), + refs: HashMap::from_iter(vec![("main".to_string(), SnapshotReference { + snapshot_id: 3055729675574597004, + retention: SnapshotRetention::Branch { + min_snapshots_to_keep: None, + max_snapshot_age_ms: None, + max_ref_age_ms: None, + }, + })]), + }; + + check_table_metadata_serde(data, expected); + } + #[test] fn test_invalid_table_uuid() -> Result<()> { let data = r#" @@ -2077,6 +2437,8 @@ mod tests { max_ref_age_ms: None, }, })]), + statistics: HashMap::new(), + partition_statistics: HashMap::new(), }; check_table_metadata_serde(&metadata, expected); @@ -2159,6 +2521,8 @@ mod tests { snapshot_log: vec![], metadata_log: Vec::new(), refs: HashMap::new(), + statistics: HashMap::new(), + partition_statistics: HashMap::new(), }; check_table_metadata_serde(&metadata, expected); @@ -2225,6 +2589,8 @@ mod tests { snapshot_log: vec![], metadata_log: Vec::new(), refs: HashMap::new(), + statistics: HashMap::new(), + partition_statistics: HashMap::new(), }; check_table_metadata_serde(&metadata, expected); diff --git a/crates/iceberg/src/spec/table_metadata_builder.rs b/crates/iceberg/src/spec/table_metadata_builder.rs index d6ea2c91f..4f200c8b2 100644 --- a/crates/iceberg/src/spec/table_metadata_builder.rs +++ b/crates/iceberg/src/spec/table_metadata_builder.rs @@ -21,12 +21,12 @@ use std::sync::Arc; use uuid::Uuid; use super::{ - FormatVersion, MetadataLog, PartitionSpec, PartitionSpecBuilder, Schema, SchemaRef, Snapshot, - SnapshotLog, SnapshotReference, SnapshotRetention, SortOrder, SortOrderRef, StructType, - TableMetadata, UnboundPartitionSpec, DEFAULT_PARTITION_SPEC_ID, DEFAULT_SCHEMA_ID, MAIN_BRANCH, - ONE_MINUTE_MS, PROPERTY_METADATA_PREVIOUS_VERSIONS_MAX, - PROPERTY_METADATA_PREVIOUS_VERSIONS_MAX_DEFAULT, RESERVED_PROPERTIES, - UNPARTITIONED_LAST_ASSIGNED_ID, + FormatVersion, MetadataLog, PartitionSpec, PartitionSpecBuilder, PartitionStatisticsFile, + Schema, SchemaRef, Snapshot, SnapshotLog, SnapshotReference, SnapshotRetention, SortOrder, + SortOrderRef, StatisticsFile, StructType, TableMetadata, UnboundPartitionSpec, + DEFAULT_PARTITION_SPEC_ID, DEFAULT_SCHEMA_ID, MAIN_BRANCH, ONE_MINUTE_MS, + PROPERTY_METADATA_PREVIOUS_VERSIONS_MAX, PROPERTY_METADATA_PREVIOUS_VERSIONS_MAX_DEFAULT, + RESERVED_PROPERTIES, UNPARTITIONED_LAST_ASSIGNED_ID, }; use crate::error::{Error, ErrorKind, Result}; use crate::{TableCreation, TableUpdate}; @@ -117,6 +117,8 @@ impl TableMetadataBuilder { metadata_log: vec![], default_sort_order_id: -1, // Overwritten immediately by add_default_sort_order refs: HashMap::default(), + statistics: HashMap::new(), + partition_statistics: HashMap::new(), }, changes: vec![], last_added_schema_id: Some(schema_id), @@ -524,6 +526,52 @@ impl TableMetadataBuilder { self } + /// Set statistics for a snapshot + pub fn set_statistics(mut self, statistics: StatisticsFile) -> Self { + self.metadata + .statistics + .insert(statistics.snapshot_id, statistics.clone()); + self.changes.push(TableUpdate::SetStatistics { + statistics: statistics.clone(), + }); + self + } + + /// Remove statistics for a snapshot + pub fn remove_statistics(mut self, snapshot_id: i64) -> Self { + let previous = self.metadata.statistics.remove(&snapshot_id); + if previous.is_some() { + self.changes + .push(TableUpdate::RemoveStatistics { snapshot_id }); + } + self + } + + /// Set partition statistics + pub fn set_partition_statistics( + mut self, + partition_statistics_file: PartitionStatisticsFile, + ) -> Self { + self.metadata.partition_statistics.insert( + partition_statistics_file.snapshot_id, + partition_statistics_file.clone(), + ); + self.changes.push(TableUpdate::SetPartitionStatistics { + partition_statistics: partition_statistics_file, + }); + self + } + + /// Remove partition statistics + pub fn remove_partition_statistics(mut self, snapshot_id: i64) -> Self { + let previous = self.metadata.partition_statistics.remove(&snapshot_id); + if previous.is_some() { + self.changes + .push(TableUpdate::RemovePartitionStatistics { snapshot_id }); + } + self + } + /// Add a schema to the table metadata. /// /// The provided `schema.schema_id` may not be used. @@ -1138,8 +1186,9 @@ impl From for TableMetadata { mod tests { use super::*; use crate::spec::{ - NestedField, NullOrder, Operation, PartitionSpec, PrimitiveType, Schema, SnapshotRetention, - SortDirection, SortField, StructType, Summary, Transform, Type, UnboundPartitionField, + BlobMetadata, NestedField, NullOrder, Operation, PartitionSpec, PrimitiveType, Schema, + SnapshotRetention, SortDirection, SortField, StructType, Summary, Transform, Type, + UnboundPartitionField, }; const TEST_LOCATION: &str = "s3://bucket/test/location"; @@ -2152,4 +2201,101 @@ mod tests { .to_string() .contains("Cannot add snapshot with sequence number")); } + + #[test] + fn test_statistics() { + let builder = builder_without_changes(FormatVersion::V2); + + let statistics = StatisticsFile { + snapshot_id: 3055729675574597004, + statistics_path: "s3://a/b/stats.puffin".to_string(), + file_size_in_bytes: 413, + file_footer_size_in_bytes: 42, + key_metadata: None, + blob_metadata: vec![BlobMetadata { + snapshot_id: 3055729675574597004, + sequence_number: 1, + fields: vec![1], + r#type: "ndv".to_string(), + properties: HashMap::new(), + }], + }; + let build_result = builder.set_statistics(statistics.clone()).build().unwrap(); + + assert_eq!( + build_result.metadata.statistics, + HashMap::from_iter(vec![(3055729675574597004, statistics.clone())]) + ); + assert_eq!(build_result.changes, vec![TableUpdate::SetStatistics { + statistics: statistics.clone() + }]); + + // Remove + let builder = build_result.metadata.into_builder(None); + let build_result = builder + .remove_statistics(statistics.snapshot_id) + .build() + .unwrap(); + + assert_eq!(build_result.metadata.statistics.len(), 0); + assert_eq!(build_result.changes, vec![TableUpdate::RemoveStatistics { + snapshot_id: statistics.snapshot_id + }]); + + // Remove again yields no changes + let builder = build_result.metadata.into_builder(None); + let build_result = builder + .remove_statistics(statistics.snapshot_id) + .build() + .unwrap(); + assert_eq!(build_result.metadata.statistics.len(), 0); + assert_eq!(build_result.changes.len(), 0); + } + + #[test] + fn test_add_partition_statistics() { + let builder = builder_without_changes(FormatVersion::V2); + + let statistics = PartitionStatisticsFile { + snapshot_id: 3055729675574597004, + statistics_path: "s3://a/b/partition-stats.parquet".to_string(), + file_size_in_bytes: 43, + }; + + let build_result = builder + .set_partition_statistics(statistics.clone()) + .build() + .unwrap(); + assert_eq!( + build_result.metadata.partition_statistics, + HashMap::from_iter(vec![(3055729675574597004, statistics.clone())]) + ); + assert_eq!(build_result.changes, vec![ + TableUpdate::SetPartitionStatistics { + partition_statistics: statistics.clone() + } + ]); + + // Remove + let builder = build_result.metadata.into_builder(None); + let build_result = builder + .remove_partition_statistics(statistics.snapshot_id) + .build() + .unwrap(); + assert_eq!(build_result.metadata.partition_statistics.len(), 0); + assert_eq!(build_result.changes, vec![ + TableUpdate::RemovePartitionStatistics { + snapshot_id: statistics.snapshot_id + } + ]); + + // Remove again yields no changes + let builder = build_result.metadata.into_builder(None); + let build_result = builder + .remove_partition_statistics(statistics.snapshot_id) + .build() + .unwrap(); + assert_eq!(build_result.metadata.partition_statistics.len(), 0); + assert_eq!(build_result.changes.len(), 0); + } } diff --git a/crates/iceberg/src/writer/file_writer/location_generator.rs b/crates/iceberg/src/writer/file_writer/location_generator.rs index abf0de4d4..3f5d4ee18 100644 --- a/crates/iceberg/src/writer/file_writer/location_generator.rs +++ b/crates/iceberg/src/writer/file_writer/location_generator.rs @@ -177,6 +177,8 @@ pub(crate) mod test { snapshot_log: Vec::new(), metadata_log: vec![], refs: HashMap::new(), + statistics: HashMap::new(), + partition_statistics: HashMap::new(), }; let file_name_genertaor = super::DefaultFileNameGenerator::new( From b17533f8312ae02f6ec1a761d3852efba2f85127 Mon Sep 17 00:00:00 2001 From: Sung Yun <107272191+sungwy@users.noreply.github.com> Date: Mon, 16 Dec 2024 11:33:12 -0500 Subject: [PATCH 13/23] bump to0.4.0 (#808) --- bindings/python/Cargo.toml | 2 +- bindings/python/pyproject.toml | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/bindings/python/Cargo.toml b/bindings/python/Cargo.toml index 64c992c9d..cb99828d6 100644 --- a/bindings/python/Cargo.toml +++ b/bindings/python/Cargo.toml @@ -17,7 +17,7 @@ [package] name = "pyiceberg_core_rust" -version = "0.1.0" +version = "0.4.0" edition = "2021" homepage = "https://rust.iceberg.apache.org" rust-version = "1.77.1" diff --git a/bindings/python/pyproject.toml b/bindings/python/pyproject.toml index c0ade9398..981bd2d4f 100644 --- a/bindings/python/pyproject.toml +++ b/bindings/python/pyproject.toml @@ -21,7 +21,7 @@ build-backend = "maturin" [project] name = "pyiceberg_core" -version = "0.1.0" +version = "0.4.0" readme = "project-description.md" classifiers = [ "Development Status :: 4 - Beta", From 43501ffac6aff8103d6c37ecc600f2c4f29ad3f4 Mon Sep 17 00:00:00 2001 From: Christian Date: Tue, 17 Dec 2024 00:52:25 +0100 Subject: [PATCH 14/23] chore(docs): Update Readme - Lakekeeper repository moved (#810) --- README.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/README.md b/README.md index cef0bb5dd..47c3d8093 100644 --- a/README.md +++ b/README.md @@ -100,7 +100,7 @@ The Apache Iceberg community is built on the principles described in the [Apache ## Users - [Databend](https://github.com/datafuselabs/databend/): An open-source cloud data warehouse that serves as a cost-effective alternative to Snowflake. -- [iceberg-catalog](https://github.com/hansetag/iceberg-catalog): A Rust implementation of the Iceberg REST Catalog specification. +- [Lakekeeper](https://github.com/lakekeeper/lakekeeper/): An Apache-licensed Iceberg REST Catalog with data access controls. - [RisingWave](https://github.com/risingwavelabs/risingwave): A Postgres-compatible SQL database designed for real-time event streaming data processing, analysis, and management. ## License From f90dbffe59e3620be5f35549736f68343cd72cb2 Mon Sep 17 00:00:00 2001 From: Sung Yun <107272191+sungwy@users.noreply.github.com> Date: Mon, 16 Dec 2024 20:39:06 -0500 Subject: [PATCH 15/23] Prep 0.4.0 release (#809) * deprecate functions * bump version to 0.4.0 * revert --- Cargo.toml | 12 +++---- README.md | 2 -- crates/catalog/rest/src/catalog.rs | 36 ------------------- crates/iceberg/src/catalog/mod.rs | 8 ----- .../src/spec/table_metadata_builder.rs | 13 ++----- 5 files changed, 8 insertions(+), 63 deletions(-) diff --git a/Cargo.toml b/Cargo.toml index c766040d7..05f2d9073 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -28,7 +28,7 @@ members = [ exclude = ["bindings/python"] [workspace.package] -version = "0.3.0" +version = "0.4.0" edition = "2021" homepage = "https://rust.iceberg.apache.org/" @@ -62,11 +62,11 @@ either = "1" env_logger = "0.11.0" fnv = "1" futures = "0.3" -iceberg = { version = "0.3.0", path = "./crates/iceberg" } -iceberg-catalog-rest = { version = "0.3.0", path = "./crates/catalog/rest" } -iceberg-catalog-hms = { version = "0.3.0", path = "./crates/catalog/hms" } -iceberg-catalog-memory = { version = "0.3.0", path = "./crates/catalog/memory" } -iceberg-datafusion = { version = "0.3.0", path = "./crates/integrations/datafusion" } +iceberg = { version = "0.4.0", path = "./crates/iceberg" } +iceberg-catalog-rest = { version = "0.4.0", path = "./crates/catalog/rest" } +iceberg-catalog-hms = { version = "0.4.0", path = "./crates/catalog/hms" } +iceberg-catalog-memory = { version = "0.4.0", path = "./crates/catalog/memory" } +iceberg-datafusion = { version = "0.4.0", path = "./crates/integrations/datafusion" } itertools = "0.13" log = "0.4" mockito = "1" diff --git a/README.md b/README.md index 47c3d8093..0036c94f1 100644 --- a/README.md +++ b/README.md @@ -23,8 +23,6 @@ Rust implementation of [Apache Iceberg™](https://iceberg.apache.org/). -Working on [v0.3.0 Release Milestone](https://github.com/apache/iceberg-rust/milestone/2) - ## Components The Apache Iceberg Rust project is composed of the following components: diff --git a/crates/catalog/rest/src/catalog.rs b/crates/catalog/rest/src/catalog.rs index 96da5dc95..7405e5350 100644 --- a/crates/catalog/rest/src/catalog.rs +++ b/crates/catalog/rest/src/catalog.rs @@ -73,12 +73,6 @@ impl RestCatalogConfig { pub(crate) fn get_token_endpoint(&self) -> String { if let Some(oauth2_uri) = self.props.get("oauth2-server-uri") { oauth2_uri.to_string() - } else if let Some(auth_url) = self.props.get("rest.authorization-url") { - log::warn!( - "'rest.authorization-url' is deprecated and will be removed in version 0.4.0. \ - Please use 'oauth2-server-uri' instead." - ); - auth_url.to_string() } else { [&self.uri, PATH_V1, "oauth", "tokens"].join("/") } @@ -924,36 +918,6 @@ mod tests { assert_eq!(headers, expected_headers); } - #[tokio::test] - async fn test_oauth_with_deprecated_auth_url() { - let mut server = Server::new_async().await; - let config_mock = create_config_mock(&mut server).await; - - let mut auth_server = Server::new_async().await; - let auth_server_path = "/some/path"; - let oauth_mock = create_oauth_mock_with_path(&mut auth_server, auth_server_path).await; - - let mut props = HashMap::new(); - props.insert("credential".to_string(), "client1:secret1".to_string()); - props.insert( - "rest.authorization-url".to_string(), - format!("{}{}", auth_server.url(), auth_server_path).to_string(), - ); - - let catalog = RestCatalog::new( - RestCatalogConfig::builder() - .uri(server.url()) - .props(props) - .build(), - ); - - let token = catalog.context().await.unwrap().client.token().await; - - oauth_mock.assert_async().await; - config_mock.assert_async().await; - assert_eq!(token, Some("ey000000000000".to_string())); - } - #[tokio::test] async fn test_oauth_with_oauth2_server_uri() { let mut server = Server::new_async().await; diff --git a/crates/iceberg/src/catalog/mod.rs b/crates/iceberg/src/catalog/mod.rs index deb4d2f34..ffafc66fa 100644 --- a/crates/iceberg/src/catalog/mod.rs +++ b/crates/iceberg/src/catalog/mod.rs @@ -367,12 +367,6 @@ pub enum TableUpdate { AddSchema { /// The schema to add. schema: Schema, - /// The last column id of the table. - #[deprecated( - since = "0.3.0", - note = "This field is handled internally, and should not be part of the update." - )] - last_column_id: Option, }, /// Set table's current schema #[serde(rename_all = "kebab-case")] @@ -1301,7 +1295,6 @@ mod tests { "#, TableUpdate::AddSchema { schema: test_schema.clone(), - last_column_id: Some(3), }, ); @@ -1340,7 +1333,6 @@ mod tests { "#, TableUpdate::AddSchema { schema: test_schema.clone(), - last_column_id: None, }, ); } diff --git a/crates/iceberg/src/spec/table_metadata_builder.rs b/crates/iceberg/src/spec/table_metadata_builder.rs index 4f200c8b2..5b4ff1234 100644 --- a/crates/iceberg/src/spec/table_metadata_builder.rs +++ b/crates/iceberg/src/spec/table_metadata_builder.rs @@ -585,7 +585,6 @@ impl TableMetadataBuilder { if schema_found { if self.last_added_schema_id != Some(new_schema_id) { self.changes.push(TableUpdate::AddSchema { - last_column_id: Some(self.metadata.last_column_id), schema: schema.clone(), }); self.last_added_schema_id = Some(new_schema_id); @@ -609,10 +608,7 @@ impl TableMetadataBuilder { .schemas .insert(new_schema_id, schema.clone().into()); - self.changes.push(TableUpdate::AddSchema { - schema, - last_column_id: Some(self.metadata.last_column_id), - }); + self.changes.push(TableUpdate::AddSchema { schema }); self.last_added_schema_id = Some(new_schema_id); @@ -1453,10 +1449,7 @@ mod tests { TableUpdate::SetLocation { location: TEST_LOCATION.to_string() }, - TableUpdate::AddSchema { - last_column_id: Some(LAST_ASSIGNED_COLUMN_ID), - schema: schema(), - }, + TableUpdate::AddSchema { schema: schema() }, TableUpdate::SetCurrentSchema { schema_id: -1 }, TableUpdate::AddSpec { // Because this is a new tables, field-ids are assigned @@ -1509,7 +1502,6 @@ mod tests { location: TEST_LOCATION.to_string() }, TableUpdate::AddSchema { - last_column_id: Some(0), schema: Schema::builder().build().unwrap(), }, TableUpdate::SetCurrentSchema { schema_id: -1 }, @@ -1751,7 +1743,6 @@ mod tests { Some(&Arc::new(added_schema.clone())) ); pretty_assertions::assert_eq!(build_result.changes[0], TableUpdate::AddSchema { - last_column_id: Some(4), schema: added_schema }); assert_eq!(build_result.changes[1], TableUpdate::SetCurrentSchema { From fa8c948538ebb7792365a295cc0ffa9b7fd0a127 Mon Sep 17 00:00:00 2001 From: Christian Date: Tue, 17 Dec 2024 02:40:27 +0100 Subject: [PATCH 16/23] feat: Add RemovePartitionSpecs table update (#804) * RemovePartitionSpecs update * address comments * fix comments --- crates/iceberg/src/catalog/mod.rs | 24 +++++++++ .../src/spec/table_metadata_builder.rs | 52 +++++++++++++++++++ 2 files changed, 76 insertions(+) diff --git a/crates/iceberg/src/catalog/mod.rs b/crates/iceberg/src/catalog/mod.rs index ffafc66fa..cbda6c905 100644 --- a/crates/iceberg/src/catalog/mod.rs +++ b/crates/iceberg/src/catalog/mod.rs @@ -440,6 +440,12 @@ pub enum TableUpdate { /// Properties to remove removals: Vec, }, + /// Remove partition specs + #[serde(rename_all = "kebab-case")] + RemovePartitionSpecs { + /// Partition spec ids to remove. + spec_ids: Vec, + }, /// Set statistics for a snapshot #[serde(with = "_serde_set_statistics")] SetStatistics { @@ -494,6 +500,9 @@ impl TableUpdate { TableUpdate::UpgradeFormatVersion { format_version } => { builder.upgrade_format_version(format_version) } + TableUpdate::RemovePartitionSpecs { spec_ids } => { + builder.remove_partition_specs(&spec_ids) + } TableUpdate::SetStatistics { statistics } => Ok(builder.set_statistics(statistics)), TableUpdate::RemoveStatistics { snapshot_id } => { Ok(builder.remove_statistics(snapshot_id)) @@ -1910,6 +1919,21 @@ mod tests { ); } + #[test] + fn test_remove_partition_specs_update() { + test_serde_json( + r#" +{ + "action": "remove-partition-specs", + "spec-ids": [1, 2] +} + "#, + TableUpdate::RemovePartitionSpecs { + spec_ids: vec![1, 2], + }, + ); + } + #[test] fn test_set_statistics_file() { test_serde_json( diff --git a/crates/iceberg/src/spec/table_metadata_builder.rs b/crates/iceberg/src/spec/table_metadata_builder.rs index 5b4ff1234..4e8c1ae76 100644 --- a/crates/iceberg/src/spec/table_metadata_builder.rs +++ b/crates/iceberg/src/spec/table_metadata_builder.rs @@ -784,6 +784,36 @@ impl TableMetadataBuilder { .set_default_partition_spec(Self::LAST_ADDED) } + /// Remove partition specs by their ids from the table metadata. + /// Does nothing if a spec id is not present. Active partition specs + /// should not be removed. + /// + /// # Errors + /// - Cannot remove the default partition spec. + pub fn remove_partition_specs(mut self, spec_ids: &[i32]) -> Result { + if spec_ids.contains(&self.metadata.default_spec.spec_id()) { + return Err(Error::new( + ErrorKind::DataInvalid, + "Cannot remove default partition spec", + )); + } + + let mut removed_specs = Vec::with_capacity(spec_ids.len()); + spec_ids.iter().for_each(|id| { + if self.metadata.partition_specs.remove(id).is_some() { + removed_specs.push(*id); + } + }); + + if !removed_specs.is_empty() { + self.changes.push(TableUpdate::RemovePartitionSpecs { + spec_ids: removed_specs, + }); + } + + Ok(self) + } + /// Add a sort order to the table metadata. /// /// The spec is bound eagerly to the current schema and must be valid for it. @@ -1584,6 +1614,21 @@ mod tests { pretty_assertions::assert_eq!(build_result.changes[0], TableUpdate::AddSpec { spec: expected_change }); + + // Remove the spec + let build_result = build_result + .metadata + .into_builder(Some( + "s3://bucket/test/location/metadata/metadata1.json".to_string(), + )) + .remove_partition_specs(&[1]) + .unwrap() + .build() + .unwrap(); + + assert_eq!(build_result.changes.len(), 1); + assert_eq!(build_result.metadata.partition_specs.len(), 1); + assert!(build_result.metadata.partition_spec_by_id(1).is_none()); } #[test] @@ -2193,6 +2238,13 @@ mod tests { .contains("Cannot add snapshot with sequence number")); } + #[test] + fn test_default_spec_cannot_be_removed() { + let builder = builder_without_changes(FormatVersion::V2); + + builder.remove_partition_specs(&[0]).unwrap_err(); + } + #[test] fn test_statistics() { let builder = builder_without_changes(FormatVersion::V2); From fbc2d42ecf22c165d5ab85b3c56a686f418aa69c Mon Sep 17 00:00:00 2001 From: Xuanwo Date: Tue, 17 Dec 2024 11:20:26 +0800 Subject: [PATCH 17/23] feat: Store file io props to allow re-build it (#802) * feat: Store file io props to allow re-build it Signed-off-by: Xuanwo * Store file io builder instead Signed-off-by: Xuanwo --------- Signed-off-by: Xuanwo --- crates/iceberg/src/io/file_io.rs | 19 +++++++++++++++---- 1 file changed, 15 insertions(+), 4 deletions(-) diff --git a/crates/iceberg/src/io/file_io.rs b/crates/iceberg/src/io/file_io.rs index 8365d622c..7eaa6aa6d 100644 --- a/crates/iceberg/src/io/file_io.rs +++ b/crates/iceberg/src/io/file_io.rs @@ -43,10 +43,20 @@ use crate::{Error, ErrorKind, Result}; /// | GCS | `storage-gcs` | `gcs` | #[derive(Clone, Debug)] pub struct FileIO { + builder: FileIOBuilder, + inner: Arc, } impl FileIO { + /// Convert FileIO into [`FileIOBuilder`] which used to build this FileIO. + /// + /// This function is useful when you want serialize and deserialize FileIO across + /// distributed systems. + pub fn into_builder(self) -> FileIOBuilder { + self.builder + } + /// Try to infer file io scheme from path. See [`FileIO`] for supported schemes. /// /// - If it's a valid url, for example `s3://bucket/a`, url scheme will be used, and the rest of the url will be ignored. @@ -134,7 +144,7 @@ impl FileIO { } /// Builder for [`FileIO`]. -#[derive(Debug)] +#[derive(Clone, Debug)] pub struct FileIOBuilder { /// This is used to infer scheme of operator. /// @@ -165,7 +175,7 @@ impl FileIOBuilder { /// Fetch the scheme string. /// /// The scheme_str will be empty if it's None. - pub(crate) fn into_parts(self) -> (String, HashMap) { + pub fn into_parts(self) -> (String, HashMap) { (self.scheme_str.unwrap_or_default(), self.props) } @@ -186,9 +196,10 @@ impl FileIOBuilder { } /// Builds [`FileIO`]. - pub fn build(self) -> crate::Result { - let storage = Storage::build(self)?; + pub fn build(self) -> Result { + let storage = Storage::build(self.clone())?; Ok(FileIO { + builder: self, inner: Arc::new(storage), }) } From f9de01b0584d3cd2e894049987f4dc9fd74a8de4 Mon Sep 17 00:00:00 2001 From: Sung Yun <107272191+sungwy@users.noreply.github.com> Date: Mon, 16 Dec 2024 23:12:33 -0500 Subject: [PATCH 18/23] chore: Generate Changelog Dependencies for 0.4.0 release (#812) * dependencies * 0.4.0 changelog * update reelease.md * typos --- CHANGELOG.md | 137 ++++++ bindings/python/DEPENDENCIES.rust.tsv | 307 ++++++++++++ crates/catalog/glue/DEPENDENCIES.rust.tsv | 365 ++++++++------- crates/catalog/hms/DEPENDENCIES.rust.tsv | 361 ++++++++------- crates/catalog/memory/DEPENDENCIES.rust.tsv | 323 +++++++------ crates/catalog/rest/DEPENDENCIES.rust.tsv | 325 +++++++------ crates/catalog/sql/DEPENDENCIES.rust.tsv | 347 +++++++------- crates/iceberg/DEPENDENCIES.rust.tsv | 323 +++++++------ .../datafusion/DEPENDENCIES.rust.tsv | 435 +++++++++--------- scripts/dependencies.py | 5 +- website/src/release.md | 2 +- 11 files changed, 1780 insertions(+), 1150 deletions(-) create mode 100644 bindings/python/DEPENDENCIES.rust.tsv diff --git a/CHANGELOG.md b/CHANGELOG.md index fc576c52f..7c3a3d68b 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -24,6 +24,143 @@ All notable changes to this project will be documented in this file. The format is based on [Keep a Changelog](https://keepachangelog.com/) and this project adheres to [Semantic Versioning](https://semver.org/). +## [v0.4.0] - 2024-12-16 +* io: add support for role arn and external id s3 props by @mattheusv in https://github.com/apache/iceberg-rust/pull/553 +* fix: ensure S3 and GCS integ tests are conditionally compiled only when the storage-s3 and storage-gcs features are enabled by @sdd in https://github.com/apache/iceberg-rust/pull/552 +* docs: fix main iceberg example by @jdockerty in https://github.com/apache/iceberg-rust/pull/554 +* io: add support to set assume role session name by @mattheusv in https://github.com/apache/iceberg-rust/pull/555 +* test: refactor datafusion test with memory catalog by @FANNG1 in https://github.com/apache/iceberg-rust/pull/557 +* chore: add clean job in Makefile by @ChinoUkaegbu in https://github.com/apache/iceberg-rust/pull/561 +* docs: Fix build website permission changed by @Xuanwo in https://github.com/apache/iceberg-rust/pull/564 +* Object Cache: caches parsed Manifests and ManifestLists for performance by @sdd in https://github.com/apache/iceberg-rust/pull/512 +* Update the paths by @Fokko in https://github.com/apache/iceberg-rust/pull/569 +* docs: Add links for released crates by @Xuanwo in https://github.com/apache/iceberg-rust/pull/570 +* Python: Use hatch for dependency management by @sungwy in https://github.com/apache/iceberg-rust/pull/572 +* Ensure that RestCatalog passes user config to FileIO by @sdd in https://github.com/apache/iceberg-rust/pull/476 +* Move `zlib` and `unicode` licenses to `allow` by @Fokko in https://github.com/apache/iceberg-rust/pull/566 +* website: Update links for 0.3.0 by @Xuanwo in https://github.com/apache/iceberg-rust/pull/573 +* feat(timestamp_ns): Implement timestamps with nanosecond precision by @Sl1mb0 in https://github.com/apache/iceberg-rust/pull/542 +* fix: correct partition-id to field-id in UnboundPartitionField by @FANNG1 in https://github.com/apache/iceberg-rust/pull/576 +* fix: Update sqlx from 0.8.0 to 0.8.1 by @FANNG1 in https://github.com/apache/iceberg-rust/pull/584 +* chore(deps): Update typed-builder requirement from 0.19 to 0.20 by @dependabot in https://github.com/apache/iceberg-rust/pull/582 +* Expose Transforms to Python Binding by @sungwy in https://github.com/apache/iceberg-rust/pull/556 +* chore(deps): Bump crate-ci/typos from 1.23.6 to 1.24.1 by @dependabot in https://github.com/apache/iceberg-rust/pull/583 +* Table Scan: Add Row Group Skipping by @sdd in https://github.com/apache/iceberg-rust/pull/558 +* chore: bump crate-ci/typos to 1.24.3 by @sdlarsen in https://github.com/apache/iceberg-rust/pull/598 +* feat: SQL Catalog - namespaces by @callum-ryan in https://github.com/apache/iceberg-rust/pull/534 +* feat: Add more fields in FileScanTask by @Xuanwo in https://github.com/apache/iceberg-rust/pull/609 +* chore(deps): Bump crate-ci/typos from 1.24.3 to 1.24.5 by @dependabot in https://github.com/apache/iceberg-rust/pull/616 +* fix: Less Panics for Snapshot timestamps by @c-thiel in https://github.com/apache/iceberg-rust/pull/614 +* feat: partition compatibility by @c-thiel in https://github.com/apache/iceberg-rust/pull/612 +* feat: SortOrder methods should take schema ref if possible by @c-thiel in https://github.com/apache/iceberg-rust/pull/613 +* feat: add `client.region` by @jdockerty in https://github.com/apache/iceberg-rust/pull/623 +* fix: Correctly calculate highest_field_id in schema by @liurenjie1024 in https://github.com/apache/iceberg-rust/pull/590 +* Feat: Normalize TableMetadata by @c-thiel in https://github.com/apache/iceberg-rust/pull/611 +* refactor(python): Expose transform as a submodule for pyiceberg_core by @Xuanwo in https://github.com/apache/iceberg-rust/pull/628 +* feat: support projection pushdown for datafusion iceberg by @FANNG1 in https://github.com/apache/iceberg-rust/pull/594 +* chore: Bump opendal to 0.50 by @Xuanwo in https://github.com/apache/iceberg-rust/pull/634 +* feat: add Sync to TransformFunction by @xxchan in https://github.com/apache/iceberg-rust/pull/638 +* feat: expose arrow type <-> iceberg type by @xxchan in https://github.com/apache/iceberg-rust/pull/637 +* doc: improve FileIO doc by @xxchan in https://github.com/apache/iceberg-rust/pull/642 +* chore(deps): Bump crate-ci/typos from 1.24.5 to 1.24.6 by @dependabot in https://github.com/apache/iceberg-rust/pull/640 +* Migrate to arrow-* v53 by @sdd in https://github.com/apache/iceberg-rust/pull/626 +* feat: expose remove_all in FileIO by @xxchan in https://github.com/apache/iceberg-rust/pull/643 +* feat (datafusion integration): convert datafusion expr filters to Iceberg Predicate by @a-agmon in https://github.com/apache/iceberg-rust/pull/588 +* feat: Add NamespaceIdent.parent() by @c-thiel in https://github.com/apache/iceberg-rust/pull/641 +* Table Scan: Add Row Selection Filtering by @sdd in https://github.com/apache/iceberg-rust/pull/565 +* fix: compile error due to merge stale PR by @xxchan in https://github.com/apache/iceberg-rust/pull/646 +* scan: change ErrorKind when table dont have spanshots by @mattheusv in https://github.com/apache/iceberg-rust/pull/608 +* fix: avoid to create operator of memory storage every time by @ZENOTME in https://github.com/apache/iceberg-rust/pull/635 +* feat (datafusion): making IcebergTableProvider public to be used without a catalog by @a-agmon in https://github.com/apache/iceberg-rust/pull/650 +* test (datafusion): add test for table provider creation by @a-agmon in https://github.com/apache/iceberg-rust/pull/651 +* fix: page index evaluator min/max args inverted by @sdd in https://github.com/apache/iceberg-rust/pull/648 +* chore: fix typo in FileIO Schemes by @wcy-fdu in https://github.com/apache/iceberg-rust/pull/653 +* fix: TableUpdate Snapshot deserialization for v1 by @c-thiel in https://github.com/apache/iceberg-rust/pull/656 +* feat: Reassign field ids for schema by @c-thiel in https://github.com/apache/iceberg-rust/pull/615 +* feat: add gcp oauth support by @twuebi in https://github.com/apache/iceberg-rust/pull/654 +* fix(arrow): Use new ParquetMetaDataReader instead by @Xuanwo in https://github.com/apache/iceberg-rust/pull/661 +* chore(deps): bump typos crate to 1.25.0 by @matthewwillian in https://github.com/apache/iceberg-rust/pull/662 +* RecordBatchTransformer: Handle schema migration and column re-ordering in table scans by @sdd in https://github.com/apache/iceberg-rust/pull/602 +* docs: installation of the new `iceberg_catalog_rest` added to the docs by @nishant-sachdeva in https://github.com/apache/iceberg-rust/pull/355 +* feat(datafusion): Support pushdown more datafusion exprs to Iceberg by @FANNG1 in https://github.com/apache/iceberg-rust/pull/649 +* feat: Derive PartialEq for FileScanTask by @Xuanwo in https://github.com/apache/iceberg-rust/pull/660 +* feat: SQL Catalog - Tables by @callum-ryan in https://github.com/apache/iceberg-rust/pull/610 +* ci: Allow install a non-debian-packaged Python package by @Xuanwo in https://github.com/apache/iceberg-rust/pull/666 +* docs: README uses iceberg-rust instead of we by @caicancai in https://github.com/apache/iceberg-rust/pull/667 +* chore(deps): Bump crate-ci/typos from 1.25.0 to 1.26.0 by @dependabot in https://github.com/apache/iceberg-rust/pull/668 +* feat: Add equality delete writer by @Dysprosium0626 in https://github.com/apache/iceberg-rust/pull/372 +* Revert "feat: Add equality delete writer (#372)" by @Xuanwo in https://github.com/apache/iceberg-rust/pull/672 +* ci: Fix CI for bindings python by @Xuanwo in https://github.com/apache/iceberg-rust/pull/678 +* fix: OpenDAL `is_exist` => `exists` by @sdd in https://github.com/apache/iceberg-rust/pull/680 +* feat: Expose ManifestEntry status by @zheilbron in https://github.com/apache/iceberg-rust/pull/681 +* feat: allow empty projection in table scan by @sundy-li in https://github.com/apache/iceberg-rust/pull/677 +* chore(deps): Bump crate-ci/typos from 1.26.0 to 1.26.8 by @dependabot in https://github.com/apache/iceberg-rust/pull/683 +* fix: bump parquet minor version by @xxchan in https://github.com/apache/iceberg-rust/pull/684 +* fix(type): fix type promote to ignore field name. by @chenzl25 in https://github.com/apache/iceberg-rust/pull/685 +* feat: implement IcebergTableProviderFactory for datafusion by @yukkit in https://github.com/apache/iceberg-rust/pull/600 +* feat: Safer PartitionSpec & SchemalessPartitionSpec by @c-thiel in https://github.com/apache/iceberg-rust/pull/645 +* chore(deps): Bump crate-ci/typos from 1.26.8 to 1.27.0 by @dependabot in https://github.com/apache/iceberg-rust/pull/687 +* feat: TableMetadata accessors for current ids of Schema, Snapshot and SortOrder by @c-thiel in https://github.com/apache/iceberg-rust/pull/688 +* chore: upgrade to DataFusion 43 by @gruuya in https://github.com/apache/iceberg-rust/pull/691 +* chore(deps): Bump crate-ci/typos from 1.27.0 to 1.27.3 by @dependabot in https://github.com/apache/iceberg-rust/pull/693 +* feat: Expose length of Iterators by @c-thiel in https://github.com/apache/iceberg-rust/pull/692 +* feat: Implement TableRequirement checks by @c-thiel in https://github.com/apache/iceberg-rust/pull/689 +* feat: Add ViewUpdate to catalog by @c-thiel in https://github.com/apache/iceberg-rust/pull/690 +* chore: update .asf.yaml by @c-thiel in https://github.com/apache/iceberg-rust/pull/701 +* datafusion: Create table provider for a snapshot. by @ryzhyk in https://github.com/apache/iceberg-rust/pull/707 +* Add Python Release Action to publish `pyiceberg_core` dist to Pypi by @sungwy in https://github.com/apache/iceberg-rust/pull/705 +* chore: Mark `last-field-id` as deprecated by @Fokko in https://github.com/apache/iceberg-rust/pull/715 +* TableMetadataBuilder by @c-thiel in https://github.com/apache/iceberg-rust/pull/587 +* Add `fallback` attribute to all `strip_option`s. by @ryzhyk in https://github.com/apache/iceberg-rust/pull/708 +* fix: Remove check of last_column_id by @liurenjie1024 in https://github.com/apache/iceberg-rust/pull/717 +* Fix error running data fusion queries - Physical input schema should be the same as the one converted from logical input schema by @FANNG1 in https://github.com/apache/iceberg-rust/pull/664 +* chore: Typo in test :) by @Fokko in https://github.com/apache/iceberg-rust/pull/727 +* Derive Clone for IcebergTableProvider by @SergeiPatiakin in https://github.com/apache/iceberg-rust/pull/722 +* fix: expand arrow to iceberg schema to handle nanosecond timestamp by @jdockerty in https://github.com/apache/iceberg-rust/pull/710 +* feat: Add equality delete writer by @ZENOTME in https://github.com/apache/iceberg-rust/pull/703 +* chore: Bump upload-artifact@v3 to v4 by @sungwy in https://github.com/apache/iceberg-rust/pull/725 +* feat: support append data file and add e2e test by @ZENOTME in https://github.com/apache/iceberg-rust/pull/349 +* chore(deps): Bump actions/setup-python from 4 to 5 by @dependabot in https://github.com/apache/iceberg-rust/pull/746 +* chore: Align argument name with doc comment by @SergeiPatiakin in https://github.com/apache/iceberg-rust/pull/750 +* fix: equality delete writer field id project by @ZENOTME in https://github.com/apache/iceberg-rust/pull/751 +* feat: expose opendal S3 options for anonymous access by @gruuya in https://github.com/apache/iceberg-rust/pull/757 +* fix: current-snapshot-id serialized to -1 in TableMetadata.json by @c-thiel in https://github.com/apache/iceberg-rust/pull/755 +* feat(puffin): Add Puffin crate and CompressionCodec by @fqaiser94 in https://github.com/apache/iceberg-rust/pull/745 +* Build: Delete branch automatically on PR merge by @manuzhang in https://github.com/apache/iceberg-rust/pull/764 +* chore(deps): bump crate-ci/typos from 1.27.3 to 1.28.1 by @jonathanc-n in https://github.com/apache/iceberg-rust/pull/769 +* chore(deps): Bump crate-ci/typos from 1.27.3 to 1.28.2 by @dependabot in https://github.com/apache/iceberg-rust/pull/767 +* Add Spark for integration tests by @Fokko in https://github.com/apache/iceberg-rust/pull/766 +* refine: refine writer interface by @ZENOTME in https://github.com/apache/iceberg-rust/pull/741 +* Clean up docker Docker by @Fokko in https://github.com/apache/iceberg-rust/pull/770 +* name mapping serde by @barronw in https://github.com/apache/iceberg-rust/pull/740 +* docker: The `archive` seems unstable by @Fokko in https://github.com/apache/iceberg-rust/pull/773 +* doc: add RisingWave to users by @xxchan in https://github.com/apache/iceberg-rust/pull/775 +* feat: Expose disable_config_load opendal S3 option by @gruuya in https://github.com/apache/iceberg-rust/pull/782 +* Support conversion of Arrow Int8 and Int16 to Iceberg Int by @gruuya in https://github.com/apache/iceberg-rust/pull/787 +* infra: Dismiss stale reviews by @liurenjie1024 in https://github.com/apache/iceberg-rust/pull/779 +* fix: return type for year and month transform should be int by @xxchan in https://github.com/apache/iceberg-rust/pull/776 +* feat: Allow for schema evolution by @Fokko in https://github.com/apache/iceberg-rust/pull/786 +* Retry object store reads on temporary errors. by @ryzhyk in https://github.com/apache/iceberg-rust/pull/788 +* refactor(puffin): Move puffin crate contents inside iceberg crate by @fqaiser94 in https://github.com/apache/iceberg-rust/pull/789 +* feat: Implement Decimal from/to bytes represents by @Xuanwo in https://github.com/apache/iceberg-rust/pull/665 +* feat: eagerly project the arrow schema to scope out non-selected fields by @gruuya in https://github.com/apache/iceberg-rust/pull/785 +* fix: wrong compute of partitions in manifest by @ZENOTME in https://github.com/apache/iceberg-rust/pull/794 +* fix: set key_metadata to Null by default by @feniljain in https://github.com/apache/iceberg-rust/pull/800 +* test: append partition data file by @feniljain in https://github.com/apache/iceberg-rust/pull/742 +* chore: Add more debug message inside error by @Xuanwo in https://github.com/apache/iceberg-rust/pull/793 +* fix: Error source from cache has been shadowed by @Xuanwo in https://github.com/apache/iceberg-rust/pull/792 +* fix(catalog/rest): Ensure token been reused correctly by @Xuanwo in https://github.com/apache/iceberg-rust/pull/801 +* feat!: Remove `BoundPartitionSpec` by @c-thiel in https://github.com/apache/iceberg-rust/pull/771 +* chore(deps): Bump crate-ci/typos from 1.28.2 to 1.28.3 by @dependabot in https://github.com/apache/iceberg-rust/pull/805 +* feat: add `DataFileWriter` tests for schema and partition by @jonathanc-n in https://github.com/apache/iceberg-rust/pull/768 +* fix: day transform compute by @ZENOTME in https://github.com/apache/iceberg-rust/pull/796 +* feat: TableMetadata Statistic Files by @c-thiel in https://github.com/apache/iceberg-rust/pull/799 +* Bump `pyiceberg_core` to 0.4.0 by @sungwy in https://github.com/apache/iceberg-rust/pull/808 +* chore(docs): Update Readme - Lakekeeper repository moved by @c-thiel in https://github.com/apache/iceberg-rust/pull/810 +* Prep 0.4.0 release by @sungwy in https://github.com/apache/iceberg-rust/pull/809 +* feat: Add RemovePartitionSpecs table update by @c-thiel in https://github.com/apache/iceberg-rust/pull/804 +* feat: Store file io props to allow re-build it by @Xuanwo in https://github.com/apache/iceberg-rust/pull/802 + ## [v0.3.0] - 2024-08-14 * Smooth out release steps by @Fokko in https://github.com/apache/iceberg-rust/pull/197 diff --git a/bindings/python/DEPENDENCIES.rust.tsv b/bindings/python/DEPENDENCIES.rust.tsv new file mode 100644 index 000000000..c24db257a --- /dev/null +++ b/bindings/python/DEPENDENCIES.rust.tsv @@ -0,0 +1,307 @@ +crate 0BSD Apache-2.0 Apache-2.0 WITH LLVM-exception BSD-2-Clause BSD-3-Clause BSL-1.0 CC0-1.0 ISC MIT MPL-2.0 OpenSSL Unicode-DFS-2016 Unlicense Zlib +addr2line@0.22.0 X X +adler@1.0.2 X X X +adler32@1.2.0 X +ahash@0.8.11 X X +aho-corasick@1.1.3 X X +alloc-no-stdlib@2.0.4 X +alloc-stdlib@0.2.2 X +allocator-api2@0.2.18 X X +android-tzdata@0.1.1 X X +android_system_properties@0.1.5 X X +anyhow@1.0.86 X X +apache-avro@0.17.0 X +array-init@2.1.0 X X +arrayvec@0.7.4 X X +arrow@53.2.0 X +arrow-arith@53.2.0 X +arrow-array@53.2.0 X +arrow-buffer@53.2.0 X +arrow-cast@53.2.0 X +arrow-csv@53.2.0 X +arrow-data@53.2.0 X +arrow-ipc@53.2.0 X +arrow-json@53.2.0 X +arrow-ord@53.2.0 X +arrow-row@53.2.0 X +arrow-schema@53.2.0 X +arrow-select@53.2.0 X +arrow-string@53.2.0 X +async-lock@3.4.0 X X +async-trait@0.1.81 X X +atoi@2.0.0 X +autocfg@1.3.0 X X +backon@1.2.0 X +backtrace@0.3.73 X X +base64@0.22.1 X X +bigdecimal@0.4.5 X X +bimap@0.6.3 X X +bitflags@1.3.2 X X +bitflags@2.6.0 X X +bitvec@1.0.1 X +block-buffer@0.10.4 X X +brotli@7.0.0 X X +brotli-decompressor@4.0.1 X X +bumpalo@3.16.0 X X +byteorder@1.5.0 X X +bytes@1.7.1 X +cc@1.1.7 X X +cfg-if@1.0.0 X X +chrono@0.4.38 X X +concurrent-queue@2.5.0 X X +const-oid@0.9.6 X X +const-random@0.1.18 X X +const-random-macro@0.1.16 X X +core-foundation-sys@0.8.6 X X +core2@0.4.0 X X +cpufeatures@0.2.12 X X +crc32c@0.6.8 X X +crc32fast@1.4.2 X X +crossbeam-channel@0.5.13 X X +crossbeam-epoch@0.9.18 X X +crossbeam-utils@0.8.20 X X +crunchy@0.2.2 X +crypto-common@0.1.6 X X +csv@1.3.0 X X +csv-core@0.1.11 X X +darling@0.20.10 X +darling_core@0.20.10 X +darling_macro@0.20.10 X +dary_heap@0.3.6 X X +derive_builder@0.20.0 X X +derive_builder_core@0.20.0 X X +derive_builder_macro@0.20.0 X X +digest@0.10.7 X X +either@1.13.0 X X +equivalent@1.0.1 X X +event-listener@5.3.1 X X +event-listener-strategy@0.5.2 X X +fastrand@2.1.0 X X +flagset@0.4.6 X +flatbuffers@24.3.25 X +flate2@1.0.31 X X +fnv@1.0.7 X X +form_urlencoded@1.2.1 X X +funty@2.0.0 X +futures@0.3.30 X X +futures-channel@0.3.30 X X +futures-core@0.3.30 X X +futures-executor@0.3.30 X X +futures-io@0.3.30 X X +futures-macro@0.3.30 X X +futures-sink@0.3.30 X X +futures-task@0.3.30 X X +futures-util@0.3.30 X X +generic-array@0.14.7 X +getrandom@0.2.15 X X +gimli@0.29.0 X X +gloo-timers@0.3.0 X X +half@2.4.1 X X +hashbrown@0.14.5 X X +heck@0.5.0 X X +hermit-abi@0.3.9 X X +hex@0.4.3 X X +hmac@0.12.1 X X +home@0.5.9 X X +http@1.1.0 X X +http-body@1.0.1 X +http-body-util@0.1.2 X +httparse@1.9.4 X X +hyper@1.4.1 X +hyper-rustls@0.27.2 X X X +hyper-util@0.1.6 X +iana-time-zone@0.1.60 X X +iana-time-zone-haiku@0.1.2 X X +iceberg@0.4.0 X +ident_case@1.0.1 X X +idna@0.5.0 X X +indexmap@2.3.0 X X +indoc@2.0.5 X X +integer-encoding@3.0.4 X +ipnet@2.9.0 X X +itertools@0.13.0 X X +itoa@1.0.11 X X +jobserver@0.1.32 X X +js-sys@0.3.69 X X +lazy_static@1.5.0 X X +lexical-core@1.0.2 X X +lexical-parse-float@1.0.2 X X +lexical-parse-integer@1.0.2 X X +lexical-util@1.0.3 X X +lexical-write-float@1.0.2 X X +lexical-write-integer@1.0.2 X X +libc@0.2.155 X X +libflate@2.1.0 X +libflate_lz77@2.1.0 X +libm@0.2.8 X X +lock_api@0.4.12 X X +log@0.4.22 X X +lz4_flex@0.11.3 X +md-5@0.10.6 X X +memchr@2.7.4 X X +memoffset@0.9.1 X +mime@0.3.17 X X +miniz_oxide@0.7.4 X X X +mio@1.0.1 X +moka@0.12.8 X X +murmur3@0.5.2 X X +num@0.4.3 X X +num-bigint@0.4.6 X X +num-complex@0.4.6 X X +num-integer@0.1.46 X X +num-iter@0.1.45 X X +num-rational@0.4.2 X X +num-traits@0.2.19 X X +object@0.36.2 X X +once_cell@1.19.0 X X +opendal@0.50.2 X +ordered-float@2.10.1 X +ordered-float@4.2.2 X +parking@2.2.1 X X +parking_lot@0.12.3 X X +parking_lot_core@0.9.10 X X +parquet@53.2.0 X +paste@1.0.15 X X +percent-encoding@2.3.1 X X +pin-project@1.1.5 X X +pin-project-internal@1.1.5 X X +pin-project-lite@0.2.14 X X +pin-utils@0.1.0 X X +pkg-config@0.3.30 X X +portable-atomic@1.7.0 X X +ppv-lite86@0.2.20 X X +proc-macro2@1.0.86 X X +pyiceberg_core_rust@0.4.0 X +pyo3@0.22.6 X X +pyo3-build-config@0.22.6 X X +pyo3-ffi@0.22.6 X X +pyo3-macros@0.22.6 X X +pyo3-macros-backend@0.22.6 X X +quad-rand@0.2.1 X +quanta@0.12.3 X +quick-xml@0.36.1 X +quote@1.0.36 X X +radium@0.7.0 X +rand@0.8.5 X X +rand_chacha@0.3.1 X X +rand_core@0.6.4 X X +raw-cpuid@11.2.0 X +redox_syscall@0.5.3 X +regex@1.10.6 X X +regex-automata@0.4.7 X X +regex-lite@0.1.6 X X +regex-syntax@0.8.4 X X +reqsign@0.16.1 X +reqwest@0.12.5 X X +ring@0.17.8 X +rle-decode-fast@1.0.3 X X +rust_decimal@1.35.0 X +rustc-demangle@0.1.24 X X +rustc_version@0.4.0 X X +rustls@0.23.12 X X X +rustls-pemfile@2.1.3 X X X +rustls-pki-types@1.7.0 X X +rustls-webpki@0.102.6 X +rustversion@1.0.17 X X +ryu@1.0.18 X X +scopeguard@1.2.0 X X +semver@1.0.23 X X +seq-macro@0.3.5 X X +serde@1.0.204 X X +serde_bytes@0.11.15 X X +serde_derive@1.0.204 X X +serde_json@1.0.122 X X +serde_repr@0.1.19 X X +serde_urlencoded@0.7.1 X X +serde_with@3.9.0 X X +serde_with_macros@3.9.0 X X +sha1@0.10.6 X X +sha2@0.10.8 X X +slab@0.4.9 X +smallvec@1.13.2 X X +snap@1.1.1 X +socket2@0.5.7 X X +spin@0.9.8 X +static_assertions@1.1.0 X X +strsim@0.11.1 X +strum@0.26.3 X +strum_macros@0.26.4 X +subtle@2.6.1 X +syn@2.0.72 X X +sync_wrapper@1.0.1 X +tagptr@0.2.0 X X +tap@1.0.1 X +target-lexicon@0.12.16 X +thiserror@1.0.63 X X +thiserror-impl@1.0.63 X X +thrift@0.17.0 X +tiny-keccak@2.0.2 X +tinyvec@1.8.0 X X X +tinyvec_macros@0.1.1 X X X +tokio@1.39.2 X +tokio-macros@2.4.0 X +tokio-rustls@0.26.0 X X +tokio-util@0.7.11 X +tower@0.4.13 X +tower-layer@0.3.2 X +tower-service@0.3.2 X +tracing@0.1.40 X +tracing-core@0.1.32 X +triomphe@0.1.11 X X +try-lock@0.2.5 X +twox-hash@1.6.3 X +typed-builder@0.19.1 X X +typed-builder@0.20.0 X X +typed-builder-macro@0.19.1 X X +typed-builder-macro@0.20.0 X X +typenum@1.17.0 X X +unicode-bidi@0.3.15 X X +unicode-ident@1.0.12 X X X +unicode-normalization@0.1.23 X X +unindent@0.2.3 X X +untrusted@0.9.0 X +url@2.5.2 X X +uuid@1.10.0 X X +version_check@0.9.5 X X +want@0.3.1 X +wasi@0.11.0+wasi-snapshot-preview1 X X X +wasm-bindgen@0.2.92 X X +wasm-bindgen-backend@0.2.92 X X +wasm-bindgen-futures@0.4.42 X X +wasm-bindgen-macro@0.2.92 X X +wasm-bindgen-macro-support@0.2.92 X X +wasm-bindgen-shared@0.2.92 X X +wasm-streams@0.4.0 X X +web-sys@0.3.69 X X +webpki-roots@0.26.3 X +winapi@0.3.9 X X +winapi-i686-pc-windows-gnu@0.4.0 X X +winapi-x86_64-pc-windows-gnu@0.4.0 X X +windows-core@0.52.0 X X +windows-sys@0.48.0 X X +windows-sys@0.52.0 X X +windows-targets@0.48.5 X X +windows-targets@0.52.6 X X +windows_aarch64_gnullvm@0.48.5 X X +windows_aarch64_gnullvm@0.52.6 X X +windows_aarch64_msvc@0.48.5 X X +windows_aarch64_msvc@0.52.6 X X +windows_i686_gnu@0.48.5 X X +windows_i686_gnu@0.52.6 X X +windows_i686_gnullvm@0.52.6 X X +windows_i686_msvc@0.48.5 X X +windows_i686_msvc@0.52.6 X X +windows_x86_64_gnu@0.48.5 X X +windows_x86_64_gnu@0.52.6 X X +windows_x86_64_gnullvm@0.48.5 X X +windows_x86_64_gnullvm@0.52.6 X X +windows_x86_64_msvc@0.48.5 X X +windows_x86_64_msvc@0.52.6 X X +winreg@0.52.0 X +wyz@0.5.1 X +zerocopy@0.7.35 X X X +zerocopy-derive@0.7.35 X X X +zeroize@1.8.1 X X +zstd@0.13.2 X +zstd-safe@7.2.1 X X +zstd-sys@2.0.12+zstd.1.5.6 X X diff --git a/crates/catalog/glue/DEPENDENCIES.rust.tsv b/crates/catalog/glue/DEPENDENCIES.rust.tsv index 735d5447b..007c54f35 100644 --- a/crates/catalog/glue/DEPENDENCIES.rust.tsv +++ b/crates/catalog/glue/DEPENDENCIES.rust.tsv @@ -1,171 +1,195 @@ -crate 0BSD Apache-2.0 Apache-2.0 WITH LLVM-exception BSD-2-Clause BSD-3-Clause BSL-1.0 CC0-1.0 ISC MIT MPL-2.0 OpenSSL Unicode-DFS-2016 Unlicense Zlib -addr2line@0.22.0 X X -adler@1.0.2 X X X +crate 0BSD Apache-2.0 Apache-2.0 WITH LLVM-exception BSD-2-Clause BSD-3-Clause BSL-1.0 CC0-1.0 ISC MIT MPL-2.0 OpenSSL Unicode-3.0 Unlicense Zlib +addr2line@0.24.2 X X +adler2@2.0.0 X X X adler32@1.2.0 X ahash@0.8.11 X X aho-corasick@1.1.3 X X alloc-no-stdlib@2.0.4 X alloc-stdlib@0.2.2 X -allocator-api2@0.2.18 X X +allocator-api2@0.2.21 X X android-tzdata@0.1.1 X X android_system_properties@0.1.5 X X -anstream@0.6.15 X X -anstyle@1.0.8 X X -anstyle-parse@0.2.5 X X -anstyle-query@1.1.1 X X -anstyle-wincon@3.0.4 X X -anyhow@1.0.86 X X +anstream@0.6.18 X X +anstyle@1.0.10 X X +anstyle-parse@0.2.6 X X +anstyle-query@1.1.2 X X +anstyle-wincon@3.0.6 X X +anyhow@1.0.94 X X apache-avro@0.17.0 X array-init@2.1.0 X X -arrayvec@0.7.4 X X -arrow-arith@52.2.0 X -arrow-array@52.2.0 X -arrow-buffer@52.2.0 X -arrow-cast@52.2.0 X -arrow-data@52.2.0 X -arrow-ipc@52.2.0 X -arrow-ord@52.2.0 X -arrow-schema@52.2.0 X -arrow-select@52.2.0 X -arrow-string@52.2.0 X -async-trait@0.1.81 X X +arrayvec@0.7.6 X X +arrow-arith@53.3.0 X +arrow-array@53.3.0 X +arrow-buffer@53.3.0 X +arrow-cast@53.3.0 X +arrow-data@53.3.0 X +arrow-ipc@53.3.0 X +arrow-ord@53.3.0 X +arrow-schema@53.3.0 X +arrow-select@53.3.0 X +arrow-string@53.3.0 X +async-lock@3.4.0 X X +async-trait@0.1.83 X X atoi@2.0.0 X -autocfg@1.3.0 X X -aws-config@1.5.5 X -aws-credential-types@1.2.0 X -aws-runtime@1.4.0 X -aws-sdk-glue@1.53.0 X -aws-sdk-sso@1.37.0 X -aws-sdk-ssooidc@1.38.0 X -aws-sdk-sts@1.37.0 X -aws-sigv4@1.2.3 X -aws-smithy-async@1.2.1 X -aws-smithy-http@0.60.9 X +autocfg@1.4.0 X X +aws-config@1.5.10 X +aws-credential-types@1.2.1 X +aws-runtime@1.4.4 X +aws-sdk-glue@1.72.0 X +aws-sdk-sso@1.50.0 X +aws-sdk-ssooidc@1.51.0 X +aws-sdk-sts@1.51.0 X +aws-sigv4@1.2.6 X +aws-smithy-async@1.2.2 X +aws-smithy-http@0.60.11 X aws-smithy-json@0.60.7 X +aws-smithy-json@0.61.1 X aws-smithy-query@0.60.7 X -aws-smithy-runtime@1.6.2 X -aws-smithy-runtime-api@1.7.2 X -aws-smithy-types@1.2.0 X -aws-smithy-xml@0.60.8 X +aws-smithy-runtime@1.7.5 X +aws-smithy-runtime-api@1.7.3 X +aws-smithy-types@1.2.10 X +aws-smithy-xml@0.60.9 X aws-types@1.3.3 X -backon@0.4.4 X -backtrace@0.3.73 X X +backon@1.3.0 X +backtrace@0.3.74 X X base64@0.21.7 X X base64@0.22.1 X X base64-simd@0.8.0 X -bigdecimal@0.4.5 X X +bigdecimal@0.4.7 X X bimap@0.6.3 X X bitflags@1.3.2 X X bitflags@2.6.0 X X bitvec@1.0.1 X block-buffer@0.10.4 X X -brotli@6.0.0 X X +brotli@7.0.0 X X brotli-decompressor@4.0.1 X X bumpalo@3.16.0 X X byteorder@1.5.0 X X -bytes@1.7.1 X +bytes@1.9.0 X bytes-utils@0.1.4 X X -cc@1.1.11 X X +cc@1.2.4 X X cfg-if@1.0.0 X X -chrono@0.4.38 X X -colorchoice@1.0.2 X X +chrono@0.4.39 X X +colorchoice@1.0.3 X X +concurrent-queue@2.5.0 X X const-oid@0.9.6 X X const-random@0.1.18 X X const-random-macro@0.1.16 X X core-foundation@0.9.4 X X core-foundation-sys@0.8.7 X X core2@0.4.0 X X -cpufeatures@0.2.13 X X +cpufeatures@0.2.16 X X crc32c@0.6.8 X X crc32fast@1.4.2 X X +crossbeam-channel@0.5.14 X X +crossbeam-epoch@0.9.18 X X +crossbeam-utils@0.8.21 X X crunchy@0.2.2 X crypto-common@0.1.6 X X darling@0.20.10 X darling_core@0.20.10 X darling_macro@0.20.10 X -dary_heap@0.3.6 X X +dary_heap@0.3.7 X X deranged@0.3.11 X X -derive_builder@0.20.0 X X -derive_builder_core@0.20.0 X X -derive_builder_macro@0.20.0 X X +derive_builder@0.20.2 X X +derive_builder_core@0.20.2 X X +derive_builder_macro@0.20.2 X X digest@0.10.7 X X +displaydoc@0.2.5 X X either@1.13.0 X X env_filter@0.1.2 X X env_logger@0.11.5 X X equivalent@1.0.1 X X -fastrand@2.1.0 X X +event-listener@5.3.1 X X +event-listener-strategy@0.5.3 X X +fastrand@2.3.0 X X flagset@0.4.6 X flatbuffers@24.3.25 X -flate2@1.0.31 X X +flate2@1.0.35 X X fnv@1.0.7 X X form_urlencoded@1.2.1 X X funty@2.0.0 X -futures@0.3.30 X X -futures-channel@0.3.30 X X -futures-core@0.3.30 X X -futures-executor@0.3.30 X X -futures-io@0.3.30 X X -futures-macro@0.3.30 X X -futures-sink@0.3.30 X X -futures-task@0.3.30 X X -futures-util@0.3.30 X X +futures@0.3.31 X X +futures-channel@0.3.31 X X +futures-core@0.3.31 X X +futures-executor@0.3.31 X X +futures-io@0.3.31 X X +futures-macro@0.3.31 X X +futures-sink@0.3.31 X X +futures-task@0.3.31 X X +futures-util@0.3.31 X X generic-array@0.14.7 X getrandom@0.2.15 X X -gimli@0.29.0 X X +gimli@0.31.1 X X +gloo-timers@0.3.0 X X h2@0.3.26 X half@2.4.1 X X hashbrown@0.14.5 X X +hashbrown@0.15.2 X X heck@0.5.0 X X -hermit-abi@0.3.9 X X hex@0.4.3 X X hmac@0.12.1 X X -home@0.5.9 X X +home@0.5.11 X X http@0.2.12 X X -http@1.1.0 X X +http@1.2.0 X X http-body@0.4.6 X http-body@1.0.1 X http-body-util@0.1.2 X -httparse@1.9.4 X X +httparse@1.9.5 X X httpdate@1.0.3 X X humantime@2.1.0 X X -hyper@0.14.30 X -hyper@1.4.1 X +hyper@0.14.32 X +hyper@1.5.2 X hyper-rustls@0.24.2 X X X -hyper-rustls@0.27.2 X X X -hyper-util@0.1.7 X -iana-time-zone@0.1.60 X X +hyper-rustls@0.27.3 X X X +hyper-util@0.1.10 X +iana-time-zone@0.1.61 X X iana-time-zone-haiku@0.1.2 X X -iceberg@0.3.0 X -iceberg-catalog-glue@0.3.0 X -iceberg-catalog-memory@0.3.0 X -iceberg_test_utils@0.3.0 X +iceberg@0.4.0 X +iceberg-catalog-glue@0.4.0 X +iceberg-catalog-memory@0.4.0 X +iceberg_test_utils@0.4.0 X +icu_collections@1.5.0 X +icu_locid@1.5.0 X +icu_locid_transform@1.5.0 X +icu_locid_transform_data@1.5.0 X +icu_normalizer@1.5.0 X +icu_normalizer_data@1.5.0 X +icu_properties@1.5.1 X +icu_properties_data@1.5.0 X +icu_provider@1.5.0 X +icu_provider_macros@1.5.0 X ident_case@1.0.1 X X -idna@0.5.0 X X -indexmap@2.4.0 X X +idna@1.0.3 X X +idna_adapter@1.2.0 X X +indexmap@2.7.0 X X integer-encoding@3.0.4 X -ipnet@2.9.0 X X +ipnet@2.10.1 X X is_terminal_polyfill@1.70.1 X X itertools@0.13.0 X X -itoa@1.0.11 X X +itoa@1.0.14 X X jobserver@0.1.32 X X -js-sys@0.3.70 X X -lexical-core@0.8.5 X X -lexical-parse-float@0.8.5 X X -lexical-parse-integer@0.8.6 X X -lexical-util@0.8.5 X X -lexical-write-float@0.8.5 X X -lexical-write-integer@0.8.5 X X -libc@0.2.155 X X +js-sys@0.3.76 X X +lexical-core@1.0.5 X X +lexical-parse-float@1.0.5 X X +lexical-parse-integer@1.0.5 X X +lexical-util@1.0.6 X X +lexical-write-float@1.0.5 X X +lexical-write-integer@1.0.5 X X +libc@0.2.168 X X libflate@2.1.0 X libflate_lz77@2.1.0 X -libm@0.2.8 X X +libm@0.2.11 X X +litemap@0.7.4 X +lock_api@0.4.12 X X log@0.4.22 X X lz4_flex@0.11.3 X md-5@0.10.6 X X memchr@2.7.4 X X mime@0.3.17 X X -miniz_oxide@0.7.4 X X X -mio@1.0.2 X +miniz_oxide@0.8.0 X X X +mio@1.0.3 X +moka@0.12.8 X X murmur3@0.5.2 X X num@0.4.3 X X num-bigint@0.4.6 X X @@ -175,66 +199,71 @@ num-integer@0.1.46 X X num-iter@0.1.45 X X num-rational@0.4.2 X X num-traits@0.2.19 X X -object@0.36.3 X X -once_cell@1.19.0 X X -opendal@0.49.0 X +object@0.36.5 X X +once_cell@1.20.2 X X +opendal@0.50.2 X openssl-probe@0.1.5 X X ordered-float@2.10.1 X -ordered-float@4.2.2 X +ordered-float@4.5.0 X outref@0.5.1 X -parquet@52.2.0 X +parking@2.2.1 X X +parking_lot@0.12.3 X X +parking_lot_core@0.9.10 X X +parquet@53.3.0 X paste@1.0.15 X X percent-encoding@2.3.1 X X -pin-project@1.1.5 X X -pin-project-internal@1.1.5 X X -pin-project-lite@0.2.14 X X +pin-project-lite@0.2.15 X X pin-utils@0.1.0 X X -pkg-config@0.3.30 X X +pkg-config@0.3.31 X X powerfmt@0.2.0 X X ppv-lite86@0.2.20 X X -proc-macro2@1.0.86 X X -quad-rand@0.2.1 X -quick-xml@0.36.1 X -quote@1.0.36 X X +proc-macro2@1.0.92 X X +quad-rand@0.2.3 X +quanta@0.12.4 X +quick-xml@0.36.2 X +quote@1.0.37 X X radium@0.7.0 X rand@0.8.5 X X rand_chacha@0.3.1 X X rand_core@0.6.4 X X -regex@1.10.6 X X -regex-automata@0.4.7 X X +raw-cpuid@11.2.0 X +redox_syscall@0.5.8 X +regex@1.11.1 X X +regex-automata@0.4.9 X X regex-lite@0.1.6 X X -regex-syntax@0.8.4 X X -reqsign@0.16.0 X -reqwest@0.12.5 X X +regex-syntax@0.8.5 X X +reqsign@0.16.1 X +reqwest@0.12.9 X X ring@0.17.8 X rle-decode-fast@1.0.3 X X -rust_decimal@1.35.0 X +rust_decimal@1.36.0 X rustc-demangle@0.1.24 X X -rustc_version@0.4.0 X X +rustc_version@0.4.1 X X rustls@0.21.12 X X X -rustls@0.23.12 X X X +rustls@0.23.20 X X X rustls-native-certs@0.6.3 X X X rustls-pemfile@1.0.4 X X X -rustls-pemfile@2.1.3 X X X -rustls-pki-types@1.8.0 X X +rustls-pemfile@2.2.0 X X X +rustls-pki-types@1.10.1 X X rustls-webpki@0.101.7 X -rustls-webpki@0.102.6 X -rustversion@1.0.17 X X +rustls-webpki@0.102.8 X +rustversion@1.0.18 X X ryu@1.0.18 X X -schannel@0.1.23 X +schannel@0.1.27 X +scopeguard@1.2.0 X X sct@0.7.1 X X X security-framework@2.11.1 X X -security-framework-sys@2.11.1 X X -semver@1.0.23 X X +security-framework-sys@2.12.1 X X +semver@1.0.24 X X seq-macro@0.3.5 X X -serde@1.0.207 X X +serde@1.0.216 X X serde_bytes@0.11.15 X X -serde_derive@1.0.207 X X -serde_json@1.0.124 X X +serde_derive@1.0.216 X X +serde_json@1.0.133 X X serde_repr@0.1.19 X X serde_urlencoded@0.7.1 X X -serde_with@3.9.0 X X -serde_with_macros@3.9.0 X X +serde_with@3.11.0 X X +serde_with_macros@3.11.0 X X sha1@0.10.6 X X sha2@0.10.8 X X shlex@1.3.0 X X @@ -242,87 +271,95 @@ signal-hook-registry@1.4.2 X X slab@0.4.9 X smallvec@1.13.2 X X snap@1.1.1 X -socket2@0.5.7 X X +socket2@0.5.8 X X spin@0.9.8 X +stable_deref_trait@1.2.0 X X static_assertions@1.1.0 X X strsim@0.11.1 X strum@0.26.3 X strum_macros@0.26.4 X subtle@2.6.1 X -syn@2.0.74 X X -sync_wrapper@1.0.1 X +syn@2.0.90 X X +sync_wrapper@1.0.2 X +synstructure@0.13.1 X +tagptr@0.2.0 X X tap@1.0.1 X -thiserror@1.0.63 X X -thiserror-impl@1.0.63 X X +thiserror@1.0.69 X X +thiserror-impl@1.0.69 X X thrift@0.17.0 X -time@0.3.36 X X +time@0.3.37 X X time-core@0.1.2 X X tiny-keccak@2.0.2 X -tinyvec@1.8.0 X X X -tinyvec_macros@0.1.1 X X X -tokio@1.39.2 X +tinystr@0.7.6 X +tokio@1.42.0 X tokio-macros@2.4.0 X tokio-rustls@0.24.1 X X -tokio-rustls@0.26.0 X X -tokio-util@0.7.11 X -tower@0.4.13 X -tower-layer@0.3.3 X +tokio-rustls@0.26.1 X X +tokio-util@0.7.13 X tower-service@0.3.3 X -tracing@0.1.40 X -tracing-attributes@0.1.27 X -tracing-core@0.1.32 X +tracing@0.1.41 X +tracing-attributes@0.1.28 X +tracing-core@0.1.33 X +triomphe@0.1.11 X X try-lock@0.2.5 X twox-hash@1.6.3 X typed-builder@0.19.1 X X +typed-builder@0.20.0 X X typed-builder-macro@0.19.1 X X +typed-builder-macro@0.20.0 X X typenum@1.17.0 X X -unicode-bidi@0.3.15 X X -unicode-ident@1.0.12 X X X -unicode-normalization@0.1.23 X X +unicode-ident@1.0.14 X X X untrusted@0.9.0 X -url@2.5.2 X X +url@2.5.4 X X urlencoding@2.1.3 X +utf16_iter@1.0.5 X X +utf8_iter@1.0.4 X X utf8parse@0.2.2 X X -uuid@1.10.0 X X +uuid@1.11.0 X X version_check@0.9.5 X X vsimd@0.8.0 X want@0.3.1 X wasi@0.11.0+wasi-snapshot-preview1 X X X -wasm-bindgen@0.2.93 X X -wasm-bindgen-backend@0.2.93 X X -wasm-bindgen-futures@0.4.43 X X -wasm-bindgen-macro@0.2.93 X X -wasm-bindgen-macro-support@0.2.93 X X -wasm-bindgen-shared@0.2.93 X X -wasm-streams@0.4.0 X X -web-sys@0.3.70 X X -webpki-roots@0.26.3 X +wasm-bindgen@0.2.99 X X +wasm-bindgen-backend@0.2.99 X X +wasm-bindgen-futures@0.4.49 X X +wasm-bindgen-macro@0.2.99 X X +wasm-bindgen-macro-support@0.2.99 X X +wasm-bindgen-shared@0.2.99 X X +wasm-streams@0.4.2 X X +web-sys@0.3.76 X X +webpki-roots@0.26.7 X +winapi@0.3.9 X X +winapi-i686-pc-windows-gnu@0.4.0 X X +winapi-x86_64-pc-windows-gnu@0.4.0 X X windows-core@0.52.0 X X -windows-sys@0.48.0 X X +windows-registry@0.2.0 X X +windows-result@0.2.0 X X +windows-strings@0.1.0 X X windows-sys@0.52.0 X X -windows-targets@0.48.5 X X +windows-sys@0.59.0 X X windows-targets@0.52.6 X X -windows_aarch64_gnullvm@0.48.5 X X windows_aarch64_gnullvm@0.52.6 X X -windows_aarch64_msvc@0.48.5 X X windows_aarch64_msvc@0.52.6 X X -windows_i686_gnu@0.48.5 X X windows_i686_gnu@0.52.6 X X windows_i686_gnullvm@0.52.6 X X -windows_i686_msvc@0.48.5 X X windows_i686_msvc@0.52.6 X X -windows_x86_64_gnu@0.48.5 X X windows_x86_64_gnu@0.52.6 X X -windows_x86_64_gnullvm@0.48.5 X X windows_x86_64_gnullvm@0.52.6 X X -windows_x86_64_msvc@0.48.5 X X windows_x86_64_msvc@0.52.6 X X -winreg@0.52.0 X +write16@1.0.0 X X +writeable@0.5.5 X wyz@0.5.1 X xmlparser@0.13.6 X X +yoke@0.7.5 X +yoke-derive@0.7.5 X zerocopy@0.7.35 X X X zerocopy-derive@0.7.35 X X X +zerofrom@0.1.5 X +zerofrom-derive@0.1.5 X zeroize@1.8.1 X X +zerovec@0.10.4 X +zerovec-derive@0.10.3 X zstd@0.13.2 X zstd-safe@7.2.1 X X -zstd-sys@2.0.12+zstd.1.5.6 X X +zstd-sys@2.0.13+zstd.1.5.6 X X diff --git a/crates/catalog/hms/DEPENDENCIES.rust.tsv b/crates/catalog/hms/DEPENDENCIES.rust.tsv index f54295ca5..f355335c1 100644 --- a/crates/catalog/hms/DEPENDENCIES.rust.tsv +++ b/crates/catalog/hms/DEPENDENCIES.rust.tsv @@ -1,152 +1,168 @@ -crate 0BSD Apache-2.0 Apache-2.0 WITH LLVM-exception BSD-2-Clause BSD-3-Clause BSL-1.0 CC0-1.0 ISC MIT MPL-2.0 OpenSSL Unicode-DFS-2016 Unlicense Zlib -addr2line@0.22.0 X X -adler@1.0.2 X X X +crate 0BSD Apache-2.0 Apache-2.0 WITH LLVM-exception BSD-2-Clause BSD-3-Clause BSL-1.0 CC0-1.0 ISC MIT MPL-2.0 OpenSSL Unicode-3.0 Unlicense Zlib +addr2line@0.24.2 X X +adler2@2.0.0 X X X adler32@1.2.0 X ahash@0.8.11 X X aho-corasick@1.1.3 X X alloc-no-stdlib@2.0.4 X alloc-stdlib@0.2.2 X -allocator-api2@0.2.18 X X +allocator-api2@0.2.21 X X android-tzdata@0.1.1 X X android_system_properties@0.1.5 X X -anstream@0.6.15 X X -anstyle@1.0.8 X X -anstyle-parse@0.2.5 X X -anstyle-query@1.1.1 X X -anstyle-wincon@3.0.4 X X -anyhow@1.0.86 X X +anstream@0.6.18 X X +anstyle@1.0.10 X X +anstyle-parse@0.2.6 X X +anstyle-query@1.1.2 X X +anstyle-wincon@3.0.6 X X +anyhow@1.0.94 X X apache-avro@0.17.0 X array-init@2.1.0 X X -arrayref@0.3.8 X -arrayvec@0.7.4 X X -arrow-arith@52.2.0 X -arrow-array@52.2.0 X -arrow-buffer@52.2.0 X -arrow-cast@52.2.0 X -arrow-data@52.2.0 X -arrow-ipc@52.2.0 X -arrow-ord@52.2.0 X -arrow-schema@52.2.0 X -arrow-select@52.2.0 X -arrow-string@52.2.0 X +arrayvec@0.7.6 X X +arrow-arith@53.3.0 X +arrow-array@53.3.0 X +arrow-buffer@53.3.0 X +arrow-cast@53.3.0 X +arrow-data@53.3.0 X +arrow-ipc@53.3.0 X +arrow-ord@53.3.0 X +arrow-schema@53.3.0 X +arrow-select@53.3.0 X +arrow-string@53.3.0 X async-broadcast@0.7.1 X X +async-lock@3.4.0 X X async-recursion@1.1.1 X X -async-trait@0.1.81 X X +async-trait@0.1.83 X X atoi@2.0.0 X -autocfg@1.3.0 X X -backon@0.4.4 X -backtrace@0.3.73 X X +autocfg@1.4.0 X X +backon@1.3.0 X +backtrace@0.3.74 X X base64@0.22.1 X X -bigdecimal@0.4.5 X X +bigdecimal@0.4.7 X X bimap@0.6.3 X X bitflags@1.3.2 X X bitflags@2.6.0 X X bitvec@1.0.1 X block-buffer@0.10.4 X X -brotli@6.0.0 X X +brotli@7.0.0 X X brotli-decompressor@4.0.1 X X bumpalo@3.16.0 X X byteorder@1.5.0 X X -bytes@1.7.1 X -cc@1.1.11 X X +bytes@1.9.0 X +cc@1.2.4 X X cfg-if@1.0.0 X X -cfg_aliases@0.1.1 X -chrono@0.4.38 X X -colorchoice@1.0.2 X X +cfg_aliases@0.2.1 X +chrono@0.4.39 X X +colorchoice@1.0.3 X X concurrent-queue@2.5.0 X X const-oid@0.9.6 X X const-random@0.1.18 X X const-random-macro@0.1.16 X X core-foundation-sys@0.8.7 X X core2@0.4.0 X X -cpufeatures@0.2.13 X X +cpufeatures@0.2.16 X X crc32c@0.6.8 X X crc32fast@1.4.2 X X -crossbeam-utils@0.8.20 X X +crossbeam-channel@0.5.14 X X +crossbeam-epoch@0.9.18 X X +crossbeam-utils@0.8.21 X X crunchy@0.2.2 X crypto-common@0.1.6 X X darling@0.20.10 X darling_core@0.20.10 X darling_macro@0.20.10 X -dary_heap@0.3.6 X X -dashmap@5.5.3 X +dary_heap@0.3.7 X X +dashmap@6.1.0 X derivative@2.2.0 X X -derive_builder@0.20.0 X X -derive_builder_core@0.20.0 X X -derive_builder_macro@0.20.0 X X +derive_builder@0.20.2 X X +derive_builder_core@0.20.2 X X +derive_builder_macro@0.20.2 X X digest@0.10.7 X X +displaydoc@0.2.5 X X either@1.13.0 X X env_filter@0.1.2 X X env_logger@0.11.5 X X equivalent@1.0.1 X X event-listener@5.3.1 X X -event-listener-strategy@0.5.2 X X -fastrand@2.1.0 X X -faststr@0.2.21 X X +event-listener-strategy@0.5.3 X X +fastrand@2.3.0 X X +faststr@0.2.27 X X flagset@0.4.6 X flatbuffers@24.3.25 X -flate2@1.0.31 X X +flate2@1.0.35 X X fnv@1.0.7 X X form_urlencoded@1.2.1 X X funty@2.0.0 X -futures@0.3.30 X X -futures-channel@0.3.30 X X -futures-core@0.3.30 X X -futures-executor@0.3.30 X X -futures-io@0.3.30 X X -futures-macro@0.3.30 X X -futures-sink@0.3.30 X X -futures-task@0.3.30 X X -futures-util@0.3.30 X X +futures@0.3.31 X X +futures-channel@0.3.31 X X +futures-core@0.3.31 X X +futures-executor@0.3.31 X X +futures-io@0.3.31 X X +futures-macro@0.3.31 X X +futures-sink@0.3.31 X X +futures-task@0.3.31 X X +futures-util@0.3.31 X X generic-array@0.14.7 X getrandom@0.2.15 X X -gimli@0.29.0 X X +gimli@0.31.1 X X +gloo-timers@0.3.0 X X half@2.4.1 X X hashbrown@0.14.5 X X +hashbrown@0.15.2 X X heck@0.5.0 X X -hermit-abi@0.3.9 X X hex@0.4.3 X X hive_metastore@0.1.0 X hmac@0.12.1 X X -home@0.5.9 X X -http@1.1.0 X X +home@0.5.11 X X +http@1.2.0 X X http-body@1.0.1 X http-body-util@0.1.2 X -httparse@1.9.4 X X +httparse@1.9.5 X X humantime@2.1.0 X X -hyper@1.4.1 X -hyper-rustls@0.27.2 X X X -hyper-util@0.1.7 X -iana-time-zone@0.1.60 X X +hyper@1.5.2 X +hyper-rustls@0.27.3 X X X +hyper-util@0.1.10 X +iana-time-zone@0.1.61 X X iana-time-zone-haiku@0.1.2 X X -iceberg@0.3.0 X -iceberg-catalog-hms@0.3.0 X -iceberg-catalog-memory@0.3.0 X -iceberg_test_utils@0.3.0 X +iceberg@0.4.0 X +iceberg-catalog-hms@0.4.0 X +iceberg-catalog-memory@0.4.0 X +iceberg_test_utils@0.4.0 X +icu_collections@1.5.0 X +icu_locid@1.5.0 X +icu_locid_transform@1.5.0 X +icu_locid_transform_data@1.5.0 X +icu_normalizer@1.5.0 X +icu_normalizer_data@1.5.0 X +icu_properties@1.5.1 X +icu_properties_data@1.5.0 X +icu_provider@1.5.0 X +icu_provider_macros@1.5.0 X ident_case@1.0.1 X X -idna@0.5.0 X X -indexmap@2.4.0 X X +idna@1.0.3 X X +idna_adapter@1.2.0 X X +indexmap@2.7.0 X X integer-encoding@3.0.4 X integer-encoding@4.0.2 X -ipnet@2.9.0 X X +ipnet@2.10.1 X X is_terminal_polyfill@1.70.1 X X itertools@0.13.0 X X -itoa@1.0.11 X X +itoa@1.0.14 X X jobserver@0.1.32 X X -js-sys@0.3.70 X X +js-sys@0.3.76 X X lazy_static@1.5.0 X X -lexical-core@0.8.5 X X -lexical-parse-float@0.8.5 X X -lexical-parse-integer@0.8.6 X X -lexical-util@0.8.5 X X -lexical-write-float@0.8.5 X X -lexical-write-integer@0.8.5 X X -libc@0.2.155 X X +lexical-core@1.0.5 X X +lexical-parse-float@1.0.5 X X +lexical-parse-integer@1.0.5 X X +lexical-util@1.0.6 X X +lexical-write-float@1.0.5 X X +lexical-write-integer@1.0.5 X X +libc@0.2.168 X X libflate@2.1.0 X libflate_lz77@2.1.0 X -libm@0.2.8 X X +libm@0.2.11 X X linked-hash-map@0.5.6 X X linkedbytes@0.1.8 X X +litemap@0.7.4 X lock_api@0.4.12 X X log@0.4.22 X X lz4_flex@0.11.3 X @@ -155,13 +171,14 @@ memchr@2.7.4 X X memoffset@0.9.1 X metainfo@0.7.12 X X mime@0.3.17 X X -miniz_oxide@0.7.4 X X X -mio@1.0.2 X +miniz_oxide@0.8.0 X X X +mio@1.0.3 X +moka@0.12.8 X X motore@0.4.1 X X motore-macros@0.4.1 X X mur3@0.1.0 X murmur3@0.5.2 X X -nix@0.28.0 X +nix@0.29.0 X num@0.4.3 X X num-bigint@0.4.6 X X num-complex@0.4.6 X X @@ -171,158 +188,172 @@ num-rational@0.4.2 X X num-traits@0.2.19 X X num_enum@0.7.3 X X X num_enum_derive@0.7.3 X X X -object@0.36.3 X X -once_cell@1.19.0 X X -opendal@0.49.0 X +object@0.36.5 X X +once_cell@1.20.2 X X +opendal@0.50.2 X ordered-float@2.10.1 X -ordered-float@4.2.2 X -page_size@0.6.0 X X -parking@2.2.0 X X +ordered-float@4.5.0 X +parking@2.2.1 X X parking_lot@0.12.3 X X parking_lot_core@0.9.10 X X -parquet@52.2.0 X +parquet@53.3.0 X paste@1.0.15 X X percent-encoding@2.3.1 X X -pilota@0.11.3 X X -pin-project@1.1.5 X X -pin-project-internal@1.1.5 X X -pin-project-lite@0.2.14 X X +pilota@0.11.8 X X +pin-project@1.1.7 X X +pin-project-internal@1.1.7 X X +pin-project-lite@0.2.15 X X pin-utils@0.1.0 X X -pkg-config@0.3.30 X X +pkg-config@0.3.31 X X ppv-lite86@0.2.20 X X -proc-macro-crate@3.1.0 X X -proc-macro2@1.0.86 X X -quad-rand@0.2.1 X -quick-xml@0.36.1 X -quote@1.0.36 X X +proc-macro-crate@3.2.0 X X +proc-macro2@1.0.92 X X +quad-rand@0.2.3 X +quanta@0.12.4 X +quick-xml@0.36.2 X +quote@1.0.37 X X radium@0.7.0 X rand@0.8.5 X X rand_chacha@0.3.1 X X rand_core@0.6.4 X X -redox_syscall@0.5.3 X -regex@1.10.6 X X -regex-automata@0.4.7 X X +raw-cpuid@11.2.0 X +redox_syscall@0.5.8 X +ref-cast@1.0.23 X X +ref-cast-impl@1.0.23 X X +regex@1.11.1 X X +regex-automata@0.4.9 X X regex-lite@0.1.6 X X -regex-syntax@0.8.4 X X -reqsign@0.16.0 X -reqwest@0.12.5 X X +regex-syntax@0.8.5 X X +reqsign@0.16.1 X +reqwest@0.12.9 X X ring@0.17.8 X rle-decode-fast@1.0.3 X X -rust_decimal@1.35.0 X +rust_decimal@1.36.0 X rustc-demangle@0.1.24 X X -rustc-hash@2.0.0 X X -rustc_version@0.4.0 X X -rustls@0.23.12 X X X -rustls-pemfile@2.1.3 X X X -rustls-pki-types@1.8.0 X X -rustls-webpki@0.102.6 X -rustversion@1.0.17 X X +rustc-hash@2.1.0 X X +rustc_version@0.4.1 X X +rustls@0.23.20 X X X +rustls-pemfile@2.2.0 X X X +rustls-pki-types@1.10.1 X X +rustls-webpki@0.102.8 X +rustversion@1.0.18 X X ryu@1.0.18 X X scopeguard@1.2.0 X X -semver@1.0.23 X X +semver@1.0.24 X X seq-macro@0.3.5 X X -serde@1.0.207 X X +serde@1.0.216 X X serde_bytes@0.11.15 X X -serde_derive@1.0.207 X X -serde_json@1.0.124 X X +serde_derive@1.0.216 X X +serde_json@1.0.133 X X serde_repr@0.1.19 X X serde_urlencoded@0.7.1 X X -serde_with@3.9.0 X X -serde_with_macros@3.9.0 X X +serde_with@3.11.0 X X +serde_with_macros@3.11.0 X X sha1@0.10.6 X X sha2@0.10.8 X X shlex@1.3.0 X X signal-hook-registry@1.4.2 X X -simdutf8@0.1.4 X X +simdutf8@0.1.5 X X slab@0.4.9 X smallvec@1.13.2 X X snap@1.1.1 X -socket2@0.5.7 X X -sonic-rs@0.3.10 X +socket2@0.5.8 X X +sonic-number@0.1.0 X +sonic-rs@0.3.17 X +sonic-simd@0.1.0 X spin@0.9.8 X +stable_deref_trait@1.2.0 X X static_assertions@1.1.0 X X strsim@0.11.1 X strum@0.26.3 X strum_macros@0.26.4 X subtle@2.6.1 X syn@1.0.109 X X -syn@2.0.74 X X -sync_wrapper@1.0.1 X +syn@2.0.90 X X +sync_wrapper@1.0.2 X +synstructure@0.13.1 X +tagptr@0.2.0 X X tap@1.0.1 X -thiserror@1.0.63 X X -thiserror-impl@1.0.63 X X +thiserror@1.0.69 X X +thiserror@2.0.7 X X +thiserror-impl@1.0.69 X X +thiserror-impl@2.0.7 X X thrift@0.17.0 X tiny-keccak@2.0.2 X -tinyvec@1.8.0 X X X -tinyvec_macros@0.1.1 X X X -tokio@1.39.2 X +tinystr@0.7.6 X +tokio@1.42.0 X tokio-macros@2.4.0 X -tokio-rustls@0.26.0 X X -tokio-stream@0.1.15 X -tokio-util@0.7.11 X +tokio-rustls@0.26.1 X X +tokio-stream@0.1.17 X +tokio-util@0.7.13 X toml_datetime@0.6.8 X X -toml_edit@0.21.1 X X -tower@0.4.13 X +toml_edit@0.22.22 X X +tower@0.5.2 X tower-layer@0.3.3 X tower-service@0.3.3 X -tracing@0.1.40 X -tracing-attributes@0.1.27 X -tracing-core@0.1.32 X +tracing@0.1.41 X +tracing-attributes@0.1.28 X +tracing-core@0.1.33 X +triomphe@0.1.11 X X try-lock@0.2.5 X twox-hash@1.6.3 X typed-builder@0.19.1 X X +typed-builder@0.20.0 X X typed-builder-macro@0.19.1 X X +typed-builder-macro@0.20.0 X X typenum@1.17.0 X X -unicode-bidi@0.3.15 X X -unicode-ident@1.0.12 X X X -unicode-normalization@0.1.23 X X +unicode-ident@1.0.14 X X X untrusted@0.9.0 X -url@2.5.2 X X +url@2.5.4 X X +utf16_iter@1.0.5 X X +utf8_iter@1.0.4 X X utf8parse@0.2.2 X X -uuid@1.10.0 X X +uuid@1.11.0 X X version_check@0.9.5 X X -volo@0.10.1 X X -volo-thrift@0.10.2 X X +volo@0.10.3 X X +volo-thrift@0.10.5 X X want@0.3.1 X wasi@0.11.0+wasi-snapshot-preview1 X X X -wasm-bindgen@0.2.93 X X -wasm-bindgen-backend@0.2.93 X X -wasm-bindgen-futures@0.4.43 X X -wasm-bindgen-macro@0.2.93 X X -wasm-bindgen-macro-support@0.2.93 X X -wasm-bindgen-shared@0.2.93 X X -wasm-streams@0.4.0 X X -web-sys@0.3.70 X X -webpki-roots@0.26.3 X +wasm-bindgen@0.2.99 X X +wasm-bindgen-backend@0.2.99 X X +wasm-bindgen-futures@0.4.49 X X +wasm-bindgen-macro@0.2.99 X X +wasm-bindgen-macro-support@0.2.99 X X +wasm-bindgen-shared@0.2.99 X X +wasm-streams@0.4.2 X X +web-sys@0.3.76 X X +webpki-roots@0.26.7 X winapi@0.3.9 X X winapi-i686-pc-windows-gnu@0.4.0 X X winapi-x86_64-pc-windows-gnu@0.4.0 X X windows-core@0.52.0 X X -windows-sys@0.48.0 X X +windows-registry@0.2.0 X X +windows-result@0.2.0 X X +windows-strings@0.1.0 X X windows-sys@0.52.0 X X -windows-targets@0.48.5 X X +windows-sys@0.59.0 X X windows-targets@0.52.6 X X -windows_aarch64_gnullvm@0.48.5 X X windows_aarch64_gnullvm@0.52.6 X X -windows_aarch64_msvc@0.48.5 X X windows_aarch64_msvc@0.52.6 X X -windows_i686_gnu@0.48.5 X X windows_i686_gnu@0.52.6 X X windows_i686_gnullvm@0.52.6 X X -windows_i686_msvc@0.48.5 X X windows_i686_msvc@0.52.6 X X -windows_x86_64_gnu@0.48.5 X X windows_x86_64_gnu@0.52.6 X X -windows_x86_64_gnullvm@0.48.5 X X windows_x86_64_gnullvm@0.52.6 X X -windows_x86_64_msvc@0.48.5 X X windows_x86_64_msvc@0.52.6 X X -winnow@0.5.40 X -winreg@0.52.0 X +winnow@0.6.20 X +write16@1.0.0 X X +writeable@0.5.5 X wyz@0.5.1 X +yoke@0.7.5 X +yoke-derive@0.7.5 X zerocopy@0.7.35 X X X zerocopy-derive@0.7.35 X X X +zerofrom@0.1.5 X +zerofrom-derive@0.1.5 X zeroize@1.8.1 X X +zerovec@0.10.4 X +zerovec-derive@0.10.3 X zstd@0.13.2 X zstd-safe@7.2.1 X X -zstd-sys@2.0.12+zstd.1.5.6 X X +zstd-sys@2.0.13+zstd.1.5.6 X X diff --git a/crates/catalog/memory/DEPENDENCIES.rust.tsv b/crates/catalog/memory/DEPENDENCIES.rust.tsv index b4617eedb..5779db261 100644 --- a/crates/catalog/memory/DEPENDENCIES.rust.tsv +++ b/crates/catalog/memory/DEPENDENCIES.rust.tsv @@ -1,139 +1,163 @@ -crate 0BSD Apache-2.0 Apache-2.0 WITH LLVM-exception BSD-2-Clause BSD-3-Clause BSL-1.0 CC0-1.0 ISC MIT MPL-2.0 OpenSSL Unicode-DFS-2016 Unlicense Zlib -addr2line@0.22.0 X X -adler@1.0.2 X X X +crate 0BSD Apache-2.0 Apache-2.0 WITH LLVM-exception BSD-2-Clause BSD-3-Clause BSL-1.0 CC0-1.0 ISC MIT MPL-2.0 OpenSSL Unicode-3.0 Unlicense Zlib +addr2line@0.24.2 X X +adler2@2.0.0 X X X adler32@1.2.0 X ahash@0.8.11 X X aho-corasick@1.1.3 X X alloc-no-stdlib@2.0.4 X alloc-stdlib@0.2.2 X -allocator-api2@0.2.18 X X +allocator-api2@0.2.21 X X android-tzdata@0.1.1 X X android_system_properties@0.1.5 X X -anstream@0.6.15 X X -anstyle@1.0.8 X X -anstyle-parse@0.2.5 X X -anstyle-query@1.1.1 X X -anstyle-wincon@3.0.4 X X -anyhow@1.0.86 X X +anstream@0.6.18 X X +anstyle@1.0.10 X X +anstyle-parse@0.2.6 X X +anstyle-query@1.1.2 X X +anstyle-wincon@3.0.6 X X +anyhow@1.0.94 X X apache-avro@0.17.0 X array-init@2.1.0 X X -arrayvec@0.7.4 X X -arrow-arith@52.2.0 X -arrow-array@52.2.0 X -arrow-buffer@52.2.0 X -arrow-cast@52.2.0 X -arrow-data@52.2.0 X -arrow-ipc@52.2.0 X -arrow-ord@52.2.0 X -arrow-schema@52.2.0 X -arrow-select@52.2.0 X -arrow-string@52.2.0 X -async-trait@0.1.81 X X +arrayvec@0.7.6 X X +arrow-arith@53.3.0 X +arrow-array@53.3.0 X +arrow-buffer@53.3.0 X +arrow-cast@53.3.0 X +arrow-data@53.3.0 X +arrow-ipc@53.3.0 X +arrow-ord@53.3.0 X +arrow-schema@53.3.0 X +arrow-select@53.3.0 X +arrow-string@53.3.0 X +async-lock@3.4.0 X X +async-trait@0.1.83 X X atoi@2.0.0 X -autocfg@1.3.0 X X -backon@0.4.4 X -backtrace@0.3.73 X X +autocfg@1.4.0 X X +backon@1.3.0 X +backtrace@0.3.74 X X base64@0.22.1 X X -bigdecimal@0.4.5 X X +bigdecimal@0.4.7 X X bimap@0.6.3 X X bitflags@1.3.2 X X +bitflags@2.6.0 X X bitvec@1.0.1 X block-buffer@0.10.4 X X -brotli@6.0.0 X X +brotli@7.0.0 X X brotli-decompressor@4.0.1 X X bumpalo@3.16.0 X X byteorder@1.5.0 X X -bytes@1.7.1 X -cc@1.1.11 X X +bytes@1.9.0 X +cc@1.2.4 X X cfg-if@1.0.0 X X -chrono@0.4.38 X X -colorchoice@1.0.2 X X +chrono@0.4.39 X X +colorchoice@1.0.3 X X +concurrent-queue@2.5.0 X X const-oid@0.9.6 X X const-random@0.1.18 X X const-random-macro@0.1.16 X X core-foundation-sys@0.8.7 X X core2@0.4.0 X X -cpufeatures@0.2.13 X X +cpufeatures@0.2.16 X X crc32c@0.6.8 X X crc32fast@1.4.2 X X +crossbeam-channel@0.5.14 X X +crossbeam-epoch@0.9.18 X X +crossbeam-utils@0.8.21 X X crunchy@0.2.2 X crypto-common@0.1.6 X X darling@0.20.10 X darling_core@0.20.10 X darling_macro@0.20.10 X -dary_heap@0.3.6 X X -derive_builder@0.20.0 X X -derive_builder_core@0.20.0 X X -derive_builder_macro@0.20.0 X X +dary_heap@0.3.7 X X +derive_builder@0.20.2 X X +derive_builder_core@0.20.2 X X +derive_builder_macro@0.20.2 X X digest@0.10.7 X X +displaydoc@0.2.5 X X either@1.13.0 X X env_filter@0.1.2 X X env_logger@0.11.5 X X -fastrand@2.1.0 X X +event-listener@5.3.1 X X +event-listener-strategy@0.5.3 X X +fastrand@2.3.0 X X flagset@0.4.6 X flatbuffers@24.3.25 X -flate2@1.0.31 X X +flate2@1.0.35 X X fnv@1.0.7 X X form_urlencoded@1.2.1 X X funty@2.0.0 X -futures@0.3.30 X X -futures-channel@0.3.30 X X -futures-core@0.3.30 X X -futures-executor@0.3.30 X X -futures-io@0.3.30 X X -futures-macro@0.3.30 X X -futures-sink@0.3.30 X X -futures-task@0.3.30 X X -futures-util@0.3.30 X X +futures@0.3.31 X X +futures-channel@0.3.31 X X +futures-core@0.3.31 X X +futures-executor@0.3.31 X X +futures-io@0.3.31 X X +futures-macro@0.3.31 X X +futures-sink@0.3.31 X X +futures-task@0.3.31 X X +futures-util@0.3.31 X X generic-array@0.14.7 X getrandom@0.2.15 X X -gimli@0.29.0 X X +gimli@0.31.1 X X +gloo-timers@0.3.0 X X half@2.4.1 X X hashbrown@0.14.5 X X +hashbrown@0.15.2 X X heck@0.5.0 X X -hermit-abi@0.3.9 X X hex@0.4.3 X X hmac@0.12.1 X X -home@0.5.9 X X -http@1.1.0 X X +home@0.5.11 X X +http@1.2.0 X X http-body@1.0.1 X http-body-util@0.1.2 X -httparse@1.9.4 X X +httparse@1.9.5 X X humantime@2.1.0 X X -hyper@1.4.1 X -hyper-rustls@0.27.2 X X X -hyper-util@0.1.7 X -iana-time-zone@0.1.60 X X +hyper@1.5.2 X +hyper-rustls@0.27.3 X X X +hyper-util@0.1.10 X +iana-time-zone@0.1.61 X X iana-time-zone-haiku@0.1.2 X X -iceberg@0.3.0 X -iceberg-catalog-memory@0.3.0 X -iceberg_test_utils@0.3.0 X +iceberg@0.4.0 X +iceberg-catalog-memory@0.4.0 X +iceberg_test_utils@0.4.0 X +icu_collections@1.5.0 X +icu_locid@1.5.0 X +icu_locid_transform@1.5.0 X +icu_locid_transform_data@1.5.0 X +icu_normalizer@1.5.0 X +icu_normalizer_data@1.5.0 X +icu_properties@1.5.1 X +icu_properties_data@1.5.0 X +icu_provider@1.5.0 X +icu_provider_macros@1.5.0 X ident_case@1.0.1 X X -idna@0.5.0 X X +idna@1.0.3 X X +idna_adapter@1.2.0 X X integer-encoding@3.0.4 X -ipnet@2.9.0 X X +ipnet@2.10.1 X X is_terminal_polyfill@1.70.1 X X itertools@0.13.0 X X -itoa@1.0.11 X X +itoa@1.0.14 X X jobserver@0.1.32 X X -js-sys@0.3.70 X X -lexical-core@0.8.5 X X -lexical-parse-float@0.8.5 X X -lexical-parse-integer@0.8.6 X X -lexical-util@0.8.5 X X -lexical-write-float@0.8.5 X X -lexical-write-integer@0.8.5 X X -libc@0.2.155 X X +js-sys@0.3.76 X X +lexical-core@1.0.5 X X +lexical-parse-float@1.0.5 X X +lexical-parse-integer@1.0.5 X X +lexical-util@1.0.6 X X +lexical-write-float@1.0.5 X X +lexical-write-integer@1.0.5 X X +libc@0.2.168 X X libflate@2.1.0 X libflate_lz77@2.1.0 X -libm@0.2.8 X X +libm@0.2.11 X X +litemap@0.7.4 X +lock_api@0.4.12 X X log@0.4.22 X X lz4_flex@0.11.3 X md-5@0.10.6 X X memchr@2.7.4 X X mime@0.3.17 X X -miniz_oxide@0.7.4 X X X -mio@1.0.2 X +miniz_oxide@0.8.0 X X X +mio@1.0.3 X +moka@0.12.8 X X murmur3@0.5.2 X X num@0.4.3 X X num-bigint@0.4.6 X X @@ -142,135 +166,148 @@ num-integer@0.1.46 X X num-iter@0.1.45 X X num-rational@0.4.2 X X num-traits@0.2.19 X X -object@0.36.3 X X -once_cell@1.19.0 X X -opendal@0.49.0 X +object@0.36.5 X X +once_cell@1.20.2 X X +opendal@0.50.2 X ordered-float@2.10.1 X -ordered-float@4.2.2 X -parquet@52.2.0 X +ordered-float@4.5.0 X +parking@2.2.1 X X +parking_lot@0.12.3 X X +parking_lot_core@0.9.10 X X +parquet@53.3.0 X paste@1.0.15 X X percent-encoding@2.3.1 X X -pin-project@1.1.5 X X -pin-project-internal@1.1.5 X X -pin-project-lite@0.2.14 X X +pin-project-lite@0.2.15 X X pin-utils@0.1.0 X X -pkg-config@0.3.30 X X +pkg-config@0.3.31 X X ppv-lite86@0.2.20 X X -proc-macro2@1.0.86 X X -quad-rand@0.2.1 X -quick-xml@0.36.1 X -quote@1.0.36 X X +proc-macro2@1.0.92 X X +quad-rand@0.2.3 X +quanta@0.12.4 X +quick-xml@0.36.2 X +quote@1.0.37 X X radium@0.7.0 X rand@0.8.5 X X rand_chacha@0.3.1 X X rand_core@0.6.4 X X -regex@1.10.6 X X -regex-automata@0.4.7 X X +raw-cpuid@11.2.0 X +redox_syscall@0.5.8 X +regex@1.11.1 X X +regex-automata@0.4.9 X X regex-lite@0.1.6 X X -regex-syntax@0.8.4 X X -reqsign@0.16.0 X -reqwest@0.12.5 X X +regex-syntax@0.8.5 X X +reqsign@0.16.1 X +reqwest@0.12.9 X X ring@0.17.8 X rle-decode-fast@1.0.3 X X -rust_decimal@1.35.0 X +rust_decimal@1.36.0 X rustc-demangle@0.1.24 X X -rustc_version@0.4.0 X X -rustls@0.23.12 X X X -rustls-pemfile@2.1.3 X X X -rustls-pki-types@1.8.0 X X -rustls-webpki@0.102.6 X -rustversion@1.0.17 X X +rustc_version@0.4.1 X X +rustls@0.23.20 X X X +rustls-pemfile@2.2.0 X X X +rustls-pki-types@1.10.1 X X +rustls-webpki@0.102.8 X +rustversion@1.0.18 X X ryu@1.0.18 X X -semver@1.0.23 X X +scopeguard@1.2.0 X X +semver@1.0.24 X X seq-macro@0.3.5 X X -serde@1.0.207 X X +serde@1.0.216 X X serde_bytes@0.11.15 X X -serde_derive@1.0.207 X X -serde_json@1.0.124 X X +serde_derive@1.0.216 X X +serde_json@1.0.133 X X serde_repr@0.1.19 X X serde_urlencoded@0.7.1 X X -serde_with@3.9.0 X X -serde_with_macros@3.9.0 X X +serde_with@3.11.0 X X +serde_with_macros@3.11.0 X X sha1@0.10.6 X X sha2@0.10.8 X X shlex@1.3.0 X X slab@0.4.9 X smallvec@1.13.2 X X snap@1.1.1 X -socket2@0.5.7 X X +socket2@0.5.8 X X spin@0.9.8 X +stable_deref_trait@1.2.0 X X static_assertions@1.1.0 X X strsim@0.11.1 X strum@0.26.3 X strum_macros@0.26.4 X subtle@2.6.1 X -syn@2.0.74 X X -sync_wrapper@1.0.1 X +syn@2.0.90 X X +sync_wrapper@1.0.2 X +synstructure@0.13.1 X +tagptr@0.2.0 X X tap@1.0.1 X -thiserror@1.0.63 X X -thiserror-impl@1.0.63 X X +thiserror@1.0.69 X X +thiserror-impl@1.0.69 X X thrift@0.17.0 X tiny-keccak@2.0.2 X -tinyvec@1.8.0 X X X -tinyvec_macros@0.1.1 X X X -tokio@1.39.2 X +tinystr@0.7.6 X +tokio@1.42.0 X tokio-macros@2.4.0 X -tokio-rustls@0.26.0 X X -tokio-util@0.7.11 X -tower@0.4.13 X -tower-layer@0.3.3 X +tokio-rustls@0.26.1 X X +tokio-util@0.7.13 X tower-service@0.3.3 X -tracing@0.1.40 X -tracing-core@0.1.32 X +tracing@0.1.41 X +tracing-core@0.1.33 X +triomphe@0.1.11 X X try-lock@0.2.5 X twox-hash@1.6.3 X typed-builder@0.19.1 X X +typed-builder@0.20.0 X X typed-builder-macro@0.19.1 X X +typed-builder-macro@0.20.0 X X typenum@1.17.0 X X -unicode-bidi@0.3.15 X X -unicode-ident@1.0.12 X X X -unicode-normalization@0.1.23 X X +unicode-ident@1.0.14 X X X untrusted@0.9.0 X -url@2.5.2 X X +url@2.5.4 X X +utf16_iter@1.0.5 X X +utf8_iter@1.0.4 X X utf8parse@0.2.2 X X -uuid@1.10.0 X X +uuid@1.11.0 X X version_check@0.9.5 X X want@0.3.1 X wasi@0.11.0+wasi-snapshot-preview1 X X X -wasm-bindgen@0.2.93 X X -wasm-bindgen-backend@0.2.93 X X -wasm-bindgen-futures@0.4.43 X X -wasm-bindgen-macro@0.2.93 X X -wasm-bindgen-macro-support@0.2.93 X X -wasm-bindgen-shared@0.2.93 X X -wasm-streams@0.4.0 X X -web-sys@0.3.70 X X -webpki-roots@0.26.3 X +wasm-bindgen@0.2.99 X X +wasm-bindgen-backend@0.2.99 X X +wasm-bindgen-futures@0.4.49 X X +wasm-bindgen-macro@0.2.99 X X +wasm-bindgen-macro-support@0.2.99 X X +wasm-bindgen-shared@0.2.99 X X +wasm-streams@0.4.2 X X +web-sys@0.3.76 X X +webpki-roots@0.26.7 X +winapi@0.3.9 X X +winapi-i686-pc-windows-gnu@0.4.0 X X +winapi-x86_64-pc-windows-gnu@0.4.0 X X windows-core@0.52.0 X X -windows-sys@0.48.0 X X +windows-registry@0.2.0 X X +windows-result@0.2.0 X X +windows-strings@0.1.0 X X windows-sys@0.52.0 X X -windows-targets@0.48.5 X X +windows-sys@0.59.0 X X windows-targets@0.52.6 X X -windows_aarch64_gnullvm@0.48.5 X X windows_aarch64_gnullvm@0.52.6 X X -windows_aarch64_msvc@0.48.5 X X windows_aarch64_msvc@0.52.6 X X -windows_i686_gnu@0.48.5 X X windows_i686_gnu@0.52.6 X X windows_i686_gnullvm@0.52.6 X X -windows_i686_msvc@0.48.5 X X windows_i686_msvc@0.52.6 X X -windows_x86_64_gnu@0.48.5 X X windows_x86_64_gnu@0.52.6 X X -windows_x86_64_gnullvm@0.48.5 X X windows_x86_64_gnullvm@0.52.6 X X -windows_x86_64_msvc@0.48.5 X X windows_x86_64_msvc@0.52.6 X X -winreg@0.52.0 X +write16@1.0.0 X X +writeable@0.5.5 X wyz@0.5.1 X +yoke@0.7.5 X +yoke-derive@0.7.5 X zerocopy@0.7.35 X X X zerocopy-derive@0.7.35 X X X +zerofrom@0.1.5 X +zerofrom-derive@0.1.5 X zeroize@1.8.1 X X +zerovec@0.10.4 X +zerovec-derive@0.10.3 X zstd@0.13.2 X zstd-safe@7.2.1 X X -zstd-sys@2.0.12+zstd.1.5.6 X X +zstd-sys@2.0.13+zstd.1.5.6 X X diff --git a/crates/catalog/rest/DEPENDENCIES.rust.tsv b/crates/catalog/rest/DEPENDENCIES.rust.tsv index 43b4ed3d3..4b4389c1c 100644 --- a/crates/catalog/rest/DEPENDENCIES.rust.tsv +++ b/crates/catalog/rest/DEPENDENCIES.rust.tsv @@ -1,147 +1,169 @@ -crate 0BSD Apache-2.0 Apache-2.0 WITH LLVM-exception BSD-2-Clause BSD-3-Clause BSL-1.0 CC0-1.0 ISC MIT MPL-2.0 OpenSSL Unicode-DFS-2016 Unlicense Zlib -addr2line@0.22.0 X X -adler@1.0.2 X X X +crate 0BSD Apache-2.0 Apache-2.0 WITH LLVM-exception BSD-2-Clause BSD-3-Clause BSL-1.0 CC0-1.0 ISC MIT MPL-2.0 OpenSSL Unicode-3.0 Unlicense Zlib +addr2line@0.24.2 X X +adler2@2.0.0 X X X adler32@1.2.0 X ahash@0.8.11 X X aho-corasick@1.1.3 X X alloc-no-stdlib@2.0.4 X alloc-stdlib@0.2.2 X -allocator-api2@0.2.18 X X +allocator-api2@0.2.21 X X android-tzdata@0.1.1 X X android_system_properties@0.1.5 X X -anstream@0.6.15 X X -anstyle@1.0.8 X X -anstyle-parse@0.2.5 X X -anstyle-query@1.1.1 X X -anstyle-wincon@3.0.4 X X -anyhow@1.0.86 X X +anstream@0.6.18 X X +anstyle@1.0.10 X X +anstyle-parse@0.2.6 X X +anstyle-query@1.1.2 X X +anstyle-wincon@3.0.6 X X +anyhow@1.0.94 X X apache-avro@0.17.0 X array-init@2.1.0 X X -arrayvec@0.7.4 X X -arrow-arith@52.2.0 X -arrow-array@52.2.0 X -arrow-buffer@52.2.0 X -arrow-cast@52.2.0 X -arrow-data@52.2.0 X -arrow-ipc@52.2.0 X -arrow-ord@52.2.0 X -arrow-schema@52.2.0 X -arrow-select@52.2.0 X -arrow-string@52.2.0 X -async-trait@0.1.81 X X +arrayvec@0.7.6 X X +arrow-arith@53.3.0 X +arrow-array@53.3.0 X +arrow-buffer@53.3.0 X +arrow-cast@53.3.0 X +arrow-data@53.3.0 X +arrow-ipc@53.3.0 X +arrow-ord@53.3.0 X +arrow-schema@53.3.0 X +arrow-select@53.3.0 X +arrow-string@53.3.0 X +async-lock@3.4.0 X X +async-trait@0.1.83 X X atoi@2.0.0 X atomic-waker@1.1.2 X X -autocfg@1.3.0 X X -backon@0.4.4 X -backtrace@0.3.73 X X +autocfg@1.4.0 X X +backon@1.3.0 X +backtrace@0.3.74 X X base64@0.22.1 X X -bigdecimal@0.4.5 X X +bigdecimal@0.4.7 X X bimap@0.6.3 X X bitflags@1.3.2 X X bitflags@2.6.0 X X bitvec@1.0.1 X block-buffer@0.10.4 X X -brotli@6.0.0 X X +brotli@7.0.0 X X brotli-decompressor@4.0.1 X X bumpalo@3.16.0 X X byteorder@1.5.0 X X -bytes@1.7.1 X -cc@1.1.11 X X +bytes@1.9.0 X +cc@1.2.4 X X cfg-if@1.0.0 X X -chrono@0.4.38 X X -colorchoice@1.0.2 X X +chrono@0.4.39 X X +colorchoice@1.0.3 X X +concurrent-queue@2.5.0 X X const-oid@0.9.6 X X const-random@0.1.18 X X const-random-macro@0.1.16 X X core-foundation-sys@0.8.7 X X core2@0.4.0 X X -cpufeatures@0.2.13 X X +cpufeatures@0.2.16 X X crc32c@0.6.8 X X crc32fast@1.4.2 X X +crossbeam-channel@0.5.14 X X +crossbeam-epoch@0.9.18 X X +crossbeam-utils@0.8.21 X X crunchy@0.2.2 X crypto-common@0.1.6 X X darling@0.20.10 X darling_core@0.20.10 X darling_macro@0.20.10 X -dary_heap@0.3.6 X X -derive_builder@0.20.0 X X -derive_builder_core@0.20.0 X X -derive_builder_macro@0.20.0 X X +dary_heap@0.3.7 X X +derive_builder@0.20.2 X X +derive_builder_core@0.20.2 X X +derive_builder_macro@0.20.2 X X digest@0.10.7 X X +displaydoc@0.2.5 X X either@1.13.0 X X env_filter@0.1.2 X X env_logger@0.11.5 X X equivalent@1.0.1 X X -fastrand@2.1.0 X X +event-listener@5.3.1 X X +event-listener-strategy@0.5.3 X X +fastrand@2.3.0 X X flagset@0.4.6 X flatbuffers@24.3.25 X -flate2@1.0.31 X X +flate2@1.0.35 X X fnv@1.0.7 X X form_urlencoded@1.2.1 X X funty@2.0.0 X -futures@0.3.30 X X -futures-channel@0.3.30 X X -futures-core@0.3.30 X X -futures-executor@0.3.30 X X -futures-io@0.3.30 X X -futures-macro@0.3.30 X X -futures-sink@0.3.30 X X -futures-task@0.3.30 X X -futures-util@0.3.30 X X +futures@0.3.31 X X +futures-channel@0.3.31 X X +futures-core@0.3.31 X X +futures-executor@0.3.31 X X +futures-io@0.3.31 X X +futures-macro@0.3.31 X X +futures-sink@0.3.31 X X +futures-task@0.3.31 X X +futures-util@0.3.31 X X generic-array@0.14.7 X getrandom@0.2.15 X X -gimli@0.29.0 X X -h2@0.4.5 X +gimli@0.31.1 X X +gloo-timers@0.3.0 X X +h2@0.4.7 X half@2.4.1 X X hashbrown@0.14.5 X X +hashbrown@0.15.2 X X heck@0.5.0 X X -hermit-abi@0.3.9 X X hex@0.4.3 X X hmac@0.12.1 X X -home@0.5.9 X X -http@1.1.0 X X +home@0.5.11 X X +http@1.2.0 X X http-body@1.0.1 X http-body-util@0.1.2 X -httparse@1.9.4 X X +httparse@1.9.5 X X httpdate@1.0.3 X X humantime@2.1.0 X X -hyper@1.4.1 X -hyper-rustls@0.27.2 X X X -hyper-util@0.1.7 X -iana-time-zone@0.1.60 X X +hyper@1.5.2 X +hyper-rustls@0.27.3 X X X +hyper-util@0.1.10 X +iana-time-zone@0.1.61 X X iana-time-zone-haiku@0.1.2 X X -iceberg@0.3.0 X -iceberg-catalog-memory@0.3.0 X -iceberg-catalog-rest@0.3.0 X -iceberg_test_utils@0.3.0 X +iceberg@0.4.0 X +iceberg-catalog-memory@0.4.0 X +iceberg-catalog-rest@0.4.0 X +iceberg_test_utils@0.4.0 X +icu_collections@1.5.0 X +icu_locid@1.5.0 X +icu_locid_transform@1.5.0 X +icu_locid_transform_data@1.5.0 X +icu_normalizer@1.5.0 X +icu_normalizer_data@1.5.0 X +icu_properties@1.5.1 X +icu_properties_data@1.5.0 X +icu_provider@1.5.0 X +icu_provider_macros@1.5.0 X ident_case@1.0.1 X X -idna@0.5.0 X X -indexmap@2.4.0 X X +idna@1.0.3 X X +idna_adapter@1.2.0 X X +indexmap@2.7.0 X X integer-encoding@3.0.4 X -ipnet@2.9.0 X X +ipnet@2.10.1 X X is_terminal_polyfill@1.70.1 X X itertools@0.13.0 X X -itoa@1.0.11 X X +itoa@1.0.14 X X jobserver@0.1.32 X X -js-sys@0.3.70 X X -lexical-core@0.8.5 X X -lexical-parse-float@0.8.5 X X -lexical-parse-integer@0.8.6 X X -lexical-util@0.8.5 X X -lexical-write-float@0.8.5 X X -lexical-write-integer@0.8.5 X X -libc@0.2.155 X X +js-sys@0.3.76 X X +lexical-core@1.0.5 X X +lexical-parse-float@1.0.5 X X +lexical-parse-integer@1.0.5 X X +lexical-util@1.0.6 X X +lexical-write-float@1.0.5 X X +lexical-write-integer@1.0.5 X X +libc@0.2.168 X X libflate@2.1.0 X libflate_lz77@2.1.0 X -libm@0.2.8 X X +libm@0.2.11 X X +litemap@0.7.4 X lock_api@0.4.12 X X log@0.4.22 X X lz4_flex@0.11.3 X md-5@0.10.6 X X memchr@2.7.4 X X mime@0.3.17 X X -miniz_oxide@0.7.4 X X X -mio@1.0.2 X +miniz_oxide@0.8.0 X X X +mio@1.0.3 X +moka@0.12.8 X X murmur3@0.5.2 X X num@0.4.3 X X num-bigint@0.4.6 X X @@ -150,139 +172,148 @@ num-integer@0.1.46 X X num-iter@0.1.45 X X num-rational@0.4.2 X X num-traits@0.2.19 X X -object@0.36.3 X X -once_cell@1.19.0 X X -opendal@0.49.0 X +object@0.36.5 X X +once_cell@1.20.2 X X +opendal@0.50.2 X ordered-float@2.10.1 X -ordered-float@4.2.2 X +ordered-float@4.5.0 X +parking@2.2.1 X X parking_lot@0.12.3 X X parking_lot_core@0.9.10 X X -parquet@52.2.0 X +parquet@53.3.0 X paste@1.0.15 X X percent-encoding@2.3.1 X X -pin-project@1.1.5 X X -pin-project-internal@1.1.5 X X -pin-project-lite@0.2.14 X X +pin-project-lite@0.2.15 X X pin-utils@0.1.0 X X -pkg-config@0.3.30 X X +pkg-config@0.3.31 X X ppv-lite86@0.2.20 X X -proc-macro2@1.0.86 X X -quad-rand@0.2.1 X -quick-xml@0.36.1 X -quote@1.0.36 X X +proc-macro2@1.0.92 X X +quad-rand@0.2.3 X +quanta@0.12.4 X +quick-xml@0.36.2 X +quote@1.0.37 X X radium@0.7.0 X rand@0.8.5 X X rand_chacha@0.3.1 X X rand_core@0.6.4 X X -redox_syscall@0.5.3 X -regex@1.10.6 X X -regex-automata@0.4.7 X X +raw-cpuid@11.2.0 X +redox_syscall@0.5.8 X +regex@1.11.1 X X +regex-automata@0.4.9 X X regex-lite@0.1.6 X X -regex-syntax@0.8.4 X X -reqsign@0.16.0 X -reqwest@0.12.5 X X +regex-syntax@0.8.5 X X +reqsign@0.16.1 X +reqwest@0.12.9 X X ring@0.17.8 X rle-decode-fast@1.0.3 X X -rust_decimal@1.35.0 X +rust_decimal@1.36.0 X rustc-demangle@0.1.24 X X -rustc_version@0.4.0 X X -rustls@0.23.12 X X X -rustls-pemfile@2.1.3 X X X -rustls-pki-types@1.8.0 X X -rustls-webpki@0.102.6 X -rustversion@1.0.17 X X +rustc_version@0.4.1 X X +rustls@0.23.20 X X X +rustls-pemfile@2.2.0 X X X +rustls-pki-types@1.10.1 X X +rustls-webpki@0.102.8 X +rustversion@1.0.18 X X ryu@1.0.18 X X scopeguard@1.2.0 X X -semver@1.0.23 X X +semver@1.0.24 X X seq-macro@0.3.5 X X -serde@1.0.207 X X +serde@1.0.216 X X serde_bytes@0.11.15 X X -serde_derive@1.0.207 X X -serde_json@1.0.124 X X +serde_derive@1.0.216 X X +serde_json@1.0.133 X X serde_repr@0.1.19 X X serde_urlencoded@0.7.1 X X -serde_with@3.9.0 X X -serde_with_macros@3.9.0 X X +serde_with@3.11.0 X X +serde_with_macros@3.11.0 X X sha1@0.10.6 X X sha2@0.10.8 X X shlex@1.3.0 X X slab@0.4.9 X smallvec@1.13.2 X X snap@1.1.1 X -socket2@0.5.7 X X +socket2@0.5.8 X X spin@0.9.8 X +stable_deref_trait@1.2.0 X X static_assertions@1.1.0 X X strsim@0.11.1 X strum@0.26.3 X strum_macros@0.26.4 X subtle@2.6.1 X -syn@2.0.74 X X -sync_wrapper@1.0.1 X +syn@2.0.90 X X +sync_wrapper@1.0.2 X +synstructure@0.13.1 X +tagptr@0.2.0 X X tap@1.0.1 X -thiserror@1.0.63 X X -thiserror-impl@1.0.63 X X +thiserror@1.0.69 X X +thiserror-impl@1.0.69 X X thrift@0.17.0 X tiny-keccak@2.0.2 X -tinyvec@1.8.0 X X X -tinyvec_macros@0.1.1 X X X -tokio@1.39.2 X +tinystr@0.7.6 X +tokio@1.42.0 X tokio-macros@2.4.0 X -tokio-rustls@0.26.0 X X -tokio-util@0.7.11 X -tower@0.4.13 X -tower-layer@0.3.3 X +tokio-rustls@0.26.1 X X +tokio-util@0.7.13 X tower-service@0.3.3 X -tracing@0.1.40 X -tracing-core@0.1.32 X +tracing@0.1.41 X +tracing-core@0.1.33 X +triomphe@0.1.11 X X try-lock@0.2.5 X twox-hash@1.6.3 X typed-builder@0.19.1 X X +typed-builder@0.20.0 X X typed-builder-macro@0.19.1 X X +typed-builder-macro@0.20.0 X X typenum@1.17.0 X X -unicode-bidi@0.3.15 X X -unicode-ident@1.0.12 X X X -unicode-normalization@0.1.23 X X +unicode-ident@1.0.14 X X X untrusted@0.9.0 X -url@2.5.2 X X +url@2.5.4 X X +utf16_iter@1.0.5 X X +utf8_iter@1.0.4 X X utf8parse@0.2.2 X X -uuid@1.10.0 X X +uuid@1.11.0 X X version_check@0.9.5 X X want@0.3.1 X wasi@0.11.0+wasi-snapshot-preview1 X X X -wasm-bindgen@0.2.93 X X -wasm-bindgen-backend@0.2.93 X X -wasm-bindgen-futures@0.4.43 X X -wasm-bindgen-macro@0.2.93 X X -wasm-bindgen-macro-support@0.2.93 X X -wasm-bindgen-shared@0.2.93 X X -wasm-streams@0.4.0 X X -web-sys@0.3.70 X X -webpki-roots@0.26.3 X +wasm-bindgen@0.2.99 X X +wasm-bindgen-backend@0.2.99 X X +wasm-bindgen-futures@0.4.49 X X +wasm-bindgen-macro@0.2.99 X X +wasm-bindgen-macro-support@0.2.99 X X +wasm-bindgen-shared@0.2.99 X X +wasm-streams@0.4.2 X X +web-sys@0.3.76 X X +webpki-roots@0.26.7 X +winapi@0.3.9 X X +winapi-i686-pc-windows-gnu@0.4.0 X X +winapi-x86_64-pc-windows-gnu@0.4.0 X X windows-core@0.52.0 X X -windows-sys@0.48.0 X X +windows-registry@0.2.0 X X +windows-result@0.2.0 X X +windows-strings@0.1.0 X X windows-sys@0.52.0 X X -windows-targets@0.48.5 X X +windows-sys@0.59.0 X X windows-targets@0.52.6 X X -windows_aarch64_gnullvm@0.48.5 X X windows_aarch64_gnullvm@0.52.6 X X -windows_aarch64_msvc@0.48.5 X X windows_aarch64_msvc@0.52.6 X X -windows_i686_gnu@0.48.5 X X windows_i686_gnu@0.52.6 X X windows_i686_gnullvm@0.52.6 X X -windows_i686_msvc@0.48.5 X X windows_i686_msvc@0.52.6 X X -windows_x86_64_gnu@0.48.5 X X windows_x86_64_gnu@0.52.6 X X -windows_x86_64_gnullvm@0.48.5 X X windows_x86_64_gnullvm@0.52.6 X X -windows_x86_64_msvc@0.48.5 X X windows_x86_64_msvc@0.52.6 X X -winreg@0.52.0 X +write16@1.0.0 X X +writeable@0.5.5 X wyz@0.5.1 X +yoke@0.7.5 X +yoke-derive@0.7.5 X zerocopy@0.7.35 X X X zerocopy-derive@0.7.35 X X X +zerofrom@0.1.5 X +zerofrom-derive@0.1.5 X zeroize@1.8.1 X X +zerovec@0.10.4 X +zerovec-derive@0.10.3 X zstd@0.13.2 X zstd-safe@7.2.1 X X -zstd-sys@2.0.12+zstd.1.5.6 X X +zstd-sys@2.0.13+zstd.1.5.6 X X diff --git a/crates/catalog/sql/DEPENDENCIES.rust.tsv b/crates/catalog/sql/DEPENDENCIES.rust.tsv index f03c70d0c..c2ed8b0df 100644 --- a/crates/catalog/sql/DEPENDENCIES.rust.tsv +++ b/crates/catalog/sql/DEPENDENCIES.rust.tsv @@ -1,147 +1,164 @@ -crate 0BSD Apache-2.0 Apache-2.0 WITH LLVM-exception BSD-2-Clause BSD-3-Clause BSL-1.0 CC0-1.0 ISC MIT MPL-2.0 OpenSSL Unicode-DFS-2016 Unlicense Zlib -addr2line@0.22.0 X X -adler@1.0.2 X X X +crate 0BSD Apache-2.0 Apache-2.0 WITH LLVM-exception BSD-2-Clause BSD-3-Clause BSL-1.0 CC0-1.0 ISC MIT MPL-2.0 OpenSSL Unicode-3.0 Unlicense Zlib +addr2line@0.24.2 X X +adler2@2.0.0 X X X adler32@1.2.0 X ahash@0.8.11 X X aho-corasick@1.1.3 X X alloc-no-stdlib@2.0.4 X alloc-stdlib@0.2.2 X -allocator-api2@0.2.18 X X +allocator-api2@0.2.21 X X android-tzdata@0.1.1 X X android_system_properties@0.1.5 X X -anstream@0.6.15 X X -anstyle@1.0.8 X X -anstyle-parse@0.2.5 X X -anstyle-query@1.1.1 X X -anstyle-wincon@3.0.4 X X -anyhow@1.0.86 X X +anstream@0.6.18 X X +anstyle@1.0.10 X X +anstyle-parse@0.2.6 X X +anstyle-query@1.1.2 X X +anstyle-wincon@3.0.6 X X +anyhow@1.0.94 X X apache-avro@0.17.0 X array-init@2.1.0 X X -arrayvec@0.7.4 X X -arrow-arith@52.2.0 X -arrow-array@52.2.0 X -arrow-buffer@52.2.0 X -arrow-cast@52.2.0 X -arrow-data@52.2.0 X -arrow-ipc@52.2.0 X -arrow-ord@52.2.0 X -arrow-schema@52.2.0 X -arrow-select@52.2.0 X -arrow-string@52.2.0 X -async-trait@0.1.81 X X +arrayvec@0.7.6 X X +arrow-arith@53.3.0 X +arrow-array@53.3.0 X +arrow-buffer@53.3.0 X +arrow-cast@53.3.0 X +arrow-data@53.3.0 X +arrow-ipc@53.3.0 X +arrow-ord@53.3.0 X +arrow-schema@53.3.0 X +arrow-select@53.3.0 X +arrow-string@53.3.0 X +async-lock@3.4.0 X X +async-trait@0.1.83 X X atoi@2.0.0 X -autocfg@1.3.0 X X -backon@0.4.4 X -backtrace@0.3.73 X X -base64@0.21.7 X X +autocfg@1.4.0 X X +backon@1.3.0 X +backtrace@0.3.74 X X base64@0.22.1 X X -bigdecimal@0.4.5 X X +bigdecimal@0.4.7 X X bimap@0.6.3 X X bitflags@1.3.2 X X bitflags@2.6.0 X X bitvec@1.0.1 X block-buffer@0.10.4 X X -brotli@6.0.0 X X +brotli@7.0.0 X X brotli-decompressor@4.0.1 X X bumpalo@3.16.0 X X byteorder@1.5.0 X X -bytes@1.7.1 X -cc@1.1.11 X X +bytes@1.9.0 X +cc@1.2.4 X X cfg-if@1.0.0 X X -chrono@0.4.38 X X -colorchoice@1.0.2 X X +chrono@0.4.39 X X +colorchoice@1.0.3 X X concurrent-queue@2.5.0 X X const-oid@0.9.6 X X const-random@0.1.18 X X const-random-macro@0.1.16 X X core-foundation-sys@0.8.7 X X core2@0.4.0 X X -cpufeatures@0.2.13 X X +cpufeatures@0.2.16 X X crc@3.2.1 X X crc-catalog@2.4.0 X X crc32c@0.6.8 X X crc32fast@1.4.2 X X -crossbeam-queue@0.3.11 X X -crossbeam-utils@0.8.20 X X +crossbeam-channel@0.5.14 X X +crossbeam-epoch@0.9.18 X X +crossbeam-queue@0.3.12 X X +crossbeam-utils@0.8.21 X X crunchy@0.2.2 X crypto-common@0.1.6 X X darling@0.20.10 X darling_core@0.20.10 X darling_macro@0.20.10 X -dary_heap@0.3.6 X X -derive_builder@0.20.0 X X -derive_builder_core@0.20.0 X X -derive_builder_macro@0.20.0 X X +dary_heap@0.3.7 X X +derive_builder@0.20.2 X X +derive_builder_core@0.20.2 X X +derive_builder_macro@0.20.2 X X digest@0.10.7 X X +displaydoc@0.2.5 X X either@1.13.0 X X env_filter@0.1.2 X X env_logger@0.11.5 X X equivalent@1.0.1 X X event-listener@5.3.1 X X -fastrand@2.1.0 X X +event-listener-strategy@0.5.3 X X +fastrand@2.3.0 X X flagset@0.4.6 X flatbuffers@24.3.25 X -flate2@1.0.31 X X -flume@0.11.0 X X +flate2@1.0.35 X X +flume@0.11.1 X X fnv@1.0.7 X X form_urlencoded@1.2.1 X X funty@2.0.0 X -futures@0.3.30 X X -futures-channel@0.3.30 X X -futures-core@0.3.30 X X -futures-executor@0.3.30 X X +futures@0.3.31 X X +futures-channel@0.3.31 X X +futures-core@0.3.31 X X +futures-executor@0.3.31 X X futures-intrusive@0.5.0 X X -futures-io@0.3.30 X X -futures-macro@0.3.30 X X -futures-sink@0.3.30 X X -futures-task@0.3.30 X X -futures-util@0.3.30 X X +futures-io@0.3.31 X X +futures-macro@0.3.31 X X +futures-sink@0.3.31 X X +futures-task@0.3.31 X X +futures-util@0.3.31 X X generic-array@0.14.7 X getrandom@0.2.15 X X -gimli@0.29.0 X X +gimli@0.31.1 X X +gloo-timers@0.3.0 X X half@2.4.1 X X hashbrown@0.14.5 X X +hashbrown@0.15.2 X X hashlink@0.9.1 X X heck@0.5.0 X X -hermit-abi@0.3.9 X X hex@0.4.3 X X hmac@0.12.1 X X -home@0.5.9 X X -http@1.1.0 X X +home@0.5.11 X X +http@1.2.0 X X http-body@1.0.1 X http-body-util@0.1.2 X -httparse@1.9.4 X X +httparse@1.9.5 X X humantime@2.1.0 X X -hyper@1.4.1 X -hyper-rustls@0.27.2 X X X -hyper-util@0.1.7 X -iana-time-zone@0.1.60 X X +hyper@1.5.2 X +hyper-rustls@0.27.3 X X X +hyper-util@0.1.10 X +iana-time-zone@0.1.61 X X iana-time-zone-haiku@0.1.2 X X -iceberg@0.3.0 X -iceberg-catalog-memory@0.3.0 X -iceberg-catalog-sql@0.3.0 X -iceberg_test_utils@0.3.0 X +iceberg@0.4.0 X +iceberg-catalog-memory@0.4.0 X +iceberg-catalog-sql@0.4.0 X +iceberg_test_utils@0.4.0 X +icu_collections@1.5.0 X +icu_locid@1.5.0 X +icu_locid_transform@1.5.0 X +icu_locid_transform_data@1.5.0 X +icu_normalizer@1.5.0 X +icu_normalizer_data@1.5.0 X +icu_properties@1.5.1 X +icu_properties_data@1.5.0 X +icu_provider@1.5.0 X +icu_provider_macros@1.5.0 X ident_case@1.0.1 X X -idna@0.5.0 X X -indexmap@2.4.0 X X +idna@1.0.3 X X +idna_adapter@1.2.0 X X +indexmap@2.7.0 X X integer-encoding@3.0.4 X -ipnet@2.9.0 X X +ipnet@2.10.1 X X is_terminal_polyfill@1.70.1 X X itertools@0.13.0 X X -itoa@1.0.11 X X +itoa@1.0.14 X X jobserver@0.1.32 X X -js-sys@0.3.70 X X -lexical-core@0.8.5 X X -lexical-parse-float@0.8.5 X X -lexical-parse-integer@0.8.6 X X -lexical-util@0.8.5 X X -lexical-write-float@0.8.5 X X -lexical-write-integer@0.8.5 X X -libc@0.2.155 X X +js-sys@0.3.76 X X +lexical-core@1.0.5 X X +lexical-parse-float@1.0.5 X X +lexical-parse-integer@1.0.5 X X +lexical-util@1.0.6 X X +lexical-write-float@1.0.5 X X +lexical-write-integer@1.0.5 X X +libc@0.2.168 X X libflate@2.1.0 X libflate_lz77@2.1.0 X -libm@0.2.8 X X -libsqlite3-sys@0.28.0 X +libm@0.2.11 X X +libsqlite3-sys@0.30.1 X +litemap@0.7.4 X lock_api@0.4.12 X X log@0.4.22 X X lz4_flex@0.11.3 X @@ -149,8 +166,9 @@ md-5@0.10.6 X X memchr@2.7.4 X X mime@0.3.17 X X minimal-lexical@0.2.1 X X -miniz_oxide@0.7.4 X X X -mio@1.0.2 X +miniz_oxide@0.8.0 X X X +mio@1.0.3 X +moka@0.12.8 X X murmur3@0.5.2 X X nom@7.1.3 X num@0.4.3 X X @@ -160,153 +178,156 @@ num-integer@0.1.46 X X num-iter@0.1.45 X X num-rational@0.4.2 X X num-traits@0.2.19 X X -object@0.36.3 X X -once_cell@1.19.0 X X -opendal@0.49.0 X +object@0.36.5 X X +once_cell@1.20.2 X X +opendal@0.50.2 X ordered-float@2.10.1 X -ordered-float@4.2.2 X -parking@2.2.0 X X +ordered-float@4.5.0 X +parking@2.2.1 X X parking_lot@0.12.3 X X parking_lot_core@0.9.10 X X -parquet@52.2.0 X +parquet@53.3.0 X paste@1.0.15 X X percent-encoding@2.3.1 X X -pin-project@1.1.5 X X -pin-project-internal@1.1.5 X X -pin-project-lite@0.2.14 X X +pin-project-lite@0.2.15 X X pin-utils@0.1.0 X X -pkg-config@0.3.30 X X +pkg-config@0.3.31 X X ppv-lite86@0.2.20 X X -proc-macro2@1.0.86 X X -quad-rand@0.2.1 X -quick-xml@0.36.1 X -quote@1.0.36 X X +proc-macro2@1.0.92 X X +quad-rand@0.2.3 X +quanta@0.12.4 X +quick-xml@0.36.2 X +quote@1.0.37 X X radium@0.7.0 X rand@0.8.5 X X rand_chacha@0.3.1 X X rand_core@0.6.4 X X -redox_syscall@0.5.3 X -regex@1.10.6 X X -regex-automata@0.4.7 X X +raw-cpuid@11.2.0 X +redox_syscall@0.5.8 X +regex@1.11.1 X X +regex-automata@0.4.9 X X regex-lite@0.1.6 X X -regex-syntax@0.8.4 X X -reqsign@0.16.0 X -reqwest@0.12.5 X X +regex-syntax@0.8.5 X X +reqsign@0.16.1 X +reqwest@0.12.9 X X ring@0.17.8 X rle-decode-fast@1.0.3 X X -rust_decimal@1.35.0 X +rust_decimal@1.36.0 X rustc-demangle@0.1.24 X X -rustc_version@0.4.0 X X -rustls@0.21.12 X X X -rustls@0.23.12 X X X -rustls-pemfile@1.0.4 X X X -rustls-pemfile@2.1.3 X X X -rustls-pki-types@1.8.0 X X -rustls-webpki@0.101.7 X -rustls-webpki@0.102.6 X -rustversion@1.0.17 X X +rustc_version@0.4.1 X X +rustls@0.23.20 X X X +rustls-pemfile@2.2.0 X X X +rustls-pki-types@1.10.1 X X +rustls-webpki@0.102.8 X +rustversion@1.0.18 X X ryu@1.0.18 X X scopeguard@1.2.0 X X -sct@0.7.1 X X X -semver@1.0.23 X X +semver@1.0.24 X X seq-macro@0.3.5 X X -serde@1.0.207 X X +serde@1.0.216 X X serde_bytes@0.11.15 X X -serde_derive@1.0.207 X X -serde_json@1.0.124 X X +serde_derive@1.0.216 X X +serde_json@1.0.133 X X serde_repr@0.1.19 X X serde_urlencoded@0.7.1 X X -serde_with@3.9.0 X X -serde_with_macros@3.9.0 X X +serde_with@3.11.0 X X +serde_with_macros@3.11.0 X X sha1@0.10.6 X X sha2@0.10.8 X X shlex@1.3.0 X X slab@0.4.9 X smallvec@1.13.2 X X snap@1.1.1 X -socket2@0.5.7 X X +socket2@0.5.8 X X spin@0.9.8 X -sqlformat@0.2.4 X X -sqlx@0.8.0 X X -sqlx-core@0.8.0 X X -sqlx-sqlite@0.8.0 X X +sqlformat@0.2.6 X X +sqlx@0.8.2 X X +sqlx-core@0.8.2 X X +sqlx-sqlite@0.8.2 X X +stable_deref_trait@1.2.0 X X static_assertions@1.1.0 X X strsim@0.11.1 X strum@0.26.3 X strum_macros@0.26.4 X subtle@2.6.1 X -syn@2.0.74 X X -sync_wrapper@1.0.1 X +syn@2.0.90 X X +sync_wrapper@1.0.2 X +synstructure@0.13.1 X +tagptr@0.2.0 X X tap@1.0.1 X -thiserror@1.0.63 X X -thiserror-impl@1.0.63 X X +thiserror@1.0.69 X X +thiserror-impl@1.0.69 X X thrift@0.17.0 X tiny-keccak@2.0.2 X -tinyvec@1.8.0 X X X -tinyvec_macros@0.1.1 X X X -tokio@1.39.2 X +tinystr@0.7.6 X +tokio@1.42.0 X tokio-macros@2.4.0 X -tokio-rustls@0.26.0 X X -tokio-stream@0.1.15 X -tokio-util@0.7.11 X -tower@0.4.13 X -tower-layer@0.3.3 X +tokio-rustls@0.26.1 X X +tokio-stream@0.1.17 X +tokio-util@0.7.13 X tower-service@0.3.3 X -tracing@0.1.40 X -tracing-attributes@0.1.27 X -tracing-core@0.1.32 X +tracing@0.1.41 X +tracing-attributes@0.1.28 X +tracing-core@0.1.33 X +triomphe@0.1.11 X X try-lock@0.2.5 X twox-hash@1.6.3 X typed-builder@0.19.1 X X +typed-builder@0.20.0 X X typed-builder-macro@0.19.1 X X +typed-builder-macro@0.20.0 X X typenum@1.17.0 X X -unicode-bidi@0.3.15 X X -unicode-ident@1.0.12 X X X -unicode-normalization@0.1.23 X X +unicode-ident@1.0.14 X X X unicode_categories@0.1.1 X X untrusted@0.9.0 X -url@2.5.2 X X +url@2.5.4 X X +utf16_iter@1.0.5 X X +utf8_iter@1.0.4 X X utf8parse@0.2.2 X X -uuid@1.10.0 X X +uuid@1.11.0 X X vcpkg@0.2.15 X X version_check@0.9.5 X X want@0.3.1 X wasi@0.11.0+wasi-snapshot-preview1 X X X -wasm-bindgen@0.2.93 X X -wasm-bindgen-backend@0.2.93 X X -wasm-bindgen-futures@0.4.43 X X -wasm-bindgen-macro@0.2.93 X X -wasm-bindgen-macro-support@0.2.93 X X -wasm-bindgen-shared@0.2.93 X X -wasm-streams@0.4.0 X X -web-sys@0.3.70 X X -webpki-roots@0.25.4 X -webpki-roots@0.26.3 X +wasm-bindgen@0.2.99 X X +wasm-bindgen-backend@0.2.99 X X +wasm-bindgen-futures@0.4.49 X X +wasm-bindgen-macro@0.2.99 X X +wasm-bindgen-macro-support@0.2.99 X X +wasm-bindgen-shared@0.2.99 X X +wasm-streams@0.4.2 X X +web-sys@0.3.76 X X +webpki-roots@0.26.7 X +winapi@0.3.9 X X +winapi-i686-pc-windows-gnu@0.4.0 X X +winapi-x86_64-pc-windows-gnu@0.4.0 X X windows-core@0.52.0 X X -windows-sys@0.48.0 X X +windows-registry@0.2.0 X X +windows-result@0.2.0 X X +windows-strings@0.1.0 X X windows-sys@0.52.0 X X -windows-targets@0.48.5 X X +windows-sys@0.59.0 X X windows-targets@0.52.6 X X -windows_aarch64_gnullvm@0.48.5 X X windows_aarch64_gnullvm@0.52.6 X X -windows_aarch64_msvc@0.48.5 X X windows_aarch64_msvc@0.52.6 X X -windows_i686_gnu@0.48.5 X X windows_i686_gnu@0.52.6 X X windows_i686_gnullvm@0.52.6 X X -windows_i686_msvc@0.48.5 X X windows_i686_msvc@0.52.6 X X -windows_x86_64_gnu@0.48.5 X X windows_x86_64_gnu@0.52.6 X X -windows_x86_64_gnullvm@0.48.5 X X windows_x86_64_gnullvm@0.52.6 X X -windows_x86_64_msvc@0.48.5 X X windows_x86_64_msvc@0.52.6 X X -winreg@0.52.0 X +write16@1.0.0 X X +writeable@0.5.5 X wyz@0.5.1 X +yoke@0.7.5 X +yoke-derive@0.7.5 X zerocopy@0.7.35 X X X zerocopy-derive@0.7.35 X X X +zerofrom@0.1.5 X +zerofrom-derive@0.1.5 X zeroize@1.8.1 X X +zerovec@0.10.4 X +zerovec-derive@0.10.3 X zstd@0.13.2 X zstd-safe@7.2.1 X X -zstd-sys@2.0.12+zstd.1.5.6 X X +zstd-sys@2.0.13+zstd.1.5.6 X X diff --git a/crates/iceberg/DEPENDENCIES.rust.tsv b/crates/iceberg/DEPENDENCIES.rust.tsv index b4617eedb..5779db261 100644 --- a/crates/iceberg/DEPENDENCIES.rust.tsv +++ b/crates/iceberg/DEPENDENCIES.rust.tsv @@ -1,139 +1,163 @@ -crate 0BSD Apache-2.0 Apache-2.0 WITH LLVM-exception BSD-2-Clause BSD-3-Clause BSL-1.0 CC0-1.0 ISC MIT MPL-2.0 OpenSSL Unicode-DFS-2016 Unlicense Zlib -addr2line@0.22.0 X X -adler@1.0.2 X X X +crate 0BSD Apache-2.0 Apache-2.0 WITH LLVM-exception BSD-2-Clause BSD-3-Clause BSL-1.0 CC0-1.0 ISC MIT MPL-2.0 OpenSSL Unicode-3.0 Unlicense Zlib +addr2line@0.24.2 X X +adler2@2.0.0 X X X adler32@1.2.0 X ahash@0.8.11 X X aho-corasick@1.1.3 X X alloc-no-stdlib@2.0.4 X alloc-stdlib@0.2.2 X -allocator-api2@0.2.18 X X +allocator-api2@0.2.21 X X android-tzdata@0.1.1 X X android_system_properties@0.1.5 X X -anstream@0.6.15 X X -anstyle@1.0.8 X X -anstyle-parse@0.2.5 X X -anstyle-query@1.1.1 X X -anstyle-wincon@3.0.4 X X -anyhow@1.0.86 X X +anstream@0.6.18 X X +anstyle@1.0.10 X X +anstyle-parse@0.2.6 X X +anstyle-query@1.1.2 X X +anstyle-wincon@3.0.6 X X +anyhow@1.0.94 X X apache-avro@0.17.0 X array-init@2.1.0 X X -arrayvec@0.7.4 X X -arrow-arith@52.2.0 X -arrow-array@52.2.0 X -arrow-buffer@52.2.0 X -arrow-cast@52.2.0 X -arrow-data@52.2.0 X -arrow-ipc@52.2.0 X -arrow-ord@52.2.0 X -arrow-schema@52.2.0 X -arrow-select@52.2.0 X -arrow-string@52.2.0 X -async-trait@0.1.81 X X +arrayvec@0.7.6 X X +arrow-arith@53.3.0 X +arrow-array@53.3.0 X +arrow-buffer@53.3.0 X +arrow-cast@53.3.0 X +arrow-data@53.3.0 X +arrow-ipc@53.3.0 X +arrow-ord@53.3.0 X +arrow-schema@53.3.0 X +arrow-select@53.3.0 X +arrow-string@53.3.0 X +async-lock@3.4.0 X X +async-trait@0.1.83 X X atoi@2.0.0 X -autocfg@1.3.0 X X -backon@0.4.4 X -backtrace@0.3.73 X X +autocfg@1.4.0 X X +backon@1.3.0 X +backtrace@0.3.74 X X base64@0.22.1 X X -bigdecimal@0.4.5 X X +bigdecimal@0.4.7 X X bimap@0.6.3 X X bitflags@1.3.2 X X +bitflags@2.6.0 X X bitvec@1.0.1 X block-buffer@0.10.4 X X -brotli@6.0.0 X X +brotli@7.0.0 X X brotli-decompressor@4.0.1 X X bumpalo@3.16.0 X X byteorder@1.5.0 X X -bytes@1.7.1 X -cc@1.1.11 X X +bytes@1.9.0 X +cc@1.2.4 X X cfg-if@1.0.0 X X -chrono@0.4.38 X X -colorchoice@1.0.2 X X +chrono@0.4.39 X X +colorchoice@1.0.3 X X +concurrent-queue@2.5.0 X X const-oid@0.9.6 X X const-random@0.1.18 X X const-random-macro@0.1.16 X X core-foundation-sys@0.8.7 X X core2@0.4.0 X X -cpufeatures@0.2.13 X X +cpufeatures@0.2.16 X X crc32c@0.6.8 X X crc32fast@1.4.2 X X +crossbeam-channel@0.5.14 X X +crossbeam-epoch@0.9.18 X X +crossbeam-utils@0.8.21 X X crunchy@0.2.2 X crypto-common@0.1.6 X X darling@0.20.10 X darling_core@0.20.10 X darling_macro@0.20.10 X -dary_heap@0.3.6 X X -derive_builder@0.20.0 X X -derive_builder_core@0.20.0 X X -derive_builder_macro@0.20.0 X X +dary_heap@0.3.7 X X +derive_builder@0.20.2 X X +derive_builder_core@0.20.2 X X +derive_builder_macro@0.20.2 X X digest@0.10.7 X X +displaydoc@0.2.5 X X either@1.13.0 X X env_filter@0.1.2 X X env_logger@0.11.5 X X -fastrand@2.1.0 X X +event-listener@5.3.1 X X +event-listener-strategy@0.5.3 X X +fastrand@2.3.0 X X flagset@0.4.6 X flatbuffers@24.3.25 X -flate2@1.0.31 X X +flate2@1.0.35 X X fnv@1.0.7 X X form_urlencoded@1.2.1 X X funty@2.0.0 X -futures@0.3.30 X X -futures-channel@0.3.30 X X -futures-core@0.3.30 X X -futures-executor@0.3.30 X X -futures-io@0.3.30 X X -futures-macro@0.3.30 X X -futures-sink@0.3.30 X X -futures-task@0.3.30 X X -futures-util@0.3.30 X X +futures@0.3.31 X X +futures-channel@0.3.31 X X +futures-core@0.3.31 X X +futures-executor@0.3.31 X X +futures-io@0.3.31 X X +futures-macro@0.3.31 X X +futures-sink@0.3.31 X X +futures-task@0.3.31 X X +futures-util@0.3.31 X X generic-array@0.14.7 X getrandom@0.2.15 X X -gimli@0.29.0 X X +gimli@0.31.1 X X +gloo-timers@0.3.0 X X half@2.4.1 X X hashbrown@0.14.5 X X +hashbrown@0.15.2 X X heck@0.5.0 X X -hermit-abi@0.3.9 X X hex@0.4.3 X X hmac@0.12.1 X X -home@0.5.9 X X -http@1.1.0 X X +home@0.5.11 X X +http@1.2.0 X X http-body@1.0.1 X http-body-util@0.1.2 X -httparse@1.9.4 X X +httparse@1.9.5 X X humantime@2.1.0 X X -hyper@1.4.1 X -hyper-rustls@0.27.2 X X X -hyper-util@0.1.7 X -iana-time-zone@0.1.60 X X +hyper@1.5.2 X +hyper-rustls@0.27.3 X X X +hyper-util@0.1.10 X +iana-time-zone@0.1.61 X X iana-time-zone-haiku@0.1.2 X X -iceberg@0.3.0 X -iceberg-catalog-memory@0.3.0 X -iceberg_test_utils@0.3.0 X +iceberg@0.4.0 X +iceberg-catalog-memory@0.4.0 X +iceberg_test_utils@0.4.0 X +icu_collections@1.5.0 X +icu_locid@1.5.0 X +icu_locid_transform@1.5.0 X +icu_locid_transform_data@1.5.0 X +icu_normalizer@1.5.0 X +icu_normalizer_data@1.5.0 X +icu_properties@1.5.1 X +icu_properties_data@1.5.0 X +icu_provider@1.5.0 X +icu_provider_macros@1.5.0 X ident_case@1.0.1 X X -idna@0.5.0 X X +idna@1.0.3 X X +idna_adapter@1.2.0 X X integer-encoding@3.0.4 X -ipnet@2.9.0 X X +ipnet@2.10.1 X X is_terminal_polyfill@1.70.1 X X itertools@0.13.0 X X -itoa@1.0.11 X X +itoa@1.0.14 X X jobserver@0.1.32 X X -js-sys@0.3.70 X X -lexical-core@0.8.5 X X -lexical-parse-float@0.8.5 X X -lexical-parse-integer@0.8.6 X X -lexical-util@0.8.5 X X -lexical-write-float@0.8.5 X X -lexical-write-integer@0.8.5 X X -libc@0.2.155 X X +js-sys@0.3.76 X X +lexical-core@1.0.5 X X +lexical-parse-float@1.0.5 X X +lexical-parse-integer@1.0.5 X X +lexical-util@1.0.6 X X +lexical-write-float@1.0.5 X X +lexical-write-integer@1.0.5 X X +libc@0.2.168 X X libflate@2.1.0 X libflate_lz77@2.1.0 X -libm@0.2.8 X X +libm@0.2.11 X X +litemap@0.7.4 X +lock_api@0.4.12 X X log@0.4.22 X X lz4_flex@0.11.3 X md-5@0.10.6 X X memchr@2.7.4 X X mime@0.3.17 X X -miniz_oxide@0.7.4 X X X -mio@1.0.2 X +miniz_oxide@0.8.0 X X X +mio@1.0.3 X +moka@0.12.8 X X murmur3@0.5.2 X X num@0.4.3 X X num-bigint@0.4.6 X X @@ -142,135 +166,148 @@ num-integer@0.1.46 X X num-iter@0.1.45 X X num-rational@0.4.2 X X num-traits@0.2.19 X X -object@0.36.3 X X -once_cell@1.19.0 X X -opendal@0.49.0 X +object@0.36.5 X X +once_cell@1.20.2 X X +opendal@0.50.2 X ordered-float@2.10.1 X -ordered-float@4.2.2 X -parquet@52.2.0 X +ordered-float@4.5.0 X +parking@2.2.1 X X +parking_lot@0.12.3 X X +parking_lot_core@0.9.10 X X +parquet@53.3.0 X paste@1.0.15 X X percent-encoding@2.3.1 X X -pin-project@1.1.5 X X -pin-project-internal@1.1.5 X X -pin-project-lite@0.2.14 X X +pin-project-lite@0.2.15 X X pin-utils@0.1.0 X X -pkg-config@0.3.30 X X +pkg-config@0.3.31 X X ppv-lite86@0.2.20 X X -proc-macro2@1.0.86 X X -quad-rand@0.2.1 X -quick-xml@0.36.1 X -quote@1.0.36 X X +proc-macro2@1.0.92 X X +quad-rand@0.2.3 X +quanta@0.12.4 X +quick-xml@0.36.2 X +quote@1.0.37 X X radium@0.7.0 X rand@0.8.5 X X rand_chacha@0.3.1 X X rand_core@0.6.4 X X -regex@1.10.6 X X -regex-automata@0.4.7 X X +raw-cpuid@11.2.0 X +redox_syscall@0.5.8 X +regex@1.11.1 X X +regex-automata@0.4.9 X X regex-lite@0.1.6 X X -regex-syntax@0.8.4 X X -reqsign@0.16.0 X -reqwest@0.12.5 X X +regex-syntax@0.8.5 X X +reqsign@0.16.1 X +reqwest@0.12.9 X X ring@0.17.8 X rle-decode-fast@1.0.3 X X -rust_decimal@1.35.0 X +rust_decimal@1.36.0 X rustc-demangle@0.1.24 X X -rustc_version@0.4.0 X X -rustls@0.23.12 X X X -rustls-pemfile@2.1.3 X X X -rustls-pki-types@1.8.0 X X -rustls-webpki@0.102.6 X -rustversion@1.0.17 X X +rustc_version@0.4.1 X X +rustls@0.23.20 X X X +rustls-pemfile@2.2.0 X X X +rustls-pki-types@1.10.1 X X +rustls-webpki@0.102.8 X +rustversion@1.0.18 X X ryu@1.0.18 X X -semver@1.0.23 X X +scopeguard@1.2.0 X X +semver@1.0.24 X X seq-macro@0.3.5 X X -serde@1.0.207 X X +serde@1.0.216 X X serde_bytes@0.11.15 X X -serde_derive@1.0.207 X X -serde_json@1.0.124 X X +serde_derive@1.0.216 X X +serde_json@1.0.133 X X serde_repr@0.1.19 X X serde_urlencoded@0.7.1 X X -serde_with@3.9.0 X X -serde_with_macros@3.9.0 X X +serde_with@3.11.0 X X +serde_with_macros@3.11.0 X X sha1@0.10.6 X X sha2@0.10.8 X X shlex@1.3.0 X X slab@0.4.9 X smallvec@1.13.2 X X snap@1.1.1 X -socket2@0.5.7 X X +socket2@0.5.8 X X spin@0.9.8 X +stable_deref_trait@1.2.0 X X static_assertions@1.1.0 X X strsim@0.11.1 X strum@0.26.3 X strum_macros@0.26.4 X subtle@2.6.1 X -syn@2.0.74 X X -sync_wrapper@1.0.1 X +syn@2.0.90 X X +sync_wrapper@1.0.2 X +synstructure@0.13.1 X +tagptr@0.2.0 X X tap@1.0.1 X -thiserror@1.0.63 X X -thiserror-impl@1.0.63 X X +thiserror@1.0.69 X X +thiserror-impl@1.0.69 X X thrift@0.17.0 X tiny-keccak@2.0.2 X -tinyvec@1.8.0 X X X -tinyvec_macros@0.1.1 X X X -tokio@1.39.2 X +tinystr@0.7.6 X +tokio@1.42.0 X tokio-macros@2.4.0 X -tokio-rustls@0.26.0 X X -tokio-util@0.7.11 X -tower@0.4.13 X -tower-layer@0.3.3 X +tokio-rustls@0.26.1 X X +tokio-util@0.7.13 X tower-service@0.3.3 X -tracing@0.1.40 X -tracing-core@0.1.32 X +tracing@0.1.41 X +tracing-core@0.1.33 X +triomphe@0.1.11 X X try-lock@0.2.5 X twox-hash@1.6.3 X typed-builder@0.19.1 X X +typed-builder@0.20.0 X X typed-builder-macro@0.19.1 X X +typed-builder-macro@0.20.0 X X typenum@1.17.0 X X -unicode-bidi@0.3.15 X X -unicode-ident@1.0.12 X X X -unicode-normalization@0.1.23 X X +unicode-ident@1.0.14 X X X untrusted@0.9.0 X -url@2.5.2 X X +url@2.5.4 X X +utf16_iter@1.0.5 X X +utf8_iter@1.0.4 X X utf8parse@0.2.2 X X -uuid@1.10.0 X X +uuid@1.11.0 X X version_check@0.9.5 X X want@0.3.1 X wasi@0.11.0+wasi-snapshot-preview1 X X X -wasm-bindgen@0.2.93 X X -wasm-bindgen-backend@0.2.93 X X -wasm-bindgen-futures@0.4.43 X X -wasm-bindgen-macro@0.2.93 X X -wasm-bindgen-macro-support@0.2.93 X X -wasm-bindgen-shared@0.2.93 X X -wasm-streams@0.4.0 X X -web-sys@0.3.70 X X -webpki-roots@0.26.3 X +wasm-bindgen@0.2.99 X X +wasm-bindgen-backend@0.2.99 X X +wasm-bindgen-futures@0.4.49 X X +wasm-bindgen-macro@0.2.99 X X +wasm-bindgen-macro-support@0.2.99 X X +wasm-bindgen-shared@0.2.99 X X +wasm-streams@0.4.2 X X +web-sys@0.3.76 X X +webpki-roots@0.26.7 X +winapi@0.3.9 X X +winapi-i686-pc-windows-gnu@0.4.0 X X +winapi-x86_64-pc-windows-gnu@0.4.0 X X windows-core@0.52.0 X X -windows-sys@0.48.0 X X +windows-registry@0.2.0 X X +windows-result@0.2.0 X X +windows-strings@0.1.0 X X windows-sys@0.52.0 X X -windows-targets@0.48.5 X X +windows-sys@0.59.0 X X windows-targets@0.52.6 X X -windows_aarch64_gnullvm@0.48.5 X X windows_aarch64_gnullvm@0.52.6 X X -windows_aarch64_msvc@0.48.5 X X windows_aarch64_msvc@0.52.6 X X -windows_i686_gnu@0.48.5 X X windows_i686_gnu@0.52.6 X X windows_i686_gnullvm@0.52.6 X X -windows_i686_msvc@0.48.5 X X windows_i686_msvc@0.52.6 X X -windows_x86_64_gnu@0.48.5 X X windows_x86_64_gnu@0.52.6 X X -windows_x86_64_gnullvm@0.48.5 X X windows_x86_64_gnullvm@0.52.6 X X -windows_x86_64_msvc@0.48.5 X X windows_x86_64_msvc@0.52.6 X X -winreg@0.52.0 X +write16@1.0.0 X X +writeable@0.5.5 X wyz@0.5.1 X +yoke@0.7.5 X +yoke-derive@0.7.5 X zerocopy@0.7.35 X X X zerocopy-derive@0.7.35 X X X +zerofrom@0.1.5 X +zerofrom-derive@0.1.5 X zeroize@1.8.1 X X +zerovec@0.10.4 X +zerovec-derive@0.10.3 X zstd@0.13.2 X zstd-safe@7.2.1 X X -zstd-sys@2.0.12+zstd.1.5.6 X X +zstd-sys@2.0.13+zstd.1.5.6 X X diff --git a/crates/integrations/datafusion/DEPENDENCIES.rust.tsv b/crates/integrations/datafusion/DEPENDENCIES.rust.tsv index 2f6a5649e..9acf41f1e 100644 --- a/crates/integrations/datafusion/DEPENDENCIES.rust.tsv +++ b/crates/integrations/datafusion/DEPENDENCIES.rust.tsv @@ -1,208 +1,211 @@ -crate 0BSD Apache-2.0 Apache-2.0 WITH LLVM-exception BSD-2-Clause BSD-3-Clause BSL-1.0 CC0-1.0 ISC MIT MIT-0 MPL-2.0 OpenSSL Unicode-DFS-2016 Unlicense Zlib -addr2line@0.22.0 X X -adler@1.0.2 X X X +crate 0BSD Apache-2.0 Apache-2.0 WITH LLVM-exception BSD-2-Clause BSD-3-Clause BSL-1.0 CC0-1.0 ISC MIT MIT-0 MPL-2.0 OpenSSL Unicode-3.0 Unlicense Zlib +addr2line@0.24.2 X X +adler2@2.0.0 X X X adler32@1.2.0 X ahash@0.8.11 X X aho-corasick@1.1.3 X X alloc-no-stdlib@2.0.4 X alloc-stdlib@0.2.2 X -allocator-api2@0.2.18 X X +allocator-api2@0.2.21 X X android-tzdata@0.1.1 X X android_system_properties@0.1.5 X X -anstream@0.6.15 X X -anstyle@1.0.8 X X -anstyle-parse@0.2.5 X X -anstyle-query@1.1.1 X X -anstyle-wincon@3.0.4 X X -anyhow@1.0.86 X X +anstream@0.6.18 X X +anstyle@1.0.10 X X +anstyle-parse@0.2.6 X X +anstyle-query@1.1.2 X X +anstyle-wincon@3.0.6 X X +anyhow@1.0.94 X X apache-avro@0.17.0 X array-init@2.1.0 X X -arrayref@0.3.8 X -arrayvec@0.7.4 X X -arrow@52.2.0 X -arrow-arith@52.2.0 X -arrow-array@52.2.0 X -arrow-buffer@52.2.0 X -arrow-cast@52.2.0 X -arrow-csv@52.2.0 X -arrow-data@52.2.0 X -arrow-ipc@52.2.0 X -arrow-json@52.2.0 X -arrow-ord@52.2.0 X -arrow-row@52.2.0 X -arrow-schema@52.2.0 X -arrow-select@52.2.0 X -arrow-string@52.2.0 X -async-broadcast@0.7.1 X X -async-compression@0.4.12 X X -async-recursion@1.1.1 X X -async-trait@0.1.81 X X +arrayref@0.3.9 X +arrayvec@0.7.6 X X +arrow@53.3.0 X +arrow-arith@53.3.0 X +arrow-array@53.3.0 X +arrow-buffer@53.3.0 X +arrow-cast@53.3.0 X +arrow-csv@53.3.0 X +arrow-data@53.3.0 X +arrow-ipc@53.3.0 X +arrow-json@53.3.0 X +arrow-ord@53.3.0 X +arrow-row@53.3.0 X +arrow-schema@53.3.0 X +arrow-select@53.3.0 X +arrow-string@53.3.0 X +async-compression@0.4.18 X X +async-lock@3.4.0 X X +async-trait@0.1.83 X X atoi@2.0.0 X -autocfg@1.3.0 X X -backon@0.4.4 X -backtrace@0.3.73 X X +autocfg@1.4.0 X X +backon@1.3.0 X +backtrace@0.3.74 X X base64@0.22.1 X X -bigdecimal@0.4.5 X X +bigdecimal@0.4.7 X X bimap@0.6.3 X X bitflags@1.3.2 X X bitflags@2.6.0 X X bitvec@1.0.1 X blake2@0.10.6 X X -blake3@1.5.3 X X X +blake3@1.5.5 X X X block-buffer@0.10.4 X X -brotli@6.0.0 X X +brotli@7.0.0 X X brotli-decompressor@4.0.1 X X bumpalo@3.16.0 X X byteorder@1.5.0 X X -bytes@1.7.1 X +bytes@1.9.0 X bzip2@0.4.4 X X bzip2-sys@0.1.11+1.0.8 X X -cc@1.1.11 X X +cc@1.2.4 X X cfg-if@1.0.0 X X -cfg_aliases@0.1.1 X -chrono@0.4.38 X X -chrono-tz@0.9.0 X X -chrono-tz-build@0.3.0 X X -colorchoice@1.0.2 X X -comfy-table@7.1.1 X +chrono@0.4.39 X X +chrono-tz@0.10.0 X X +chrono-tz-build@0.4.0 X X +colorchoice@1.0.3 X X +comfy-table@7.1.3 X concurrent-queue@2.5.0 X X const-oid@0.9.6 X X const-random@0.1.18 X X const-random-macro@0.1.16 X X -constant_time_eq@0.3.0 X X X +constant_time_eq@0.3.1 X X X core-foundation-sys@0.8.7 X X core2@0.4.0 X X -cpufeatures@0.2.13 X X +cpufeatures@0.2.16 X X crc32c@0.6.8 X X crc32fast@1.4.2 X X -crossbeam-utils@0.8.20 X X +crossbeam-channel@0.5.14 X X +crossbeam-epoch@0.9.18 X X +crossbeam-utils@0.8.21 X X crunchy@0.2.2 X crypto-common@0.1.6 X X -csv@1.3.0 X X +csv@1.3.1 X X csv-core@0.1.11 X X darling@0.20.10 X darling_core@0.20.10 X darling_macro@0.20.10 X -dary_heap@0.3.6 X X -dashmap@5.5.3 X -dashmap@6.0.1 X -datafusion@41.0.0 X -datafusion-catalog@41.0.0 X -datafusion-common@41.0.0 X -datafusion-common-runtime@41.0.0 X -datafusion-execution@41.0.0 X -datafusion-expr@41.0.0 X -datafusion-functions@41.0.0 X -datafusion-functions-aggregate@41.0.0 X -datafusion-functions-nested@41.0.0 X -datafusion-optimizer@41.0.0 X -datafusion-physical-expr@41.0.0 X -datafusion-physical-expr-common@41.0.0 X -datafusion-physical-optimizer@41.0.0 X -datafusion-physical-plan@41.0.0 X -datafusion-sql@41.0.0 X -derivative@2.2.0 X X -derive_builder@0.20.0 X X -derive_builder_core@0.20.0 X X -derive_builder_macro@0.20.0 X X +dary_heap@0.3.7 X X +dashmap@6.1.0 X +datafusion@43.0.0 X +datafusion-catalog@43.0.0 X +datafusion-common@43.0.0 X +datafusion-common-runtime@43.0.0 X +datafusion-execution@43.0.0 X +datafusion-expr@43.0.0 X +datafusion-expr-common@43.0.0 X +datafusion-functions@43.0.0 X +datafusion-functions-aggregate@43.0.0 X +datafusion-functions-aggregate-common@43.0.0 X +datafusion-functions-nested@43.0.0 X +datafusion-functions-window@43.0.0 X +datafusion-functions-window-common@43.0.0 X +datafusion-optimizer@43.0.0 X +datafusion-physical-expr@43.0.0 X +datafusion-physical-expr-common@43.0.0 X +datafusion-physical-optimizer@43.0.0 X +datafusion-physical-plan@43.0.0 X +datafusion-sql@43.0.0 X +derive_builder@0.20.2 X X +derive_builder_core@0.20.2 X X +derive_builder_macro@0.20.2 X X digest@0.10.7 X X -doc-comment@0.3.3 X +displaydoc@0.2.5 X X either@1.13.0 X X env_filter@0.1.2 X X env_logger@0.11.5 X X equivalent@1.0.1 X X -errno@0.3.9 X X +errno@0.3.10 X X event-listener@5.3.1 X X -event-listener-strategy@0.5.2 X X -fastrand@2.1.0 X X -faststr@0.2.21 X X +event-listener-strategy@0.5.3 X X +fastrand@2.3.0 X X fixedbitset@0.4.2 X X flagset@0.4.6 X flatbuffers@24.3.25 X -flate2@1.0.31 X X +flate2@1.0.35 X X fnv@1.0.7 X X form_urlencoded@1.2.1 X X funty@2.0.0 X -futures@0.3.30 X X -futures-channel@0.3.30 X X -futures-core@0.3.30 X X -futures-executor@0.3.30 X X -futures-io@0.3.30 X X -futures-macro@0.3.30 X X -futures-sink@0.3.30 X X -futures-task@0.3.30 X X -futures-util@0.3.30 X X +futures@0.3.31 X X +futures-channel@0.3.31 X X +futures-core@0.3.31 X X +futures-executor@0.3.31 X X +futures-io@0.3.31 X X +futures-macro@0.3.31 X X +futures-sink@0.3.31 X X +futures-task@0.3.31 X X +futures-util@0.3.31 X X generic-array@0.14.7 X getrandom@0.2.15 X X -gimli@0.29.0 X X +gimli@0.31.1 X X glob@0.3.1 X X +gloo-timers@0.3.0 X X half@2.4.1 X X hashbrown@0.14.5 X X -heck@0.4.1 X X +hashbrown@0.15.2 X X heck@0.5.0 X X hermit-abi@0.3.9 X X hex@0.4.3 X X -hive_metastore@0.1.0 X hmac@0.12.1 X X -home@0.5.9 X X -http@1.1.0 X X +home@0.5.11 X X +http@1.2.0 X X http-body@1.0.1 X http-body-util@0.1.2 X -httparse@1.9.4 X X +httparse@1.9.5 X X humantime@2.1.0 X X -hyper@1.4.1 X -hyper-rustls@0.27.2 X X X -hyper-util@0.1.7 X -iana-time-zone@0.1.60 X X +hyper@1.5.2 X +hyper-rustls@0.27.3 X X X +hyper-util@0.1.10 X +iana-time-zone@0.1.61 X X iana-time-zone-haiku@0.1.2 X X -iceberg@0.3.0 X -iceberg-catalog-hms@0.3.0 X -iceberg-catalog-memory@0.3.0 X -iceberg-datafusion@0.3.0 X -iceberg_test_utils@0.3.0 X +iceberg@0.4.0 X +iceberg-catalog-memory@0.4.0 X +iceberg-datafusion@0.4.0 X +iceberg_test_utils@0.4.0 X +icu_collections@1.5.0 X +icu_locid@1.5.0 X +icu_locid_transform@1.5.0 X +icu_locid_transform_data@1.5.0 X +icu_normalizer@1.5.0 X +icu_normalizer_data@1.5.0 X +icu_properties@1.5.1 X +icu_properties_data@1.5.0 X +icu_provider@1.5.0 X +icu_provider_macros@1.5.0 X ident_case@1.0.1 X X -idna@0.5.0 X X -indexmap@2.4.0 X X +idna@1.0.3 X X +idna_adapter@1.2.0 X X +indexmap@2.7.0 X X instant@0.1.13 X integer-encoding@3.0.4 X -integer-encoding@4.0.2 X -ipnet@2.9.0 X X +ipnet@2.10.1 X X is_terminal_polyfill@1.70.1 X X -itertools@0.12.1 X X itertools@0.13.0 X X -itoa@1.0.11 X X +itoa@1.0.14 X X jobserver@0.1.32 X X -js-sys@0.3.70 X X +js-sys@0.3.76 X X lazy_static@1.5.0 X X -lexical-core@0.8.5 X X -lexical-parse-float@0.8.5 X X -lexical-parse-integer@0.8.6 X X -lexical-util@0.8.5 X X -lexical-write-float@0.8.5 X X -lexical-write-integer@0.8.5 X X -libc@0.2.155 X X +lexical-core@1.0.5 X X +lexical-parse-float@1.0.5 X X +lexical-parse-integer@1.0.5 X X +lexical-util@1.0.6 X X +lexical-write-float@1.0.5 X X +lexical-write-integer@1.0.5 X X +libc@0.2.168 X X libflate@2.1.0 X libflate_lz77@2.1.0 X -libm@0.2.8 X X -linked-hash-map@0.5.6 X X -linkedbytes@0.1.8 X X +libm@0.2.11 X X linux-raw-sys@0.4.14 X X X +litemap@0.7.4 X lock_api@0.4.12 X X log@0.4.22 X X lz4_flex@0.11.3 X lzma-sys@0.1.20 X X md-5@0.10.6 X X memchr@2.7.4 X X -memoffset@0.9.1 X -metainfo@0.7.12 X X mime@0.3.17 X X -miniz_oxide@0.7.4 X X X -mio@1.0.2 X -motore@0.4.1 X X -motore-macros@0.4.1 X X -mur3@0.1.0 X +miniz_oxide@0.8.0 X X X +mio@1.0.3 X +moka@0.12.8 X X murmur3@0.5.2 X X -nix@0.28.0 X num@0.4.3 X X num-bigint@0.4.6 X X num-complex@0.4.6 X X @@ -211,19 +214,16 @@ num-iter@0.1.45 X X num-rational@0.4.2 X X num-traits@0.2.19 X X num_cpus@1.16.0 X X -num_enum@0.7.3 X X X -num_enum_derive@0.7.3 X X X -object@0.36.3 X X -object_store@0.10.2 X X -once_cell@1.19.0 X X -opendal@0.49.0 X +object@0.36.5 X X +object_store@0.11.1 X X +once_cell@1.20.2 X X +opendal@0.50.2 X ordered-float@2.10.1 X -ordered-float@4.2.2 X -page_size@0.6.0 X X -parking@2.2.0 X X +ordered-float@4.5.0 X +parking@2.2.1 X X parking_lot@0.12.3 X X parking_lot_core@0.9.10 X X -parquet@52.2.0 X +parquet@53.3.0 X parse-zoneinfo@0.3.1 X paste@1.0.15 X X percent-encoding@2.3.1 X X @@ -232,160 +232,151 @@ phf@0.11.2 X phf_codegen@0.11.2 X phf_generator@0.11.2 X phf_shared@0.11.2 X -pilota@0.11.3 X X -pin-project@1.1.5 X X -pin-project-internal@1.1.5 X X -pin-project-lite@0.2.14 X X +pin-project-lite@0.2.15 X X pin-utils@0.1.0 X X -pkg-config@0.3.30 X X +pkg-config@0.3.31 X X ppv-lite86@0.2.20 X X -proc-macro-crate@3.1.0 X X -proc-macro2@1.0.86 X X -quad-rand@0.2.1 X -quick-xml@0.36.1 X -quote@1.0.36 X X +proc-macro2@1.0.92 X X +quad-rand@0.2.3 X +quanta@0.12.4 X +quick-xml@0.36.2 X +quote@1.0.37 X X radium@0.7.0 X rand@0.8.5 X X rand_chacha@0.3.1 X X rand_core@0.6.4 X X -redox_syscall@0.5.3 X -regex@1.10.6 X X -regex-automata@0.4.7 X X +raw-cpuid@11.2.0 X +redox_syscall@0.5.8 X +regex@1.11.1 X X +regex-automata@0.4.9 X X regex-lite@0.1.6 X X -regex-syntax@0.8.4 X X -reqsign@0.16.0 X -reqwest@0.12.5 X X +regex-syntax@0.8.5 X X +reqsign@0.16.1 X +reqwest@0.12.9 X X ring@0.17.8 X rle-decode-fast@1.0.3 X X -rust_decimal@1.35.0 X +rust_decimal@1.36.0 X rustc-demangle@0.1.24 X X -rustc-hash@2.0.0 X X -rustc_version@0.4.0 X X -rustix@0.38.34 X X X -rustls@0.23.12 X X X -rustls-pemfile@2.1.3 X X X -rustls-pki-types@1.8.0 X X -rustls-webpki@0.102.6 X -rustversion@1.0.17 X X +rustc_version@0.4.1 X X +rustix@0.38.42 X X X +rustls@0.23.20 X X X +rustls-pemfile@2.2.0 X X X +rustls-pki-types@1.10.1 X X +rustls-webpki@0.102.8 X +rustversion@1.0.18 X X ryu@1.0.18 X X same-file@1.0.6 X X scopeguard@1.2.0 X X -semver@1.0.23 X X +semver@1.0.24 X X seq-macro@0.3.5 X X -serde@1.0.207 X X +serde@1.0.216 X X serde_bytes@0.11.15 X X -serde_derive@1.0.207 X X -serde_json@1.0.124 X X +serde_derive@1.0.216 X X +serde_json@1.0.133 X X serde_repr@0.1.19 X X serde_urlencoded@0.7.1 X X -serde_with@3.9.0 X X -serde_with_macros@3.9.0 X X +serde_with@3.11.0 X X +serde_with_macros@3.11.0 X X sha1@0.10.6 X X sha2@0.10.8 X X shlex@1.3.0 X X -signal-hook-registry@1.4.2 X X -simdutf8@0.1.4 X X siphasher@0.3.11 X X slab@0.4.9 X smallvec@1.13.2 X X -snafu@0.7.5 X X -snafu-derive@0.7.5 X X +snafu@0.8.5 X X +snafu-derive@0.8.5 X X snap@1.1.1 X -socket2@0.5.7 X X -sonic-rs@0.3.10 X +socket2@0.5.8 X X spin@0.9.8 X -sqlparser@0.49.0 X +sqlparser@0.51.0 X sqlparser_derive@0.2.2 X +stable_deref_trait@1.2.0 X X static_assertions@1.1.0 X X strsim@0.11.1 X strum@0.26.3 X strum_macros@0.26.4 X subtle@2.6.1 X -syn@1.0.109 X X -syn@2.0.74 X X -sync_wrapper@1.0.1 X +syn@2.0.90 X X +sync_wrapper@1.0.2 X +synstructure@0.13.1 X +tagptr@0.2.0 X X tap@1.0.1 X -tempfile@3.12.0 X X -thiserror@1.0.63 X X -thiserror-impl@1.0.63 X X +tempfile@3.14.0 X X +thiserror@1.0.69 X X +thiserror-impl@1.0.69 X X thrift@0.17.0 X tiny-keccak@2.0.2 X -tinyvec@1.8.0 X X X -tinyvec_macros@0.1.1 X X X -tokio@1.39.2 X +tinystr@0.7.6 X +tokio@1.42.0 X tokio-macros@2.4.0 X -tokio-rustls@0.26.0 X X -tokio-stream@0.1.15 X -tokio-util@0.7.11 X -toml_datetime@0.6.8 X X -toml_edit@0.21.1 X X -tower@0.4.13 X -tower-layer@0.3.3 X +tokio-rustls@0.26.1 X X +tokio-util@0.7.13 X tower-service@0.3.3 X -tracing@0.1.40 X -tracing-attributes@0.1.27 X -tracing-core@0.1.32 X +tracing@0.1.41 X +tracing-attributes@0.1.28 X +tracing-core@0.1.33 X +triomphe@0.1.11 X X try-lock@0.2.5 X twox-hash@1.6.3 X typed-builder@0.19.1 X X +typed-builder@0.20.0 X X typed-builder-macro@0.19.1 X X +typed-builder-macro@0.20.0 X X typenum@1.17.0 X X -unicode-bidi@0.3.15 X X -unicode-ident@1.0.12 X X X -unicode-normalization@0.1.23 X X -unicode-segmentation@1.11.0 X X -unicode-width@0.1.13 X X +unicode-ident@1.0.14 X X X +unicode-segmentation@1.12.0 X X +unicode-width@0.2.0 X X untrusted@0.9.0 X -url@2.5.2 X X +url@2.5.4 X X +utf16_iter@1.0.5 X X +utf8_iter@1.0.4 X X utf8parse@0.2.2 X X -uuid@1.10.0 X X +uuid@1.11.0 X X version_check@0.9.5 X X -volo@0.10.1 X X -volo-thrift@0.10.2 X X walkdir@2.5.0 X X want@0.3.1 X wasi@0.11.0+wasi-snapshot-preview1 X X X -wasm-bindgen@0.2.93 X X -wasm-bindgen-backend@0.2.93 X X -wasm-bindgen-futures@0.4.43 X X -wasm-bindgen-macro@0.2.93 X X -wasm-bindgen-macro-support@0.2.93 X X -wasm-bindgen-shared@0.2.93 X X -wasm-streams@0.4.0 X X -web-sys@0.3.70 X X -webpki-roots@0.26.3 X +wasm-bindgen@0.2.99 X X +wasm-bindgen-backend@0.2.99 X X +wasm-bindgen-futures@0.4.49 X X +wasm-bindgen-macro@0.2.99 X X +wasm-bindgen-macro-support@0.2.99 X X +wasm-bindgen-shared@0.2.99 X X +wasm-streams@0.4.2 X X +web-sys@0.3.76 X X +webpki-roots@0.26.7 X winapi@0.3.9 X X winapi-i686-pc-windows-gnu@0.4.0 X X winapi-util@0.1.9 X X winapi-x86_64-pc-windows-gnu@0.4.0 X X windows-core@0.52.0 X X -windows-sys@0.48.0 X X +windows-registry@0.2.0 X X +windows-result@0.2.0 X X +windows-strings@0.1.0 X X windows-sys@0.52.0 X X windows-sys@0.59.0 X X -windows-targets@0.48.5 X X windows-targets@0.52.6 X X -windows_aarch64_gnullvm@0.48.5 X X windows_aarch64_gnullvm@0.52.6 X X -windows_aarch64_msvc@0.48.5 X X windows_aarch64_msvc@0.52.6 X X -windows_i686_gnu@0.48.5 X X windows_i686_gnu@0.52.6 X X windows_i686_gnullvm@0.52.6 X X -windows_i686_msvc@0.48.5 X X windows_i686_msvc@0.52.6 X X -windows_x86_64_gnu@0.48.5 X X windows_x86_64_gnu@0.52.6 X X -windows_x86_64_gnullvm@0.48.5 X X windows_x86_64_gnullvm@0.52.6 X X -windows_x86_64_msvc@0.48.5 X X windows_x86_64_msvc@0.52.6 X X -winnow@0.5.40 X -winreg@0.52.0 X +write16@1.0.0 X X +writeable@0.5.5 X wyz@0.5.1 X xz2@0.1.7 X X +yoke@0.7.5 X +yoke-derive@0.7.5 X zerocopy@0.7.35 X X X zerocopy-derive@0.7.35 X X X +zerofrom@0.1.5 X +zerofrom-derive@0.1.5 X zeroize@1.8.1 X X +zerovec@0.10.4 X +zerovec-derive@0.10.3 X zstd@0.13.2 X zstd-safe@7.2.1 X X -zstd-sys@2.0.12+zstd.1.5.6 X X +zstd-sys@2.0.13+zstd.1.5.6 X X diff --git a/scripts/dependencies.py b/scripts/dependencies.py index 1c8db96b9..32fff7cc6 100644 --- a/scripts/dependencies.py +++ b/scripts/dependencies.py @@ -27,8 +27,9 @@ "crates/catalog/memory", "crates/catalog/rest", "crates/catalog/sql", - "crates/integrations/datafusion" - + "crates/integrations/datafusion", + + "bindings/python" ] diff --git a/website/src/release.md b/website/src/release.md index a822b1e72..dc94bb68d 100644 --- a/website/src/release.md +++ b/website/src/release.md @@ -111,7 +111,7 @@ Please note that this version is the exact version of the release, not the relea ### Update docs -- Update `CHANGELOG.md`, refer to [Generate Release Note](reference/generate_release_note.md) for more information. +- Update `CHANGELOG.md` by Drafting a new release [note on Github Releases](https://github.com/apache/iceberg-rust/releases/new) ### Generate dependencies list From 74a85e7530e40c56a2a3075be63fd943e632501d Mon Sep 17 00:00:00 2001 From: Sung Yun <107272191+sungwy@users.noreply.github.com> Date: Tue, 17 Dec 2024 20:50:11 -0500 Subject: [PATCH 19/23] chmod+x (#817) --- scripts/verify.py | 0 1 file changed, 0 insertions(+), 0 deletions(-) mode change 100644 => 100755 scripts/verify.py diff --git a/scripts/verify.py b/scripts/verify.py old mode 100644 new mode 100755 From e1f24c1af466f128c7bd24333b7975f900def3fa Mon Sep 17 00:00:00 2001 From: Fokko Driesprong Date: Fri, 20 Dec 2024 06:15:37 +0100 Subject: [PATCH 20/23] Chore: Add `AboveMax` and `BelowMin` (#820) And add some nice `.to` conversions. --- crates/iceberg/src/expr/predicate.rs | 153 +++++++++++++++++- crates/iceberg/src/spec/values.rs | 231 +++++++++++++++++++++++++++ 2 files changed, 383 insertions(+), 1 deletion(-) diff --git a/crates/iceberg/src/expr/predicate.rs b/crates/iceberg/src/expr/predicate.rs index 76befb6d8..e0f6a7845 100644 --- a/crates/iceberg/src/expr/predicate.rs +++ b/crates/iceberg/src/expr/predicate.rs @@ -29,7 +29,7 @@ use serde::{Deserialize, Serialize}; use crate::error::Result; use crate::expr::{Bind, BoundReference, PredicateOperator, Reference}; -use crate::spec::{Datum, SchemaRef}; +use crate::spec::{Datum, PrimitiveLiteral, SchemaRef}; use crate::{Error, ErrorKind}; /// Logical expression, such as `AND`, `OR`, `NOT`. @@ -400,6 +400,37 @@ impl Bind for Predicate { Predicate::Binary(expr) => { let bound_expr = expr.bind(schema, case_sensitive)?; let bound_literal = bound_expr.literal.to(&bound_expr.term.field().field_type)?; + + match bound_literal.literal() { + PrimitiveLiteral::AboveMax => match &bound_expr.op { + &PredicateOperator::LessThan + | &PredicateOperator::LessThanOrEq + | &PredicateOperator::NotEq => { + return Ok(BoundPredicate::AlwaysTrue); + } + &PredicateOperator::GreaterThan + | &PredicateOperator::GreaterThanOrEq + | &PredicateOperator::Eq => { + return Ok(BoundPredicate::AlwaysFalse); + } + _ => {} + }, + PrimitiveLiteral::BelowMin => match &bound_expr.op { + &PredicateOperator::GreaterThan + | &PredicateOperator::GreaterThanOrEq + | &PredicateOperator::NotEq => { + return Ok(BoundPredicate::AlwaysTrue); + } + &PredicateOperator::LessThan + | &PredicateOperator::LessThanOrEq + | &PredicateOperator::Eq => { + return Ok(BoundPredicate::AlwaysFalse); + } + _ => {} + }, + _ => {} + } + Ok(BoundPredicate::Binary(BinaryExpression::new( bound_expr.op, bound_expr.term, @@ -1086,6 +1117,126 @@ mod tests { test_bound_predicate_serialize_diserialize(bound_expr); } + #[test] + fn test_bind_equal_to_above_max() { + let schema = table_schema_simple(); + // int32 can hold up to 2147483647 + let expr = Reference::new("bar").equal_to(Datum::long(2147483648i64)); + let bound_expr = expr.bind(schema, true).unwrap(); + assert_eq!(&format!("{bound_expr}"), "False"); + test_bound_predicate_serialize_diserialize(bound_expr); + } + + #[test] + fn test_bind_equal_to_below_min() { + let schema = table_schema_simple(); + // int32 can hold up to -2147483647 + let expr = Reference::new("bar").equal_to(Datum::long(-2147483649i64)); + let bound_expr = expr.bind(schema, true).unwrap(); + assert_eq!(&format!("{bound_expr}"), "False"); + test_bound_predicate_serialize_diserialize(bound_expr); + } + + #[test] + fn test_bind_not_equal_to_above_max() { + let schema = table_schema_simple(); + // int32 can hold up to 2147483647 + let expr = Reference::new("bar").not_equal_to(Datum::long(2147483648i64)); + let bound_expr = expr.bind(schema, true).unwrap(); + assert_eq!(&format!("{bound_expr}"), "True"); + test_bound_predicate_serialize_diserialize(bound_expr); + } + + #[test] + fn test_bind_not_equal_to_below_min() { + let schema = table_schema_simple(); + // int32 can hold up to -2147483647 + let expr = Reference::new("bar").not_equal_to(Datum::long(-2147483649i64)); + let bound_expr = expr.bind(schema, true).unwrap(); + assert_eq!(&format!("{bound_expr}"), "True"); + test_bound_predicate_serialize_diserialize(bound_expr); + } + + #[test] + fn test_bind_less_than_above_max() { + let schema = table_schema_simple(); + // int32 can hold up to 2147483647 + let expr = Reference::new("bar").less_than(Datum::long(2147483648i64)); + let bound_expr = expr.bind(schema, true).unwrap(); + assert_eq!(&format!("{bound_expr}"), "True"); + test_bound_predicate_serialize_diserialize(bound_expr); + } + + #[test] + fn test_bind_less_than_below_min() { + let schema = table_schema_simple(); + // int32 can hold up to -2147483647 + let expr = Reference::new("bar").less_than(Datum::long(-2147483649i64)); + let bound_expr = expr.bind(schema, true).unwrap(); + assert_eq!(&format!("{bound_expr}"), "False"); + test_bound_predicate_serialize_diserialize(bound_expr); + } + + #[test] + fn test_bind_less_than_or_equal_to_above_max() { + let schema = table_schema_simple(); + // int32 can hold up to 2147483647 + let expr = Reference::new("bar").less_than_or_equal_to(Datum::long(2147483648i64)); + let bound_expr = expr.bind(schema, true).unwrap(); + assert_eq!(&format!("{bound_expr}"), "True"); + test_bound_predicate_serialize_diserialize(bound_expr); + } + + #[test] + fn test_bind_less_than_or_equal_to_below_min() { + let schema = table_schema_simple(); + // int32 can hold up to -2147483647 + let expr = Reference::new("bar").less_than_or_equal_to(Datum::long(-2147483649i64)); + let bound_expr = expr.bind(schema, true).unwrap(); + assert_eq!(&format!("{bound_expr}"), "False"); + test_bound_predicate_serialize_diserialize(bound_expr); + } + + #[test] + fn test_bind_great_than_above_max() { + let schema = table_schema_simple(); + // int32 can hold up to 2147483647 + let expr = Reference::new("bar").greater_than(Datum::long(2147483648i64)); + let bound_expr = expr.bind(schema, true).unwrap(); + assert_eq!(&format!("{bound_expr}"), "False"); + test_bound_predicate_serialize_diserialize(bound_expr); + } + + #[test] + fn test_bind_great_than_below_min() { + let schema = table_schema_simple(); + // int32 can hold up to -2147483647 + let expr = Reference::new("bar").greater_than(Datum::long(-2147483649i64)); + let bound_expr = expr.bind(schema, true).unwrap(); + assert_eq!(&format!("{bound_expr}"), "True"); + test_bound_predicate_serialize_diserialize(bound_expr); + } + + #[test] + fn test_bind_great_than_or_equal_to_above_max() { + let schema = table_schema_simple(); + // int32 can hold up to 2147483647 + let expr = Reference::new("bar").greater_than_or_equal_to(Datum::long(2147483648i64)); + let bound_expr = expr.bind(schema, true).unwrap(); + assert_eq!(&format!("{bound_expr}"), "False"); + test_bound_predicate_serialize_diserialize(bound_expr); + } + + #[test] + fn test_bind_great_than_or_equal_to_below_min() { + let schema = table_schema_simple(); + // int32 can hold up to -2147483647 + let expr = Reference::new("bar").greater_than_or_equal_to(Datum::long(-2147483649i64)); + let bound_expr = expr.bind(schema, true).unwrap(); + assert_eq!(&format!("{bound_expr}"), "True"); + test_bound_predicate_serialize_diserialize(bound_expr); + } + #[test] fn test_bind_equal_to_wrong_type() { let schema = table_schema_simple(); diff --git a/crates/iceberg/src/spec/values.rs b/crates/iceberg/src/spec/values.rs index 38326b808..f92ca263d 100644 --- a/crates/iceberg/src/spec/values.rs +++ b/crates/iceberg/src/spec/values.rs @@ -55,6 +55,11 @@ use crate::{ensure_data_valid, Error, ErrorKind}; /// Maximum value for [`PrimitiveType::Time`] type in microseconds, e.g. 23 hours 59 minutes 59 seconds 999999 microseconds. const MAX_TIME_VALUE: i64 = 24 * 60 * 60 * 1_000_000i64 - 1; +const INT_MAX: i32 = 2147483647; +const INT_MIN: i32 = -2147483648; +const LONG_MAX: i64 = 9223372036854775807; +const LONG_MIN: i64 = -9223372036854775808; + /// Values present in iceberg type #[derive(Clone, Debug, PartialEq, Hash, Eq)] pub enum PrimitiveLiteral { @@ -76,6 +81,10 @@ pub enum PrimitiveLiteral { Int128(i128), /// Stored as 16-byte little-endian UInt128(u128), + /// When a number is larger than it can hold + AboveMax, + /// When a number is smaller than it can hold + BelowMin, } impl PrimitiveLiteral { @@ -489,6 +498,12 @@ impl Datum { ByteBuf::from(bytes) } + PrimitiveLiteral::AboveMax | PrimitiveLiteral::BelowMin => { + return Err(Error::new( + ErrorKind::DataInvalid, + "Cannot convert AboveMax or BelowMin to bytes".to_string(), + )); + } }; Ok(buf) @@ -1093,6 +1108,42 @@ impl Datum { } } + fn i64_to_i32 + PartialOrd>(val: T) -> Datum { + if val > INT_MAX as i64 { + Datum::new(PrimitiveType::Int, PrimitiveLiteral::AboveMax) + } else if val < INT_MIN as i64 { + Datum::new(PrimitiveType::Int, PrimitiveLiteral::BelowMin) + } else { + Datum::int(val.into() as i32) + } + } + + fn i128_to_i32 + PartialOrd>(val: T) -> Datum { + if val > INT_MAX as i128 { + Datum::new(PrimitiveType::Int, PrimitiveLiteral::AboveMax) + } else if val < INT_MIN as i128 { + Datum::new(PrimitiveType::Int, PrimitiveLiteral::BelowMin) + } else { + Datum::int(val.into() as i32) + } + } + + fn i128_to_i64 + PartialOrd>(val: T) -> Datum { + if val > LONG_MAX as i128 { + Datum::new(PrimitiveType::Long, PrimitiveLiteral::AboveMax) + } else if val < LONG_MIN as i128 { + Datum::new(PrimitiveType::Long, PrimitiveLiteral::BelowMin) + } else { + Datum::long(val.into() as i64) + } + } + + fn string_to_i128>(s: S) -> Result { + return s.as_ref().parse::().map_err(|e| { + Error::new(ErrorKind::DataInvalid, "Can't parse string to i128.").with_source(e) + }); + } + /// Convert the datum to `target_type`. pub fn to(self, target_type: &Type) -> Result { match target_type { @@ -1101,6 +1152,36 @@ impl Datum { (PrimitiveLiteral::Int(val), _, PrimitiveType::Int) => Ok(Datum::int(*val)), (PrimitiveLiteral::Int(val), _, PrimitiveType::Date) => Ok(Datum::date(*val)), (PrimitiveLiteral::Int(val), _, PrimitiveType::Long) => Ok(Datum::long(*val)), + (PrimitiveLiteral::Long(val), _, PrimitiveType::Int) => { + Ok(Datum::i64_to_i32(*val)) + } + (PrimitiveLiteral::Long(val), _, PrimitiveType::Timestamp) => { + Ok(Datum::timestamp_micros(*val)) + } + (PrimitiveLiteral::Long(val), _, PrimitiveType::Timestamptz) => { + Ok(Datum::timestamptz_micros(*val)) + } + // Let's wait with nano's until this clears up: https://github.com/apache/iceberg/pull/11775 + (PrimitiveLiteral::Int128(val), _, PrimitiveType::Long) => { + Ok(Datum::i128_to_i64(*val)) + } + + (PrimitiveLiteral::String(val), _, PrimitiveType::Boolean) => { + Datum::bool_from_str(val) + } + (PrimitiveLiteral::String(val), _, PrimitiveType::Int) => { + Datum::string_to_i128(val).map(Datum::i128_to_i32) + } + (PrimitiveLiteral::String(val), _, PrimitiveType::Long) => { + Datum::string_to_i128(val).map(Datum::i128_to_i64) + } + (PrimitiveLiteral::String(val), _, PrimitiveType::Timestamp) => { + Datum::timestamp_from_str(val) + } + (PrimitiveLiteral::String(val), _, PrimitiveType::Timestamptz) => { + Datum::timestamptz_from_str(val) + } + // TODO: implement more type conversions (_, self_type, target_type) if self_type == target_type => Ok(self), _ => Err(Error::new( @@ -2047,6 +2128,7 @@ impl Literal { PrimitiveLiteral::String(any) => Box::new(any), PrimitiveLiteral::UInt128(any) => Box::new(any), PrimitiveLiteral::Int128(any) => Box::new(any), + PrimitiveLiteral::AboveMax | PrimitiveLiteral::BelowMin => unimplemented!(), }, _ => unimplemented!(), } @@ -2365,6 +2447,12 @@ mod _serde { super::PrimitiveLiteral::Int128(v) => { RawLiteralEnum::Bytes(ByteBuf::from(v.to_be_bytes())) } + super::PrimitiveLiteral::AboveMax | super::PrimitiveLiteral::BelowMin => { + return Err(Error::new( + ErrorKind::DataInvalid, + "Can't convert AboveMax or BelowMax", + )); + } }, Literal::Struct(r#struct) => { let mut required = Vec::new(); @@ -3604,4 +3692,147 @@ mod tests { assert_eq!(result, expected); } + + #[test] + fn test_datum_long_convert_to_int() { + let datum = Datum::long(12345); + + let result = datum.to(&Primitive(PrimitiveType::Int)).unwrap(); + + let expected = Datum::int(12345); + + assert_eq!(result, expected); + } + + #[test] + fn test_datum_long_convert_to_int_above_max() { + let datum = Datum::long(INT_MAX as i64 + 1); + + let result = datum.to(&Primitive(PrimitiveType::Int)).unwrap(); + + let expected = Datum::new(PrimitiveType::Int, PrimitiveLiteral::AboveMax); + + assert_eq!(result, expected); + } + + #[test] + fn test_datum_long_convert_to_int_below_min() { + let datum = Datum::long(INT_MIN as i64 - 1); + + let result = datum.to(&Primitive(PrimitiveType::Int)).unwrap(); + + let expected = Datum::new(PrimitiveType::Int, PrimitiveLiteral::BelowMin); + + assert_eq!(result, expected); + } + + #[test] + fn test_datum_long_convert_to_timestamp() { + let datum = Datum::long(12345); + + let result = datum.to(&Primitive(PrimitiveType::Timestamp)).unwrap(); + + let expected = Datum::timestamp_micros(12345); + + assert_eq!(result, expected); + } + + #[test] + fn test_datum_long_convert_to_timestamptz() { + let datum = Datum::long(12345); + + let result = datum.to(&Primitive(PrimitiveType::Timestamptz)).unwrap(); + + let expected = Datum::timestamptz_micros(12345); + + assert_eq!(result, expected); + } + + #[test] + fn test_datum_decimal_convert_to_long() { + let datum = Datum::decimal(12345).unwrap(); + + let result = datum.to(&Primitive(PrimitiveType::Long)).unwrap(); + + let expected = Datum::long(12345); + + assert_eq!(result, expected); + } + + #[test] + fn test_datum_decimal_convert_to_long_above_max() { + let datum = Datum::decimal(LONG_MAX as i128 + 1).unwrap(); + + let result = datum.to(&Primitive(PrimitiveType::Long)).unwrap(); + + let expected = Datum::new(PrimitiveType::Long, PrimitiveLiteral::AboveMax); + + assert_eq!(result, expected); + } + + #[test] + fn test_datum_decimal_convert_to_long_below_min() { + let datum = Datum::decimal(LONG_MIN as i128 - 1).unwrap(); + + let result = datum.to(&Primitive(PrimitiveType::Long)).unwrap(); + + let expected = Datum::new(PrimitiveType::Long, PrimitiveLiteral::BelowMin); + + assert_eq!(result, expected); + } + + #[test] + fn test_datum_string_convert_to_boolean() { + let datum = Datum::string("true"); + + let result = datum.to(&Primitive(PrimitiveType::Boolean)).unwrap(); + + let expected = Datum::bool(true); + + assert_eq!(result, expected); + } + + #[test] + fn test_datum_string_convert_to_int() { + let datum = Datum::string("12345"); + + let result = datum.to(&Primitive(PrimitiveType::Int)).unwrap(); + + let expected = Datum::int(12345); + + assert_eq!(result, expected); + } + + #[test] + fn test_datum_string_convert_to_long() { + let datum = Datum::string("12345"); + + let result = datum.to(&Primitive(PrimitiveType::Long)).unwrap(); + + let expected = Datum::long(12345); + + assert_eq!(result, expected); + } + + #[test] + fn test_datum_string_convert_to_timestamp() { + let datum = Datum::string("1925-05-20T19:25:00.000"); + + let result = datum.to(&Primitive(PrimitiveType::Timestamp)).unwrap(); + + let expected = Datum::timestamp_micros(-1407990900000000); + + assert_eq!(result, expected); + } + + #[test] + fn test_datum_string_convert_to_timestamptz() { + let datum = Datum::string("1925-05-20T19:25:00.000 UTC"); + + let result = datum.to(&Primitive(PrimitiveType::Timestamptz)).unwrap(); + + let expected = Datum::timestamptz_micros(-1407990900000000); + + assert_eq!(result, expected); + } } From 0777fa7349cd04dbe2a64805d0505e6465f94ad6 Mon Sep 17 00:00:00 2001 From: Fokko Driesprong Date: Fri, 20 Dec 2024 08:21:51 +0100 Subject: [PATCH 21/23] fix: Reading a table with positional deletes should fail (#826) * A table with positional deletes shoulds fail * Add possible fix * Comment and refactor --- crates/iceberg/src/scan.rs | 22 ++++++++++++------- .../testdata/spark/entrypoint.sh | 2 ++ .../testdata/spark/provision.py | 12 +++++++++- .../tests/read_positional_deletes.rs | 19 ++++++++++++++-- 4 files changed, 44 insertions(+), 11 deletions(-) diff --git a/crates/iceberg/src/scan.rs b/crates/iceberg/src/scan.rs index 89cc21bbf..1366d9414 100644 --- a/crates/iceberg/src/scan.rs +++ b/crates/iceberg/src/scan.rs @@ -364,9 +364,8 @@ impl TableScan { let manifest_list = self.plan_context.get_manifest_list().await?; - // get the [`ManifestFile`]s from the [`ManifestList`], filtering out any - // whose content type is not Data or whose partitions cannot match this - // scan's filter + // get the [`ManifestFile`]s from the [`ManifestList`], filtering out + // partitions cannot match the scan's filter let manifest_file_contexts = self .plan_context .build_manifest_file_contexts(manifest_list, manifest_entry_ctx_tx)?; @@ -619,15 +618,22 @@ impl PlanContext { manifest_list: Arc, sender: Sender, ) -> Result>>> { - let filtered_entries = manifest_list - .entries() + let entries = manifest_list.entries(); + + if entries .iter() - .filter(|manifest_file| manifest_file.content == ManifestContentType::Data); + .any(|e| e.content != ManifestContentType::Data) + { + return Err(Error::new( + ErrorKind::FeatureUnsupported, + "Merge-on-read is not yet supported", + )); + } // TODO: Ideally we could ditch this intermediate Vec as we return an iterator. let mut filtered_mfcs = vec![]; if self.predicate.is_some() { - for manifest_file in filtered_entries { + for manifest_file in entries { let partition_bound_predicate = self.get_partition_filter(manifest_file)?; // evaluate the ManifestFile against the partition filter. Skip @@ -649,7 +655,7 @@ impl PlanContext { } } } else { - for manifest_file in filtered_entries { + for manifest_file in entries { let mfc = self.create_manifest_file_context(manifest_file, None, sender.clone()); filtered_mfcs.push(Ok(mfc)); } diff --git a/crates/integration_tests/testdata/spark/entrypoint.sh b/crates/integration_tests/testdata/spark/entrypoint.sh index abbcc9332..90db14d0b 100755 --- a/crates/integration_tests/testdata/spark/entrypoint.sh +++ b/crates/integration_tests/testdata/spark/entrypoint.sh @@ -18,6 +18,8 @@ # under the License. # +set -e + start-master.sh -p 7077 start-worker.sh spark://spark-iceberg:7077 start-history-server.sh diff --git a/crates/integration_tests/testdata/spark/provision.py b/crates/integration_tests/testdata/spark/provision.py index 1d5ffcb85..364e366b4 100755 --- a/crates/integration_tests/testdata/spark/provision.py +++ b/crates/integration_tests/testdata/spark/provision.py @@ -18,7 +18,17 @@ from pyspark.sql import SparkSession from pyspark.sql.functions import current_date, date_add, expr -spark = SparkSession.builder.getOrCreate() +# The configuration is important, otherwise we get many small +# parquet files with a single row. When a positional delete +# hits the Parquet file with one row, the parquet file gets +# dropped instead of having a merge-on-read delete file. +spark = ( + SparkSession + .builder + .config("spark.sql.shuffle.partitions", "1") + .config("spark.default.parallelism", "1") + .getOrCreate() +) spark.sql( f""" diff --git a/crates/integration_tests/tests/read_positional_deletes.rs b/crates/integration_tests/tests/read_positional_deletes.rs index ebd5ea67f..41ca057a6 100644 --- a/crates/integration_tests/tests/read_positional_deletes.rs +++ b/crates/integration_tests/tests/read_positional_deletes.rs @@ -17,6 +17,7 @@ //! Integration tests for rest catalog. +use iceberg::ErrorKind::FeatureUnsupported; use iceberg::{Catalog, TableIdent}; use iceberg_integration_tests::set_test_fixture; @@ -34,6 +35,20 @@ async fn test_read_table_with_positional_deletes() { .await .unwrap(); - // 😱 If we don't support positional deletes, we should not be able to plan them - println!("{:?}", table.scan().build().unwrap()); + let scan = table.scan().build().unwrap(); + println!("{:?}", scan); + + assert!(scan + .to_arrow() + .await + .is_err_and(|e| e.kind() == FeatureUnsupported)); + + // 😱 If we don't support positional deletes, we should fail when we try to read a table that + // has positional deletes. The table has 12 rows, and 2 are deleted, see provision.py + + // When we get support for it: + // let batch_stream = scan.to_arrow().await.unwrap(); + // let batches: Vec<_> = batch_stream.try_collect().await.is_err(); + // let num_rows: usize = batches.iter().map(|v| v.num_rows()).sum(); + // assert_eq!(num_rows, 10); } From 982db0e2cbdf59d2c3b4d2d7d5b7c45393dbb2e6 Mon Sep 17 00:00:00 2001 From: Sung Yun <107272191+sungwy@users.noreply.github.com> Date: Fri, 20 Dec 2024 09:37:18 -0500 Subject: [PATCH 22/23] chore: Updated Changelog for 0.4.0-rc3 (#830) * add updated changelog for 0.4.0rc3 * leave out #820 --- CHANGELOG.md | 2 ++ 1 file changed, 2 insertions(+) diff --git a/CHANGELOG.md b/CHANGELOG.md index 7c3a3d68b..70e6f36d3 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -160,6 +160,8 @@ and this project adheres to [Semantic Versioning](https://semver.org/). * Prep 0.4.0 release by @sungwy in https://github.com/apache/iceberg-rust/pull/809 * feat: Add RemovePartitionSpecs table update by @c-thiel in https://github.com/apache/iceberg-rust/pull/804 * feat: Store file io props to allow re-build it by @Xuanwo in https://github.com/apache/iceberg-rust/pull/802 +* chore: chmod +x on `verify.py` script by @sungwy in https://github.com/apache/iceberg-rust/pull/817 +* fix: Reading a table with positional deletes should fail by @Fokko in https://github.com/apache/iceberg-rust/pull/826 ## [v0.3.0] - 2024-08-14 From c8f5d91239951dee0b37594dfdb6f7f9bc74258c Mon Sep 17 00:00:00 2001 From: Marko Grujic Date: Fri, 20 Dec 2024 16:09:42 +0100 Subject: [PATCH 23/23] Align schemas for DataFusion plan and stream (#829) --- .../datafusion/src/physical_plan/scan.rs | 5 +--- .../tests/integration_datafusion_test.rs | 25 +++++++++++++++++-- 2 files changed, 24 insertions(+), 6 deletions(-) diff --git a/crates/integrations/datafusion/src/physical_plan/scan.rs b/crates/integrations/datafusion/src/physical_plan/scan.rs index a51048428..eaf2f94e5 100644 --- a/crates/integrations/datafusion/src/physical_plan/scan.rs +++ b/crates/integrations/datafusion/src/physical_plan/scan.rs @@ -45,8 +45,6 @@ pub(crate) struct IcebergTableScan { table: Table, /// Snapshot of the table to scan. snapshot_id: Option, - /// A reference-counted arrow `Schema`. - schema: ArrowSchemaRef, /// Stores certain, often expensive to compute, /// plan properties used in query optimization. plan_properties: PlanProperties, @@ -76,7 +74,6 @@ impl IcebergTableScan { Self { table, snapshot_id, - schema, plan_properties, projection, predicates, @@ -134,7 +131,7 @@ impl ExecutionPlan for IcebergTableScan { let stream = futures::stream::once(fut).try_flatten(); Ok(Box::pin(RecordBatchStreamAdapter::new( - self.schema.clone(), + self.schema(), stream, ))) } diff --git a/crates/integrations/datafusion/tests/integration_datafusion_test.rs b/crates/integrations/datafusion/tests/integration_datafusion_test.rs index d320c8ef0..715635e06 100644 --- a/crates/integrations/datafusion/tests/integration_datafusion_test.rs +++ b/crates/integrations/datafusion/tests/integration_datafusion_test.rs @@ -22,8 +22,9 @@ use std::sync::Arc; use std::vec; use datafusion::arrow::array::{Array, StringArray}; -use datafusion::arrow::datatypes::DataType; +use datafusion::arrow::datatypes::{DataType, Field, Schema as ArrowSchema}; use datafusion::execution::context::SessionContext; +use datafusion::parquet::arrow::PARQUET_FIELD_ID_META_KEY; use iceberg::io::FileIOBuilder; use iceberg::spec::{NestedField, PrimitiveType, Schema, StructType, Type}; use iceberg::{Catalog, NamespaceIdent, Result, TableCreation}; @@ -83,7 +84,7 @@ fn get_table_creation( } #[tokio::test] -async fn test_provider_get_table_schema() -> Result<()> { +async fn test_provider_plan_stream_schema() -> Result<()> { let iceberg_catalog = get_iceberg_catalog(); let namespace = NamespaceIdent::new("test_provider_get_table_schema".to_string()); set_test_namespace(&iceberg_catalog, &namespace).await?; @@ -111,6 +112,26 @@ async fn test_provider_get_table_schema() -> Result<()> { assert!(!field.is_nullable()) } + let df = ctx + .sql("select foo2 from catalog.test_provider_get_table_schema.my_table") + .await + .unwrap(); + + let task_ctx = Arc::new(df.task_ctx()); + let plan = df.create_physical_plan().await.unwrap(); + let stream = plan.execute(1, task_ctx).unwrap(); + + // Ensure both the plan and the stream conform to the same schema + assert_eq!(plan.schema(), stream.schema()); + assert_eq!( + stream.schema().as_ref(), + &ArrowSchema::new(vec![Field::new("foo2", DataType::Utf8, false) + .with_metadata(HashMap::from([( + PARQUET_FIELD_ID_META_KEY.to_string(), + "2".to_string(), + )]))]), + ); + Ok(()) }