From 856597b60a846b9287788e35f5b4e113e71e2516 Mon Sep 17 00:00:00 2001 From: Shawn Chang Date: Fri, 17 Oct 2025 03:02:33 -0700 Subject: [PATCH 01/47] feat(writer): Add clustered and fanout writer (#1735) ## Which issue does this PR close? - Closes #1572 #1573 ## What changes are included in this PR? New: - Added new `partitioning` module with `PartitioningWriter` trait - `ClusteredWriter`: Optimized for pre-sorted data, requires writing in partition order - `FanoutWriter`: Flexible writer that can handle data from any partition at any time Modification: - (BREAKING) Modified `DataFileWriterBuilder` to support dynamic partition assignment - Updated DataFusion integration to use the new writer API ## Are these changes tested? Added unit tests --- .../writer/base_writer/data_file_writer.rs | 26 +- .../base_writer/equality_delete_writer.rs | 42 +- .../src/writer/file_writer/rolling_writer.rs | 9 +- crates/iceberg/src/writer/mod.rs | 28 +- .../writer/partitioning/clustered_writer.rs | 520 ++++++++++++++++++ .../src/writer/partitioning/fanout_writer.rs | 385 +++++++++++++ crates/iceberg/src/writer/partitioning/mod.rs | 49 ++ .../shared_tests/append_data_file_test.rs | 4 +- .../append_partition_data_file_test.rs | 30 +- .../shared_tests/conflict_commit_test.rs | 4 +- .../tests/shared_tests/scan_all_type.rs | 4 +- .../datafusion/src/physical_plan/write.rs | 7 +- 12 files changed, 1024 insertions(+), 84 deletions(-) create mode 100644 crates/iceberg/src/writer/partitioning/clustered_writer.rs create mode 100644 crates/iceberg/src/writer/partitioning/fanout_writer.rs create mode 100644 crates/iceberg/src/writer/partitioning/mod.rs 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 a950547d33..b9de4efda7 100644 --- a/crates/iceberg/src/writer/base_writer/data_file_writer.rs +++ b/crates/iceberg/src/writer/base_writer/data_file_writer.rs @@ -30,7 +30,6 @@ use crate::{Error, ErrorKind, Result}; #[derive(Clone, Debug)] pub struct DataFileWriterBuilder { inner: RollingFileWriterBuilder, - partition_key: Option, } impl DataFileWriterBuilder @@ -40,14 +39,8 @@ where F: FileNameGenerator, { /// Create a new `DataFileWriterBuilder` using a `RollingFileWriterBuilder`. - pub fn new( - inner_builder: RollingFileWriterBuilder, - partition_key: Option, - ) -> Self { - Self { - inner: inner_builder, - partition_key, - } + pub fn new(inner: RollingFileWriterBuilder) -> Self { + Self { inner } } } @@ -60,10 +53,10 @@ where { type R = DataFileWriter; - async fn build(self) -> Result { + async fn build(self, partition_key: Option) -> Result { Ok(DataFileWriter { inner: Some(self.inner.clone().build()), - partition_key: self.partition_key, + partition_key, }) } } @@ -194,8 +187,8 @@ mod test { file_name_gen, ); - let mut data_file_writer = DataFileWriterBuilder::new(rolling_file_writer_builder, None) - .build() + let mut data_file_writer = DataFileWriterBuilder::new(rolling_file_writer_builder) + .build(None) .await .unwrap(); @@ -280,10 +273,9 @@ mod test { file_name_gen, ); - let mut data_file_writer = - DataFileWriterBuilder::new(rolling_file_writer_builder, Some(partition_key)) - .build() - .await?; + let mut data_file_writer = DataFileWriterBuilder::new(rolling_file_writer_builder) + .build(Some(partition_key)) + .await?; let arrow_schema = arrow_schema::Schema::new(vec![ Field::new("id", DataType::Int32, false).with_metadata(HashMap::from([( diff --git a/crates/iceberg/src/writer/base_writer/equality_delete_writer.rs b/crates/iceberg/src/writer/base_writer/equality_delete_writer.rs index 6740ed435c..bd96c80bb3 100644 --- a/crates/iceberg/src/writer/base_writer/equality_delete_writer.rs +++ b/crates/iceberg/src/writer/base_writer/equality_delete_writer.rs @@ -66,16 +66,11 @@ pub struct EqualityDeleteWriterConfig { equality_ids: Vec, // Projector used to project the data chunk into specific fields. projector: RecordBatchProjector, - partition_key: Option, } impl EqualityDeleteWriterConfig { /// Create a new `DataFileWriterConfig` with equality ids. - pub fn new( - equality_ids: Vec, - original_schema: SchemaRef, - partition_key: Option, - ) -> Result { + pub fn new(equality_ids: Vec, original_schema: SchemaRef) -> Result { let original_arrow_schema = Arc::new(schema_to_arrow_schema(&original_schema)?); let projector = RecordBatchProjector::new( original_arrow_schema, @@ -110,7 +105,6 @@ impl EqualityDeleteWriterConfig { Ok(Self { equality_ids, projector, - partition_key, }) } @@ -129,12 +123,12 @@ where { type R = EqualityDeleteFileWriter; - async fn build(self) -> Result { + async fn build(self, partition_key: Option) -> Result { Ok(EqualityDeleteFileWriter { - inner: Some(self.inner.clone().build()), // todo revisit this, probably still need a builder for rolling writer + inner: Some(self.inner.clone().build()), projector: self.config.projector, equality_ids: self.config.equality_ids, - partition_key: self.config.partition_key, + partition_key, }) } } @@ -428,7 +422,7 @@ mod test { let equality_ids = vec![0_i32, 8]; let equality_config = - EqualityDeleteWriterConfig::new(equality_ids, Arc::new(schema), None).unwrap(); + EqualityDeleteWriterConfig::new(equality_ids, Arc::new(schema)).unwrap(); let delete_schema = arrow_schema_to_schema(equality_config.projected_arrow_schema_ref()).unwrap(); let projector = equality_config.projector.clone(); @@ -444,7 +438,7 @@ mod test { ); let mut equality_delete_writer = EqualityDeleteFileWriterBuilder::new(rolling_writer_builder, equality_config) - .build() + .build(None) .await?; // write @@ -531,19 +525,19 @@ mod test { .unwrap(), ); // Float and Double are not allowed to be used for equality delete - assert!(EqualityDeleteWriterConfig::new(vec![0], schema.clone(), None).is_err()); - assert!(EqualityDeleteWriterConfig::new(vec![1], schema.clone(), None).is_err()); + assert!(EqualityDeleteWriterConfig::new(vec![0], schema.clone()).is_err()); + assert!(EqualityDeleteWriterConfig::new(vec![1], schema.clone()).is_err()); // Struct is not allowed to be used for equality delete - assert!(EqualityDeleteWriterConfig::new(vec![3], schema.clone(), None).is_err()); + assert!(EqualityDeleteWriterConfig::new(vec![3], schema.clone()).is_err()); // Nested field of struct is allowed to be used for equality delete - assert!(EqualityDeleteWriterConfig::new(vec![4], schema.clone(), None).is_ok()); + assert!(EqualityDeleteWriterConfig::new(vec![4], schema.clone()).is_ok()); // Nested field of map is not allowed to be used for equality delete - assert!(EqualityDeleteWriterConfig::new(vec![7], schema.clone(), None).is_err()); - assert!(EqualityDeleteWriterConfig::new(vec![8], schema.clone(), None).is_err()); - assert!(EqualityDeleteWriterConfig::new(vec![9], schema.clone(), None).is_err()); + assert!(EqualityDeleteWriterConfig::new(vec![7], schema.clone()).is_err()); + assert!(EqualityDeleteWriterConfig::new(vec![8], schema.clone()).is_err()); + assert!(EqualityDeleteWriterConfig::new(vec![9], schema.clone()).is_err()); // Nested field of list is not allowed to be used for equality delete - assert!(EqualityDeleteWriterConfig::new(vec![10], schema.clone(), None).is_err()); - assert!(EqualityDeleteWriterConfig::new(vec![11], schema.clone(), None).is_err()); + assert!(EqualityDeleteWriterConfig::new(vec![10], schema.clone()).is_err()); + assert!(EqualityDeleteWriterConfig::new(vec![11], schema.clone()).is_err()); Ok(()) } @@ -597,7 +591,7 @@ mod test { .unwrap(), ); let equality_ids = vec![0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13]; - let config = EqualityDeleteWriterConfig::new(equality_ids, schema.clone(), None).unwrap(); + let config = EqualityDeleteWriterConfig::new(equality_ids, schema.clone()).unwrap(); let delete_arrow_schema = config.projected_arrow_schema_ref().clone(); let delete_schema = arrow_schema_to_schema(&delete_arrow_schema).unwrap(); @@ -611,7 +605,7 @@ mod test { ); let mut equality_delete_writer = EqualityDeleteFileWriterBuilder::new(rolling_writer_builder, config) - .build() + .build(None) .await?; // prepare data @@ -795,7 +789,7 @@ mod test { let to_write = RecordBatch::try_new(arrow_schema.clone(), columns).unwrap(); let equality_ids = vec![0_i32, 2, 5]; let equality_config = - EqualityDeleteWriterConfig::new(equality_ids, Arc::new(schema), None).unwrap(); + EqualityDeleteWriterConfig::new(equality_ids, Arc::new(schema)).unwrap(); let projector = equality_config.projector.clone(); // check diff --git a/crates/iceberg/src/writer/file_writer/rolling_writer.rs b/crates/iceberg/src/writer/file_writer/rolling_writer.rs index 0003617043..b68010304c 100644 --- a/crates/iceberg/src/writer/file_writer/rolling_writer.rs +++ b/crates/iceberg/src/writer/file_writer/rolling_writer.rs @@ -329,11 +329,10 @@ mod tests { file_name_gen, ); - let data_file_writer_builder = - DataFileWriterBuilder::new(rolling_file_writer_builder, None); + let data_file_writer_builder = DataFileWriterBuilder::new(rolling_file_writer_builder); // Create writer - let mut writer = data_file_writer_builder.build().await?; + let mut writer = data_file_writer_builder.build(None).await?; // Create test data let arrow_schema = make_test_arrow_schema(); @@ -388,10 +387,10 @@ mod tests { file_name_gen, ); - let data_file_writer_builder = DataFileWriterBuilder::new(rolling_writer_builder, None); + let data_file_writer_builder = DataFileWriterBuilder::new(rolling_writer_builder); // Create writer - let mut writer = data_file_writer_builder.build().await?; + let mut writer = data_file_writer_builder.build(None).await?; // Create test data let arrow_schema = make_test_arrow_schema(); diff --git a/crates/iceberg/src/writer/mod.rs b/crates/iceberg/src/writer/mod.rs index d5a8a66861..1da3fa6790 100644 --- a/crates/iceberg/src/writer/mod.rs +++ b/crates/iceberg/src/writer/mod.rs @@ -100,10 +100,9 @@ //! ); //! //! // Create a data file writer using parquet file writer builder. -//! let data_file_writer_builder = -//! DataFileWriterBuilder::new(rolling_file_writer_builder, None); +//! let data_file_writer_builder = DataFileWriterBuilder::new(rolling_file_writer_builder); //! // Build the data file writer -//! let mut data_file_writer = data_file_writer_builder.build().await?; +//! let mut data_file_writer = data_file_writer_builder.build(None).await?; //! //! // Write the data using data_file_writer... //! @@ -122,7 +121,7 @@ //! use arrow_array::RecordBatch; //! use iceberg::io::FileIOBuilder; //! use iceberg::memory::MemoryCatalogBuilder; -//! use iceberg::spec::DataFile; +//! use iceberg::spec::{DataFile, PartitionKey}; //! use iceberg::writer::base_writer::data_file_writer::DataFileWriterBuilder; //! use iceberg::writer::file_writer::ParquetWriterBuilder; //! use iceberg::writer::file_writer::location_generator::{ @@ -149,9 +148,9 @@ //! impl IcebergWriterBuilder for LatencyRecordWriterBuilder { //! type R = LatencyRecordWriter; //! -//! async fn build(self) -> Result { +//! async fn build(self, partition_key: Option) -> Result { //! Ok(LatencyRecordWriter { -//! inner_writer: self.inner_writer_builder.build().await?, +//! inner_writer: self.inner_writer_builder.build(partition_key).await?, //! }) //! } //! } @@ -231,12 +230,14 @@ //! ); //! //! // Create a data file writer builder using rolling file writer. -//! let data_file_writer_builder = -//! DataFileWriterBuilder::new(rolling_file_writer_builder, Some(partition_key)); +//! let data_file_writer_builder = DataFileWriterBuilder::new(rolling_file_writer_builder); //! // Create latency record writer using data file writer builder. //! let latency_record_builder = LatencyRecordWriterBuilder::new(data_file_writer_builder); //! // Build the final writer -//! let mut latency_record_data_file_writer = latency_record_builder.build().await.unwrap(); +//! let mut latency_record_data_file_writer = latency_record_builder +//! .build(Some(partition_key)) +//! .await +//! .unwrap(); //! //! Ok(()) //! } @@ -244,11 +245,14 @@ pub mod base_writer; pub mod file_writer; +/// Provides partition-aware writers +/// TODO examples +pub mod partitioning; use arrow_array::RecordBatch; use crate::Result; -use crate::spec::DataFile; +use crate::spec::{DataFile, PartitionKey}; type DefaultInput = RecordBatch; type DefaultOutput = Vec; @@ -260,8 +264,8 @@ pub trait IcebergWriterBuilder: { /// The associated writer type. type R: IcebergWriter; - /// Build the iceberg writer. - async fn build(self) -> Result; + /// Build the iceberg writer with an optional partition key. + async fn build(self, partition_key: Option) -> Result; } /// The iceberg writer used to write data to iceberg table. diff --git a/crates/iceberg/src/writer/partitioning/clustered_writer.rs b/crates/iceberg/src/writer/partitioning/clustered_writer.rs new file mode 100644 index 0000000000..c141b212f5 --- /dev/null +++ b/crates/iceberg/src/writer/partitioning/clustered_writer.rs @@ -0,0 +1,520 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +//! This module provides the `ClusteredWriter` implementation. + +use std::collections::HashSet; +use std::marker::PhantomData; + +use async_trait::async_trait; + +use crate::spec::{PartitionKey, Struct}; +use crate::writer::partitioning::PartitioningWriter; +use crate::writer::{DefaultInput, DefaultOutput, IcebergWriter, IcebergWriterBuilder}; +use crate::{Error, ErrorKind, Result}; + +/// A writer that writes data to a single partition at a time. +/// +/// This writer expects input data to be sorted by partition key. It maintains only one +/// active writer at a time, making it memory efficient for sorted data. +/// +/// # Type Parameters +/// +/// * `B` - The inner writer builder type +/// * `I` - Input type (defaults to `RecordBatch`) +/// * `O` - Output collection type (defaults to `Vec`) +pub struct ClusteredWriter +where + B: IcebergWriterBuilder, + O: IntoIterator + FromIterator<::Item>, + ::Item: Clone, +{ + inner_builder: B, + current_writer: Option, + current_partition: Option, + closed_partitions: HashSet, + output: Vec<::Item>, + _phantom: PhantomData, +} + +impl ClusteredWriter +where + B: IcebergWriterBuilder, + I: Send + 'static, + O: IntoIterator + FromIterator<::Item>, + ::Item: Send + Clone, +{ + /// Create a new `ClusteredWriter`. + pub fn new(inner_builder: B) -> Self { + Self { + inner_builder, + current_writer: None, + current_partition: None, + closed_partitions: HashSet::new(), + output: Vec::new(), + _phantom: PhantomData, + } + } + + /// Closes the current writer if it exists, flushes the written data to output, and record closed partition. + async fn close_current_writer(&mut self) -> Result<()> { + if let Some(mut writer) = self.current_writer.take() { + self.output.extend(writer.close().await?); + + // Add the current partition to the set of closed partitions + if let Some(current_partition) = self.current_partition.take() { + self.closed_partitions.insert(current_partition); + } + } + + Ok(()) + } +} + +#[async_trait] +impl PartitioningWriter for ClusteredWriter +where + B: IcebergWriterBuilder, + I: Send + 'static, + O: IntoIterator + FromIterator<::Item> + Send + 'static, + ::Item: Send + Clone, +{ + async fn write(&mut self, partition_key: PartitionKey, input: I) -> Result<()> { + let partition_value = partition_key.data(); + + // Check if this partition has been closed already + if self.closed_partitions.contains(partition_value) { + return Err(Error::new( + ErrorKind::Unexpected, + format!( + "The input is not sorted! Cannot write to partition that was previously closed: {:?}", + partition_key + ), + )); + } + + // Check if we need to switch to a new partition + let need_new_writer = match &self.current_partition { + Some(current) => current != partition_value, + None => true, + }; + + if need_new_writer { + self.close_current_writer().await?; + + // Create a new writer for the new partition + self.current_writer = Some( + self.inner_builder + .clone() + .build(Some(partition_key.clone())) + .await?, + ); + self.current_partition = Some(partition_value.clone()); + } + + // do write + self.current_writer + .as_mut() + .expect("Writer should be initialized") + .write(input) + .await + } + + async fn close(mut self) -> Result { + self.close_current_writer().await?; + + // Collect all output items into the output collection type + Ok(O::from_iter(self.output)) + } +} + +#[cfg(test)] +mod tests { + use std::collections::HashMap; + use std::sync::Arc; + + use arrow_array::{Int32Array, RecordBatch, StringArray}; + use arrow_schema::{DataType, Field, Schema}; + use parquet::arrow::PARQUET_FIELD_ID_META_KEY; + use parquet::file::properties::WriterProperties; + use tempfile::TempDir; + + use super::*; + use crate::io::FileIOBuilder; + use crate::spec::{DataFileFormat, NestedField, PrimitiveType, Type}; + use crate::writer::base_writer::data_file_writer::DataFileWriterBuilder; + use crate::writer::file_writer::ParquetWriterBuilder; + use crate::writer::file_writer::location_generator::{ + DefaultFileNameGenerator, DefaultLocationGenerator, + }; + use crate::writer::file_writer::rolling_writer::RollingFileWriterBuilder; + + #[tokio::test] + async fn test_clustered_writer_single_partition() -> Result<()> { + let temp_dir = TempDir::new()?; + let file_io = FileIOBuilder::new_fs_io().build()?; + let location_gen = DefaultLocationGenerator::with_data_location( + temp_dir.path().to_str().unwrap().to_string(), + ); + let file_name_gen = + DefaultFileNameGenerator::new("test".to_string(), None, DataFileFormat::Parquet); + + // Create schema with partition field + let schema = Arc::new( + crate::spec::Schema::builder() + .with_schema_id(1) + .with_fields(vec![ + NestedField::required(1, "id", Type::Primitive(PrimitiveType::Int)).into(), + NestedField::required(2, "name", Type::Primitive(PrimitiveType::String)).into(), + NestedField::required(3, "region", Type::Primitive(PrimitiveType::String)) + .into(), + ]) + .build()?, + ); + + // Create partition spec and key + let partition_spec = crate::spec::PartitionSpec::builder(schema.clone()).build()?; + let partition_value = + crate::spec::Struct::from_iter([Some(crate::spec::Literal::string("US"))]); + let partition_key = + crate::spec::PartitionKey::new(partition_spec, schema.clone(), partition_value.clone()); + + // Create writer builder + let parquet_writer_builder = + ParquetWriterBuilder::new(WriterProperties::builder().build(), schema.clone()); + + // Create rolling file writer builder + let rolling_writer_builder = RollingFileWriterBuilder::new_with_default_file_size( + parquet_writer_builder, + file_io.clone(), + location_gen, + file_name_gen, + ); + + // Create data file writer builder + let data_file_writer_builder = DataFileWriterBuilder::new(rolling_writer_builder); + + // Create clustered writer + let mut writer = ClusteredWriter::new(data_file_writer_builder); + + // Create test data with proper field ID metadata + let arrow_schema = Schema::new(vec![ + Field::new("id", DataType::Int32, false).with_metadata(HashMap::from([( + PARQUET_FIELD_ID_META_KEY.to_string(), + 1.to_string(), + )])), + Field::new("name", DataType::Utf8, false).with_metadata(HashMap::from([( + PARQUET_FIELD_ID_META_KEY.to_string(), + 2.to_string(), + )])), + Field::new("region", DataType::Utf8, false).with_metadata(HashMap::from([( + PARQUET_FIELD_ID_META_KEY.to_string(), + 3.to_string(), + )])), + ]); + + let batch1 = RecordBatch::try_new(Arc::new(arrow_schema.clone()), vec![ + Arc::new(Int32Array::from(vec![1, 2])), + Arc::new(StringArray::from(vec!["Alice", "Bob"])), + Arc::new(StringArray::from(vec!["US", "US"])), + ])?; + + let batch2 = RecordBatch::try_new(Arc::new(arrow_schema.clone()), vec![ + Arc::new(Int32Array::from(vec![3, 4])), + Arc::new(StringArray::from(vec!["Charlie", "Dave"])), + Arc::new(StringArray::from(vec!["US", "US"])), + ])?; + + // Write data to the same partition (this should work) + writer.write(partition_key.clone(), batch1).await?; + writer.write(partition_key.clone(), batch2).await?; + + // Close writer and get data files + let data_files = writer.close().await?; + + // Verify at least one file was created + assert!( + !data_files.is_empty(), + "Expected at least one data file to be created" + ); + + // Verify that all data files have the correct partition value + for data_file in &data_files { + assert_eq!(data_file.partition, partition_value); + } + + Ok(()) + } + + #[tokio::test] + async fn test_clustered_writer_sorted_partitions() -> Result<()> { + let temp_dir = TempDir::new()?; + let file_io = FileIOBuilder::new_fs_io().build()?; + let location_gen = DefaultLocationGenerator::with_data_location( + temp_dir.path().to_str().unwrap().to_string(), + ); + let file_name_gen = + DefaultFileNameGenerator::new("test".to_string(), None, DataFileFormat::Parquet); + + // Create schema with partition field + let schema = Arc::new( + crate::spec::Schema::builder() + .with_schema_id(1) + .with_fields(vec![ + NestedField::required(1, "id", Type::Primitive(PrimitiveType::Int)).into(), + NestedField::required(2, "name", Type::Primitive(PrimitiveType::String)).into(), + NestedField::required(3, "region", Type::Primitive(PrimitiveType::String)) + .into(), + ]) + .build()?, + ); + + // Create partition spec + let partition_spec = crate::spec::PartitionSpec::builder(schema.clone()).build()?; + + // Create partition keys for different regions (in sorted order) + let partition_value_asia = + crate::spec::Struct::from_iter([Some(crate::spec::Literal::string("ASIA"))]); + let partition_key_asia = crate::spec::PartitionKey::new( + partition_spec.clone(), + schema.clone(), + partition_value_asia.clone(), + ); + + let partition_value_eu = + crate::spec::Struct::from_iter([Some(crate::spec::Literal::string("EU"))]); + let partition_key_eu = crate::spec::PartitionKey::new( + partition_spec.clone(), + schema.clone(), + partition_value_eu.clone(), + ); + + let partition_value_us = + crate::spec::Struct::from_iter([Some(crate::spec::Literal::string("US"))]); + let partition_key_us = crate::spec::PartitionKey::new( + partition_spec.clone(), + schema.clone(), + partition_value_us.clone(), + ); + + // Create writer builder + let parquet_writer_builder = + ParquetWriterBuilder::new(WriterProperties::builder().build(), schema.clone()); + + // Create rolling file writer builder + let rolling_writer_builder = RollingFileWriterBuilder::new_with_default_file_size( + parquet_writer_builder, + file_io.clone(), + location_gen, + file_name_gen, + ); + + // Create data file writer builder + let data_file_writer_builder = DataFileWriterBuilder::new(rolling_writer_builder); + + // Create clustered writer + let mut writer = ClusteredWriter::new(data_file_writer_builder); + + // Create test data with proper field ID metadata + let arrow_schema = Schema::new(vec![ + Field::new("id", DataType::Int32, false).with_metadata(HashMap::from([( + PARQUET_FIELD_ID_META_KEY.to_string(), + 1.to_string(), + )])), + Field::new("name", DataType::Utf8, false).with_metadata(HashMap::from([( + PARQUET_FIELD_ID_META_KEY.to_string(), + 2.to_string(), + )])), + Field::new("region", DataType::Utf8, false).with_metadata(HashMap::from([( + PARQUET_FIELD_ID_META_KEY.to_string(), + 3.to_string(), + )])), + ]); + + // Create batches for different partitions (in sorted order) + let batch_asia = RecordBatch::try_new(Arc::new(arrow_schema.clone()), vec![ + Arc::new(Int32Array::from(vec![1, 2])), + Arc::new(StringArray::from(vec!["Alice", "Bob"])), + Arc::new(StringArray::from(vec!["ASIA", "ASIA"])), + ])?; + + let batch_eu = RecordBatch::try_new(Arc::new(arrow_schema.clone()), vec![ + Arc::new(Int32Array::from(vec![3, 4])), + Arc::new(StringArray::from(vec!["Charlie", "Dave"])), + Arc::new(StringArray::from(vec!["EU", "EU"])), + ])?; + + let batch_us = RecordBatch::try_new(Arc::new(arrow_schema.clone()), vec![ + Arc::new(Int32Array::from(vec![5, 6])), + Arc::new(StringArray::from(vec!["Eve", "Frank"])), + Arc::new(StringArray::from(vec!["US", "US"])), + ])?; + + // Write data in sorted partition order (this should work) + writer.write(partition_key_asia.clone(), batch_asia).await?; + writer.write(partition_key_eu.clone(), batch_eu).await?; + writer.write(partition_key_us.clone(), batch_us).await?; + + // Close writer and get data files + let data_files = writer.close().await?; + + // Verify files were created for all partitions + assert!( + data_files.len() >= 3, + "Expected at least 3 data files (one per partition), got {}", + data_files.len() + ); + + // Verify that we have files for each partition + let mut partitions_found = std::collections::HashSet::new(); + for data_file in &data_files { + partitions_found.insert(data_file.partition.clone()); + } + + assert!( + partitions_found.contains(&partition_value_asia), + "Missing ASIA partition" + ); + assert!( + partitions_found.contains(&partition_value_eu), + "Missing EU partition" + ); + assert!( + partitions_found.contains(&partition_value_us), + "Missing US partition" + ); + + Ok(()) + } + + #[tokio::test] + async fn test_clustered_writer_unsorted_partitions_error() -> Result<()> { + let temp_dir = TempDir::new()?; + let file_io = FileIOBuilder::new_fs_io().build()?; + let location_gen = DefaultLocationGenerator::with_data_location( + temp_dir.path().to_str().unwrap().to_string(), + ); + let file_name_gen = + DefaultFileNameGenerator::new("test".to_string(), None, DataFileFormat::Parquet); + + // Create schema with partition field + let schema = Arc::new( + crate::spec::Schema::builder() + .with_schema_id(1) + .with_fields(vec![ + NestedField::required(1, "id", Type::Primitive(PrimitiveType::Int)).into(), + NestedField::required(2, "name", Type::Primitive(PrimitiveType::String)).into(), + NestedField::required(3, "region", Type::Primitive(PrimitiveType::String)) + .into(), + ]) + .build()?, + ); + + // Create partition spec + let partition_spec = crate::spec::PartitionSpec::builder(schema.clone()).build()?; + + // Create partition keys for different regions + let partition_value_us = + crate::spec::Struct::from_iter([Some(crate::spec::Literal::string("US"))]); + let partition_key_us = crate::spec::PartitionKey::new( + partition_spec.clone(), + schema.clone(), + partition_value_us.clone(), + ); + + let partition_value_eu = + crate::spec::Struct::from_iter([Some(crate::spec::Literal::string("EU"))]); + let partition_key_eu = crate::spec::PartitionKey::new( + partition_spec.clone(), + schema.clone(), + partition_value_eu.clone(), + ); + + // Create writer builder + let parquet_writer_builder = + ParquetWriterBuilder::new(WriterProperties::builder().build(), schema.clone()); + + // Create rolling file writer builder + let rolling_writer_builder = RollingFileWriterBuilder::new_with_default_file_size( + parquet_writer_builder, + file_io.clone(), + location_gen, + file_name_gen, + ); + + // Create data file writer builder + let data_file_writer_builder = DataFileWriterBuilder::new(rolling_writer_builder); + + // Create clustered writer + let mut writer = ClusteredWriter::new(data_file_writer_builder); + + // Create test data with proper field ID metadata + let arrow_schema = Schema::new(vec![ + Field::new("id", DataType::Int32, false).with_metadata(HashMap::from([( + PARQUET_FIELD_ID_META_KEY.to_string(), + 1.to_string(), + )])), + Field::new("name", DataType::Utf8, false).with_metadata(HashMap::from([( + PARQUET_FIELD_ID_META_KEY.to_string(), + 2.to_string(), + )])), + Field::new("region", DataType::Utf8, false).with_metadata(HashMap::from([( + PARQUET_FIELD_ID_META_KEY.to_string(), + 3.to_string(), + )])), + ]); + + // Create batches for different partitions + let batch_us = RecordBatch::try_new(Arc::new(arrow_schema.clone()), vec![ + Arc::new(Int32Array::from(vec![1, 2])), + Arc::new(StringArray::from(vec!["Alice", "Bob"])), + Arc::new(StringArray::from(vec!["US", "US"])), + ])?; + + let batch_eu = RecordBatch::try_new(Arc::new(arrow_schema.clone()), vec![ + Arc::new(Int32Array::from(vec![3, 4])), + Arc::new(StringArray::from(vec!["Charlie", "Dave"])), + Arc::new(StringArray::from(vec!["EU", "EU"])), + ])?; + + let batch_us2 = RecordBatch::try_new(Arc::new(arrow_schema.clone()), vec![ + Arc::new(Int32Array::from(vec![5])), + Arc::new(StringArray::from(vec!["Eve"])), + Arc::new(StringArray::from(vec!["US"])), + ])?; + + // Write data to US partition first + writer.write(partition_key_us.clone(), batch_us).await?; + + // Write data to EU partition (this closes US partition) + writer.write(partition_key_eu.clone(), batch_eu).await?; + + // Try to write to US partition again - this should fail because data is not sorted + let result = writer.write(partition_key_us.clone(), batch_us2).await; + + assert!(result.is_err(), "Expected error when writing unsorted data"); + + let error = result.unwrap_err(); + assert!( + error.to_string().contains("The input is not sorted"), + "Expected 'input is not sorted' error, got: {}", + error + ); + + Ok(()) + } +} diff --git a/crates/iceberg/src/writer/partitioning/fanout_writer.rs b/crates/iceberg/src/writer/partitioning/fanout_writer.rs new file mode 100644 index 0000000000..796c1a4888 --- /dev/null +++ b/crates/iceberg/src/writer/partitioning/fanout_writer.rs @@ -0,0 +1,385 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +//! This module provides the `FanoutWriter` implementation. + +use std::collections::HashMap; +use std::marker::PhantomData; + +use async_trait::async_trait; + +use crate::spec::{PartitionKey, Struct}; +use crate::writer::partitioning::PartitioningWriter; +use crate::writer::{DefaultInput, DefaultOutput, IcebergWriter, IcebergWriterBuilder}; +use crate::{Error, ErrorKind, Result}; + +/// A writer that can write data to multiple partitions simultaneously. +/// +/// Unlike `ClusteredWriter` which expects sorted input and maintains only one active writer, +/// `FanoutWriter` can handle unsorted data by maintaining multiple active writers in a map. +/// This allows writing to any partition at any time, but uses more memory as all writers +/// remain active until the writer is closed. +/// +/// # Type Parameters +/// +/// * `B` - The inner writer builder type +/// * `I` - Input type (defaults to `RecordBatch`) +/// * `O` - Output collection type (defaults to `Vec`) +pub struct FanoutWriter +where + B: IcebergWriterBuilder, + O: IntoIterator + FromIterator<::Item>, + ::Item: Clone, +{ + inner_builder: B, + partition_writers: HashMap, + output: Vec<::Item>, + _phantom: PhantomData, +} + +impl FanoutWriter +where + B: IcebergWriterBuilder, + I: Send + 'static, + O: IntoIterator + FromIterator<::Item>, + ::Item: Send + Clone, +{ + /// Create a new `FanoutWriter`. + pub fn new(inner_builder: B) -> Self { + Self { + inner_builder, + partition_writers: HashMap::new(), + output: Vec::new(), + _phantom: PhantomData, + } + } + + /// Get or create a writer for the specified partition. + async fn get_or_create_writer(&mut self, partition_key: &PartitionKey) -> Result<&mut B::R> { + if !self.partition_writers.contains_key(partition_key.data()) { + let writer = self + .inner_builder + .clone() + .build(Some(partition_key.clone())) + .await?; + self.partition_writers + .insert(partition_key.data().clone(), writer); + } + + self.partition_writers + .get_mut(partition_key.data()) + .ok_or_else(|| { + Error::new( + ErrorKind::Unexpected, + "Failed to get partition writer after creation", + ) + }) + } +} + +#[async_trait] +impl PartitioningWriter for FanoutWriter +where + B: IcebergWriterBuilder, + I: Send + 'static, + O: IntoIterator + FromIterator<::Item> + Send + 'static, + ::Item: Send + Clone, +{ + async fn write(&mut self, partition_key: PartitionKey, input: I) -> Result<()> { + let writer = self.get_or_create_writer(&partition_key).await?; + writer.write(input).await + } + + async fn close(mut self) -> Result { + // Close all partition writers + for (_, mut writer) in self.partition_writers { + self.output.extend(writer.close().await?); + } + + // Collect all output items into the output collection type + Ok(O::from_iter(self.output)) + } +} + +#[cfg(test)] +mod tests { + use std::collections::HashMap; + use std::sync::Arc; + + use arrow_array::{Int32Array, RecordBatch, StringArray}; + use arrow_schema::{DataType, Field, Schema}; + use parquet::arrow::PARQUET_FIELD_ID_META_KEY; + use parquet::file::properties::WriterProperties; + use tempfile::TempDir; + + use super::*; + use crate::io::FileIOBuilder; + use crate::spec::{ + DataFileFormat, Literal, NestedField, PartitionKey, PartitionSpec, PrimitiveType, Struct, + Type, + }; + use crate::writer::base_writer::data_file_writer::DataFileWriterBuilder; + use crate::writer::file_writer::ParquetWriterBuilder; + use crate::writer::file_writer::location_generator::{ + DefaultFileNameGenerator, DefaultLocationGenerator, + }; + use crate::writer::file_writer::rolling_writer::RollingFileWriterBuilder; + + #[tokio::test] + async fn test_fanout_writer_single_partition() -> Result<()> { + let temp_dir = TempDir::new()?; + let file_io = FileIOBuilder::new_fs_io().build()?; + let location_gen = DefaultLocationGenerator::with_data_location( + temp_dir.path().to_str().unwrap().to_string(), + ); + let file_name_gen = + DefaultFileNameGenerator::new("test".to_string(), None, DataFileFormat::Parquet); + + // Create schema with partition field + let schema = Arc::new( + crate::spec::Schema::builder() + .with_schema_id(1) + .with_fields(vec![ + NestedField::required(1, "id", Type::Primitive(PrimitiveType::Int)).into(), + NestedField::required(2, "name", Type::Primitive(PrimitiveType::String)).into(), + NestedField::required(3, "region", Type::Primitive(PrimitiveType::String)) + .into(), + ]) + .build()?, + ); + + // Create partition spec - using the same pattern as data_file_writer tests + let partition_spec = PartitionSpec::builder(schema.clone()).build()?; + let partition_value = Struct::from_iter([Some(Literal::string("US"))]); + let partition_key = + PartitionKey::new(partition_spec, schema.clone(), partition_value.clone()); + + // Create writer builder + let parquet_writer_builder = + ParquetWriterBuilder::new(WriterProperties::builder().build(), schema.clone()); + + // Create rolling file writer builder + let rolling_writer_builder = RollingFileWriterBuilder::new_with_default_file_size( + parquet_writer_builder, + file_io.clone(), + location_gen, + file_name_gen, + ); + + // Create data file writer builder + let data_file_writer_builder = DataFileWriterBuilder::new(rolling_writer_builder); + + // Create fanout writer + let mut writer = FanoutWriter::new(data_file_writer_builder); + + // Create test data with proper field ID metadata + let arrow_schema = Schema::new(vec![ + Field::new("id", DataType::Int32, false).with_metadata(HashMap::from([( + PARQUET_FIELD_ID_META_KEY.to_string(), + 1.to_string(), + )])), + Field::new("name", DataType::Utf8, false).with_metadata(HashMap::from([( + PARQUET_FIELD_ID_META_KEY.to_string(), + 2.to_string(), + )])), + Field::new("region", DataType::Utf8, false).with_metadata(HashMap::from([( + PARQUET_FIELD_ID_META_KEY.to_string(), + 3.to_string(), + )])), + ]); + + let batch1 = RecordBatch::try_new(Arc::new(arrow_schema.clone()), vec![ + Arc::new(Int32Array::from(vec![1, 2])), + Arc::new(StringArray::from(vec!["Alice", "Bob"])), + Arc::new(StringArray::from(vec!["US", "US"])), + ])?; + + let batch2 = RecordBatch::try_new(Arc::new(arrow_schema.clone()), vec![ + Arc::new(Int32Array::from(vec![3, 4])), + Arc::new(StringArray::from(vec!["Charlie", "Dave"])), + Arc::new(StringArray::from(vec!["US", "US"])), + ])?; + + // Write data to the same partition + writer.write(partition_key.clone(), batch1).await?; + writer.write(partition_key.clone(), batch2).await?; + + // Close writer and get data files + let data_files = writer.close().await?; + + // Verify at least one file was created + assert!( + !data_files.is_empty(), + "Expected at least one data file to be created" + ); + + // Verify that all data files have the correct partition value + for data_file in &data_files { + assert_eq!(data_file.partition, partition_value); + } + + Ok(()) + } + + #[tokio::test] + async fn test_fanout_writer_multiple_partitions() -> Result<()> { + let temp_dir = TempDir::new()?; + let file_io = FileIOBuilder::new_fs_io().build()?; + let location_gen = DefaultLocationGenerator::with_data_location( + temp_dir.path().to_str().unwrap().to_string(), + ); + let file_name_gen = + DefaultFileNameGenerator::new("test".to_string(), None, DataFileFormat::Parquet); + + // Create schema with partition field + let schema = Arc::new( + crate::spec::Schema::builder() + .with_schema_id(1) + .with_fields(vec![ + NestedField::required(1, "id", Type::Primitive(PrimitiveType::Int)).into(), + NestedField::required(2, "name", Type::Primitive(PrimitiveType::String)).into(), + NestedField::required(3, "region", Type::Primitive(PrimitiveType::String)) + .into(), + ]) + .build()?, + ); + + // Create partition spec + let partition_spec = PartitionSpec::builder(schema.clone()).build()?; + + // Create partition keys for different regions + let partition_value_us = Struct::from_iter([Some(Literal::string("US"))]); + let partition_key_us = PartitionKey::new( + partition_spec.clone(), + schema.clone(), + partition_value_us.clone(), + ); + + let partition_value_eu = Struct::from_iter([Some(Literal::string("EU"))]); + let partition_key_eu = PartitionKey::new( + partition_spec.clone(), + schema.clone(), + partition_value_eu.clone(), + ); + + let partition_value_asia = Struct::from_iter([Some(Literal::string("ASIA"))]); + let partition_key_asia = PartitionKey::new( + partition_spec.clone(), + schema.clone(), + partition_value_asia.clone(), + ); + + // Create writer builder + let parquet_writer_builder = + ParquetWriterBuilder::new(WriterProperties::builder().build(), schema.clone()); + + // Create rolling file writer builder + let rolling_writer_builder = RollingFileWriterBuilder::new_with_default_file_size( + parquet_writer_builder, + file_io.clone(), + location_gen, + file_name_gen, + ); + + // Create data file writer builder + let data_file_writer_builder = DataFileWriterBuilder::new(rolling_writer_builder); + + // Create fanout writer + let mut writer = FanoutWriter::new(data_file_writer_builder); + + // Create test data with proper field ID metadata + let arrow_schema = Schema::new(vec![ + Field::new("id", DataType::Int32, false).with_metadata(HashMap::from([( + PARQUET_FIELD_ID_META_KEY.to_string(), + 1.to_string(), + )])), + Field::new("name", DataType::Utf8, false).with_metadata(HashMap::from([( + PARQUET_FIELD_ID_META_KEY.to_string(), + 2.to_string(), + )])), + Field::new("region", DataType::Utf8, false).with_metadata(HashMap::from([( + PARQUET_FIELD_ID_META_KEY.to_string(), + 3.to_string(), + )])), + ]); + + // Create batches for different partitions + let batch_us1 = RecordBatch::try_new(Arc::new(arrow_schema.clone()), vec![ + Arc::new(Int32Array::from(vec![1, 2])), + Arc::new(StringArray::from(vec!["Alice", "Bob"])), + Arc::new(StringArray::from(vec!["US", "US"])), + ])?; + + let batch_eu1 = RecordBatch::try_new(Arc::new(arrow_schema.clone()), vec![ + Arc::new(Int32Array::from(vec![3, 4])), + Arc::new(StringArray::from(vec!["Charlie", "Dave"])), + Arc::new(StringArray::from(vec!["EU", "EU"])), + ])?; + + let batch_us2 = RecordBatch::try_new(Arc::new(arrow_schema.clone()), vec![ + Arc::new(Int32Array::from(vec![5])), + Arc::new(StringArray::from(vec!["Eve"])), + Arc::new(StringArray::from(vec!["US"])), + ])?; + + let batch_asia1 = RecordBatch::try_new(Arc::new(arrow_schema.clone()), vec![ + Arc::new(Int32Array::from(vec![6, 7])), + Arc::new(StringArray::from(vec!["Frank", "Grace"])), + Arc::new(StringArray::from(vec!["ASIA", "ASIA"])), + ])?; + + // Write data in mixed partition order to demonstrate fanout capability + // This is the key difference from ClusteredWriter - we can write to any partition at any time + writer.write(partition_key_us.clone(), batch_us1).await?; + writer.write(partition_key_eu.clone(), batch_eu1).await?; + writer.write(partition_key_us.clone(), batch_us2).await?; // Back to US partition + writer + .write(partition_key_asia.clone(), batch_asia1) + .await?; + + // Close writer and get data files + let data_files = writer.close().await?; + + // Verify files were created for all partitions + assert!( + data_files.len() >= 3, + "Expected at least 3 data files (one per partition), got {}", + data_files.len() + ); + + // Verify that we have files for each partition + let mut partitions_found = std::collections::HashSet::new(); + for data_file in &data_files { + partitions_found.insert(data_file.partition.clone()); + } + + assert!( + partitions_found.contains(&partition_value_us), + "Missing US partition" + ); + assert!( + partitions_found.contains(&partition_value_eu), + "Missing EU partition" + ); + assert!( + partitions_found.contains(&partition_value_asia), + "Missing ASIA partition" + ); + + Ok(()) + } +} diff --git a/crates/iceberg/src/writer/partitioning/mod.rs b/crates/iceberg/src/writer/partitioning/mod.rs new file mode 100644 index 0000000000..36acb2a987 --- /dev/null +++ b/crates/iceberg/src/writer/partitioning/mod.rs @@ -0,0 +1,49 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +pub mod clustered_writer; +pub mod fanout_writer; + +use crate::Result; +use crate::spec::PartitionKey; +use crate::writer::{DefaultInput, DefaultOutput}; + +/// A writer that can write data to partitioned tables. +/// +/// This trait provides methods for writing data with partition keys and +/// closing the writer to retrieve the output. +#[async_trait::async_trait] +pub trait PartitioningWriter: Send + 'static { + /// Write data with a partition key. + /// + /// # Parameters + /// + /// * `partition_key` - Partition key to determine which partition to write to + /// * `input` - The input data to write + /// + /// # Returns + /// + /// `Ok(())` on success, or an error if the write operation fails. + async fn write(&mut self, partition_key: PartitionKey, input: I) -> Result<()>; + + /// Close the writer and return the output. + /// + /// # Returns + /// + /// The accumulated output from all write operations. + async fn close(self) -> Result; +} diff --git a/crates/integration_tests/tests/shared_tests/append_data_file_test.rs b/crates/integration_tests/tests/shared_tests/append_data_file_test.rs index f4cba959ee..bedc975102 100644 --- a/crates/integration_tests/tests/shared_tests/append_data_file_test.rs +++ b/crates/integration_tests/tests/shared_tests/append_data_file_test.rs @@ -82,8 +82,8 @@ async fn test_append_data_file() { location_generator.clone(), file_name_generator.clone(), ); - let data_file_writer_builder = DataFileWriterBuilder::new(rolling_file_writer_builder, None); - let mut data_file_writer = data_file_writer_builder.build().await.unwrap(); + let data_file_writer_builder = DataFileWriterBuilder::new(rolling_file_writer_builder); + let mut data_file_writer = data_file_writer_builder.build(None).await.unwrap(); let col1 = StringArray::from(vec![Some("foo"), Some("bar"), None, Some("baz")]); let col2 = Int32Array::from(vec![Some(1), Some(2), Some(3), Some(4)]); let col3 = BooleanArray::from(vec![Some(true), Some(false), None, Some(false)]); diff --git a/crates/integration_tests/tests/shared_tests/append_partition_data_file_test.rs b/crates/integration_tests/tests/shared_tests/append_partition_data_file_test.rs index 0da88f1a09..a305ec0842 100644 --- a/crates/integration_tests/tests/shared_tests/append_partition_data_file_test.rs +++ b/crates/integration_tests/tests/shared_tests/append_partition_data_file_test.rs @@ -106,13 +106,11 @@ async fn test_append_partition_data_file() { file_name_generator.clone(), ); - let mut data_file_writer_valid = DataFileWriterBuilder::new( - rolling_file_writer_builder.clone(), - Some(partition_key.clone()), - ) - .build() - .await - .unwrap(); + let mut data_file_writer_valid = + DataFileWriterBuilder::new(rolling_file_writer_builder.clone()) + .build(Some(partition_key.clone())) + .await + .unwrap(); let col1 = StringArray::from(vec![Some("foo1"), Some("foo2")]); let col2 = Int32Array::from(vec![ @@ -191,11 +189,10 @@ async fn test_schema_incompatible_partition_type( catalog: &dyn Catalog, ) { // test writing different "type" of partition than mentioned in schema - let mut data_file_writer_invalid = - DataFileWriterBuilder::new(rolling_file_writer_builder, Some(partition_key)) - .build() - .await - .unwrap(); + let mut data_file_writer_invalid = DataFileWriterBuilder::new(rolling_file_writer_builder) + .build(Some(partition_key)) + .await + .unwrap(); data_file_writer_invalid.write(batch.clone()).await.unwrap(); let data_file_invalid = data_file_writer_invalid.close().await.unwrap(); @@ -221,11 +218,10 @@ async fn test_schema_incompatible_partition_fields( catalog: &dyn Catalog, ) { // test writing different number of partition fields than mentioned in schema - let mut data_file_writer_invalid = - DataFileWriterBuilder::new(rolling_file_writer_builder, Some(partition_key)) - .build() - .await - .unwrap(); + let mut data_file_writer_invalid = DataFileWriterBuilder::new(rolling_file_writer_builder) + .build(Some(partition_key)) + .await + .unwrap(); data_file_writer_invalid.write(batch.clone()).await.unwrap(); let data_file_invalid = data_file_writer_invalid.close().await.unwrap(); diff --git a/crates/integration_tests/tests/shared_tests/conflict_commit_test.rs b/crates/integration_tests/tests/shared_tests/conflict_commit_test.rs index a248fa707a..fc529cc3d2 100644 --- a/crates/integration_tests/tests/shared_tests/conflict_commit_test.rs +++ b/crates/integration_tests/tests/shared_tests/conflict_commit_test.rs @@ -81,8 +81,8 @@ async fn test_append_data_file_conflict() { location_generator.clone(), file_name_generator.clone(), ); - let data_file_writer_builder = DataFileWriterBuilder::new(rolling_file_writer_builder, None); - let mut data_file_writer = data_file_writer_builder.build().await.unwrap(); + let data_file_writer_builder = DataFileWriterBuilder::new(rolling_file_writer_builder); + let mut data_file_writer = data_file_writer_builder.build(None).await.unwrap(); let col1 = StringArray::from(vec![Some("foo"), Some("bar"), None, Some("baz")]); let col2 = Int32Array::from(vec![Some(1), Some(2), Some(3), Some(4)]); let col3 = BooleanArray::from(vec![Some(true), Some(false), None, Some(false)]); diff --git a/crates/integration_tests/tests/shared_tests/scan_all_type.rs b/crates/integration_tests/tests/shared_tests/scan_all_type.rs index 1125de11a5..7a2907d4cb 100644 --- a/crates/integration_tests/tests/shared_tests/scan_all_type.rs +++ b/crates/integration_tests/tests/shared_tests/scan_all_type.rs @@ -163,8 +163,8 @@ async fn test_scan_all_type() { location_generator.clone(), file_name_generator.clone(), ); - let data_file_writer_builder = DataFileWriterBuilder::new(rolling_file_writer_builder, None); - let mut data_file_writer = data_file_writer_builder.build().await.unwrap(); + let data_file_writer_builder = DataFileWriterBuilder::new(rolling_file_writer_builder); + let mut data_file_writer = data_file_writer_builder.build(None).await.unwrap(); // Prepare data let col1 = Int32Array::from(vec![1, 2, 3, 4, 5]); diff --git a/crates/integrations/datafusion/src/physical_plan/write.rs b/crates/integrations/datafusion/src/physical_plan/write.rs index 7bff239222..dff40a3c0d 100644 --- a/crates/integrations/datafusion/src/physical_plan/write.rs +++ b/crates/integrations/datafusion/src/physical_plan/write.rs @@ -262,6 +262,7 @@ impl ExecutionPlan for IcebergWriteExec { }; let file_io = self.table.file_io().clone(); + // todo location_gen and file_name_gen should be configurable let location_generator = DefaultLocationGenerator::new(self.table.metadata().clone()) .map_err(to_datafusion_error)?; // todo filename prefix/suffix should be configurable @@ -274,8 +275,7 @@ impl ExecutionPlan for IcebergWriteExec { location_generator, file_name_generator, ); - // todo specify partition key when partitioning writer is supported - let data_file_writer_builder = DataFileWriterBuilder::new(rolling_writer_builder, None); + let data_file_writer_builder = DataFileWriterBuilder::new(rolling_writer_builder); // Get input data let data = execute_input_stream( @@ -291,7 +291,8 @@ impl ExecutionPlan for IcebergWriteExec { // Create write stream let stream = futures::stream::once(async move { let mut writer = data_file_writer_builder - .build() + // todo specify partition key when partitioning writer is supported + .build(None) .await .map_err(to_datafusion_error)?; let mut input_stream = data; From 826d98f94675c8dad0faae838a543cd3f00fd2be Mon Sep 17 00:00:00 2001 From: Yoshiki Kudo <335g@users.noreply.github.com> Date: Fri, 17 Oct 2025 19:04:10 +0900 Subject: [PATCH 02/47] feat(catalog): impl builder for SqlCatalog (#1666) MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit ## Which issue does this PR close? - Closes #1626 ## What changes are included in this PR? - Introduced a new `SqlCatalogBuilder` implementation for `iceberg::CatalogBuilder`. - By transitioning the builder implementation from `SqlCatalogConfig` to `SqlCatalogBuilder`, we've simplified the process for creating `SqlCatalog` objects. - Enabled generation of `Box` for the `SqlCatalog` via the `load` function (#1260) ## Are these changes tested? Yes (only tested whether it could be made) --------- Co-authored-by: Yoshiki Kudo --- Cargo.lock | 6 +- Cargo.toml | 2 + crates/catalog/loader/Cargo.toml | 5 + crates/catalog/loader/src/lib.rs | 34 +++ crates/catalog/sql/Cargo.toml | 3 +- crates/catalog/sql/src/catalog.rs | 374 ++++++++++++++++++++++++++++-- crates/catalog/sql/src/lib.rs | 37 +++ 7 files changed, 434 insertions(+), 27 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index 16ffef0654..87b97b7c78 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -3712,6 +3712,9 @@ dependencies = [ "iceberg-catalog-hms", "iceberg-catalog-rest", "iceberg-catalog-s3tables", + "iceberg-catalog-sql", + "sqlx", + "tempfile", "tokio", ] @@ -3757,13 +3760,12 @@ version = "0.7.0" dependencies = [ "async-trait", "iceberg", - "iceberg_test_utils", "itertools 0.13.0", "regex", "sqlx", + "strum 0.27.2", "tempfile", "tokio", - "typed-builder 0.20.1", ] [[package]] diff --git a/Cargo.toml b/Cargo.toml index 46c99cc3d8..7735d702cf 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -78,6 +78,7 @@ http = "1.2" iceberg = { version = "0.7.0", path = "./crates/iceberg" } iceberg-catalog-glue = { version = "0.7.0", path = "./crates/catalog/glue" } iceberg-catalog-hms = { version = "0.7.0", path = "./crates/catalog/hms" } +iceberg-catalog-sql = { version = "0.7.0", path = "./crates/catalog/sql" } iceberg-catalog-rest = { version = "0.7.0", path = "./crates/catalog/rest" } iceberg-catalog-s3tables = { version = "0.7.0", path = "./crates/catalog/s3tables" } iceberg-datafusion = { version = "0.7.0", path = "./crates/integrations/datafusion" } @@ -111,6 +112,7 @@ serde_repr = "0.1.16" serde_with = "3.4" smol = "2.0.2" sqllogictest = "0.28.3" +sqlx = { version = "0.8.1", default-features = false } stacker = "0.1.20" strum = "0.27.2" tempfile = "3.18" diff --git a/crates/catalog/loader/Cargo.toml b/crates/catalog/loader/Cargo.toml index 014f84a3a1..d4b925fb94 100644 --- a/crates/catalog/loader/Cargo.toml +++ b/crates/catalog/loader/Cargo.toml @@ -34,5 +34,10 @@ iceberg-catalog-rest = { workspace = true } iceberg-catalog-glue = { workspace = true } iceberg-catalog-s3tables = { workspace = true } iceberg-catalog-hms = { workspace = true } +iceberg-catalog-sql = { workspace = true } tokio = { workspace = true } async-trait = { workspace = true } + +[dev-dependencies] +sqlx = { workspace = true, features = ["runtime-tokio", "sqlite", "migrate"] } +tempfile = { workspace = true } diff --git a/crates/catalog/loader/src/lib.rs b/crates/catalog/loader/src/lib.rs index 9c18ab4e5f..e118ef86a9 100644 --- a/crates/catalog/loader/src/lib.rs +++ b/crates/catalog/loader/src/lib.rs @@ -24,6 +24,7 @@ use iceberg_catalog_glue::GlueCatalogBuilder; use iceberg_catalog_hms::HmsCatalogBuilder; use iceberg_catalog_rest::RestCatalogBuilder; use iceberg_catalog_s3tables::S3TablesCatalogBuilder; +use iceberg_catalog_sql::SqlCatalogBuilder; /// A CatalogBuilderFactory creating a new catalog builder. type CatalogBuilderFactory = fn() -> Box; @@ -34,6 +35,7 @@ static CATALOG_REGISTRY: &[(&str, CatalogBuilderFactory)] = &[ ("glue", || Box::new(GlueCatalogBuilder::default())), ("s3tables", || Box::new(S3TablesCatalogBuilder::default())), ("hms", || Box::new(HmsCatalogBuilder::default())), + ("sql", || Box::new(SqlCatalogBuilder::default())), ]; /// Return the list of supported catalog types. @@ -108,6 +110,9 @@ impl CatalogLoader<'_> { mod tests { use std::collections::HashMap; + use sqlx::migrate::MigrateDatabase; + use tempfile::TempDir; + use crate::{CatalogLoader, load}; #[tokio::test] @@ -220,6 +225,35 @@ mod tests { assert!(catalog.is_ok()); } + fn temp_path() -> String { + let temp_dir = TempDir::new().unwrap(); + temp_dir.path().to_str().unwrap().to_string() + } + + #[tokio::test] + async fn test_catalog_loader_pattern_sql_catalog() { + use iceberg_catalog_sql::{SQL_CATALOG_PROP_URI, SQL_CATALOG_PROP_WAREHOUSE}; + + let uri = format!("sqlite:{}", temp_path()); + sqlx::Sqlite::create_database(&uri).await.unwrap(); + + let catalog_loader = load("sql").unwrap(); + let catalog = catalog_loader + .load( + "sql".to_string(), + HashMap::from([ + (SQL_CATALOG_PROP_URI.to_string(), uri), + ( + SQL_CATALOG_PROP_WAREHOUSE.to_string(), + "s3://warehouse".to_string(), + ), + ]), + ) + .await; + + assert!(catalog.is_ok()); + } + #[tokio::test] async fn test_error_message_includes_supported_types() { let err = match load("does-not-exist") { diff --git a/crates/catalog/sql/Cargo.toml b/crates/catalog/sql/Cargo.toml index 33ca700bf7..262422c284 100644 --- a/crates/catalog/sql/Cargo.toml +++ b/crates/catalog/sql/Cargo.toml @@ -32,10 +32,9 @@ repository = { workspace = true } async-trait = { workspace = true } iceberg = { workspace = true } sqlx = { version = "0.8.1", features = ["any"], default-features = false } -typed-builder = { workspace = true } +strum = { workspace = true } [dev-dependencies] -iceberg_test_utils = { path = "../../test_utils", features = ["tests"] } itertools = { workspace = true } regex = "1.10.5" sqlx = { version = "0.8.1", features = [ diff --git a/crates/catalog/sql/src/catalog.rs b/crates/catalog/sql/src/catalog.rs index c1040c65a0..b6dc4f835b 100644 --- a/crates/catalog/sql/src/catalog.rs +++ b/crates/catalog/sql/src/catalog.rs @@ -16,6 +16,7 @@ // under the License. use std::collections::{HashMap, HashSet}; +use std::str::FromStr; use std::time::Duration; use async_trait::async_trait; @@ -23,17 +24,23 @@ use iceberg::io::FileIO; use iceberg::spec::{TableMetadata, TableMetadataBuilder}; use iceberg::table::Table; use iceberg::{ - Catalog, Error, ErrorKind, MetadataLocation, Namespace, NamespaceIdent, Result, TableCommit, - TableCreation, TableIdent, + Catalog, CatalogBuilder, Error, ErrorKind, MetadataLocation, Namespace, NamespaceIdent, Result, + TableCommit, TableCreation, TableIdent, }; use sqlx::any::{AnyPoolOptions, AnyQueryResult, AnyRow, install_default_drivers}; use sqlx::{Any, AnyPool, Row, Transaction}; -use typed_builder::TypedBuilder; use crate::error::{ from_sqlx_error, no_such_namespace_err, no_such_table_err, table_already_exists_err, }; +/// catalog URI +pub const SQL_CATALOG_PROP_URI: &str = "uri"; +/// catalog warehouse location +pub const SQL_CATALOG_PROP_WAREHOUSE: &str = "warehouse"; +/// catalog sql bind style +pub const SQL_CATALOG_PROP_BIND_STYLE: &str = "sql_bind_style"; + static CATALOG_TABLE_NAME: &str = "iceberg_tables"; static CATALOG_FIELD_CATALOG_NAME: &str = "catalog_name"; static CATALOG_FIELD_TABLE_NAME: &str = "table_name"; @@ -54,6 +61,126 @@ static MAX_CONNECTIONS: u32 = 10; // Default the SQL pool to 10 connections if n static IDLE_TIMEOUT: u64 = 10; // Default the maximum idle timeout per connection to 10s before it is closed static TEST_BEFORE_ACQUIRE: bool = true; // Default the health-check of each connection to enabled prior to returning +/// Builder for [`SqlCatalog`] +#[derive(Debug)] +pub struct SqlCatalogBuilder(SqlCatalogConfig); + +impl Default for SqlCatalogBuilder { + fn default() -> Self { + Self(SqlCatalogConfig { + uri: "".to_string(), + name: "".to_string(), + warehouse_location: "".to_string(), + sql_bind_style: SqlBindStyle::DollarNumeric, + props: HashMap::new(), + }) + } +} + +impl SqlCatalogBuilder { + /// Configure the database URI + /// + /// If `SQL_CATALOG_PROP_URI` has a value set in `props` during `SqlCatalogBuilder::load`, + /// that value takes precedence, and the value specified by this method will not be used. + pub fn uri(mut self, uri: impl Into) -> Self { + self.0.uri = uri.into(); + self + } + + /// Configure the warehouse location + /// + /// If `SQL_CATALOG_PROP_WAREHOUSE` has a value set in `props` during `SqlCatalogBuilder::load`, + /// that value takes precedence, and the value specified by this method will not be used. + pub fn warehouse_location(mut self, location: impl Into) -> Self { + self.0.warehouse_location = location.into(); + self + } + + /// Configure the bound SQL Statement + /// + /// If `SQL_CATALOG_PROP_BIND_STYLE` has a value set in `props` during `SqlCatalogBuilder::load`, + /// that value takes precedence, and the value specified by this method will not be used. + pub fn sql_bind_style(mut self, sql_bind_style: SqlBindStyle) -> Self { + self.0.sql_bind_style = sql_bind_style; + self + } + + /// Configure the any properties + /// + /// If the same key has values set in `props` during `SqlCatalogBuilder::load`, + /// those values will take precedence. + pub fn props(mut self, props: HashMap) -> Self { + for (k, v) in props { + self.0.props.insert(k, v); + } + self + } + + /// Set a new property on the property to be configured. + /// When multiple methods are executed with the same key, + /// the later-set value takes precedence. + /// + /// If the same key has values set in `props` during `SqlCatalogBuilder::load`, + /// those values will take precedence. + pub fn prop(mut self, key: impl Into, value: impl Into) -> Self { + self.0.props.insert(key.into(), value.into()); + self + } +} + +impl CatalogBuilder for SqlCatalogBuilder { + type C = SqlCatalog; + + fn load( + mut self, + name: impl Into, + props: HashMap, + ) -> impl Future> + Send { + let name = name.into(); + + for (k, v) in props { + self.0.props.insert(k, v); + } + + if let Some(uri) = self.0.props.remove(SQL_CATALOG_PROP_URI) { + self.0.uri = uri; + } + if let Some(warehouse_location) = self.0.props.remove(SQL_CATALOG_PROP_WAREHOUSE) { + self.0.warehouse_location = warehouse_location; + } + + let mut valid_sql_bind_style = true; + if let Some(sql_bind_style) = self.0.props.remove(SQL_CATALOG_PROP_BIND_STYLE) { + if let Ok(sql_bind_style) = SqlBindStyle::from_str(&sql_bind_style) { + self.0.sql_bind_style = sql_bind_style; + } else { + valid_sql_bind_style = false; + } + } + + async move { + if name.trim().is_empty() { + Err(Error::new( + ErrorKind::DataInvalid, + "Catalog name cannot be empty", + )) + } else if !valid_sql_bind_style { + Err(Error::new( + ErrorKind::DataInvalid, + format!( + "`{}` values are valid only if they're `{}` or `{}`", + SQL_CATALOG_PROP_BIND_STYLE, + SqlBindStyle::DollarNumeric, + SqlBindStyle::QMark + ), + )) + } else { + SqlCatalog::new(self.0).await + } + } + } +} + /// A struct representing the SQL catalog configuration. /// /// This struct contains various parameters that are used to configure a SQL catalog, @@ -62,14 +189,12 @@ static TEST_BEFORE_ACQUIRE: bool = true; // Default the health-check of each con /// The options available for this parameter include: /// - `SqlBindStyle::DollarNumeric`: Binds SQL statements using `$1`, `$2`, etc., as placeholders. This is for PostgreSQL databases. /// - `SqlBindStyle::QuestionMark`: Binds SQL statements using `?` as a placeholder. This is for MySQL and SQLite databases. -#[derive(Debug, TypedBuilder)] -pub struct SqlCatalogConfig { +#[derive(Debug)] +struct SqlCatalogConfig { uri: String, name: String, warehouse_location: String, - file_io: FileIO, sql_bind_style: SqlBindStyle, - #[builder(default)] props: HashMap, } @@ -83,7 +208,7 @@ pub struct SqlCatalog { sql_bind_style: SqlBindStyle, } -#[derive(Debug, PartialEq)] +#[derive(Debug, PartialEq, strum::EnumString, strum::Display)] /// Set the SQL parameter bind style to either $1..$N (Postgres style) or ? (SQLite/MySQL/MariaDB) pub enum SqlBindStyle { /// DollarNumeric uses parameters of the form `$1..$N``, which is the Postgres style @@ -94,7 +219,8 @@ pub enum SqlBindStyle { impl SqlCatalog { /// Create new sql catalog instance - pub async fn new(config: SqlCatalogConfig) -> Result { + async fn new(config: SqlCatalogConfig) -> Result { + let fileio = FileIO::from_path(&config.warehouse_location)?.build()?; install_default_drivers(); let max_connections: u32 = config .props @@ -150,7 +276,7 @@ impl SqlCatalog { name: config.name.to_owned(), connection: pool, warehouse_location: config.warehouse_location, - fileio: config.file_io, + fileio, sql_bind_style: config.sql_bind_style, }) } @@ -804,17 +930,19 @@ mod tests { use std::collections::{HashMap, HashSet}; use std::hash::Hash; - use iceberg::io::FileIOBuilder; use iceberg::spec::{NestedField, PartitionSpec, PrimitiveType, Schema, SortOrder, Type}; use iceberg::table::Table; - use iceberg::{Catalog, Namespace, NamespaceIdent, TableCreation, TableIdent}; + use iceberg::{Catalog, CatalogBuilder, Namespace, NamespaceIdent, TableCreation, TableIdent}; use itertools::Itertools; use regex::Regex; use sqlx::migrate::MigrateDatabase; use tempfile::TempDir; - use crate::catalog::NAMESPACE_LOCATION_PROPERTY_KEY; - use crate::{SqlBindStyle, SqlCatalog, SqlCatalogConfig}; + use crate::catalog::{ + NAMESPACE_LOCATION_PROPERTY_KEY, SQL_CATALOG_PROP_BIND_STYLE, SQL_CATALOG_PROP_URI, + SQL_CATALOG_PROP_WAREHOUSE, + }; + use crate::{SqlBindStyle, SqlCatalogBuilder}; const UUID_REGEX_STR: &str = "[0-9a-f]{8}-[0-9a-f]{4}-[0-9a-f]{4}-[0-9a-f]{4}-[0-9a-f]{12}"; @@ -835,15 +963,18 @@ mod tests { let sql_lite_uri = format!("sqlite:{}", temp_path()); sqlx::Sqlite::create_database(&sql_lite_uri).await.unwrap(); - let config = SqlCatalogConfig::builder() - .uri(sql_lite_uri.to_string()) - .name("iceberg".to_string()) - .warehouse_location(warehouse_location) - .file_io(FileIOBuilder::new_fs_io().build().unwrap()) - .sql_bind_style(SqlBindStyle::QMark) - .build(); - - SqlCatalog::new(config).await.unwrap() + let props = HashMap::from_iter([ + (SQL_CATALOG_PROP_URI.to_string(), sql_lite_uri.to_string()), + (SQL_CATALOG_PROP_WAREHOUSE.to_string(), warehouse_location), + ( + SQL_CATALOG_PROP_BIND_STYLE.to_string(), + SqlBindStyle::DollarNumeric.to_string(), + ), + ]); + SqlCatalogBuilder::default() + .load("iceberg", props) + .await + .unwrap() } async fn create_namespace(catalog: &C, namespace_ident: &NamespaceIdent) { @@ -942,6 +1073,203 @@ mod tests { new_sql_catalog(warehouse_loc.clone()).await; } + #[tokio::test] + async fn test_builder_method() { + let sql_lite_uri = format!("sqlite:{}", temp_path()); + sqlx::Sqlite::create_database(&sql_lite_uri).await.unwrap(); + let warehouse_location = temp_path(); + + let catalog = SqlCatalogBuilder::default() + .uri(sql_lite_uri.to_string()) + .warehouse_location(warehouse_location.clone()) + .sql_bind_style(SqlBindStyle::QMark) + .load("iceberg", HashMap::default()) + .await; + assert!(catalog.is_ok()); + + let catalog = catalog.unwrap(); + assert!(catalog.warehouse_location == warehouse_location); + assert!(catalog.sql_bind_style == SqlBindStyle::QMark); + } + + /// Overwriting an sqlite database with a non-existent path causes + /// catalog generation to fail + #[tokio::test] + async fn test_builder_props_non_existent_path_fails() { + let sql_lite_uri = format!("sqlite:{}", temp_path()); + let sql_lite_uri2 = format!("sqlite:{}", temp_path()); + sqlx::Sqlite::create_database(&sql_lite_uri).await.unwrap(); + let warehouse_location = temp_path(); + + let catalog = SqlCatalogBuilder::default() + .uri(sql_lite_uri) + .warehouse_location(warehouse_location) + .load( + "iceberg", + HashMap::from_iter([(SQL_CATALOG_PROP_URI.to_string(), sql_lite_uri2)]), + ) + .await; + assert!(catalog.is_err()); + } + + /// Even when an invalid URI is specified in a builder method, + /// it can be successfully overridden with a valid URI in props + /// for catalog generation to succeed. + #[tokio::test] + async fn test_builder_props_set_valid_uri() { + let sql_lite_uri = format!("sqlite:{}", temp_path()); + let sql_lite_uri2 = format!("sqlite:{}", temp_path()); + sqlx::Sqlite::create_database(&sql_lite_uri).await.unwrap(); + let warehouse_location = temp_path(); + + let catalog = SqlCatalogBuilder::default() + .uri(sql_lite_uri2) + .warehouse_location(warehouse_location) + .load( + "iceberg", + HashMap::from_iter([(SQL_CATALOG_PROP_URI.to_string(), sql_lite_uri.clone())]), + ) + .await; + assert!(catalog.is_ok()); + } + + /// values assigned via props take precedence + #[tokio::test] + async fn test_builder_props_take_precedence() { + let sql_lite_uri = format!("sqlite:{}", temp_path()); + sqlx::Sqlite::create_database(&sql_lite_uri).await.unwrap(); + let warehouse_location = temp_path(); + let warehouse_location2 = temp_path(); + + let catalog = SqlCatalogBuilder::default() + .warehouse_location(warehouse_location2) + .sql_bind_style(SqlBindStyle::DollarNumeric) + .load( + "iceberg", + HashMap::from_iter([ + (SQL_CATALOG_PROP_URI.to_string(), sql_lite_uri), + ( + SQL_CATALOG_PROP_WAREHOUSE.to_string(), + warehouse_location.clone(), + ), + ( + SQL_CATALOG_PROP_BIND_STYLE.to_string(), + SqlBindStyle::QMark.to_string(), + ), + ]), + ) + .await; + + assert!(catalog.is_ok()); + + let catalog = catalog.unwrap(); + assert!(catalog.warehouse_location == warehouse_location); + assert!(catalog.sql_bind_style == SqlBindStyle::QMark); + } + + /// values assigned via props take precedence + #[tokio::test] + async fn test_builder_props_take_precedence_props() { + let sql_lite_uri = format!("sqlite:{}", temp_path()); + let sql_lite_uri2 = format!("sqlite:{}", temp_path()); + sqlx::Sqlite::create_database(&sql_lite_uri).await.unwrap(); + let warehouse_location = temp_path(); + let warehouse_location2 = temp_path(); + + let props = HashMap::from_iter([ + (SQL_CATALOG_PROP_URI.to_string(), sql_lite_uri.clone()), + ( + SQL_CATALOG_PROP_WAREHOUSE.to_string(), + warehouse_location.clone(), + ), + ( + SQL_CATALOG_PROP_BIND_STYLE.to_string(), + SqlBindStyle::QMark.to_string(), + ), + ]); + let props2 = HashMap::from_iter([ + (SQL_CATALOG_PROP_URI.to_string(), sql_lite_uri2.clone()), + ( + SQL_CATALOG_PROP_WAREHOUSE.to_string(), + warehouse_location2.clone(), + ), + ( + SQL_CATALOG_PROP_BIND_STYLE.to_string(), + SqlBindStyle::DollarNumeric.to_string(), + ), + ]); + + let catalog = SqlCatalogBuilder::default() + .props(props2) + .load("iceberg", props) + .await; + + assert!(catalog.is_ok()); + + let catalog = catalog.unwrap(); + assert!(catalog.warehouse_location == warehouse_location); + assert!(catalog.sql_bind_style == SqlBindStyle::QMark); + } + + /// values assigned via props take precedence + #[tokio::test] + async fn test_builder_props_take_precedence_prop() { + let sql_lite_uri = format!("sqlite:{}", temp_path()); + let sql_lite_uri2 = format!("sqlite:{}", temp_path()); + sqlx::Sqlite::create_database(&sql_lite_uri).await.unwrap(); + let warehouse_location = temp_path(); + let warehouse_location2 = temp_path(); + + let props = HashMap::from_iter([ + (SQL_CATALOG_PROP_URI.to_string(), sql_lite_uri.clone()), + ( + SQL_CATALOG_PROP_WAREHOUSE.to_string(), + warehouse_location.clone(), + ), + ( + SQL_CATALOG_PROP_BIND_STYLE.to_string(), + SqlBindStyle::QMark.to_string(), + ), + ]); + + let catalog = SqlCatalogBuilder::default() + .prop(SQL_CATALOG_PROP_URI.to_string(), sql_lite_uri2) + .prop(SQL_CATALOG_PROP_WAREHOUSE.to_string(), warehouse_location2) + .prop( + SQL_CATALOG_PROP_BIND_STYLE.to_string(), + SqlBindStyle::DollarNumeric.to_string(), + ) + .load("iceberg", props) + .await; + + assert!(catalog.is_ok()); + + let catalog = catalog.unwrap(); + assert!(catalog.warehouse_location == warehouse_location); + assert!(catalog.sql_bind_style == SqlBindStyle::QMark); + } + + /// invalid value for `SqlBindStyle` causes catalog creation to fail + #[tokio::test] + async fn test_builder_props_invalid_bind_style_fails() { + let sql_lite_uri = format!("sqlite:{}", temp_path()); + sqlx::Sqlite::create_database(&sql_lite_uri).await.unwrap(); + let warehouse_location = temp_path(); + + let catalog = SqlCatalogBuilder::default() + .load( + "iceberg", + HashMap::from_iter([ + (SQL_CATALOG_PROP_URI.to_string(), sql_lite_uri), + (SQL_CATALOG_PROP_WAREHOUSE.to_string(), warehouse_location), + (SQL_CATALOG_PROP_BIND_STYLE.to_string(), "AAA".to_string()), + ]), + ) + .await; + + assert!(catalog.is_err()); + } + #[tokio::test] async fn test_list_namespaces_returns_empty_vector() { let warehouse_loc = temp_path(); diff --git a/crates/catalog/sql/src/lib.rs b/crates/catalog/sql/src/lib.rs index 6861dab3f8..b76006ed3b 100644 --- a/crates/catalog/sql/src/lib.rs +++ b/crates/catalog/sql/src/lib.rs @@ -16,6 +16,43 @@ // under the License. //! Iceberg sql catalog implementation. +//! +//! To build a sql catalog with configurations +//! # Example +//! +//! ```rust, no_run +//! use std::collections::HashMap; +//! +//! use iceberg::CatalogBuilder; +//! use iceberg_catalog_sql::{ +//! SQL_CATALOG_PROP_BIND_STYLE, SQL_CATALOG_PROP_URI, SQL_CATALOG_PROP_WAREHOUSE, +//! SqlBindStyle, SqlCatalogBuilder, +//! }; +//! +//! #[tokio::main] +//! async fn main() { +//! let catalog = SqlCatalogBuilder::default() +//! .load( +//! "sql", +//! HashMap::from_iter([ +//! ( +//! SQL_CATALOG_PROP_URI.to_string(), +//! "http://localhost:8181".to_string(), +//! ), +//! ( +//! SQL_CATALOG_PROP_WAREHOUSE.to_string(), +//! "s3://warehouse".to_string(), +//! ), +//! ( +//! SQL_CATALOG_PROP_BIND_STYLE.to_string(), +//! SqlBindStyle::QMark.to_string(), +//! ), +//! ]), +//! ) +//! .await +//! .unwrap(); +//! } +//! ``` #![deny(missing_docs)] From fa07ec6f26c785de332c75e4f19d87040d543dc3 Mon Sep 17 00:00:00 2001 From: Dylan Date: Fri, 17 Oct 2025 19:03:15 +0800 Subject: [PATCH 03/47] fix: fix read parquert file when schema change (#1750) --- crates/iceberg/src/arrow/reader.rs | 129 ++++++++++++++++++++++++----- 1 file changed, 106 insertions(+), 23 deletions(-) diff --git a/crates/iceberg/src/arrow/reader.rs b/crates/iceberg/src/arrow/reader.rs index 564b281107..ff4cff0a64 100644 --- a/crates/iceberg/src/arrow/reader.rs +++ b/crates/iceberg/src/arrow/reader.rs @@ -583,35 +583,25 @@ impl ArrowReader { true }); - if column_map.len() != leaf_field_ids.len() { - let missing_fields = leaf_field_ids - .iter() - .filter(|field_id| !column_map.contains_key(field_id)) - .collect::>(); - return Err(Error::new( - ErrorKind::DataInvalid, - format!( - "Parquet schema {} and Iceberg schema {} do not match.", - iceberg_schema, iceberg_schema_of_task - ), - ) - .with_context("column_map", format! {"{:?}", column_map}) - .with_context("field_ids", format! {"{:?}", leaf_field_ids}) - .with_context("missing_fields", format! {"{:?}", missing_fields})); - } - + // Only project columns that exist in the Parquet file. + // Missing columns will be added by RecordBatchTransformer with default/NULL values. + // This supports schema evolution where new columns are added to the table schema + // but old Parquet files don't have them yet. let mut indices = vec![]; for field_id in leaf_field_ids { if let Some(col_idx) = column_map.get(&field_id) { indices.push(*col_idx); - } else { - return Err(Error::new( - ErrorKind::DataInvalid, - format!("Field {} is not found in Parquet schema.", field_id), - )); } + // Skip fields that don't exist in the Parquet file - they will be added later + } + + if indices.is_empty() { + // If no columns from the projection exist in the file, project all columns + // This can happen if all requested columns are new and need to be added by the transformer + Ok(ProjectionMask::all()) + } else { + Ok(ProjectionMask::leaves(parquet_schema, indices)) } - Ok(ProjectionMask::leaves(parquet_schema, indices)) } } @@ -1958,4 +1948,97 @@ message schema { Arc::new(SchemaDescriptor::new(Arc::new(schema))) } + + /// Test schema evolution: reading old Parquet file (with only column 'a') + /// using a newer table schema (with columns 'a' and 'b'). + /// This tests that: + /// 1. get_arrow_projection_mask allows missing columns + /// 2. RecordBatchTransformer adds missing column 'b' with NULL values + #[tokio::test] + async fn test_schema_evolution_add_column() { + use arrow_array::{Array, Int32Array}; + + // New table schema: columns 'a' and 'b' (b was added later, file only has 'a') + let new_schema = Arc::new( + Schema::builder() + .with_schema_id(2) + .with_fields(vec![ + NestedField::required(1, "a", Type::Primitive(PrimitiveType::Int)).into(), + NestedField::optional(2, "b", Type::Primitive(PrimitiveType::Int)).into(), + ]) + .build() + .unwrap(), + ); + + // Create Arrow schema for old Parquet file (only has column 'a') + let arrow_schema_old = Arc::new(ArrowSchema::new(vec![ + Field::new("a", DataType::Int32, false).with_metadata(HashMap::from([( + PARQUET_FIELD_ID_META_KEY.to_string(), + "1".to_string(), + )])), + ])); + + // Write old Parquet file with only column 'a' + let tmp_dir = TempDir::new().unwrap(); + let table_location = tmp_dir.path().to_str().unwrap().to_string(); + let file_io = FileIO::from_path(&table_location).unwrap().build().unwrap(); + + let data_a = Arc::new(Int32Array::from(vec![1, 2, 3])) as ArrayRef; + let to_write = RecordBatch::try_new(arrow_schema_old.clone(), vec![data_a]).unwrap(); + + let props = WriterProperties::builder() + .set_compression(Compression::SNAPPY) + .build(); + let file = File::create(format!("{}/old_file.parquet", &table_location)).unwrap(); + let mut writer = ArrowWriter::try_new(file, to_write.schema(), Some(props)).unwrap(); + writer.write(&to_write).expect("Writing batch"); + writer.close().unwrap(); + + // Read the old Parquet file using the NEW schema (with column 'b') + let reader = ArrowReaderBuilder::new(file_io).build(); + let tasks = Box::pin(futures::stream::iter( + vec![Ok(FileScanTask { + start: 0, + length: 0, + record_count: None, + data_file_path: format!("{}/old_file.parquet", table_location), + data_file_format: DataFileFormat::Parquet, + schema: new_schema.clone(), + project_field_ids: vec![1, 2], // Request both columns 'a' and 'b' + predicate: None, + deletes: vec![], + })] + .into_iter(), + )) as FileScanTaskStream; + + let result = reader + .read(tasks) + .unwrap() + .try_collect::>() + .await + .unwrap(); + + // Verify we got the correct data + assert_eq!(result.len(), 1); + let batch = &result[0]; + + // Should have 2 columns now + assert_eq!(batch.num_columns(), 2); + assert_eq!(batch.num_rows(), 3); + + // Column 'a' should have the original data + let col_a = batch + .column(0) + .as_primitive::(); + assert_eq!(col_a.values(), &[1, 2, 3]); + + // Column 'b' should be all NULLs (it didn't exist in the old file) + let col_b = batch + .column(1) + .as_primitive::(); + assert_eq!(col_b.null_count(), 3); + assert!(col_b.is_null(0)); + assert!(col_b.is_null(1)); + assert!(col_b.is_null(2)); + } } From 5c6907a84a95a563fb057f85376b32360d9fe65a Mon Sep 17 00:00:00 2001 From: Shawn Chang Date: Fri, 12 Sep 2025 15:28:17 -0700 Subject: [PATCH 04/47] Concentrated Storage trait, need to fix a lot --- crates/iceberg/src/io/file_io.rs | 133 ++++++++++++++++--------------- crates/iceberg/src/io/storage.rs | 95 +++++++++++++++++++--- 2 files changed, 154 insertions(+), 74 deletions(-) diff --git a/crates/iceberg/src/io/file_io.rs b/crates/iceberg/src/io/file_io.rs index 6e2d152ed7..d2020ad062 100644 --- a/crates/iceberg/src/io/file_io.rs +++ b/crates/iceberg/src/io/file_io.rs @@ -17,16 +17,49 @@ use std::any::{Any, TypeId}; use std::collections::HashMap; +use std::fmt::Debug; use std::ops::Range; use std::sync::Arc; - +use async_trait::async_trait; use bytes::Bytes; -use opendal::Operator; use url::Url; -use super::storage::Storage; +use super::storage::OpenDALStorage; use crate::{Error, ErrorKind, Result}; +#[async_trait] +pub trait Storage: Debug + Send + Sync { + /// Check if a file exists at the given path + async fn exists(&self, path: &str) -> Result; + + /// Get metadata from an input path + async fn metadata(&self, path: &str) -> Result; + + /// Read bytes from a path + async fn read(&self, path: &str) -> Result; + + /// Get FileRead from a path + async fn reader(&self, path: &str) -> Result>; + + /// Write bytes to an output path + async fn write(&self, path: &str, bs: Bytes) -> Result<()>; + + /// Get FileWrite from a path + async fn writer(&self, path: &str) -> Result>; + + /// Delete a file at the given path + async fn delete(&self, path: &str) -> Result<()>; + + /// Remove a directory and all its contents recursively + async fn remove_dir_all(&self, path: &str) -> Result<()>; + + /// Create a new input file for reading + fn new_input(&self, path: &str) -> Result; + + /// Create a new output file for writing + fn new_output(&self, path: &str) -> Result; +} + /// FileIO implementation, used to manipulate files in underlying storage. /// /// # Note @@ -48,7 +81,7 @@ use crate::{Error, ErrorKind, Result}; pub struct FileIO { builder: FileIOBuilder, - inner: Arc, + inner: Arc, } impl FileIO { @@ -89,8 +122,9 @@ impl FileIO { /// /// * path: It should be *absolute* path starting with scheme string used to construct [`FileIO`]. pub async fn delete(&self, path: impl AsRef) -> Result<()> { - let (op, relative_path) = self.inner.create_operator(&path)?; - Ok(op.delete(relative_path).await?) + // let (op, relative_path) = self.inner.create_operator(&path)?; + // Ok(op.delete(relative_path).await?) + self.inner.delete(path.as_ref()).await } /// Remove the path and all nested dirs and files recursively. @@ -100,8 +134,8 @@ impl FileIO { /// * path: It should be *absolute* path starting with scheme string used to construct [`FileIO`]. #[deprecated(note = "use remove_dir_all instead", since = "0.4.0")] pub async fn remove_all(&self, path: impl AsRef) -> Result<()> { - let (op, relative_path) = self.inner.create_operator(&path)?; - Ok(op.remove_all(relative_path).await?) + // todo this should be removed as it doesn't exist in the new trait + self.inner.remove_dir_all(path.as_ref()).await } /// Remove the path and all nested dirs and files recursively. @@ -116,13 +150,7 @@ impl FileIO { /// - If the path is a empty directory, this function will remove the directory itself. /// - If the path is a non-empty directory, this function will remove the directory and all nested files and directories. pub async fn remove_dir_all(&self, path: impl AsRef) -> Result<()> { - let (op, relative_path) = self.inner.create_operator(&path)?; - let path = if relative_path.ends_with('/') { - relative_path.to_string() - } else { - format!("{relative_path}/") - }; - Ok(op.remove_all(&path).await?) + self.inner.remove_dir_all(path.as_ref()).await } /// Check file exists. @@ -131,8 +159,7 @@ impl FileIO { /// /// * path: It should be *absolute* path starting with scheme string used to construct [`FileIO`]. pub async fn exists(&self, path: impl AsRef) -> Result { - let (op, relative_path) = self.inner.create_operator(&path)?; - Ok(op.exists(relative_path).await?) + self.inner.exists(path.as_ref()).await } /// Creates input file. @@ -141,14 +168,7 @@ impl FileIO { /// /// * path: It should be *absolute* path starting with scheme string used to construct [`FileIO`]. pub fn new_input(&self, path: impl AsRef) -> Result { - let (op, relative_path) = self.inner.create_operator(&path)?; - let path = path.as_ref().to_string(); - let relative_path_pos = path.len() - relative_path.len(); - Ok(InputFile { - op, - path, - relative_path_pos, - }) + self.inner.new_input(path.as_ref()) } /// Creates output file. @@ -157,14 +177,7 @@ impl FileIO { /// /// * path: It should be *absolute* path starting with scheme string used to construct [`FileIO`]. pub fn new_output(&self, path: impl AsRef) -> Result { - let (op, relative_path) = self.inner.create_operator(&path)?; - let path = path.as_ref().to_string(); - let relative_path_pos = path.len() - relative_path.len(); - Ok(OutputFile { - op, - path, - relative_path_pos, - }) + self.inner.new_output(path.as_ref()) } } @@ -273,7 +286,7 @@ impl FileIOBuilder { /// Builds [`FileIO`]. pub fn build(self) -> Result { - let storage = Storage::build(self.clone())?; + let storage = OpenDALStorage::build(self.clone())?; Ok(FileIO { builder: self, inner: Arc::new(storage), @@ -313,11 +326,12 @@ impl FileRead for opendal::Reader { /// Input file is used for reading from files. #[derive(Debug)] pub struct InputFile { - op: Operator, + pub storage: Arc, // Absolution path of file. - path: String, - // Relative path of file to uri, starts at [`relative_path_pos`] - relative_path_pos: usize, + pub path: String, + // todo should remove this? Should always pass down a full path + // // Relative path of file to uri, starts at [`relative_path_pos`] + // relative_path_pos: usize, } impl InputFile { @@ -328,34 +342,29 @@ impl InputFile { /// Check if file exists. pub async fn exists(&self) -> crate::Result { - Ok(self.op.exists(&self.path[self.relative_path_pos..]).await?) + self.storage.exists(&self.path).await } /// Fetch and returns metadata of file. pub async fn metadata(&self) -> crate::Result { - let meta = self.op.stat(&self.path[self.relative_path_pos..]).await?; - - Ok(FileMetadata { - size: meta.content_length(), - }) + self.storage.metadata(&self.path).await } /// Read and returns whole content of file. /// /// For continuous reading, use [`Self::reader`] instead. pub async fn read(&self) -> crate::Result { - Ok(self - .op - .read(&self.path[self.relative_path_pos..]) - .await? - .to_bytes()) + self + .storage + .read(&self.path) + .await } /// Creates [`FileRead`] for continuous reading. /// /// For one-time reading, use [`Self::read`] instead. - pub async fn reader(&self) -> crate::Result> { - Ok(self.op.reader(&self.path[self.relative_path_pos..]).await?) + pub async fn reader(&self) -> crate::Result> { + self.storage.reader(&self.path).await } } @@ -404,11 +413,12 @@ impl FileWrite for Box { /// Output file is used for writing to files.. #[derive(Debug)] pub struct OutputFile { - op: Operator, + pub storage: Arc, // Absolution path of file. - path: String, - // Relative path of file to uri, starts at [`relative_path_pos`] - relative_path_pos: usize, + pub path: String, + // todo should always pass down a full path + // // Relative path of file to uri, starts at [`relative_path_pos`] + // relative_path_pos: usize, } impl OutputFile { @@ -419,22 +429,21 @@ impl OutputFile { /// Checks if file exists. pub async fn exists(&self) -> Result { - Ok(self.op.exists(&self.path[self.relative_path_pos..]).await?) + Ok(self.storage.exists(&self.path).await?) } /// Deletes file. /// /// If the file does not exist, it will not return error. pub async fn delete(&self) -> Result<()> { - Ok(self.op.delete(&self.path[self.relative_path_pos..]).await?) + Ok(self.storage.delete(&self.path).await?) } /// Converts into [`InputFile`]. pub fn to_input_file(self) -> InputFile { InputFile { - op: self.op, + storage: self.storage, path: self.path, - relative_path_pos: self.relative_path_pos, } } @@ -445,9 +454,7 @@ impl OutputFile { /// Calling `write` will overwrite the file if it exists. /// For continuous writing, use [`Self::writer`]. pub async fn write(&self, bs: Bytes) -> crate::Result<()> { - let mut writer = self.writer().await?; - writer.write(bs).await?; - writer.close().await + self.storage.write(self.path.as_str(), bs).await } /// Creates output file for continuous writing. @@ -457,7 +464,7 @@ impl OutputFile { /// For one-time writing, use [`Self::write`] instead. pub async fn writer(&self) -> crate::Result> { Ok(Box::new( - self.op.writer(&self.path[self.relative_path_pos..]).await?, + self.storage.writer(&self.path).await? )) } } diff --git a/crates/iceberg/src/io/storage.rs b/crates/iceberg/src/io/storage.rs index 3de4f10dbf..e0fae92abe 100644 --- a/crates/iceberg/src/io/storage.rs +++ b/crates/iceberg/src/io/storage.rs @@ -16,7 +16,7 @@ // under the License. use std::sync::Arc; - +use bytes::Bytes; use opendal::layers::RetryLayer; #[cfg(feature = "storage-azdls")] use opendal::services::AzdlsConfig; @@ -30,14 +30,14 @@ use opendal::{Operator, Scheme}; #[cfg(feature = "storage-azdls")] use super::AzureStorageScheme; -use super::FileIOBuilder; +use super::{FileIOBuilder, FileMetadata, FileRead, FileWrite, InputFile, OutputFile, Storage}; #[cfg(feature = "storage-s3")] use crate::io::CustomAwsCredentialLoader; use crate::{Error, ErrorKind}; /// The storage carries all supported storage services in iceberg -#[derive(Debug)] -pub(crate) enum Storage { +#[derive(Debug, Clone)] +pub(crate) enum OpenDALStorage { #[cfg(feature = "storage-memory")] Memory(Operator), #[cfg(feature = "storage-fs")] @@ -67,7 +67,80 @@ pub(crate) enum Storage { }, } -impl Storage { +#[async_trait::async_trait] +impl Storage for OpenDALStorage { + async fn exists(&self, path: &str) -> crate::Result { + let (op, relative_path) = self.create_operator(&path)?; + Ok(op.exists(relative_path).await?) + } + + async fn metadata(&self, path: &str) -> crate::Result { + let (op, relative_path) = self.create_operator(&path)?; + let meta = op.stat(relative_path).await?; + + Ok(FileMetadata { + size: meta.content_length(), + }) + } + + async fn read(&self, path: &str) -> crate::Result { + let (op, relative_path) = self.create_operator(&path)?; + Ok(op.read(relative_path).await?.to_bytes()) + } + + async fn reader(&self, path: &str) -> crate::Result> { + let (op, relative_path) = self.create_operator(&path)?; + Ok(Box::new(op.reader(relative_path).await?)) + } + + async fn write(&self, path: &str, bs: Bytes) -> crate::Result<()> { + let mut writer = self.writer(path).await?; + writer.write(bs).await?; + writer.close().await + } + + async fn writer(&self, path: &str) -> crate::Result> { + let (op, relative_path) = self.create_operator(&path)?; + Ok(Box::new( + op.writer(relative_path).await?, + )) + } + + async fn delete(&self, path: &str) -> crate::Result<()> { + let (op, relative_path) = self.create_operator(&path)?; + Ok(op.delete(relative_path).await?) + } + + async fn remove_dir_all(&self, path: &str) -> crate::Result<()> { + let (op, relative_path) = self.create_operator(&path)?; + let path = if relative_path.ends_with('/') { + relative_path.to_string() + } else { + format!("{relative_path}/") + }; + Ok(op.remove_all(&path).await?) + } + + fn new_input(&self, path: &str) -> crate::Result { + let storage = Arc::new(self.clone()); + let path = path.to_string(); + Ok(InputFile { + storage, + path, + }) + } + + fn new_output(&self, path: &str) -> crate::Result { + let storage = Arc::new(self.clone()); + let path = path.to_string(); + Ok(OutputFile { + storage, + path, + }) + } +} + +impl OpenDALStorage { /// Convert iceberg config to opendal config. pub(crate) fn build(file_io_builder: FileIOBuilder) -> crate::Result { let (scheme_str, props, extensions) = file_io_builder.into_parts(); @@ -129,7 +202,7 @@ impl Storage { let path = path.as_ref(); let (operator, relative_path): (Operator, &str) = match self { #[cfg(feature = "storage-memory")] - Storage::Memory(op) => { + OpenDALStorage::Memory(op) => { if let Some(stripped) = path.strip_prefix("memory:/") { Ok::<_, crate::Error>((op.clone(), stripped)) } else { @@ -137,7 +210,7 @@ impl Storage { } } #[cfg(feature = "storage-fs")] - Storage::LocalFs => { + OpenDALStorage::LocalFs => { let op = super::fs_config_build()?; if let Some(stripped) = path.strip_prefix("file:/") { @@ -147,7 +220,7 @@ impl Storage { } } #[cfg(feature = "storage-s3")] - Storage::S3 { + OpenDALStorage::S3 { configured_scheme, config, customized_credential_load, @@ -167,7 +240,7 @@ impl Storage { } } #[cfg(feature = "storage-gcs")] - Storage::Gcs { config } => { + OpenDALStorage::Gcs { config } => { let operator = super::gcs_config_build(config, path)?; let prefix = format!("gs://{}/", operator.info().name()); if path.starts_with(&prefix) { @@ -180,7 +253,7 @@ impl Storage { } } #[cfg(feature = "storage-oss")] - Storage::Oss { config } => { + OpenDALStorage::Oss { config } => { let op = super::oss_config_build(config, path)?; // Check prefix of oss path. @@ -195,7 +268,7 @@ impl Storage { } } #[cfg(feature = "storage-azdls")] - Storage::Azdls { + OpenDALStorage::Azdls { configured_scheme, config, } => super::azdls_create_operator(path, config, configured_scheme), From 9a27198ce8f22efaf10433718a28c0918972d32e Mon Sep 17 00:00:00 2001 From: Shawn Chang Date: Wed, 17 Sep 2025 16:26:56 -0700 Subject: [PATCH 05/47] pure storage, fixed compile issues --- crates/iceberg/src/arrow/reader.rs | 12 ++++++------ crates/iceberg/src/io/file_io.rs | 17 +++++++++++------ crates/iceberg/src/io/storage.rs | 26 ++++++++++++++------------ crates/iceberg/src/puffin/metadata.rs | 5 ++--- crates/iceberg/src/puffin/reader.rs | 2 +- 5 files changed, 34 insertions(+), 28 deletions(-) diff --git a/crates/iceberg/src/arrow/reader.rs b/crates/iceberg/src/arrow/reader.rs index ff4cff0a64..077961d493 100644 --- a/crates/iceberg/src/arrow/reader.rs +++ b/crates/iceberg/src/arrow/reader.rs @@ -327,7 +327,7 @@ impl ArrowReader { data_file_path: &str, file_io: FileIO, should_load_page_index: bool, - ) -> Result>> { + ) -> Result> { // Get the metadata for the Parquet file we need to read and build // a reader for the data within let parquet_file = file_io.new_input(data_file_path)?; @@ -1312,18 +1312,18 @@ impl BoundPredicateVisitor for PredicateConverter<'_> { } /// ArrowFileReader is a wrapper around a FileRead that impls parquets AsyncFileReader. -pub struct ArrowFileReader { +pub struct ArrowFileReader { meta: FileMetadata, preload_column_index: bool, preload_offset_index: bool, preload_page_index: bool, metadata_size_hint: Option, - r: R, + r: Box, } -impl ArrowFileReader { +impl ArrowFileReader { /// Create a new ArrowFileReader - pub fn new(meta: FileMetadata, r: R) -> Self { + pub fn new(meta: FileMetadata, r: Box) -> Self { Self { meta, preload_column_index: false, @@ -1362,7 +1362,7 @@ impl ArrowFileReader { } } -impl AsyncFileReader for ArrowFileReader { +impl AsyncFileReader for ArrowFileReader { fn get_bytes(&mut self, range: Range) -> BoxFuture<'_, parquet::errors::Result> { Box::pin( self.r diff --git a/crates/iceberg/src/io/file_io.rs b/crates/iceberg/src/io/file_io.rs index d2020ad062..d45a52462d 100644 --- a/crates/iceberg/src/io/file_io.rs +++ b/crates/iceberg/src/io/file_io.rs @@ -27,6 +27,7 @@ use url::Url; use super::storage::OpenDALStorage; use crate::{Error, ErrorKind, Result}; +/// todo doc #[async_trait] pub trait Storage: Debug + Send + Sync { /// Check if a file exists at the given path @@ -326,8 +327,10 @@ impl FileRead for opendal::Reader { /// Input file is used for reading from files. #[derive(Debug)] pub struct InputFile { + /// todo doc pub storage: Arc, // Absolution path of file. + /// todo doc pub path: String, // todo should remove this? Should always pass down a full path // // Relative path of file to uri, starts at [`relative_path_pos`] @@ -353,7 +356,7 @@ impl InputFile { /// Read and returns whole content of file. /// /// For continuous reading, use [`Self::reader`] instead. - pub async fn read(&self) -> crate::Result { + pub async fn read(&self) -> Result { self .storage .read(&self.path) @@ -363,7 +366,7 @@ impl InputFile { /// Creates [`FileRead`] for continuous reading. /// /// For one-time reading, use [`Self::read`] instead. - pub async fn reader(&self) -> crate::Result> { + pub async fn reader(&self) -> Result> { self.storage.reader(&self.path).await } } @@ -374,17 +377,17 @@ impl InputFile { /// /// It's possible for us to remove the async_trait, but we need to figure /// out how to handle the object safety. -#[async_trait::async_trait] -pub trait FileWrite: Send + Unpin + 'static { +#[async_trait] +pub trait FileWrite: Send + Sync + Unpin + 'static { /// Write bytes to file. /// /// TODO: we can support writing non-contiguous bytes in the future. - async fn write(&mut self, bs: Bytes) -> crate::Result<()>; + async fn write(&mut self, bs: Bytes) -> Result<()>; /// Close file. /// /// Calling close on closed file will generate an error. - async fn close(&mut self) -> crate::Result<()>; + async fn close(&mut self) -> Result<()>; } #[async_trait::async_trait] @@ -413,8 +416,10 @@ impl FileWrite for Box { /// Output file is used for writing to files.. #[derive(Debug)] pub struct OutputFile { + /// todo fix pub qualifier pub storage: Arc, // Absolution path of file. + /// todo fix pub qualifier pub path: String, // todo should always pass down a full path // // Relative path of file to uri, starts at [`relative_path_pos`] diff --git a/crates/iceberg/src/io/storage.rs b/crates/iceberg/src/io/storage.rs index e0fae92abe..0a8274de21 100644 --- a/crates/iceberg/src/io/storage.rs +++ b/crates/iceberg/src/io/storage.rs @@ -15,7 +15,9 @@ // specific language governing permissions and limitations // under the License. +use crate::Result; use std::sync::Arc; +use async_trait::async_trait; use bytes::Bytes; use opendal::layers::RetryLayer; #[cfg(feature = "storage-azdls")] @@ -67,14 +69,14 @@ pub(crate) enum OpenDALStorage { }, } -#[async_trait::async_trait] +#[async_trait] impl Storage for OpenDALStorage { - async fn exists(&self, path: &str) -> crate::Result { + async fn exists(&self, path: &str) -> Result { let (op, relative_path) = self.create_operator(&path)?; Ok(op.exists(relative_path).await?) } - async fn metadata(&self, path: &str) -> crate::Result { + async fn metadata(&self, path: &str) -> Result { let (op, relative_path) = self.create_operator(&path)?; let meta = op.stat(relative_path).await?; @@ -83,35 +85,35 @@ impl Storage for OpenDALStorage { }) } - async fn read(&self, path: &str) -> crate::Result { + async fn read(&self, path: &str) -> Result { let (op, relative_path) = self.create_operator(&path)?; Ok(op.read(relative_path).await?.to_bytes()) } - async fn reader(&self, path: &str) -> crate::Result> { + async fn reader(&self, path: &str) -> Result> { let (op, relative_path) = self.create_operator(&path)?; Ok(Box::new(op.reader(relative_path).await?)) } - async fn write(&self, path: &str, bs: Bytes) -> crate::Result<()> { + async fn write(&self, path: &str, bs: Bytes) -> Result<()> { let mut writer = self.writer(path).await?; writer.write(bs).await?; writer.close().await } - async fn writer(&self, path: &str) -> crate::Result> { + async fn writer(&self, path: &str) -> Result> { let (op, relative_path) = self.create_operator(&path)?; Ok(Box::new( op.writer(relative_path).await?, )) } - async fn delete(&self, path: &str) -> crate::Result<()> { + async fn delete(&self, path: &str) -> Result<()> { let (op, relative_path) = self.create_operator(&path)?; Ok(op.delete(relative_path).await?) } - async fn remove_dir_all(&self, path: &str) -> crate::Result<()> { + async fn remove_dir_all(&self, path: &str) -> Result<()> { let (op, relative_path) = self.create_operator(&path)?; let path = if relative_path.ends_with('/') { relative_path.to_string() @@ -121,7 +123,7 @@ impl Storage for OpenDALStorage { Ok(op.remove_all(&path).await?) } - fn new_input(&self, path: &str) -> crate::Result { + fn new_input(&self, path: &str) -> Result { let storage = Arc::new(self.clone()); let path = path.to_string(); Ok(InputFile { @@ -130,7 +132,7 @@ impl Storage for OpenDALStorage { }) } - fn new_output(&self, path: &str) -> crate::Result { + fn new_output(&self, path: &str) -> Result { let storage = Arc::new(self.clone()); let path = path.to_string(); Ok(OutputFile { @@ -142,7 +144,7 @@ impl Storage for OpenDALStorage { impl OpenDALStorage { /// Convert iceberg config to opendal config. - pub(crate) fn build(file_io_builder: FileIOBuilder) -> crate::Result { + pub(crate) fn build(file_io_builder: FileIOBuilder) -> Result { let (scheme_str, props, extensions) = file_io_builder.into_parts(); let scheme = Self::parse_scheme(&scheme_str)?; diff --git a/crates/iceberg/src/puffin/metadata.rs b/crates/iceberg/src/puffin/metadata.rs index 6039c7f820..e2fea212b7 100644 --- a/crates/iceberg/src/puffin/metadata.rs +++ b/crates/iceberg/src/puffin/metadata.rs @@ -16,7 +16,6 @@ // under the License. use std::collections::{HashMap, HashSet}; - use bytes::Bytes; use serde::{Deserialize, Serialize}; @@ -288,9 +287,9 @@ impl FileMetadata { let input_file_length = input_file.metadata().await?.size; let footer_payload_length = - FileMetadata::read_footer_payload_length(&file_read, input_file_length).await?; + FileMetadata::read_footer_payload_length(file_read.as_ref(), input_file_length).await?; let footer_bytes = - FileMetadata::read_footer_bytes(&file_read, input_file_length, footer_payload_length) + FileMetadata::read_footer_bytes(file_read.as_ref(), input_file_length, footer_payload_length) .await?; let magic_length = FileMetadata::MAGIC_LENGTH as usize; diff --git a/crates/iceberg/src/puffin/reader.rs b/crates/iceberg/src/puffin/reader.rs index dce53d93f0..8941d4a8fc 100644 --- a/crates/iceberg/src/puffin/reader.rs +++ b/crates/iceberg/src/puffin/reader.rs @@ -18,7 +18,7 @@ use tokio::sync::OnceCell; use crate::Result; -use crate::io::{FileRead, InputFile}; +use crate::io::InputFile; use crate::puffin::blob::Blob; use crate::puffin::metadata::{BlobMetadata, FileMetadata}; From 535de901901b65c5c553261a46942e5329a95319 Mon Sep 17 00:00:00 2001 From: Shawn Chang Date: Wed, 17 Sep 2025 16:56:08 -0700 Subject: [PATCH 06/47] fmt is good --- crates/iceberg/src/io/file_io.rs | 24 ++++++++++-------------- crates/iceberg/src/io/storage.rs | 20 ++++++-------------- crates/iceberg/src/puffin/metadata.rs | 10 +++++++--- 3 files changed, 23 insertions(+), 31 deletions(-) diff --git a/crates/iceberg/src/io/file_io.rs b/crates/iceberg/src/io/file_io.rs index d45a52462d..e362e67ea7 100644 --- a/crates/iceberg/src/io/file_io.rs +++ b/crates/iceberg/src/io/file_io.rs @@ -20,6 +20,7 @@ use std::collections::HashMap; use std::fmt::Debug; use std::ops::Range; use std::sync::Arc; + use async_trait::async_trait; use bytes::Bytes; use url::Url; @@ -32,16 +33,16 @@ use crate::{Error, ErrorKind, Result}; pub trait Storage: Debug + Send + Sync { /// Check if a file exists at the given path async fn exists(&self, path: &str) -> Result; - + /// Get metadata from an input path async fn metadata(&self, path: &str) -> Result; - + /// Read bytes from a path async fn read(&self, path: &str) -> Result; - + /// Get FileRead from a path async fn reader(&self, path: &str) -> Result>; - + /// Write bytes to an output path async fn write(&self, path: &str, bs: Bytes) -> Result<()>; @@ -319,7 +320,7 @@ pub trait FileRead: Send + Sync + Unpin + 'static { #[async_trait::async_trait] impl FileRead for opendal::Reader { - async fn read(&self, range: Range) -> crate::Result { + async fn read(&self, range: Range) -> Result { Ok(opendal::Reader::read(self, range).await?.to_bytes()) } } @@ -344,12 +345,12 @@ impl InputFile { } /// Check if file exists. - pub async fn exists(&self) -> crate::Result { + pub async fn exists(&self) -> Result { self.storage.exists(&self.path).await } /// Fetch and returns metadata of file. - pub async fn metadata(&self) -> crate::Result { + pub async fn metadata(&self) -> Result { self.storage.metadata(&self.path).await } @@ -357,10 +358,7 @@ impl InputFile { /// /// For continuous reading, use [`Self::reader`] instead. pub async fn read(&self) -> Result { - self - .storage - .read(&self.path) - .await + self.storage.read(&self.path).await } /// Creates [`FileRead`] for continuous reading. @@ -468,9 +466,7 @@ impl OutputFile { /// /// For one-time writing, use [`Self::write`] instead. pub async fn writer(&self) -> crate::Result> { - Ok(Box::new( - self.storage.writer(&self.path).await? - )) + Ok(Box::new(self.storage.writer(&self.path).await?)) } } diff --git a/crates/iceberg/src/io/storage.rs b/crates/iceberg/src/io/storage.rs index 0a8274de21..62c6b8a31e 100644 --- a/crates/iceberg/src/io/storage.rs +++ b/crates/iceberg/src/io/storage.rs @@ -15,8 +15,8 @@ // specific language governing permissions and limitations // under the License. -use crate::Result; use std::sync::Arc; + use async_trait::async_trait; use bytes::Bytes; use opendal::layers::RetryLayer; @@ -35,7 +35,7 @@ use super::AzureStorageScheme; use super::{FileIOBuilder, FileMetadata, FileRead, FileWrite, InputFile, OutputFile, Storage}; #[cfg(feature = "storage-s3")] use crate::io::CustomAwsCredentialLoader; -use crate::{Error, ErrorKind}; +use crate::{Error, ErrorKind, Result}; /// The storage carries all supported storage services in iceberg #[derive(Debug, Clone)] @@ -79,7 +79,7 @@ impl Storage for OpenDALStorage { async fn metadata(&self, path: &str) -> Result { let (op, relative_path) = self.create_operator(&path)?; let meta = op.stat(relative_path).await?; - + Ok(FileMetadata { size: meta.content_length(), }) @@ -103,9 +103,7 @@ impl Storage for OpenDALStorage { async fn writer(&self, path: &str) -> Result> { let (op, relative_path) = self.create_operator(&path)?; - Ok(Box::new( - op.writer(relative_path).await?, - )) + Ok(Box::new(op.writer(relative_path).await?)) } async fn delete(&self, path: &str) -> Result<()> { @@ -126,19 +124,13 @@ impl Storage for OpenDALStorage { fn new_input(&self, path: &str) -> Result { let storage = Arc::new(self.clone()); let path = path.to_string(); - Ok(InputFile { - storage, - path, - }) + Ok(InputFile { storage, path }) } fn new_output(&self, path: &str) -> Result { let storage = Arc::new(self.clone()); let path = path.to_string(); - Ok(OutputFile { - storage, - path, - }) + Ok(OutputFile { storage, path }) } } diff --git a/crates/iceberg/src/puffin/metadata.rs b/crates/iceberg/src/puffin/metadata.rs index e2fea212b7..2686db06d6 100644 --- a/crates/iceberg/src/puffin/metadata.rs +++ b/crates/iceberg/src/puffin/metadata.rs @@ -16,6 +16,7 @@ // under the License. use std::collections::{HashMap, HashSet}; + use bytes::Bytes; use serde::{Deserialize, Serialize}; @@ -288,9 +289,12 @@ impl FileMetadata { let input_file_length = input_file.metadata().await?.size; let footer_payload_length = FileMetadata::read_footer_payload_length(file_read.as_ref(), input_file_length).await?; - let footer_bytes = - FileMetadata::read_footer_bytes(file_read.as_ref(), input_file_length, footer_payload_length) - .await?; + let footer_bytes = FileMetadata::read_footer_bytes( + file_read.as_ref(), + input_file_length, + footer_payload_length, + ) + .await?; let magic_length = FileMetadata::MAGIC_LENGTH as usize; // check first four bytes of footer From 901db63d41a28acc5bcc2b32027834ebfde69630 Mon Sep 17 00:00:00 2001 From: Shawn Chang Date: Thu, 16 Oct 2025 19:07:07 -0700 Subject: [PATCH 07/47] builder, registry, split by schemes --- crates/iceberg/src/io/file_io.rs | 26 ++- crates/iceberg/src/io/mod.rs | 2 + crates/iceberg/src/io/storage_azdls.rs | 124 ++++++++++++- crates/iceberg/src/io/storage_builder.rs | 210 +++++++++++++++++++++++ crates/iceberg/src/io/storage_fs.rs | 113 +++++++++++- crates/iceberg/src/io/storage_gcs.rs | 117 ++++++++++++- crates/iceberg/src/io/storage_memory.rs | 109 +++++++++++- crates/iceberg/src/io/storage_oss.rs | 114 ++++++++++++ crates/iceberg/src/io/storage_s3.rs | 147 +++++++++++++++- 9 files changed, 951 insertions(+), 11 deletions(-) create mode 100644 crates/iceberg/src/io/storage_builder.rs diff --git a/crates/iceberg/src/io/file_io.rs b/crates/iceberg/src/io/file_io.rs index e362e67ea7..4ce81d8f04 100644 --- a/crates/iceberg/src/io/file_io.rs +++ b/crates/iceberg/src/io/file_io.rs @@ -25,10 +25,9 @@ use async_trait::async_trait; use bytes::Bytes; use url::Url; -use super::storage::OpenDALStorage; use crate::{Error, ErrorKind, Result}; -/// todo doc +/// Trait for storage operations in Iceberg #[async_trait] pub trait Storage: Debug + Send + Sync { /// Check if a file exists at the given path @@ -62,6 +61,16 @@ pub trait Storage: Debug + Send + Sync { fn new_output(&self, path: &str) -> Result; } +/// Common interface for all storage builders. +pub trait StorageBuilder: Debug + Send + Sync { + /// Create a new storage instance with the given properties and extensions. + fn build( + &self, + props: HashMap, + extensions: Extensions, + ) -> Result>; +} + /// FileIO implementation, used to manipulate files in underlying storage. /// /// # Note @@ -288,10 +297,19 @@ impl FileIOBuilder { /// Builds [`FileIO`]. pub fn build(self) -> Result { - let storage = OpenDALStorage::build(self.clone())?; + // Use the scheme to determine the storage type + let scheme = self.scheme_str.clone().unwrap_or_default(); + + // Create registry and get builder + let registry = crate::io::StorageBuilderRegistry::new(); + let builder = registry.get_builder(scheme.as_str())?; + + // Build storage with props and extensions + let storage = builder.build(self.props.clone(), self.extensions.clone())?; + Ok(FileIO { builder: self, - inner: Arc::new(storage), + inner: storage, }) } } diff --git a/crates/iceberg/src/io/mod.rs b/crates/iceberg/src/io/mod.rs index 5eb5964345..5f234961fc 100644 --- a/crates/iceberg/src/io/mod.rs +++ b/crates/iceberg/src/io/mod.rs @@ -68,8 +68,10 @@ mod file_io; mod storage; +mod storage_builder; pub use file_io::*; +pub use storage_builder::StorageBuilderRegistry; pub(crate) mod object_cache; #[cfg(feature = "storage-azdls")] diff --git a/crates/iceberg/src/io/storage_azdls.rs b/crates/iceberg/src/io/storage_azdls.rs index fe12167f6f..2afccd5a01 100644 --- a/crates/iceberg/src/io/storage_azdls.rs +++ b/crates/iceberg/src/io/storage_azdls.rs @@ -18,11 +18,18 @@ use std::collections::HashMap; use std::fmt::Display; use std::str::FromStr; +use std::sync::Arc; -use opendal::Configurator; +use async_trait::async_trait; +use bytes::Bytes; use opendal::services::AzdlsConfig; +use opendal::{Configurator, Operator}; use url::Url; +use crate::io::{ + Extensions, FileMetadata, FileRead, FileWrite, InputFile, OutputFile, Storage, + StorageBuilder, +}; use crate::{Error, ErrorKind, Result, ensure_data_valid}; /// A connection string. @@ -125,7 +132,7 @@ pub(crate) fn azdls_create_operator<'a>( /// paths are expected to contain the `dfs` storage service. /// - `wasb[s]` is used to refer to files in Blob Storage directly; paths are /// expected to contain the `blob` storage service. -#[derive(Debug, PartialEq)] +#[derive(Debug, Clone, PartialEq)] pub(crate) enum AzureStorageScheme { Abfs, Abfss, @@ -597,3 +604,116 @@ mod tests { } } } + +/// Azure Data Lake Storage implementation using OpenDAL +#[derive(Debug, Clone)] +pub struct OpenDALAzdlsStorage { + /// Because Azdls accepts multiple possible schemes, we store the full + /// passed scheme here to later validate schemes passed via paths. + configured_scheme: AzureStorageScheme, + config: Arc, +} + +impl OpenDALAzdlsStorage { + /// Creates operator from path. + fn create_operator<'a>(&self, path: &'a str) -> Result<(Operator, &'a str)> { + let (op, relative_path) = azdls_create_operator(path, &self.config, &self.configured_scheme)?; + let op = op.layer(opendal::layers::RetryLayer::new()); + Ok((op, relative_path)) + } +} + +#[async_trait] +impl Storage for OpenDALAzdlsStorage { + async fn exists(&self, path: &str) -> Result { + let (op, relative_path) = self.create_operator(path)?; + Ok(op.exists(relative_path).await?) + } + + async fn metadata(&self, path: &str) -> Result { + let (op, relative_path) = self.create_operator(path)?; + let meta = op.stat(relative_path).await?; + Ok(FileMetadata { + size: meta.content_length(), + }) + } + + async fn read(&self, path: &str) -> Result { + let (op, relative_path) = self.create_operator(path)?; + Ok(op.read(relative_path).await?.to_bytes()) + } + + async fn reader(&self, path: &str) -> Result> { + let (op, relative_path) = self.create_operator(path)?; + Ok(Box::new(op.reader(relative_path).await?)) + } + + async fn write(&self, path: &str, bs: Bytes) -> Result<()> { + let mut writer = self.writer(path).await?; + writer.write(bs).await?; + writer.close().await + } + + async fn writer(&self, path: &str) -> Result> { + let (op, relative_path) = self.create_operator(path)?; + Ok(Box::new(op.writer(relative_path).await?)) + } + + async fn delete(&self, path: &str) -> Result<()> { + let (op, relative_path) = self.create_operator(path)?; + Ok(op.delete(relative_path).await?) + } + + async fn remove_dir_all(&self, path: &str) -> Result<()> { + let (op, relative_path) = self.create_operator(path)?; + let path = if relative_path.ends_with('/') { + relative_path.to_string() + } else { + format!("{relative_path}/") + }; + Ok(op.remove_all(&path).await?) + } + + fn new_input(&self, path: &str) -> Result { + Ok(InputFile { + storage: Arc::new(self.clone()), + path: path.to_string(), + }) + } + + fn new_output(&self, path: &str) -> Result { + Ok(OutputFile { + storage: Arc::new(self.clone()), + path: path.to_string(), + }) + } +} + +/// Builder for Azure Data Lake Storage +#[derive(Debug)] +pub struct OpenDALAzdlsStorageBuilder; + +impl StorageBuilder for OpenDALAzdlsStorageBuilder { + fn build( + &self, + props: HashMap, + _extensions: Extensions, + ) -> Result> { + // Get the scheme string from the props or use default + let scheme_str = props + .get("scheme_str") + .cloned() + .unwrap_or_else(|| "abfs".to_string()); + + // Parse the scheme + let scheme = scheme_str.parse::()?; + + // Parse Azdls config from props + let config = azdls_config_parse(props)?; + + Ok(Arc::new(OpenDALAzdlsStorage { + configured_scheme: scheme, + config: Arc::new(config), + })) + } +} diff --git a/crates/iceberg/src/io/storage_builder.rs b/crates/iceberg/src/io/storage_builder.rs new file mode 100644 index 0000000000..aacd68bb00 --- /dev/null +++ b/crates/iceberg/src/io/storage_builder.rs @@ -0,0 +1,210 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +use std::collections::HashMap; +use std::sync::Arc; + +use crate::io::StorageBuilder; +#[cfg(feature = "storage-azdls")] +use crate::io::storage_azdls::OpenDALAzdlsStorageBuilder; +#[cfg(feature = "storage-fs")] +use crate::io::storage_fs::OpenDALFsStorageBuilder; +#[cfg(feature = "storage-gcs")] +use crate::io::storage_gcs::OpenDALGcsStorageBuilder; +#[cfg(feature = "storage-memory")] +use crate::io::storage_memory::OpenDALMemoryStorageBuilder; +#[cfg(feature = "storage-oss")] +use crate::io::storage_oss::OpenDALOssStorageBuilder; +#[cfg(feature = "storage-s3")] +use crate::io::storage_s3::OpenDALS3StorageBuilder; +use crate::{Error, ErrorKind, Result}; + +/// A registry of storage builders. +/// +/// The registry allows you to register custom storage builders for different URI schemes. +/// By default, it includes builders for all enabled storage features. +/// +/// # Example +/// +/// ```rust +/// use iceberg::io::StorageBuilderRegistry; +/// +/// // Create a new registry with default builders +/// let registry = StorageBuilderRegistry::new(); +/// +/// // Get supported storage types +/// let types = registry.supported_types(); +/// println!("Supported storage types: {:?}", types); +/// +/// // Get a builder for a specific scheme +/// # #[cfg(feature = "storage-memory")] +/// # { +/// let builder = registry.get_builder("memory").unwrap(); +/// # } +/// ``` +/// +/// You can also register custom storage builders: +/// +/// ```rust,ignore +/// use std::sync::Arc; +/// use iceberg::io::{StorageBuilderRegistry, StorageBuilder}; +/// +/// let mut registry = StorageBuilderRegistry::new(); +/// +/// // Register a custom storage builder +/// registry.register("custom", Arc::new(MyCustomStorageBuilder)); +/// ``` +#[derive(Debug, Clone)] +pub struct StorageBuilderRegistry { + builders: HashMap>, +} + +impl StorageBuilderRegistry { + /// Create a new storage registry with default builders based on enabled features. + pub fn new() -> Self { + let mut builders: HashMap> = HashMap::new(); + + #[cfg(feature = "storage-memory")] + { + let builder = Arc::new(OpenDALMemoryStorageBuilder) as Arc; + builders.insert("memory".to_string(), builder); + } + + #[cfg(feature = "storage-fs")] + { + let builder = Arc::new(OpenDALFsStorageBuilder) as Arc; + builders.insert("file".to_string(), builder.clone()); + builders.insert("".to_string(), builder); + } + + #[cfg(feature = "storage-s3")] + { + let builder = Arc::new(OpenDALS3StorageBuilder) as Arc; + builders.insert("s3".to_string(), builder.clone()); + builders.insert("s3a".to_string(), builder); + } + + #[cfg(feature = "storage-gcs")] + { + let builder = Arc::new(OpenDALGcsStorageBuilder) as Arc; + builders.insert("gs".to_string(), builder.clone()); + builders.insert("gcs".to_string(), builder); + } + + #[cfg(feature = "storage-oss")] + { + let builder = Arc::new(OpenDALOssStorageBuilder) as Arc; + builders.insert("oss".to_string(), builder); + } + + #[cfg(feature = "storage-azdls")] + { + let builder = Arc::new(OpenDALAzdlsStorageBuilder) as Arc; + builders.insert("abfs".to_string(), builder.clone()); + builders.insert("abfss".to_string(), builder.clone()); + builders.insert("wasb".to_string(), builder.clone()); + builders.insert("wasbs".to_string(), builder); + } + + Self { builders } + } + + /// Register a custom storage builder for a given scheme. + pub fn register(&mut self, scheme: impl Into, builder: Arc) { + self.builders.insert(scheme.into(), builder); + } + + /// Get a storage builder by scheme. + pub fn get_builder(&self, scheme: &str) -> Result> { + let key = scheme.trim(); + self.builders + .iter() + .find(|(k, _)| k.eq_ignore_ascii_case(key)) + .map(|(_, builder)| builder.clone()) + .ok_or_else(|| { + Error::new( + ErrorKind::FeatureUnsupported, + format!( + "Unsupported storage type: {}. Supported types: {}", + scheme, + self.supported_types().join(", ") + ), + ) + }) + } + + /// Return the list of supported storage types. + pub fn supported_types(&self) -> Vec { + self.builders.keys().cloned().collect() + } +} + +impl Default for StorageBuilderRegistry { + fn default() -> Self { + Self::new() + } +} + +#[cfg(test)] +mod tests { + use super::*; + + #[test] + fn test_storage_builder_registry_new() { + let registry = StorageBuilderRegistry::new(); + let types = registry.supported_types(); + + // At least one storage type should be available + assert!(!types.is_empty()); + } + + #[test] + #[cfg(feature = "storage-memory")] + fn test_storage_builder_registry_get_builder() { + let registry = StorageBuilderRegistry::new(); + + // Should be able to get memory storage builder + let builder = registry.get_builder("memory"); + assert!(builder.is_ok()); + + // Should be case-insensitive + let builder = registry.get_builder("MEMORY"); + assert!(builder.is_ok()); + } + + #[test] + fn test_storage_builder_registry_unsupported_type() { + let registry = StorageBuilderRegistry::new(); + + // Should return error for unsupported type + let result = registry.get_builder("unsupported"); + assert!(result.is_err()); + } + + #[test] + #[cfg(feature = "storage-memory")] + fn test_storage_builder_registry_clone() { + let registry = StorageBuilderRegistry::new(); + let cloned = registry.clone(); + + // Both should have the same builders + assert_eq!( + registry.supported_types().len(), + cloned.supported_types().len() + ); + } +} diff --git a/crates/iceberg/src/io/storage_fs.rs b/crates/iceberg/src/io/storage_fs.rs index d3e121a085..eb3a59fa7e 100644 --- a/crates/iceberg/src/io/storage_fs.rs +++ b/crates/iceberg/src/io/storage_fs.rs @@ -15,9 +15,18 @@ // specific language governing permissions and limitations // under the License. -use opendal::Operator; +use std::collections::HashMap; +use std::sync::Arc; + +use async_trait::async_trait; +use bytes::Bytes; use opendal::services::FsConfig; +use opendal::Operator; +use crate::io::{ + Extensions, FileMetadata, FileRead, FileWrite, InputFile, OutputFile, Storage, + StorageBuilder, +}; use crate::Result; /// Build new opendal operator from give path. @@ -27,3 +36,105 @@ pub(crate) fn fs_config_build() -> Result { Ok(Operator::from_config(cfg)?.finish()) } + +/// Filesystem storage implementation using OpenDAL +#[derive(Debug, Clone)] +pub struct OpenDALFsStorage; + +impl OpenDALFsStorage { + /// Extract relative path from file:// URLs + fn extract_relative_path<'a>(&self, path: &'a str) -> &'a str { + if let Some(stripped) = path.strip_prefix("file:/") { + stripped + } else { + &path[1..] + } + } +} + +#[async_trait] +impl Storage for OpenDALFsStorage { + async fn exists(&self, path: &str) -> Result { + let relative_path = self.extract_relative_path(path); + let op = fs_config_build()?; + Ok(op.exists(relative_path).await?) + } + + async fn metadata(&self, path: &str) -> Result { + let relative_path = self.extract_relative_path(path); + let op = fs_config_build()?; + let meta = op.stat(relative_path).await?; + Ok(FileMetadata { + size: meta.content_length(), + }) + } + + async fn read(&self, path: &str) -> Result { + let relative_path = self.extract_relative_path(path); + let op = fs_config_build()?; + Ok(op.read(relative_path).await?.to_bytes()) + } + + async fn reader(&self, path: &str) -> Result> { + let relative_path = self.extract_relative_path(path); + let op = fs_config_build()?; + Ok(Box::new(op.reader(relative_path).await?)) + } + + async fn write(&self, path: &str, bs: Bytes) -> Result<()> { + let mut writer = self.writer(path).await?; + writer.write(bs).await?; + writer.close().await + } + + async fn writer(&self, path: &str) -> Result> { + let relative_path = self.extract_relative_path(path); + let op = fs_config_build()?; + Ok(Box::new(op.writer(relative_path).await?)) + } + + async fn delete(&self, path: &str) -> Result<()> { + let relative_path = self.extract_relative_path(path); + let op = fs_config_build()?; + Ok(op.delete(relative_path).await?) + } + + async fn remove_dir_all(&self, path: &str) -> Result<()> { + let relative_path = self.extract_relative_path(path); + let op = fs_config_build()?; + let path = if relative_path.ends_with('/') { + relative_path.to_string() + } else { + format!("{relative_path}/") + }; + Ok(op.remove_all(&path).await?) + } + + fn new_input(&self, path: &str) -> Result { + Ok(InputFile { + storage: Arc::new(self.clone()), + path: path.to_string(), + }) + } + + fn new_output(&self, path: &str) -> Result { + Ok(OutputFile { + storage: Arc::new(self.clone()), + path: path.to_string(), + }) + } +} + +/// Builder for OpenDAL Filesystem storage +#[derive(Debug)] +pub struct OpenDALFsStorageBuilder; + +impl StorageBuilder for OpenDALFsStorageBuilder { + fn build( + &self, + _props: HashMap, + _extensions: Extensions, + ) -> Result> { + Ok(Arc::new(OpenDALFsStorage)) + } +} diff --git a/crates/iceberg/src/io/storage_gcs.rs b/crates/iceberg/src/io/storage_gcs.rs index 8c3d914c86..e0fed11024 100644 --- a/crates/iceberg/src/io/storage_gcs.rs +++ b/crates/iceberg/src/io/storage_gcs.rs @@ -17,12 +17,18 @@ //! Google Cloud Storage properties use std::collections::HashMap; +use std::sync::Arc; -use opendal::Operator; +use async_trait::async_trait; +use bytes::Bytes; use opendal::services::GcsConfig; +use opendal::Operator; use url::Url; -use crate::io::is_truthy; +use crate::io::{ + Extensions, FileMetadata, FileRead, FileWrite, InputFile, OutputFile, Storage, + StorageBuilder, is_truthy, +}; use crate::{Error, ErrorKind, Result}; // Reference: https://github.com/apache/iceberg/blob/main/gcp/src/main/java/org/apache/iceberg/gcp/GCPProperties.java @@ -104,3 +110,110 @@ pub(crate) fn gcs_config_build(cfg: &GcsConfig, path: &str) -> Result cfg.bucket = bucket.to_string(); Ok(Operator::from_config(cfg)?.finish()) } + +/// GCS storage implementation using OpenDAL +#[derive(Debug, Clone)] +pub struct OpenDALGcsStorage { + config: Arc, +} + +impl OpenDALGcsStorage { + /// Creates operator from path. + fn create_operator<'a>(&self, path: &'a str) -> Result<(Operator, &'a str)> { + let operator = gcs_config_build(&self.config, path)?; + let prefix = format!("gs://{}/", operator.info().name()); + + if path.starts_with(&prefix) { + let op = operator.layer(opendal::layers::RetryLayer::new()); + Ok((op, &path[prefix.len()..])) + } else { + Err(Error::new( + ErrorKind::DataInvalid, + format!("Invalid gcs url: {}, should start with {}", path, prefix), + )) + } + } +} + +#[async_trait] +impl Storage for OpenDALGcsStorage { + async fn exists(&self, path: &str) -> Result { + let (op, relative_path) = self.create_operator(path)?; + Ok(op.exists(relative_path).await?) + } + + async fn metadata(&self, path: &str) -> Result { + let (op, relative_path) = self.create_operator(path)?; + let meta = op.stat(relative_path).await?; + Ok(FileMetadata { + size: meta.content_length(), + }) + } + + async fn read(&self, path: &str) -> Result { + let (op, relative_path) = self.create_operator(path)?; + Ok(op.read(relative_path).await?.to_bytes()) + } + + async fn reader(&self, path: &str) -> Result> { + let (op, relative_path) = self.create_operator(path)?; + Ok(Box::new(op.reader(relative_path).await?)) + } + + async fn write(&self, path: &str, bs: Bytes) -> Result<()> { + let mut writer = self.writer(path).await?; + writer.write(bs).await?; + writer.close().await + } + + async fn writer(&self, path: &str) -> Result> { + let (op, relative_path) = self.create_operator(path)?; + Ok(Box::new(op.writer(relative_path).await?)) + } + + async fn delete(&self, path: &str) -> Result<()> { + let (op, relative_path) = self.create_operator(path)?; + Ok(op.delete(relative_path).await?) + } + + async fn remove_dir_all(&self, path: &str) -> Result<()> { + let (op, relative_path) = self.create_operator(path)?; + let path = if relative_path.ends_with('/') { + relative_path.to_string() + } else { + format!("{relative_path}/") + }; + Ok(op.remove_all(&path).await?) + } + + fn new_input(&self, path: &str) -> Result { + Ok(InputFile { + storage: Arc::new(self.clone()), + path: path.to_string(), + }) + } + + fn new_output(&self, path: &str) -> Result { + Ok(OutputFile { + storage: Arc::new(self.clone()), + path: path.to_string(), + }) + } +} + +/// Builder for OpenDAL GCS storage +#[derive(Debug)] +pub struct OpenDALGcsStorageBuilder; + +impl StorageBuilder for OpenDALGcsStorageBuilder { + fn build( + &self, + props: HashMap, + _extensions: Extensions, + ) -> Result> { + let cfg = gcs_config_parse(props)?; + Ok(Arc::new(OpenDALGcsStorage { + config: Arc::new(cfg), + })) + } +} diff --git a/crates/iceberg/src/io/storage_memory.rs b/crates/iceberg/src/io/storage_memory.rs index b8023717b6..2c51d5af37 100644 --- a/crates/iceberg/src/io/storage_memory.rs +++ b/crates/iceberg/src/io/storage_memory.rs @@ -15,11 +15,118 @@ // specific language governing permissions and limitations // under the License. -use opendal::Operator; +use std::collections::HashMap; +use std::sync::Arc; + +use async_trait::async_trait; +use bytes::Bytes; use opendal::services::MemoryConfig; +use opendal::Operator; +use crate::io::{ + Extensions, FileMetadata, FileRead, FileWrite, InputFile, OutputFile, Storage, + StorageBuilder, +}; use crate::Result; pub(crate) fn memory_config_build() -> Result { Ok(Operator::from_config(MemoryConfig::default())?.finish()) } + +/// Memory storage implementation using OpenDAL +#[derive(Debug, Clone)] +pub struct OpenDALMemoryStorage { + op: Operator, +} + +impl OpenDALMemoryStorage { + /// Extract relative path from memory:// URLs + fn extract_relative_path<'a>(&self, path: &'a str) -> &'a str { + if let Some(stripped) = path.strip_prefix("memory:/") { + stripped + } else { + &path[1..] + } + } +} + +#[async_trait] +impl Storage for OpenDALMemoryStorage { + async fn exists(&self, path: &str) -> Result { + let relative_path = self.extract_relative_path(path); + Ok(self.op.exists(relative_path).await?) + } + + async fn metadata(&self, path: &str) -> Result { + let relative_path = self.extract_relative_path(path); + let meta = self.op.stat(relative_path).await?; + Ok(FileMetadata { + size: meta.content_length(), + }) + } + + async fn read(&self, path: &str) -> Result { + let relative_path = self.extract_relative_path(path); + Ok(self.op.read(relative_path).await?.to_bytes()) + } + + async fn reader(&self, path: &str) -> Result> { + let relative_path = self.extract_relative_path(path); + Ok(Box::new(self.op.reader(relative_path).await?)) + } + + async fn write(&self, path: &str, bs: Bytes) -> Result<()> { + let mut writer = self.writer(path).await?; + writer.write(bs).await?; + writer.close().await + } + + async fn writer(&self, path: &str) -> Result> { + let relative_path = self.extract_relative_path(path); + Ok(Box::new(self.op.writer(relative_path).await?)) + } + + async fn delete(&self, path: &str) -> Result<()> { + let relative_path = self.extract_relative_path(path); + Ok(self.op.delete(relative_path).await?) + } + + async fn remove_dir_all(&self, path: &str) -> Result<()> { + let relative_path = self.extract_relative_path(path); + let path = if relative_path.ends_with('/') { + relative_path.to_string() + } else { + format!("{relative_path}/") + }; + Ok(self.op.remove_all(&path).await?) + } + + fn new_input(&self, path: &str) -> Result { + Ok(InputFile { + storage: Arc::new(self.clone()), + path: path.to_string(), + }) + } + + fn new_output(&self, path: &str) -> Result { + Ok(OutputFile { + storage: Arc::new(self.clone()), + path: path.to_string(), + }) + } +} + +/// Builder for OpenDAL Memory storage +#[derive(Debug)] +pub struct OpenDALMemoryStorageBuilder; + +impl StorageBuilder for OpenDALMemoryStorageBuilder { + fn build( + &self, + _props: HashMap, + _extensions: Extensions, + ) -> Result> { + let op = Operator::from_config(MemoryConfig::default())?.finish(); + Ok(Arc::new(OpenDALMemoryStorage { op })) + } +} diff --git a/crates/iceberg/src/io/storage_oss.rs b/crates/iceberg/src/io/storage_oss.rs index 8bfffc6ca8..43c5cbb5c8 100644 --- a/crates/iceberg/src/io/storage_oss.rs +++ b/crates/iceberg/src/io/storage_oss.rs @@ -16,11 +16,18 @@ // under the License. use std::collections::HashMap; +use std::sync::Arc; +use async_trait::async_trait; +use bytes::Bytes; use opendal::services::OssConfig; use opendal::{Configurator, Operator}; use url::Url; +use crate::io::{ + Extensions, FileMetadata, FileRead, FileWrite, InputFile, OutputFile, Storage, + StorageBuilder, +}; use crate::{Error, ErrorKind, Result}; /// Required configuration arguments for creating an Aliyun OSS Operator with OpenDAL: @@ -64,3 +71,110 @@ pub(crate) fn oss_config_build(cfg: &OssConfig, path: &str) -> Result Ok(Operator::new(builder)?.finish()) } + +/// OSS storage implementation using OpenDAL +#[derive(Debug, Clone)] +pub struct OpenDALOssStorage { + config: Arc, +} + +impl OpenDALOssStorage { + /// Creates operator from path. + fn create_operator<'a>(&self, path: &'a str) -> Result<(Operator, &'a str)> { + let op = oss_config_build(&self.config, path)?; + let prefix = format!("oss://{}/", op.info().name()); + + if path.starts_with(&prefix) { + let op = op.layer(opendal::layers::RetryLayer::new()); + Ok((op, &path[prefix.len()..])) + } else { + Err(Error::new( + ErrorKind::DataInvalid, + format!("Invalid oss url: {}, should start with {}", path, prefix), + )) + } + } +} + +#[async_trait] +impl Storage for OpenDALOssStorage { + async fn exists(&self, path: &str) -> Result { + let (op, relative_path) = self.create_operator(path)?; + Ok(op.exists(relative_path).await?) + } + + async fn metadata(&self, path: &str) -> Result { + let (op, relative_path) = self.create_operator(path)?; + let meta = op.stat(relative_path).await?; + Ok(FileMetadata { + size: meta.content_length(), + }) + } + + async fn read(&self, path: &str) -> Result { + let (op, relative_path) = self.create_operator(path)?; + Ok(op.read(relative_path).await?.to_bytes()) + } + + async fn reader(&self, path: &str) -> Result> { + let (op, relative_path) = self.create_operator(path)?; + Ok(Box::new(op.reader(relative_path).await?)) + } + + async fn write(&self, path: &str, bs: Bytes) -> Result<()> { + let mut writer = self.writer(path).await?; + writer.write(bs).await?; + writer.close().await + } + + async fn writer(&self, path: &str) -> Result> { + let (op, relative_path) = self.create_operator(path)?; + Ok(Box::new(op.writer(relative_path).await?)) + } + + async fn delete(&self, path: &str) -> Result<()> { + let (op, relative_path) = self.create_operator(path)?; + Ok(op.delete(relative_path).await?) + } + + async fn remove_dir_all(&self, path: &str) -> Result<()> { + let (op, relative_path) = self.create_operator(path)?; + let path = if relative_path.ends_with('/') { + relative_path.to_string() + } else { + format!("{relative_path}/") + }; + Ok(op.remove_all(&path).await?) + } + + fn new_input(&self, path: &str) -> Result { + Ok(InputFile { + storage: Arc::new(self.clone()), + path: path.to_string(), + }) + } + + fn new_output(&self, path: &str) -> Result { + Ok(OutputFile { + storage: Arc::new(self.clone()), + path: path.to_string(), + }) + } +} + +/// Builder for OpenDAL OSS storage +#[derive(Debug)] +pub struct OpenDALOssStorageBuilder; + +impl StorageBuilder for OpenDALOssStorageBuilder { + fn build( + &self, + props: HashMap, + _extensions: Extensions, + ) -> Result> { + let cfg = oss_config_parse(props)?; + Ok(Arc::new(OpenDALOssStorage { + config: Arc::new(cfg), + })) + } +} diff --git a/crates/iceberg/src/io/storage_s3.rs b/crates/iceberg/src/io/storage_s3.rs index f2408331c5..24d96a2874 100644 --- a/crates/iceberg/src/io/storage_s3.rs +++ b/crates/iceberg/src/io/storage_s3.rs @@ -19,13 +19,17 @@ use std::collections::HashMap; use std::sync::Arc; use async_trait::async_trait; +use bytes::Bytes; use opendal::services::S3Config; use opendal::{Configurator, Operator}; pub use reqsign::{AwsCredential, AwsCredentialLoad}; use reqwest::Client; use url::Url; -use crate::io::is_truthy; +use crate::io::{ + Extensions, FileMetadata, FileRead, FileWrite, InputFile, OutputFile, Storage, StorageBuilder, + is_truthy, +}; use crate::{Error, ErrorKind, Result}; /// Following are arguments for [s3 file io](https://py.iceberg.apache.org/configuration/#s3). @@ -214,3 +218,144 @@ impl AwsCredentialLoad for CustomAwsCredentialLoader { self.0.load_credential(client).await } } + +/// S3 storage implementation using OpenDAL +#[derive(Debug, Clone)] +pub struct OpenDALS3Storage { + /// s3 storage could have `s3://` and `s3a://`. + /// Storing the scheme string here to return the correct path. + configured_scheme: String, + config: Arc, + customized_credential_load: Option, +} + +impl OpenDALS3Storage { + /// Creates operator from path. + /// + /// # Arguments + /// + /// * path: It should be *absolute* path starting with scheme string used to construct [`FileIO`]. + /// + /// # Returns + /// + /// The return value consists of two parts: + /// + /// * An [`opendal::Operator`] instance used to operate on file. + /// * Relative path to the root uri of [`opendal::Operator`]. + fn create_operator<'a>(&self, path: &'a str) -> Result<(Operator, &'a str)> { + let op = s3_config_build(&self.config, &self.customized_credential_load, path)?; + let op_info = op.info(); + + // Check prefix of s3 path. + let prefix = format!("{}://{}/", self.configured_scheme, op_info.name()); + if path.starts_with(&prefix) { + // Add retry layer for transient errors + let op = op.layer(opendal::layers::RetryLayer::new()); + Ok((op, &path[prefix.len()..])) + } else { + Err(Error::new( + ErrorKind::DataInvalid, + format!("Invalid s3 url: {}, should start with {}", path, prefix), + )) + } + } +} + +#[async_trait] +impl Storage for OpenDALS3Storage { + async fn exists(&self, path: &str) -> Result { + let (op, relative_path) = self.create_operator(path)?; + Ok(op.exists(relative_path).await?) + } + + async fn metadata(&self, path: &str) -> Result { + let (op, relative_path) = self.create_operator(path)?; + let meta = op.stat(relative_path).await?; + + Ok(FileMetadata { + size: meta.content_length(), + }) + } + + async fn read(&self, path: &str) -> Result { + let (op, relative_path) = self.create_operator(path)?; + Ok(op.read(relative_path).await?.to_bytes()) + } + + async fn reader(&self, path: &str) -> Result> { + let (op, relative_path) = self.create_operator(path)?; + Ok(Box::new(op.reader(relative_path).await?)) + } + + async fn write(&self, path: &str, bs: Bytes) -> Result<()> { + let mut writer = self.writer(path).await?; + writer.write(bs).await?; + writer.close().await + } + + async fn writer(&self, path: &str) -> Result> { + let (op, relative_path) = self.create_operator(path)?; + Ok(Box::new(op.writer(relative_path).await?)) + } + + async fn delete(&self, path: &str) -> Result<()> { + let (op, relative_path) = self.create_operator(path)?; + Ok(op.delete(relative_path).await?) + } + + async fn remove_dir_all(&self, path: &str) -> Result<()> { + let (op, relative_path) = self.create_operator(path)?; + let path = if relative_path.ends_with('/') { + relative_path.to_string() + } else { + format!("{relative_path}/") + }; + Ok(op.remove_all(&path).await?) + } + + fn new_input(&self, path: &str) -> Result { + Ok(InputFile { + storage: Arc::new(self.clone()), + path: path.to_string(), + }) + } + + fn new_output(&self, path: &str) -> Result { + Ok(OutputFile { + storage: Arc::new(self.clone()), + path: path.to_string(), + }) + } +} + +/// Builder for S3 storage +#[derive(Debug)] +pub struct OpenDALS3StorageBuilder; + +impl StorageBuilder for OpenDALS3StorageBuilder { + fn build( + &self, + props: HashMap, + extensions: Extensions, + ) -> Result> { + // Get the scheme string from the props or use "s3" as default + let scheme_str = props + .get("scheme_str") + .cloned() + .unwrap_or_else(|| "s3".to_string()); + + // Parse S3 config from props + let config = s3_config_parse(props)?; + + // Get customized credential loader from extensions if available + let customized_credential_load = extensions + .get::() + .map(Arc::unwrap_or_clone); + + Ok(Arc::new(OpenDALS3Storage { + configured_scheme: scheme_str, + config: Arc::new(config), + customized_credential_load, + })) + } +} From e5516b8db8db4a694de672eb60a1f98640daba20 Mon Sep 17 00:00:00 2001 From: Shawn Chang Date: Thu, 16 Oct 2025 19:24:07 -0700 Subject: [PATCH 08/47] Cleanup OpenDALStorage enum --- crates/iceberg/src/io/file_io.rs | 55 +-- crates/iceberg/src/io/mod.rs | 7 +- crates/iceberg/src/io/storage.rs | 517 ++++++++++++----------- crates/iceberg/src/io/storage_azdls.rs | 6 +- crates/iceberg/src/io/storage_builder.rs | 210 --------- crates/iceberg/src/io/storage_fs.rs | 7 +- crates/iceberg/src/io/storage_gcs.rs | 8 +- crates/iceberg/src/io/storage_memory.rs | 7 +- crates/iceberg/src/io/storage_oss.rs | 5 +- 9 files changed, 285 insertions(+), 537 deletions(-) delete mode 100644 crates/iceberg/src/io/storage_builder.rs diff --git a/crates/iceberg/src/io/file_io.rs b/crates/iceberg/src/io/file_io.rs index 4ce81d8f04..d64b45db3f 100644 --- a/crates/iceberg/src/io/file_io.rs +++ b/crates/iceberg/src/io/file_io.rs @@ -25,52 +25,11 @@ use async_trait::async_trait; use bytes::Bytes; use url::Url; +// Re-export traits from storage module +pub use super::storage::{Storage, StorageBuilder}; +use crate::io::StorageBuilderRegistry; use crate::{Error, ErrorKind, Result}; -/// Trait for storage operations in Iceberg -#[async_trait] -pub trait Storage: Debug + Send + Sync { - /// Check if a file exists at the given path - async fn exists(&self, path: &str) -> Result; - - /// Get metadata from an input path - async fn metadata(&self, path: &str) -> Result; - - /// Read bytes from a path - async fn read(&self, path: &str) -> Result; - - /// Get FileRead from a path - async fn reader(&self, path: &str) -> Result>; - - /// Write bytes to an output path - async fn write(&self, path: &str, bs: Bytes) -> Result<()>; - - /// Get FileWrite from a path - async fn writer(&self, path: &str) -> Result>; - - /// Delete a file at the given path - async fn delete(&self, path: &str) -> Result<()>; - - /// Remove a directory and all its contents recursively - async fn remove_dir_all(&self, path: &str) -> Result<()>; - - /// Create a new input file for reading - fn new_input(&self, path: &str) -> Result; - - /// Create a new output file for writing - fn new_output(&self, path: &str) -> Result; -} - -/// Common interface for all storage builders. -pub trait StorageBuilder: Debug + Send + Sync { - /// Create a new storage instance with the given properties and extensions. - fn build( - &self, - props: HashMap, - extensions: Extensions, - ) -> Result>; -} - /// FileIO implementation, used to manipulate files in underlying storage. /// /// # Note @@ -133,8 +92,6 @@ impl FileIO { /// /// * path: It should be *absolute* path starting with scheme string used to construct [`FileIO`]. pub async fn delete(&self, path: impl AsRef) -> Result<()> { - // let (op, relative_path) = self.inner.create_operator(&path)?; - // Ok(op.delete(relative_path).await?) self.inner.delete(path.as_ref()).await } @@ -299,11 +256,11 @@ impl FileIOBuilder { pub fn build(self) -> Result { // Use the scheme to determine the storage type let scheme = self.scheme_str.clone().unwrap_or_default(); - + // Create registry and get builder - let registry = crate::io::StorageBuilderRegistry::new(); + let registry = StorageBuilderRegistry::new(); let builder = registry.get_builder(scheme.as_str())?; - + // Build storage with props and extensions let storage = builder.build(self.props.clone(), self.extensions.clone())?; diff --git a/crates/iceberg/src/io/mod.rs b/crates/iceberg/src/io/mod.rs index 5f234961fc..2397e2561f 100644 --- a/crates/iceberg/src/io/mod.rs +++ b/crates/iceberg/src/io/mod.rs @@ -68,10 +68,9 @@ mod file_io; mod storage; -mod storage_builder; pub use file_io::*; -pub use storage_builder::StorageBuilderRegistry; +pub use storage::{Storage, StorageBuilder, StorageBuilderRegistry}; pub(crate) mod object_cache; #[cfg(feature = "storage-azdls")] @@ -89,12 +88,8 @@ mod storage_s3; #[cfg(feature = "storage-azdls")] pub use storage_azdls::*; -#[cfg(feature = "storage-fs")] -use storage_fs::*; #[cfg(feature = "storage-gcs")] pub use storage_gcs::*; -#[cfg(feature = "storage-memory")] -use storage_memory::*; #[cfg(feature = "storage-oss")] pub use storage_oss::*; #[cfg(feature = "storage-s3")] diff --git a/crates/iceberg/src/io/storage.rs b/crates/iceberg/src/io/storage.rs index 62c6b8a31e..03f5b77155 100644 --- a/crates/iceberg/src/io/storage.rs +++ b/crates/iceberg/src/io/storage.rs @@ -15,287 +15,296 @@ // specific language governing permissions and limitations // under the License. +//! Storage traits and implementations for Iceberg. +//! +//! This module provides the core storage abstraction used throughout Iceberg Rust. +//! Storage implementations handle reading and writing files across different backends +//! (S3, GCS, Azure, local filesystem, etc.). + +use std::collections::HashMap; +use std::fmt::Debug; use std::sync::Arc; use async_trait::async_trait; use bytes::Bytes; -use opendal::layers::RetryLayer; -#[cfg(feature = "storage-azdls")] -use opendal::services::AzdlsConfig; -#[cfg(feature = "storage-gcs")] -use opendal::services::GcsConfig; -#[cfg(feature = "storage-oss")] -use opendal::services::OssConfig; -#[cfg(feature = "storage-s3")] -use opendal::services::S3Config; -use opendal::{Operator, Scheme}; - -#[cfg(feature = "storage-azdls")] -use super::AzureStorageScheme; -use super::{FileIOBuilder, FileMetadata, FileRead, FileWrite, InputFile, OutputFile, Storage}; -#[cfg(feature = "storage-s3")] -use crate::io::CustomAwsCredentialLoader; -use crate::{Error, ErrorKind, Result}; - -/// The storage carries all supported storage services in iceberg + +use super::{Extensions, FileMetadata, FileRead, FileWrite, InputFile, OutputFile}; +use crate::Result; + +/// Trait for storage operations in Iceberg. +/// +/// This trait defines the interface for all storage backends. Implementations +/// provide access to different storage systems like S3, GCS, Azure, local filesystem, etc. +/// +/// # Example +/// +/// ```rust,ignore +/// use iceberg::io::Storage; +/// +/// async fn example(storage: Arc) -> Result<()> { +/// // Check if file exists +/// if storage.exists("s3://bucket/path/file.parquet").await? { +/// // Read file +/// let data = storage.read("s3://bucket/path/file.parquet").await?; +/// } +/// Ok(()) +/// } +/// ``` +#[async_trait] +pub trait Storage: Debug + Send + Sync { + /// Check if a file exists at the given path + async fn exists(&self, path: &str) -> Result; + + /// Get metadata from an input path + async fn metadata(&self, path: &str) -> Result; + + /// Read bytes from a path + async fn read(&self, path: &str) -> Result; + + /// Get FileRead from a path + async fn reader(&self, path: &str) -> Result>; + + /// Write bytes to an output path + async fn write(&self, path: &str, bs: Bytes) -> Result<()>; + + /// Get FileWrite from a path + async fn writer(&self, path: &str) -> Result>; + + /// Delete a file at the given path + async fn delete(&self, path: &str) -> Result<()>; + + /// Remove a directory and all its contents recursively + async fn remove_dir_all(&self, path: &str) -> Result<()>; + + /// Create a new input file for reading + fn new_input(&self, path: &str) -> Result; + + /// Create a new output file for writing + fn new_output(&self, path: &str) -> Result; +} + +/// Common interface for all storage builders. +/// +/// Storage builders are responsible for creating storage instances from configuration +/// properties and extensions. Each storage backend (S3, GCS, etc.) provides its own +/// builder implementation. +/// +/// # Example +/// +/// ```rust,ignore +/// use iceberg::io::{StorageBuilder, Extensions}; +/// use std::collections::HashMap; +/// +/// struct MyStorageBuilder; +/// +/// impl StorageBuilder for MyStorageBuilder { +/// fn build( +/// &self, +/// props: HashMap, +/// extensions: Extensions, +/// ) -> Result> { +/// // Parse configuration and create storage +/// Ok(Arc::new(MyStorage::new(props)?)) +/// } +/// } +/// ``` +pub trait StorageBuilder: Debug + Send + Sync { + /// Create a new storage instance with the given properties and extensions. + /// + /// # Arguments + /// + /// * `props` - Configuration properties for the storage backend + /// * `extensions` - Additional extensions (e.g., custom credential loaders) + /// + /// # Returns + /// + /// An `Arc` that can be used for file operations. + fn build( + &self, + props: HashMap, + extensions: Extensions, + ) -> Result>; +} + +/// A registry of storage builders. +/// +/// The registry allows you to register custom storage builders for different URI schemes. +/// By default, it includes builders for all enabled storage features. +/// +/// # Example +/// +/// ```rust +/// use iceberg::io::StorageBuilderRegistry; +/// +/// // Create a new registry with default builders +/// let registry = StorageBuilderRegistry::new(); +/// +/// // Get supported storage types +/// let types = registry.supported_types(); +/// println!("Supported storage types: {:?}", types); +/// +/// // Get a builder for a specific scheme +/// # #[cfg(feature = "storage-memory")] +/// # { +/// let builder = registry.get_builder("memory").unwrap(); +/// # } +/// ``` +/// +/// You can also register custom storage builders: +/// +/// ```rust,ignore +/// use std::sync::Arc; +/// use iceberg::io::{StorageBuilderRegistry, StorageBuilder}; +/// +/// let mut registry = StorageBuilderRegistry::new(); +/// +/// // Register a custom storage builder +/// registry.register("custom", Arc::new(MyCustomStorageBuilder)); +/// ``` #[derive(Debug, Clone)] -pub(crate) enum OpenDALStorage { - #[cfg(feature = "storage-memory")] - Memory(Operator), - #[cfg(feature = "storage-fs")] - LocalFs, - /// Expects paths of the form `s3[a]:///`. - #[cfg(feature = "storage-s3")] - S3 { - /// s3 storage could have `s3://` and `s3a://`. - /// Storing the scheme string here to return the correct path. - configured_scheme: String, - config: Arc, - customized_credential_load: Option, - }, - #[cfg(feature = "storage-gcs")] - Gcs { config: Arc }, - #[cfg(feature = "storage-oss")] - Oss { config: Arc }, - /// Expects paths of the form - /// `abfs[s]://@.dfs./` or - /// `wasb[s]://@.blob./`. - #[cfg(feature = "storage-azdls")] - Azdls { - /// Because Azdls accepts multiple possible schemes, we store the full - /// passed scheme here to later validate schemes passed via paths. - configured_scheme: AzureStorageScheme, - config: Arc, - }, +pub struct StorageBuilderRegistry { + builders: HashMap>, } -#[async_trait] -impl Storage for OpenDALStorage { - async fn exists(&self, path: &str) -> Result { - let (op, relative_path) = self.create_operator(&path)?; - Ok(op.exists(relative_path).await?) - } +impl StorageBuilderRegistry { + /// Create a new storage registry with default builders based on enabled features. + pub fn new() -> Self { + let mut builders: HashMap> = HashMap::new(); - async fn metadata(&self, path: &str) -> Result { - let (op, relative_path) = self.create_operator(&path)?; - let meta = op.stat(relative_path).await?; + #[cfg(feature = "storage-memory")] + { + use crate::io::storage_memory::OpenDALMemoryStorageBuilder; + let builder = Arc::new(OpenDALMemoryStorageBuilder) as Arc; + builders.insert("memory".to_string(), builder); + } - Ok(FileMetadata { - size: meta.content_length(), - }) - } + #[cfg(feature = "storage-fs")] + { + use crate::io::storage_fs::OpenDALFsStorageBuilder; + let builder = Arc::new(OpenDALFsStorageBuilder) as Arc; + builders.insert("file".to_string(), builder.clone()); + builders.insert("".to_string(), builder); + } - async fn read(&self, path: &str) -> Result { - let (op, relative_path) = self.create_operator(&path)?; - Ok(op.read(relative_path).await?.to_bytes()) - } + #[cfg(feature = "storage-s3")] + { + use crate::io::storage_s3::OpenDALS3StorageBuilder; + let builder = Arc::new(OpenDALS3StorageBuilder) as Arc; + builders.insert("s3".to_string(), builder.clone()); + builders.insert("s3a".to_string(), builder); + } - async fn reader(&self, path: &str) -> Result> { - let (op, relative_path) = self.create_operator(&path)?; - Ok(Box::new(op.reader(relative_path).await?)) - } + #[cfg(feature = "storage-gcs")] + { + use crate::io::storage_gcs::OpenDALGcsStorageBuilder; + let builder = Arc::new(OpenDALGcsStorageBuilder) as Arc; + builders.insert("gs".to_string(), builder.clone()); + builders.insert("gcs".to_string(), builder); + } - async fn write(&self, path: &str, bs: Bytes) -> Result<()> { - let mut writer = self.writer(path).await?; - writer.write(bs).await?; - writer.close().await - } + #[cfg(feature = "storage-oss")] + { + use crate::io::storage_oss::OpenDALOssStorageBuilder; + let builder = Arc::new(OpenDALOssStorageBuilder) as Arc; + builders.insert("oss".to_string(), builder); + } + + #[cfg(feature = "storage-azdls")] + { + use crate::io::storage_azdls::OpenDALAzdlsStorageBuilder; + let builder = Arc::new(OpenDALAzdlsStorageBuilder) as Arc; + builders.insert("abfs".to_string(), builder.clone()); + builders.insert("abfss".to_string(), builder.clone()); + builders.insert("wasb".to_string(), builder.clone()); + builders.insert("wasbs".to_string(), builder); + } - async fn writer(&self, path: &str) -> Result> { - let (op, relative_path) = self.create_operator(&path)?; - Ok(Box::new(op.writer(relative_path).await?)) + Self { builders } } - async fn delete(&self, path: &str) -> Result<()> { - let (op, relative_path) = self.create_operator(&path)?; - Ok(op.delete(relative_path).await?) + /// Register a custom storage builder for a given scheme. + pub fn register(&mut self, scheme: impl Into, builder: Arc) { + self.builders.insert(scheme.into(), builder); } - async fn remove_dir_all(&self, path: &str) -> Result<()> { - let (op, relative_path) = self.create_operator(&path)?; - let path = if relative_path.ends_with('/') { - relative_path.to_string() - } else { - format!("{relative_path}/") - }; - Ok(op.remove_all(&path).await?) + /// Get a storage builder by scheme. + pub fn get_builder(&self, scheme: &str) -> Result> { + let key = scheme.trim(); + self.builders + .iter() + .find(|(k, _)| k.eq_ignore_ascii_case(key)) + .map(|(_, builder)| builder.clone()) + .ok_or_else(|| { + use crate::{Error, ErrorKind}; + Error::new( + ErrorKind::FeatureUnsupported, + format!( + "Unsupported storage type: {}. Supported types: {}", + scheme, + self.supported_types().join(", ") + ), + ) + }) } - fn new_input(&self, path: &str) -> Result { - let storage = Arc::new(self.clone()); - let path = path.to_string(); - Ok(InputFile { storage, path }) + /// Return the list of supported storage types. + pub fn supported_types(&self) -> Vec { + self.builders.keys().cloned().collect() } +} - fn new_output(&self, path: &str) -> Result { - let storage = Arc::new(self.clone()); - let path = path.to_string(); - Ok(OutputFile { storage, path }) +impl Default for StorageBuilderRegistry { + fn default() -> Self { + Self::new() } } -impl OpenDALStorage { - /// Convert iceberg config to opendal config. - pub(crate) fn build(file_io_builder: FileIOBuilder) -> Result { - let (scheme_str, props, extensions) = file_io_builder.into_parts(); - let scheme = Self::parse_scheme(&scheme_str)?; - - match scheme { - #[cfg(feature = "storage-memory")] - Scheme::Memory => Ok(Self::Memory(super::memory_config_build()?)), - #[cfg(feature = "storage-fs")] - Scheme::Fs => Ok(Self::LocalFs), - #[cfg(feature = "storage-s3")] - Scheme::S3 => Ok(Self::S3 { - configured_scheme: scheme_str, - config: super::s3_config_parse(props)?.into(), - customized_credential_load: extensions - .get::() - .map(Arc::unwrap_or_clone), - }), - #[cfg(feature = "storage-gcs")] - Scheme::Gcs => Ok(Self::Gcs { - config: super::gcs_config_parse(props)?.into(), - }), - #[cfg(feature = "storage-oss")] - Scheme::Oss => Ok(Self::Oss { - config: super::oss_config_parse(props)?.into(), - }), - #[cfg(feature = "storage-azdls")] - Scheme::Azdls => { - let scheme = scheme_str.parse::()?; - Ok(Self::Azdls { - config: super::azdls_config_parse(props)?.into(), - configured_scheme: scheme, - }) - } - // Update doc on [`FileIO`] when adding new schemes. - _ => Err(Error::new( - ErrorKind::FeatureUnsupported, - format!("Constructing file io from scheme: {scheme} not supported now",), - )), - } +#[cfg(test)] +mod tests { + use super::*; + + #[test] + fn test_storage_builder_registry_new() { + let registry = StorageBuilderRegistry::new(); + let types = registry.supported_types(); + + // At least one storage type should be available + assert!(!types.is_empty()); } - /// Creates operator from path. - /// - /// # Arguments - /// - /// * path: It should be *absolute* path starting with scheme string used to construct [`FileIO`]. - /// - /// # Returns - /// - /// The return value consists of two parts: - /// - /// * An [`opendal::Operator`] instance used to operate on file. - /// * Relative path to the root uri of [`opendal::Operator`]. - pub(crate) fn create_operator<'a>( - &self, - path: &'a impl AsRef, - ) -> crate::Result<(Operator, &'a str)> { - let path = path.as_ref(); - let (operator, relative_path): (Operator, &str) = match self { - #[cfg(feature = "storage-memory")] - OpenDALStorage::Memory(op) => { - if let Some(stripped) = path.strip_prefix("memory:/") { - Ok::<_, crate::Error>((op.clone(), stripped)) - } else { - Ok::<_, crate::Error>((op.clone(), &path[1..])) - } - } - #[cfg(feature = "storage-fs")] - OpenDALStorage::LocalFs => { - let op = super::fs_config_build()?; - - if let Some(stripped) = path.strip_prefix("file:/") { - Ok::<_, crate::Error>((op, stripped)) - } else { - Ok::<_, crate::Error>((op, &path[1..])) - } - } - #[cfg(feature = "storage-s3")] - OpenDALStorage::S3 { - configured_scheme, - config, - customized_credential_load, - } => { - let op = super::s3_config_build(config, customized_credential_load, path)?; - let op_info = op.info(); - - // Check prefix of s3 path. - let prefix = format!("{}://{}/", configured_scheme, op_info.name()); - if path.starts_with(&prefix) { - Ok((op, &path[prefix.len()..])) - } else { - Err(Error::new( - ErrorKind::DataInvalid, - format!("Invalid s3 url: {}, should start with {}", path, prefix), - )) - } - } - #[cfg(feature = "storage-gcs")] - OpenDALStorage::Gcs { config } => { - let operator = super::gcs_config_build(config, path)?; - let prefix = format!("gs://{}/", operator.info().name()); - if path.starts_with(&prefix) { - Ok((operator, &path[prefix.len()..])) - } else { - Err(Error::new( - ErrorKind::DataInvalid, - format!("Invalid gcs url: {}, should start with {}", path, prefix), - )) - } - } - #[cfg(feature = "storage-oss")] - OpenDALStorage::Oss { config } => { - let op = super::oss_config_build(config, path)?; - - // Check prefix of oss path. - let prefix = format!("oss://{}/", op.info().name()); - if path.starts_with(&prefix) { - Ok((op, &path[prefix.len()..])) - } else { - Err(Error::new( - ErrorKind::DataInvalid, - format!("Invalid oss url: {}, should start with {}", path, prefix), - )) - } - } - #[cfg(feature = "storage-azdls")] - OpenDALStorage::Azdls { - configured_scheme, - config, - } => super::azdls_create_operator(path, config, configured_scheme), - #[cfg(all( - not(feature = "storage-s3"), - not(feature = "storage-fs"), - not(feature = "storage-gcs"), - not(feature = "storage-oss"), - not(feature = "storage-azdls"), - ))] - _ => Err(Error::new( - ErrorKind::FeatureUnsupported, - "No storage service has been enabled", - )), - }?; - - // Transient errors are common for object stores; however there's no - // harm in retrying temporary failures for other storage backends as well. - let operator = operator.layer(RetryLayer::new()); - - Ok((operator, relative_path)) + #[test] + #[cfg(feature = "storage-memory")] + fn test_storage_builder_registry_get_builder() { + let registry = StorageBuilderRegistry::new(); + + // Should be able to get memory storage builder + let builder = registry.get_builder("memory"); + assert!(builder.is_ok()); + + // Should be case-insensitive + let builder = registry.get_builder("MEMORY"); + assert!(builder.is_ok()); } - /// Parse scheme. - fn parse_scheme(scheme: &str) -> crate::Result { - match scheme { - "memory" => Ok(Scheme::Memory), - "file" | "" => Ok(Scheme::Fs), - "s3" | "s3a" => Ok(Scheme::S3), - "gs" | "gcs" => Ok(Scheme::Gcs), - "oss" => Ok(Scheme::Oss), - "abfss" | "abfs" | "wasbs" | "wasb" => Ok(Scheme::Azdls), - s => Ok(s.parse::()?), - } + #[test] + fn test_storage_builder_registry_unsupported_type() { + let registry = StorageBuilderRegistry::new(); + + // Should return error for unsupported type + let result = registry.get_builder("unsupported"); + assert!(result.is_err()); + } + + #[test] + #[cfg(feature = "storage-memory")] + fn test_storage_builder_registry_clone() { + let registry = StorageBuilderRegistry::new(); + let cloned = registry.clone(); + + // Both should have the same builders + assert_eq!( + registry.supported_types().len(), + cloned.supported_types().len() + ); } } diff --git a/crates/iceberg/src/io/storage_azdls.rs b/crates/iceberg/src/io/storage_azdls.rs index 2afccd5a01..ff9b5f6a5a 100644 --- a/crates/iceberg/src/io/storage_azdls.rs +++ b/crates/iceberg/src/io/storage_azdls.rs @@ -27,8 +27,7 @@ use opendal::{Configurator, Operator}; use url::Url; use crate::io::{ - Extensions, FileMetadata, FileRead, FileWrite, InputFile, OutputFile, Storage, - StorageBuilder, + Extensions, FileMetadata, FileRead, FileWrite, InputFile, OutputFile, Storage, StorageBuilder, }; use crate::{Error, ErrorKind, Result, ensure_data_valid}; @@ -617,7 +616,8 @@ pub struct OpenDALAzdlsStorage { impl OpenDALAzdlsStorage { /// Creates operator from path. fn create_operator<'a>(&self, path: &'a str) -> Result<(Operator, &'a str)> { - let (op, relative_path) = azdls_create_operator(path, &self.config, &self.configured_scheme)?; + let (op, relative_path) = + azdls_create_operator(path, &self.config, &self.configured_scheme)?; let op = op.layer(opendal::layers::RetryLayer::new()); Ok((op, relative_path)) } diff --git a/crates/iceberg/src/io/storage_builder.rs b/crates/iceberg/src/io/storage_builder.rs deleted file mode 100644 index aacd68bb00..0000000000 --- a/crates/iceberg/src/io/storage_builder.rs +++ /dev/null @@ -1,210 +0,0 @@ -// Licensed to the Apache Software Foundation (ASF) under one -// or more contributor license agreements. See the NOTICE file -// distributed with this work for additional information -// regarding copyright ownership. The ASF licenses this file -// to you under the Apache License, Version 2.0 (the -// "License"); you may not use this file except in compliance -// with the License. You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, -// software distributed under the License is distributed on an -// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -// KIND, either express or implied. See the License for the -// specific language governing permissions and limitations -// under the License. - -use std::collections::HashMap; -use std::sync::Arc; - -use crate::io::StorageBuilder; -#[cfg(feature = "storage-azdls")] -use crate::io::storage_azdls::OpenDALAzdlsStorageBuilder; -#[cfg(feature = "storage-fs")] -use crate::io::storage_fs::OpenDALFsStorageBuilder; -#[cfg(feature = "storage-gcs")] -use crate::io::storage_gcs::OpenDALGcsStorageBuilder; -#[cfg(feature = "storage-memory")] -use crate::io::storage_memory::OpenDALMemoryStorageBuilder; -#[cfg(feature = "storage-oss")] -use crate::io::storage_oss::OpenDALOssStorageBuilder; -#[cfg(feature = "storage-s3")] -use crate::io::storage_s3::OpenDALS3StorageBuilder; -use crate::{Error, ErrorKind, Result}; - -/// A registry of storage builders. -/// -/// The registry allows you to register custom storage builders for different URI schemes. -/// By default, it includes builders for all enabled storage features. -/// -/// # Example -/// -/// ```rust -/// use iceberg::io::StorageBuilderRegistry; -/// -/// // Create a new registry with default builders -/// let registry = StorageBuilderRegistry::new(); -/// -/// // Get supported storage types -/// let types = registry.supported_types(); -/// println!("Supported storage types: {:?}", types); -/// -/// // Get a builder for a specific scheme -/// # #[cfg(feature = "storage-memory")] -/// # { -/// let builder = registry.get_builder("memory").unwrap(); -/// # } -/// ``` -/// -/// You can also register custom storage builders: -/// -/// ```rust,ignore -/// use std::sync::Arc; -/// use iceberg::io::{StorageBuilderRegistry, StorageBuilder}; -/// -/// let mut registry = StorageBuilderRegistry::new(); -/// -/// // Register a custom storage builder -/// registry.register("custom", Arc::new(MyCustomStorageBuilder)); -/// ``` -#[derive(Debug, Clone)] -pub struct StorageBuilderRegistry { - builders: HashMap>, -} - -impl StorageBuilderRegistry { - /// Create a new storage registry with default builders based on enabled features. - pub fn new() -> Self { - let mut builders: HashMap> = HashMap::new(); - - #[cfg(feature = "storage-memory")] - { - let builder = Arc::new(OpenDALMemoryStorageBuilder) as Arc; - builders.insert("memory".to_string(), builder); - } - - #[cfg(feature = "storage-fs")] - { - let builder = Arc::new(OpenDALFsStorageBuilder) as Arc; - builders.insert("file".to_string(), builder.clone()); - builders.insert("".to_string(), builder); - } - - #[cfg(feature = "storage-s3")] - { - let builder = Arc::new(OpenDALS3StorageBuilder) as Arc; - builders.insert("s3".to_string(), builder.clone()); - builders.insert("s3a".to_string(), builder); - } - - #[cfg(feature = "storage-gcs")] - { - let builder = Arc::new(OpenDALGcsStorageBuilder) as Arc; - builders.insert("gs".to_string(), builder.clone()); - builders.insert("gcs".to_string(), builder); - } - - #[cfg(feature = "storage-oss")] - { - let builder = Arc::new(OpenDALOssStorageBuilder) as Arc; - builders.insert("oss".to_string(), builder); - } - - #[cfg(feature = "storage-azdls")] - { - let builder = Arc::new(OpenDALAzdlsStorageBuilder) as Arc; - builders.insert("abfs".to_string(), builder.clone()); - builders.insert("abfss".to_string(), builder.clone()); - builders.insert("wasb".to_string(), builder.clone()); - builders.insert("wasbs".to_string(), builder); - } - - Self { builders } - } - - /// Register a custom storage builder for a given scheme. - pub fn register(&mut self, scheme: impl Into, builder: Arc) { - self.builders.insert(scheme.into(), builder); - } - - /// Get a storage builder by scheme. - pub fn get_builder(&self, scheme: &str) -> Result> { - let key = scheme.trim(); - self.builders - .iter() - .find(|(k, _)| k.eq_ignore_ascii_case(key)) - .map(|(_, builder)| builder.clone()) - .ok_or_else(|| { - Error::new( - ErrorKind::FeatureUnsupported, - format!( - "Unsupported storage type: {}. Supported types: {}", - scheme, - self.supported_types().join(", ") - ), - ) - }) - } - - /// Return the list of supported storage types. - pub fn supported_types(&self) -> Vec { - self.builders.keys().cloned().collect() - } -} - -impl Default for StorageBuilderRegistry { - fn default() -> Self { - Self::new() - } -} - -#[cfg(test)] -mod tests { - use super::*; - - #[test] - fn test_storage_builder_registry_new() { - let registry = StorageBuilderRegistry::new(); - let types = registry.supported_types(); - - // At least one storage type should be available - assert!(!types.is_empty()); - } - - #[test] - #[cfg(feature = "storage-memory")] - fn test_storage_builder_registry_get_builder() { - let registry = StorageBuilderRegistry::new(); - - // Should be able to get memory storage builder - let builder = registry.get_builder("memory"); - assert!(builder.is_ok()); - - // Should be case-insensitive - let builder = registry.get_builder("MEMORY"); - assert!(builder.is_ok()); - } - - #[test] - fn test_storage_builder_registry_unsupported_type() { - let registry = StorageBuilderRegistry::new(); - - // Should return error for unsupported type - let result = registry.get_builder("unsupported"); - assert!(result.is_err()); - } - - #[test] - #[cfg(feature = "storage-memory")] - fn test_storage_builder_registry_clone() { - let registry = StorageBuilderRegistry::new(); - let cloned = registry.clone(); - - // Both should have the same builders - assert_eq!( - registry.supported_types().len(), - cloned.supported_types().len() - ); - } -} diff --git a/crates/iceberg/src/io/storage_fs.rs b/crates/iceberg/src/io/storage_fs.rs index eb3a59fa7e..791ab3e9fc 100644 --- a/crates/iceberg/src/io/storage_fs.rs +++ b/crates/iceberg/src/io/storage_fs.rs @@ -20,14 +20,13 @@ use std::sync::Arc; use async_trait::async_trait; use bytes::Bytes; -use opendal::services::FsConfig; use opendal::Operator; +use opendal::services::FsConfig; +use crate::Result; use crate::io::{ - Extensions, FileMetadata, FileRead, FileWrite, InputFile, OutputFile, Storage, - StorageBuilder, + Extensions, FileMetadata, FileRead, FileWrite, InputFile, OutputFile, Storage, StorageBuilder, }; -use crate::Result; /// Build new opendal operator from give path. pub(crate) fn fs_config_build() -> Result { diff --git a/crates/iceberg/src/io/storage_gcs.rs b/crates/iceberg/src/io/storage_gcs.rs index e0fed11024..65cff80de3 100644 --- a/crates/iceberg/src/io/storage_gcs.rs +++ b/crates/iceberg/src/io/storage_gcs.rs @@ -21,13 +21,13 @@ use std::sync::Arc; use async_trait::async_trait; use bytes::Bytes; -use opendal::services::GcsConfig; use opendal::Operator; +use opendal::services::GcsConfig; use url::Url; use crate::io::{ - Extensions, FileMetadata, FileRead, FileWrite, InputFile, OutputFile, Storage, - StorageBuilder, is_truthy, + Extensions, FileMetadata, FileRead, FileWrite, InputFile, OutputFile, Storage, StorageBuilder, + is_truthy, }; use crate::{Error, ErrorKind, Result}; @@ -122,7 +122,7 @@ impl OpenDALGcsStorage { fn create_operator<'a>(&self, path: &'a str) -> Result<(Operator, &'a str)> { let operator = gcs_config_build(&self.config, path)?; let prefix = format!("gs://{}/", operator.info().name()); - + if path.starts_with(&prefix) { let op = operator.layer(opendal::layers::RetryLayer::new()); Ok((op, &path[prefix.len()..])) diff --git a/crates/iceberg/src/io/storage_memory.rs b/crates/iceberg/src/io/storage_memory.rs index 2c51d5af37..c406929109 100644 --- a/crates/iceberg/src/io/storage_memory.rs +++ b/crates/iceberg/src/io/storage_memory.rs @@ -20,14 +20,13 @@ use std::sync::Arc; use async_trait::async_trait; use bytes::Bytes; -use opendal::services::MemoryConfig; use opendal::Operator; +use opendal::services::MemoryConfig; +use crate::Result; use crate::io::{ - Extensions, FileMetadata, FileRead, FileWrite, InputFile, OutputFile, Storage, - StorageBuilder, + Extensions, FileMetadata, FileRead, FileWrite, InputFile, OutputFile, Storage, StorageBuilder, }; -use crate::Result; pub(crate) fn memory_config_build() -> Result { Ok(Operator::from_config(MemoryConfig::default())?.finish()) diff --git a/crates/iceberg/src/io/storage_oss.rs b/crates/iceberg/src/io/storage_oss.rs index 43c5cbb5c8..fe230c7289 100644 --- a/crates/iceberg/src/io/storage_oss.rs +++ b/crates/iceberg/src/io/storage_oss.rs @@ -25,8 +25,7 @@ use opendal::{Configurator, Operator}; use url::Url; use crate::io::{ - Extensions, FileMetadata, FileRead, FileWrite, InputFile, OutputFile, Storage, - StorageBuilder, + Extensions, FileMetadata, FileRead, FileWrite, InputFile, OutputFile, Storage, StorageBuilder, }; use crate::{Error, ErrorKind, Result}; @@ -83,7 +82,7 @@ impl OpenDALOssStorage { fn create_operator<'a>(&self, path: &'a str) -> Result<(Operator, &'a str)> { let op = oss_config_build(&self.config, path)?; let prefix = format!("oss://{}/", op.info().name()); - + if path.starts_with(&prefix) { let op = op.layer(opendal::layers::RetryLayer::new()); Ok((op, &path[prefix.len()..])) From f065f8b5de1f78137e7f34394d2edc44f532a67e Mon Sep 17 00:00:00 2001 From: Shawn Chang Date: Thu, 16 Oct 2025 19:31:30 -0700 Subject: [PATCH 09/47] clean up --- crates/iceberg/src/io/file_io.rs | 4 ++-- crates/iceberg/src/io/storage_memory.rs | 4 ---- 2 files changed, 2 insertions(+), 6 deletions(-) diff --git a/crates/iceberg/src/io/file_io.rs b/crates/iceberg/src/io/file_io.rs index d64b45db3f..71b903a22a 100644 --- a/crates/iceberg/src/io/file_io.rs +++ b/crates/iceberg/src/io/file_io.rs @@ -407,14 +407,14 @@ impl OutputFile { /// Checks if file exists. pub async fn exists(&self) -> Result { - Ok(self.storage.exists(&self.path).await?) + self.storage.exists(&self.path).await? } /// Deletes file. /// /// If the file does not exist, it will not return error. pub async fn delete(&self) -> Result<()> { - Ok(self.storage.delete(&self.path).await?) + self.storage.delete(&self.path).await? } /// Converts into [`InputFile`]. diff --git a/crates/iceberg/src/io/storage_memory.rs b/crates/iceberg/src/io/storage_memory.rs index c406929109..9bad67d30c 100644 --- a/crates/iceberg/src/io/storage_memory.rs +++ b/crates/iceberg/src/io/storage_memory.rs @@ -28,10 +28,6 @@ use crate::io::{ Extensions, FileMetadata, FileRead, FileWrite, InputFile, OutputFile, Storage, StorageBuilder, }; -pub(crate) fn memory_config_build() -> Result { - Ok(Operator::from_config(MemoryConfig::default())?.finish()) -} - /// Memory storage implementation using OpenDAL #[derive(Debug, Clone)] pub struct OpenDALMemoryStorage { From 3b161ed54f22186ec2d344e4b8b933b4d091bd0e Mon Sep 17 00:00:00 2001 From: Shawn Chang Date: Thu, 16 Oct 2025 19:34:24 -0700 Subject: [PATCH 10/47] minor --- crates/iceberg/src/io/file_io.rs | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/crates/iceberg/src/io/file_io.rs b/crates/iceberg/src/io/file_io.rs index 71b903a22a..82dd98b733 100644 --- a/crates/iceberg/src/io/file_io.rs +++ b/crates/iceberg/src/io/file_io.rs @@ -407,14 +407,14 @@ impl OutputFile { /// Checks if file exists. pub async fn exists(&self) -> Result { - self.storage.exists(&self.path).await? + self.storage.exists(&self.path).await } /// Deletes file. /// /// If the file does not exist, it will not return error. pub async fn delete(&self) -> Result<()> { - self.storage.delete(&self.path).await? + self.storage.delete(&self.path).await } /// Converts into [`InputFile`]. From 9b5c1aac0c1c037a2a9557da3015df6f05b4af0d Mon Sep 17 00:00:00 2001 From: Shawn Chang Date: Fri, 17 Oct 2025 15:48:54 -0700 Subject: [PATCH 11/47] encapsulate input and output file better --- crates/iceberg/src/io/file_io.rs | 61 ++++++++++++++++--------- crates/iceberg/src/io/storage_azdls.rs | 10 +--- crates/iceberg/src/io/storage_fs.rs | 10 +--- crates/iceberg/src/io/storage_gcs.rs | 10 +--- crates/iceberg/src/io/storage_memory.rs | 10 +--- crates/iceberg/src/io/storage_oss.rs | 10 +--- crates/iceberg/src/io/storage_s3.rs | 10 +--- 7 files changed, 52 insertions(+), 69 deletions(-) diff --git a/crates/iceberg/src/io/file_io.rs b/crates/iceberg/src/io/file_io.rs index 82dd98b733..5c8d994ae3 100644 --- a/crates/iceberg/src/io/file_io.rs +++ b/crates/iceberg/src/io/file_io.rs @@ -303,17 +303,28 @@ impl FileRead for opendal::Reader { /// Input file is used for reading from files. #[derive(Debug)] pub struct InputFile { - /// todo doc - pub storage: Arc, - // Absolution path of file. - /// todo doc - pub path: String, - // todo should remove this? Should always pass down a full path - // // Relative path of file to uri, starts at [`relative_path_pos`] - // relative_path_pos: usize, + /// Storage backend for this input file. + storage: Arc, + /// Absolute path to the file. + path: String, } impl InputFile { + /// Creates a new input file. + /// + /// # Arguments + /// + /// * `storage` - The storage backend to use + /// * `path` - Absolute path to the file + pub fn new(storage: Arc, path: String) -> Self { + Self { storage, path } + } + + /// Returns the storage backend for this input file. + pub fn storage(&self) -> &Arc { + &self.storage + } + /// Absolute path to root uri. pub fn location(&self) -> &str { &self.path @@ -386,20 +397,31 @@ impl FileWrite for Box { } } -/// Output file is used for writing to files.. +/// Output file is used for writing to files. #[derive(Debug)] pub struct OutputFile { - /// todo fix pub qualifier - pub storage: Arc, - // Absolution path of file. - /// todo fix pub qualifier - pub path: String, - // todo should always pass down a full path - // // Relative path of file to uri, starts at [`relative_path_pos`] - // relative_path_pos: usize, + /// Storage backend for this output file. + storage: Arc, + /// Absolute path to the file. + path: String, } impl OutputFile { + /// Creates a new output file. + /// + /// # Arguments + /// + /// * `storage` - The storage backend to use + /// * `path` - Absolute path to the file + pub fn new(storage: Arc, path: String) -> Self { + Self { storage, path } + } + + /// Returns the storage backend for this output file. + pub fn storage(&self) -> &Arc { + &self.storage + } + /// Relative path to root uri. pub fn location(&self) -> &str { &self.path @@ -419,10 +441,7 @@ impl OutputFile { /// Converts into [`InputFile`]. pub fn to_input_file(self) -> InputFile { - InputFile { - storage: self.storage, - path: self.path, - } + InputFile::new(self.storage, self.path) } /// Create a new output file with given bytes. diff --git a/crates/iceberg/src/io/storage_azdls.rs b/crates/iceberg/src/io/storage_azdls.rs index ff9b5f6a5a..22da4dd125 100644 --- a/crates/iceberg/src/io/storage_azdls.rs +++ b/crates/iceberg/src/io/storage_azdls.rs @@ -675,17 +675,11 @@ impl Storage for OpenDALAzdlsStorage { } fn new_input(&self, path: &str) -> Result { - Ok(InputFile { - storage: Arc::new(self.clone()), - path: path.to_string(), - }) + Ok(InputFile::new(Arc::new(self.clone()), path.to_string())) } fn new_output(&self, path: &str) -> Result { - Ok(OutputFile { - storage: Arc::new(self.clone()), - path: path.to_string(), - }) + Ok(OutputFile::new(Arc::new(self.clone()), path.to_string())) } } diff --git a/crates/iceberg/src/io/storage_fs.rs b/crates/iceberg/src/io/storage_fs.rs index 791ab3e9fc..0b612351f7 100644 --- a/crates/iceberg/src/io/storage_fs.rs +++ b/crates/iceberg/src/io/storage_fs.rs @@ -110,17 +110,11 @@ impl Storage for OpenDALFsStorage { } fn new_input(&self, path: &str) -> Result { - Ok(InputFile { - storage: Arc::new(self.clone()), - path: path.to_string(), - }) + Ok(InputFile::new(Arc::new(self.clone()), path.to_string())) } fn new_output(&self, path: &str) -> Result { - Ok(OutputFile { - storage: Arc::new(self.clone()), - path: path.to_string(), - }) + Ok(OutputFile::new(Arc::new(self.clone()), path.to_string())) } } diff --git a/crates/iceberg/src/io/storage_gcs.rs b/crates/iceberg/src/io/storage_gcs.rs index 65cff80de3..f3dc0212a9 100644 --- a/crates/iceberg/src/io/storage_gcs.rs +++ b/crates/iceberg/src/io/storage_gcs.rs @@ -187,17 +187,11 @@ impl Storage for OpenDALGcsStorage { } fn new_input(&self, path: &str) -> Result { - Ok(InputFile { - storage: Arc::new(self.clone()), - path: path.to_string(), - }) + Ok(InputFile::new(Arc::new(self.clone()), path.to_string())) } fn new_output(&self, path: &str) -> Result { - Ok(OutputFile { - storage: Arc::new(self.clone()), - path: path.to_string(), - }) + Ok(OutputFile::new(Arc::new(self.clone()), path.to_string())) } } diff --git a/crates/iceberg/src/io/storage_memory.rs b/crates/iceberg/src/io/storage_memory.rs index 9bad67d30c..b72a0b7abc 100644 --- a/crates/iceberg/src/io/storage_memory.rs +++ b/crates/iceberg/src/io/storage_memory.rs @@ -97,17 +97,11 @@ impl Storage for OpenDALMemoryStorage { } fn new_input(&self, path: &str) -> Result { - Ok(InputFile { - storage: Arc::new(self.clone()), - path: path.to_string(), - }) + Ok(InputFile::new(Arc::new(self.clone()), path.to_string())) } fn new_output(&self, path: &str) -> Result { - Ok(OutputFile { - storage: Arc::new(self.clone()), - path: path.to_string(), - }) + Ok(OutputFile::new(Arc::new(self.clone()), path.to_string())) } } diff --git a/crates/iceberg/src/io/storage_oss.rs b/crates/iceberg/src/io/storage_oss.rs index fe230c7289..c070edef01 100644 --- a/crates/iceberg/src/io/storage_oss.rs +++ b/crates/iceberg/src/io/storage_oss.rs @@ -147,17 +147,11 @@ impl Storage for OpenDALOssStorage { } fn new_input(&self, path: &str) -> Result { - Ok(InputFile { - storage: Arc::new(self.clone()), - path: path.to_string(), - }) + Ok(InputFile::new(Arc::new(self.clone()), path.to_string())) } fn new_output(&self, path: &str) -> Result { - Ok(OutputFile { - storage: Arc::new(self.clone()), - path: path.to_string(), - }) + Ok(OutputFile::new(Arc::new(self.clone()), path.to_string())) } } diff --git a/crates/iceberg/src/io/storage_s3.rs b/crates/iceberg/src/io/storage_s3.rs index 24d96a2874..be7bb623af 100644 --- a/crates/iceberg/src/io/storage_s3.rs +++ b/crates/iceberg/src/io/storage_s3.rs @@ -314,17 +314,11 @@ impl Storage for OpenDALS3Storage { } fn new_input(&self, path: &str) -> Result { - Ok(InputFile { - storage: Arc::new(self.clone()), - path: path.to_string(), - }) + Ok(InputFile::new(Arc::new(self.clone()), path.to_string())) } fn new_output(&self, path: &str) -> Result { - Ok(OutputFile { - storage: Arc::new(self.clone()), - path: path.to_string(), - }) + Ok(OutputFile::new(Arc::new(self.clone()), path.to_string())) } } From f0bd0abff3210075019456ff74e986e16f74c83e Mon Sep 17 00:00:00 2001 From: Shawn Chang Date: Fri, 17 Oct 2025 16:38:46 -0700 Subject: [PATCH 12/47] add with_registry --- crates/iceberg/src/io/file_io.rs | 296 ++++++++++++++++++++++++++++++- 1 file changed, 293 insertions(+), 3 deletions(-) diff --git a/crates/iceberg/src/io/file_io.rs b/crates/iceberg/src/io/file_io.rs index 5c8d994ae3..1077014d0d 100644 --- a/crates/iceberg/src/io/file_io.rs +++ b/crates/iceberg/src/io/file_io.rs @@ -175,6 +175,29 @@ impl Extensions { } /// Builder for [`FileIO`]. +/// +/// # Custom Storage Implementations +/// +/// You can use custom storage implementations by creating a custom +/// [`StorageBuilderRegistry`] and registering your storage builder: +/// +/// ```rust,ignore +/// use iceberg::io::{StorageBuilderRegistry, StorageBuilder, FileIOBuilder}; +/// use std::sync::Arc; +/// +/// // Create your custom storage builder +/// let my_builder = Arc::new(MyCustomStorageBuilder); +/// +/// // Register it with a custom scheme +/// let mut registry = StorageBuilderRegistry::new(); +/// registry.register("mycustom", my_builder); +/// +/// // Use it to build FileIO +/// let file_io = FileIOBuilder::new("mycustom") +/// .with_prop("key", "value") +/// .with_registry(registry) +/// .build()?; +/// ``` #[derive(Clone, Debug)] pub struct FileIOBuilder { /// This is used to infer scheme of operator. @@ -185,6 +208,8 @@ pub struct FileIOBuilder { props: HashMap, /// Optional extensions to configure the underlying FileIO behavior. extensions: Extensions, + /// Optional custom registry. If None, a default registry will be created. + registry: Option, } impl FileIOBuilder { @@ -195,6 +220,7 @@ impl FileIOBuilder { scheme_str: Some(scheme_str.to_string()), props: HashMap::default(), extensions: Extensions::default(), + registry: None, } } @@ -204,17 +230,26 @@ impl FileIOBuilder { scheme_str: None, props: HashMap::default(), extensions: Extensions::default(), + registry: None, } } /// Fetch the scheme string. /// /// The scheme_str will be empty if it's None. - pub fn into_parts(self) -> (String, HashMap, Extensions) { + pub fn into_parts( + self, + ) -> ( + String, + HashMap, + Extensions, + Option, + ) { ( self.scheme_str.unwrap_or_default(), self.props, self.extensions, + self.registry, ) } @@ -252,13 +287,38 @@ impl FileIOBuilder { self.extensions.get::() } + /// Sets a custom storage builder registry. + /// + /// This allows you to register custom storage implementations that can be used + /// when building the FileIO. If not set, a default registry with built-in + /// storage types will be used. + /// + /// # Example + /// + /// ```rust,ignore + /// use iceberg::io::{StorageBuilderRegistry, FileIOBuilder}; + /// use std::sync::Arc; + /// + /// let mut registry = StorageBuilderRegistry::new(); + /// registry.register("mycustom", Arc::new(MyCustomStorageBuilder)); + /// + /// let file_io = FileIOBuilder::new("mycustom") + /// .with_registry(registry) + /// .build()?; + /// ``` + pub fn with_registry(mut self, registry: StorageBuilderRegistry) -> Self { + self.registry = Some(registry); + self + } + /// Builds [`FileIO`]. pub fn build(self) -> Result { // Use the scheme to determine the storage type let scheme = self.scheme_str.clone().unwrap_or_default(); - // Create registry and get builder - let registry = StorageBuilderRegistry::new(); + // Use custom registry if provided, otherwise create default + let registry = self.registry.clone().unwrap_or_default(); + let builder = registry.get_builder(scheme.as_str())?; // Build storage with props and extensions @@ -466,16 +526,94 @@ impl OutputFile { #[cfg(test)] mod tests { + use std::collections::HashMap; use std::fs::{File, create_dir_all}; use std::io::Write; use std::path::Path; + use std::sync::Arc; + use async_trait::async_trait; use bytes::Bytes; use futures::AsyncReadExt; use futures::io::AllowStdIo; use tempfile::TempDir; use super::{FileIO, FileIOBuilder}; + use crate::io::{ + Extensions, FileMetadata, FileRead, FileWrite, InputFile, OutputFile, Storage, + StorageBuilder, StorageBuilderRegistry, + }; + + // Dummy storage implementation for testing custom registries + #[derive(Debug, Clone)] + struct DummyStorage { + _scheme: String, + } + + #[async_trait] + impl Storage for DummyStorage { + async fn exists(&self, _path: &str) -> crate::Result { + Ok(true) + } + + async fn metadata(&self, _path: &str) -> crate::Result { + Ok(FileMetadata { size: 0 }) + } + + async fn read(&self, _path: &str) -> crate::Result { + Ok(Bytes::new()) + } + + async fn reader(&self, _path: &str) -> crate::Result> { + Err(crate::Error::new( + crate::ErrorKind::FeatureUnsupported, + "DummyStorage does not support reader", + )) + } + + async fn write(&self, _path: &str, _bs: Bytes) -> crate::Result<()> { + Ok(()) + } + + async fn writer(&self, _path: &str) -> crate::Result> { + Err(crate::Error::new( + crate::ErrorKind::FeatureUnsupported, + "DummyStorage does not support writer", + )) + } + + async fn delete(&self, _path: &str) -> crate::Result<()> { + Ok(()) + } + + async fn remove_dir_all(&self, _path: &str) -> crate::Result<()> { + Ok(()) + } + + fn new_input(&self, path: &str) -> crate::Result { + Ok(InputFile::new(Arc::new(self.clone()), path.to_string())) + } + + fn new_output(&self, path: &str) -> crate::Result { + Ok(OutputFile::new(Arc::new(self.clone()), path.to_string())) + } + } + + // Dummy storage builder for testing + #[derive(Debug)] + struct DummyStorageBuilder; + + impl StorageBuilder for DummyStorageBuilder { + fn build( + &self, + _props: HashMap, + _extensions: Extensions, + ) -> crate::Result> { + Ok(Arc::new(DummyStorage { + _scheme: "dummy".to_string(), + })) + } + } fn create_local_file_io() -> FileIO { FileIOBuilder::new_fs_io().build().unwrap() @@ -619,4 +757,156 @@ mod tests { io.delete(&path).await.unwrap(); assert!(!io.exists(&path).await.unwrap()); } + + #[test] + fn test_custom_registry() { + // Create a custom registry and register the dummy storage + let mut registry = StorageBuilderRegistry::new(); + registry.register("dummy", Arc::new(DummyStorageBuilder)); + + // Build FileIO with custom storage + let file_io = FileIOBuilder::new("dummy") + .with_registry(registry) + .build() + .unwrap(); + + // Verify we can create files with the custom storage + assert!(file_io.new_output("dummy://test.txt").is_ok()); + assert!(file_io.new_input("dummy://test.txt").is_ok()); + } + + #[tokio::test] + async fn test_custom_registry_operations() { + // Define a dummy storage that tracks operations + #[derive(Debug, Clone)] + struct TrackingStorage { + written: Arc>>, + } + + #[async_trait] + impl Storage for TrackingStorage { + async fn exists(&self, _path: &str) -> crate::Result { + Ok(true) + } + + async fn metadata(&self, _path: &str) -> crate::Result { + Ok(FileMetadata { size: 42 }) + } + + async fn read(&self, _path: &str) -> crate::Result { + Ok(Bytes::from("test data")) + } + + async fn reader(&self, _path: &str) -> crate::Result> { + Err(crate::Error::new( + crate::ErrorKind::FeatureUnsupported, + "TrackingStorage does not support reader", + )) + } + + async fn write(&self, path: &str, _bs: Bytes) -> crate::Result<()> { + self.written.lock().unwrap().push(path.to_string()); + Ok(()) + } + + async fn writer(&self, _path: &str) -> crate::Result> { + Err(crate::Error::new( + crate::ErrorKind::FeatureUnsupported, + "TrackingStorage does not support writer", + )) + } + + async fn delete(&self, _path: &str) -> crate::Result<()> { + Ok(()) + } + + async fn remove_dir_all(&self, _path: &str) -> crate::Result<()> { + Ok(()) + } + + fn new_input(&self, path: &str) -> crate::Result { + Ok(InputFile::new(Arc::new(self.clone()), path.to_string())) + } + + fn new_output(&self, path: &str) -> crate::Result { + Ok(OutputFile::new(Arc::new(self.clone()), path.to_string())) + } + } + + // Define a builder for tracking storage + #[derive(Debug)] + struct TrackingStorageBuilder { + written: Arc>>, + } + + impl StorageBuilder for TrackingStorageBuilder { + fn build( + &self, + _props: HashMap, + _extensions: Extensions, + ) -> crate::Result> { + Ok(Arc::new(TrackingStorage { + written: self.written.clone(), + })) + } + } + + // Create tracking storage + let written = Arc::new(std::sync::Mutex::new(Vec::new())); + let mut registry = StorageBuilderRegistry::new(); + registry.register( + "tracking", + Arc::new(TrackingStorageBuilder { + written: written.clone(), + }), + ); + + // Build FileIO with tracking storage + let file_io = FileIOBuilder::new("tracking") + .with_registry(registry) + .build() + .unwrap(); + + // Perform operations + let output = file_io.new_output("tracking://bucket/file.txt").unwrap(); + output.write(Bytes::from("test")).await.unwrap(); + + let input = file_io.new_input("tracking://bucket/file.txt").unwrap(); + let data = input.read().await.unwrap(); + assert_eq!(data, Bytes::from("test data")); + + let metadata = input.metadata().await.unwrap(); + assert_eq!(metadata.size, 42); + + // Verify write was tracked + let tracked = written.lock().unwrap(); + assert_eq!(tracked.len(), 1); + assert_eq!(tracked[0], "tracking://bucket/file.txt"); + } + + #[test] + fn test_into_parts_includes_registry() { + let registry = StorageBuilderRegistry::new(); + + let builder = FileIOBuilder::new("memory") + .with_prop("key", "value") + .with_registry(registry.clone()); + + let (scheme, props, _extensions, returned_registry) = builder.into_parts(); + + assert_eq!(scheme, "memory"); + assert_eq!(props.get("key"), Some(&"value".to_string())); + assert!(returned_registry.is_some()); + } + + #[test] + fn test_into_parts_without_registry() { + let builder = FileIOBuilder::new("memory").with_prop("key", "value"); + + let (scheme, props, _extensions, returned_registry) = builder.into_parts(); + + assert_eq!(scheme, "memory"); + assert_eq!(props.get("key"), Some(&"value".to_string())); + assert!(returned_registry.is_none()); + } } From 463c545de1715e69a1287ba900ef17f1acaf4ef6 Mon Sep 17 00:00:00 2001 From: Shawn Chang Date: Fri, 17 Oct 2025 16:46:43 -0700 Subject: [PATCH 13/47] minor --- crates/iceberg/src/io/mod.rs | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/crates/iceberg/src/io/mod.rs b/crates/iceberg/src/io/mod.rs index 2397e2561f..f4c186a5c6 100644 --- a/crates/iceberg/src/io/mod.rs +++ b/crates/iceberg/src/io/mod.rs @@ -88,8 +88,12 @@ mod storage_s3; #[cfg(feature = "storage-azdls")] pub use storage_azdls::*; +#[cfg(feature = "storage-fs")] +use storage_fs::*; #[cfg(feature = "storage-gcs")] pub use storage_gcs::*; +#[cfg(feature = "storage-memory")] +use storage_memory::*; #[cfg(feature = "storage-oss")] pub use storage_oss::*; #[cfg(feature = "storage-s3")] From 96fe4a0a65f1369ab165418b0163687ed5867ea5 Mon Sep 17 00:00:00 2001 From: Shawn Chang Date: Fri, 17 Oct 2025 16:50:15 -0700 Subject: [PATCH 14/47] minor --- crates/iceberg/src/io/file_io.rs | 4 ---- crates/iceberg/src/io/mod.rs | 4 ---- 2 files changed, 8 deletions(-) diff --git a/crates/iceberg/src/io/file_io.rs b/crates/iceberg/src/io/file_io.rs index 1077014d0d..2ca5a77ddd 100644 --- a/crates/iceberg/src/io/file_io.rs +++ b/crates/iceberg/src/io/file_io.rs @@ -363,9 +363,7 @@ impl FileRead for opendal::Reader { /// Input file is used for reading from files. #[derive(Debug)] pub struct InputFile { - /// Storage backend for this input file. storage: Arc, - /// Absolute path to the file. path: String, } @@ -460,9 +458,7 @@ impl FileWrite for Box { /// Output file is used for writing to files. #[derive(Debug)] pub struct OutputFile { - /// Storage backend for this output file. storage: Arc, - /// Absolute path to the file. path: String, } diff --git a/crates/iceberg/src/io/mod.rs b/crates/iceberg/src/io/mod.rs index f4c186a5c6..2397e2561f 100644 --- a/crates/iceberg/src/io/mod.rs +++ b/crates/iceberg/src/io/mod.rs @@ -88,12 +88,8 @@ mod storage_s3; #[cfg(feature = "storage-azdls")] pub use storage_azdls::*; -#[cfg(feature = "storage-fs")] -use storage_fs::*; #[cfg(feature = "storage-gcs")] pub use storage_gcs::*; -#[cfg(feature = "storage-memory")] -use storage_memory::*; #[cfg(feature = "storage-oss")] pub use storage_oss::*; #[cfg(feature = "storage-s3")] From cf6998e88b61b04306001fa8d656a945ccc7dbc1 Mon Sep 17 00:00:00 2001 From: Shawn Chang Date: Fri, 17 Oct 2025 17:29:14 -0700 Subject: [PATCH 15/47] Pass scheme strings cleanly --- crates/iceberg/src/io/file_io.rs | 7 +++++-- crates/iceberg/src/io/mod.rs | 6 ++++++ crates/iceberg/src/io/storage_azdls.rs | 3 ++- crates/iceberg/src/io/storage_s3.rs | 4 ++-- 4 files changed, 15 insertions(+), 5 deletions(-) diff --git a/crates/iceberg/src/io/file_io.rs b/crates/iceberg/src/io/file_io.rs index 2ca5a77ddd..5ef8818033 100644 --- a/crates/iceberg/src/io/file_io.rs +++ b/crates/iceberg/src/io/file_io.rs @@ -27,7 +27,7 @@ use url::Url; // Re-export traits from storage module pub use super::storage::{Storage, StorageBuilder}; -use crate::io::StorageBuilderRegistry; +use crate::io::{StorageBuilderRegistry, STORAGE_LOCATION_SCHEME}; use crate::{Error, ErrorKind, Result}; /// FileIO implementation, used to manipulate files in underlying storage. @@ -321,8 +321,11 @@ impl FileIOBuilder { let builder = registry.get_builder(scheme.as_str())?; + let mut props_with_scheme = self.props.clone(); + props_with_scheme.insert(STORAGE_LOCATION_SCHEME.to_string(), scheme); + // Build storage with props and extensions - let storage = builder.build(self.props.clone(), self.extensions.clone())?; + let storage = builder.build(props_with_scheme, self.extensions.clone())?; Ok(FileIO { builder: self, diff --git a/crates/iceberg/src/io/mod.rs b/crates/iceberg/src/io/mod.rs index 2397e2561f..f1a7600777 100644 --- a/crates/iceberg/src/io/mod.rs +++ b/crates/iceberg/src/io/mod.rs @@ -73,6 +73,12 @@ pub use file_io::*; pub use storage::{Storage, StorageBuilder, StorageBuilderRegistry}; pub(crate) mod object_cache; +/// Property key used to pass the scheme string to storage builders. +/// +/// Custom storage implementations can use this constant to retrieve the scheme +/// from the properties map passed to `StorageBuilder::build()`. +pub const STORAGE_LOCATION_SCHEME: &str = "iceberg.storage.location.scheme"; + #[cfg(feature = "storage-azdls")] mod storage_azdls; #[cfg(feature = "storage-fs")] diff --git a/crates/iceberg/src/io/storage_azdls.rs b/crates/iceberg/src/io/storage_azdls.rs index 22da4dd125..2f963956b2 100644 --- a/crates/iceberg/src/io/storage_azdls.rs +++ b/crates/iceberg/src/io/storage_azdls.rs @@ -28,6 +28,7 @@ use url::Url; use crate::io::{ Extensions, FileMetadata, FileRead, FileWrite, InputFile, OutputFile, Storage, StorageBuilder, + STORAGE_LOCATION_SCHEME, }; use crate::{Error, ErrorKind, Result, ensure_data_valid}; @@ -695,7 +696,7 @@ impl StorageBuilder for OpenDALAzdlsStorageBuilder { ) -> Result> { // Get the scheme string from the props or use default let scheme_str = props - .get("scheme_str") + .get(STORAGE_LOCATION_SCHEME) .cloned() .unwrap_or_else(|| "abfs".to_string()); diff --git a/crates/iceberg/src/io/storage_s3.rs b/crates/iceberg/src/io/storage_s3.rs index be7bb623af..63640ae591 100644 --- a/crates/iceberg/src/io/storage_s3.rs +++ b/crates/iceberg/src/io/storage_s3.rs @@ -28,7 +28,7 @@ use url::Url; use crate::io::{ Extensions, FileMetadata, FileRead, FileWrite, InputFile, OutputFile, Storage, StorageBuilder, - is_truthy, + is_truthy, STORAGE_LOCATION_SCHEME, }; use crate::{Error, ErrorKind, Result}; @@ -334,7 +334,7 @@ impl StorageBuilder for OpenDALS3StorageBuilder { ) -> Result> { // Get the scheme string from the props or use "s3" as default let scheme_str = props - .get("scheme_str") + .get(STORAGE_LOCATION_SCHEME) .cloned() .unwrap_or_else(|| "s3".to_string()); From a46f94712c0a660c406de174c0fba8ddd2e4db19 Mon Sep 17 00:00:00 2001 From: Shawn Chang Date: Fri, 17 Oct 2025 17:32:01 -0700 Subject: [PATCH 16/47] daily fmt fix --- crates/iceberg/src/io/file_io.rs | 2 +- crates/iceberg/src/io/mod.rs | 5 +---- crates/iceberg/src/io/storage_azdls.rs | 4 ++-- crates/iceberg/src/io/storage_s3.rs | 4 ++-- 4 files changed, 6 insertions(+), 9 deletions(-) diff --git a/crates/iceberg/src/io/file_io.rs b/crates/iceberg/src/io/file_io.rs index 5ef8818033..f519613844 100644 --- a/crates/iceberg/src/io/file_io.rs +++ b/crates/iceberg/src/io/file_io.rs @@ -27,7 +27,7 @@ use url::Url; // Re-export traits from storage module pub use super::storage::{Storage, StorageBuilder}; -use crate::io::{StorageBuilderRegistry, STORAGE_LOCATION_SCHEME}; +use crate::io::{STORAGE_LOCATION_SCHEME, StorageBuilderRegistry}; use crate::{Error, ErrorKind, Result}; /// FileIO implementation, used to manipulate files in underlying storage. diff --git a/crates/iceberg/src/io/mod.rs b/crates/iceberg/src/io/mod.rs index f1a7600777..209be5321d 100644 --- a/crates/iceberg/src/io/mod.rs +++ b/crates/iceberg/src/io/mod.rs @@ -73,10 +73,7 @@ pub use file_io::*; pub use storage::{Storage, StorageBuilder, StorageBuilderRegistry}; pub(crate) mod object_cache; -/// Property key used to pass the scheme string to storage builders. -/// -/// Custom storage implementations can use this constant to retrieve the scheme -/// from the properties map passed to `StorageBuilder::build()`. +/// Property key used to pass the scheme string from FileIOBuilder to StorageBuilder. pub const STORAGE_LOCATION_SCHEME: &str = "iceberg.storage.location.scheme"; #[cfg(feature = "storage-azdls")] diff --git a/crates/iceberg/src/io/storage_azdls.rs b/crates/iceberg/src/io/storage_azdls.rs index 2f963956b2..f0a85e44b1 100644 --- a/crates/iceberg/src/io/storage_azdls.rs +++ b/crates/iceberg/src/io/storage_azdls.rs @@ -27,8 +27,8 @@ use opendal::{Configurator, Operator}; use url::Url; use crate::io::{ - Extensions, FileMetadata, FileRead, FileWrite, InputFile, OutputFile, Storage, StorageBuilder, - STORAGE_LOCATION_SCHEME, + Extensions, FileMetadata, FileRead, FileWrite, InputFile, OutputFile, STORAGE_LOCATION_SCHEME, + Storage, StorageBuilder, }; use crate::{Error, ErrorKind, Result, ensure_data_valid}; diff --git a/crates/iceberg/src/io/storage_s3.rs b/crates/iceberg/src/io/storage_s3.rs index 63640ae591..cc8d84ca77 100644 --- a/crates/iceberg/src/io/storage_s3.rs +++ b/crates/iceberg/src/io/storage_s3.rs @@ -27,8 +27,8 @@ use reqwest::Client; use url::Url; use crate::io::{ - Extensions, FileMetadata, FileRead, FileWrite, InputFile, OutputFile, Storage, StorageBuilder, - is_truthy, STORAGE_LOCATION_SCHEME, + Extensions, FileMetadata, FileRead, FileWrite, InputFile, OutputFile, STORAGE_LOCATION_SCHEME, + Storage, StorageBuilder, is_truthy, }; use crate::{Error, ErrorKind, Result}; From 790e0253a6d1ba83a54f51cf8a8ed62a5c2da7f5 Mon Sep 17 00:00:00 2001 From: Peter Nguyen Date: Fri, 17 Oct 2025 20:13:21 -0700 Subject: [PATCH 17/47] docs: Fix broken orbstack and podman links in CONTRIBUTING.md (#1757) ## Which issue does this PR close? - Closes #1756 ## What changes are included in this PR? Fix broken orbstack and podman links in CONTRIBUTING.md ## Are these changes tested? N/A --- CONTRIBUTING.md | 4 ++-- website/src/reference/orbstack.md | 4 ++-- 2 files changed, 4 insertions(+), 4 deletions(-) diff --git a/CONTRIBUTING.md b/CONTRIBUTING.md index e4b175b055..0d05c3ddac 100644 --- a/CONTRIBUTING.md +++ b/CONTRIBUTING.md @@ -103,8 +103,8 @@ cargo 1.69.0 (6e9a83356 2023-04-12) Currently, iceberg-rust uses Docker to set up environment for integration tests. Native Docker has some limitations, please check (https://github.com/apache/iceberg-rust/pull/748). Please use Orbstack or Podman. -For MacOS users, you can install [OrbStack as a docker alternative](reference/orbstack.md). -For Podman users, refer to [Using Podman instead of Docker](reference/podman.md) +For MacOS users, you can install [OrbStack as a docker alternative](website/src/reference/orbstack.md). +For Podman users, refer to [Using Podman instead of Docker](website/src/reference/podman.md) ## Build diff --git a/website/src/reference/orbstack.md b/website/src/reference/orbstack.md index cb00849d6b..cbd6566359 100644 --- a/website/src/reference/orbstack.md +++ b/website/src/reference/orbstack.md @@ -25,7 +25,7 @@ 2. Migrate Docker data ```shell - orbstack migrate docker + orb migrate docker ``` 3. (Optional) Add registry mirrors @@ -36,4 +36,4 @@ { "registry-mirrors": [""] } - ``` \ No newline at end of file + ``` From 12f20605018650fa6cc8f0d44aff25895dcd1e04 Mon Sep 17 00:00:00 2001 From: Shawn Chang Date: Fri, 17 Oct 2025 20:33:03 -0700 Subject: [PATCH 18/47] optimize tests --- crates/iceberg/src/io/file_io.rs | 240 +++++++++++++++---------------- 1 file changed, 115 insertions(+), 125 deletions(-) diff --git a/crates/iceberg/src/io/file_io.rs b/crates/iceberg/src/io/file_io.rs index f519613844..e72dd1061f 100644 --- a/crates/iceberg/src/io/file_io.rs +++ b/crates/iceberg/src/io/file_io.rs @@ -529,7 +529,7 @@ mod tests { use std::fs::{File, create_dir_all}; use std::io::Write; use std::path::Path; - use std::sync::Arc; + use std::sync::{Arc, Mutex, MutexGuard}; use async_trait::async_trait; use bytes::Bytes; @@ -538,78 +538,107 @@ mod tests { use tempfile::TempDir; use super::{FileIO, FileIOBuilder}; + use crate::{Error, ErrorKind, Result}; use crate::io::{ - Extensions, FileMetadata, FileRead, FileWrite, InputFile, OutputFile, Storage, - StorageBuilder, StorageBuilderRegistry, + Extensions, FileMetadata, FileRead, FileWrite, InputFile, OutputFile, + STORAGE_LOCATION_SCHEME, Storage, StorageBuilder, StorageBuilderRegistry, }; - // Dummy storage implementation for testing custom registries + // Test storage implementation that tracks write operations #[derive(Debug, Clone)] - struct DummyStorage { - _scheme: String, + struct TestStorage { + written: Arc>>, + received_props: HashMap, + } + + #[allow(dead_code)] + impl TestStorage { + pub fn written(&self) -> MutexGuard<'_, Vec> { + self.written.lock().unwrap() + } + + pub fn received_props(&self) -> &HashMap { + &self.received_props + } } #[async_trait] - impl Storage for DummyStorage { - async fn exists(&self, _path: &str) -> crate::Result { + impl Storage for TestStorage { + async fn exists(&self, _path: &str) -> Result { Ok(true) } - async fn metadata(&self, _path: &str) -> crate::Result { - Ok(FileMetadata { size: 0 }) + async fn metadata(&self, _path: &str) -> Result { + Ok(FileMetadata { size: 42 }) } - async fn read(&self, _path: &str) -> crate::Result { - Ok(Bytes::new()) + async fn read(&self, _path: &str) -> Result { + Ok(Bytes::from("test data")) } - async fn reader(&self, _path: &str) -> crate::Result> { - Err(crate::Error::new( - crate::ErrorKind::FeatureUnsupported, - "DummyStorage does not support reader", + async fn reader(&self, _path: &str) -> Result> { + Err(Error::new( + ErrorKind::FeatureUnsupported, + "TestStorage does not support reader", )) } - async fn write(&self, _path: &str, _bs: Bytes) -> crate::Result<()> { + async fn write(&self, path: &str, _bs: Bytes) -> Result<()> { + self.written.lock().unwrap().push(path.to_string()); Ok(()) } - async fn writer(&self, _path: &str) -> crate::Result> { - Err(crate::Error::new( - crate::ErrorKind::FeatureUnsupported, - "DummyStorage does not support writer", + async fn writer(&self, _path: &str) -> Result> { + Err(Error::new( + ErrorKind::FeatureUnsupported, + "TestStorage does not support writer", )) } - async fn delete(&self, _path: &str) -> crate::Result<()> { + async fn delete(&self, _path: &str) -> Result<()> { Ok(()) } - async fn remove_dir_all(&self, _path: &str) -> crate::Result<()> { + async fn remove_dir_all(&self, _path: &str) -> Result<()> { Ok(()) } - fn new_input(&self, path: &str) -> crate::Result { + fn new_input(&self, path: &str) -> Result { Ok(InputFile::new(Arc::new(self.clone()), path.to_string())) } - fn new_output(&self, path: &str) -> crate::Result { + fn new_output(&self, path: &str) -> Result { Ok(OutputFile::new(Arc::new(self.clone()), path.to_string())) } } - // Dummy storage builder for testing + // Test storage builder #[derive(Debug)] - struct DummyStorageBuilder; + struct TestStorageBuilder { + written: Arc>>, + received_props: Arc>>, + } + + impl TestStorageBuilder { + pub fn written(&self) -> MutexGuard<'_, Vec> { + self.written.lock().unwrap() + } - impl StorageBuilder for DummyStorageBuilder { + pub fn received_props(&self) -> MutexGuard<'_, HashMap> { + self.received_props.lock().unwrap() + } + } + + impl StorageBuilder for TestStorageBuilder { fn build( &self, - _props: HashMap, + props: HashMap, _extensions: Extensions, - ) -> crate::Result> { - Ok(Arc::new(DummyStorage { - _scheme: "dummy".to_string(), + ) -> Result> { + *self.received_props.lock().unwrap() = props.clone(); + Ok(Arc::new(TestStorage { + written: self.written.clone(), + received_props: props, })) } } @@ -759,118 +788,48 @@ mod tests { #[test] fn test_custom_registry() { - // Create a custom registry and register the dummy storage + // Create a custom registry and register test storage + let builder = Arc::new(TestStorageBuilder { + written: Arc::new(Mutex::new(Vec::new())), + received_props: Arc::new(Mutex::new(HashMap::new())), + }); + let mut registry = StorageBuilderRegistry::new(); - registry.register("dummy", Arc::new(DummyStorageBuilder)); + registry.register("test", builder.clone()); // Build FileIO with custom storage - let file_io = FileIOBuilder::new("dummy") + let file_io = FileIOBuilder::new("test") .with_registry(registry) .build() .unwrap(); // Verify we can create files with the custom storage - assert!(file_io.new_output("dummy://test.txt").is_ok()); - assert!(file_io.new_input("dummy://test.txt").is_ok()); + assert!(file_io.new_output("test://test.txt").is_ok()); + assert!(file_io.new_input("test://test.txt").is_ok()); } #[tokio::test] async fn test_custom_registry_operations() { - // Define a dummy storage that tracks operations - #[derive(Debug, Clone)] - struct TrackingStorage { - written: Arc>>, - } - - #[async_trait] - impl Storage for TrackingStorage { - async fn exists(&self, _path: &str) -> crate::Result { - Ok(true) - } - - async fn metadata(&self, _path: &str) -> crate::Result { - Ok(FileMetadata { size: 42 }) - } - - async fn read(&self, _path: &str) -> crate::Result { - Ok(Bytes::from("test data")) - } - - async fn reader(&self, _path: &str) -> crate::Result> { - Err(crate::Error::new( - crate::ErrorKind::FeatureUnsupported, - "TrackingStorage does not support reader", - )) - } - - async fn write(&self, path: &str, _bs: Bytes) -> crate::Result<()> { - self.written.lock().unwrap().push(path.to_string()); - Ok(()) - } - - async fn writer(&self, _path: &str) -> crate::Result> { - Err(crate::Error::new( - crate::ErrorKind::FeatureUnsupported, - "TrackingStorage does not support writer", - )) - } - - async fn delete(&self, _path: &str) -> crate::Result<()> { - Ok(()) - } - - async fn remove_dir_all(&self, _path: &str) -> crate::Result<()> { - Ok(()) - } - - fn new_input(&self, path: &str) -> crate::Result { - Ok(InputFile::new(Arc::new(self.clone()), path.to_string())) - } - - fn new_output(&self, path: &str) -> crate::Result { - Ok(OutputFile::new(Arc::new(self.clone()), path.to_string())) - } - } - - // Define a builder for tracking storage - #[derive(Debug)] - struct TrackingStorageBuilder { - written: Arc>>, - } - - impl StorageBuilder for TrackingStorageBuilder { - fn build( - &self, - _props: HashMap, - _extensions: Extensions, - ) -> crate::Result> { - Ok(Arc::new(TrackingStorage { - written: self.written.clone(), - })) - } - } - - // Create tracking storage - let written = Arc::new(std::sync::Mutex::new(Vec::new())); + // Create test storage with write tracking + let builder = Arc::new(TestStorageBuilder { + written: Arc::new(Mutex::new(Vec::new())), + received_props: Arc::new(Mutex::new(HashMap::new())), + }); + let mut registry = StorageBuilderRegistry::new(); - registry.register( - "tracking", - Arc::new(TrackingStorageBuilder { - written: written.clone(), - }), - ); + registry.register("test", builder.clone()); - // Build FileIO with tracking storage - let file_io = FileIOBuilder::new("tracking") + // Build FileIO with test storage + let file_io = FileIOBuilder::new("test") .with_registry(registry) .build() .unwrap(); // Perform operations - let output = file_io.new_output("tracking://bucket/file.txt").unwrap(); + let output = file_io.new_output("test://bucket/file.txt").unwrap(); output.write(Bytes::from("test")).await.unwrap(); - let input = file_io.new_input("tracking://bucket/file.txt").unwrap(); + let input = file_io.new_input("test://bucket/file.txt").unwrap(); let data = input.read().await.unwrap(); assert_eq!(data, Bytes::from("test data")); @@ -878,9 +837,40 @@ mod tests { assert_eq!(metadata.size, 42); // Verify write was tracked - let tracked = written.lock().unwrap(); + let tracked = builder.written(); assert_eq!(tracked.len(), 1); - assert_eq!(tracked[0], "tracking://bucket/file.txt"); + assert_eq!(tracked[0], "test://bucket/file.txt"); + } + + #[test] + fn test_scheme_and_props_propagation() { + // Create test storage that captures props + let builder = Arc::new(TestStorageBuilder { + written: Arc::new(Mutex::new(Vec::new())), + received_props: Arc::new(Mutex::new(HashMap::new())), + }); + + let mut registry = StorageBuilderRegistry::new(); + registry.register("myscheme", builder.clone()); + + // Build FileIO with custom scheme and additional props + let file_io = FileIOBuilder::new("myscheme") + .with_prop("custom.prop", "custom_value") + .with_registry(registry) + .build() + .unwrap(); + + // Verify the storage was created + assert!(file_io.new_output("myscheme://test.txt").is_ok()); + + // Verify the scheme was propagated to the builder + let props = builder.received_props(); + assert_eq!( + props.get(STORAGE_LOCATION_SCHEME), + Some(&"myscheme".to_string()) + ); + // Verify custom props were also passed + assert_eq!(props.get("custom.prop"), Some(&"custom_value".to_string())); } #[test] From 1ae79f8c587bb8e30b5e4dbd7113abbf71a2b571 Mon Sep 17 00:00:00 2001 From: Shawn Chang Date: Fri, 17 Oct 2025 20:34:22 -0700 Subject: [PATCH 19/47] fmt fix --- crates/iceberg/src/io/file_io.rs | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/crates/iceberg/src/io/file_io.rs b/crates/iceberg/src/io/file_io.rs index e72dd1061f..276b71d063 100644 --- a/crates/iceberg/src/io/file_io.rs +++ b/crates/iceberg/src/io/file_io.rs @@ -538,11 +538,11 @@ mod tests { use tempfile::TempDir; use super::{FileIO, FileIOBuilder}; - use crate::{Error, ErrorKind, Result}; use crate::io::{ Extensions, FileMetadata, FileRead, FileWrite, InputFile, OutputFile, STORAGE_LOCATION_SCHEME, Storage, StorageBuilder, StorageBuilderRegistry, }; + use crate::{Error, ErrorKind, Result}; // Test storage implementation that tracks write operations #[derive(Debug, Clone)] @@ -793,7 +793,7 @@ mod tests { written: Arc::new(Mutex::new(Vec::new())), received_props: Arc::new(Mutex::new(HashMap::new())), }); - + let mut registry = StorageBuilderRegistry::new(); registry.register("test", builder.clone()); @@ -815,7 +815,7 @@ mod tests { written: Arc::new(Mutex::new(Vec::new())), received_props: Arc::new(Mutex::new(HashMap::new())), }); - + let mut registry = StorageBuilderRegistry::new(); registry.register("test", builder.clone()); @@ -849,7 +849,7 @@ mod tests { written: Arc::new(Mutex::new(Vec::new())), received_props: Arc::new(Mutex::new(HashMap::new())), }); - + let mut registry = StorageBuilderRegistry::new(); registry.register("myscheme", builder.clone()); From b7d7937a907b4ffdfe577fbb3213630ea17634b2 Mon Sep 17 00:00:00 2001 From: Shawn Chang Date: Fri, 17 Oct 2025 20:48:36 -0700 Subject: [PATCH 20/47] minor --- crates/iceberg/src/io/file_io.rs | 1 - 1 file changed, 1 deletion(-) diff --git a/crates/iceberg/src/io/file_io.rs b/crates/iceberg/src/io/file_io.rs index 276b71d063..d0c8a3a1b8 100644 --- a/crates/iceberg/src/io/file_io.rs +++ b/crates/iceberg/src/io/file_io.rs @@ -102,7 +102,6 @@ impl FileIO { /// * path: It should be *absolute* path starting with scheme string used to construct [`FileIO`]. #[deprecated(note = "use remove_dir_all instead", since = "0.4.0")] pub async fn remove_all(&self, path: impl AsRef) -> Result<()> { - // todo this should be removed as it doesn't exist in the new trait self.inner.remove_dir_all(path.as_ref()).await } From c2b5c830570284c88922ea325036b86a0393e6e6 Mon Sep 17 00:00:00 2001 From: Shawn Chang Date: Fri, 17 Oct 2025 20:51:02 -0700 Subject: [PATCH 21/47] minor againnnn --- crates/iceberg/src/io/file_io.rs | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/crates/iceberg/src/io/file_io.rs b/crates/iceberg/src/io/file_io.rs index d0c8a3a1b8..48b9396bc9 100644 --- a/crates/iceberg/src/io/file_io.rs +++ b/crates/iceberg/src/io/file_io.rs @@ -26,8 +26,8 @@ use bytes::Bytes; use url::Url; // Re-export traits from storage module -pub use super::storage::{Storage, StorageBuilder}; -use crate::io::{STORAGE_LOCATION_SCHEME, StorageBuilderRegistry}; +pub use super::storage::{Storage, StorageBuilder, StorageBuilderRegistry}; +use crate::io::STORAGE_LOCATION_SCHEME; use crate::{Error, ErrorKind, Result}; /// FileIO implementation, used to manipulate files in underlying storage. From 2861dabab8931ebb0e07bcf47c47990355b7ad55 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Sun, 19 Oct 2025 16:56:17 -0400 Subject: [PATCH 22/47] chore(deps): Bump tokio from 1.47.1 to 1.48.0 (#1763) Bumps [tokio](https://github.com/tokio-rs/tokio) from 1.47.1 to 1.48.0.
Release notes

Sourced from tokio's releases.

Tokio v1.48.0

1.48.0 (October 14th, 2025)

The MSRV is increased to 1.71.

Added

  • fs: add File::max_buf_size (#7594)
  • io: export Chain of AsyncReadExt::chain (#7599)
  • net: add SocketAddr::as_abstract_name (#7491)
  • net: add TcpStream::quickack and TcpStream::set_quickack (#7490)
  • net: implement AsRef<Self> for TcpStream and UnixStream (#7573)
  • task: add LocalKey::try_get (#7666)
  • task: implement Ord for task::Id (#7530)

Changed

  • deps: bump windows-sys to version 0.61 (#7645)
  • fs: preserve max_buf_size when cloning a File (#7593)
  • macros: suppress clippy::unwrap_in_result in #[tokio::main] (#7651)
  • net: remove PollEvented noise from Debug formats (#7675)
  • process: upgrade Command::spawn_with to use FnOnce (#7511)
  • sync: remove inner mutex in SetOnce (#7554)
  • sync: use UnsafeCell::get_mut in Mutex::get_mut and RwLock::get_mut (#7569)
  • time: reduce the generated code size of Timeout<T>::poll (#7535)

Fixed

  • macros: fix hygiene issue in join! and try_join! (#7638)
  • net: fix copy/paste errors in udp peek methods (#7604)
  • process: fix error when runtime is shut down on nightly-2025-10-12 (#7672)
  • runtime: use release ordering in wake_by_ref() even if already woken (#7622)
  • sync: close the broadcast::Sender in broadcast::Sender::new() (#7629)
  • sync: fix implementation of unused RwLock::try_* methods (#7587)

Unstable

  • tokio: use cargo features instead of --cfg flags for taskdump and io_uring (#7655, #7621)
  • fs: support io_uring in fs::write (#7567)
  • fs: support io_uring with File::open() (#7617)
  • fs: support io_uring with OpenOptions (#7321)
  • macros: add local runtime flavor (#7375, #7597)

Documented

  • io: clarify the zero capacity case of AsyncRead::poll_read (#7580)
  • io: fix typos in the docs of AsyncFd readiness guards (#7583)
  • net: clarify socket gets closed on drop (#7526)
  • net: clarify the behavior of UCred::pid() on Cygwin (#7611)
  • net: clarify the supported platform of set_reuseport() and reuseport() (#7628)

... (truncated)

Commits
  • 556820f chore: prepare Tokio v1.48.0 (#7677)
  • fd1659a chore: prepare tokio-macros v2.6.0 (#7676)
  • 53e8aca ci: update nightly version to 2025-10-12 (#7670)
  • 9e5527d process: fix error when runtime is shut down on nightly-2025-10-12 (#7672)
  • 25a24de net: remove PollEvented noise from Debug formats (#7675)
  • c1fa25f task: clarify the behavior of several spawn_local methods (#7669)
  • e7e02fc fs: use FileOptions inside fs::File to support uring (#7617)
  • f7a7f62 ci: remove cargo-deny Unicode-DFS-2016 license exception config (#7619)
  • d1f1499 tokio: use cargo feature for taskdump support instead of cfg (#7655)
  • ad6f618 runtime: clarify the behavior of Handle::block_on (#7665)
  • Additional commits viewable in compare view

[![Dependabot compatibility score](https://dependabot-badges.githubapp.com/badges/compatibility_score?dependency-name=tokio&package-manager=cargo&previous-version=1.47.1&new-version=1.48.0)](https://docs.github.com/en/github/managing-security-vulnerabilities/about-dependabot-security-updates#about-compatibility-scores) Dependabot will resolve any conflicts with this PR as long as you don't alter it yourself. You can also trigger a rebase manually by commenting `@dependabot rebase`. [//]: # (dependabot-automerge-start) [//]: # (dependabot-automerge-end) ---
Dependabot commands and options
You can trigger Dependabot actions by commenting on this PR: - `@dependabot rebase` will rebase this PR - `@dependabot recreate` will recreate this PR, overwriting any edits that have been made to it - `@dependabot merge` will merge this PR after your CI passes on it - `@dependabot squash and merge` will squash and merge this PR after your CI passes on it - `@dependabot cancel merge` will cancel a previously requested merge and block automerging - `@dependabot reopen` will reopen this PR if it is closed - `@dependabot close` will close this PR and stop Dependabot recreating it. You can achieve the same result by closing it manually - `@dependabot show ignore conditions` will show all of the ignore conditions of the specified dependency - `@dependabot ignore this major version` will close this PR and stop Dependabot creating any more for this major version (unless you reopen the PR or upgrade to it yourself) - `@dependabot ignore this minor version` will close this PR and stop Dependabot creating any more for this minor version (unless you reopen the PR or upgrade to it yourself) - `@dependabot ignore this dependency` will close this PR and stop Dependabot creating any more for this dependency (unless you reopen the PR or upgrade to it yourself)
Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- Cargo.lock | 83 ++++++++---------------------------------------------- 1 file changed, 12 insertions(+), 71 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index 87b97b7c78..6a2a33238f 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -2,15 +2,6 @@ # It is not intended for manual editing. version = 4 -[[package]] -name = "addr2line" -version = "0.24.2" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "dfbe277e56a376000877090da837660b4427aad530e3028d44e0bffe4f89a1c1" -dependencies = [ - "gimli", -] - [[package]] name = "adler2" version = "2.0.1" @@ -1060,21 +1051,6 @@ dependencies = [ "tokio", ] -[[package]] -name = "backtrace" -version = "0.3.75" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "6806a6321ec58106fea15becdad98371e28d92ccbc7c8f1b3b6dd724fe8f1002" -dependencies = [ - "addr2line", - "cfg-if", - "libc", - "miniz_oxide", - "object", - "rustc-demangle", - "windows-targets 0.52.6", -] - [[package]] name = "base64" version = "0.21.7" @@ -1566,7 +1542,7 @@ version = "3.0.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "fde0e0ec90c9dfb3b4b1a0891a7dcd0e2bffde2f7efed5fe7c9bb00e5bfb915e" dependencies = [ - "windows-sys 0.59.0", + "windows-sys 0.48.0", ] [[package]] @@ -2927,7 +2903,7 @@ checksum = "0ce92ff622d6dadf7349484f42c93271a0d49b7cc4d466a936405bacbe10aa78" dependencies = [ "cfg-if", "rustix 1.0.8", - "windows-sys 0.59.0", + "windows-sys 0.52.0", ] [[package]] @@ -3186,12 +3162,6 @@ dependencies = [ "wasm-bindgen", ] -[[package]] -name = "gimli" -version = "0.31.1" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "07e28edb80900c19c28f1072f2e8aeca7fa06b23cd4169cefe1af5aa3260783f" - [[package]] name = "glob" version = "0.3.3" @@ -3561,7 +3531,7 @@ dependencies = [ "libc", "percent-encoding", "pin-project-lite", - "socket2 0.6.0", + "socket2 0.5.10", "tokio", "tower-service", "tracing", @@ -4049,17 +4019,6 @@ dependencies = [ "tokio", ] -[[package]] -name = "io-uring" -version = "0.7.10" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "046fa2d4d00aea763528b4950358d0ead425372445dc8ff86312b3c69ff7727b" -dependencies = [ - "bitflags", - "cfg-if", - "libc", -] - [[package]] name = "ipnet" version = "2.11.0" @@ -4846,15 +4805,6 @@ version = "0.4.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "830b246a0e5f20af87141b25c173cd1b609bd7779a4617d6ec582abaf90870f3" -[[package]] -name = "object" -version = "0.36.7" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "62948e14d923ea95ea2c7c86c71013138b66525b86bdc08d2dcc262bdb497b87" -dependencies = [ - "memchr", -] - [[package]] name = "object_store" version = "0.12.3" @@ -5661,7 +5611,7 @@ dependencies = [ "quinn-udp", "rustc-hash 2.1.1", "rustls 0.23.31", - "socket2 0.6.0", + "socket2 0.5.10", "thiserror 2.0.16", "tokio", "tracing", @@ -5698,7 +5648,7 @@ dependencies = [ "cfg_aliases", "libc", "once_cell", - "socket2 0.6.0", + "socket2 0.5.10", "tracing", "windows-sys 0.60.2", ] @@ -6135,12 +6085,6 @@ dependencies = [ "serde_json", ] -[[package]] -name = "rustc-demangle" -version = "0.1.26" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "56f7d92ca342cea22a06f2121d944b4fd82af56988c270852495420f961d4ace" - [[package]] name = "rustc-hash" version = "1.1.0" @@ -6175,7 +6119,7 @@ dependencies = [ "errno", "libc", "linux-raw-sys 0.4.15", - "windows-sys 0.59.0", + "windows-sys 0.52.0", ] [[package]] @@ -7093,7 +7037,7 @@ dependencies = [ "cfg-if", "libc", "psm", - "windows-sys 0.59.0", + "windows-sys 0.52.0", ] [[package]] @@ -7432,29 +7376,26 @@ checksum = "1f3ccbac311fea05f86f61904b462b55fb3df8837a366dfc601a0161d0532f20" [[package]] name = "tokio" -version = "1.47.1" +version = "1.48.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "89e49afdadebb872d3145a5638b59eb0691ea23e46ca484037cfab3b76b95038" +checksum = "ff360e02eab121e0bc37a2d3b4d4dc622e6eda3a8e5253d5435ecf5bd4c68408" dependencies = [ - "backtrace", "bytes", - "io-uring", "libc", "mio", "parking_lot", "pin-project-lite", "signal-hook-registry", - "slab", "socket2 0.6.0", "tokio-macros", - "windows-sys 0.59.0", + "windows-sys 0.61.0", ] [[package]] name = "tokio-macros" -version = "2.5.0" +version = "2.6.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "6e06d43f1345a3bcd39f6a56dbb7dcab2ba47e68e8ac134855e7e2bdbaf8cab8" +checksum = "af407857209536a95c8e56f8231ef2c2e2aff839b22e07a1ffcbc617e9db9fa5" dependencies = [ "proc-macro2", "quote", From 1244acd08ef56e5f15b0961c66a8492a6f6d83a2 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Sun, 19 Oct 2025 16:57:05 -0400 Subject: [PATCH 23/47] chore(deps): Bump backon from 1.5.2 to 1.6.0 (#1762) Bumps [backon](https://github.com/Xuanwo/backon) from 1.5.2 to 1.6.0.
Commits

[![Dependabot compatibility score](https://dependabot-badges.githubapp.com/badges/compatibility_score?dependency-name=backon&package-manager=cargo&previous-version=1.5.2&new-version=1.6.0)](https://docs.github.com/en/github/managing-security-vulnerabilities/about-dependabot-security-updates#about-compatibility-scores) Dependabot will resolve any conflicts with this PR as long as you don't alter it yourself. You can also trigger a rebase manually by commenting `@dependabot rebase`. [//]: # (dependabot-automerge-start) [//]: # (dependabot-automerge-end) ---
Dependabot commands and options
You can trigger Dependabot actions by commenting on this PR: - `@dependabot rebase` will rebase this PR - `@dependabot recreate` will recreate this PR, overwriting any edits that have been made to it - `@dependabot merge` will merge this PR after your CI passes on it - `@dependabot squash and merge` will squash and merge this PR after your CI passes on it - `@dependabot cancel merge` will cancel a previously requested merge and block automerging - `@dependabot reopen` will reopen this PR if it is closed - `@dependabot close` will close this PR and stop Dependabot recreating it. You can achieve the same result by closing it manually - `@dependabot show ignore conditions` will show all of the ignore conditions of the specified dependency - `@dependabot ignore this major version` will close this PR and stop Dependabot creating any more for this major version (unless you reopen the PR or upgrade to it yourself) - `@dependabot ignore this minor version` will close this PR and stop Dependabot creating any more for this minor version (unless you reopen the PR or upgrade to it yourself) - `@dependabot ignore this dependency` will close this PR and stop Dependabot creating any more for this dependency (unless you reopen the PR or upgrade to it yourself)
Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- Cargo.lock | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index 6a2a33238f..2adfb0a868 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -1042,9 +1042,9 @@ dependencies = [ [[package]] name = "backon" -version = "1.5.2" +version = "1.6.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "592277618714fbcecda9a02ba7a8781f319d26532a88553bbacc77ba5d2b3a8d" +checksum = "cffb0e931875b666fc4fcb20fee52e9bbd1ef836fd9e9e04ec21555f9f85f7ef" dependencies = [ "fastrand", "gloo-timers", From d33f3bb77ede1bf481bf71d9ddb45cb4cdcbd858 Mon Sep 17 00:00:00 2001 From: Amogh Jahagirdar Date: Mon, 20 Oct 2025 10:48:51 -0600 Subject: [PATCH 24/47] fix: global eq delete matching should apply to only strictly older files, and fix partition scoped matching to consider spec id (#1758) ## Which issue does this PR close? - Closes #1759. ## What changes are included in this PR? This changes scan planning of equality deletes so that: 1. we only match against eq deletes which are strictly older than the data file. Right now it looks like we incorrectly over-apply based on the seq number for global equality deletes. 2. Partition scoped deletes (both equality and position) are compared correctly by also factoring in the spec ID. It's not quite enough to compare just off the tuple, we should also compare based off the spec ID as well. ## Are these changes tested? Added unit tests which are scoped to testing delete index matching logic. If tests are not included in your PR, please explain why (for example, are they covered by existing tests)? --> --- crates/iceberg/src/delete_file_index.rs | 292 +++++++++++++++++++++++- 1 file changed, 283 insertions(+), 9 deletions(-) diff --git a/crates/iceberg/src/delete_file_index.rs b/crates/iceberg/src/delete_file_index.rs index d8f7a872e1..4f6fd28483 100644 --- a/crates/iceberg/src/delete_file_index.rs +++ b/crates/iceberg/src/delete_file_index.rs @@ -42,7 +42,7 @@ enum DeleteFileIndexState { #[derive(Debug)] struct PopulatedDeleteFileIndex { #[allow(dead_code)] - global_deletes: Vec>, + global_equality_deletes: Vec>, eq_deletes_by_partition: HashMap>>, pos_deletes_by_partition: HashMap>>, // TODO: do we need this? @@ -65,7 +65,8 @@ impl DeleteFileIndex { spawn({ let state = state.clone(); async move { - let delete_files = delete_file_stream.collect::>().await; + let delete_files: Vec = + delete_file_stream.collect::>().await; let populated_delete_file_index = PopulatedDeleteFileIndex::new(delete_files); @@ -114,7 +115,7 @@ impl PopulatedDeleteFileIndex { /// /// 1. The partition information is extracted from each delete file's manifest entry. /// 2. If the partition is empty and the delete file is not a positional delete, - /// it is added to the `global_deletes` vector + /// it is added to the `global_equality_deletes` vector /// 3. Otherwise, the delete file is added to one of two hash maps based on its content type. fn new(files: Vec) -> PopulatedDeleteFileIndex { let mut eq_deletes_by_partition: HashMap>> = @@ -122,7 +123,7 @@ impl PopulatedDeleteFileIndex { let mut pos_deletes_by_partition: HashMap>> = HashMap::default(); - let mut global_deletes: Vec> = vec![]; + let mut global_equality_deletes: Vec> = vec![]; files.into_iter().for_each(|ctx| { let arc_ctx = Arc::new(ctx); @@ -133,7 +134,7 @@ impl PopulatedDeleteFileIndex { if partition.fields().is_empty() { // TODO: confirm we're good to skip here if we encounter a pos del if arc_ctx.manifest_entry.content_type() != DataContentType::PositionDeletes { - global_deletes.push(arc_ctx); + global_equality_deletes.push(arc_ctx); return; } } @@ -153,7 +154,7 @@ impl PopulatedDeleteFileIndex { }); PopulatedDeleteFileIndex { - global_deletes, + global_equality_deletes, eq_deletes_by_partition, pos_deletes_by_partition, } @@ -167,12 +168,12 @@ impl PopulatedDeleteFileIndex { ) -> Vec { let mut results = vec![]; - self.global_deletes + self.global_equality_deletes .iter() - // filter that returns true if the provided delete file's sequence number is **greater than or equal to** `seq_num` + // filter that returns true if the provided delete file's sequence number is **greater than** `seq_num` .filter(|&delete| { seq_num - .map(|seq_num| delete.manifest_entry.sequence_number() >= Some(seq_num)) + .map(|seq_num| delete.manifest_entry.sequence_number() > Some(seq_num)) .unwrap_or_else(|| true) }) .for_each(|delete| results.push(delete.as_ref().into())); @@ -185,6 +186,7 @@ impl PopulatedDeleteFileIndex { seq_num .map(|seq_num| delete.manifest_entry.sequence_number() > Some(seq_num)) .unwrap_or_else(|| true) + && data_file.partition_spec_id == delete.partition_spec_id }) .for_each(|delete| results.push(delete.as_ref().into())); } @@ -201,6 +203,7 @@ impl PopulatedDeleteFileIndex { seq_num .map(|seq_num| delete.manifest_entry.sequence_number() >= Some(seq_num)) .unwrap_or_else(|| true) + && data_file.partition_spec_id == delete.partition_spec_id }) .for_each(|delete| results.push(delete.as_ref().into())); } @@ -208,3 +211,274 @@ impl PopulatedDeleteFileIndex { results } } + +#[cfg(test)] +mod tests { + use uuid::Uuid; + + use super::*; + use crate::spec::{ + DataContentType, DataFileBuilder, DataFileFormat, Literal, ManifestEntry, ManifestStatus, + Struct, + }; + + #[test] + fn test_delete_file_index_unpartitioned() { + let deletes: Vec = vec![ + build_added_manifest_entry(4, &build_unpartitioned_eq_delete()), + build_added_manifest_entry(6, &build_unpartitioned_eq_delete()), + build_added_manifest_entry(5, &build_unpartitioned_pos_delete()), + build_added_manifest_entry(6, &build_unpartitioned_pos_delete()), + ]; + + let delete_file_paths: Vec = deletes + .iter() + .map(|file| file.file_path().to_string()) + .collect(); + + let delete_contexts: Vec = deletes + .into_iter() + .map(|entry| DeleteFileContext { + manifest_entry: entry.into(), + partition_spec_id: 0, + }) + .collect(); + + let delete_file_index = PopulatedDeleteFileIndex::new(delete_contexts); + + let data_file = build_unpartitioned_data_file(); + + // All deletes apply to sequence 0 + let delete_files_to_apply_for_seq_0 = + delete_file_index.get_deletes_for_data_file(&data_file, Some(0)); + assert_eq!(delete_files_to_apply_for_seq_0.len(), 4); + + // All deletes apply to sequence 3 + let delete_files_to_apply_for_seq_3 = + delete_file_index.get_deletes_for_data_file(&data_file, Some(3)); + assert_eq!(delete_files_to_apply_for_seq_3.len(), 4); + + // Last 3 deletes apply to sequence 4 + let delete_files_to_apply_for_seq_4 = + delete_file_index.get_deletes_for_data_file(&data_file, Some(4)); + let actual_paths_to_apply_for_seq_4: Vec = delete_files_to_apply_for_seq_4 + .into_iter() + .map(|file| file.file_path) + .collect(); + + assert_eq!( + actual_paths_to_apply_for_seq_4, + delete_file_paths[delete_file_paths.len() - 3..] + ); + + // Last 3 deletes apply to sequence 5 + let delete_files_to_apply_for_seq_5 = + delete_file_index.get_deletes_for_data_file(&data_file, Some(5)); + let actual_paths_to_apply_for_seq_5: Vec = delete_files_to_apply_for_seq_5 + .into_iter() + .map(|file| file.file_path) + .collect(); + assert_eq!( + actual_paths_to_apply_for_seq_5, + delete_file_paths[delete_file_paths.len() - 3..] + ); + + // Only the last position delete applies to sequence 6 + let delete_files_to_apply_for_seq_6 = + delete_file_index.get_deletes_for_data_file(&data_file, Some(6)); + let actual_paths_to_apply_for_seq_6: Vec = delete_files_to_apply_for_seq_6 + .into_iter() + .map(|file| file.file_path) + .collect(); + assert_eq!( + actual_paths_to_apply_for_seq_6, + delete_file_paths[delete_file_paths.len() - 1..] + ); + + // The 2 global equality deletes should match against any partitioned file + let partitioned_file = + build_partitioned_data_file(&Struct::from_iter([Some(Literal::long(100))]), 1); + + let delete_files_to_apply_for_partitioned_file = + delete_file_index.get_deletes_for_data_file(&partitioned_file, Some(0)); + let actual_paths_to_apply_for_partitioned_file: Vec = + delete_files_to_apply_for_partitioned_file + .into_iter() + .map(|file| file.file_path) + .collect(); + assert_eq!( + actual_paths_to_apply_for_partitioned_file, + delete_file_paths[..2] + ); + } + + #[test] + fn test_delete_file_index_partitioned() { + let partition_one = Struct::from_iter([Some(Literal::long(100))]); + let spec_id = 1; + let deletes: Vec = vec![ + build_added_manifest_entry(4, &build_partitioned_eq_delete(&partition_one, spec_id)), + build_added_manifest_entry(6, &build_partitioned_eq_delete(&partition_one, spec_id)), + build_added_manifest_entry(5, &build_partitioned_pos_delete(&partition_one, spec_id)), + build_added_manifest_entry(6, &build_partitioned_pos_delete(&partition_one, spec_id)), + ]; + + let delete_file_paths: Vec = deletes + .iter() + .map(|file| file.file_path().to_string()) + .collect(); + + let delete_contexts: Vec = deletes + .into_iter() + .map(|entry| DeleteFileContext { + manifest_entry: entry.into(), + partition_spec_id: spec_id, + }) + .collect(); + + let delete_file_index = PopulatedDeleteFileIndex::new(delete_contexts); + + let partitioned_file = + build_partitioned_data_file(&Struct::from_iter([Some(Literal::long(100))]), spec_id); + + // All deletes apply to sequence 0 + let delete_files_to_apply_for_seq_0 = + delete_file_index.get_deletes_for_data_file(&partitioned_file, Some(0)); + assert_eq!(delete_files_to_apply_for_seq_0.len(), 4); + + // All deletes apply to sequence 3 + let delete_files_to_apply_for_seq_3 = + delete_file_index.get_deletes_for_data_file(&partitioned_file, Some(3)); + assert_eq!(delete_files_to_apply_for_seq_3.len(), 4); + + // Last 3 deletes apply to sequence 4 + let delete_files_to_apply_for_seq_4 = + delete_file_index.get_deletes_for_data_file(&partitioned_file, Some(4)); + let actual_paths_to_apply_for_seq_4: Vec = delete_files_to_apply_for_seq_4 + .into_iter() + .map(|file| file.file_path) + .collect(); + + assert_eq!( + actual_paths_to_apply_for_seq_4, + delete_file_paths[delete_file_paths.len() - 3..] + ); + + // Last 3 deletes apply to sequence 5 + let delete_files_to_apply_for_seq_5 = + delete_file_index.get_deletes_for_data_file(&partitioned_file, Some(5)); + let actual_paths_to_apply_for_seq_5: Vec = delete_files_to_apply_for_seq_5 + .into_iter() + .map(|file| file.file_path) + .collect(); + assert_eq!( + actual_paths_to_apply_for_seq_5, + delete_file_paths[delete_file_paths.len() - 3..] + ); + + // Only the last position delete applies to sequence 6 + let delete_files_to_apply_for_seq_6 = + delete_file_index.get_deletes_for_data_file(&partitioned_file, Some(6)); + let actual_paths_to_apply_for_seq_6: Vec = delete_files_to_apply_for_seq_6 + .into_iter() + .map(|file| file.file_path) + .collect(); + assert_eq!( + actual_paths_to_apply_for_seq_6, + delete_file_paths[delete_file_paths.len() - 1..] + ); + + // Data file with different partition tuples does not match any delete files + let partitioned_second_file = + build_partitioned_data_file(&Struct::from_iter([Some(Literal::long(200))]), 1); + let delete_files_to_apply_for_different_partition = + delete_file_index.get_deletes_for_data_file(&partitioned_second_file, Some(0)); + let actual_paths_to_apply_for_different_partition: Vec = + delete_files_to_apply_for_different_partition + .into_iter() + .map(|file| file.file_path) + .collect(); + assert!(actual_paths_to_apply_for_different_partition.is_empty()); + + // Data file with same tuple but different spec ID does not match any delete files + let partitioned_different_spec = build_partitioned_data_file(&partition_one, 2); + let delete_files_to_apply_for_different_spec = + delete_file_index.get_deletes_for_data_file(&partitioned_different_spec, Some(0)); + let actual_paths_to_apply_for_different_spec: Vec = + delete_files_to_apply_for_different_spec + .into_iter() + .map(|file| file.file_path) + .collect(); + assert!(actual_paths_to_apply_for_different_spec.is_empty()); + } + + fn build_unpartitioned_eq_delete() -> DataFile { + build_partitioned_eq_delete(&Struct::empty(), 0) + } + + fn build_partitioned_eq_delete(partition: &Struct, spec_id: i32) -> DataFile { + DataFileBuilder::default() + .file_path(format!("{}_equality_delete.parquet", Uuid::new_v4())) + .file_format(DataFileFormat::Parquet) + .content(DataContentType::EqualityDeletes) + .equality_ids(Some(vec![1])) + .record_count(1) + .partition(partition.clone()) + .partition_spec_id(spec_id) + .file_size_in_bytes(100) + .build() + .unwrap() + } + + fn build_unpartitioned_pos_delete() -> DataFile { + build_partitioned_pos_delete(&Struct::empty(), 0) + } + + fn build_partitioned_pos_delete(partition: &Struct, spec_id: i32) -> DataFile { + DataFileBuilder::default() + .file_path(format!("{}-pos-delete.parquet", Uuid::new_v4())) + .file_format(DataFileFormat::Parquet) + .content(DataContentType::PositionDeletes) + .record_count(1) + .referenced_data_file(Some("/some-data-file.parquet".to_string())) + .partition(partition.clone()) + .partition_spec_id(spec_id) + .file_size_in_bytes(100) + .build() + .unwrap() + } + + fn build_unpartitioned_data_file() -> DataFile { + DataFileBuilder::default() + .file_path(format!("{}-data.parquet", Uuid::new_v4())) + .file_format(DataFileFormat::Parquet) + .content(DataContentType::Data) + .record_count(100) + .partition(Struct::empty()) + .partition_spec_id(0) + .file_size_in_bytes(100) + .build() + .unwrap() + } + + fn build_partitioned_data_file(partition_value: &Struct, spec_id: i32) -> DataFile { + DataFileBuilder::default() + .file_path(format!("{}-data.parquet", Uuid::new_v4())) + .file_format(DataFileFormat::Parquet) + .content(DataContentType::Data) + .record_count(100) + .partition(partition_value.clone()) + .partition_spec_id(spec_id) + .file_size_in_bytes(100) + .build() + .unwrap() + } + + fn build_added_manifest_entry(data_seq_number: i64, file: &DataFile) -> ManifestEntry { + ManifestEntry::builder() + .status(ManifestStatus::Added) + .sequence_number(data_seq_number) + .data_file(file.clone()) + .build() + } +} From 777d9dc3746759229a0c0b6a13c30c5414efe7e5 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Tue, 21 Oct 2025 11:28:03 +0800 Subject: [PATCH 25/47] chore(deps): Bump apache/skywalking-eyes from 0.7.0 to 0.8.0 (#1760) Bumps [apache/skywalking-eyes](https://github.com/apache/skywalking-eyes) from 0.7.0 to 0.8.0.
Commits
  • 61275cc Add fsf-free and osi-approved options (#250)
  • d1a97df Update example workflow in the README for the Ruby scenario (#248)
  • 2ca7b4d fix: Gemfile / Gemspec parser to ignore commented dependencies (#249)
  • 454b8ea feat: add some compatibility licenses (#247)
  • 36f75de feat: Compatibility matrix: MIT & Ruby (#209)
  • 18cb362 GemfileLockResolver adjusted to exclude all specs when runtime dependencies a...
  • 5a19c65 Fixed a panic caused by nil dependency specs in Ruby Gemfile.lock resolver (#...
  • 0d9c4df fix: return error in license check, add MIT header (#206)
  • 3655e78 Ruby dependency license scanning support via Gemfile.lock. (#205)
  • 07a607f Bump golang.org/x/oauth2 from 0.5.0 to 0.27.0 (#204)
  • Additional commits viewable in compare view

[![Dependabot compatibility score](https://dependabot-badges.githubapp.com/badges/compatibility_score?dependency-name=apache/skywalking-eyes&package-manager=github_actions&previous-version=0.7.0&new-version=0.8.0)](https://docs.github.com/en/github/managing-security-vulnerabilities/about-dependabot-security-updates#about-compatibility-scores) Dependabot will resolve any conflicts with this PR as long as you don't alter it yourself. You can also trigger a rebase manually by commenting `@dependabot rebase`. [//]: # (dependabot-automerge-start) [//]: # (dependabot-automerge-end) ---
Dependabot commands and options
You can trigger Dependabot actions by commenting on this PR: - `@dependabot rebase` will rebase this PR - `@dependabot recreate` will recreate this PR, overwriting any edits that have been made to it - `@dependabot merge` will merge this PR after your CI passes on it - `@dependabot squash and merge` will squash and merge this PR after your CI passes on it - `@dependabot cancel merge` will cancel a previously requested merge and block automerging - `@dependabot reopen` will reopen this PR if it is closed - `@dependabot close` will close this PR and stop Dependabot recreating it. You can achieve the same result by closing it manually - `@dependabot show ignore conditions` will show all of the ignore conditions of the specified dependency - `@dependabot ignore this major version` will close this PR and stop Dependabot creating any more for this major version (unless you reopen the PR or upgrade to it yourself) - `@dependabot ignore this minor version` will close this PR and stop Dependabot creating any more for this minor version (unless you reopen the PR or upgrade to it yourself) - `@dependabot ignore this dependency` will close this PR and stop Dependabot creating any more for this dependency (unless you reopen the PR or upgrade to it yourself)
Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- .github/workflows/ci.yml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/.github/workflows/ci.yml b/.github/workflows/ci.yml index aa3f54d317..3207ae87b1 100644 --- a/.github/workflows/ci.yml +++ b/.github/workflows/ci.yml @@ -58,7 +58,7 @@ jobs: uses: ./.github/actions/setup-builder - name: Check License Header - uses: apache/skywalking-eyes/header@v0.7.0 + uses: apache/skywalking-eyes/header@v0.8.0 - name: Check toml format run: make check-toml From 4d20ff08d7c79abbe5bdc17d8e770f077b66dfbf Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Tue, 21 Oct 2025 12:58:50 +0800 Subject: [PATCH 26/47] chore(deps): Bump rust_decimal from 1.38.0 to 1.39.0 (#1761) MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Bumps [rust_decimal](https://github.com/paupino/rust-decimal) from 1.38.0 to 1.39.0.
Release notes

Sourced from rust_decimal's releases.

1.39.0

What's Changed

New Contributors

Full Changelog: https://github.com/paupino/rust-decimal/compare/1.38.0...1.39.0

Commits

[![Dependabot compatibility score](https://dependabot-badges.githubapp.com/badges/compatibility_score?dependency-name=rust_decimal&package-manager=cargo&previous-version=1.38.0&new-version=1.39.0)](https://docs.github.com/en/github/managing-security-vulnerabilities/about-dependabot-security-updates#about-compatibility-scores) Dependabot will resolve any conflicts with this PR as long as you don't alter it yourself. You can also trigger a rebase manually by commenting `@dependabot rebase`. [//]: # (dependabot-automerge-start) [//]: # (dependabot-automerge-end) ---
Dependabot commands and options
You can trigger Dependabot actions by commenting on this PR: - `@dependabot rebase` will rebase this PR - `@dependabot recreate` will recreate this PR, overwriting any edits that have been made to it - `@dependabot merge` will merge this PR after your CI passes on it - `@dependabot squash and merge` will squash and merge this PR after your CI passes on it - `@dependabot cancel merge` will cancel a previously requested merge and block automerging - `@dependabot reopen` will reopen this PR if it is closed - `@dependabot close` will close this PR and stop Dependabot recreating it. You can achieve the same result by closing it manually - `@dependabot show ignore conditions` will show all of the ignore conditions of the specified dependency - `@dependabot ignore this major version` will close this PR and stop Dependabot creating any more for this major version (unless you reopen the PR or upgrade to it yourself) - `@dependabot ignore this minor version` will close this PR and stop Dependabot creating any more for this minor version (unless you reopen the PR or upgrade to it yourself) - `@dependabot ignore this dependency` will close this PR and stop Dependabot creating any more for this dependency (unless you reopen the PR or upgrade to it yourself)
Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> Co-authored-by: Renjie Liu --- Cargo.lock | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index 2adfb0a868..b048c3db56 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -6070,9 +6070,9 @@ dependencies = [ [[package]] name = "rust_decimal" -version = "1.38.0" +version = "1.39.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c8975fc98059f365204d635119cf9c5a60ae67b841ed49b5422a9a7e56cdfac0" +checksum = "35affe401787a9bd846712274d97654355d21b2a2c092a3139aabe31e9022282" dependencies = [ "arrayvec", "borsh", From b3b5afed126ee5962dc3a4ca23cd735694449df8 Mon Sep 17 00:00:00 2001 From: Florian Valeye Date: Tue, 21 Oct 2025 11:53:58 +0200 Subject: [PATCH 27/47] feat(datafusion): implement the project node to add the partition columns (#1602) ## Which issue does this PR close? - Closes #1542 ## What changes are included in this PR? Implement a physical execution plan node that projects Iceberg partition columns from source data, supporting nested fields and all Iceberg transforms. ## Are these changes tested? Yes, with unit tests --------- Signed-off-by: Florian Valeye --- crates/iceberg/src/arrow/mod.rs | 3 +- .../src/arrow/record_batch_projector.rs | 71 ++- crates/iceberg/src/transform/mod.rs | 4 +- .../datafusion/src/physical_plan/mod.rs | 2 + .../datafusion/src/physical_plan/project.rs | 488 ++++++++++++++++++ 5 files changed, 563 insertions(+), 5 deletions(-) create mode 100644 crates/integrations/datafusion/src/physical_plan/project.rs diff --git a/crates/iceberg/src/arrow/mod.rs b/crates/iceberg/src/arrow/mod.rs index d32cbeb8f9..28116a4b5e 100644 --- a/crates/iceberg/src/arrow/mod.rs +++ b/crates/iceberg/src/arrow/mod.rs @@ -28,7 +28,8 @@ pub mod delete_file_loader; pub(crate) mod delete_filter; mod reader; -pub(crate) mod record_batch_projector; +/// RecordBatch projection utilities +pub mod record_batch_projector; pub(crate) mod record_batch_transformer; mod value; diff --git a/crates/iceberg/src/arrow/record_batch_projector.rs b/crates/iceberg/src/arrow/record_batch_projector.rs index 7ca28c25c7..45de0212e8 100644 --- a/crates/iceberg/src/arrow/record_batch_projector.rs +++ b/crates/iceberg/src/arrow/record_batch_projector.rs @@ -20,13 +20,16 @@ use std::sync::Arc; use arrow_array::{ArrayRef, RecordBatch, StructArray, make_array}; use arrow_buffer::NullBuffer; use arrow_schema::{DataType, Field, FieldRef, Fields, Schema, SchemaRef}; +use parquet::arrow::PARQUET_FIELD_ID_META_KEY; +use crate::arrow::schema::schema_to_arrow_schema; use crate::error::Result; +use crate::spec::Schema as IcebergSchema; use crate::{Error, ErrorKind}; /// Help to project specific field from `RecordBatch`` according to the fields id. -#[derive(Clone, Debug)] -pub(crate) struct RecordBatchProjector { +#[derive(Clone, Debug, PartialEq, Eq)] +pub struct RecordBatchProjector { // A vector of vectors, where each inner vector represents the index path to access a specific field in a nested structure. // E.g. [[0], [1, 2]] means the first field is accessed directly from the first column, // while the second field is accessed from the second column and then from its third subcolumn (second column must be a struct column). @@ -77,6 +80,46 @@ impl RecordBatchProjector { }) } + /// Create RecordBatchProjector using Iceberg schema. + /// + /// This constructor converts the Iceberg schema to Arrow schema with field ID metadata, + /// then uses the standard field ID lookup for projection. + /// + /// # Arguments + /// * `iceberg_schema` - The Iceberg schema for field ID mapping + /// * `target_field_ids` - The field IDs to project + pub fn from_iceberg_schema( + iceberg_schema: Arc, + target_field_ids: &[i32], + ) -> Result { + let arrow_schema_with_ids = Arc::new(schema_to_arrow_schema(&iceberg_schema)?); + + let field_id_fetch_func = |field: &Field| -> Result> { + if let Some(value) = field.metadata().get(PARQUET_FIELD_ID_META_KEY) { + let field_id = value.parse::().map_err(|e| { + Error::new( + ErrorKind::DataInvalid, + "Failed to parse field id".to_string(), + ) + .with_context("value", value) + .with_source(e) + })?; + Ok(Some(field_id as i64)) + } else { + Ok(None) + } + }; + + let searchable_field_func = |_field: &Field| -> bool { true }; + + Self::new( + arrow_schema_with_ids, + target_field_ids, + field_id_fetch_func, + searchable_field_func, + ) + } + fn fetch_field_index( fields: &Fields, index_vec: &mut Vec, @@ -129,7 +172,7 @@ impl RecordBatchProjector { } /// Do projection with columns - pub(crate) fn project_column(&self, batch: &[ArrayRef]) -> Result> { + pub fn project_column(&self, batch: &[ArrayRef]) -> Result> { self.field_indices .iter() .map(|index_vec| Self::get_column_by_field_index(batch, index_vec)) @@ -166,6 +209,7 @@ mod test { use arrow_schema::{DataType, Field, Fields, Schema}; use crate::arrow::record_batch_projector::RecordBatchProjector; + use crate::spec::{NestedField, PrimitiveType, Schema as IcebergSchema, Type}; use crate::{Error, ErrorKind}; #[test] @@ -293,4 +337,25 @@ mod test { RecordBatchProjector::new(schema.clone(), &[3], field_id_fetch_func, |_| true); assert!(projector.is_ok()); } + + #[test] + fn test_from_iceberg_schema() { + let iceberg_schema = IcebergSchema::builder() + .with_schema_id(0) + .with_fields(vec![ + NestedField::required(1, "id", Type::Primitive(PrimitiveType::Int)).into(), + NestedField::required(2, "name", Type::Primitive(PrimitiveType::String)).into(), + NestedField::optional(3, "age", Type::Primitive(PrimitiveType::Int)).into(), + ]) + .build() + .unwrap(); + + let projector = + RecordBatchProjector::from_iceberg_schema(Arc::new(iceberg_schema), &[1, 3]).unwrap(); + + assert_eq!(projector.field_indices.len(), 2); + assert_eq!(projector.projected_schema_ref().fields().len(), 2); + assert_eq!(projector.projected_schema_ref().field(0).name(), "id"); + assert_eq!(projector.projected_schema_ref().field(1).name(), "age"); + } } diff --git a/crates/iceberg/src/transform/mod.rs b/crates/iceberg/src/transform/mod.rs index 4cc0d1fe8f..809d2dafe0 100644 --- a/crates/iceberg/src/transform/mod.rs +++ b/crates/iceberg/src/transform/mod.rs @@ -17,6 +17,8 @@ //! Transform function used to compute partition values. +use std::fmt::Debug; + use arrow_array::ArrayRef; use crate::spec::{Datum, Transform}; @@ -29,7 +31,7 @@ mod truncate; mod void; /// TransformFunction is a trait that defines the interface for all transform functions. -pub trait TransformFunction: Send + Sync { +pub trait TransformFunction: Send + Sync + Debug { /// transform will take an input array and transform it into a new array. /// The implementation of this function will need to check and downcast the input to specific /// type. diff --git a/crates/integrations/datafusion/src/physical_plan/mod.rs b/crates/integrations/datafusion/src/physical_plan/mod.rs index fcfd11a453..ce923b8662 100644 --- a/crates/integrations/datafusion/src/physical_plan/mod.rs +++ b/crates/integrations/datafusion/src/physical_plan/mod.rs @@ -18,9 +18,11 @@ pub(crate) mod commit; pub(crate) mod expr_to_predicate; pub(crate) mod metadata_scan; +pub(crate) mod project; pub(crate) mod scan; pub(crate) mod write; pub(crate) const DATA_FILES_COL_NAME: &str = "data_files"; +pub use project::project_with_partition; pub use scan::IcebergTableScan; diff --git a/crates/integrations/datafusion/src/physical_plan/project.rs b/crates/integrations/datafusion/src/physical_plan/project.rs new file mode 100644 index 0000000000..4bfe8192b0 --- /dev/null +++ b/crates/integrations/datafusion/src/physical_plan/project.rs @@ -0,0 +1,488 @@ +// 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. + +//! Partition value projection for Iceberg tables. + +use std::sync::Arc; + +use datafusion::arrow::array::{ArrayRef, RecordBatch, StructArray}; +use datafusion::arrow::datatypes::{DataType, Schema as ArrowSchema}; +use datafusion::common::Result as DFResult; +use datafusion::error::DataFusionError; +use datafusion::physical_expr::PhysicalExpr; +use datafusion::physical_expr::expressions::Column; +use datafusion::physical_plan::projection::ProjectionExec; +use datafusion::physical_plan::{ColumnarValue, ExecutionPlan}; +use iceberg::arrow::record_batch_projector::RecordBatchProjector; +use iceberg::spec::{PartitionSpec, Schema}; +use iceberg::table::Table; +use iceberg::transform::BoxedTransformFunction; + +use crate::to_datafusion_error; + +/// Column name for the combined partition values struct +const PARTITION_VALUES_COLUMN: &str = "_partition"; + +/// Extends an ExecutionPlan with partition value calculations for Iceberg tables. +/// +/// This function takes an input ExecutionPlan and extends it with an additional column +/// containing calculated partition values based on the table's partition specification. +/// For unpartitioned tables, returns the original plan unchanged. +/// +/// # Arguments +/// * `input` - The input ExecutionPlan to extend +/// * `table` - The Iceberg table with partition specification +/// +/// # Returns +/// * `Ok(Arc)` - Extended plan with partition values column +/// * `Err` - If partition spec is not found or transformation fails +pub fn project_with_partition( + input: Arc, + table: &Table, +) -> DFResult> { + let metadata = table.metadata(); + let partition_spec = metadata.default_partition_spec(); + let table_schema = metadata.current_schema(); + + if partition_spec.is_unpartitioned() { + return Ok(input); + } + + let input_schema = input.schema(); + // TODO: Validate that input_schema matches the Iceberg table schema. + // See: https://github.com/apache/iceberg-rust/issues/1752 + let partition_type = build_partition_type(partition_spec, table_schema.as_ref())?; + let calculator = PartitionValueCalculator::new( + partition_spec.as_ref().clone(), + table_schema.as_ref().clone(), + partition_type, + )?; + + let mut projection_exprs: Vec<(Arc, String)> = + Vec::with_capacity(input_schema.fields().len() + 1); + + for (index, field) in input_schema.fields().iter().enumerate() { + let column_expr = Arc::new(Column::new(field.name(), index)); + projection_exprs.push((column_expr, field.name().clone())); + } + + let partition_expr = Arc::new(PartitionExpr::new(calculator)); + projection_exprs.push((partition_expr, PARTITION_VALUES_COLUMN.to_string())); + + let projection = ProjectionExec::try_new(projection_exprs, input)?; + Ok(Arc::new(projection)) +} + +/// PhysicalExpr implementation for partition value calculation +#[derive(Debug, Clone)] +struct PartitionExpr { + calculator: Arc, +} + +impl PartitionExpr { + fn new(calculator: PartitionValueCalculator) -> Self { + Self { + calculator: Arc::new(calculator), + } + } +} + +// Manual PartialEq/Eq implementations for pointer-based equality +// (two PartitionExpr are equal if they share the same calculator instance) +impl PartialEq for PartitionExpr { + fn eq(&self, other: &Self) -> bool { + Arc::ptr_eq(&self.calculator, &other.calculator) + } +} + +impl Eq for PartitionExpr {} + +impl PhysicalExpr for PartitionExpr { + fn as_any(&self) -> &dyn std::any::Any { + self + } + + fn data_type(&self, _input_schema: &ArrowSchema) -> DFResult { + Ok(self.calculator.partition_type.clone()) + } + + fn nullable(&self, _input_schema: &ArrowSchema) -> DFResult { + Ok(false) + } + + fn evaluate(&self, batch: &RecordBatch) -> DFResult { + let array = self.calculator.calculate(batch)?; + Ok(ColumnarValue::Array(array)) + } + + fn children(&self) -> Vec<&Arc> { + vec![] + } + + fn with_new_children( + self: Arc, + _children: Vec>, + ) -> DFResult> { + Ok(self) + } + + fn fmt_sql(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + let field_names: Vec = self + .calculator + .partition_spec + .fields() + .iter() + .map(|pf| format!("{}({})", pf.transform, pf.name)) + .collect(); + write!(f, "iceberg_partition_values[{}]", field_names.join(", ")) + } +} + +impl std::fmt::Display for PartitionExpr { + fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + let field_names: Vec<&str> = self + .calculator + .partition_spec + .fields() + .iter() + .map(|pf| pf.name.as_str()) + .collect(); + write!(f, "iceberg_partition_values({})", field_names.join(", ")) + } +} + +impl std::hash::Hash for PartitionExpr { + fn hash(&self, state: &mut H) { + // Two PartitionExpr are equal if they share the same calculator Arc + Arc::as_ptr(&self.calculator).hash(state); + } +} + +/// Calculator for partition values in Iceberg tables +#[derive(Debug)] +struct PartitionValueCalculator { + partition_spec: PartitionSpec, + partition_type: DataType, + projector: RecordBatchProjector, + transform_functions: Vec, +} + +impl PartitionValueCalculator { + fn new( + partition_spec: PartitionSpec, + table_schema: Schema, + partition_type: DataType, + ) -> DFResult { + if partition_spec.is_unpartitioned() { + return Err(DataFusionError::Internal( + "Cannot create partition calculator for unpartitioned table".to_string(), + )); + } + + let transform_functions: Result, _> = partition_spec + .fields() + .iter() + .map(|pf| iceberg::transform::create_transform_function(&pf.transform)) + .collect(); + + let transform_functions = transform_functions.map_err(to_datafusion_error)?; + + let source_field_ids: Vec = partition_spec + .fields() + .iter() + .map(|pf| pf.source_id) + .collect(); + + let projector = RecordBatchProjector::from_iceberg_schema( + Arc::new(table_schema.clone()), + &source_field_ids, + ) + .map_err(to_datafusion_error)?; + + Ok(Self { + partition_spec, + partition_type, + projector, + transform_functions, + }) + } + + fn calculate(&self, batch: &RecordBatch) -> DFResult { + let source_columns = self + .projector + .project_column(batch.columns()) + .map_err(to_datafusion_error)?; + + let expected_struct_fields = match &self.partition_type { + DataType::Struct(fields) => fields.clone(), + _ => { + return Err(DataFusionError::Internal( + "Expected partition type must be a struct".to_string(), + )); + } + }; + + let mut partition_values = Vec::with_capacity(self.partition_spec.fields().len()); + + for (source_column, transform_fn) in source_columns.iter().zip(&self.transform_functions) { + let partition_value = transform_fn + .transform(source_column.clone()) + .map_err(to_datafusion_error)?; + + partition_values.push(partition_value); + } + + let struct_array = StructArray::try_new(expected_struct_fields, partition_values, None) + .map_err(|e| DataFusionError::ArrowError(e, None))?; + + Ok(Arc::new(struct_array)) + } +} + +fn build_partition_type( + partition_spec: &PartitionSpec, + table_schema: &Schema, +) -> DFResult { + let partition_struct_type = partition_spec + .partition_type(table_schema) + .map_err(to_datafusion_error)?; + + iceberg::arrow::type_to_arrow_type(&iceberg::spec::Type::Struct(partition_struct_type)) + .map_err(to_datafusion_error) +} + +#[cfg(test)] +mod tests { + use datafusion::arrow::array::Int32Array; + use datafusion::arrow::datatypes::{Field, Fields}; + use datafusion::physical_plan::empty::EmptyExec; + use iceberg::spec::{NestedField, PrimitiveType, StructType, Transform, Type}; + + use super::*; + + #[test] + fn test_partition_calculator_basic() { + let table_schema = Schema::builder() + .with_schema_id(0) + .with_fields(vec![ + NestedField::required(1, "id", Type::Primitive(PrimitiveType::Int)).into(), + NestedField::required(2, "name", Type::Primitive(PrimitiveType::String)).into(), + ]) + .build() + .unwrap(); + + let partition_spec = iceberg::spec::PartitionSpec::builder(Arc::new(table_schema.clone())) + .add_partition_field("id", "id_partition", Transform::Identity) + .unwrap() + .build() + .unwrap(); + + let _arrow_schema = Arc::new(ArrowSchema::new(vec![ + Field::new("id", DataType::Int32, false), + Field::new("name", DataType::Utf8, false), + ])); + + let partition_type = build_partition_type(&partition_spec, &table_schema).unwrap(); + let calculator = PartitionValueCalculator::new( + partition_spec.clone(), + table_schema, + partition_type.clone(), + ) + .unwrap(); + + assert_eq!(calculator.partition_type, partition_type); + } + + #[test] + fn test_partition_expr_with_projection() { + let table_schema = Schema::builder() + .with_schema_id(0) + .with_fields(vec![ + NestedField::required(1, "id", Type::Primitive(PrimitiveType::Int)).into(), + NestedField::required(2, "name", Type::Primitive(PrimitiveType::String)).into(), + ]) + .build() + .unwrap(); + + let partition_spec = iceberg::spec::PartitionSpec::builder(Arc::new(table_schema.clone())) + .add_partition_field("id", "id_partition", Transform::Identity) + .unwrap() + .build() + .unwrap(); + + let arrow_schema = Arc::new(ArrowSchema::new(vec![ + Field::new("id", DataType::Int32, false), + Field::new("name", DataType::Utf8, false), + ])); + + let input = Arc::new(EmptyExec::new(arrow_schema.clone())); + + let partition_type = build_partition_type(&partition_spec, &table_schema).unwrap(); + let calculator = + PartitionValueCalculator::new(partition_spec, table_schema, partition_type).unwrap(); + + let mut projection_exprs: Vec<(Arc, String)> = + Vec::with_capacity(arrow_schema.fields().len() + 1); + for (i, field) in arrow_schema.fields().iter().enumerate() { + let column_expr = Arc::new(Column::new(field.name(), i)); + projection_exprs.push((column_expr, field.name().clone())); + } + + let partition_expr = Arc::new(PartitionExpr::new(calculator)); + projection_exprs.push((partition_expr, PARTITION_VALUES_COLUMN.to_string())); + + let projection = ProjectionExec::try_new(projection_exprs, input).unwrap(); + let result = Arc::new(projection); + + assert_eq!(result.schema().fields().len(), 3); + assert_eq!(result.schema().field(0).name(), "id"); + assert_eq!(result.schema().field(1).name(), "name"); + assert_eq!(result.schema().field(2).name(), "_partition"); + } + + #[test] + fn test_partition_expr_evaluate() { + let table_schema = Schema::builder() + .with_schema_id(0) + .with_fields(vec![ + NestedField::required(1, "id", Type::Primitive(PrimitiveType::Int)).into(), + NestedField::required(2, "data", Type::Primitive(PrimitiveType::String)).into(), + ]) + .build() + .unwrap(); + + let partition_spec = iceberg::spec::PartitionSpec::builder(Arc::new(table_schema.clone())) + .add_partition_field("id", "id_partition", Transform::Identity) + .unwrap() + .build() + .unwrap(); + + let arrow_schema = Arc::new(ArrowSchema::new(vec![ + Field::new("id", DataType::Int32, false), + Field::new("data", DataType::Utf8, false), + ])); + + let batch = RecordBatch::try_new(arrow_schema.clone(), vec![ + Arc::new(Int32Array::from(vec![10, 20, 30])), + Arc::new(datafusion::arrow::array::StringArray::from(vec![ + "a", "b", "c", + ])), + ]) + .unwrap(); + + let partition_type = build_partition_type(&partition_spec, &table_schema).unwrap(); + let calculator = + PartitionValueCalculator::new(partition_spec, table_schema, partition_type.clone()) + .unwrap(); + let expr = PartitionExpr::new(calculator); + + assert_eq!(expr.data_type(&arrow_schema).unwrap(), partition_type); + assert!(!expr.nullable(&arrow_schema).unwrap()); + + let result = expr.evaluate(&batch).unwrap(); + match result { + ColumnarValue::Array(array) => { + let struct_array = array.as_any().downcast_ref::().unwrap(); + let id_partition = struct_array + .column_by_name("id_partition") + .unwrap() + .as_any() + .downcast_ref::() + .unwrap(); + assert_eq!(id_partition.value(0), 10); + assert_eq!(id_partition.value(1), 20); + assert_eq!(id_partition.value(2), 30); + } + _ => panic!("Expected array result"), + } + } + + #[test] + fn test_nested_partition() { + let address_struct = StructType::new(vec![ + NestedField::required(3, "street", Type::Primitive(PrimitiveType::String)).into(), + NestedField::required(4, "city", Type::Primitive(PrimitiveType::String)).into(), + ]); + + let table_schema = Schema::builder() + .with_schema_id(0) + .with_fields(vec![ + NestedField::required(1, "id", Type::Primitive(PrimitiveType::Int)).into(), + NestedField::required(2, "address", Type::Struct(address_struct)).into(), + ]) + .build() + .unwrap(); + + let partition_spec = iceberg::spec::PartitionSpec::builder(Arc::new(table_schema.clone())) + .add_partition_field("address.city", "city_partition", Transform::Identity) + .unwrap() + .build() + .unwrap(); + + let struct_fields = Fields::from(vec![ + Field::new("street", DataType::Utf8, false), + Field::new("city", DataType::Utf8, false), + ]); + + let arrow_schema = Arc::new(ArrowSchema::new(vec![ + Field::new("id", DataType::Int32, false), + Field::new("address", DataType::Struct(struct_fields), false), + ])); + + let street_array = Arc::new(datafusion::arrow::array::StringArray::from(vec![ + "123 Main St", + "456 Oak Ave", + ])); + let city_array = Arc::new(datafusion::arrow::array::StringArray::from(vec![ + "New York", + "Los Angeles", + ])); + + let struct_array = StructArray::from(vec![ + ( + Arc::new(Field::new("street", DataType::Utf8, false)), + street_array as ArrayRef, + ), + ( + Arc::new(Field::new("city", DataType::Utf8, false)), + city_array as ArrayRef, + ), + ]); + + let batch = RecordBatch::try_new(arrow_schema.clone(), vec![ + Arc::new(Int32Array::from(vec![1, 2])), + Arc::new(struct_array), + ]) + .unwrap(); + + let partition_type = build_partition_type(&partition_spec, &table_schema).unwrap(); + let calculator = + PartitionValueCalculator::new(partition_spec, table_schema, partition_type).unwrap(); + let array = calculator.calculate(&batch).unwrap(); + + let struct_array = array.as_any().downcast_ref::().unwrap(); + let city_partition = struct_array + .column_by_name("city_partition") + .unwrap() + .as_any() + .downcast_ref::() + .unwrap(); + + assert_eq!(city_partition.value(0), "New York"); + assert_eq!(city_partition.value(1), "Los Angeles"); + } +} From 3aa7ade28e6e56304700c163e1cbfdd8b0069921 Mon Sep 17 00:00:00 2001 From: Raminder Singh Date: Wed, 22 Oct 2025 15:03:44 +0530 Subject: [PATCH 28/47] fix: snapshot was producing empty summary (#1767) ## Which issue does this PR close? - Closes #. There was no open issue for this. I noticed this while trying out iceberg-rust. ## What changes are included in this PR? The first line in the method `SnapshotProducer::write_added_manifest` (`let added_data_files = std::mem::take(&mut self.added_data_files);`) sets the `self.added_data_files` to an empty vec. `SnapshotProducer::write_added_manifest` is called via the call chain `SnapshotProducer::commit` -> `SnapshotProducer::manifest_file` -> `SnapshotProducer::write_added_manifest`). Hence, if `SnapshotProducer::summary` is called after calling `SnapshotProducer::manifest_file`, the summary produced was empty due to empty `self.added_data_files`. This PR rearranges the code in `SnapshotProducer::commit` to make sure the produced summary is not empty. ## Are these changes tested? No new tests have been added for this as there were no previous tests. --- crates/iceberg/src/transaction/snapshot.rs | 23 ++++++++++++---------- 1 file changed, 13 insertions(+), 10 deletions(-) diff --git a/crates/iceberg/src/transaction/snapshot.rs b/crates/iceberg/src/transaction/snapshot.rs index a03b8dc490..93dd819d5c 100644 --- a/crates/iceberg/src/transaction/snapshot.rs +++ b/crates/iceberg/src/transaction/snapshot.rs @@ -380,17 +380,8 @@ impl<'a> SnapshotProducer<'a> { snapshot_produce_operation: OP, process: MP, ) -> Result { - let new_manifests = self - .manifest_file(&snapshot_produce_operation, &process) - .await?; - let next_seq_num = self.table.metadata().next_sequence_number(); - - let summary = self.summary(&snapshot_produce_operation).map_err(|err| { - Error::new(ErrorKind::Unexpected, "Failed to create snapshot summary.").with_source(err) - })?; - let manifest_list_path = self.generate_manifest_list_file_path(0); - + let next_seq_num = self.table.metadata().next_sequence_number(); let mut manifest_list_writer = match self.table.metadata().format_version() { FormatVersion::V1 => ManifestListWriter::v1( self.table @@ -408,6 +399,18 @@ impl<'a> SnapshotProducer<'a> { next_seq_num, ), }; + + // Calling self.summary() before self.manifest_file() is important because self.added_data_files + // will be set to an empty vec after self.manifest_file() returns, resulting in an empty summary + // being generated. + let summary = self.summary(&snapshot_produce_operation).map_err(|err| { + Error::new(ErrorKind::Unexpected, "Failed to create snapshot summary.").with_source(err) + })?; + + let new_manifests = self + .manifest_file(&snapshot_produce_operation, &process) + .await?; + manifest_list_writer.add_manifests(new_manifests.into_iter())?; manifest_list_writer.close().await?; From 915778ebb3245f611ddb0b16076bc7c573a3ea56 Mon Sep 17 00:00:00 2001 From: Shawn Chang Date: Wed, 22 Oct 2025 02:50:20 -0700 Subject: [PATCH 29/47] docs: Add examples for PartitioningWriter (#1754) ## Which issue does this PR close? - Closes #1745 ## What changes are included in this PR? - As title, added examples and documentation ## Are these changes tested? --- crates/iceberg/src/writer/mod.rs | 143 +++++++++++++++++- crates/iceberg/src/writer/partitioning/mod.rs | 6 + 2 files changed, 147 insertions(+), 2 deletions(-) diff --git a/crates/iceberg/src/writer/mod.rs b/crates/iceberg/src/writer/mod.rs index 1da3fa6790..a7892d49e1 100644 --- a/crates/iceberg/src/writer/mod.rs +++ b/crates/iceberg/src/writer/mod.rs @@ -242,11 +242,150 @@ //! Ok(()) //! } //! ``` +//! +//! # Adding Partitioning to Data File Writers +//! +//! You can wrap a `DataFileWriter` with partitioning writers to handle partitioned tables. +//! Iceberg provides two partitioning strategies: +//! +//! ## FanoutWriter - For Unsorted Data +//! +//! Wraps the data file writer to handle unsorted data by maintaining multiple active writers. +//! Use this when your data is not pre-sorted by partition key. Writes to different partitions +//! can happen in any order, even interleaved. +//! +//! ```rust, no_run +//! # // Same setup as the simple example above... +//! # use iceberg::memory::{MEMORY_CATALOG_WAREHOUSE, MemoryCatalogBuilder}; +//! # use iceberg::writer::file_writer::rolling_writer::RollingFileWriterBuilder; +//! # use iceberg::{Catalog, CatalogBuilder, Result, TableIdent}; +//! # use iceberg::writer::base_writer::data_file_writer::DataFileWriterBuilder; +//! # use iceberg::writer::file_writer::ParquetWriterBuilder; +//! # use iceberg::writer::file_writer::location_generator::{ +//! # DefaultFileNameGenerator, DefaultLocationGenerator, +//! # }; +//! # use parquet::file::properties::WriterProperties; +//! # use std::collections::HashMap; +//! # #[tokio::main] +//! # async fn main() -> Result<()> { +//! # let catalog = MemoryCatalogBuilder::default() +//! # .load("memory", HashMap::from([(MEMORY_CATALOG_WAREHOUSE.to_string(), "file:///path/to/warehouse".to_string())])) +//! # .await?; +//! # let table = catalog.load_table(&TableIdent::from_strs(["hello", "world"])?).await?; +//! # 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()); +//! # let rolling_writer_builder = RollingFileWriterBuilder::new_with_default_file_size( +//! # parquet_writer_builder, table.file_io().clone(), location_generator, file_name_generator); +//! # let data_file_writer_builder = DataFileWriterBuilder::new(rolling_writer_builder); +//! +//! // Wrap the data file writer with FanoutWriter for partitioning +//! use iceberg::writer::partitioning::fanout_writer::FanoutWriter; +//! use iceberg::writer::partitioning::PartitioningWriter; +//! use iceberg::spec::{Literal, PartitionKey, Struct}; +//! +//! let mut fanout_writer = FanoutWriter::new(data_file_writer_builder); +//! +//! // Create partition keys for different regions +//! let schema = table.metadata().current_schema().clone(); +//! let partition_spec = table.metadata().default_partition_spec().as_ref().clone(); +//! +//! let partition_key_us = PartitionKey::new( +//! partition_spec.clone(), +//! schema.clone(), +//! Struct::from_iter([Some(Literal::string("US"))]), +//! ); +//! +//! let partition_key_eu = PartitionKey::new( +//! partition_spec.clone(), +//! schema.clone(), +//! Struct::from_iter([Some(Literal::string("EU"))]), +//! ); +//! +//! // Write to different partitions in any order - can interleave partition writes +//! // fanout_writer.write(partition_key_us.clone(), batch_us1).await?; +//! // fanout_writer.write(partition_key_eu.clone(), batch_eu1).await?; +//! // fanout_writer.write(partition_key_us.clone(), batch_us2).await?; // Back to US - OK! +//! // fanout_writer.write(partition_key_eu.clone(), batch_eu2).await?; // Back to EU - OK! +//! +//! let data_files = fanout_writer.close().await?; +//! # Ok(()) +//! # } +//! ``` +//! +//! ## ClusteredWriter - For Sorted Data +//! +//! Wraps the data file writer for pre-sorted data. More memory efficient as it maintains +//! only one active writer at a time, but requires input sorted by partition key. +//! +//! ```rust, no_run +//! # // Same setup as the simple example above... +//! # use iceberg::memory::{MEMORY_CATALOG_WAREHOUSE, MemoryCatalogBuilder}; +//! # use iceberg::writer::file_writer::rolling_writer::RollingFileWriterBuilder; +//! # use iceberg::{Catalog, CatalogBuilder, Result, TableIdent}; +//! # use iceberg::writer::base_writer::data_file_writer::DataFileWriterBuilder; +//! # use iceberg::writer::file_writer::ParquetWriterBuilder; +//! # use iceberg::writer::file_writer::location_generator::{ +//! # DefaultFileNameGenerator, DefaultLocationGenerator, +//! # }; +//! # use parquet::file::properties::WriterProperties; +//! # use std::collections::HashMap; +//! # #[tokio::main] +//! # async fn main() -> Result<()> { +//! # let catalog = MemoryCatalogBuilder::default() +//! # .load("memory", HashMap::from([(MEMORY_CATALOG_WAREHOUSE.to_string(), "file:///path/to/warehouse".to_string())])) +//! # .await?; +//! # let table = catalog.load_table(&TableIdent::from_strs(["hello", "world"])?).await?; +//! # 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()); +//! # let rolling_writer_builder = RollingFileWriterBuilder::new_with_default_file_size( +//! # parquet_writer_builder, table.file_io().clone(), location_generator, file_name_generator); +//! # let data_file_writer_builder = DataFileWriterBuilder::new(rolling_writer_builder); +//! +//! // Wrap the data file writer with ClusteredWriter for sorted partitioning +//! use iceberg::writer::partitioning::clustered_writer::ClusteredWriter; +//! use iceberg::writer::partitioning::PartitioningWriter; +//! use iceberg::spec::{Literal, PartitionKey, Struct}; +//! +//! let mut clustered_writer = ClusteredWriter::new(data_file_writer_builder); +//! +//! // Create partition keys (must write in sorted order) +//! let schema = table.metadata().current_schema().clone(); +//! let partition_spec = table.metadata().default_partition_spec().as_ref().clone(); +//! +//! let partition_key_asia = PartitionKey::new( +//! partition_spec.clone(), +//! schema.clone(), +//! Struct::from_iter([Some(Literal::string("ASIA"))]), +//! ); +//! +//! let partition_key_eu = PartitionKey::new( +//! partition_spec.clone(), +//! schema.clone(), +//! Struct::from_iter([Some(Literal::string("EU"))]), +//! ); +//! +//! let partition_key_us = PartitionKey::new( +//! partition_spec.clone(), +//! schema.clone(), +//! Struct::from_iter([Some(Literal::string("US"))]), +//! ); +//! +//! // Write to partitions in sorted order (ASIA -> EU -> US) +//! // clustered_writer.write(partition_key_asia, batch_asia).await?; +//! // clustered_writer.write(partition_key_eu, batch_eu).await?; +//! // clustered_writer.write(partition_key_us, batch_us).await?; +//! // Writing back to ASIA would fail since data must be sorted! +//! +//! let data_files = clustered_writer.close().await?; +//! +//! Ok(()) +//! } +//! ``` pub mod base_writer; pub mod file_writer; -/// Provides partition-aware writers -/// TODO examples pub mod partitioning; use arrow_array::RecordBatch; diff --git a/crates/iceberg/src/writer/partitioning/mod.rs b/crates/iceberg/src/writer/partitioning/mod.rs index 36acb2a987..f63a9d0d26 100644 --- a/crates/iceberg/src/writer/partitioning/mod.rs +++ b/crates/iceberg/src/writer/partitioning/mod.rs @@ -15,6 +15,12 @@ // specific language governing permissions and limitations // under the License. +//! Partitioning writers for handling partitioned Iceberg tables. +//! +//! This module provides two strategies for writing to partitioned tables: +//! - [`FanoutWriter`](fanout_writer::FanoutWriter): Handles unsorted data by maintaining multiple active writers +//! - [`ClusteredWriter`](clustered_writer::ClusteredWriter): Optimized for pre-sorted data with single active writer + pub mod clustered_writer; pub mod fanout_writer; From 406d4168acac4f6164dde66ac579586f770b4298 Mon Sep 17 00:00:00 2001 From: Leon Lin Date: Thu, 23 Oct 2025 17:46:05 -0700 Subject: [PATCH 30/47] feat(sqllogictest): Add support for iceberg datafusion sqllogictest integration (#1764) ## Which issue does this PR close? - Relates to #1211 - Part of #1621 - Add support for iceberg datafusion integration with sqllogictests ## What changes are included in this PR? * Add sqllogictests entry point `sqllogictests.rs` * Update `Schedule` to track `EngineRunner` instead of `Engine` * Update datafusion engine to run slt file with sqllogictest ## Are these changes tested? * `cargo test --test sqllogictests` ``` Finished `test` profile [unoptimized + debuginfo] target(s) in 0.46s Running tests/sqllogictests.rs (target/debug/deps/sqllogictests-8ee6c1e0fdedebe4) running 1 test test schedule: df_test.toml ... ok test result: ok. 1 passed; 0 failed; 0 ignored; 0 measured; 0 filtered out; finished in 0.01s ``` --------- Co-authored-by: Renjie Liu --- .github/workflows/ci.yml | 12 + Cargo.lock | 1286 ++++++++--------- Cargo.toml | 3 + crates/sqllogictest/Cargo.toml | 12 + crates/sqllogictest/src/engine/datafusion.rs | 56 +- crates/sqllogictest/src/engine/mod.rs | 73 +- crates/sqllogictest/src/error.rs | 6 + crates/sqllogictest/src/lib.rs | 7 +- crates/sqllogictest/src/schedule.rs | 24 +- .../testdata/schedules/df_test.toml | 23 + .../testdata/slts/df_test/show_tables.slt | 34 + crates/sqllogictest/tests/sqllogictests.rs | 87 ++ 12 files changed, 862 insertions(+), 761 deletions(-) create mode 100644 crates/sqllogictest/testdata/schedules/df_test.toml create mode 100644 crates/sqllogictest/testdata/slts/df_test/show_tables.slt create mode 100644 crates/sqllogictest/tests/sqllogictests.rs diff --git a/.github/workflows/ci.yml b/.github/workflows/ci.yml index 3207ae87b1..d547397808 100644 --- a/.github/workflows/ci.yml +++ b/.github/workflows/ci.yml @@ -89,6 +89,18 @@ jobs: - macos-latest - windows-latest steps: + - name: Maximize build space (Ubuntu) + if: matrix.os == 'ubuntu-latest' + uses: easimon/maximize-build-space@master + with: + remove-dotnet: "true" + remove-android: "true" + remove-haskell: "true" + remove-codeql: "true" + remove-docker-images: "true" + root-reserve-mb: 10240 + temp-reserve-mb: 10240 + - uses: actions/checkout@v5 - name: Setup Rust toolchain diff --git a/Cargo.lock b/Cargo.lock index b048c3db56..e27e05129d 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -44,7 +44,7 @@ checksum = "5a15f179cd60c4584b8a8c596927aadc462e27f2ca70c04e0071964a73ba7a75" dependencies = [ "cfg-if", "const-random", - "getrandom 0.3.3", + "getrandom 0.3.4", "once_cell", "serde", "version_check", @@ -92,9 +92,9 @@ dependencies = [ [[package]] name = "anstream" -version = "0.6.20" +version = "0.6.21" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "3ae563653d1938f79b1ab1b5e668c87c76a9930414574a6583a7b7e11a8e6192" +checksum = "43d5b281e737544384e969a5ccad3f1cdd24b48086a0fc1b2a5262a26b8f4f4a" dependencies = [ "anstyle", "anstyle-parse", @@ -107,9 +107,9 @@ dependencies = [ [[package]] name = "anstyle" -version = "1.0.11" +version = "1.0.13" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "862ed96ca487e809f1c8e5a8447f6ee2cf102f846893800b20cebdf541fc6bbd" +checksum = "5192cca8006f1fd4f7237516f40fa183bb07f8fbdfedaa0036de5ea9b0b45e78" [[package]] name = "anstyle-parse" @@ -142,9 +142,9 @@ dependencies = [ [[package]] name = "anyhow" -version = "1.0.99" +version = "1.0.100" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "b0674a1ddeecb70197781e945de4b3b8ffb61fa939a5597bcf48503737663100" +checksum = "a23eb6b1614318a8071c9b2521f36b424b2c83db5eb3a0fead4a6c0809af6e61" [[package]] name = "apache-avro" @@ -195,7 +195,7 @@ dependencies = [ "serde_json", "strum 0.27.2", "strum_macros 0.27.2", - "thiserror 2.0.16", + "thiserror 2.0.17", "uuid", "zstd", ] @@ -356,7 +356,7 @@ dependencies = [ "arrow-schema", "chrono", "half", - "indexmap 2.11.0", + "indexmap 2.12.0", "lexical-core", "memchr", "num", @@ -505,9 +505,9 @@ dependencies = [ [[package]] name = "async-fs" -version = "2.1.3" +version = "2.2.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "09f7e37c0ed80b2a977691c47dae8625cfb21e205827106c64f7c588766b2e50" +checksum = "8034a681df4aed8b8edbd7fbe472401ecf009251c8b40556b304567052e294c5" dependencies = [ "async-lock", "blocking", @@ -516,20 +516,20 @@ dependencies = [ [[package]] name = "async-io" -version = "2.5.0" +version = "2.6.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "19634d6336019ef220f09fd31168ce5c184b295cbf80345437cc36094ef223ca" +checksum = "456b8a8feb6f42d237746d4b3e9a178494627745c3c56c6ea55d92ba50d026fc" dependencies = [ - "async-lock", + "autocfg", "cfg-if", "concurrent-queue", "futures-io", "futures-lite", "parking", "polling", - "rustix 1.0.8", + "rustix", "slab", - "windows-sys 0.60.2", + "windows-sys 0.61.2", ] [[package]] @@ -556,9 +556,9 @@ dependencies = [ [[package]] name = "async-process" -version = "2.4.0" +version = "2.5.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "65daa13722ad51e6ab1a1b9c01299142bc75135b337923cfa10e79bbbd669f00" +checksum = "fc50921ec0055cdd8a16de48773bfeec5c972598674347252c0399676be7da75" dependencies = [ "async-channel", "async-io", @@ -569,7 +569,7 @@ dependencies = [ "cfg-if", "event-listener", "futures-lite", - "rustix 1.0.8", + "rustix", ] [[package]] @@ -580,14 +580,14 @@ checksum = "3b43422f69d8ff38f95f1b2bb76517c91589a924d1559a0e935d7c8ce0274c11" dependencies = [ "proc-macro2", "quote", - "syn 2.0.106", + "syn 2.0.107", ] [[package]] name = "async-signal" -version = "0.2.12" +version = "0.2.13" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "f567af260ef69e1d52c2b560ce0ea230763e6fbb9214a85d768760a920e3e3c1" +checksum = "43c070bbf59cd3570b6b2dd54cd772527c7c3620fce8be898406dd3ed6adc64c" dependencies = [ "async-io", "async-lock", @@ -595,10 +595,10 @@ dependencies = [ "cfg-if", "futures-core", "futures-io", - "rustix 1.0.8", + "rustix", "signal-hook-registry", "slab", - "windows-sys 0.60.2", + "windows-sys 0.61.2", ] [[package]] @@ -615,7 +615,7 @@ checksum = "9035ad2d096bed7955a320ee7e2230574d28fd3c3a0f186cbea1ff3c7eed5dbb" dependencies = [ "proc-macro2", "quote", - "syn 2.0.106", + "syn 2.0.107", ] [[package]] @@ -641,9 +641,9 @@ checksum = "c08606f8c3cbf4ce6ec8e28fb0014a2c086708fe954eaa885384a6165172e7e8" [[package]] name = "aws-config" -version = "1.8.6" +version = "1.8.8" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "8bc1b40fb26027769f16960d2f4a6bc20c4bb755d403e552c8c1a73af433c246" +checksum = "37cf2b6af2a95a20e266782b4f76f1a5e12bf412a9db2de9c1e9123b9d8c0ad8" dependencies = [ "aws-credential-types", "aws-runtime", @@ -671,9 +671,9 @@ dependencies = [ [[package]] name = "aws-credential-types" -version = "1.2.6" +version = "1.2.8" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d025db5d9f52cbc413b167136afb3d8aeea708c0d8884783cf6253be5e22f6f2" +checksum = "faf26925f4a5b59eb76722b63c2892b1d70d06fa053c72e4a100ec308c1d47bc" dependencies = [ "aws-smithy-async", "aws-smithy-runtime-api", @@ -683,9 +683,9 @@ dependencies = [ [[package]] name = "aws-lc-rs" -version = "1.13.3" +version = "1.14.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "5c953fe1ba023e6b7730c0d4b031d06f267f23a46167dcbd40316644b10a17ba" +checksum = "879b6c89592deb404ba4dc0ae6b58ffd1795c78991cbb5b8bc441c48a070440d" dependencies = [ "aws-lc-sys", "zeroize", @@ -693,9 +693,9 @@ dependencies = [ [[package]] name = "aws-lc-sys" -version = "0.30.0" +version = "0.32.3" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "dbfd150b5dbdb988bcc8fb1fe787eb6b7ee6180ca24da683b61ea5405f3d43ff" +checksum = "107a4e9d9cab9963e04e84bb8dee0e25f2a987f9a8bad5ed054abd439caa8f8c" dependencies = [ "bindgen", "cc", @@ -706,9 +706,9 @@ dependencies = [ [[package]] name = "aws-runtime" -version = "1.5.10" +version = "1.5.12" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c034a1bc1d70e16e7f4e4caf7e9f7693e4c9c24cd91cf17c2a0b21abaebc7c8b" +checksum = "bfa006bb32360ed90ac51203feafb9d02e3d21046e1fd3a450a404b90ea73e5d" dependencies = [ "aws-credential-types", "aws-sigv4", @@ -730,9 +730,9 @@ dependencies = [ [[package]] name = "aws-sdk-glue" -version = "1.119.0" +version = "1.125.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "9170d379508a356718ca6c27ffa5c00b5077ecdf7ab5cb27a70834a58e00dd13" +checksum = "df436173c8acd8c207ecff1aedfd0992d44d0d6d1e84be73e30dab7746a20363" dependencies = [ "aws-credential-types", "aws-runtime", @@ -752,9 +752,9 @@ dependencies = [ [[package]] name = "aws-sdk-s3tables" -version = "1.37.0" +version = "1.40.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "ba5b1975c398d8778a07ddd3de2bff50f03455907d4524c5e74eab05954ac95c" +checksum = "4f5b63d53d1536c2332c4ed11737c5de3eda1ef59c3565290b2f247d0d712a5e" dependencies = [ "aws-credential-types", "aws-runtime", @@ -774,9 +774,9 @@ dependencies = [ [[package]] name = "aws-sdk-sso" -version = "1.83.0" +version = "1.86.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "643cd43af212d2a1c4dedff6f044d7e1961e5d9e7cfe773d70f31d9842413886" +checksum = "4a0abbfab841446cce6e87af853a3ba2cc1bc9afcd3f3550dd556c43d434c86d" dependencies = [ "aws-credential-types", "aws-runtime", @@ -796,9 +796,9 @@ dependencies = [ [[package]] name = "aws-sdk-ssooidc" -version = "1.84.0" +version = "1.88.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "20ec4a95bd48e0db7a424356a161f8d87bd6a4f0af37204775f0da03d9e39fc3" +checksum = "9a68d675582afea0e94d38b6ca9c5aaae4ca14f1d36faa6edb19b42e687e70d7" dependencies = [ "aws-credential-types", "aws-runtime", @@ -818,9 +818,9 @@ dependencies = [ [[package]] name = "aws-sdk-sts" -version = "1.85.0" +version = "1.88.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "410309ad0df4606bc721aff0d89c3407682845453247213a0ccc5ff8801ee107" +checksum = "d30990923f4f675523c51eb1c0dec9b752fb267b36a61e83cbc219c9d86da715" dependencies = [ "aws-credential-types", "aws-runtime", @@ -841,9 +841,9 @@ dependencies = [ [[package]] name = "aws-sigv4" -version = "1.3.4" +version = "1.3.5" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "084c34162187d39e3740cb635acd73c4e3a551a36146ad6fe8883c929c9f876c" +checksum = "bffc03068fbb9c8dd5ce1c6fb240678a5cffb86fb2b7b1985c999c4b83c8df68" dependencies = [ "aws-credential-types", "aws-smithy-http", @@ -863,9 +863,9 @@ dependencies = [ [[package]] name = "aws-smithy-async" -version = "1.2.5" +version = "1.2.6" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "1e190749ea56f8c42bf15dd76c65e14f8f765233e6df9b0506d9d934ebef867c" +checksum = "127fcfad33b7dfc531141fda7e1c402ac65f88aca5511a4d31e2e3d2cd01ce9c" dependencies = [ "futures-util", "pin-project-lite", @@ -874,9 +874,9 @@ dependencies = [ [[package]] name = "aws-smithy-http" -version = "0.62.3" +version = "0.62.4" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "7c4dacf2d38996cf729f55e7a762b30918229917eca115de45dfa8dfb97796c9" +checksum = "3feafd437c763db26aa04e0cc7591185d0961e64c61885bece0fb9d50ceac671" dependencies = [ "aws-smithy-runtime-api", "aws-smithy-types", @@ -894,9 +894,9 @@ dependencies = [ [[package]] name = "aws-smithy-http-client" -version = "1.1.1" +version = "1.1.3" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "147e8eea63a40315d704b97bf9bc9b8c1402ae94f89d5ad6f7550d963309da1b" +checksum = "1053b5e587e6fa40ce5a79ea27957b04ba660baa02b28b7436f64850152234f1" dependencies = [ "aws-smithy-async", "aws-smithy-runtime-api", @@ -913,38 +913,38 @@ dependencies = [ "hyper-util", "pin-project-lite", "rustls 0.21.12", - "rustls 0.23.31", - "rustls-native-certs 0.8.1", + "rustls 0.23.33", + "rustls-native-certs 0.8.2", "rustls-pki-types", "tokio", - "tokio-rustls 0.26.2", + "tokio-rustls 0.26.4", "tower", "tracing", ] [[package]] name = "aws-smithy-json" -version = "0.61.5" +version = "0.61.6" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "eaa31b350998e703e9826b2104dd6f63be0508666e1aba88137af060e8944047" +checksum = "cff418fc8ec5cadf8173b10125f05c2e7e1d46771406187b2c878557d4503390" dependencies = [ "aws-smithy-types", ] [[package]] name = "aws-smithy-observability" -version = "0.1.3" +version = "0.1.4" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "9364d5989ac4dd918e5cc4c4bdcc61c9be17dcd2586ea7f69e348fc7c6cab393" +checksum = "2d1881b1ea6d313f9890710d65c158bdab6fb08c91ea825f74c1c8c357baf4cc" dependencies = [ "aws-smithy-runtime-api", ] [[package]] name = "aws-smithy-query" -version = "0.60.7" +version = "0.60.8" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "f2fbd61ceb3fe8a1cb7352e42689cec5335833cd9f94103a61e98f9bb61c64bb" +checksum = "d28a63441360c477465f80c7abac3b9c4d075ca638f982e605b7dc2a2c7156c9" dependencies = [ "aws-smithy-types", "urlencoding", @@ -952,9 +952,9 @@ dependencies = [ [[package]] name = "aws-smithy-runtime" -version = "1.9.1" +version = "1.9.3" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d3946acbe1ead1301ba6862e712c7903ca9bb230bdf1fbd1b5ac54158ef2ab1f" +checksum = "40ab99739082da5347660c556689256438defae3bcefd66c52b095905730e404" dependencies = [ "aws-smithy-async", "aws-smithy-http", @@ -976,9 +976,9 @@ dependencies = [ [[package]] name = "aws-smithy-runtime-api" -version = "1.9.0" +version = "1.9.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "07f5e0fc8a6b3f2303f331b94504bbf754d85488f402d6f1dd7a6080f99afe56" +checksum = "3683c5b152d2ad753607179ed71988e8cfd52964443b4f74fd8e552d0bbfeb46" dependencies = [ "aws-smithy-async", "aws-smithy-types", @@ -993,9 +993,9 @@ dependencies = [ [[package]] name = "aws-smithy-types" -version = "1.3.2" +version = "1.3.3" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d498595448e43de7f4296b7b7a18a8a02c61ec9349128c80a368f7c3b4ab11a8" +checksum = "9f5b3a7486f6690ba25952cabf1e7d75e34d69eaff5081904a47bc79074d6457" dependencies = [ "base64-simd", "bytes", @@ -1019,18 +1019,18 @@ dependencies = [ [[package]] name = "aws-smithy-xml" -version = "0.60.10" +version = "0.60.11" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "3db87b96cb1b16c024980f133968d52882ca0daaee3a086c6decc500f6c99728" +checksum = "e9c34127e8c624bc2999f3b657e749c1393bedc9cd97b92a804db8ced4d2e163" dependencies = [ "xmlparser", ] [[package]] name = "aws-types" -version = "1.3.8" +version = "1.3.9" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "b069d19bf01e46298eaedd7c6f283fe565a59263e53eebec945f3e6398f42390" +checksum = "e2fd329bf0e901ff3f60425691410c69094dc2a1f34b331f37bfc4e9ac1565a1" dependencies = [ "aws-credential-types", "aws-smithy-async", @@ -1081,9 +1081,9 @@ checksum = "55248b47b0caf0546f7988906588779981c43bb1bc9d0c44087278f80cdb44ba" [[package]] name = "bigdecimal" -version = "0.4.8" +version = "0.4.9" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "1a22f228ab7a1b23027ccc6c350b72868017af7ea8356fbdf19f8d991c690013" +checksum = "560f42649de9fa436b73517378a147ec21f6c997a546581df4b4b31677828934" dependencies = [ "autocfg", "libm", @@ -1101,34 +1101,31 @@ checksum = "230c5f1ca6a325a32553f8640d31ac9b49f2411e901e427570154868b46da4f7" [[package]] name = "bindgen" -version = "0.69.5" +version = "0.72.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "271383c67ccabffb7381723dea0672a673f292304fcb45c01cc648c7a8d58088" +checksum = "993776b509cfb49c750f11b8f07a46fa23e0a1386ffc01fb1e7d343efc387895" dependencies = [ "bitflags", "cexpr", "clang-sys", - "itertools 0.12.1", - "lazy_static", - "lazycell", + "itertools 0.13.0", "log", "prettyplease", "proc-macro2", "quote", "regex", - "rustc-hash 1.1.0", + "rustc-hash", "shlex", - "syn 2.0.106", - "which", + "syn 2.0.107", ] [[package]] name = "bitflags" -version = "2.9.4" +version = "2.10.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "2261d10cca569e4643e526d8dc2e62e433cc8aba21ab764233731f8d369bf394" +checksum = "812e12b5285cc515a9c72a5c1d3b6d46a19dac5acfef5265968c166106e31dd3" dependencies = [ - "serde", + "serde_core", ] [[package]] @@ -1198,9 +1195,9 @@ dependencies = [ [[package]] name = "bon" -version = "3.7.2" +version = "3.8.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c2529c31017402be841eb45892278a6c21a000c0a17643af326c73a73f83f0fb" +checksum = "ebeb9aaf9329dff6ceb65c689ca3db33dbf15f324909c60e4e5eef5701ce31b1" dependencies = [ "bon-macros", "rustversion", @@ -1208,9 +1205,9 @@ dependencies = [ [[package]] name = "bon-macros" -version = "3.7.2" +version = "3.8.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d82020dadcb845a345591863adb65d74fa8dc5c18a0b6d408470e13b7adc7005" +checksum = "77e9d642a7e3a318e37c2c9427b5a6a48aa1ad55dcd986f3034ab2239045a645" dependencies = [ "darling 0.21.3", "ident_case", @@ -1218,7 +1215,7 @@ dependencies = [ "proc-macro2", "quote", "rustversion", - "syn 2.0.106", + "syn 2.0.107", ] [[package]] @@ -1241,7 +1238,7 @@ dependencies = [ "proc-macro-crate", "proc-macro2", "quote", - "syn 2.0.106", + "syn 2.0.107", ] [[package]] @@ -1305,9 +1302,9 @@ dependencies = [ [[package]] name = "bytemuck" -version = "1.23.2" +version = "1.24.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "3995eaeebcdf32f91f980d360f78732ddc061097ab4e39991ae7a6ace9194677" +checksum = "1fbdf580320f38b612e485521afda1ee26d10cc9884efaaa750d383e13e3c5f4" [[package]] name = "byteorder" @@ -1374,9 +1371,9 @@ dependencies = [ [[package]] name = "cc" -version = "1.2.36" +version = "1.2.41" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "5252b3d2648e5eedbc1a6f501e3c795e07025c1e93bbf8bbdd6eef7f447a6d54" +checksum = "ac9fe6cdbb24b6ade63616c0a0688e45bb56732262c158df3c0c4bea4ca47cb7" dependencies = [ "find-msvc-tools", "jobserver", @@ -1395,9 +1392,9 @@ dependencies = [ [[package]] name = "cfg-if" -version = "1.0.3" +version = "1.0.4" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "2fd1289c04a9ea8cb22300a459a72a385d7c73d3259e2ed7dcb2af674838cfa9" +checksum = "9330f8b2ff13f34540b44e946ef35111825727b38d33286ef986142615121801" [[package]] name = "cfg_aliases" @@ -1416,7 +1413,7 @@ dependencies = [ "num-traits", "serde", "wasm-bindgen", - "windows-link 0.2.0", + "windows-link", ] [[package]] @@ -1474,9 +1471,9 @@ dependencies = [ [[package]] name = "clap" -version = "4.5.47" +version = "4.5.49" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "7eac00902d9d136acd712710d71823fb8ac8004ca445a89e73a41d45aa712931" +checksum = "f4512b90fa68d3a9932cea5184017c5d200f5921df706d45e853537dea51508f" dependencies = [ "clap_builder", "clap_derive", @@ -1484,9 +1481,9 @@ dependencies = [ [[package]] name = "clap_builder" -version = "4.5.47" +version = "4.5.49" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "2ad9bbf750e73b5884fb8a211a9424a1906c1e156724260fdae972f31d70e1d6" +checksum = "0025e98baa12e766c67ba13ff4695a887a1eba19569aad00a472546795bd6730" dependencies = [ "anstream", "anstyle", @@ -1496,21 +1493,21 @@ dependencies = [ [[package]] name = "clap_derive" -version = "4.5.47" +version = "4.5.49" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "bbfd7eae0b0f1a6e63d4b13c9c478de77c2eb546fba158ad50b4203dc24b9f9c" +checksum = "2a0b5487afeab2deb2ff4e03a807ad1a03ac532ff5a2cee5d86884440c7f7671" dependencies = [ "heck", "proc-macro2", "quote", - "syn 2.0.106", + "syn 2.0.107", ] [[package]] name = "clap_lex" -version = "0.7.5" +version = "0.7.6" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "b94f61472cee1439c0b966b47e3aca9ae07e45d070759512cd390ea2bebc6675" +checksum = "a1d728cc89cf3aee9ff92b05e62b19ee65a02b5702cff7d5a377e32c6ae29d8d" [[package]] name = "clipboard-win" @@ -1542,17 +1539,17 @@ version = "3.0.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "fde0e0ec90c9dfb3b4b1a0891a7dcd0e2bffde2f7efed5fe7c9bb00e5bfb915e" dependencies = [ - "windows-sys 0.48.0", + "windows-sys 0.59.0", ] [[package]] name = "comfy-table" -version = "7.2.0" +version = "7.2.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "3f8e18d0dca9578507f13f9803add0df13362b02c501c1c17734f0dbb52eaf0b" +checksum = "b03b7db8e0b4b2fdad6c551e634134e99ec000e5c8c3b6856c65e8bbaded7a3b" dependencies = [ "unicode-segmentation", - "unicode-width 0.2.1", + "unicode-width 0.2.2", ] [[package]] @@ -1573,7 +1570,7 @@ dependencies = [ "encode_unicode", "libc", "once_cell", - "unicode-width 0.2.1", + "unicode-width 0.2.2", "windows-sys 0.59.0", ] @@ -1747,21 +1744,21 @@ dependencies = [ [[package]] name = "csv" -version = "1.3.1" +version = "1.4.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "acdc4883a9c96732e4733212c01447ebd805833b7275a73ca3ee080fd77afdaf" +checksum = "52cd9d68cf7efc6ddfaaee42e7288d3a99d613d4b50f76ce9827ae0c6e14f938" dependencies = [ "csv-core", "itoa", "ryu", - "serde", + "serde_core", ] [[package]] name = "csv-core" -version = "0.1.12" +version = "0.1.13" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "7d02f3b0da4c6504f86e9cd789d8dbafab48c2321be74e9987593de5a894d93d" +checksum = "704a3c26996a80471189265814dbc2c257598b96b8a7feae2d31ace646bb9782" dependencies = [ "memchr", ] @@ -1773,7 +1770,7 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "32a2785755761f3ddc1492979ce1e48d2c00d09311c39e4466429188f3dd6501" dependencies = [ "quote", - "syn 2.0.106", + "syn 2.0.107", ] [[package]] @@ -1807,7 +1804,7 @@ dependencies = [ "proc-macro2", "quote", "strsim", - "syn 2.0.106", + "syn 2.0.107", ] [[package]] @@ -1821,7 +1818,7 @@ dependencies = [ "proc-macro2", "quote", "strsim", - "syn 2.0.106", + "syn 2.0.107", ] [[package]] @@ -1832,7 +1829,7 @@ checksum = "fc34b93ccb385b40dc71c6fceac4b2ad23662c7eeb248cf10d529b7e055b6ead" dependencies = [ "darling_core 0.20.11", "quote", - "syn 2.0.106", + "syn 2.0.107", ] [[package]] @@ -1843,14 +1840,14 @@ checksum = "d38308df82d1080de0afee5d069fa14b0326a88c14f15c5ccda35b4a6c414c81" dependencies = [ "darling_core 0.21.3", "quote", - "syn 2.0.106", + "syn 2.0.107", ] [[package]] name = "dary_heap" -version = "0.3.7" +version = "0.3.8" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "04d2cd9c18b9f454ed67da600630b021a8a80bf33f8c95896ab33aaf1c26b728" +checksum = "06d2e3287df1c007e74221c49ca10a95d557349e54b3a75dc2fb14712c751f04" [[package]] name = "dashmap" @@ -2008,7 +2005,7 @@ dependencies = [ "base64 0.22.1", "half", "hashbrown 0.14.5", - "indexmap 2.11.0", + "indexmap 2.12.0", "libc", "log", "object_store", @@ -2212,7 +2209,7 @@ dependencies = [ "datafusion-functions-aggregate-common", "datafusion-functions-window-common", "datafusion-physical-expr-common", - "indexmap 2.11.0", + "indexmap 2.12.0", "paste", "recursive", "serde_json", @@ -2227,7 +2224,7 @@ checksum = "70fafb3a045ed6c49cfca0cd090f62cf871ca6326cc3355cb0aaf1260fa760b6" dependencies = [ "arrow", "datafusion-common", - "indexmap 2.11.0", + "indexmap 2.12.0", "itertools 0.14.0", "paste", ] @@ -2368,7 +2365,7 @@ checksum = "2821de7cb0362d12e75a5196b636a59ea3584ec1e1cc7dc6f5e34b9e8389d251" dependencies = [ "datafusion-expr", "quote", - "syn 2.0.106", + "syn 2.0.107", ] [[package]] @@ -2382,7 +2379,7 @@ dependencies = [ "datafusion-common", "datafusion-expr", "datafusion-physical-expr", - "indexmap 2.11.0", + "indexmap 2.12.0", "itertools 0.14.0", "log", "recursive", @@ -2405,11 +2402,11 @@ dependencies = [ "datafusion-physical-expr-common", "half", "hashbrown 0.14.5", - "indexmap 2.11.0", + "indexmap 2.12.0", "itertools 0.14.0", "log", "paste", - "petgraph 0.8.2", + "petgraph 0.8.3", ] [[package]] @@ -2467,7 +2464,7 @@ dependencies = [ "futures", "half", "hashbrown 0.14.5", - "indexmap 2.11.0", + "indexmap 2.12.0", "itertools 0.14.0", "log", "parking_lot", @@ -2525,7 +2522,7 @@ dependencies = [ "bigdecimal", "datafusion-common", "datafusion-expr", - "indexmap 2.11.0", + "indexmap 2.12.0", "log", "recursive", "regex", @@ -2555,7 +2552,7 @@ dependencies = [ "sqllogictest", "sqlparser", "tempfile", - "thiserror 2.0.16", + "thiserror 2.0.17", "tokio", ] @@ -2591,12 +2588,12 @@ dependencies = [ [[package]] name = "deranged" -version = "0.5.3" +version = "0.5.4" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d630bccd429a5bb5a64b5e94f693bfc48c9f8566418fda4c494cc94f911f87cc" +checksum = "a41953f86f8a05768a6cda24def994fd2f424b04ec5c719cf89989779f199071" dependencies = [ "powerfmt", - "serde", + "serde_core", ] [[package]] @@ -2617,7 +2614,7 @@ dependencies = [ "darling 0.20.11", "proc-macro2", "quote", - "syn 2.0.106", + "syn 2.0.107", ] [[package]] @@ -2627,7 +2624,7 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "ab63b0e2bf4d5928aff72e83a7dace85d7bba5fe12dcc3c5a572d78caffd3f3c" dependencies = [ "derive_builder_core", - "syn 2.0.106", + "syn 2.0.107", ] [[package]] @@ -2672,7 +2669,7 @@ dependencies = [ "libc", "option-ext", "redox_users", - "windows-sys 0.61.0", + "windows-sys 0.61.2", ] [[package]] @@ -2683,7 +2680,7 @@ checksum = "97369cbbc041bc366949bc74d34658d6cda5621039731c6310521892a3a20ae0" dependencies = [ "proc-macro2", "quote", - "syn 2.0.106", + "syn 2.0.107", ] [[package]] @@ -2734,7 +2731,7 @@ dependencies = [ "enum-ordinalize", "proc-macro2", "quote", - "syn 2.0.106", + "syn 2.0.107", ] [[package]] @@ -2775,14 +2772,14 @@ checksum = "0d28318a75d4aead5c4db25382e8ef717932d0346600cacae6357eb5941bc5ff" dependencies = [ "proc-macro2", "quote", - "syn 2.0.106", + "syn 2.0.107", ] [[package]] name = "env_filter" -version = "0.1.3" +version = "0.1.4" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "186e05a59d4c50738528153b83b0b0194d3a29507dfec16eccd4b342903397d0" +checksum = "1bf3c259d255ca70051b30e2e95b5446cdb8949ac4cd22c0d7fd634d89f568e2" dependencies = [ "log", "regex", @@ -2809,12 +2806,12 @@ checksum = "877a4ace8713b0bcf2a4e7eec82529c029f1d0619886d18145fea96c3ffe5c0f" [[package]] name = "errno" -version = "0.3.13" +version = "0.3.14" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "778e2ac28f6c47af28e4907f13ffd1e1ddbd400980a9abd7c8df189bf578a5ad" +checksum = "39cab71617ae0d63f51a36d69f866391735b51691dbda63cf6f96d042b63efeb" dependencies = [ "libc", - "windows-sys 0.60.2", + "windows-sys 0.61.2", ] [[package]] @@ -2890,7 +2887,7 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "baec6a0289d7f1fe5665586ef7340af82e3037207bef60f5785e57569776f0c8" dependencies = [ "bytes", - "rkyv 0.8.11", + "rkyv 0.8.12", "serde", "simdutf8", ] @@ -2902,15 +2899,15 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "0ce92ff622d6dadf7349484f42c93271a0d49b7cc4d466a936405bacbe10aa78" dependencies = [ "cfg-if", - "rustix 1.0.8", - "windows-sys 0.52.0", + "rustix", + "windows-sys 0.59.0", ] [[package]] name = "find-msvc-tools" -version = "0.1.1" +version = "0.1.4" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "7fd99930f64d146689264c637b5af2f0233a933bef0d8570e2526bf9e083192d" +checksum = "52051878f80a721bb68ebfbc930e07b65ba72f2da88968ea5c06fd6ca3d3a127" [[package]] name = "fixedbitset" @@ -2920,9 +2917,9 @@ checksum = "1d674e81391d1e1ab681a28d99df07927c6d4aa5b027d7da16ba32d1d21ecd99" [[package]] name = "flatbuffers" -version = "25.2.10" +version = "25.9.23" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "1045398c1bfd89168b5fd3f1fc11f6e70b34f6f66300c87d44d3de849463abf1" +checksum = "09b6620799e7340ebd9968d2e0708eb82cf1971e9a16821e2091b6d6e475eed5" dependencies = [ "bitflags", "rustc_version", @@ -2930,9 +2927,9 @@ dependencies = [ [[package]] name = "flate2" -version = "1.1.2" +version = "1.1.4" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "4a3d7db9596fecd151c5f638c0ee5d5bd487b6e0ea232e5dc96d5250f6f94b1d" +checksum = "dc5a4e564e38c699f2880d3fda590bedc2e69f3f84cd48b457bd892ce61d0aa9" dependencies = [ "crc32fast", "libz-rs-sys", @@ -2979,9 +2976,9 @@ checksum = "28dd6caf6059519a65843af8fe2a3ae298b14b80179855aeb4adc2c1934ee619" [[package]] name = "fs-err" -version = "3.1.1" +version = "3.1.3" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "88d7be93788013f265201256d58f04936a8079ad5dc898743aa20525f503b683" +checksum = "6ad492b2cf1d89d568a43508ab24f98501fe03f2f31c01e1d0fe7366a71745d2" dependencies = [ "autocfg", ] @@ -3078,7 +3075,7 @@ checksum = "162ee34ebcb7c64a8abebc059ce0fee27c2262618d7b60ed8faf72fef13c3650" dependencies = [ "proc-macro2", "quote", - "syn 2.0.106", + "syn 2.0.107", ] [[package]] @@ -3111,25 +3108,11 @@ dependencies = [ "slab", ] -[[package]] -name = "generator" -version = "0.8.7" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "605183a538e3e2a9c1038635cc5c2d194e2ee8fd0d1b66b8349fad7dbacce5a2" -dependencies = [ - "cc", - "cfg-if", - "libc", - "log", - "rustversion", - "windows", -] - [[package]] name = "generic-array" -version = "0.14.7" +version = "0.14.9" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "85649ca51fd72272d7821adaf274ad91c288277713d9c18820d8499a7ff69e9a" +checksum = "4bb6743198531e02858aeaea5398fcc883e71851fcbcb5a2f773e2fb6cb1edf2" dependencies = [ "typenum", "version_check", @@ -3144,21 +3127,21 @@ dependencies = [ "cfg-if", "js-sys", "libc", - "wasi 0.11.1+wasi-snapshot-preview1", + "wasi", "wasm-bindgen", ] [[package]] name = "getrandom" -version = "0.3.3" +version = "0.3.4" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "26145e563e54f2cadc477553f1ec5ee650b00862f0a58bcd12cbdc5f0ea2d2f4" +checksum = "899def5c37c4fd7b2664648c28120ecec138e4d395b459e5ca34f9cce2dd77fd" dependencies = [ "cfg-if", "js-sys", "libc", "r-efi", - "wasi 0.14.4+wasi-0.2.4", + "wasip2", "wasm-bindgen", ] @@ -3170,9 +3153,9 @@ checksum = "0cc23270f6e1808e30a928bdc84dea0b9b4136a8bc82338574f23baf47bbd280" [[package]] name = "globset" -version = "0.4.16" +version = "0.4.17" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "54a1028dfc5f5df5da8a56a73e6c153c9a9708ec57232470703592a3f18e49f5" +checksum = "eab69130804d941f8075cfd713bf8848a2c3b3f201a9457a11e6f87e1ab62305" dependencies = [ "aho-corasick", "bstr", @@ -3216,7 +3199,7 @@ dependencies = [ "futures-sink", "futures-util", "http 0.2.12", - "indexmap 2.11.0", + "indexmap 2.12.0", "slab", "tokio", "tokio-util", @@ -3235,7 +3218,7 @@ dependencies = [ "futures-core", "futures-sink", "http 1.3.1", - "indexmap 2.11.0", + "indexmap 2.12.0", "slab", "tokio", "tokio-util", @@ -3244,13 +3227,14 @@ dependencies = [ [[package]] name = "half" -version = "2.6.0" +version = "2.7.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "459196ed295495a68f7d7fe1d84f6c4b7ff0e21fe3017b2f283c6fac3ad803c9" +checksum = "6ea2d84b969582b4b1864a92dc5d27cd2b77b622a8d79306834f1be5ba20d84b" dependencies = [ "cfg-if", "crunchy", "num-traits", + "zerocopy", ] [[package]] @@ -3283,6 +3267,12 @@ dependencies = [ "foldhash", ] +[[package]] +name = "hashbrown" +version = "0.16.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "5419bdc4f6a9207fbeba6d11b604d481addf78ecd10c11ad51e76c2f6482748d" + [[package]] name = "hashlink" version = "0.10.0" @@ -3428,9 +3418,9 @@ dependencies = [ [[package]] name = "humantime" -version = "2.2.0" +version = "2.3.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "9b112acc8b3adf4b107a8ec20977da0273a8c386765a3ec0229bd500a1443f9f" +checksum = "135b12329e5e3ce057a9f972339ea52bc954fe1e9358ef27f95e89716fbc5424" [[package]] name = "hyper" @@ -3504,20 +3494,20 @@ dependencies = [ "http 1.3.1", "hyper 1.7.0", "hyper-util", - "rustls 0.23.31", - "rustls-native-certs 0.8.1", + "rustls 0.23.33", + "rustls-native-certs 0.8.2", "rustls-pki-types", "tokio", - "tokio-rustls 0.26.2", + "tokio-rustls 0.26.4", "tower-service", - "webpki-roots 1.0.2", + "webpki-roots 1.0.3", ] [[package]] name = "hyper-util" -version = "0.1.16" +version = "0.1.17" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "8d9b05277c7e8da2c93a568989bb6207bef0112e8d17df7a6eda4a3cf143bc5e" +checksum = "3c6995591a8f1380fcb4ba966a252a4b29188d51d2b89e3a252f5305be65aea8" dependencies = [ "base64 0.22.1", "bytes", @@ -3531,7 +3521,7 @@ dependencies = [ "libc", "percent-encoding", "pin-project-lite", - "socket2 0.5.10", + "socket2 0.6.1", "tokio", "tower-service", "tracing", @@ -3539,9 +3529,9 @@ dependencies = [ [[package]] name = "iana-time-zone" -version = "0.1.63" +version = "0.1.64" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "b0c919e5debc312ad217002b8048a17b7d83f80703865bbfcfebb0458b0b27d8" +checksum = "33e57f83510bb73707521ebaffa789ec8caf86f9657cad665b092b581d40e9fb" dependencies = [ "android_system_properties", "core-foundation-sys", @@ -3813,7 +3803,12 @@ dependencies = [ "datafusion", "datafusion-sqllogictest", "enum-ordinalize", + "env_logger", + "iceberg", + "iceberg-datafusion", "indicatif", + "libtest-mimic", + "log", "serde", "sqllogictest", "tokio", @@ -3944,9 +3939,9 @@ dependencies = [ [[package]] name = "ignore" -version = "0.4.23" +version = "0.4.24" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "6d89fd380afde86567dfba715db065673989d6253f42b88179abd3eae47bda4b" +checksum = "81776e6f9464432afcc28d03e52eb101c93b6f0566f52aef2427663e700f0403" dependencies = [ "crossbeam-deque", "globset", @@ -3971,13 +3966,14 @@ dependencies = [ [[package]] name = "indexmap" -version = "2.11.0" +version = "2.12.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "f2481980430f9f78649238835720ddccc57e52df14ffce1c6f37391d61b563e9" +checksum = "6717a8d2a5a929a1a2eb43a12812498ed141a0bcfb7e8f7844fbdbe4303bba9f" dependencies = [ "equivalent", - "hashbrown 0.15.5", + "hashbrown 0.16.0", "serde", + "serde_core", ] [[package]] @@ -3989,7 +3985,7 @@ dependencies = [ "console", "number_prefix", "portable-atomic", - "unicode-width 0.2.1", + "unicode-width 0.2.2", "web-time", ] @@ -4041,15 +4037,6 @@ version = "1.70.1" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "7943c866cc5cd64cbc25b2e01621d07fa8eb2a1a23160ee81ce38704e97b8ecf" -[[package]] -name = "itertools" -version = "0.12.1" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "ba291022dbbd398a455acf126c1e341954079855bc60dfdda641363bd6922569" -dependencies = [ - "either", -] - [[package]] name = "itertools" version = "0.13.0" @@ -4095,7 +4082,7 @@ checksum = "03343451ff899767262ec32146f6d559dd759fdadf42ff0e227c7c48f72594b4" dependencies = [ "proc-macro2", "quote", - "syn 2.0.106", + "syn 2.0.107", ] [[package]] @@ -4104,15 +4091,15 @@ version = "0.1.34" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "9afb3de4395d6b3e67a780b6de64b51c978ecf11cb9a462c66be7d4ca9039d33" dependencies = [ - "getrandom 0.3.3", + "getrandom 0.3.4", "libc", ] [[package]] name = "js-sys" -version = "0.3.78" +version = "0.3.81" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "0c0b063578492ceec17683ef2f8c5e89121fbd0b172cbc280635ab7567db2738" +checksum = "ec48937a97411dcb524a265206ccd4c90bb711fca92b2792c407f268825b9305" dependencies = [ "once_cell", "wasm-bindgen", @@ -4142,17 +4129,11 @@ dependencies = [ "spin", ] -[[package]] -name = "lazycell" -version = "1.3.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "830d08ce1d1d941e6b30645f1a0eb5643013d835ce3779a5fc208261dbe10f55" - [[package]] name = "lexical-core" -version = "1.0.5" +version = "1.0.6" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "b765c31809609075565a70b4b71402281283aeda7ecaf4818ac14a7b2ade8958" +checksum = "7d8d125a277f807e55a77304455eb7b1cb52f2b18c143b60e766c120bd64a594" dependencies = [ "lexical-parse-float", "lexical-parse-integer", @@ -4163,60 +4144,53 @@ dependencies = [ [[package]] name = "lexical-parse-float" -version = "1.0.5" +version = "1.0.6" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "de6f9cb01fb0b08060209a057c048fcbab8717b4c1ecd2eac66ebfe39a65b0f2" +checksum = "52a9f232fbd6f550bc0137dcb5f99ab674071ac2d690ac69704593cb4abbea56" dependencies = [ "lexical-parse-integer", "lexical-util", - "static_assertions", ] [[package]] name = "lexical-parse-integer" -version = "1.0.5" +version = "1.0.6" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "72207aae22fc0a121ba7b6d479e42cbfea549af1479c3f3a4f12c70dd66df12e" +checksum = "9a7a039f8fb9c19c996cd7b2fcce303c1b2874fe1aca544edc85c4a5f8489b34" dependencies = [ "lexical-util", - "static_assertions", ] [[package]] name = "lexical-util" -version = "1.0.6" +version = "1.0.7" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "5a82e24bf537fd24c177ffbbdc6ebcc8d54732c35b50a3f28cc3f4e4c949a0b3" -dependencies = [ - "static_assertions", -] +checksum = "2604dd126bb14f13fb5d1bd6a66155079cb9fa655b37f875b3a742c705dbed17" [[package]] name = "lexical-write-float" -version = "1.0.5" +version = "1.0.6" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c5afc668a27f460fb45a81a757b6bf2f43c2d7e30cb5a2dcd3abf294c78d62bd" +checksum = "50c438c87c013188d415fbabbb1dceb44249ab81664efbd31b14ae55dabb6361" dependencies = [ "lexical-util", "lexical-write-integer", - "static_assertions", ] [[package]] name = "lexical-write-integer" -version = "1.0.5" +version = "1.0.6" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "629ddff1a914a836fb245616a7888b62903aae58fa771e1d83943035efa0f978" +checksum = "409851a618475d2d5796377cad353802345cba92c867d9fbcde9cf4eac4e14df" dependencies = [ "lexical-util", - "static_assertions", ] [[package]] name = "libc" -version = "0.2.175" +version = "0.2.177" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "6a82ae493e598baaea5209805c49bbf2ea7de956d50d7da0da1164f9c6d28543" +checksum = "2874a2af47a2325c2001a6e6fad9b16a53b802102b528163885171cf92b15976" [[package]] name = "libflate" @@ -4244,12 +4218,12 @@ dependencies = [ [[package]] name = "libloading" -version = "0.8.8" +version = "0.8.9" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "07033963ba89ebaf1584d767badaa2e8fcec21aedea6b8c0346d487d49c28667" +checksum = "d7c4b02199fee7c5d21a5ae7d8cfa79a6ef5bb2fc834d6e9058e89c825efdc55" dependencies = [ "cfg-if", - "windows-targets 0.53.3", + "windows-link", ] [[package]] @@ -4270,9 +4244,9 @@ dependencies = [ [[package]] name = "libredox" -version = "0.1.9" +version = "0.1.10" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "391290121bad3d37fbddad76d8f5d1c1c314cfc646d143d7e07a3086ddff0ce3" +checksum = "416f7e718bdb06000964960ffa43b4335ad4012ae8b99060261aa4a8088d5ccb" dependencies = [ "bitflags", "libc", @@ -4330,15 +4304,9 @@ dependencies = [ [[package]] name = "linux-raw-sys" -version = "0.4.15" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d26c52dbd32dccf2d10cac7725f8eae5296885fb5703b261f7d0a0739ec807ab" - -[[package]] -name = "linux-raw-sys" -version = "0.9.4" +version = "0.11.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "cd945864f07fe9f5371a27ad7b52a172b4b499999f1d97574c9fa68373937e12" +checksum = "df1d3c3b53da64cf5760482273a98e575c651a67eec7f77df96b5b642de8f039" [[package]] name = "litemap" @@ -4348,11 +4316,10 @@ checksum = "241eaef5fd12c88705a01fc1066c48c4b36e0dd4377dcdc7ec3942cea7a69956" [[package]] name = "lock_api" -version = "0.4.13" +version = "0.4.14" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "96936507f153605bddfcda068dd804796c84324ed2510809e5b2a624c81da765" +checksum = "224399e74b87b5f3557511d98dff8b14089b3dadafcab6bb93eab67d3aace965" dependencies = [ - "autocfg", "scopeguard", ] @@ -4362,19 +4329,6 @@ version = "0.4.28" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "34080505efa8e45a4b816c349525ebe327ceaa8559756f0356cba97ef3bf7432" -[[package]] -name = "loom" -version = "0.7.2" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "419e0dc8046cb947daa77eb95ae174acfbddb7673b4151f56d1eed8e93fbfaca" -dependencies = [ - "cfg-if", - "generator", - "scoped-tls", - "tracing", - "tracing-subscriber", -] - [[package]] name = "lru-slab" version = "0.1.2" @@ -4401,15 +4355,6 @@ dependencies = [ "pkg-config", ] -[[package]] -name = "matchers" -version = "0.2.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d1525a2a28c7f4fa0fc98bb91ae755d1e2d1505079e05539e35bc876b5d65ae9" -dependencies = [ - "regex-automata", -] - [[package]] name = "md-5" version = "0.10.6" @@ -4422,9 +4367,9 @@ dependencies = [ [[package]] name = "memchr" -version = "2.7.5" +version = "2.7.6" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "32a282da65faaf38286cf3be983213fcf1d2e2a58700e808f83f4ea9a4804bc0" +checksum = "f52b00d39961fc5b2736ea853c9cc86238e165017a493d1d5c8eac6bdc4cc273" [[package]] name = "memoffset" @@ -4444,7 +4389,7 @@ dependencies = [ "ahash 0.8.12", "faststr", "paste", - "rustc-hash 2.1.1", + "rustc-hash", "tokio", ] @@ -4470,17 +4415,18 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "1fa76a2c86f704bdb222d66965fb3d63269ce38518b83cb0575fca855ebb6316" dependencies = [ "adler2", + "simd-adler32", ] [[package]] name = "mio" -version = "1.0.4" +version = "1.1.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "78bed444cc8a2160f01cbcf811ef18cac863ad68ae8ca62092e8db51d51c761c" +checksum = "69d83b0086dc8ecf3ce9ae2874b2d1290252e2a30720bea58a5c6639b0092873" dependencies = [ "libc", - "wasi 0.11.1+wasi-snapshot-preview1", - "windows-sys 0.59.0", + "wasi", + "windows-sys 0.61.2", ] [[package]] @@ -4506,7 +4452,7 @@ dependencies = [ "cfg-if", "proc-macro2", "quote", - "syn 2.0.106", + "syn 2.0.107", ] [[package]] @@ -4535,23 +4481,22 @@ dependencies = [ [[package]] name = "moka" -version = "0.12.10" +version = "0.12.11" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a9321642ca94a4282428e6ea4af8cc2ca4eac48ac7a6a4ea8f33f76d0ce70926" +checksum = "8261cd88c312e0004c1d51baad2980c66528dfdb2bee62003e643a4d8f86b077" dependencies = [ "async-lock", "crossbeam-channel", "crossbeam-epoch", "crossbeam-utils", + "equivalent", "event-listener", "futures-util", - "loom", "parking_lot", "portable-atomic", "rustc_version", "smallvec", "tagptr", - "thiserror 1.0.69", "uuid", ] @@ -4575,7 +4520,7 @@ checksum = "b40e46c845ac234bcba19db7ab252bc2778cbadd516a466d2f12b1580852d136" dependencies = [ "proc-macro2", "quote", - "syn 2.0.106", + "syn 2.0.107", ] [[package]] @@ -4586,22 +4531,22 @@ checksum = "1d87ecb2933e8aeadb3e3a02b828fed80a7528047e68b4f424523a0981a3a084" [[package]] name = "munge" -version = "0.4.6" +version = "0.4.7" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d7feb0b48aa0a25f9fe0899482c6e1379ee7a11b24a53073eacdecb9adb6dc60" +checksum = "5e17401f259eba956ca16491461b6e8f72913a0a114e39736ce404410f915a0c" dependencies = [ "munge_macro", ] [[package]] name = "munge_macro" -version = "0.4.6" +version = "0.4.7" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "f2e3795a5d2da581a8b252fec6022eee01aea10161a4d1bf237d4cbe47f7e988" +checksum = "4568f25ccbd45ab5d5603dc34318c1ec56b117531781260002151b8530a9f931" dependencies = [ "proc-macro2", "quote", - "syn 2.0.106", + "syn 2.0.107", ] [[package]] @@ -4662,11 +4607,11 @@ dependencies = [ [[package]] name = "nu-ansi-term" -version = "0.50.1" +version = "0.50.3" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d4a28e057d01f97e61255210fcff094d74ed0466038633e95017f5beb68e4399" +checksum = "7957b9740744892f114936ab4a57b3f487491bbeafaf8083688b16841a4240e5" dependencies = [ - "windows-sys 0.52.0", + "windows-sys 0.61.2", ] [[package]] @@ -4779,9 +4724,9 @@ dependencies = [ [[package]] name = "num_enum" -version = "0.7.4" +version = "0.7.5" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a973b4e44ce6cad84ce69d797acf9a044532e4184c4f267913d1b546a0727b7a" +checksum = "b1207a7e20ad57b847bbddc6776b968420d38292bbfe2089accff5e19e82454c" dependencies = [ "num_enum_derive", "rustversion", @@ -4789,14 +4734,14 @@ dependencies = [ [[package]] name = "num_enum_derive" -version = "0.7.4" +version = "0.7.5" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "77e878c846a8abae00dd069496dbe8751b16ac1c3d6bd2a7283a938e8228f90d" +checksum = "ff32365de1b6743cb203b710788263c44a03de03802daf96092f2da4fe6ba4d7" dependencies = [ "proc-macro-crate", "proc-macro2", "quote", - "syn 2.0.106", + "syn 2.0.107", ] [[package]] @@ -4807,9 +4752,9 @@ checksum = "830b246a0e5f20af87141b25c173cd1b609bd7779a4617d6ec582abaf90870f3" [[package]] name = "object_store" -version = "0.12.3" +version = "0.12.4" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "efc4f07659e11cd45a341cd24d71e683e3be65d9ff1f8150061678fe60437496" +checksum = "4c1be0c6c22ec0817cdc77d3842f721a17fd30ab6965001415b5402a74e6b740" dependencies = [ "async-trait", "base64 0.22.1", @@ -4833,7 +4778,7 @@ dependencies = [ "serde", "serde_json", "serde_urlencoded", - "thiserror 2.0.16", + "thiserror 2.0.17", "tokio", "tracing", "url", @@ -4856,9 +4801,9 @@ checksum = "a4895175b425cb1f87721b59f0f286c2092bd4af812243672510e1ac53e2e0ad" [[package]] name = "opendal" -version = "0.54.0" +version = "0.54.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "ffb9838d0575c6dbaf3fcec7255af8d5771996d4af900bbb6fa9a314dec00a1a" +checksum = "42afda58fa2cf50914402d132cc1caacff116a85d10c72ab2082bb7c50021754" dependencies = [ "anyhow", "backon", @@ -4873,7 +4818,7 @@ dependencies = [ "log", "md-5", "percent-encoding", - "quick-xml 0.37.5", + "quick-xml 0.38.3", "reqsign", "reqwest", "serde", @@ -4932,9 +4877,9 @@ checksum = "1a80800c0488c3a21695ea981a54918fbb37abf04f4d0720c453632255e2ff0e" [[package]] name = "owo-colors" -version = "4.2.2" +version = "4.2.3" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "48dd4f4a2c8405440fd0462561f0e5806bd0f77e86f51c761481bdd4018b545e" +checksum = "9c6901729fa79e91a0913333229e9ca5dc725089d1c363b2f4b4760709dc4a52" [[package]] name = "parking" @@ -4944,9 +4889,9 @@ checksum = "f38d5652c16fde515bb1ecef450ab0f6a219d619a7274976324d5e377f7dceba" [[package]] name = "parking_lot" -version = "0.12.4" +version = "0.12.5" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "70d58bf43669b5795d1576d0641cfb6fbb2057bf629506267a92807158584a13" +checksum = "93857453250e3077bd71ff98b6a65ea6621a19bb0f559a85248955ac12c45a1a" dependencies = [ "lock_api", "parking_lot_core", @@ -4954,15 +4899,15 @@ dependencies = [ [[package]] name = "parking_lot_core" -version = "0.9.11" +version = "0.9.12" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "bc838d2a56b5b1a6c25f55575dfc605fabb63bb2365f6c2353ef9159aa69e4a5" +checksum = "2621685985a2ebf1c516881c026032ac7deafcda1a2c9b7850dc81e3dfcb64c1" dependencies = [ "cfg-if", "libc", "redox_syscall", "smallvec", - "windows-targets 0.52.6", + "windows-link", ] [[package]] @@ -5065,12 +5010,12 @@ dependencies = [ [[package]] name = "pem" -version = "3.0.5" +version = "3.0.6" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "38af38e8470ac9dee3ce1bae1af9c1671fffc44ddfd8bd1d0a3445bf349a8ef3" +checksum = "1d30c53c26bc5b31a98cd02d20f25a7c8567146caf63ed593a9d87b2775291be" dependencies = [ "base64 0.22.1", - "serde", + "serde_core", ] [[package]] @@ -5090,20 +5035,19 @@ checksum = "9b4f627cb1b25917193a259e49bdad08f671f8d9708acfd5fe0a8c1455d87220" [[package]] name = "pest" -version = "2.8.1" +version = "2.8.3" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "1db05f56d34358a8b1066f67cbb203ee3e7ed2ba674a6263a1d5ec6db2204323" +checksum = "989e7521a040efde50c3ab6bbadafbe15ab6dc042686926be59ac35d74607df4" dependencies = [ "memchr", - "thiserror 2.0.16", "ucd-trie", ] [[package]] name = "pest_derive" -version = "2.8.1" +version = "2.8.3" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "bb056d9e8ea77922845ec74a1c4e8fb17e7c218cc4fc11a15c5d25e189aa40bc" +checksum = "187da9a3030dbafabbbfb20cb323b976dc7b7ce91fcd84f2f74d6e31d378e2de" dependencies = [ "pest", "pest_generator", @@ -5111,22 +5055,22 @@ dependencies = [ [[package]] name = "pest_generator" -version = "2.8.1" +version = "2.8.3" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "87e404e638f781eb3202dc82db6760c8ae8a1eeef7fb3fa8264b2ef280504966" +checksum = "49b401d98f5757ebe97a26085998d6c0eecec4995cad6ab7fc30ffdf4b052843" dependencies = [ "pest", "pest_meta", "proc-macro2", "quote", - "syn 2.0.106", + "syn 2.0.107", ] [[package]] name = "pest_meta" -version = "2.8.1" +version = "2.8.3" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "edd1101f170f5903fde0914f899bb503d9ff5271d7ba76bbb70bea63690cc0d5" +checksum = "72f27a2cfee9f9039c4d86faa5af122a0ac3851441a34865b8a043b46be0065a" dependencies = [ "pest", "sha2", @@ -5139,18 +5083,18 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "3672b37090dbd86368a4145bc067582552b29c27377cad4e0a306c97f9bd7772" dependencies = [ "fixedbitset", - "indexmap 2.11.0", + "indexmap 2.12.0", ] [[package]] name = "petgraph" -version = "0.8.2" +version = "0.8.3" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "54acf3a685220b533e437e264e4d932cfbdc4cc7ec0cd232ed73c08d03b8a7ca" +checksum = "8701b58ea97060d5e5b155d383a69952a60943f0e6dfe30b04c287beb0b27455" dependencies = [ "fixedbitset", "hashbrown 0.15.5", - "indexmap 2.11.0", + "indexmap 2.12.0", "serde", ] @@ -5249,7 +5193,7 @@ checksum = "6e918e4ff8c4549eb882f14b3a4bc8c8bc93de829416eacf579f1207a8fbf861" dependencies = [ "proc-macro2", "quote", - "syn 2.0.106", + "syn 2.0.107", ] [[package]] @@ -5321,16 +5265,16 @@ checksum = "7edddbd0b52d732b21ad9a5fab5c704c14cd949e5e9a1ec5929a24fded1b904c" [[package]] name = "polling" -version = "3.10.0" +version = "3.11.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "b5bd19146350fe804f7cb2669c851c03d69da628803dab0d98018142aaa5d829" +checksum = "5d0e4f59085d47d8241c88ead0f274e8a0cb551f3625263c05eb8dd897c34218" dependencies = [ "cfg-if", "concurrent-queue", "hermit-abi", "pin-project-lite", - "rustix 1.0.8", - "windows-sys 0.60.2", + "rustix", + "windows-sys 0.61.2", ] [[package]] @@ -5356,9 +5300,9 @@ dependencies = [ [[package]] name = "postgres-protocol" -version = "0.6.8" +version = "0.6.9" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "76ff0abab4a9b844b93ef7b81f1efc0a366062aaef2cd702c76256b5dc075c54" +checksum = "fbef655056b916eb868048276cfd5d6a7dea4f81560dfd047f97c8c6fe3fcfd4" dependencies = [ "base64 0.22.1", "byteorder", @@ -5374,9 +5318,9 @@ dependencies = [ [[package]] name = "postgres-types" -version = "0.2.9" +version = "0.2.11" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "613283563cd90e1dfc3518d548caee47e0e725455ed619881f5cf21f36de4b48" +checksum = "ef4605b7c057056dd35baeb6ac0c0338e4975b1f2bef0f65da953285eb007095" dependencies = [ "bytes", "fallible-iterator", @@ -5450,16 +5394,16 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "479ca8adacdd7ce8f1fb39ce9ecccbfe93a3f1344b3d0d97f20bc0196208f62b" dependencies = [ "proc-macro2", - "syn 2.0.106", + "syn 2.0.107", ] [[package]] name = "proc-macro-crate" -version = "3.3.0" +version = "3.4.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "edce586971a4dfaa28950c6f18ed55e0406c1ab88bbce2c6f6293a7aaba73d35" +checksum = "219cb19e96be00ab2e37d6e299658a0cfa83e52429179969b0f0121b4ac46983" dependencies = [ - "toml_edit", + "toml_edit 0.23.7", ] [[package]] @@ -5497,7 +5441,7 @@ dependencies = [ "prost", "prost-types", "regex", - "syn 2.0.106", + "syn 2.0.107", "tempfile", ] @@ -5511,7 +5455,7 @@ dependencies = [ "itertools 0.14.0", "proc-macro2", "quote", - "syn 2.0.106", + "syn 2.0.107", ] [[package]] @@ -5525,9 +5469,9 @@ dependencies = [ [[package]] name = "psm" -version = "0.1.26" +version = "0.1.27" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "6e944464ec8536cd1beb0bbfd96987eb5e3b72f2ecdafdc5c769a37f1fa2ae1f" +checksum = "e66fcd288453b748497d8fb18bccc83a16b0518e3906d4b8df0a8d42d93dbb1c" dependencies = [ "cc", ] @@ -5543,11 +5487,11 @@ dependencies = [ [[package]] name = "ptr_meta" -version = "0.3.0" +version = "0.3.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "fe9e76f66d3f9606f44e45598d155cb13ecf09f4a28199e48daf8c8fc937ea90" +checksum = "0b9a0cf95a1196af61d4f1cbdab967179516d9a4a4312af1f31948f8f6224a79" dependencies = [ - "ptr_meta_derive 0.3.0", + "ptr_meta_derive 0.3.1", ] [[package]] @@ -5563,13 +5507,13 @@ dependencies = [ [[package]] name = "ptr_meta_derive" -version = "0.3.0" +version = "0.3.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "ca414edb151b4c8d125c12566ab0d74dc9cdba36fb80eb7b848c15f495fd32d1" +checksum = "7347867d0a7e1208d93b46767be83e2b8f978c3dad35f775ac8d8847551d6fe1" dependencies = [ "proc-macro2", "quote", - "syn 2.0.106", + "syn 2.0.107", ] [[package]] @@ -5609,10 +5553,10 @@ dependencies = [ "pin-project-lite", "quinn-proto", "quinn-udp", - "rustc-hash 2.1.1", - "rustls 0.23.31", - "socket2 0.5.10", - "thiserror 2.0.16", + "rustc-hash", + "rustls 0.23.33", + "socket2 0.6.1", + "thiserror 2.0.17", "tokio", "tracing", "web-time", @@ -5625,15 +5569,15 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "f1906b49b0c3bc04b5fe5d86a77925ae6524a19b816ae38ce1e426255f1d8a31" dependencies = [ "bytes", - "getrandom 0.3.3", + "getrandom 0.3.4", "lru-slab", "rand 0.9.2", "ring", - "rustc-hash 2.1.1", - "rustls 0.23.31", + "rustc-hash", + "rustls 0.23.33", "rustls-pki-types", "slab", - "thiserror 2.0.16", + "thiserror 2.0.17", "tinyvec", "tracing", "web-time", @@ -5648,16 +5592,16 @@ dependencies = [ "cfg_aliases", "libc", "once_cell", - "socket2 0.5.10", + "socket2 0.6.1", "tracing", "windows-sys 0.60.2", ] [[package]] name = "quote" -version = "1.0.40" +version = "1.0.41" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "1885c039570dc00dcb4ff087a89e185fd56bae234ddc7f056a945bf36467248d" +checksum = "ce25767e7b499d1b604768e7cde645d14cc8584231ea6b295e9c9eb22c02e1d1" dependencies = [ "proc-macro2", ] @@ -5686,11 +5630,11 @@ dependencies = [ [[package]] name = "rancor" -version = "0.1.0" +version = "0.1.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "caf5f7161924b9d1cea0e4cabc97c372cea92b5f927fc13c6bca67157a0ad947" +checksum = "a063ea72381527c2a0561da9c80000ef822bdd7c3241b1cc1b12100e3df081ee" dependencies = [ - "ptr_meta 0.3.0", + "ptr_meta 0.3.1", ] [[package]] @@ -5751,7 +5695,7 @@ version = "0.9.3" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "99d9a13982dcf210057a8a78572b2217b667c3beacbf3a0d8b454f6f82837d38" dependencies = [ - "getrandom 0.3.3", + "getrandom 0.3.4", ] [[package]] @@ -5771,14 +5715,14 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "76009fbe0614077fc1a2ce255e3a1881a2e3a3527097d5dc6d8212c585e7e38b" dependencies = [ "quote", - "syn 2.0.106", + "syn 2.0.107", ] [[package]] name = "redox_syscall" -version = "0.5.17" +version = "0.5.18" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "5407465600fb0548f1442edf71dd20683c6ed326200ace4b1ef0763521bb3b77" +checksum = "ed2bf2547551a7053d6fdfafda3f938979645c44812fbfcda098faae3f1a362d" dependencies = [ "bitflags", ] @@ -5791,34 +5735,34 @@ checksum = "a4e608c6638b9c18977b00b475ac1f28d14e84b27d8d42f70e0bf1e3dec127ac" dependencies = [ "getrandom 0.2.16", "libredox", - "thiserror 2.0.16", + "thiserror 2.0.17", ] [[package]] name = "ref-cast" -version = "1.0.24" +version = "1.0.25" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "4a0ae411dbe946a674d89546582cea4ba2bb8defac896622d6496f14c23ba5cf" +checksum = "f354300ae66f76f1c85c5f84693f0ce81d747e2c3f21a45fef496d89c960bf7d" dependencies = [ "ref-cast-impl", ] [[package]] name = "ref-cast-impl" -version = "1.0.24" +version = "1.0.25" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "1165225c21bff1f3bbce98f5a1f889949bc902d3575308cc7b0de30b4f6d27c7" +checksum = "b7186006dcb21920990093f30e3dea63b7d6e977bf1256be20c3563a5db070da" dependencies = [ "proc-macro2", "quote", - "syn 2.0.106", + "syn 2.0.107", ] [[package]] name = "regex" -version = "1.12.1" +version = "1.12.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "4a52d8d02cacdb176ef4678de6c052efb4b3da14b78e4db683a4252762be5433" +checksum = "843bc0191f75f3e22651ae5f1e72939ab2f72a4bc30fa80a066bd66edefc24d4" dependencies = [ "aho-corasick", "memchr", @@ -5828,9 +5772,9 @@ dependencies = [ [[package]] name = "regex-automata" -version = "0.4.12" +version = "0.4.13" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "722166aa0d7438abbaa4d5cc2c649dac844e8c56d82fb3d33e9c34b5cd268fc6" +checksum = "5276caf25ac86c8d810222b3dbb938e512c55c6831a10f3e6ed1c93b84041f1c" dependencies = [ "aho-corasick", "memchr", @@ -5839,15 +5783,15 @@ dependencies = [ [[package]] name = "regex-lite" -version = "0.1.7" +version = "0.1.8" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "943f41321c63ef1c92fd763bfe054d2668f7f225a5c29f0105903dc2fc04ba30" +checksum = "8d942b98df5e658f56f20d592c7f868833fe38115e65c33003d8cd224b0155da" [[package]] name = "regex-syntax" -version = "0.8.6" +version = "0.8.8" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "caf4aa5b0f434c91fe5c7f1ecb6a5ece2130b02ad2a590589dda5146df959001" +checksum = "7a2d987857b319362043e95f5353c0535c1f58eec5336fdfcf626430af7def58" [[package]] name = "regress" @@ -5870,9 +5814,9 @@ dependencies = [ [[package]] name = "rend" -version = "0.5.2" +version = "0.5.3" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a35e8a6bf28cd121053a66aa2e6a2e3eaffad4a60012179f0e864aa5ffeff215" +checksum = "cadadef317c2f20755a64d7fdc48f9e7178ee6b0e1f7fce33fa60f1d68a276e6" [[package]] name = "reqsign" @@ -5908,9 +5852,9 @@ dependencies = [ [[package]] name = "reqwest" -version = "0.12.23" +version = "0.12.24" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d429f34c8092b2d42c7c93cec323bb4adeb7c67698f70839adec842ec10c7ceb" +checksum = "9d0946410b9f7b082a427e4ef5c8ff541a88b357bc6c637c40db3a68ac70a36f" dependencies = [ "base64 0.22.1", "bytes", @@ -5928,15 +5872,15 @@ dependencies = [ "percent-encoding", "pin-project-lite", "quinn", - "rustls 0.23.31", - "rustls-native-certs 0.8.1", + "rustls 0.23.33", + "rustls-native-certs 0.8.2", "rustls-pki-types", "serde", "serde_json", "serde_urlencoded", "sync_wrapper", "tokio", - "tokio-rustls 0.26.2", + "tokio-rustls 0.26.4", "tokio-util", "tower", "tower-http", @@ -5946,7 +5890,7 @@ dependencies = [ "wasm-bindgen-futures", "wasm-streams", "web-sys", - "webpki-roots 1.0.2", + "webpki-roots 1.0.3", ] [[package]] @@ -5983,18 +5927,18 @@ dependencies = [ [[package]] name = "rkyv" -version = "0.8.11" +version = "0.8.12" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "19f5c3e5da784cd8c69d32cdc84673f3204536ca56e1fa01be31a74b92c932ac" +checksum = "35a640b26f007713818e9a9b65d34da1cf58538207b052916a83d80e43f3ffa4" dependencies = [ "bytes", "hashbrown 0.15.5", - "indexmap 2.11.0", + "indexmap 2.12.0", "munge", - "ptr_meta 0.3.0", + "ptr_meta 0.3.1", "rancor", - "rend 0.5.2", - "rkyv_derive 0.8.11", + "rend 0.5.3", + "rkyv_derive 0.8.12", "tinyvec", "uuid", ] @@ -6012,13 +5956,13 @@ dependencies = [ [[package]] name = "rkyv_derive" -version = "0.8.11" +version = "0.8.12" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "4270433626cffc9c4c1d3707dd681f2a2718d3d7b09ad754bec137acecda8d22" +checksum = "bd83f5f173ff41e00337d97f6572e416d022ef8a19f371817259ae960324c482" dependencies = [ "proc-macro2", "quote", - "syn 2.0.106", + "syn 2.0.107", ] [[package]] @@ -6085,12 +6029,6 @@ dependencies = [ "serde_json", ] -[[package]] -name = "rustc-hash" -version = "1.1.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "08d43f7aa6b08d49f382cde6a7982047c3426db949b1424bc4b7ec9ae12c6ce2" - [[package]] name = "rustc-hash" version = "2.1.1" @@ -6111,28 +6049,15 @@ dependencies = [ [[package]] name = "rustix" -version = "0.38.44" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "fdb5bc1ae2baa591800df16c9ca78619bf65c0488b41b96ccec5d11220d8c154" -dependencies = [ - "bitflags", - "errno", - "libc", - "linux-raw-sys 0.4.15", - "windows-sys 0.52.0", -] - -[[package]] -name = "rustix" -version = "1.0.8" +version = "1.1.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "11181fbabf243db407ef8df94a6ce0b2f9a733bd8be4ad02b4eda9602296cac8" +checksum = "cd15f8a2c5551a84d56efdc1cd049089e409ac19a3072d5037a17fd70719ff3e" dependencies = [ "bitflags", "errno", "libc", - "linux-raw-sys 0.9.4", - "windows-sys 0.60.2", + "linux-raw-sys", + "windows-sys 0.61.2", ] [[package]] @@ -6149,15 +6074,15 @@ dependencies = [ [[package]] name = "rustls" -version = "0.23.31" +version = "0.23.33" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c0ebcbd2f03de0fc1122ad9bb24b127a5a6cd51d72604a3f3c50ac459762b6cc" +checksum = "751e04a496ca00bb97a5e043158d23d66b5aabf2e1d5aa2a0aaebb1aafe6f82c" dependencies = [ "aws-lc-rs", "once_cell", "ring", "rustls-pki-types", - "rustls-webpki 0.103.4", + "rustls-webpki 0.103.7", "subtle", "zeroize", ] @@ -6176,14 +6101,14 @@ dependencies = [ [[package]] name = "rustls-native-certs" -version = "0.8.1" +version = "0.8.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "7fcff2dd52b58a8d98a70243663a0d234c4e2b79235637849d15913394a247d3" +checksum = "9980d917ebb0c0536119ba501e90834767bffc3d60641457fd84a1f3fd337923" dependencies = [ "openssl-probe", "rustls-pki-types", "schannel", - "security-framework 3.4.0", + "security-framework 3.5.1", ] [[package]] @@ -6226,9 +6151,9 @@ dependencies = [ [[package]] name = "rustls-webpki" -version = "0.103.4" +version = "0.103.7" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "0a17884ae0c1b773f1ccd2bd4a8c72f16da897310a98b0e84bf349ad5ead92fc" +checksum = "e10b3f4191e8a80e6b43eebabfac91e5dcecebb27a71f04e820c47ec41d314bf" dependencies = [ "aws-lc-rs", "ring", @@ -6259,7 +6184,7 @@ dependencies = [ "nix 0.30.1", "radix_trie", "unicode-segmentation", - "unicode-width 0.2.1", + "unicode-width 0.2.2", "utf8parse", "windows-sys 0.59.0", ] @@ -6290,11 +6215,11 @@ dependencies = [ [[package]] name = "schannel" -version = "0.1.27" +version = "0.1.28" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "1f29ebaa345f945cec9fbbc532eb307f0fdad8161f281b6369539c8d84876b3d" +checksum = "891d81b926048e76efe18581bf793546b4c0eaf8448d72be8de2bbee5fd166e1" dependencies = [ - "windows-sys 0.59.0", + "windows-sys 0.61.2", ] [[package]] @@ -6342,15 +6267,9 @@ dependencies = [ "proc-macro2", "quote", "serde_derive_internals", - "syn 2.0.106", + "syn 2.0.107", ] -[[package]] -name = "scoped-tls" -version = "1.0.1" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "e1cf6437eb19a8f4a6cc0f7dca544973b0b78843adbfeb3683d1a94a0024a294" - [[package]] name = "scopeguard" version = "1.2.0" @@ -6399,9 +6318,9 @@ dependencies = [ [[package]] name = "security-framework" -version = "3.4.0" +version = "3.5.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "60b369d18893388b345804dc0007963c99b7d665ae71d275812d828c6f089640" +checksum = "b3297343eaf830f66ede390ea39da1d462b6b0c1b000f420d0a83f898bbbe6ef" dependencies = [ "bitflags", "core-foundation 0.10.1", @@ -6422,11 +6341,12 @@ dependencies = [ [[package]] name = "semver" -version = "1.0.26" +version = "1.0.27" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "56e6fa9c48d24d85fb3de5ad847117517440f6beceb7798af16b4a87d616b8d0" +checksum = "d767eb0aabc880b29956c35734170f26ed551a859dbd361d140cdbeca61ab1e2" dependencies = [ "serde", + "serde_core", ] [[package]] @@ -6437,31 +6357,42 @@ checksum = "1bc711410fbe7399f390ca1c3b60ad0f53f80e95c5eb935e52268a0e2cd49acc" [[package]] name = "serde" -version = "1.0.219" +version = "1.0.228" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "5f0e2c6ed6606019b4e29e69dbaba95b11854410e5347d525002456dbbb786b6" +checksum = "9a8e94ea7f378bd32cbbd37198a4a91436180c5bb472411e48b5ec2e2124ae9e" dependencies = [ + "serde_core", "serde_derive", ] [[package]] name = "serde_bytes" -version = "0.11.17" +version = "0.11.19" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "8437fd221bde2d4ca316d61b90e337e9e702b3820b87d63caa9ba6c02bd06d96" +checksum = "a5d440709e79d88e51ac01c4b72fc6cb7314017bb7da9eeff678aa94c10e3ea8" dependencies = [ "serde", + "serde_core", +] + +[[package]] +name = "serde_core" +version = "1.0.228" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "41d385c7d4ca58e59fc732af25c3983b67ac852c1a25000afe1175de458b67ad" +dependencies = [ + "serde_derive", ] [[package]] name = "serde_derive" -version = "1.0.219" +version = "1.0.228" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "5b0276cf7f2c73365f7157c8123c21cd9a50fbbd844757af28ca1f5925fc2a00" +checksum = "d540f220d3187173da220f885ab66608367b6574e925011a9353e4badda91d79" dependencies = [ "proc-macro2", "quote", - "syn 2.0.106", + "syn 2.0.107", ] [[package]] @@ -6472,19 +6403,20 @@ checksum = "18d26a20a969b9e3fdf2fc2d9f21eda6c40e2de84c9408bb5d3b05d499aae711" dependencies = [ "proc-macro2", "quote", - "syn 2.0.106", + "syn 2.0.107", ] [[package]] name = "serde_json" -version = "1.0.143" +version = "1.0.145" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d401abef1d108fbd9cbaebc3e46611f4b1021f714a0597a71f41ee463f5f4a5a" +checksum = "402a6f66d8c709116cf22f558eab210f5a50187f702eb4d7e5ef38d9a7f1c79c" dependencies = [ "itoa", "memchr", "ryu", "serde", + "serde_core", ] [[package]] @@ -6495,7 +6427,7 @@ checksum = "175ee3e80ae9982737ca543e96133087cbd9a485eecc3bc4de9c1a37b47ea59c" dependencies = [ "proc-macro2", "quote", - "syn 2.0.106", + "syn 2.0.107", ] [[package]] @@ -6516,7 +6448,7 @@ dependencies = [ "proc-macro2", "quote", "serde", - "syn 2.0.106", + "syn 2.0.107", ] [[package]] @@ -6533,19 +6465,18 @@ dependencies = [ [[package]] name = "serde_with" -version = "3.14.1" +version = "3.15.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c522100790450cf78eeac1507263d0a350d4d5b30df0c8e1fe051a10c22b376e" +checksum = "6093cd8c01b25262b84927e0f7151692158fab02d961e04c979d3903eba7ecc5" dependencies = [ "base64 0.22.1", "chrono", "hex", "indexmap 1.9.3", - "indexmap 2.11.0", + "indexmap 2.12.0", "schemars 0.9.0", "schemars 1.0.4", - "serde", - "serde_derive", + "serde_core", "serde_json", "serde_with_macros", "time", @@ -6553,14 +6484,14 @@ dependencies = [ [[package]] name = "serde_with_macros" -version = "3.14.1" +version = "3.15.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "327ada00f7d64abaac1e55a6911e90cf665aa051b9a561c7006c157f4633135e" +checksum = "a7e6c180db0816026a61afa1cff5344fb7ebded7e4d3062772179f2501481c27" dependencies = [ "darling 0.21.3", "proc-macro2", "quote", - "syn 2.0.106", + "syn 2.0.107", ] [[package]] @@ -6569,7 +6500,7 @@ version = "0.9.34+deprecated" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "6a8b1a1a2ebf674015cc02edccce75287f1a0130d394307b36743c2f5d504b47" dependencies = [ - "indexmap 2.11.0", + "indexmap 2.12.0", "itoa", "ryu", "serde", @@ -6632,6 +6563,12 @@ dependencies = [ "rand_core 0.6.4", ] +[[package]] +name = "simd-adler32" +version = "0.3.7" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "d66dc143e6b11c1eddc06d5c423cfc97062865baf299914ab64caa38182078fe" + [[package]] name = "simdutf8" version = "0.1.5" @@ -6652,7 +6589,7 @@ checksum = "297f631f50729c8c99b84667867963997ec0b50f32b2a7dbcab828ef0541e8bb" dependencies = [ "num-bigint", "num-traits", - "thiserror 2.0.16", + "thiserror 2.0.17", "time", ] @@ -6722,12 +6659,12 @@ dependencies = [ [[package]] name = "socket2" -version = "0.6.0" +version = "0.6.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "233504af464074f9d066d7b5416c5f9b894a5862a6506e306f7b816cdd6f1807" +checksum = "17129e116933cf371d018bb80ae557e889637989d8638274fb25622827b03881" dependencies = [ "libc", - "windows-sys 0.59.0", + "windows-sys 0.60.2", ] [[package]] @@ -6756,7 +6693,7 @@ dependencies = [ "simdutf8", "sonic-number", "sonic-simd", - "thiserror 2.0.16", + "thiserror 2.0.17", ] [[package]] @@ -6808,7 +6745,7 @@ dependencies = [ "similar", "subst", "tempfile", - "thiserror 2.0.16", + "thiserror 2.0.17", "tracing", ] @@ -6831,7 +6768,7 @@ checksum = "da5fc6819faabb412da764b99d3b713bb55083c11e7e0c00144d386cd6a1939c" dependencies = [ "proc-macro2", "quote", - "syn 2.0.106", + "syn 2.0.107", ] [[package]] @@ -6865,17 +6802,17 @@ dependencies = [ "futures-util", "hashbrown 0.15.5", "hashlink", - "indexmap 2.11.0", + "indexmap 2.12.0", "log", "memchr", "once_cell", "percent-encoding", - "rustls 0.23.31", + "rustls 0.23.33", "serde", "serde_json", "sha2", "smallvec", - "thiserror 2.0.16", + "thiserror 2.0.17", "tokio", "tokio-stream", "tracing", @@ -6893,7 +6830,7 @@ dependencies = [ "quote", "sqlx-core", "sqlx-macros-core", - "syn 2.0.106", + "syn 2.0.107", ] [[package]] @@ -6914,7 +6851,7 @@ dependencies = [ "sha2", "sqlx-core", "sqlx-sqlite", - "syn 2.0.106", + "syn 2.0.107", "tokio", "url", ] @@ -6955,7 +6892,7 @@ dependencies = [ "smallvec", "sqlx-core", "stringprep", - "thiserror 2.0.16", + "thiserror 2.0.17", "tracing", "whoami", ] @@ -6992,7 +6929,7 @@ dependencies = [ "smallvec", "sqlx-core", "stringprep", - "thiserror 2.0.16", + "thiserror 2.0.17", "tracing", "whoami", ] @@ -7016,36 +6953,30 @@ dependencies = [ "serde", "serde_urlencoded", "sqlx-core", - "thiserror 2.0.16", + "thiserror 2.0.17", "tracing", "url", ] [[package]] name = "stable_deref_trait" -version = "1.2.0" +version = "1.2.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a8f112729512f8e442d81f95a8a7ddf2b7c6b8a1a6f509a95864142b30cab2d3" +checksum = "6ce2be8dc25455e1f91df71bfa12ad37d7af1092ae736f3a6cd0e37bc7810596" [[package]] name = "stacker" -version = "0.1.21" +version = "0.1.22" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "cddb07e32ddb770749da91081d8d0ac3a16f1a569a18b20348cd371f5dead06b" +checksum = "e1f8b29fb42aafcea4edeeb6b2f2d7ecd0d969c48b4cf0d2e64aafc471dd6e59" dependencies = [ "cc", "cfg-if", "libc", "psm", - "windows-sys 0.52.0", + "windows-sys 0.59.0", ] -[[package]] -name = "static_assertions" -version = "1.1.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a2eb9349b6444b326872e140eb1cf5e7c522154d69e7a0ffb0fb81c06b37543f" - [[package]] name = "stringprep" version = "0.1.5" @@ -7088,7 +7019,7 @@ dependencies = [ "proc-macro2", "quote", "rustversion", - "syn 2.0.106", + "syn 2.0.107", ] [[package]] @@ -7100,7 +7031,7 @@ dependencies = [ "heck", "proc-macro2", "quote", - "syn 2.0.106", + "syn 2.0.107", ] [[package]] @@ -7133,7 +7064,7 @@ dependencies = [ "serde", "serde_json", "serde_yaml", - "syn 2.0.106", + "syn 2.0.107", "typify", "walkdir", ] @@ -7157,9 +7088,9 @@ dependencies = [ [[package]] name = "syn" -version = "2.0.106" +version = "2.0.107" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "ede7c438028d4436d71104916910f5bb611972c5cfd7f89b8300a8186e6fada6" +checksum = "2a26dbd934e5451d21ef060c018dae56fc073894c5a7896f882928a76e6d081b" dependencies = [ "proc-macro2", "quote", @@ -7183,7 +7114,7 @@ checksum = "728a70f3dbaf5bab7f0c4b1ac8d7ae5ea60a4b5549c8a5914361c99147a709d2" dependencies = [ "proc-macro2", "quote", - "syn 2.0.106", + "syn 2.0.107", ] [[package]] @@ -7205,10 +7136,10 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "2d31c77bdf42a745371d260a26ca7163f1e0924b64afa0b688e61b5a9fa02f16" dependencies = [ "fastrand", - "getrandom 0.3.3", + "getrandom 0.3.4", "once_cell", - "rustix 1.0.8", - "windows-sys 0.61.0", + "rustix", + "windows-sys 0.61.2", ] [[package]] @@ -7250,11 +7181,11 @@ dependencies = [ [[package]] name = "thiserror" -version = "2.0.16" +version = "2.0.17" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "3467d614147380f2e4e374161426ff399c91084acd2363eaf549172b3d5e60c0" +checksum = "f63587ca0f12b72a0600bcba1d40081f830876000bb46dd2337a3051618f4fc8" dependencies = [ - "thiserror-impl 2.0.16", + "thiserror-impl 2.0.17", ] [[package]] @@ -7265,18 +7196,18 @@ checksum = "4fee6c4efc90059e10f81e6d42c60a18f76588c3d74cb83a0b242a2b6c7504c1" dependencies = [ "proc-macro2", "quote", - "syn 2.0.106", + "syn 2.0.107", ] [[package]] name = "thiserror-impl" -version = "2.0.16" +version = "2.0.17" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "6c5e1be1c48b9172ee610da68fd9cd2770e7a4056cb3fc98710ee6906f0c7960" +checksum = "3ff15c8ecd7de3849db632e14d18d2571fa09dfc5ed93479bc4485c7a517c913" dependencies = [ "proc-macro2", "quote", - "syn 2.0.106", + "syn 2.0.107", ] [[package]] @@ -7312,11 +7243,12 @@ dependencies = [ [[package]] name = "time" -version = "0.3.43" +version = "0.3.44" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "83bde6f1ec10e72d583d91623c939f623002284ef622b87de38cfd546cbf2031" +checksum = "91e7d9e3bb61134e77bde20dd4825b97c010155709965fedf0f49bb138e52a9d" dependencies = [ "deranged", + "itoa", "num-conv", "powerfmt", "serde", @@ -7386,9 +7318,9 @@ dependencies = [ "parking_lot", "pin-project-lite", "signal-hook-registry", - "socket2 0.6.0", + "socket2 0.6.1", "tokio-macros", - "windows-sys 0.61.0", + "windows-sys 0.61.2", ] [[package]] @@ -7399,7 +7331,7 @@ checksum = "af407857209536a95c8e56f8231ef2c2e2aff839b22e07a1ffcbc617e9db9fa5" dependencies = [ "proc-macro2", "quote", - "syn 2.0.106", + "syn 2.0.107", ] [[package]] @@ -7414,11 +7346,11 @@ dependencies = [ [[package]] name = "tokio-rustls" -version = "0.26.2" +version = "0.26.4" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "8e727b36a1a0e8b74c376ac2211e40c2c8af09fb4013c60d910495810f008e9b" +checksum = "1729aa945f29d91ba541258c8df89027d5792d85a8841fb65e8bf0f4ede4ef61" dependencies = [ - "rustls 0.23.31", + "rustls 0.23.33", "tokio", ] @@ -7454,8 +7386,8 @@ checksum = "dc1beb996b9d83529a9e75c17a1686767d148d70663143c7854d8b4a09ced362" dependencies = [ "serde", "serde_spanned", - "toml_datetime", - "toml_edit", + "toml_datetime 0.6.11", + "toml_edit 0.22.27", ] [[package]] @@ -7467,20 +7399,50 @@ dependencies = [ "serde", ] +[[package]] +name = "toml_datetime" +version = "0.7.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "f2cdb639ebbc97961c51720f858597f7f24c4fc295327923af55b74c3c724533" +dependencies = [ + "serde_core", +] + [[package]] name = "toml_edit" version = "0.22.27" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "41fe8c660ae4257887cf66394862d21dbca4a6ddd26f04a3560410406a2f819a" dependencies = [ - "indexmap 2.11.0", + "indexmap 2.12.0", "serde", "serde_spanned", - "toml_datetime", + "toml_datetime 0.6.11", "toml_write", "winnow", ] +[[package]] +name = "toml_edit" +version = "0.23.7" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "6485ef6d0d9b5d0ec17244ff7eb05310113c3f316f2d14200d4de56b3cb98f8d" +dependencies = [ + "indexmap 2.12.0", + "toml_datetime 0.7.3", + "toml_parser", + "winnow", +] + +[[package]] +name = "toml_parser" +version = "1.0.4" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "c0cbe268d35bdb4bb5a56a2de88d0ad0eb70af5384a99d648cd4b3d04039800e" +dependencies = [ + "winnow", +] + [[package]] name = "toml_write" version = "0.1.2" @@ -7552,7 +7514,7 @@ checksum = "81383ab64e72a7a8b8e13130c49e3dab29def6d0c7d76a03087b3cf71c5c6903" dependencies = [ "proc-macro2", "quote", - "syn 2.0.106", + "syn 2.0.107", ] [[package]] @@ -7582,14 +7544,10 @@ version = "0.3.20" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "2054a14f5307d601f88daf0553e1cbf472acc4f2c51afab632431cdcd72124d5" dependencies = [ - "matchers", "nu-ansi-term", - "once_cell", - "regex-automata", "sharded-slab", "smallvec", "thread_local", - "tracing", "tracing-core", "tracing-log", ] @@ -7632,7 +7590,7 @@ checksum = "f9534daa9fd3ed0bd911d462a37f172228077e7abf18c18a5f67199d959205f8" dependencies = [ "proc-macro2", "quote", - "syn 2.0.106", + "syn 2.0.107", ] [[package]] @@ -7643,14 +7601,14 @@ checksum = "3c36781cc0e46a83726d9879608e4cf6c2505237e263a8eb8c24502989cfdb28" dependencies = [ "proc-macro2", "quote", - "syn 2.0.106", + "syn 2.0.107", ] [[package]] name = "typenum" -version = "1.18.0" +version = "1.19.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "1dccffe3ce07af9386bfd29e80c0ab1a8205a2fc34e4bcd40364df902cfa8f3f" +checksum = "562d481066bde0658276a35467c4af00bdc6ee726305698a55b86e61d7ad82bb" [[package]] name = "typify" @@ -7677,8 +7635,8 @@ dependencies = [ "semver", "serde", "serde_json", - "syn 2.0.106", - "thiserror 2.0.16", + "syn 2.0.107", + "thiserror 2.0.17", "unicode-ident", ] @@ -7695,7 +7653,7 @@ dependencies = [ "serde", "serde_json", "serde_tokenstream", - "syn 2.0.106", + "syn 2.0.107", "typify-impl", ] @@ -7763,9 +7721,9 @@ checksum = "5c1cb5db39152898a79168971543b1cb5020dff7fe43c8dc468b0885f5e29df5" [[package]] name = "unicode-ident" -version = "1.0.18" +version = "1.0.19" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "5a5f39404a5da50712a4c1eecf25e90dd62b613502b7e925fd4e4d19b5c96512" +checksum = "f63a545481291138910575129486daeaf8ac54aee4387fe7906919f7830c7d9d" [[package]] name = "unicode-normalization" @@ -7796,9 +7754,9 @@ checksum = "7dd6e30e90baa6f72411720665d41d89b9a3d039dc45b8faea1ddd07f617f6af" [[package]] name = "unicode-width" -version = "0.2.1" +version = "0.2.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "4a1a07cc7db3810833284e8d372ccdc6da29741639ecc70c9ec107df0fa6154c" +checksum = "b4ac048d71ede7ee76d585517add45da530660ef4390e49b098733c6e897f254" [[package]] name = "unsafe-libyaml" @@ -7848,7 +7806,7 @@ version = "1.18.1" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "2f87b8aa10b915a06587d0dec516c282ff295b475d94abf425d62b57710070a2" dependencies = [ - "getrandom 0.3.3", + "getrandom 0.3.4", "js-sys", "serde", "wasm-bindgen", @@ -7891,7 +7849,7 @@ dependencies = [ "pin-project", "rand 0.9.2", "socket2 0.5.10", - "thiserror 2.0.16", + "thiserror 2.0.17", "tokio", "tokio-stream", "tower", @@ -7919,10 +7877,10 @@ dependencies = [ "paste", "pilota", "pin-project", - "rustc-hash 2.1.1", + "rustc-hash", "scopeguard", "sonic-rs", - "thiserror 2.0.16", + "thiserror 2.0.17", "tokio", "tracing", "volo", @@ -7960,10 +7918,10 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "ccf3ec651a847eb01de73ccad15eb7d99f80485de043efb2f370cd654f4ea44b" [[package]] -name = "wasi" -version = "0.14.4+wasi-0.2.4" +name = "wasip2" +version = "1.0.1+wasi-0.2.4" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "88a5f4a424faf49c3c2c344f166f0662341d470ea185e939657aaff130f0ec4a" +checksum = "0562428422c63773dad2c345a1882263bbf4d65cf3f42e90921f787ef5ad58e7" dependencies = [ "wit-bindgen", ] @@ -7976,9 +7934,9 @@ checksum = "b8dad83b4f25e74f184f64c43b150b91efe7647395b42289f38e50566d82855b" [[package]] name = "wasm-bindgen" -version = "0.2.101" +version = "0.2.104" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "7e14915cadd45b529bb8d1f343c4ed0ac1de926144b746e2710f9cd05df6603b" +checksum = "c1da10c01ae9f1ae40cbfac0bac3b1e724b320abfcf52229f80b547c0d250e2d" dependencies = [ "cfg-if", "once_cell", @@ -7989,23 +7947,23 @@ dependencies = [ [[package]] name = "wasm-bindgen-backend" -version = "0.2.101" +version = "0.2.104" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "e28d1ba982ca7923fd01448d5c30c6864d0a14109560296a162f80f305fb93bb" +checksum = "671c9a5a66f49d8a47345ab942e2cb93c7d1d0339065d4f8139c486121b43b19" dependencies = [ "bumpalo", "log", "proc-macro2", "quote", - "syn 2.0.106", + "syn 2.0.107", "wasm-bindgen-shared", ] [[package]] name = "wasm-bindgen-futures" -version = "0.4.51" +version = "0.4.54" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "0ca85039a9b469b38336411d6d6ced91f3fc87109a2a27b0c197663f5144dffe" +checksum = "7e038d41e478cc73bae0ff9b36c60cff1c98b8f38f8d7e8061e79ee63608ac5c" dependencies = [ "cfg-if", "js-sys", @@ -8016,9 +7974,9 @@ dependencies = [ [[package]] name = "wasm-bindgen-macro" -version = "0.2.101" +version = "0.2.104" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "7c3d463ae3eff775b0c45df9da45d68837702ac35af998361e2c84e7c5ec1b0d" +checksum = "7ca60477e4c59f5f2986c50191cd972e3a50d8a95603bc9434501cf156a9a119" dependencies = [ "quote", "wasm-bindgen-macro-support", @@ -8026,22 +7984,22 @@ dependencies = [ [[package]] name = "wasm-bindgen-macro-support" -version = "0.2.101" +version = "0.2.104" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "7bb4ce89b08211f923caf51d527662b75bdc9c9c7aab40f86dcb9fb85ac552aa" +checksum = "9f07d2f20d4da7b26400c9f4a0511e6e0345b040694e8a75bd41d578fa4421d7" dependencies = [ "proc-macro2", "quote", - "syn 2.0.106", + "syn 2.0.107", "wasm-bindgen-backend", "wasm-bindgen-shared", ] [[package]] name = "wasm-bindgen-shared" -version = "0.2.101" +version = "0.2.104" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "f143854a3b13752c6950862c906306adb27c7e839f7414cec8fea35beab624c1" +checksum = "bad67dc8b2a1a6e5448428adec4c3e84c43e561d8c9ee8a9e5aabeb193ec41d1" dependencies = [ "unicode-ident", ] @@ -8061,9 +8019,9 @@ dependencies = [ [[package]] name = "web-sys" -version = "0.3.78" +version = "0.3.81" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "77e4b637749ff0d92b8fad63aa1f7cff3cbe125fd49c175cd6345e7272638b12" +checksum = "9367c417a924a74cae129e6a2ae3b47fabb1f8995595ab474029da749a8be120" dependencies = [ "js-sys", "wasm-bindgen", @@ -8085,30 +8043,18 @@ version = "0.26.11" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "521bc38abb08001b01866da9f51eb7c5d647a19260e00054a8c7fd5f9e57f7a9" dependencies = [ - "webpki-roots 1.0.2", + "webpki-roots 1.0.3", ] [[package]] name = "webpki-roots" -version = "1.0.2" +version = "1.0.3" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "7e8983c3ab33d6fb807cfcdad2491c4ea8cbc8ed839181c7dfd9c67c83e261b2" +checksum = "32b130c0d2d49f8b6889abc456e795e82525204f27c42cf767cf0d7734e089b8" dependencies = [ "rustls-pki-types", ] -[[package]] -name = "which" -version = "4.4.2" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "87ba24419a2078cd2b0f2ede2691b6c66d8e47836da3b6db8265ebad47afbfc7" -dependencies = [ - "either", - "home", - "once_cell", - "rustix 0.38.44", -] - [[package]] name = "whoami" version = "1.6.1" @@ -8125,115 +8071,66 @@ version = "0.1.11" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "c2a7b1c03c876122aa43f3020e6c3c3ee5c05081c9a00739faf7503aeba10d22" dependencies = [ - "windows-sys 0.61.0", -] - -[[package]] -name = "windows" -version = "0.61.3" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "9babd3a767a4c1aef6900409f85f5d53ce2544ccdfaa86dad48c91782c6d6893" -dependencies = [ - "windows-collections", - "windows-core", - "windows-future", - "windows-link 0.1.3", - "windows-numerics", -] - -[[package]] -name = "windows-collections" -version = "0.2.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "3beeceb5e5cfd9eb1d76b381630e82c4241ccd0d27f1a39ed41b2760b255c5e8" -dependencies = [ - "windows-core", + "windows-sys 0.61.2", ] [[package]] name = "windows-core" -version = "0.61.2" +version = "0.62.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c0fdd3ddb90610c7638aa2b3a3ab2904fb9e5cdbecc643ddb3647212781c4ae3" +checksum = "b8e83a14d34d0623b51dce9581199302a221863196a1dde71a7663a4c2be9deb" dependencies = [ "windows-implement", "windows-interface", - "windows-link 0.1.3", + "windows-link", "windows-result", "windows-strings", ] -[[package]] -name = "windows-future" -version = "0.2.1" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "fc6a41e98427b19fe4b73c550f060b59fa592d7d686537eebf9385621bfbad8e" -dependencies = [ - "windows-core", - "windows-link 0.1.3", - "windows-threading", -] - [[package]] name = "windows-implement" -version = "0.60.0" +version = "0.60.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a47fddd13af08290e67f4acabf4b459f647552718f683a7b415d290ac744a836" +checksum = "053e2e040ab57b9dc951b72c264860db7eb3b0200ba345b4e4c3b14f67855ddf" dependencies = [ "proc-macro2", "quote", - "syn 2.0.106", + "syn 2.0.107", ] [[package]] name = "windows-interface" -version = "0.59.1" +version = "0.59.3" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "bd9211b69f8dcdfa817bfd14bf1c97c9188afa36f4750130fcdf3f400eca9fa8" +checksum = "3f316c4a2570ba26bbec722032c4099d8c8bc095efccdc15688708623367e358" dependencies = [ "proc-macro2", "quote", - "syn 2.0.106", + "syn 2.0.107", ] [[package]] name = "windows-link" -version = "0.1.3" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "5e6ad25900d524eaabdbbb96d20b4311e1e7ae1699af4fb28c17ae66c80d798a" - -[[package]] -name = "windows-link" -version = "0.2.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "45e46c0661abb7180e7b9c281db115305d49ca1709ab8242adf09666d2173c65" - -[[package]] -name = "windows-numerics" -version = "0.2.0" +version = "0.2.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "9150af68066c4c5c07ddc0ce30421554771e528bde427614c61038bc2c92c2b1" -dependencies = [ - "windows-core", - "windows-link 0.1.3", -] +checksum = "f0805222e57f7521d6a62e36fa9163bc891acd422f971defe97d64e70d0a4fe5" [[package]] name = "windows-result" -version = "0.3.4" +version = "0.4.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "56f42bd332cc6c8eac5af113fc0c1fd6a8fd2aa08a0119358686e5160d0586c6" +checksum = "7781fa89eaf60850ac3d2da7af8e5242a5ea78d1a11c49bf2910bb5a73853eb5" dependencies = [ - "windows-link 0.1.3", + "windows-link", ] [[package]] name = "windows-strings" -version = "0.4.2" +version = "0.5.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "56e6c93f3a0c3b36176cb1327a4958a0353d5d166c2a35cb268ace15e91d3b57" +checksum = "7837d08f69c77cf6b07689544538e017c1bfcf57e34b4c0ff58e6c2cd3b37091" dependencies = [ - "windows-link 0.1.3", + "windows-link", ] [[package]] @@ -8269,16 +8166,16 @@ version = "0.60.2" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "f2f500e4d28234f72040990ec9d39e3a6b950f9f22d3dba18416c35882612bcb" dependencies = [ - "windows-targets 0.53.3", + "windows-targets 0.53.5", ] [[package]] name = "windows-sys" -version = "0.61.0" +version = "0.61.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "e201184e40b2ede64bc2ea34968b28e33622acdbbf37104f0e4a33f7abe657aa" +checksum = "ae137229bcbd6cdf0f7b80a31df61766145077ddf49416a728b02cb3921ff3fc" dependencies = [ - "windows-link 0.2.0", + "windows-link", ] [[package]] @@ -8314,28 +8211,19 @@ dependencies = [ [[package]] name = "windows-targets" -version = "0.53.3" +version = "0.53.5" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d5fe6031c4041849d7c496a8ded650796e7b6ecc19df1a431c1a363342e5dc91" +checksum = "4945f9f551b88e0d65f3db0bc25c33b8acea4d9e41163edf90dcd0b19f9069f3" dependencies = [ - "windows-link 0.1.3", - "windows_aarch64_gnullvm 0.53.0", - "windows_aarch64_msvc 0.53.0", - "windows_i686_gnu 0.53.0", - "windows_i686_gnullvm 0.53.0", - "windows_i686_msvc 0.53.0", - "windows_x86_64_gnu 0.53.0", - "windows_x86_64_gnullvm 0.53.0", - "windows_x86_64_msvc 0.53.0", -] - -[[package]] -name = "windows-threading" -version = "0.1.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "b66463ad2e0ea3bbf808b7f1d371311c80e115c0b71d60efc142cafbcfb057a6" -dependencies = [ - "windows-link 0.1.3", + "windows-link", + "windows_aarch64_gnullvm 0.53.1", + "windows_aarch64_msvc 0.53.1", + "windows_i686_gnu 0.53.1", + "windows_i686_gnullvm 0.53.1", + "windows_i686_msvc 0.53.1", + "windows_x86_64_gnu 0.53.1", + "windows_x86_64_gnullvm 0.53.1", + "windows_x86_64_msvc 0.53.1", ] [[package]] @@ -8352,9 +8240,9 @@ checksum = "32a4622180e7a0ec044bb555404c800bc9fd9ec262ec147edd5989ccd0c02cd3" [[package]] name = "windows_aarch64_gnullvm" -version = "0.53.0" +version = "0.53.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "86b8d5f90ddd19cb4a147a5fa63ca848db3df085e25fee3cc10b39b6eebae764" +checksum = "a9d8416fa8b42f5c947f8482c43e7d89e73a173cead56d044f6a56104a6d1b53" [[package]] name = "windows_aarch64_msvc" @@ -8370,9 +8258,9 @@ checksum = "09ec2a7bb152e2252b53fa7803150007879548bc709c039df7627cabbd05d469" [[package]] name = "windows_aarch64_msvc" -version = "0.53.0" +version = "0.53.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c7651a1f62a11b8cbd5e0d42526e55f2c99886c77e007179efff86c2b137e66c" +checksum = "b9d782e804c2f632e395708e99a94275910eb9100b2114651e04744e9b125006" [[package]] name = "windows_i686_gnu" @@ -8388,9 +8276,9 @@ checksum = "8e9b5ad5ab802e97eb8e295ac6720e509ee4c243f69d781394014ebfe8bbfa0b" [[package]] name = "windows_i686_gnu" -version = "0.53.0" +version = "0.53.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c1dc67659d35f387f5f6c479dc4e28f1d4bb90ddd1a5d3da2e5d97b42d6272c3" +checksum = "960e6da069d81e09becb0ca57a65220ddff016ff2d6af6a223cf372a506593a3" [[package]] name = "windows_i686_gnullvm" @@ -8400,9 +8288,9 @@ checksum = "0eee52d38c090b3caa76c563b86c3a4bd71ef1a819287c19d586d7334ae8ed66" [[package]] name = "windows_i686_gnullvm" -version = "0.53.0" +version = "0.53.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "9ce6ccbdedbf6d6354471319e781c0dfef054c81fbc7cf83f338a4296c0cae11" +checksum = "fa7359d10048f68ab8b09fa71c3daccfb0e9b559aed648a8f95469c27057180c" [[package]] name = "windows_i686_msvc" @@ -8418,9 +8306,9 @@ checksum = "240948bc05c5e7c6dabba28bf89d89ffce3e303022809e73deaefe4f6ec56c66" [[package]] name = "windows_i686_msvc" -version = "0.53.0" +version = "0.53.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "581fee95406bb13382d2f65cd4a908ca7b1e4c2f1917f143ba16efe98a589b5d" +checksum = "1e7ac75179f18232fe9c285163565a57ef8d3c89254a30685b57d83a38d326c2" [[package]] name = "windows_x86_64_gnu" @@ -8436,9 +8324,9 @@ checksum = "147a5c80aabfbf0c7d901cb5895d1de30ef2907eb21fbbab29ca94c5b08b1a78" [[package]] name = "windows_x86_64_gnu" -version = "0.53.0" +version = "0.53.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "2e55b5ac9ea33f2fc1716d1742db15574fd6fc8dadc51caab1c16a3d3b4190ba" +checksum = "9c3842cdd74a865a8066ab39c8a7a473c0778a3f29370b5fd6b4b9aa7df4a499" [[package]] name = "windows_x86_64_gnullvm" @@ -8454,9 +8342,9 @@ checksum = "24d5b23dc417412679681396f2b49f3de8c1473deb516bd34410872eff51ed0d" [[package]] name = "windows_x86_64_gnullvm" -version = "0.53.0" +version = "0.53.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "0a6e035dd0599267ce1ee132e51c27dd29437f63325753051e71dd9e42406c57" +checksum = "0ffa179e2d07eee8ad8f57493436566c7cc30ac536a3379fdf008f47f6bb7ae1" [[package]] name = "windows_x86_64_msvc" @@ -8472,9 +8360,9 @@ checksum = "589f6da84c646204747d1270a2a5661ea66ed1cced2631d546fdfb155959f9ec" [[package]] name = "windows_x86_64_msvc" -version = "0.53.0" +version = "0.53.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "271414315aff87387382ec3d271b52d7ae78726f5d44ac98b4f4030c91880486" +checksum = "d6bbff5f0aada427a1e5a6da5f1f98158182f26556f345ac9e04d36d0ebed650" [[package]] name = "winnow" @@ -8487,9 +8375,9 @@ dependencies = [ [[package]] name = "wit-bindgen" -version = "0.45.1" +version = "0.46.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "5c573471f125075647d03df72e026074b7203790d41351cd6edc96f46bcccd36" +checksum = "f17a85883d4e6d00e8a97c586de764dabcc06133f7f1d55dce5cdc070ad7fe59" [[package]] name = "writeable" @@ -8547,7 +8435,7 @@ checksum = "38da3c9736e16c5d3c8c597a9aaa5d1fa565d0532ae05e27c24aa62fb32c0ab6" dependencies = [ "proc-macro2", "quote", - "syn 2.0.106", + "syn 2.0.107", "synstructure", ] @@ -8568,7 +8456,7 @@ checksum = "88d2b8d9c68ad2b9e4340d7832716a4d21a22a1154777ad56ea55c51a9cf3831" dependencies = [ "proc-macro2", "quote", - "syn 2.0.106", + "syn 2.0.107", ] [[package]] @@ -8588,15 +8476,15 @@ checksum = "d71e5d6e06ab090c67b5e44993ec16b72dcbaabc526db883a360057678b48502" dependencies = [ "proc-macro2", "quote", - "syn 2.0.106", + "syn 2.0.107", "synstructure", ] [[package]] name = "zeroize" -version = "1.8.1" +version = "1.8.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "ced3678a2879b30306d323f4542626697a464a97c0a07c9aebf7ebca65cd4dde" +checksum = "b97154e67e32c85465826e8bcc1c59429aaaf107c1e4a9e53c8d8ccd5eff88d0" [[package]] name = "zerotrie" @@ -8628,7 +8516,7 @@ checksum = "5b96237efa0c878c64bd89c436f661be4e46b2f3eff1ebb976f7ef2321d2f58f" dependencies = [ "proc-macro2", "quote", - "syn 2.0.106", + "syn 2.0.107", ] [[package]] diff --git a/Cargo.toml b/Cargo.toml index 7735d702cf..6167380594 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -68,6 +68,7 @@ datafusion-sqllogictest = "48.0.1" derive_builder = "0.20" dirs = "6" enum-ordinalize = "4.3.0" +env_logger = "0.11.8" expect-test = "1" faststr = "0.2.31" fnv = "1.0.7" @@ -84,7 +85,9 @@ iceberg-catalog-s3tables = { version = "0.7.0", path = "./crates/catalog/s3table iceberg-datafusion = { version = "0.7.0", path = "./crates/integrations/datafusion" } indicatif = "0.17" itertools = "0.13" +libtest-mimic = "0.8.1" linkedbytes = "0.1.8" +log = "0.4.28" metainfo = "0.7.14" mimalloc = "0.1.46" mockall = "0.13.1" diff --git a/crates/sqllogictest/Cargo.toml b/crates/sqllogictest/Cargo.toml index e335de433e..e826ad7ae0 100644 --- a/crates/sqllogictest/Cargo.toml +++ b/crates/sqllogictest/Cargo.toml @@ -30,13 +30,25 @@ async-trait = { workspace = true } datafusion = { workspace = true } datafusion-sqllogictest = { workspace = true } enum-ordinalize = { workspace = true } +env_logger = { workspace = true } +iceberg = { workspace = true } +iceberg-datafusion = { workspace = true } indicatif = { workspace = true } +log = { workspace = true } sqllogictest = { workspace = true } toml = { workspace = true } serde = { workspace = true } tracing = { workspace = true } tokio = { workspace = true } +[dev-dependencies] +libtest-mimic = { workspace = true } + +[[test]] +harness = false +name = "sqllogictests" +path = "tests/sqllogictests.rs" + [package.metadata.cargo-machete] # These dependencies are added to ensure minimal dependency version ignored = ["enum-ordinalize"] diff --git a/crates/sqllogictest/src/engine/datafusion.rs b/crates/sqllogictest/src/engine/datafusion.rs index 49d7273d60..b3e37d9206 100644 --- a/crates/sqllogictest/src/engine/datafusion.rs +++ b/crates/sqllogictest/src/engine/datafusion.rs @@ -15,53 +15,77 @@ // specific language governing permissions and limitations // under the License. +use std::collections::HashMap; use std::path::{Path, PathBuf}; use std::sync::Arc; -use anyhow::{Context, anyhow}; use datafusion::catalog::CatalogProvider; use datafusion::prelude::{SessionConfig, SessionContext}; use datafusion_sqllogictest::DataFusion; +use iceberg::CatalogBuilder; +use iceberg::memory::{MEMORY_CATALOG_WAREHOUSE, MemoryCatalogBuilder}; +use iceberg_datafusion::IcebergCatalogProvider; use indicatif::ProgressBar; -use sqllogictest::runner::AsyncDB; use toml::Table as TomlTable; -use crate::engine::EngineRunner; +use crate::engine::{EngineRunner, run_slt_with_runner}; use crate::error::Result; pub struct DataFusionEngine { - datafusion: DataFusion, + test_data_path: PathBuf, + session_context: SessionContext, } #[async_trait::async_trait] impl EngineRunner for DataFusionEngine { async fn run_slt_file(&mut self, path: &Path) -> Result<()> { - let content = std::fs::read_to_string(path) - .with_context(|| format!("Failed to read slt file {:?}", path)) - .map_err(|e| anyhow!(e))?; + let ctx = self.session_context.clone(); + let testdata = self.test_data_path.clone(); - self.datafusion - .run(content.as_str()) - .await - .with_context(|| format!("Failed to run slt file {:?}", path)) - .map_err(|e| anyhow!(e))?; + let runner = sqllogictest::Runner::new({ + move || { + let ctx = ctx.clone(); + let testdata = testdata.clone(); + async move { + // Everything here is owned; no `self` capture. + Ok(DataFusion::new(ctx, testdata, ProgressBar::new(100))) + } + } + }); - Ok(()) + run_slt_with_runner(runner, path).await } } impl DataFusionEngine { pub async fn new(config: TomlTable) -> Result { - let session_config = SessionConfig::new().with_target_partitions(4); + let session_config = SessionConfig::new() + .with_target_partitions(4) + .with_information_schema(true); let ctx = SessionContext::new_with_config(session_config); ctx.register_catalog("default", Self::create_catalog(&config).await?); Ok(Self { - datafusion: DataFusion::new(ctx, PathBuf::from("testdata"), ProgressBar::new(100)), + test_data_path: PathBuf::from("testdata"), + session_context: ctx, }) } async fn create_catalog(_: &TomlTable) -> anyhow::Result> { - todo!() + // TODO: support dynamic catalog configuration + // See: https://github.com/apache/iceberg-rust/issues/1780 + let catalog = MemoryCatalogBuilder::default() + .load( + "memory", + HashMap::from([( + MEMORY_CATALOG_WAREHOUSE.to_string(), + "memory://".to_string(), + )]), + ) + .await?; + + Ok(Arc::new( + IcebergCatalogProvider::try_new(Arc::new(catalog)).await?, + )) } } diff --git a/crates/sqllogictest/src/engine/mod.rs b/crates/sqllogictest/src/engine/mod.rs index a1d34dd9bc..f8a405a7cc 100644 --- a/crates/sqllogictest/src/engine/mod.rs +++ b/crates/sqllogictest/src/engine/mod.rs @@ -19,70 +19,75 @@ mod datafusion; use std::path::Path; +use anyhow::anyhow; +use sqllogictest::{AsyncDB, MakeConnection, Runner, parse_file}; use toml::Table as TomlTable; use crate::engine::datafusion::DataFusionEngine; -use crate::error::Result; +use crate::error::{Error, Result}; -const KEY_TYPE: &str = "type"; const TYPE_DATAFUSION: &str = "datafusion"; #[async_trait::async_trait] -pub trait EngineRunner: Sized { +pub trait EngineRunner: Send { async fn run_slt_file(&mut self, path: &Path) -> Result<()>; } -pub enum Engine { - DataFusion(DataFusionEngine), +pub async fn load_engine_runner( + engine_type: &str, + cfg: TomlTable, +) -> Result> { + match engine_type { + TYPE_DATAFUSION => Ok(Box::new(DataFusionEngine::new(cfg).await?)), + _ => Err(anyhow::anyhow!("Unsupported engine type: {}", engine_type).into()), + } } -impl Engine { - pub async fn new(config: TomlTable) -> Result { - let engine_type = config - .get(KEY_TYPE) - .ok_or_else(|| anyhow::anyhow!("Missing required key: {KEY_TYPE}"))? - .as_str() - .ok_or_else(|| anyhow::anyhow!("Config value for {KEY_TYPE} must be a string"))?; - - match engine_type { - TYPE_DATAFUSION => { - let engine = DataFusionEngine::new(config).await?; - Ok(Engine::DataFusion(engine)) - } - _ => Err(anyhow::anyhow!("Unsupported engine type: {engine_type}").into()), +pub async fn run_slt_with_runner( + mut runner: Runner, + step_slt_file: impl AsRef, +) -> Result<()> +where + D: AsyncDB + Send + 'static, + M: MakeConnection + Send + 'static, +{ + let path = step_slt_file.as_ref().canonicalize()?; + let records = parse_file(&path).map_err(|e| Error(anyhow!("parsing slt file failed: {e}")))?; + + for record in records { + if let Err(err) = runner.run_async(record).await { + return Err(Error(anyhow!("SLT record execution failed: {err}"))); } } - pub async fn run_slt_file(&mut self, path: &Path) -> Result<()> { - match self { - Engine::DataFusion(engine) => engine.run_slt_file(path).await, - } - } + Ok(()) } #[cfg(test)] mod tests { - use toml::Table as TomlTable; - - use crate::engine::Engine; + use crate::engine::{TYPE_DATAFUSION, load_engine_runner}; #[tokio::test] - async fn test_engine_new_missing_type_key() { - let config = TomlTable::new(); - let result = Engine::new(config).await; + async fn test_engine_invalid_type() { + let input = r#" + [engines] + random = { type = "random_engine", url = "http://localhost:8181" } + "#; + let tbl = toml::from_str(input).unwrap(); + let result = load_engine_runner("random_engine", tbl).await; assert!(result.is_err()); } #[tokio::test] - async fn test_engine_invalid_type() { + async fn test_load_datafusion() { let input = r#" [engines] - random = { type = "random_engine", url = "http://localhost:8181" } + df = { type = "datafusion" } "#; let tbl = toml::from_str(input).unwrap(); - let result = Engine::new(tbl).await; + let result = load_engine_runner(TYPE_DATAFUSION, tbl).await; - assert!(result.is_err()); + assert!(result.is_ok()); } } diff --git a/crates/sqllogictest/src/error.rs b/crates/sqllogictest/src/error.rs index 01eb364e2a..2bf5a09d6b 100644 --- a/crates/sqllogictest/src/error.rs +++ b/crates/sqllogictest/src/error.rs @@ -44,3 +44,9 @@ impl From for Error { Self(value) } } + +impl From for Error { + fn from(value: std::io::Error) -> Self { + Self(value.into()) + } +} diff --git a/crates/sqllogictest/src/lib.rs b/crates/sqllogictest/src/lib.rs index 7b17727d1a..34a3c654aa 100644 --- a/crates/sqllogictest/src/lib.rs +++ b/crates/sqllogictest/src/lib.rs @@ -18,9 +18,6 @@ // This lib contains codes copied from // [Apache Datafusion](https://github.com/apache/datafusion/tree/main/datafusion/sqllogictest) -#[allow(dead_code)] mod engine; -#[allow(dead_code)] -mod error; -#[allow(dead_code)] -mod schedule; +pub mod error; +pub mod schedule; diff --git a/crates/sqllogictest/src/schedule.rs b/crates/sqllogictest/src/schedule.rs index d3284949bc..7c13ad4d12 100644 --- a/crates/sqllogictest/src/schedule.rs +++ b/crates/sqllogictest/src/schedule.rs @@ -24,11 +24,11 @@ use serde::{Deserialize, Serialize}; use toml::{Table as TomlTable, Value}; use tracing::info; -use crate::engine::Engine; +use crate::engine::{EngineRunner, load_engine_runner}; pub struct Schedule { /// Engine names to engine instances - engines: HashMap, + engines: HashMap>, /// List of test steps to run steps: Vec, /// Path of the schedule file @@ -44,7 +44,11 @@ pub struct Step { } impl Schedule { - pub fn new(engines: HashMap, steps: Vec, schedule_file: String) -> Self { + pub fn new( + engines: HashMap>, + steps: Vec, + schedule_file: String, + ) -> Self { Self { engines, steps, @@ -102,7 +106,9 @@ impl Schedule { Ok(()) } - async fn parse_engines(table: &TomlTable) -> anyhow::Result> { + async fn parse_engines( + table: &TomlTable, + ) -> anyhow::Result>> { let engines_tbl = table .get("engines") .with_context(|| "Schedule file must have an 'engines' table")? @@ -117,9 +123,13 @@ impl Schedule { .ok_or_else(|| anyhow!("Config of engine '{name}' is not a table"))? .clone(); - let engine = Engine::new(cfg_tbl) - .await - .with_context(|| format!("Failed to construct engine '{name}'"))?; + let engine_type = cfg_tbl + .get("type") + .ok_or_else(|| anyhow::anyhow!("Engine {name} doesn't have a 'type' field"))? + .as_str() + .ok_or_else(|| anyhow::anyhow!("Engine {name} type must be a string"))?; + + let engine = load_engine_runner(engine_type, cfg_tbl.clone()).await?; if engines.insert(name.clone(), engine).is_some() { return Err(anyhow!("Duplicate engine '{name}'")); diff --git a/crates/sqllogictest/testdata/schedules/df_test.toml b/crates/sqllogictest/testdata/schedules/df_test.toml new file mode 100644 index 0000000000..0733744951 --- /dev/null +++ b/crates/sqllogictest/testdata/schedules/df_test.toml @@ -0,0 +1,23 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, +# software distributed under the License is distributed on an +# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +# KIND, either express or implied. See the License for the +# specific language governing permissions and limitations +# under the License. + +[engines] +df = { type = "datafusion" } + +[[steps]] +engine = "df" +slt = "df_test/show_tables.slt" \ No newline at end of file diff --git a/crates/sqllogictest/testdata/slts/df_test/show_tables.slt b/crates/sqllogictest/testdata/slts/df_test/show_tables.slt new file mode 100644 index 0000000000..34709d7359 --- /dev/null +++ b/crates/sqllogictest/testdata/slts/df_test/show_tables.slt @@ -0,0 +1,34 @@ +# 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. + +query TTTT rowsort +SHOW TABLES +---- +datafusion information_schema columns VIEW +datafusion information_schema df_settings VIEW +datafusion information_schema parameters VIEW +datafusion information_schema routines VIEW +datafusion information_schema schemata VIEW +datafusion information_schema tables VIEW +datafusion information_schema views VIEW +default information_schema columns VIEW +default information_schema df_settings VIEW +default information_schema parameters VIEW +default information_schema routines VIEW +default information_schema schemata VIEW +default information_schema tables VIEW +default information_schema views VIEW \ No newline at end of file diff --git a/crates/sqllogictest/tests/sqllogictests.rs b/crates/sqllogictest/tests/sqllogictests.rs new file mode 100644 index 0000000000..434d9d78cc --- /dev/null +++ b/crates/sqllogictest/tests/sqllogictests.rs @@ -0,0 +1,87 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +use std::fs; +use std::path::PathBuf; + +use iceberg_sqllogictest::schedule::Schedule; +use libtest_mimic::{Arguments, Trial}; +use tokio::runtime::Handle; + +pub fn main() { + env_logger::init(); + + let rt = tokio::runtime::Builder::new_multi_thread() + .enable_all() + .build() + .unwrap(); + + // Parse command line arguments + let args = Arguments::from_args(); + + let tests = collect_trials(rt.handle().clone()).unwrap(); + let result = libtest_mimic::run(&args, tests); + + drop(rt); + + result.exit(); +} + +pub(crate) fn collect_trials(handle: Handle) -> anyhow::Result> { + let schedule_files = collect_schedule_files()?; + log::debug!( + "Found {} schedules files: {:?}", + schedule_files.len(), + &schedule_files + ); + let mut trials = Vec::with_capacity(schedule_files.len()); + for schedule_file in schedule_files { + let h = handle.clone(); + let trial_name = format!( + "schedule: {}", + schedule_file + .file_name() + .expect("Schedule file should have a name") + .to_string_lossy() + ); + let trial = Trial::test(trial_name, move || { + Ok(h.block_on(run_schedule(schedule_file.clone()))?) + }); + trials.push(trial); + } + Ok(trials) +} + +pub(crate) fn collect_schedule_files() -> anyhow::Result> { + let dir = PathBuf::from(format!("{}/testdata/schedules", env!("CARGO_MANIFEST_DIR"))); + let mut schedule_files = Vec::with_capacity(32); + for entry in fs::read_dir(&dir)? { + let entry = entry?; + let path = entry.path(); + if path.is_file() { + schedule_files.push(fs::canonicalize(dir.join(path))?); + } + } + Ok(schedule_files) +} + +pub(crate) async fn run_schedule(schedule_file: PathBuf) -> anyhow::Result<()> { + let schedules = Schedule::from_file(schedule_file).await?; + schedules.run().await?; + + Ok(()) +} From ddbcae46d4b6739b0d21fbf44de85b140b3d272a Mon Sep 17 00:00:00 2001 From: Matt Butrovich Date: Fri, 24 Oct 2025 08:03:49 -0400 Subject: [PATCH 31/47] fix(build): Pin home version after merging #1764 (#1783) --- Cargo.toml | 1 + crates/integrations/playground/Cargo.toml | 3 ++- 2 files changed, 3 insertions(+), 1 deletion(-) diff --git a/Cargo.toml b/Cargo.toml index 6167380594..1c007376ad 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -75,6 +75,7 @@ fnv = "1.0.7" fs-err = "3.1.0" futures = "0.3" hive_metastore = "0.2.0" +home = "=0.5.11" http = "1.2" iceberg = { version = "0.7.0", path = "./crates/iceberg" } iceberg-catalog-glue = { version = "0.7.0", path = "./crates/catalog/glue" } diff --git a/crates/integrations/playground/Cargo.toml b/crates/integrations/playground/Cargo.toml index 8ba983d058..3f6774be19 100644 --- a/crates/integrations/playground/Cargo.toml +++ b/crates/integrations/playground/Cargo.toml @@ -33,6 +33,7 @@ datafusion = { workspace = true } datafusion-cli = { workspace = true } dirs = { workspace = true } fs-err = { workspace = true } +home = { workspace = true } iceberg = { workspace = true } iceberg-catalog-rest = { workspace = true } iceberg-datafusion = { workspace = true } @@ -45,4 +46,4 @@ tracing-subscriber = { workspace = true } [package.metadata.cargo-machete] # These dependencies are added to ensure minimal dependency version -ignored = ["stacker", "mimalloc"] +ignored = ["stacker", "mimalloc", "home"] From 104a01679cecbe4f6c1fba4d8aaca5ce099a4693 Mon Sep 17 00:00:00 2001 From: Shawn Chang Date: Sun, 26 Oct 2025 20:32:54 -0700 Subject: [PATCH 32/47] minor: Update Cargo.lock to add home (#1785) ## Which issue does this PR close? - A minor followup of #1783 ## What changes are included in this PR? - Update Cargo.lock to add home ## Are these changes tested? No --- Cargo.lock | 1 + 1 file changed, 1 insertion(+) diff --git a/Cargo.lock b/Cargo.lock index e27e05129d..1e520bd448 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -3783,6 +3783,7 @@ dependencies = [ "datafusion-cli", "dirs", "fs-err", + "home", "iceberg", "iceberg-catalog-rest", "iceberg-datafusion", From 1691419c4ca42e7979373188ac239fd6569df885 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Mon, 27 Oct 2025 12:22:31 +0800 Subject: [PATCH 33/47] chore(deps): Bump aws-sdk-s3tables from 1.40.0 to 1.41.0 (#1790) --- Cargo.lock | 18 +++++++++--------- 1 file changed, 9 insertions(+), 9 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index 1e520bd448..4895381610 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -752,9 +752,9 @@ dependencies = [ [[package]] name = "aws-sdk-s3tables" -version = "1.40.0" +version = "1.41.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "4f5b63d53d1536c2332c4ed11737c5de3eda1ef59c3565290b2f247d0d712a5e" +checksum = "761f176da526badb4c3dbd67ee1da2faf3dc1e537ed229355f7590d80595ae35" dependencies = [ "aws-credential-types", "aws-runtime", @@ -2669,7 +2669,7 @@ dependencies = [ "libc", "option-ext", "redox_users", - "windows-sys 0.61.2", + "windows-sys 0.59.0", ] [[package]] @@ -2811,7 +2811,7 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "39cab71617ae0d63f51a36d69f866391735b51691dbda63cf6f96d042b63efeb" dependencies = [ "libc", - "windows-sys 0.61.2", + "windows-sys 0.59.0", ] [[package]] @@ -4612,7 +4612,7 @@ version = "0.50.3" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "7957b9740744892f114936ab4a57b3f487491bbeafaf8083688b16841a4240e5" dependencies = [ - "windows-sys 0.61.2", + "windows-sys 0.59.0", ] [[package]] @@ -5595,7 +5595,7 @@ dependencies = [ "once_cell", "socket2 0.6.1", "tracing", - "windows-sys 0.60.2", + "windows-sys 0.59.0", ] [[package]] @@ -6058,7 +6058,7 @@ dependencies = [ "errno", "libc", "linux-raw-sys", - "windows-sys 0.61.2", + "windows-sys 0.59.0", ] [[package]] @@ -7140,7 +7140,7 @@ dependencies = [ "getrandom 0.3.4", "once_cell", "rustix", - "windows-sys 0.61.2", + "windows-sys 0.59.0", ] [[package]] @@ -8072,7 +8072,7 @@ version = "0.1.11" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "c2a7b1c03c876122aa43f3020e6c3c3ee5c05081c9a00739faf7503aeba10d22" dependencies = [ - "windows-sys 0.61.2", + "windows-sys 0.48.0", ] [[package]] From 0141385c581cc1ae6dd47c7a4b293aa75deda8f0 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Mon, 27 Oct 2025 13:09:34 +0800 Subject: [PATCH 34/47] chore(deps): Bump rand from 0.8.5 to 0.9.2 (#1789) MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Bumps [rand](https://github.com/rust-random/rand) from 0.8.5 to 0.9.2.
Changelog

Sourced from rand's changelog.

[0.9.2 — 2025-07-20]

Deprecated

  • Deprecate rand::rngs::mock module and StepRng generator (#1634)

Additions

  • Enable WeightedIndex<usize> (de)serialization (#1646)

[0.9.1] - 2025-04-17

Security and unsafe

  • Revise "not a crypto library" policy again (#1565)
  • Remove zerocopy dependency from rand (#1579)

Fixes

  • Fix feature simd_support for recent nightly rust (#1586)

Changes

  • Allow fn rand::seq::index::sample_weighted and fn IndexedRandom::choose_multiple_weighted to return fewer than amount results (#1623), reverting an undocumented change (#1382) to the previous release.

Additions

  • Add rand::distr::Alphabetic distribution. (#1587)
  • Re-export rand_core (#1604)

[0.9.0] - 2025-01-27

Security and unsafe

  • Policy: "rand is not a crypto library" (#1514)
  • Remove fork-protection from ReseedingRng and ThreadRng. Instead, it is recommended to call ThreadRng::reseed on fork. (#1379)
  • Use zerocopy to replace some unsafe code (#1349, #1393, #1446, #1502)

Dependencies

  • Bump the MSRV to 1.63.0 (#1207, #1246, #1269, #1341, #1416, #1536); note that 1.60.0 may work for dependents when using --ignore-rust-version
  • Update to rand_core v0.9.0 (#1558)

Features

  • Support std feature without getrandom or rand_chacha (#1354)
  • Enable feature small_rng by default (#1455)
  • Remove implicit feature rand_chacha; use std_rng instead. (#1473)
  • Rename feature serde1 to serde (#1477)
  • Rename feature getrandom to os_rng (#1537)
  • Add feature thread_rng (#1547)

API changes: rand_core traits

  • Add fn RngCore::read_adapter implementing std::io::Read (#1267)
  • Add trait CryptoBlockRng: BlockRngCore; make trait CryptoRng: RngCore (#1273)
  • Add traits TryRngCore, TryCryptoRng (#1424, #1499)
  • Rename fn SeedableRng::from_rng -> try_from_rng and add infallible variant fn from_rng (#1424)
  • Rename fn SeedableRng::from_entropy -> from_os_rng and add fallible variant fn try_from_os_rng (#1424)
  • Add bounds Clone and AsRef to associated type SeedableRng::Seed (#1491)

API changes: Rng trait and top-level fns

  • Rename fn rand::thread_rng() to rand::rng() and remove from the prelude (#1506)

... (truncated)

Commits

[![Dependabot compatibility score](https://dependabot-badges.githubapp.com/badges/compatibility_score?dependency-name=rand&package-manager=cargo&previous-version=0.8.5&new-version=0.9.2)](https://docs.github.com/en/github/managing-security-vulnerabilities/about-dependabot-security-updates#about-compatibility-scores) Dependabot will resolve any conflicts with this PR as long as you don't alter it yourself. You can also trigger a rebase manually by commenting `@dependabot rebase`. [//]: # (dependabot-automerge-start) [//]: # (dependabot-automerge-end) ---
Dependabot commands and options
You can trigger Dependabot actions by commenting on this PR: - `@dependabot rebase` will rebase this PR - `@dependabot recreate` will recreate this PR, overwriting any edits that have been made to it - `@dependabot merge` will merge this PR after your CI passes on it - `@dependabot squash and merge` will squash and merge this PR after your CI passes on it - `@dependabot cancel merge` will cancel a previously requested merge and block automerging - `@dependabot reopen` will reopen this PR if it is closed - `@dependabot close` will close this PR and stop Dependabot recreating it. You can achieve the same result by closing it manually - `@dependabot show ignore conditions` will show all of the ignore conditions of the specified dependency - `@dependabot ignore this major version` will close this PR and stop Dependabot creating any more for this major version (unless you reopen the PR or upgrade to it yourself) - `@dependabot ignore this minor version` will close this PR and stop Dependabot creating any more for this minor version (unless you reopen the PR or upgrade to it yourself) - `@dependabot ignore this dependency` will close this PR and stop Dependabot creating any more for this dependency (unless you reopen the PR or upgrade to it yourself)
Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> Co-authored-by: Renjie Liu From f1b1ddfcb3926052c32d20106aa4b4e31ff8d59f Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Mon, 27 Oct 2025 13:51:34 +0800 Subject: [PATCH 35/47] chore(deps): Bump actions/download-artifact from 5 to 6 (#1788) MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Bumps [actions/download-artifact](https://github.com/actions/download-artifact) from 5 to 6.
Release notes

Sourced from actions/download-artifact's releases.

v6.0.0

What's Changed

BREAKING CHANGE: this update supports Node v24.x. This is not a breaking change per-se but we're treating it as such.

New Contributors

Full Changelog: https://github.com/actions/download-artifact/compare/v5...v6.0.0

Commits
  • 018cc2c Merge pull request #438 from actions/danwkennedy/prepare-6.0.0
  • 815651c Revert "Remove github.dep.yml"
  • bb3a066 Remove github.dep.yml
  • fa1ce46 Prepare v6.0.0
  • 4a24838 Merge pull request #431 from danwkennedy/patch-1
  • 5e3251c Readme: spell out the first use of GHES
  • abefc31 Merge pull request #424 from actions/yacaovsnc/update_readme
  • ac43a60 Update README with artifact extraction details
  • de96f46 Merge pull request #417 from actions/yacaovsnc/update_readme
  • 7993cb4 Remove migration guide for artifact download changes
  • Additional commits viewable in compare view

[![Dependabot compatibility score](https://dependabot-badges.githubapp.com/badges/compatibility_score?dependency-name=actions/download-artifact&package-manager=github_actions&previous-version=5&new-version=6)](https://docs.github.com/en/github/managing-security-vulnerabilities/about-dependabot-security-updates#about-compatibility-scores) Dependabot will resolve any conflicts with this PR as long as you don't alter it yourself. You can also trigger a rebase manually by commenting `@dependabot rebase`. [//]: # (dependabot-automerge-start) [//]: # (dependabot-automerge-end) ---
Dependabot commands and options
You can trigger Dependabot actions by commenting on this PR: - `@dependabot rebase` will rebase this PR - `@dependabot recreate` will recreate this PR, overwriting any edits that have been made to it - `@dependabot merge` will merge this PR after your CI passes on it - `@dependabot squash and merge` will squash and merge this PR after your CI passes on it - `@dependabot cancel merge` will cancel a previously requested merge and block automerging - `@dependabot reopen` will reopen this PR if it is closed - `@dependabot close` will close this PR and stop Dependabot recreating it. You can achieve the same result by closing it manually - `@dependabot show ignore conditions` will show all of the ignore conditions of the specified dependency - `@dependabot ignore this major version` will close this PR and stop Dependabot creating any more for this major version (unless you reopen the PR or upgrade to it yourself) - `@dependabot ignore this minor version` will close this PR and stop Dependabot creating any more for this minor version (unless you reopen the PR or upgrade to it yourself) - `@dependabot ignore this dependency` will close this PR and stop Dependabot creating any more for this dependency (unless you reopen the PR or upgrade to it yourself)
Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> Co-authored-by: Renjie Liu --- .github/workflows/release_python.yml | 2 +- .github/workflows/release_python_nightly.yml | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/.github/workflows/release_python.yml b/.github/workflows/release_python.yml index 828e4e16b3..3608614918 100644 --- a/.github/workflows/release_python.yml +++ b/.github/workflows/release_python.yml @@ -178,7 +178,7 @@ jobs: steps: - name: Download all the dists - uses: actions/download-artifact@v5 + uses: actions/download-artifact@v6 with: pattern: wheels-* merge-multiple: true diff --git a/.github/workflows/release_python_nightly.yml b/.github/workflows/release_python_nightly.yml index 5e6fc3ce38..6624925d58 100644 --- a/.github/workflows/release_python_nightly.yml +++ b/.github/workflows/release_python_nightly.yml @@ -120,7 +120,7 @@ jobs: steps: - name: Download all the dists - uses: actions/download-artifact@v5 + uses: actions/download-artifact@v6 with: pattern: wheels-* merge-multiple: true From 044b45b9e7d2b1444c32463cc836c5a0e348fbb4 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Mon, 27 Oct 2025 14:24:04 +0800 Subject: [PATCH 36/47] chore(deps): Bump actions/upload-artifact from 4 to 5 (#1787) MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Bumps [actions/upload-artifact](https://github.com/actions/upload-artifact) from 4 to 5.
Release notes

Sourced from actions/upload-artifact's releases.

v5.0.0

What's Changed

BREAKING CHANGE: this update supports Node v24.x. This is not a breaking change per-se but we're treating it as such.

New Contributors

Full Changelog: https://github.com/actions/upload-artifact/compare/v4...v5.0.0

v4.6.2

What's Changed

New Contributors

Full Changelog: https://github.com/actions/upload-artifact/compare/v4...v4.6.2

v4.6.1

What's Changed

Full Changelog: https://github.com/actions/upload-artifact/compare/v4...v4.6.1

v4.6.0

What's Changed

Full Changelog: https://github.com/actions/upload-artifact/compare/v4...v4.6.0

v4.5.0

What's Changed

New Contributors

... (truncated)

Commits
  • 330a01c Merge pull request #734 from actions/danwkennedy/prepare-5.0.0
  • 03f2824 Update github.dep.yml
  • 905a1ec Prepare v5.0.0
  • 2d9f9cd Merge pull request #725 from patrikpolyak/patch-1
  • 9687587 Merge branch 'main' into patch-1
  • 2848b2c Merge pull request #727 from danwkennedy/patch-1
  • 9b51177 Spell out the first use of GHES
  • cd231ca Update GHES guidance to include reference to Node 20 version
  • de65e23 Merge pull request #712 from actions/nebuk89-patch-1
  • 8747d8c Update README.md
  • Additional commits viewable in compare view

[![Dependabot compatibility score](https://dependabot-badges.githubapp.com/badges/compatibility_score?dependency-name=actions/upload-artifact&package-manager=github_actions&previous-version=4&new-version=5)](https://docs.github.com/en/github/managing-security-vulnerabilities/about-dependabot-security-updates#about-compatibility-scores) Dependabot will resolve any conflicts with this PR as long as you don't alter it yourself. You can also trigger a rebase manually by commenting `@dependabot rebase`. [//]: # (dependabot-automerge-start) [//]: # (dependabot-automerge-end) ---
Dependabot commands and options
You can trigger Dependabot actions by commenting on this PR: - `@dependabot rebase` will rebase this PR - `@dependabot recreate` will recreate this PR, overwriting any edits that have been made to it - `@dependabot merge` will merge this PR after your CI passes on it - `@dependabot squash and merge` will squash and merge this PR after your CI passes on it - `@dependabot cancel merge` will cancel a previously requested merge and block automerging - `@dependabot reopen` will reopen this PR if it is closed - `@dependabot close` will close this PR and stop Dependabot recreating it. You can achieve the same result by closing it manually - `@dependabot show ignore conditions` will show all of the ignore conditions of the specified dependency - `@dependabot ignore this major version` will close this PR and stop Dependabot creating any more for this major version (unless you reopen the PR or upgrade to it yourself) - `@dependabot ignore this minor version` will close this PR and stop Dependabot creating any more for this minor version (unless you reopen the PR or upgrade to it yourself) - `@dependabot ignore this dependency` will close this PR and stop Dependabot creating any more for this dependency (unless you reopen the PR or upgrade to it yourself)
Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> Co-authored-by: Renjie Liu --- .github/workflows/release_python.yml | 4 ++-- .github/workflows/release_python_nightly.yml | 4 ++-- 2 files changed, 4 insertions(+), 4 deletions(-) diff --git a/.github/workflows/release_python.yml b/.github/workflows/release_python.yml index 3608614918..8208373cf0 100644 --- a/.github/workflows/release_python.yml +++ b/.github/workflows/release_python.yml @@ -107,7 +107,7 @@ jobs: command: sdist args: -o dist - name: Upload sdist - uses: actions/upload-artifact@v4 + uses: actions/upload-artifact@v5 with: name: wheels-sdist path: bindings/python/dist @@ -159,7 +159,7 @@ jobs: command: build args: --release -o dist - name: Upload wheels - uses: actions/upload-artifact@v4 + uses: actions/upload-artifact@v5 with: name: wheels-${{ matrix.os }}-${{ matrix.target }} path: bindings/python/dist diff --git a/.github/workflows/release_python_nightly.yml b/.github/workflows/release_python_nightly.yml index 6624925d58..821f95ac4e 100644 --- a/.github/workflows/release_python_nightly.yml +++ b/.github/workflows/release_python_nightly.yml @@ -56,7 +56,7 @@ jobs: args: -o dist - name: Upload sdist - uses: actions/upload-artifact@v4 + uses: actions/upload-artifact@v5 with: name: wheels-sdist path: bindings/python/dist @@ -102,7 +102,7 @@ jobs: args: --release -o dist - name: Upload wheels - uses: actions/upload-artifact@v4 + uses: actions/upload-artifact@v5 with: name: wheels-${{ matrix.os }}-${{ matrix.target }} path: bindings/python/dist From cd6005724ff511b8a2844cf78bd54c9bb5a5493c Mon Sep 17 00:00:00 2001 From: Matt Butrovich Date: Mon, 27 Oct 2025 07:30:59 -0400 Subject: [PATCH 37/47] fix(reader): filter row groups when FileScanTask contains byte ranges (#1779) --- crates/iceberg/src/arrow/reader.rs | 254 ++++++++++++++++++++++++++++- 1 file changed, 248 insertions(+), 6 deletions(-) diff --git a/crates/iceberg/src/arrow/reader.rs b/crates/iceberg/src/arrow/reader.rs index ff4cff0a64..720b9363b9 100644 --- a/crates/iceberg/src/arrow/reader.rs +++ b/crates/iceberg/src/arrow/reader.rs @@ -224,15 +224,16 @@ impl ArrowReader { } }; - // There are two possible sources both for potential lists of selected RowGroup indices, - // and for `RowSelection`s. - // Selected RowGroup index lists can come from two sources: + // There are three possible sources for potential lists of selected RowGroup indices, + // and two for `RowSelection`s. + // Selected RowGroup index lists can come from three sources: + // * When task.start and task.length specify a byte range (file splitting); // * When there are equality delete files that are applicable; // * When there is a scan predicate and row_group_filtering_enabled = true. // `RowSelection`s can be created in either or both of the following cases: // * When there are positional delete files that are applicable; // * When there is a scan predicate and row_selection_enabled = true - // Note that, in the former case we only perform row group filtering when + // Note that row group filtering from predicates only happens when // there is a scan predicate AND row_group_filtering_enabled = true, // but we perform row selection filtering if there are applicable // equality delete files OR (there is a scan predicate AND row_selection_enabled), @@ -241,6 +242,17 @@ impl ArrowReader { let mut selected_row_group_indices = None; let mut row_selection = None; + // Filter row groups based on byte range from task.start and task.length. + // If both start and length are 0, read the entire file (backwards compatibility). + if task.start != 0 || task.length != 0 { + let byte_range_filtered_row_groups = Self::filter_row_groups_by_byte_range( + record_batch_stream_builder.metadata(), + task.start, + task.length, + )?; + selected_row_group_indices = Some(byte_range_filtered_row_groups); + } + if let Some(predicate) = final_predicate { let (iceberg_field_ids, field_id_map) = Self::build_field_id_set_and_map( record_batch_stream_builder.parquet_schema(), @@ -256,14 +268,26 @@ impl ArrowReader { record_batch_stream_builder = record_batch_stream_builder.with_row_filter(row_filter); if row_group_filtering_enabled { - let result = Self::get_selected_row_group_indices( + let predicate_filtered_row_groups = Self::get_selected_row_group_indices( &predicate, record_batch_stream_builder.metadata(), &field_id_map, &task.schema, )?; - selected_row_group_indices = Some(result); + // Merge predicate-based filtering with byte range filtering (if present) + // by taking the intersection of both filters + selected_row_group_indices = match selected_row_group_indices { + Some(byte_range_filtered) => { + // Keep only row groups that are in both filters + let intersection: Vec = byte_range_filtered + .into_iter() + .filter(|idx| predicate_filtered_row_groups.contains(idx)) + .collect(); + Some(intersection) + } + None => Some(predicate_filtered_row_groups), + }; } if row_selection_enabled { @@ -717,6 +741,36 @@ impl ArrowReader { Ok(results.into_iter().flatten().collect::>().into()) } + + /// Filters row groups by byte range to support Iceberg's file splitting. + /// + /// Iceberg splits large files at row group boundaries, so we only read row groups + /// whose byte ranges overlap with [start, start+length). + fn filter_row_groups_by_byte_range( + parquet_metadata: &Arc, + start: u64, + length: u64, + ) -> Result> { + let row_groups = parquet_metadata.row_groups(); + let mut selected = Vec::new(); + let end = start + length; + + // Row groups are stored sequentially after the 4-byte magic header. + let mut current_byte_offset = 4u64; + + for (idx, row_group) in row_groups.iter().enumerate() { + let row_group_size = row_group.compressed_size() as u64; + let row_group_end = current_byte_offset + row_group_size; + + if current_byte_offset < end && start < row_group_end { + selected.push(idx); + } + + current_byte_offset = row_group_end; + } + + Ok(selected) + } } /// Build the map of parquet field id to Parquet column index in the schema. @@ -1949,6 +2003,194 @@ message schema { Arc::new(SchemaDescriptor::new(Arc::new(schema))) } + /// Verifies that file splits respect byte ranges and only read specific row groups. + #[tokio::test] + async fn test_file_splits_respect_byte_ranges() { + use arrow_array::Int32Array; + use parquet::file::reader::{FileReader, SerializedFileReader}; + + let schema = Arc::new( + Schema::builder() + .with_schema_id(1) + .with_fields(vec![ + NestedField::required(1, "id", Type::Primitive(PrimitiveType::Int)).into(), + ]) + .build() + .unwrap(), + ); + + let arrow_schema = Arc::new(ArrowSchema::new(vec![ + Field::new("id", DataType::Int32, false).with_metadata(HashMap::from([( + PARQUET_FIELD_ID_META_KEY.to_string(), + "1".to_string(), + )])), + ])); + + let tmp_dir = TempDir::new().unwrap(); + let table_location = tmp_dir.path().to_str().unwrap().to_string(); + let file_path = format!("{}/multi_row_group.parquet", &table_location); + + // Force each batch into its own row group for testing byte range filtering. + let batch1 = RecordBatch::try_new(arrow_schema.clone(), vec![Arc::new(Int32Array::from( + (0..100).collect::>(), + ))]) + .unwrap(); + let batch2 = RecordBatch::try_new(arrow_schema.clone(), vec![Arc::new(Int32Array::from( + (100..200).collect::>(), + ))]) + .unwrap(); + let batch3 = RecordBatch::try_new(arrow_schema.clone(), vec![Arc::new(Int32Array::from( + (200..300).collect::>(), + ))]) + .unwrap(); + + let props = WriterProperties::builder() + .set_compression(Compression::SNAPPY) + .set_max_row_group_size(100) + .build(); + + let file = File::create(&file_path).unwrap(); + let mut writer = ArrowWriter::try_new(file, arrow_schema.clone(), Some(props)).unwrap(); + writer.write(&batch1).expect("Writing batch 1"); + writer.write(&batch2).expect("Writing batch 2"); + writer.write(&batch3).expect("Writing batch 3"); + writer.close().unwrap(); + + // Read the file metadata to get row group byte positions + let file = File::open(&file_path).unwrap(); + let reader = SerializedFileReader::new(file).unwrap(); + let metadata = reader.metadata(); + + println!("File has {} row groups", metadata.num_row_groups()); + assert_eq!(metadata.num_row_groups(), 3, "Expected 3 row groups"); + + // Get byte positions for each row group + let row_group_0 = metadata.row_group(0); + let row_group_1 = metadata.row_group(1); + let row_group_2 = metadata.row_group(2); + + let rg0_start = 4u64; // Parquet files start with 4-byte magic "PAR1" + let rg1_start = rg0_start + row_group_0.compressed_size() as u64; + let rg2_start = rg1_start + row_group_1.compressed_size() as u64; + let file_end = rg2_start + row_group_2.compressed_size() as u64; + + println!( + "Row group 0: {} rows, starts at byte {}, {} bytes compressed", + row_group_0.num_rows(), + rg0_start, + row_group_0.compressed_size() + ); + println!( + "Row group 1: {} rows, starts at byte {}, {} bytes compressed", + row_group_1.num_rows(), + rg1_start, + row_group_1.compressed_size() + ); + println!( + "Row group 2: {} rows, starts at byte {}, {} bytes compressed", + row_group_2.num_rows(), + rg2_start, + row_group_2.compressed_size() + ); + + let file_io = FileIO::from_path(&table_location).unwrap().build().unwrap(); + let reader = ArrowReaderBuilder::new(file_io).build(); + + // Task 1: read only the first row group + let task1 = FileScanTask { + start: rg0_start, + length: row_group_0.compressed_size() as u64, + record_count: Some(100), + data_file_path: file_path.clone(), + data_file_format: DataFileFormat::Parquet, + schema: schema.clone(), + project_field_ids: vec![1], + predicate: None, + deletes: vec![], + }; + + // Task 2: read the second and third row groups + let task2 = FileScanTask { + start: rg1_start, + length: file_end - rg1_start, + record_count: Some(200), + data_file_path: file_path.clone(), + data_file_format: DataFileFormat::Parquet, + schema: schema.clone(), + project_field_ids: vec![1], + predicate: None, + deletes: vec![], + }; + + let tasks1 = Box::pin(futures::stream::iter(vec![Ok(task1)])) as FileScanTaskStream; + let result1 = reader + .clone() + .read(tasks1) + .unwrap() + .try_collect::>() + .await + .unwrap(); + + let total_rows_task1: usize = result1.iter().map(|b| b.num_rows()).sum(); + println!( + "Task 1 (bytes {}-{}) returned {} rows", + rg0_start, + rg0_start + row_group_0.compressed_size() as u64, + total_rows_task1 + ); + + let tasks2 = Box::pin(futures::stream::iter(vec![Ok(task2)])) as FileScanTaskStream; + let result2 = reader + .read(tasks2) + .unwrap() + .try_collect::>() + .await + .unwrap(); + + let total_rows_task2: usize = result2.iter().map(|b| b.num_rows()).sum(); + println!( + "Task 2 (bytes {}-{}) returned {} rows", + rg1_start, file_end, total_rows_task2 + ); + + assert_eq!( + total_rows_task1, 100, + "Task 1 should read only the first row group (100 rows), but got {} rows", + total_rows_task1 + ); + + assert_eq!( + total_rows_task2, 200, + "Task 2 should read only the second+third row groups (200 rows), but got {} rows", + total_rows_task2 + ); + + // Verify the actual data values are correct (not just the row count) + if total_rows_task1 > 0 { + let first_batch = &result1[0]; + let id_col = first_batch + .column(0) + .as_primitive::(); + let first_val = id_col.value(0); + let last_val = id_col.value(id_col.len() - 1); + println!("Task 1 data range: {} to {}", first_val, last_val); + + assert_eq!(first_val, 0, "Task 1 should start with id=0"); + assert_eq!(last_val, 99, "Task 1 should end with id=99"); + } + + if total_rows_task2 > 0 { + let first_batch = &result2[0]; + let id_col = first_batch + .column(0) + .as_primitive::(); + let first_val = id_col.value(0); + println!("Task 2 first value: {}", first_val); + + assert_eq!(first_val, 100, "Task 2 should start with id=100, not id=0"); + } + } + /// Test schema evolution: reading old Parquet file (with only column 'a') /// using a newer table schema (with columns 'a' and 'b'). /// This tests that: From 0a06c3e241a297998ebdcdf7c20f22d03c4e23f2 Mon Sep 17 00:00:00 2001 From: Shawn Chang Date: Tue, 28 Oct 2025 02:28:38 -0700 Subject: [PATCH 38/47] refactor(arrow,datafusion): Reuse PartitionValueCalculator in RecordBatchPartitionSplitter (#1781) ## Which issue does this PR close? - Closes #1786 - Covered some of changes from the previous draft: #1769 ## What changes are included in this PR? - Move PartitionValueCalculator to core/arrow so it can be reused by RecordBatchPartitionSplitter - Allow skipping partition value calculation in partition splitter for projected batches - Return rather than pairs in RecordBatchPartitionSplitter::split ## Are these changes tested? Added uts --- crates/iceberg/src/arrow/mod.rs | 7 +- .../src/arrow/partition_value_calculator.rs | 254 ++++++++++++ .../arrow/record_batch_partition_splitter.rs | 377 +++++++++++++----- .../datafusion/src/physical_plan/project.rs | 190 ++------- 4 files changed, 580 insertions(+), 248 deletions(-) create mode 100644 crates/iceberg/src/arrow/partition_value_calculator.rs diff --git a/crates/iceberg/src/arrow/mod.rs b/crates/iceberg/src/arrow/mod.rs index 28116a4b5e..c091c45177 100644 --- a/crates/iceberg/src/arrow/mod.rs +++ b/crates/iceberg/src/arrow/mod.rs @@ -35,4 +35,9 @@ mod value; pub use reader::*; pub use value::*; -pub(crate) mod record_batch_partition_splitter; +/// Partition value calculator for computing partition values +pub mod partition_value_calculator; +pub use partition_value_calculator::*; +/// Record batch partition splitter for partitioned tables +pub mod record_batch_partition_splitter; +pub use record_batch_partition_splitter::*; diff --git a/crates/iceberg/src/arrow/partition_value_calculator.rs b/crates/iceberg/src/arrow/partition_value_calculator.rs new file mode 100644 index 0000000000..1409503451 --- /dev/null +++ b/crates/iceberg/src/arrow/partition_value_calculator.rs @@ -0,0 +1,254 @@ +// 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. + +//! Partition value calculation for Iceberg tables. +//! +//! This module provides utilities for calculating partition values from record batches +//! based on a partition specification. + +use std::sync::Arc; + +use arrow_array::{ArrayRef, RecordBatch, StructArray}; +use arrow_schema::DataType; + +use super::record_batch_projector::RecordBatchProjector; +use super::type_to_arrow_type; +use crate::spec::{PartitionSpec, Schema, StructType, Type}; +use crate::transform::{BoxedTransformFunction, create_transform_function}; +use crate::{Error, ErrorKind, Result}; + +/// Calculator for partition values in Iceberg tables. +/// +/// This struct handles the projection of source columns and application of +/// partition transforms to compute partition values for a given record batch. +#[derive(Debug)] +pub struct PartitionValueCalculator { + projector: RecordBatchProjector, + transform_functions: Vec, + partition_type: StructType, + partition_arrow_type: DataType, +} + +impl PartitionValueCalculator { + /// Create a new PartitionValueCalculator. + /// + /// # Arguments + /// + /// * `partition_spec` - The partition specification + /// * `table_schema` - The Iceberg table schema + /// + /// # Returns + /// + /// Returns a new `PartitionValueCalculator` instance or an error if initialization fails. + /// + /// # Errors + /// + /// Returns an error if: + /// - The partition spec is unpartitioned + /// - Transform function creation fails + /// - Projector initialization fails + pub fn try_new(partition_spec: &PartitionSpec, table_schema: &Schema) -> Result { + if partition_spec.is_unpartitioned() { + return Err(Error::new( + ErrorKind::DataInvalid, + "Cannot create partition calculator for unpartitioned table", + )); + } + + // Create transform functions for each partition field + let transform_functions: Vec = partition_spec + .fields() + .iter() + .map(|pf| create_transform_function(&pf.transform)) + .collect::>>()?; + + // Extract source field IDs for projection + let source_field_ids: Vec = partition_spec + .fields() + .iter() + .map(|pf| pf.source_id) + .collect(); + + // Create projector for extracting source columns + let projector = RecordBatchProjector::from_iceberg_schema( + Arc::new(table_schema.clone()), + &source_field_ids, + )?; + + // Get partition type information + let partition_type = partition_spec.partition_type(table_schema)?; + let partition_arrow_type = type_to_arrow_type(&Type::Struct(partition_type.clone()))?; + + Ok(Self { + projector, + transform_functions, + partition_type, + partition_arrow_type, + }) + } + + /// Get the partition type as an Iceberg StructType. + pub fn partition_type(&self) -> &StructType { + &self.partition_type + } + + /// Get the partition type as an Arrow DataType. + pub fn partition_arrow_type(&self) -> &DataType { + &self.partition_arrow_type + } + + /// Calculate partition values for a record batch. + /// + /// This method: + /// 1. Projects the source columns from the batch + /// 2. Applies partition transforms to each source column + /// 3. Constructs a StructArray containing the partition values + /// + /// # Arguments + /// + /// * `batch` - The record batch to calculate partition values for + /// + /// # Returns + /// + /// Returns an ArrayRef containing a StructArray of partition values, or an error if calculation fails. + /// + /// # Errors + /// + /// Returns an error if: + /// - Column projection fails + /// - Transform application fails + /// - StructArray construction fails + pub fn calculate(&self, batch: &RecordBatch) -> Result { + // Project source columns from the batch + let source_columns = self.projector.project_column(batch.columns())?; + + // Get expected struct fields for the result + let expected_struct_fields = match &self.partition_arrow_type { + DataType::Struct(fields) => fields.clone(), + _ => { + return Err(Error::new( + ErrorKind::DataInvalid, + "Expected partition type must be a struct", + )); + } + }; + + // Apply transforms to each source column + let mut partition_values = Vec::with_capacity(self.transform_functions.len()); + for (source_column, transform_fn) in source_columns.iter().zip(&self.transform_functions) { + let partition_value = transform_fn.transform(source_column.clone())?; + partition_values.push(partition_value); + } + + // Construct the StructArray + let struct_array = StructArray::try_new(expected_struct_fields, partition_values, None) + .map_err(|e| { + Error::new( + ErrorKind::DataInvalid, + format!("Failed to create partition struct array: {}", e), + ) + })?; + + Ok(Arc::new(struct_array)) + } +} + +#[cfg(test)] +mod tests { + use std::sync::Arc; + + use arrow_array::{Int32Array, RecordBatch, StringArray}; + use arrow_schema::{Field, Schema as ArrowSchema}; + + use super::*; + use crate::spec::{NestedField, PartitionSpecBuilder, PrimitiveType, Transform}; + + #[test] + fn test_partition_calculator_identity_transform() { + let table_schema = Schema::builder() + .with_schema_id(0) + .with_fields(vec![ + NestedField::required(1, "id", Type::Primitive(PrimitiveType::Int)).into(), + NestedField::required(2, "name", Type::Primitive(PrimitiveType::String)).into(), + ]) + .build() + .unwrap(); + + let partition_spec = PartitionSpecBuilder::new(Arc::new(table_schema.clone())) + .add_partition_field("id", "id_partition", Transform::Identity) + .unwrap() + .build() + .unwrap(); + + let calculator = PartitionValueCalculator::try_new(&partition_spec, &table_schema).unwrap(); + + // Verify partition type + assert_eq!(calculator.partition_type().fields().len(), 1); + assert_eq!(calculator.partition_type().fields()[0].name, "id_partition"); + + // Create test batch + let arrow_schema = Arc::new(ArrowSchema::new(vec![ + Field::new("id", DataType::Int32, false), + Field::new("name", DataType::Utf8, false), + ])); + + let batch = RecordBatch::try_new(arrow_schema, vec![ + Arc::new(Int32Array::from(vec![10, 20, 30])), + Arc::new(StringArray::from(vec!["a", "b", "c"])), + ]) + .unwrap(); + + // Calculate partition values + let result = calculator.calculate(&batch).unwrap(); + let struct_array = result.as_any().downcast_ref::().unwrap(); + + let id_partition = struct_array + .column_by_name("id_partition") + .unwrap() + .as_any() + .downcast_ref::() + .unwrap(); + + assert_eq!(id_partition.value(0), 10); + assert_eq!(id_partition.value(1), 20); + assert_eq!(id_partition.value(2), 30); + } + + #[test] + fn test_partition_calculator_unpartitioned_error() { + let table_schema = Schema::builder() + .with_schema_id(0) + .with_fields(vec![ + NestedField::required(1, "id", Type::Primitive(PrimitiveType::Int)).into(), + ]) + .build() + .unwrap(); + + let partition_spec = PartitionSpecBuilder::new(Arc::new(table_schema.clone())) + .build() + .unwrap(); + + let result = PartitionValueCalculator::try_new(&partition_spec, &table_schema); + assert!(result.is_err()); + assert!( + result + .unwrap_err() + .to_string() + .contains("unpartitioned table") + ); + } +} diff --git a/crates/iceberg/src/arrow/record_batch_partition_splitter.rs b/crates/iceberg/src/arrow/record_batch_partition_splitter.rs index 704a4e9c15..66371fac16 100644 --- a/crates/iceberg/src/arrow/record_batch_partition_splitter.rs +++ b/crates/iceberg/src/arrow/record_batch_partition_splitter.rs @@ -19,137 +19,169 @@ use std::collections::HashMap; use std::sync::Arc; use arrow_array::{ArrayRef, BooleanArray, RecordBatch, StructArray}; -use arrow_schema::{DataType, SchemaRef as ArrowSchemaRef}; use arrow_select::filter::filter_record_batch; -use itertools::Itertools; -use parquet::arrow::PARQUET_FIELD_ID_META_KEY; use super::arrow_struct_to_literal; -use super::record_batch_projector::RecordBatchProjector; -use crate::arrow::type_to_arrow_type; -use crate::spec::{Literal, PartitionSpecRef, SchemaRef, Struct, StructType, Type}; -use crate::transform::{BoxedTransformFunction, create_transform_function}; +use super::partition_value_calculator::PartitionValueCalculator; +use crate::spec::{Literal, PartitionKey, PartitionSpecRef, SchemaRef, StructType}; use crate::{Error, ErrorKind, Result}; +/// Column name for the projected partition values struct +pub const PROJECTED_PARTITION_VALUE_COLUMN: &str = "_partition"; + /// The splitter used to split the record batch into multiple record batches by the partition spec. /// 1. It will project and transform the input record batch based on the partition spec, get the partitioned record batch. /// 2. Split the input record batch into multiple record batches based on the partitioned record batch. +/// +/// # Partition Value Modes +/// +/// The splitter supports two modes for obtaining partition values: +/// - **Computed mode** (`calculator` is `Some`): Computes partition values from source columns using transforms +/// - **Pre-computed mode** (`calculator` is `None`): Expects a `_partition` column in the input batch // # TODO // Remove this after partition writer supported. #[allow(dead_code)] pub struct RecordBatchPartitionSplitter { schema: SchemaRef, partition_spec: PartitionSpecRef, - projector: RecordBatchProjector, - transform_functions: Vec, - + calculator: Option, partition_type: StructType, - partition_arrow_type: DataType, } // # TODO // Remove this after partition writer supported. #[allow(dead_code)] impl RecordBatchPartitionSplitter { + /// Create a new RecordBatchPartitionSplitter. + /// + /// # Arguments + /// + /// * `iceberg_schema` - The Iceberg schema reference + /// * `partition_spec` - The partition specification reference + /// * `calculator` - Optional calculator for computing partition values from source columns. + /// - `Some(calculator)`: Compute partition values from source columns using transforms + /// - `None`: Expect a pre-computed `_partition` column in the input batch + /// + /// # Returns + /// + /// Returns a new `RecordBatchPartitionSplitter` instance or an error if initialization fails. pub fn new( - input_schema: ArrowSchemaRef, iceberg_schema: SchemaRef, partition_spec: PartitionSpecRef, + calculator: Option, ) -> Result { - let projector = RecordBatchProjector::new( - input_schema, - &partition_spec - .fields() - .iter() - .map(|field| field.source_id) - .collect::>(), - // The source columns, selected by ids, must be a primitive type and cannot be contained in a map or list, but may be nested in a struct. - // ref: https://iceberg.apache.org/spec/#partitioning - |field| { - if !field.data_type().is_primitive() { - return Ok(None); - } - field - .metadata() - .get(PARQUET_FIELD_ID_META_KEY) - .map(|s| { - s.parse::() - .map_err(|e| Error::new(ErrorKind::Unexpected, e.to_string())) - }) - .transpose() - }, - |_| true, - )?; - let transform_functions = partition_spec - .fields() - .iter() - .map(|field| create_transform_function(&field.transform)) - .collect::>>()?; - let partition_type = partition_spec.partition_type(&iceberg_schema)?; - let partition_arrow_type = type_to_arrow_type(&Type::Struct(partition_type.clone()))?; Ok(Self { schema: iceberg_schema, partition_spec, - projector, - transform_functions, + calculator, partition_type, - partition_arrow_type, }) } - fn partition_columns_to_struct(&self, partition_columns: Vec) -> Result> { - let arrow_struct_array = { - let partition_arrow_fields = { - let DataType::Struct(fields) = &self.partition_arrow_type else { - return Err(Error::new( - ErrorKind::DataInvalid, - "The partition arrow type is not a struct type", - )); - }; - fields.clone() - }; - Arc::new(StructArray::try_new( - partition_arrow_fields, - partition_columns, - None, - )?) as ArrayRef - }; - let struct_array = { - let struct_array = arrow_struct_to_literal(&arrow_struct_array, &self.partition_type)?; + /// Create a new RecordBatchPartitionSplitter with computed partition values. + /// + /// This is a convenience method that creates a calculator and initializes the splitter + /// to compute partition values from source columns. + /// + /// # Arguments + /// + /// * `iceberg_schema` - The Iceberg schema reference + /// * `partition_spec` - The partition specification reference + /// + /// # Returns + /// + /// Returns a new `RecordBatchPartitionSplitter` instance or an error if initialization fails. + pub fn new_with_computed_values( + iceberg_schema: SchemaRef, + partition_spec: PartitionSpecRef, + ) -> Result { + let calculator = PartitionValueCalculator::try_new(&partition_spec, &iceberg_schema)?; + Self::new(iceberg_schema, partition_spec, Some(calculator)) + } + + /// Create a new RecordBatchPartitionSplitter expecting pre-computed partition values. + /// + /// This is a convenience method that initializes the splitter to expect a `_partition` + /// column in the input batches. + /// + /// # Arguments + /// + /// * `iceberg_schema` - The Iceberg schema reference + /// * `partition_spec` - The partition specification reference + /// + /// # Returns + /// + /// Returns a new `RecordBatchPartitionSplitter` instance or an error if initialization fails. + pub fn new_with_precomputed_values( + iceberg_schema: SchemaRef, + partition_spec: PartitionSpecRef, + ) -> Result { + Self::new(iceberg_schema, partition_spec, None) + } + + /// Split the record batch into multiple record batches based on the partition spec. + pub fn split(&self, batch: &RecordBatch) -> Result> { + let partition_structs = if let Some(calculator) = &self.calculator { + // Compute partition values from source columns using calculator + let partition_array = calculator.calculate(batch)?; + let struct_array = arrow_struct_to_literal(&partition_array, &self.partition_type)?; + struct_array .into_iter() .map(|s| { - if let Some(s) = s { - if let Literal::Struct(s) = s { - Ok(s) - } else { - Err(Error::new( - ErrorKind::DataInvalid, - "The struct is not a struct literal", - )) - } + if let Some(Literal::Struct(s)) = s { + Ok(s) } else { - Err(Error::new(ErrorKind::DataInvalid, "The struct is null")) + Err(Error::new( + ErrorKind::DataInvalid, + "Partition value is not a struct literal or is null", + )) } }) .collect::>>()? - }; + } else { + // Extract partition values from pre-computed partition column + let partition_column = batch + .column_by_name(PROJECTED_PARTITION_VALUE_COLUMN) + .ok_or_else(|| { + Error::new( + ErrorKind::DataInvalid, + format!( + "Partition column '{}' not found in batch", + PROJECTED_PARTITION_VALUE_COLUMN + ), + ) + })?; - Ok(struct_array) - } + let partition_struct_array = partition_column + .as_any() + .downcast_ref::() + .ok_or_else(|| { + Error::new( + ErrorKind::DataInvalid, + "Partition column is not a StructArray", + ) + })?; - /// Split the record batch into multiple record batches based on the partition spec. - pub fn split(&self, batch: &RecordBatch) -> Result> { - let source_columns = self.projector.project_column(batch.columns())?; - let partition_columns = source_columns - .into_iter() - .zip_eq(self.transform_functions.iter()) - .map(|(source_column, transform_function)| transform_function.transform(source_column)) - .collect::>>()?; + let arrow_struct_array = Arc::new(partition_struct_array.clone()) as ArrayRef; + let struct_array = arrow_struct_to_literal(&arrow_struct_array, &self.partition_type)?; - let partition_structs = self.partition_columns_to_struct(partition_columns)?; + struct_array + .into_iter() + .map(|s| { + if let Some(Literal::Struct(s)) = s { + Ok(s) + } else { + Err(Error::new( + ErrorKind::DataInvalid, + "Partition value is not a struct literal or is null", + )) + } + }) + .collect::>>()? + }; // Group the batch by row value. let mut group_ids = HashMap::new(); @@ -172,8 +204,15 @@ impl RecordBatchPartitionSplitter { filter.into() }; + // Create PartitionKey from the partition struct + let partition_key = PartitionKey::new( + self.partition_spec.as_ref().clone(), + self.schema.clone(), + row, + ); + // filter the RecordBatch - partition_batches.push((row, filter_record_batch(batch, &filter_array)?)); + partition_batches.push((partition_key, filter_record_batch(batch, &filter_array)?)); } Ok(partition_batches) @@ -185,11 +224,13 @@ mod tests { use std::sync::Arc; use arrow_array::{Int32Array, RecordBatch, StringArray}; + use arrow_schema::DataType; + use parquet::arrow::PARQUET_FIELD_ID_META_KEY; use super::*; use crate::arrow::schema_to_arrow_schema; use crate::spec::{ - NestedField, PartitionSpecBuilder, PrimitiveLiteral, Schema, Transform, + NestedField, PartitionSpecBuilder, PrimitiveLiteral, Schema, Struct, Transform, Type, UnboundPartitionField, }; @@ -227,14 +268,14 @@ mod tests { .build() .unwrap(), ); - let input_schema = Arc::new(schema_to_arrow_schema(&schema).unwrap()); let partition_splitter = - RecordBatchPartitionSplitter::new(input_schema.clone(), schema.clone(), partition_spec) + RecordBatchPartitionSplitter::new_with_computed_values(schema.clone(), partition_spec) .expect("Failed to create splitter"); + let arrow_schema = Arc::new(schema_to_arrow_schema(&schema).unwrap()); let id_array = Int32Array::from(vec![1, 2, 1, 3, 2, 3, 1]); let data_array = StringArray::from(vec!["a", "b", "c", "d", "e", "f", "g"]); - let batch = RecordBatch::try_new(input_schema.clone(), vec![ + let batch = RecordBatch::try_new(arrow_schema.clone(), vec![ Arc::new(id_array), Arc::new(data_array), ]) @@ -243,8 +284,8 @@ mod tests { let mut partitioned_batches = partition_splitter .split(&batch) .expect("Failed to split RecordBatch"); - partitioned_batches.sort_by_key(|(row, _)| { - if let PrimitiveLiteral::Int(i) = row.fields()[0] + partitioned_batches.sort_by_key(|(partition_key, _)| { + if let PrimitiveLiteral::Int(i) = partition_key.data().fields()[0] .as_ref() .unwrap() .as_primitive_literal() @@ -260,7 +301,7 @@ mod tests { // check the first partition let expected_id_array = Int32Array::from(vec![1, 1, 1]); let expected_data_array = StringArray::from(vec!["a", "c", "g"]); - let expected_batch = RecordBatch::try_new(input_schema.clone(), vec![ + let expected_batch = RecordBatch::try_new(arrow_schema.clone(), vec![ Arc::new(expected_id_array), Arc::new(expected_data_array), ]) @@ -271,7 +312,7 @@ mod tests { // check the second partition let expected_id_array = Int32Array::from(vec![2, 2]); let expected_data_array = StringArray::from(vec!["b", "e"]); - let expected_batch = RecordBatch::try_new(input_schema.clone(), vec![ + let expected_batch = RecordBatch::try_new(arrow_schema.clone(), vec![ Arc::new(expected_id_array), Arc::new(expected_data_array), ]) @@ -282,7 +323,7 @@ mod tests { // check the third partition let expected_id_array = Int32Array::from(vec![3, 3]); let expected_data_array = StringArray::from(vec!["d", "f"]); - let expected_batch = RecordBatch::try_new(input_schema.clone(), vec![ + let expected_batch = RecordBatch::try_new(arrow_schema.clone(), vec![ Arc::new(expected_id_array), Arc::new(expected_data_array), ]) @@ -292,7 +333,7 @@ mod tests { let partition_values = partitioned_batches .iter() - .map(|(row, _)| row.clone()) + .map(|(partition_key, _)| partition_key.data().clone()) .collect::>(); // check partition value is struct(1), struct(2), struct(3) assert_eq!(partition_values, vec![ @@ -301,4 +342,144 @@ mod tests { Struct::from_iter(vec![Some(Literal::int(3))]), ]); } + + #[test] + fn test_record_batch_partition_split_with_partition_column() { + use arrow_array::StructArray; + use arrow_schema::{Field, Schema as ArrowSchema}; + + let schema = Arc::new( + 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 = Arc::new( + PartitionSpecBuilder::new(schema.clone()) + .with_spec_id(1) + .add_unbound_field(UnboundPartitionField { + source_id: 1, + field_id: None, + name: "id_bucket".to_string(), + transform: Transform::Identity, + }) + .unwrap() + .build() + .unwrap(), + ); + + // Create input schema with _partition column + // Note: partition field IDs start from 1000 by default + let partition_field = Field::new("id_bucket", DataType::Int32, false).with_metadata( + HashMap::from([(PARQUET_FIELD_ID_META_KEY.to_string(), "1000".to_string())]), + ); + let partition_struct_field = Field::new( + PROJECTED_PARTITION_VALUE_COLUMN, + DataType::Struct(vec![partition_field.clone()].into()), + false, + ); + + let input_schema = Arc::new(ArrowSchema::new(vec![ + Field::new("id", DataType::Int32, false), + Field::new("name", DataType::Utf8, false), + partition_struct_field, + ])); + + // Create splitter expecting pre-computed partition column + let partition_splitter = RecordBatchPartitionSplitter::new_with_precomputed_values( + schema.clone(), + partition_spec, + ) + .expect("Failed to create splitter"); + + // Create test data with pre-computed partition column + let id_array = Int32Array::from(vec![1, 2, 1, 3, 2, 3, 1]); + let data_array = StringArray::from(vec!["a", "b", "c", "d", "e", "f", "g"]); + + // Create partition column (same values as id for Identity transform) + let partition_values = Int32Array::from(vec![1, 2, 1, 3, 2, 3, 1]); + let partition_struct = StructArray::from(vec![( + Arc::new(partition_field), + Arc::new(partition_values) as ArrayRef, + )]); + + let batch = RecordBatch::try_new(input_schema.clone(), vec![ + Arc::new(id_array), + Arc::new(data_array), + Arc::new(partition_struct), + ]) + .expect("Failed to create RecordBatch"); + + // Split using the pre-computed partition column + let mut partitioned_batches = partition_splitter + .split(&batch) + .expect("Failed to split RecordBatch"); + + partitioned_batches.sort_by_key(|(partition_key, _)| { + if let PrimitiveLiteral::Int(i) = partition_key.data().fields()[0] + .as_ref() + .unwrap() + .as_primitive_literal() + .unwrap() + { + i + } else { + panic!("The partition value is not a int"); + } + }); + + assert_eq!(partitioned_batches.len(), 3); + + // Helper to extract id and name values from a batch + let extract_values = |batch: &RecordBatch| -> (Vec, Vec) { + let id_col = batch + .column(0) + .as_any() + .downcast_ref::() + .unwrap(); + let name_col = batch + .column(1) + .as_any() + .downcast_ref::() + .unwrap(); + ( + id_col.values().to_vec(), + name_col.iter().map(|s| s.unwrap().to_string()).collect(), + ) + }; + + // Verify partition 1: id=1, names=["a", "c", "g"] + let (key, batch) = &partitioned_batches[0]; + assert_eq!(key.data(), &Struct::from_iter(vec![Some(Literal::int(1))])); + let (ids, names) = extract_values(batch); + assert_eq!(ids, vec![1, 1, 1]); + assert_eq!(names, vec!["a", "c", "g"]); + + // Verify partition 2: id=2, names=["b", "e"] + let (key, batch) = &partitioned_batches[1]; + assert_eq!(key.data(), &Struct::from_iter(vec![Some(Literal::int(2))])); + let (ids, names) = extract_values(batch); + assert_eq!(ids, vec![2, 2]); + assert_eq!(names, vec!["b", "e"]); + + // Verify partition 3: id=3, names=["d", "f"] + let (key, batch) = &partitioned_batches[2]; + assert_eq!(key.data(), &Struct::from_iter(vec![Some(Literal::int(3))])); + let (ids, names) = extract_values(batch); + assert_eq!(ids, vec![3, 3]); + assert_eq!(names, vec!["d", "f"]); + } } diff --git a/crates/integrations/datafusion/src/physical_plan/project.rs b/crates/integrations/datafusion/src/physical_plan/project.rs index 4bfe8192b0..17492176a4 100644 --- a/crates/integrations/datafusion/src/physical_plan/project.rs +++ b/crates/integrations/datafusion/src/physical_plan/project.rs @@ -19,24 +19,19 @@ use std::sync::Arc; -use datafusion::arrow::array::{ArrayRef, RecordBatch, StructArray}; +use datafusion::arrow::array::RecordBatch; use datafusion::arrow::datatypes::{DataType, Schema as ArrowSchema}; use datafusion::common::Result as DFResult; -use datafusion::error::DataFusionError; use datafusion::physical_expr::PhysicalExpr; use datafusion::physical_expr::expressions::Column; use datafusion::physical_plan::projection::ProjectionExec; use datafusion::physical_plan::{ColumnarValue, ExecutionPlan}; -use iceberg::arrow::record_batch_projector::RecordBatchProjector; -use iceberg::spec::{PartitionSpec, Schema}; +use iceberg::arrow::{PROJECTED_PARTITION_VALUE_COLUMN, PartitionValueCalculator}; +use iceberg::spec::PartitionSpec; use iceberg::table::Table; -use iceberg::transform::BoxedTransformFunction; use crate::to_datafusion_error; -/// Column name for the combined partition values struct -const PARTITION_VALUES_COLUMN: &str = "_partition"; - /// Extends an ExecutionPlan with partition value calculations for Iceberg tables. /// /// This function takes an input ExecutionPlan and extends it with an additional column @@ -65,12 +60,9 @@ pub fn project_with_partition( let input_schema = input.schema(); // TODO: Validate that input_schema matches the Iceberg table schema. // See: https://github.com/apache/iceberg-rust/issues/1752 - let partition_type = build_partition_type(partition_spec, table_schema.as_ref())?; - let calculator = PartitionValueCalculator::new( - partition_spec.as_ref().clone(), - table_schema.as_ref().clone(), - partition_type, - )?; + let calculator = + PartitionValueCalculator::try_new(partition_spec.as_ref(), table_schema.as_ref()) + .map_err(to_datafusion_error)?; let mut projection_exprs: Vec<(Arc, String)> = Vec::with_capacity(input_schema.fields().len() + 1); @@ -80,8 +72,8 @@ pub fn project_with_partition( projection_exprs.push((column_expr, field.name().clone())); } - let partition_expr = Arc::new(PartitionExpr::new(calculator)); - projection_exprs.push((partition_expr, PARTITION_VALUES_COLUMN.to_string())); + let partition_expr = Arc::new(PartitionExpr::new(calculator, partition_spec.clone())); + projection_exprs.push((partition_expr, PROJECTED_PARTITION_VALUE_COLUMN.to_string())); let projection = ProjectionExec::try_new(projection_exprs, input)?; Ok(Arc::new(projection)) @@ -91,21 +83,24 @@ pub fn project_with_partition( #[derive(Debug, Clone)] struct PartitionExpr { calculator: Arc, + partition_spec: Arc, } impl PartitionExpr { - fn new(calculator: PartitionValueCalculator) -> Self { + fn new(calculator: PartitionValueCalculator, partition_spec: Arc) -> Self { Self { calculator: Arc::new(calculator), + partition_spec, } } } // Manual PartialEq/Eq implementations for pointer-based equality -// (two PartitionExpr are equal if they share the same calculator instance) +// (two PartitionExpr are equal if they share the same calculator and partition_spec instances) impl PartialEq for PartitionExpr { fn eq(&self, other: &Self) -> bool { Arc::ptr_eq(&self.calculator, &other.calculator) + && Arc::ptr_eq(&self.partition_spec, &other.partition_spec) } } @@ -117,7 +112,7 @@ impl PhysicalExpr for PartitionExpr { } fn data_type(&self, _input_schema: &ArrowSchema) -> DFResult { - Ok(self.calculator.partition_type.clone()) + Ok(self.calculator.partition_arrow_type().clone()) } fn nullable(&self, _input_schema: &ArrowSchema) -> DFResult { @@ -125,7 +120,10 @@ impl PhysicalExpr for PartitionExpr { } fn evaluate(&self, batch: &RecordBatch) -> DFResult { - let array = self.calculator.calculate(batch)?; + let array = self + .calculator + .calculate(batch) + .map_err(to_datafusion_error)?; Ok(ColumnarValue::Array(array)) } @@ -142,7 +140,6 @@ impl PhysicalExpr for PartitionExpr { fn fmt_sql(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { let field_names: Vec = self - .calculator .partition_spec .fields() .iter() @@ -155,7 +152,6 @@ impl PhysicalExpr for PartitionExpr { impl std::fmt::Display for PartitionExpr { fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { let field_names: Vec<&str> = self - .calculator .partition_spec .fields() .iter() @@ -167,110 +163,18 @@ impl std::fmt::Display for PartitionExpr { impl std::hash::Hash for PartitionExpr { fn hash(&self, state: &mut H) { - // Two PartitionExpr are equal if they share the same calculator Arc + // Two PartitionExpr are equal if they share the same calculator and partition_spec Arcs Arc::as_ptr(&self.calculator).hash(state); + Arc::as_ptr(&self.partition_spec).hash(state); } } -/// Calculator for partition values in Iceberg tables -#[derive(Debug)] -struct PartitionValueCalculator { - partition_spec: PartitionSpec, - partition_type: DataType, - projector: RecordBatchProjector, - transform_functions: Vec, -} - -impl PartitionValueCalculator { - fn new( - partition_spec: PartitionSpec, - table_schema: Schema, - partition_type: DataType, - ) -> DFResult { - if partition_spec.is_unpartitioned() { - return Err(DataFusionError::Internal( - "Cannot create partition calculator for unpartitioned table".to_string(), - )); - } - - let transform_functions: Result, _> = partition_spec - .fields() - .iter() - .map(|pf| iceberg::transform::create_transform_function(&pf.transform)) - .collect(); - - let transform_functions = transform_functions.map_err(to_datafusion_error)?; - - let source_field_ids: Vec = partition_spec - .fields() - .iter() - .map(|pf| pf.source_id) - .collect(); - - let projector = RecordBatchProjector::from_iceberg_schema( - Arc::new(table_schema.clone()), - &source_field_ids, - ) - .map_err(to_datafusion_error)?; - - Ok(Self { - partition_spec, - partition_type, - projector, - transform_functions, - }) - } - - fn calculate(&self, batch: &RecordBatch) -> DFResult { - let source_columns = self - .projector - .project_column(batch.columns()) - .map_err(to_datafusion_error)?; - - let expected_struct_fields = match &self.partition_type { - DataType::Struct(fields) => fields.clone(), - _ => { - return Err(DataFusionError::Internal( - "Expected partition type must be a struct".to_string(), - )); - } - }; - - let mut partition_values = Vec::with_capacity(self.partition_spec.fields().len()); - - for (source_column, transform_fn) in source_columns.iter().zip(&self.transform_functions) { - let partition_value = transform_fn - .transform(source_column.clone()) - .map_err(to_datafusion_error)?; - - partition_values.push(partition_value); - } - - let struct_array = StructArray::try_new(expected_struct_fields, partition_values, None) - .map_err(|e| DataFusionError::ArrowError(e, None))?; - - Ok(Arc::new(struct_array)) - } -} - -fn build_partition_type( - partition_spec: &PartitionSpec, - table_schema: &Schema, -) -> DFResult { - let partition_struct_type = partition_spec - .partition_type(table_schema) - .map_err(to_datafusion_error)?; - - iceberg::arrow::type_to_arrow_type(&iceberg::spec::Type::Struct(partition_struct_type)) - .map_err(to_datafusion_error) -} - #[cfg(test)] mod tests { - use datafusion::arrow::array::Int32Array; + use datafusion::arrow::array::{ArrayRef, Int32Array, StructArray}; use datafusion::arrow::datatypes::{Field, Fields}; use datafusion::physical_plan::empty::EmptyExec; - use iceberg::spec::{NestedField, PrimitiveType, StructType, Transform, Type}; + use iceberg::spec::{NestedField, PrimitiveType, Schema, StructType, Transform, Type}; use super::*; @@ -291,20 +195,11 @@ mod tests { .build() .unwrap(); - let _arrow_schema = Arc::new(ArrowSchema::new(vec![ - Field::new("id", DataType::Int32, false), - Field::new("name", DataType::Utf8, false), - ])); - - let partition_type = build_partition_type(&partition_spec, &table_schema).unwrap(); - let calculator = PartitionValueCalculator::new( - partition_spec.clone(), - table_schema, - partition_type.clone(), - ) - .unwrap(); + let calculator = PartitionValueCalculator::try_new(&partition_spec, &table_schema).unwrap(); - assert_eq!(calculator.partition_type, partition_type); + // Verify partition type + assert_eq!(calculator.partition_type().fields().len(), 1); + assert_eq!(calculator.partition_type().fields()[0].name, "id_partition"); } #[test] @@ -318,11 +213,13 @@ mod tests { .build() .unwrap(); - let partition_spec = iceberg::spec::PartitionSpec::builder(Arc::new(table_schema.clone())) - .add_partition_field("id", "id_partition", Transform::Identity) - .unwrap() - .build() - .unwrap(); + let partition_spec = Arc::new( + iceberg::spec::PartitionSpec::builder(Arc::new(table_schema.clone())) + .add_partition_field("id", "id_partition", Transform::Identity) + .unwrap() + .build() + .unwrap(), + ); let arrow_schema = Arc::new(ArrowSchema::new(vec![ Field::new("id", DataType::Int32, false), @@ -331,9 +228,7 @@ mod tests { let input = Arc::new(EmptyExec::new(arrow_schema.clone())); - let partition_type = build_partition_type(&partition_spec, &table_schema).unwrap(); - let calculator = - PartitionValueCalculator::new(partition_spec, table_schema, partition_type).unwrap(); + let calculator = PartitionValueCalculator::try_new(&partition_spec, &table_schema).unwrap(); let mut projection_exprs: Vec<(Arc, String)> = Vec::with_capacity(arrow_schema.fields().len() + 1); @@ -342,8 +237,8 @@ mod tests { projection_exprs.push((column_expr, field.name().clone())); } - let partition_expr = Arc::new(PartitionExpr::new(calculator)); - projection_exprs.push((partition_expr, PARTITION_VALUES_COLUMN.to_string())); + let partition_expr = Arc::new(PartitionExpr::new(calculator, partition_spec)); + projection_exprs.push((partition_expr, PROJECTED_PARTITION_VALUE_COLUMN.to_string())); let projection = ProjectionExec::try_new(projection_exprs, input).unwrap(); let result = Arc::new(projection); @@ -384,11 +279,10 @@ mod tests { ]) .unwrap(); - let partition_type = build_partition_type(&partition_spec, &table_schema).unwrap(); - let calculator = - PartitionValueCalculator::new(partition_spec, table_schema, partition_type.clone()) - .unwrap(); - let expr = PartitionExpr::new(calculator); + let partition_spec = Arc::new(partition_spec); + let calculator = PartitionValueCalculator::try_new(&partition_spec, &table_schema).unwrap(); + let partition_type = calculator.partition_arrow_type().clone(); + let expr = PartitionExpr::new(calculator, partition_spec); assert_eq!(expr.data_type(&arrow_schema).unwrap(), partition_type); assert!(!expr.nullable(&arrow_schema).unwrap()); @@ -469,9 +363,7 @@ mod tests { ]) .unwrap(); - let partition_type = build_partition_type(&partition_spec, &table_schema).unwrap(); - let calculator = - PartitionValueCalculator::new(partition_spec, table_schema, partition_type).unwrap(); + let calculator = PartitionValueCalculator::try_new(&partition_spec, &table_schema).unwrap(); let array = calculator.calculate(&batch).unwrap(); let struct_array = array.as_any().downcast_ref::().unwrap(); From 127eeb8dec905dbbc48fb065d6c73a67b82cdb88 Mon Sep 17 00:00:00 2001 From: Jannik Steinmann Date: Tue, 28 Oct 2025 16:50:10 +0100 Subject: [PATCH 39/47] feat: Update Datafusion to v49 (#1704) ## Which issue does this PR close? Closes #1702 ## What changes are included in this PR? - a dependency upgrade from Datafusion 48 to 49 - a dependency specificity change from an exact pinned patch version to a broader major version (IMO this is better for broader compatibility) - fixes to breaking APIs ## Are these changes tested? --------- Signed-off-by: Xuanwo Co-authored-by: Xuanwo --- Cargo.lock | 185 +++++++++++------- Cargo.toml | 26 +-- bindings/python/Cargo.lock | 175 +++++++++++------ bindings/python/Cargo.toml | 4 +- bindings/python/pyproject.toml | 2 +- .../datafusion/src/physical_plan/commit.rs | 5 +- .../datafusion/src/physical_plan/write.rs | 5 +- .../src/table/table_provider_factory.rs | 2 +- 8 files changed, 250 insertions(+), 154 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index 4895381610..f4085e59e3 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -341,6 +341,7 @@ dependencies = [ "arrow-schema", "flatbuffers", "lz4_flex", + "zstd", ] [[package]] @@ -1350,6 +1351,15 @@ dependencies = [ "bzip2-sys", ] +[[package]] +name = "bzip2" +version = "0.6.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "bea8dcd42434048e4f7a304411d9273a411f647446c1234a65ce0554923f4cff" +dependencies = [ + "libbz2-rs-sys", +] + [[package]] name = "bzip2-sys" version = "0.1.13+1.0.8" @@ -1865,16 +1875,16 @@ dependencies = [ [[package]] name = "datafusion" -version = "48.0.1" +version = "49.0.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "8a11e19a7ccc5bb979c95c1dceef663eab39c9061b3bbf8d1937faf0f03bf41f" +checksum = "69dfeda1633bf8ec75b068d9f6c27cdc392ffcf5ff83128d5dbab65b73c1fd02" dependencies = [ "arrow", "arrow-ipc", "arrow-schema", "async-trait", "bytes", - "bzip2 0.5.2", + "bzip2 0.6.0", "chrono", "datafusion-catalog", "datafusion-catalog-listing", @@ -1902,6 +1912,7 @@ dependencies = [ "datafusion-sql", "flate2", "futures", + "hex", "itertools 0.14.0", "log", "object_store", @@ -1920,9 +1931,9 @@ dependencies = [ [[package]] name = "datafusion-catalog" -version = "48.0.1" +version = "49.0.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "94985e67cab97b1099db2a7af11f31a45008b282aba921c1e1d35327c212ec18" +checksum = "2848fd1e85e2953116dab9cc2eb109214b0888d7bbd2230e30c07f1794f642c0" dependencies = [ "arrow", "async-trait", @@ -1946,9 +1957,9 @@ dependencies = [ [[package]] name = "datafusion-catalog-listing" -version = "48.0.1" +version = "49.0.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "e002df133bdb7b0b9b429d89a69aa77b35caeadee4498b2ce1c7c23a99516988" +checksum = "051a1634628c2d1296d4e326823e7536640d87a118966cdaff069b68821ad53b" dependencies = [ "arrow", "async-trait", @@ -1969,9 +1980,9 @@ dependencies = [ [[package]] name = "datafusion-cli" -version = "48.0.1" +version = "49.0.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "85591b54815b0c9d3fbf3b0425b75e9fc49ef73e6886d6c19d622986e2551b53" +checksum = "5966ac4973bf66cf5c189046b1b29de90db9ba059ce7193137dd089e277495de" dependencies = [ "arrow", "async-trait", @@ -1982,6 +1993,7 @@ dependencies = [ "dirs", "env_logger", "futures", + "log", "mimalloc", "object_store", "parking_lot", @@ -1994,17 +2006,19 @@ dependencies = [ [[package]] name = "datafusion-common" -version = "48.0.1" +version = "49.0.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "e13242fc58fd753787b0a538e5ae77d356cb9d0656fa85a591a33c5f106267f6" +checksum = "765e4ad4ef7a4500e389a3f1e738791b71ff4c29fd00912c2f541d62b25da096" dependencies = [ "ahash 0.8.12", "apache-avro 0.17.0", "arrow", "arrow-ipc", "base64 0.22.1", + "chrono", "half", "hashbrown 0.14.5", + "hex", "indexmap 2.12.0", "libc", "log", @@ -2019,9 +2033,9 @@ dependencies = [ [[package]] name = "datafusion-common-runtime" -version = "48.0.1" +version = "49.0.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d2239f964e95c3a5d6b4a8cde07e646de8995c1396a7fd62c6e784f5341db499" +checksum = "40a2ae8393051ce25d232a6065c4558ab5a535c9637d5373bacfd464ac88ea12" dependencies = [ "futures", "log", @@ -2030,15 +2044,15 @@ dependencies = [ [[package]] name = "datafusion-datasource" -version = "48.0.1" +version = "49.0.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "2cf792579bc8bf07d1b2f68c2d5382f8a63679cce8fbebfd4ba95742b6e08864" +checksum = "90cd841a77f378bc1a5c4a1c37345e1885a9203b008203f9f4b3a769729bf330" dependencies = [ "arrow", "async-compression", "async-trait", "bytes", - "bzip2 0.5.2", + "bzip2 0.6.0", "chrono", "datafusion-common", "datafusion-common-runtime", @@ -2066,9 +2080,9 @@ dependencies = [ [[package]] name = "datafusion-datasource-avro" -version = "48.0.1" +version = "49.0.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "4de733d231abb0fba663ff60fd37bf7171fa8b2e46e8a99e41362001821d116e" +checksum = "3cba1696aa919da9517d29164d45f5902d6cc281f718e8d3bfe98bd52cd1142c" dependencies = [ "apache-avro 0.17.0", "arrow", @@ -2091,9 +2105,9 @@ dependencies = [ [[package]] name = "datafusion-datasource-csv" -version = "48.0.1" +version = "49.0.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "cfc114f9a1415174f3e8d2719c371fc72092ef2195a7955404cfe6b2ba29a706" +checksum = "77f4a2c64939c6f0dd15b246723a699fa30d59d0133eb36a86e8ff8c6e2a8dc6" dependencies = [ "arrow", "async-trait", @@ -2116,9 +2130,9 @@ dependencies = [ [[package]] name = "datafusion-datasource-json" -version = "48.0.1" +version = "49.0.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d88dd5e215c420a52362b9988ecd4cefd71081b730663d4f7d886f706111fc75" +checksum = "11387aaf931b2993ad9273c63ddca33f05aef7d02df9b70fb757429b4b71cdae" dependencies = [ "arrow", "async-trait", @@ -2141,9 +2155,9 @@ dependencies = [ [[package]] name = "datafusion-datasource-parquet" -version = "48.0.1" +version = "49.0.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "33692acdd1fbe75280d14f4676fe43f39e9cb36296df56575aa2cac9a819e4cf" +checksum = "028f430c5185120bf806347848b8d8acd9823f4038875b3820eeefa35f2bb4a2" dependencies = [ "arrow", "async-trait", @@ -2159,8 +2173,10 @@ dependencies = [ "datafusion-physical-expr-common", "datafusion-physical-optimizer", "datafusion-physical-plan", + "datafusion-pruning", "datafusion-session", "futures", + "hex", "itertools 0.14.0", "log", "object_store", @@ -2172,15 +2188,15 @@ dependencies = [ [[package]] name = "datafusion-doc" -version = "48.0.1" +version = "49.0.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "e0e7b648387b0c1937b83cb328533c06c923799e73a9e3750b762667f32662c0" +checksum = "8ff336d1d755399753a9e4fbab001180e346fc8bfa063a97f1214b82274c00f8" [[package]] name = "datafusion-execution" -version = "48.0.1" +version = "49.0.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "9609d83d52ff8315283c6dad3b97566e877d8f366fab4c3297742f33dcd636c7" +checksum = "042ea192757d1b2d7dcf71643e7ff33f6542c7704f00228d8b85b40003fd8e0f" dependencies = [ "arrow", "dashmap", @@ -2197,11 +2213,12 @@ dependencies = [ [[package]] name = "datafusion-expr" -version = "48.0.1" +version = "49.0.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "e75230cd67f650ef0399eb00f54d4a073698f2c0262948298e5299fc7324da63" +checksum = "025222545d6d7fab71e2ae2b356526a1df67a2872222cbae7535e557a42abd2e" dependencies = [ "arrow", + "async-trait", "chrono", "datafusion-common", "datafusion-doc", @@ -2218,9 +2235,9 @@ dependencies = [ [[package]] name = "datafusion-expr-common" -version = "48.0.1" +version = "49.0.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "70fafb3a045ed6c49cfca0cd090f62cf871ca6326cc3355cb0aaf1260fa760b6" +checksum = "9d5c267104849d5fa6d81cf5ba88f35ecd58727729c5eb84066c25227b644ae2" dependencies = [ "arrow", "datafusion-common", @@ -2231,9 +2248,9 @@ dependencies = [ [[package]] name = "datafusion-functions" -version = "48.0.1" +version = "49.0.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "cdf9a9cf655265861a20453b1e58357147eab59bdc90ce7f2f68f1f35104d3bb" +checksum = "c620d105aa208fcee45c588765483314eb415f5571cfd6c1bae3a59c5b4d15bb" dependencies = [ "arrow", "arrow-buffer", @@ -2260,9 +2277,9 @@ dependencies = [ [[package]] name = "datafusion-functions-aggregate" -version = "48.0.1" +version = "49.0.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "7f07e49733d847be0a05235e17b884d326a2fd402c97a89fe8bcf0bfba310005" +checksum = "35f61d5198a35ed368bf3aacac74f0d0fa33de7a7cb0c57e9f68ab1346d2f952" dependencies = [ "ahash 0.8.12", "arrow", @@ -2281,9 +2298,9 @@ dependencies = [ [[package]] name = "datafusion-functions-aggregate-common" -version = "48.0.1" +version = "49.0.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "4512607e10d72b0b0a1dc08f42cb5bd5284cb8348b7fea49dc83409493e32b1b" +checksum = "13efdb17362be39b5024f6da0d977ffe49c0212929ec36eec550e07e2bc7812f" dependencies = [ "ahash 0.8.12", "arrow", @@ -2294,9 +2311,9 @@ dependencies = [ [[package]] name = "datafusion-functions-nested" -version = "48.0.1" +version = "49.0.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "2ab331806e34f5545e5f03396e4d5068077395b1665795d8f88c14ec4f1e0b7a" +checksum = "9187678af567d7c9e004b72a0b6dc5b0a00ebf4901cb3511ed2db4effe092e66" dependencies = [ "arrow", "arrow-ord", @@ -2306,6 +2323,7 @@ dependencies = [ "datafusion-expr", "datafusion-functions", "datafusion-functions-aggregate", + "datafusion-functions-aggregate-common", "datafusion-macros", "datafusion-physical-expr-common", "itertools 0.14.0", @@ -2315,9 +2333,9 @@ dependencies = [ [[package]] name = "datafusion-functions-table" -version = "48.0.1" +version = "49.0.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d4ac2c0be983a06950ef077e34e0174aa0cb9e346f3aeae459823158037ade37" +checksum = "ecf156589cc21ef59fe39c7a9a841b4a97394549643bbfa88cc44e8588cf8fe5" dependencies = [ "arrow", "async-trait", @@ -2331,9 +2349,9 @@ dependencies = [ [[package]] name = "datafusion-functions-window" -version = "48.0.1" +version = "49.0.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "36f3d92731de384c90906941d36dcadf6a86d4128409a9c5cd916662baed5f53" +checksum = "edcb25e3e369f1366ec9a261456e45b5aad6ea1c0c8b4ce546587207c501ed9e" dependencies = [ "arrow", "datafusion-common", @@ -2349,9 +2367,9 @@ dependencies = [ [[package]] name = "datafusion-functions-window-common" -version = "48.0.1" +version = "49.0.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c679f8bf0971704ec8fd4249fcbb2eb49d6a12cc3e7a840ac047b4928d3541b5" +checksum = "8996a8e11174d0bd7c62dc2f316485affc6ae5ffd5b8a68b508137ace2310294" dependencies = [ "datafusion-common", "datafusion-physical-expr-common", @@ -2359,9 +2377,9 @@ dependencies = [ [[package]] name = "datafusion-macros" -version = "48.0.1" +version = "49.0.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "2821de7cb0362d12e75a5196b636a59ea3584ec1e1cc7dc6f5e34b9e8389d251" +checksum = "95ee8d1be549eb7316f437035f2cec7ec42aba8374096d807c4de006a3b5d78a" dependencies = [ "datafusion-expr", "quote", @@ -2370,14 +2388,15 @@ dependencies = [ [[package]] name = "datafusion-optimizer" -version = "48.0.1" +version = "49.0.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "1594c7a97219ede334f25347ad8d57056621e7f4f35a0693c8da876e10dd6a53" +checksum = "c9fa98671458254928af854e5f6c915e66b860a8bde505baea0ff2892deab74d" dependencies = [ "arrow", "chrono", "datafusion-common", "datafusion-expr", + "datafusion-expr-common", "datafusion-physical-expr", "indexmap 2.12.0", "itertools 0.14.0", @@ -2389,9 +2408,9 @@ dependencies = [ [[package]] name = "datafusion-physical-expr" -version = "48.0.1" +version = "49.0.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "dc6da0f2412088d23f6b01929dedd687b5aee63b19b674eb73d00c3eb3c883b7" +checksum = "3515d51531cca5f7b5a6f3ea22742b71bb36fc378b465df124ff9a2fa349b002" dependencies = [ "ahash 0.8.12", "arrow", @@ -2411,9 +2430,9 @@ dependencies = [ [[package]] name = "datafusion-physical-expr-common" -version = "48.0.1" +version = "49.0.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "dcb0dbd9213078a593c3fe28783beaa625a4e6c6a6c797856ee2ba234311fb96" +checksum = "24485475d9c618a1d33b2a3dad003d946dc7a7bbf0354d125301abc0a5a79e3e" dependencies = [ "ahash 0.8.12", "arrow", @@ -2425,9 +2444,9 @@ dependencies = [ [[package]] name = "datafusion-physical-optimizer" -version = "48.0.1" +version = "49.0.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "6d140854b2db3ef8ac611caad12bfb2e1e1de827077429322a6188f18fc0026a" +checksum = "b9da411a0a64702f941a12af2b979434d14ec5d36c6f49296966b2c7639cbb3a" dependencies = [ "arrow", "datafusion-common", @@ -2437,6 +2456,7 @@ dependencies = [ "datafusion-physical-expr", "datafusion-physical-expr-common", "datafusion-physical-plan", + "datafusion-pruning", "itertools 0.14.0", "log", "recursive", @@ -2444,9 +2464,9 @@ dependencies = [ [[package]] name = "datafusion-physical-plan" -version = "48.0.1" +version = "49.0.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "b46cbdf21a01206be76d467f325273b22c559c744a012ead5018dfe79597de08" +checksum = "a6d168282bb7b54880bb3159f89b51c047db4287f5014d60c3ef4c6e1468212b" dependencies = [ "ahash 0.8.12", "arrow", @@ -2472,11 +2492,29 @@ dependencies = [ "tokio", ] +[[package]] +name = "datafusion-pruning" +version = "49.0.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "391a457b9d23744c53eeb89edd1027424cba100581488d89800ed841182df905" +dependencies = [ + "arrow", + "arrow-schema", + "datafusion-common", + "datafusion-datasource", + "datafusion-expr-common", + "datafusion-physical-expr", + "datafusion-physical-expr-common", + "datafusion-physical-plan", + "itertools 0.14.0", + "log", +] + [[package]] name = "datafusion-session" -version = "48.0.1" +version = "49.0.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "3a72733766ddb5b41534910926e8da5836622316f6283307fd9fb7e19811a59c" +checksum = "053201c2bb729c7938f85879034df2b5a52cfaba16f1b3b66ab8505c81b2aad3" dependencies = [ "arrow", "async-trait", @@ -2498,9 +2536,9 @@ dependencies = [ [[package]] name = "datafusion-spark" -version = "48.0.1" +version = "49.0.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "4f469b1301a5a2dec8b944eacb7b1194ed8c0f8203ce460b7451bb6fa2e89060" +checksum = "126ceb2436b498ddd66c0a9f58304bce6b903f622c681c0ca9e52cf229c55ea0" dependencies = [ "arrow", "datafusion-catalog", @@ -2514,9 +2552,9 @@ dependencies = [ [[package]] name = "datafusion-sql" -version = "48.0.1" +version = "49.0.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c5162338cdec9cc7ea13a0e6015c361acad5ec1d88d83f7c86301f789473971f" +checksum = "9082779be8ce4882189b229c0cff4393bd0808282a7194130c9f32159f185e25" dependencies = [ "arrow", "bigdecimal", @@ -2531,9 +2569,9 @@ dependencies = [ [[package]] name = "datafusion-sqllogictest" -version = "48.0.1" +version = "49.0.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d3a05ab096d3dd4c234f4ecf3c9c08b71396ac1810d4d57d3a13049bbf8a1b30" +checksum = "2c2358f74d54fb9895b3b962f14877e414327eaaad4a95154abacc3e6bdcd38b" dependencies = [ "arrow", "async-trait", @@ -2558,9 +2596,9 @@ dependencies = [ [[package]] name = "datafusion-substrait" -version = "48.0.1" +version = "49.0.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "af6029c08002772fb2c38a191d21a8b03af78750f42d9b896222f6b0e16c84cf" +checksum = "4189e480ae42f06ade81a6d12853c5f3f84fbe237d73fb8795c712b6f1923afb" dependencies = [ "async-recursion", "async-trait", @@ -4187,6 +4225,12 @@ dependencies = [ "lexical-util", ] +[[package]] +name = "libbz2-rs-sys" +version = "0.2.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "2c4a545a15244c7d945065b5d392b2d2d7f21526fba56ce51467b06ed445e8f7" + [[package]] name = "libc" version = "0.2.177" @@ -4938,6 +4982,7 @@ dependencies = [ "num-bigint", "object_store", "paste", + "ring", "seq-macro", "simdutf8", "snap", @@ -7047,9 +7092,9 @@ dependencies = [ [[package]] name = "substrait" -version = "0.56.0" +version = "0.58.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "13de2e20128f2a018dab1cfa30be83ae069219a65968c6f89df66ad124de2397" +checksum = "de6d24c270c6c672a86c183c3a8439ba46c1936f93cf7296aa692de3b0ff0228" dependencies = [ "heck", "pbjson", diff --git a/Cargo.toml b/Cargo.toml index 1c007376ad..49eeff038a 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -42,14 +42,14 @@ rust-version = "1.87" anyhow = "1.0.72" apache-avro = { version = "0.20", features = ["zstandard"] } array-init = "2" -arrow-arith = { version = "55.1" } -arrow-array = { version = "55.1" } -arrow-buffer = { version = "55.1" } -arrow-cast = { version = "55.1" } -arrow-ord = { version = "55.1" } -arrow-schema = { version = "55.1" } -arrow-select = { version = "55.1" } -arrow-string = { version = "55.1" } +arrow-arith = { version = "55.2" } +arrow-array = { version = "55.2" } +arrow-buffer = { version = "55.2" } +arrow-cast = { version = "55.2" } +arrow-ord = { version = "55.2" } +arrow-schema = { version = "55.2" } +arrow-select = { version = "55.2" } +arrow-string = { version = "55.2" } as-any = "0.3.2" async-trait = "0.1.88" aws-config = "1.8.1" @@ -62,9 +62,9 @@ bytes = "1.10" chrono = "0.4.41" clap = { version = "4.5.41", features = ["derive", "cargo"] } ctor = "0.2.8" -datafusion = "48.0.1" -datafusion-cli = "48.0.1" -datafusion-sqllogictest = "48.0.1" +datafusion = "49" +datafusion-cli = "49" +datafusion-sqllogictest = "49" derive_builder = "0.20" dirs = "6" enum-ordinalize = "4.3.0" @@ -99,7 +99,7 @@ num-bigint = "0.4.6" once_cell = "1.20" opendal = "0.54.0" ordered-float = "4" -parquet = "55.1" +parquet = "55.2" pilota = "0.11.10" port_scanner = "0.1.5" pretty_assertions = "1.4" @@ -107,7 +107,7 @@ rand = "0.8.5" regex = "1.10.5" reqwest = { version = "0.12.12", default-features = false, features = ["json"] } roaring = { version = "0.11" } -rust_decimal = "1.37.1" +rust_decimal = "1.37.2" serde = { version = "1.0.219", features = ["rc"] } serde_bytes = "0.11.17" serde_derive = "1.0.219" diff --git a/bindings/python/Cargo.lock b/bindings/python/Cargo.lock index 1a619dca0f..70e95a37aa 100644 --- a/bindings/python/Cargo.lock +++ b/bindings/python/Cargo.lock @@ -157,6 +157,7 @@ dependencies = [ "strum_macros", "thiserror 2.0.16", "uuid", + "zstd", ] [[package]] @@ -301,6 +302,7 @@ dependencies = [ "arrow-schema", "flatbuffers", "lz4_flex", + "zstd", ] [[package]] @@ -429,7 +431,7 @@ version = "0.4.19" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "06575e6a9673580f52661c92107baabffbf41e2141373441cbcdc47cb733003c" dependencies = [ - "bzip2", + "bzip2 0.5.2", "flate2", "futures-core", "memchr", @@ -717,6 +719,15 @@ dependencies = [ "bzip2-sys", ] +[[package]] +name = "bzip2" +version = "0.6.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "bea8dcd42434048e4f7a304411d9273a411f647446c1234a65ce0554923f4cff" +dependencies = [ + "libbz2-rs-sys", +] + [[package]] name = "bzip2-sys" version = "0.1.13+1.0.8" @@ -1030,16 +1041,16 @@ dependencies = [ [[package]] name = "datafusion" -version = "48.0.1" +version = "49.0.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "8a11e19a7ccc5bb979c95c1dceef663eab39c9061b3bbf8d1937faf0f03bf41f" +checksum = "69dfeda1633bf8ec75b068d9f6c27cdc392ffcf5ff83128d5dbab65b73c1fd02" dependencies = [ "arrow", "arrow-ipc", "arrow-schema", "async-trait", "bytes", - "bzip2", + "bzip2 0.6.0", "chrono", "datafusion-catalog", "datafusion-catalog-listing", @@ -1066,6 +1077,7 @@ dependencies = [ "datafusion-sql", "flate2", "futures", + "hex", "itertools 0.14.0", "log", "object_store", @@ -1084,9 +1096,9 @@ dependencies = [ [[package]] name = "datafusion-catalog" -version = "48.0.1" +version = "49.0.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "94985e67cab97b1099db2a7af11f31a45008b282aba921c1e1d35327c212ec18" +checksum = "2848fd1e85e2953116dab9cc2eb109214b0888d7bbd2230e30c07f1794f642c0" dependencies = [ "arrow", "async-trait", @@ -1110,9 +1122,9 @@ dependencies = [ [[package]] name = "datafusion-catalog-listing" -version = "48.0.1" +version = "49.0.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "e002df133bdb7b0b9b429d89a69aa77b35caeadee4498b2ce1c7c23a99516988" +checksum = "051a1634628c2d1296d4e326823e7536640d87a118966cdaff069b68821ad53b" dependencies = [ "arrow", "async-trait", @@ -1133,16 +1145,18 @@ dependencies = [ [[package]] name = "datafusion-common" -version = "48.0.1" +version = "49.0.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "e13242fc58fd753787b0a538e5ae77d356cb9d0656fa85a591a33c5f106267f6" +checksum = "765e4ad4ef7a4500e389a3f1e738791b71ff4c29fd00912c2f541d62b25da096" dependencies = [ "ahash 0.8.12", "arrow", "arrow-ipc", "base64", + "chrono", "half", "hashbrown 0.14.5", + "hex", "indexmap 2.11.0", "libc", "log", @@ -1157,9 +1171,9 @@ dependencies = [ [[package]] name = "datafusion-common-runtime" -version = "48.0.1" +version = "49.0.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d2239f964e95c3a5d6b4a8cde07e646de8995c1396a7fd62c6e784f5341db499" +checksum = "40a2ae8393051ce25d232a6065c4558ab5a535c9637d5373bacfd464ac88ea12" dependencies = [ "futures", "log", @@ -1168,15 +1182,15 @@ dependencies = [ [[package]] name = "datafusion-datasource" -version = "48.0.1" +version = "49.0.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "2cf792579bc8bf07d1b2f68c2d5382f8a63679cce8fbebfd4ba95742b6e08864" +checksum = "90cd841a77f378bc1a5c4a1c37345e1885a9203b008203f9f4b3a769729bf330" dependencies = [ "arrow", "async-compression", "async-trait", "bytes", - "bzip2", + "bzip2 0.6.0", "chrono", "datafusion-common", "datafusion-common-runtime", @@ -1204,9 +1218,9 @@ dependencies = [ [[package]] name = "datafusion-datasource-csv" -version = "48.0.1" +version = "49.0.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "cfc114f9a1415174f3e8d2719c371fc72092ef2195a7955404cfe6b2ba29a706" +checksum = "77f4a2c64939c6f0dd15b246723a699fa30d59d0133eb36a86e8ff8c6e2a8dc6" dependencies = [ "arrow", "async-trait", @@ -1229,9 +1243,9 @@ dependencies = [ [[package]] name = "datafusion-datasource-json" -version = "48.0.1" +version = "49.0.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d88dd5e215c420a52362b9988ecd4cefd71081b730663d4f7d886f706111fc75" +checksum = "11387aaf931b2993ad9273c63ddca33f05aef7d02df9b70fb757429b4b71cdae" dependencies = [ "arrow", "async-trait", @@ -1254,9 +1268,9 @@ dependencies = [ [[package]] name = "datafusion-datasource-parquet" -version = "48.0.1" +version = "49.0.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "33692acdd1fbe75280d14f4676fe43f39e9cb36296df56575aa2cac9a819e4cf" +checksum = "028f430c5185120bf806347848b8d8acd9823f4038875b3820eeefa35f2bb4a2" dependencies = [ "arrow", "async-trait", @@ -1272,8 +1286,10 @@ dependencies = [ "datafusion-physical-expr-common", "datafusion-physical-optimizer", "datafusion-physical-plan", + "datafusion-pruning", "datafusion-session", "futures", + "hex", "itertools 0.14.0", "log", "object_store", @@ -1285,15 +1301,15 @@ dependencies = [ [[package]] name = "datafusion-doc" -version = "48.0.1" +version = "49.0.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "e0e7b648387b0c1937b83cb328533c06c923799e73a9e3750b762667f32662c0" +checksum = "8ff336d1d755399753a9e4fbab001180e346fc8bfa063a97f1214b82274c00f8" [[package]] name = "datafusion-execution" -version = "48.0.1" +version = "49.0.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "9609d83d52ff8315283c6dad3b97566e877d8f366fab4c3297742f33dcd636c7" +checksum = "042ea192757d1b2d7dcf71643e7ff33f6542c7704f00228d8b85b40003fd8e0f" dependencies = [ "arrow", "dashmap", @@ -1310,11 +1326,12 @@ dependencies = [ [[package]] name = "datafusion-expr" -version = "48.0.1" +version = "49.0.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "e75230cd67f650ef0399eb00f54d4a073698f2c0262948298e5299fc7324da63" +checksum = "025222545d6d7fab71e2ae2b356526a1df67a2872222cbae7535e557a42abd2e" dependencies = [ "arrow", + "async-trait", "chrono", "datafusion-common", "datafusion-doc", @@ -1331,9 +1348,9 @@ dependencies = [ [[package]] name = "datafusion-expr-common" -version = "48.0.1" +version = "49.0.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "70fafb3a045ed6c49cfca0cd090f62cf871ca6326cc3355cb0aaf1260fa760b6" +checksum = "9d5c267104849d5fa6d81cf5ba88f35ecd58727729c5eb84066c25227b644ae2" dependencies = [ "arrow", "datafusion-common", @@ -1344,9 +1361,9 @@ dependencies = [ [[package]] name = "datafusion-ffi" -version = "48.0.1" +version = "49.0.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "257de77f836e9bad3e0274d4ffc88555ebf559ae30d4e9c674b9809104c1cc3b" +checksum = "ec21805d9df2d834e4c6ddfbf8a1bed2bd460b89b01686fe0dcd1cee06d0b60f" dependencies = [ "abi_stable", "arrow", @@ -1366,9 +1383,9 @@ dependencies = [ [[package]] name = "datafusion-functions" -version = "48.0.1" +version = "49.0.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "cdf9a9cf655265861a20453b1e58357147eab59bdc90ce7f2f68f1f35104d3bb" +checksum = "c620d105aa208fcee45c588765483314eb415f5571cfd6c1bae3a59c5b4d15bb" dependencies = [ "arrow", "arrow-buffer", @@ -1395,9 +1412,9 @@ dependencies = [ [[package]] name = "datafusion-functions-aggregate" -version = "48.0.1" +version = "49.0.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "7f07e49733d847be0a05235e17b884d326a2fd402c97a89fe8bcf0bfba310005" +checksum = "35f61d5198a35ed368bf3aacac74f0d0fa33de7a7cb0c57e9f68ab1346d2f952" dependencies = [ "ahash 0.8.12", "arrow", @@ -1416,9 +1433,9 @@ dependencies = [ [[package]] name = "datafusion-functions-aggregate-common" -version = "48.0.1" +version = "49.0.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "4512607e10d72b0b0a1dc08f42cb5bd5284cb8348b7fea49dc83409493e32b1b" +checksum = "13efdb17362be39b5024f6da0d977ffe49c0212929ec36eec550e07e2bc7812f" dependencies = [ "ahash 0.8.12", "arrow", @@ -1429,9 +1446,9 @@ dependencies = [ [[package]] name = "datafusion-functions-nested" -version = "48.0.1" +version = "49.0.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "2ab331806e34f5545e5f03396e4d5068077395b1665795d8f88c14ec4f1e0b7a" +checksum = "9187678af567d7c9e004b72a0b6dc5b0a00ebf4901cb3511ed2db4effe092e66" dependencies = [ "arrow", "arrow-ord", @@ -1441,6 +1458,7 @@ dependencies = [ "datafusion-expr", "datafusion-functions", "datafusion-functions-aggregate", + "datafusion-functions-aggregate-common", "datafusion-macros", "datafusion-physical-expr-common", "itertools 0.14.0", @@ -1450,9 +1468,9 @@ dependencies = [ [[package]] name = "datafusion-functions-table" -version = "48.0.1" +version = "49.0.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d4ac2c0be983a06950ef077e34e0174aa0cb9e346f3aeae459823158037ade37" +checksum = "ecf156589cc21ef59fe39c7a9a841b4a97394549643bbfa88cc44e8588cf8fe5" dependencies = [ "arrow", "async-trait", @@ -1466,9 +1484,9 @@ dependencies = [ [[package]] name = "datafusion-functions-window" -version = "48.0.1" +version = "49.0.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "36f3d92731de384c90906941d36dcadf6a86d4128409a9c5cd916662baed5f53" +checksum = "edcb25e3e369f1366ec9a261456e45b5aad6ea1c0c8b4ce546587207c501ed9e" dependencies = [ "arrow", "datafusion-common", @@ -1484,9 +1502,9 @@ dependencies = [ [[package]] name = "datafusion-functions-window-common" -version = "48.0.1" +version = "49.0.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c679f8bf0971704ec8fd4249fcbb2eb49d6a12cc3e7a840ac047b4928d3541b5" +checksum = "8996a8e11174d0bd7c62dc2f316485affc6ae5ffd5b8a68b508137ace2310294" dependencies = [ "datafusion-common", "datafusion-physical-expr-common", @@ -1494,9 +1512,9 @@ dependencies = [ [[package]] name = "datafusion-macros" -version = "48.0.1" +version = "49.0.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "2821de7cb0362d12e75a5196b636a59ea3584ec1e1cc7dc6f5e34b9e8389d251" +checksum = "95ee8d1be549eb7316f437035f2cec7ec42aba8374096d807c4de006a3b5d78a" dependencies = [ "datafusion-expr", "quote", @@ -1505,14 +1523,15 @@ dependencies = [ [[package]] name = "datafusion-optimizer" -version = "48.0.1" +version = "49.0.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "1594c7a97219ede334f25347ad8d57056621e7f4f35a0693c8da876e10dd6a53" +checksum = "c9fa98671458254928af854e5f6c915e66b860a8bde505baea0ff2892deab74d" dependencies = [ "arrow", "chrono", "datafusion-common", "datafusion-expr", + "datafusion-expr-common", "datafusion-physical-expr", "indexmap 2.11.0", "itertools 0.14.0", @@ -1524,9 +1543,9 @@ dependencies = [ [[package]] name = "datafusion-physical-expr" -version = "48.0.1" +version = "49.0.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "dc6da0f2412088d23f6b01929dedd687b5aee63b19b674eb73d00c3eb3c883b7" +checksum = "3515d51531cca5f7b5a6f3ea22742b71bb36fc378b465df124ff9a2fa349b002" dependencies = [ "ahash 0.8.12", "arrow", @@ -1546,9 +1565,9 @@ dependencies = [ [[package]] name = "datafusion-physical-expr-common" -version = "48.0.1" +version = "49.0.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "dcb0dbd9213078a593c3fe28783beaa625a4e6c6a6c797856ee2ba234311fb96" +checksum = "24485475d9c618a1d33b2a3dad003d946dc7a7bbf0354d125301abc0a5a79e3e" dependencies = [ "ahash 0.8.12", "arrow", @@ -1560,9 +1579,9 @@ dependencies = [ [[package]] name = "datafusion-physical-optimizer" -version = "48.0.1" +version = "49.0.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "6d140854b2db3ef8ac611caad12bfb2e1e1de827077429322a6188f18fc0026a" +checksum = "b9da411a0a64702f941a12af2b979434d14ec5d36c6f49296966b2c7639cbb3a" dependencies = [ "arrow", "datafusion-common", @@ -1572,6 +1591,7 @@ dependencies = [ "datafusion-physical-expr", "datafusion-physical-expr-common", "datafusion-physical-plan", + "datafusion-pruning", "itertools 0.14.0", "log", "recursive", @@ -1579,9 +1599,9 @@ dependencies = [ [[package]] name = "datafusion-physical-plan" -version = "48.0.1" +version = "49.0.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "b46cbdf21a01206be76d467f325273b22c559c744a012ead5018dfe79597de08" +checksum = "a6d168282bb7b54880bb3159f89b51c047db4287f5014d60c3ef4c6e1468212b" dependencies = [ "ahash 0.8.12", "arrow", @@ -1609,9 +1629,9 @@ dependencies = [ [[package]] name = "datafusion-proto" -version = "48.0.1" +version = "49.0.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "e3fc7a2744332c2ef8804274c21f9fa664b4ca5889169250a6fd6b649ee5d16c" +checksum = "1b36a0c84f4500efd90487a004b533bd81de1f2bb3f143f71b7526f33b85d2e2" dependencies = [ "arrow", "chrono", @@ -1625,20 +1645,38 @@ dependencies = [ [[package]] name = "datafusion-proto-common" -version = "48.0.1" +version = "49.0.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "800add86852f12e3d249867425de2224c1e9fb7adc2930460548868781fbeded" +checksum = "2ec788be522806740ad6372c0a2f7e45fb37cb37f786d9b77933add49cdd058f" dependencies = [ "arrow", "datafusion-common", "prost", ] +[[package]] +name = "datafusion-pruning" +version = "49.0.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "391a457b9d23744c53eeb89edd1027424cba100581488d89800ed841182df905" +dependencies = [ + "arrow", + "arrow-schema", + "datafusion-common", + "datafusion-datasource", + "datafusion-expr-common", + "datafusion-physical-expr", + "datafusion-physical-expr-common", + "datafusion-physical-plan", + "itertools 0.14.0", + "log", +] + [[package]] name = "datafusion-session" -version = "48.0.1" +version = "49.0.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "3a72733766ddb5b41534910926e8da5836622316f6283307fd9fb7e19811a59c" +checksum = "053201c2bb729c7938f85879034df2b5a52cfaba16f1b3b66ab8505c81b2aad3" dependencies = [ "arrow", "async-trait", @@ -1660,9 +1698,9 @@ dependencies = [ [[package]] name = "datafusion-sql" -version = "48.0.1" +version = "49.0.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c5162338cdec9cc7ea13a0e6015c361acad5ec1d88d83f7c86301f789473971f" +checksum = "9082779be8ce4882189b229c0cff4393bd0808282a7194130c9f32159f185e25" dependencies = [ "arrow", "bigdecimal", @@ -2617,6 +2655,12 @@ dependencies = [ "static_assertions", ] +[[package]] +name = "libbz2-rs-sys" +version = "0.2.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "2c4a545a15244c7d945065b5d392b2d2d7f21526fba56ce51467b06ed445e8f7" + [[package]] name = "libc" version = "0.2.175" @@ -3048,6 +3092,7 @@ dependencies = [ "num-bigint", "object_store", "paste", + "ring", "seq-macro", "simdutf8", "snap", diff --git a/bindings/python/Cargo.toml b/bindings/python/Cargo.toml index 879a1450b7..f7f33d0729 100644 --- a/bindings/python/Cargo.toml +++ b/bindings/python/Cargo.toml @@ -31,9 +31,9 @@ license = "Apache-2.0" crate-type = ["cdylib"] [dependencies] -arrow = { version = "55.1", features = ["pyarrow", "chrono-tz"] } +arrow = { version = "55.2", features = ["pyarrow", "chrono-tz"] } iceberg = { path = "../../crates/iceberg" } pyo3 = { version = "0.24.1", features = ["extension-module", "abi3-py39"] } iceberg-datafusion = { path = "../../crates/integrations/datafusion" } -datafusion-ffi = { version = "48.0.1" } +datafusion-ffi = { version = "49" } tokio = { version = "1.46.1", default-features = false } diff --git a/bindings/python/pyproject.toml b/bindings/python/pyproject.toml index 41877e1952..c659d9b1a9 100644 --- a/bindings/python/pyproject.toml +++ b/bindings/python/pyproject.toml @@ -49,7 +49,7 @@ include = [ ignore = ["F403", "F405"] [tool.hatch.envs.dev] -dependencies = ["maturin>=1.0,<2.0", "pytest>=8.3.2", "datafusion==45.*", "pyiceberg[sql-sqlite,pyarrow]>=0.10.0", "fastavro>=1.11.1", "pydantic<2.12.0"] +dependencies = ["maturin>=1.0,<2.0", "pytest>=8.3.2", "datafusion==49.*", "pyiceberg[sql-sqlite,pyarrow]>=0.10.0", "fastavro>=1.11.1", "pydantic<2.12.0"] [tool.hatch.envs.dev.scripts] build = "maturin build --out dist --sdist" diff --git a/crates/integrations/datafusion/src/physical_plan/commit.rs b/crates/integrations/datafusion/src/physical_plan/commit.rs index 48373f11e2..6bba89fce5 100644 --- a/crates/integrations/datafusion/src/physical_plan/commit.rs +++ b/crates/integrations/datafusion/src/physical_plan/commit.rs @@ -86,7 +86,10 @@ impl IcebergCommitExec { let count_array = Arc::new(UInt64Array::from(vec![count])) as ArrayRef; RecordBatch::try_from_iter_with_nullable(vec![("count", count_array, false)]).map_err(|e| { - DataFusionError::ArrowError(e, Some("Failed to make count batch!".to_string())) + DataFusionError::ArrowError( + Box::new(e), + Some("Failed to make count batch!".to_string()), + ) }) } diff --git a/crates/integrations/datafusion/src/physical_plan/write.rs b/crates/integrations/datafusion/src/physical_plan/write.rs index dff40a3c0d..e46019c541 100644 --- a/crates/integrations/datafusion/src/physical_plan/write.rs +++ b/crates/integrations/datafusion/src/physical_plan/write.rs @@ -96,7 +96,10 @@ impl IcebergWriteExec { let files_array = Arc::new(StringArray::from(data_files)) as ArrayRef; RecordBatch::try_new(Self::make_result_schema(), vec![files_array]).map_err(|e| { - DataFusionError::ArrowError(e, Some("Failed to make result batch".to_string())) + DataFusionError::ArrowError( + Box::new(e), + Some("Failed to make result batch".to_string()), + ) }) } diff --git a/crates/integrations/datafusion/src/table/table_provider_factory.rs b/crates/integrations/datafusion/src/table/table_provider_factory.rs index a6d3146e54..e8e87dd318 100644 --- a/crates/integrations/datafusion/src/table/table_provider_factory.rs +++ b/crates/integrations/datafusion/src/table/table_provider_factory.rs @@ -241,7 +241,7 @@ mod tests { options: Default::default(), table_partition_cols: Default::default(), order_exprs: Default::default(), - constraints: Constraints::empty(), + constraints: Constraints::default(), column_defaults: Default::default(), if_not_exists: Default::default(), temporary: false, From 16ddd0e242f34d806f39e2f91a08a6cb9592e88e Mon Sep 17 00:00:00 2001 From: Kevin Liu Date: Tue, 28 Oct 2025 09:12:19 -0700 Subject: [PATCH 40/47] deps: unpin pydantic (#1793) ## Which issue does this PR close? - Closes #. ## What changes are included in this PR? Revert of #1737 pydantic release a fix in 2.12.3 which will be automatically used by resolver (see simliar in pyiceberg, https://github.com/apache/iceberg-python/pull/2635) ## Are these changes tested? --- bindings/python/pyproject.toml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/bindings/python/pyproject.toml b/bindings/python/pyproject.toml index c659d9b1a9..a9c3c290c7 100644 --- a/bindings/python/pyproject.toml +++ b/bindings/python/pyproject.toml @@ -49,7 +49,7 @@ include = [ ignore = ["F403", "F405"] [tool.hatch.envs.dev] -dependencies = ["maturin>=1.0,<2.0", "pytest>=8.3.2", "datafusion==49.*", "pyiceberg[sql-sqlite,pyarrow]>=0.10.0", "fastavro>=1.11.1", "pydantic<2.12.0"] +dependencies = ["maturin>=1.0,<2.0", "pytest>=8.3.2", "datafusion==49.*", "pyiceberg[sql-sqlite,pyarrow]>=0.10.0", "fastavro>=1.11.1"] [tool.hatch.envs.dev.scripts] build = "maturin build --out dist --sdist" From b20a2554a1a95e881e92ec4312fd4f75fad6da2f Mon Sep 17 00:00:00 2001 From: Matt Butrovich Date: Tue, 28 Oct 2025 13:01:57 -0400 Subject: [PATCH 41/47] feat(reader): Add Date32 support to RecordBatchTransformer create_column (#1792) ## Which issue does this PR close? - Partially address #1749. Iceberg Java has a test that does a schema change requires the `RecordBatchTransformer` to add a Date32 column, which it currently does not support. ## What changes are included in this PR? Add match arms for `Date32` type in `create_column`. ## Are these changes tested? New test that mirrors Iceberg Java's `TestSelect.readAndWriteWithBranchAfterSchemaChange` --------- Co-authored-by: Xuanwo --- .../src/arrow/record_batch_transformer.rs | 82 ++++++++++++++++++- 1 file changed, 79 insertions(+), 3 deletions(-) diff --git a/crates/iceberg/src/arrow/record_batch_transformer.rs b/crates/iceberg/src/arrow/record_batch_transformer.rs index 779f1cc625..71fe59dea5 100644 --- a/crates/iceberg/src/arrow/record_batch_transformer.rs +++ b/crates/iceberg/src/arrow/record_batch_transformer.rs @@ -19,8 +19,8 @@ use std::collections::HashMap; use std::sync::Arc; use arrow_array::{ - Array as ArrowArray, ArrayRef, BinaryArray, BooleanArray, Float32Array, Float64Array, - Int32Array, Int64Array, NullArray, RecordBatch, RecordBatchOptions, StringArray, + Array as ArrowArray, ArrayRef, BinaryArray, BooleanArray, Date32Array, Float32Array, + Float64Array, Int32Array, Int64Array, NullArray, RecordBatch, RecordBatchOptions, StringArray, }; use arrow_cast::cast; use arrow_schema::{ @@ -401,6 +401,13 @@ impl RecordBatchTransformer { let vals: Vec> = vec![None; num_rows]; Arc::new(Int32Array::from(vals)) } + (DataType::Date32, Some(PrimitiveLiteral::Int(value))) => { + Arc::new(Date32Array::from(vec![*value; num_rows])) + } + (DataType::Date32, None) => { + let vals: Vec> = vec![None; num_rows]; + Arc::new(Date32Array::from(vals)) + } (DataType::Int64, Some(PrimitiveLiteral::Long(value))) => { Arc::new(Int64Array::from(vec![*value; num_rows])) } @@ -453,7 +460,8 @@ mod test { use std::sync::Arc; use arrow_array::{ - Float32Array, Float64Array, Int32Array, Int64Array, RecordBatch, StringArray, + Array, Date32Array, Float32Array, Float64Array, Int32Array, Int64Array, RecordBatch, + StringArray, }; use arrow_schema::{DataType, Field, Schema as ArrowSchema}; use parquet::arrow::PARQUET_FIELD_ID_META_KEY; @@ -509,6 +517,74 @@ mod test { assert_eq!(result, expected); } + #[test] + fn schema_evolution_adds_date_column_with_nulls() { + // Reproduces TestSelect.readAndWriteWithBranchAfterSchemaChange from iceberg-spark. + // When reading old snapshots after adding a DATE column, the transformer must + // populate the new column with NULL values since old files lack this field. + let snapshot_schema = Arc::new( + Schema::builder() + .with_schema_id(1) + .with_fields(vec![ + NestedField::required(1, "id", Type::Primitive(PrimitiveType::Int)).into(), + NestedField::optional(2, "name", Type::Primitive(PrimitiveType::String)).into(), + NestedField::optional(3, "date_col", Type::Primitive(PrimitiveType::Date)) + .into(), + ]) + .build() + .unwrap(), + ); + let projected_iceberg_field_ids = [1, 2, 3]; + + let mut transformer = + RecordBatchTransformer::build(snapshot_schema, &projected_iceberg_field_ids); + + let file_schema = Arc::new(ArrowSchema::new(vec![ + simple_field("id", DataType::Int32, false, "1"), + simple_field("name", DataType::Utf8, true, "2"), + ])); + + let file_batch = RecordBatch::try_new(file_schema, vec![ + Arc::new(Int32Array::from(vec![1, 2, 3])), + Arc::new(StringArray::from(vec![ + Some("Alice"), + Some("Bob"), + Some("Charlie"), + ])), + ]) + .unwrap(); + + let result = transformer.process_record_batch(file_batch).unwrap(); + + assert_eq!(result.num_columns(), 3); + assert_eq!(result.num_rows(), 3); + + let id_column = result + .column(0) + .as_any() + .downcast_ref::() + .unwrap(); + assert_eq!(id_column.values(), &[1, 2, 3]); + + let name_column = result + .column(1) + .as_any() + .downcast_ref::() + .unwrap(); + assert_eq!(name_column.value(0), "Alice"); + assert_eq!(name_column.value(1), "Bob"); + assert_eq!(name_column.value(2), "Charlie"); + + let date_column = result + .column(2) + .as_any() + .downcast_ref::() + .unwrap(); + assert!(date_column.is_null(0)); + assert!(date_column.is_null(1)); + assert!(date_column.is_null(2)); + } + pub fn source_record_batch() -> RecordBatch { RecordBatch::try_new( arrow_schema_promotion_addition_and_renaming_required(), From 87acf3cbb77ecd5c0ed3e1b2eb4e8327223bc706 Mon Sep 17 00:00:00 2001 From: Shawn Chang Date: Tue, 28 Oct 2025 10:34:01 -0700 Subject: [PATCH 42/47] feat(catalog): Implement update_table for S3TablesCatalog (#1594) ## Which issue does this PR close? - Addresses the S3Table part of #1389 ## What changes are included in this PR? - Implemented update_table for S3TablesCatalog ## Are these changes tested? added a test --------- Signed-off-by: Xuanwo Co-authored-by: Xuanwo --- Cargo.lock | 1 + crates/catalog/s3tables/Cargo.toml | 1 + crates/catalog/s3tables/src/catalog.rs | 199 ++++++++++++++++++++----- 3 files changed, 167 insertions(+), 34 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index f4085e59e3..808c222dd3 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -3743,6 +3743,7 @@ dependencies = [ name = "iceberg-catalog-s3tables" version = "0.7.0" dependencies = [ + "anyhow", "async-trait", "aws-config", "aws-sdk-s3tables", diff --git a/crates/catalog/s3tables/Cargo.toml b/crates/catalog/s3tables/Cargo.toml index c5df489693..66fb70fefc 100644 --- a/crates/catalog/s3tables/Cargo.toml +++ b/crates/catalog/s3tables/Cargo.toml @@ -29,6 +29,7 @@ license = { workspace = true } repository = { workspace = true } [dependencies] +anyhow = { workspace = true } async-trait = { workspace = true } aws-config = { workspace = true } aws-sdk-s3tables = { workspace = true } diff --git a/crates/catalog/s3tables/src/catalog.rs b/crates/catalog/s3tables/src/catalog.rs index 858bb92896..daa6590551 100644 --- a/crates/catalog/s3tables/src/catalog.rs +++ b/crates/catalog/s3tables/src/catalog.rs @@ -23,6 +23,7 @@ use aws_sdk_s3tables::operation::create_table::CreateTableOutput; use aws_sdk_s3tables::operation::get_namespace::GetNamespaceOutput; use aws_sdk_s3tables::operation::get_table::GetTableOutput; use aws_sdk_s3tables::operation::list_tables::ListTablesOutput; +use aws_sdk_s3tables::operation::update_table_metadata_location::UpdateTableMetadataLocationError; use aws_sdk_s3tables::types::OpenTableFormat; use iceberg::io::{FileIO, FileIOBuilder}; use iceberg::spec::{TableMetadata, TableMetadataBuilder}; @@ -189,6 +190,39 @@ impl S3TablesCatalog { file_io, }) } + + async fn load_table_with_version_token( + &self, + table_ident: &TableIdent, + ) -> Result<(Table, String)> { + let req = self + .s3tables_client + .get_table() + .table_bucket_arn(self.config.table_bucket_arn.clone()) + .namespace(table_ident.namespace().to_url_string()) + .name(table_ident.name()); + let resp: GetTableOutput = req.send().await.map_err(from_aws_sdk_error)?; + + // when a table is created, it's possible that the metadata location is not set. + let metadata_location = resp.metadata_location().ok_or_else(|| { + Error::new( + ErrorKind::Unexpected, + format!( + "Table {} does not have metadata location", + table_ident.name() + ), + ) + })?; + let metadata = TableMetadata::read_from(&self.file_io, metadata_location).await?; + + let table = Table::builder() + .identifier(table_ident.clone()) + .metadata(metadata) + .metadata_location(metadata_location) + .file_io(self.file_io.clone()) + .build()?; + Ok((table, resp.version_token)) + } } #[async_trait] @@ -477,33 +511,7 @@ impl Catalog for S3TablesCatalog { /// - Errors from the underlying database query process, converted using /// `from_aws_sdk_error`. async fn load_table(&self, table_ident: &TableIdent) -> Result { - let req = self - .s3tables_client - .get_table() - .table_bucket_arn(self.config.table_bucket_arn.clone()) - .namespace(table_ident.namespace().to_url_string()) - .name(table_ident.name()); - let resp: GetTableOutput = req.send().await.map_err(from_aws_sdk_error)?; - - // when a table is created, it's possible that the metadata location is not set. - let metadata_location = resp.metadata_location().ok_or_else(|| { - Error::new( - ErrorKind::Unexpected, - format!( - "Table {} does not have metadata location", - table_ident.name() - ), - ) - })?; - let metadata = TableMetadata::read_from(&self.file_io, metadata_location).await?; - - let table = Table::builder() - .identifier(table_ident.clone()) - .metadata(metadata) - .metadata_location(metadata_location) - .file_io(self.file_io.clone()) - .build()?; - Ok(table) + Ok(self.load_table_with_version_token(table_ident).await?.0) } /// Drops an existing table from the s3tables catalog. @@ -589,13 +597,50 @@ impl Catalog for S3TablesCatalog { } /// Updates an existing table within the s3tables catalog. - /// - /// This function is still in development and will always return an error. - async fn update_table(&self, _commit: TableCommit) -> Result
{ - Err(Error::new( - ErrorKind::FeatureUnsupported, - "Updating a table is not supported yet", - )) + async fn update_table(&self, commit: TableCommit) -> Result
{ + let table_ident = commit.identifier().clone(); + let table_namespace = table_ident.namespace(); + let (current_table, version_token) = + self.load_table_with_version_token(&table_ident).await?; + + let staged_table = commit.apply(current_table)?; + let staged_metadata_location = staged_table.metadata_location_result()?; + + staged_table + .metadata() + .write_to(staged_table.file_io(), staged_metadata_location) + .await?; + + let builder = self + .s3tables_client + .update_table_metadata_location() + .table_bucket_arn(&self.config.table_bucket_arn) + .namespace(table_namespace.to_url_string()) + .name(table_ident.name()) + .version_token(version_token) + .metadata_location(staged_metadata_location); + + let _ = builder.send().await.map_err(|e| { + let error = e.into_service_error(); + match error { + UpdateTableMetadataLocationError::ConflictException(_) => Error::new( + ErrorKind::CatalogCommitConflicts, + format!("Commit conflicted for table: {table_ident}"), + ) + .with_retryable(true), + UpdateTableMetadataLocationError::NotFoundException(_) => Error::new( + ErrorKind::TableNotFound, + format!("Table {table_ident} is not found"), + ), + _ => Error::new( + ErrorKind::Unexpected, + "Operation failed for hitting aws sdk error", + ), + } + .with_source(anyhow::Error::msg(format!("aws sdk error: {:?}", error))) + })?; + + Ok(staged_table) } } @@ -611,6 +656,7 @@ where T: std::fmt::Debug { #[cfg(test)] mod tests { use iceberg::spec::{NestedField, PrimitiveType, Schema, Type}; + use iceberg::transaction::{ApplyTransactionAction, Transaction}; use super::*; @@ -737,6 +783,91 @@ mod tests { catalog.drop_namespace(&namespace).await.unwrap(); } + #[tokio::test] + async fn test_s3tables_update_table() { + let catalog = match load_s3tables_catalog_from_env().await { + Ok(Some(catalog)) => catalog, + Ok(None) => return, + Err(e) => panic!("Error loading catalog: {}", e), + }; + + // Create a test namespace and table + let namespace = NamespaceIdent::new("test_s3tables_update_table".to_string()); + let table_ident = + TableIdent::new(namespace.clone(), "test_s3tables_update_table".to_string()); + + // Clean up any existing resources from previous test runs + catalog.drop_table(&table_ident).await.ok(); + catalog.drop_namespace(&namespace).await.ok(); + + // Create namespace and table + catalog + .create_namespace(&namespace, HashMap::new()) + .await + .unwrap(); + + let creation = { + let schema = Schema::builder() + .with_schema_id(0) + .with_fields(vec![ + NestedField::required(1, "foo", Type::Primitive(PrimitiveType::Int)).into(), + NestedField::required(2, "bar", Type::Primitive(PrimitiveType::String)).into(), + ]) + .build() + .unwrap(); + TableCreation::builder() + .name(table_ident.name().to_string()) + .properties(HashMap::new()) + .schema(schema) + .build() + }; + + let table = catalog.create_table(&namespace, creation).await.unwrap(); + + // Create a transaction to update the table + let tx = Transaction::new(&table); + + // Store the original metadata location for comparison + let original_metadata_location = table.metadata_location(); + + // Update table properties using the transaction + let tx = tx + .update_table_properties() + .set("test_property".to_string(), "test_value".to_string()) + .apply(tx) + .unwrap(); + + // Commit the transaction to the catalog + let updated_table = tx.commit(&catalog).await.unwrap(); + + // Verify the update was successful + assert_eq!( + updated_table.metadata().properties().get("test_property"), + Some(&"test_value".to_string()) + ); + + // Verify the metadata location has been updated + assert_ne!( + updated_table.metadata_location(), + original_metadata_location, + "Metadata location should be updated after commit" + ); + + // Load the table again from the catalog to verify changes were persisted + let reloaded_table = catalog.load_table(&table_ident).await.unwrap(); + + // Verify the reloaded table matches the updated table + assert_eq!( + reloaded_table.metadata().properties().get("test_property"), + Some(&"test_value".to_string()) + ); + assert_eq!( + reloaded_table.metadata_location(), + updated_table.metadata_location(), + "Reloaded table should have the same metadata location as the updated table" + ); + } + #[tokio::test] async fn test_builder_load_missing_bucket_arn() { let builder = S3TablesCatalogBuilder::default(); From 4a6ea15a7dd73e7c68b0a24441820b13bdd22a92 Mon Sep 17 00:00:00 2001 From: Jannik Steinmann Date: Tue, 28 Oct 2025 19:34:28 +0100 Subject: [PATCH 43/47] feat: Update Datafusion to v50 (#1728) ## Which issue does this PR close? - Closes #. ## What changes are included in this PR? ## Are these changes tested? --------- Signed-off-by: Xuanwo Co-authored-by: Xuanwo --- .github/workflows/bindings_python_ci.yml | 2 +- Cargo.lock | 763 ++++++------- Cargo.toml | 42 +- bindings/python/Cargo.lock | 1327 ++++++++++------------ bindings/python/Cargo.toml | 6 +- bindings/python/pyproject.toml | 2 +- crates/catalog/sql/Cargo.toml | 2 +- crates/iceberg/src/arrow/reader.rs | 11 +- 8 files changed, 956 insertions(+), 1199 deletions(-) diff --git a/.github/workflows/bindings_python_ci.yml b/.github/workflows/bindings_python_ci.yml index 5c904b319d..ae3a6d1a54 100644 --- a/.github/workflows/bindings_python_ci.yml +++ b/.github/workflows/bindings_python_ci.yml @@ -91,6 +91,6 @@ jobs: shell: bash run: | set -e - pip install hatch==1.12.0 + pip install hatch==1.14.2 hatch run dev:pip install dist/pyiceberg_core-*.whl --force-reinstall hatch run dev:test diff --git a/Cargo.lock b/Cargo.lock index 808c222dd3..cd18718290 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -8,12 +8,6 @@ version = "2.0.1" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "320119579fcad9c21884f5c4861d16174d0e06250625266f50fe6898340abefa" -[[package]] -name = "adler32" -version = "1.2.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "aae1277d39aeec15cb388266ecc24b11c80469deae6067e17a1a7aa9e5c1f234" - [[package]] name = "aes" version = "0.8.4" @@ -146,35 +140,6 @@ version = "1.0.100" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "a23eb6b1614318a8071c9b2521f36b424b2c83db5eb3a0fead4a6c0809af6e61" -[[package]] -name = "apache-avro" -version = "0.17.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "1aef82843a0ec9f8b19567445ad2421ceeb1d711514384bdd3d49fe37102ee13" -dependencies = [ - "bigdecimal", - "bzip2 0.4.4", - "crc32fast", - "digest", - "libflate", - "log", - "num-bigint", - "quad-rand", - "rand 0.8.5", - "regex-lite", - "serde", - "serde_bytes", - "serde_json", - "snap", - "strum 0.26.3", - "strum_macros 0.26.4", - "thiserror 1.0.69", - "typed-builder 0.19.1", - "uuid", - "xz2", - "zstd", -] - [[package]] name = "apache-avro" version = "0.20.0" @@ -183,6 +148,8 @@ checksum = "3a033b4ced7c585199fb78ef50fca7fe2f444369ec48080c5fd072efa1a03cc7" dependencies = [ "bigdecimal", "bon", + "bzip2 0.6.1", + "crc32fast", "digest", "log", "miniz_oxide", @@ -193,13 +160,24 @@ dependencies = [ "serde", "serde_bytes", "serde_json", + "snap", "strum 0.27.2", "strum_macros 0.27.2", "thiserror 2.0.17", "uuid", + "xz2", "zstd", ] +[[package]] +name = "ar_archive_writer" +version = "0.2.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "f0c269894b6fe5e9d7ada0cf69b5bf847ff35bc25fc271f08e1d080fce80339a" +dependencies = [ + "object", +] + [[package]] name = "array-init" version = "2.1.0" @@ -220,9 +198,9 @@ checksum = "7c02d123df017efcdfbd739ef81735b36c5ba83ec3c59c80a9d7ecc718f92e50" [[package]] name = "arrow" -version = "55.2.0" +version = "56.2.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "f3f15b4c6b148206ff3a2b35002e08929c2462467b62b9c02036d9c34f9ef994" +checksum = "6e833808ff2d94ed40d9379848a950d995043c7fb3e81a30b383f4c6033821cc" dependencies = [ "arrow-arith", "arrow-array", @@ -241,9 +219,9 @@ dependencies = [ [[package]] name = "arrow-arith" -version = "55.2.0" +version = "56.2.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "30feb679425110209ae35c3fbf82404a39a4c0436bb3ec36164d8bffed2a4ce4" +checksum = "ad08897b81588f60ba983e3ca39bda2b179bdd84dced378e7df81a5313802ef8" dependencies = [ "arrow-array", "arrow-buffer", @@ -255,9 +233,9 @@ dependencies = [ [[package]] name = "arrow-array" -version = "55.2.0" +version = "56.2.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "70732f04d285d49054a48b72c54f791bb3424abae92d27aafdf776c98af161c8" +checksum = "8548ca7c070d8db9ce7aa43f37393e4bfcf3f2d3681df278490772fd1673d08d" dependencies = [ "ahash 0.8.12", "arrow-buffer", @@ -266,15 +244,15 @@ dependencies = [ "chrono", "chrono-tz 0.10.4", "half", - "hashbrown 0.15.5", + "hashbrown 0.16.0", "num", ] [[package]] name = "arrow-buffer" -version = "55.2.0" +version = "56.2.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "169b1d5d6cb390dd92ce582b06b23815c7953e9dfaaea75556e89d890d19993d" +checksum = "e003216336f70446457e280807a73899dd822feaf02087d31febca1363e2fccc" dependencies = [ "bytes", "half", @@ -283,9 +261,9 @@ dependencies = [ [[package]] name = "arrow-cast" -version = "55.2.0" +version = "56.2.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "e4f12eccc3e1c05a766cafb31f6a60a46c2f8efec9b74c6e0648766d30686af8" +checksum = "919418a0681298d3a77d1a315f625916cb5678ad0d74b9c60108eb15fd083023" dependencies = [ "arrow-array", "arrow-buffer", @@ -304,9 +282,9 @@ dependencies = [ [[package]] name = "arrow-csv" -version = "55.2.0" +version = "56.2.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "012c9fef3f4a11573b2c74aec53712ff9fdae4a95f4ce452d1bbf088ee00f06b" +checksum = "bfa9bf02705b5cf762b6f764c65f04ae9082c7cfc4e96e0c33548ee3f67012eb" dependencies = [ "arrow-array", "arrow-cast", @@ -319,9 +297,9 @@ dependencies = [ [[package]] name = "arrow-data" -version = "55.2.0" +version = "56.2.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "8de1ce212d803199684b658fc4ba55fb2d7e87b213de5af415308d2fee3619c2" +checksum = "a5c64fff1d142f833d78897a772f2e5b55b36cb3e6320376f0961ab0db7bd6d0" dependencies = [ "arrow-buffer", "arrow-schema", @@ -331,14 +309,15 @@ dependencies = [ [[package]] name = "arrow-ipc" -version = "55.2.0" +version = "56.2.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d9ea5967e8b2af39aff5d9de2197df16e305f47f404781d3230b2dc672da5d92" +checksum = "1d3594dcddccc7f20fd069bc8e9828ce37220372680ff638c5e00dea427d88f5" dependencies = [ "arrow-array", "arrow-buffer", "arrow-data", "arrow-schema", + "arrow-select", "flatbuffers", "lz4_flex", "zstd", @@ -346,9 +325,9 @@ dependencies = [ [[package]] name = "arrow-json" -version = "55.2.0" +version = "56.2.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "5709d974c4ea5be96d900c01576c7c0b99705f4a3eec343648cb1ca863988a9c" +checksum = "88cf36502b64a127dc659e3b305f1d993a544eab0d48cce704424e62074dc04b" dependencies = [ "arrow-array", "arrow-buffer", @@ -368,9 +347,9 @@ dependencies = [ [[package]] name = "arrow-ord" -version = "55.2.0" +version = "56.2.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "6506e3a059e3be23023f587f79c82ef0bcf6d293587e3272d20f2d30b969b5a7" +checksum = "3c8f82583eb4f8d84d4ee55fd1cb306720cddead7596edce95b50ee418edf66f" dependencies = [ "arrow-array", "arrow-buffer", @@ -381,9 +360,9 @@ dependencies = [ [[package]] name = "arrow-row" -version = "55.2.0" +version = "56.2.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "52bf7393166beaf79b4bed9bfdf19e97472af32ce5b6b48169d321518a08cae2" +checksum = "9d07ba24522229d9085031df6b94605e0f4b26e099fb7cdeec37abd941a73753" dependencies = [ "arrow-array", "arrow-buffer", @@ -394,9 +373,9 @@ dependencies = [ [[package]] name = "arrow-schema" -version = "55.2.0" +version = "56.2.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "af7686986a3bf2254c9fb130c623cdcb2f8e1f15763e7c71c310f0834da3d292" +checksum = "b3aa9e59c611ebc291c28582077ef25c97f1975383f1479b12f3b9ffee2ffabe" dependencies = [ "serde", "serde_json", @@ -404,9 +383,9 @@ dependencies = [ [[package]] name = "arrow-select" -version = "55.2.0" +version = "56.2.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "dd2b45757d6a2373faa3352d02ff5b54b098f5e21dccebc45a21806bc34501e5" +checksum = "8c41dbbd1e97bfcaee4fcb30e29105fb2c75e4d82ae4de70b792a5d3f66b2e7a" dependencies = [ "ahash 0.8.12", "arrow-array", @@ -418,9 +397,9 @@ dependencies = [ [[package]] name = "arrow-string" -version = "55.2.0" +version = "56.2.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "0377d532850babb4d927a06294314b316e23311503ed580ec6ce6a0158f49d40" +checksum = "53f5183c150fbc619eede22b861ea7c0eebed8eaac0333eaa7f6da5205fd504d" dependencies = [ "arrow-array", "arrow-buffer", @@ -581,7 +560,7 @@ checksum = "3b43422f69d8ff38f95f1b2bb76517c91589a924d1559a0e935d7c8ce0274c11" dependencies = [ "proc-macro2", "quote", - "syn 2.0.107", + "syn 2.0.108", ] [[package]] @@ -616,7 +595,7 @@ checksum = "9035ad2d096bed7955a320ee7e2230574d28fd3c3a0f186cbea1ff3c7eed5dbb" dependencies = [ "proc-macro2", "quote", - "syn 2.0.107", + "syn 2.0.108", ] [[package]] @@ -797,9 +776,9 @@ dependencies = [ [[package]] name = "aws-sdk-ssooidc" -version = "1.88.0" +version = "1.89.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "9a68d675582afea0e94d38b6ca9c5aaae4ca14f1d36faa6edb19b42e687e70d7" +checksum = "695dc67bb861ccb8426c9129b91c30e266a0e3d85650cafdf62fcca14c8fd338" dependencies = [ "aws-credential-types", "aws-runtime", @@ -914,7 +893,7 @@ dependencies = [ "hyper-util", "pin-project-lite", "rustls 0.21.12", - "rustls 0.23.33", + "rustls 0.23.34", "rustls-native-certs 0.8.2", "rustls-pki-types", "tokio", @@ -1117,7 +1096,7 @@ dependencies = [ "regex", "rustc-hash", "shlex", - "syn 2.0.107", + "syn 2.0.108", ] [[package]] @@ -1216,7 +1195,7 @@ dependencies = [ "proc-macro2", "quote", "rustversion", - "syn 2.0.107", + "syn 2.0.108", ] [[package]] @@ -1239,7 +1218,7 @@ dependencies = [ "proc-macro-crate", "proc-macro2", "quote", - "syn 2.0.107", + "syn 2.0.108", ] [[package]] @@ -1265,9 +1244,9 @@ dependencies = [ [[package]] name = "bstr" -version = "1.12.0" +version = "1.12.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "234113d19d0d7d613b40e86fb654acf958910802bcceab913a4f9e7cda03b1a4" +checksum = "63044e1ae8e69f3b5a92c736ca6269b8d12fa7efe39bf34ddb06d102cf0e2cab" dependencies = [ "memchr", "serde", @@ -1332,16 +1311,6 @@ dependencies = [ "either", ] -[[package]] -name = "bzip2" -version = "0.4.4" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "bdb116a6ef3f6c3698828873ad02c3014b3c85cadb88496095628e3ef1e347f8" -dependencies = [ - "bzip2-sys", - "libc", -] - [[package]] name = "bzip2" version = "0.5.2" @@ -1353,9 +1322,9 @@ dependencies = [ [[package]] name = "bzip2" -version = "0.6.0" +version = "0.6.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "bea8dcd42434048e4f7a304411d9273a411f647446c1234a65ce0554923f4cff" +checksum = "f3a53fac24f34a81bc9954b5d6cfce0c21e18ec6959f44f56e8e90e4bb7c346c" dependencies = [ "libbz2-rs-sys", ] @@ -1381,9 +1350,9 @@ dependencies = [ [[package]] name = "cc" -version = "1.2.41" +version = "1.2.43" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "ac9fe6cdbb24b6ade63616c0a0688e45bb56732262c158df3c0c4bea4ca47cb7" +checksum = "739eb0f94557554b3ca9a86d2d37bebd49c5e6d0c1d2bda35ba5bdac830befc2" dependencies = [ "find-msvc-tools", "jobserver", @@ -1481,9 +1450,9 @@ dependencies = [ [[package]] name = "clap" -version = "4.5.49" +version = "4.5.50" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "f4512b90fa68d3a9932cea5184017c5d200f5921df706d45e853537dea51508f" +checksum = "0c2cfd7bf8a6017ddaa4e32ffe7403d547790db06bd171c1c53926faab501623" dependencies = [ "clap_builder", "clap_derive", @@ -1491,9 +1460,9 @@ dependencies = [ [[package]] name = "clap_builder" -version = "4.5.49" +version = "4.5.50" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "0025e98baa12e766c67ba13ff4695a887a1eba19569aad00a472546795bd6730" +checksum = "0a4c05b9e80c5ccd3a7ef080ad7b6ba7d6fc00a985b8b157197075677c82c7a0" dependencies = [ "anstream", "anstyle", @@ -1510,7 +1479,7 @@ dependencies = [ "heck", "proc-macro2", "quote", - "syn 2.0.107", + "syn 2.0.108", ] [[package]] @@ -1554,11 +1523,12 @@ dependencies = [ [[package]] name = "comfy-table" -version = "7.2.1" +version = "7.1.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "b03b7db8e0b4b2fdad6c551e634134e99ec000e5c8c3b6856c65e8bbaded7a3b" +checksum = "e0d05af1e006a2407bedef5af410552494ce5be9090444dbbcb57258c1af3d56" dependencies = [ - "unicode-segmentation", + "strum 0.26.3", + "strum_macros 0.26.4", "unicode-width 0.2.2", ] @@ -1573,15 +1543,15 @@ dependencies = [ [[package]] name = "console" -version = "0.15.11" +version = "0.16.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "054ccb5b10f9f2cbf51eb355ca1d05c2d279ce1804688d0db74b4733a5aeafd8" +checksum = "b430743a6eb14e9764d4260d4c0d8123087d504eeb9c48f2b2a5e810dd369df4" dependencies = [ "encode_unicode", "libc", "once_cell", "unicode-width 0.2.2", - "windows-sys 0.59.0", + "windows-sys 0.61.2", ] [[package]] @@ -1642,15 +1612,6 @@ version = "0.8.7" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "773648b94d0e5d620f64f280777445740e61fe701025087ec8b57f45c791888b" -[[package]] -name = "core2" -version = "0.4.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "b49ba7ef1ad6107f8824dbe97de947cbaac53c44e7f9756a1fba0d37c1eec505" -dependencies = [ - "memchr", -] - [[package]] name = "cpufeatures" version = "0.2.17" @@ -1780,7 +1741,7 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "32a2785755761f3ddc1492979ce1e48d2c00d09311c39e4466429188f3dd6501" dependencies = [ "quote", - "syn 2.0.107", + "syn 2.0.108", ] [[package]] @@ -1814,7 +1775,7 @@ dependencies = [ "proc-macro2", "quote", "strsim", - "syn 2.0.107", + "syn 2.0.108", ] [[package]] @@ -1828,7 +1789,7 @@ dependencies = [ "proc-macro2", "quote", "strsim", - "syn 2.0.107", + "syn 2.0.108", ] [[package]] @@ -1839,7 +1800,7 @@ checksum = "fc34b93ccb385b40dc71c6fceac4b2ad23662c7eeb248cf10d529b7e055b6ead" dependencies = [ "darling_core 0.20.11", "quote", - "syn 2.0.107", + "syn 2.0.108", ] [[package]] @@ -1850,15 +1811,9 @@ checksum = "d38308df82d1080de0afee5d069fa14b0326a88c14f15c5ccda35b4a6c414c81" dependencies = [ "darling_core 0.21.3", "quote", - "syn 2.0.107", + "syn 2.0.108", ] -[[package]] -name = "dary_heap" -version = "0.3.8" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "06d2e3287df1c007e74221c49ca10a95d557349e54b3a75dc2fb14712c751f04" - [[package]] name = "dashmap" version = "6.1.0" @@ -1875,16 +1830,16 @@ dependencies = [ [[package]] name = "datafusion" -version = "49.0.2" +version = "50.3.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "69dfeda1633bf8ec75b068d9f6c27cdc392ffcf5ff83128d5dbab65b73c1fd02" +checksum = "2af15bb3c6ffa33011ef579f6b0bcbe7c26584688bd6c994f548e44df67f011a" dependencies = [ "arrow", "arrow-ipc", "arrow-schema", "async-trait", "bytes", - "bzip2 0.6.0", + "bzip2 0.6.1", "chrono", "datafusion-catalog", "datafusion-catalog-listing", @@ -1905,6 +1860,7 @@ dependencies = [ "datafusion-functions-window", "datafusion-optimizer", "datafusion-physical-expr", + "datafusion-physical-expr-adapter", "datafusion-physical-expr-common", "datafusion-physical-optimizer", "datafusion-physical-plan", @@ -1931,9 +1887,9 @@ dependencies = [ [[package]] name = "datafusion-catalog" -version = "49.0.2" +version = "50.3.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "2848fd1e85e2953116dab9cc2eb109214b0888d7bbd2230e30c07f1794f642c0" +checksum = "187622262ad8f7d16d3be9202b4c1e0116f1c9aa387e5074245538b755261621" dependencies = [ "arrow", "async-trait", @@ -1957,9 +1913,9 @@ dependencies = [ [[package]] name = "datafusion-catalog-listing" -version = "49.0.2" +version = "50.3.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "051a1634628c2d1296d4e326823e7536640d87a118966cdaff069b68821ad53b" +checksum = "9657314f0a32efd0382b9a46fdeb2d233273ece64baa68a7c45f5a192daf0f83" dependencies = [ "arrow", "async-trait", @@ -1980,9 +1936,9 @@ dependencies = [ [[package]] name = "datafusion-cli" -version = "49.0.2" +version = "50.3.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "5966ac4973bf66cf5c189046b1b29de90db9ba059ce7193137dd089e277495de" +checksum = "6a0b9c821d14e79070f42ea3a6d6618ced04d94277f0a32301918d7a022c250f" dependencies = [ "arrow", "async-trait", @@ -2006,12 +1962,12 @@ dependencies = [ [[package]] name = "datafusion-common" -version = "49.0.2" +version = "50.3.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "765e4ad4ef7a4500e389a3f1e738791b71ff4c29fd00912c2f541d62b25da096" +checksum = "5a83760d9a13122d025fbdb1d5d5aaf93dd9ada5e90ea229add92aa30898b2d1" dependencies = [ "ahash 0.8.12", - "apache-avro 0.17.0", + "apache-avro", "arrow", "arrow-ipc", "base64 0.22.1", @@ -2033,9 +1989,9 @@ dependencies = [ [[package]] name = "datafusion-common-runtime" -version = "49.0.2" +version = "50.3.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "40a2ae8393051ce25d232a6065c4558ab5a535c9637d5373bacfd464ac88ea12" +checksum = "5b6234a6c7173fe5db1c6c35c01a12b2aa0f803a3007feee53483218817f8b1e" dependencies = [ "futures", "log", @@ -2044,21 +2000,22 @@ dependencies = [ [[package]] name = "datafusion-datasource" -version = "49.0.2" +version = "50.3.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "90cd841a77f378bc1a5c4a1c37345e1885a9203b008203f9f4b3a769729bf330" +checksum = "7256c9cb27a78709dd42d0c80f0178494637209cac6e29d5c93edd09b6721b86" dependencies = [ "arrow", "async-compression", "async-trait", "bytes", - "bzip2 0.6.0", + "bzip2 0.6.1", "chrono", "datafusion-common", "datafusion-common-runtime", "datafusion-execution", "datafusion-expr", "datafusion-physical-expr", + "datafusion-physical-expr-adapter", "datafusion-physical-expr-common", "datafusion-physical-plan", "datafusion-session", @@ -2080,11 +2037,11 @@ dependencies = [ [[package]] name = "datafusion-datasource-avro" -version = "49.0.2" +version = "50.3.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "3cba1696aa919da9517d29164d45f5902d6cc281f718e8d3bfe98bd52cd1142c" +checksum = "10d40b6953ebc9099b37adfd12fde97eb73ff0cee44355c6dea64b8a4537d561" dependencies = [ - "apache-avro 0.17.0", + "apache-avro", "arrow", "async-trait", "bytes", @@ -2105,9 +2062,9 @@ dependencies = [ [[package]] name = "datafusion-datasource-csv" -version = "49.0.2" +version = "50.3.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "77f4a2c64939c6f0dd15b246723a699fa30d59d0133eb36a86e8ff8c6e2a8dc6" +checksum = "64533a90f78e1684bfb113d200b540f18f268134622d7c96bbebc91354d04825" dependencies = [ "arrow", "async-trait", @@ -2130,9 +2087,9 @@ dependencies = [ [[package]] name = "datafusion-datasource-json" -version = "49.0.2" +version = "50.3.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "11387aaf931b2993ad9273c63ddca33f05aef7d02df9b70fb757429b4b71cdae" +checksum = "8d7ebeb12c77df0aacad26f21b0d033aeede423a64b2b352f53048a75bf1d6e6" dependencies = [ "arrow", "async-trait", @@ -2155,9 +2112,9 @@ dependencies = [ [[package]] name = "datafusion-datasource-parquet" -version = "49.0.2" +version = "50.3.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "028f430c5185120bf806347848b8d8acd9823f4038875b3820eeefa35f2bb4a2" +checksum = "09e783c4c7d7faa1199af2df4761c68530634521b176a8d1331ddbc5a5c75133" dependencies = [ "arrow", "async-trait", @@ -2170,6 +2127,7 @@ dependencies = [ "datafusion-expr", "datafusion-functions-aggregate", "datafusion-physical-expr", + "datafusion-physical-expr-adapter", "datafusion-physical-expr-common", "datafusion-physical-optimizer", "datafusion-physical-plan", @@ -2188,17 +2146,18 @@ dependencies = [ [[package]] name = "datafusion-doc" -version = "49.0.2" +version = "50.3.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "8ff336d1d755399753a9e4fbab001180e346fc8bfa063a97f1214b82274c00f8" +checksum = "99ee6b1d9a80d13f9deb2291f45c07044b8e62fb540dbde2453a18be17a36429" [[package]] name = "datafusion-execution" -version = "49.0.2" +version = "50.3.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "042ea192757d1b2d7dcf71643e7ff33f6542c7704f00228d8b85b40003fd8e0f" +checksum = "a4cec0a57653bec7b933fb248d3ffa3fa3ab3bd33bd140dc917f714ac036f531" dependencies = [ "arrow", + "async-trait", "dashmap", "datafusion-common", "datafusion-expr", @@ -2206,6 +2165,7 @@ dependencies = [ "log", "object_store", "parking_lot", + "parquet", "rand 0.9.2", "tempfile", "url", @@ -2213,9 +2173,9 @@ dependencies = [ [[package]] name = "datafusion-expr" -version = "49.0.2" +version = "50.3.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "025222545d6d7fab71e2ae2b356526a1df67a2872222cbae7535e557a42abd2e" +checksum = "ef76910bdca909722586389156d0aa4da4020e1631994d50fadd8ad4b1aa05fe" dependencies = [ "arrow", "async-trait", @@ -2235,9 +2195,9 @@ dependencies = [ [[package]] name = "datafusion-expr-common" -version = "49.0.2" +version = "50.3.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "9d5c267104849d5fa6d81cf5ba88f35ecd58727729c5eb84066c25227b644ae2" +checksum = "6d155ccbda29591ca71a1344dd6bed26c65a4438072b400df9db59447f590bb6" dependencies = [ "arrow", "datafusion-common", @@ -2248,9 +2208,9 @@ dependencies = [ [[package]] name = "datafusion-functions" -version = "49.0.2" +version = "50.3.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c620d105aa208fcee45c588765483314eb415f5571cfd6c1bae3a59c5b4d15bb" +checksum = "7de2782136bd6014670fd84fe3b0ca3b3e4106c96403c3ae05c0598577139977" dependencies = [ "arrow", "arrow-buffer", @@ -2277,9 +2237,9 @@ dependencies = [ [[package]] name = "datafusion-functions-aggregate" -version = "49.0.2" +version = "50.3.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "35f61d5198a35ed368bf3aacac74f0d0fa33de7a7cb0c57e9f68ab1346d2f952" +checksum = "07331fc13603a9da97b74fd8a273f4238222943dffdbbed1c4c6f862a30105bf" dependencies = [ "ahash 0.8.12", "arrow", @@ -2298,9 +2258,9 @@ dependencies = [ [[package]] name = "datafusion-functions-aggregate-common" -version = "49.0.2" +version = "50.3.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "13efdb17362be39b5024f6da0d977ffe49c0212929ec36eec550e07e2bc7812f" +checksum = "b5951e572a8610b89968a09b5420515a121fbc305c0258651f318dc07c97ab17" dependencies = [ "ahash 0.8.12", "arrow", @@ -2311,9 +2271,9 @@ dependencies = [ [[package]] name = "datafusion-functions-nested" -version = "49.0.2" +version = "50.3.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "9187678af567d7c9e004b72a0b6dc5b0a00ebf4901cb3511ed2db4effe092e66" +checksum = "fdacca9302c3d8fc03f3e94f338767e786a88a33f5ebad6ffc0e7b50364b9ea3" dependencies = [ "arrow", "arrow-ord", @@ -2333,9 +2293,9 @@ dependencies = [ [[package]] name = "datafusion-functions-table" -version = "49.0.2" +version = "50.3.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "ecf156589cc21ef59fe39c7a9a841b4a97394549643bbfa88cc44e8588cf8fe5" +checksum = "8c37ff8a99434fbbad604a7e0669717c58c7c4f14c472d45067c4b016621d981" dependencies = [ "arrow", "async-trait", @@ -2349,9 +2309,9 @@ dependencies = [ [[package]] name = "datafusion-functions-window" -version = "49.0.2" +version = "50.3.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "edcb25e3e369f1366ec9a261456e45b5aad6ea1c0c8b4ce546587207c501ed9e" +checksum = "48e2aea7c79c926cffabb13dc27309d4eaeb130f4a21c8ba91cdd241c813652b" dependencies = [ "arrow", "datafusion-common", @@ -2367,9 +2327,9 @@ dependencies = [ [[package]] name = "datafusion-functions-window-common" -version = "49.0.2" +version = "50.3.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "8996a8e11174d0bd7c62dc2f316485affc6ae5ffd5b8a68b508137ace2310294" +checksum = "0fead257ab5fd2ffc3b40fda64da307e20de0040fe43d49197241d9de82a487f" dependencies = [ "datafusion-common", "datafusion-physical-expr-common", @@ -2377,20 +2337,20 @@ dependencies = [ [[package]] name = "datafusion-macros" -version = "49.0.2" +version = "50.3.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "95ee8d1be549eb7316f437035f2cec7ec42aba8374096d807c4de006a3b5d78a" +checksum = "ec6f637bce95efac05cdfb9b6c19579ed4aa5f6b94d951cfa5bb054b7bb4f730" dependencies = [ "datafusion-expr", "quote", - "syn 2.0.107", + "syn 2.0.108", ] [[package]] name = "datafusion-optimizer" -version = "49.0.2" +version = "50.3.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c9fa98671458254928af854e5f6c915e66b860a8bde505baea0ff2892deab74d" +checksum = "c6583ef666ae000a613a837e69e456681a9faa96347bf3877661e9e89e141d8a" dependencies = [ "arrow", "chrono", @@ -2408,9 +2368,9 @@ dependencies = [ [[package]] name = "datafusion-physical-expr" -version = "49.0.2" +version = "50.3.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "3515d51531cca5f7b5a6f3ea22742b71bb36fc378b465df124ff9a2fa349b002" +checksum = "c8668103361a272cbbe3a61f72eca60c9b7c706e87cc3565bcf21e2b277b84f6" dependencies = [ "ahash 0.8.12", "arrow", @@ -2424,15 +2384,31 @@ dependencies = [ "indexmap 2.12.0", "itertools 0.14.0", "log", + "parking_lot", "paste", "petgraph 0.8.3", ] +[[package]] +name = "datafusion-physical-expr-adapter" +version = "50.3.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "815acced725d30601b397e39958e0e55630e0a10d66ef7769c14ae6597298bb0" +dependencies = [ + "arrow", + "datafusion-common", + "datafusion-expr", + "datafusion-functions", + "datafusion-physical-expr", + "datafusion-physical-expr-common", + "itertools 0.14.0", +] + [[package]] name = "datafusion-physical-expr-common" -version = "49.0.2" +version = "50.3.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "24485475d9c618a1d33b2a3dad003d946dc7a7bbf0354d125301abc0a5a79e3e" +checksum = "6652fe7b5bf87e85ed175f571745305565da2c0b599d98e697bcbedc7baa47c3" dependencies = [ "ahash 0.8.12", "arrow", @@ -2444,9 +2420,9 @@ dependencies = [ [[package]] name = "datafusion-physical-optimizer" -version = "49.0.2" +version = "50.3.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "b9da411a0a64702f941a12af2b979434d14ec5d36c6f49296966b2c7639cbb3a" +checksum = "49b7d623eb6162a3332b564a0907ba00895c505d101b99af78345f1acf929b5c" dependencies = [ "arrow", "datafusion-common", @@ -2464,9 +2440,9 @@ dependencies = [ [[package]] name = "datafusion-physical-plan" -version = "49.0.2" +version = "50.3.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a6d168282bb7b54880bb3159f89b51c047db4287f5014d60c3ef4c6e1468212b" +checksum = "e2f7f778a1a838dec124efb96eae6144237d546945587557c9e6936b3414558c" dependencies = [ "ahash 0.8.12", "arrow", @@ -2478,6 +2454,7 @@ dependencies = [ "datafusion-common-runtime", "datafusion-execution", "datafusion-expr", + "datafusion-functions-aggregate-common", "datafusion-functions-window-common", "datafusion-physical-expr", "datafusion-physical-expr-common", @@ -2494,9 +2471,9 @@ dependencies = [ [[package]] name = "datafusion-pruning" -version = "49.0.0" +version = "50.3.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "391a457b9d23744c53eeb89edd1027424cba100581488d89800ed841182df905" +checksum = "cd1e59e2ca14fe3c30f141600b10ad8815e2856caa59ebbd0e3e07cd3d127a65" dependencies = [ "arrow", "arrow-schema", @@ -2512,9 +2489,9 @@ dependencies = [ [[package]] name = "datafusion-session" -version = "49.0.2" +version = "50.3.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "053201c2bb729c7938f85879034df2b5a52cfaba16f1b3b66ab8505c81b2aad3" +checksum = "21ef8e2745583619bd7a49474e8f45fbe98ebb31a133f27802217125a7b3d58d" dependencies = [ "arrow", "async-trait", @@ -2536,11 +2513,13 @@ dependencies = [ [[package]] name = "datafusion-spark" -version = "49.0.2" +version = "50.3.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "126ceb2436b498ddd66c0a9f58304bce6b903f622c681c0ca9e52cf229c55ea0" +checksum = "613efb6666a7d42fcb922b90cd0daa2b25ea486d141350e5d3e86e46df28309a" dependencies = [ "arrow", + "chrono", + "crc32fast", "datafusion-catalog", "datafusion-common", "datafusion-execution", @@ -2548,13 +2527,16 @@ dependencies = [ "datafusion-functions", "datafusion-macros", "log", + "sha1", + "url", + "xxhash-rust", ] [[package]] name = "datafusion-sql" -version = "49.0.2" +version = "50.3.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "9082779be8ce4882189b229c0cff4393bd0808282a7194130c9f32159f185e25" +checksum = "89abd9868770386fede29e5a4b14f49c0bf48d652c3b9d7a8a0332329b87d50b" dependencies = [ "arrow", "bigdecimal", @@ -2569,9 +2551,9 @@ dependencies = [ [[package]] name = "datafusion-sqllogictest" -version = "49.0.2" +version = "50.3.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "2c2358f74d54fb9895b3b962f14877e414327eaaad4a95154abacc3e6bdcd38b" +checksum = "17598193dd875ca895400c51ccab1c30fceb1855220dc60aa415a4db7c95a2d7" dependencies = [ "arrow", "async-trait", @@ -2596,9 +2578,9 @@ dependencies = [ [[package]] name = "datafusion-substrait" -version = "49.0.2" +version = "50.3.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "4189e480ae42f06ade81a6d12853c5f3f84fbe237d73fb8795c712b6f1923afb" +checksum = "eaa011a3814d91a03ab655ad41bbe5e57b203b2859281af8fe2c30aebbbcc5d9" dependencies = [ "async-recursion", "async-trait", @@ -2611,6 +2593,7 @@ dependencies = [ "substrait", "tokio", "url", + "uuid", ] [[package]] @@ -2626,9 +2609,9 @@ dependencies = [ [[package]] name = "deranged" -version = "0.5.4" +version = "0.5.5" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a41953f86f8a05768a6cda24def994fd2f424b04ec5c719cf89989779f199071" +checksum = "ececcb659e7ba858fb4f10388c250a7252eb0a27373f1a72b8748afdd248e587" dependencies = [ "powerfmt", "serde_core", @@ -2652,7 +2635,7 @@ dependencies = [ "darling 0.20.11", "proc-macro2", "quote", - "syn 2.0.107", + "syn 2.0.108", ] [[package]] @@ -2662,7 +2645,7 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "ab63b0e2bf4d5928aff72e83a7dace85d7bba5fe12dcc3c5a572d78caffd3f3c" dependencies = [ "derive_builder_core", - "syn 2.0.107", + "syn 2.0.108", ] [[package]] @@ -2707,7 +2690,7 @@ dependencies = [ "libc", "option-ext", "redox_users", - "windows-sys 0.59.0", + "windows-sys 0.61.2", ] [[package]] @@ -2718,7 +2701,7 @@ checksum = "97369cbbc041bc366949bc74d34658d6cda5621039731c6310521892a3a20ae0" dependencies = [ "proc-macro2", "quote", - "syn 2.0.107", + "syn 2.0.108", ] [[package]] @@ -2769,7 +2752,7 @@ dependencies = [ "enum-ordinalize", "proc-macro2", "quote", - "syn 2.0.107", + "syn 2.0.108", ] [[package]] @@ -2795,22 +2778,22 @@ checksum = "c34f04666d835ff5d62e058c3995147c06f42fe86ff053337632bca83e42702d" [[package]] name = "enum-ordinalize" -version = "4.3.0" +version = "4.3.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "fea0dcfa4e54eeb516fe454635a95753ddd39acda650ce703031c6973e315dd5" +checksum = "4a1091a7bb1f8f2c4b28f1fe2cef4980ca2d410a3d727d67ecc3178c9b0800f0" dependencies = [ "enum-ordinalize-derive", ] [[package]] name = "enum-ordinalize-derive" -version = "4.3.1" +version = "4.3.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "0d28318a75d4aead5c4db25382e8ef717932d0346600cacae6357eb5941bc5ff" +checksum = "8ca9601fb2d62598ee17836250842873a413586e5d7ed88b356e38ddbb0ec631" dependencies = [ "proc-macro2", "quote", - "syn 2.0.107", + "syn 2.0.108", ] [[package]] @@ -2849,7 +2832,7 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "39cab71617ae0d63f51a36d69f866391735b51691dbda63cf6f96d042b63efeb" dependencies = [ "libc", - "windows-sys 0.59.0", + "windows-sys 0.61.2", ] [[package]] @@ -2965,9 +2948,9 @@ dependencies = [ [[package]] name = "flate2" -version = "1.1.4" +version = "1.1.5" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "dc5a4e564e38c699f2880d3fda590bedc2e69f3f84cd48b457bd892ce61d0aa9" +checksum = "bfe33edd8e85a12a67454e37f8c75e730830d83e313556ab9ebf9ee7fbeb3bfb" dependencies = [ "crc32fast", "libz-rs-sys", @@ -3113,7 +3096,7 @@ checksum = "162ee34ebcb7c64a8abebc059ce0fee27c2262618d7b60ed8faf72fef13c3650" dependencies = [ "proc-macro2", "quote", - "syn 2.0.107", + "syn 2.0.108", ] [[package]] @@ -3191,9 +3174,9 @@ checksum = "0cc23270f6e1808e30a928bdc84dea0b9b4136a8bc82338574f23baf47bbd280" [[package]] name = "globset" -version = "0.4.17" +version = "0.4.18" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "eab69130804d941f8075cfd713bf8848a2c3b3f201a9457a11e6f87e1ab62305" +checksum = "52dfc19153a48bde0cbd630453615c8151bce3a5adfac7a0aebfbf0a1e1f57e3" dependencies = [ "aho-corasick", "bstr", @@ -3532,7 +3515,7 @@ dependencies = [ "http 1.3.1", "hyper 1.7.0", "hyper-util", - "rustls 0.23.33", + "rustls 0.23.34", "rustls-native-certs 0.8.2", "rustls-pki-types", "tokio", @@ -3594,7 +3577,7 @@ name = "iceberg" version = "0.7.0" dependencies = [ "anyhow", - "apache-avro 0.20.0", + "apache-avro", "array-init", "arrow-arith", "arrow-array", @@ -3645,7 +3628,7 @@ dependencies = [ "tera", "thrift", "tokio", - "typed-builder 0.20.1", + "typed-builder", "url", "uuid", "zstd", @@ -3735,7 +3718,7 @@ dependencies = [ "serde_json", "tokio", "tracing", - "typed-builder 0.20.1", + "typed-builder", "uuid", ] @@ -3866,9 +3849,9 @@ dependencies = [ [[package]] name = "icu_collections" -version = "2.0.0" +version = "2.1.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "200072f5d0e3614556f94a9930d5dc3e0662a652823904c3a75dc3b0af7fee47" +checksum = "f578a71f2bfaf7ceb30b519a645ae48024b45f9eecbe060a31a004d7b4ba9462" dependencies = [ "displaydoc", "potential_utf", @@ -3879,9 +3862,9 @@ dependencies = [ [[package]] name = "icu_locale_core" -version = "2.0.0" +version = "2.1.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "0cde2700ccaed3872079a65fb1a78f6c0a36c91570f28755dda67bc8f7d9f00a" +checksum = "4c219b62bf5a06801012446193fdfcbd7970e876823aba4c62def2ce957dcb44" dependencies = [ "displaydoc", "litemap", @@ -3892,11 +3875,10 @@ dependencies = [ [[package]] name = "icu_normalizer" -version = "2.0.0" +version = "2.1.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "436880e8e18df4d7bbc06d58432329d6458cc84531f7ac5f024e93deadb37979" +checksum = "33747cecc725eebb47ac503fab725e395d50cb7889ae490a1359f130611d4cc5" dependencies = [ - "displaydoc", "icu_collections", "icu_normalizer_data", "icu_properties", @@ -3907,42 +3889,38 @@ dependencies = [ [[package]] name = "icu_normalizer_data" -version = "2.0.0" +version = "2.1.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "00210d6893afc98edb752b664b8890f0ef174c8adbb8d0be9710fa66fbbf72d3" +checksum = "d6ce2d23e1b3c45624ba6a23e2c767e01c9680e0c0800b39c7abfff9565175d8" [[package]] name = "icu_properties" -version = "2.0.1" +version = "2.1.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "016c619c1eeb94efb86809b015c58f479963de65bdb6253345c1a1276f22e32b" +checksum = "6d70f9b6574c79f7a83ea5ce72cc88d271a3e77355c5f7748a107e751d8617fb" dependencies = [ - "displaydoc", "icu_collections", "icu_locale_core", "icu_properties_data", "icu_provider", - "potential_utf", "zerotrie", "zerovec", ] [[package]] name = "icu_properties_data" -version = "2.0.1" +version = "2.1.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "298459143998310acd25ffe6810ed544932242d3f07083eee1084d83a71bd632" +checksum = "17fa55bf868e28e638ed132bcee1e5c21ba2c1e52c15e7c78b781858e7b54342" [[package]] name = "icu_provider" -version = "2.0.0" +version = "2.1.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "03c80da27b5f4187909049ee2d72f276f0d9f99a42c306bd0131ecfe04d8e5af" +checksum = "f64958e359123591ae1f17a27b5fc9ebdb50c98b04e0401146154de1d8fe3e44" dependencies = [ "displaydoc", "icu_locale_core", - "stable_deref_trait", - "tinystr", "writeable", "yoke", "zerofrom", @@ -4018,14 +3996,14 @@ dependencies = [ [[package]] name = "indicatif" -version = "0.17.11" +version = "0.18.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "183b3088984b400f4cfac3620d5e076c84da5364016b4f49473de574b2586235" +checksum = "ade6dfcba0dfb62ad59e59e7241ec8912af34fd29e0e743e3db992bd278e8b65" dependencies = [ "console", - "number_prefix", "portable-atomic", "unicode-width 0.2.2", + "unit-prefix", "web-time", ] @@ -4073,9 +4051,9 @@ dependencies = [ [[package]] name = "is_terminal_polyfill" -version = "1.70.1" +version = "1.70.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "7943c866cc5cd64cbc25b2e01621d07fa8eb2a1a23160ee81ce38704e97b8ecf" +checksum = "a6cb138bb79a146c1bd460005623e142ef0181e3d0219cb493e02f7d08a35695" [[package]] name = "itertools" @@ -4122,7 +4100,7 @@ checksum = "03343451ff899767262ec32146f6d559dd759fdadf42ff0e227c7c48f72594b4" dependencies = [ "proc-macro2", "quote", - "syn 2.0.107", + "syn 2.0.108", ] [[package]] @@ -4137,9 +4115,9 @@ dependencies = [ [[package]] name = "js-sys" -version = "0.3.81" +version = "0.3.82" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "ec48937a97411dcb524a265206ccd4c90bb711fca92b2792c407f268825b9305" +checksum = "b011eec8cc36da2aab2d5cff675ec18454fad408585853910a202391cf9f8e65" dependencies = [ "once_cell", "wasm-bindgen", @@ -4238,30 +4216,6 @@ version = "0.2.177" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "2874a2af47a2325c2001a6e6fad9b16a53b802102b528163885171cf92b15976" -[[package]] -name = "libflate" -version = "2.1.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "45d9dfdc14ea4ef0900c1cddbc8dcd553fbaacd8a4a282cf4018ae9dd04fb21e" -dependencies = [ - "adler32", - "core2", - "crc32fast", - "dary_heap", - "libflate_lz77", -] - -[[package]] -name = "libflate_lz77" -version = "2.1.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "e6e0d73b369f386f1c44abd9c570d5318f55ccde816ff4b562fa452e5182863d" -dependencies = [ - "core2", - "hashbrown 0.14.5", - "rle-decode-fast", -] - [[package]] name = "libloading" version = "0.8.9" @@ -4356,9 +4310,9 @@ checksum = "df1d3c3b53da64cf5760482273a98e575c651a67eec7f77df96b5b642de8f039" [[package]] name = "litemap" -version = "0.8.0" +version = "0.8.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "241eaef5fd12c88705a01fc1066c48c4b36e0dd4377dcdc7ec3942cea7a69956" +checksum = "6373607a59f0be73a39b6fe456b8192fcc3585f602af20751600e974dd455e77" [[package]] name = "lock_api" @@ -4498,7 +4452,7 @@ dependencies = [ "cfg-if", "proc-macro2", "quote", - "syn 2.0.107", + "syn 2.0.108", ] [[package]] @@ -4566,7 +4520,7 @@ checksum = "b40e46c845ac234bcba19db7ab252bc2778cbadd516a466d2f12b1580852d136" dependencies = [ "proc-macro2", "quote", - "syn 2.0.107", + "syn 2.0.108", ] [[package]] @@ -4592,7 +4546,7 @@ checksum = "4568f25ccbd45ab5d5603dc34318c1ec56b117531781260002151b8530a9f931" dependencies = [ "proc-macro2", "quote", - "syn 2.0.107", + "syn 2.0.108", ] [[package]] @@ -4657,7 +4611,7 @@ version = "0.50.3" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "7957b9740744892f114936ab4a57b3f487491bbeafaf8083688b16841a4240e5" dependencies = [ - "windows-sys 0.59.0", + "windows-sys 0.61.2", ] [[package]] @@ -4787,14 +4741,17 @@ dependencies = [ "proc-macro-crate", "proc-macro2", "quote", - "syn 2.0.107", + "syn 2.0.108", ] [[package]] -name = "number_prefix" -version = "0.4.0" +name = "object" +version = "0.32.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "830b246a0e5f20af87141b25c173cd1b609bd7779a4617d6ec582abaf90870f3" +checksum = "a6a622008b6e321afc04970976f62ee297fdbaa6f95318ca343e3eebb9648441" +dependencies = [ + "memchr", +] [[package]] name = "object_store" @@ -4841,9 +4798,9 @@ checksum = "42f5e15c9953c5e4ccceeb2e7382a716482c34515315f7b03532b8b4e8393d2d" [[package]] name = "once_cell_polyfill" -version = "1.70.1" +version = "1.70.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a4895175b425cb1f87721b59f0f286c2092bd4af812243672510e1ac53e2e0ad" +checksum = "384b8ab6d37215f3c5301a95a4accb5d64aa607f1fcb26a11b5303878451b4fe" [[package]] name = "opendal" @@ -4958,9 +4915,9 @@ dependencies = [ [[package]] name = "parquet" -version = "55.2.0" +version = "56.2.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "b17da4150748086bd43352bc77372efa9b6e3dbd06a04831d2a98c041c225cfa" +checksum = "f0dbd48ad52d7dccf8ea1b90a3ddbfaea4f69878dd7683e51c507d4bc52b5b27" dependencies = [ "ahash 0.8.12", "arrow-array", @@ -4977,7 +4934,7 @@ dependencies = [ "flate2", "futures", "half", - "hashbrown 0.15.5", + "hashbrown 0.16.0", "lz4_flex", "num", "num-bigint", @@ -5110,7 +5067,7 @@ dependencies = [ "pest_meta", "proc-macro2", "quote", - "syn 2.0.107", + "syn 2.0.108", ] [[package]] @@ -5240,7 +5197,7 @@ checksum = "6e918e4ff8c4549eb882f14b3a4bc8c8bc93de829416eacf579f1207a8fbf861" dependencies = [ "proc-macro2", "quote", - "syn 2.0.107", + "syn 2.0.108", ] [[package]] @@ -5376,9 +5333,9 @@ dependencies = [ [[package]] name = "potential_utf" -version = "0.1.3" +version = "0.1.4" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "84df19adbe5b5a0782edcab45899906947ab039ccf4573713735ee7de1e6b08a" +checksum = "b73949432f5e2a09657003c25bca5e19a0e9c84f8058ca374f49e0ebe605af77" dependencies = [ "zerovec", ] @@ -5441,7 +5398,7 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "479ca8adacdd7ce8f1fb39ce9ecccbfe93a3f1344b3d0d97f20bc0196208f62b" dependencies = [ "proc-macro2", - "syn 2.0.107", + "syn 2.0.108", ] [[package]] @@ -5455,9 +5412,9 @@ dependencies = [ [[package]] name = "proc-macro2" -version = "1.0.101" +version = "1.0.103" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "89ae43fd86e4158d6db51ad8e2b80f313af9cc74f5c0e03ccb87de09998732de" +checksum = "5ee95bc4ef87b8d5ba32e8b7714ccc834865276eab0aed5c9958d00ec45f49e8" dependencies = [ "unicode-ident", ] @@ -5488,7 +5445,7 @@ dependencies = [ "prost", "prost-types", "regex", - "syn 2.0.107", + "syn 2.0.108", "tempfile", ] @@ -5502,7 +5459,7 @@ dependencies = [ "itertools 0.14.0", "proc-macro2", "quote", - "syn 2.0.107", + "syn 2.0.108", ] [[package]] @@ -5516,10 +5473,11 @@ dependencies = [ [[package]] name = "psm" -version = "0.1.27" +version = "0.1.28" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "e66fcd288453b748497d8fb18bccc83a16b0518e3906d4b8df0a8d42d93dbb1c" +checksum = "d11f2fedc3b7dafdc2851bc52f277377c5473d378859be234bc7ebb593144d01" dependencies = [ + "ar_archive_writer", "cc", ] @@ -5560,7 +5518,7 @@ checksum = "7347867d0a7e1208d93b46767be83e2b8f978c3dad35f775ac8d8847551d6fe1" dependencies = [ "proc-macro2", "quote", - "syn 2.0.107", + "syn 2.0.108", ] [[package]] @@ -5601,7 +5559,7 @@ dependencies = [ "quinn-proto", "quinn-udp", "rustc-hash", - "rustls 0.23.33", + "rustls 0.23.34", "socket2 0.6.1", "thiserror 2.0.17", "tokio", @@ -5621,7 +5579,7 @@ dependencies = [ "rand 0.9.2", "ring", "rustc-hash", - "rustls 0.23.33", + "rustls 0.23.34", "rustls-pki-types", "slab", "thiserror 2.0.17", @@ -5641,7 +5599,7 @@ dependencies = [ "once_cell", "socket2 0.6.1", "tracing", - "windows-sys 0.59.0", + "windows-sys 0.60.2", ] [[package]] @@ -5762,7 +5720,7 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "76009fbe0614077fc1a2ce255e3a1881a2e3a3527097d5dc6d8212c585e7e38b" dependencies = [ "quote", - "syn 2.0.107", + "syn 2.0.108", ] [[package]] @@ -5802,7 +5760,7 @@ checksum = "b7186006dcb21920990093f30e3dea63b7d6e977bf1256be20c3563a5db070da" dependencies = [ "proc-macro2", "quote", - "syn 2.0.107", + "syn 2.0.108", ] [[package]] @@ -5919,7 +5877,7 @@ dependencies = [ "percent-encoding", "pin-project-lite", "quinn", - "rustls 0.23.33", + "rustls 0.23.34", "rustls-native-certs 0.8.2", "rustls-pki-types", "serde", @@ -6009,15 +5967,9 @@ checksum = "bd83f5f173ff41e00337d97f6572e416d022ef8a19f371817259ae960324c482" dependencies = [ "proc-macro2", "quote", - "syn 2.0.107", + "syn 2.0.108", ] -[[package]] -name = "rle-decode-fast" -version = "1.0.3" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "3582f63211428f83597b51b2ddb88e2a91a9d52d12831f9d08f5e624e8977422" - [[package]] name = "roaring" version = "0.11.2" @@ -6104,7 +6056,7 @@ dependencies = [ "errno", "libc", "linux-raw-sys", - "windows-sys 0.59.0", + "windows-sys 0.61.2", ] [[package]] @@ -6121,9 +6073,9 @@ dependencies = [ [[package]] name = "rustls" -version = "0.23.33" +version = "0.23.34" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "751e04a496ca00bb97a5e043158d23d66b5aabf2e1d5aa2a0aaebb1aafe6f82c" +checksum = "6a9586e9ee2b4f8fab52a0048ca7334d7024eef48e2cb9407e3497bb7cab7fa7" dependencies = [ "aws-lc-rs", "once_cell", @@ -6178,9 +6130,9 @@ dependencies = [ [[package]] name = "rustls-pki-types" -version = "1.12.0" +version = "1.13.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "229a4a4c221013e7e1f1a043678c5cc39fe5171437c88fb47151a21e6f5b5c79" +checksum = "94182ad936a0c91c324cd46c6511b9510ed16af436d7b5bab34beab0afd55f7a" dependencies = [ "web-time", "zeroize", @@ -6216,9 +6168,9 @@ checksum = "b39cdef0fa800fc44525c84ccb54a029961a8215f9619753635a9c0d2538d46d" [[package]] name = "rustyline" -version = "16.0.0" +version = "17.0.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "62fd9ca5ebc709e8535e8ef7c658eb51457987e48c98ead2be482172accc408d" +checksum = "e902948a25149d50edc1a8e0141aad50f54e22ba83ff988cf8f7c9ef07f50564" dependencies = [ "bitflags", "cfg-if", @@ -6233,7 +6185,7 @@ dependencies = [ "unicode-segmentation", "unicode-width 0.2.2", "utf8parse", - "windows-sys 0.59.0", + "windows-sys 0.60.2", ] [[package]] @@ -6314,7 +6266,7 @@ dependencies = [ "proc-macro2", "quote", "serde_derive_internals", - "syn 2.0.107", + "syn 2.0.108", ] [[package]] @@ -6439,7 +6391,7 @@ checksum = "d540f220d3187173da220f885ab66608367b6574e925011a9353e4badda91d79" dependencies = [ "proc-macro2", "quote", - "syn 2.0.107", + "syn 2.0.108", ] [[package]] @@ -6450,7 +6402,7 @@ checksum = "18d26a20a969b9e3fdf2fc2d9f21eda6c40e2de84c9408bb5d3b05d499aae711" dependencies = [ "proc-macro2", "quote", - "syn 2.0.107", + "syn 2.0.108", ] [[package]] @@ -6474,7 +6426,7 @@ checksum = "175ee3e80ae9982737ca543e96133087cbd9a485eecc3bc4de9c1a37b47ea59c" dependencies = [ "proc-macro2", "quote", - "syn 2.0.107", + "syn 2.0.108", ] [[package]] @@ -6495,7 +6447,7 @@ dependencies = [ "proc-macro2", "quote", "serde", - "syn 2.0.107", + "syn 2.0.108", ] [[package]] @@ -6512,9 +6464,9 @@ dependencies = [ [[package]] name = "serde_with" -version = "3.15.0" +version = "3.15.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "6093cd8c01b25262b84927e0f7151692158fab02d961e04c979d3903eba7ecc5" +checksum = "aa66c845eee442168b2c8134fec70ac50dc20e760769c8ba0ad1319ca1959b04" dependencies = [ "base64 0.22.1", "chrono", @@ -6531,14 +6483,14 @@ dependencies = [ [[package]] name = "serde_with_macros" -version = "3.15.0" +version = "3.15.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a7e6c180db0816026a61afa1cff5344fb7ebded7e4d3062772179f2501481c27" +checksum = "b91a903660542fced4e99881aa481bdbaec1634568ee02e0b8bd57c64cb38955" dependencies = [ "darling 0.21.3", "proc-macro2", "quote", - "syn 2.0.107", + "syn 2.0.108", ] [[package]] @@ -6798,9 +6750,9 @@ dependencies = [ [[package]] name = "sqlparser" -version = "0.55.0" +version = "0.58.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c4521174166bac1ff04fe16ef4524c70144cd29682a45978978ca3d7f4e0be11" +checksum = "ec4b661c54b1e4b603b37873a18c59920e4c51ea8ea2cf527d925424dbd4437c" dependencies = [ "log", "recursive", @@ -6815,7 +6767,7 @@ checksum = "da5fc6819faabb412da764b99d3b713bb55083c11e7e0c00144d386cd6a1939c" dependencies = [ "proc-macro2", "quote", - "syn 2.0.107", + "syn 2.0.108", ] [[package]] @@ -6854,7 +6806,7 @@ dependencies = [ "memchr", "once_cell", "percent-encoding", - "rustls 0.23.33", + "rustls 0.23.34", "serde", "serde_json", "sha2", @@ -6877,7 +6829,7 @@ dependencies = [ "quote", "sqlx-core", "sqlx-macros-core", - "syn 2.0.107", + "syn 2.0.108", ] [[package]] @@ -6898,7 +6850,7 @@ dependencies = [ "sha2", "sqlx-core", "sqlx-sqlite", - "syn 2.0.107", + "syn 2.0.108", "tokio", "url", ] @@ -7066,7 +7018,7 @@ dependencies = [ "proc-macro2", "quote", "rustversion", - "syn 2.0.107", + "syn 2.0.108", ] [[package]] @@ -7078,7 +7030,7 @@ dependencies = [ "heck", "proc-macro2", "quote", - "syn 2.0.107", + "syn 2.0.108", ] [[package]] @@ -7111,7 +7063,7 @@ dependencies = [ "serde", "serde_json", "serde_yaml", - "syn 2.0.107", + "syn 2.0.108", "typify", "walkdir", ] @@ -7135,9 +7087,9 @@ dependencies = [ [[package]] name = "syn" -version = "2.0.107" +version = "2.0.108" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "2a26dbd934e5451d21ef060c018dae56fc073894c5a7896f882928a76e6d081b" +checksum = "da58917d35242480a05c2897064da0a80589a2a0476c9a3f2fdc83b53502e917" dependencies = [ "proc-macro2", "quote", @@ -7161,7 +7113,7 @@ checksum = "728a70f3dbaf5bab7f0c4b1ac8d7ae5ea60a4b5549c8a5914361c99147a709d2" dependencies = [ "proc-macro2", "quote", - "syn 2.0.107", + "syn 2.0.108", ] [[package]] @@ -7186,7 +7138,7 @@ dependencies = [ "getrandom 0.3.4", "once_cell", "rustix", - "windows-sys 0.59.0", + "windows-sys 0.61.2", ] [[package]] @@ -7243,7 +7195,7 @@ checksum = "4fee6c4efc90059e10f81e6d42c60a18f76588c3d74cb83a0b242a2b6c7504c1" dependencies = [ "proc-macro2", "quote", - "syn 2.0.107", + "syn 2.0.108", ] [[package]] @@ -7254,7 +7206,7 @@ checksum = "3ff15c8ecd7de3849db632e14d18d2571fa09dfc5ed93479bc4485c7a517c913" dependencies = [ "proc-macro2", "quote", - "syn 2.0.107", + "syn 2.0.108", ] [[package]] @@ -7330,9 +7282,9 @@ dependencies = [ [[package]] name = "tinystr" -version = "0.8.1" +version = "0.8.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "5d4f6d1145dcb577acf783d4e601bc1d76a13337bb54e6233add580b07344c8b" +checksum = "42d3e9c45c09de15d06dd8acf5f4e0e399e85927b7f00711024eb7ae10fa4869" dependencies = [ "displaydoc", "zerovec", @@ -7378,7 +7330,7 @@ checksum = "af407857209536a95c8e56f8231ef2c2e2aff839b22e07a1ffcbc617e9db9fa5" dependencies = [ "proc-macro2", "quote", - "syn 2.0.107", + "syn 2.0.108", ] [[package]] @@ -7397,7 +7349,7 @@ version = "0.26.4" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "1729aa945f29d91ba541258c8df89027d5792d85a8841fb65e8bf0f4ede4ef61" dependencies = [ - "rustls 0.23.33", + "rustls 0.23.34", "tokio", ] @@ -7561,7 +7513,7 @@ checksum = "81383ab64e72a7a8b8e13130c49e3dab29def6d0c7d76a03087b3cf71c5c6903" dependencies = [ "proc-macro2", "quote", - "syn 2.0.107", + "syn 2.0.108", ] [[package]] @@ -7611,33 +7563,13 @@ version = "2.1.2" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "9ea3136b675547379c4bd395ca6b938e5ad3c3d20fad76e7fe85f9e0d011419c" -[[package]] -name = "typed-builder" -version = "0.19.1" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a06fbd5b8de54c5f7c91f6fe4cebb949be2125d7758e630bb58b1d831dbce600" -dependencies = [ - "typed-builder-macro 0.19.1", -] - [[package]] name = "typed-builder" version = "0.20.1" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "cd9d30e3a08026c78f246b173243cf07b3696d274debd26680773b6773c2afc7" dependencies = [ - "typed-builder-macro 0.20.1", -] - -[[package]] -name = "typed-builder-macro" -version = "0.19.1" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "f9534daa9fd3ed0bd911d462a37f172228077e7abf18c18a5f67199d959205f8" -dependencies = [ - "proc-macro2", - "quote", - "syn 2.0.107", + "typed-builder-macro", ] [[package]] @@ -7648,7 +7580,7 @@ checksum = "3c36781cc0e46a83726d9879608e4cf6c2505237e263a8eb8c24502989cfdb28" dependencies = [ "proc-macro2", "quote", - "syn 2.0.107", + "syn 2.0.108", ] [[package]] @@ -7682,7 +7614,7 @@ dependencies = [ "semver", "serde", "serde_json", - "syn 2.0.107", + "syn 2.0.108", "thiserror 2.0.17", "unicode-ident", ] @@ -7700,7 +7632,7 @@ dependencies = [ "serde", "serde_json", "serde_tokenstream", - "syn 2.0.107", + "syn 2.0.108", "typify-impl", ] @@ -7768,9 +7700,9 @@ checksum = "5c1cb5db39152898a79168971543b1cb5020dff7fe43c8dc468b0885f5e29df5" [[package]] name = "unicode-ident" -version = "1.0.19" +version = "1.0.20" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "f63a545481291138910575129486daeaf8ac54aee4387fe7906919f7830c7d9d" +checksum = "462eeb75aeb73aea900253ce739c8e18a67423fadf006037cd3ff27e82748a06" [[package]] name = "unicode-normalization" @@ -7805,6 +7737,12 @@ version = "0.2.2" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "b4ac048d71ede7ee76d585517add45da530660ef4390e49b098733c6e897f254" +[[package]] +name = "unit-prefix" +version = "0.5.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "323402cff2dd658f39ca17c789b502021b3f18707c91cdf22e3838e1b4023817" + [[package]] name = "unsafe-libyaml" version = "0.2.11" @@ -7981,9 +7919,9 @@ checksum = "b8dad83b4f25e74f184f64c43b150b91efe7647395b42289f38e50566d82855b" [[package]] name = "wasm-bindgen" -version = "0.2.104" +version = "0.2.105" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c1da10c01ae9f1ae40cbfac0bac3b1e724b320abfcf52229f80b547c0d250e2d" +checksum = "da95793dfc411fbbd93f5be7715b0578ec61fe87cb1a42b12eb625caa5c5ea60" dependencies = [ "cfg-if", "once_cell", @@ -7992,25 +7930,11 @@ dependencies = [ "wasm-bindgen-shared", ] -[[package]] -name = "wasm-bindgen-backend" -version = "0.2.104" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "671c9a5a66f49d8a47345ab942e2cb93c7d1d0339065d4f8139c486121b43b19" -dependencies = [ - "bumpalo", - "log", - "proc-macro2", - "quote", - "syn 2.0.107", - "wasm-bindgen-shared", -] - [[package]] name = "wasm-bindgen-futures" -version = "0.4.54" +version = "0.4.55" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "7e038d41e478cc73bae0ff9b36c60cff1c98b8f38f8d7e8061e79ee63608ac5c" +checksum = "551f88106c6d5e7ccc7cd9a16f312dd3b5d36ea8b4954304657d5dfba115d4a0" dependencies = [ "cfg-if", "js-sys", @@ -8021,9 +7945,9 @@ dependencies = [ [[package]] name = "wasm-bindgen-macro" -version = "0.2.104" +version = "0.2.105" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "7ca60477e4c59f5f2986c50191cd972e3a50d8a95603bc9434501cf156a9a119" +checksum = "04264334509e04a7bf8690f2384ef5265f05143a4bff3889ab7a3269adab59c2" dependencies = [ "quote", "wasm-bindgen-macro-support", @@ -8031,22 +7955,22 @@ dependencies = [ [[package]] name = "wasm-bindgen-macro-support" -version = "0.2.104" +version = "0.2.105" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "9f07d2f20d4da7b26400c9f4a0511e6e0345b040694e8a75bd41d578fa4421d7" +checksum = "420bc339d9f322e562942d52e115d57e950d12d88983a14c79b86859ee6c7ebc" dependencies = [ + "bumpalo", "proc-macro2", "quote", - "syn 2.0.107", - "wasm-bindgen-backend", + "syn 2.0.108", "wasm-bindgen-shared", ] [[package]] name = "wasm-bindgen-shared" -version = "0.2.104" +version = "0.2.105" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "bad67dc8b2a1a6e5448428adec4c3e84c43e561d8c9ee8a9e5aabeb193ec41d1" +checksum = "76f218a38c84bcb33c25ec7059b07847d465ce0e0a76b995e134a45adcb6af76" dependencies = [ "unicode-ident", ] @@ -8066,9 +7990,9 @@ dependencies = [ [[package]] name = "web-sys" -version = "0.3.81" +version = "0.3.82" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "9367c417a924a74cae129e6a2ae3b47fabb1f8995595ab474029da749a8be120" +checksum = "3a1f95c0d03a47f4ae1f7a64643a6bb97465d9b740f0fa8f90ea33915c99a9a1" dependencies = [ "js-sys", "wasm-bindgen", @@ -8118,7 +8042,7 @@ version = "0.1.11" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "c2a7b1c03c876122aa43f3020e6c3c3ee5c05081c9a00739faf7503aeba10d22" dependencies = [ - "windows-sys 0.48.0", + "windows-sys 0.61.2", ] [[package]] @@ -8142,7 +8066,7 @@ checksum = "053e2e040ab57b9dc951b72c264860db7eb3b0200ba345b4e4c3b14f67855ddf" dependencies = [ "proc-macro2", "quote", - "syn 2.0.107", + "syn 2.0.108", ] [[package]] @@ -8153,7 +8077,7 @@ checksum = "3f316c4a2570ba26bbec722032c4099d8c8bc095efccdc15688708623367e358" dependencies = [ "proc-macro2", "quote", - "syn 2.0.107", + "syn 2.0.108", ] [[package]] @@ -8428,9 +8352,9 @@ checksum = "f17a85883d4e6d00e8a97c586de764dabcc06133f7f1d55dce5cdc070ad7fe59" [[package]] name = "writeable" -version = "0.6.1" +version = "0.6.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "ea2f10b9bb0928dfb1b42b65e1f9e36f7f54dbdf08457afefb38afcdec4fa2bb" +checksum = "9edde0db4769d2dc68579893f2306b26c6ecfbe0ef499b013d731b7b9247e0b9" [[package]] name = "wyz" @@ -8447,6 +8371,12 @@ version = "0.13.6" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "66fee0b777b0f5ac1c69bb06d361268faafa61cd4682ae064a171c16c433e9e4" +[[package]] +name = "xxhash-rust" +version = "0.8.15" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "fdd20c5420375476fbd4394763288da7eb0cc0b8c11deed431a91562af7335d3" + [[package]] name = "xz2" version = "0.1.7" @@ -8464,11 +8394,10 @@ checksum = "cfe53a6657fd280eaa890a3bc59152892ffa3e30101319d168b781ed6529b049" [[package]] name = "yoke" -version = "0.8.0" +version = "0.8.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "5f41bb01b8226ef4bfd589436a297c53d118f65921786300e427be8d487695cc" +checksum = "72d6e5c6afb84d73944e5cedb052c4680d5657337201555f9f2a16b7406d4954" dependencies = [ - "serde", "stable_deref_trait", "yoke-derive", "zerofrom", @@ -8476,13 +8405,13 @@ dependencies = [ [[package]] name = "yoke-derive" -version = "0.8.0" +version = "0.8.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "38da3c9736e16c5d3c8c597a9aaa5d1fa565d0532ae05e27c24aa62fb32c0ab6" +checksum = "b659052874eb698efe5b9e8cf382204678a0086ebf46982b79d6ca3182927e5d" dependencies = [ "proc-macro2", "quote", - "syn 2.0.107", + "syn 2.0.108", "synstructure", ] @@ -8503,7 +8432,7 @@ checksum = "88d2b8d9c68ad2b9e4340d7832716a4d21a22a1154777ad56ea55c51a9cf3831" dependencies = [ "proc-macro2", "quote", - "syn 2.0.107", + "syn 2.0.108", ] [[package]] @@ -8523,7 +8452,7 @@ checksum = "d71e5d6e06ab090c67b5e44993ec16b72dcbaabc526db883a360057678b48502" dependencies = [ "proc-macro2", "quote", - "syn 2.0.107", + "syn 2.0.108", "synstructure", ] @@ -8535,9 +8464,9 @@ checksum = "b97154e67e32c85465826e8bcc1c59429aaaf107c1e4a9e53c8d8ccd5eff88d0" [[package]] name = "zerotrie" -version = "0.2.2" +version = "0.2.3" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "36f0bbd478583f79edad978b407914f61b2972f5af6fa089686016be8f9af595" +checksum = "2a59c17a5562d507e4b54960e8569ebee33bee890c70aa3fe7b97e85a9fd7851" dependencies = [ "displaydoc", "yoke", @@ -8546,9 +8475,9 @@ dependencies = [ [[package]] name = "zerovec" -version = "0.11.4" +version = "0.11.5" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "e7aa2bd55086f1ab526693ecbe444205da57e25f4489879da80635a46d90e73b" +checksum = "6c28719294829477f525be0186d13efa9a3c602f7ec202ca9e353d310fb9a002" dependencies = [ "yoke", "zerofrom", @@ -8557,13 +8486,13 @@ dependencies = [ [[package]] name = "zerovec-derive" -version = "0.11.1" +version = "0.11.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "5b96237efa0c878c64bd89c436f661be4e46b2f3eff1ebb976f7ef2321d2f58f" +checksum = "eadce39539ca5cb3985590102671f2567e659fca9666581ad3411d59207951f3" dependencies = [ "proc-macro2", "quote", - "syn 2.0.107", + "syn 2.0.108", ] [[package]] diff --git a/Cargo.toml b/Cargo.toml index 49eeff038a..6f4385cf1c 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -42,17 +42,17 @@ rust-version = "1.87" anyhow = "1.0.72" apache-avro = { version = "0.20", features = ["zstandard"] } array-init = "2" -arrow-arith = { version = "55.2" } -arrow-array = { version = "55.2" } -arrow-buffer = { version = "55.2" } -arrow-cast = { version = "55.2" } -arrow-ord = { version = "55.2" } -arrow-schema = { version = "55.2" } -arrow-select = { version = "55.2" } -arrow-string = { version = "55.2" } +arrow-arith = "56.2" +arrow-array = "56.2" +arrow-buffer = "56.2" +arrow-cast = "56.2" +arrow-ord = "56.2" +arrow-schema = "56.2" +arrow-select = "56.2" +arrow-string = "56.2" as-any = "0.3.2" -async-trait = "0.1.88" -aws-config = "1.8.1" +async-trait = "0.1.89" +aws-config = "1.8.7" aws-sdk-glue = "1.39" aws-sdk-s3tables = "1.28.0" backon = "1.5.1" @@ -60,11 +60,11 @@ base64 = "0.22.1" bimap = "0.6" bytes = "1.10" chrono = "0.4.41" -clap = { version = "4.5.41", features = ["derive", "cargo"] } +clap = { version = "4.5.48", features = ["derive", "cargo"] } ctor = "0.2.8" -datafusion = "49" -datafusion-cli = "49" -datafusion-sqllogictest = "49" +datafusion = "50" +datafusion-cli = "50" +datafusion-sqllogictest = "50" derive_builder = "0.20" dirs = "6" enum-ordinalize = "4.3.0" @@ -84,7 +84,7 @@ iceberg-catalog-sql = { version = "0.7.0", path = "./crates/catalog/sql" } iceberg-catalog-rest = { version = "0.7.0", path = "./crates/catalog/rest" } iceberg-catalog-s3tables = { version = "0.7.0", path = "./crates/catalog/s3tables" } iceberg-datafusion = { version = "0.7.0", path = "./crates/integrations/datafusion" } -indicatif = "0.17" +indicatif = "0.18" itertools = "0.13" libtest-mimic = "0.8.1" linkedbytes = "0.1.8" @@ -99,12 +99,12 @@ num-bigint = "0.4.6" once_cell = "1.20" opendal = "0.54.0" ordered-float = "4" -parquet = "55.2" +parquet = "56.2" pilota = "0.11.10" port_scanner = "0.1.5" pretty_assertions = "1.4" rand = "0.8.5" -regex = "1.10.5" +regex = "1.11.3" reqwest = { version = "0.12.12", default-features = false, features = ["json"] } roaring = { version = "0.11" } rust_decimal = "1.37.2" @@ -122,12 +122,12 @@ strum = "0.27.2" tempfile = "3.18" tera = "1" thrift = "0.17.0" -tokio = { version = "1.46.1", default-features = false } +tokio = { version = "1.47", default-features = false } toml = "0.8" -tracing = "0.1.40" -tracing-subscriber = "0.3.8" +tracing = "0.1.41" +tracing-subscriber = "0.3.20" typed-builder = "0.20" -url = "2.5.4" +url = "2.5.7" uuid = { version = "1.18", features = ["v7"] } volo = "0.10.6" volo-thrift = "0.10.8" diff --git a/bindings/python/Cargo.lock b/bindings/python/Cargo.lock index 70e95a37aa..8249414b8d 100644 --- a/bindings/python/Cargo.lock +++ b/bindings/python/Cargo.lock @@ -50,15 +50,6 @@ dependencies = [ "core_extensions", ] -[[package]] -name = "addr2line" -version = "0.24.2" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "dfbe277e56a376000877090da837660b4427aad530e3028d44e0bffe4f89a1c1" -dependencies = [ - "gimli", -] - [[package]] name = "adler2" version = "2.0.1" @@ -84,7 +75,7 @@ checksum = "5a15f179cd60c4584b8a8c596927aadc462e27f2ca70c04e0071964a73ba7a75" dependencies = [ "cfg-if", "const-random", - "getrandom 0.3.3", + "getrandom 0.3.4", "once_cell", "version_check", "zerocopy", @@ -131,9 +122,9 @@ dependencies = [ [[package]] name = "anyhow" -version = "1.0.99" +version = "1.0.100" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "b0674a1ddeecb70197781e945de4b3b8ffb61fa939a5597bcf48503737663100" +checksum = "a23eb6b1614318a8071c9b2521f36b424b2c83db5eb3a0fead4a6c0809af6e61" [[package]] name = "apache-avro" @@ -153,13 +144,22 @@ dependencies = [ "serde", "serde_bytes", "serde_json", - "strum", - "strum_macros", - "thiserror 2.0.16", + "strum 0.27.2", + "strum_macros 0.27.2", + "thiserror", "uuid", "zstd", ] +[[package]] +name = "ar_archive_writer" +version = "0.2.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "f0c269894b6fe5e9d7ada0cf69b5bf847ff35bc25fc271f08e1d080fce80339a" +dependencies = [ + "object", +] + [[package]] name = "array-init" version = "2.1.0" @@ -180,9 +180,9 @@ checksum = "7c02d123df017efcdfbd739ef81735b36c5ba83ec3c59c80a9d7ecc718f92e50" [[package]] name = "arrow" -version = "55.2.0" +version = "56.2.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "f3f15b4c6b148206ff3a2b35002e08929c2462467b62b9c02036d9c34f9ef994" +checksum = "6e833808ff2d94ed40d9379848a950d995043c7fb3e81a30b383f4c6033821cc" dependencies = [ "arrow-arith", "arrow-array", @@ -202,9 +202,9 @@ dependencies = [ [[package]] name = "arrow-arith" -version = "55.2.0" +version = "56.2.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "30feb679425110209ae35c3fbf82404a39a4c0436bb3ec36164d8bffed2a4ce4" +checksum = "ad08897b81588f60ba983e3ca39bda2b179bdd84dced378e7df81a5313802ef8" dependencies = [ "arrow-array", "arrow-buffer", @@ -216,9 +216,9 @@ dependencies = [ [[package]] name = "arrow-array" -version = "55.2.0" +version = "56.2.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "70732f04d285d49054a48b72c54f791bb3424abae92d27aafdf776c98af161c8" +checksum = "8548ca7c070d8db9ce7aa43f37393e4bfcf3f2d3681df278490772fd1673d08d" dependencies = [ "ahash 0.8.12", "arrow-buffer", @@ -227,15 +227,15 @@ dependencies = [ "chrono", "chrono-tz", "half", - "hashbrown 0.15.5", + "hashbrown 0.16.0", "num", ] [[package]] name = "arrow-buffer" -version = "55.2.0" +version = "56.2.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "169b1d5d6cb390dd92ce582b06b23815c7953e9dfaaea75556e89d890d19993d" +checksum = "e003216336f70446457e280807a73899dd822feaf02087d31febca1363e2fccc" dependencies = [ "bytes", "half", @@ -244,9 +244,9 @@ dependencies = [ [[package]] name = "arrow-cast" -version = "55.2.0" +version = "56.2.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "e4f12eccc3e1c05a766cafb31f6a60a46c2f8efec9b74c6e0648766d30686af8" +checksum = "919418a0681298d3a77d1a315f625916cb5678ad0d74b9c60108eb15fd083023" dependencies = [ "arrow-array", "arrow-buffer", @@ -265,9 +265,9 @@ dependencies = [ [[package]] name = "arrow-csv" -version = "55.2.0" +version = "56.2.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "012c9fef3f4a11573b2c74aec53712ff9fdae4a95f4ce452d1bbf088ee00f06b" +checksum = "bfa9bf02705b5cf762b6f764c65f04ae9082c7cfc4e96e0c33548ee3f67012eb" dependencies = [ "arrow-array", "arrow-cast", @@ -280,9 +280,9 @@ dependencies = [ [[package]] name = "arrow-data" -version = "55.2.0" +version = "56.2.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "8de1ce212d803199684b658fc4ba55fb2d7e87b213de5af415308d2fee3619c2" +checksum = "a5c64fff1d142f833d78897a772f2e5b55b36cb3e6320376f0961ab0db7bd6d0" dependencies = [ "arrow-buffer", "arrow-schema", @@ -292,14 +292,15 @@ dependencies = [ [[package]] name = "arrow-ipc" -version = "55.2.0" +version = "56.2.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d9ea5967e8b2af39aff5d9de2197df16e305f47f404781d3230b2dc672da5d92" +checksum = "1d3594dcddccc7f20fd069bc8e9828ce37220372680ff638c5e00dea427d88f5" dependencies = [ "arrow-array", "arrow-buffer", "arrow-data", "arrow-schema", + "arrow-select", "flatbuffers", "lz4_flex", "zstd", @@ -307,9 +308,9 @@ dependencies = [ [[package]] name = "arrow-json" -version = "55.2.0" +version = "56.2.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "5709d974c4ea5be96d900c01576c7c0b99705f4a3eec343648cb1ca863988a9c" +checksum = "88cf36502b64a127dc659e3b305f1d993a544eab0d48cce704424e62074dc04b" dependencies = [ "arrow-array", "arrow-buffer", @@ -318,7 +319,7 @@ dependencies = [ "arrow-schema", "chrono", "half", - "indexmap 2.11.0", + "indexmap 2.12.0", "lexical-core", "memchr", "num", @@ -329,9 +330,9 @@ dependencies = [ [[package]] name = "arrow-ord" -version = "55.2.0" +version = "56.2.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "6506e3a059e3be23023f587f79c82ef0bcf6d293587e3272d20f2d30b969b5a7" +checksum = "3c8f82583eb4f8d84d4ee55fd1cb306720cddead7596edce95b50ee418edf66f" dependencies = [ "arrow-array", "arrow-buffer", @@ -342,9 +343,9 @@ dependencies = [ [[package]] name = "arrow-pyarrow" -version = "55.2.0" +version = "56.2.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "0e55ecf16b9b61d433f6e63c72fc6afcf2597d7db96583de88ebb887d1822268" +checksum = "7d924b32e96f8bb74d94cd82bd97b313c432fcb0ea331689ef9e7c6b8be4b258" dependencies = [ "arrow-array", "arrow-data", @@ -354,9 +355,9 @@ dependencies = [ [[package]] name = "arrow-row" -version = "55.2.0" +version = "56.2.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "52bf7393166beaf79b4bed9bfdf19e97472af32ce5b6b48169d321518a08cae2" +checksum = "9d07ba24522229d9085031df6b94605e0f4b26e099fb7cdeec37abd941a73753" dependencies = [ "arrow-array", "arrow-buffer", @@ -367,9 +368,9 @@ dependencies = [ [[package]] name = "arrow-schema" -version = "55.2.0" +version = "56.2.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "af7686986a3bf2254c9fb130c623cdcb2f8e1f15763e7c71c310f0834da3d292" +checksum = "b3aa9e59c611ebc291c28582077ef25c97f1975383f1479b12f3b9ffee2ffabe" dependencies = [ "bitflags", "serde", @@ -378,9 +379,9 @@ dependencies = [ [[package]] name = "arrow-select" -version = "55.2.0" +version = "56.2.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "dd2b45757d6a2373faa3352d02ff5b54b098f5e21dccebc45a21806bc34501e5" +checksum = "8c41dbbd1e97bfcaee4fcb30e29105fb2c75e4d82ae4de70b792a5d3f66b2e7a" dependencies = [ "ahash 0.8.12", "arrow-array", @@ -392,9 +393,9 @@ dependencies = [ [[package]] name = "arrow-string" -version = "55.2.0" +version = "56.2.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "0377d532850babb4d927a06294314b316e23311503ed580ec6ce6a0158f49d40" +checksum = "53f5183c150fbc619eede22b861ea7c0eebed8eaac0333eaa7f6da5205fd504d" dependencies = [ "arrow-array", "arrow-buffer", @@ -470,7 +471,7 @@ checksum = "9035ad2d096bed7955a320ee7e2230574d28fd3c3a0f186cbea1ff3c7eed5dbb" dependencies = [ "proc-macro2", "quote", - "syn 2.0.106", + "syn 2.0.108", ] [[package]] @@ -496,30 +497,15 @@ checksum = "c08606f8c3cbf4ce6ec8e28fb0014a2c086708fe954eaa885384a6165172e7e8" [[package]] name = "backon" -version = "1.5.2" +version = "1.6.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "592277618714fbcecda9a02ba7a8781f319d26532a88553bbacc77ba5d2b3a8d" +checksum = "cffb0e931875b666fc4fcb20fee52e9bbd1ef836fd9e9e04ec21555f9f85f7ef" dependencies = [ "fastrand", "gloo-timers", "tokio", ] -[[package]] -name = "backtrace" -version = "0.3.75" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "6806a6321ec58106fea15becdad98371e28d92ccbc7c8f1b3b6dd724fe8f1002" -dependencies = [ - "addr2line", - "cfg-if", - "libc", - "miniz_oxide", - "object", - "rustc-demangle", - "windows-targets 0.52.6", -] - [[package]] name = "base64" version = "0.22.1" @@ -528,9 +514,9 @@ checksum = "72b3254f16251a8381aa12e40e3c4d2f0199f8c6508fbecb9d91f575e0fbb8c6" [[package]] name = "bigdecimal" -version = "0.4.8" +version = "0.4.9" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "1a22f228ab7a1b23027ccc6c350b72868017af7ea8356fbdf19f8d991c690013" +checksum = "560f42649de9fa436b73517378a147ec21f6c997a546581df4b4b31677828934" dependencies = [ "autocfg", "libm", @@ -548,9 +534,9 @@ checksum = "230c5f1ca6a325a32553f8640d31ac9b49f2411e901e427570154868b46da4f7" [[package]] name = "bitflags" -version = "2.9.4" +version = "2.10.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "2261d10cca569e4643e526d8dc2e62e433cc8aba21ab764233731f8d369bf394" +checksum = "812e12b5285cc515a9c72a5c1d3b6d46a19dac5acfef5265968c166106e31dd3" [[package]] name = "bitvec" @@ -597,9 +583,9 @@ dependencies = [ [[package]] name = "bon" -version = "3.7.2" +version = "3.8.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c2529c31017402be841eb45892278a6c21a000c0a17643af326c73a73f83f0fb" +checksum = "ebeb9aaf9329dff6ceb65c689ca3db33dbf15f324909c60e4e5eef5701ce31b1" dependencies = [ "bon-macros", "rustversion", @@ -607,9 +593,9 @@ dependencies = [ [[package]] name = "bon-macros" -version = "3.7.2" +version = "3.8.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d82020dadcb845a345591863adb65d74fa8dc5c18a0b6d408470e13b7adc7005" +checksum = "77e9d642a7e3a318e37c2c9427b5a6a48aa1ad55dcd986f3034ab2239045a645" dependencies = [ "darling 0.21.3", "ident_case", @@ -617,7 +603,7 @@ dependencies = [ "proc-macro2", "quote", "rustversion", - "syn 2.0.106", + "syn 2.0.108", ] [[package]] @@ -640,7 +626,7 @@ dependencies = [ "proc-macro-crate", "proc-macro2", "quote", - "syn 2.0.106", + "syn 2.0.108", ] [[package]] @@ -694,9 +680,9 @@ dependencies = [ [[package]] name = "bytemuck" -version = "1.23.2" +version = "1.24.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "3995eaeebcdf32f91f980d360f78732ddc061097ab4e39991ae7a6ace9194677" +checksum = "1fbdf580320f38b612e485521afda1ee26d10cc9884efaaa750d383e13e3c5f4" [[package]] name = "byteorder" @@ -721,9 +707,9 @@ dependencies = [ [[package]] name = "bzip2" -version = "0.6.0" +version = "0.6.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "bea8dcd42434048e4f7a304411d9273a411f647446c1234a65ce0554923f4cff" +checksum = "f3a53fac24f34a81bc9954b5d6cfce0c21e18ec6959f44f56e8e90e4bb7c346c" dependencies = [ "libbz2-rs-sys", ] @@ -740,9 +726,9 @@ dependencies = [ [[package]] name = "cc" -version = "1.2.36" +version = "1.2.43" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "5252b3d2648e5eedbc1a6f501e3c795e07025c1e93bbf8bbdd6eef7f447a6d54" +checksum = "739eb0f94557554b3ca9a86d2d37bebd49c5e6d0c1d2bda35ba5bdac830befc2" dependencies = [ "find-msvc-tools", "jobserver", @@ -752,9 +738,9 @@ dependencies = [ [[package]] name = "cfg-if" -version = "1.0.3" +version = "1.0.4" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "2fd1289c04a9ea8cb22300a459a72a385d7c73d3259e2ed7dcb2af674838cfa9" +checksum = "9330f8b2ff13f34540b44e946ef35111825727b38d33286ef986142615121801" [[package]] name = "cfg_aliases" @@ -773,7 +759,7 @@ dependencies = [ "num-traits", "serde", "wasm-bindgen", - "windows-link 0.2.0", + "windows-link", ] [[package]] @@ -788,11 +774,12 @@ dependencies = [ [[package]] name = "comfy-table" -version = "7.2.0" +version = "7.1.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "3f8e18d0dca9578507f13f9803add0df13362b02c501c1c17734f0dbb52eaf0b" +checksum = "e0d05af1e006a2407bedef5af410552494ce5be9090444dbbcb57258c1af3d56" dependencies = [ - "unicode-segmentation", + "strum 0.26.3", + "strum_macros 0.26.4", "unicode-width", ] @@ -833,9 +820,9 @@ dependencies = [ [[package]] name = "const_panic" -version = "0.2.14" +version = "0.2.15" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "bb8a602185c3c95b52f86dc78e55a6df9a287a7a93ddbcf012509930880cf879" +checksum = "e262cdaac42494e3ae34c43969f9cdeb7da178bdb4b66fa6a1ea2edb4c8ae652" dependencies = [ "typewit", ] @@ -936,21 +923,21 @@ dependencies = [ [[package]] name = "csv" -version = "1.3.1" +version = "1.4.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "acdc4883a9c96732e4733212c01447ebd805833b7275a73ca3ee080fd77afdaf" +checksum = "52cd9d68cf7efc6ddfaaee42e7288d3a99d613d4b50f76ce9827ae0c6e14f938" dependencies = [ "csv-core", "itoa", "ryu", - "serde", + "serde_core", ] [[package]] name = "csv-core" -version = "0.1.12" +version = "0.1.13" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "7d02f3b0da4c6504f86e9cd789d8dbafab48c2321be74e9987593de5a894d93d" +checksum = "704a3c26996a80471189265814dbc2c257598b96b8a7feae2d31ace646bb9782" dependencies = [ "memchr", ] @@ -986,7 +973,7 @@ dependencies = [ "proc-macro2", "quote", "strsim", - "syn 2.0.106", + "syn 2.0.108", ] [[package]] @@ -1000,7 +987,7 @@ dependencies = [ "proc-macro2", "quote", "strsim", - "syn 2.0.106", + "syn 2.0.108", ] [[package]] @@ -1011,7 +998,7 @@ checksum = "fc34b93ccb385b40dc71c6fceac4b2ad23662c7eeb248cf10d529b7e055b6ead" dependencies = [ "darling_core 0.20.11", "quote", - "syn 2.0.106", + "syn 2.0.108", ] [[package]] @@ -1022,7 +1009,7 @@ checksum = "d38308df82d1080de0afee5d069fa14b0326a88c14f15c5ccda35b4a6c414c81" dependencies = [ "darling_core 0.21.3", "quote", - "syn 2.0.106", + "syn 2.0.108", ] [[package]] @@ -1041,16 +1028,16 @@ dependencies = [ [[package]] name = "datafusion" -version = "49.0.2" +version = "50.3.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "69dfeda1633bf8ec75b068d9f6c27cdc392ffcf5ff83128d5dbab65b73c1fd02" +checksum = "2af15bb3c6ffa33011ef579f6b0bcbe7c26584688bd6c994f548e44df67f011a" dependencies = [ "arrow", "arrow-ipc", "arrow-schema", "async-trait", "bytes", - "bzip2 0.6.0", + "bzip2 0.6.1", "chrono", "datafusion-catalog", "datafusion-catalog-listing", @@ -1070,6 +1057,7 @@ dependencies = [ "datafusion-functions-window", "datafusion-optimizer", "datafusion-physical-expr", + "datafusion-physical-expr-adapter", "datafusion-physical-expr-common", "datafusion-physical-optimizer", "datafusion-physical-plan", @@ -1077,7 +1065,6 @@ dependencies = [ "datafusion-sql", "flate2", "futures", - "hex", "itertools 0.14.0", "log", "object_store", @@ -1096,9 +1083,9 @@ dependencies = [ [[package]] name = "datafusion-catalog" -version = "49.0.2" +version = "50.3.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "2848fd1e85e2953116dab9cc2eb109214b0888d7bbd2230e30c07f1794f642c0" +checksum = "187622262ad8f7d16d3be9202b4c1e0116f1c9aa387e5074245538b755261621" dependencies = [ "arrow", "async-trait", @@ -1122,9 +1109,9 @@ dependencies = [ [[package]] name = "datafusion-catalog-listing" -version = "49.0.2" +version = "50.3.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "051a1634628c2d1296d4e326823e7536640d87a118966cdaff069b68821ad53b" +checksum = "9657314f0a32efd0382b9a46fdeb2d233273ece64baa68a7c45f5a192daf0f83" dependencies = [ "arrow", "async-trait", @@ -1145,9 +1132,9 @@ dependencies = [ [[package]] name = "datafusion-common" -version = "49.0.2" +version = "50.3.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "765e4ad4ef7a4500e389a3f1e738791b71ff4c29fd00912c2f541d62b25da096" +checksum = "5a83760d9a13122d025fbdb1d5d5aaf93dd9ada5e90ea229add92aa30898b2d1" dependencies = [ "ahash 0.8.12", "arrow", @@ -1156,8 +1143,7 @@ dependencies = [ "chrono", "half", "hashbrown 0.14.5", - "hex", - "indexmap 2.11.0", + "indexmap 2.12.0", "libc", "log", "object_store", @@ -1171,9 +1157,9 @@ dependencies = [ [[package]] name = "datafusion-common-runtime" -version = "49.0.2" +version = "50.3.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "40a2ae8393051ce25d232a6065c4558ab5a535c9637d5373bacfd464ac88ea12" +checksum = "5b6234a6c7173fe5db1c6c35c01a12b2aa0f803a3007feee53483218817f8b1e" dependencies = [ "futures", "log", @@ -1182,21 +1168,22 @@ dependencies = [ [[package]] name = "datafusion-datasource" -version = "49.0.2" +version = "50.3.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "90cd841a77f378bc1a5c4a1c37345e1885a9203b008203f9f4b3a769729bf330" +checksum = "7256c9cb27a78709dd42d0c80f0178494637209cac6e29d5c93edd09b6721b86" dependencies = [ "arrow", "async-compression", "async-trait", "bytes", - "bzip2 0.6.0", + "bzip2 0.6.1", "chrono", "datafusion-common", "datafusion-common-runtime", "datafusion-execution", "datafusion-expr", "datafusion-physical-expr", + "datafusion-physical-expr-adapter", "datafusion-physical-expr-common", "datafusion-physical-plan", "datafusion-session", @@ -1218,9 +1205,9 @@ dependencies = [ [[package]] name = "datafusion-datasource-csv" -version = "49.0.2" +version = "50.3.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "77f4a2c64939c6f0dd15b246723a699fa30d59d0133eb36a86e8ff8c6e2a8dc6" +checksum = "64533a90f78e1684bfb113d200b540f18f268134622d7c96bbebc91354d04825" dependencies = [ "arrow", "async-trait", @@ -1243,9 +1230,9 @@ dependencies = [ [[package]] name = "datafusion-datasource-json" -version = "49.0.2" +version = "50.3.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "11387aaf931b2993ad9273c63ddca33f05aef7d02df9b70fb757429b4b71cdae" +checksum = "8d7ebeb12c77df0aacad26f21b0d033aeede423a64b2b352f53048a75bf1d6e6" dependencies = [ "arrow", "async-trait", @@ -1268,9 +1255,9 @@ dependencies = [ [[package]] name = "datafusion-datasource-parquet" -version = "49.0.2" +version = "50.3.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "028f430c5185120bf806347848b8d8acd9823f4038875b3820eeefa35f2bb4a2" +checksum = "09e783c4c7d7faa1199af2df4761c68530634521b176a8d1331ddbc5a5c75133" dependencies = [ "arrow", "async-trait", @@ -1283,13 +1270,13 @@ dependencies = [ "datafusion-expr", "datafusion-functions-aggregate", "datafusion-physical-expr", + "datafusion-physical-expr-adapter", "datafusion-physical-expr-common", "datafusion-physical-optimizer", "datafusion-physical-plan", "datafusion-pruning", "datafusion-session", "futures", - "hex", "itertools 0.14.0", "log", "object_store", @@ -1301,17 +1288,18 @@ dependencies = [ [[package]] name = "datafusion-doc" -version = "49.0.2" +version = "50.3.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "8ff336d1d755399753a9e4fbab001180e346fc8bfa063a97f1214b82274c00f8" +checksum = "99ee6b1d9a80d13f9deb2291f45c07044b8e62fb540dbde2453a18be17a36429" [[package]] name = "datafusion-execution" -version = "49.0.2" +version = "50.3.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "042ea192757d1b2d7dcf71643e7ff33f6542c7704f00228d8b85b40003fd8e0f" +checksum = "a4cec0a57653bec7b933fb248d3ffa3fa3ab3bd33bd140dc917f714ac036f531" dependencies = [ "arrow", + "async-trait", "dashmap", "datafusion-common", "datafusion-expr", @@ -1326,9 +1314,9 @@ dependencies = [ [[package]] name = "datafusion-expr" -version = "49.0.2" +version = "50.3.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "025222545d6d7fab71e2ae2b356526a1df67a2872222cbae7535e557a42abd2e" +checksum = "ef76910bdca909722586389156d0aa4da4020e1631994d50fadd8ad4b1aa05fe" dependencies = [ "arrow", "async-trait", @@ -1339,7 +1327,7 @@ dependencies = [ "datafusion-functions-aggregate-common", "datafusion-functions-window-common", "datafusion-physical-expr-common", - "indexmap 2.11.0", + "indexmap 2.12.0", "paste", "recursive", "serde_json", @@ -1348,22 +1336,22 @@ dependencies = [ [[package]] name = "datafusion-expr-common" -version = "49.0.2" +version = "50.3.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "9d5c267104849d5fa6d81cf5ba88f35ecd58727729c5eb84066c25227b644ae2" +checksum = "6d155ccbda29591ca71a1344dd6bed26c65a4438072b400df9db59447f590bb6" dependencies = [ "arrow", "datafusion-common", - "indexmap 2.11.0", + "indexmap 2.12.0", "itertools 0.14.0", "paste", ] [[package]] name = "datafusion-ffi" -version = "49.0.2" +version = "50.3.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "ec21805d9df2d834e4c6ddfbf8a1bed2bd460b89b01686fe0dcd1cee06d0b60f" +checksum = "25ddb7c4e645df080c27dad13a198d191da328dd1c98e198664a7a0f64b335cc" dependencies = [ "abi_stable", "arrow", @@ -1383,9 +1371,9 @@ dependencies = [ [[package]] name = "datafusion-functions" -version = "49.0.2" +version = "50.3.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c620d105aa208fcee45c588765483314eb415f5571cfd6c1bae3a59c5b4d15bb" +checksum = "7de2782136bd6014670fd84fe3b0ca3b3e4106c96403c3ae05c0598577139977" dependencies = [ "arrow", "arrow-buffer", @@ -1412,9 +1400,9 @@ dependencies = [ [[package]] name = "datafusion-functions-aggregate" -version = "49.0.2" +version = "50.3.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "35f61d5198a35ed368bf3aacac74f0d0fa33de7a7cb0c57e9f68ab1346d2f952" +checksum = "07331fc13603a9da97b74fd8a273f4238222943dffdbbed1c4c6f862a30105bf" dependencies = [ "ahash 0.8.12", "arrow", @@ -1433,9 +1421,9 @@ dependencies = [ [[package]] name = "datafusion-functions-aggregate-common" -version = "49.0.2" +version = "50.3.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "13efdb17362be39b5024f6da0d977ffe49c0212929ec36eec550e07e2bc7812f" +checksum = "b5951e572a8610b89968a09b5420515a121fbc305c0258651f318dc07c97ab17" dependencies = [ "ahash 0.8.12", "arrow", @@ -1446,9 +1434,9 @@ dependencies = [ [[package]] name = "datafusion-functions-nested" -version = "49.0.2" +version = "50.3.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "9187678af567d7c9e004b72a0b6dc5b0a00ebf4901cb3511ed2db4effe092e66" +checksum = "fdacca9302c3d8fc03f3e94f338767e786a88a33f5ebad6ffc0e7b50364b9ea3" dependencies = [ "arrow", "arrow-ord", @@ -1468,9 +1456,9 @@ dependencies = [ [[package]] name = "datafusion-functions-table" -version = "49.0.2" +version = "50.3.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "ecf156589cc21ef59fe39c7a9a841b4a97394549643bbfa88cc44e8588cf8fe5" +checksum = "8c37ff8a99434fbbad604a7e0669717c58c7c4f14c472d45067c4b016621d981" dependencies = [ "arrow", "async-trait", @@ -1484,9 +1472,9 @@ dependencies = [ [[package]] name = "datafusion-functions-window" -version = "49.0.2" +version = "50.3.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "edcb25e3e369f1366ec9a261456e45b5aad6ea1c0c8b4ce546587207c501ed9e" +checksum = "48e2aea7c79c926cffabb13dc27309d4eaeb130f4a21c8ba91cdd241c813652b" dependencies = [ "arrow", "datafusion-common", @@ -1502,9 +1490,9 @@ dependencies = [ [[package]] name = "datafusion-functions-window-common" -version = "49.0.2" +version = "50.3.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "8996a8e11174d0bd7c62dc2f316485affc6ae5ffd5b8a68b508137ace2310294" +checksum = "0fead257ab5fd2ffc3b40fda64da307e20de0040fe43d49197241d9de82a487f" dependencies = [ "datafusion-common", "datafusion-physical-expr-common", @@ -1512,20 +1500,20 @@ dependencies = [ [[package]] name = "datafusion-macros" -version = "49.0.2" +version = "50.3.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "95ee8d1be549eb7316f437035f2cec7ec42aba8374096d807c4de006a3b5d78a" +checksum = "ec6f637bce95efac05cdfb9b6c19579ed4aa5f6b94d951cfa5bb054b7bb4f730" dependencies = [ "datafusion-expr", "quote", - "syn 2.0.106", + "syn 2.0.108", ] [[package]] name = "datafusion-optimizer" -version = "49.0.2" +version = "50.3.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c9fa98671458254928af854e5f6c915e66b860a8bde505baea0ff2892deab74d" +checksum = "c6583ef666ae000a613a837e69e456681a9faa96347bf3877661e9e89e141d8a" dependencies = [ "arrow", "chrono", @@ -1533,7 +1521,7 @@ dependencies = [ "datafusion-expr", "datafusion-expr-common", "datafusion-physical-expr", - "indexmap 2.11.0", + "indexmap 2.12.0", "itertools 0.14.0", "log", "recursive", @@ -1543,9 +1531,9 @@ dependencies = [ [[package]] name = "datafusion-physical-expr" -version = "49.0.2" +version = "50.3.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "3515d51531cca5f7b5a6f3ea22742b71bb36fc378b465df124ff9a2fa349b002" +checksum = "c8668103361a272cbbe3a61f72eca60c9b7c706e87cc3565bcf21e2b277b84f6" dependencies = [ "ahash 0.8.12", "arrow", @@ -1556,18 +1544,34 @@ dependencies = [ "datafusion-physical-expr-common", "half", "hashbrown 0.14.5", - "indexmap 2.11.0", + "indexmap 2.12.0", "itertools 0.14.0", "log", + "parking_lot", "paste", "petgraph", ] +[[package]] +name = "datafusion-physical-expr-adapter" +version = "50.3.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "815acced725d30601b397e39958e0e55630e0a10d66ef7769c14ae6597298bb0" +dependencies = [ + "arrow", + "datafusion-common", + "datafusion-expr", + "datafusion-functions", + "datafusion-physical-expr", + "datafusion-physical-expr-common", + "itertools 0.14.0", +] + [[package]] name = "datafusion-physical-expr-common" -version = "49.0.2" +version = "50.3.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "24485475d9c618a1d33b2a3dad003d946dc7a7bbf0354d125301abc0a5a79e3e" +checksum = "6652fe7b5bf87e85ed175f571745305565da2c0b599d98e697bcbedc7baa47c3" dependencies = [ "ahash 0.8.12", "arrow", @@ -1579,9 +1583,9 @@ dependencies = [ [[package]] name = "datafusion-physical-optimizer" -version = "49.0.2" +version = "50.3.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "b9da411a0a64702f941a12af2b979434d14ec5d36c6f49296966b2c7639cbb3a" +checksum = "49b7d623eb6162a3332b564a0907ba00895c505d101b99af78345f1acf929b5c" dependencies = [ "arrow", "datafusion-common", @@ -1599,9 +1603,9 @@ dependencies = [ [[package]] name = "datafusion-physical-plan" -version = "49.0.2" +version = "50.3.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a6d168282bb7b54880bb3159f89b51c047db4287f5014d60c3ef4c6e1468212b" +checksum = "e2f7f778a1a838dec124efb96eae6144237d546945587557c9e6936b3414558c" dependencies = [ "ahash 0.8.12", "arrow", @@ -1613,13 +1617,14 @@ dependencies = [ "datafusion-common-runtime", "datafusion-execution", "datafusion-expr", + "datafusion-functions-aggregate-common", "datafusion-functions-window-common", "datafusion-physical-expr", "datafusion-physical-expr-common", "futures", "half", "hashbrown 0.14.5", - "indexmap 2.11.0", + "indexmap 2.12.0", "itertools 0.14.0", "log", "parking_lot", @@ -1629,9 +1634,9 @@ dependencies = [ [[package]] name = "datafusion-proto" -version = "49.0.2" +version = "50.3.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "1b36a0c84f4500efd90487a004b533bd81de1f2bb3f143f71b7526f33b85d2e2" +checksum = "a7df9f606892e6af45763d94d210634eec69b9bb6ced5353381682ff090028a3" dependencies = [ "arrow", "chrono", @@ -1645,9 +1650,9 @@ dependencies = [ [[package]] name = "datafusion-proto-common" -version = "49.0.2" +version = "50.3.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "2ec788be522806740ad6372c0a2f7e45fb37cb37f786d9b77933add49cdd058f" +checksum = "b4b14f288ca4ef77743d9672cafecf3adfffff0b9b04af9af79ecbeaaf736901" dependencies = [ "arrow", "datafusion-common", @@ -1656,9 +1661,9 @@ dependencies = [ [[package]] name = "datafusion-pruning" -version = "49.0.0" +version = "50.3.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "391a457b9d23744c53eeb89edd1027424cba100581488d89800ed841182df905" +checksum = "cd1e59e2ca14fe3c30f141600b10ad8815e2856caa59ebbd0e3e07cd3d127a65" dependencies = [ "arrow", "arrow-schema", @@ -1674,9 +1679,9 @@ dependencies = [ [[package]] name = "datafusion-session" -version = "49.0.2" +version = "50.3.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "053201c2bb729c7938f85879034df2b5a52cfaba16f1b3b66ab8505c81b2aad3" +checksum = "21ef8e2745583619bd7a49474e8f45fbe98ebb31a133f27802217125a7b3d58d" dependencies = [ "arrow", "async-trait", @@ -1698,15 +1703,15 @@ dependencies = [ [[package]] name = "datafusion-sql" -version = "49.0.2" +version = "50.3.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "9082779be8ce4882189b229c0cff4393bd0808282a7194130c9f32159f185e25" +checksum = "89abd9868770386fede29e5a4b14f49c0bf48d652c3b9d7a8a0332329b87d50b" dependencies = [ "arrow", "bigdecimal", "datafusion-common", "datafusion-expr", - "indexmap 2.11.0", + "indexmap 2.12.0", "log", "recursive", "regex", @@ -1715,12 +1720,12 @@ dependencies = [ [[package]] name = "deranged" -version = "0.5.3" +version = "0.5.5" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d630bccd429a5bb5a64b5e94f693bfc48c9f8566418fda4c494cc94f911f87cc" +checksum = "ececcb659e7ba858fb4f10388c250a7252eb0a27373f1a72b8748afdd248e587" dependencies = [ "powerfmt", - "serde", + "serde_core", ] [[package]] @@ -1741,7 +1746,7 @@ dependencies = [ "darling 0.20.11", "proc-macro2", "quote", - "syn 2.0.106", + "syn 2.0.108", ] [[package]] @@ -1751,7 +1756,7 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "ab63b0e2bf4d5928aff72e83a7dace85d7bba5fe12dcc3c5a572d78caffd3f3c" dependencies = [ "derive_builder_core", - "syn 2.0.106", + "syn 2.0.108", ] [[package]] @@ -1774,7 +1779,7 @@ checksum = "97369cbbc041bc366949bc74d34658d6cda5621039731c6310521892a3a20ae0" dependencies = [ "proc-macro2", "quote", - "syn 2.0.106", + "syn 2.0.108", ] [[package]] @@ -1812,12 +1817,12 @@ checksum = "877a4ace8713b0bcf2a4e7eec82529c029f1d0619886d18145fea96c3ffe5c0f" [[package]] name = "errno" -version = "0.3.13" +version = "0.3.14" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "778e2ac28f6c47af28e4907f13ffd1e1ddbd400980a9abd7c8df189bf578a5ad" +checksum = "39cab71617ae0d63f51a36d69f866391735b51691dbda63cf6f96d042b63efeb" dependencies = [ "libc", - "windows-sys 0.60.2", + "windows-sys 0.61.2", ] [[package]] @@ -1859,9 +1864,9 @@ checksum = "37909eebbb50d72f9059c3b6d82c0463f2ff062c9e95845c43a6c9c0355411be" [[package]] name = "find-msvc-tools" -version = "0.1.1" +version = "0.1.4" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "7fd99930f64d146689264c637b5af2f0233a933bef0d8570e2526bf9e083192d" +checksum = "52051878f80a721bb68ebfbc930e07b65ba72f2da88968ea5c06fd6ca3d3a127" [[package]] name = "fixedbitset" @@ -1871,9 +1876,9 @@ checksum = "1d674e81391d1e1ab681a28d99df07927c6d4aa5b027d7da16ba32d1d21ecd99" [[package]] name = "flatbuffers" -version = "25.2.10" +version = "25.9.23" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "1045398c1bfd89168b5fd3f1fc11f6e70b34f6f66300c87d44d3de849463abf1" +checksum = "09b6620799e7340ebd9968d2e0708eb82cf1971e9a16821e2091b6d6e475eed5" dependencies = [ "bitflags", "rustc_version", @@ -1881,9 +1886,9 @@ dependencies = [ [[package]] name = "flate2" -version = "1.1.2" +version = "1.1.5" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "4a3d7db9596fecd151c5f638c0ee5d5bd487b6e0ea232e5dc96d5250f6f94b1d" +checksum = "bfe33edd8e85a12a67454e37f8c75e730830d83e313556ab9ebf9ee7fbeb3bfb" dependencies = [ "crc32fast", "libz-rs-sys", @@ -1973,7 +1978,7 @@ checksum = "162ee34ebcb7c64a8abebc059ce0fee27c2262618d7b60ed8faf72fef13c3650" dependencies = [ "proc-macro2", "quote", - "syn 2.0.106", + "syn 2.0.108", ] [[package]] @@ -2015,25 +2020,11 @@ dependencies = [ "cfg-if", ] -[[package]] -name = "generator" -version = "0.8.7" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "605183a538e3e2a9c1038635cc5c2d194e2ee8fd0d1b66b8349fad7dbacce5a2" -dependencies = [ - "cc", - "cfg-if", - "libc", - "log", - "rustversion", - "windows", -] - [[package]] name = "generic-array" -version = "0.14.7" +version = "0.14.9" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "85649ca51fd72272d7821adaf274ad91c288277713d9c18820d8499a7ff69e9a" +checksum = "4bb6743198531e02858aeaea5398fcc883e71851fcbcb5a2f773e2fb6cb1edf2" dependencies = [ "typenum", "version_check", @@ -2048,30 +2039,24 @@ dependencies = [ "cfg-if", "js-sys", "libc", - "wasi 0.11.1+wasi-snapshot-preview1", + "wasi", "wasm-bindgen", ] [[package]] name = "getrandom" -version = "0.3.3" +version = "0.3.4" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "26145e563e54f2cadc477553f1ec5ee650b00862f0a58bcd12cbdc5f0ea2d2f4" +checksum = "899def5c37c4fd7b2664648c28120ecec138e4d395b459e5ca34f9cce2dd77fd" dependencies = [ "cfg-if", "js-sys", "libc", "r-efi", - "wasi 0.14.4+wasi-0.2.4", + "wasip2", "wasm-bindgen", ] -[[package]] -name = "gimli" -version = "0.31.1" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "07e28edb80900c19c28f1072f2e8aeca7fa06b23cd4169cefe1af5aa3260783f" - [[package]] name = "glob" version = "0.3.3" @@ -2092,13 +2077,14 @@ dependencies = [ [[package]] name = "half" -version = "2.6.0" +version = "2.7.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "459196ed295495a68f7d7fe1d84f6c4b7ff0e21fe3017b2f283c6fac3ad803c9" +checksum = "6ea2d84b969582b4b1864a92dc5d27cd2b77b622a8d79306834f1be5ba20d84b" dependencies = [ "cfg-if", "crunchy", "num-traits", + "zerocopy", ] [[package]] @@ -2126,11 +2112,15 @@ version = "0.15.5" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "9229cfe53dfd69f0609a49f65461bd93001ea1ef889cd5529dd176593f5338a1" dependencies = [ - "allocator-api2", - "equivalent", "foldhash", ] +[[package]] +name = "hashbrown" +version = "0.16.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "5419bdc4f6a9207fbeba6d11b604d481addf78ecd10c11ad51e76c2f6482748d" + [[package]] name = "heck" version = "0.5.0" @@ -2209,9 +2199,9 @@ checksum = "6dbf3de79e51f3d586ab4cb9d5c3e2c14aa28ed23d180cf89b4df0454a69cc87" [[package]] name = "humantime" -version = "2.2.0" +version = "2.3.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "9b112acc8b3adf4b107a8ec20977da0273a8c386765a3ec0229bd500a1443f9f" +checksum = "135b12329e5e3ce057a9f972339ea52bc954fe1e9358ef27f95e89716fbc5424" [[package]] name = "hyper" @@ -2253,9 +2243,9 @@ dependencies = [ [[package]] name = "hyper-util" -version = "0.1.16" +version = "0.1.17" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "8d9b05277c7e8da2c93a568989bb6207bef0112e8d17df7a6eda4a3cf143bc5e" +checksum = "3c6995591a8f1380fcb4ba966a252a4b29188d51d2b89e3a252f5305be65aea8" dependencies = [ "base64", "bytes", @@ -2277,9 +2267,9 @@ dependencies = [ [[package]] name = "iana-time-zone" -version = "0.1.63" +version = "0.1.64" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "b0c919e5debc312ad217002b8048a17b7d83f80703865bbfcfebb0458b0b27d8" +checksum = "33e57f83510bb73707521ebaffa789ec8caf86f9657cad665b092b581d40e9fb" dependencies = [ "android_system_properties", "core-foundation-sys", @@ -2344,7 +2334,7 @@ dependencies = [ "serde_json", "serde_repr", "serde_with", - "strum", + "strum 0.27.2", "thrift", "tokio", "typed-builder", @@ -2369,9 +2359,9 @@ dependencies = [ [[package]] name = "icu_collections" -version = "2.0.0" +version = "2.1.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "200072f5d0e3614556f94a9930d5dc3e0662a652823904c3a75dc3b0af7fee47" +checksum = "f578a71f2bfaf7ceb30b519a645ae48024b45f9eecbe060a31a004d7b4ba9462" dependencies = [ "displaydoc", "potential_utf", @@ -2382,9 +2372,9 @@ dependencies = [ [[package]] name = "icu_locale_core" -version = "2.0.0" +version = "2.1.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "0cde2700ccaed3872079a65fb1a78f6c0a36c91570f28755dda67bc8f7d9f00a" +checksum = "4c219b62bf5a06801012446193fdfcbd7970e876823aba4c62def2ce957dcb44" dependencies = [ "displaydoc", "litemap", @@ -2395,11 +2385,10 @@ dependencies = [ [[package]] name = "icu_normalizer" -version = "2.0.0" +version = "2.1.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "436880e8e18df4d7bbc06d58432329d6458cc84531f7ac5f024e93deadb37979" +checksum = "33747cecc725eebb47ac503fab725e395d50cb7889ae490a1359f130611d4cc5" dependencies = [ - "displaydoc", "icu_collections", "icu_normalizer_data", "icu_properties", @@ -2410,42 +2399,38 @@ dependencies = [ [[package]] name = "icu_normalizer_data" -version = "2.0.0" +version = "2.1.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "00210d6893afc98edb752b664b8890f0ef174c8adbb8d0be9710fa66fbbf72d3" +checksum = "d6ce2d23e1b3c45624ba6a23e2c767e01c9680e0c0800b39c7abfff9565175d8" [[package]] name = "icu_properties" -version = "2.0.1" +version = "2.1.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "016c619c1eeb94efb86809b015c58f479963de65bdb6253345c1a1276f22e32b" +checksum = "6d70f9b6574c79f7a83ea5ce72cc88d271a3e77355c5f7748a107e751d8617fb" dependencies = [ - "displaydoc", "icu_collections", "icu_locale_core", "icu_properties_data", "icu_provider", - "potential_utf", "zerotrie", "zerovec", ] [[package]] name = "icu_properties_data" -version = "2.0.1" +version = "2.1.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "298459143998310acd25ffe6810ed544932242d3f07083eee1084d83a71bd632" +checksum = "17fa55bf868e28e638ed132bcee1e5c21ba2c1e52c15e7c78b781858e7b54342" [[package]] name = "icu_provider" -version = "2.0.0" +version = "2.1.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "03c80da27b5f4187909049ee2d72f276f0d9f99a42c306bd0131ecfe04d8e5af" +checksum = "f64958e359123591ae1f17a27b5fc9ebdb50c98b04e0401146154de1d8fe3e44" dependencies = [ "displaydoc", "icu_locale_core", - "stable_deref_trait", - "tinystr", "writeable", "yoke", "zerofrom", @@ -2493,20 +2478,24 @@ dependencies = [ [[package]] name = "indexmap" -version = "2.11.0" +version = "2.12.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "f2481980430f9f78649238835720ddccc57e52df14ffce1c6f37391d61b563e9" +checksum = "6717a8d2a5a929a1a2eb43a12812498ed141a0bcfb7e8f7844fbdbe4303bba9f" dependencies = [ "equivalent", - "hashbrown 0.15.5", + "hashbrown 0.16.0", "serde", + "serde_core", ] [[package]] name = "indoc" -version = "2.0.6" +version = "2.0.7" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "f4c7245a08504955605670dbf141fceab975f15ca21570696aebe9d2e71576bd" +checksum = "79cf5c93f93228cf8efb3ba362535fb11199ac548a09ce117c9b1adc3030d706" +dependencies = [ + "rustversion", +] [[package]] name = "integer-encoding" @@ -2514,17 +2503,6 @@ version = "3.0.4" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "8bb03732005da905c88227371639bf1ad885cc712789c011c31c5fb3ab3ccf02" -[[package]] -name = "io-uring" -version = "0.7.10" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "046fa2d4d00aea763528b4950358d0ead425372445dc8ff86312b3c69ff7727b" -dependencies = [ - "bitflags", - "cfg-if", - "libc", -] - [[package]] name = "ipnet" version = "2.11.0" @@ -2571,31 +2549,25 @@ version = "0.1.34" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "9afb3de4395d6b3e67a780b6de64b51c978ecf11cb9a462c66be7d4ca9039d33" dependencies = [ - "getrandom 0.3.3", + "getrandom 0.3.4", "libc", ] [[package]] name = "js-sys" -version = "0.3.78" +version = "0.3.82" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "0c0b063578492ceec17683ef2f8c5e89121fbd0b172cbc280635ab7567db2738" +checksum = "b011eec8cc36da2aab2d5cff675ec18454fad408585853910a202391cf9f8e65" dependencies = [ "once_cell", "wasm-bindgen", ] -[[package]] -name = "lazy_static" -version = "1.5.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "bbd2bcb4c963f2ddae06a2efc7e9f3591312473c50c6685e1f298068316e66fe" - [[package]] name = "lexical-core" -version = "1.0.5" +version = "1.0.6" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "b765c31809609075565a70b4b71402281283aeda7ecaf4818ac14a7b2ade8958" +checksum = "7d8d125a277f807e55a77304455eb7b1cb52f2b18c143b60e766c120bd64a594" dependencies = [ "lexical-parse-float", "lexical-parse-integer", @@ -2606,53 +2578,46 @@ dependencies = [ [[package]] name = "lexical-parse-float" -version = "1.0.5" +version = "1.0.6" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "de6f9cb01fb0b08060209a057c048fcbab8717b4c1ecd2eac66ebfe39a65b0f2" +checksum = "52a9f232fbd6f550bc0137dcb5f99ab674071ac2d690ac69704593cb4abbea56" dependencies = [ "lexical-parse-integer", "lexical-util", - "static_assertions", ] [[package]] name = "lexical-parse-integer" -version = "1.0.5" +version = "1.0.6" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "72207aae22fc0a121ba7b6d479e42cbfea549af1479c3f3a4f12c70dd66df12e" +checksum = "9a7a039f8fb9c19c996cd7b2fcce303c1b2874fe1aca544edc85c4a5f8489b34" dependencies = [ "lexical-util", - "static_assertions", ] [[package]] name = "lexical-util" -version = "1.0.6" +version = "1.0.7" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "5a82e24bf537fd24c177ffbbdc6ebcc8d54732c35b50a3f28cc3f4e4c949a0b3" -dependencies = [ - "static_assertions", -] +checksum = "2604dd126bb14f13fb5d1bd6a66155079cb9fa655b37f875b3a742c705dbed17" [[package]] name = "lexical-write-float" -version = "1.0.5" +version = "1.0.6" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c5afc668a27f460fb45a81a757b6bf2f43c2d7e30cb5a2dcd3abf294c78d62bd" +checksum = "50c438c87c013188d415fbabbb1dceb44249ab81664efbd31b14ae55dabb6361" dependencies = [ "lexical-util", "lexical-write-integer", - "static_assertions", ] [[package]] name = "lexical-write-integer" -version = "1.0.5" +version = "1.0.6" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "629ddff1a914a836fb245616a7888b62903aae58fa771e1d83943035efa0f978" +checksum = "409851a618475d2d5796377cad353802345cba92c867d9fbcde9cf4eac4e14df" dependencies = [ "lexical-util", - "static_assertions", ] [[package]] @@ -2663,9 +2628,9 @@ checksum = "2c4a545a15244c7d945065b5d392b2d2d7f21526fba56ce51467b06ed445e8f7" [[package]] name = "libc" -version = "0.2.175" +version = "0.2.177" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "6a82ae493e598baaea5209805c49bbf2ea7de956d50d7da0da1164f9c6d28543" +checksum = "2874a2af47a2325c2001a6e6fad9b16a53b802102b528163885171cf92b15976" [[package]] name = "libloading" @@ -2694,23 +2659,22 @@ dependencies = [ [[package]] name = "linux-raw-sys" -version = "0.9.4" +version = "0.11.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "cd945864f07fe9f5371a27ad7b52a172b4b499999f1d97574c9fa68373937e12" +checksum = "df1d3c3b53da64cf5760482273a98e575c651a67eec7f77df96b5b642de8f039" [[package]] name = "litemap" -version = "0.8.0" +version = "0.8.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "241eaef5fd12c88705a01fc1066c48c4b36e0dd4377dcdc7ec3942cea7a69956" +checksum = "6373607a59f0be73a39b6fe456b8192fcc3585f602af20751600e974dd455e77" [[package]] name = "lock_api" -version = "0.4.13" +version = "0.4.14" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "96936507f153605bddfcda068dd804796c84324ed2510809e5b2a624c81da765" +checksum = "224399e74b87b5f3557511d98dff8b14089b3dadafcab6bb93eab67d3aace965" dependencies = [ - "autocfg", "scopeguard", ] @@ -2720,19 +2684,6 @@ version = "0.4.28" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "34080505efa8e45a4b816c349525ebe327ceaa8559756f0356cba97ef3bf7432" -[[package]] -name = "loom" -version = "0.7.2" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "419e0dc8046cb947daa77eb95ae174acfbddb7673b4151f56d1eed8e93fbfaca" -dependencies = [ - "cfg-if", - "generator", - "scoped-tls", - "tracing", - "tracing-subscriber", -] - [[package]] name = "lru-slab" version = "0.1.2" @@ -2759,15 +2710,6 @@ dependencies = [ "pkg-config", ] -[[package]] -name = "matchers" -version = "0.2.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d1525a2a28c7f4fa0fc98bb91ae755d1e2d1505079e05539e35bc876b5d65ae9" -dependencies = [ - "regex-automata", -] - [[package]] name = "md-5" version = "0.10.6" @@ -2780,9 +2722,9 @@ dependencies = [ [[package]] name = "memchr" -version = "2.7.5" +version = "2.7.6" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "32a282da65faaf38286cf3be983213fcf1d2e2a58700e808f83f4ea9a4804bc0" +checksum = "f52b00d39961fc5b2736ea853c9cc86238e165017a493d1d5c8eac6bdc4cc273" [[package]] name = "memoffset" @@ -2800,38 +2742,38 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "1fa76a2c86f704bdb222d66965fb3d63269ce38518b83cb0575fca855ebb6316" dependencies = [ "adler2", + "simd-adler32", ] [[package]] name = "mio" -version = "1.0.4" +version = "1.1.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "78bed444cc8a2160f01cbcf811ef18cac863ad68ae8ca62092e8db51d51c761c" +checksum = "69d83b0086dc8ecf3ce9ae2874b2d1290252e2a30720bea58a5c6639b0092873" dependencies = [ "libc", - "wasi 0.11.1+wasi-snapshot-preview1", - "windows-sys 0.59.0", + "wasi", + "windows-sys 0.61.2", ] [[package]] name = "moka" -version = "0.12.10" +version = "0.12.11" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a9321642ca94a4282428e6ea4af8cc2ca4eac48ac7a6a4ea8f33f76d0ce70926" +checksum = "8261cd88c312e0004c1d51baad2980c66528dfdb2bee62003e643a4d8f86b077" dependencies = [ "async-lock", "crossbeam-channel", "crossbeam-epoch", "crossbeam-utils", + "equivalent", "event-listener", "futures-util", - "loom", "parking_lot", "portable-atomic", "rustc_version", "smallvec", "tagptr", - "thiserror 1.0.69", "uuid", ] @@ -2841,15 +2783,6 @@ version = "0.5.2" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "9252111cf132ba0929b6f8e030cac2a24b507f3a4d6db6fb2896f27b354c714b" -[[package]] -name = "nu-ansi-term" -version = "0.50.1" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d4a28e057d01f97e61255210fcff094d74ed0466038633e95017f5beb68e4399" -dependencies = [ - "windows-sys 0.52.0", -] - [[package]] name = "num" version = "0.4.3" @@ -2943,18 +2876,18 @@ dependencies = [ [[package]] name = "object" -version = "0.36.7" +version = "0.32.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "62948e14d923ea95ea2c7c86c71013138b66525b86bdc08d2dcc262bdb497b87" +checksum = "a6a622008b6e321afc04970976f62ee297fdbaa6f95318ca343e3eebb9648441" dependencies = [ "memchr", ] [[package]] name = "object_store" -version = "0.12.3" +version = "0.12.4" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "efc4f07659e11cd45a341cd24d71e683e3be65d9ff1f8150061678fe60437496" +checksum = "4c1be0c6c22ec0817cdc77d3842f721a17fd30ab6965001415b5402a74e6b740" dependencies = [ "async-trait", "bytes", @@ -2965,7 +2898,7 @@ dependencies = [ "itertools 0.14.0", "parking_lot", "percent-encoding", - "thiserror 2.0.16", + "thiserror", "tokio", "tracing", "url", @@ -2982,9 +2915,9 @@ checksum = "42f5e15c9953c5e4ccceeb2e7382a716482c34515315f7b03532b8b4e8393d2d" [[package]] name = "opendal" -version = "0.54.0" +version = "0.54.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "ffb9838d0575c6dbaf3fcec7255af8d5771996d4af900bbb6fa9a314dec00a1a" +checksum = "42afda58fa2cf50914402d132cc1caacff116a85d10c72ab2082bb7c50021754" dependencies = [ "anyhow", "backon", @@ -2999,7 +2932,7 @@ dependencies = [ "log", "md-5", "percent-encoding", - "quick-xml", + "quick-xml 0.38.3", "reqsign", "reqwest", "serde", @@ -3044,9 +2977,9 @@ checksum = "f38d5652c16fde515bb1ecef450ab0f6a219d619a7274976324d5e377f7dceba" [[package]] name = "parking_lot" -version = "0.12.4" +version = "0.12.5" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "70d58bf43669b5795d1576d0641cfb6fbb2057bf629506267a92807158584a13" +checksum = "93857453250e3077bd71ff98b6a65ea6621a19bb0f559a85248955ac12c45a1a" dependencies = [ "lock_api", "parking_lot_core", @@ -3054,22 +2987,22 @@ dependencies = [ [[package]] name = "parking_lot_core" -version = "0.9.11" +version = "0.9.12" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "bc838d2a56b5b1a6c25f55575dfc605fabb63bb2365f6c2353ef9159aa69e4a5" +checksum = "2621685985a2ebf1c516881c026032ac7deafcda1a2c9b7850dc81e3dfcb64c1" dependencies = [ "cfg-if", "libc", "redox_syscall", "smallvec", - "windows-targets 0.52.6", + "windows-link", ] [[package]] name = "parquet" -version = "55.2.0" +version = "56.2.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "b17da4150748086bd43352bc77372efa9b6e3dbd06a04831d2a98c041c225cfa" +checksum = "f0dbd48ad52d7dccf8ea1b90a3ddbfaea4f69878dd7683e51c507d4bc52b5b27" dependencies = [ "ahash 0.8.12", "arrow-array", @@ -3086,7 +3019,7 @@ dependencies = [ "flate2", "futures", "half", - "hashbrown 0.15.5", + "hashbrown 0.16.0", "lz4_flex", "num", "num-bigint", @@ -3116,13 +3049,13 @@ checksum = "9b4f627cb1b25917193a259e49bdad08f671f8d9708acfd5fe0a8c1455d87220" [[package]] name = "petgraph" -version = "0.8.2" +version = "0.8.3" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "54acf3a685220b533e437e264e4d932cfbdc4cc7ec0cd232ed73c08d03b8a7ca" +checksum = "8701b58ea97060d5e5b155d383a69952a60943f0e6dfe30b04c287beb0b27455" dependencies = [ "fixedbitset", "hashbrown 0.15.5", - "indexmap 2.11.0", + "indexmap 2.12.0", "serde", ] @@ -3170,9 +3103,9 @@ checksum = "f84267b20a16ea918e43c6a88433c2d54fa145c92a811b5b047ccbe153674483" [[package]] name = "potential_utf" -version = "0.1.3" +version = "0.1.4" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "84df19adbe5b5a0782edcab45899906947ab039ccf4573713735ee7de1e6b08a" +checksum = "b73949432f5e2a09657003c25bca5e19a0e9c84f8058ca374f49e0ebe605af77" dependencies = [ "zerovec", ] @@ -3199,23 +3132,23 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "479ca8adacdd7ce8f1fb39ce9ecccbfe93a3f1344b3d0d97f20bc0196208f62b" dependencies = [ "proc-macro2", - "syn 2.0.106", + "syn 2.0.108", ] [[package]] name = "proc-macro-crate" -version = "3.3.0" +version = "3.4.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "edce586971a4dfaa28950c6f18ed55e0406c1ab88bbce2c6f6293a7aaba73d35" +checksum = "219cb19e96be00ab2e37d6e299658a0cfa83e52429179969b0f0121b4ac46983" dependencies = [ "toml_edit", ] [[package]] name = "proc-macro2" -version = "1.0.101" +version = "1.0.103" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "89ae43fd86e4158d6db51ad8e2b80f313af9cc74f5c0e03ccb87de09998732de" +checksum = "5ee95bc4ef87b8d5ba32e8b7714ccc834865276eab0aed5c9958d00ec45f49e8" dependencies = [ "unicode-ident", ] @@ -3240,15 +3173,16 @@ dependencies = [ "itertools 0.14.0", "proc-macro2", "quote", - "syn 2.0.106", + "syn 2.0.108", ] [[package]] name = "psm" -version = "0.1.26" +version = "0.1.28" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "6e944464ec8536cd1beb0bbfd96987eb5e3b72f2ecdafdc5c769a37f1fa2ae1f" +checksum = "d11f2fedc3b7dafdc2851bc52f277377c5473d378859be234bc7ebb593144d01" dependencies = [ + "ar_archive_writer", "cc", ] @@ -3286,11 +3220,10 @@ dependencies = [ [[package]] name = "pyo3" -version = "0.24.2" +version = "0.25.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "e5203598f366b11a02b13aa20cab591229ff0a89fd121a308a5df751d5fc9219" +checksum = "8970a78afe0628a3e3430376fc5fd76b6b45c4d43360ffd6cdd40bdde72b682a" dependencies = [ - "cfg-if", "indoc", "libc", "memoffset", @@ -3304,9 +3237,9 @@ dependencies = [ [[package]] name = "pyo3-build-config" -version = "0.24.2" +version = "0.25.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "99636d423fa2ca130fa5acde3059308006d46f98caac629418e53f7ebb1e9999" +checksum = "458eb0c55e7ece017adeba38f2248ff3ac615e53660d7c71a238d7d2a01c7598" dependencies = [ "once_cell", "target-lexicon", @@ -3314,9 +3247,9 @@ dependencies = [ [[package]] name = "pyo3-ffi" -version = "0.24.2" +version = "0.25.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "78f9cf92ba9c409279bc3305b5409d90db2d2c22392d443a87df3a1adad59e33" +checksum = "7114fe5457c61b276ab77c5055f206295b812608083644a5c5b2640c3102565c" dependencies = [ "libc", "pyo3-build-config", @@ -3324,27 +3257,27 @@ dependencies = [ [[package]] name = "pyo3-macros" -version = "0.24.2" +version = "0.25.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "0b999cb1a6ce21f9a6b147dcf1be9ffedf02e0043aec74dc390f3007047cecd9" +checksum = "a8725c0a622b374d6cb051d11a0983786448f7785336139c3c94f5aa6bef7e50" dependencies = [ "proc-macro2", "pyo3-macros-backend", "quote", - "syn 2.0.106", + "syn 2.0.108", ] [[package]] name = "pyo3-macros-backend" -version = "0.24.2" +version = "0.25.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "822ece1c7e1012745607d5cf0bcb2874769f0f7cb34c4cde03b9358eb9ef911a" +checksum = "4109984c22491085343c05b0dbc54ddc405c3cf7b4374fc533f5c3313a572ccc" dependencies = [ "heck", "proc-macro2", "pyo3-build-config", "quote", - "syn 2.0.106", + "syn 2.0.108", ] [[package]] @@ -3363,6 +3296,16 @@ dependencies = [ "serde", ] +[[package]] +name = "quick-xml" +version = "0.38.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "42a232e7487fc2ef313d96dde7948e7a3c05101870d8985e4fd8d26aedd27b89" +dependencies = [ + "memchr", + "serde", +] + [[package]] name = "quinn" version = "0.11.9" @@ -3377,7 +3320,7 @@ dependencies = [ "rustc-hash", "rustls", "socket2", - "thiserror 2.0.16", + "thiserror", "tokio", "tracing", "web-time", @@ -3390,7 +3333,7 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "f1906b49b0c3bc04b5fe5d86a77925ae6524a19b816ae38ce1e426255f1d8a31" dependencies = [ "bytes", - "getrandom 0.3.3", + "getrandom 0.3.4", "lru-slab", "rand 0.9.2", "ring", @@ -3398,7 +3341,7 @@ dependencies = [ "rustls", "rustls-pki-types", "slab", - "thiserror 2.0.16", + "thiserror", "tinyvec", "tracing", "web-time", @@ -3420,9 +3363,9 @@ dependencies = [ [[package]] name = "quote" -version = "1.0.40" +version = "1.0.41" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "1885c039570dc00dcb4ff087a89e185fd56bae234ddc7f056a945bf36467248d" +checksum = "ce25767e7b499d1b604768e7cde645d14cc8584231ea6b295e9c9eb22c02e1d1" dependencies = [ "proc-macro2", ] @@ -3495,7 +3438,7 @@ version = "0.9.3" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "99d9a13982dcf210057a8a78572b2217b667c3beacbf3a0d8b454f6f82837d38" dependencies = [ - "getrandom 0.3.3", + "getrandom 0.3.4", ] [[package]] @@ -3515,43 +3458,43 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "76009fbe0614077fc1a2ce255e3a1881a2e3a3527097d5dc6d8212c585e7e38b" dependencies = [ "quote", - "syn 2.0.106", + "syn 2.0.108", ] [[package]] name = "redox_syscall" -version = "0.5.17" +version = "0.5.18" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "5407465600fb0548f1442edf71dd20683c6ed326200ace4b1ef0763521bb3b77" +checksum = "ed2bf2547551a7053d6fdfafda3f938979645c44812fbfcda098faae3f1a362d" dependencies = [ "bitflags", ] [[package]] name = "ref-cast" -version = "1.0.24" +version = "1.0.25" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "4a0ae411dbe946a674d89546582cea4ba2bb8defac896622d6496f14c23ba5cf" +checksum = "f354300ae66f76f1c85c5f84693f0ce81d747e2c3f21a45fef496d89c960bf7d" dependencies = [ "ref-cast-impl", ] [[package]] name = "ref-cast-impl" -version = "1.0.24" +version = "1.0.25" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "1165225c21bff1f3bbce98f5a1f889949bc902d3575308cc7b0de30b4f6d27c7" +checksum = "b7186006dcb21920990093f30e3dea63b7d6e977bf1256be20c3563a5db070da" dependencies = [ "proc-macro2", "quote", - "syn 2.0.106", + "syn 2.0.108", ] [[package]] name = "regex" -version = "1.11.2" +version = "1.12.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "23d7fd106d8c02486a8d64e778353d1cffe08ce79ac2e82f540c86d0facf6912" +checksum = "843bc0191f75f3e22651ae5f1e72939ab2f72a4bc30fa80a066bd66edefc24d4" dependencies = [ "aho-corasick", "memchr", @@ -3561,9 +3504,9 @@ dependencies = [ [[package]] name = "regex-automata" -version = "0.4.10" +version = "0.4.13" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "6b9458fa0bfeeac22b5ca447c63aaf45f28439a709ccd244698632f9aa6394d6" +checksum = "5276caf25ac86c8d810222b3dbb938e512c55c6831a10f3e6ed1c93b84041f1c" dependencies = [ "aho-corasick", "memchr", @@ -3572,15 +3515,15 @@ dependencies = [ [[package]] name = "regex-lite" -version = "0.1.7" +version = "0.1.8" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "943f41321c63ef1c92fd763bfe054d2668f7f225a5c29f0105903dc2fc04ba30" +checksum = "8d942b98df5e658f56f20d592c7f868833fe38115e65c33003d8cd224b0155da" [[package]] name = "regex-syntax" -version = "0.8.6" +version = "0.8.8" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "caf4aa5b0f434c91fe5c7f1ecb6a5ece2130b02ad2a590589dda5146df959001" +checksum = "7a2d987857b319362043e95f5353c0535c1f58eec5336fdfcf626430af7def58" [[package]] name = "rend" @@ -3618,7 +3561,7 @@ dependencies = [ "http", "log", "percent-encoding", - "quick-xml", + "quick-xml 0.37.5", "rand 0.8.5", "reqwest", "rust-ini", @@ -3631,9 +3574,9 @@ dependencies = [ [[package]] name = "reqwest" -version = "0.12.23" +version = "0.12.24" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d429f34c8092b2d42c7c93cec323bb4adeb7c67698f70839adec842ec10c7ceb" +checksum = "9d0946410b9f7b082a427e4ef5c8ff541a88b357bc6c637c40db3a68ac70a36f" dependencies = [ "base64", "bytes", @@ -3735,9 +3678,9 @@ dependencies = [ [[package]] name = "rust_decimal" -version = "1.37.2" +version = "1.39.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "b203a6425500a03e0919c42d3c47caca51e79f1132046626d2c8871c5092035d" +checksum = "35affe401787a9bd846712274d97654355d21b2a2c092a3139aabe31e9022282" dependencies = [ "arrayvec", "borsh", @@ -3749,12 +3692,6 @@ dependencies = [ "serde_json", ] -[[package]] -name = "rustc-demangle" -version = "0.1.26" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "56f7d92ca342cea22a06f2121d944b4fd82af56988c270852495420f961d4ace" - [[package]] name = "rustc-hash" version = "2.1.1" @@ -3772,22 +3709,22 @@ dependencies = [ [[package]] name = "rustix" -version = "1.0.8" +version = "1.1.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "11181fbabf243db407ef8df94a6ce0b2f9a733bd8be4ad02b4eda9602296cac8" +checksum = "cd15f8a2c5551a84d56efdc1cd049089e409ac19a3072d5037a17fd70719ff3e" dependencies = [ "bitflags", "errno", "libc", "linux-raw-sys", - "windows-sys 0.60.2", + "windows-sys 0.61.2", ] [[package]] name = "rustls" -version = "0.23.31" +version = "0.23.34" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c0ebcbd2f03de0fc1122ad9bb24b127a5a6cd51d72604a3f3c50ac459762b6cc" +checksum = "6a9586e9ee2b4f8fab52a0048ca7334d7024eef48e2cb9407e3497bb7cab7fa7" dependencies = [ "once_cell", "ring", @@ -3799,9 +3736,9 @@ dependencies = [ [[package]] name = "rustls-pki-types" -version = "1.12.0" +version = "1.13.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "229a4a4c221013e7e1f1a043678c5cc39fe5171437c88fb47151a21e6f5b5c79" +checksum = "94182ad936a0c91c324cd46c6511b9510ed16af436d7b5bab34beab0afd55f7a" dependencies = [ "web-time", "zeroize", @@ -3809,9 +3746,9 @@ dependencies = [ [[package]] name = "rustls-webpki" -version = "0.103.4" +version = "0.103.7" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "0a17884ae0c1b773f1ccd2bd4a8c72f16da897310a98b0e84bf349ad5ead92fc" +checksum = "e10b3f4191e8a80e6b43eebabfac91e5dcecebb27a71f04e820c47ec41d314bf" dependencies = [ "ring", "rustls-pki-types", @@ -3863,12 +3800,6 @@ dependencies = [ "serde_json", ] -[[package]] -name = "scoped-tls" -version = "1.0.1" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "e1cf6437eb19a8f4a6cc0f7dca544973b0b78843adbfeb3683d1a94a0024a294" - [[package]] name = "scopeguard" version = "1.2.0" @@ -3883,9 +3814,9 @@ checksum = "1c107b6f4780854c8b126e228ea8869f4d7b71260f962fefb57b996b8959ba6b" [[package]] name = "semver" -version = "1.0.26" +version = "1.0.27" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "56e6fa9c48d24d85fb3de5ad847117517440f6beceb7798af16b4a87d616b8d0" +checksum = "d767eb0aabc880b29956c35734170f26ed551a859dbd361d140cdbeca61ab1e2" [[package]] name = "seq-macro" @@ -3895,43 +3826,55 @@ checksum = "1bc711410fbe7399f390ca1c3b60ad0f53f80e95c5eb935e52268a0e2cd49acc" [[package]] name = "serde" -version = "1.0.219" +version = "1.0.228" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "5f0e2c6ed6606019b4e29e69dbaba95b11854410e5347d525002456dbbb786b6" +checksum = "9a8e94ea7f378bd32cbbd37198a4a91436180c5bb472411e48b5ec2e2124ae9e" dependencies = [ + "serde_core", "serde_derive", ] [[package]] name = "serde_bytes" -version = "0.11.17" +version = "0.11.19" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "8437fd221bde2d4ca316d61b90e337e9e702b3820b87d63caa9ba6c02bd06d96" +checksum = "a5d440709e79d88e51ac01c4b72fc6cb7314017bb7da9eeff678aa94c10e3ea8" dependencies = [ "serde", + "serde_core", +] + +[[package]] +name = "serde_core" +version = "1.0.228" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "41d385c7d4ca58e59fc732af25c3983b67ac852c1a25000afe1175de458b67ad" +dependencies = [ + "serde_derive", ] [[package]] name = "serde_derive" -version = "1.0.219" +version = "1.0.228" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "5b0276cf7f2c73365f7157c8123c21cd9a50fbbd844757af28ca1f5925fc2a00" +checksum = "d540f220d3187173da220f885ab66608367b6574e925011a9353e4badda91d79" dependencies = [ "proc-macro2", "quote", - "syn 2.0.106", + "syn 2.0.108", ] [[package]] name = "serde_json" -version = "1.0.143" +version = "1.0.145" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d401abef1d108fbd9cbaebc3e46611f4b1021f714a0597a71f41ee463f5f4a5a" +checksum = "402a6f66d8c709116cf22f558eab210f5a50187f702eb4d7e5ef38d9a7f1c79c" dependencies = [ "itoa", "memchr", "ryu", "serde", + "serde_core", ] [[package]] @@ -3942,7 +3885,7 @@ checksum = "175ee3e80ae9982737ca543e96133087cbd9a485eecc3bc4de9c1a37b47ea59c" dependencies = [ "proc-macro2", "quote", - "syn 2.0.106", + "syn 2.0.108", ] [[package]] @@ -3959,19 +3902,18 @@ dependencies = [ [[package]] name = "serde_with" -version = "3.14.0" +version = "3.15.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "f2c45cd61fefa9db6f254525d46e392b852e0e61d9a1fd36e5bd183450a556d5" +checksum = "aa66c845eee442168b2c8134fec70ac50dc20e760769c8ba0ad1319ca1959b04" dependencies = [ "base64", "chrono", "hex", "indexmap 1.9.3", - "indexmap 2.11.0", + "indexmap 2.12.0", "schemars 0.9.0", "schemars 1.0.4", - "serde", - "serde_derive", + "serde_core", "serde_json", "serde_with_macros", "time", @@ -3979,14 +3921,14 @@ dependencies = [ [[package]] name = "serde_with_macros" -version = "3.14.0" +version = "3.15.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "de90945e6565ce0d9a25098082ed4ee4002e047cb59892c318d66821e14bb30f" +checksum = "b91a903660542fced4e99881aa481bdbaec1634568ee02e0b8bd57c64cb38955" dependencies = [ - "darling 0.20.11", + "darling 0.21.3", "proc-macro2", "quote", - "syn 2.0.106", + "syn 2.0.108", ] [[package]] @@ -4011,21 +3953,18 @@ dependencies = [ "digest", ] -[[package]] -name = "sharded-slab" -version = "0.1.7" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "f40ca3c46823713e0d4209592e8d6e826aa57e928f09752619fc696c499637f6" -dependencies = [ - "lazy_static", -] - [[package]] name = "shlex" version = "1.3.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "0fda2ff0d084019ba4d7c6f371c95d8fd75ce3524c3cb8fb653a3023f6323e64" +[[package]] +name = "simd-adler32" +version = "0.3.7" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "d66dc143e6b11c1eddc06d5c423cfc97062865baf299914ab64caa38182078fe" + [[package]] name = "simdutf8" version = "0.1.5" @@ -4058,19 +3997,19 @@ checksum = "1b6b67fb9a61334225b5b790716f609cd58395f895b3fe8b328786812a40bc3b" [[package]] name = "socket2" -version = "0.6.0" +version = "0.6.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "233504af464074f9d066d7b5416c5f9b894a5862a6506e306f7b816cdd6f1807" +checksum = "17129e116933cf371d018bb80ae557e889637989d8638274fb25622827b03881" dependencies = [ "libc", - "windows-sys 0.59.0", + "windows-sys 0.60.2", ] [[package]] name = "sqlparser" -version = "0.55.0" +version = "0.58.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c4521174166bac1ff04fe16ef4524c70144cd29682a45978978ca3d7f4e0be11" +checksum = "ec4b661c54b1e4b603b37873a18c59920e4c51ea8ea2cf527d925424dbd4437c" dependencies = [ "log", "recursive", @@ -4085,20 +4024,20 @@ checksum = "da5fc6819faabb412da764b99d3b713bb55083c11e7e0c00144d386cd6a1939c" dependencies = [ "proc-macro2", "quote", - "syn 2.0.106", + "syn 2.0.108", ] [[package]] name = "stable_deref_trait" -version = "1.2.0" +version = "1.2.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a8f112729512f8e442d81f95a8a7ddf2b7c6b8a1a6f509a95864142b30cab2d3" +checksum = "6ce2be8dc25455e1f91df71bfa12ad37d7af1092ae736f3a6cd0e37bc7810596" [[package]] name = "stacker" -version = "0.1.21" +version = "0.1.22" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "cddb07e32ddb770749da91081d8d0ac3a16f1a569a18b20348cd371f5dead06b" +checksum = "e1f8b29fb42aafcea4edeeb6b2f2d7ecd0d969c48b4cf0d2e64aafc471dd6e59" dependencies = [ "cc", "cfg-if", @@ -4107,25 +4046,38 @@ dependencies = [ "windows-sys 0.59.0", ] -[[package]] -name = "static_assertions" -version = "1.1.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a2eb9349b6444b326872e140eb1cf5e7c522154d69e7a0ffb0fb81c06b37543f" - [[package]] name = "strsim" version = "0.11.1" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "7da8b5736845d9f2fcb837ea5d9e2628564b3b043a70948a3f0b778838c5fb4f" +[[package]] +name = "strum" +version = "0.26.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "8fec0f0aef304996cf250b31b5a10dee7980c85da9d759361292b8bca5a18f06" + [[package]] name = "strum" version = "0.27.2" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "af23d6f6c1a224baef9d3f61e287d2761385a5b88fdab4eb4c6f11aeb54c4bcf" dependencies = [ - "strum_macros", + "strum_macros 0.27.2", +] + +[[package]] +name = "strum_macros" +version = "0.26.4" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "4c6bee85a5a24955dc440386795aa378cd9cf82acd5f764469152d2270e581be" +dependencies = [ + "heck", + "proc-macro2", + "quote", + "rustversion", + "syn 2.0.108", ] [[package]] @@ -4137,7 +4089,7 @@ dependencies = [ "heck", "proc-macro2", "quote", - "syn 2.0.106", + "syn 2.0.108", ] [[package]] @@ -4159,9 +4111,9 @@ dependencies = [ [[package]] name = "syn" -version = "2.0.106" +version = "2.0.108" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "ede7c438028d4436d71104916910f5bb611972c5cfd7f89b8300a8186e6fada6" +checksum = "da58917d35242480a05c2897064da0a80589a2a0476c9a3f2fdc83b53502e917" dependencies = [ "proc-macro2", "quote", @@ -4185,7 +4137,7 @@ checksum = "728a70f3dbaf5bab7f0c4b1ac8d7ae5ea60a4b5549c8a5914361c99147a709d2" dependencies = [ "proc-macro2", "quote", - "syn 2.0.106", + "syn 2.0.108", ] [[package]] @@ -4202,70 +4154,41 @@ checksum = "55937e1799185b12863d447f42597ed69d9928686b8d88a1df17376a097d8369" [[package]] name = "target-lexicon" -version = "0.13.2" +version = "0.13.3" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "e502f78cdbb8ba4718f566c418c52bc729126ffd16baee5baa718cf25dd5a69a" +checksum = "df7f62577c25e07834649fc3b39fafdc597c0a3527dc1c60129201ccfcbaa50c" [[package]] name = "tempfile" -version = "3.21.0" +version = "3.23.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "15b61f8f20e3a6f7e0649d825294eaf317edce30f82cf6026e7e4cb9222a7d1e" +checksum = "2d31c77bdf42a745371d260a26ca7163f1e0924b64afa0b688e61b5a9fa02f16" dependencies = [ "fastrand", - "getrandom 0.3.3", + "getrandom 0.3.4", "once_cell", "rustix", - "windows-sys 0.60.2", + "windows-sys 0.61.2", ] [[package]] name = "thiserror" -version = "1.0.69" +version = "2.0.17" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "b6aaf5339b578ea85b50e080feb250a3e8ae8cfcdff9a461c9ec2904bc923f52" +checksum = "f63587ca0f12b72a0600bcba1d40081f830876000bb46dd2337a3051618f4fc8" dependencies = [ - "thiserror-impl 1.0.69", -] - -[[package]] -name = "thiserror" -version = "2.0.16" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "3467d614147380f2e4e374161426ff399c91084acd2363eaf549172b3d5e60c0" -dependencies = [ - "thiserror-impl 2.0.16", + "thiserror-impl", ] [[package]] name = "thiserror-impl" -version = "1.0.69" +version = "2.0.17" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "4fee6c4efc90059e10f81e6d42c60a18f76588c3d74cb83a0b242a2b6c7504c1" +checksum = "3ff15c8ecd7de3849db632e14d18d2571fa09dfc5ed93479bc4485c7a517c913" dependencies = [ "proc-macro2", "quote", - "syn 2.0.106", -] - -[[package]] -name = "thiserror-impl" -version = "2.0.16" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "6c5e1be1c48b9172ee610da68fd9cd2770e7a4056cb3fc98710ee6906f0c7960" -dependencies = [ - "proc-macro2", - "quote", - "syn 2.0.106", -] - -[[package]] -name = "thread_local" -version = "1.1.9" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "f60246a4944f24f6e018aa17cdeffb7818b76356965d03b07d6a9886e8962185" -dependencies = [ - "cfg-if", + "syn 2.0.108", ] [[package]] @@ -4292,11 +4215,12 @@ dependencies = [ [[package]] name = "time" -version = "0.3.43" +version = "0.3.44" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "83bde6f1ec10e72d583d91623c939f623002284ef622b87de38cfd546cbf2031" +checksum = "91e7d9e3bb61134e77bde20dd4825b97c010155709965fedf0f49bb138e52a9d" dependencies = [ "deranged", + "itoa", "num-conv", "powerfmt", "serde", @@ -4331,9 +4255,9 @@ dependencies = [ [[package]] name = "tinystr" -version = "0.8.1" +version = "0.8.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "5d4f6d1145dcb577acf783d4e601bc1d76a13337bb54e6233add580b07344c8b" +checksum = "42d3e9c45c09de15d06dd8acf5f4e0e399e85927b7f00711024eb7ae10fa4869" dependencies = [ "displaydoc", "zerovec", @@ -4356,38 +4280,35 @@ checksum = "1f3ccbac311fea05f86f61904b462b55fb3df8837a366dfc601a0161d0532f20" [[package]] name = "tokio" -version = "1.47.1" +version = "1.48.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "89e49afdadebb872d3145a5638b59eb0691ea23e46ca484037cfab3b76b95038" +checksum = "ff360e02eab121e0bc37a2d3b4d4dc622e6eda3a8e5253d5435ecf5bd4c68408" dependencies = [ - "backtrace", "bytes", - "io-uring", "libc", "mio", "pin-project-lite", - "slab", "socket2", "tokio-macros", - "windows-sys 0.59.0", + "windows-sys 0.61.2", ] [[package]] name = "tokio-macros" -version = "2.5.0" +version = "2.6.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "6e06d43f1345a3bcd39f6a56dbb7dcab2ba47e68e8ac134855e7e2bdbaf8cab8" +checksum = "af407857209536a95c8e56f8231ef2c2e2aff839b22e07a1ffcbc617e9db9fa5" dependencies = [ "proc-macro2", "quote", - "syn 2.0.106", + "syn 2.0.108", ] [[package]] name = "tokio-rustls" -version = "0.26.2" +version = "0.26.4" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "8e727b36a1a0e8b74c376ac2211e40c2c8af09fb4013c60d910495810f008e9b" +checksum = "1729aa945f29d91ba541258c8df89027d5792d85a8841fb65e8bf0f4ede4ef61" dependencies = [ "rustls", "tokio", @@ -4408,18 +4329,31 @@ dependencies = [ [[package]] name = "toml_datetime" -version = "0.6.11" +version = "0.7.3" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "22cddaf88f4fbc13c51aebbf5f8eceb5c7c5a9da2ac40a13519eb5b0a0e8f11c" +checksum = "f2cdb639ebbc97961c51720f858597f7f24c4fc295327923af55b74c3c724533" +dependencies = [ + "serde_core", +] [[package]] name = "toml_edit" -version = "0.22.27" +version = "0.23.7" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "41fe8c660ae4257887cf66394862d21dbca4a6ddd26f04a3560410406a2f819a" +checksum = "6485ef6d0d9b5d0ec17244ff7eb05310113c3f316f2d14200d4de56b3cb98f8d" dependencies = [ - "indexmap 2.11.0", + "indexmap 2.12.0", "toml_datetime", + "toml_parser", + "winnow", +] + +[[package]] +name = "toml_parser" +version = "1.0.4" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "c0cbe268d35bdb4bb5a56a2de88d0ad0eb70af5384a99d648cd4b3d04039800e" +dependencies = [ "winnow", ] @@ -4487,7 +4421,7 @@ checksum = "81383ab64e72a7a8b8e13130c49e3dab29def6d0c7d76a03087b3cf71c5c6903" dependencies = [ "proc-macro2", "quote", - "syn 2.0.106", + "syn 2.0.108", ] [[package]] @@ -4497,36 +4431,6 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "b9d12581f227e93f094d3af2ae690a574abb8a2b9b7a96e7cfe9647b2b617678" dependencies = [ "once_cell", - "valuable", -] - -[[package]] -name = "tracing-log" -version = "0.2.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "ee855f1f400bd0e5c02d150ae5de3840039a3f54b025156404e34c23c03f47c3" -dependencies = [ - "log", - "once_cell", - "tracing-core", -] - -[[package]] -name = "tracing-subscriber" -version = "0.3.20" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "2054a14f5307d601f88daf0553e1cbf472acc4f2c51afab632431cdcd72124d5" -dependencies = [ - "matchers", - "nu-ansi-term", - "once_cell", - "regex-automata", - "sharded-slab", - "smallvec", - "thread_local", - "tracing", - "tracing-core", - "tracing-log", ] [[package]] @@ -4579,26 +4483,26 @@ checksum = "3c36781cc0e46a83726d9879608e4cf6c2505237e263a8eb8c24502989cfdb28" dependencies = [ "proc-macro2", "quote", - "syn 2.0.106", + "syn 2.0.108", ] [[package]] name = "typenum" -version = "1.18.0" +version = "1.19.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "1dccffe3ce07af9386bfd29e80c0ab1a8205a2fc34e4bcd40364df902cfa8f3f" +checksum = "562d481066bde0658276a35467c4af00bdc6ee726305698a55b86e61d7ad82bb" [[package]] name = "typewit" -version = "1.14.1" +version = "1.14.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "4c98488b93df24b7c794d6a58c4198d7a2abde676324beaca84f7fb5b39c0811" +checksum = "f8c1ae7cc0fdb8b842d65d127cb981574b0d2b249b74d1c7a2986863dc134f71" [[package]] name = "unicode-ident" -version = "1.0.18" +version = "1.0.20" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "5a5f39404a5da50712a4c1eecf25e90dd62b613502b7e925fd4e4d19b5c96512" +checksum = "462eeb75aeb73aea900253ce739c8e18a67423fadf006037cd3ff27e82748a06" [[package]] name = "unicode-segmentation" @@ -4608,9 +4512,9 @@ checksum = "f6ccf251212114b54433ec949fd6a7841275f9ada20dddd2f29e9ceea4501493" [[package]] name = "unicode-width" -version = "0.2.1" +version = "0.2.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "4a1a07cc7db3810833284e8d372ccdc6da29741639ecc70c9ec107df0fa6154c" +checksum = "b4ac048d71ede7ee76d585517add45da530660ef4390e49b098733c6e897f254" [[package]] name = "unindent" @@ -4648,18 +4552,12 @@ version = "1.18.1" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "2f87b8aa10b915a06587d0dec516c282ff295b475d94abf425d62b57710070a2" dependencies = [ - "getrandom 0.3.3", + "getrandom 0.3.4", "js-sys", "serde", "wasm-bindgen", ] -[[package]] -name = "valuable" -version = "0.1.1" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "ba73ea9cf16a25df0c8caa16c51acb937d5712a8429db78a3ee29d5dcacd3a65" - [[package]] name = "version_check" version = "0.9.5" @@ -4692,19 +4590,19 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "ccf3ec651a847eb01de73ccad15eb7d99f80485de043efb2f370cd654f4ea44b" [[package]] -name = "wasi" -version = "0.14.4+wasi-0.2.4" +name = "wasip2" +version = "1.0.1+wasi-0.2.4" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "88a5f4a424faf49c3c2c344f166f0662341d470ea185e939657aaff130f0ec4a" +checksum = "0562428422c63773dad2c345a1882263bbf4d65cf3f42e90921f787ef5ad58e7" dependencies = [ "wit-bindgen", ] [[package]] name = "wasm-bindgen" -version = "0.2.101" +version = "0.2.105" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "7e14915cadd45b529bb8d1f343c4ed0ac1de926144b746e2710f9cd05df6603b" +checksum = "da95793dfc411fbbd93f5be7715b0578ec61fe87cb1a42b12eb625caa5c5ea60" dependencies = [ "cfg-if", "once_cell", @@ -4713,25 +4611,11 @@ dependencies = [ "wasm-bindgen-shared", ] -[[package]] -name = "wasm-bindgen-backend" -version = "0.2.101" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "e28d1ba982ca7923fd01448d5c30c6864d0a14109560296a162f80f305fb93bb" -dependencies = [ - "bumpalo", - "log", - "proc-macro2", - "quote", - "syn 2.0.106", - "wasm-bindgen-shared", -] - [[package]] name = "wasm-bindgen-futures" -version = "0.4.51" +version = "0.4.55" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "0ca85039a9b469b38336411d6d6ced91f3fc87109a2a27b0c197663f5144dffe" +checksum = "551f88106c6d5e7ccc7cd9a16f312dd3b5d36ea8b4954304657d5dfba115d4a0" dependencies = [ "cfg-if", "js-sys", @@ -4742,9 +4626,9 @@ dependencies = [ [[package]] name = "wasm-bindgen-macro" -version = "0.2.101" +version = "0.2.105" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "7c3d463ae3eff775b0c45df9da45d68837702ac35af998361e2c84e7c5ec1b0d" +checksum = "04264334509e04a7bf8690f2384ef5265f05143a4bff3889ab7a3269adab59c2" dependencies = [ "quote", "wasm-bindgen-macro-support", @@ -4752,22 +4636,22 @@ dependencies = [ [[package]] name = "wasm-bindgen-macro-support" -version = "0.2.101" +version = "0.2.105" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "7bb4ce89b08211f923caf51d527662b75bdc9c9c7aab40f86dcb9fb85ac552aa" +checksum = "420bc339d9f322e562942d52e115d57e950d12d88983a14c79b86859ee6c7ebc" dependencies = [ + "bumpalo", "proc-macro2", "quote", - "syn 2.0.106", - "wasm-bindgen-backend", + "syn 2.0.108", "wasm-bindgen-shared", ] [[package]] name = "wasm-bindgen-shared" -version = "0.2.101" +version = "0.2.105" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "f143854a3b13752c6950862c906306adb27c7e839f7414cec8fea35beab624c1" +checksum = "76f218a38c84bcb33c25ec7059b07847d465ce0e0a76b995e134a45adcb6af76" dependencies = [ "unicode-ident", ] @@ -4787,9 +4671,9 @@ dependencies = [ [[package]] name = "web-sys" -version = "0.3.78" +version = "0.3.82" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "77e4b637749ff0d92b8fad63aa1f7cff3cbe125fd49c175cd6345e7272638b12" +checksum = "3a1f95c0d03a47f4ae1f7a64643a6bb97465d9b740f0fa8f90ea33915c99a9a1" dependencies = [ "js-sys", "wasm-bindgen", @@ -4807,9 +4691,9 @@ dependencies = [ [[package]] name = "webpki-roots" -version = "1.0.2" +version = "1.0.3" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "7e8983c3ab33d6fb807cfcdad2491c4ea8cbc8ed839181c7dfd9c67c83e261b2" +checksum = "32b130c0d2d49f8b6889abc456e795e82525204f27c42cf767cf0d7734e089b8" dependencies = [ "rustls-pki-types", ] @@ -4836,7 +4720,7 @@ version = "0.1.11" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "c2a7b1c03c876122aa43f3020e6c3c3ee5c05081c9a00739faf7503aeba10d22" dependencies = [ - "windows-sys 0.61.0", + "windows-sys 0.61.2", ] [[package]] @@ -4845,112 +4729,63 @@ version = "0.4.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "712e227841d057c1ee1cd2fb22fa7e5a5461ae8e48fa2ca79ec42cfc1931183f" -[[package]] -name = "windows" -version = "0.61.3" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "9babd3a767a4c1aef6900409f85f5d53ce2544ccdfaa86dad48c91782c6d6893" -dependencies = [ - "windows-collections", - "windows-core", - "windows-future", - "windows-link 0.1.3", - "windows-numerics", -] - -[[package]] -name = "windows-collections" -version = "0.2.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "3beeceb5e5cfd9eb1d76b381630e82c4241ccd0d27f1a39ed41b2760b255c5e8" -dependencies = [ - "windows-core", -] - [[package]] name = "windows-core" -version = "0.61.2" +version = "0.62.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c0fdd3ddb90610c7638aa2b3a3ab2904fb9e5cdbecc643ddb3647212781c4ae3" +checksum = "b8e83a14d34d0623b51dce9581199302a221863196a1dde71a7663a4c2be9deb" dependencies = [ "windows-implement", "windows-interface", - "windows-link 0.1.3", + "windows-link", "windows-result", "windows-strings", ] -[[package]] -name = "windows-future" -version = "0.2.1" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "fc6a41e98427b19fe4b73c550f060b59fa592d7d686537eebf9385621bfbad8e" -dependencies = [ - "windows-core", - "windows-link 0.1.3", - "windows-threading", -] - [[package]] name = "windows-implement" -version = "0.60.0" +version = "0.60.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a47fddd13af08290e67f4acabf4b459f647552718f683a7b415d290ac744a836" +checksum = "053e2e040ab57b9dc951b72c264860db7eb3b0200ba345b4e4c3b14f67855ddf" dependencies = [ "proc-macro2", "quote", - "syn 2.0.106", + "syn 2.0.108", ] [[package]] name = "windows-interface" -version = "0.59.1" +version = "0.59.3" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "bd9211b69f8dcdfa817bfd14bf1c97c9188afa36f4750130fcdf3f400eca9fa8" +checksum = "3f316c4a2570ba26bbec722032c4099d8c8bc095efccdc15688708623367e358" dependencies = [ "proc-macro2", "quote", - "syn 2.0.106", + "syn 2.0.108", ] [[package]] name = "windows-link" -version = "0.1.3" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "5e6ad25900d524eaabdbbb96d20b4311e1e7ae1699af4fb28c17ae66c80d798a" - -[[package]] -name = "windows-link" -version = "0.2.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "45e46c0661abb7180e7b9c281db115305d49ca1709ab8242adf09666d2173c65" - -[[package]] -name = "windows-numerics" -version = "0.2.0" +version = "0.2.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "9150af68066c4c5c07ddc0ce30421554771e528bde427614c61038bc2c92c2b1" -dependencies = [ - "windows-core", - "windows-link 0.1.3", -] +checksum = "f0805222e57f7521d6a62e36fa9163bc891acd422f971defe97d64e70d0a4fe5" [[package]] name = "windows-result" -version = "0.3.4" +version = "0.4.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "56f42bd332cc6c8eac5af113fc0c1fd6a8fd2aa08a0119358686e5160d0586c6" +checksum = "7781fa89eaf60850ac3d2da7af8e5242a5ea78d1a11c49bf2910bb5a73853eb5" dependencies = [ - "windows-link 0.1.3", + "windows-link", ] [[package]] name = "windows-strings" -version = "0.4.2" +version = "0.5.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "56e6c93f3a0c3b36176cb1327a4958a0353d5d166c2a35cb268ace15e91d3b57" +checksum = "7837d08f69c77cf6b07689544538e017c1bfcf57e34b4c0ff58e6c2cd3b37091" dependencies = [ - "windows-link 0.1.3", + "windows-link", ] [[package]] @@ -4977,16 +4812,16 @@ version = "0.60.2" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "f2f500e4d28234f72040990ec9d39e3a6b950f9f22d3dba18416c35882612bcb" dependencies = [ - "windows-targets 0.53.3", + "windows-targets 0.53.5", ] [[package]] name = "windows-sys" -version = "0.61.0" +version = "0.61.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "e201184e40b2ede64bc2ea34968b28e33622acdbbf37104f0e4a33f7abe657aa" +checksum = "ae137229bcbd6cdf0f7b80a31df61766145077ddf49416a728b02cb3921ff3fc" dependencies = [ - "windows-link 0.2.0", + "windows-link", ] [[package]] @@ -5007,28 +4842,19 @@ dependencies = [ [[package]] name = "windows-targets" -version = "0.53.3" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d5fe6031c4041849d7c496a8ded650796e7b6ecc19df1a431c1a363342e5dc91" -dependencies = [ - "windows-link 0.1.3", - "windows_aarch64_gnullvm 0.53.0", - "windows_aarch64_msvc 0.53.0", - "windows_i686_gnu 0.53.0", - "windows_i686_gnullvm 0.53.0", - "windows_i686_msvc 0.53.0", - "windows_x86_64_gnu 0.53.0", - "windows_x86_64_gnullvm 0.53.0", - "windows_x86_64_msvc 0.53.0", -] - -[[package]] -name = "windows-threading" -version = "0.1.0" +version = "0.53.5" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "b66463ad2e0ea3bbf808b7f1d371311c80e115c0b71d60efc142cafbcfb057a6" +checksum = "4945f9f551b88e0d65f3db0bc25c33b8acea4d9e41163edf90dcd0b19f9069f3" dependencies = [ - "windows-link 0.1.3", + "windows-link", + "windows_aarch64_gnullvm 0.53.1", + "windows_aarch64_msvc 0.53.1", + "windows_i686_gnu 0.53.1", + "windows_i686_gnullvm 0.53.1", + "windows_i686_msvc 0.53.1", + "windows_x86_64_gnu 0.53.1", + "windows_x86_64_gnullvm 0.53.1", + "windows_x86_64_msvc 0.53.1", ] [[package]] @@ -5039,9 +4865,9 @@ checksum = "32a4622180e7a0ec044bb555404c800bc9fd9ec262ec147edd5989ccd0c02cd3" [[package]] name = "windows_aarch64_gnullvm" -version = "0.53.0" +version = "0.53.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "86b8d5f90ddd19cb4a147a5fa63ca848db3df085e25fee3cc10b39b6eebae764" +checksum = "a9d8416fa8b42f5c947f8482c43e7d89e73a173cead56d044f6a56104a6d1b53" [[package]] name = "windows_aarch64_msvc" @@ -5051,9 +4877,9 @@ checksum = "09ec2a7bb152e2252b53fa7803150007879548bc709c039df7627cabbd05d469" [[package]] name = "windows_aarch64_msvc" -version = "0.53.0" +version = "0.53.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c7651a1f62a11b8cbd5e0d42526e55f2c99886c77e007179efff86c2b137e66c" +checksum = "b9d782e804c2f632e395708e99a94275910eb9100b2114651e04744e9b125006" [[package]] name = "windows_i686_gnu" @@ -5063,9 +4889,9 @@ checksum = "8e9b5ad5ab802e97eb8e295ac6720e509ee4c243f69d781394014ebfe8bbfa0b" [[package]] name = "windows_i686_gnu" -version = "0.53.0" +version = "0.53.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c1dc67659d35f387f5f6c479dc4e28f1d4bb90ddd1a5d3da2e5d97b42d6272c3" +checksum = "960e6da069d81e09becb0ca57a65220ddff016ff2d6af6a223cf372a506593a3" [[package]] name = "windows_i686_gnullvm" @@ -5075,9 +4901,9 @@ checksum = "0eee52d38c090b3caa76c563b86c3a4bd71ef1a819287c19d586d7334ae8ed66" [[package]] name = "windows_i686_gnullvm" -version = "0.53.0" +version = "0.53.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "9ce6ccbdedbf6d6354471319e781c0dfef054c81fbc7cf83f338a4296c0cae11" +checksum = "fa7359d10048f68ab8b09fa71c3daccfb0e9b559aed648a8f95469c27057180c" [[package]] name = "windows_i686_msvc" @@ -5087,9 +4913,9 @@ checksum = "240948bc05c5e7c6dabba28bf89d89ffce3e303022809e73deaefe4f6ec56c66" [[package]] name = "windows_i686_msvc" -version = "0.53.0" +version = "0.53.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "581fee95406bb13382d2f65cd4a908ca7b1e4c2f1917f143ba16efe98a589b5d" +checksum = "1e7ac75179f18232fe9c285163565a57ef8d3c89254a30685b57d83a38d326c2" [[package]] name = "windows_x86_64_gnu" @@ -5099,9 +4925,9 @@ checksum = "147a5c80aabfbf0c7d901cb5895d1de30ef2907eb21fbbab29ca94c5b08b1a78" [[package]] name = "windows_x86_64_gnu" -version = "0.53.0" +version = "0.53.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "2e55b5ac9ea33f2fc1716d1742db15574fd6fc8dadc51caab1c16a3d3b4190ba" +checksum = "9c3842cdd74a865a8066ab39c8a7a473c0778a3f29370b5fd6b4b9aa7df4a499" [[package]] name = "windows_x86_64_gnullvm" @@ -5111,9 +4937,9 @@ checksum = "24d5b23dc417412679681396f2b49f3de8c1473deb516bd34410872eff51ed0d" [[package]] name = "windows_x86_64_gnullvm" -version = "0.53.0" +version = "0.53.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "0a6e035dd0599267ce1ee132e51c27dd29437f63325753051e71dd9e42406c57" +checksum = "0ffa179e2d07eee8ad8f57493436566c7cc30ac536a3379fdf008f47f6bb7ae1" [[package]] name = "windows_x86_64_msvc" @@ -5123,9 +4949,9 @@ checksum = "589f6da84c646204747d1270a2a5661ea66ed1cced2631d546fdfb155959f9ec" [[package]] name = "windows_x86_64_msvc" -version = "0.53.0" +version = "0.53.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "271414315aff87387382ec3d271b52d7ae78726f5d44ac98b4f4030c91880486" +checksum = "d6bbff5f0aada427a1e5a6da5f1f98158182f26556f345ac9e04d36d0ebed650" [[package]] name = "winnow" @@ -5138,15 +4964,15 @@ dependencies = [ [[package]] name = "wit-bindgen" -version = "0.45.1" +version = "0.46.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "5c573471f125075647d03df72e026074b7203790d41351cd6edc96f46bcccd36" +checksum = "f17a85883d4e6d00e8a97c586de764dabcc06133f7f1d55dce5cdc070ad7fe59" [[package]] name = "writeable" -version = "0.6.1" +version = "0.6.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "ea2f10b9bb0928dfb1b42b65e1f9e36f7f54dbdf08457afefb38afcdec4fa2bb" +checksum = "9edde0db4769d2dc68579893f2306b26c6ecfbe0ef499b013d731b7b9247e0b9" [[package]] name = "wyz" @@ -5168,11 +4994,10 @@ dependencies = [ [[package]] name = "yoke" -version = "0.8.0" +version = "0.8.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "5f41bb01b8226ef4bfd589436a297c53d118f65921786300e427be8d487695cc" +checksum = "72d6e5c6afb84d73944e5cedb052c4680d5657337201555f9f2a16b7406d4954" dependencies = [ - "serde", "stable_deref_trait", "yoke-derive", "zerofrom", @@ -5180,13 +5005,13 @@ dependencies = [ [[package]] name = "yoke-derive" -version = "0.8.0" +version = "0.8.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "38da3c9736e16c5d3c8c597a9aaa5d1fa565d0532ae05e27c24aa62fb32c0ab6" +checksum = "b659052874eb698efe5b9e8cf382204678a0086ebf46982b79d6ca3182927e5d" dependencies = [ "proc-macro2", "quote", - "syn 2.0.106", + "syn 2.0.108", "synstructure", ] @@ -5207,7 +5032,7 @@ checksum = "88d2b8d9c68ad2b9e4340d7832716a4d21a22a1154777ad56ea55c51a9cf3831" dependencies = [ "proc-macro2", "quote", - "syn 2.0.106", + "syn 2.0.108", ] [[package]] @@ -5227,21 +5052,21 @@ checksum = "d71e5d6e06ab090c67b5e44993ec16b72dcbaabc526db883a360057678b48502" dependencies = [ "proc-macro2", "quote", - "syn 2.0.106", + "syn 2.0.108", "synstructure", ] [[package]] name = "zeroize" -version = "1.8.1" +version = "1.8.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "ced3678a2879b30306d323f4542626697a464a97c0a07c9aebf7ebca65cd4dde" +checksum = "b97154e67e32c85465826e8bcc1c59429aaaf107c1e4a9e53c8d8ccd5eff88d0" [[package]] name = "zerotrie" -version = "0.2.2" +version = "0.2.3" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "36f0bbd478583f79edad978b407914f61b2972f5af6fa089686016be8f9af595" +checksum = "2a59c17a5562d507e4b54960e8569ebee33bee890c70aa3fe7b97e85a9fd7851" dependencies = [ "displaydoc", "yoke", @@ -5250,9 +5075,9 @@ dependencies = [ [[package]] name = "zerovec" -version = "0.11.4" +version = "0.11.5" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "e7aa2bd55086f1ab526693ecbe444205da57e25f4489879da80635a46d90e73b" +checksum = "6c28719294829477f525be0186d13efa9a3c602f7ec202ca9e353d310fb9a002" dependencies = [ "yoke", "zerofrom", @@ -5261,13 +5086,13 @@ dependencies = [ [[package]] name = "zerovec-derive" -version = "0.11.1" +version = "0.11.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "5b96237efa0c878c64bd89c436f661be4e46b2f3eff1ebb976f7ef2321d2f58f" +checksum = "eadce39539ca5cb3985590102671f2567e659fca9666581ad3411d59207951f3" dependencies = [ "proc-macro2", "quote", - "syn 2.0.106", + "syn 2.0.108", ] [[package]] diff --git a/bindings/python/Cargo.toml b/bindings/python/Cargo.toml index f7f33d0729..b8c1efe694 100644 --- a/bindings/python/Cargo.toml +++ b/bindings/python/Cargo.toml @@ -31,9 +31,9 @@ license = "Apache-2.0" crate-type = ["cdylib"] [dependencies] -arrow = { version = "55.2", features = ["pyarrow", "chrono-tz"] } +arrow = { version = "56", features = ["pyarrow", "chrono-tz"] } iceberg = { path = "../../crates/iceberg" } -pyo3 = { version = "0.24.1", features = ["extension-module", "abi3-py39"] } +pyo3 = { version = "0.25", features = ["extension-module", "abi3-py39"] } iceberg-datafusion = { path = "../../crates/integrations/datafusion" } -datafusion-ffi = { version = "49" } +datafusion-ffi = { version = "50" } tokio = { version = "1.46.1", default-features = false } diff --git a/bindings/python/pyproject.toml b/bindings/python/pyproject.toml index a9c3c290c7..516424e0b0 100644 --- a/bindings/python/pyproject.toml +++ b/bindings/python/pyproject.toml @@ -49,7 +49,7 @@ include = [ ignore = ["F403", "F405"] [tool.hatch.envs.dev] -dependencies = ["maturin>=1.0,<2.0", "pytest>=8.3.2", "datafusion==49.*", "pyiceberg[sql-sqlite,pyarrow]>=0.10.0", "fastavro>=1.11.1"] +dependencies = ["maturin>=1.0,<2.0", "pytest>=8.3.2", "datafusion==50.*", "pyiceberg[sql-sqlite,pyarrow]>=0.10.0", "fastavro>=1.11.1"] [tool.hatch.envs.dev.scripts] build = "maturin build --out dist --sdist" diff --git a/crates/catalog/sql/Cargo.toml b/crates/catalog/sql/Cargo.toml index 262422c284..eff605f466 100644 --- a/crates/catalog/sql/Cargo.toml +++ b/crates/catalog/sql/Cargo.toml @@ -36,7 +36,7 @@ strum = { workspace = true } [dev-dependencies] itertools = { workspace = true } -regex = "1.10.5" +regex = { workspace = true } sqlx = { version = "0.8.1", features = [ "tls-rustls", "runtime-tokio", diff --git a/crates/iceberg/src/arrow/reader.rs b/crates/iceberg/src/arrow/reader.rs index 720b9363b9..c6f5af2f2a 100644 --- a/crates/iceberg/src/arrow/reader.rs +++ b/crates/iceberg/src/arrow/reader.rs @@ -39,7 +39,9 @@ use parquet::arrow::arrow_reader::{ }; use parquet::arrow::async_reader::AsyncFileReader; use parquet::arrow::{PARQUET_FIELD_ID_META_KEY, ParquetRecordBatchStreamBuilder, ProjectionMask}; -use parquet::file::metadata::{ParquetMetaData, ParquetMetaDataReader, RowGroupMetaData}; +use parquet::file::metadata::{ + PageIndexPolicy, ParquetMetaData, ParquetMetaDataReader, RowGroupMetaData, +}; use parquet::schema::types::{SchemaDescriptor, Type as ParquetType}; use crate::arrow::caching_delete_file_loader::CachingDeleteFileLoader; @@ -1434,9 +1436,10 @@ impl AsyncFileReader for ArrowFileReader { async move { let reader = ParquetMetaDataReader::new() .with_prefetch_hint(self.metadata_size_hint) - .with_column_indexes(self.preload_column_index) - .with_page_indexes(self.preload_page_index) - .with_offset_indexes(self.preload_offset_index); + // Set the page policy first because it updates both column and offset policies. + .with_page_index_policy(PageIndexPolicy::from(self.preload_page_index)) + .with_column_index_policy(PageIndexPolicy::from(self.preload_column_index)) + .with_offset_index_policy(PageIndexPolicy::from(self.preload_offset_index)); let size = self.meta.size; let meta = reader.load_and_finish(self, size).await?; From a76e73863b84845dca566ae3e73defa40592cb15 Mon Sep 17 00:00:00 2001 From: Xuanwo Date: Wed, 29 Oct 2025 09:25:48 +0800 Subject: [PATCH 44/47] ci: Migrate to uv for python (#1796) ## Which issue does this PR close? - Closes https://github.com/apache/iceberg-rust/issues/1794 ## What changes are included in this PR? Migrate to uv ## Are these changes tested? --------- Signed-off-by: Xuanwo --- .github/workflows/bindings_python_ci.yml | 21 +- .licenserc.yaml | 19 +- bindings/python/README.md | 14 +- bindings/python/pyproject.toml | 18 +- bindings/python/uv.lock | 767 +++++++++++++++++++++++ 5 files changed, 811 insertions(+), 28 deletions(-) create mode 100644 bindings/python/uv.lock diff --git a/.github/workflows/bindings_python_ci.yml b/.github/workflows/bindings_python_ci.yml index ae3a6d1a54..9ea2ac0b90 100644 --- a/.github/workflows/bindings_python_ci.yml +++ b/.github/workflows/bindings_python_ci.yml @@ -80,17 +80,28 @@ jobs: - uses: actions/checkout@v5 - uses: actions/setup-python@v6 with: - python-version: 3.9 + python-version: 3.12 - uses: PyO3/maturin-action@v1 with: working-directory: "bindings/python" command: build args: --out dist --sdist + - name: Install uv + shell: bash + run: | + python -m pip install uv==0.9.3 + - name: Sync dependencies + working-directory: "bindings/python" + shell: bash + run: | + uv sync --group dev --no-install-project + - name: Install built wheel + working-directory: "bindings/python" + shell: bash + run: | + uv pip install --reinstall dist/pyiceberg_core-*.whl - name: Run tests working-directory: "bindings/python" shell: bash run: | - set -e - pip install hatch==1.14.2 - hatch run dev:pip install dist/pyiceberg_core-*.whl --force-reinstall - hatch run dev:test + uv run --group dev --no-sync pytest diff --git a/.licenserc.yaml b/.licenserc.yaml index 01625e32cb..da87374c3b 100644 --- a/.licenserc.yaml +++ b/.licenserc.yaml @@ -21,16 +21,17 @@ header: copyright-owner: Apache Software Foundation paths-ignore: - - 'LICENSE' - - 'NOTICE' - - '.gitattributes' - - '**/*.json' + - "LICENSE" + - "NOTICE" + - ".gitattributes" + - "**/*.json" # Generated content by mdbook - - 'website/book' + - "website/book" # Generated content by scripts - - '**/DEPENDENCIES.*.tsv' + - "**/DEPENDENCIES.*.tsv" # Release distributions - - 'dist/*' - - 'Cargo.lock' - - '.github/PULL_REQUEST_TEMPLATE.md' + - "dist/*" + - "Cargo.lock" + - "bindings/python/uv.lock" + - ".github/PULL_REQUEST_TEMPLATE.md" comment: on-failure diff --git a/bindings/python/README.md b/bindings/python/README.md index 2dddc71ac1..a91599d9a4 100644 --- a/bindings/python/README.md +++ b/bindings/python/README.md @@ -23,22 +23,26 @@ This project is used to build an Iceberg-rust powered core for [PyIceberg](https ## Setup -Install Hatch: +Install [uv](https://docs.astral.sh/uv/getting-started/installation/): ```shell -pip install hatch==1.12.0 +pip install uv==0.9.3 ``` -Hatch uses [uv](https://docs.astral.sh/uv/) as a backend by default, so [make sure that it is installed](https://docs.astral.sh/uv/getting-started/installation/) as well. +Set up the development environment: + +```shell +uv sync --group dev +``` ## Build ```shell -hatch run dev:develop +uv run --group dev maturin develop ``` ## Test ```shell -hatch run dev:test +uv run --group dev pytest ``` diff --git a/bindings/python/pyproject.toml b/bindings/python/pyproject.toml index 516424e0b0..c79c9b520c 100644 --- a/bindings/python/pyproject.toml +++ b/bindings/python/pyproject.toml @@ -25,14 +25,13 @@ classifiers = [ "Intended Audience :: Developers", "License :: OSI Approved :: Apache Software License", "Operating System :: OS Independent", - "Programming Language :: Python :: 3.9", "Programming Language :: Python :: 3.10", "Programming Language :: Python :: 3.11", "Programming Language :: Python :: 3.12", ] name = "pyiceberg-core" readme = "project-description.md" -requires-python = "~=3.9" +requires-python = ">=3.10,<3.13" dynamic = ["version"] license = { file = "LICENSE" } @@ -48,10 +47,11 @@ include = [ [tool.ruff.lint] ignore = ["F403", "F405"] -[tool.hatch.envs.dev] -dependencies = ["maturin>=1.0,<2.0", "pytest>=8.3.2", "datafusion==50.*", "pyiceberg[sql-sqlite,pyarrow]>=0.10.0", "fastavro>=1.11.1"] - -[tool.hatch.envs.dev.scripts] -build = "maturin build --out dist --sdist" -develop = "maturin develop" -test = "pytest" +[dependency-groups] +dev = [ + "maturin>=1.0,<2.0", + "pytest>=8.3.2", + "datafusion==50.*", + "pyiceberg[sql-sqlite,pyarrow] @ git+https://github.com/apache/iceberg-python.git@d3eb149fe99b14e714a06e727b69cd617c6c052d", + "fastavro>=1.11.1", +] diff --git a/bindings/python/uv.lock b/bindings/python/uv.lock new file mode 100644 index 0000000000..614478d338 --- /dev/null +++ b/bindings/python/uv.lock @@ -0,0 +1,767 @@ +version = 1 +revision = 3 +requires-python = ">=3.10, <3.13" + +[[package]] +name = "annotated-types" +version = "0.7.0" +source = { registry = "https://pypi.org/simple" } +sdist = { url = "https://files.pythonhosted.org/packages/ee/67/531ea369ba64dcff5ec9c3402f9f51bf748cec26dde048a2f973a4eea7f5/annotated_types-0.7.0.tar.gz", hash = "sha256:aff07c09a53a08bc8cfccb9c85b05f1aa9a2a6f23728d790723543408344ce89", size = 16081, upload-time = "2024-05-20T21:33:25.928Z" } +wheels = [ + { url = "https://files.pythonhosted.org/packages/78/b6/6307fbef88d9b5ee7421e68d78a9f162e0da4900bc5f5793f6d3d0e34fb8/annotated_types-0.7.0-py3-none-any.whl", hash = "sha256:1f02e8b43a8fbbc3f3e0d4f0f4bfc8131bcb4eebe8849b8e5c773f3a1c582a53", size = 13643, upload-time = "2024-05-20T21:33:24.1Z" }, +] + +[[package]] +name = "cachetools" +version = "6.2.1" +source = { registry = "https://pypi.org/simple" } +sdist = { url = "https://files.pythonhosted.org/packages/cc/7e/b975b5814bd36faf009faebe22c1072a1fa1168db34d285ef0ba071ad78c/cachetools-6.2.1.tar.gz", hash = "sha256:3f391e4bd8f8bf0931169baf7456cc822705f4e2a31f840d218f445b9a854201", size = 31325, upload-time = "2025-10-12T14:55:30.139Z" } +wheels = [ + { url = "https://files.pythonhosted.org/packages/96/c5/1e741d26306c42e2bf6ab740b2202872727e0f606033c9dd713f8b93f5a8/cachetools-6.2.1-py3-none-any.whl", hash = "sha256:09868944b6dde876dfd44e1d47e18484541eaf12f26f29b7af91b26cc892d701", size = 11280, upload-time = "2025-10-12T14:55:28.382Z" }, +] + +[[package]] +name = "certifi" +version = "2025.10.5" +source = { registry = "https://pypi.org/simple" } +sdist = { url = "https://files.pythonhosted.org/packages/4c/5b/b6ce21586237c77ce67d01dc5507039d444b630dd76611bbca2d8e5dcd91/certifi-2025.10.5.tar.gz", hash = "sha256:47c09d31ccf2acf0be3f701ea53595ee7e0b8fa08801c6624be771df09ae7b43", size = 164519, upload-time = "2025-10-05T04:12:15.808Z" } +wheels = [ + { url = "https://files.pythonhosted.org/packages/e4/37/af0d2ef3967ac0d6113837b44a4f0bfe1328c2b9763bd5b1744520e5cfed/certifi-2025.10.5-py3-none-any.whl", hash = "sha256:0f212c2744a9bb6de0c56639a6f68afe01ecd92d91f14ae897c4fe7bbeeef0de", size = 163286, upload-time = "2025-10-05T04:12:14.03Z" }, +] + +[[package]] +name = "charset-normalizer" +version = "3.4.4" +source = { registry = "https://pypi.org/simple" } +sdist = { url = "https://files.pythonhosted.org/packages/13/69/33ddede1939fdd074bce5434295f38fae7136463422fe4fd3e0e89b98062/charset_normalizer-3.4.4.tar.gz", hash = "sha256:94537985111c35f28720e43603b8e7b43a6ecfb2ce1d3058bbe955b73404e21a", size = 129418, upload-time = "2025-10-14T04:42:32.879Z" } +wheels = [ + { url = "https://files.pythonhosted.org/packages/1f/b8/6d51fc1d52cbd52cd4ccedd5b5b2f0f6a11bbf6765c782298b0f3e808541/charset_normalizer-3.4.4-cp310-cp310-macosx_10_9_universal2.whl", hash = "sha256:e824f1492727fa856dd6eda4f7cee25f8518a12f3c4a56a74e8095695089cf6d", size = 209709, upload-time = "2025-10-14T04:40:11.385Z" }, + { url = "https://files.pythonhosted.org/packages/5c/af/1f9d7f7faafe2ddfb6f72a2e07a548a629c61ad510fe60f9630309908fef/charset_normalizer-3.4.4-cp310-cp310-manylinux2014_aarch64.manylinux_2_17_aarch64.manylinux_2_28_aarch64.whl", hash = "sha256:4bd5d4137d500351a30687c2d3971758aac9a19208fc110ccb9d7188fbe709e8", size = 148814, upload-time = "2025-10-14T04:40:13.135Z" }, + { url = "https://files.pythonhosted.org/packages/79/3d/f2e3ac2bbc056ca0c204298ea4e3d9db9b4afe437812638759db2c976b5f/charset_normalizer-3.4.4-cp310-cp310-manylinux2014_armv7l.manylinux_2_17_armv7l.manylinux_2_31_armv7l.whl", hash = "sha256:027f6de494925c0ab2a55eab46ae5129951638a49a34d87f4c3eda90f696b4ad", size = 144467, upload-time = "2025-10-14T04:40:14.728Z" }, + { url = "https://files.pythonhosted.org/packages/ec/85/1bf997003815e60d57de7bd972c57dc6950446a3e4ccac43bc3070721856/charset_normalizer-3.4.4-cp310-cp310-manylinux2014_ppc64le.manylinux_2_17_ppc64le.manylinux_2_28_ppc64le.whl", hash = "sha256:f820802628d2694cb7e56db99213f930856014862f3fd943d290ea8438d07ca8", size = 162280, upload-time = "2025-10-14T04:40:16.14Z" }, + { url = "https://files.pythonhosted.org/packages/3e/8e/6aa1952f56b192f54921c436b87f2aaf7c7a7c3d0d1a765547d64fd83c13/charset_normalizer-3.4.4-cp310-cp310-manylinux2014_s390x.manylinux_2_17_s390x.manylinux_2_28_s390x.whl", hash = "sha256:798d75d81754988d2565bff1b97ba5a44411867c0cf32b77a7e8f8d84796b10d", size = 159454, upload-time = "2025-10-14T04:40:17.567Z" }, + { url = "https://files.pythonhosted.org/packages/36/3b/60cbd1f8e93aa25d1c669c649b7a655b0b5fb4c571858910ea9332678558/charset_normalizer-3.4.4-cp310-cp310-manylinux2014_x86_64.manylinux_2_17_x86_64.manylinux_2_28_x86_64.whl", hash = "sha256:9d1bb833febdff5c8927f922386db610b49db6e0d4f4ee29601d71e7c2694313", size = 153609, upload-time = "2025-10-14T04:40:19.08Z" }, + { url = "https://files.pythonhosted.org/packages/64/91/6a13396948b8fd3c4b4fd5bc74d045f5637d78c9675585e8e9fbe5636554/charset_normalizer-3.4.4-cp310-cp310-manylinux_2_31_riscv64.manylinux_2_39_riscv64.whl", hash = "sha256:9cd98cdc06614a2f768d2b7286d66805f94c48cde050acdbbb7db2600ab3197e", size = 151849, upload-time = "2025-10-14T04:40:20.607Z" }, + { url = "https://files.pythonhosted.org/packages/b7/7a/59482e28b9981d105691e968c544cc0df3b7d6133152fb3dcdc8f135da7a/charset_normalizer-3.4.4-cp310-cp310-musllinux_1_2_aarch64.whl", hash = "sha256:077fbb858e903c73f6c9db43374fd213b0b6a778106bc7032446a8e8b5b38b93", size = 151586, upload-time = "2025-10-14T04:40:21.719Z" }, + { url = "https://files.pythonhosted.org/packages/92/59/f64ef6a1c4bdd2baf892b04cd78792ed8684fbc48d4c2afe467d96b4df57/charset_normalizer-3.4.4-cp310-cp310-musllinux_1_2_armv7l.whl", hash = "sha256:244bfb999c71b35de57821b8ea746b24e863398194a4014e4c76adc2bbdfeff0", size = 145290, upload-time = "2025-10-14T04:40:23.069Z" }, + { url = "https://files.pythonhosted.org/packages/6b/63/3bf9f279ddfa641ffa1962b0db6a57a9c294361cc2f5fcac997049a00e9c/charset_normalizer-3.4.4-cp310-cp310-musllinux_1_2_ppc64le.whl", hash = "sha256:64b55f9dce520635f018f907ff1b0df1fdc31f2795a922fb49dd14fbcdf48c84", size = 163663, upload-time = "2025-10-14T04:40:24.17Z" }, + { url = "https://files.pythonhosted.org/packages/ed/09/c9e38fc8fa9e0849b172b581fd9803bdf6e694041127933934184e19f8c3/charset_normalizer-3.4.4-cp310-cp310-musllinux_1_2_riscv64.whl", hash = "sha256:faa3a41b2b66b6e50f84ae4a68c64fcd0c44355741c6374813a800cd6695db9e", size = 151964, upload-time = "2025-10-14T04:40:25.368Z" }, + { url = "https://files.pythonhosted.org/packages/d2/d1/d28b747e512d0da79d8b6a1ac18b7ab2ecfd81b2944c4c710e166d8dd09c/charset_normalizer-3.4.4-cp310-cp310-musllinux_1_2_s390x.whl", hash = "sha256:6515f3182dbe4ea06ced2d9e8666d97b46ef4c75e326b79bb624110f122551db", size = 161064, upload-time = "2025-10-14T04:40:26.806Z" }, + { url = "https://files.pythonhosted.org/packages/bb/9a/31d62b611d901c3b9e5500c36aab0ff5eb442043fb3a1c254200d3d397d9/charset_normalizer-3.4.4-cp310-cp310-musllinux_1_2_x86_64.whl", hash = "sha256:cc00f04ed596e9dc0da42ed17ac5e596c6ccba999ba6bd92b0e0aef2f170f2d6", size = 155015, upload-time = "2025-10-14T04:40:28.284Z" }, + { url = "https://files.pythonhosted.org/packages/1f/f3/107e008fa2bff0c8b9319584174418e5e5285fef32f79d8ee6a430d0039c/charset_normalizer-3.4.4-cp310-cp310-win32.whl", hash = "sha256:f34be2938726fc13801220747472850852fe6b1ea75869a048d6f896838c896f", size = 99792, upload-time = "2025-10-14T04:40:29.613Z" }, + { url = "https://files.pythonhosted.org/packages/eb/66/e396e8a408843337d7315bab30dbf106c38966f1819f123257f5520f8a96/charset_normalizer-3.4.4-cp310-cp310-win_amd64.whl", hash = "sha256:a61900df84c667873b292c3de315a786dd8dac506704dea57bc957bd31e22c7d", size = 107198, upload-time = "2025-10-14T04:40:30.644Z" }, + { url = "https://files.pythonhosted.org/packages/b5/58/01b4f815bf0312704c267f2ccb6e5d42bcc7752340cd487bc9f8c3710597/charset_normalizer-3.4.4-cp310-cp310-win_arm64.whl", hash = "sha256:cead0978fc57397645f12578bfd2d5ea9138ea0fac82b2f63f7f7c6877986a69", size = 100262, upload-time = "2025-10-14T04:40:32.108Z" }, + { url = "https://files.pythonhosted.org/packages/ed/27/c6491ff4954e58a10f69ad90aca8a1b6fe9c5d3c6f380907af3c37435b59/charset_normalizer-3.4.4-cp311-cp311-macosx_10_9_universal2.whl", hash = "sha256:6e1fcf0720908f200cd21aa4e6750a48ff6ce4afe7ff5a79a90d5ed8a08296f8", size = 206988, upload-time = "2025-10-14T04:40:33.79Z" }, + { url = "https://files.pythonhosted.org/packages/94/59/2e87300fe67ab820b5428580a53cad894272dbb97f38a7a814a2a1ac1011/charset_normalizer-3.4.4-cp311-cp311-manylinux2014_aarch64.manylinux_2_17_aarch64.manylinux_2_28_aarch64.whl", hash = "sha256:5f819d5fe9234f9f82d75bdfa9aef3a3d72c4d24a6e57aeaebba32a704553aa0", size = 147324, upload-time = "2025-10-14T04:40:34.961Z" }, + { url = "https://files.pythonhosted.org/packages/07/fb/0cf61dc84b2b088391830f6274cb57c82e4da8bbc2efeac8c025edb88772/charset_normalizer-3.4.4-cp311-cp311-manylinux2014_armv7l.manylinux_2_17_armv7l.manylinux_2_31_armv7l.whl", hash = "sha256:a59cb51917aa591b1c4e6a43c132f0cdc3c76dbad6155df4e28ee626cc77a0a3", size = 142742, upload-time = "2025-10-14T04:40:36.105Z" }, + { url = "https://files.pythonhosted.org/packages/62/8b/171935adf2312cd745d290ed93cf16cf0dfe320863ab7cbeeae1dcd6535f/charset_normalizer-3.4.4-cp311-cp311-manylinux2014_ppc64le.manylinux_2_17_ppc64le.manylinux_2_28_ppc64le.whl", hash = "sha256:8ef3c867360f88ac904fd3f5e1f902f13307af9052646963ee08ff4f131adafc", size = 160863, upload-time = "2025-10-14T04:40:37.188Z" }, + { url = "https://files.pythonhosted.org/packages/09/73/ad875b192bda14f2173bfc1bc9a55e009808484a4b256748d931b6948442/charset_normalizer-3.4.4-cp311-cp311-manylinux2014_s390x.manylinux_2_17_s390x.manylinux_2_28_s390x.whl", hash = "sha256:d9e45d7faa48ee908174d8fe84854479ef838fc6a705c9315372eacbc2f02897", size = 157837, upload-time = "2025-10-14T04:40:38.435Z" }, + { url = "https://files.pythonhosted.org/packages/6d/fc/de9cce525b2c5b94b47c70a4b4fb19f871b24995c728e957ee68ab1671ea/charset_normalizer-3.4.4-cp311-cp311-manylinux2014_x86_64.manylinux_2_17_x86_64.manylinux_2_28_x86_64.whl", hash = "sha256:840c25fb618a231545cbab0564a799f101b63b9901f2569faecd6b222ac72381", size = 151550, upload-time = "2025-10-14T04:40:40.053Z" }, + { url = "https://files.pythonhosted.org/packages/55/c2/43edd615fdfba8c6f2dfbd459b25a6b3b551f24ea21981e23fb768503ce1/charset_normalizer-3.4.4-cp311-cp311-manylinux_2_31_riscv64.manylinux_2_39_riscv64.whl", hash = "sha256:ca5862d5b3928c4940729dacc329aa9102900382fea192fc5e52eb69d6093815", size = 149162, upload-time = "2025-10-14T04:40:41.163Z" }, + { url = "https://files.pythonhosted.org/packages/03/86/bde4ad8b4d0e9429a4e82c1e8f5c659993a9a863ad62c7df05cf7b678d75/charset_normalizer-3.4.4-cp311-cp311-musllinux_1_2_aarch64.whl", hash = "sha256:d9c7f57c3d666a53421049053eaacdd14bbd0a528e2186fcb2e672effd053bb0", size = 150019, upload-time = "2025-10-14T04:40:42.276Z" }, + { url = "https://files.pythonhosted.org/packages/1f/86/a151eb2af293a7e7bac3a739b81072585ce36ccfb4493039f49f1d3cae8c/charset_normalizer-3.4.4-cp311-cp311-musllinux_1_2_armv7l.whl", hash = "sha256:277e970e750505ed74c832b4bf75dac7476262ee2a013f5574dd49075879e161", size = 143310, upload-time = "2025-10-14T04:40:43.439Z" }, + { url = "https://files.pythonhosted.org/packages/b5/fe/43dae6144a7e07b87478fdfc4dbe9efd5defb0e7ec29f5f58a55aeef7bf7/charset_normalizer-3.4.4-cp311-cp311-musllinux_1_2_ppc64le.whl", hash = "sha256:31fd66405eaf47bb62e8cd575dc621c56c668f27d46a61d975a249930dd5e2a4", size = 162022, upload-time = "2025-10-14T04:40:44.547Z" }, + { url = "https://files.pythonhosted.org/packages/80/e6/7aab83774f5d2bca81f42ac58d04caf44f0cc2b65fc6db2b3b2e8a05f3b3/charset_normalizer-3.4.4-cp311-cp311-musllinux_1_2_riscv64.whl", hash = "sha256:0d3d8f15c07f86e9ff82319b3d9ef6f4bf907608f53fe9d92b28ea9ae3d1fd89", size = 149383, upload-time = "2025-10-14T04:40:46.018Z" }, + { url = "https://files.pythonhosted.org/packages/4f/e8/b289173b4edae05c0dde07f69f8db476a0b511eac556dfe0d6bda3c43384/charset_normalizer-3.4.4-cp311-cp311-musllinux_1_2_s390x.whl", hash = "sha256:9f7fcd74d410a36883701fafa2482a6af2ff5ba96b9a620e9e0721e28ead5569", size = 159098, upload-time = "2025-10-14T04:40:47.081Z" }, + { url = "https://files.pythonhosted.org/packages/d8/df/fe699727754cae3f8478493c7f45f777b17c3ef0600e28abfec8619eb49c/charset_normalizer-3.4.4-cp311-cp311-musllinux_1_2_x86_64.whl", hash = "sha256:ebf3e58c7ec8a8bed6d66a75d7fb37b55e5015b03ceae72a8e7c74495551e224", size = 152991, upload-time = "2025-10-14T04:40:48.246Z" }, + { url = "https://files.pythonhosted.org/packages/1a/86/584869fe4ddb6ffa3bd9f491b87a01568797fb9bd8933f557dba9771beaf/charset_normalizer-3.4.4-cp311-cp311-win32.whl", hash = "sha256:eecbc200c7fd5ddb9a7f16c7decb07b566c29fa2161a16cf67b8d068bd21690a", size = 99456, upload-time = "2025-10-14T04:40:49.376Z" }, + { url = "https://files.pythonhosted.org/packages/65/f6/62fdd5feb60530f50f7e38b4f6a1d5203f4d16ff4f9f0952962c044e919a/charset_normalizer-3.4.4-cp311-cp311-win_amd64.whl", hash = "sha256:5ae497466c7901d54b639cf42d5b8c1b6a4fead55215500d2f486d34db48d016", size = 106978, upload-time = "2025-10-14T04:40:50.844Z" }, + { url = "https://files.pythonhosted.org/packages/7a/9d/0710916e6c82948b3be62d9d398cb4fcf4e97b56d6a6aeccd66c4b2f2bd5/charset_normalizer-3.4.4-cp311-cp311-win_arm64.whl", hash = "sha256:65e2befcd84bc6f37095f5961e68a6f077bf44946771354a28ad434c2cce0ae1", size = 99969, upload-time = "2025-10-14T04:40:52.272Z" }, + { url = "https://files.pythonhosted.org/packages/f3/85/1637cd4af66fa687396e757dec650f28025f2a2f5a5531a3208dc0ec43f2/charset_normalizer-3.4.4-cp312-cp312-macosx_10_13_universal2.whl", hash = "sha256:0a98e6759f854bd25a58a73fa88833fba3b7c491169f86ce1180c948ab3fd394", size = 208425, upload-time = "2025-10-14T04:40:53.353Z" }, + { url = "https://files.pythonhosted.org/packages/9d/6a/04130023fef2a0d9c62d0bae2649b69f7b7d8d24ea5536feef50551029df/charset_normalizer-3.4.4-cp312-cp312-manylinux2014_aarch64.manylinux_2_17_aarch64.manylinux_2_28_aarch64.whl", hash = "sha256:b5b290ccc2a263e8d185130284f8501e3e36c5e02750fc6b6bdeb2e9e96f1e25", size = 148162, upload-time = "2025-10-14T04:40:54.558Z" }, + { url = "https://files.pythonhosted.org/packages/78/29/62328d79aa60da22c9e0b9a66539feae06ca0f5a4171ac4f7dc285b83688/charset_normalizer-3.4.4-cp312-cp312-manylinux2014_armv7l.manylinux_2_17_armv7l.manylinux_2_31_armv7l.whl", hash = "sha256:74bb723680f9f7a6234dcf67aea57e708ec1fbdf5699fb91dfd6f511b0a320ef", size = 144558, upload-time = "2025-10-14T04:40:55.677Z" }, + { url = "https://files.pythonhosted.org/packages/86/bb/b32194a4bf15b88403537c2e120b817c61cd4ecffa9b6876e941c3ee38fe/charset_normalizer-3.4.4-cp312-cp312-manylinux2014_ppc64le.manylinux_2_17_ppc64le.manylinux_2_28_ppc64le.whl", hash = "sha256:f1e34719c6ed0b92f418c7c780480b26b5d9c50349e9a9af7d76bf757530350d", size = 161497, upload-time = "2025-10-14T04:40:57.217Z" }, + { url = "https://files.pythonhosted.org/packages/19/89/a54c82b253d5b9b111dc74aca196ba5ccfcca8242d0fb64146d4d3183ff1/charset_normalizer-3.4.4-cp312-cp312-manylinux2014_s390x.manylinux_2_17_s390x.manylinux_2_28_s390x.whl", hash = "sha256:2437418e20515acec67d86e12bf70056a33abdacb5cb1655042f6538d6b085a8", size = 159240, upload-time = "2025-10-14T04:40:58.358Z" }, + { url = "https://files.pythonhosted.org/packages/c0/10/d20b513afe03acc89ec33948320a5544d31f21b05368436d580dec4e234d/charset_normalizer-3.4.4-cp312-cp312-manylinux2014_x86_64.manylinux_2_17_x86_64.manylinux_2_28_x86_64.whl", hash = "sha256:11d694519d7f29d6cd09f6ac70028dba10f92f6cdd059096db198c283794ac86", size = 153471, upload-time = "2025-10-14T04:40:59.468Z" }, + { url = "https://files.pythonhosted.org/packages/61/fa/fbf177b55bdd727010f9c0a3c49eefa1d10f960e5f09d1d887bf93c2e698/charset_normalizer-3.4.4-cp312-cp312-manylinux_2_31_riscv64.manylinux_2_39_riscv64.whl", hash = "sha256:ac1c4a689edcc530fc9d9aa11f5774b9e2f33f9a0c6a57864e90908f5208d30a", size = 150864, upload-time = "2025-10-14T04:41:00.623Z" }, + { url = "https://files.pythonhosted.org/packages/05/12/9fbc6a4d39c0198adeebbde20b619790e9236557ca59fc40e0e3cebe6f40/charset_normalizer-3.4.4-cp312-cp312-musllinux_1_2_aarch64.whl", hash = "sha256:21d142cc6c0ec30d2efee5068ca36c128a30b0f2c53c1c07bd78cb6bc1d3be5f", size = 150647, upload-time = "2025-10-14T04:41:01.754Z" }, + { url = "https://files.pythonhosted.org/packages/ad/1f/6a9a593d52e3e8c5d2b167daf8c6b968808efb57ef4c210acb907c365bc4/charset_normalizer-3.4.4-cp312-cp312-musllinux_1_2_armv7l.whl", hash = "sha256:5dbe56a36425d26d6cfb40ce79c314a2e4dd6211d51d6d2191c00bed34f354cc", size = 145110, upload-time = "2025-10-14T04:41:03.231Z" }, + { url = "https://files.pythonhosted.org/packages/30/42/9a52c609e72471b0fc54386dc63c3781a387bb4fe61c20231a4ebcd58bdd/charset_normalizer-3.4.4-cp312-cp312-musllinux_1_2_ppc64le.whl", hash = "sha256:5bfbb1b9acf3334612667b61bd3002196fe2a1eb4dd74d247e0f2a4d50ec9bbf", size = 162839, upload-time = "2025-10-14T04:41:04.715Z" }, + { url = "https://files.pythonhosted.org/packages/c4/5b/c0682bbf9f11597073052628ddd38344a3d673fda35a36773f7d19344b23/charset_normalizer-3.4.4-cp312-cp312-musllinux_1_2_riscv64.whl", hash = "sha256:d055ec1e26e441f6187acf818b73564e6e6282709e9bcb5b63f5b23068356a15", size = 150667, upload-time = "2025-10-14T04:41:05.827Z" }, + { url = "https://files.pythonhosted.org/packages/e4/24/a41afeab6f990cf2daf6cb8c67419b63b48cf518e4f56022230840c9bfb2/charset_normalizer-3.4.4-cp312-cp312-musllinux_1_2_s390x.whl", hash = "sha256:af2d8c67d8e573d6de5bc30cdb27e9b95e49115cd9baad5ddbd1a6207aaa82a9", size = 160535, upload-time = "2025-10-14T04:41:06.938Z" }, + { url = "https://files.pythonhosted.org/packages/2a/e5/6a4ce77ed243c4a50a1fecca6aaaab419628c818a49434be428fe24c9957/charset_normalizer-3.4.4-cp312-cp312-musllinux_1_2_x86_64.whl", hash = "sha256:780236ac706e66881f3b7f2f32dfe90507a09e67d1d454c762cf642e6e1586e0", size = 154816, upload-time = "2025-10-14T04:41:08.101Z" }, + { url = "https://files.pythonhosted.org/packages/a8/ef/89297262b8092b312d29cdb2517cb1237e51db8ecef2e9af5edbe7b683b1/charset_normalizer-3.4.4-cp312-cp312-win32.whl", hash = "sha256:5833d2c39d8896e4e19b689ffc198f08ea58116bee26dea51e362ecc7cd3ed26", size = 99694, upload-time = "2025-10-14T04:41:09.23Z" }, + { url = "https://files.pythonhosted.org/packages/3d/2d/1e5ed9dd3b3803994c155cd9aacb60c82c331bad84daf75bcb9c91b3295e/charset_normalizer-3.4.4-cp312-cp312-win_amd64.whl", hash = "sha256:a79cfe37875f822425b89a82333404539ae63dbdddf97f84dcbc3d339aae9525", size = 107131, upload-time = "2025-10-14T04:41:10.467Z" }, + { url = "https://files.pythonhosted.org/packages/d0/d9/0ed4c7098a861482a7b6a95603edce4c0d9db2311af23da1fb2b75ec26fc/charset_normalizer-3.4.4-cp312-cp312-win_arm64.whl", hash = "sha256:376bec83a63b8021bb5c8ea75e21c4ccb86e7e45ca4eb81146091b56599b80c3", size = 100390, upload-time = "2025-10-14T04:41:11.915Z" }, + { url = "https://files.pythonhosted.org/packages/0a/4c/925909008ed5a988ccbb72dcc897407e5d6d3bd72410d69e051fc0c14647/charset_normalizer-3.4.4-py3-none-any.whl", hash = "sha256:7a32c560861a02ff789ad905a2fe94e3f840803362c84fecf1851cb4cf3dc37f", size = 53402, upload-time = "2025-10-14T04:42:31.76Z" }, +] + +[[package]] +name = "click" +version = "8.3.0" +source = { registry = "https://pypi.org/simple" } +dependencies = [ + { name = "colorama", marker = "sys_platform == 'win32'" }, +] +sdist = { url = "https://files.pythonhosted.org/packages/46/61/de6cd827efad202d7057d93e0fed9294b96952e188f7384832791c7b2254/click-8.3.0.tar.gz", hash = "sha256:e7b8232224eba16f4ebe410c25ced9f7875cb5f3263ffc93cc3e8da705e229c4", size = 276943, upload-time = "2025-09-18T17:32:23.696Z" } +wheels = [ + { url = "https://files.pythonhosted.org/packages/db/d3/9dcc0f5797f070ec8edf30fbadfb200e71d9db6b84d211e3b2085a7589a0/click-8.3.0-py3-none-any.whl", hash = "sha256:9b9f285302c6e3064f4330c05f05b81945b2a39544279343e6e7c5f27a9baddc", size = 107295, upload-time = "2025-09-18T17:32:22.42Z" }, +] + +[[package]] +name = "colorama" +version = "0.4.6" +source = { registry = "https://pypi.org/simple" } +sdist = { url = "https://files.pythonhosted.org/packages/d8/53/6f443c9a4a8358a93a6792e2acffb9d9d5cb0a5cfd8802644b7b1c9a02e4/colorama-0.4.6.tar.gz", hash = "sha256:08695f5cb7ed6e0531a20572697297273c47b8cae5a63ffc6d6ed5c201be6e44", size = 27697, upload-time = "2022-10-25T02:36:22.414Z" } +wheels = [ + { url = "https://files.pythonhosted.org/packages/d1/d6/3965ed04c63042e047cb6a3e6ed1a63a35087b6a609aa3a15ed8ac56c221/colorama-0.4.6-py2.py3-none-any.whl", hash = "sha256:4f1d9991f5acc0ca119f9d443620b77f9d6b33703e51011c16baf57afb285fc6", size = 25335, upload-time = "2022-10-25T02:36:20.889Z" }, +] + +[[package]] +name = "datafusion" +version = "50.1.0" +source = { registry = "https://pypi.org/simple" } +dependencies = [ + { name = "pyarrow" }, + { name = "typing-extensions" }, +] +sdist = { url = "https://files.pythonhosted.org/packages/fa/cc/e8e8f7c472e93e7a560203ac40ac319b926029007c0dad873dbba97f9f2d/datafusion-50.1.0.tar.gz", hash = "sha256:d8b8f027c7ce2498cda1589d3ce6d8720798963e031660fbe4d2e26e172442ec", size = 188103, upload-time = "2025-10-20T12:39:23.802Z" } +wheels = [ + { url = "https://files.pythonhosted.org/packages/1f/6e/f9e2d5d935024a79fd549b5ce1d05549d26a027aab800727d492ac036504/datafusion-50.1.0-cp39-abi3-macosx_10_12_x86_64.whl", hash = "sha256:aeaa3c7bcf630bbea962b8fe75d300d98eaf7e2a5edf98e6a0130a1bec3543ea", size = 29280689, upload-time = "2025-10-20T12:39:06.913Z" }, + { url = "https://files.pythonhosted.org/packages/db/58/2dc473240f552d3620186b527c04397f82b36f02243afaf49f0813c84a17/datafusion-50.1.0-cp39-abi3-macosx_11_0_arm64.whl", hash = "sha256:85727df82c818103092c3ee18d198365833d3e44c2921d2b378d4d682798e511", size = 26140751, upload-time = "2025-10-20T12:39:09.95Z" }, + { url = "https://files.pythonhosted.org/packages/00/ba/8d8aa1df96e0666752e5c9d406d440495df2014d315b2a95bbef9856b23e/datafusion-50.1.0-cp39-abi3-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:49f5bd0edb2bf2d00625beeb46a115e1421db2e1b14b535f7c17cc0927f36b8a", size = 32165290, upload-time = "2025-10-20T12:39:13.713Z" }, + { url = "https://files.pythonhosted.org/packages/11/9a/afce9586145b3ed153d75364b21102a6a95260940352e06b7c6709e9d2db/datafusion-50.1.0-cp39-abi3-manylinux_2_28_aarch64.whl", hash = "sha256:5c9c2f70922ddedf54d8abd4ba9585a5026c3409438f5aafc1ad0428a67a4d1f", size = 29982398, upload-time = "2025-10-20T12:39:16.823Z" }, + { url = "https://files.pythonhosted.org/packages/51/a3/41ef1c565770ef0c4060ee3fd50367dd06816f70a5be1ef41fbd7c3975e8/datafusion-50.1.0-cp39-abi3-win_amd64.whl", hash = "sha256:145c8f2e969c9cc51dc6af8a185ec39739ebeb5d680f9fe0020e005564ed40a8", size = 31258359, upload-time = "2025-10-20T12:39:21.731Z" }, +] + +[[package]] +name = "exceptiongroup" +version = "1.3.0" +source = { registry = "https://pypi.org/simple" } +dependencies = [ + { name = "typing-extensions" }, +] +sdist = { url = "https://files.pythonhosted.org/packages/0b/9f/a65090624ecf468cdca03533906e7c69ed7588582240cfe7cc9e770b50eb/exceptiongroup-1.3.0.tar.gz", hash = "sha256:b241f5885f560bc56a59ee63ca4c6a8bfa46ae4ad651af316d4e81817bb9fd88", size = 29749, upload-time = "2025-05-10T17:42:51.123Z" } +wheels = [ + { url = "https://files.pythonhosted.org/packages/36/f4/c6e662dade71f56cd2f3735141b265c3c79293c109549c1e6933b0651ffc/exceptiongroup-1.3.0-py3-none-any.whl", hash = "sha256:4d111e6e0c13d0644cad6ddaa7ed0261a0b36971f6d23e7ec9b4b9097da78a10", size = 16674, upload-time = "2025-05-10T17:42:49.33Z" }, +] + +[[package]] +name = "fastavro" +version = "1.12.1" +source = { registry = "https://pypi.org/simple" } +sdist = { url = "https://files.pythonhosted.org/packages/65/8b/fa2d3287fd2267be6261d0177c6809a7fa12c5600ddb33490c8dc29e77b2/fastavro-1.12.1.tar.gz", hash = "sha256:2f285be49e45bc047ab2f6bed040bb349da85db3f3c87880e4b92595ea093b2b", size = 1025661, upload-time = "2025-10-10T15:40:55.41Z" } +wheels = [ + { url = "https://files.pythonhosted.org/packages/40/a0/077fd7cbfc143152cb96780cb592ed6cb6696667d8bc1b977745eb2255a8/fastavro-1.12.1-cp310-cp310-macosx_10_9_universal2.whl", hash = "sha256:00650ca533907361edda22e6ffe8cf87ab2091c5d8aee5c8000b0f2dcdda7ed3", size = 1000335, upload-time = "2025-10-10T15:40:59.834Z" }, + { url = "https://files.pythonhosted.org/packages/a0/ae/a115e027f3a75df237609701b03ecba0b7f0aa3d77fe0161df533fde1eb7/fastavro-1.12.1-cp310-cp310-manylinux2014_aarch64.manylinux_2_17_aarch64.manylinux_2_28_aarch64.whl", hash = "sha256:ac76d6d95f909c72ee70d314b460b7e711d928845771531d823eb96a10952d26", size = 3221067, upload-time = "2025-10-10T15:41:04.399Z" }, + { url = "https://files.pythonhosted.org/packages/94/4e/c4991c3eec0175af9a8a0c161b88089cb7bf7fe353b3e3be1bc4cf9036b2/fastavro-1.12.1-cp310-cp310-manylinux2014_x86_64.manylinux_2_17_x86_64.manylinux_2_28_x86_64.whl", hash = "sha256:1f55eef18c41d4476bd32a82ed5dd86aabc3f614e1b66bdb09ffa291612e1670", size = 3228979, upload-time = "2025-10-10T15:41:06.738Z" }, + { url = "https://files.pythonhosted.org/packages/21/0c/f2afb8eaea38799ccb1ed07d68bf2659f2e313f1902bbd36774cf6a1bef9/fastavro-1.12.1-cp310-cp310-musllinux_1_2_aarch64.whl", hash = "sha256:81563e1f93570e6565487cdb01ba241a36a00e58cff9c5a0614af819d1155d8f", size = 3160740, upload-time = "2025-10-10T15:41:08.731Z" }, + { url = "https://files.pythonhosted.org/packages/0d/1a/f4d367924b40b86857862c1fa65f2afba94ddadf298b611e610a676a29e5/fastavro-1.12.1-cp310-cp310-musllinux_1_2_x86_64.whl", hash = "sha256:bec207360f76f0b3de540758a297193c5390e8e081c43c3317f610b1414d8c8f", size = 3235787, upload-time = "2025-10-10T15:41:10.869Z" }, + { url = "https://files.pythonhosted.org/packages/90/ec/8db9331896e3dfe4f71b2b3c23f2e97fbbfd90129777467ca9f8bafccb74/fastavro-1.12.1-cp310-cp310-win_amd64.whl", hash = "sha256:c0390bfe4a9f8056a75ac6785fbbff8f5e317f5356481d2e29ec980877d2314b", size = 449350, upload-time = "2025-10-10T15:41:12.104Z" }, + { url = "https://files.pythonhosted.org/packages/a0/e9/31c64b47cefc0951099e7c0c8c8ea1c931edd1350f34d55c27cbfbb08df1/fastavro-1.12.1-cp311-cp311-macosx_10_9_universal2.whl", hash = "sha256:6b632b713bc5d03928a87d811fa4a11d5f25cd43e79c161e291c7d3f7aa740fd", size = 1016585, upload-time = "2025-10-10T15:41:13.717Z" }, + { url = "https://files.pythonhosted.org/packages/10/76/111560775b548f5d8d828c1b5285ff90e2d2745643fb80ecbf115344eea4/fastavro-1.12.1-cp311-cp311-manylinux2014_aarch64.manylinux_2_17_aarch64.manylinux_2_28_aarch64.whl", hash = "sha256:eaa7ab3769beadcebb60f0539054c7755f63bd9cf7666e2c15e615ab605f89a8", size = 3404629, upload-time = "2025-10-10T15:41:15.642Z" }, + { url = "https://files.pythonhosted.org/packages/b0/07/6bb93cb963932146c2b6c5c765903a0a547ad9f0f8b769a4a9aad8c06369/fastavro-1.12.1-cp311-cp311-manylinux2014_x86_64.manylinux_2_17_x86_64.manylinux_2_28_x86_64.whl", hash = "sha256:123fb221df3164abd93f2d042c82f538a1d5a43ce41375f12c91ce1355a9141e", size = 3428594, upload-time = "2025-10-10T15:41:17.779Z" }, + { url = "https://files.pythonhosted.org/packages/d1/67/8115ec36b584197ea737ec79e3499e1f1b640b288d6c6ee295edd13b80f6/fastavro-1.12.1-cp311-cp311-musllinux_1_2_aarch64.whl", hash = "sha256:632a4e3ff223f834ddb746baae0cc7cee1068eb12c32e4d982c2fee8a5b483d0", size = 3344145, upload-time = "2025-10-10T15:41:19.89Z" }, + { url = "https://files.pythonhosted.org/packages/9e/9e/a7cebb3af967e62539539897c10138fa0821668ec92525d1be88a9cd3ee6/fastavro-1.12.1-cp311-cp311-musllinux_1_2_x86_64.whl", hash = "sha256:83e6caf4e7a8717d932a3b1ff31595ad169289bbe1128a216be070d3a8391671", size = 3431942, upload-time = "2025-10-10T15:41:22.076Z" }, + { url = "https://files.pythonhosted.org/packages/c0/d1/7774ddfb8781c5224294c01a593ebce2ad3289b948061c9701bd1903264d/fastavro-1.12.1-cp311-cp311-win_amd64.whl", hash = "sha256:b91a0fe5a173679a6c02d53ca22dcaad0a2c726b74507e0c1c2e71a7c3f79ef9", size = 450542, upload-time = "2025-10-10T15:41:23.333Z" }, + { url = "https://files.pythonhosted.org/packages/7c/f0/10bd1a3d08667fa0739e2b451fe90e06df575ec8b8ba5d3135c70555c9bd/fastavro-1.12.1-cp312-cp312-macosx_10_13_universal2.whl", hash = "sha256:509818cb24b98a804fc80be9c5fed90f660310ae3d59382fc811bfa187122167", size = 1009057, upload-time = "2025-10-10T15:41:24.556Z" }, + { url = "https://files.pythonhosted.org/packages/78/ad/0d985bc99e1fa9e74c636658000ba38a5cd7f5ab2708e9c62eaf736ecf1a/fastavro-1.12.1-cp312-cp312-manylinux2014_aarch64.manylinux_2_17_aarch64.manylinux_2_28_aarch64.whl", hash = "sha256:089e155c0c76e0d418d7e79144ce000524dd345eab3bc1e9c5ae69d500f71b14", size = 3391866, upload-time = "2025-10-10T15:41:26.882Z" }, + { url = "https://files.pythonhosted.org/packages/0d/9e/b4951dc84ebc34aac69afcbfbb22ea4a91080422ec2bfd2c06076ff1d419/fastavro-1.12.1-cp312-cp312-manylinux2014_x86_64.manylinux_2_17_x86_64.manylinux_2_28_x86_64.whl", hash = "sha256:44cbff7518901c91a82aab476fcab13d102e4999499df219d481b9e15f61af34", size = 3458005, upload-time = "2025-10-10T15:41:29.017Z" }, + { url = "https://files.pythonhosted.org/packages/af/f8/5a8df450a9f55ca8441f22ea0351d8c77809fc121498b6970daaaf667a21/fastavro-1.12.1-cp312-cp312-musllinux_1_2_aarch64.whl", hash = "sha256:a275e48df0b1701bb764b18a8a21900b24cf882263cb03d35ecdba636bbc830b", size = 3295258, upload-time = "2025-10-10T15:41:31.564Z" }, + { url = "https://files.pythonhosted.org/packages/99/b2/40f25299111d737e58b85696e91138a66c25b7334f5357e7ac2b0e8966f8/fastavro-1.12.1-cp312-cp312-musllinux_1_2_x86_64.whl", hash = "sha256:2de72d786eb38be6b16d556b27232b1bf1b2797ea09599507938cdb7a9fe3e7c", size = 3430328, upload-time = "2025-10-10T15:41:33.689Z" }, + { url = "https://files.pythonhosted.org/packages/e0/07/85157a7c57c5f8b95507d7829b5946561e5ee656ff80e9dd9a757f53ddaf/fastavro-1.12.1-cp312-cp312-win_amd64.whl", hash = "sha256:9090f0dee63fe022ee9cc5147483366cc4171c821644c22da020d6b48f576b4f", size = 444140, upload-time = "2025-10-10T15:41:34.902Z" }, +] + +[[package]] +name = "fsspec" +version = "2025.9.0" +source = { registry = "https://pypi.org/simple" } +sdist = { url = "https://files.pythonhosted.org/packages/de/e0/bab50af11c2d75c9c4a2a26a5254573c0bd97cea152254401510950486fa/fsspec-2025.9.0.tar.gz", hash = "sha256:19fd429483d25d28b65ec68f9f4adc16c17ea2c7c7bf54ec61360d478fb19c19", size = 304847, upload-time = "2025-09-02T19:10:49.215Z" } +wheels = [ + { url = "https://files.pythonhosted.org/packages/47/71/70db47e4f6ce3e5c37a607355f80da8860a33226be640226ac52cb05ef2e/fsspec-2025.9.0-py3-none-any.whl", hash = "sha256:530dc2a2af60a414a832059574df4a6e10cce927f6f4a78209390fe38955cfb7", size = 199289, upload-time = "2025-09-02T19:10:47.708Z" }, +] + +[[package]] +name = "greenlet" +version = "3.2.4" +source = { registry = "https://pypi.org/simple" } +sdist = { url = "https://files.pythonhosted.org/packages/03/b8/704d753a5a45507a7aab61f18db9509302ed3d0a27ac7e0359ec2905b1a6/greenlet-3.2.4.tar.gz", hash = "sha256:0dca0d95ff849f9a364385f36ab49f50065d76964944638be9691e1832e9f86d", size = 188260, upload-time = "2025-08-07T13:24:33.51Z" } +wheels = [ + { url = "https://files.pythonhosted.org/packages/7d/ed/6bfa4109fcb23a58819600392564fea69cdc6551ffd5e69ccf1d52a40cbc/greenlet-3.2.4-cp310-cp310-macosx_11_0_universal2.whl", hash = "sha256:8c68325b0d0acf8d91dde4e6f930967dd52a5302cd4062932a6b2e7c2969f47c", size = 271061, upload-time = "2025-08-07T13:17:15.373Z" }, + { url = "https://files.pythonhosted.org/packages/2a/fc/102ec1a2fc015b3a7652abab7acf3541d58c04d3d17a8d3d6a44adae1eb1/greenlet-3.2.4-cp310-cp310-manylinux2014_aarch64.manylinux_2_17_aarch64.whl", hash = "sha256:94385f101946790ae13da500603491f04a76b6e4c059dab271b3ce2e283b2590", size = 629475, upload-time = "2025-08-07T13:42:54.009Z" }, + { url = "https://files.pythonhosted.org/packages/c5/26/80383131d55a4ac0fb08d71660fd77e7660b9db6bdb4e8884f46d9f2cc04/greenlet-3.2.4-cp310-cp310-manylinux2014_ppc64le.manylinux_2_17_ppc64le.whl", hash = "sha256:f10fd42b5ee276335863712fa3da6608e93f70629c631bf77145021600abc23c", size = 640802, upload-time = "2025-08-07T13:45:25.52Z" }, + { url = "https://files.pythonhosted.org/packages/9f/7c/e7833dbcd8f376f3326bd728c845d31dcde4c84268d3921afcae77d90d08/greenlet-3.2.4-cp310-cp310-manylinux2014_s390x.manylinux_2_17_s390x.whl", hash = "sha256:c8c9e331e58180d0d83c5b7999255721b725913ff6bc6cf39fa2a45841a4fd4b", size = 636703, upload-time = "2025-08-07T13:53:12.622Z" }, + { url = "https://files.pythonhosted.org/packages/e9/49/547b93b7c0428ede7b3f309bc965986874759f7d89e4e04aeddbc9699acb/greenlet-3.2.4-cp310-cp310-manylinux2014_x86_64.manylinux_2_17_x86_64.whl", hash = "sha256:58b97143c9cc7b86fc458f215bd0932f1757ce649e05b640fea2e79b54cedb31", size = 635417, upload-time = "2025-08-07T13:18:25.189Z" }, + { url = "https://files.pythonhosted.org/packages/7f/91/ae2eb6b7979e2f9b035a9f612cf70f1bf54aad4e1d125129bef1eae96f19/greenlet-3.2.4-cp310-cp310-manylinux_2_24_x86_64.manylinux_2_28_x86_64.whl", hash = "sha256:c2ca18a03a8cfb5b25bc1cbe20f3d9a4c80d8c3b13ba3df49ac3961af0b1018d", size = 584358, upload-time = "2025-08-07T13:18:23.708Z" }, + { url = "https://files.pythonhosted.org/packages/f7/85/433de0c9c0252b22b16d413c9407e6cb3b41df7389afc366ca204dbc1393/greenlet-3.2.4-cp310-cp310-musllinux_1_1_aarch64.whl", hash = "sha256:9fe0a28a7b952a21e2c062cd5756d34354117796c6d9215a87f55e38d15402c5", size = 1113550, upload-time = "2025-08-07T13:42:37.467Z" }, + { url = "https://files.pythonhosted.org/packages/a1/8d/88f3ebd2bc96bf7747093696f4335a0a8a4c5acfcf1b757717c0d2474ba3/greenlet-3.2.4-cp310-cp310-musllinux_1_1_x86_64.whl", hash = "sha256:8854167e06950ca75b898b104b63cc646573aa5fef1353d4508ecdd1ee76254f", size = 1137126, upload-time = "2025-08-07T13:18:20.239Z" }, + { url = "https://files.pythonhosted.org/packages/d6/6f/b60b0291d9623c496638c582297ead61f43c4b72eef5e9c926ef4565ec13/greenlet-3.2.4-cp310-cp310-win_amd64.whl", hash = "sha256:73f49b5368b5359d04e18d15828eecc1806033db5233397748f4ca813ff1056c", size = 298654, upload-time = "2025-08-07T13:50:00.469Z" }, + { url = "https://files.pythonhosted.org/packages/a4/de/f28ced0a67749cac23fecb02b694f6473f47686dff6afaa211d186e2ef9c/greenlet-3.2.4-cp311-cp311-macosx_11_0_universal2.whl", hash = "sha256:96378df1de302bc38e99c3a9aa311967b7dc80ced1dcc6f171e99842987882a2", size = 272305, upload-time = "2025-08-07T13:15:41.288Z" }, + { url = "https://files.pythonhosted.org/packages/09/16/2c3792cba130000bf2a31c5272999113f4764fd9d874fb257ff588ac779a/greenlet-3.2.4-cp311-cp311-manylinux2014_aarch64.manylinux_2_17_aarch64.whl", hash = "sha256:1ee8fae0519a337f2329cb78bd7a8e128ec0f881073d43f023c7b8d4831d5246", size = 632472, upload-time = "2025-08-07T13:42:55.044Z" }, + { url = "https://files.pythonhosted.org/packages/ae/8f/95d48d7e3d433e6dae5b1682e4292242a53f22df82e6d3dda81b1701a960/greenlet-3.2.4-cp311-cp311-manylinux2014_ppc64le.manylinux_2_17_ppc64le.whl", hash = "sha256:94abf90142c2a18151632371140b3dba4dee031633fe614cb592dbb6c9e17bc3", size = 644646, upload-time = "2025-08-07T13:45:26.523Z" }, + { url = "https://files.pythonhosted.org/packages/d5/5e/405965351aef8c76b8ef7ad370e5da58d57ef6068df197548b015464001a/greenlet-3.2.4-cp311-cp311-manylinux2014_s390x.manylinux_2_17_s390x.whl", hash = "sha256:4d1378601b85e2e5171b99be8d2dc85f594c79967599328f95c1dc1a40f1c633", size = 640519, upload-time = "2025-08-07T13:53:13.928Z" }, + { url = "https://files.pythonhosted.org/packages/25/5d/382753b52006ce0218297ec1b628e048c4e64b155379331f25a7316eb749/greenlet-3.2.4-cp311-cp311-manylinux2014_x86_64.manylinux_2_17_x86_64.whl", hash = "sha256:0db5594dce18db94f7d1650d7489909b57afde4c580806b8d9203b6e79cdc079", size = 639707, upload-time = "2025-08-07T13:18:27.146Z" }, + { url = "https://files.pythonhosted.org/packages/1f/8e/abdd3f14d735b2929290a018ecf133c901be4874b858dd1c604b9319f064/greenlet-3.2.4-cp311-cp311-manylinux_2_24_x86_64.manylinux_2_28_x86_64.whl", hash = "sha256:2523e5246274f54fdadbce8494458a2ebdcdbc7b802318466ac5606d3cded1f8", size = 587684, upload-time = "2025-08-07T13:18:25.164Z" }, + { url = "https://files.pythonhosted.org/packages/5d/65/deb2a69c3e5996439b0176f6651e0052542bb6c8f8ec2e3fba97c9768805/greenlet-3.2.4-cp311-cp311-musllinux_1_1_aarch64.whl", hash = "sha256:1987de92fec508535687fb807a5cea1560f6196285a4cde35c100b8cd632cc52", size = 1116647, upload-time = "2025-08-07T13:42:38.655Z" }, + { url = "https://files.pythonhosted.org/packages/3f/cc/b07000438a29ac5cfb2194bfc128151d52f333cee74dd7dfe3fb733fc16c/greenlet-3.2.4-cp311-cp311-musllinux_1_1_x86_64.whl", hash = "sha256:55e9c5affaa6775e2c6b67659f3a71684de4c549b3dd9afca3bc773533d284fa", size = 1142073, upload-time = "2025-08-07T13:18:21.737Z" }, + { url = "https://files.pythonhosted.org/packages/d8/0f/30aef242fcab550b0b3520b8e3561156857c94288f0332a79928c31a52cf/greenlet-3.2.4-cp311-cp311-win_amd64.whl", hash = "sha256:9c40adce87eaa9ddb593ccb0fa6a07caf34015a29bf8d344811665b573138db9", size = 299100, upload-time = "2025-08-07T13:44:12.287Z" }, + { url = "https://files.pythonhosted.org/packages/44/69/9b804adb5fd0671f367781560eb5eb586c4d495277c93bde4307b9e28068/greenlet-3.2.4-cp312-cp312-macosx_11_0_universal2.whl", hash = "sha256:3b67ca49f54cede0186854a008109d6ee71f66bd57bb36abd6d0a0267b540cdd", size = 274079, upload-time = "2025-08-07T13:15:45.033Z" }, + { url = "https://files.pythonhosted.org/packages/46/e9/d2a80c99f19a153eff70bc451ab78615583b8dac0754cfb942223d2c1a0d/greenlet-3.2.4-cp312-cp312-manylinux2014_aarch64.manylinux_2_17_aarch64.whl", hash = "sha256:ddf9164e7a5b08e9d22511526865780a576f19ddd00d62f8a665949327fde8bb", size = 640997, upload-time = "2025-08-07T13:42:56.234Z" }, + { url = "https://files.pythonhosted.org/packages/3b/16/035dcfcc48715ccd345f3a93183267167cdd162ad123cd93067d86f27ce4/greenlet-3.2.4-cp312-cp312-manylinux2014_ppc64le.manylinux_2_17_ppc64le.whl", hash = "sha256:f28588772bb5fb869a8eb331374ec06f24a83a9c25bfa1f38b6993afe9c1e968", size = 655185, upload-time = "2025-08-07T13:45:27.624Z" }, + { url = "https://files.pythonhosted.org/packages/31/da/0386695eef69ffae1ad726881571dfe28b41970173947e7c558d9998de0f/greenlet-3.2.4-cp312-cp312-manylinux2014_s390x.manylinux_2_17_s390x.whl", hash = "sha256:5c9320971821a7cb77cfab8d956fa8e39cd07ca44b6070db358ceb7f8797c8c9", size = 649926, upload-time = "2025-08-07T13:53:15.251Z" }, + { url = "https://files.pythonhosted.org/packages/68/88/69bf19fd4dc19981928ceacbc5fd4bb6bc2215d53199e367832e98d1d8fe/greenlet-3.2.4-cp312-cp312-manylinux2014_x86_64.manylinux_2_17_x86_64.whl", hash = "sha256:c60a6d84229b271d44b70fb6e5fa23781abb5d742af7b808ae3f6efd7c9c60f6", size = 651839, upload-time = "2025-08-07T13:18:30.281Z" }, + { url = "https://files.pythonhosted.org/packages/19/0d/6660d55f7373b2ff8152401a83e02084956da23ae58cddbfb0b330978fe9/greenlet-3.2.4-cp312-cp312-manylinux_2_24_x86_64.manylinux_2_28_x86_64.whl", hash = "sha256:3b3812d8d0c9579967815af437d96623f45c0f2ae5f04e366de62a12d83a8fb0", size = 607586, upload-time = "2025-08-07T13:18:28.544Z" }, + { url = "https://files.pythonhosted.org/packages/8e/1a/c953fdedd22d81ee4629afbb38d2f9d71e37d23caace44775a3a969147d4/greenlet-3.2.4-cp312-cp312-musllinux_1_1_aarch64.whl", hash = "sha256:abbf57b5a870d30c4675928c37278493044d7c14378350b3aa5d484fa65575f0", size = 1123281, upload-time = "2025-08-07T13:42:39.858Z" }, + { url = "https://files.pythonhosted.org/packages/3f/c7/12381b18e21aef2c6bd3a636da1088b888b97b7a0362fac2e4de92405f97/greenlet-3.2.4-cp312-cp312-musllinux_1_1_x86_64.whl", hash = "sha256:20fb936b4652b6e307b8f347665e2c615540d4b42b3b4c8a321d8286da7e520f", size = 1151142, upload-time = "2025-08-07T13:18:22.981Z" }, + { url = "https://files.pythonhosted.org/packages/e9/08/b0814846b79399e585f974bbeebf5580fbe59e258ea7be64d9dfb253c84f/greenlet-3.2.4-cp312-cp312-win_amd64.whl", hash = "sha256:a7d4e128405eea3814a12cc2605e0e6aedb4035bf32697f72deca74de4105e02", size = 299899, upload-time = "2025-08-07T13:38:53.448Z" }, +] + +[[package]] +name = "idna" +version = "3.11" +source = { registry = "https://pypi.org/simple" } +sdist = { url = "https://files.pythonhosted.org/packages/6f/6d/0703ccc57f3a7233505399edb88de3cbd678da106337b9fcde432b65ed60/idna-3.11.tar.gz", hash = "sha256:795dafcc9c04ed0c1fb032c2aa73654d8e8c5023a7df64a53f39190ada629902", size = 194582, upload-time = "2025-10-12T14:55:20.501Z" } +wheels = [ + { url = "https://files.pythonhosted.org/packages/0e/61/66938bbb5fc52dbdf84594873d5b51fb1f7c7794e9c0f5bd885f30bc507b/idna-3.11-py3-none-any.whl", hash = "sha256:771a87f49d9defaf64091e6e6fe9c18d4833f140bd19464795bc32d966ca37ea", size = 71008, upload-time = "2025-10-12T14:55:18.883Z" }, +] + +[[package]] +name = "iniconfig" +version = "2.3.0" +source = { registry = "https://pypi.org/simple" } +sdist = { url = "https://files.pythonhosted.org/packages/72/34/14ca021ce8e5dfedc35312d08ba8bf51fdd999c576889fc2c24cb97f4f10/iniconfig-2.3.0.tar.gz", hash = "sha256:c76315c77db068650d49c5b56314774a7804df16fee4402c1f19d6d15d8c4730", size = 20503, upload-time = "2025-10-18T21:55:43.219Z" } +wheels = [ + { url = "https://files.pythonhosted.org/packages/cb/b1/3846dd7f199d53cb17f49cba7e651e9ce294d8497c8c150530ed11865bb8/iniconfig-2.3.0-py3-none-any.whl", hash = "sha256:f631c04d2c48c52b84d0d0549c99ff3859c98df65b3101406327ecc7d53fbf12", size = 7484, upload-time = "2025-10-18T21:55:41.639Z" }, +] + +[[package]] +name = "markdown-it-py" +version = "4.0.0" +source = { registry = "https://pypi.org/simple" } +dependencies = [ + { name = "mdurl" }, +] +sdist = { url = "https://files.pythonhosted.org/packages/5b/f5/4ec618ed16cc4f8fb3b701563655a69816155e79e24a17b651541804721d/markdown_it_py-4.0.0.tar.gz", hash = "sha256:cb0a2b4aa34f932c007117b194e945bd74e0ec24133ceb5bac59009cda1cb9f3", size = 73070, upload-time = "2025-08-11T12:57:52.854Z" } +wheels = [ + { url = "https://files.pythonhosted.org/packages/94/54/e7d793b573f298e1c9013b8c4dade17d481164aa517d1d7148619c2cedbf/markdown_it_py-4.0.0-py3-none-any.whl", hash = "sha256:87327c59b172c5011896038353a81343b6754500a08cd7a4973bb48c6d578147", size = 87321, upload-time = "2025-08-11T12:57:51.923Z" }, +] + +[[package]] +name = "maturin" +version = "1.9.6" +source = { registry = "https://pypi.org/simple" } +dependencies = [ + { name = "tomli", marker = "python_full_version < '3.11'" }, +] +sdist = { url = "https://files.pythonhosted.org/packages/9a/35/c3370188492f4c139c7a318f438d01b8185c216303c49c4bc885c98b6afb/maturin-1.9.6.tar.gz", hash = "sha256:2c2ae37144811d365509889ed7220b0598487f1278c2441829c3abf56cc6324a", size = 214846, upload-time = "2025-10-07T12:45:08.408Z" } +wheels = [ + { url = "https://files.pythonhosted.org/packages/55/5c/b435418ba4ba2647a1f7a95d53314991b1e556e656ae276dea993c3bce1d/maturin-1.9.6-py3-none-linux_armv6l.whl", hash = "sha256:26e3ab1a42a7145824210e9d763f6958f2c46afb1245ddd0bab7d78b1f59bb3f", size = 8134483, upload-time = "2025-10-07T12:44:44.274Z" }, + { url = "https://files.pythonhosted.org/packages/4d/1c/8e58eda6601f328b412cdeeaa88a9b6a10e591e2a73f313e8c0154d68385/maturin-1.9.6-py3-none-macosx_10_12_x86_64.macosx_11_0_arm64.macosx_10_12_universal2.whl", hash = "sha256:5263dda3f71feef2e4122baf5c4620e4b3710dbb7f2121f85a337182de214369", size = 15776470, upload-time = "2025-10-07T12:44:47.476Z" }, + { url = "https://files.pythonhosted.org/packages/6c/33/8c967cce6848cdd87a2e442c86120ac644b80c5ed4c32e3291bde6a17df8/maturin-1.9.6-py3-none-macosx_10_12_x86_64.whl", hash = "sha256:fe78262c2800c92f67d1ce3c0f6463f958a692cc67bfb572e5dbf5b4b696a8ba", size = 8226557, upload-time = "2025-10-07T12:44:49.844Z" }, + { url = "https://files.pythonhosted.org/packages/58/bd/3e2675cdc8b7270700ba30c663c852a35694441732a107ac30ebd6878bd8/maturin-1.9.6-py3-none-manylinux_2_12_i686.manylinux2010_i686.musllinux_1_1_i686.whl", hash = "sha256:7ab827c6e8c022eb2e1e7fb6deede54549c8460b20ccc2e9268cc6e8cde957a8", size = 8166544, upload-time = "2025-10-07T12:44:51.396Z" }, + { url = "https://files.pythonhosted.org/packages/58/1f/a2047ddf2230e700d5f8a13dd4b9af5ce806ad380c32e58105888205926e/maturin-1.9.6-py3-none-manylinux_2_12_x86_64.manylinux2010_x86_64.musllinux_1_1_x86_64.whl", hash = "sha256:0246202377c49449315305209f45c8ecef6e2d6bd27a04b5b6f1ab3e4ea47238", size = 8641010, upload-time = "2025-10-07T12:44:53.658Z" }, + { url = "https://files.pythonhosted.org/packages/be/1f/265d63c7aa6faf363d4a3f23396f51bc6b4d5c7680a4190ae68dba25dea2/maturin-1.9.6-py3-none-manylinux_2_17_aarch64.manylinux2014_aarch64.musllinux_1_1_aarch64.whl", hash = "sha256:f5bac167700fbb6f8c8ed1a97b494522554b4432d7578e11403b894b6a91d99f", size = 7965945, upload-time = "2025-10-07T12:44:55.248Z" }, + { url = "https://files.pythonhosted.org/packages/4c/ca/a8e61979ccfe080948bcc1bddd79356157aee687134df7fb013050cec783/maturin-1.9.6-py3-none-manylinux_2_17_armv7l.manylinux2014_armv7l.musllinux_1_1_armv7l.whl", hash = "sha256:7f53d3b1d8396d3fea3e1ee5fd37558bca5719090f3d194ba1c02b0b56327ae3", size = 7978820, upload-time = "2025-10-07T12:44:56.919Z" }, + { url = "https://files.pythonhosted.org/packages/bf/4a/81b412f8ad02a99801ef19ec059fba0822d1d28fb44cb6a92e722f05f278/maturin-1.9.6-py3-none-manylinux_2_17_ppc64le.manylinux2014_ppc64le.musllinux_1_1_ppc64le.whl", hash = "sha256:7f506eb358386d94d6ec3208c003130cf4b69cab26034fc0cbbf8bf83afa4c2e", size = 10452064, upload-time = "2025-10-07T12:44:58.232Z" }, + { url = "https://files.pythonhosted.org/packages/5b/12/cc96c7a8cb51d8dcc9badd886c361caa1526fba7fa69d1e7892e613b71d4/maturin-1.9.6-py3-none-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:f2d6984ab690af509f525dbd2b130714207c06ebb14a5814edbe1e42b17ae0de", size = 8852401, upload-time = "2025-10-07T12:44:59.8Z" }, + { url = "https://files.pythonhosted.org/packages/51/8e/653ac3c9f2c25cdd81aefb0a2d17ff140ca5a14504f5e3c7f94dcfe4dbb7/maturin-1.9.6-py3-none-manylinux_2_31_riscv64.whl", hash = "sha256:5c2252b0956bb331460ac750c805ddf0d9b44442449fc1f16e3b66941689d0bc", size = 8425057, upload-time = "2025-10-07T12:45:01.711Z" }, + { url = "https://files.pythonhosted.org/packages/db/29/f13490328764ae9bfc1da55afc5b707cebe4fa75ad7a1573bfa82cfae0c6/maturin-1.9.6-py3-none-win32.whl", hash = "sha256:f2c58d29ebdd4346fd004e6be213d071fdd94a77a16aa91474a21a4f9dbf6309", size = 7165956, upload-time = "2025-10-07T12:45:03.766Z" }, + { url = "https://files.pythonhosted.org/packages/db/9f/dd51e5ac1fce47581b8efa03d77a03f928c0ef85b6e48a61dfa37b6b85a2/maturin-1.9.6-py3-none-win_amd64.whl", hash = "sha256:1b39a5d82572c240d20d9e8be024d722dfb311d330c5e28ddeb615211755941a", size = 8145722, upload-time = "2025-10-07T12:45:05.487Z" }, + { url = "https://files.pythonhosted.org/packages/65/f2/e97aaba6d0d78c5871771bf9dd71d4eb8dac15df9109cf452748d2207412/maturin-1.9.6-py3-none-win_arm64.whl", hash = "sha256:ac02a30083553d2a781c10cd6f5480119bf6692fd177e743267406cad2ad198c", size = 6857006, upload-time = "2025-10-07T12:45:06.813Z" }, +] + +[[package]] +name = "mdurl" +version = "0.1.2" +source = { registry = "https://pypi.org/simple" } +sdist = { url = "https://files.pythonhosted.org/packages/d6/54/cfe61301667036ec958cb99bd3efefba235e65cdeb9c84d24a8293ba1d90/mdurl-0.1.2.tar.gz", hash = "sha256:bb413d29f5eea38f31dd4754dd7377d4465116fb207585f97bf925588687c1ba", size = 8729, upload-time = "2022-08-14T12:40:10.846Z" } +wheels = [ + { url = "https://files.pythonhosted.org/packages/b3/38/89ba8ad64ae25be8de66a6d463314cf1eb366222074cfda9ee839c56a4b4/mdurl-0.1.2-py3-none-any.whl", hash = "sha256:84008a41e51615a49fc9966191ff91509e3c40b939176e643fd50a5c2196b8f8", size = 9979, upload-time = "2022-08-14T12:40:09.779Z" }, +] + +[[package]] +name = "mmh3" +version = "5.2.0" +source = { registry = "https://pypi.org/simple" } +sdist = { url = "https://files.pythonhosted.org/packages/a7/af/f28c2c2f51f31abb4725f9a64bc7863d5f491f6539bd26aee2a1d21a649e/mmh3-5.2.0.tar.gz", hash = "sha256:1efc8fec8478e9243a78bb993422cf79f8ff85cb4cf6b79647480a31e0d950a8", size = 33582, upload-time = "2025-07-29T07:43:48.49Z" } +wheels = [ + { url = "https://files.pythonhosted.org/packages/b9/2b/870f0ff5ecf312c58500f45950751f214b7068665e66e9bfd8bc2595587c/mmh3-5.2.0-cp310-cp310-macosx_10_9_universal2.whl", hash = "sha256:81c504ad11c588c8629536b032940f2a359dda3b6cbfd4ad8f74cb24dcd1b0bc", size = 56119, upload-time = "2025-07-29T07:41:39.117Z" }, + { url = "https://files.pythonhosted.org/packages/3b/88/eb9a55b3f3cf43a74d6bfa8db0e2e209f966007777a1dc897c52c008314c/mmh3-5.2.0-cp310-cp310-macosx_10_9_x86_64.whl", hash = "sha256:0b898cecff57442724a0f52bf42c2de42de63083a91008fb452887e372f9c328", size = 40634, upload-time = "2025-07-29T07:41:40.626Z" }, + { url = "https://files.pythonhosted.org/packages/d1/4c/8e4b3878bf8435c697d7ce99940a3784eb864521768069feaccaff884a17/mmh3-5.2.0-cp310-cp310-macosx_11_0_arm64.whl", hash = "sha256:be1374df449465c9f2500e62eee73a39db62152a8bdfbe12ec5b5c1cd451344d", size = 40080, upload-time = "2025-07-29T07:41:41.791Z" }, + { url = "https://files.pythonhosted.org/packages/45/ac/0a254402c8c5ca424a0a9ebfe870f5665922f932830f0a11a517b6390a09/mmh3-5.2.0-cp310-cp310-manylinux1_i686.manylinux_2_28_i686.manylinux_2_5_i686.whl", hash = "sha256:b0d753ad566c721faa33db7e2e0eddd74b224cdd3eaf8481d76c926603c7a00e", size = 95321, upload-time = "2025-07-29T07:41:42.659Z" }, + { url = "https://files.pythonhosted.org/packages/39/8e/29306d5eca6dfda4b899d22c95b5420db4e0ffb7e0b6389b17379654ece5/mmh3-5.2.0-cp310-cp310-manylinux1_x86_64.manylinux_2_28_x86_64.manylinux_2_5_x86_64.whl", hash = "sha256:dfbead5575f6470c17e955b94f92d62a03dfc3d07f2e6f817d9b93dc211a1515", size = 101220, upload-time = "2025-07-29T07:41:43.572Z" }, + { url = "https://files.pythonhosted.org/packages/49/f7/0dd1368e531e52a17b5b8dd2f379cce813bff2d0978a7748a506f1231152/mmh3-5.2.0-cp310-cp310-manylinux2014_aarch64.manylinux_2_17_aarch64.manylinux_2_28_aarch64.whl", hash = "sha256:7434a27754049144539d2099a6d2da5d88b8bdeedf935180bf42ad59b3607aa3", size = 103991, upload-time = "2025-07-29T07:41:44.914Z" }, + { url = "https://files.pythonhosted.org/packages/35/06/abc7122c40f4abbfcef01d2dac6ec0b77ede9757e5be8b8a40a6265b1274/mmh3-5.2.0-cp310-cp310-manylinux2014_ppc64le.manylinux_2_17_ppc64le.manylinux_2_28_ppc64le.whl", hash = "sha256:cadc16e8ea64b5d9a47363013e2bea469e121e6e7cb416a7593aeb24f2ad122e", size = 110894, upload-time = "2025-07-29T07:41:45.849Z" }, + { url = "https://files.pythonhosted.org/packages/f4/2f/837885759afa4baccb8e40456e1cf76a4f3eac835b878c727ae1286c5f82/mmh3-5.2.0-cp310-cp310-manylinux2014_s390x.manylinux_2_17_s390x.manylinux_2_28_s390x.whl", hash = "sha256:d765058da196f68dc721116cab335e696e87e76720e6ef8ee5a24801af65e63d", size = 118327, upload-time = "2025-07-29T07:41:47.224Z" }, + { url = "https://files.pythonhosted.org/packages/40/cc/5683ba20a21bcfb3f1605b1c474f46d30354f728a7412201f59f453d405a/mmh3-5.2.0-cp310-cp310-musllinux_1_2_aarch64.whl", hash = "sha256:8b0c53fe0994beade1ad7c0f13bd6fec980a0664bfbe5a6a7d64500b9ab76772", size = 101701, upload-time = "2025-07-29T07:41:48.259Z" }, + { url = "https://files.pythonhosted.org/packages/0e/24/99ab3fb940150aec8a26dbdfc39b200b5592f6aeb293ec268df93e054c30/mmh3-5.2.0-cp310-cp310-musllinux_1_2_i686.whl", hash = "sha256:49037d417419863b222ae47ee562b2de9c3416add0a45c8d7f4e864be8dc4f89", size = 96712, upload-time = "2025-07-29T07:41:49.467Z" }, + { url = "https://files.pythonhosted.org/packages/61/04/d7c4cb18f1f001ede2e8aed0f9dbbfad03d161c9eea4fffb03f14f4523e5/mmh3-5.2.0-cp310-cp310-musllinux_1_2_ppc64le.whl", hash = "sha256:6ecb4e750d712abde046858ee6992b65c93f1f71b397fce7975c3860c07365d2", size = 110302, upload-time = "2025-07-29T07:41:50.387Z" }, + { url = "https://files.pythonhosted.org/packages/d8/bf/4dac37580cfda74425a4547500c36fa13ef581c8a756727c37af45e11e9a/mmh3-5.2.0-cp310-cp310-musllinux_1_2_s390x.whl", hash = "sha256:382a6bb3f8c6532ea084e7acc5be6ae0c6effa529240836d59352398f002e3fc", size = 111929, upload-time = "2025-07-29T07:41:51.348Z" }, + { url = "https://files.pythonhosted.org/packages/eb/b1/49f0a582c7a942fb71ddd1ec52b7d21d2544b37d2b2d994551346a15b4f6/mmh3-5.2.0-cp310-cp310-musllinux_1_2_x86_64.whl", hash = "sha256:7733ec52296fc1ba22e9b90a245c821adbb943e98c91d8a330a2254612726106", size = 100111, upload-time = "2025-07-29T07:41:53.139Z" }, + { url = "https://files.pythonhosted.org/packages/dc/94/ccec09f438caeb2506f4c63bb3b99aa08a9e09880f8fc047295154756210/mmh3-5.2.0-cp310-cp310-win32.whl", hash = "sha256:127c95336f2a98c51e7682341ab7cb0be3adb9df0819ab8505a726ed1801876d", size = 40783, upload-time = "2025-07-29T07:41:54.463Z" }, + { url = "https://files.pythonhosted.org/packages/ea/f4/8d39a32c8203c1cdae88fdb04d1ea4aa178c20f159df97f4c5a2eaec702c/mmh3-5.2.0-cp310-cp310-win_amd64.whl", hash = "sha256:419005f84ba1cab47a77465a2a843562dadadd6671b8758bf179d82a15ca63eb", size = 41549, upload-time = "2025-07-29T07:41:55.295Z" }, + { url = "https://files.pythonhosted.org/packages/cc/a1/30efb1cd945e193f62574144dd92a0c9ee6463435e4e8ffce9b9e9f032f0/mmh3-5.2.0-cp310-cp310-win_arm64.whl", hash = "sha256:d22c9dcafed659fadc605538946c041722b6d1104fe619dbf5cc73b3c8a0ded8", size = 39335, upload-time = "2025-07-29T07:41:56.194Z" }, + { url = "https://files.pythonhosted.org/packages/f7/87/399567b3796e134352e11a8b973cd470c06b2ecfad5468fe580833be442b/mmh3-5.2.0-cp311-cp311-macosx_10_9_universal2.whl", hash = "sha256:7901c893e704ee3c65f92d39b951f8f34ccf8e8566768c58103fb10e55afb8c1", size = 56107, upload-time = "2025-07-29T07:41:57.07Z" }, + { url = "https://files.pythonhosted.org/packages/c3/09/830af30adf8678955b247d97d3d9543dd2fd95684f3cd41c0cd9d291da9f/mmh3-5.2.0-cp311-cp311-macosx_10_9_x86_64.whl", hash = "sha256:4a5f5536b1cbfa72318ab3bfc8a8188b949260baed186b75f0abc75b95d8c051", size = 40635, upload-time = "2025-07-29T07:41:57.903Z" }, + { url = "https://files.pythonhosted.org/packages/07/14/eaba79eef55b40d653321765ac5e8f6c9ac38780b8a7c2a2f8df8ee0fb72/mmh3-5.2.0-cp311-cp311-macosx_11_0_arm64.whl", hash = "sha256:cedac4f4054b8f7859e5aed41aaa31ad03fce6851901a7fdc2af0275ac533c10", size = 40078, upload-time = "2025-07-29T07:41:58.772Z" }, + { url = "https://files.pythonhosted.org/packages/bb/26/83a0f852e763f81b2265d446b13ed6d49ee49e1fc0c47b9655977e6f3d81/mmh3-5.2.0-cp311-cp311-manylinux1_i686.manylinux_2_28_i686.manylinux_2_5_i686.whl", hash = "sha256:eb756caf8975882630ce4e9fbbeb9d3401242a72528230422c9ab3a0d278e60c", size = 97262, upload-time = "2025-07-29T07:41:59.678Z" }, + { url = "https://files.pythonhosted.org/packages/00/7d/b7133b10d12239aeaebf6878d7eaf0bf7d3738c44b4aba3c564588f6d802/mmh3-5.2.0-cp311-cp311-manylinux1_x86_64.manylinux_2_28_x86_64.manylinux_2_5_x86_64.whl", hash = "sha256:097e13c8b8a66c5753c6968b7640faefe85d8e38992703c1f666eda6ef4c3762", size = 103118, upload-time = "2025-07-29T07:42:01.197Z" }, + { url = "https://files.pythonhosted.org/packages/7b/3e/62f0b5dce2e22fd5b7d092aba285abd7959ea2b17148641e029f2eab1ffa/mmh3-5.2.0-cp311-cp311-manylinux2014_aarch64.manylinux_2_17_aarch64.manylinux_2_28_aarch64.whl", hash = "sha256:a7c0c7845566b9686480e6a7e9044db4afb60038d5fabd19227443f0104eeee4", size = 106072, upload-time = "2025-07-29T07:42:02.601Z" }, + { url = "https://files.pythonhosted.org/packages/66/84/ea88bb816edfe65052c757a1c3408d65c4201ddbd769d4a287b0f1a628b2/mmh3-5.2.0-cp311-cp311-manylinux2014_ppc64le.manylinux_2_17_ppc64le.manylinux_2_28_ppc64le.whl", hash = "sha256:61ac226af521a572700f863d6ecddc6ece97220ce7174e311948ff8c8919a363", size = 112925, upload-time = "2025-07-29T07:42:03.632Z" }, + { url = "https://files.pythonhosted.org/packages/2e/13/c9b1c022807db575fe4db806f442d5b5784547e2e82cff36133e58ea31c7/mmh3-5.2.0-cp311-cp311-manylinux2014_s390x.manylinux_2_17_s390x.manylinux_2_28_s390x.whl", hash = "sha256:582f9dbeefe15c32a5fa528b79b088b599a1dfe290a4436351c6090f90ddebb8", size = 120583, upload-time = "2025-07-29T07:42:04.991Z" }, + { url = "https://files.pythonhosted.org/packages/8a/5f/0e2dfe1a38f6a78788b7eb2b23432cee24623aeabbc907fed07fc17d6935/mmh3-5.2.0-cp311-cp311-musllinux_1_2_aarch64.whl", hash = "sha256:2ebfc46b39168ab1cd44670a32ea5489bcbc74a25795c61b6d888c5c2cf654ed", size = 99127, upload-time = "2025-07-29T07:42:05.929Z" }, + { url = "https://files.pythonhosted.org/packages/77/27/aefb7d663b67e6a0c4d61a513c83e39ba2237e8e4557fa7122a742a23de5/mmh3-5.2.0-cp311-cp311-musllinux_1_2_i686.whl", hash = "sha256:1556e31e4bd0ac0c17eaf220be17a09c171d7396919c3794274cb3415a9d3646", size = 98544, upload-time = "2025-07-29T07:42:06.87Z" }, + { url = "https://files.pythonhosted.org/packages/ab/97/a21cc9b1a7c6e92205a1b5fa030cdf62277d177570c06a239eca7bd6dd32/mmh3-5.2.0-cp311-cp311-musllinux_1_2_ppc64le.whl", hash = "sha256:81df0dae22cd0da87f1c978602750f33d17fb3d21fb0f326c89dc89834fea79b", size = 106262, upload-time = "2025-07-29T07:42:07.804Z" }, + { url = "https://files.pythonhosted.org/packages/43/18/db19ae82ea63c8922a880e1498a75342311f8aa0c581c4dd07711473b5f7/mmh3-5.2.0-cp311-cp311-musllinux_1_2_s390x.whl", hash = "sha256:eba01ec3bd4a49b9ac5ca2bc6a73ff5f3af53374b8556fcc2966dd2af9eb7779", size = 109824, upload-time = "2025-07-29T07:42:08.735Z" }, + { url = "https://files.pythonhosted.org/packages/9f/f5/41dcf0d1969125fc6f61d8618b107c79130b5af50b18a4651210ea52ab40/mmh3-5.2.0-cp311-cp311-musllinux_1_2_x86_64.whl", hash = "sha256:e9a011469b47b752e7d20de296bb34591cdfcbe76c99c2e863ceaa2aa61113d2", size = 97255, upload-time = "2025-07-29T07:42:09.706Z" }, + { url = "https://files.pythonhosted.org/packages/32/b3/cce9eaa0efac1f0e735bb178ef9d1d2887b4927fe0ec16609d5acd492dda/mmh3-5.2.0-cp311-cp311-win32.whl", hash = "sha256:bc44fc2b886243d7c0d8daeb37864e16f232e5b56aaec27cc781d848264cfd28", size = 40779, upload-time = "2025-07-29T07:42:10.546Z" }, + { url = "https://files.pythonhosted.org/packages/7c/e9/3fa0290122e6d5a7041b50ae500b8a9f4932478a51e48f209a3879fe0b9b/mmh3-5.2.0-cp311-cp311-win_amd64.whl", hash = "sha256:8ebf241072cf2777a492d0e09252f8cc2b3edd07dfdb9404b9757bffeb4f2cee", size = 41549, upload-time = "2025-07-29T07:42:11.399Z" }, + { url = "https://files.pythonhosted.org/packages/3a/54/c277475b4102588e6f06b2e9095ee758dfe31a149312cdbf62d39a9f5c30/mmh3-5.2.0-cp311-cp311-win_arm64.whl", hash = "sha256:b5f317a727bba0e633a12e71228bc6a4acb4f471a98b1c003163b917311ea9a9", size = 39336, upload-time = "2025-07-29T07:42:12.209Z" }, + { url = "https://files.pythonhosted.org/packages/bf/6a/d5aa7edb5c08e0bd24286c7d08341a0446f9a2fbbb97d96a8a6dd81935ee/mmh3-5.2.0-cp312-cp312-macosx_10_13_universal2.whl", hash = "sha256:384eda9361a7bf83a85e09447e1feafe081034af9dd428893701b959230d84be", size = 56141, upload-time = "2025-07-29T07:42:13.456Z" }, + { url = "https://files.pythonhosted.org/packages/08/49/131d0fae6447bc4a7299ebdb1a6fb9d08c9f8dcf97d75ea93e8152ddf7ab/mmh3-5.2.0-cp312-cp312-macosx_10_13_x86_64.whl", hash = "sha256:2c9da0d568569cc87315cb063486d761e38458b8ad513fedd3dc9263e1b81bcd", size = 40681, upload-time = "2025-07-29T07:42:14.306Z" }, + { url = "https://files.pythonhosted.org/packages/8f/6f/9221445a6bcc962b7f5ff3ba18ad55bba624bacdc7aa3fc0a518db7da8ec/mmh3-5.2.0-cp312-cp312-macosx_11_0_arm64.whl", hash = "sha256:86d1be5d63232e6eb93c50881aea55ff06eb86d8e08f9b5417c8c9b10db9db96", size = 40062, upload-time = "2025-07-29T07:42:15.08Z" }, + { url = "https://files.pythonhosted.org/packages/1e/d4/6bb2d0fef81401e0bb4c297d1eb568b767de4ce6fc00890bc14d7b51ecc4/mmh3-5.2.0-cp312-cp312-manylinux1_i686.manylinux_2_28_i686.manylinux_2_5_i686.whl", hash = "sha256:bf7bee43e17e81671c447e9c83499f53d99bf440bc6d9dc26a841e21acfbe094", size = 97333, upload-time = "2025-07-29T07:42:16.436Z" }, + { url = "https://files.pythonhosted.org/packages/44/e0/ccf0daff8134efbb4fbc10a945ab53302e358c4b016ada9bf97a6bdd50c1/mmh3-5.2.0-cp312-cp312-manylinux1_x86_64.manylinux_2_28_x86_64.manylinux_2_5_x86_64.whl", hash = "sha256:7aa18cdb58983ee660c9c400b46272e14fa253c675ed963d3812487f8ca42037", size = 103310, upload-time = "2025-07-29T07:42:17.796Z" }, + { url = "https://files.pythonhosted.org/packages/02/63/1965cb08a46533faca0e420e06aff8bbaf9690a6f0ac6ae6e5b2e4544687/mmh3-5.2.0-cp312-cp312-manylinux2014_aarch64.manylinux_2_17_aarch64.manylinux_2_28_aarch64.whl", hash = "sha256:ae9d032488fcec32d22be6542d1a836f00247f40f320844dbb361393b5b22773", size = 106178, upload-time = "2025-07-29T07:42:19.281Z" }, + { url = "https://files.pythonhosted.org/packages/c2/41/c883ad8e2c234013f27f92061200afc11554ea55edd1bcf5e1accd803a85/mmh3-5.2.0-cp312-cp312-manylinux2014_ppc64le.manylinux_2_17_ppc64le.manylinux_2_28_ppc64le.whl", hash = "sha256:e1861fb6b1d0453ed7293200139c0a9011eeb1376632e048e3766945b13313c5", size = 113035, upload-time = "2025-07-29T07:42:20.356Z" }, + { url = "https://files.pythonhosted.org/packages/df/b5/1ccade8b1fa625d634a18bab7bf08a87457e09d5ec8cf83ca07cbea9d400/mmh3-5.2.0-cp312-cp312-manylinux2014_s390x.manylinux_2_17_s390x.manylinux_2_28_s390x.whl", hash = "sha256:99bb6a4d809aa4e528ddfe2c85dd5239b78b9dd14be62cca0329db78505e7b50", size = 120784, upload-time = "2025-07-29T07:42:21.377Z" }, + { url = "https://files.pythonhosted.org/packages/77/1c/919d9171fcbdcdab242e06394464ccf546f7d0f3b31e0d1e3a630398782e/mmh3-5.2.0-cp312-cp312-musllinux_1_2_aarch64.whl", hash = "sha256:1f8d8b627799f4e2fcc7c034fed8f5f24dc7724ff52f69838a3d6d15f1ad4765", size = 99137, upload-time = "2025-07-29T07:42:22.344Z" }, + { url = "https://files.pythonhosted.org/packages/66/8a/1eebef5bd6633d36281d9fc83cf2e9ba1ba0e1a77dff92aacab83001cee4/mmh3-5.2.0-cp312-cp312-musllinux_1_2_i686.whl", hash = "sha256:b5995088dd7023d2d9f310a0c67de5a2b2e06a570ecfd00f9ff4ab94a67cde43", size = 98664, upload-time = "2025-07-29T07:42:23.269Z" }, + { url = "https://files.pythonhosted.org/packages/13/41/a5d981563e2ee682b21fb65e29cc0f517a6734a02b581359edd67f9d0360/mmh3-5.2.0-cp312-cp312-musllinux_1_2_ppc64le.whl", hash = "sha256:1a5f4d2e59d6bba8ef01b013c472741835ad961e7c28f50c82b27c57748744a4", size = 106459, upload-time = "2025-07-29T07:42:24.238Z" }, + { url = "https://files.pythonhosted.org/packages/24/31/342494cd6ab792d81e083680875a2c50fa0c5df475ebf0b67784f13e4647/mmh3-5.2.0-cp312-cp312-musllinux_1_2_s390x.whl", hash = "sha256:fd6e6c3d90660d085f7e73710eab6f5545d4854b81b0135a3526e797009dbda3", size = 110038, upload-time = "2025-07-29T07:42:25.629Z" }, + { url = "https://files.pythonhosted.org/packages/28/44/efda282170a46bb4f19c3e2b90536513b1d821c414c28469a227ca5a1789/mmh3-5.2.0-cp312-cp312-musllinux_1_2_x86_64.whl", hash = "sha256:c4a2f3d83879e3de2eb8cbf562e71563a8ed15ee9b9c2e77ca5d9f73072ac15c", size = 97545, upload-time = "2025-07-29T07:42:27.04Z" }, + { url = "https://files.pythonhosted.org/packages/68/8f/534ae319c6e05d714f437e7206f78c17e66daca88164dff70286b0e8ea0c/mmh3-5.2.0-cp312-cp312-win32.whl", hash = "sha256:2421b9d665a0b1ad724ec7332fb5a98d075f50bc51a6ff854f3a1882bd650d49", size = 40805, upload-time = "2025-07-29T07:42:28.032Z" }, + { url = "https://files.pythonhosted.org/packages/b8/f6/f6abdcfefcedab3c964868048cfe472764ed358c2bf6819a70dd4ed4ed3a/mmh3-5.2.0-cp312-cp312-win_amd64.whl", hash = "sha256:72d80005b7634a3a2220f81fbeb94775ebd12794623bb2e1451701ea732b4aa3", size = 41597, upload-time = "2025-07-29T07:42:28.894Z" }, + { url = "https://files.pythonhosted.org/packages/15/fd/f7420e8cbce45c259c770cac5718badf907b302d3a99ec587ba5ce030237/mmh3-5.2.0-cp312-cp312-win_arm64.whl", hash = "sha256:3d6bfd9662a20c054bc216f861fa330c2dac7c81e7fb8307b5e32ab5b9b4d2e0", size = 39350, upload-time = "2025-07-29T07:42:29.794Z" }, +] + +[[package]] +name = "packaging" +version = "25.0" +source = { registry = "https://pypi.org/simple" } +sdist = { url = "https://files.pythonhosted.org/packages/a1/d4/1fc4078c65507b51b96ca8f8c3ba19e6a61c8253c72794544580a7b6c24d/packaging-25.0.tar.gz", hash = "sha256:d443872c98d677bf60f6a1f2f8c1cb748e8fe762d2bf9d3148b5599295b0fc4f", size = 165727, upload-time = "2025-04-19T11:48:59.673Z" } +wheels = [ + { url = "https://files.pythonhosted.org/packages/20/12/38679034af332785aac8774540895e234f4d07f7545804097de4b666afd8/packaging-25.0-py3-none-any.whl", hash = "sha256:29572ef2b1f17581046b3a2227d5c611fb25ec70ca1ba8554b24b0e69331a484", size = 66469, upload-time = "2025-04-19T11:48:57.875Z" }, +] + +[[package]] +name = "pluggy" +version = "1.6.0" +source = { registry = "https://pypi.org/simple" } +sdist = { url = "https://files.pythonhosted.org/packages/f9/e2/3e91f31a7d2b083fe6ef3fa267035b518369d9511ffab804f839851d2779/pluggy-1.6.0.tar.gz", hash = "sha256:7dcc130b76258d33b90f61b658791dede3486c3e6bfb003ee5c9bfb396dd22f3", size = 69412, upload-time = "2025-05-15T12:30:07.975Z" } +wheels = [ + { url = "https://files.pythonhosted.org/packages/54/20/4d324d65cc6d9205fabedc306948156824eb9f0ee1633355a8f7ec5c66bf/pluggy-1.6.0-py3-none-any.whl", hash = "sha256:e920276dd6813095e9377c0bc5566d94c932c33b27a3e3945d8389c374dd4746", size = 20538, upload-time = "2025-05-15T12:30:06.134Z" }, +] + +[[package]] +name = "pyarrow" +version = "22.0.0" +source = { registry = "https://pypi.org/simple" } +sdist = { url = "https://files.pythonhosted.org/packages/30/53/04a7fdc63e6056116c9ddc8b43bc28c12cdd181b85cbeadb79278475f3ae/pyarrow-22.0.0.tar.gz", hash = "sha256:3d600dc583260d845c7d8a6db540339dd883081925da2bd1c5cb808f720b3cd9", size = 1151151, upload-time = "2025-10-24T12:30:00.762Z" } +wheels = [ + { url = "https://files.pythonhosted.org/packages/d9/9b/cb3f7e0a345353def531ca879053e9ef6b9f38ed91aebcf68b09ba54dec0/pyarrow-22.0.0-cp310-cp310-macosx_12_0_arm64.whl", hash = "sha256:77718810bd3066158db1e95a63c160ad7ce08c6b0710bc656055033e39cdad88", size = 34223968, upload-time = "2025-10-24T10:03:31.21Z" }, + { url = "https://files.pythonhosted.org/packages/6c/41/3184b8192a120306270c5307f105b70320fdaa592c99843c5ef78aaefdcf/pyarrow-22.0.0-cp310-cp310-macosx_12_0_x86_64.whl", hash = "sha256:44d2d26cda26d18f7af7db71453b7b783788322d756e81730acb98f24eb90ace", size = 35942085, upload-time = "2025-10-24T10:03:38.146Z" }, + { url = "https://files.pythonhosted.org/packages/d9/3d/a1eab2f6f08001f9fb714b8ed5cfb045e2fe3e3e3c0c221f2c9ed1e6d67d/pyarrow-22.0.0-cp310-cp310-manylinux_2_28_aarch64.whl", hash = "sha256:b9d71701ce97c95480fecb0039ec5bb889e75f110da72005743451339262f4ce", size = 44964613, upload-time = "2025-10-24T10:03:46.516Z" }, + { url = "https://files.pythonhosted.org/packages/46/46/a1d9c24baf21cfd9ce994ac820a24608decf2710521b29223d4334985127/pyarrow-22.0.0-cp310-cp310-manylinux_2_28_x86_64.whl", hash = "sha256:710624ab925dc2b05a6229d47f6f0dac1c1155e6ed559be7109f684eba048a48", size = 47627059, upload-time = "2025-10-24T10:03:55.353Z" }, + { url = "https://files.pythonhosted.org/packages/3a/4c/f711acb13075c1391fd54bc17e078587672c575f8de2a6e62509af026dcf/pyarrow-22.0.0-cp310-cp310-musllinux_1_2_aarch64.whl", hash = "sha256:f963ba8c3b0199f9d6b794c90ec77545e05eadc83973897a4523c9e8d84e9340", size = 47947043, upload-time = "2025-10-24T10:04:05.408Z" }, + { url = "https://files.pythonhosted.org/packages/4e/70/1f3180dd7c2eab35c2aca2b29ace6c519f827dcd4cfeb8e0dca41612cf7a/pyarrow-22.0.0-cp310-cp310-musllinux_1_2_x86_64.whl", hash = "sha256:bd0d42297ace400d8febe55f13fdf46e86754842b860c978dfec16f081e5c653", size = 50206505, upload-time = "2025-10-24T10:04:15.786Z" }, + { url = "https://files.pythonhosted.org/packages/80/07/fea6578112c8c60ffde55883a571e4c4c6bc7049f119d6b09333b5cc6f73/pyarrow-22.0.0-cp310-cp310-win_amd64.whl", hash = "sha256:00626d9dc0f5ef3a75fe63fd68b9c7c8302d2b5bbc7f74ecaedba83447a24f84", size = 28101641, upload-time = "2025-10-24T10:04:22.57Z" }, + { url = "https://files.pythonhosted.org/packages/2e/b7/18f611a8cdc43417f9394a3ccd3eace2f32183c08b9eddc3d17681819f37/pyarrow-22.0.0-cp311-cp311-macosx_12_0_arm64.whl", hash = "sha256:3e294c5eadfb93d78b0763e859a0c16d4051fc1c5231ae8956d61cb0b5666f5a", size = 34272022, upload-time = "2025-10-24T10:04:28.973Z" }, + { url = "https://files.pythonhosted.org/packages/26/5c/f259e2526c67eb4b9e511741b19870a02363a47a35edbebc55c3178db22d/pyarrow-22.0.0-cp311-cp311-macosx_12_0_x86_64.whl", hash = "sha256:69763ab2445f632d90b504a815a2a033f74332997052b721002298ed6de40f2e", size = 35995834, upload-time = "2025-10-24T10:04:35.467Z" }, + { url = "https://files.pythonhosted.org/packages/50/8d/281f0f9b9376d4b7f146913b26fac0aa2829cd1ee7e997f53a27411bbb92/pyarrow-22.0.0-cp311-cp311-manylinux_2_28_aarch64.whl", hash = "sha256:b41f37cabfe2463232684de44bad753d6be08a7a072f6a83447eeaf0e4d2a215", size = 45030348, upload-time = "2025-10-24T10:04:43.366Z" }, + { url = "https://files.pythonhosted.org/packages/f5/e5/53c0a1c428f0976bf22f513d79c73000926cb00b9c138d8e02daf2102e18/pyarrow-22.0.0-cp311-cp311-manylinux_2_28_x86_64.whl", hash = "sha256:35ad0f0378c9359b3f297299c3309778bb03b8612f987399a0333a560b43862d", size = 47699480, upload-time = "2025-10-24T10:04:51.486Z" }, + { url = "https://files.pythonhosted.org/packages/95/e1/9dbe4c465c3365959d183e6345d0a8d1dc5b02ca3f8db4760b3bc834cf25/pyarrow-22.0.0-cp311-cp311-musllinux_1_2_aarch64.whl", hash = "sha256:8382ad21458075c2e66a82a29d650f963ce51c7708c7c0ff313a8c206c4fd5e8", size = 48011148, upload-time = "2025-10-24T10:04:59.585Z" }, + { url = "https://files.pythonhosted.org/packages/c5/b4/7caf5d21930061444c3cf4fa7535c82faf5263e22ce43af7c2759ceb5b8b/pyarrow-22.0.0-cp311-cp311-musllinux_1_2_x86_64.whl", hash = "sha256:1a812a5b727bc09c3d7ea072c4eebf657c2f7066155506ba31ebf4792f88f016", size = 50276964, upload-time = "2025-10-24T10:05:08.175Z" }, + { url = "https://files.pythonhosted.org/packages/ae/f3/cec89bd99fa3abf826f14d4e53d3d11340ce6f6af4d14bdcd54cd83b6576/pyarrow-22.0.0-cp311-cp311-win_amd64.whl", hash = "sha256:ec5d40dd494882704fb876c16fa7261a69791e784ae34e6b5992e977bd2e238c", size = 28106517, upload-time = "2025-10-24T10:05:14.314Z" }, + { url = "https://files.pythonhosted.org/packages/af/63/ba23862d69652f85b615ca14ad14f3bcfc5bf1b99ef3f0cd04ff93fdad5a/pyarrow-22.0.0-cp312-cp312-macosx_12_0_arm64.whl", hash = "sha256:bea79263d55c24a32b0d79c00a1c58bb2ee5f0757ed95656b01c0fb310c5af3d", size = 34211578, upload-time = "2025-10-24T10:05:21.583Z" }, + { url = "https://files.pythonhosted.org/packages/b1/d0/f9ad86fe809efd2bcc8be32032fa72e8b0d112b01ae56a053006376c5930/pyarrow-22.0.0-cp312-cp312-macosx_12_0_x86_64.whl", hash = "sha256:12fe549c9b10ac98c91cf791d2945e878875d95508e1a5d14091a7aaa66d9cf8", size = 35989906, upload-time = "2025-10-24T10:05:29.485Z" }, + { url = "https://files.pythonhosted.org/packages/b4/a8/f910afcb14630e64d673f15904ec27dd31f1e009b77033c365c84e8c1e1d/pyarrow-22.0.0-cp312-cp312-manylinux_2_28_aarch64.whl", hash = "sha256:334f900ff08ce0423407af97e6c26ad5d4e3b0763645559ece6fbf3747d6a8f5", size = 45021677, upload-time = "2025-10-24T10:05:38.274Z" }, + { url = "https://files.pythonhosted.org/packages/13/95/aec81f781c75cd10554dc17a25849c720d54feafb6f7847690478dcf5ef8/pyarrow-22.0.0-cp312-cp312-manylinux_2_28_x86_64.whl", hash = "sha256:c6c791b09c57ed76a18b03f2631753a4960eefbbca80f846da8baefc6491fcfe", size = 47726315, upload-time = "2025-10-24T10:05:47.314Z" }, + { url = "https://files.pythonhosted.org/packages/bb/d4/74ac9f7a54cfde12ee42734ea25d5a3c9a45db78f9def949307a92720d37/pyarrow-22.0.0-cp312-cp312-musllinux_1_2_aarch64.whl", hash = "sha256:c3200cb41cdbc65156e5f8c908d739b0dfed57e890329413da2748d1a2cd1a4e", size = 47990906, upload-time = "2025-10-24T10:05:58.254Z" }, + { url = "https://files.pythonhosted.org/packages/2e/71/fedf2499bf7a95062eafc989ace56572f3343432570e1c54e6599d5b88da/pyarrow-22.0.0-cp312-cp312-musllinux_1_2_x86_64.whl", hash = "sha256:ac93252226cf288753d8b46280f4edf3433bf9508b6977f8dd8526b521a1bbb9", size = 50306783, upload-time = "2025-10-24T10:06:08.08Z" }, + { url = "https://files.pythonhosted.org/packages/68/ed/b202abd5a5b78f519722f3d29063dda03c114711093c1995a33b8e2e0f4b/pyarrow-22.0.0-cp312-cp312-win_amd64.whl", hash = "sha256:44729980b6c50a5f2bfcc2668d36c569ce17f8b17bccaf470c4313dcbbf13c9d", size = 27972883, upload-time = "2025-10-24T10:06:14.204Z" }, +] + +[[package]] +name = "pydantic" +version = "2.11.10" +source = { registry = "https://pypi.org/simple" } +dependencies = [ + { name = "annotated-types" }, + { name = "pydantic-core" }, + { name = "typing-extensions" }, + { name = "typing-inspection" }, +] +sdist = { url = "https://files.pythonhosted.org/packages/ae/54/ecab642b3bed45f7d5f59b38443dcb36ef50f85af192e6ece103dbfe9587/pydantic-2.11.10.tar.gz", hash = "sha256:dc280f0982fbda6c38fada4e476dc0a4f3aeaf9c6ad4c28df68a666ec3c61423", size = 788494, upload-time = "2025-10-04T10:40:41.338Z" } +wheels = [ + { url = "https://files.pythonhosted.org/packages/bd/1f/73c53fcbfb0b5a78f91176df41945ca466e71e9d9d836e5c522abda39ee7/pydantic-2.11.10-py3-none-any.whl", hash = "sha256:802a655709d49bd004c31e865ef37da30b540786a46bfce02333e0e24b5fe29a", size = 444823, upload-time = "2025-10-04T10:40:39.055Z" }, +] + +[[package]] +name = "pydantic-core" +version = "2.33.2" +source = { registry = "https://pypi.org/simple" } +dependencies = [ + { name = "typing-extensions" }, +] +sdist = { url = "https://files.pythonhosted.org/packages/ad/88/5f2260bdfae97aabf98f1778d43f69574390ad787afb646292a638c923d4/pydantic_core-2.33.2.tar.gz", hash = "sha256:7cb8bc3605c29176e1b105350d2e6474142d7c1bd1d9327c4a9bdb46bf827acc", size = 435195, upload-time = "2025-04-23T18:33:52.104Z" } +wheels = [ + { url = "https://files.pythonhosted.org/packages/e5/92/b31726561b5dae176c2d2c2dc43a9c5bfba5d32f96f8b4c0a600dd492447/pydantic_core-2.33.2-cp310-cp310-macosx_10_12_x86_64.whl", hash = "sha256:2b3d326aaef0c0399d9afffeb6367d5e26ddc24d351dbc9c636840ac355dc5d8", size = 2028817, upload-time = "2025-04-23T18:30:43.919Z" }, + { url = "https://files.pythonhosted.org/packages/a3/44/3f0b95fafdaca04a483c4e685fe437c6891001bf3ce8b2fded82b9ea3aa1/pydantic_core-2.33.2-cp310-cp310-macosx_11_0_arm64.whl", hash = "sha256:0e5b2671f05ba48b94cb90ce55d8bdcaaedb8ba00cc5359f6810fc918713983d", size = 1861357, upload-time = "2025-04-23T18:30:46.372Z" }, + { url = "https://files.pythonhosted.org/packages/30/97/e8f13b55766234caae05372826e8e4b3b96e7b248be3157f53237682e43c/pydantic_core-2.33.2-cp310-cp310-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:0069c9acc3f3981b9ff4cdfaf088e98d83440a4c7ea1bc07460af3d4dc22e72d", size = 1898011, upload-time = "2025-04-23T18:30:47.591Z" }, + { url = "https://files.pythonhosted.org/packages/9b/a3/99c48cf7bafc991cc3ee66fd544c0aae8dc907b752f1dad2d79b1b5a471f/pydantic_core-2.33.2-cp310-cp310-manylinux_2_17_armv7l.manylinux2014_armv7l.whl", hash = "sha256:d53b22f2032c42eaaf025f7c40c2e3b94568ae077a606f006d206a463bc69572", size = 1982730, upload-time = "2025-04-23T18:30:49.328Z" }, + { url = "https://files.pythonhosted.org/packages/de/8e/a5b882ec4307010a840fb8b58bd9bf65d1840c92eae7534c7441709bf54b/pydantic_core-2.33.2-cp310-cp310-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:0405262705a123b7ce9f0b92f123334d67b70fd1f20a9372b907ce1080c7ba02", size = 2136178, upload-time = "2025-04-23T18:30:50.907Z" }, + { url = "https://files.pythonhosted.org/packages/e4/bb/71e35fc3ed05af6834e890edb75968e2802fe98778971ab5cba20a162315/pydantic_core-2.33.2-cp310-cp310-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:4b25d91e288e2c4e0662b8038a28c6a07eaac3e196cfc4ff69de4ea3db992a1b", size = 2736462, upload-time = "2025-04-23T18:30:52.083Z" }, + { url = "https://files.pythonhosted.org/packages/31/0d/c8f7593e6bc7066289bbc366f2235701dcbebcd1ff0ef8e64f6f239fb47d/pydantic_core-2.33.2-cp310-cp310-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:6bdfe4b3789761f3bcb4b1ddf33355a71079858958e3a552f16d5af19768fef2", size = 2005652, upload-time = "2025-04-23T18:30:53.389Z" }, + { url = "https://files.pythonhosted.org/packages/d2/7a/996d8bd75f3eda405e3dd219ff5ff0a283cd8e34add39d8ef9157e722867/pydantic_core-2.33.2-cp310-cp310-manylinux_2_5_i686.manylinux1_i686.whl", hash = "sha256:efec8db3266b76ef9607c2c4c419bdb06bf335ae433b80816089ea7585816f6a", size = 2113306, upload-time = "2025-04-23T18:30:54.661Z" }, + { url = "https://files.pythonhosted.org/packages/ff/84/daf2a6fb2db40ffda6578a7e8c5a6e9c8affb251a05c233ae37098118788/pydantic_core-2.33.2-cp310-cp310-musllinux_1_1_aarch64.whl", hash = "sha256:031c57d67ca86902726e0fae2214ce6770bbe2f710dc33063187a68744a5ecac", size = 2073720, upload-time = "2025-04-23T18:30:56.11Z" }, + { url = "https://files.pythonhosted.org/packages/77/fb/2258da019f4825128445ae79456a5499c032b55849dbd5bed78c95ccf163/pydantic_core-2.33.2-cp310-cp310-musllinux_1_1_armv7l.whl", hash = "sha256:f8de619080e944347f5f20de29a975c2d815d9ddd8be9b9b7268e2e3ef68605a", size = 2244915, upload-time = "2025-04-23T18:30:57.501Z" }, + { url = "https://files.pythonhosted.org/packages/d8/7a/925ff73756031289468326e355b6fa8316960d0d65f8b5d6b3a3e7866de7/pydantic_core-2.33.2-cp310-cp310-musllinux_1_1_x86_64.whl", hash = "sha256:73662edf539e72a9440129f231ed3757faab89630d291b784ca99237fb94db2b", size = 2241884, upload-time = "2025-04-23T18:30:58.867Z" }, + { url = "https://files.pythonhosted.org/packages/0b/b0/249ee6d2646f1cdadcb813805fe76265745c4010cf20a8eba7b0e639d9b2/pydantic_core-2.33.2-cp310-cp310-win32.whl", hash = "sha256:0a39979dcbb70998b0e505fb1556a1d550a0781463ce84ebf915ba293ccb7e22", size = 1910496, upload-time = "2025-04-23T18:31:00.078Z" }, + { url = "https://files.pythonhosted.org/packages/66/ff/172ba8f12a42d4b552917aa65d1f2328990d3ccfc01d5b7c943ec084299f/pydantic_core-2.33.2-cp310-cp310-win_amd64.whl", hash = "sha256:b0379a2b24882fef529ec3b4987cb5d003b9cda32256024e6fe1586ac45fc640", size = 1955019, upload-time = "2025-04-23T18:31:01.335Z" }, + { url = "https://files.pythonhosted.org/packages/3f/8d/71db63483d518cbbf290261a1fc2839d17ff89fce7089e08cad07ccfce67/pydantic_core-2.33.2-cp311-cp311-macosx_10_12_x86_64.whl", hash = "sha256:4c5b0a576fb381edd6d27f0a85915c6daf2f8138dc5c267a57c08a62900758c7", size = 2028584, upload-time = "2025-04-23T18:31:03.106Z" }, + { url = "https://files.pythonhosted.org/packages/24/2f/3cfa7244ae292dd850989f328722d2aef313f74ffc471184dc509e1e4e5a/pydantic_core-2.33.2-cp311-cp311-macosx_11_0_arm64.whl", hash = "sha256:e799c050df38a639db758c617ec771fd8fb7a5f8eaaa4b27b101f266b216a246", size = 1855071, upload-time = "2025-04-23T18:31:04.621Z" }, + { url = "https://files.pythonhosted.org/packages/b3/d3/4ae42d33f5e3f50dd467761304be2fa0a9417fbf09735bc2cce003480f2a/pydantic_core-2.33.2-cp311-cp311-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:dc46a01bf8d62f227d5ecee74178ffc448ff4e5197c756331f71efcc66dc980f", size = 1897823, upload-time = "2025-04-23T18:31:06.377Z" }, + { url = "https://files.pythonhosted.org/packages/f4/f3/aa5976e8352b7695ff808599794b1fba2a9ae2ee954a3426855935799488/pydantic_core-2.33.2-cp311-cp311-manylinux_2_17_armv7l.manylinux2014_armv7l.whl", hash = "sha256:a144d4f717285c6d9234a66778059f33a89096dfb9b39117663fd8413d582dcc", size = 1983792, upload-time = "2025-04-23T18:31:07.93Z" }, + { url = "https://files.pythonhosted.org/packages/d5/7a/cda9b5a23c552037717f2b2a5257e9b2bfe45e687386df9591eff7b46d28/pydantic_core-2.33.2-cp311-cp311-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:73cf6373c21bc80b2e0dc88444f41ae60b2f070ed02095754eb5a01df12256de", size = 2136338, upload-time = "2025-04-23T18:31:09.283Z" }, + { url = "https://files.pythonhosted.org/packages/2b/9f/b8f9ec8dd1417eb9da784e91e1667d58a2a4a7b7b34cf4af765ef663a7e5/pydantic_core-2.33.2-cp311-cp311-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:3dc625f4aa79713512d1976fe9f0bc99f706a9dee21dfd1810b4bbbf228d0e8a", size = 2730998, upload-time = "2025-04-23T18:31:11.7Z" }, + { url = "https://files.pythonhosted.org/packages/47/bc/cd720e078576bdb8255d5032c5d63ee5c0bf4b7173dd955185a1d658c456/pydantic_core-2.33.2-cp311-cp311-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:881b21b5549499972441da4758d662aeea93f1923f953e9cbaff14b8b9565aef", size = 2003200, upload-time = "2025-04-23T18:31:13.536Z" }, + { url = "https://files.pythonhosted.org/packages/ca/22/3602b895ee2cd29d11a2b349372446ae9727c32e78a94b3d588a40fdf187/pydantic_core-2.33.2-cp311-cp311-manylinux_2_5_i686.manylinux1_i686.whl", hash = "sha256:bdc25f3681f7b78572699569514036afe3c243bc3059d3942624e936ec93450e", size = 2113890, upload-time = "2025-04-23T18:31:15.011Z" }, + { url = "https://files.pythonhosted.org/packages/ff/e6/e3c5908c03cf00d629eb38393a98fccc38ee0ce8ecce32f69fc7d7b558a7/pydantic_core-2.33.2-cp311-cp311-musllinux_1_1_aarch64.whl", hash = "sha256:fe5b32187cbc0c862ee201ad66c30cf218e5ed468ec8dc1cf49dec66e160cc4d", size = 2073359, upload-time = "2025-04-23T18:31:16.393Z" }, + { url = "https://files.pythonhosted.org/packages/12/e7/6a36a07c59ebefc8777d1ffdaf5ae71b06b21952582e4b07eba88a421c79/pydantic_core-2.33.2-cp311-cp311-musllinux_1_1_armv7l.whl", hash = "sha256:bc7aee6f634a6f4a95676fcb5d6559a2c2a390330098dba5e5a5f28a2e4ada30", size = 2245883, upload-time = "2025-04-23T18:31:17.892Z" }, + { url = "https://files.pythonhosted.org/packages/16/3f/59b3187aaa6cc0c1e6616e8045b284de2b6a87b027cce2ffcea073adf1d2/pydantic_core-2.33.2-cp311-cp311-musllinux_1_1_x86_64.whl", hash = "sha256:235f45e5dbcccf6bd99f9f472858849f73d11120d76ea8707115415f8e5ebebf", size = 2241074, upload-time = "2025-04-23T18:31:19.205Z" }, + { url = "https://files.pythonhosted.org/packages/e0/ed/55532bb88f674d5d8f67ab121a2a13c385df382de2a1677f30ad385f7438/pydantic_core-2.33.2-cp311-cp311-win32.whl", hash = "sha256:6368900c2d3ef09b69cb0b913f9f8263b03786e5b2a387706c5afb66800efd51", size = 1910538, upload-time = "2025-04-23T18:31:20.541Z" }, + { url = "https://files.pythonhosted.org/packages/fe/1b/25b7cccd4519c0b23c2dd636ad39d381abf113085ce4f7bec2b0dc755eb1/pydantic_core-2.33.2-cp311-cp311-win_amd64.whl", hash = "sha256:1e063337ef9e9820c77acc768546325ebe04ee38b08703244c1309cccc4f1bab", size = 1952909, upload-time = "2025-04-23T18:31:22.371Z" }, + { url = "https://files.pythonhosted.org/packages/49/a9/d809358e49126438055884c4366a1f6227f0f84f635a9014e2deb9b9de54/pydantic_core-2.33.2-cp311-cp311-win_arm64.whl", hash = "sha256:6b99022f1d19bc32a4c2a0d544fc9a76e3be90f0b3f4af413f87d38749300e65", size = 1897786, upload-time = "2025-04-23T18:31:24.161Z" }, + { url = "https://files.pythonhosted.org/packages/18/8a/2b41c97f554ec8c71f2a8a5f85cb56a8b0956addfe8b0efb5b3d77e8bdc3/pydantic_core-2.33.2-cp312-cp312-macosx_10_12_x86_64.whl", hash = "sha256:a7ec89dc587667f22b6a0b6579c249fca9026ce7c333fc142ba42411fa243cdc", size = 2009000, upload-time = "2025-04-23T18:31:25.863Z" }, + { url = "https://files.pythonhosted.org/packages/a1/02/6224312aacb3c8ecbaa959897af57181fb6cf3a3d7917fd44d0f2917e6f2/pydantic_core-2.33.2-cp312-cp312-macosx_11_0_arm64.whl", hash = "sha256:3c6db6e52c6d70aa0d00d45cdb9b40f0433b96380071ea80b09277dba021ddf7", size = 1847996, upload-time = "2025-04-23T18:31:27.341Z" }, + { url = "https://files.pythonhosted.org/packages/d6/46/6dcdf084a523dbe0a0be59d054734b86a981726f221f4562aed313dbcb49/pydantic_core-2.33.2-cp312-cp312-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:4e61206137cbc65e6d5256e1166f88331d3b6238e082d9f74613b9b765fb9025", size = 1880957, upload-time = "2025-04-23T18:31:28.956Z" }, + { url = "https://files.pythonhosted.org/packages/ec/6b/1ec2c03837ac00886ba8160ce041ce4e325b41d06a034adbef11339ae422/pydantic_core-2.33.2-cp312-cp312-manylinux_2_17_armv7l.manylinux2014_armv7l.whl", hash = "sha256:eb8c529b2819c37140eb51b914153063d27ed88e3bdc31b71198a198e921e011", size = 1964199, upload-time = "2025-04-23T18:31:31.025Z" }, + { url = "https://files.pythonhosted.org/packages/2d/1d/6bf34d6adb9debd9136bd197ca72642203ce9aaaa85cfcbfcf20f9696e83/pydantic_core-2.33.2-cp312-cp312-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:c52b02ad8b4e2cf14ca7b3d918f3eb0ee91e63b3167c32591e57c4317e134f8f", size = 2120296, upload-time = "2025-04-23T18:31:32.514Z" }, + { url = "https://files.pythonhosted.org/packages/e0/94/2bd0aaf5a591e974b32a9f7123f16637776c304471a0ab33cf263cf5591a/pydantic_core-2.33.2-cp312-cp312-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:96081f1605125ba0855dfda83f6f3df5ec90c61195421ba72223de35ccfb2f88", size = 2676109, upload-time = "2025-04-23T18:31:33.958Z" }, + { url = "https://files.pythonhosted.org/packages/f9/41/4b043778cf9c4285d59742281a769eac371b9e47e35f98ad321349cc5d61/pydantic_core-2.33.2-cp312-cp312-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:8f57a69461af2a5fa6e6bbd7a5f60d3b7e6cebb687f55106933188e79ad155c1", size = 2002028, upload-time = "2025-04-23T18:31:39.095Z" }, + { url = "https://files.pythonhosted.org/packages/cb/d5/7bb781bf2748ce3d03af04d5c969fa1308880e1dca35a9bd94e1a96a922e/pydantic_core-2.33.2-cp312-cp312-manylinux_2_5_i686.manylinux1_i686.whl", hash = "sha256:572c7e6c8bb4774d2ac88929e3d1f12bc45714ae5ee6d9a788a9fb35e60bb04b", size = 2100044, upload-time = "2025-04-23T18:31:41.034Z" }, + { url = "https://files.pythonhosted.org/packages/fe/36/def5e53e1eb0ad896785702a5bbfd25eed546cdcf4087ad285021a90ed53/pydantic_core-2.33.2-cp312-cp312-musllinux_1_1_aarch64.whl", hash = "sha256:db4b41f9bd95fbe5acd76d89920336ba96f03e149097365afe1cb092fceb89a1", size = 2058881, upload-time = "2025-04-23T18:31:42.757Z" }, + { url = "https://files.pythonhosted.org/packages/01/6c/57f8d70b2ee57fc3dc8b9610315949837fa8c11d86927b9bb044f8705419/pydantic_core-2.33.2-cp312-cp312-musllinux_1_1_armv7l.whl", hash = "sha256:fa854f5cf7e33842a892e5c73f45327760bc7bc516339fda888c75ae60edaeb6", size = 2227034, upload-time = "2025-04-23T18:31:44.304Z" }, + { url = "https://files.pythonhosted.org/packages/27/b9/9c17f0396a82b3d5cbea4c24d742083422639e7bb1d5bf600e12cb176a13/pydantic_core-2.33.2-cp312-cp312-musllinux_1_1_x86_64.whl", hash = "sha256:5f483cfb75ff703095c59e365360cb73e00185e01aaea067cd19acffd2ab20ea", size = 2234187, upload-time = "2025-04-23T18:31:45.891Z" }, + { url = "https://files.pythonhosted.org/packages/b0/6a/adf5734ffd52bf86d865093ad70b2ce543415e0e356f6cacabbc0d9ad910/pydantic_core-2.33.2-cp312-cp312-win32.whl", hash = "sha256:9cb1da0f5a471435a7bc7e439b8a728e8b61e59784b2af70d7c169f8dd8ae290", size = 1892628, upload-time = "2025-04-23T18:31:47.819Z" }, + { url = "https://files.pythonhosted.org/packages/43/e4/5479fecb3606c1368d496a825d8411e126133c41224c1e7238be58b87d7e/pydantic_core-2.33.2-cp312-cp312-win_amd64.whl", hash = "sha256:f941635f2a3d96b2973e867144fde513665c87f13fe0e193c158ac51bfaaa7b2", size = 1955866, upload-time = "2025-04-23T18:31:49.635Z" }, + { url = "https://files.pythonhosted.org/packages/0d/24/8b11e8b3e2be9dd82df4b11408a67c61bb4dc4f8e11b5b0fc888b38118b5/pydantic_core-2.33.2-cp312-cp312-win_arm64.whl", hash = "sha256:cca3868ddfaccfbc4bfb1d608e2ccaaebe0ae628e1416aeb9c4d88c001bb45ab", size = 1888894, upload-time = "2025-04-23T18:31:51.609Z" }, + { url = "https://files.pythonhosted.org/packages/30/68/373d55e58b7e83ce371691f6eaa7175e3a24b956c44628eb25d7da007917/pydantic_core-2.33.2-pp310-pypy310_pp73-macosx_10_12_x86_64.whl", hash = "sha256:5c4aa4e82353f65e548c476b37e64189783aa5384903bfea4f41580f255fddfa", size = 2023982, upload-time = "2025-04-23T18:32:53.14Z" }, + { url = "https://files.pythonhosted.org/packages/a4/16/145f54ac08c96a63d8ed6442f9dec17b2773d19920b627b18d4f10a061ea/pydantic_core-2.33.2-pp310-pypy310_pp73-macosx_11_0_arm64.whl", hash = "sha256:d946c8bf0d5c24bf4fe333af284c59a19358aa3ec18cb3dc4370080da1e8ad29", size = 1858412, upload-time = "2025-04-23T18:32:55.52Z" }, + { url = "https://files.pythonhosted.org/packages/41/b1/c6dc6c3e2de4516c0bb2c46f6a373b91b5660312342a0cf5826e38ad82fa/pydantic_core-2.33.2-pp310-pypy310_pp73-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:87b31b6846e361ef83fedb187bb5b4372d0da3f7e28d85415efa92d6125d6e6d", size = 1892749, upload-time = "2025-04-23T18:32:57.546Z" }, + { url = "https://files.pythonhosted.org/packages/12/73/8cd57e20afba760b21b742106f9dbdfa6697f1570b189c7457a1af4cd8a0/pydantic_core-2.33.2-pp310-pypy310_pp73-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:aa9d91b338f2df0508606f7009fde642391425189bba6d8c653afd80fd6bb64e", size = 2067527, upload-time = "2025-04-23T18:32:59.771Z" }, + { url = "https://files.pythonhosted.org/packages/e3/d5/0bb5d988cc019b3cba4a78f2d4b3854427fc47ee8ec8e9eaabf787da239c/pydantic_core-2.33.2-pp310-pypy310_pp73-manylinux_2_5_i686.manylinux1_i686.whl", hash = "sha256:2058a32994f1fde4ca0480ab9d1e75a0e8c87c22b53a3ae66554f9af78f2fe8c", size = 2108225, upload-time = "2025-04-23T18:33:04.51Z" }, + { url = "https://files.pythonhosted.org/packages/f1/c5/00c02d1571913d496aabf146106ad8239dc132485ee22efe08085084ff7c/pydantic_core-2.33.2-pp310-pypy310_pp73-musllinux_1_1_aarch64.whl", hash = "sha256:0e03262ab796d986f978f79c943fc5f620381be7287148b8010b4097f79a39ec", size = 2069490, upload-time = "2025-04-23T18:33:06.391Z" }, + { url = "https://files.pythonhosted.org/packages/22/a8/dccc38768274d3ed3a59b5d06f59ccb845778687652daa71df0cab4040d7/pydantic_core-2.33.2-pp310-pypy310_pp73-musllinux_1_1_armv7l.whl", hash = "sha256:1a8695a8d00c73e50bff9dfda4d540b7dee29ff9b8053e38380426a85ef10052", size = 2237525, upload-time = "2025-04-23T18:33:08.44Z" }, + { url = "https://files.pythonhosted.org/packages/d4/e7/4f98c0b125dda7cf7ccd14ba936218397b44f50a56dd8c16a3091df116c3/pydantic_core-2.33.2-pp310-pypy310_pp73-musllinux_1_1_x86_64.whl", hash = "sha256:fa754d1850735a0b0e03bcffd9d4b4343eb417e47196e4485d9cca326073a42c", size = 2238446, upload-time = "2025-04-23T18:33:10.313Z" }, + { url = "https://files.pythonhosted.org/packages/ce/91/2ec36480fdb0b783cd9ef6795753c1dea13882f2e68e73bce76ae8c21e6a/pydantic_core-2.33.2-pp310-pypy310_pp73-win_amd64.whl", hash = "sha256:a11c8d26a50bfab49002947d3d237abe4d9e4b5bdc8846a63537b6488e197808", size = 2066678, upload-time = "2025-04-23T18:33:12.224Z" }, + { url = "https://files.pythonhosted.org/packages/7b/27/d4ae6487d73948d6f20dddcd94be4ea43e74349b56eba82e9bdee2d7494c/pydantic_core-2.33.2-pp311-pypy311_pp73-macosx_10_12_x86_64.whl", hash = "sha256:dd14041875d09cc0f9308e37a6f8b65f5585cf2598a53aa0123df8b129d481f8", size = 2025200, upload-time = "2025-04-23T18:33:14.199Z" }, + { url = "https://files.pythonhosted.org/packages/f1/b8/b3cb95375f05d33801024079b9392a5ab45267a63400bf1866e7ce0f0de4/pydantic_core-2.33.2-pp311-pypy311_pp73-macosx_11_0_arm64.whl", hash = "sha256:d87c561733f66531dced0da6e864f44ebf89a8fba55f31407b00c2f7f9449593", size = 1859123, upload-time = "2025-04-23T18:33:16.555Z" }, + { url = "https://files.pythonhosted.org/packages/05/bc/0d0b5adeda59a261cd30a1235a445bf55c7e46ae44aea28f7bd6ed46e091/pydantic_core-2.33.2-pp311-pypy311_pp73-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:2f82865531efd18d6e07a04a17331af02cb7a651583c418df8266f17a63c6612", size = 1892852, upload-time = "2025-04-23T18:33:18.513Z" }, + { url = "https://files.pythonhosted.org/packages/3e/11/d37bdebbda2e449cb3f519f6ce950927b56d62f0b84fd9cb9e372a26a3d5/pydantic_core-2.33.2-pp311-pypy311_pp73-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:2bfb5112df54209d820d7bf9317c7a6c9025ea52e49f46b6a2060104bba37de7", size = 2067484, upload-time = "2025-04-23T18:33:20.475Z" }, + { url = "https://files.pythonhosted.org/packages/8c/55/1f95f0a05ce72ecb02a8a8a1c3be0579bbc29b1d5ab68f1378b7bebc5057/pydantic_core-2.33.2-pp311-pypy311_pp73-manylinux_2_5_i686.manylinux1_i686.whl", hash = "sha256:64632ff9d614e5eecfb495796ad51b0ed98c453e447a76bcbeeb69615079fc7e", size = 2108896, upload-time = "2025-04-23T18:33:22.501Z" }, + { url = "https://files.pythonhosted.org/packages/53/89/2b2de6c81fa131f423246a9109d7b2a375e83968ad0800d6e57d0574629b/pydantic_core-2.33.2-pp311-pypy311_pp73-musllinux_1_1_aarch64.whl", hash = "sha256:f889f7a40498cc077332c7ab6b4608d296d852182211787d4f3ee377aaae66e8", size = 2069475, upload-time = "2025-04-23T18:33:24.528Z" }, + { url = "https://files.pythonhosted.org/packages/b8/e9/1f7efbe20d0b2b10f6718944b5d8ece9152390904f29a78e68d4e7961159/pydantic_core-2.33.2-pp311-pypy311_pp73-musllinux_1_1_armv7l.whl", hash = "sha256:de4b83bb311557e439b9e186f733f6c645b9417c84e2eb8203f3f820a4b988bf", size = 2239013, upload-time = "2025-04-23T18:33:26.621Z" }, + { url = "https://files.pythonhosted.org/packages/3c/b2/5309c905a93811524a49b4e031e9851a6b00ff0fb668794472ea7746b448/pydantic_core-2.33.2-pp311-pypy311_pp73-musllinux_1_1_x86_64.whl", hash = "sha256:82f68293f055f51b51ea42fafc74b6aad03e70e191799430b90c13d643059ebb", size = 2238715, upload-time = "2025-04-23T18:33:28.656Z" }, + { url = "https://files.pythonhosted.org/packages/32/56/8a7ca5d2cd2cda1d245d34b1c9a942920a718082ae8e54e5f3e5a58b7add/pydantic_core-2.33.2-pp311-pypy311_pp73-win_amd64.whl", hash = "sha256:329467cecfb529c925cf2bbd4d60d2c509bc2fb52a20c1045bf09bb70971a9c1", size = 2066757, upload-time = "2025-04-23T18:33:30.645Z" }, +] + +[[package]] +name = "pygments" +version = "2.19.2" +source = { registry = "https://pypi.org/simple" } +sdist = { url = "https://files.pythonhosted.org/packages/b0/77/a5b8c569bf593b0140bde72ea885a803b82086995367bf2037de0159d924/pygments-2.19.2.tar.gz", hash = "sha256:636cb2477cec7f8952536970bc533bc43743542f70392ae026374600add5b887", size = 4968631, upload-time = "2025-06-21T13:39:12.283Z" } +wheels = [ + { url = "https://files.pythonhosted.org/packages/c7/21/705964c7812476f378728bdf590ca4b771ec72385c533964653c68e86bdc/pygments-2.19.2-py3-none-any.whl", hash = "sha256:86540386c03d588bb81d44bc3928634ff26449851e99741617ecb9037ee5ec0b", size = 1225217, upload-time = "2025-06-21T13:39:07.939Z" }, +] + +[[package]] +name = "pyiceberg" +version = "0.10.0" +source = { git = "https://github.com/apache/iceberg-python.git?rev=d3eb149fe99b14e714a06e727b69cd617c6c052d#d3eb149fe99b14e714a06e727b69cd617c6c052d" } +dependencies = [ + { name = "cachetools" }, + { name = "click" }, + { name = "fsspec" }, + { name = "mmh3" }, + { name = "pydantic" }, + { name = "pyparsing" }, + { name = "pyroaring" }, + { name = "requests" }, + { name = "rich" }, + { name = "sortedcontainers" }, + { name = "strictyaml" }, + { name = "tenacity" }, +] + +[package.optional-dependencies] +pyarrow = [ + { name = "pyarrow" }, + { name = "pyiceberg-core" }, +] +sql-sqlite = [ + { name = "sqlalchemy" }, +] + +[[package]] +name = "pyiceberg-core" +source = { editable = "." } + +[package.dev-dependencies] +dev = [ + { name = "datafusion" }, + { name = "fastavro" }, + { name = "maturin" }, + { name = "pyiceberg", extra = ["pyarrow", "sql-sqlite"] }, + { name = "pytest" }, +] + +[package.metadata] + +[package.metadata.requires-dev] +dev = [ + { name = "datafusion", specifier = "==50.*" }, + { name = "fastavro", specifier = ">=1.11.1" }, + { name = "maturin", specifier = ">=1.0,<2.0" }, + { name = "pyiceberg", extras = ["sql-sqlite", "pyarrow"], git = "https://github.com/apache/iceberg-python.git?rev=d3eb149fe99b14e714a06e727b69cd617c6c052d" }, + { name = "pytest", specifier = ">=8.3.2" }, +] + +[[package]] +name = "pyparsing" +version = "3.2.5" +source = { registry = "https://pypi.org/simple" } +sdist = { url = "https://files.pythonhosted.org/packages/f2/a5/181488fc2b9d093e3972d2a472855aae8a03f000592dbfce716a512b3359/pyparsing-3.2.5.tar.gz", hash = "sha256:2df8d5b7b2802ef88e8d016a2eb9c7aeaa923529cd251ed0fe4608275d4105b6", size = 1099274, upload-time = "2025-09-21T04:11:06.277Z" } +wheels = [ + { url = "https://files.pythonhosted.org/packages/10/5e/1aa9a93198c6b64513c9d7752de7422c06402de6600a8767da1524f9570b/pyparsing-3.2.5-py3-none-any.whl", hash = "sha256:e38a4f02064cf41fe6593d328d0512495ad1f3d8a91c4f73fc401b3079a59a5e", size = 113890, upload-time = "2025-09-21T04:11:04.117Z" }, +] + +[[package]] +name = "pyroaring" +version = "1.0.3" +source = { registry = "https://pypi.org/simple" } +sdist = { url = "https://files.pythonhosted.org/packages/0f/e4/975f0fa77fc3590820b4a3ac49704644b389795409bc12eb91729f845812/pyroaring-1.0.3.tar.gz", hash = "sha256:cd7392d1c010c9e41c11c62cd0610c8852e7e9698b1f7f6c2fcdefe50e7ef6da", size = 188688, upload-time = "2025-10-09T09:08:22.448Z" } +wheels = [ + { url = "https://files.pythonhosted.org/packages/3f/87/f19328d35f29b1d634f8f2127941927b37c6b68890912a4668e1a272d54d/pyroaring-1.0.3-cp310-cp310-macosx_10_9_universal2.whl", hash = "sha256:c10e4cfbe203a578c78808406af491e3615d5e46cf69a7709050243346cd68bc", size = 670434, upload-time = "2025-10-09T09:06:34.89Z" }, + { url = "https://files.pythonhosted.org/packages/2f/fa/d933448844925728990e2256bbfa4f18cc3956d43d548270317128038015/pyroaring-1.0.3-cp310-cp310-macosx_10_9_x86_64.whl", hash = "sha256:cc329c62e504f2531c4008240f31736bcd2dee4339071f1eac0648068e6d17fa", size = 367332, upload-time = "2025-10-09T09:06:36.549Z" }, + { url = "https://files.pythonhosted.org/packages/39/e4/ab74296c3aac8f3ceb800354c2f811de2c4c518b3ca3a7a28e0599740cff/pyroaring-1.0.3-cp310-cp310-macosx_11_0_arm64.whl", hash = "sha256:8c7fb6ddf6ef31148f0939bc5c26b681d63df301ee1e372525012dd7bfe4a30a", size = 311443, upload-time = "2025-10-09T09:06:37.617Z" }, + { url = "https://files.pythonhosted.org/packages/6b/71/13ff623f3bba340ea7cc841883d7a0eaba1bec7e2d4e0d6759b89a4ce754/pyroaring-1.0.3-cp310-cp310-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:cd18446832ea04a7d33bd6b78270b0be14eabcda5937af3428d6cb3d2bf98e54", size = 1853845, upload-time = "2025-10-09T09:06:38.706Z" }, + { url = "https://files.pythonhosted.org/packages/2b/05/40c0b37d78b16842c924b87b4fa491f4b20ed0e40c6255c872df45314247/pyroaring-1.0.3-cp310-cp310-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:4f0cbc766df2a24e28f23d69b66bbec64e691799219fd82c2f2236f03fc88e2e", size = 2045170, upload-time = "2025-10-09T09:06:40.229Z" }, + { url = "https://files.pythonhosted.org/packages/78/c6/6560c61d2f5c30fbb8f7b9a1c7d02068e78a7e8cd336eb294ec70896a80a/pyroaring-1.0.3-cp310-cp310-manylinux_2_24_armv7l.manylinux_2_31_armv7l.whl", hash = "sha256:96a51e96f8f473381615f0f852f7238ad0a47f28e4a35e9f082468c5cfe4e9c3", size = 1791073, upload-time = "2025-10-09T09:06:42.092Z" }, + { url = "https://files.pythonhosted.org/packages/33/9e/0c91d4dbc4ec7bea9dcd3c203cfb8d96ed9df3c46981c0b22e9f17e98296/pyroaring-1.0.3-cp310-cp310-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:654af38b1f1c9bdc27b4f6d331fc5d91599df96e72a6df1886f4d95eea60ab29", size = 1789048, upload-time = "2025-10-09T09:06:43.392Z" }, + { url = "https://files.pythonhosted.org/packages/e9/48/51af418321cda2a1cfa64a48397ea0b73da74afe5c53a862525476d8a42c/pyroaring-1.0.3-cp310-cp310-musllinux_1_2_aarch64.whl", hash = "sha256:6721036afa31c07bdcbb4fcafa166660cf9c2eac695dcd495f8778549fa55899", size = 2838811, upload-time = "2025-10-09T09:06:44.588Z" }, + { url = "https://files.pythonhosted.org/packages/b2/69/7bc7070b35f72706e3870b5856e73656b9065bedae90268da5d77be00b15/pyroaring-1.0.3-cp310-cp310-musllinux_1_2_armv7l.whl", hash = "sha256:0caa10f20329d09233fac6550b2adce4d9f173f748a9a9a5ea3b7033827dfe2d", size = 2640299, upload-time = "2025-10-09T09:06:46.159Z" }, + { url = "https://files.pythonhosted.org/packages/d0/87/5f2f590973d454e79ee8729aca888f9bb2d6018f7c9816bf66000cbc5e88/pyroaring-1.0.3-cp310-cp310-musllinux_1_2_i686.whl", hash = "sha256:f109be8af937e85c52cb920d3fd120db52b172f59460852d2e3d2e3d13a4f52a", size = 2965427, upload-time = "2025-10-09T09:06:47.558Z" }, + { url = "https://files.pythonhosted.org/packages/e4/c8/1b425503141681db94941d9d3f41333bbd61975c3fc62d95122c372da85a/pyroaring-1.0.3-cp310-cp310-musllinux_1_2_x86_64.whl", hash = "sha256:ddc80bfcd313c7c524a2742d263e73cae088b6a611b77dcc46fa90c306f6dace", size = 3084447, upload-time = "2025-10-09T09:06:48.727Z" }, + { url = "https://files.pythonhosted.org/packages/51/b9/2ac712ea90bd1e0d7e49e5e26c0c5aad1d77f21cf520a7edf46a477f5217/pyroaring-1.0.3-cp310-cp310-win32.whl", hash = "sha256:5a183f5ec069757fe5b60e37f7c6fa8a53178eacf0d76601b739e2890edee036", size = 204956, upload-time = "2025-10-09T09:06:49.801Z" }, + { url = "https://files.pythonhosted.org/packages/6d/b9/1d4859c74d05f72b86dc0b308e6221e814a178459301cea9bcd084b4a92b/pyroaring-1.0.3-cp310-cp310-win_amd64.whl", hash = "sha256:051bd9a66ce855a1143faa2b879ea6c6ca2905209e172ce9eedf79834897c730", size = 253778, upload-time = "2025-10-09T09:06:50.68Z" }, + { url = "https://files.pythonhosted.org/packages/64/20/b421100bd14b6a1074945af1418671630e1b8c4996ef000ac4e363785ead/pyroaring-1.0.3-cp310-cp310-win_arm64.whl", hash = "sha256:3043ff5c85375310ca3cd3e01944e03026e0ec07885e52dfabcfcd9dc303867f", size = 219330, upload-time = "2025-10-09T09:06:52.011Z" }, + { url = "https://files.pythonhosted.org/packages/39/ed/5e555dd99b12318ea1c7666b773fc4f097aeb609eeb1c1b3da519d445f71/pyroaring-1.0.3-cp311-cp311-macosx_10_9_universal2.whl", hash = "sha256:755cdac1f9a1b7b5c621e570d4f6dbcf3b8e4a1e35a66f976104ecb35dce4ed2", size = 675916, upload-time = "2025-10-09T09:06:53.174Z" }, + { url = "https://files.pythonhosted.org/packages/da/06/dd8a9a87b90c4560f8384ab1dbafcd40c2a16f6777a07334a8e341bd7383/pyroaring-1.0.3-cp311-cp311-macosx_10_9_x86_64.whl", hash = "sha256:ebab073db620f26f0ba11e13fa2f35e3b1298209fba47b6bc8cb6f0e2c9627f9", size = 369743, upload-time = "2025-10-09T09:06:54.421Z" }, + { url = "https://files.pythonhosted.org/packages/35/aa/da882011045ddacffe818a4fcbdd7e609a15f9c83d536222ec5b17af4aa9/pyroaring-1.0.3-cp311-cp311-macosx_11_0_arm64.whl", hash = "sha256:684fb8dffe19bdb7f91897c65eac6eee23b1e46043c47eb24288f28a1170fe04", size = 313981, upload-time = "2025-10-09T09:06:55.514Z" }, + { url = "https://files.pythonhosted.org/packages/ed/3c/f6534844b02e2505ccdc9aae461c9838ab96f72b5688c045448761735512/pyroaring-1.0.3-cp311-cp311-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:678d31fc24e82945a1bfb14816c77823983382ffea76985d494782aa2f058427", size = 1923181, upload-time = "2025-10-09T09:06:56.897Z" }, + { url = "https://files.pythonhosted.org/packages/ea/82/9f1a85ba33e3d89b9cdb8183fb2fd2f25720d10742dd8827508ccccc13ae/pyroaring-1.0.3-cp311-cp311-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:7d815f624e0285db3669f673d1725cb754b120ec70d0032d7c7166103a96c96d", size = 2113222, upload-time = "2025-10-09T09:06:58.388Z" }, + { url = "https://files.pythonhosted.org/packages/a7/f8/4d4340971cbc1379f987c847080bcb7f9765a57e122f392c3a3485c9587e/pyroaring-1.0.3-cp311-cp311-manylinux_2_24_armv7l.manylinux_2_31_armv7l.whl", hash = "sha256:57fd5b80dacb8e888402b6b7508a734c6a527063e4e24e882ff2e0fd90721ada", size = 1837385, upload-time = "2025-10-09T09:06:59.449Z" }, + { url = "https://files.pythonhosted.org/packages/c6/58/d14cc561685e4c224af26b4fdb4f6c7e643294ac5a4b29f178b5cbb71af1/pyroaring-1.0.3-cp311-cp311-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:ab26a7a45a0bb46c00394d1a60a9f2d57c220f84586e30d59b39784b0f94aee6", size = 1856170, upload-time = "2025-10-09T09:07:00.608Z" }, + { url = "https://files.pythonhosted.org/packages/d1/d2/d2d9790c373f6438d4d0958bc4c79f3dc77826d8553743ff3f64acdc9ab3/pyroaring-1.0.3-cp311-cp311-musllinux_1_2_aarch64.whl", hash = "sha256:9232f3f606315d59049c128154100fd05008d5c5c211e48b21848cd41ee64d26", size = 2909282, upload-time = "2025-10-09T09:07:02.124Z" }, + { url = "https://files.pythonhosted.org/packages/bc/28/4b2277982302b5b406998064ca1eaef1a79e4ea87185f511e33e7a7e3511/pyroaring-1.0.3-cp311-cp311-musllinux_1_2_armv7l.whl", hash = "sha256:f34b44b3ec3df97b978799f2901fefb2a48d367496fd1cde3cc5fe8b3bc13510", size = 2701034, upload-time = "2025-10-09T09:07:03.403Z" }, + { url = "https://files.pythonhosted.org/packages/d2/91/b2340193825fa2431cf735f0ecb23206fb31f386fecca38336935a294513/pyroaring-1.0.3-cp311-cp311-musllinux_1_2_i686.whl", hash = "sha256:25a83ec6bac3106568bd3fdd316f0fee52aa0be8c72da565ad02b10ae7905924", size = 3028962, upload-time = "2025-10-09T09:07:05.558Z" }, + { url = "https://files.pythonhosted.org/packages/07/ea/ad79073cc5d8dcca35d1a955bb886d96905e9dacc58d1971fda012a5ad18/pyroaring-1.0.3-cp311-cp311-musllinux_1_2_x86_64.whl", hash = "sha256:c17d4ec53b5b6b333d9a9515051213a691293ada785dc8c025d3641482597ed3", size = 3152109, upload-time = "2025-10-09T09:07:06.887Z" }, + { url = "https://files.pythonhosted.org/packages/9a/de/f55a1093acb16d25ff9811546823e59078e4a3e56d2eb0ff5d10f696933d/pyroaring-1.0.3-cp311-cp311-win32.whl", hash = "sha256:d54024459ace600f1d1ffbc6dc3c60eb47cca3b678701f06148f59e10f6f8d7b", size = 204246, upload-time = "2025-10-09T09:07:08.036Z" }, + { url = "https://files.pythonhosted.org/packages/c6/e5/36bf3039733b8e00732892c9334b2f5309f38e72af0b3b40b8729b5857a3/pyroaring-1.0.3-cp311-cp311-win_amd64.whl", hash = "sha256:c28750148ef579a7447a8cb60b39e5943e03f8c29bce8f2788728f6f23d1887a", size = 254637, upload-time = "2025-10-09T09:07:09.103Z" }, + { url = "https://files.pythonhosted.org/packages/d6/e8/e2b78e595b5a82a6014af327614756a55f17ec4120a2ab197f1762641316/pyroaring-1.0.3-cp311-cp311-win_arm64.whl", hash = "sha256:535d8deccbd8db2c6bf38629243e9646756905574a742b2a72ff51d6461d616c", size = 219597, upload-time = "2025-10-09T09:07:10.38Z" }, + { url = "https://files.pythonhosted.org/packages/dd/09/a5376d55672e0535019ba1469888909d0046cea0cfb969a4aa1f99caaf22/pyroaring-1.0.3-cp312-cp312-macosx_10_13_universal2.whl", hash = "sha256:add3e4c78eb590a76526ecce8d1566eecdd5822e351c36b3697997f4a80ed808", size = 681056, upload-time = "2025-10-09T09:07:11.497Z" }, + { url = "https://files.pythonhosted.org/packages/23/dd/78f59d361bd9ebf8de3660408b0c48664ade0a057ebcf4b207d99ac1a698/pyroaring-1.0.3-cp312-cp312-macosx_10_13_x86_64.whl", hash = "sha256:ebaffe846cf4ba4f00ce6b8a9f39613f24e2d09447e77be4fa6e898bc36451b6", size = 375111, upload-time = "2025-10-09T09:07:12.597Z" }, + { url = "https://files.pythonhosted.org/packages/bf/03/10dc93f83a5453eb40a69c79106a8385b40aa12cf4531ca72bd9d7f45cb2/pyroaring-1.0.3-cp312-cp312-macosx_11_0_arm64.whl", hash = "sha256:a9459f27498f97d08031a34a5ead230b77eb0ab3cc3d85b7f54faa2fd548acd6", size = 314319, upload-time = "2025-10-09T09:07:13.579Z" }, + { url = "https://files.pythonhosted.org/packages/86/9e/b00c38a7e62a73e152055f593595c37152e61fc2896fd11538a7c71fbe4e/pyroaring-1.0.3-cp312-cp312-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:f2b2eb8bd1c35c772994889be9f7dda09477475d7aa1e2af9ab4ef18619326f6", size = 1869251, upload-time = "2025-10-09T09:07:14.584Z" }, + { url = "https://files.pythonhosted.org/packages/4f/33/f32d00ca105b66303deab43d027c3574c8ade8525dac0e5b50a9fb4d1b76/pyroaring-1.0.3-cp312-cp312-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:d31f4c1c906f1af14ce61a3959d04a14a64c594f8a768399146a45bbd341f21f", size = 2071551, upload-time = "2025-10-09T09:07:15.713Z" }, + { url = "https://files.pythonhosted.org/packages/5d/89/e953cae181ba4c7523334855a1ca0ae8eeea3cee8d7cd39c56bd99709d3f/pyroaring-1.0.3-cp312-cp312-manylinux_2_24_armv7l.manylinux_2_31_armv7l.whl", hash = "sha256:53be988fc86698d56c11049bfe5113a2f6990adb1fa2782b29636509808b6aa7", size = 1781071, upload-time = "2025-10-09T09:07:17.19Z" }, + { url = "https://files.pythonhosted.org/packages/fa/db/65d4be532e68b62a84a9c89b24d0a1394f452f484fa29392142d9a3b9c48/pyroaring-1.0.3-cp312-cp312-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:7df84d223424523b19a23781f4246cc247fd6d821e1bc0853c2f25669136f7d0", size = 1795670, upload-time = "2025-10-09T09:07:18.524Z" }, + { url = "https://files.pythonhosted.org/packages/f5/9e/684ea0568ce7d30fc4e01ad1c666e9ce1a5b1702fa630231f4f6bdb96539/pyroaring-1.0.3-cp312-cp312-musllinux_1_2_aarch64.whl", hash = "sha256:34a781f1f9766897f63ef18be129827340ae37764015b83fdcff1efb9e29136d", size = 2849305, upload-time = "2025-10-09T09:07:20.388Z" }, + { url = "https://files.pythonhosted.org/packages/7c/fd/d7773a2adf91f45d8924197954c66b1694325afd2f27e02edaac07338402/pyroaring-1.0.3-cp312-cp312-musllinux_1_2_armv7l.whl", hash = "sha256:1f414343b4ed0756734328cdf2a91022fc54503769e3f8d79bd0b672ea815a16", size = 2692843, upload-time = "2025-10-09T09:07:22.042Z" }, + { url = "https://files.pythonhosted.org/packages/13/72/b8a99ba138eebd8ff9bf8d15f3942e9e43e8e45723e2e6b7b09e542b7448/pyroaring-1.0.3-cp312-cp312-musllinux_1_2_i686.whl", hash = "sha256:d16ae185c72dc64f76335dbe53e53a892e78115adc92194957d1b7ef74d230b9", size = 2983440, upload-time = "2025-10-09T09:07:23.419Z" }, + { url = "https://files.pythonhosted.org/packages/ca/94/e6ed1f682d850e039c71b2032bacdefc5082dc809796cf34b9e6f24c604d/pyroaring-1.0.3-cp312-cp312-musllinux_1_2_x86_64.whl", hash = "sha256:f888447bf22dde7759108bfe6dfbeb6bbb61b14948de9c4cb6843c4dd57e2215", size = 3117542, upload-time = "2025-10-09T09:07:25.104Z" }, + { url = "https://files.pythonhosted.org/packages/8f/89/d55b0ed3e098ef89c421b43b748afe3d90eb250cab50b9e53e3a3449ac58/pyroaring-1.0.3-cp312-cp312-win32.whl", hash = "sha256:fbbdc44c51a0a3efd7be3dbe04466278ce098fcd101aa1905849319042159770", size = 205118, upload-time = "2025-10-09T09:07:26.532Z" }, + { url = "https://files.pythonhosted.org/packages/c8/e1/b71fef6a73efb50110d33d714235ff7059f4ebae98dc474b6549b322f48f/pyroaring-1.0.3-cp312-cp312-win_amd64.whl", hash = "sha256:3b217c4b3ad953b4c759a0d2f9bd95316f0c345b9f7adb49e6ded7a1f5106bd4", size = 260629, upload-time = "2025-10-09T09:07:27.528Z" }, + { url = "https://files.pythonhosted.org/packages/57/33/66ee872079c9c47512d6e17d374bcad8d91350c24dc20fbe678c34b33745/pyroaring-1.0.3-cp312-cp312-win_arm64.whl", hash = "sha256:e6bcf838564c21bab8fe6c2748b4990d4cd90612d8c470c04889def7bb5114ea", size = 219032, upload-time = "2025-10-09T09:07:28.754Z" }, +] + +[[package]] +name = "pytest" +version = "8.4.2" +source = { registry = "https://pypi.org/simple" } +dependencies = [ + { name = "colorama", marker = "sys_platform == 'win32'" }, + { name = "exceptiongroup", marker = "python_full_version < '3.11'" }, + { name = "iniconfig" }, + { name = "packaging" }, + { name = "pluggy" }, + { name = "pygments" }, + { name = "tomli", marker = "python_full_version < '3.11'" }, +] +sdist = { url = "https://files.pythonhosted.org/packages/a3/5c/00a0e072241553e1a7496d638deababa67c5058571567b92a7eaa258397c/pytest-8.4.2.tar.gz", hash = "sha256:86c0d0b93306b961d58d62a4db4879f27fe25513d4b969df351abdddb3c30e01", size = 1519618, upload-time = "2025-09-04T14:34:22.711Z" } +wheels = [ + { url = "https://files.pythonhosted.org/packages/a8/a4/20da314d277121d6534b3a980b29035dcd51e6744bd79075a6ce8fa4eb8d/pytest-8.4.2-py3-none-any.whl", hash = "sha256:872f880de3fc3a5bdc88a11b39c9710c3497a547cfa9320bc3c5e62fbf272e79", size = 365750, upload-time = "2025-09-04T14:34:20.226Z" }, +] + +[[package]] +name = "python-dateutil" +version = "2.9.0.post0" +source = { registry = "https://pypi.org/simple" } +dependencies = [ + { name = "six" }, +] +sdist = { url = "https://files.pythonhosted.org/packages/66/c0/0c8b6ad9f17a802ee498c46e004a0eb49bc148f2fd230864601a86dcf6db/python-dateutil-2.9.0.post0.tar.gz", hash = "sha256:37dd54208da7e1cd875388217d5e00ebd4179249f90fb72437e91a35459a0ad3", size = 342432, upload-time = "2024-03-01T18:36:20.211Z" } +wheels = [ + { url = "https://files.pythonhosted.org/packages/ec/57/56b9bcc3c9c6a792fcbaf139543cee77261f3651ca9da0c93f5c1221264b/python_dateutil-2.9.0.post0-py2.py3-none-any.whl", hash = "sha256:a8b2bc7bffae282281c8140a97d3aa9c14da0b136dfe83f850eea9a5f7470427", size = 229892, upload-time = "2024-03-01T18:36:18.57Z" }, +] + +[[package]] +name = "requests" +version = "2.32.5" +source = { registry = "https://pypi.org/simple" } +dependencies = [ + { name = "certifi" }, + { name = "charset-normalizer" }, + { name = "idna" }, + { name = "urllib3" }, +] +sdist = { url = "https://files.pythonhosted.org/packages/c9/74/b3ff8e6c8446842c3f5c837e9c3dfcfe2018ea6ecef224c710c85ef728f4/requests-2.32.5.tar.gz", hash = "sha256:dbba0bac56e100853db0ea71b82b4dfd5fe2bf6d3754a8893c3af500cec7d7cf", size = 134517, upload-time = "2025-08-18T20:46:02.573Z" } +wheels = [ + { url = "https://files.pythonhosted.org/packages/1e/db/4254e3eabe8020b458f1a747140d32277ec7a271daf1d235b70dc0b4e6e3/requests-2.32.5-py3-none-any.whl", hash = "sha256:2462f94637a34fd532264295e186976db0f5d453d1cdd31473c85a6a161affb6", size = 64738, upload-time = "2025-08-18T20:46:00.542Z" }, +] + +[[package]] +name = "rich" +version = "14.2.0" +source = { registry = "https://pypi.org/simple" } +dependencies = [ + { name = "markdown-it-py" }, + { name = "pygments" }, +] +sdist = { url = "https://files.pythonhosted.org/packages/fb/d2/8920e102050a0de7bfabeb4c4614a49248cf8d5d7a8d01885fbb24dc767a/rich-14.2.0.tar.gz", hash = "sha256:73ff50c7c0c1c77c8243079283f4edb376f0f6442433aecb8ce7e6d0b92d1fe4", size = 219990, upload-time = "2025-10-09T14:16:53.064Z" } +wheels = [ + { url = "https://files.pythonhosted.org/packages/25/7a/b0178788f8dc6cafce37a212c99565fa1fe7872c70c6c9c1e1a372d9d88f/rich-14.2.0-py3-none-any.whl", hash = "sha256:76bc51fe2e57d2b1be1f96c524b890b816e334ab4c1e45888799bfaab0021edd", size = 243393, upload-time = "2025-10-09T14:16:51.245Z" }, +] + +[[package]] +name = "six" +version = "1.17.0" +source = { registry = "https://pypi.org/simple" } +sdist = { url = "https://files.pythonhosted.org/packages/94/e7/b2c673351809dca68a0e064b6af791aa332cf192da575fd474ed7d6f16a2/six-1.17.0.tar.gz", hash = "sha256:ff70335d468e7eb6ec65b95b99d3a2836546063f63acc5171de367e834932a81", size = 34031, upload-time = "2024-12-04T17:35:28.174Z" } +wheels = [ + { url = "https://files.pythonhosted.org/packages/b7/ce/149a00dd41f10bc29e5921b496af8b574d8413afcd5e30dfa0ed46c2cc5e/six-1.17.0-py2.py3-none-any.whl", hash = "sha256:4721f391ed90541fddacab5acf947aa0d3dc7d27b2e1e8eda2be8970586c3274", size = 11050, upload-time = "2024-12-04T17:35:26.475Z" }, +] + +[[package]] +name = "sortedcontainers" +version = "2.4.0" +source = { registry = "https://pypi.org/simple" } +sdist = { url = "https://files.pythonhosted.org/packages/e8/c4/ba2f8066cceb6f23394729afe52f3bf7adec04bf9ed2c820b39e19299111/sortedcontainers-2.4.0.tar.gz", hash = "sha256:25caa5a06cc30b6b83d11423433f65d1f9d76c4c6a0c90e3379eaa43b9bfdb88", size = 30594, upload-time = "2021-05-16T22:03:42.897Z" } +wheels = [ + { url = "https://files.pythonhosted.org/packages/32/46/9cb0e58b2deb7f82b84065f37f3bffeb12413f947f9388e4cac22c4621ce/sortedcontainers-2.4.0-py2.py3-none-any.whl", hash = "sha256:a163dcaede0f1c021485e957a39245190e74249897e2ae4b2aa38595db237ee0", size = 29575, upload-time = "2021-05-16T22:03:41.177Z" }, +] + +[[package]] +name = "sqlalchemy" +version = "2.0.44" +source = { registry = "https://pypi.org/simple" } +dependencies = [ + { name = "greenlet", marker = "platform_machine == 'AMD64' or platform_machine == 'WIN32' or platform_machine == 'aarch64' or platform_machine == 'amd64' or platform_machine == 'ppc64le' or platform_machine == 'win32' or platform_machine == 'x86_64'" }, + { name = "typing-extensions" }, +] +sdist = { url = "https://files.pythonhosted.org/packages/f0/f2/840d7b9496825333f532d2e3976b8eadbf52034178aac53630d09fe6e1ef/sqlalchemy-2.0.44.tar.gz", hash = "sha256:0ae7454e1ab1d780aee69fd2aae7d6b8670a581d8847f2d1e0f7ddfbf47e5a22", size = 9819830, upload-time = "2025-10-10T14:39:12.935Z" } +wheels = [ + { url = "https://files.pythonhosted.org/packages/a2/a7/e9ccfa7eecaf34c6f57d8cb0bb7cbdeeff27017cc0f5d0ca90fdde7a7c0d/sqlalchemy-2.0.44-cp310-cp310-macosx_10_9_x86_64.whl", hash = "sha256:7c77f3080674fc529b1bd99489378c7f63fcb4ba7f8322b79732e0258f0ea3ce", size = 2137282, upload-time = "2025-10-10T15:36:10.965Z" }, + { url = "https://files.pythonhosted.org/packages/b1/e1/50bc121885bdf10833a4f65ecbe9fe229a3215f4d65a58da8a181734cae3/sqlalchemy-2.0.44-cp310-cp310-macosx_11_0_arm64.whl", hash = "sha256:4c26ef74ba842d61635b0152763d057c8d48215d5be9bb8b7604116a059e9985", size = 2127322, upload-time = "2025-10-10T15:36:12.428Z" }, + { url = "https://files.pythonhosted.org/packages/46/f2/a8573b7230a3ce5ee4b961a2d510d71b43872513647398e595b744344664/sqlalchemy-2.0.44-cp310-cp310-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:f4a172b31785e2f00780eccab00bc240ccdbfdb8345f1e6063175b3ff12ad1b0", size = 3214772, upload-time = "2025-10-10T15:34:15.09Z" }, + { url = "https://files.pythonhosted.org/packages/4a/d8/c63d8adb6a7edaf8dcb6f75a2b1e9f8577960a1e489606859c4d73e7d32b/sqlalchemy-2.0.44-cp310-cp310-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:f9480c0740aabd8cb29c329b422fb65358049840b34aba0adf63162371d2a96e", size = 3214434, upload-time = "2025-10-10T15:47:00.473Z" }, + { url = "https://files.pythonhosted.org/packages/ee/a6/243d277a4b54fae74d4797957a7320a5c210c293487f931cbe036debb697/sqlalchemy-2.0.44-cp310-cp310-musllinux_1_2_aarch64.whl", hash = "sha256:17835885016b9e4d0135720160db3095dc78c583e7b902b6be799fb21035e749", size = 3155365, upload-time = "2025-10-10T15:34:17.932Z" }, + { url = "https://files.pythonhosted.org/packages/5f/f8/6a39516ddd75429fd4ee5a0d72e4c80639fab329b2467c75f363c2ed9751/sqlalchemy-2.0.44-cp310-cp310-musllinux_1_2_x86_64.whl", hash = "sha256:cbe4f85f50c656d753890f39468fcd8190c5f08282caf19219f684225bfd5fd2", size = 3178910, upload-time = "2025-10-10T15:47:02.346Z" }, + { url = "https://files.pythonhosted.org/packages/43/f0/118355d4ad3c39d9a2f5ee4c7304a9665b3571482777357fa9920cd7a6b4/sqlalchemy-2.0.44-cp310-cp310-win32.whl", hash = "sha256:2fcc4901a86ed81dc76703f3b93ff881e08761c63263c46991081fd7f034b165", size = 2105624, upload-time = "2025-10-10T15:38:15.552Z" }, + { url = "https://files.pythonhosted.org/packages/61/83/6ae5f9466f8aa5d0dcebfff8c9c33b98b27ce23292df3b990454b3d434fd/sqlalchemy-2.0.44-cp310-cp310-win_amd64.whl", hash = "sha256:9919e77403a483ab81e3423151e8ffc9dd992c20d2603bf17e4a8161111e55f5", size = 2129240, upload-time = "2025-10-10T15:38:17.175Z" }, + { url = "https://files.pythonhosted.org/packages/e3/81/15d7c161c9ddf0900b076b55345872ed04ff1ed6a0666e5e94ab44b0163c/sqlalchemy-2.0.44-cp311-cp311-macosx_10_9_x86_64.whl", hash = "sha256:0fe3917059c7ab2ee3f35e77757062b1bea10a0b6ca633c58391e3f3c6c488dd", size = 2140517, upload-time = "2025-10-10T15:36:15.64Z" }, + { url = "https://files.pythonhosted.org/packages/d4/d5/4abd13b245c7d91bdf131d4916fd9e96a584dac74215f8b5bc945206a974/sqlalchemy-2.0.44-cp311-cp311-macosx_11_0_arm64.whl", hash = "sha256:de4387a354ff230bc979b46b2207af841dc8bf29847b6c7dbe60af186d97aefa", size = 2130738, upload-time = "2025-10-10T15:36:16.91Z" }, + { url = "https://files.pythonhosted.org/packages/cb/3c/8418969879c26522019c1025171cefbb2a8586b6789ea13254ac602986c0/sqlalchemy-2.0.44-cp311-cp311-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:c3678a0fb72c8a6a29422b2732fe423db3ce119c34421b5f9955873eb9b62c1e", size = 3304145, upload-time = "2025-10-10T15:34:19.569Z" }, + { url = "https://files.pythonhosted.org/packages/94/2d/fdb9246d9d32518bda5d90f4b65030b9bf403a935cfe4c36a474846517cb/sqlalchemy-2.0.44-cp311-cp311-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:3cf6872a23601672d61a68f390e44703442639a12ee9dd5a88bbce52a695e46e", size = 3304511, upload-time = "2025-10-10T15:47:05.088Z" }, + { url = "https://files.pythonhosted.org/packages/7d/fb/40f2ad1da97d5c83f6c1269664678293d3fe28e90ad17a1093b735420549/sqlalchemy-2.0.44-cp311-cp311-musllinux_1_2_aarch64.whl", hash = "sha256:329aa42d1be9929603f406186630135be1e7a42569540577ba2c69952b7cf399", size = 3235161, upload-time = "2025-10-10T15:34:21.193Z" }, + { url = "https://files.pythonhosted.org/packages/95/cb/7cf4078b46752dca917d18cf31910d4eff6076e5b513c2d66100c4293d83/sqlalchemy-2.0.44-cp311-cp311-musllinux_1_2_x86_64.whl", hash = "sha256:70e03833faca7166e6a9927fbee7c27e6ecde436774cd0b24bbcc96353bce06b", size = 3261426, upload-time = "2025-10-10T15:47:07.196Z" }, + { url = "https://files.pythonhosted.org/packages/f8/3b/55c09b285cb2d55bdfa711e778bdffdd0dc3ffa052b0af41f1c5d6e582fa/sqlalchemy-2.0.44-cp311-cp311-win32.whl", hash = "sha256:253e2f29843fb303eca6b2fc645aca91fa7aa0aa70b38b6950da92d44ff267f3", size = 2105392, upload-time = "2025-10-10T15:38:20.051Z" }, + { url = "https://files.pythonhosted.org/packages/c7/23/907193c2f4d680aedbfbdf7bf24c13925e3c7c292e813326c1b84a0b878e/sqlalchemy-2.0.44-cp311-cp311-win_amd64.whl", hash = "sha256:7a8694107eb4308a13b425ca8c0e67112f8134c846b6e1f722698708741215d5", size = 2130293, upload-time = "2025-10-10T15:38:21.601Z" }, + { url = "https://files.pythonhosted.org/packages/62/c4/59c7c9b068e6813c898b771204aad36683c96318ed12d4233e1b18762164/sqlalchemy-2.0.44-cp312-cp312-macosx_10_13_x86_64.whl", hash = "sha256:72fea91746b5890f9e5e0997f16cbf3d53550580d76355ba2d998311b17b2250", size = 2139675, upload-time = "2025-10-10T16:03:31.064Z" }, + { url = "https://files.pythonhosted.org/packages/d6/ae/eeb0920537a6f9c5a3708e4a5fc55af25900216bdb4847ec29cfddf3bf3a/sqlalchemy-2.0.44-cp312-cp312-macosx_11_0_arm64.whl", hash = "sha256:585c0c852a891450edbb1eaca8648408a3cc125f18cf433941fa6babcc359e29", size = 2127726, upload-time = "2025-10-10T16:03:35.934Z" }, + { url = "https://files.pythonhosted.org/packages/d8/d5/2ebbabe0379418eda8041c06b0b551f213576bfe4c2f09d77c06c07c8cc5/sqlalchemy-2.0.44-cp312-cp312-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:9b94843a102efa9ac68a7a30cd46df3ff1ed9c658100d30a725d10d9c60a2f44", size = 3327603, upload-time = "2025-10-10T15:35:28.322Z" }, + { url = "https://files.pythonhosted.org/packages/45/e5/5aa65852dadc24b7d8ae75b7efb8d19303ed6ac93482e60c44a585930ea5/sqlalchemy-2.0.44-cp312-cp312-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:119dc41e7a7defcefc57189cfa0e61b1bf9c228211aba432b53fb71ef367fda1", size = 3337842, upload-time = "2025-10-10T15:43:45.431Z" }, + { url = "https://files.pythonhosted.org/packages/41/92/648f1afd3f20b71e880ca797a960f638d39d243e233a7082c93093c22378/sqlalchemy-2.0.44-cp312-cp312-musllinux_1_2_aarch64.whl", hash = "sha256:0765e318ee9179b3718c4fd7ba35c434f4dd20332fbc6857a5e8df17719c24d7", size = 3264558, upload-time = "2025-10-10T15:35:29.93Z" }, + { url = "https://files.pythonhosted.org/packages/40/cf/e27d7ee61a10f74b17740918e23cbc5bc62011b48282170dc4c66da8ec0f/sqlalchemy-2.0.44-cp312-cp312-musllinux_1_2_x86_64.whl", hash = "sha256:2e7b5b079055e02d06a4308d0481658e4f06bc7ef211567edc8f7d5dce52018d", size = 3301570, upload-time = "2025-10-10T15:43:48.407Z" }, + { url = "https://files.pythonhosted.org/packages/3b/3d/3116a9a7b63e780fb402799b6da227435be878b6846b192f076d2f838654/sqlalchemy-2.0.44-cp312-cp312-win32.whl", hash = "sha256:846541e58b9a81cce7dee8329f352c318de25aa2f2bbe1e31587eb1f057448b4", size = 2103447, upload-time = "2025-10-10T15:03:21.678Z" }, + { url = "https://files.pythonhosted.org/packages/25/83/24690e9dfc241e6ab062df82cc0df7f4231c79ba98b273fa496fb3dd78ed/sqlalchemy-2.0.44-cp312-cp312-win_amd64.whl", hash = "sha256:7cbcb47fd66ab294703e1644f78971f6f2f1126424d2b300678f419aa73c7b6e", size = 2130912, upload-time = "2025-10-10T15:03:24.656Z" }, + { url = "https://files.pythonhosted.org/packages/9c/5e/6a29fa884d9fb7ddadf6b69490a9d45fded3b38541713010dad16b77d015/sqlalchemy-2.0.44-py3-none-any.whl", hash = "sha256:19de7ca1246fbef9f9d1bff8f1ab25641569df226364a0e40457dc5457c54b05", size = 1928718, upload-time = "2025-10-10T15:29:45.32Z" }, +] + +[[package]] +name = "strictyaml" +version = "1.7.3" +source = { registry = "https://pypi.org/simple" } +dependencies = [ + { name = "python-dateutil" }, +] +sdist = { url = "https://files.pythonhosted.org/packages/b3/08/efd28d49162ce89c2ad61a88bd80e11fb77bc9f6c145402589112d38f8af/strictyaml-1.7.3.tar.gz", hash = "sha256:22f854a5fcab42b5ddba8030a0e4be51ca89af0267961c8d6cfa86395586c407", size = 115206, upload-time = "2023-03-10T12:50:27.062Z" } +wheels = [ + { url = "https://files.pythonhosted.org/packages/96/7c/a81ef5ef10978dd073a854e0fa93b5d8021d0594b639cc8f6453c3c78a1d/strictyaml-1.7.3-py3-none-any.whl", hash = "sha256:fb5c8a4edb43bebb765959e420f9b3978d7f1af88c80606c03fb420888f5d1c7", size = 123917, upload-time = "2023-03-10T12:50:17.242Z" }, +] + +[[package]] +name = "tenacity" +version = "9.1.2" +source = { registry = "https://pypi.org/simple" } +sdist = { url = "https://files.pythonhosted.org/packages/0a/d4/2b0cd0fe285e14b36db076e78c93766ff1d529d70408bd1d2a5a84f1d929/tenacity-9.1.2.tar.gz", hash = "sha256:1169d376c297e7de388d18b4481760d478b0e99a777cad3a9c86e556f4b697cb", size = 48036, upload-time = "2025-04-02T08:25:09.966Z" } +wheels = [ + { url = "https://files.pythonhosted.org/packages/e5/30/643397144bfbfec6f6ef821f36f33e57d35946c44a2352d3c9f0ae847619/tenacity-9.1.2-py3-none-any.whl", hash = "sha256:f77bf36710d8b73a50b2dd155c97b870017ad21afe6ab300326b0371b3b05138", size = 28248, upload-time = "2025-04-02T08:25:07.678Z" }, +] + +[[package]] +name = "tomli" +version = "2.3.0" +source = { registry = "https://pypi.org/simple" } +sdist = { url = "https://files.pythonhosted.org/packages/52/ed/3f73f72945444548f33eba9a87fc7a6e969915e7b1acc8260b30e1f76a2f/tomli-2.3.0.tar.gz", hash = "sha256:64be704a875d2a59753d80ee8a533c3fe183e3f06807ff7dc2232938ccb01549", size = 17392, upload-time = "2025-10-08T22:01:47.119Z" } +wheels = [ + { url = "https://files.pythonhosted.org/packages/b3/2e/299f62b401438d5fe1624119c723f5d877acc86a4c2492da405626665f12/tomli-2.3.0-cp311-cp311-macosx_10_9_x86_64.whl", hash = "sha256:88bd15eb972f3664f5ed4b57c1634a97153b4bac4479dcb6a495f41921eb7f45", size = 153236, upload-time = "2025-10-08T22:01:00.137Z" }, + { url = "https://files.pythonhosted.org/packages/86/7f/d8fffe6a7aefdb61bced88fcb5e280cfd71e08939da5894161bd71bea022/tomli-2.3.0-cp311-cp311-macosx_11_0_arm64.whl", hash = "sha256:883b1c0d6398a6a9d29b508c331fa56adbcdff647f6ace4dfca0f50e90dfd0ba", size = 148084, upload-time = "2025-10-08T22:01:01.63Z" }, + { url = "https://files.pythonhosted.org/packages/47/5c/24935fb6a2ee63e86d80e4d3b58b222dafaf438c416752c8b58537c8b89a/tomli-2.3.0-cp311-cp311-manylinux2014_aarch64.manylinux_2_17_aarch64.manylinux_2_28_aarch64.whl", hash = "sha256:d1381caf13ab9f300e30dd8feadb3de072aeb86f1d34a8569453ff32a7dea4bf", size = 234832, upload-time = "2025-10-08T22:01:02.543Z" }, + { url = "https://files.pythonhosted.org/packages/89/da/75dfd804fc11e6612846758a23f13271b76d577e299592b4371a4ca4cd09/tomli-2.3.0-cp311-cp311-manylinux2014_x86_64.manylinux_2_17_x86_64.manylinux_2_28_x86_64.whl", hash = "sha256:a0e285d2649b78c0d9027570d4da3425bdb49830a6156121360b3f8511ea3441", size = 242052, upload-time = "2025-10-08T22:01:03.836Z" }, + { url = "https://files.pythonhosted.org/packages/70/8c/f48ac899f7b3ca7eb13af73bacbc93aec37f9c954df3c08ad96991c8c373/tomli-2.3.0-cp311-cp311-musllinux_1_2_aarch64.whl", hash = "sha256:0a154a9ae14bfcf5d8917a59b51ffd5a3ac1fd149b71b47a3a104ca4edcfa845", size = 239555, upload-time = "2025-10-08T22:01:04.834Z" }, + { url = "https://files.pythonhosted.org/packages/ba/28/72f8afd73f1d0e7829bfc093f4cb98ce0a40ffc0cc997009ee1ed94ba705/tomli-2.3.0-cp311-cp311-musllinux_1_2_x86_64.whl", hash = "sha256:74bf8464ff93e413514fefd2be591c3b0b23231a77f901db1eb30d6f712fc42c", size = 245128, upload-time = "2025-10-08T22:01:05.84Z" }, + { url = "https://files.pythonhosted.org/packages/b6/eb/a7679c8ac85208706d27436e8d421dfa39d4c914dcf5fa8083a9305f58d9/tomli-2.3.0-cp311-cp311-win32.whl", hash = "sha256:00b5f5d95bbfc7d12f91ad8c593a1659b6387b43f054104cda404be6bda62456", size = 96445, upload-time = "2025-10-08T22:01:06.896Z" }, + { url = "https://files.pythonhosted.org/packages/0a/fe/3d3420c4cb1ad9cb462fb52967080575f15898da97e21cb6f1361d505383/tomli-2.3.0-cp311-cp311-win_amd64.whl", hash = "sha256:4dc4ce8483a5d429ab602f111a93a6ab1ed425eae3122032db7e9acf449451be", size = 107165, upload-time = "2025-10-08T22:01:08.107Z" }, + { url = "https://files.pythonhosted.org/packages/ff/b7/40f36368fcabc518bb11c8f06379a0fd631985046c038aca08c6d6a43c6e/tomli-2.3.0-cp312-cp312-macosx_10_13_x86_64.whl", hash = "sha256:d7d86942e56ded512a594786a5ba0a5e521d02529b3826e7761a05138341a2ac", size = 154891, upload-time = "2025-10-08T22:01:09.082Z" }, + { url = "https://files.pythonhosted.org/packages/f9/3f/d9dd692199e3b3aab2e4e4dd948abd0f790d9ded8cd10cbaae276a898434/tomli-2.3.0-cp312-cp312-macosx_11_0_arm64.whl", hash = "sha256:73ee0b47d4dad1c5e996e3cd33b8a76a50167ae5f96a2607cbe8cc773506ab22", size = 148796, upload-time = "2025-10-08T22:01:10.266Z" }, + { url = "https://files.pythonhosted.org/packages/60/83/59bff4996c2cf9f9387a0f5a3394629c7efa5ef16142076a23a90f1955fa/tomli-2.3.0-cp312-cp312-manylinux2014_aarch64.manylinux_2_17_aarch64.manylinux_2_28_aarch64.whl", hash = "sha256:792262b94d5d0a466afb5bc63c7daa9d75520110971ee269152083270998316f", size = 242121, upload-time = "2025-10-08T22:01:11.332Z" }, + { url = "https://files.pythonhosted.org/packages/45/e5/7c5119ff39de8693d6baab6c0b6dcb556d192c165596e9fc231ea1052041/tomli-2.3.0-cp312-cp312-manylinux2014_x86_64.manylinux_2_17_x86_64.manylinux_2_28_x86_64.whl", hash = "sha256:4f195fe57ecceac95a66a75ac24d9d5fbc98ef0962e09b2eddec5d39375aae52", size = 250070, upload-time = "2025-10-08T22:01:12.498Z" }, + { url = "https://files.pythonhosted.org/packages/45/12/ad5126d3a278f27e6701abde51d342aa78d06e27ce2bb596a01f7709a5a2/tomli-2.3.0-cp312-cp312-musllinux_1_2_aarch64.whl", hash = "sha256:e31d432427dcbf4d86958c184b9bfd1e96b5b71f8eb17e6d02531f434fd335b8", size = 245859, upload-time = "2025-10-08T22:01:13.551Z" }, + { url = "https://files.pythonhosted.org/packages/fb/a1/4d6865da6a71c603cfe6ad0e6556c73c76548557a8d658f9e3b142df245f/tomli-2.3.0-cp312-cp312-musllinux_1_2_x86_64.whl", hash = "sha256:7b0882799624980785240ab732537fcfc372601015c00f7fc367c55308c186f6", size = 250296, upload-time = "2025-10-08T22:01:14.614Z" }, + { url = "https://files.pythonhosted.org/packages/a0/b7/a7a7042715d55c9ba6e8b196d65d2cb662578b4d8cd17d882d45322b0d78/tomli-2.3.0-cp312-cp312-win32.whl", hash = "sha256:ff72b71b5d10d22ecb084d345fc26f42b5143c5533db5e2eaba7d2d335358876", size = 97124, upload-time = "2025-10-08T22:01:15.629Z" }, + { url = "https://files.pythonhosted.org/packages/06/1e/f22f100db15a68b520664eb3328fb0ae4e90530887928558112c8d1f4515/tomli-2.3.0-cp312-cp312-win_amd64.whl", hash = "sha256:1cb4ed918939151a03f33d4242ccd0aa5f11b3547d0cf30f7c74a408a5b99878", size = 107698, upload-time = "2025-10-08T22:01:16.51Z" }, + { url = "https://files.pythonhosted.org/packages/77/b8/0135fadc89e73be292b473cb820b4f5a08197779206b33191e801feeae40/tomli-2.3.0-py3-none-any.whl", hash = "sha256:e95b1af3c5b07d9e643909b5abbec77cd9f1217e6d0bca72b0234736b9fb1f1b", size = 14408, upload-time = "2025-10-08T22:01:46.04Z" }, +] + +[[package]] +name = "typing-extensions" +version = "4.15.0" +source = { registry = "https://pypi.org/simple" } +sdist = { url = "https://files.pythonhosted.org/packages/72/94/1a15dd82efb362ac84269196e94cf00f187f7ed21c242792a923cdb1c61f/typing_extensions-4.15.0.tar.gz", hash = "sha256:0cea48d173cc12fa28ecabc3b837ea3cf6f38c6d1136f85cbaaf598984861466", size = 109391, upload-time = "2025-08-25T13:49:26.313Z" } +wheels = [ + { url = "https://files.pythonhosted.org/packages/18/67/36e9267722cc04a6b9f15c7f3441c2363321a3ea07da7ae0c0707beb2a9c/typing_extensions-4.15.0-py3-none-any.whl", hash = "sha256:f0fa19c6845758ab08074a0cfa8b7aecb71c999ca73d62883bc25cc018c4e548", size = 44614, upload-time = "2025-08-25T13:49:24.86Z" }, +] + +[[package]] +name = "typing-inspection" +version = "0.4.2" +source = { registry = "https://pypi.org/simple" } +dependencies = [ + { name = "typing-extensions" }, +] +sdist = { url = "https://files.pythonhosted.org/packages/55/e3/70399cb7dd41c10ac53367ae42139cf4b1ca5f36bb3dc6c9d33acdb43655/typing_inspection-0.4.2.tar.gz", hash = "sha256:ba561c48a67c5958007083d386c3295464928b01faa735ab8547c5692e87f464", size = 75949, upload-time = "2025-10-01T02:14:41.687Z" } +wheels = [ + { url = "https://files.pythonhosted.org/packages/dc/9b/47798a6c91d8bdb567fe2698fe81e0c6b7cb7ef4d13da4114b41d239f65d/typing_inspection-0.4.2-py3-none-any.whl", hash = "sha256:4ed1cacbdc298c220f1bd249ed5287caa16f34d44ef4e9c3d0cbad5b521545e7", size = 14611, upload-time = "2025-10-01T02:14:40.154Z" }, +] + +[[package]] +name = "urllib3" +version = "2.5.0" +source = { registry = "https://pypi.org/simple" } +sdist = { url = "https://files.pythonhosted.org/packages/15/22/9ee70a2574a4f4599c47dd506532914ce044817c7752a79b6a51286319bc/urllib3-2.5.0.tar.gz", hash = "sha256:3fc47733c7e419d4bc3f6b3dc2b4f890bb743906a30d56ba4a5bfa4bbff92760", size = 393185, upload-time = "2025-06-18T14:07:41.644Z" } +wheels = [ + { url = "https://files.pythonhosted.org/packages/a7/c2/fe1e52489ae3122415c51f387e221dd0773709bad6c6cdaa599e8a2c5185/urllib3-2.5.0-py3-none-any.whl", hash = "sha256:e6b01673c0fa6a13e374b50871808eb3bf7046c4b125b216f6bf1cc604cff0dc", size = 129795, upload-time = "2025-06-18T14:07:40.39Z" }, +] From 3cb18e789fb10b837b7449f6634f2b7e1c90bbb5 Mon Sep 17 00:00:00 2001 From: Xuanwo Date: Wed, 29 Oct 2025 10:58:02 +0800 Subject: [PATCH 45/47] ci: Relax msrv check thanks to rust 2024 (#1795) ## Which issue does this PR close? - Closes #. ## What changes are included in this PR? This PR is going to relax the MSRV check thanks to rust 2024. Since starts from rust 2024, cargo has [msrv aware dep resolve](https://doc.rust-lang.org/edition-guide/rust-2024/cargo-resolver.html). This means that dep's upgrade no a problem now. ## Are these changes tested? Signed-off-by: Xuanwo Co-authored-by: Renjie Liu --- Makefile | 5 +---- 1 file changed, 1 insertion(+), 4 deletions(-) diff --git a/Makefile b/Makefile index 1b98fe8fa1..ecdada3df5 100644 --- a/Makefile +++ b/Makefile @@ -45,10 +45,7 @@ NIGHTLY_VERSION := $(shell awk -F'"' '/^channel/ {print $$2}' rust-toolchain.tom MSRV_VERSION := $(shell awk -F'"' '/^rust-version/ {print $$2}' Cargo.toml) check-msrv: - @set -e; \ - trap 'git restore Cargo.lock' EXIT; \ - cargo +$(NIGHTLY_VERSION) generate-lockfile -Z direct-minimal-versions; \ - cargo +$(MSRV_VERSION) check --locked --workspace + cargo +$(MSRV_VERSION) check --workspace check: check-fmt check-clippy check-toml cargo-machete From f10c0ab0e4b30c44679614231e1f299a20205697 Mon Sep 17 00:00:00 2001 From: Xuanwo Date: Wed, 29 Oct 2025 11:05:05 +0800 Subject: [PATCH 46/47] ci: Don't dismiss stale review to make contribution easier (#1799) MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit ## Which issue does this PR close? - Closes #. ## What changes are included in this PR? I made this change to avoid dismissing stale reviews to make contributions easier. iceberg is quite active, and we nearly have conflicts every day. It can be a pain for contributors to get approved, especially after resolving conflicts—they then have to wait for reviewers to give another approval. I think we can make it clear that trivial changes like resolving merge conflicts are allowed, but adding other code changes aren't. We can just add this to the docs so it’s easier for everyone. ## Are these changes tested? --- .asf.yaml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/.asf.yaml b/.asf.yaml index 1a766f5330..842ffc39a3 100644 --- a/.asf.yaml +++ b/.asf.yaml @@ -40,7 +40,7 @@ github: required_pull_request_reviews: required_approving_review_count: 1 - dismiss_stale_reviews: true + dismiss_stale_reviews: false required_linear_history: true del_branch_on_merge: true From d3d3127bf43bd6102acc13dbb064ae4633f8e923 Mon Sep 17 00:00:00 2001 From: Kevin Liu Date: Wed, 29 Oct 2025 01:13:18 -0700 Subject: [PATCH 47/47] add Makefile to bindings/python (#1800) ## Which issue does this PR close? - Closes #. ## What changes are included in this PR? Follow up to #1796 This PR adds ``` make install make build make test ``` to `bindings/python/` ## Are these changes tested? --- .github/workflows/bindings_python_ci.yml | 22 ++++++++++++-------- bindings/python/Makefile | 26 ++++++++++++++++++++++++ bindings/python/README.md | 6 +++--- 3 files changed, 42 insertions(+), 12 deletions(-) create mode 100644 bindings/python/Makefile diff --git a/.github/workflows/bindings_python_ci.yml b/.github/workflows/bindings_python_ci.yml index 9ea2ac0b90..55e7e5bbd0 100644 --- a/.github/workflows/bindings_python_ci.yml +++ b/.github/workflows/bindings_python_ci.yml @@ -56,17 +56,21 @@ jobs: runs-on: ubuntu-latest steps: - uses: actions/checkout@v5 + - uses: astral-sh/setup-uv@v6 + with: + version: "0.9.3" + enable-cache: true - name: Install tools run: | - pip install ruff + uv tool install ruff - name: Check format working-directory: "bindings/python" run: | - ruff format . --diff + uvx ruff format . --diff - name: Check style working-directory: "bindings/python" run: | - ruff check . + uvx ruff check . test: runs-on: ${{ matrix.os }} @@ -86,15 +90,15 @@ jobs: working-directory: "bindings/python" command: build args: --out dist --sdist - - name: Install uv - shell: bash - run: | - python -m pip install uv==0.9.3 + - uses: astral-sh/setup-uv@v6 + with: + version: "0.9.3" + enable-cache: true - name: Sync dependencies working-directory: "bindings/python" shell: bash run: | - uv sync --group dev --no-install-project + make install - name: Install built wheel working-directory: "bindings/python" shell: bash @@ -104,4 +108,4 @@ jobs: working-directory: "bindings/python" shell: bash run: | - uv run --group dev --no-sync pytest + make test diff --git a/bindings/python/Makefile b/bindings/python/Makefile new file mode 100644 index 0000000000..c36a1091dd --- /dev/null +++ b/bindings/python/Makefile @@ -0,0 +1,26 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, +# software distributed under the License is distributed on an +# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +# KIND, either express or implied. See the License for the +# specific language governing permissions and limitations +# under the License. + + +install: + uv sync --group dev --no-install-project + +build: + uv run maturin develop + +test: + uv run --no-sync pytest diff --git a/bindings/python/README.md b/bindings/python/README.md index a91599d9a4..320a9a401b 100644 --- a/bindings/python/README.md +++ b/bindings/python/README.md @@ -32,17 +32,17 @@ pip install uv==0.9.3 Set up the development environment: ```shell -uv sync --group dev +make install ``` ## Build ```shell -uv run --group dev maturin develop +make build ``` ## Test ```shell -uv run --group dev pytest +make test ```