From 372d31c6622fd305600616935248805ddc823e1c Mon Sep 17 00:00:00 2001 From: liurenjie1024 Date: Thu, 22 Aug 2024 14:09:25 +0800 Subject: [PATCH 1/9] Apply changes from Renjie --- crates/sqllogictests/Cargo.toml | 25 ++ crates/sqllogictests/src/engine/conversion.rs | 101 +++++++ .../src/engine/datafusion/error.rs | 50 ++++ .../src/engine/datafusion/mod.rs | 26 ++ .../src/engine/datafusion/normalize.rs | 279 ++++++++++++++++++ .../src/engine/datafusion/runner.rs | 87 ++++++ crates/sqllogictests/src/engine/mod.rs | 16 + crates/sqllogictests/src/engine/output.rs | 57 ++++ .../sqllogictests/src/engine/sparksql/mod.rs | 30 ++ crates/sqllogictests/src/lib.rs | 21 ++ crates/sqllogictests/src/schedule.rs | 38 +++ 11 files changed, 730 insertions(+) create mode 100644 crates/sqllogictests/Cargo.toml create mode 100644 crates/sqllogictests/src/engine/conversion.rs create mode 100644 crates/sqllogictests/src/engine/datafusion/error.rs create mode 100644 crates/sqllogictests/src/engine/datafusion/mod.rs create mode 100644 crates/sqllogictests/src/engine/datafusion/normalize.rs create mode 100644 crates/sqllogictests/src/engine/datafusion/runner.rs create mode 100644 crates/sqllogictests/src/engine/mod.rs create mode 100644 crates/sqllogictests/src/engine/output.rs create mode 100644 crates/sqllogictests/src/engine/sparksql/mod.rs create mode 100644 crates/sqllogictests/src/lib.rs create mode 100644 crates/sqllogictests/src/schedule.rs diff --git a/crates/sqllogictests/Cargo.toml b/crates/sqllogictests/Cargo.toml new file mode 100644 index 0000000000..f5667e24db --- /dev/null +++ b/crates/sqllogictests/Cargo.toml @@ -0,0 +1,25 @@ +[package] +name = "sqllogictests" +version.workspace = true +edition.workspace = true +homepage.workspace = true +repository.workspace = true +license.workspace = true +rust-version.workspace = true + +[dependencies] +arrow-schema = { workspace = true } +arrow-array= { workspace = true } +async-trait = { workspace = true } +sqllogictest = "0.21.0" +datafusion = { workspace = true, default-features = true} +datafusion-common = { workspace = true, default-features = true} +thiserror = "1.0.63" +sqlparser = {workspace = true} +itertools = "0.13.0" +half = "2.4.1" +bigdecimal = "0.4.1" +rust_decimal = { version = "1.27.0" } +tempfile = { workspace = true } +log = "0.4.22" +tokio = "1.38.0" \ No newline at end of file diff --git a/crates/sqllogictests/src/engine/conversion.rs b/crates/sqllogictests/src/engine/conversion.rs new file mode 100644 index 0000000000..937f43520b --- /dev/null +++ b/crates/sqllogictests/src/engine/conversion.rs @@ -0,0 +1,101 @@ +// 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 arrow_array::types::{Decimal128Type, Decimal256Type, DecimalType}; +use bigdecimal::BigDecimal; +use datafusion_common::arrow::datatypes::i256; +use half::f16; +use rust_decimal::prelude::*; + +/// Represents a constant for NULL string in your database. +pub const NULL_STR: &str = "NULL"; + +pub(crate) fn bool_to_str(value: bool) -> String { + if value { + "true".to_string() + } else { + "false".to_string() + } +} + +pub(crate) fn varchar_to_str(value: &str) -> String { + if value.is_empty() { + "(empty)".to_string() + } else { + value.trim_end_matches('\n').to_string() + } +} + +pub(crate) fn f16_to_str(value: f16) -> String { + if value.is_nan() { + // The sign of NaN can be different depending on platform. + // So the string representation of NaN ignores the sign. + "NaN".to_string() + } else if value == f16::INFINITY { + "Infinity".to_string() + } else if value == f16::NEG_INFINITY { + "-Infinity".to_string() + } else { + big_decimal_to_str(BigDecimal::from_str(&value.to_string()).unwrap()) + } +} + +pub(crate) fn f32_to_str(value: f32) -> String { + if value.is_nan() { + // The sign of NaN can be different depending on platform. + // So the string representation of NaN ignores the sign. + "NaN".to_string() + } else if value == f32::INFINITY { + "Infinity".to_string() + } else if value == f32::NEG_INFINITY { + "-Infinity".to_string() + } else { + big_decimal_to_str(BigDecimal::from_str(&value.to_string()).unwrap()) + } +} + +pub(crate) fn f64_to_str(value: f64) -> String { + if value.is_nan() { + // The sign of NaN can be different depending on platform. + // So the string representation of NaN ignores the sign. + "NaN".to_string() + } else if value == f64::INFINITY { + "Infinity".to_string() + } else if value == f64::NEG_INFINITY { + "-Infinity".to_string() + } else { + big_decimal_to_str(BigDecimal::from_str(&value.to_string()).unwrap()) + } +} + +pub(crate) fn i128_to_str(value: i128, precision: &u8, scale: &i8) -> String { + big_decimal_to_str( + BigDecimal::from_str(&Decimal128Type::format_decimal(value, *precision, *scale)) + .unwrap(), + ) +} + +pub(crate) fn i256_to_str(value: i256, precision: &u8, scale: &i8) -> String { + big_decimal_to_str( + BigDecimal::from_str(&Decimal256Type::format_decimal(value, *precision, *scale)) + .unwrap(), + ) +} + +pub(crate) fn big_decimal_to_str(value: BigDecimal) -> String { + value.round(12).normalized().to_string() +} \ No newline at end of file diff --git a/crates/sqllogictests/src/engine/datafusion/error.rs b/crates/sqllogictests/src/engine/datafusion/error.rs new file mode 100644 index 0000000000..ec479516cd --- /dev/null +++ b/crates/sqllogictests/src/engine/datafusion/error.rs @@ -0,0 +1,50 @@ +// 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 arrow_schema::ArrowError; +use datafusion_common::DataFusionError; +use sqllogictest::TestError; +use sqlparser::parser::ParserError; +use thiserror::Error; + +pub type Result = std::result::Result; + +/// DataFusion sql-logicaltest error +#[derive(Debug, Error)] +pub enum DFSqlLogicTestError { + /// Error from sqllogictest-rs + #[error("SqlLogicTest error(from sqllogictest-rs crate): {0}")] + SqlLogicTest(#[from] TestError), + /// Error from datafusion + #[error("DataFusion error: {0}")] + DataFusion(#[from] DataFusionError), + /// Error returned when SQL is syntactically incorrect. + #[error("SQL Parser error: {0}")] + Sql(#[from] ParserError), + /// Error from arrow-rs + #[error("Arrow error: {0}")] + Arrow(#[from] ArrowError), + /// Generic error + #[error("Other Error: {0}")] + Other(String), +} + +impl From for DFSqlLogicTestError { + fn from(value: String) -> Self { + DFSqlLogicTestError::Other(value) + } +} \ No newline at end of file diff --git a/crates/sqllogictests/src/engine/datafusion/mod.rs b/crates/sqllogictests/src/engine/datafusion/mod.rs new file mode 100644 index 0000000000..da2c218585 --- /dev/null +++ b/crates/sqllogictests/src/engine/datafusion/mod.rs @@ -0,0 +1,26 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + + +/// DataFusion engine implementation for sqllogictest. + +mod error; +mod normalize; +mod runner; + +pub use error::*; +pub use runner::*; \ No newline at end of file diff --git a/crates/sqllogictests/src/engine/datafusion/normalize.rs b/crates/sqllogictests/src/engine/datafusion/normalize.rs new file mode 100644 index 0000000000..d4e883d21d --- /dev/null +++ b/crates/sqllogictests/src/engine/datafusion/normalize.rs @@ -0,0 +1,279 @@ +// 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 arrow_schema::Fields; +use datafusion_common::format::DEFAULT_FORMAT_OPTIONS; +use datafusion_common::DataFusionError; +use std::path::PathBuf; +use std::sync::OnceLock; +use arrow_array::{ArrayRef, RecordBatch}; +use crate::engines::output::DFColumnType; + +use super::super::conversion::*; +use super::error::{DFSqlLogicTestError, Result}; + +/// Converts `batches` to a result as expected by sqllogicteset. +pub(crate) fn convert_batches(batches: Vec) -> Result>> { + if batches.is_empty() { + Ok(vec![]) + } else { + let schema = batches[0].schema(); + let mut rows = vec![]; + for batch in batches { + // Verify schema + if !schema.contains(&batch.schema()) { + return Err(DFSqlLogicTestError::DataFusion(DataFusionError::Internal( + format!( + "Schema mismatch. Previously had\n{:#?}\n\nGot:\n{:#?}", + &schema, + batch.schema() + ), + ))); + } + + let new_rows = convert_batch(batch)? + .into_iter() + .flat_map(expand_row) + .map(normalize_paths); + rows.extend(new_rows); + } + Ok(rows) + } +} + +/// special case rows that have newlines in them (like explain plans) +// +/// Transform inputs like: +/// ```text +/// [ +/// "logical_plan", +/// "Sort: d.b ASC NULLS LAST\n Projection: d.b, MAX(d.a) AS max_a", +/// ] +/// ``` +/// +/// Into one cell per line, adding lines if necessary +/// ```text +/// [ +/// "logical_plan", +/// ] +/// [ +/// "Sort: d.b ASC NULLS LAST", +/// ] +/// [ <--- newly added row +/// "|-- Projection: d.b, MAX(d.a) AS max_a", +/// ] +/// ``` +fn expand_row(mut row: Vec) -> impl Iterator> { + use itertools::Either; + use std::iter::once; + + // check last cell + if let Some(cell) = row.pop() { + let lines: Vec<_> = cell.split('\n').collect(); + + // no newlines in last cell + if lines.len() < 2 { + row.push(cell); + return Either::Left(once(row)); + } + + // form new rows with each additional line + let new_lines: Vec<_> = lines + .into_iter() + .enumerate() + .map(|(idx, l)| { + // replace any leading spaces with '-' as + // `sqllogictest` ignores whitespace differences + // + // See https://github.com/apache/datafusion/issues/6328 + let content = l.trim_start(); + let new_prefix = "-".repeat(l.len() - content.len()); + // maintain for each line a number, so + // reviewing explain result changes is easier + let line_num = idx + 1; + vec![format!("{line_num:02}){new_prefix}{content}")] + }) + .collect(); + + Either::Right(once(row).chain(new_lines)) + } else { + Either::Left(once(row)) + } +} + +/// normalize path references +/// +/// ```text +/// CsvExec: files={1 group: [[path/to/datafusion/testing/data/csv/aggregate_test_100.csv]]}, ... +/// ``` +/// +/// into: +/// +/// ```text +/// CsvExec: files={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, ... +/// ``` +fn normalize_paths(mut row: Vec) -> Vec { + row.iter_mut().for_each(|s| { + let workspace_root: &str = workspace_root().as_ref(); + if s.contains(workspace_root) { + *s = s.replace(workspace_root, "WORKSPACE_ROOT"); + } + }); + row +} + +/// return the location of the datafusion checkout +fn workspace_root() -> &'static object_store::path::Path { + static WORKSPACE_ROOT_LOCK: OnceLock = OnceLock::new(); + WORKSPACE_ROOT_LOCK.get_or_init(|| { + // e.g. /Software/datafusion/datafusion/core + let dir = PathBuf::from(env!("CARGO_MANIFEST_DIR")); + + // e.g. /Software/datafusion/datafusion + let workspace_root = dir + .parent() + .expect("Can not find parent of datafusion/core") + // e.g. /Software/datafusion + .parent() + .expect("parent of datafusion") + .to_string_lossy(); + + let sanitized_workplace_root = if cfg!(windows) { + // Object store paths are delimited with `/`, e.g. `/datafusion/datafusion/testing/data/csv/aggregate_test_100.csv`. + // The default windows delimiter is `\`, so the workplace path is `datafusion\datafusion`. + workspace_root + .replace(std::path::MAIN_SEPARATOR, object_store::path::DELIMITER) + } else { + workspace_root.to_string() + }; + + object_store::path::Path::parse(sanitized_workplace_root).unwrap() + }) +} + +/// Convert a single batch to a `Vec>` for comparison +fn convert_batch(batch: RecordBatch) -> Result>> { + (0..batch.num_rows()) + .map(|row| { + batch + .columns() + .iter() + .map(|col| cell_to_string(col, row)) + .collect::>>() + }) + .collect() +} + +macro_rules! get_row_value { + ($array_type:ty, $column: ident, $row: ident) => {{ + let array = $column.as_any().downcast_ref::<$array_type>().unwrap(); + + array.value($row) + }}; +} + +/// Normalizes the content of a single cell in RecordBatch prior to printing. +/// +/// This is to make the output comparable to the semi-standard .slt format +/// +/// Normalizations applied to [NULL Values and empty strings] +/// +/// [NULL Values and empty strings]: https://duckdb.org/dev/sqllogictest/result_verification#null-values-and-empty-strings +/// +/// Floating numbers are rounded to have a consistent representation with the Postgres runner. +/// +pub fn cell_to_string(col: &ArrayRef, row: usize) -> Result { + if !col.is_valid(row) { + // represent any null value with the string "NULL" + Ok(NULL_STR.to_string()) + } else { + match col.data_type() { + DataType::Null => Ok(NULL_STR.to_string()), + DataType::Boolean => { + Ok(bool_to_str(get_row_value!(array::BooleanArray, col, row))) + } + DataType::Float16 => { + Ok(f16_to_str(get_row_value!(array::Float16Array, col, row))) + } + DataType::Float32 => { + Ok(f32_to_str(get_row_value!(array::Float32Array, col, row))) + } + DataType::Float64 => { + Ok(f64_to_str(get_row_value!(array::Float64Array, col, row))) + } + DataType::Decimal128(precision, scale) => { + let value = get_row_value!(array::Decimal128Array, col, row); + Ok(i128_to_str(value, precision, scale)) + } + DataType::Decimal256(precision, scale) => { + let value = get_row_value!(array::Decimal256Array, col, row); + Ok(i256_to_str(value, precision, scale)) + } + DataType::LargeUtf8 => Ok(varchar_to_str(get_row_value!( + array::LargeStringArray, + col, + row + ))), + DataType::Utf8 => { + Ok(varchar_to_str(get_row_value!(array::StringArray, col, row))) + } + DataType::Utf8View => Ok(varchar_to_str(get_row_value!( + array::StringViewArray, + col, + row + ))), + _ => { + let f = ArrayFormatter::try_new(col.as_ref(), &DEFAULT_FORMAT_OPTIONS); + Ok(f.unwrap().value(row).to_string()) + } + } + .map_err(DFSqlLogicTestError::Arrow) + } +} + +/// Converts columns to a result as expected by sqllogicteset. +pub(crate) fn convert_schema_to_types(columns: &Fields) -> Vec { + columns + .iter() + .map(|f| f.data_type()) + .map(|data_type| match data_type { + DataType::Boolean => DFColumnType::Boolean, + DataType::Int8 + | DataType::Int16 + | DataType::Int32 + | DataType::Int64 + | DataType::UInt8 + | DataType::UInt16 + | DataType::UInt32 + | DataType::UInt64 => DFColumnType::Integer, + DataType::Float16 + | DataType::Float32 + | DataType::Float64 + | DataType::Decimal128(_, _) + | DataType::Decimal256(_, _) => DFColumnType::Float, + DataType::Utf8 | DataType::LargeUtf8 | DataType::Utf8View => { + DFColumnType::Text + } + DataType::Date32 + | DataType::Date64 + | DataType::Time32(_) + | DataType::Time64(_) => DFColumnType::DateTime, + DataType::Timestamp(_, _) => DFColumnType::Timestamp, + _ => DFColumnType::Another, + }) + .collect() +} \ No newline at end of file diff --git a/crates/sqllogictests/src/engine/datafusion/runner.rs b/crates/sqllogictests/src/engine/datafusion/runner.rs new file mode 100644 index 0000000000..e39f62b1be --- /dev/null +++ b/crates/sqllogictests/src/engine/datafusion/runner.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::sync::Arc; +use std::{path::PathBuf, time::Duration}; +use arrow_array::RecordBatch; +use async_trait::async_trait; +use datafusion::physical_plan::common::collect; +use datafusion::physical_plan::execute_stream; +use datafusion::prelude::SessionContext; +use log::info; +use sqllogictest::DBOutput; + +use super::{error::Result, normalize, DFSqlLogicTestError}; + +use crate::engine::output::{DFColumnType, DFOutput}; + +pub struct DataFusionEngine { + ctx: SessionContext, + relative_path: PathBuf, +} + +impl DataFusionEngine { + pub fn new(ctx: SessionContext, relative_path: PathBuf) -> Self { + Self { ctx, relative_path } + } +} + +#[async_trait] +impl sqllogictest::AsyncDB for DataFusionEngine { + type Error = DFSqlLogicTestError; + type ColumnType = DFColumnType; + + async fn run(&mut self, sql: &str) -> Result { + info!( + "[{}] Running query: \"{}\"", + self.relative_path.display(), + sql + ); + run_query(&self.ctx, sql).await + } + + /// Engine name of current database. + fn engine_name(&self) -> &str { + "DataFusion" + } + + /// [`DataFusionEngine`] calls this function to perform sleep. + /// + /// The default implementation is `std::thread::sleep`, which is universal to any async runtime + /// but would block the current thread. If you are running in tokio runtime, you should override + /// this by `tokio::time::sleep`. + async fn sleep(dur: Duration) { + tokio::time::sleep(dur).await; + } +} + +async fn run_query(ctx: &SessionContext, sql: impl Into) -> Result { + let df = ctx.sql(sql.into().as_str()).await?; + let task_ctx = Arc::new(df.task_ctx()); + let plan = df.create_physical_plan().await?; + + let stream = execute_stream(plan, task_ctx)?; + let types = normalize::convert_schema_to_types(stream.schema().fields()); + let results: Vec = collect(stream).await?; + let rows = normalize::convert_batches(results)?; + + if rows.is_empty() && types.is_empty() { + Ok(DBOutput::StatementComplete(0)) + } else { + Ok(DBOutput::Rows { types, rows }) + } +} \ No newline at end of file diff --git a/crates/sqllogictests/src/engine/mod.rs b/crates/sqllogictests/src/engine/mod.rs new file mode 100644 index 0000000000..ba2a1f4151 --- /dev/null +++ b/crates/sqllogictests/src/engine/mod.rs @@ -0,0 +1,16 @@ +mod datafusion; + +use std::sync::Arc; +pub use datafusion::*; + +mod sparksql; +mod conversion; +mod output; + +pub use sparksql::*; + + +pub enum Engine { + DataFusion(Arc), + SparkSQL(Arc), +} \ No newline at end of file diff --git a/crates/sqllogictests/src/engine/output.rs b/crates/sqllogictests/src/engine/output.rs new file mode 100644 index 0000000000..ae1030ca4a --- /dev/null +++ b/crates/sqllogictests/src/engine/output.rs @@ -0,0 +1,57 @@ +// 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 sqllogictest::{ColumnType, DBOutput}; + +#[derive(Debug, PartialEq, Eq, Clone)] +pub enum DFColumnType { + Boolean, + DateTime, + Integer, + Float, + Text, + Timestamp, + Another, +} + +impl ColumnType for DFColumnType { + fn from_char(value: char) -> Option { + match value { + 'B' => Some(Self::Boolean), + 'D' => Some(Self::DateTime), + 'I' => Some(Self::Integer), + 'P' => Some(Self::Timestamp), + 'R' => Some(Self::Float), + 'T' => Some(Self::Text), + _ => Some(Self::Another), + } + } + + fn to_char(&self) -> char { + match self { + Self::Boolean => 'B', + Self::DateTime => 'D', + Self::Integer => 'I', + Self::Timestamp => 'P', + Self::Float => 'R', + Self::Text => 'T', + Self::Another => '?', + } + } +} + +pub(crate) type DFOutput = DBOutput; \ No newline at end of file diff --git a/crates/sqllogictests/src/engine/sparksql/mod.rs b/crates/sqllogictests/src/engine/sparksql/mod.rs new file mode 100644 index 0000000000..aeff2616e8 --- /dev/null +++ b/crates/sqllogictests/src/engine/sparksql/mod.rs @@ -0,0 +1,30 @@ +// 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 sqllogictest::{AsyncDB, DBOutput}; + +/// SparkSql engine implementation for sqllogictest. +pub struct SparkSqlEngine; + +impl AsyncDB for SparkSqlEngine { + type Error = (); + type ColumnType = (); + + async fn run(&mut self, sql: &str) -> Result, Self::Error> { + todo!() + } +} diff --git a/crates/sqllogictests/src/lib.rs b/crates/sqllogictests/src/lib.rs new file mode 100644 index 0000000000..d01802f61d --- /dev/null +++ b/crates/sqllogictests/src/lib.rs @@ -0,0 +1,21 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +// This lib contains codes copied from +// [Apache Datafusion](https://github.com/apache/datafusion/tree/main/datafusion/sqllogictest) +mod engine; +mod schedule; diff --git a/crates/sqllogictests/src/schedule.rs b/crates/sqllogictests/src/schedule.rs new file mode 100644 index 0000000000..26e789bb09 --- /dev/null +++ b/crates/sqllogictests/src/schedule.rs @@ -0,0 +1,38 @@ +// 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::path::Path; +use crate::engine::Engine; + +/// Schedule of engines to run tests. +pub struct Schedule { + /// Map of engine names to engine instances. + engines: HashMap, + /// List of steps to run, each step is a sql file. + steps: Vec, +} + +impl Schedule { + pub async fn parse>(_schedule_def_file: P) -> Self { + todo!() + } + + pub async fn run(mut self) { + todo!() + } +} From 6ea9c3c6c03359ab88a5d976ff3b4f3d4667e410 Mon Sep 17 00:00:00 2001 From: Leon Lin Date: Wed, 13 Aug 2025 14:28:00 -0700 Subject: [PATCH 2/9] Working on sqllogic test --- Cargo.lock | 2175 +++++++++-------- Cargo.toml | 3 + .../integrations/datafusion/src/table/mod.rs | 2 +- crates/sqllogictest/Cargo.toml | 23 + crates/sqllogictest/src/display/conversion.rs | 82 + crates/sqllogictest/src/display/mod.rs | 1 + crates/sqllogictest/src/engine/datafusion.rs | 113 +- crates/sqllogictest/src/engine/mod.rs | 14 +- crates/sqllogictest/src/engine/spark.rs | 117 + crates/sqllogictest/src/lib.rs | 6 +- crates/sqllogictest/src/schedule.rs | 163 ++ .../testdata/docker/docker-compose.yml | 78 + .../docker/spark/spark-connect-server.sh | 27 + .../sqllogictest/testdata/schedules/test.toml | 11 + .../testdata/slts/demo/prepare.slt | 18 + .../testdata/slts/demo/verify.slt | 6 + crates/sqllogictest/tests/sqllogictests.rs | 112 + 17 files changed, 1937 insertions(+), 1014 deletions(-) create mode 100644 crates/sqllogictest/src/display/conversion.rs create mode 100644 crates/sqllogictest/src/display/mod.rs create mode 100644 crates/sqllogictest/src/engine/spark.rs create mode 100644 crates/sqllogictest/src/schedule.rs create mode 100644 crates/sqllogictest/testdata/docker/docker-compose.yml create mode 100755 crates/sqllogictest/testdata/docker/spark/spark-connect-server.sh create mode 100644 crates/sqllogictest/testdata/schedules/test.toml create mode 100644 crates/sqllogictest/testdata/slts/demo/prepare.slt create mode 100644 crates/sqllogictest/testdata/slts/demo/verify.slt create mode 100644 crates/sqllogictest/tests/sqllogictests.rs diff --git a/Cargo.lock b/Cargo.lock index 48a995242f..5c50e74490 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -13,9 +13,9 @@ dependencies = [ [[package]] name = "adler2" -version = "2.0.1" +version = "2.0.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "320119579fcad9c21884f5c4861d16174d0e06250625266f50fe6898340abefa" +checksum = "512761e0bb2578dd7380c6baaa0f4ce03e84f95e960231d1dec8bf4d7d6e2627" [[package]] name = "adler32" @@ -40,24 +40,24 @@ version = "0.7.8" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "891477e0c6a8957309ee5c45a6368af3ae14bb510732d2684ffa19af310920f9" dependencies = [ - "getrandom 0.2.16", + "getrandom 0.2.15", "once_cell", "version_check", ] [[package]] name = "ahash" -version = "0.8.12" +version = "0.8.11" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "5a15f179cd60c4584b8a8c596927aadc462e27f2ca70c04e0071964a73ba7a75" +checksum = "e89da841a80418a9b391ebaea17f5c112ffaaa96f621d2c285b5174da76b9011" dependencies = [ "cfg-if", "const-random", - "getrandom 0.3.3", + "getrandom 0.2.15", "once_cell", "serde", "version_check", - "zerocopy", + "zerocopy 0.7.35", ] [[package]] @@ -107,9 +107,9 @@ dependencies = [ [[package]] name = "anstream" -version = "0.6.20" +version = "0.6.18" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "3ae563653d1938f79b1ab1b5e668c87c76a9930414574a6583a7b7e11a8e6192" +checksum = "8acc5369981196006228e28809f761875c0327210a891e941f4c683b3a99529b" dependencies = [ "anstyle", "anstyle-parse", @@ -122,44 +122,44 @@ dependencies = [ [[package]] name = "anstyle" -version = "1.0.11" +version = "1.0.10" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "862ed96ca487e809f1c8e5a8447f6ee2cf102f846893800b20cebdf541fc6bbd" +checksum = "55cc3b69f167a1ef2e161439aa98aed94e6028e5f9a59be9a6ffb47aef1651f9" [[package]] name = "anstyle-parse" -version = "0.2.7" +version = "0.2.6" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "4e7644824f0aa2c7b9384579234ef10eb7efb6a0deb83f9630a49594dd9c15c2" +checksum = "3b2d16507662817a6a20a9ea92df6652ee4f94f914589377d69f3b21bc5798a9" dependencies = [ "utf8parse", ] [[package]] name = "anstyle-query" -version = "1.1.4" +version = "1.1.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "9e231f6134f61b71076a3eab506c379d4f36122f2af15a9ff04415ea4c3339e2" +checksum = "79947af37f4177cfead1110013d678905c37501914fba0efea834c3fe9a8d60c" dependencies = [ - "windows-sys 0.60.2", + "windows-sys 0.59.0", ] [[package]] name = "anstyle-wincon" -version = "3.0.10" +version = "3.0.7" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "3e0633414522a32ffaac8ac6cc8f748e090c5717661fddeea04219e2344f5f2a" +checksum = "ca3534e77181a9cc07539ad51f2141fe32f6c3ffd4df76db8ad92346b003ae4e" dependencies = [ "anstyle", - "once_cell_polyfill", - "windows-sys 0.60.2", + "once_cell", + "windows-sys 0.59.0", ] [[package]] name = "anyhow" -version = "1.0.99" +version = "1.0.95" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "b0674a1ddeecb70197781e945de4b3b8ffb61fa939a5597bcf48503737663100" +checksum = "34ac096ce696dc2fcabef30516bb13c0a68a11d30131d3df6f04711467681b04" [[package]] name = "apache-avro" @@ -210,9 +210,9 @@ checksum = "7c02d123df017efcdfbd739ef81735b36c5ba83ec3c59c80a9d7ecc718f92e50" [[package]] name = "arrow" -version = "55.2.0" +version = "55.0.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "f3f15b4c6b148206ff3a2b35002e08929c2462467b62b9c02036d9c34f9ef994" +checksum = "3095aaf545942ff5abd46654534f15b03a90fba78299d661e045e5d587222f0d" dependencies = [ "arrow-arith", "arrow-array", @@ -249,14 +249,14 @@ version = "55.2.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "70732f04d285d49054a48b72c54f791bb3424abae92d27aafdf776c98af161c8" dependencies = [ - "ahash 0.8.12", + "ahash 0.8.11", "arrow-buffer", "arrow-data", "arrow-schema", "chrono", - "chrono-tz 0.10.4", + "chrono-tz 0.10.1", "half", - "hashbrown 0.15.5", + "hashbrown 0.15.2", "num", ] @@ -294,9 +294,9 @@ dependencies = [ [[package]] name = "arrow-csv" -version = "55.2.0" +version = "55.0.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "012c9fef3f4a11573b2c74aec53712ff9fdae4a95f4ce452d1bbf088ee00f06b" +checksum = "510db7dfbb4d5761826516cc611d97b3a68835d0ece95b034a052601109c0b1b" dependencies = [ "arrow-array", "arrow-cast", @@ -304,6 +304,7 @@ dependencies = [ "chrono", "csv", "csv-core", + "lazy_static", "regex", ] @@ -335,9 +336,9 @@ dependencies = [ [[package]] name = "arrow-json" -version = "55.2.0" +version = "55.0.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "5709d974c4ea5be96d900c01576c7c0b99705f4a3eec343648cb1ca863988a9c" +checksum = "d8dafd17a05449e31e0114d740530e0ada7379d7cb9c338fd65b09a8130960b0" dependencies = [ "arrow-array", "arrow-buffer", @@ -346,7 +347,7 @@ dependencies = [ "arrow-schema", "chrono", "half", - "indexmap 2.10.0", + "indexmap 2.9.0", "lexical-core", "memchr", "num", @@ -370,9 +371,9 @@ dependencies = [ [[package]] name = "arrow-row" -version = "55.2.0" +version = "55.0.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "52bf7393166beaf79b4bed9bfdf19e97472af32ce5b6b48169d321518a08cae2" +checksum = "9be8a2a4e5e7d9c822b2b8095ecd77010576d824f654d347817640acfc97d229" dependencies = [ "arrow-array", "arrow-buffer", @@ -393,7 +394,7 @@ version = "55.2.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "dd2b45757d6a2373faa3352d02ff5b54b098f5e21dccebc45a21806bc34501e5" dependencies = [ - "ahash 0.8.12", + "ahash 0.8.11", "arrow-array", "arrow-buffer", "arrow-data", @@ -450,7 +451,7 @@ version = "0.7.2" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "435a87a52755b8f27fcf321ac4f04b2802e337c8c4872923137471ec39c37532" dependencies = [ - "event-listener 5.4.1", + "event-listener 5.4.0", "event-listener-strategy", "futures-core", "pin-project-lite", @@ -469,9 +470,9 @@ dependencies = [ [[package]] name = "async-channel" -version = "2.5.0" +version = "2.3.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "924ed96dd52d1b75e9c1a3e6275715fd320f5f9439fb5a4a11fa51f4221158d2" +checksum = "89b47800b0be77592da0afd425cc03468052844aff33b84e33cc696f64e77b6a" dependencies = [ "concurrent-queue", "event-listener-strategy", @@ -498,15 +499,14 @@ dependencies = [ [[package]] name = "async-executor" -version = "1.13.2" +version = "1.13.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "bb812ffb58524bdd10860d7d974e2f01cc0950c2438a74ee5ec2e2280c6c4ffa" +checksum = "30ca9a001c1e8ba5149f91a74362376cc6bc5b919d92d988668657bd570bdcec" dependencies = [ "async-task", "concurrent-queue", "fastrand", "futures-lite", - "pin-project-lite", "slab", ] @@ -516,7 +516,7 @@ version = "2.4.1" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "05b1b633a2115cd122d73b955eadd9916c18c8f510ec9cd1686404c60ad1c29c" dependencies = [ - "async-channel 2.5.0", + "async-channel 2.3.1", "async-executor", "async-io", "async-lock", @@ -527,9 +527,9 @@ dependencies = [ [[package]] name = "async-io" -version = "2.5.0" +version = "2.4.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "19634d6336019ef220f09fd31168ce5c184b295cbf80345437cc36094ef223ca" +checksum = "43a2b323ccce0a1d90b449fd71f2a06ca7faa7c54c2751f06c9bd851fc061059" dependencies = [ "async-lock", "cfg-if", @@ -538,18 +538,19 @@ dependencies = [ "futures-lite", "parking", "polling", - "rustix 1.0.8", + "rustix 0.38.44", "slab", - "windows-sys 0.60.2", + "tracing", + "windows-sys 0.59.0", ] [[package]] name = "async-lock" -version = "3.4.1" +version = "3.4.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "5fd03604047cee9b6ce9de9f70c6cd540a0520c813cbd49bae61f33ab80ed1dc" +checksum = "ff6e472cdea888a4bd64f342f09b3f50e1886d32afe8df3d663c01140b811b18" dependencies = [ - "event-listener 5.4.1", + "event-listener 5.4.0", "event-listener-strategy", "pin-project-lite", ] @@ -562,14 +563,14 @@ checksum = "3b43422f69d8ff38f95f1b2bb76517c91589a924d1559a0e935d7c8ce0274c11" dependencies = [ "proc-macro2", "quote", - "syn 2.0.106", + "syn 2.0.101", ] [[package]] name = "async-std" -version = "1.13.2" +version = "1.13.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "2c8e079a4ab67ae52b7403632e4618815d6db36d2a010cfe41b02c1b1578f93b" +checksum = "c634475f29802fde2b8f0b505b1bd00dfe4df7d4a000f0b36f7671197d5c3615" dependencies = [ "async-attributes", "async-channel 1.9.0", @@ -592,6 +593,28 @@ dependencies = [ "wasm-bindgen-futures", ] +[[package]] +name = "async-stream" +version = "0.3.6" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "0b5a71a6f37880a80d1d7f19efd781e4b5de42c88f0722cc13bcb6cc2cfe8476" +dependencies = [ + "async-stream-impl", + "futures-core", + "pin-project-lite", +] + +[[package]] +name = "async-stream-impl" +version = "0.3.6" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "c7c24de15d275a1ecfd47a380fb4d5ec9bfe0933f309ed5e705b775596a3574d" +dependencies = [ + "proc-macro2", + "quote", + "syn 2.0.101", +] + [[package]] name = "async-task" version = "4.7.1" @@ -600,13 +623,13 @@ checksum = "8b75356056920673b02621b35afd0f7dda9306d03c79a30f5c56c44cf256e3de" [[package]] name = "async-trait" -version = "0.1.89" +version = "0.1.88" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "9035ad2d096bed7955a320ee7e2230574d28fd3c3a0f186cbea1ff3c7eed5dbb" +checksum = "e539d3fca749fcee5236ab05e93a52867dd549cc157c8cb7f99595f3cedffdb5" dependencies = [ "proc-macro2", "quote", - "syn 2.0.106", + "syn 2.0.101", ] [[package]] @@ -626,15 +649,15 @@ checksum = "1505bd5d3d116872e7271a6d4e16d81d0c8570876c8de68093a09ac269d8aac0" [[package]] name = "autocfg" -version = "1.5.0" +version = "1.4.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c08606f8c3cbf4ce6ec8e28fb0014a2c086708fe954eaa885384a6165172e7e8" +checksum = "ace50bade8e6234aa140d9a2f552bbee1db4d353f69b8217bc503490fc1a9f26" [[package]] name = "aws-config" -version = "1.8.5" +version = "1.6.3" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c478f5b10ce55c9a33f87ca3404ca92768b144fc1bfdede7c0121214a8283a25" +checksum = "02a18fd934af6ae7ca52410d4548b98eb895aab0f1ea417d168d85db1434a141" dependencies = [ "aws-credential-types", "aws-runtime", @@ -662,9 +685,9 @@ dependencies = [ [[package]] name = "aws-credential-types" -version = "1.2.5" +version = "1.2.3" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "1541072f81945fa1251f8795ef6c92c4282d74d59f88498ae7d4bf00f0ebdad9" +checksum = "687bc16bc431a8533fe0097c7f0182874767f920989d7260950172ae8e3c4465" dependencies = [ "aws-smithy-async", "aws-smithy-runtime-api", @@ -674,9 +697,9 @@ dependencies = [ [[package]] name = "aws-lc-rs" -version = "1.13.3" +version = "1.13.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "5c953fe1ba023e6b7730c0d4b031d06f267f23a46167dcbd40316644b10a17ba" +checksum = "19b756939cb2f8dc900aa6dcd505e6e2428e9cae7ff7b028c49e3946efa70878" dependencies = [ "aws-lc-sys", "zeroize", @@ -684,9 +707,9 @@ dependencies = [ [[package]] name = "aws-lc-sys" -version = "0.30.0" +version = "0.28.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "dbfd150b5dbdb988bcc8fb1fe787eb6b7ee6180ca24da683b61ea5405f3d43ff" +checksum = "bfa9b6986f250236c27e5a204062434a773a13243d2ffc2955f37bdba4c5c6a1" dependencies = [ "bindgen", "cc", @@ -697,9 +720,9 @@ dependencies = [ [[package]] name = "aws-runtime" -version = "1.5.10" +version = "1.5.7" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c034a1bc1d70e16e7f4e4caf7e9f7693e4c9c24cd91cf17c2a0b21abaebc7c8b" +checksum = "6c4063282c69991e57faab9e5cb21ae557e59f5b0fb285c196335243df8dc25c" dependencies = [ "aws-credential-types", "aws-sigv4", @@ -721,9 +744,9 @@ dependencies = [ [[package]] name = "aws-sdk-glue" -version = "1.116.0" +version = "1.99.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "b4b44ba32edf94e0723dfa1ee340170925858012ac0c981f4ee220c7455014bf" +checksum = "6d3964f2ad78d3fe39a327f109faaf0d0d7b2aa1742d99fb9434a36c99ca542a" dependencies = [ "aws-credential-types", "aws-runtime", @@ -743,9 +766,9 @@ dependencies = [ [[package]] name = "aws-sdk-s3tables" -version = "1.35.0" +version = "1.25.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c716a0dc793c4f21fd1f99737dc041cb7e1b82e9eb0bcf834f538b484ba667df" +checksum = "3382afb09173e6f30369399041f0d3920031b14878d4b013ba7d3f04052359dd" dependencies = [ "aws-credential-types", "aws-runtime", @@ -765,9 +788,9 @@ dependencies = [ [[package]] name = "aws-sdk-sso" -version = "1.81.0" +version = "1.72.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "79ede098271e3471036c46957cba2ba30888f53bda2515bf04b560614a30a36e" +checksum = "13118ad30741222f67b1a18e5071385863914da05124652b38e172d6d3d9ce31" dependencies = [ "aws-credential-types", "aws-runtime", @@ -787,9 +810,9 @@ dependencies = [ [[package]] name = "aws-sdk-ssooidc" -version = "1.82.0" +version = "1.73.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "43326f724ba2cc957e6f3deac0ca1621a3e5d4146f5970c24c8a108dac33070f" +checksum = "f879a8572b4683a8f84f781695bebf2f25cf11a81a2693c31fc0e0215c2c1726" dependencies = [ "aws-credential-types", "aws-runtime", @@ -809,9 +832,9 @@ dependencies = [ [[package]] name = "aws-sdk-sts" -version = "1.83.0" +version = "1.73.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a5468593c47efc31fdbe6c902d1a5fde8d9c82f78a3f8ccfe907b1e9434748cb" +checksum = "f1e9c3c24e36183e2f698235ed38dcfbbdff1d09b9232dc866c4be3011e0b47e" dependencies = [ "aws-credential-types", "aws-runtime", @@ -832,9 +855,9 @@ dependencies = [ [[package]] name = "aws-sigv4" -version = "1.3.4" +version = "1.3.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "084c34162187d39e3740cb635acd73c4e3a551a36146ad6fe8883c929c9f876c" +checksum = "3503af839bd8751d0bdc5a46b9cac93a003a353e635b0c12cf2376b5b53e41ea" dependencies = [ "aws-credential-types", "aws-smithy-http", @@ -865,9 +888,9 @@ dependencies = [ [[package]] name = "aws-smithy-http" -version = "0.62.3" +version = "0.62.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "7c4dacf2d38996cf729f55e7a762b30918229917eca115de45dfa8dfb97796c9" +checksum = "99335bec6cdc50a346fda1437f9fefe33abf8c99060739a546a16457f2862ca9" dependencies = [ "aws-smithy-runtime-api", "aws-smithy-types", @@ -885,39 +908,37 @@ dependencies = [ [[package]] name = "aws-smithy-http-client" -version = "1.1.0" +version = "1.0.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "4fdbad9bd9dbcc6c5e68c311a841b54b70def3ca3b674c42fbebb265980539f8" +checksum = "8aff1159006441d02e57204bf57a1b890ba68bedb6904ffd2873c1c4c11c546b" dependencies = [ "aws-smithy-async", "aws-smithy-runtime-api", "aws-smithy-types", - "h2 0.3.27", - "h2 0.4.12", + "h2 0.4.7", "http 0.2.12", "http 1.3.1", "http-body 0.4.6", "hyper 0.14.32", - "hyper 1.7.0", + "hyper 1.6.0", "hyper-rustls 0.24.2", - "hyper-rustls 0.27.7", + "hyper-rustls 0.27.5", "hyper-util", "pin-project-lite", "rustls 0.21.12", - "rustls 0.23.31", + "rustls 0.23.23", "rustls-native-certs 0.8.1", "rustls-pki-types", "tokio", - "tokio-rustls 0.26.2", - "tower", + "tower 0.5.2", "tracing", ] [[package]] name = "aws-smithy-json" -version = "0.61.4" +version = "0.61.3" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a16e040799d29c17412943bdbf488fd75db04112d0c0d4b9290bacf5ae0014b9" +checksum = "92144e45819cae7dc62af23eac5a038a58aa544432d2102609654376a900bd07" dependencies = [ "aws-smithy-types", ] @@ -943,9 +964,9 @@ dependencies = [ [[package]] name = "aws-smithy-runtime" -version = "1.9.0" +version = "1.8.3" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a3d57c8b53a72d15c8e190475743acf34e4996685e346a3448dd54ef696fc6e0" +checksum = "14302f06d1d5b7d333fd819943075b13d27c7700b414f574c3c35859bfb55d5e" dependencies = [ "aws-smithy-async", "aws-smithy-http", @@ -967,9 +988,9 @@ dependencies = [ [[package]] name = "aws-smithy-runtime-api" -version = "1.9.0" +version = "1.8.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "07f5e0fc8a6b3f2303f331b94504bbf754d85488f402d6f1dd7a6080f99afe56" +checksum = "a1e5d9e3a80a18afa109391fb5ad09c3daf887b516c6fd805a157c6ea7994a57" dependencies = [ "aws-smithy-async", "aws-smithy-types", @@ -984,9 +1005,9 @@ dependencies = [ [[package]] name = "aws-smithy-types" -version = "1.3.2" +version = "1.3.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d498595448e43de7f4296b7b7a18a8a02c61ec9349128c80a368f7c3b4ab11a8" +checksum = "40076bd09fadbc12d5e026ae080d0930defa606856186e31d83ccc6a255eeaf3" dependencies = [ "base64-simd", "bytes", @@ -1010,18 +1031,18 @@ dependencies = [ [[package]] name = "aws-smithy-xml" -version = "0.60.10" +version = "0.60.9" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "3db87b96cb1b16c024980f133968d52882ca0daaee3a086c6decc500f6c99728" +checksum = "ab0b0166827aa700d3dc519f72f8b3a91c35d0b8d042dc5d643a91e6f80648fc" dependencies = [ "xmlparser", ] [[package]] name = "aws-types" -version = "1.3.8" +version = "1.3.7" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "b069d19bf01e46298eaedd7c6f283fe565a59263e53eebec945f3e6398f42390" +checksum = "8a322fec39e4df22777ed3ad8ea868ac2f94cd15e1a55f6ee8d8d6305057689a" dependencies = [ "aws-credential-types", "aws-smithy-async", @@ -1031,11 +1052,56 @@ dependencies = [ "tracing", ] +[[package]] +name = "axum" +version = "0.6.20" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "3b829e4e32b91e643de6eafe82b1d90675f5874230191a4ffbc1b336dec4d6bf" +dependencies = [ + "async-trait", + "axum-core", + "bitflags 1.3.2", + "bytes", + "futures-util", + "http 0.2.12", + "http-body 0.4.6", + "hyper 0.14.32", + "itoa", + "matchit", + "memchr", + "mime", + "percent-encoding", + "pin-project-lite", + "rustversion", + "serde", + "sync_wrapper 0.1.2", + "tower 0.4.13", + "tower-layer", + "tower-service", +] + +[[package]] +name = "axum-core" +version = "0.3.4" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "759fa577a247914fd3f7f76d62972792636412fbfd634cd452f6a385a74d2d2c" +dependencies = [ + "async-trait", + "bytes", + "futures-util", + "http 0.2.12", + "http-body 0.4.6", + "mime", + "rustversion", + "tower-layer", + "tower-service", +] + [[package]] name = "backon" -version = "1.5.2" +version = "1.5.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "592277618714fbcecda9a02ba7a8781f319d26532a88553bbacc77ba5d2b3a8d" +checksum = "302eaff5357a264a2c42f127ecb8bac761cf99749fc3dc95677e2743991f99e7" dependencies = [ "fastrand", "gloo-timers", @@ -1044,9 +1110,9 @@ dependencies = [ [[package]] name = "backtrace" -version = "0.3.75" +version = "0.3.74" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "6806a6321ec58106fea15becdad98371e28d92ccbc7c8f1b3b6dd724fe8f1002" +checksum = "8d82cb332cdfaed17ae235a638438ac4d4839913cc2af585c3c6746e8f8bee1a" dependencies = [ "addr2line", "cfg-if", @@ -1081,9 +1147,9 @@ dependencies = [ [[package]] name = "base64ct" -version = "1.8.0" +version = "1.6.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "55248b47b0caf0546f7988906588779981c43bb1bc9d0c44087278f80cdb44ba" +checksum = "8c3c1a368f70d6cf7302d78f8f7093da241fb8e8807c05cc9e51a125895a6d5b" [[package]] name = "bigdecimal" @@ -1111,7 +1177,7 @@ version = "0.69.5" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "271383c67ccabffb7381723dea0672a673f292304fcb45c01cc648c7a8d58088" dependencies = [ - "bitflags", + "bitflags 2.8.0", "cexpr", "clang-sys", "itertools 0.12.1", @@ -1124,15 +1190,21 @@ dependencies = [ "regex", "rustc-hash 1.1.0", "shlex", - "syn 2.0.106", + "syn 2.0.101", "which", ] [[package]] name = "bitflags" -version = "2.9.2" +version = "1.3.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "bef38d45163c2f1dde094a7dfd33ccf595c92905c8f8f4fdc18d06fb1037718a" + +[[package]] +name = "bitflags" +version = "2.8.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "6a65b545ab31d687cff52899d4890855fec459eb6afe0da6417b8a18da87aa29" +checksum = "8f68f53c83ab957f72c32642f3868eec03eb974d1fb82e453128456482613d36" dependencies = [ "serde", ] @@ -1191,11 +1263,11 @@ dependencies = [ [[package]] name = "blocking" -version = "1.6.2" +version = "1.6.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "e83f8d02be6967315521be875afa792a316e28d57b5a2d401897e2a7921b7f21" +checksum = "703f41c54fc768e63e091340b424302bb1c29ef4aa0c7f10fe849dfb114d29ea" dependencies = [ - "async-channel 2.5.0", + "async-channel 2.3.1", "async-task", "futures-io", "futures-lite", @@ -1204,9 +1276,9 @@ dependencies = [ [[package]] name = "borsh" -version = "1.5.7" +version = "1.5.5" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "ad8646f98db542e39fc66e68a20b2144f6a732636df7c2354e74645faaa433ce" +checksum = "5430e3be710b68d984d1391c854eb431a9d548640711faa54eecb1df93db91cc" dependencies = [ "borsh-derive", "cfg_aliases", @@ -1214,22 +1286,22 @@ dependencies = [ [[package]] name = "borsh-derive" -version = "1.5.7" +version = "1.5.5" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "fdd1d3c0c2f5833f22386f252fe8ed005c7f59fdcddeef025c01b4c3b9fd9ac3" +checksum = "f8b668d39970baad5356d7c83a86fee3a539e6f93bf6764c97368243e17a0487" dependencies = [ "once_cell", "proc-macro-crate", "proc-macro2", "quote", - "syn 2.0.106", + "syn 2.0.101", ] [[package]] name = "brotli" -version = "8.0.2" +version = "8.0.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "4bd8b9603c7aa97359dbd97ecf258968c95f3adddd6db2f7e7a5bef101c84560" +checksum = "9991eea70ea4f293524138648e41ee89b0b2b12ddef3b255effa43c8056e0e0d" dependencies = [ "alloc-no-stdlib", "alloc-stdlib", @@ -1248,9 +1320,9 @@ dependencies = [ [[package]] name = "bstr" -version = "1.12.0" +version = "1.11.3" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "234113d19d0d7d613b40e86fb654acf958910802bcceab913a4f9e7cda03b1a4" +checksum = "531a9155a481e2ee699d4f98f43c0ca4ff8ee1bfd55c31e9e98fb29d2b176fe0" dependencies = [ "memchr", "serde", @@ -1258,9 +1330,9 @@ dependencies = [ [[package]] name = "bumpalo" -version = "3.19.0" +version = "3.17.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "46c5e41b57b8bba42a04676d81cb89e9ee8e859a1a66f80a5a72e1cb76b34d43" +checksum = "1628fb46dfa0b37568d12e5edd512553eccf6a22a78e8bde00bb4aed84d5bdbf" [[package]] name = "bytecheck" @@ -1286,9 +1358,9 @@ dependencies = [ [[package]] name = "bytemuck" -version = "1.23.2" +version = "1.21.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "3995eaeebcdf32f91f980d360f78732ddc061097ab4e39991ae7a6ace9194677" +checksum = "ef657dfab802224e671f5818e9a4935f9b1957ed18e58292690cc39e7a4092a3" [[package]] name = "byteorder" @@ -1298,9 +1370,9 @@ checksum = "1fd0f2584146f6f2ef48085050886acf353beff7305ebd1ae69500e27c67f64b" [[package]] name = "bytes" -version = "1.10.1" +version = "1.10.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d71b6127be86fdcfddb610f7182ac57211d4b18a3e9c82eb2d17662f2227ad6a" +checksum = "f61dac84819c6588b558454b194026eb1f09c293b9036ae9b159e74e73ab6cf9" dependencies = [ "serde", ] @@ -1355,9 +1427,9 @@ dependencies = [ [[package]] name = "cc" -version = "1.2.33" +version = "1.2.14" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "3ee0f8803222ba5a7e2777dd72ca451868909b1ac410621b676adf07280e9b5f" +checksum = "0c3d1b2e905a3a7b00a6141adb0e4c0bb941d11caf55349d863942a1cc44e3c9" dependencies = [ "jobserver", "libc", @@ -1375,9 +1447,9 @@ dependencies = [ [[package]] name = "cfg-if" -version = "1.0.1" +version = "1.0.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "9555578bc9e57714c812a1f84e4fc5b4d21fcb063490c624de019f7464c91268" +checksum = "baf1de4339761588bc0619e3cbc0120ee582ebb74b53b4efbf79117bd2da40fd" [[package]] name = "cfg_aliases" @@ -1407,18 +1479,19 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "93698b29de5e97ad0ae26447b344c482a7284c737d9ddc5f9e52b74a336671bb" dependencies = [ "chrono", - "chrono-tz-build", - "phf 0.11.3", + "chrono-tz-build 0.3.0", + "phf", ] [[package]] name = "chrono-tz" -version = "0.10.4" +version = "0.10.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a6139a8597ed92cf816dfb33f5dd6cf0bb93a6adc938f11039f371bc5bcd26c3" +checksum = "9c6ac4f2c0bf0f44e9161aec9675e1050aa4a530663c4a9e37e108fa948bca9f" dependencies = [ "chrono", - "phf 0.12.1", + "chrono-tz-build 0.4.0", + "phf", ] [[package]] @@ -1428,7 +1501,17 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "0c088aee841df9c3041febbb73934cfc39708749bf96dc827e3359cd39ef11b1" dependencies = [ "parse-zoneinfo", - "phf 0.11.3", + "phf", + "phf_codegen", +] + +[[package]] +name = "chrono-tz-build" +version = "0.4.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "e94fea34d77a245229e7746bd2beb786cd2a896f306ff491fb8cecb3074b10a7" +dependencies = [ + "parse-zoneinfo", "phf_codegen", ] @@ -1455,9 +1538,9 @@ dependencies = [ [[package]] name = "clap" -version = "4.5.45" +version = "4.5.35" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "1fc0e74a703892159f5ae7d3aac52c8e6c392f5ae5f359c70b5881d60aaac318" +checksum = "d8aa86934b44c19c50f87cc2790e19f54f7a67aedb64101c2e1a2e5ecfb73944" dependencies = [ "clap_builder", "clap_derive", @@ -1465,9 +1548,9 @@ dependencies = [ [[package]] name = "clap_builder" -version = "4.5.44" +version = "4.5.35" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "b3e7f4214277f3c7aa526a59dd3fbe306a370daee1f8b7b8c987069cd8e888a8" +checksum = "2414dbb2dd0695280da6ea9261e327479e9d37b0630f6b53ba2a11c60c679fd9" dependencies = [ "anstream", "anstyle", @@ -1477,27 +1560,27 @@ dependencies = [ [[package]] name = "clap_derive" -version = "4.5.45" +version = "4.5.32" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "14cb31bb0a7d536caef2639baa7fad459e15c3144efefa6dbd1c84562c4739f6" +checksum = "09176aae279615badda0765c0c0b3f6ed53f4709118af73cf4655d85d1530cd7" dependencies = [ "heck", "proc-macro2", "quote", - "syn 2.0.106", + "syn 2.0.101", ] [[package]] name = "clap_lex" -version = "0.7.5" +version = "0.7.4" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "b94f61472cee1439c0b966b47e3aca9ae07e45d070759512cd390ea2bebc6675" +checksum = "f46ad14479a25103f283c0f10005961cf086d8dc42205bb44c46ac563475dca6" [[package]] name = "clipboard-win" -version = "5.4.1" +version = "5.4.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "bde03770d3df201d4fb868f2c9c59e66a3e4e2bd06692a0fe701e7103c7e84d4" +checksum = "15efe7a882b08f34e38556b14f2fb3daa98769d06c7f0c1b076dfd0d983bc892" dependencies = [ "error-code", ] @@ -1513,16 +1596,17 @@ dependencies = [ [[package]] name = "colorchoice" -version = "1.0.4" +version = "1.0.3" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "b05b61dc5112cbb17e4b6cd61790d9845d13888356391624cbe7e41efeac1e75" +checksum = "5b63caa9aa9397e2d9480a9b13673856c78d8ac123288526c37d7839f2a86990" [[package]] name = "colored" -version = "3.0.0" +version = "2.2.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "fde0e0ec90c9dfb3b4b1a0891a7dcd0e2bffde2f7efed5fe7c9bb00e5bfb915e" +checksum = "117725a109d387c937a1533ce01b450cbde6b88abceea8473c4d7a85853cda3c" dependencies = [ + "lazy_static", "windows-sys 0.59.0", ] @@ -1533,7 +1617,7 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "4a65ebfec4fb190b6f90e944a817d60499ee0744e582530e2c9900a22e591d9a" dependencies = [ "unicode-segmentation", - "unicode-width 0.2.1", + "unicode-width 0.2.0", ] [[package]] @@ -1554,7 +1638,7 @@ dependencies = [ "encode_unicode", "libc", "once_cell", - "unicode-width 0.2.1", + "unicode-width 0.2.0", "windows-sys 0.59.0", ] @@ -1579,7 +1663,7 @@ version = "0.1.16" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "f9d839f2a20b0aee515dc581a6172f2321f96cab76c1a38a4c584a194955390e" dependencies = [ - "getrandom 0.2.16", + "getrandom 0.2.15", "once_cell", "tiny-keccak", ] @@ -1602,9 +1686,9 @@ dependencies = [ [[package]] name = "core-foundation" -version = "0.10.1" +version = "0.10.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "b2a6cd9ae233e7f62ba4e9353e81a88df7fc8a5987b8d445b4d90c879bd156f6" +checksum = "b55271e5c8c478ad3f38ad24ef34923091e0548492a266d19b3c0b4d82574c63" dependencies = [ "core-foundation-sys", "libc", @@ -1636,9 +1720,9 @@ dependencies = [ [[package]] name = "crc" -version = "3.3.0" +version = "3.2.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "9710d3b3739c2e349eb44fe848ad0b7c8cb1e42bd87ee49371df2f7acaf3e675" +checksum = "69e6e4d7b33a94f0991c26729976b10ebde1d34c3ee82408fb536164fa10d636" dependencies = [ "crc-catalog", ] @@ -1660,9 +1744,9 @@ dependencies = [ [[package]] name = "crc32fast" -version = "1.5.0" +version = "1.4.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "9481c1c90cbf2ac953f07c8d4a58aa3945c425b7185c9154d67a65e4230da511" +checksum = "a97769d94ddab943e4510d138150169a2758b5ef3eb191a9ee688de3e23ef7b3" dependencies = [ "cfg-if", ] @@ -1712,9 +1796,9 @@ checksum = "d0a5c400df2834b80a4c3327b3aad3a4c4cd4de0629063962b03235697506a28" [[package]] name = "crunchy" -version = "0.2.4" +version = "0.2.3" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "460fbee9c2c2f33933d720630a6a0bac33ba7053db5344fac858d4b8952d77d5" +checksum = "43da5946c66ffcc7745f48db692ffbb10a83bfe0afd96235c5c2a4fb23994929" [[package]] name = "crypto-common" @@ -1754,14 +1838,14 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "32a2785755761f3ddc1492979ce1e48d2c00d09311c39e4466429188f3dd6501" dependencies = [ "quote", - "syn 2.0.106", + "syn 2.0.101", ] [[package]] name = "darling" -version = "0.20.11" +version = "0.20.10" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "fc7f46116c46ff9ab3eb1597a45688b6715c6e628b5c133e288e709a29bcb4ee" +checksum = "6f63b86c8a8826a49b8c21f08a2d07338eec8d900540f8630dc76284be802989" dependencies = [ "darling_core", "darling_macro", @@ -1769,27 +1853,27 @@ dependencies = [ [[package]] name = "darling_core" -version = "0.20.11" +version = "0.20.10" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "0d00b9596d185e565c2207a0b01f8bd1a135483d02d9b7b0a54b11da8d53412e" +checksum = "95133861a8032aaea082871032f5815eb9e98cef03fa916ab4500513994df9e5" dependencies = [ "fnv", "ident_case", "proc-macro2", "quote", "strsim", - "syn 2.0.106", + "syn 2.0.101", ] [[package]] name = "darling_macro" -version = "0.20.11" +version = "0.20.10" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "fc34b93ccb385b40dc71c6fceac4b2ad23662c7eeb248cf10d529b7e055b6ead" +checksum = "d336a2a514f6ccccaa3e09b02d41d35330c07ddf03a62165fcec10bb561c7806" dependencies = [ "darling_core", "quote", - "syn 2.0.106", + "syn 2.0.101", ] [[package]] @@ -1948,14 +2032,14 @@ version = "47.0.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "0734015d81c8375eb5d4869b7f7ecccc2ee8d6cb81948ef737cd0e7b743bd69c" dependencies = [ - "ahash 0.8.12", + "ahash 0.8.11", "apache-avro", "arrow", "arrow-ipc", "base64 0.22.1", "half", "hashbrown 0.14.5", - "indexmap 2.10.0", + "indexmap 2.9.0", "libc", "log", "object_store", @@ -2159,7 +2243,7 @@ dependencies = [ "datafusion-functions-aggregate-common", "datafusion-functions-window-common", "datafusion-physical-expr-common", - "indexmap 2.10.0", + "indexmap 2.9.0", "paste", "recursive", "serde_json", @@ -2174,7 +2258,7 @@ checksum = "422ac9cf3b22bbbae8cdf8ceb33039107fde1b5492693168f13bd566b1bcc839" dependencies = [ "arrow", "datafusion-common", - "indexmap 2.10.0", + "indexmap 2.9.0", "itertools 0.14.0", "paste", ] @@ -2214,7 +2298,7 @@ version = "47.0.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "408a05dafdc70d05a38a29005b8b15e21b0238734dab1e98483fcb58038c5aba" dependencies = [ - "ahash 0.8.12", + "ahash 0.8.11", "arrow", "datafusion-common", "datafusion-doc", @@ -2235,7 +2319,7 @@ version = "47.0.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "756d21da2dd6c9bef97af1504970ff56cbf35d03fbd4ffd62827f02f4d2279d4" dependencies = [ - "ahash 0.8.12", + "ahash 0.8.11", "arrow", "datafusion-common", "datafusion-expr-common", @@ -2314,7 +2398,7 @@ checksum = "df6f88d7ee27daf8b108ba910f9015176b36fbc72902b1ca5c2a5f1d1717e1a1" dependencies = [ "datafusion-expr", "quote", - "syn 2.0.106", + "syn 2.0.101", ] [[package]] @@ -2328,7 +2412,7 @@ dependencies = [ "datafusion-common", "datafusion-expr", "datafusion-physical-expr", - "indexmap 2.10.0", + "indexmap 2.9.0", "itertools 0.14.0", "log", "recursive", @@ -2342,7 +2426,7 @@ version = "47.0.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "64c536062b0076f4e30084065d805f389f9fe38af0ca75bcbac86bc5e9fbab65" dependencies = [ - "ahash 0.8.12", + "ahash 0.8.11", "arrow", "datafusion-common", "datafusion-expr", @@ -2351,11 +2435,11 @@ dependencies = [ "datafusion-physical-expr-common", "half", "hashbrown 0.14.5", - "indexmap 2.10.0", + "indexmap 2.9.0", "itertools 0.14.0", "log", "paste", - "petgraph", + "petgraph 0.7.1", ] [[package]] @@ -2364,7 +2448,7 @@ version = "47.0.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "f8a92b53b3193fac1916a1c5b8e3f4347c526f6822e56b71faa5fb372327a863" dependencies = [ - "ahash 0.8.12", + "ahash 0.8.11", "arrow", "datafusion-common", "datafusion-expr-common", @@ -2397,7 +2481,7 @@ version = "47.0.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "690c615db468c2e5fe5085b232d8b1c088299a6c63d87fd960a354a71f7acb55" dependencies = [ - "ahash 0.8.12", + "ahash 0.8.11", "arrow", "arrow-ord", "arrow-schema", @@ -2413,7 +2497,7 @@ dependencies = [ "futures", "half", "hashbrown 0.14.5", - "indexmap 2.10.0", + "indexmap 2.9.0", "itertools 0.14.0", "log", "parking_lot", @@ -2455,7 +2539,7 @@ dependencies = [ "bigdecimal", "datafusion-common", "datafusion-expr", - "indexmap 2.10.0", + "indexmap 2.9.0", "log", "recursive", "regex", @@ -2483,15 +2567,15 @@ dependencies = [ "sqllogictest", "sqlparser", "tempfile", - "thiserror 2.0.15", + "thiserror 2.0.12", "tokio", ] [[package]] name = "der" -version = "0.7.10" +version = "0.7.9" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "e7c1832837b905bbfb5101e07cc24c8deddf52f93225eee6ead5f4d63d53ddcb" +checksum = "f55bf8e7b65898637379c1b74eb1551107c8294ed26d855ceb9fd1a09cfc9bc0" dependencies = [ "const-oid", "pem-rfc7468", @@ -2500,14 +2584,25 @@ dependencies = [ [[package]] name = "deranged" -version = "0.4.0" +version = "0.3.11" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "9c9e6a11ca8224451684bc0d7d5a7adbf8f2fd6887261a1cfc3c0432f9d4068e" +checksum = "b42b6fa04a440b495c8b04d0e71b707c585f83cb9cb28cf8cd0d976c315e31b4" dependencies = [ "powerfmt", "serde", ] +[[package]] +name = "derivative" +version = "2.2.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "fcc3dd5e9e9c0b295d6e1e4d811fb6f157d5ffd784b8d202fc62eac8035a770b" +dependencies = [ + "proc-macro2", + "quote", + "syn 1.0.109", +] + [[package]] name = "derive_builder" version = "0.20.2" @@ -2526,7 +2621,7 @@ dependencies = [ "darling", "proc-macro2", "quote", - "syn 2.0.106", + "syn 2.0.101", ] [[package]] @@ -2536,14 +2631,14 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "ab63b0e2bf4d5928aff72e83a7dace85d7bba5fe12dcc3c5a572d78caffd3f3c" dependencies = [ "derive_builder_core", - "syn 2.0.106", + "syn 2.0.101", ] [[package]] name = "deunicode" -version = "1.6.2" +version = "1.6.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "abd57806937c9cc163efc8ea3910e00a62e2aeb0b8119f1793a978088f8f6b04" +checksum = "339544cc9e2c4dc3fc7149fd630c5f22263a4fdf18a98afd0075784968b5cf00" [[package]] name = "diff" @@ -2581,7 +2676,7 @@ dependencies = [ "libc", "option-ext", "redox_users", - "windows-sys 0.60.2", + "windows-sys 0.59.0", ] [[package]] @@ -2592,14 +2687,14 @@ checksum = "97369cbbc041bc366949bc74d34658d6cda5621039731c6310521892a3a20ae0" dependencies = [ "proc-macro2", "quote", - "syn 2.0.106", + "syn 2.0.101", ] [[package]] name = "dissimilar" -version = "1.0.10" +version = "1.0.9" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "8975ffdaa0ef3661bfe02dbdcc06c9f829dfafe6a3c474de366a8d5e44276921" +checksum = "59f8e79d1fbf76bdfbde321e902714bf6c49df88a7dda6fc682fc2979226962d" [[package]] name = "dlv-list" @@ -2630,9 +2725,9 @@ checksum = "92773504d58c093f6de2459af4af33faa518c13451eb8f2b5698ed3d36e7c813" [[package]] name = "dyn-clone" -version = "1.0.20" +version = "1.0.19" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d0881ea181b1df73ff77ffaaf9c7544ecc11e82fba9b5f27b262a3c73a332555" +checksum = "1c7a8fb8a9fbf66c1f703fe16184d10ca0ee9d23be5b4436400408ba54a95005" [[package]] name = "educe" @@ -2643,7 +2738,7 @@ dependencies = [ "enum-ordinalize", "proc-macro2", "quote", - "syn 2.0.106", + "syn 2.0.101", ] [[package]] @@ -2684,7 +2779,7 @@ checksum = "0d28318a75d4aead5c4db25382e8ef717932d0346600cacae6357eb5941bc5ff" dependencies = [ "proc-macro2", "quote", - "syn 2.0.106", + "syn 2.0.101", ] [[package]] @@ -2718,19 +2813,19 @@ checksum = "877a4ace8713b0bcf2a4e7eec82529c029f1d0619886d18145fea96c3ffe5c0f" [[package]] name = "errno" -version = "0.3.13" +version = "0.3.10" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "778e2ac28f6c47af28e4907f13ffd1e1ddbd400980a9abd7c8df189bf578a5ad" +checksum = "33d852cb9b869c2a9b3df2f71a3074817f01e1844f839a144f5fcef059a4eb5d" dependencies = [ "libc", - "windows-sys 0.60.2", + "windows-sys 0.59.0", ] [[package]] name = "error-code" -version = "3.3.2" +version = "3.3.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "dea2df4cf52843e0452895c455a1a2cfbb842a1e7329671acf418fdc53ed4c59" +checksum = "a5d9305ccc6942a704f4335694ecd3de2ea531b114ac2d51f5f843750787a92f" [[package]] name = "escape8259" @@ -2757,9 +2852,9 @@ checksum = "0206175f82b8d6bf6652ff7d71a1e27fd2e4efde587fd368662814d6ec1d9ce0" [[package]] name = "event-listener" -version = "5.4.1" +version = "5.4.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "e13b66accf52311f30a0db42147dadea9850cb48cd070028831ae5f5d4b856ab" +checksum = "3492acde4c3fc54c845eaab3eed8bd00c7a7d881f78bfc801e43a93dec1331ae" dependencies = [ "concurrent-queue", "parking", @@ -2768,11 +2863,11 @@ dependencies = [ [[package]] name = "event-listener-strategy" -version = "0.5.4" +version = "0.5.3" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "8be9f3dfaaffdae2972880079a491a1a8bb7cbed0b8dd7a347f668b4150a3b93" +checksum = "3c3e4e0dd3673c1139bf041f3008816d9cf2946bbfac2945c09e523b8d7b05b2" dependencies = [ - "event-listener 5.4.1", + "event-listener 5.4.0", "pin-project-lite", ] @@ -2805,7 +2900,7 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "a6503af7917fea18ffef8f7e8553fb8dff89e2e6837e94e09dd7fb069c82d62c" dependencies = [ "bytes", - "rkyv 0.8.11", + "rkyv 0.8.10", "serde", "simdutf8", ] @@ -2817,10 +2912,16 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "0ce92ff622d6dadf7349484f42c93271a0d49b7cc4d466a936405bacbe10aa78" dependencies = [ "cfg-if", - "rustix 1.0.8", + "rustix 1.0.1", "windows-sys 0.59.0", ] +[[package]] +name = "fixedbitset" +version = "0.4.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "0ce7134b9999ecaf8bcd65542e436736ef32ddca1b3e06094cb6ec5755203b80" + [[package]] name = "fixedbitset" version = "0.5.7" @@ -2833,15 +2934,15 @@ version = "25.2.10" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "1045398c1bfd89168b5fd3f1fc11f6e70b34f6f66300c87d44d3de849463abf1" dependencies = [ - "bitflags", + "bitflags 2.8.0", "rustc_version", ] [[package]] name = "flate2" -version = "1.1.2" +version = "1.1.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "4a3d7db9596fecd151c5f638c0ee5d5bd487b6e0ea232e5dc96d5250f6f94b1d" +checksum = "7ced92e76e966ca2fd84c8f7aa01a4aea65b0eb6648d72f7c8f3e2764a67fece" dependencies = [ "crc32fast", "libz-rs-sys", @@ -2867,9 +2968,9 @@ checksum = "3f9eec918d3f24069decb9af1554cad7c880e2da24a9afd88aca000531ab82c1" [[package]] name = "foldhash" -version = "0.1.5" +version = "0.1.4" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d9c4f5dac5e15c24eb999c26181a6ca40b39fe946cbe4c263c7209467bc83af2" +checksum = "a0d2fde1f7b3d48b8395d5f2de76c18a528bd6a9cdde438df747bfcba3e05d6f" [[package]] name = "form_urlencoded" @@ -2888,9 +2989,9 @@ checksum = "28dd6caf6059519a65843af8fe2a3ae298b14b80179855aeb4adc2c1934ee619" [[package]] name = "fs-err" -version = "3.1.1" +version = "3.1.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "88d7be93788013f265201256d58f04936a8079ad5dc898743aa20525f503b683" +checksum = "1f89bda4c2a21204059a977ed3bfe746677dfd137b83c339e702b0ac91d482aa" dependencies = [ "autocfg", ] @@ -2968,9 +3069,9 @@ checksum = "9e5c1b78ca4aae1ac06c48a526a655760685149f0d465d21f37abfe57ce075c6" [[package]] name = "futures-lite" -version = "2.6.1" +version = "2.6.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "f78e10609fe0e0b3f4157ffab1876319b5b0db102a2c60dc4626306dc46b44ad" +checksum = "f5edaec856126859abb19ed65f39e90fea3a9574b9707f13539acf4abf7eb532" dependencies = [ "fastrand", "futures-core", @@ -2987,7 +3088,7 @@ checksum = "162ee34ebcb7c64a8abebc059ce0fee27c2262618d7b60ed8faf72fef13c3650" dependencies = [ "proc-macro2", "quote", - "syn 2.0.106", + "syn 2.0.101", ] [[package]] @@ -3022,11 +3123,10 @@ dependencies = [ [[package]] name = "generator" -version = "0.8.5" +version = "0.8.4" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d18470a76cb7f8ff746cf1f7470914f900252ec36bbc40b569d74b1258446827" +checksum = "cc6bd114ceda131d3b1d665eba35788690ad37f5916457286b32ab6fd3c438dd" dependencies = [ - "cc", "cfg-if", "libc", "log", @@ -3046,29 +3146,27 @@ dependencies = [ [[package]] name = "getrandom" -version = "0.2.16" +version = "0.2.15" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "335ff9f135e4384c8150d6f27c6daed433577f86b4750418338c01a1a2528592" +checksum = "c4567c8db10ae91089c99af84c68c38da3ec2f087c3f82960bcdbf3656b6f4d7" dependencies = [ "cfg-if", "js-sys", "libc", - "wasi 0.11.1+wasi-snapshot-preview1", + "wasi 0.11.0+wasi-snapshot-preview1", "wasm-bindgen", ] [[package]] name = "getrandom" -version = "0.3.3" +version = "0.3.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "26145e563e54f2cadc477553f1ec5ee650b00862f0a58bcd12cbdc5f0ea2d2f4" +checksum = "43a49c392881ce6d5c3b8cb70f98717b7c07aabbdff06687b9030dbfbe2725f8" dependencies = [ "cfg-if", - "js-sys", "libc", - "r-efi", - "wasi 0.14.2+wasi-0.2.4", - "wasm-bindgen", + "wasi 0.13.3+wasi-0.2.2", + "windows-targets 0.52.6", ] [[package]] @@ -3079,15 +3177,15 @@ checksum = "07e28edb80900c19c28f1072f2e8aeca7fa06b23cd4169cefe1af5aa3260783f" [[package]] name = "glob" -version = "0.3.3" +version = "0.3.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "0cc23270f6e1808e30a928bdc84dea0b9b4136a8bc82338574f23baf47bbd280" +checksum = "a8d1add55171497b4705a648c6b583acafb01d58050a51727785f0b2c8e0a2b2" [[package]] name = "globset" -version = "0.4.16" +version = "0.4.15" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "54a1028dfc5f5df5da8a56a73e6c153c9a9708ec57232470703592a3f18e49f5" +checksum = "15f1ce686646e7f1e19bf7d5533fe443a45dbfb990e00629110797578b42fb19" dependencies = [ "aho-corasick", "bstr", @@ -3102,7 +3200,7 @@ version = "0.9.1" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "0bf760ebf69878d9fd8f110c89703d90ce35095324d1f1edcb595c63945ee757" dependencies = [ - "bitflags", + "bitflags 2.8.0", "ignore", "walkdir", ] @@ -3121,9 +3219,9 @@ dependencies = [ [[package]] name = "h2" -version = "0.3.27" +version = "0.3.26" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "0beca50380b1fc32983fc1cb4587bfa4bb9e78fc259aad4a0032d2080309222d" +checksum = "81fe527a889e1532da5c525686d96d4c2e74cdd345badf8dfef9f6b39dd5f5e8" dependencies = [ "bytes", "fnv", @@ -3131,7 +3229,7 @@ dependencies = [ "futures-sink", "futures-util", "http 0.2.12", - "indexmap 2.10.0", + "indexmap 2.9.0", "slab", "tokio", "tokio-util", @@ -3140,9 +3238,9 @@ dependencies = [ [[package]] name = "h2" -version = "0.4.12" +version = "0.4.7" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "f3c0b69cfcb4e1b9f1bf2f53f95f766e4661169728ec61cd3fe5a0166f2d1386" +checksum = "ccae279728d634d083c00f6099cb58f01cc99c145b84b8be2f6c74618d79922e" dependencies = [ "atomic-waker", "bytes", @@ -3150,7 +3248,7 @@ dependencies = [ "futures-core", "futures-sink", "http 1.3.1", - "indexmap 2.10.0", + "indexmap 2.9.0", "slab", "tokio", "tokio-util", @@ -3183,15 +3281,15 @@ version = "0.14.5" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "e5274423e17b7c9fc20b6e7e208532f9b19825d82dfd615708b70edd83df41f1" dependencies = [ - "ahash 0.8.12", + "ahash 0.8.11", "allocator-api2", ] [[package]] name = "hashbrown" -version = "0.15.5" +version = "0.15.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "9229cfe53dfd69f0609a49f65461bd93001ea1ef889cd5529dd176593f5338a1" +checksum = "bf151400ff0baff5465007dd2f3e717f3fe502074ca563069ce3a6629d07b289" dependencies = [ "allocator-api2", "equivalent", @@ -3204,7 +3302,7 @@ version = "0.10.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "7382cf6263419f2d8df38c55d7da83da5c18aef87fc7a7fc1fb1e344edfe14c1" dependencies = [ - "hashbrown 0.15.5", + "hashbrown 0.15.2", ] [[package]] @@ -3215,9 +3313,15 @@ checksum = "2304e00983f87ffb38b55b444b5e3b60a884b5d30c0fca7d82fe33449bbe55ea" [[package]] name = "hermit-abi" -version = "0.5.2" +version = "0.3.9" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "d231dfb89cfffdbc30e7fc41579ed6066ad03abda9e567ccafae602b97ec5024" + +[[package]] +name = "hermit-abi" +version = "0.4.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "fc0fef456e4baa96da950455cd02c081ca953b141298e41db3fc7e36b1da849c" +checksum = "fbf6a919d6cf397374f7dfeeea91d974c7c0a7221d0d0f4f20d859d329e53fcc" [[package]] name = "hex" @@ -3227,9 +3331,9 @@ checksum = "7f24254aa9a54b5c858eaee2f5bccdb46aaf0e486a595ed5fd8f86ba55232a70" [[package]] name = "hive_metastore" -version = "0.2.0" +version = "0.1.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a699b5fcbc7f4614c4fcd8154bedcaab8b741e524e5f1b7bf1e9b1dca9917d8a" +checksum = "35f502759a3b4517dc44d06d8cdaddb942b7930bb81ebf7d645e209cfc7b7e43" dependencies = [ "anyhow", "pilota", @@ -3309,12 +3413,12 @@ dependencies = [ [[package]] name = "http-body-util" -version = "0.1.3" +version = "0.1.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "b021d93e26becf5dc7e1b75b1bed1fd93124b374ceb73f43d4d4eafec896a64a" +checksum = "793429d76616a256bcb62c2a2ec2bed781c8307e797e2598c50010f2bee2544f" dependencies = [ "bytes", - "futures-core", + "futures-util", "http 1.3.1", "http-body 1.0.1", "pin-project-lite", @@ -3322,9 +3426,9 @@ dependencies = [ [[package]] name = "httparse" -version = "1.10.1" +version = "1.10.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "6dbf3de79e51f3d586ab4cb9d5c3e2c14aa28ed23d180cf89b4df0454a69cc87" +checksum = "f2d708df4e7140240a16cd6ab0ab65c972d7433ab77819ea693fde9c43811e2a" [[package]] name = "httpdate" @@ -3343,9 +3447,9 @@ dependencies = [ [[package]] name = "humantime" -version = "2.2.0" +version = "2.1.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "9b112acc8b3adf4b107a8ec20977da0273a8c386765a3ec0229bd500a1443f9f" +checksum = "9a3a5bfb195931eeb336b2a7b4d761daec841b97f947d34394601737a7bba5e4" [[package]] name = "hyper" @@ -3357,14 +3461,14 @@ dependencies = [ "futures-channel", "futures-core", "futures-util", - "h2 0.3.27", + "h2 0.3.26", "http 0.2.12", "http-body 0.4.6", "httparse", "httpdate", "itoa", "pin-project-lite", - "socket2 0.5.10", + "socket2 0.5.8", "tokio", "tower-service", "tracing", @@ -3373,22 +3477,20 @@ dependencies = [ [[package]] name = "hyper" -version = "1.7.0" +version = "1.6.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "eb3aa54a13a0dfe7fbe3a59e0c76093041720fdc77b110cc0fc260fafb4dc51e" +checksum = "cc2b571658e38e0c01b1fdca3bbbe93c00d3d71693ff2770043f8c29bc7d6f80" dependencies = [ - "atomic-waker", "bytes", "futures-channel", - "futures-core", - "h2 0.4.12", + "futures-util", + "h2 0.4.7", "http 1.3.1", "http-body 1.0.1", "httparse", "httpdate", "itoa", "pin-project-lite", - "pin-utils", "smallvec", "tokio", "want", @@ -3412,41 +3514,49 @@ dependencies = [ [[package]] name = "hyper-rustls" -version = "0.27.7" +version = "0.27.5" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "e3c93eb611681b207e1fe55d5a71ecf91572ec8a6705cdb6857f7d8d5242cf58" +checksum = "2d191583f3da1305256f22463b9bb0471acad48a4e534a5218b9963e9c1f59b2" dependencies = [ + "futures-util", "http 1.3.1", - "hyper 1.7.0", + "hyper 1.6.0", "hyper-util", - "rustls 0.23.31", + "rustls 0.23.23", "rustls-native-certs 0.8.1", "rustls-pki-types", "tokio", - "tokio-rustls 0.26.2", + "tokio-rustls 0.26.1", "tower-service", - "webpki-roots 1.0.2", + "webpki-roots", +] + +[[package]] +name = "hyper-timeout" +version = "0.4.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "bbb958482e8c7be4bc3cf272a766a2b0bf1a6755e7a6ae777f017a31d11b13b1" +dependencies = [ + "hyper 0.14.32", + "pin-project-lite", + "tokio", + "tokio-io-timeout", ] [[package]] name = "hyper-util" -version = "0.1.16" +version = "0.1.10" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "8d9b05277c7e8da2c93a568989bb6207bef0112e8d17df7a6eda4a3cf143bc5e" +checksum = "df2dcfbe0677734ab2f3ffa7fa7bfd4706bfdc1ef393f2ee30184aed67e631b4" dependencies = [ - "base64 0.22.1", "bytes", "futures-channel", - "futures-core", "futures-util", "http 1.3.1", "http-body 1.0.1", - "hyper 1.7.0", - "ipnet", - "libc", - "percent-encoding", + "hyper 1.6.0", "pin-project-lite", - "socket2 0.6.0", + "socket2 0.5.8", "tokio", "tower-service", "tracing", @@ -3454,17 +3564,16 @@ dependencies = [ [[package]] name = "iana-time-zone" -version = "0.1.63" +version = "0.1.61" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "b0c919e5debc312ad217002b8048a17b7d83f80703865bbfcfebb0458b0b27d8" +checksum = "235e081f3925a06703c2d0117ea8b91f042756fd6e7a6e5d901e8ca1a996b220" dependencies = [ "android_system_properties", "core-foundation-sys", "iana-time-zone-haiku", "js-sys", - "log", "wasm-bindgen", - "windows-core", + "windows-core 0.52.0", ] [[package]] @@ -3527,7 +3636,7 @@ dependencies = [ "serde_json", "serde_repr", "serde_with", - "strum 0.27.2", + "strum 0.27.1", "tempfile", "tera", "thrift", @@ -3561,6 +3670,7 @@ dependencies = [ "serde_json", "tokio", "tracing", + "typed-builder 0.20.1", ] [[package]] @@ -3596,7 +3706,6 @@ dependencies = [ "iceberg", "iceberg-catalog-glue", "iceberg-catalog-rest", - "iceberg-catalog-s3tables", "tokio", ] @@ -3627,6 +3736,7 @@ dependencies = [ name = "iceberg-catalog-s3tables" version = "0.6.0" dependencies = [ + "anyhow", "async-trait", "aws-config", "aws-sdk-s3tables", @@ -3634,6 +3744,7 @@ dependencies = [ "iceberg_test_utils", "itertools 0.13.0", "tokio", + "typed-builder 0.20.1", ] [[package]] @@ -3722,12 +3833,28 @@ name = "iceberg-sqllogictest" version = "0.6.0" dependencies = [ "anyhow", + "arrow", "async-trait", + "bigdecimal", "datafusion", + "datafusion-common", "datafusion-sqllogictest", "enum-ordinalize", + "env_logger", + "half", + "iceberg", + "iceberg-catalog-rest", + "iceberg-datafusion", + "iceberg_test_utils", "indicatif", + "itertools 0.13.0", + "libtest-mimic 0.7.3", + "log", + "rust_decimal", + "spark-connect-rs", "sqllogictest", + "tempfile", + "tokio", "toml", ] @@ -3741,22 +3868,21 @@ dependencies = [ [[package]] name = "icu_collections" -version = "2.0.0" +version = "1.5.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "200072f5d0e3614556f94a9930d5dc3e0662a652823904c3a75dc3b0af7fee47" +checksum = "db2fa452206ebee18c4b5c2274dbf1de17008e874b4dc4f0aea9d01ca79e4526" dependencies = [ "displaydoc", - "potential_utf", "yoke", "zerofrom", "zerovec", ] [[package]] -name = "icu_locale_core" -version = "2.0.0" +name = "icu_locid" +version = "1.5.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "0cde2700ccaed3872079a65fb1a78f6c0a36c91570f28755dda67bc8f7d9f00a" +checksum = "13acbb8371917fc971be86fc8057c41a64b521c184808a698c02acc242dbf637" dependencies = [ "displaydoc", "litemap", @@ -3765,11 +3891,31 @@ dependencies = [ "zerovec", ] +[[package]] +name = "icu_locid_transform" +version = "1.5.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "01d11ac35de8e40fdeda00d9e1e9d92525f3f9d887cdd7aa81d727596788b54e" +dependencies = [ + "displaydoc", + "icu_locid", + "icu_locid_transform_data", + "icu_provider", + "tinystr", + "zerovec", +] + +[[package]] +name = "icu_locid_transform_data" +version = "1.5.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "fdc8ff3388f852bede6b579ad4e978ab004f139284d7b28715f773507b946f6e" + [[package]] name = "icu_normalizer" -version = "2.0.0" +version = "1.5.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "436880e8e18df4d7bbc06d58432329d6458cc84531f7ac5f024e93deadb37979" +checksum = "19ce3e0da2ec68599d193c93d088142efd7f9c5d6fc9b803774855747dc6a84f" dependencies = [ "displaydoc", "icu_collections", @@ -3777,54 +3923,67 @@ dependencies = [ "icu_properties", "icu_provider", "smallvec", + "utf16_iter", + "utf8_iter", + "write16", "zerovec", ] [[package]] name = "icu_normalizer_data" -version = "2.0.0" +version = "1.5.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "00210d6893afc98edb752b664b8890f0ef174c8adbb8d0be9710fa66fbbf72d3" +checksum = "f8cafbf7aa791e9b22bec55a167906f9e1215fd475cd22adfcf660e03e989516" [[package]] name = "icu_properties" -version = "2.0.1" +version = "1.5.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "016c619c1eeb94efb86809b015c58f479963de65bdb6253345c1a1276f22e32b" +checksum = "93d6020766cfc6302c15dbbc9c8778c37e62c14427cb7f6e601d849e092aeef5" dependencies = [ "displaydoc", "icu_collections", - "icu_locale_core", + "icu_locid_transform", "icu_properties_data", "icu_provider", - "potential_utf", - "zerotrie", + "tinystr", "zerovec", ] [[package]] name = "icu_properties_data" -version = "2.0.1" +version = "1.5.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "298459143998310acd25ffe6810ed544932242d3f07083eee1084d83a71bd632" +checksum = "67a8effbc3dd3e4ba1afa8ad918d5684b8868b3b26500753effea8d2eed19569" [[package]] name = "icu_provider" -version = "2.0.0" +version = "1.5.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "03c80da27b5f4187909049ee2d72f276f0d9f99a42c306bd0131ecfe04d8e5af" +checksum = "6ed421c8a8ef78d3e2dbc98a973be2f3770cb42b606e3ab18d6237c4dfde68d9" dependencies = [ "displaydoc", - "icu_locale_core", + "icu_locid", + "icu_provider_macros", "stable_deref_trait", "tinystr", "writeable", "yoke", "zerofrom", - "zerotrie", "zerovec", ] +[[package]] +name = "icu_provider_macros" +version = "1.5.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "1ec89e9337638ecdc08744df490b221a7399bf8d164eb52a665454e60e075ad6" +dependencies = [ + "proc-macro2", + "quote", + "syn 2.0.101", +] + [[package]] name = "ident_case" version = "1.0.1" @@ -3844,9 +4003,9 @@ dependencies = [ [[package]] name = "idna_adapter" -version = "1.2.1" +version = "1.2.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "3acae9609540aa318d1bc588455225fb2085b9ed0c4f6bd0d9d5bcd86f1a0344" +checksum = "daca1df1c957320b2cf139ac61e7bd64fed304c5040df000a745aa1de3b4ef71" dependencies = [ "icu_normalizer", "icu_properties", @@ -3881,12 +4040,12 @@ dependencies = [ [[package]] name = "indexmap" -version = "2.10.0" +version = "2.9.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "fe4cd85333e22411419a0bcae1297d25e58c9443848b11dc6a86fefe8c78a661" +checksum = "cea70ddb795996207ad57735b50c5982d8844f38ba9ee5f1aedcfb708a2aa11e" dependencies = [ "equivalent", - "hashbrown 0.15.5", + "hashbrown 0.15.2", "serde", ] @@ -3899,15 +4058,15 @@ dependencies = [ "console", "number_prefix", "portable-atomic", - "unicode-width 0.2.1", + "unicode-width 0.2.0", "web-time", ] [[package]] name = "inout" -version = "0.1.4" +version = "0.1.3" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "879f10e63c20629ecabbb64a8010319738c66a5cd0c29b02d63d272b03751d01" +checksum = "a0c10553d664a4d0bcff9f4215d0aac67a639cc68ef660840afe309b807bc9f5" dependencies = [ "block-padding", "generic-array", @@ -3931,11 +4090,11 @@ dependencies = [ [[package]] name = "io-uring" -version = "0.7.9" +version = "0.7.8" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d93587f37623a1a17d94ef2bc9ada592f5465fe7732084ab7beefabe5c77c0c4" +checksum = "b86e202f00093dcba4275d4636b93ef9dd75d025ae560d2521b45ea28ab49013" dependencies = [ - "bitflags", + "bitflags 2.8.0", "cfg-if", "libc", ] @@ -3946,16 +4105,6 @@ version = "2.11.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "469fb0b9cefa57e3ef31275ee7cacb78f2fdca44e4765491884a2b119d4eb130" -[[package]] -name = "iri-string" -version = "0.7.8" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "dbc5ebe9c3a1a7a5127f920a418f7585e9e758e911d0466ed004f393b0e380b2" -dependencies = [ - "memchr", - "serde", -] - [[package]] name = "is_terminal_polyfill" version = "1.70.1" @@ -3991,15 +4140,15 @@ dependencies = [ [[package]] name = "itoa" -version = "1.0.15" +version = "1.0.14" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "4a5f13b858c8d314ee3e8f639011f7ccefe71f97f96e50151fb991f267928e2c" +checksum = "d75a2a4b1b190afb6f5425f10f6a8f959d2ea0b9c2b1d79553551850539e4674" [[package]] name = "jiff" -version = "0.2.15" +version = "0.2.6" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "be1f93b8b1eb69c77f24bbb0afdf66f54b632ee39af40ca21c4365a1d7347e49" +checksum = "1f33145a5cbea837164362c7bd596106eb7c5198f97d1ba6f6ebb3223952e488" dependencies = [ "jiff-static", "log", @@ -4010,22 +4159,21 @@ dependencies = [ [[package]] name = "jiff-static" -version = "0.2.15" +version = "0.2.6" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "03343451ff899767262ec32146f6d559dd759fdadf42ff0e227c7c48f72594b4" +checksum = "43ce13c40ec6956157a3635d97a1ee2df323b263f09ea14165131289cb0f5c19" dependencies = [ "proc-macro2", "quote", - "syn 2.0.106", + "syn 2.0.101", ] [[package]] name = "jobserver" -version = "0.1.33" +version = "0.1.32" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "38f262f097c174adebe41eb73d66ae9c06b2844fb0da69969647bbddd9b0538a" +checksum = "48d1dbcbbeb6a7fec7e059840aa538bd62aaccf972c7346c4d9d2059312853d0" dependencies = [ - "getrandom 0.3.3", "libc", ] @@ -4144,9 +4292,9 @@ dependencies = [ [[package]] name = "libc" -version = "0.2.175" +version = "0.2.174" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "6a82ae493e598baaea5209805c49bbf2ea7de956d50d7da0da1164f9c6d28543" +checksum = "1171693293099992e19cddea4e8b849964e9846f4acee11b3948bcc337be8776" [[package]] name = "libflate" @@ -4174,25 +4322,25 @@ dependencies = [ [[package]] name = "libloading" -version = "0.8.8" +version = "0.8.6" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "07033963ba89ebaf1584d767badaa2e8fcec21aedea6b8c0346d487d49c28667" +checksum = "fc2f4eb4bc735547cfed7c0a4922cbd04a4655978c09b54f1f7b228750664c34" dependencies = [ "cfg-if", - "windows-targets 0.53.3", + "windows-targets 0.52.6", ] [[package]] name = "libm" -version = "0.2.15" +version = "0.2.11" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "f9fbbcab51052fe104eb5e5d351cf728d30a5be1fe14d9be8a3b097481fb97de" +checksum = "8355be11b20d696c8f18f6cc018c4e372165b1fa8126cef092399c9951984ffa" [[package]] name = "libmimalloc-sys" -version = "0.1.43" +version = "0.1.42" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "bf88cd67e9de251c1781dbe2f641a1a3ad66eaae831b8a2c38fbdc5ddae16d4d" +checksum = "ec9d6fac27761dabcd4ee73571cdb06b7022dc99089acbe5435691edffaac0f4" dependencies = [ "cc", "libc", @@ -4200,13 +4348,12 @@ dependencies = [ [[package]] name = "libredox" -version = "0.1.9" +version = "0.1.3" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "391290121bad3d37fbddad76d8f5d1c1c314cfc646d143d7e07a3086ddff0ce3" +checksum = "c0ff37bd590ca25063e35af745c343cb7a0271906fb7b37e4813e8f79f00268d" dependencies = [ - "bitflags", + "bitflags 2.8.0", "libc", - "redox_syscall", ] [[package]] @@ -4220,6 +4367,18 @@ dependencies = [ "vcpkg", ] +[[package]] +name = "libtest-mimic" +version = "0.7.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "cc0bda45ed5b3a2904262c1bb91e526127aa70e7ef3758aba2ef93cf896b9b58" +dependencies = [ + "clap", + "escape8259", + "termcolor", + "threadpool", +] + [[package]] name = "libtest-mimic" version = "0.8.1" @@ -4234,9 +4393,9 @@ dependencies = [ [[package]] name = "libz-rs-sys" -version = "0.5.1" +version = "0.5.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "172a788537a2221661b480fee8dc5f96c580eb34fa88764d3205dc356c7e4221" +checksum = "6489ca9bd760fe9642d7644e827b0c9add07df89857b0416ee15c1cc1a3b8c5a" dependencies = [ "zlib-rs", ] @@ -4249,9 +4408,9 @@ checksum = "0717cef1bc8b636c6e1c1bbdefc09e6322da8a9321966e8928ef80d20f7f770f" [[package]] name = "linkedbytes" -version = "0.1.16" +version = "0.1.8" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "80dd357febf29a3b1e37d0ff3509b035077769e5f4af161ff32edc3e97a78548" +checksum = "487640b2e3f554987c1345fc71c3eda34d6d750fba63a39147205ccc93f920cd" dependencies = [ "bytes", "faststr", @@ -4266,21 +4425,21 @@ checksum = "d26c52dbd32dccf2d10cac7725f8eae5296885fb5703b261f7d0a0739ec807ab" [[package]] name = "linux-raw-sys" -version = "0.9.4" +version = "0.9.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "cd945864f07fe9f5371a27ad7b52a172b4b499999f1d97574c9fa68373937e12" +checksum = "6db9c683daf087dc577b7506e9695b3d556a9f3849903fa28186283afd6809e9" [[package]] name = "litemap" -version = "0.8.0" +version = "0.7.4" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "241eaef5fd12c88705a01fc1066c48c4b36e0dd4377dcdc7ec3942cea7a69956" +checksum = "4ee93343901ab17bd981295f2cf0026d4ad018c7c31ba84549a4ddbb47a45104" [[package]] name = "lock_api" -version = "0.4.13" +version = "0.4.12" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "96936507f153605bddfcda068dd804796c84324ed2510809e5b2a624c81da765" +checksum = "07af8b9cdd281b7915f413fa73f29ebd5d55d0d3f0155584dade1ff18cea1b17" dependencies = [ "autocfg", "scopeguard", @@ -4288,9 +4447,9 @@ dependencies = [ [[package]] name = "log" -version = "0.4.27" +version = "0.4.25" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "13dc2df351e3202783a1fe0d44375f7295ffb4049267b0f3018346dc122a1d94" +checksum = "04cbf5b083de1c7e0222a7a51dbfdba1cbe1c6ab0b15e29fff3f6c077fd9cd9f" dependencies = [ "value-bag", ] @@ -4308,19 +4467,13 @@ dependencies = [ "tracing-subscriber", ] -[[package]] -name = "lru-slab" -version = "0.1.2" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "112b39cec0b298b6c1999fee3e31427f74f676e4cb9879ed1a121b43661a4154" - [[package]] name = "lz4_flex" -version = "0.11.5" +version = "0.11.3" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "08ab2867e3eeeca90e844d1940eab391c9dc5228783db2ed999acbc0a9ed375a" +checksum = "75761162ae2b0e580d7e7c390558127e5f01b4194debd6221fd8c207fc80e3f5" dependencies = [ - "twox-hash", + "twox-hash 1.6.3", ] [[package]] @@ -4343,6 +4496,12 @@ dependencies = [ "regex-automata 0.1.10", ] +[[package]] +name = "matchit" +version = "0.7.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "0e7465ac9959cc2b1404e8e2367b43684a6d13790fe23056cc8c6c5a6b7bcb94" + [[package]] name = "md-5" version = "0.10.6" @@ -4355,9 +4514,9 @@ dependencies = [ [[package]] name = "memchr" -version = "2.7.5" +version = "2.7.4" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "32a282da65faaf38286cf3be983213fcf1d2e2a58700e808f83f4ea9a4804bc0" +checksum = "78ca9ab1a0babb1e7d5695e3530886289c18cf2f87ec19a575a0abdce112e3a3" [[package]] name = "memoffset" @@ -4374,7 +4533,7 @@ version = "0.7.14" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "cdc67a1d6ef0340a7f5152b9fd34ad7477b4d518920f7557267f1fc6e5a62641" dependencies = [ - "ahash 0.8.12", + "ahash 0.8.11", "faststr", "paste", "rustc-hash 2.1.1", @@ -4383,13 +4542,19 @@ dependencies = [ [[package]] name = "mimalloc" -version = "0.1.47" +version = "0.1.46" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "b1791cbe101e95af5764f06f20f6760521f7158f69dbf9d6baf941ee1bf6bc40" +checksum = "995942f432bbb4822a7e9c3faa87a695185b0d09273ba85f097b54f4e458f2af" dependencies = [ "libmimalloc-sys", ] +[[package]] +name = "mime" +version = "0.3.17" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "6877bb514081ee2a7ff5ef9de3281f14a4dd4bceac4c09388074a6b5df8a139a" + [[package]] name = "minimal-lexical" version = "0.2.1" @@ -4398,22 +4563,22 @@ checksum = "68354c5c6bd36d73ff3feceb05efa59b6acb7626617f4962be322a825e61f79a" [[package]] name = "miniz_oxide" -version = "0.8.9" +version = "0.8.8" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "1fa76a2c86f704bdb222d66965fb3d63269ce38518b83cb0575fca855ebb6316" +checksum = "3be647b768db090acb35d5ec5db2b0e1f1de11133ca123b9eacf5137868f892a" dependencies = [ "adler2", ] [[package]] name = "mio" -version = "1.0.4" +version = "1.0.3" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "78bed444cc8a2160f01cbcf811ef18cac863ad68ae8ca62092e8db51d51c761c" +checksum = "2886843bf800fba2e3377cff24abf6379b4c4d5c6681eaf9ea5b0d15090450bd" dependencies = [ "libc", - "wasi 0.11.1+wasi-snapshot-preview1", - "windows-sys 0.59.0", + "wasi 0.11.0+wasi-snapshot-preview1", + "windows-sys 0.52.0", ] [[package]] @@ -4439,14 +4604,14 @@ dependencies = [ "cfg-if", "proc-macro2", "quote", - "syn 2.0.106", + "syn 2.0.101", ] [[package]] name = "mockito" -version = "1.7.0" +version = "1.6.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "7760e0e418d9b7e5777c0374009ca4c93861b9066f18cb334a20ce50ab63aa48" +checksum = "652cd6d169a36eaf9d1e6bce1a221130439a966d7f27858af66a33a66e9c4ee2" dependencies = [ "assert-json-diff", "bytes", @@ -4455,10 +4620,10 @@ dependencies = [ "http 1.3.1", "http-body 1.0.1", "http-body-util", - "hyper 1.7.0", + "hyper 1.6.0", "hyper-util", "log", - "rand 0.9.2", + "rand 0.8.5", "regex", "serde_json", "serde_urlencoded", @@ -4476,7 +4641,7 @@ dependencies = [ "crossbeam-channel", "crossbeam-epoch", "crossbeam-utils", - "event-listener 5.4.1", + "event-listener 5.4.0", "futures-util", "loom", "parking_lot", @@ -4508,27 +4673,33 @@ checksum = "b40e46c845ac234bcba19db7ab252bc2778cbadd516a466d2f12b1580852d136" dependencies = [ "proc-macro2", "quote", - "syn 2.0.106", + "syn 2.0.101", ] +[[package]] +name = "multimap" +version = "0.10.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "1d87ecb2933e8aeadb3e3a02b828fed80a7528047e68b4f424523a0981a3a084" + [[package]] name = "munge" -version = "0.4.6" +version = "0.4.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d7feb0b48aa0a25f9fe0899482c6e1379ee7a11b24a53073eacdecb9adb6dc60" +checksum = "64142d38c84badf60abf06ff9bd80ad2174306a5b11bd4706535090a30a419df" dependencies = [ "munge_macro", ] [[package]] name = "munge_macro" -version = "0.4.6" +version = "0.4.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "f2e3795a5d2da581a8b252fec6022eee01aea10161a4d1bf237d4cbe47f7e988" +checksum = "1bb5c1d8184f13f7d0ccbeeca0def2f9a181bce2624302793005f5ca8aa62e5e" dependencies = [ "proc-macro2", "quote", - "syn 2.0.106", + "syn 2.0.101", ] [[package]] @@ -4558,7 +4729,7 @@ version = "0.29.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "71e2746dc3a24dd78b3cfcb7be93368c6de9963d30f43a6a73998a9cf4b17b46" dependencies = [ - "bitflags", + "bitflags 2.8.0", "cfg-if", "cfg_aliases", "libc", @@ -4685,34 +4856,33 @@ dependencies = [ [[package]] name = "num_cpus" -version = "1.17.0" +version = "1.16.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "91df4bbde75afed763b708b7eee1e8e7651e02d97f6d5dd763e89367e957b23b" +checksum = "4161fcb6d602d4d2081af7c3a45852d875a03dd337a6bfdd6e06407b61342a43" dependencies = [ - "hermit-abi", + "hermit-abi 0.3.9", "libc", ] [[package]] name = "num_enum" -version = "0.7.4" +version = "0.7.3" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a973b4e44ce6cad84ce69d797acf9a044532e4184c4f267913d1b546a0727b7a" +checksum = "4e613fc340b2220f734a8595782c551f1250e969d87d3be1ae0579e8d4065179" dependencies = [ "num_enum_derive", - "rustversion", ] [[package]] name = "num_enum_derive" -version = "0.7.4" +version = "0.7.3" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "77e878c846a8abae00dd069496dbe8751b16ac1c3d6bd2a7283a938e8228f90d" +checksum = "af1844ef2428cc3e1cb900be36181049ef3d3193c63e43026cfe202983b27a56" dependencies = [ "proc-macro-crate", "proc-macro2", "quote", - "syn 2.0.106", + "syn 2.0.101", ] [[package]] @@ -4732,9 +4902,9 @@ dependencies = [ [[package]] name = "object_store" -version = "0.12.3" +version = "0.12.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "efc4f07659e11cd45a341cd24d71e683e3be65d9ff1f8150061678fe60437496" +checksum = "e9ce831b09395f933addbc56d894d889e4b226eba304d4e7adbab591e26daf1e" dependencies = [ "async-trait", "base64 0.22.1", @@ -4745,39 +4915,31 @@ dependencies = [ "http 1.3.1", "http-body-util", "humantime", - "hyper 1.7.0", + "hyper 1.6.0", "itertools 0.14.0", "md-5", "parking_lot", "percent-encoding", - "quick-xml 0.38.1", - "rand 0.9.2", + "quick-xml", + "rand 0.8.5", "reqwest", "ring", "rustls-pemfile 2.2.0", "serde", "serde_json", "serde_urlencoded", - "thiserror 2.0.15", + "thiserror 2.0.12", "tokio", "tracing", "url", "walkdir", - "wasm-bindgen-futures", - "web-time", ] [[package]] name = "once_cell" -version = "1.21.3" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "42f5e15c9953c5e4ccceeb2e7382a716482c34515315f7b03532b8b4e8393d2d" - -[[package]] -name = "once_cell_polyfill" -version = "1.70.1" +version = "1.21.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a4895175b425cb1f87721b59f0f286c2092bd4af812243672510e1ac53e2e0ad" +checksum = "d75b0bedcc4fe52caa0e03d9f1151a323e4aa5e2d78ba3580400cd3c9e2bc4bc" [[package]] name = "opendal" @@ -4792,13 +4954,13 @@ dependencies = [ "chrono", "crc32c", "futures", - "getrandom 0.2.16", + "getrandom 0.2.15", "http 1.3.1", "http-body 1.0.1", "log", "md-5", "percent-encoding", - "quick-xml 0.37.5", + "quick-xml", "reqsign", "reqwest", "serde", @@ -4863,9 +5025,9 @@ checksum = "b15813163c1d831bf4a13c3610c05c0d03b39feb07f7e09fa234dac9b15aaf39" [[package]] name = "owo-colors" -version = "4.2.2" +version = "4.2.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "48dd4f4a2c8405440fd0462561f0e5806bd0f77e86f51c761481bdd4018b545e" +checksum = "1036865bb9422d3300cf723f657c2851d0e9ab12567854b1f4eba3d77decf564" [[package]] name = "parking" @@ -4875,9 +5037,9 @@ checksum = "f38d5652c16fde515bb1ecef450ab0f6a219d619a7274976324d5e377f7dceba" [[package]] name = "parking_lot" -version = "0.12.4" +version = "0.12.3" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "70d58bf43669b5795d1576d0641cfb6fbb2057bf629506267a92807158584a13" +checksum = "f1bf18183cf54e8d6059647fc3063646a1801cf30896933ec2311622cc4b9a27" dependencies = [ "lock_api", "parking_lot_core", @@ -4885,9 +5047,9 @@ dependencies = [ [[package]] name = "parking_lot_core" -version = "0.9.11" +version = "0.9.10" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "bc838d2a56b5b1a6c25f55575dfc605fabb63bb2365f6c2353ef9159aa69e4a5" +checksum = "1e401f977ab385c9e4e3ab30627d6f26d00e2c73eef317493c4ec6d468726cf8" dependencies = [ "cfg-if", "libc", @@ -4902,7 +5064,7 @@ version = "55.2.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "b17da4150748086bd43352bc77372efa9b6e3dbd06a04831d2a98c041c225cfa" dependencies = [ - "ahash 0.8.12", + "ahash 0.8.11", "arrow-array", "arrow-buffer", "arrow-cast", @@ -4917,7 +5079,7 @@ dependencies = [ "flate2", "futures", "half", - "hashbrown 0.15.5", + "hashbrown 0.15.2", "lz4_flex", "num", "num-bigint", @@ -4928,7 +5090,7 @@ dependencies = [ "snap", "thrift", "tokio", - "twox-hash", + "twox-hash 2.1.0", "zstd", ] @@ -4959,9 +5121,9 @@ dependencies = [ [[package]] name = "pem" -version = "3.0.5" +version = "3.0.4" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "38af38e8470ac9dee3ce1bae1af9c1671fffc44ddfd8bd1d0a3445bf349a8ef3" +checksum = "8e459365e590736a54c3fa561947c84837534b8e9af6fc5bf781307e82658fae" dependencies = [ "base64 0.22.1", "serde", @@ -4984,20 +5146,20 @@ checksum = "e3148f5046208a5d56bcfc03053e3ca6334e51da8dfb19b6cdc8b306fae3283e" [[package]] name = "pest" -version = "2.8.1" +version = "2.7.15" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "1db05f56d34358a8b1066f67cbb203ee3e7ed2ba674a6263a1d5ec6db2204323" +checksum = "8b7cafe60d6cf8e62e1b9b2ea516a089c008945bb5a275416789e7db0bc199dc" dependencies = [ "memchr", - "thiserror 2.0.15", + "thiserror 2.0.12", "ucd-trie", ] [[package]] name = "pest_derive" -version = "2.8.1" +version = "2.7.15" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "bb056d9e8ea77922845ec74a1c4e8fb17e7c218cc4fc11a15c5d25e189aa40bc" +checksum = "816518421cfc6887a0d62bf441b6ffb4536fcc926395a69e1a85852d4363f57e" dependencies = [ "pest", "pest_generator", @@ -5005,53 +5167,55 @@ dependencies = [ [[package]] name = "pest_generator" -version = "2.8.1" +version = "2.7.15" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "87e404e638f781eb3202dc82db6760c8ae8a1eeef7fb3fa8264b2ef280504966" +checksum = "7d1396fd3a870fc7838768d171b4616d5c91f6cc25e377b673d714567d99377b" dependencies = [ "pest", "pest_meta", "proc-macro2", "quote", - "syn 2.0.106", + "syn 2.0.101", ] [[package]] name = "pest_meta" -version = "2.8.1" +version = "2.7.15" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "edd1101f170f5903fde0914f899bb503d9ff5271d7ba76bbb70bea63690cc0d5" +checksum = "e1e58089ea25d717bfd31fb534e4f3afcc2cc569c70de3e239778991ea3b7dea" dependencies = [ + "once_cell", "pest", "sha2", ] [[package]] name = "petgraph" -version = "0.7.1" +version = "0.6.5" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "3672b37090dbd86368a4145bc067582552b29c27377cad4e0a306c97f9bd7772" +checksum = "b4c5cc86750666a3ed20bdaf5ca2a0344f9c67674cae0515bec2da16fbaa47db" dependencies = [ - "fixedbitset", - "indexmap 2.10.0", + "fixedbitset 0.4.2", + "indexmap 2.9.0", ] [[package]] -name = "phf" -version = "0.11.3" +name = "petgraph" +version = "0.7.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "1fd6780a80ae0c52cc120a26a1a42c1ae51b247a253e4e06113d23d2c2edd078" +checksum = "3672b37090dbd86368a4145bc067582552b29c27377cad4e0a306c97f9bd7772" dependencies = [ - "phf_shared 0.11.3", + "fixedbitset 0.5.7", + "indexmap 2.9.0", ] [[package]] name = "phf" -version = "0.12.1" +version = "0.11.3" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "913273894cec178f401a31ec4b656318d95473527be05c0752cc41cdc32be8b7" +checksum = "1fd6780a80ae0c52cc120a26a1a42c1ae51b247a253e4e06113d23d2c2edd078" dependencies = [ - "phf_shared 0.12.1", + "phf_shared", ] [[package]] @@ -5061,7 +5225,7 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "aef8048c789fa5e851558d709946d6d79a8ff88c0440c587967f8e94bfb1216a" dependencies = [ "phf_generator", - "phf_shared 0.11.3", + "phf_shared", ] [[package]] @@ -5070,7 +5234,7 @@ version = "0.11.3" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "3c80231409c20246a13fddb31776fb942c38553c51e871f8cbd687a4cfb5843d" dependencies = [ - "phf_shared 0.11.3", + "phf_shared", "rand 0.8.5", ] @@ -5083,25 +5247,17 @@ dependencies = [ "siphasher", ] -[[package]] -name = "phf_shared" -version = "0.12.1" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "06005508882fb681fd97892ecff4b7fd0fee13ef1aa569f8695dae7ab9099981" -dependencies = [ - "siphasher", -] - [[package]] name = "pilota" -version = "0.11.10" +version = "0.11.8" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "520e32b295867834617b8a352b6fd4837e578f328f4e48813db4904ebc0d70b2" +checksum = "d18f400b02a75df232f315ad31c6d15b1b649237151ebe8b242eee1657330137" dependencies = [ - "ahash 0.8.12", + "ahash 0.8.11", "anyhow", "async-recursion", "bytes", + "derivative", "faststr", "integer-encoding 4.0.2", "lazy_static", @@ -5116,22 +5272,22 @@ dependencies = [ [[package]] name = "pin-project" -version = "1.1.10" +version = "1.1.9" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "677f1add503faace112b9f1373e43e9e054bfdd22ff1a63c1bc485eaec6a6a8a" +checksum = "dfe2e71e1471fe07709406bf725f710b02927c9c54b2b5b2ec0e8087d97c327d" dependencies = [ "pin-project-internal", ] [[package]] name = "pin-project-internal" -version = "1.1.10" +version = "1.1.9" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "6e918e4ff8c4549eb882f14b3a4bc8c8bc93de829416eacf579f1207a8fbf861" +checksum = "f6e859e6e5bd50440ab63c47e3ebabc90f26251f7c73c3d3e837b74a1cc3fa67" dependencies = [ "proc-macro2", "quote", - "syn 2.0.106", + "syn 2.0.101", ] [[package]] @@ -5197,22 +5353,23 @@ dependencies = [ [[package]] name = "pkg-config" -version = "0.3.32" +version = "0.3.31" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "7edddbd0b52d732b21ad9a5fab5c704c14cd949e5e9a1ec5929a24fded1b904c" +checksum = "953ec861398dccce10c670dfeaf3ec4911ca479e9c02154b3a215178c5f566f2" [[package]] name = "polling" -version = "3.10.0" +version = "3.7.4" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "b5bd19146350fe804f7cb2669c851c03d69da628803dab0d98018142aaa5d829" +checksum = "a604568c3202727d1507653cb121dbd627a58684eb09a820fd746bee38b4442f" dependencies = [ "cfg-if", "concurrent-queue", - "hermit-abi", + "hermit-abi 0.4.0", "pin-project-lite", - "rustix 1.0.8", - "windows-sys 0.60.2", + "rustix 0.38.44", + "tracing", + "windows-sys 0.59.0", ] [[package]] @@ -5223,9 +5380,9 @@ checksum = "325a6d2ac5dee293c3b2612d4993b98aec1dff096b0a2dae70ed7d95784a05da" [[package]] name = "portable-atomic" -version = "1.11.1" +version = "1.10.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "f84267b20a16ea918e43c6a88433c2d54fa145c92a811b5b047ccbe153674483" +checksum = "280dc24453071f1b63954171985a0b0d30058d287960968b9b2aca264c8d4ee6" [[package]] name = "portable-atomic-util" @@ -5249,7 +5406,7 @@ dependencies = [ "hmac", "md-5", "memchr", - "rand 0.9.2", + "rand 0.9.0", "sha2", "stringprep", ] @@ -5265,15 +5422,6 @@ dependencies = [ "postgres-protocol", ] -[[package]] -name = "potential_utf" -version = "0.1.2" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "e5a7c30837279ca13e7c867e9e40053bc68740f988cb07f7ca6df43cc734b585" -dependencies = [ - "zerovec", -] - [[package]] name = "powerfmt" version = "0.2.0" @@ -5282,11 +5430,11 @@ checksum = "439ee305def115ba05938db6eb1644ff94165c5ab5e9420d1c1bcedbba909391" [[package]] name = "ppv-lite86" -version = "0.2.21" +version = "0.2.20" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "85eae3c4ed2f50dcfe72643da4befc30deadb458a9b590d720cde2f2b1e97da9" +checksum = "77957b295656769bb8ad2b6a6b09d897d94f05c41b069aede1fcdaa675eaea04" dependencies = [ - "zerocopy", + "zerocopy 0.7.35", ] [[package]] @@ -5327,37 +5475,90 @@ dependencies = [ [[package]] name = "prettyplease" -version = "0.2.37" +version = "0.2.32" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "479ca8adacdd7ce8f1fb39ce9ecccbfe93a3f1344b3d0d97f20bc0196208f62b" +checksum = "664ec5419c51e34154eec046ebcba56312d5a2fc3b09a06da188e1ad21afadf6" dependencies = [ "proc-macro2", - "syn 2.0.106", + "syn 2.0.101", ] [[package]] name = "proc-macro-crate" -version = "3.3.0" +version = "3.2.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "edce586971a4dfaa28950c6f18ed55e0406c1ab88bbce2c6f6293a7aaba73d35" +checksum = "8ecf48c7ca261d60b74ab1a7b20da18bede46776b2e55535cb958eb595c5fa7b" dependencies = [ "toml_edit", ] [[package]] name = "proc-macro2" -version = "1.0.101" +version = "1.0.93" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "89ae43fd86e4158d6db51ad8e2b80f313af9cc74f5c0e03ccb87de09998732de" +checksum = "60946a68e5f9d28b0dc1c21bb8a97ee7d018a8b322fa57838ba31cc878e22d99" dependencies = [ "unicode-ident", ] +[[package]] +name = "prost" +version = "0.12.6" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "deb1435c188b76130da55f17a466d252ff7b1418b2ad3e037d127b94e3411f29" +dependencies = [ + "bytes", + "prost-derive", +] + +[[package]] +name = "prost-build" +version = "0.12.6" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "22505a5c94da8e3b7c2996394d1c933236c4d743e81a410bcca4e6989fc066a4" +dependencies = [ + "bytes", + "heck", + "itertools 0.12.1", + "log", + "multimap", + "once_cell", + "petgraph 0.6.5", + "prettyplease", + "prost", + "prost-types", + "regex", + "syn 2.0.101", + "tempfile", +] + +[[package]] +name = "prost-derive" +version = "0.12.6" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "81bddcdb20abf9501610992b6759a4c888aef7d1a7247ef75e2404275ac24af1" +dependencies = [ + "anyhow", + "itertools 0.12.1", + "proc-macro2", + "quote", + "syn 2.0.101", +] + +[[package]] +name = "prost-types" +version = "0.12.6" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "9091c90b0a32608e984ff2fa4091273cbdd755d54935c51d520887f4a1dbd5b0" +dependencies = [ + "prost", +] + [[package]] name = "psm" -version = "0.1.26" +version = "0.1.25" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "6e944464ec8536cd1beb0bbfd96987eb5e3b72f2ecdafdc5c769a37f1fa2ae1f" +checksum = "f58e5423e24c18cc840e1c98370b3993c6649cd1678b4d24318bcf0a083cbe88" dependencies = [ "cc", ] @@ -5399,7 +5600,7 @@ checksum = "ca414edb151b4c8d125c12566ab0d74dc9cdba36fb80eb7b848c15f495fd32d1" dependencies = [ "proc-macro2", "quote", - "syn 2.0.106", + "syn 2.0.101", ] [[package]] @@ -5410,19 +5611,9 @@ checksum = "5a651516ddc9168ebd67b24afd085a718be02f8858fe406591b013d101ce2f40" [[package]] name = "quick-xml" -version = "0.37.5" +version = "0.37.4" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "331e97a1af0bf59823e6eadffe373d7b27f485be8748f71471c662c1f269b7fb" -dependencies = [ - "memchr", - "serde", -] - -[[package]] -name = "quick-xml" -version = "0.38.1" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "9845d9dccf565065824e69f9f235fafba1587031eda353c1f1561cd6a6be78f4" +checksum = "a4ce8c88de324ff838700f36fb6ab86c96df0e3c4ab6ef3a9b2044465cce1369" dependencies = [ "memchr", "serde", @@ -5430,40 +5621,37 @@ dependencies = [ [[package]] name = "quinn" -version = "0.11.8" +version = "0.11.6" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "626214629cda6781b6dc1d316ba307189c85ba657213ce642d9c77670f8202c8" +checksum = "62e96808277ec6f97351a2380e6c25114bc9e67037775464979f3037c92d05ef" dependencies = [ "bytes", - "cfg_aliases", "pin-project-lite", "quinn-proto", "quinn-udp", "rustc-hash 2.1.1", - "rustls 0.23.31", - "socket2 0.5.10", - "thiserror 2.0.15", + "rustls 0.23.23", + "socket2 0.5.8", + "thiserror 2.0.12", "tokio", "tracing", - "web-time", ] [[package]] name = "quinn-proto" -version = "0.11.12" +version = "0.11.9" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "49df843a9161c85bb8aae55f101bc0bac8bcafd637a620d9122fd7e0b2f7422e" +checksum = "a2fe5ef3495d7d2e377ff17b1a8ce2ee2ec2a18cde8b6ad6619d65d0701c135d" dependencies = [ "bytes", - "getrandom 0.3.3", - "lru-slab", - "rand 0.9.2", + "getrandom 0.2.15", + "rand 0.8.5", "ring", "rustc-hash 2.1.1", - "rustls 0.23.31", + "rustls 0.23.23", "rustls-pki-types", "slab", - "thiserror 2.0.15", + "thiserror 2.0.12", "tinyvec", "tracing", "web-time", @@ -5471,14 +5659,14 @@ dependencies = [ [[package]] name = "quinn-udp" -version = "0.5.13" +version = "0.5.10" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "fcebb1209ee276352ef14ff8732e24cc2b02bbac986cd74a4c81bcb2f9881970" +checksum = "e46f3055866785f6b92bc6164b76be02ca8f2eb4b002c0354b28cf4c119e5944" dependencies = [ "cfg_aliases", "libc", "once_cell", - "socket2 0.5.10", + "socket2 0.5.8", "tracing", "windows-sys 0.59.0", ] @@ -5492,12 +5680,6 @@ dependencies = [ "proc-macro2", ] -[[package]] -name = "r-efi" -version = "5.3.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "69cdb34c158ceb288df11e18b4bd39de994f6657d83847bdffdbd7f346754b0f" - [[package]] name = "radium" version = "0.7.0" @@ -5537,12 +5719,13 @@ dependencies = [ [[package]] name = "rand" -version = "0.9.2" +version = "0.9.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "6db2770f06117d490610c7488547d543617b21bfa07796d7a12f6f1bd53850d1" +checksum = "3779b94aeb87e8bd4e834cee3650289ee9e0d5677f976ecdb6d219e5f4f6cd94" dependencies = [ "rand_chacha 0.9.0", - "rand_core 0.9.3", + "rand_core 0.9.1", + "zerocopy 0.8.18", ] [[package]] @@ -5562,7 +5745,7 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "d3022b5f1df60f26e1ffddd6c66e8aa15de382ae63b3a0c1bfc0e4d3e3f325cb" dependencies = [ "ppv-lite86", - "rand_core 0.9.3", + "rand_core 0.9.1", ] [[package]] @@ -5571,17 +5754,18 @@ version = "0.6.4" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "ec0be4795e2f6a28069bec0b5ff3e2ac9bafc99e6a9a7dc3547996c5c816922c" dependencies = [ - "getrandom 0.2.16", + "getrandom 0.2.15", "serde", ] [[package]] name = "rand_core" -version = "0.9.3" +version = "0.9.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "99d9a13982dcf210057a8a78572b2217b667c3beacbf3a0d8b454f6f82837d38" +checksum = "a88e0da7a2c97baa202165137c158d0a2e824ac465d13d81046727b34cb247d3" dependencies = [ - "getrandom 0.3.3", + "getrandom 0.3.1", + "zerocopy 0.8.18", ] [[package]] @@ -5601,47 +5785,47 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "76009fbe0614077fc1a2ce255e3a1881a2e3a3527097d5dc6d8212c585e7e38b" dependencies = [ "quote", - "syn 2.0.106", + "syn 2.0.101", ] [[package]] name = "redox_syscall" -version = "0.5.17" +version = "0.5.8" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "5407465600fb0548f1442edf71dd20683c6ed326200ace4b1ef0763521bb3b77" +checksum = "03a862b389f93e68874fbf580b9de08dd02facb9a788ebadaf4a3fd33cf58834" dependencies = [ - "bitflags", + "bitflags 2.8.0", ] [[package]] name = "redox_users" -version = "0.5.2" +version = "0.5.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a4e608c6638b9c18977b00b475ac1f28d14e84b27d8d42f70e0bf1e3dec127ac" +checksum = "dd6f9d3d47bdd2ad6945c5015a226ec6155d0bcdfd8f7cd29f86b71f8de99d2b" dependencies = [ - "getrandom 0.2.16", + "getrandom 0.2.15", "libredox", - "thiserror 2.0.15", + "thiserror 2.0.12", ] [[package]] name = "ref-cast" -version = "1.0.24" +version = "1.0.23" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "4a0ae411dbe946a674d89546582cea4ba2bb8defac896622d6496f14c23ba5cf" +checksum = "ccf0a6f84d5f1d581da8b41b47ec8600871962f2a528115b542b362d4b744931" dependencies = [ "ref-cast-impl", ] [[package]] name = "ref-cast-impl" -version = "1.0.24" +version = "1.0.23" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "1165225c21bff1f3bbce98f5a1f889949bc902d3575308cc7b0de30b4f6d27c7" +checksum = "bcc303e793d3734489387d205e9b186fac9c6cfacedd98cbb2e8a5943595f3e6" dependencies = [ "proc-macro2", "quote", - "syn 2.0.106", + "syn 2.0.101", ] [[package]] @@ -5711,16 +5895,16 @@ checksum = "a35e8a6bf28cd121053a66aa2e6a2e3eaffad4a60012179f0e864aa5ffeff215" [[package]] name = "reqsign" -version = "0.16.5" +version = "0.16.3" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "43451dbf3590a7590684c25fb8d12ecdcc90ed3ac123433e500447c7d77ed701" +checksum = "9323c0afb30e54f793f4705b10c890395bccc87c6e6ea62c4e7e82d09a380dc6" dependencies = [ "anyhow", "async-trait", "base64 0.22.1", "chrono", "form_urlencoded", - "getrandom 0.2.16", + "getrandom 0.2.15", "hex", "hmac", "home", @@ -5729,7 +5913,7 @@ dependencies = [ "log", "once_cell", "percent-encoding", - "quick-xml 0.37.5", + "quick-xml", "rand 0.8.5", "reqwest", "rsa", @@ -5738,61 +5922,64 @@ dependencies = [ "serde_json", "sha1", "sha2", - "tokio", ] [[package]] name = "reqwest" -version = "0.12.23" +version = "0.12.12" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d429f34c8092b2d42c7c93cec323bb4adeb7c67698f70839adec842ec10c7ceb" +checksum = "43e734407157c3c2034e0258f5e4473ddb361b1e85f95a66690d67264d7cd1da" dependencies = [ "base64 0.22.1", "bytes", "futures-core", "futures-util", - "h2 0.4.12", + "h2 0.4.7", "http 1.3.1", "http-body 1.0.1", "http-body-util", - "hyper 1.7.0", - "hyper-rustls 0.27.7", + "hyper 1.6.0", + "hyper-rustls 0.27.5", "hyper-util", + "ipnet", "js-sys", "log", + "mime", + "once_cell", "percent-encoding", "pin-project-lite", "quinn", - "rustls 0.23.31", + "rustls 0.23.23", "rustls-native-certs 0.8.1", + "rustls-pemfile 2.2.0", "rustls-pki-types", "serde", "serde_json", "serde_urlencoded", - "sync_wrapper", + "sync_wrapper 1.0.2", "tokio", - "tokio-rustls 0.26.2", + "tokio-rustls 0.26.1", "tokio-util", - "tower", - "tower-http", + "tower 0.5.2", "tower-service", "url", "wasm-bindgen", "wasm-bindgen-futures", "wasm-streams", "web-sys", - "webpki-roots 1.0.2", + "webpki-roots", + "windows-registry", ] [[package]] name = "ring" -version = "0.17.14" +version = "0.17.13" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a4689e6c2294d81e88dc6261c768b63bc4fcdb852be6d1352498b114f61383b7" +checksum = "70ac5d832aa16abd7d1def883a8545280c20a60f523a370aa3a9617c2b8550ee" dependencies = [ "cc", "cfg-if", - "getrandom 0.2.16", + "getrandom 0.2.15", "libc", "untrusted", "windows-sys 0.52.0", @@ -5818,18 +6005,18 @@ dependencies = [ [[package]] name = "rkyv" -version = "0.8.11" +version = "0.8.10" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "19f5c3e5da784cd8c69d32cdc84673f3204536ca56e1fa01be31a74b92c932ac" +checksum = "1e147371c75553e1e2fcdb483944a8540b8438c31426279553b9a8182a9b7b65" dependencies = [ "bytes", - "hashbrown 0.15.5", - "indexmap 2.10.0", + "hashbrown 0.15.2", + "indexmap 2.9.0", "munge", "ptr_meta 0.3.0", "rancor", "rend 0.5.2", - "rkyv_derive 0.8.11", + "rkyv_derive 0.8.10", "tinyvec", "uuid", ] @@ -5847,13 +6034,13 @@ dependencies = [ [[package]] name = "rkyv_derive" -version = "0.8.11" +version = "0.8.10" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "4270433626cffc9c4c1d3707dd681f2a2718d3d7b09ad754bec137acecda8d22" +checksum = "246b40ac189af6c675d124b802e8ef6d5246c53e17367ce9501f8f66a81abb7a" dependencies = [ "proc-macro2", "quote", - "syn 2.0.106", + "syn 2.0.101", ] [[package]] @@ -5874,9 +6061,9 @@ dependencies = [ [[package]] name = "rsa" -version = "0.9.8" +version = "0.9.7" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "78928ac1ed176a5ca1d17e578a1825f3d81ca54cf41053a592584b020cfd691b" +checksum = "47c75d7c5c6b673e58bf54d8544a9f432e3a925b0e80f7cd3602ab5c50c55519" dependencies = [ "const-oid", "digest", @@ -5906,9 +6093,9 @@ dependencies = [ [[package]] name = "rust_decimal" -version = "1.37.2" +version = "1.37.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "b203a6425500a03e0919c42d3c47caca51e79f1132046626d2c8871c5092035d" +checksum = "faa7de2ba56ac291bd90c6b9bece784a52ae1411f9506544b3eae36dd2356d50" dependencies = [ "arrayvec", "borsh", @@ -5923,9 +6110,9 @@ dependencies = [ [[package]] name = "rustc-demangle" -version = "0.1.26" +version = "0.1.24" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "56f7d92ca342cea22a06f2121d944b4fd82af56988c270852495420f961d4ace" +checksum = "719b953e2095829ee67db738b3bfa9fa368c94900df327b3f07fe6e794d2fe1f" [[package]] name = "rustc-hash" @@ -5957,7 +6144,7 @@ version = "0.38.44" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "fdb5bc1ae2baa591800df16c9ca78619bf65c0488b41b96ccec5d11220d8c154" dependencies = [ - "bitflags", + "bitflags 2.8.0", "errno", "libc", "linux-raw-sys 0.4.15", @@ -5966,15 +6153,15 @@ dependencies = [ [[package]] name = "rustix" -version = "1.0.8" +version = "1.0.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "11181fbabf243db407ef8df94a6ce0b2f9a733bd8be4ad02b4eda9602296cac8" +checksum = "dade4812df5c384711475be5fcd8c162555352945401aed22a35bffeab61f657" dependencies = [ - "bitflags", + "bitflags 2.8.0", "errno", "libc", - "linux-raw-sys 0.9.4", - "windows-sys 0.60.2", + "linux-raw-sys 0.9.2", + "windows-sys 0.59.0", ] [[package]] @@ -5991,15 +6178,15 @@ dependencies = [ [[package]] name = "rustls" -version = "0.23.31" +version = "0.23.23" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c0ebcbd2f03de0fc1122ad9bb24b127a5a6cd51d72604a3f3c50ac459762b6cc" +checksum = "47796c98c480fce5406ef69d1c76378375492c3b0a0de587be0c1d9feb12f395" dependencies = [ "aws-lc-rs", "once_cell", "ring", "rustls-pki-types", - "rustls-webpki 0.103.4", + "rustls-webpki 0.102.8", "subtle", "zeroize", ] @@ -6025,7 +6212,7 @@ dependencies = [ "openssl-probe", "rustls-pki-types", "schannel", - "security-framework 3.3.0", + "security-framework 3.2.0", ] [[package]] @@ -6048,12 +6235,11 @@ dependencies = [ [[package]] name = "rustls-pki-types" -version = "1.12.0" +version = "1.11.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "229a4a4c221013e7e1f1a043678c5cc39fe5171437c88fb47151a21e6f5b5c79" +checksum = "917ce264624a4b4db1c364dcc35bfca9ded014d0a958cd47ad3e960e988ea51c" dependencies = [ "web-time", - "zeroize", ] [[package]] @@ -6068,9 +6254,9 @@ dependencies = [ [[package]] name = "rustls-webpki" -version = "0.103.4" +version = "0.102.8" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "0a17884ae0c1b773f1ccd2bd4a8c72f16da897310a98b0e84bf349ad5ead92fc" +checksum = "64ca1bc8749bd4cf37b5ce386cc146580777b4e8572c7b97baf22c83f444bee9" dependencies = [ "aws-lc-rs", "ring", @@ -6080,9 +6266,9 @@ dependencies = [ [[package]] name = "rustversion" -version = "1.0.22" +version = "1.0.19" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "b39cdef0fa800fc44525c84ccb54a029961a8215f9619753635a9c0d2538d46d" +checksum = "f7c45b9784283f1b2e7fb61b42047c2fd678ef0960d4f6f1eba131594cc369d4" [[package]] name = "rustyline" @@ -6090,7 +6276,7 @@ version = "15.0.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "2ee1e066dc922e513bda599c6ccb5f3bb2b0ea5870a579448f2622993f0a9a2f" dependencies = [ - "bitflags", + "bitflags 2.8.0", "cfg-if", "clipboard-win", "fd-lock", @@ -6101,16 +6287,16 @@ dependencies = [ "nix", "radix_trie", "unicode-segmentation", - "unicode-width 0.2.1", + "unicode-width 0.2.0", "utf8parse", "windows-sys 0.59.0", ] [[package]] name = "ryu" -version = "1.0.20" +version = "1.0.19" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "28d3b2b1366ec20994f1fd18c3c594f05c5dd4bc44d8bb0c1c632c8d6829481f" +checksum = "6ea1a2d0a644769cc99faa24c3ad26b379b786fe7c36fd3c546254801650e6dd" [[package]] name = "salsa20" @@ -6208,7 +6394,7 @@ version = "2.11.1" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "897b2245f0b511c87893af39b033e5ca9cce68824c4d7e7630b5a1d339658d02" dependencies = [ - "bitflags", + "bitflags 2.8.0", "core-foundation 0.9.4", "core-foundation-sys", "libc", @@ -6217,12 +6403,12 @@ dependencies = [ [[package]] name = "security-framework" -version = "3.3.0" +version = "3.2.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "80fb1d92c5028aa318b4b8bd7302a5bfcf48be96a37fc6fc790f806b0004ee0c" +checksum = "271720403f46ca04f7ba6f55d438f8bd878d6b8ca0a1046e8228c4145bcbb316" dependencies = [ - "bitflags", - "core-foundation 0.10.1", + "bitflags 2.8.0", + "core-foundation 0.10.0", "core-foundation-sys", "libc", "security-framework-sys", @@ -6240,50 +6426,50 @@ dependencies = [ [[package]] name = "semver" -version = "1.0.26" +version = "1.0.25" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "56e6fa9c48d24d85fb3de5ad847117517440f6beceb7798af16b4a87d616b8d0" +checksum = "f79dfe2d285b0488816f30e700a7438c5a73d816b5b7d3ac72fbc48b0d185e03" [[package]] name = "seq-macro" -version = "0.3.6" +version = "0.3.5" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "1bc711410fbe7399f390ca1c3b60ad0f53f80e95c5eb935e52268a0e2cd49acc" +checksum = "a3f0bf26fd526d2a95683cd0f87bf103b8539e2ca1ef48ce002d67aad59aa0b4" [[package]] name = "serde" -version = "1.0.219" +version = "1.0.217" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "5f0e2c6ed6606019b4e29e69dbaba95b11854410e5347d525002456dbbb786b6" +checksum = "02fc4265df13d6fa1d00ecff087228cc0a2b5f3c0e87e258d8b94a156e984c70" dependencies = [ "serde_derive", ] [[package]] name = "serde_bytes" -version = "0.11.17" +version = "0.11.15" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "8437fd221bde2d4ca316d61b90e337e9e702b3820b87d63caa9ba6c02bd06d96" +checksum = "387cc504cb06bb40a96c8e04e951fe01854cf6bc921053c954e4a606d9675c6a" dependencies = [ "serde", ] [[package]] name = "serde_derive" -version = "1.0.219" +version = "1.0.217" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "5b0276cf7f2c73365f7157c8123c21cd9a50fbbd844757af28ca1f5925fc2a00" +checksum = "5a9bf7cf98d04a2b28aead066b7496853d4779c9cc183c440dbac457641e19a0" dependencies = [ "proc-macro2", "quote", - "syn 2.0.106", + "syn 2.0.101", ] [[package]] name = "serde_json" -version = "1.0.143" +version = "1.0.138" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d401abef1d108fbd9cbaebc3e46611f4b1021f714a0597a71f41ee463f5f4a5a" +checksum = "d434192e7da787e94a6ea7e9670b26a036d0ca41e0b7efb2676dd32bae872949" dependencies = [ "itoa", "memchr", @@ -6293,20 +6479,20 @@ dependencies = [ [[package]] name = "serde_repr" -version = "0.1.20" +version = "0.1.19" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "175ee3e80ae9982737ca543e96133087cbd9a485eecc3bc4de9c1a37b47ea59c" +checksum = "6c64451ba24fc7a6a2d60fc75dd9c83c90903b19028d4eff35e88fc1e86564e9" dependencies = [ "proc-macro2", "quote", - "syn 2.0.106", + "syn 2.0.101", ] [[package]] name = "serde_spanned" -version = "0.6.9" +version = "0.6.8" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "bf41e0cfaf7226dca15e8197172c295a782857fcb97fad1808a166870dee75a3" +checksum = "87607cb1398ed59d48732e575a4c28a7a8ebf2454b964fe3f224f2afc07909e1" dependencies = [ "serde", ] @@ -6333,7 +6519,7 @@ dependencies = [ "chrono", "hex", "indexmap 1.9.3", - "indexmap 2.10.0", + "indexmap 2.9.0", "schemars 0.9.0", "schemars 1.0.4", "serde", @@ -6352,7 +6538,7 @@ dependencies = [ "darling", "proc-macro2", "quote", - "syn 2.0.106", + "syn 2.0.101", ] [[package]] @@ -6368,9 +6554,9 @@ dependencies = [ [[package]] name = "sha2" -version = "0.10.9" +version = "0.10.8" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a7507d819769d01a365ab707794a4084392c824f54a7a6a7862f8c3d0892b283" +checksum = "793db75ad2bcafc3ffa7c68b215fee268f537982cd901d132f89c6343f3a3dc8" dependencies = [ "cfg-if", "cpufeatures", @@ -6394,9 +6580,9 @@ checksum = "0fda2ff0d084019ba4d7c6f371c95d8fd75ce3524c3cb8fb653a3023f6323e64" [[package]] name = "signal-hook-registry" -version = "1.4.6" +version = "1.4.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "b2a4719bff48cee6b39d12c020eeb490953ad2443b7055bd0b21fca26bd8c28b" +checksum = "a9e9e0b4211b72e7b8b6e85c807d36c212bdb33ea8587f7569562a84df5465b1" dependencies = [ "libc", ] @@ -6431,7 +6617,7 @@ checksum = "297f631f50729c8c99b84667867963997ec0b50f32b2a7dbcab828ef0541e8bb" dependencies = [ "num-bigint", "num-traits", - "thiserror 2.0.15", + "thiserror 2.0.12", "time", ] @@ -6443,9 +6629,12 @@ checksum = "56199f7ddabf13fe5074ce809e7d3f42b42ae711800501b5b16ea82ad029c39d" [[package]] name = "slab" -version = "0.4.11" +version = "0.4.9" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "7a2ae44ef20feb57a68b23d846850f861394c2e02dc425a50098ae8c90267589" +checksum = "8f92a496fb766b417c996b9c5e57daf2f7ad3b0bebe1ccfca4856390e3d3bb67" +dependencies = [ + "autocfg", +] [[package]] name = "slug" @@ -6459,9 +6648,9 @@ dependencies = [ [[package]] name = "smallvec" -version = "1.15.1" +version = "1.14.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "67b1b7a3b5fe4f1376887184045fcf45c69e92af734b7aaddc05fb777b6fbd03" +checksum = "7fcf8323ef1faaee30a44a340193b1ac6814fd9b7b4e88e9d4519a3e4abe1cfd" dependencies = [ "serde", ] @@ -6474,9 +6663,9 @@ checksum = "1b6b67fb9a61334225b5b790716f609cd58395f895b3fe8b328786812a40bc3b" [[package]] name = "socket2" -version = "0.5.10" +version = "0.5.8" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "e22376abed350d73dd1cd119b57ffccad95b4e585a7cda43e286245ce23c0678" +checksum = "c970269d99b64e60ec3bd6ad27270092a5394c4e309314b18ae3fe575695fbe8" dependencies = [ "libc", "windows-sys 0.52.0", @@ -6518,18 +6707,42 @@ dependencies = [ "simdutf8", "sonic-number", "sonic-simd", - "thiserror 2.0.15", + "thiserror 2.0.12", ] [[package]] name = "sonic-simd" -version = "0.1.1" +version = "0.1.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "b421f7b6aa4a5de8f685aaf398dfaa828346ee639d2b1c1061ab43d40baa6223" +checksum = "940a24e82c9a97483ef66cef06b92160a8fa5cd74042c57c10b24d99d169d2fc" dependencies = [ "cfg-if", ] +[[package]] +name = "spark-connect-rs" +version = "0.0.2" +source = "git+https://github.com/apache/spark-connect-rust.git?rev=061cb3ecb187b039141f20c722c7984e915f3b9d#061cb3ecb187b039141f20c722c7984e915f3b9d" +dependencies = [ + "arrow", + "arrow-ipc", + "chrono", + "futures-util", + "http-body 0.4.6", + "prost", + "prost-types", + "rand 0.9.0", + "regex", + "serde_json", + "thiserror 2.0.12", + "tokio", + "tonic", + "tonic-build", + "tower 0.5.2", + "url", + "uuid", +] + [[package]] name = "spin" version = "0.9.8" @@ -6551,9 +6764,9 @@ dependencies = [ [[package]] name = "sqllogictest" -version = "0.28.3" +version = "0.28.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "9fcbf91368a8d6807093d94f274fa4d0978cd78a310fee1d20368c545a606f7a" +checksum = "ee6199c1e008acc669b1e5873c138bf3ad4f8709ccd5c5d88913e664ae4f75de" dependencies = [ "async-trait", "educe", @@ -6562,7 +6775,7 @@ dependencies = [ "glob", "humantime", "itertools 0.13.0", - "libtest-mimic", + "libtest-mimic 0.8.1", "md-5", "owo-colors", "rand 0.8.5", @@ -6570,7 +6783,7 @@ dependencies = [ "similar", "subst", "tempfile", - "thiserror 2.0.15", + "thiserror 2.0.12", "tracing", ] @@ -6593,14 +6806,14 @@ checksum = "da5fc6819faabb412da764b99d3b713bb55083c11e7e0c00144d386cd6a1939c" dependencies = [ "proc-macro2", "quote", - "syn 2.0.106", + "syn 2.0.101", ] [[package]] name = "sqlx" -version = "0.8.6" +version = "0.8.3" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "1fefb893899429669dcdd979aff487bd78f4064e5e7907e4269081e0ef7d97dc" +checksum = "4410e73b3c0d8442c5f99b425d7a435b5ee0ae4167b3196771dd3f7a01be745f" dependencies = [ "sqlx-core", "sqlx-macros", @@ -6611,58 +6824,58 @@ dependencies = [ [[package]] name = "sqlx-core" -version = "0.8.6" +version = "0.8.3" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "ee6798b1838b6a0f69c007c133b8df5866302197e404e8b6ee8ed3e3a5e68dc6" +checksum = "6a007b6936676aa9ab40207cde35daab0a04b823be8ae004368c0793b96a61e0" dependencies = [ - "base64 0.22.1", "bytes", "crc", "crossbeam-queue", "either", - "event-listener 5.4.1", + "event-listener 5.4.0", "futures-core", "futures-intrusive", "futures-io", "futures-util", - "hashbrown 0.15.5", + "hashbrown 0.15.2", "hashlink", - "indexmap 2.10.0", + "indexmap 2.9.0", "log", "memchr", "once_cell", "percent-encoding", - "rustls 0.23.31", + "rustls 0.23.23", + "rustls-pemfile 2.2.0", "serde", "serde_json", "sha2", "smallvec", - "thiserror 2.0.15", + "thiserror 2.0.12", "tokio", "tokio-stream", "tracing", "url", - "webpki-roots 0.26.11", + "webpki-roots", ] [[package]] name = "sqlx-macros" -version = "0.8.6" +version = "0.8.3" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a2d452988ccaacfbf5e0bdbc348fb91d7c8af5bee192173ac3636b5fb6e6715d" +checksum = "3112e2ad78643fef903618d78cf0aec1cb3134b019730edb039b69eaf531f310" dependencies = [ "proc-macro2", "quote", "sqlx-core", "sqlx-macros-core", - "syn 2.0.106", + "syn 2.0.101", ] [[package]] name = "sqlx-macros-core" -version = "0.8.6" +version = "0.8.3" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "19a9c1841124ac5a61741f96e1d9e2ec77424bf323962dd894bdb93f37d5219b" +checksum = "4e9f90acc5ab146a99bf5061a7eb4976b573f560bc898ef3bf8435448dd5e7ad" dependencies = [ "dotenvy", "either", @@ -6676,20 +6889,21 @@ dependencies = [ "sha2", "sqlx-core", "sqlx-sqlite", - "syn 2.0.106", + "syn 2.0.101", + "tempfile", "tokio", "url", ] [[package]] name = "sqlx-mysql" -version = "0.8.6" +version = "0.8.3" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "aa003f0038df784eb8fecbbac13affe3da23b45194bd57dba231c8f48199c526" +checksum = "4560278f0e00ce64938540546f59f590d60beee33fffbd3b9cd47851e5fff233" dependencies = [ "atoi", "base64 0.22.1", - "bitflags", + "bitflags 2.8.0", "byteorder", "bytes", "crc", @@ -6717,20 +6931,20 @@ dependencies = [ "smallvec", "sqlx-core", "stringprep", - "thiserror 2.0.15", + "thiserror 2.0.12", "tracing", "whoami", ] [[package]] name = "sqlx-postgres" -version = "0.8.6" +version = "0.8.3" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "db58fcd5a53cf07c184b154801ff91347e4c30d17a3562a635ff028ad5deda46" +checksum = "c5b98a57f363ed6764d5b3a12bfedf62f07aa16e1856a7ddc2a0bb190a959613" dependencies = [ "atoi", "base64 0.22.1", - "bitflags", + "bitflags 2.8.0", "byteorder", "crc", "dotenvy", @@ -6754,16 +6968,16 @@ dependencies = [ "smallvec", "sqlx-core", "stringprep", - "thiserror 2.0.15", + "thiserror 2.0.12", "tracing", "whoami", ] [[package]] name = "sqlx-sqlite" -version = "0.8.6" +version = "0.8.3" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c2d12fe70b2c1b4401038055f90f151b78208de1f9f89a7dbfd41587a10c3eea" +checksum = "f85ca71d3a5b24e64e1d08dd8fe36c6c95c339a896cc33068148906784620540" dependencies = [ "atoi", "flume", @@ -6778,7 +6992,6 @@ dependencies = [ "serde", "serde_urlencoded", "sqlx-core", - "thiserror 2.0.15", "tracing", "url", ] @@ -6791,9 +7004,9 @@ checksum = "a8f112729512f8e442d81f95a8a7ddf2b7c6b8a1a6f509a95864142b30cab2d3" [[package]] name = "stacker" -version = "0.1.21" +version = "0.1.20" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "cddb07e32ddb770749da91081d8d0ac3a16f1a569a18b20348cd371f5dead06b" +checksum = "601f9201feb9b09c00266478bf459952b9ef9a6b94edb2f21eba14ab681a60a9" dependencies = [ "cc", "cfg-if", @@ -6833,11 +7046,11 @@ checksum = "8fec0f0aef304996cf250b31b5a10dee7980c85da9d759361292b8bca5a18f06" [[package]] name = "strum" -version = "0.27.2" +version = "0.27.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "af23d6f6c1a224baef9d3f61e287d2761385a5b88fdab4eb4c6f11aeb54c4bcf" +checksum = "f64def088c51c9510a8579e3c5d67c65349dcf755e5479ad3d010aa6454e2c32" dependencies = [ - "strum_macros 0.27.2", + "strum_macros 0.27.1", ] [[package]] @@ -6850,26 +7063,27 @@ dependencies = [ "proc-macro2", "quote", "rustversion", - "syn 2.0.106", + "syn 2.0.101", ] [[package]] name = "strum_macros" -version = "0.27.2" +version = "0.27.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "7695ce3845ea4b33927c055a39dc438a45b059f7c1b3d91d38d10355fb8cbca7" +checksum = "c77a8c5abcaf0f9ce05d62342b7d298c346515365c36b673df4ebe3ced01fde8" dependencies = [ "heck", "proc-macro2", "quote", - "syn 2.0.106", + "rustversion", + "syn 2.0.101", ] [[package]] name = "subst" -version = "0.3.8" +version = "0.3.7" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "0a9a86e5144f63c2d18334698269a8bfae6eece345c70b64821ea5b35054ec99" +checksum = "33e7942675ea19db01ef8cf15a1e6443007208e6c74568bd64162da26d40160d" dependencies = [ "memchr", "unicode-width 0.1.14", @@ -6894,15 +7108,21 @@ dependencies = [ [[package]] name = "syn" -version = "2.0.106" +version = "2.0.101" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "ede7c438028d4436d71104916910f5bb611972c5cfd7f89b8300a8186e6fada6" +checksum = "8ce2b7fc941b3a24138a0a7cf8e858bfc6a992e7978a068a5c760deb0ed43caf" dependencies = [ "proc-macro2", "quote", "unicode-ident", ] +[[package]] +name = "sync_wrapper" +version = "0.1.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "2047c6ded9c721764247e62cd3b03c09ffc529b2ba5b10ec482ae507a4a70160" + [[package]] name = "sync_wrapper" version = "1.0.2" @@ -6914,13 +7134,13 @@ dependencies = [ [[package]] name = "synstructure" -version = "0.13.2" +version = "0.13.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "728a70f3dbaf5bab7f0c4b1ac8d7ae5ea60a4b5549c8a5914361c99147a709d2" +checksum = "c8af7666ab7b6390ab78131fb5b0fce11d6b7a6951602017c35fa82800708971" dependencies = [ "proc-macro2", "quote", - "syn 2.0.106", + "syn 2.0.101", ] [[package]] @@ -6942,9 +7162,9 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "e8a64e3985349f2441a1a9ef0b853f869006c3855f2cda6862a94d26ebb9d6a1" dependencies = [ "fastrand", - "getrandom 0.3.3", + "getrandom 0.3.1", "once_cell", - "rustix 1.0.8", + "rustix 1.0.1", "windows-sys 0.59.0", ] @@ -6970,6 +7190,15 @@ dependencies = [ "unic-segment", ] +[[package]] +name = "termcolor" +version = "1.4.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "06794f8f6c5c898b3275aebefa6b8a1cb24cd2c6c79397ab15774837a0bc5755" +dependencies = [ + "winapi-util", +] + [[package]] name = "termtree" version = "0.5.1" @@ -6987,11 +7216,11 @@ dependencies = [ [[package]] name = "thiserror" -version = "2.0.15" +version = "2.0.12" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "80d76d3f064b981389ecb4b6b7f45a0bf9fdac1d5b9204c7bd6714fecc302850" +checksum = "567b8a2dae586314f7be2a752ec7474332959c6460e02bde30d702a66d488708" dependencies = [ - "thiserror-impl 2.0.15", + "thiserror-impl 2.0.12", ] [[package]] @@ -7002,27 +7231,28 @@ checksum = "4fee6c4efc90059e10f81e6d42c60a18f76588c3d74cb83a0b242a2b6c7504c1" dependencies = [ "proc-macro2", "quote", - "syn 2.0.106", + "syn 2.0.101", ] [[package]] name = "thiserror-impl" -version = "2.0.15" +version = "2.0.12" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "44d29feb33e986b6ea906bd9c3559a856983f92371b3eaa5e83782a351623de0" +checksum = "7f7cf42b4507d8ea322120659672cf1b9dbb93f8f2d4ecfd6e51350ff5b17a1d" dependencies = [ "proc-macro2", "quote", - "syn 2.0.106", + "syn 2.0.101", ] [[package]] name = "thread_local" -version = "1.1.9" +version = "1.1.8" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "f60246a4944f24f6e018aa17cdeffb7818b76356965d03b07d6a9886e8962185" +checksum = "8b9ef9bad013ada3808854ceac7b46812a6465ba368859a37e2100283d2d719c" dependencies = [ "cfg-if", + "once_cell", ] [[package]] @@ -7049,9 +7279,9 @@ dependencies = [ [[package]] name = "time" -version = "0.3.41" +version = "0.3.37" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "8a7619e19bc266e0f9c5e6686659d394bc57973859340060a69221e57dbc0c40" +checksum = "35e7868883861bd0e56d9ac6efcaaca0d6d5d82a2a7ec8209ff492c07cf37b21" dependencies = [ "deranged", "itoa", @@ -7064,15 +7294,15 @@ dependencies = [ [[package]] name = "time-core" -version = "0.1.4" +version = "0.1.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c9e9a38711f559d9e3ce1cdb06dd7c5b8ea546bc90052da6d06bb76da74bb07c" +checksum = "ef927ca75afb808a4d64dd374f00a2adf8d0fcff8e7b184af886c3c87ec4a3f3" [[package]] name = "time-macros" -version = "0.2.22" +version = "0.2.19" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "3526739392ec93fd8b359c8e98514cb3e8e021beb4e5f597b00a0221f8ed8a49" +checksum = "2834e6017e3e5e4b9834939793b282bc03b37a3336245fa820e35e233e2a85de" dependencies = [ "num-conv", "time-core", @@ -7089,9 +7319,9 @@ dependencies = [ [[package]] name = "tinystr" -version = "0.8.1" +version = "0.7.6" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "5d4f6d1145dcb577acf783d4e601bc1d76a13337bb54e6233add580b07344c8b" +checksum = "9117f5d4db391c1cf6927e7bea3db74b9a1c1add8f7eda9ffd5364f40f57b82f" dependencies = [ "displaydoc", "zerovec", @@ -7099,9 +7329,9 @@ dependencies = [ [[package]] name = "tinyvec" -version = "1.10.0" +version = "1.8.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "bfa5fdc3bce6191a1dbc8c02d5c8bffcf557bafa17c124c5264a458f1b0613fa" +checksum = "022db8904dfa342efe721985167e9fcd16c29b226db4397ed752a761cfce81e8" dependencies = [ "tinyvec_macros", ] @@ -7114,9 +7344,9 @@ checksum = "1f3ccbac311fea05f86f61904b462b55fb3df8837a366dfc601a0161d0532f20" [[package]] name = "tokio" -version = "1.47.1" +version = "1.47.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "89e49afdadebb872d3145a5638b59eb0691ea23e46ca484037cfab3b76b95038" +checksum = "43864ed400b6043a4757a25c7a64a8efde741aed79a056a2fb348a406701bb35" dependencies = [ "backtrace", "bytes", @@ -7132,6 +7362,16 @@ dependencies = [ "windows-sys 0.59.0", ] +[[package]] +name = "tokio-io-timeout" +version = "1.2.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "0bd86198d9ee903fedd2f9a2e72014287c0d9167e4ae43b5853007205dda1b76" +dependencies = [ + "pin-project-lite", + "tokio", +] + [[package]] name = "tokio-macros" version = "2.5.0" @@ -7140,7 +7380,7 @@ checksum = "6e06d43f1345a3bcd39f6a56dbb7dcab2ba47e68e8ac134855e7e2bdbaf8cab8" dependencies = [ "proc-macro2", "quote", - "syn 2.0.106", + "syn 2.0.101", ] [[package]] @@ -7155,11 +7395,11 @@ dependencies = [ [[package]] name = "tokio-rustls" -version = "0.26.2" +version = "0.26.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "8e727b36a1a0e8b74c376ac2211e40c2c8af09fb4013c60d910495810f008e9b" +checksum = "5f6d0975eaace0cf0fcadee4e4aaa5da15b5c079146f2cffb67c113be122bf37" dependencies = [ - "rustls 0.23.31", + "rustls 0.23.23", "tokio", ] @@ -7176,9 +7416,9 @@ dependencies = [ [[package]] name = "tokio-util" -version = "0.7.16" +version = "0.7.15" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "14307c986784f72ef81c89db7d9e28d6ac26d16213b109ea501696195e6e3ce5" +checksum = "66a539a9ad6d5d281510d5bd368c973d636c02dbf8a67300bfb6b950696ad7df" dependencies = [ "bytes", "futures-core", @@ -7189,9 +7429,9 @@ dependencies = [ [[package]] name = "toml" -version = "0.8.23" +version = "0.8.20" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "dc1beb996b9d83529a9e75c17a1686767d148d70663143c7854d8b4a09ced362" +checksum = "cd87a5cdd6ffab733b2f74bc4fd7ee5fff6634124999ac278c35fc78c6120148" dependencies = [ "serde", "serde_spanned", @@ -7201,62 +7441,97 @@ dependencies = [ [[package]] name = "toml_datetime" -version = "0.6.11" +version = "0.6.8" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "22cddaf88f4fbc13c51aebbf5f8eceb5c7c5a9da2ac40a13519eb5b0a0e8f11c" +checksum = "0dd7358ecb8fc2f8d014bf86f6f638ce72ba252a2c3a2572f2a795f1d23efb41" dependencies = [ "serde", ] [[package]] name = "toml_edit" -version = "0.22.27" +version = "0.22.24" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "41fe8c660ae4257887cf66394862d21dbca4a6ddd26f04a3560410406a2f819a" +checksum = "17b4795ff5edd201c7cd6dca065ae59972ce77d1b80fa0a84d94950ece7d1474" dependencies = [ - "indexmap 2.10.0", + "indexmap 2.9.0", "serde", "serde_spanned", "toml_datetime", - "toml_write", "winnow", ] [[package]] -name = "toml_write" -version = "0.1.2" +name = "tonic" +version = "0.11.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "5d99f8c9a7727884afe522e9bd5edbfc91a3312b36a77b5fb8926e4c31a41801" +checksum = "76c4eb7a4e9ef9d4763600161f12f5070b92a578e1b634db88a6887844c91a13" +dependencies = [ + "async-stream", + "async-trait", + "axum", + "base64 0.21.7", + "bytes", + "h2 0.3.26", + "http 0.2.12", + "http-body 0.4.6", + "hyper 0.14.32", + "hyper-timeout", + "percent-encoding", + "pin-project", + "prost", + "tokio", + "tokio-stream", + "tower 0.4.13", + "tower-layer", + "tower-service", + "tracing", +] + +[[package]] +name = "tonic-build" +version = "0.11.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "be4ef6dd70a610078cb4e338a0f79d06bc759ff1b22d2120c2ff02ae264ba9c2" +dependencies = [ + "prettyplease", + "proc-macro2", + "prost-build", + "quote", + "syn 2.0.101", +] [[package]] name = "tower" -version = "0.5.2" +version = "0.4.13" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d039ad9159c98b70ecfd540b2573b97f7f52c3e8d9f8ad57a24b916a536975f9" +checksum = "b8fa9be0de6cf49e536ce1851f987bd21a43b771b09473c3549a6c853db37c1c" dependencies = [ "futures-core", "futures-util", + "indexmap 1.9.3", + "pin-project", "pin-project-lite", - "sync_wrapper", + "rand 0.8.5", + "slab", "tokio", + "tokio-util", "tower-layer", "tower-service", + "tracing", ] [[package]] -name = "tower-http" -version = "0.6.6" +name = "tower" +version = "0.5.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "adc82fd73de2a9722ac5da747f12383d2bfdb93591ee6c58486e0097890f05f2" +checksum = "d039ad9159c98b70ecfd540b2573b97f7f52c3e8d9f8ad57a24b916a536975f9" dependencies = [ - "bitflags", - "bytes", + "futures-core", "futures-util", - "http 1.3.1", - "http-body 1.0.1", - "iri-string", "pin-project-lite", - "tower", + "sync_wrapper 1.0.2", + "tokio", "tower-layer", "tower-service", ] @@ -7287,20 +7562,20 @@ dependencies = [ [[package]] name = "tracing-attributes" -version = "0.1.30" +version = "0.1.28" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "81383ab64e72a7a8b8e13130c49e3dab29def6d0c7d76a03087b3cf71c5c6903" +checksum = "395ae124c09f9e6918a2310af6038fba074bcf474ac352496d5910dd59a2226d" dependencies = [ "proc-macro2", "quote", - "syn 2.0.106", + "syn 2.0.101", ] [[package]] name = "tracing-core" -version = "0.1.34" +version = "0.1.33" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "b9d12581f227e93f094d3af2ae690a574abb8a2b9b7a96e7cfe9647b2b617678" +checksum = "e672c95779cf947c5311f83787af4fa8fffd12fb27e4993211a84bdfd9610f9c" dependencies = [ "once_cell", "valuable", @@ -7349,9 +7624,19 @@ checksum = "e421abadd41a4225275504ea4d6566923418b7f05506fbc9c0fe86ba7396114b" [[package]] name = "twox-hash" -version = "2.1.1" +version = "1.6.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "97fee6b57c6a41524a810daee9286c02d7752c4253064d0b05472833a438f675" +dependencies = [ + "cfg-if", + "static_assertions", +] + +[[package]] +name = "twox-hash" +version = "2.1.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "8b907da542cbced5261bd3256de1b3a1bf340a3d37f93425a07362a1d687de56" +checksum = "e7b17f197b3050ba473acf9181f7b1d3b66d1cf7356c6cc57886662276e65908" [[package]] name = "typed-builder" @@ -7379,7 +7664,7 @@ checksum = "f9534daa9fd3ed0bd911d462a37f172228077e7abf18c18a5f67199d959205f8" dependencies = [ "proc-macro2", "quote", - "syn 2.0.106", + "syn 2.0.101", ] [[package]] @@ -7390,14 +7675,14 @@ checksum = "3c36781cc0e46a83726d9879608e4cf6c2505237e263a8eb8c24502989cfdb28" dependencies = [ "proc-macro2", "quote", - "syn 2.0.106", + "syn 2.0.101", ] [[package]] name = "typenum" -version = "1.18.0" +version = "1.17.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "1dccffe3ce07af9386bfd29e80c0ab1a8205a2fc34e4bcd40364df902cfa8f3f" +checksum = "42ff0bf0c66b8238c6f3b578df37d0b7848e55df8577b3f74f92a69acceeb825" [[package]] name = "ucd-trie" @@ -7463,9 +7748,9 @@ checksum = "5c1cb5db39152898a79168971543b1cb5020dff7fe43c8dc468b0885f5e29df5" [[package]] name = "unicode-ident" -version = "1.0.18" +version = "1.0.16" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "5a5f39404a5da50712a4c1eecf25e90dd62b613502b7e925fd4e4d19b5c96512" +checksum = "a210d160f08b701c8721ba1c726c11662f877ea6b7094007e1ca9a1041945034" [[package]] name = "unicode-normalization" @@ -7496,9 +7781,9 @@ checksum = "7dd6e30e90baa6f72411720665d41d89b9a3d039dc45b8faea1ddd07f617f6af" [[package]] name = "unicode-width" -version = "0.2.1" +version = "0.2.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "4a1a07cc7db3810833284e8d372ccdc6da29741639ecc70c9ec107df0fa6154c" +checksum = "1fc81956842c57dac11422a97c3b8195a1ff727f06e85c84ed2e8aa277c9a0fd" [[package]] name = "untrusted" @@ -7523,6 +7808,12 @@ version = "2.1.3" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "daf8dba3b7eb870caf1ddeed7bc9d2a049f3cfdfae7cb521b087cc33ae4c49da" +[[package]] +name = "utf16_iter" +version = "1.0.5" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "c8232dd3cdaed5356e0f716d285e4b40b932ac434100fe9b7e0e8e935b9e6246" + [[package]] name = "utf8_iter" version = "1.0.4" @@ -7537,11 +7828,11 @@ checksum = "06abde3611657adf66d383f00b093d7faecc7fa57071cce2578660c9f1010821" [[package]] name = "uuid" -version = "1.18.0" +version = "1.17.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "f33196643e165781c20a5ead5582283a7dacbb87855d867fbc2df3f81eddc1be" +checksum = "3cf4199d1e5d15ddd86a694e4d0dffa9c323ce759fea589f00fef9d81cc1931d" dependencies = [ - "getrandom 0.3.3", + "getrandom 0.3.1", "js-sys", "serde", "wasm-bindgen", @@ -7555,9 +7846,9 @@ checksum = "ba73ea9cf16a25df0c8caa16c51acb937d5712a8429db78a3ee29d5dcacd3a65" [[package]] name = "value-bag" -version = "1.11.1" +version = "1.10.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "943ce29a8a743eb10d6082545d861b24f9d1b160b7d741e0f2cdf726bec909c5" +checksum = "3ef4c4aa54d5d05a279399bfa921ec387b7aba77caf7a682ae8d86785b8fdad2" [[package]] name = "vcpkg" @@ -7588,12 +7879,12 @@ dependencies = [ "nix", "once_cell", "pin-project", - "rand 0.9.2", - "socket2 0.5.10", - "thiserror 2.0.15", + "rand 0.9.0", + "socket2 0.5.8", + "thiserror 2.0.12", "tokio", "tokio-stream", - "tower", + "tower 0.5.2", "tracing", ] @@ -7603,7 +7894,7 @@ version = "0.10.8" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "a4a41a936651c3b8206339898466fd4f39596cfedf789e5ee6a8d02a45feb45f" dependencies = [ - "ahash 0.8.12", + "ahash 0.8.11", "anyhow", "bytes", "chrono", @@ -7621,7 +7912,7 @@ dependencies = [ "rustc-hash 2.1.1", "scopeguard", "sonic-rs", - "thiserror 2.0.15", + "thiserror 2.0.12", "tokio", "tracing", "volo", @@ -7654,15 +7945,15 @@ dependencies = [ [[package]] name = "wasi" -version = "0.11.1+wasi-snapshot-preview1" +version = "0.11.0+wasi-snapshot-preview1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "ccf3ec651a847eb01de73ccad15eb7d99f80485de043efb2f370cd654f4ea44b" +checksum = "9c8d87e72b64a3b4db28d11ce29237c246188f4f51057d65a7eab63b7987e423" [[package]] name = "wasi" -version = "0.14.2+wasi-0.2.4" +version = "0.13.3+wasi-0.2.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "9683f9a5a998d873c0d21fcbe3c083009670149a8fab228644b8bd36b2c48cb3" +checksum = "26816d2e1a4a36a2940b96c5296ce403917633dff8f3440e9b236ed6f6bacad2" dependencies = [ "wit-bindgen-rt", ] @@ -7695,7 +7986,7 @@ dependencies = [ "log", "proc-macro2", "quote", - "syn 2.0.106", + "syn 2.0.101", "wasm-bindgen-shared", ] @@ -7730,7 +8021,7 @@ checksum = "8ae87ea40c9f689fc23f209965b6fb8a99ad69aeeb0231408be24920604395de" dependencies = [ "proc-macro2", "quote", - "syn 2.0.106", + "syn 2.0.101", "wasm-bindgen-backend", "wasm-bindgen-shared", ] @@ -7779,18 +8070,9 @@ dependencies = [ [[package]] name = "webpki-roots" -version = "0.26.11" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "521bc38abb08001b01866da9f51eb7c5d647a19260e00054a8c7fd5f9e57f7a9" -dependencies = [ - "webpki-roots 1.0.2", -] - -[[package]] -name = "webpki-roots" -version = "1.0.2" +version = "0.26.8" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "7e8983c3ab33d6fb807cfcdad2491c4ea8cbc8ed839181c7dfd9c67c83e261b2" +checksum = "2210b291f7ea53617fbafcc4939f10914214ec15aace5ba62293a668f322c5c9" dependencies = [ "rustls-pki-types", ] @@ -7809,11 +8091,11 @@ dependencies = [ [[package]] name = "whoami" -version = "1.6.1" +version = "1.5.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "5d4a4db5077702ca3015d3d02d74974948aba2ad9e12ab7df718ee64ccd7e97d" +checksum = "372d5b87f58ec45c384ba03563b03544dc5fadc3983e434b286913f5b4a9bb6d" dependencies = [ - "libredox", + "redox_syscall", "wasite", ] @@ -7850,104 +8132,92 @@ checksum = "712e227841d057c1ee1cd2fb22fa7e5a5461ae8e48fa2ca79ec42cfc1931183f" [[package]] name = "windows" -version = "0.61.3" +version = "0.58.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "9babd3a767a4c1aef6900409f85f5d53ce2544ccdfaa86dad48c91782c6d6893" +checksum = "dd04d41d93c4992d421894c18c8b43496aa748dd4c081bac0dc93eb0489272b6" dependencies = [ - "windows-collections", - "windows-core", - "windows-future", - "windows-link", - "windows-numerics", + "windows-core 0.58.0", + "windows-targets 0.52.6", ] [[package]] -name = "windows-collections" -version = "0.2.0" +name = "windows-core" +version = "0.52.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "3beeceb5e5cfd9eb1d76b381630e82c4241ccd0d27f1a39ed41b2760b255c5e8" +checksum = "33ab640c8d7e35bf8ba19b884ba838ceb4fba93a4e8c65a9059d08afcfc683d9" dependencies = [ - "windows-core", + "windows-targets 0.52.6", ] [[package]] name = "windows-core" -version = "0.61.2" +version = "0.58.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c0fdd3ddb90610c7638aa2b3a3ab2904fb9e5cdbecc643ddb3647212781c4ae3" +checksum = "6ba6d44ec8c2591c134257ce647b7ea6b20335bf6379a27dac5f1641fcf59f99" dependencies = [ "windows-implement", "windows-interface", - "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", - "windows-threading", + "windows-targets 0.52.6", ] [[package]] name = "windows-implement" -version = "0.60.0" +version = "0.58.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a47fddd13af08290e67f4acabf4b459f647552718f683a7b415d290ac744a836" +checksum = "2bbd5b46c938e506ecbce286b6628a02171d56153ba733b6c741fc627ec9579b" dependencies = [ "proc-macro2", "quote", - "syn 2.0.106", + "syn 2.0.101", ] [[package]] name = "windows-interface" -version = "0.59.1" +version = "0.58.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "bd9211b69f8dcdfa817bfd14bf1c97c9188afa36f4750130fcdf3f400eca9fa8" +checksum = "053c4c462dc91d3b1504c6fe5a726dd15e216ba718e84a0e46a88fbe5ded3515" dependencies = [ "proc-macro2", "quote", - "syn 2.0.106", + "syn 2.0.101", ] [[package]] name = "windows-link" -version = "0.1.3" +version = "0.1.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "5e6ad25900d524eaabdbbb96d20b4311e1e7ae1699af4fb28c17ae66c80d798a" +checksum = "76840935b766e1b0a05c0066835fb9ec80071d4c09a16f6bd5f7e655e3c14c38" [[package]] -name = "windows-numerics" +name = "windows-registry" version = "0.2.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "9150af68066c4c5c07ddc0ce30421554771e528bde427614c61038bc2c92c2b1" +checksum = "e400001bb720a623c1c69032f8e3e4cf09984deec740f007dd2b03ec864804b0" dependencies = [ - "windows-core", - "windows-link", + "windows-result", + "windows-strings", + "windows-targets 0.52.6", ] [[package]] name = "windows-result" -version = "0.3.4" +version = "0.2.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "56f42bd332cc6c8eac5af113fc0c1fd6a8fd2aa08a0119358686e5160d0586c6" +checksum = "1d1043d8214f791817bab27572aaa8af63732e11bf84aa21a45a78d6c317ae0e" dependencies = [ - "windows-link", + "windows-targets 0.52.6", ] [[package]] name = "windows-strings" -version = "0.4.2" +version = "0.1.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "56e6c93f3a0c3b36176cb1327a4958a0353d5d166c2a35cb268ace15e91d3b57" +checksum = "4cd9b125c486025df0eabcb585e62173c6c9eddcec5d117d3b6e8c30e2ee4d10" dependencies = [ - "windows-link", + "windows-result", + "windows-targets 0.52.6", ] [[package]] @@ -7977,15 +8247,6 @@ dependencies = [ "windows-targets 0.52.6", ] -[[package]] -name = "windows-sys" -version = "0.60.2" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "f2f500e4d28234f72040990ec9d39e3a6b950f9f22d3dba18416c35882612bcb" -dependencies = [ - "windows-targets 0.53.3", -] - [[package]] name = "windows-targets" version = "0.48.5" @@ -8010,39 +8271,13 @@ dependencies = [ "windows_aarch64_gnullvm 0.52.6", "windows_aarch64_msvc 0.52.6", "windows_i686_gnu 0.52.6", - "windows_i686_gnullvm 0.52.6", + "windows_i686_gnullvm", "windows_i686_msvc 0.52.6", "windows_x86_64_gnu 0.52.6", "windows_x86_64_gnullvm 0.52.6", "windows_x86_64_msvc 0.52.6", ] -[[package]] -name = "windows-targets" -version = "0.53.3" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d5fe6031c4041849d7c496a8ded650796e7b6ecc19df1a431c1a363342e5dc91" -dependencies = [ - "windows-link", - "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", -] - [[package]] name = "windows_aarch64_gnullvm" version = "0.48.5" @@ -8055,12 +8290,6 @@ version = "0.52.6" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "32a4622180e7a0ec044bb555404c800bc9fd9ec262ec147edd5989ccd0c02cd3" -[[package]] -name = "windows_aarch64_gnullvm" -version = "0.53.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "86b8d5f90ddd19cb4a147a5fa63ca848db3df085e25fee3cc10b39b6eebae764" - [[package]] name = "windows_aarch64_msvc" version = "0.48.5" @@ -8073,12 +8302,6 @@ version = "0.52.6" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "09ec2a7bb152e2252b53fa7803150007879548bc709c039df7627cabbd05d469" -[[package]] -name = "windows_aarch64_msvc" -version = "0.53.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c7651a1f62a11b8cbd5e0d42526e55f2c99886c77e007179efff86c2b137e66c" - [[package]] name = "windows_i686_gnu" version = "0.48.5" @@ -8091,24 +8314,12 @@ version = "0.52.6" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "8e9b5ad5ab802e97eb8e295ac6720e509ee4c243f69d781394014ebfe8bbfa0b" -[[package]] -name = "windows_i686_gnu" -version = "0.53.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c1dc67659d35f387f5f6c479dc4e28f1d4bb90ddd1a5d3da2e5d97b42d6272c3" - [[package]] name = "windows_i686_gnullvm" version = "0.52.6" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "0eee52d38c090b3caa76c563b86c3a4bd71ef1a819287c19d586d7334ae8ed66" -[[package]] -name = "windows_i686_gnullvm" -version = "0.53.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "9ce6ccbdedbf6d6354471319e781c0dfef054c81fbc7cf83f338a4296c0cae11" - [[package]] name = "windows_i686_msvc" version = "0.48.5" @@ -8121,12 +8332,6 @@ version = "0.52.6" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "240948bc05c5e7c6dabba28bf89d89ffce3e303022809e73deaefe4f6ec56c66" -[[package]] -name = "windows_i686_msvc" -version = "0.53.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "581fee95406bb13382d2f65cd4a908ca7b1e4c2f1917f143ba16efe98a589b5d" - [[package]] name = "windows_x86_64_gnu" version = "0.48.5" @@ -8139,12 +8344,6 @@ version = "0.52.6" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "147a5c80aabfbf0c7d901cb5895d1de30ef2907eb21fbbab29ca94c5b08b1a78" -[[package]] -name = "windows_x86_64_gnu" -version = "0.53.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "2e55b5ac9ea33f2fc1716d1742db15574fd6fc8dadc51caab1c16a3d3b4190ba" - [[package]] name = "windows_x86_64_gnullvm" version = "0.48.5" @@ -8157,12 +8356,6 @@ version = "0.52.6" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "24d5b23dc417412679681396f2b49f3de8c1473deb516bd34410872eff51ed0d" -[[package]] -name = "windows_x86_64_gnullvm" -version = "0.53.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "0a6e035dd0599267ce1ee132e51c27dd29437f63325753051e71dd9e42406c57" - [[package]] name = "windows_x86_64_msvc" version = "0.48.5" @@ -8175,35 +8368,35 @@ version = "0.52.6" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "589f6da84c646204747d1270a2a5661ea66ed1cced2631d546fdfb155959f9ec" -[[package]] -name = "windows_x86_64_msvc" -version = "0.53.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "271414315aff87387382ec3d271b52d7ae78726f5d44ac98b4f4030c91880486" - [[package]] name = "winnow" -version = "0.7.12" +version = "0.7.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "f3edebf492c8125044983378ecb5766203ad3b4c2f7a922bd7dd207f6d443e95" +checksum = "59690dea168f2198d1a3b0cac23b8063efcd11012f10ae4698f284808c8ef603" dependencies = [ "memchr", ] [[package]] name = "wit-bindgen-rt" -version = "0.39.0" +version = "0.33.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "6f42320e61fe2cfd34354ecb597f86f413484a798ba44a8ca1165c58d42da6c1" +checksum = "3268f3d866458b787f390cf61f4bbb563b922d091359f9608842999eaee3943c" dependencies = [ - "bitflags", + "bitflags 2.8.0", ] +[[package]] +name = "write16" +version = "1.0.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "d1890f4022759daae28ed4fe62859b1236caebfc61ede2f63ed4e695f3f6d936" + [[package]] name = "writeable" -version = "0.6.1" +version = "0.5.5" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "ea2f10b9bb0928dfb1b42b65e1f9e36f7f54dbdf08457afefb38afcdec4fa2bb" +checksum = "1e9df38ee2d2c3c5948ea468a8406ff0db0b29ae1ffde1bcf20ef305bcc95c51" [[package]] name = "wyz" @@ -8237,9 +8430,9 @@ checksum = "cfe53a6657fd280eaa890a3bc59152892ffa3e30101319d168b781ed6529b049" [[package]] name = "yoke" -version = "0.8.0" +version = "0.7.5" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "5f41bb01b8226ef4bfd589436a297c53d118f65921786300e427be8d487695cc" +checksum = "120e6aef9aa629e3d4f52dc8cc43a015c7724194c97dfaf45180d2daf2b77f40" dependencies = [ "serde", "stable_deref_trait", @@ -8249,54 +8442,75 @@ dependencies = [ [[package]] name = "yoke-derive" -version = "0.8.0" +version = "0.7.5" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "38da3c9736e16c5d3c8c597a9aaa5d1fa565d0532ae05e27c24aa62fb32c0ab6" +checksum = "2380878cad4ac9aac1e2435f3eb4020e8374b5f13c296cb75b4620ff8e229154" dependencies = [ "proc-macro2", "quote", - "syn 2.0.106", + "syn 2.0.101", "synstructure", ] [[package]] name = "zerocopy" -version = "0.8.26" +version = "0.7.35" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "1039dd0d3c310cf05de012d8a39ff557cb0d23087fd44cad61df08fc31907a2f" +checksum = "1b9b4fd18abc82b8136838da5d50bae7bdea537c574d8dc1a34ed098d6c166f0" dependencies = [ - "zerocopy-derive", + "byteorder", + "zerocopy-derive 0.7.35", +] + +[[package]] +name = "zerocopy" +version = "0.8.18" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "79386d31a42a4996e3336b0919ddb90f81112af416270cff95b5f5af22b839c2" +dependencies = [ + "zerocopy-derive 0.8.18", +] + +[[package]] +name = "zerocopy-derive" +version = "0.7.35" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "fa4f8080344d4671fb4e831a13ad1e68092748387dfc4f55e356242fae12ce3e" +dependencies = [ + "proc-macro2", + "quote", + "syn 2.0.101", ] [[package]] name = "zerocopy-derive" -version = "0.8.26" +version = "0.8.18" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "9ecf5b4cc5364572d7f4c329661bcc82724222973f2cab6f050a4e5c22f75181" +checksum = "76331675d372f91bf8d17e13afbd5fe639200b73d01f0fc748bb059f9cca2db7" dependencies = [ "proc-macro2", "quote", - "syn 2.0.106", + "syn 2.0.101", ] [[package]] name = "zerofrom" -version = "0.1.6" +version = "0.1.5" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "50cc42e0333e05660c3587f3bf9d0478688e15d870fab3346451ce7f8c9fbea5" +checksum = "cff3ee08c995dee1859d998dea82f7374f2826091dd9cd47def953cae446cd2e" dependencies = [ "zerofrom-derive", ] [[package]] name = "zerofrom-derive" -version = "0.1.6" +version = "0.1.5" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d71e5d6e06ab090c67b5e44993ec16b72dcbaabc526db883a360057678b48502" +checksum = "595eed982f7d355beb85837f651fa22e90b3c044842dc7f2c2842c086f295808" dependencies = [ "proc-macro2", "quote", - "syn 2.0.106", + "syn 2.0.101", "synstructure", ] @@ -8306,22 +8520,11 @@ version = "1.8.1" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "ced3678a2879b30306d323f4542626697a464a97c0a07c9aebf7ebca65cd4dde" -[[package]] -name = "zerotrie" -version = "0.2.2" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "36f0bbd478583f79edad978b407914f61b2972f5af6fa089686016be8f9af595" -dependencies = [ - "displaydoc", - "yoke", - "zerofrom", -] - [[package]] name = "zerovec" -version = "0.11.4" +version = "0.10.4" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "e7aa2bd55086f1ab526693ecbe444205da57e25f4489879da80635a46d90e73b" +checksum = "aa2b893d79df23bfb12d5461018d408ea19dfafe76c2c7ef6d4eba614f8ff079" dependencies = [ "yoke", "zerofrom", @@ -8330,44 +8533,44 @@ dependencies = [ [[package]] name = "zerovec-derive" -version = "0.11.1" +version = "0.10.3" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "5b96237efa0c878c64bd89c436f661be4e46b2f3eff1ebb976f7ef2321d2f58f" +checksum = "6eafa6dfb17584ea3e2bd6e76e0cc15ad7af12b09abdd1ca55961bed9b1063c6" dependencies = [ "proc-macro2", "quote", - "syn 2.0.106", + "syn 2.0.101", ] [[package]] name = "zlib-rs" -version = "0.5.1" +version = "0.5.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "626bd9fa9734751fc50d6060752170984d7053f5a39061f524cda68023d4db8a" +checksum = "868b928d7949e09af2f6086dfc1e01936064cc7a819253bce650d4e2a2d63ba8" [[package]] name = "zstd" -version = "0.13.3" +version = "0.13.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "e91ee311a569c327171651566e07972200e76fcfe2242a4fa446149a3881c08a" +checksum = "fcf2b778a664581e31e389454a7072dab1647606d44f7feea22cd5abb9c9f3f9" dependencies = [ "zstd-safe", ] [[package]] name = "zstd-safe" -version = "7.2.4" +version = "7.2.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "8f49c4d5f0abb602a93fb8736af2a4f4dd9512e36f7f570d66e65ff867ed3b9d" +checksum = "54a3ab4db68cea366acc5c897c7b4d4d1b8994a9cd6e6f841f8964566a419059" dependencies = [ "zstd-sys", ] [[package]] name = "zstd-sys" -version = "2.0.15+zstd.1.5.7" +version = "2.0.13+zstd.1.5.6" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "eb81183ddd97d0c74cedf1d50d85c8d08c1b8b68ee863bdee9e706eedba1a237" +checksum = "38ff0f21cfee8f97d94cef41359e0c89aa6113028ab0291aa8ca0038995a95aa" dependencies = [ "cc", "pkg-config", diff --git a/Cargo.toml b/Cargo.toml index 868284e751..f7494ec878 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -42,6 +42,7 @@ rust-version = "1.85" anyhow = "1.0.72" apache-avro = "0.17" array-init = "2" +arrow = { version = "55" } arrow-arith = { version = "55" } arrow-array = { version = "55" } arrow-buffer = { version = "55" } @@ -65,6 +66,7 @@ clap = { version = "4.5.35", features = ["derive", "cargo"] } ctor = "0.2.8" datafusion = "47" datafusion-cli = "47" +datafusion-common = { version = "47" } datafusion-sqllogictest = "47" derive_builder = "0.20" dirs = "6" @@ -77,6 +79,7 @@ futures = "0.3" hive_metastore = "0.2.0" http = "1.2" iceberg = { version = "0.6.0", path = "./crates/iceberg" } +iceberg-catalog-loader = { version = "0.6.0", path = "./crates/catalog/loader" } iceberg-catalog-rest = { version = "0.6.0", path = "./crates/catalog/rest" } iceberg-catalog-glue = { version = "0.6.0", path = "./crates/catalog/glue" } iceberg-catalog-s3tables = { version = "0.6.0", path = "./crates/catalog/s3tables" } diff --git a/crates/integrations/datafusion/src/table/mod.rs b/crates/integrations/datafusion/src/table/mod.rs index 7f741a534a..ed87afdcdd 100644 --- a/crates/integrations/datafusion/src/table/mod.rs +++ b/crates/integrations/datafusion/src/table/mod.rs @@ -172,7 +172,7 @@ mod tests { async fn get_test_table_from_metadata_file() -> Table { let metadata_file_name = "TableMetadataV2Valid.json"; let metadata_file_path = format!( - "{}/tests/test_data/{}", + "{}/tests/testdata/{}", env!("CARGO_MANIFEST_DIR"), metadata_file_name ); diff --git a/crates/sqllogictest/Cargo.toml b/crates/sqllogictest/Cargo.toml index ba149daeab..963be88339 100644 --- a/crates/sqllogictest/Cargo.toml +++ b/crates/sqllogictest/Cargo.toml @@ -25,14 +25,37 @@ rust-version = { workspace = true } version = { workspace = true } [dependencies] +arrow = { workspace = true } anyhow = { workspace = true } async-trait = { workspace = true } datafusion = { workspace = true } +datafusion-common = { workspace = true, default-features = true } datafusion-sqllogictest = { workspace = true } enum-ordinalize = { workspace = true } indicatif = { workspace = true } +iceberg = { workspace = true } +iceberg_test_utils = { path = "../test_utils", features = ["tests"] } +iceberg-catalog-rest = { workspace = true } +iceberg-datafusion = { workspace = true } sqllogictest = { workspace = true } toml = { workspace = true } +tokio = { workspace = true } +env_logger = "0.11.8" +log = "0.4.25" +itertools = "0.13.0" +tempfile = { workspace = true } +spark-connect-rs = { git = "https://github.com/apache/spark-connect-rust.git", rev = "061cb3ecb187b039141f20c722c7984e915f3b9d" } +half = "2.6.0" +bigdecimal = "0.4.1" +rust_decimal = { version = "1.27.0" } + +[dev-dependencies] +libtest-mimic = "0.7.3" + +[[test]] +harness = false +name = "sqllogictests" +path = "tests/sqllogictests.rs" [package.metadata.cargo-machete] # These dependencies are added to ensure minimal dependency version diff --git a/crates/sqllogictest/src/display/conversion.rs b/crates/sqllogictest/src/display/conversion.rs new file mode 100644 index 0000000000..d68afaaf00 --- /dev/null +++ b/crates/sqllogictest/src/display/conversion.rs @@ -0,0 +1,82 @@ +use arrow::array::types::{Decimal128Type, Decimal256Type, DecimalType}; +use arrow::datatypes::i256; +use bigdecimal::BigDecimal; +use half::f16; +use rust_decimal::prelude::*; + +/// Represents a constant for NULL string in your database. +pub const NULL_STR: &str = "NULL"; + +pub(crate) fn bool_to_str(value: bool) -> String { + if value { + "true".to_string() + } else { + "false".to_string() + } +} + +pub(crate) fn varchar_to_str(value: &str) -> String { + if value.is_empty() { + "(empty)".to_string() + } else { + value.trim_end_matches('\n').to_string() + } +} + +pub(crate) fn f16_to_str(value: f16) -> String { + if value.is_nan() { + // The sign of NaN can be different depending on platform. + // So the string representation of NaN ignores the sign. + "NaN".to_string() + } else if value == f16::INFINITY { + "Infinity".to_string() + } else if value == f16::NEG_INFINITY { + "-Infinity".to_string() + } else { + big_decimal_to_str(BigDecimal::from_str(&value.to_string()).unwrap()) + } +} + +pub(crate) fn f32_to_str(value: f32) -> String { + if value.is_nan() { + // The sign of NaN can be different depending on platform. + // So the string representation of NaN ignores the sign. + "NaN".to_string() + } else if value == f32::INFINITY { + "Infinity".to_string() + } else if value == f32::NEG_INFINITY { + "-Infinity".to_string() + } else { + big_decimal_to_str(BigDecimal::from_str(&value.to_string()).unwrap()) + } +} + +pub(crate) fn f64_to_str(value: f64) -> String { + if value.is_nan() { + // The sign of NaN can be different depending on platform. + // So the string representation of NaN ignores the sign. + "NaN".to_string() + } else if value == f64::INFINITY { + "Infinity".to_string() + } else if value == f64::NEG_INFINITY { + "-Infinity".to_string() + } else { + big_decimal_to_str(BigDecimal::from_str(&value.to_string()).unwrap()) + } +} + +pub(crate) fn i128_to_str(value: i128, precision: &u8, scale: &i8) -> String { + big_decimal_to_str( + BigDecimal::from_str(&Decimal128Type::format_decimal(value, *precision, *scale)).unwrap(), + ) +} + +pub(crate) fn i256_to_str(value: i256, precision: &u8, scale: &i8) -> String { + big_decimal_to_str( + BigDecimal::from_str(&Decimal256Type::format_decimal(value, *precision, *scale)).unwrap(), + ) +} + +pub(crate) fn big_decimal_to_str(value: BigDecimal) -> String { + value.round(12).normalized().to_string() +} diff --git a/crates/sqllogictest/src/display/mod.rs b/crates/sqllogictest/src/display/mod.rs new file mode 100644 index 0000000000..f237773c30 --- /dev/null +++ b/crates/sqllogictest/src/display/mod.rs @@ -0,0 +1 @@ +mod conversion; diff --git a/crates/sqllogictest/src/engine/datafusion.rs b/crates/sqllogictest/src/engine/datafusion.rs index f95cfb247d..6c3216f6d3 100644 --- a/crates/sqllogictest/src/engine/datafusion.rs +++ b/crates/sqllogictest/src/engine/datafusion.rs @@ -15,6 +15,7 @@ // specific language governing permissions and limitations // under the License. +use std::collections::HashMap; use std::path::{Path, PathBuf}; use std::sync::Arc; @@ -22,46 +23,114 @@ use anyhow::{Context, anyhow}; use datafusion::catalog::CatalogProvider; use datafusion::prelude::{SessionConfig, SessionContext}; use datafusion_sqllogictest::DataFusion; +use iceberg::CatalogBuilder; +use iceberg_catalog_rest::{REST_CATALOG_PROP_URI, RestCatalogBuilder}; +use iceberg_datafusion::IcebergCatalogProvider; use indicatif::ProgressBar; use sqllogictest::runner::AsyncDB; +use sqllogictest::{MakeConnection, Record, parse_file}; use toml::Table as TomlTable; -use crate::engine::Engine; -use crate::error::Result; +use crate::engine::EngineRunner; +use crate::error::{Error, Result}; pub struct DataFusionEngine { - datafusion: DataFusion, + ctx: SessionContext, + relative_path: PathBuf, + pb: ProgressBar, + config: TomlTable, } #[async_trait::async_trait] -impl Engine for DataFusionEngine { - async fn new(config: TomlTable) -> Result { +impl EngineRunner for DataFusionEngine { + async fn run_slt_file(&mut self, path: &Path) -> Result<()> { + let path_dir = path.to_str().unwrap(); + println!("engine running slt file on path: {path_dir}"); + let session_config = SessionConfig::new().with_target_partitions(4); let ctx = SessionContext::new_with_config(session_config); - ctx.register_catalog("default", Self::create_catalog(&config).await?); + ctx.register_catalog("demo", Self::create_catalog(&self.config).await?); - Ok(Self { - datafusion: DataFusion::new(ctx, PathBuf::from("testdata"), ProgressBar::new(100)), - }) - } + let runner = sqllogictest::Runner::new(|| async { + Ok(DataFusion::new( + ctx.clone(), + self.relative_path.clone(), + self.pb.clone(), + )) + }); - 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 result: std::result::Result<(), Error> = Self::run_file_in_runner(path, runner).await; + self.pb.finish_and_clear(); - self.datafusion - .run(content.as_str()) - .await - .with_context(|| format!("Failed to run slt file {:?}", path)) - .map_err(|e| anyhow!(e))?; - - Ok(()) + result } } impl DataFusionEngine { + pub async fn new(config: TomlTable) -> Result { + let session_config = SessionConfig::new().with_target_partitions(4); + let ctx = SessionContext::new_with_config(session_config); + ctx.register_catalog("demo", Self::create_catalog(&config).await?); + Ok(Self { + ctx, + relative_path: PathBuf::from("testdata"), + pb: ProgressBar::new(100), + config, + }) + } + async fn create_catalog(_: &TomlTable) -> anyhow::Result> { - todo!() + let catalog = RestCatalogBuilder::default() + .load( + "rest", + HashMap::from([ + ( + REST_CATALOG_PROP_URI.to_string(), + "http://localhost:8181".to_string(), + ), + ( + "s3.endpoint".to_string(), + "http://localhost:9000".to_string(), + ), + ("s3.access-key-id".to_string(), "admin".to_string()), + ("s3.secret-access-key".to_string(), "password".to_string()), + ("s3.region".to_string(), "us-east-1".to_string()), + ("s3.disable-config-load".to_string(), "true".to_string()), + ]), + ) + .await?; + + Ok(Arc::new( + IcebergCatalogProvider::try_new(Arc::new(catalog)).await?, + )) + } + + async fn run_file_in_runner>( + path: &Path, + mut runner: sqllogictest::Runner, + ) -> Result<()> { + println!("run file in runner"); + + let records = parse_file(&path).context("Failed to parse slt file")?; + + let mut errs = vec![]; + for record in records.into_iter() { + if let Record::Halt { .. } = record { + break; + } + if let Err(err) = runner.run_async(record).await { + errs.push(format!("{err}")); + } + } + + if !errs.is_empty() { + let mut msg = format!("{} errors in file {}\n\n", errs.len(), path.display()); + for (i, err) in errs.iter().enumerate() { + msg.push_str(&format!("{}. {err}\n\n", i + 1)); + } + return Err(Error(anyhow!(msg))); + } + + Ok(()) } } diff --git a/crates/sqllogictest/src/engine/mod.rs b/crates/sqllogictest/src/engine/mod.rs index 61722f663f..35faa7eff7 100644 --- a/crates/sqllogictest/src/engine/mod.rs +++ b/crates/sqllogictest/src/engine/mod.rs @@ -16,15 +16,25 @@ // under the License. mod datafusion; +mod spark; use std::path::Path; use toml::Table as TomlTable; +use crate::engine::datafusion::DataFusionEngine; +use crate::engine::spark::SparkEngine; use crate::error::Result; #[async_trait::async_trait] -pub trait Engine: Sized { - async fn new(config: TomlTable) -> Result; +pub trait EngineRunner { async fn run_slt_file(&mut self, path: &Path) -> Result<()>; } + +pub async fn load_engine(typ: &str, cfg: TomlTable) -> Result> { + match typ { + "datafusion" => Ok(Box::new(DataFusionEngine::new(cfg).await?)), + "spark-connect" => Ok(Box::new(SparkEngine::new(cfg).await?)), + _ => Err(anyhow::anyhow!("Unsupported engine type: {}", typ).into()), + } +} diff --git a/crates/sqllogictest/src/engine/spark.rs b/crates/sqllogictest/src/engine/spark.rs new file mode 100644 index 0000000000..1473c07ea0 --- /dev/null +++ b/crates/sqllogictest/src/engine/spark.rs @@ -0,0 +1,117 @@ +use std::path::Path; +use std::time::Duration; + +use anyhow::{Context, anyhow}; +use datafusion_sqllogictest::{DFColumnType, DFOutput, convert_schema_to_types, convert_batches}; +use spark_connect_rs::{SparkSession, SparkSessionBuilder}; +use sqllogictest::{AsyncDB, DBOutput, Record, parse_file}; +use toml::Table as TomlTable; + +use crate::engine::EngineRunner; +use crate::error::{Error, Result}; + +pub type SparkOutput = DBOutput; + +pub struct SparkEngine { + session: SparkSession, +} + +#[async_trait::async_trait] +impl AsyncDB for SparkEngine { + type Error = Error; + type ColumnType = DFColumnType; + + async fn run(&mut self, sql: &str) -> Result> { + Self::run_query(&self.session, sql).await + } + + async fn shutdown(&mut self) {} + + fn engine_name(&self) -> &str { + "SparkConnect" + } + + async fn sleep(dur: Duration) { + tokio::time::sleep(dur).await; + } +} + +#[async_trait::async_trait] +impl EngineRunner for SparkEngine { + async fn run_slt_file(&mut self, path: &Path) -> crate::error::Result<()> { + let path_dir = path.to_str().unwrap(); + println!("engine running slt file on path: {path_dir}"); + + let session = self.session.clone(); + let runner = sqllogictest::Runner::new(move || { + let session = session.clone(); + async move { Ok(SparkEngine { session }) } + }); + + let result: std::result::Result<(), Error> = Self::run_file_in_runner(path, runner).await; + + result + } +} + +impl SparkEngine { + pub async fn new(configs: TomlTable) -> Result { + let url = configs + .get("url") + .ok_or_else(|| anyhow!("url property doesn't exist for spark engine"))? + .as_str() + .ok_or_else(|| anyhow!("url property is not a string for spark engine"))?; + + let session = SparkSessionBuilder::remote(url) + .app_name("SparkConnect") + .build() + .await + .map_err(|e| anyhow!(e))?; + + Ok(Self { session }) + } + + pub async fn run_query(session: &SparkSession, sql: impl Into) -> Result { + let df = session.sql(sql.into().as_str()).await.unwrap(); + let batches = df.collect().await.unwrap(); + let schema = batches.schema(); + let types = convert_schema_to_types(schema.fields()); + + // Convert batches to rows of strings + let rows = convert_batches(vec![batches]).unwrap(); + + Ok(DBOutput::Rows { types, rows }) + } + + async fn run_file_in_runner( + path: &Path, + mut runner: sqllogictest::Runner, + ) -> Result<()> + where + M: sqllogictest::MakeConnection, + { + println!("run file in runner"); + + let records = parse_file(&path).context("Failed to parse slt file")?; + + let mut errs = vec![]; + for record in records.into_iter() { + if let Record::Halt { .. } = record { + break; + } + if let Err(err) = runner.run_async(record).await { + errs.push(format!("{err}")); + } + } + + if !errs.is_empty() { + let mut msg = format!("{} errors in file {}\n\n", errs.len(), path.display()); + for (i, err) in errs.iter().enumerate() { + msg.push_str(&format!("{}. {err}\n\n", i + 1)); + } + return Err(Error(anyhow!(msg))); + } + + Ok(()) + } +} diff --git a/crates/sqllogictest/src/lib.rs b/crates/sqllogictest/src/lib.rs index c72d50c429..82cd668aeb 100644 --- a/crates/sqllogictest/src/lib.rs +++ b/crates/sqllogictest/src/lib.rs @@ -18,7 +18,7 @@ // This lib contains codes copied from // [Apache Datafusion](https://github.com/apache/datafusion/tree/main/datafusion/sqllogictest) -#[allow(dead_code)] +mod display; mod engine; -#[allow(dead_code)] -mod error; +pub mod error; +pub mod schedule; diff --git a/crates/sqllogictest/src/schedule.rs b/crates/sqllogictest/src/schedule.rs new file mode 100644 index 0000000000..4f8566c0bb --- /dev/null +++ b/crates/sqllogictest/src/schedule.rs @@ -0,0 +1,163 @@ +// 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::fs::read_to_string; +use std::path::{Path, PathBuf}; + +use anyhow::anyhow; +use iceberg::{Catalog, CatalogBuilder}; +use iceberg_catalog_rest::RestCatalogBuilder; +use itertools::Itertools; +use toml::{Table, Value}; + +use crate::engine::{EngineRunner, load_engine}; + +/// Schedule of engines to run tests. +/// Controls the engine, storage, and catalog being used for the test steps +pub struct Schedule { + // Map of engine names to engine instances. + engines: HashMap>, + // List of steps to run, each step is a sql file. + steps: Vec, + // catalog: Box, +} + +pub struct Step { + /// Name of engine to execute. + engine_name: String, + /// Name of sql file. + sql: String, +} + +impl Schedule { + pub async fn parse>(schedule_def_file: P) -> anyhow::Result { + let content = read_to_string(schedule_def_file)?; + let toml_value = content.parse::()?; + let toml_table = toml_value + .as_table() + .ok_or_else(|| anyhow::anyhow!("Schedule file must be a TOML table"))?; + + let engines = Schedule::parse_engines(toml_table).await?; + let steps = Schedule::parse_steps(toml_table).await?; + + Ok(Self { engines, steps }) + } + + async fn parse_engines( + table: &Table, + ) -> anyhow::Result>> { + println!("parsing engine..."); + let engines = table + .get("engines") + .ok_or_else(|| anyhow::anyhow!("Schedule file must have an 'engines' table"))? + .as_table() + .ok_or_else(|| anyhow::anyhow!("'engines' must be a table"))?; + + let mut result = HashMap::new(); + for (name, engine_config) in engines { + println!("engine: {name}, config: {engine_config}"); + let engine_configs = engine_config + .as_table() + .ok_or_else(|| anyhow::anyhow!("Config of engine {name} is not a table"))?; + + println!("name {name}, engine config {engine_configs}"); + + let typ = engine_configs + .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(typ, engine_configs.clone()).await?; + + result.insert(name.clone(), engine); + } + + Ok(result) + } + + async fn parse_catalog() -> anyhow::Result> { + let catalog = RestCatalogBuilder::default() + .load("rest", HashMap::from([])) + .await?; + + Ok(Box::new(catalog)) + } + + async fn parse_steps(table: &Table) -> anyhow::Result> { + let steps = table + .get("steps") + .ok_or_else(|| anyhow!("steps not found"))? + .as_array() + .ok_or_else(|| anyhow!("steps is not array"))?; + + steps.iter().map(Schedule::parse_step).try_collect() + } + + fn parse_step(value: &Value) -> anyhow::Result { + let t = value + .as_table() + .ok_or_else(|| anyhow!("Step must be a table!"))?; + + let engine_name = t + .get("engine") + .ok_or_else(|| anyhow!("Property engine is missing in step"))? + .as_str() + .ok_or_else(|| anyhow!("Property engine is not a string in step"))? + .to_string(); + + let sql = t + .get("sql") + .ok_or_else(|| anyhow!("Property sql is missing in step"))? + .as_str() + .ok_or_else(|| anyhow!("Property sqlis not a string in step"))? + .to_string(); + + println!("engine: {engine_name}, sql: {sql}"); + Ok(Step { engine_name, sql }) + } + + pub async fn run(mut self) -> anyhow::Result<()> { + println!("running steps"); + + for step_idx in 0..self.steps.len() { + self.run_step(step_idx).await?; + } + + Ok(()) + } + + async fn run_step(&mut self, step_index: usize) -> anyhow::Result<()> { + println!("running step: {step_index}"); + + let step = &self.steps[step_index]; + + let engine = self + .engines + .get_mut(&step.engine_name) + .ok_or_else(|| anyhow!("Engine {} not found!", step.engine_name))?; + + let step_sql_path = PathBuf::from(format!( + "{}/testdata/slts/{}", + env!("CARGO_MANIFEST_DIR"), + &step.sql + )); + engine.run_slt_file(step_sql_path.as_path()).await?; + Ok(()) + } +} diff --git a/crates/sqllogictest/testdata/docker/docker-compose.yml b/crates/sqllogictest/testdata/docker/docker-compose.yml new file mode 100644 index 0000000000..5b30aa8f99 --- /dev/null +++ b/crates/sqllogictest/testdata/docker/docker-compose.yml @@ -0,0 +1,78 @@ +networks: + rest_bridge: + +services: + rest: + image: apache/iceberg-rest-fixture:1.9.2 + environment: + - AWS_ACCESS_KEY_ID=admin + - AWS_SECRET_ACCESS_KEY=password + - AWS_REGION=us-east-1 + - CATALOG_CATALOG__IMPL=org.apache.iceberg.jdbc.JdbcCatalog + - CATALOG_URI=jdbc:sqlite:file:/tmp/iceberg_rest_mode=memory + - CATALOG_WAREHOUSE=s3://icebergdata/demo + - CATALOG_IO__IMPL=org.apache.iceberg.aws.s3.S3FileIO + - CATALOG_S3_ENDPOINT=http://minio:9000 + depends_on: + - minio + networks: + rest_bridge: + ports: + - 8181:8181 + expose: + - 8181 + + minio: + image: minio/minio:RELEASE.2025-05-24T17-08-30Z + environment: + - MINIO_ROOT_USER=admin + - MINIO_ROOT_PASSWORD=password + - MINIO_DOMAIN=minio + - MINIO_DEFAULT_BUCKETS=icebergdata + hostname: icebergdata.minio + networks: + rest_bridge: + ports: + - 9000:9000 + - 9001:9001 + expose: + - 9001 + - 9000 + command: ["server", "/data", "--console-address", ":9001"] + + mc: + depends_on: + - minio + image: minio/mc:RELEASE.2025-05-21T01-59-54Z + environment: + - AWS_ACCESS_KEY_ID=admin + - AWS_SECRET_ACCESS_KEY=password + - AWS_REGION=us-east-1 + entrypoint: > + /bin/sh -c " until (/usr/bin/mc alias set minio http://minio:9000 admin password) do echo '...waiting...' && sleep 1; done; /usr/bin/mc rm -r --force minio/icebergdata; /usr/bin/mc mb minio/icebergdata; /usr/bin/mc policy set public minio/icebergdata; tail -f /dev/null " + networks: + rest_bridge: + + spark: + depends_on: + - rest + - minio + image: apache/spark:3.5.6-java17 + environment: + - AWS_ACCESS_KEY_ID=admin + - AWS_SECRET_ACCESS_KEY=password + - AWS_REGION=us-east-1 + - SPARK_HOME=/opt/spark + - PATH=/usr/local/sbin:/usr/local/bin:/usr/sbin:/usr/bin:/sbin:/bin:/opt/spark/bin:/opt/spark/sbin + user: root + networks: + rest_bridge: + ports: + - "15002:15002" + healthcheck: + test: netstat -ltn | grep -c 15002 + interval: 1s + retries: 1200 + volumes: + - ./spark:/spark-script + entrypoint: [ "/spark-script/spark-connect-server.sh" ] \ No newline at end of file diff --git a/crates/sqllogictest/testdata/docker/spark/spark-connect-server.sh b/crates/sqllogictest/testdata/docker/spark/spark-connect-server.sh new file mode 100755 index 0000000000..6bc2877d27 --- /dev/null +++ b/crates/sqllogictest/testdata/docker/spark/spark-connect-server.sh @@ -0,0 +1,27 @@ +#!/bin/bash + +set -ex + +SPARK_VERSION="3.5.6" +ICEBERG_VERSION="1.9.2" + +PACKAGES="org.apache.iceberg:iceberg-spark-runtime-3.5_2.12:$ICEBERG_VERSION" +PACKAGES="$PACKAGES,org.apache.iceberg:iceberg-aws-bundle:$ICEBERG_VERSION" +PACKAGES="$PACKAGES,org.apache.spark:spark-connect_2.12:$SPARK_VERSION" + +/opt/spark/sbin/start-connect-server.sh \ + --packages $PACKAGES \ + --master local[3] \ + --conf spark.driver.extraJavaOptions="-Dlog4j.configuration=file:///spark-script/log4j2.properties" \ + --conf spark.driver.bindAddress=0.0.0.0 \ + --conf spark.sql.catalog.default=org.apache.iceberg.spark.SparkCatalog \ + --conf spark.sql.extensions=org.apache.iceberg.spark.extensions.IcebergSparkSessionExtensions \ + --conf spark.sql.catalog.default.catalog-impl=org.apache.iceberg.rest.RESTCatalog \ + --conf spark.sql.catalog.default.uri=http://rest:8181 \ + --conf spark.sql.catalog.default.s3.endpoint=http://minio:9000 \ + --conf spark.sql.catalog.default.s3.path.style.access=true \ + --conf spark.sql.catalog.default.s3.access.key=admin \ + --conf spark.sql.catalog.default.s3.secret.key=password \ + --conf spark.sql.defaultCatalog=default + +tail -f /opt/spark/logs/spark*.out diff --git a/crates/sqllogictest/testdata/schedules/test.toml b/crates/sqllogictest/testdata/schedules/test.toml new file mode 100644 index 0000000000..109c37e7b6 --- /dev/null +++ b/crates/sqllogictest/testdata/schedules/test.toml @@ -0,0 +1,11 @@ +[engines] +sc = { type = "spark-connect", url = "sc://localhost:15002" } +df = { type = "datafusion", url = "http://localhost:8181" } + +[[steps]] +engine = "sc" +sql = "demo/prepare.slt" + +[[steps]] +engine = "df" +sql = "demo/verify.slt" diff --git a/crates/sqllogictest/testdata/slts/demo/prepare.slt b/crates/sqllogictest/testdata/slts/demo/prepare.slt new file mode 100644 index 0000000000..e25e4682d6 --- /dev/null +++ b/crates/sqllogictest/testdata/slts/demo/prepare.slt @@ -0,0 +1,18 @@ +statement ok +CREATE DATABASE IF NOT EXISTS s1; + +statement ok +USE DATABASE s1; + +statement ok +CREATE TABLE t1 (id INTEGER); + +statement ok +INSERT INTO t1 VALUES (1), (2), (3); + +query I +SELECT * FROM t1 ORDER BY id +---- +1 +2 +3 diff --git a/crates/sqllogictest/testdata/slts/demo/verify.slt b/crates/sqllogictest/testdata/slts/demo/verify.slt new file mode 100644 index 0000000000..f8744e2120 --- /dev/null +++ b/crates/sqllogictest/testdata/slts/demo/verify.slt @@ -0,0 +1,6 @@ +query I +SELECT * FROM demo.s1.t1 ORDER BY id; +---- +1 +2 +3 diff --git a/crates/sqllogictest/tests/sqllogictests.rs b/crates/sqllogictest/tests/sqllogictests.rs new file mode 100644 index 0000000000..4d9353dab8 --- /dev/null +++ b/crates/sqllogictest/tests/sqllogictests.rs @@ -0,0 +1,112 @@ +// 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 std::sync::RwLock; + +use iceberg_sqllogictest::schedule; +use iceberg_sqllogictest::schedule::Schedule; +use iceberg_test_utils::docker::DockerCompose; +use iceberg_test_utils::normalize_test_name; +use libtest_mimic::{Arguments, Trial}; +use tokio::runtime::Handle; + +static DOCKER_COMPOSE_ENV: RwLock> = RwLock::new(None); + +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(); + + setup_env(); + + let tests = collect_trials(rt.handle().clone()).unwrap(); + let result = libtest_mimic::run(&args, tests); + + teardown_env(); + drop(rt); + + result.exit(); +} + +fn setup_env() { + let mut guard = DOCKER_COMPOSE_ENV.write().unwrap(); + let docker_compose = DockerCompose::new( + normalize_test_name(module_path!()), + format!("{}/testdata/docker", env!("CARGO_MANIFEST_DIR")), + ); + docker_compose.up(); + guard.replace(docker_compose); +} + +fn teardown_env() { + let mut guard = DOCKER_COMPOSE_ENV.write().unwrap(); + guard.take(); +} + +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 schedule_file_name = schedule_file.file_name().unwrap().to_string_lossy(); + let schedules = Schedule::parse(schedule_file).await?; + schedules.run().await?; + + Ok(()) +} From 74cc6aac669464b4d3275043f2865d79ac9f034c Mon Sep 17 00:00:00 2001 From: Leon Lin Date: Thu, 21 Aug 2025 00:21:07 -0700 Subject: [PATCH 3/9] Update deps --- Cargo.lock | 1896 +++++++++++++++++++++++++++------------------------- 1 file changed, 1001 insertions(+), 895 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index 5c50e74490..38464a945d 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -13,9 +13,9 @@ dependencies = [ [[package]] name = "adler2" -version = "2.0.0" +version = "2.0.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "512761e0bb2578dd7380c6baaa0f4ce03e84f95e960231d1dec8bf4d7d6e2627" +checksum = "320119579fcad9c21884f5c4861d16174d0e06250625266f50fe6898340abefa" [[package]] name = "adler32" @@ -40,24 +40,24 @@ version = "0.7.8" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "891477e0c6a8957309ee5c45a6368af3ae14bb510732d2684ffa19af310920f9" dependencies = [ - "getrandom 0.2.15", + "getrandom 0.2.16", "once_cell", "version_check", ] [[package]] name = "ahash" -version = "0.8.11" +version = "0.8.12" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "e89da841a80418a9b391ebaea17f5c112ffaaa96f621d2c285b5174da76b9011" +checksum = "5a15f179cd60c4584b8a8c596927aadc462e27f2ca70c04e0071964a73ba7a75" dependencies = [ "cfg-if", "const-random", - "getrandom 0.2.15", + "getrandom 0.3.3", "once_cell", "serde", "version_check", - "zerocopy 0.7.35", + "zerocopy", ] [[package]] @@ -107,9 +107,9 @@ dependencies = [ [[package]] name = "anstream" -version = "0.6.18" +version = "0.6.20" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "8acc5369981196006228e28809f761875c0327210a891e941f4c683b3a99529b" +checksum = "3ae563653d1938f79b1ab1b5e668c87c76a9930414574a6583a7b7e11a8e6192" dependencies = [ "anstyle", "anstyle-parse", @@ -122,44 +122,44 @@ dependencies = [ [[package]] name = "anstyle" -version = "1.0.10" +version = "1.0.11" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "55cc3b69f167a1ef2e161439aa98aed94e6028e5f9a59be9a6ffb47aef1651f9" +checksum = "862ed96ca487e809f1c8e5a8447f6ee2cf102f846893800b20cebdf541fc6bbd" [[package]] name = "anstyle-parse" -version = "0.2.6" +version = "0.2.7" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "3b2d16507662817a6a20a9ea92df6652ee4f94f914589377d69f3b21bc5798a9" +checksum = "4e7644824f0aa2c7b9384579234ef10eb7efb6a0deb83f9630a49594dd9c15c2" dependencies = [ "utf8parse", ] [[package]] name = "anstyle-query" -version = "1.1.2" +version = "1.1.4" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "79947af37f4177cfead1110013d678905c37501914fba0efea834c3fe9a8d60c" +checksum = "9e231f6134f61b71076a3eab506c379d4f36122f2af15a9ff04415ea4c3339e2" dependencies = [ - "windows-sys 0.59.0", + "windows-sys 0.60.2", ] [[package]] name = "anstyle-wincon" -version = "3.0.7" +version = "3.0.10" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "ca3534e77181a9cc07539ad51f2141fe32f6c3ffd4df76db8ad92346b003ae4e" +checksum = "3e0633414522a32ffaac8ac6cc8f748e090c5717661fddeea04219e2344f5f2a" dependencies = [ "anstyle", - "once_cell", - "windows-sys 0.59.0", + "once_cell_polyfill", + "windows-sys 0.60.2", ] [[package]] name = "anyhow" -version = "1.0.95" +version = "1.0.99" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "34ac096ce696dc2fcabef30516bb13c0a68a11d30131d3df6f04711467681b04" +checksum = "b0674a1ddeecb70197781e945de4b3b8ffb61fa939a5597bcf48503737663100" [[package]] name = "apache-avro" @@ -210,9 +210,9 @@ checksum = "7c02d123df017efcdfbd739ef81735b36c5ba83ec3c59c80a9d7ecc718f92e50" [[package]] name = "arrow" -version = "55.0.0" +version = "55.2.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "3095aaf545942ff5abd46654534f15b03a90fba78299d661e045e5d587222f0d" +checksum = "f3f15b4c6b148206ff3a2b35002e08929c2462467b62b9c02036d9c34f9ef994" dependencies = [ "arrow-arith", "arrow-array", @@ -249,14 +249,14 @@ version = "55.2.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "70732f04d285d49054a48b72c54f791bb3424abae92d27aafdf776c98af161c8" dependencies = [ - "ahash 0.8.11", + "ahash 0.8.12", "arrow-buffer", "arrow-data", "arrow-schema", "chrono", - "chrono-tz 0.10.1", + "chrono-tz 0.10.4", "half", - "hashbrown 0.15.2", + "hashbrown 0.15.5", "num", ] @@ -294,9 +294,9 @@ dependencies = [ [[package]] name = "arrow-csv" -version = "55.0.0" +version = "55.2.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "510db7dfbb4d5761826516cc611d97b3a68835d0ece95b034a052601109c0b1b" +checksum = "012c9fef3f4a11573b2c74aec53712ff9fdae4a95f4ce452d1bbf088ee00f06b" dependencies = [ "arrow-array", "arrow-cast", @@ -304,7 +304,6 @@ dependencies = [ "chrono", "csv", "csv-core", - "lazy_static", "regex", ] @@ -336,9 +335,9 @@ dependencies = [ [[package]] name = "arrow-json" -version = "55.0.0" +version = "55.2.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d8dafd17a05449e31e0114d740530e0ada7379d7cb9c338fd65b09a8130960b0" +checksum = "5709d974c4ea5be96d900c01576c7c0b99705f4a3eec343648cb1ca863988a9c" dependencies = [ "arrow-array", "arrow-buffer", @@ -347,7 +346,7 @@ dependencies = [ "arrow-schema", "chrono", "half", - "indexmap 2.9.0", + "indexmap 2.10.0", "lexical-core", "memchr", "num", @@ -371,9 +370,9 @@ dependencies = [ [[package]] name = "arrow-row" -version = "55.0.0" +version = "55.2.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "9be8a2a4e5e7d9c822b2b8095ecd77010576d824f654d347817640acfc97d229" +checksum = "52bf7393166beaf79b4bed9bfdf19e97472af32ce5b6b48169d321518a08cae2" dependencies = [ "arrow-array", "arrow-buffer", @@ -394,7 +393,7 @@ version = "55.2.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "dd2b45757d6a2373faa3352d02ff5b54b098f5e21dccebc45a21806bc34501e5" dependencies = [ - "ahash 0.8.11", + "ahash 0.8.12", "arrow-array", "arrow-buffer", "arrow-data", @@ -451,7 +450,7 @@ version = "0.7.2" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "435a87a52755b8f27fcf321ac4f04b2802e337c8c4872923137471ec39c37532" dependencies = [ - "event-listener 5.4.0", + "event-listener 5.4.1", "event-listener-strategy", "futures-core", "pin-project-lite", @@ -470,9 +469,9 @@ dependencies = [ [[package]] name = "async-channel" -version = "2.3.1" +version = "2.5.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "89b47800b0be77592da0afd425cc03468052844aff33b84e33cc696f64e77b6a" +checksum = "924ed96dd52d1b75e9c1a3e6275715fd320f5f9439fb5a4a11fa51f4221158d2" dependencies = [ "concurrent-queue", "event-listener-strategy", @@ -499,14 +498,15 @@ dependencies = [ [[package]] name = "async-executor" -version = "1.13.1" +version = "1.13.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "30ca9a001c1e8ba5149f91a74362376cc6bc5b919d92d988668657bd570bdcec" +checksum = "bb812ffb58524bdd10860d7d974e2f01cc0950c2438a74ee5ec2e2280c6c4ffa" dependencies = [ "async-task", "concurrent-queue", "fastrand", "futures-lite", + "pin-project-lite", "slab", ] @@ -516,7 +516,7 @@ version = "2.4.1" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "05b1b633a2115cd122d73b955eadd9916c18c8f510ec9cd1686404c60ad1c29c" dependencies = [ - "async-channel 2.3.1", + "async-channel 2.5.0", "async-executor", "async-io", "async-lock", @@ -527,9 +527,9 @@ dependencies = [ [[package]] name = "async-io" -version = "2.4.0" +version = "2.5.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "43a2b323ccce0a1d90b449fd71f2a06ca7faa7c54c2751f06c9bd851fc061059" +checksum = "19634d6336019ef220f09fd31168ce5c184b295cbf80345437cc36094ef223ca" dependencies = [ "async-lock", "cfg-if", @@ -538,19 +538,18 @@ dependencies = [ "futures-lite", "parking", "polling", - "rustix 0.38.44", + "rustix 1.0.8", "slab", - "tracing", - "windows-sys 0.59.0", + "windows-sys 0.60.2", ] [[package]] name = "async-lock" -version = "3.4.0" +version = "3.4.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "ff6e472cdea888a4bd64f342f09b3f50e1886d32afe8df3d663c01140b811b18" +checksum = "5fd03604047cee9b6ce9de9f70c6cd540a0520c813cbd49bae61f33ab80ed1dc" dependencies = [ - "event-listener 5.4.0", + "event-listener 5.4.1", "event-listener-strategy", "pin-project-lite", ] @@ -563,14 +562,14 @@ checksum = "3b43422f69d8ff38f95f1b2bb76517c91589a924d1559a0e935d7c8ce0274c11" dependencies = [ "proc-macro2", "quote", - "syn 2.0.101", + "syn 2.0.106", ] [[package]] name = "async-std" -version = "1.13.0" +version = "1.13.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c634475f29802fde2b8f0b505b1bd00dfe4df7d4a000f0b36f7671197d5c3615" +checksum = "2c8e079a4ab67ae52b7403632e4618815d6db36d2a010cfe41b02c1b1578f93b" dependencies = [ "async-attributes", "async-channel 1.9.0", @@ -612,7 +611,7 @@ checksum = "c7c24de15d275a1ecfd47a380fb4d5ec9bfe0933f309ed5e705b775596a3574d" dependencies = [ "proc-macro2", "quote", - "syn 2.0.101", + "syn 2.0.106", ] [[package]] @@ -623,13 +622,13 @@ checksum = "8b75356056920673b02621b35afd0f7dda9306d03c79a30f5c56c44cf256e3de" [[package]] name = "async-trait" -version = "0.1.88" +version = "0.1.89" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "e539d3fca749fcee5236ab05e93a52867dd549cc157c8cb7f99595f3cedffdb5" +checksum = "9035ad2d096bed7955a320ee7e2230574d28fd3c3a0f186cbea1ff3c7eed5dbb" dependencies = [ "proc-macro2", "quote", - "syn 2.0.101", + "syn 2.0.106", ] [[package]] @@ -649,15 +648,15 @@ checksum = "1505bd5d3d116872e7271a6d4e16d81d0c8570876c8de68093a09ac269d8aac0" [[package]] name = "autocfg" -version = "1.4.0" +version = "1.5.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "ace50bade8e6234aa140d9a2f552bbee1db4d353f69b8217bc503490fc1a9f26" +checksum = "c08606f8c3cbf4ce6ec8e28fb0014a2c086708fe954eaa885384a6165172e7e8" [[package]] name = "aws-config" -version = "1.6.3" +version = "1.8.5" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "02a18fd934af6ae7ca52410d4548b98eb895aab0f1ea417d168d85db1434a141" +checksum = "c478f5b10ce55c9a33f87ca3404ca92768b144fc1bfdede7c0121214a8283a25" dependencies = [ "aws-credential-types", "aws-runtime", @@ -685,9 +684,9 @@ dependencies = [ [[package]] name = "aws-credential-types" -version = "1.2.3" +version = "1.2.5" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "687bc16bc431a8533fe0097c7f0182874767f920989d7260950172ae8e3c4465" +checksum = "1541072f81945fa1251f8795ef6c92c4282d74d59f88498ae7d4bf00f0ebdad9" dependencies = [ "aws-smithy-async", "aws-smithy-runtime-api", @@ -697,9 +696,9 @@ dependencies = [ [[package]] name = "aws-lc-rs" -version = "1.13.0" +version = "1.13.3" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "19b756939cb2f8dc900aa6dcd505e6e2428e9cae7ff7b028c49e3946efa70878" +checksum = "5c953fe1ba023e6b7730c0d4b031d06f267f23a46167dcbd40316644b10a17ba" dependencies = [ "aws-lc-sys", "zeroize", @@ -707,9 +706,9 @@ dependencies = [ [[package]] name = "aws-lc-sys" -version = "0.28.2" +version = "0.30.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "bfa9b6986f250236c27e5a204062434a773a13243d2ffc2955f37bdba4c5c6a1" +checksum = "dbfd150b5dbdb988bcc8fb1fe787eb6b7ee6180ca24da683b61ea5405f3d43ff" dependencies = [ "bindgen", "cc", @@ -720,9 +719,9 @@ dependencies = [ [[package]] name = "aws-runtime" -version = "1.5.7" +version = "1.5.10" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "6c4063282c69991e57faab9e5cb21ae557e59f5b0fb285c196335243df8dc25c" +checksum = "c034a1bc1d70e16e7f4e4caf7e9f7693e4c9c24cd91cf17c2a0b21abaebc7c8b" dependencies = [ "aws-credential-types", "aws-sigv4", @@ -744,9 +743,9 @@ dependencies = [ [[package]] name = "aws-sdk-glue" -version = "1.99.0" +version = "1.116.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "6d3964f2ad78d3fe39a327f109faaf0d0d7b2aa1742d99fb9434a36c99ca542a" +checksum = "b4b44ba32edf94e0723dfa1ee340170925858012ac0c981f4ee220c7455014bf" dependencies = [ "aws-credential-types", "aws-runtime", @@ -766,9 +765,9 @@ dependencies = [ [[package]] name = "aws-sdk-s3tables" -version = "1.25.0" +version = "1.35.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "3382afb09173e6f30369399041f0d3920031b14878d4b013ba7d3f04052359dd" +checksum = "c716a0dc793c4f21fd1f99737dc041cb7e1b82e9eb0bcf834f538b484ba667df" dependencies = [ "aws-credential-types", "aws-runtime", @@ -788,9 +787,9 @@ dependencies = [ [[package]] name = "aws-sdk-sso" -version = "1.72.0" +version = "1.81.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "13118ad30741222f67b1a18e5071385863914da05124652b38e172d6d3d9ce31" +checksum = "79ede098271e3471036c46957cba2ba30888f53bda2515bf04b560614a30a36e" dependencies = [ "aws-credential-types", "aws-runtime", @@ -810,9 +809,9 @@ dependencies = [ [[package]] name = "aws-sdk-ssooidc" -version = "1.73.0" +version = "1.82.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "f879a8572b4683a8f84f781695bebf2f25cf11a81a2693c31fc0e0215c2c1726" +checksum = "43326f724ba2cc957e6f3deac0ca1621a3e5d4146f5970c24c8a108dac33070f" dependencies = [ "aws-credential-types", "aws-runtime", @@ -832,9 +831,9 @@ dependencies = [ [[package]] name = "aws-sdk-sts" -version = "1.73.0" +version = "1.83.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "f1e9c3c24e36183e2f698235ed38dcfbbdff1d09b9232dc866c4be3011e0b47e" +checksum = "a5468593c47efc31fdbe6c902d1a5fde8d9c82f78a3f8ccfe907b1e9434748cb" dependencies = [ "aws-credential-types", "aws-runtime", @@ -855,9 +854,9 @@ dependencies = [ [[package]] name = "aws-sigv4" -version = "1.3.1" +version = "1.3.4" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "3503af839bd8751d0bdc5a46b9cac93a003a353e635b0c12cf2376b5b53e41ea" +checksum = "084c34162187d39e3740cb635acd73c4e3a551a36146ad6fe8883c929c9f876c" dependencies = [ "aws-credential-types", "aws-smithy-http", @@ -888,9 +887,9 @@ dependencies = [ [[package]] name = "aws-smithy-http" -version = "0.62.1" +version = "0.62.3" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "99335bec6cdc50a346fda1437f9fefe33abf8c99060739a546a16457f2862ca9" +checksum = "7c4dacf2d38996cf729f55e7a762b30918229917eca115de45dfa8dfb97796c9" dependencies = [ "aws-smithy-runtime-api", "aws-smithy-types", @@ -908,37 +907,39 @@ dependencies = [ [[package]] name = "aws-smithy-http-client" -version = "1.0.1" +version = "1.1.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "8aff1159006441d02e57204bf57a1b890ba68bedb6904ffd2873c1c4c11c546b" +checksum = "4fdbad9bd9dbcc6c5e68c311a841b54b70def3ca3b674c42fbebb265980539f8" dependencies = [ "aws-smithy-async", "aws-smithy-runtime-api", "aws-smithy-types", - "h2 0.4.7", + "h2 0.3.27", + "h2 0.4.12", "http 0.2.12", "http 1.3.1", "http-body 0.4.6", "hyper 0.14.32", - "hyper 1.6.0", + "hyper 1.7.0", "hyper-rustls 0.24.2", - "hyper-rustls 0.27.5", + "hyper-rustls 0.27.7", "hyper-util", "pin-project-lite", "rustls 0.21.12", - "rustls 0.23.23", + "rustls 0.23.31", "rustls-native-certs 0.8.1", "rustls-pki-types", "tokio", + "tokio-rustls 0.26.2", "tower 0.5.2", "tracing", ] [[package]] name = "aws-smithy-json" -version = "0.61.3" +version = "0.61.4" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "92144e45819cae7dc62af23eac5a038a58aa544432d2102609654376a900bd07" +checksum = "a16e040799d29c17412943bdbf488fd75db04112d0c0d4b9290bacf5ae0014b9" dependencies = [ "aws-smithy-types", ] @@ -964,9 +965,9 @@ dependencies = [ [[package]] name = "aws-smithy-runtime" -version = "1.8.3" +version = "1.9.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "14302f06d1d5b7d333fd819943075b13d27c7700b414f574c3c35859bfb55d5e" +checksum = "a3d57c8b53a72d15c8e190475743acf34e4996685e346a3448dd54ef696fc6e0" dependencies = [ "aws-smithy-async", "aws-smithy-http", @@ -988,9 +989,9 @@ dependencies = [ [[package]] name = "aws-smithy-runtime-api" -version = "1.8.0" +version = "1.9.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a1e5d9e3a80a18afa109391fb5ad09c3daf887b516c6fd805a157c6ea7994a57" +checksum = "07f5e0fc8a6b3f2303f331b94504bbf754d85488f402d6f1dd7a6080f99afe56" dependencies = [ "aws-smithy-async", "aws-smithy-types", @@ -1005,9 +1006,9 @@ dependencies = [ [[package]] name = "aws-smithy-types" -version = "1.3.1" +version = "1.3.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "40076bd09fadbc12d5e026ae080d0930defa606856186e31d83ccc6a255eeaf3" +checksum = "d498595448e43de7f4296b7b7a18a8a02c61ec9349128c80a368f7c3b4ab11a8" dependencies = [ "base64-simd", "bytes", @@ -1031,18 +1032,18 @@ dependencies = [ [[package]] name = "aws-smithy-xml" -version = "0.60.9" +version = "0.60.10" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "ab0b0166827aa700d3dc519f72f8b3a91c35d0b8d042dc5d643a91e6f80648fc" +checksum = "3db87b96cb1b16c024980f133968d52882ca0daaee3a086c6decc500f6c99728" dependencies = [ "xmlparser", ] [[package]] name = "aws-types" -version = "1.3.7" +version = "1.3.8" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "8a322fec39e4df22777ed3ad8ea868ac2f94cd15e1a55f6ee8d8d6305057689a" +checksum = "b069d19bf01e46298eaedd7c6f283fe565a59263e53eebec945f3e6398f42390" dependencies = [ "aws-credential-types", "aws-smithy-async", @@ -1099,9 +1100,9 @@ dependencies = [ [[package]] name = "backon" -version = "1.5.1" +version = "1.5.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "302eaff5357a264a2c42f127ecb8bac761cf99749fc3dc95677e2743991f99e7" +checksum = "592277618714fbcecda9a02ba7a8781f319d26532a88553bbacc77ba5d2b3a8d" dependencies = [ "fastrand", "gloo-timers", @@ -1110,9 +1111,9 @@ dependencies = [ [[package]] name = "backtrace" -version = "0.3.74" +version = "0.3.75" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "8d82cb332cdfaed17ae235a638438ac4d4839913cc2af585c3c6746e8f8bee1a" +checksum = "6806a6321ec58106fea15becdad98371e28d92ccbc7c8f1b3b6dd724fe8f1002" dependencies = [ "addr2line", "cfg-if", @@ -1147,9 +1148,9 @@ dependencies = [ [[package]] name = "base64ct" -version = "1.6.0" +version = "1.8.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "8c3c1a368f70d6cf7302d78f8f7093da241fb8e8807c05cc9e51a125895a6d5b" +checksum = "55248b47b0caf0546f7988906588779981c43bb1bc9d0c44087278f80cdb44ba" [[package]] name = "bigdecimal" @@ -1177,7 +1178,7 @@ version = "0.69.5" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "271383c67ccabffb7381723dea0672a673f292304fcb45c01cc648c7a8d58088" dependencies = [ - "bitflags 2.8.0", + "bitflags 2.9.2", "cexpr", "clang-sys", "itertools 0.12.1", @@ -1190,7 +1191,7 @@ dependencies = [ "regex", "rustc-hash 1.1.0", "shlex", - "syn 2.0.101", + "syn 2.0.106", "which", ] @@ -1202,9 +1203,9 @@ checksum = "bef38d45163c2f1dde094a7dfd33ccf595c92905c8f8f4fdc18d06fb1037718a" [[package]] name = "bitflags" -version = "2.8.0" +version = "2.9.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "8f68f53c83ab957f72c32642f3868eec03eb974d1fb82e453128456482613d36" +checksum = "6a65b545ab31d687cff52899d4890855fec459eb6afe0da6417b8a18da87aa29" dependencies = [ "serde", ] @@ -1263,11 +1264,11 @@ dependencies = [ [[package]] name = "blocking" -version = "1.6.1" +version = "1.6.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "703f41c54fc768e63e091340b424302bb1c29ef4aa0c7f10fe849dfb114d29ea" +checksum = "e83f8d02be6967315521be875afa792a316e28d57b5a2d401897e2a7921b7f21" dependencies = [ - "async-channel 2.3.1", + "async-channel 2.5.0", "async-task", "futures-io", "futures-lite", @@ -1276,9 +1277,9 @@ dependencies = [ [[package]] name = "borsh" -version = "1.5.5" +version = "1.5.7" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "5430e3be710b68d984d1391c854eb431a9d548640711faa54eecb1df93db91cc" +checksum = "ad8646f98db542e39fc66e68a20b2144f6a732636df7c2354e74645faaa433ce" dependencies = [ "borsh-derive", "cfg_aliases", @@ -1286,22 +1287,22 @@ dependencies = [ [[package]] name = "borsh-derive" -version = "1.5.5" +version = "1.5.7" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "f8b668d39970baad5356d7c83a86fee3a539e6f93bf6764c97368243e17a0487" +checksum = "fdd1d3c0c2f5833f22386f252fe8ed005c7f59fdcddeef025c01b4c3b9fd9ac3" dependencies = [ "once_cell", "proc-macro-crate", "proc-macro2", "quote", - "syn 2.0.101", + "syn 2.0.106", ] [[package]] name = "brotli" -version = "8.0.1" +version = "8.0.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "9991eea70ea4f293524138648e41ee89b0b2b12ddef3b255effa43c8056e0e0d" +checksum = "4bd8b9603c7aa97359dbd97ecf258968c95f3adddd6db2f7e7a5bef101c84560" dependencies = [ "alloc-no-stdlib", "alloc-stdlib", @@ -1320,9 +1321,9 @@ dependencies = [ [[package]] name = "bstr" -version = "1.11.3" +version = "1.12.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "531a9155a481e2ee699d4f98f43c0ca4ff8ee1bfd55c31e9e98fb29d2b176fe0" +checksum = "234113d19d0d7d613b40e86fb654acf958910802bcceab913a4f9e7cda03b1a4" dependencies = [ "memchr", "serde", @@ -1330,9 +1331,9 @@ dependencies = [ [[package]] name = "bumpalo" -version = "3.17.0" +version = "3.19.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "1628fb46dfa0b37568d12e5edd512553eccf6a22a78e8bde00bb4aed84d5bdbf" +checksum = "46c5e41b57b8bba42a04676d81cb89e9ee8e859a1a66f80a5a72e1cb76b34d43" [[package]] name = "bytecheck" @@ -1358,9 +1359,9 @@ dependencies = [ [[package]] name = "bytemuck" -version = "1.21.0" +version = "1.23.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "ef657dfab802224e671f5818e9a4935f9b1957ed18e58292690cc39e7a4092a3" +checksum = "3995eaeebcdf32f91f980d360f78732ddc061097ab4e39991ae7a6ace9194677" [[package]] name = "byteorder" @@ -1370,9 +1371,9 @@ checksum = "1fd0f2584146f6f2ef48085050886acf353beff7305ebd1ae69500e27c67f64b" [[package]] name = "bytes" -version = "1.10.0" +version = "1.10.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "f61dac84819c6588b558454b194026eb1f09c293b9036ae9b159e74e73ab6cf9" +checksum = "d71b6127be86fdcfddb610f7182ac57211d4b18a3e9c82eb2d17662f2227ad6a" dependencies = [ "serde", ] @@ -1427,9 +1428,9 @@ dependencies = [ [[package]] name = "cc" -version = "1.2.14" +version = "1.2.33" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "0c3d1b2e905a3a7b00a6141adb0e4c0bb941d11caf55349d863942a1cc44e3c9" +checksum = "3ee0f8803222ba5a7e2777dd72ca451868909b1ac410621b676adf07280e9b5f" dependencies = [ "jobserver", "libc", @@ -1447,9 +1448,9 @@ dependencies = [ [[package]] name = "cfg-if" -version = "1.0.0" +version = "1.0.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "baf1de4339761588bc0619e3cbc0120ee582ebb74b53b4efbf79117bd2da40fd" +checksum = "9555578bc9e57714c812a1f84e4fc5b4d21fcb063490c624de019f7464c91268" [[package]] name = "cfg_aliases" @@ -1479,19 +1480,18 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "93698b29de5e97ad0ae26447b344c482a7284c737d9ddc5f9e52b74a336671bb" dependencies = [ "chrono", - "chrono-tz-build 0.3.0", - "phf", + "chrono-tz-build", + "phf 0.11.3", ] [[package]] name = "chrono-tz" -version = "0.10.1" +version = "0.10.4" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "9c6ac4f2c0bf0f44e9161aec9675e1050aa4a530663c4a9e37e108fa948bca9f" +checksum = "a6139a8597ed92cf816dfb33f5dd6cf0bb93a6adc938f11039f371bc5bcd26c3" dependencies = [ "chrono", - "chrono-tz-build 0.4.0", - "phf", + "phf 0.12.1", ] [[package]] @@ -1501,17 +1501,7 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "0c088aee841df9c3041febbb73934cfc39708749bf96dc827e3359cd39ef11b1" dependencies = [ "parse-zoneinfo", - "phf", - "phf_codegen", -] - -[[package]] -name = "chrono-tz-build" -version = "0.4.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "e94fea34d77a245229e7746bd2beb786cd2a896f306ff491fb8cecb3074b10a7" -dependencies = [ - "parse-zoneinfo", + "phf 0.11.3", "phf_codegen", ] @@ -1538,9 +1528,9 @@ dependencies = [ [[package]] name = "clap" -version = "4.5.35" +version = "4.5.45" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d8aa86934b44c19c50f87cc2790e19f54f7a67aedb64101c2e1a2e5ecfb73944" +checksum = "1fc0e74a703892159f5ae7d3aac52c8e6c392f5ae5f359c70b5881d60aaac318" dependencies = [ "clap_builder", "clap_derive", @@ -1548,9 +1538,9 @@ dependencies = [ [[package]] name = "clap_builder" -version = "4.5.35" +version = "4.5.44" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "2414dbb2dd0695280da6ea9261e327479e9d37b0630f6b53ba2a11c60c679fd9" +checksum = "b3e7f4214277f3c7aa526a59dd3fbe306a370daee1f8b7b8c987069cd8e888a8" dependencies = [ "anstream", "anstyle", @@ -1560,27 +1550,27 @@ dependencies = [ [[package]] name = "clap_derive" -version = "4.5.32" +version = "4.5.45" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "09176aae279615badda0765c0c0b3f6ed53f4709118af73cf4655d85d1530cd7" +checksum = "14cb31bb0a7d536caef2639baa7fad459e15c3144efefa6dbd1c84562c4739f6" dependencies = [ "heck", "proc-macro2", "quote", - "syn 2.0.101", + "syn 2.0.106", ] [[package]] name = "clap_lex" -version = "0.7.4" +version = "0.7.5" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "f46ad14479a25103f283c0f10005961cf086d8dc42205bb44c46ac563475dca6" +checksum = "b94f61472cee1439c0b966b47e3aca9ae07e45d070759512cd390ea2bebc6675" [[package]] name = "clipboard-win" -version = "5.4.0" +version = "5.4.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "15efe7a882b08f34e38556b14f2fb3daa98769d06c7f0c1b076dfd0d983bc892" +checksum = "bde03770d3df201d4fb868f2c9c59e66a3e4e2bd06692a0fe701e7103c7e84d4" dependencies = [ "error-code", ] @@ -1596,17 +1586,16 @@ dependencies = [ [[package]] name = "colorchoice" -version = "1.0.3" +version = "1.0.4" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "5b63caa9aa9397e2d9480a9b13673856c78d8ac123288526c37d7839f2a86990" +checksum = "b05b61dc5112cbb17e4b6cd61790d9845d13888356391624cbe7e41efeac1e75" [[package]] name = "colored" -version = "2.2.0" +version = "3.0.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "117725a109d387c937a1533ce01b450cbde6b88abceea8473c4d7a85853cda3c" +checksum = "fde0e0ec90c9dfb3b4b1a0891a7dcd0e2bffde2f7efed5fe7c9bb00e5bfb915e" dependencies = [ - "lazy_static", "windows-sys 0.59.0", ] @@ -1617,7 +1606,7 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "4a65ebfec4fb190b6f90e944a817d60499ee0744e582530e2c9900a22e591d9a" dependencies = [ "unicode-segmentation", - "unicode-width 0.2.0", + "unicode-width 0.2.1", ] [[package]] @@ -1638,7 +1627,7 @@ dependencies = [ "encode_unicode", "libc", "once_cell", - "unicode-width 0.2.0", + "unicode-width 0.2.1", "windows-sys 0.59.0", ] @@ -1663,7 +1652,7 @@ version = "0.1.16" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "f9d839f2a20b0aee515dc581a6172f2321f96cab76c1a38a4c584a194955390e" dependencies = [ - "getrandom 0.2.15", + "getrandom 0.2.16", "once_cell", "tiny-keccak", ] @@ -1686,9 +1675,9 @@ dependencies = [ [[package]] name = "core-foundation" -version = "0.10.0" +version = "0.10.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "b55271e5c8c478ad3f38ad24ef34923091e0548492a266d19b3c0b4d82574c63" +checksum = "b2a6cd9ae233e7f62ba4e9353e81a88df7fc8a5987b8d445b4d90c879bd156f6" dependencies = [ "core-foundation-sys", "libc", @@ -1720,9 +1709,9 @@ dependencies = [ [[package]] name = "crc" -version = "3.2.1" +version = "3.3.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "69e6e4d7b33a94f0991c26729976b10ebde1d34c3ee82408fb536164fa10d636" +checksum = "9710d3b3739c2e349eb44fe848ad0b7c8cb1e42bd87ee49371df2f7acaf3e675" dependencies = [ "crc-catalog", ] @@ -1744,9 +1733,9 @@ dependencies = [ [[package]] name = "crc32fast" -version = "1.4.2" +version = "1.5.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a97769d94ddab943e4510d138150169a2758b5ef3eb191a9ee688de3e23ef7b3" +checksum = "9481c1c90cbf2ac953f07c8d4a58aa3945c425b7185c9154d67a65e4230da511" dependencies = [ "cfg-if", ] @@ -1796,9 +1785,9 @@ checksum = "d0a5c400df2834b80a4c3327b3aad3a4c4cd4de0629063962b03235697506a28" [[package]] name = "crunchy" -version = "0.2.3" +version = "0.2.4" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "43da5946c66ffcc7745f48db692ffbb10a83bfe0afd96235c5c2a4fb23994929" +checksum = "460fbee9c2c2f33933d720630a6a0bac33ba7053db5344fac858d4b8952d77d5" [[package]] name = "crypto-common" @@ -1838,14 +1827,14 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "32a2785755761f3ddc1492979ce1e48d2c00d09311c39e4466429188f3dd6501" dependencies = [ "quote", - "syn 2.0.101", + "syn 2.0.106", ] [[package]] name = "darling" -version = "0.20.10" +version = "0.20.11" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "6f63b86c8a8826a49b8c21f08a2d07338eec8d900540f8630dc76284be802989" +checksum = "fc7f46116c46ff9ab3eb1597a45688b6715c6e628b5c133e288e709a29bcb4ee" dependencies = [ "darling_core", "darling_macro", @@ -1853,27 +1842,27 @@ dependencies = [ [[package]] name = "darling_core" -version = "0.20.10" +version = "0.20.11" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "95133861a8032aaea082871032f5815eb9e98cef03fa916ab4500513994df9e5" +checksum = "0d00b9596d185e565c2207a0b01f8bd1a135483d02d9b7b0a54b11da8d53412e" dependencies = [ "fnv", "ident_case", "proc-macro2", "quote", "strsim", - "syn 2.0.101", + "syn 2.0.106", ] [[package]] name = "darling_macro" -version = "0.20.10" +version = "0.20.11" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d336a2a514f6ccccaa3e09b02d41d35330c07ddf03a62165fcec10bb561c7806" +checksum = "fc34b93ccb385b40dc71c6fceac4b2ad23662c7eeb248cf10d529b7e055b6ead" dependencies = [ "darling_core", "quote", - "syn 2.0.101", + "syn 2.0.106", ] [[package]] @@ -2032,14 +2021,14 @@ version = "47.0.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "0734015d81c8375eb5d4869b7f7ecccc2ee8d6cb81948ef737cd0e7b743bd69c" dependencies = [ - "ahash 0.8.11", + "ahash 0.8.12", "apache-avro", "arrow", "arrow-ipc", "base64 0.22.1", "half", "hashbrown 0.14.5", - "indexmap 2.9.0", + "indexmap 2.10.0", "libc", "log", "object_store", @@ -2243,7 +2232,7 @@ dependencies = [ "datafusion-functions-aggregate-common", "datafusion-functions-window-common", "datafusion-physical-expr-common", - "indexmap 2.9.0", + "indexmap 2.10.0", "paste", "recursive", "serde_json", @@ -2258,7 +2247,7 @@ checksum = "422ac9cf3b22bbbae8cdf8ceb33039107fde1b5492693168f13bd566b1bcc839" dependencies = [ "arrow", "datafusion-common", - "indexmap 2.9.0", + "indexmap 2.10.0", "itertools 0.14.0", "paste", ] @@ -2298,7 +2287,7 @@ version = "47.0.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "408a05dafdc70d05a38a29005b8b15e21b0238734dab1e98483fcb58038c5aba" dependencies = [ - "ahash 0.8.11", + "ahash 0.8.12", "arrow", "datafusion-common", "datafusion-doc", @@ -2319,7 +2308,7 @@ version = "47.0.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "756d21da2dd6c9bef97af1504970ff56cbf35d03fbd4ffd62827f02f4d2279d4" dependencies = [ - "ahash 0.8.11", + "ahash 0.8.12", "arrow", "datafusion-common", "datafusion-expr-common", @@ -2398,7 +2387,7 @@ checksum = "df6f88d7ee27daf8b108ba910f9015176b36fbc72902b1ca5c2a5f1d1717e1a1" dependencies = [ "datafusion-expr", "quote", - "syn 2.0.101", + "syn 2.0.106", ] [[package]] @@ -2412,7 +2401,7 @@ dependencies = [ "datafusion-common", "datafusion-expr", "datafusion-physical-expr", - "indexmap 2.9.0", + "indexmap 2.10.0", "itertools 0.14.0", "log", "recursive", @@ -2426,7 +2415,7 @@ version = "47.0.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "64c536062b0076f4e30084065d805f389f9fe38af0ca75bcbac86bc5e9fbab65" dependencies = [ - "ahash 0.8.11", + "ahash 0.8.12", "arrow", "datafusion-common", "datafusion-expr", @@ -2435,7 +2424,7 @@ dependencies = [ "datafusion-physical-expr-common", "half", "hashbrown 0.14.5", - "indexmap 2.9.0", + "indexmap 2.10.0", "itertools 0.14.0", "log", "paste", @@ -2448,7 +2437,7 @@ version = "47.0.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "f8a92b53b3193fac1916a1c5b8e3f4347c526f6822e56b71faa5fb372327a863" dependencies = [ - "ahash 0.8.11", + "ahash 0.8.12", "arrow", "datafusion-common", "datafusion-expr-common", @@ -2481,7 +2470,7 @@ version = "47.0.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "690c615db468c2e5fe5085b232d8b1c088299a6c63d87fd960a354a71f7acb55" dependencies = [ - "ahash 0.8.11", + "ahash 0.8.12", "arrow", "arrow-ord", "arrow-schema", @@ -2497,7 +2486,7 @@ dependencies = [ "futures", "half", "hashbrown 0.14.5", - "indexmap 2.9.0", + "indexmap 2.10.0", "itertools 0.14.0", "log", "parking_lot", @@ -2539,7 +2528,7 @@ dependencies = [ "bigdecimal", "datafusion-common", "datafusion-expr", - "indexmap 2.9.0", + "indexmap 2.10.0", "log", "recursive", "regex", @@ -2567,15 +2556,15 @@ dependencies = [ "sqllogictest", "sqlparser", "tempfile", - "thiserror 2.0.12", + "thiserror 2.0.15", "tokio", ] [[package]] name = "der" -version = "0.7.9" +version = "0.7.10" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "f55bf8e7b65898637379c1b74eb1551107c8294ed26d855ceb9fd1a09cfc9bc0" +checksum = "e7c1832837b905bbfb5101e07cc24c8deddf52f93225eee6ead5f4d63d53ddcb" dependencies = [ "const-oid", "pem-rfc7468", @@ -2584,25 +2573,14 @@ dependencies = [ [[package]] name = "deranged" -version = "0.3.11" +version = "0.4.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "b42b6fa04a440b495c8b04d0e71b707c585f83cb9cb28cf8cd0d976c315e31b4" +checksum = "9c9e6a11ca8224451684bc0d7d5a7adbf8f2fd6887261a1cfc3c0432f9d4068e" dependencies = [ "powerfmt", "serde", ] -[[package]] -name = "derivative" -version = "2.2.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "fcc3dd5e9e9c0b295d6e1e4d811fb6f157d5ffd784b8d202fc62eac8035a770b" -dependencies = [ - "proc-macro2", - "quote", - "syn 1.0.109", -] - [[package]] name = "derive_builder" version = "0.20.2" @@ -2621,7 +2599,7 @@ dependencies = [ "darling", "proc-macro2", "quote", - "syn 2.0.101", + "syn 2.0.106", ] [[package]] @@ -2631,14 +2609,14 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "ab63b0e2bf4d5928aff72e83a7dace85d7bba5fe12dcc3c5a572d78caffd3f3c" dependencies = [ "derive_builder_core", - "syn 2.0.101", + "syn 2.0.106", ] [[package]] name = "deunicode" -version = "1.6.0" +version = "1.6.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "339544cc9e2c4dc3fc7149fd630c5f22263a4fdf18a98afd0075784968b5cf00" +checksum = "abd57806937c9cc163efc8ea3910e00a62e2aeb0b8119f1793a978088f8f6b04" [[package]] name = "diff" @@ -2676,7 +2654,7 @@ dependencies = [ "libc", "option-ext", "redox_users", - "windows-sys 0.59.0", + "windows-sys 0.60.2", ] [[package]] @@ -2687,14 +2665,14 @@ checksum = "97369cbbc041bc366949bc74d34658d6cda5621039731c6310521892a3a20ae0" dependencies = [ "proc-macro2", "quote", - "syn 2.0.101", + "syn 2.0.106", ] [[package]] name = "dissimilar" -version = "1.0.9" +version = "1.0.10" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "59f8e79d1fbf76bdfbde321e902714bf6c49df88a7dda6fc682fc2979226962d" +checksum = "8975ffdaa0ef3661bfe02dbdcc06c9f829dfafe6a3c474de366a8d5e44276921" [[package]] name = "dlv-list" @@ -2725,9 +2703,9 @@ checksum = "92773504d58c093f6de2459af4af33faa518c13451eb8f2b5698ed3d36e7c813" [[package]] name = "dyn-clone" -version = "1.0.19" +version = "1.0.20" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "1c7a8fb8a9fbf66c1f703fe16184d10ca0ee9d23be5b4436400408ba54a95005" +checksum = "d0881ea181b1df73ff77ffaaf9c7544ecc11e82fba9b5f27b262a3c73a332555" [[package]] name = "educe" @@ -2738,7 +2716,7 @@ dependencies = [ "enum-ordinalize", "proc-macro2", "quote", - "syn 2.0.101", + "syn 2.0.106", ] [[package]] @@ -2779,7 +2757,7 @@ checksum = "0d28318a75d4aead5c4db25382e8ef717932d0346600cacae6357eb5941bc5ff" dependencies = [ "proc-macro2", "quote", - "syn 2.0.101", + "syn 2.0.106", ] [[package]] @@ -2813,19 +2791,19 @@ checksum = "877a4ace8713b0bcf2a4e7eec82529c029f1d0619886d18145fea96c3ffe5c0f" [[package]] name = "errno" -version = "0.3.10" +version = "0.3.13" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "33d852cb9b869c2a9b3df2f71a3074817f01e1844f839a144f5fcef059a4eb5d" +checksum = "778e2ac28f6c47af28e4907f13ffd1e1ddbd400980a9abd7c8df189bf578a5ad" dependencies = [ "libc", - "windows-sys 0.59.0", + "windows-sys 0.60.2", ] [[package]] name = "error-code" -version = "3.3.1" +version = "3.3.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a5d9305ccc6942a704f4335694ecd3de2ea531b114ac2d51f5f843750787a92f" +checksum = "dea2df4cf52843e0452895c455a1a2cfbb842a1e7329671acf418fdc53ed4c59" [[package]] name = "escape8259" @@ -2852,9 +2830,9 @@ checksum = "0206175f82b8d6bf6652ff7d71a1e27fd2e4efde587fd368662814d6ec1d9ce0" [[package]] name = "event-listener" -version = "5.4.0" +version = "5.4.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "3492acde4c3fc54c845eaab3eed8bd00c7a7d881f78bfc801e43a93dec1331ae" +checksum = "e13b66accf52311f30a0db42147dadea9850cb48cd070028831ae5f5d4b856ab" dependencies = [ "concurrent-queue", "parking", @@ -2863,11 +2841,11 @@ dependencies = [ [[package]] name = "event-listener-strategy" -version = "0.5.3" +version = "0.5.4" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "3c3e4e0dd3673c1139bf041f3008816d9cf2946bbfac2945c09e523b8d7b05b2" +checksum = "8be9f3dfaaffdae2972880079a491a1a8bb7cbed0b8dd7a347f668b4150a3b93" dependencies = [ - "event-listener 5.4.0", + "event-listener 5.4.1", "pin-project-lite", ] @@ -2900,7 +2878,7 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "a6503af7917fea18ffef8f7e8553fb8dff89e2e6837e94e09dd7fb069c82d62c" dependencies = [ "bytes", - "rkyv 0.8.10", + "rkyv 0.8.11", "serde", "simdutf8", ] @@ -2912,7 +2890,7 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "0ce92ff622d6dadf7349484f42c93271a0d49b7cc4d466a936405bacbe10aa78" dependencies = [ "cfg-if", - "rustix 1.0.1", + "rustix 1.0.8", "windows-sys 0.59.0", ] @@ -2934,15 +2912,15 @@ version = "25.2.10" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "1045398c1bfd89168b5fd3f1fc11f6e70b34f6f66300c87d44d3de849463abf1" dependencies = [ - "bitflags 2.8.0", + "bitflags 2.9.2", "rustc_version", ] [[package]] name = "flate2" -version = "1.1.1" +version = "1.1.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "7ced92e76e966ca2fd84c8f7aa01a4aea65b0eb6648d72f7c8f3e2764a67fece" +checksum = "4a3d7db9596fecd151c5f638c0ee5d5bd487b6e0ea232e5dc96d5250f6f94b1d" dependencies = [ "crc32fast", "libz-rs-sys", @@ -2968,9 +2946,9 @@ checksum = "3f9eec918d3f24069decb9af1554cad7c880e2da24a9afd88aca000531ab82c1" [[package]] name = "foldhash" -version = "0.1.4" +version = "0.1.5" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a0d2fde1f7b3d48b8395d5f2de76c18a528bd6a9cdde438df747bfcba3e05d6f" +checksum = "d9c4f5dac5e15c24eb999c26181a6ca40b39fe946cbe4c263c7209467bc83af2" [[package]] name = "form_urlencoded" @@ -2989,9 +2967,9 @@ checksum = "28dd6caf6059519a65843af8fe2a3ae298b14b80179855aeb4adc2c1934ee619" [[package]] name = "fs-err" -version = "3.1.0" +version = "3.1.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "1f89bda4c2a21204059a977ed3bfe746677dfd137b83c339e702b0ac91d482aa" +checksum = "88d7be93788013f265201256d58f04936a8079ad5dc898743aa20525f503b683" dependencies = [ "autocfg", ] @@ -3069,9 +3047,9 @@ checksum = "9e5c1b78ca4aae1ac06c48a526a655760685149f0d465d21f37abfe57ce075c6" [[package]] name = "futures-lite" -version = "2.6.0" +version = "2.6.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "f5edaec856126859abb19ed65f39e90fea3a9574b9707f13539acf4abf7eb532" +checksum = "f78e10609fe0e0b3f4157ffab1876319b5b0db102a2c60dc4626306dc46b44ad" dependencies = [ "fastrand", "futures-core", @@ -3088,7 +3066,7 @@ checksum = "162ee34ebcb7c64a8abebc059ce0fee27c2262618d7b60ed8faf72fef13c3650" dependencies = [ "proc-macro2", "quote", - "syn 2.0.101", + "syn 2.0.106", ] [[package]] @@ -3123,10 +3101,11 @@ dependencies = [ [[package]] name = "generator" -version = "0.8.4" +version = "0.8.5" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "cc6bd114ceda131d3b1d665eba35788690ad37f5916457286b32ab6fd3c438dd" +checksum = "d18470a76cb7f8ff746cf1f7470914f900252ec36bbc40b569d74b1258446827" dependencies = [ + "cc", "cfg-if", "libc", "log", @@ -3146,27 +3125,29 @@ dependencies = [ [[package]] name = "getrandom" -version = "0.2.15" +version = "0.2.16" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c4567c8db10ae91089c99af84c68c38da3ec2f087c3f82960bcdbf3656b6f4d7" +checksum = "335ff9f135e4384c8150d6f27c6daed433577f86b4750418338c01a1a2528592" dependencies = [ "cfg-if", "js-sys", "libc", - "wasi 0.11.0+wasi-snapshot-preview1", + "wasi 0.11.1+wasi-snapshot-preview1", "wasm-bindgen", ] [[package]] name = "getrandom" -version = "0.3.1" +version = "0.3.3" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "43a49c392881ce6d5c3b8cb70f98717b7c07aabbdff06687b9030dbfbe2725f8" +checksum = "26145e563e54f2cadc477553f1ec5ee650b00862f0a58bcd12cbdc5f0ea2d2f4" dependencies = [ "cfg-if", + "js-sys", "libc", - "wasi 0.13.3+wasi-0.2.2", - "windows-targets 0.52.6", + "r-efi", + "wasi 0.14.2+wasi-0.2.4", + "wasm-bindgen", ] [[package]] @@ -3177,15 +3158,15 @@ checksum = "07e28edb80900c19c28f1072f2e8aeca7fa06b23cd4169cefe1af5aa3260783f" [[package]] name = "glob" -version = "0.3.2" +version = "0.3.3" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a8d1add55171497b4705a648c6b583acafb01d58050a51727785f0b2c8e0a2b2" +checksum = "0cc23270f6e1808e30a928bdc84dea0b9b4136a8bc82338574f23baf47bbd280" [[package]] name = "globset" -version = "0.4.15" +version = "0.4.16" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "15f1ce686646e7f1e19bf7d5533fe443a45dbfb990e00629110797578b42fb19" +checksum = "54a1028dfc5f5df5da8a56a73e6c153c9a9708ec57232470703592a3f18e49f5" dependencies = [ "aho-corasick", "bstr", @@ -3200,7 +3181,7 @@ version = "0.9.1" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "0bf760ebf69878d9fd8f110c89703d90ce35095324d1f1edcb595c63945ee757" dependencies = [ - "bitflags 2.8.0", + "bitflags 2.9.2", "ignore", "walkdir", ] @@ -3219,9 +3200,9 @@ dependencies = [ [[package]] name = "h2" -version = "0.3.26" +version = "0.3.27" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "81fe527a889e1532da5c525686d96d4c2e74cdd345badf8dfef9f6b39dd5f5e8" +checksum = "0beca50380b1fc32983fc1cb4587bfa4bb9e78fc259aad4a0032d2080309222d" dependencies = [ "bytes", "fnv", @@ -3229,7 +3210,7 @@ dependencies = [ "futures-sink", "futures-util", "http 0.2.12", - "indexmap 2.9.0", + "indexmap 2.10.0", "slab", "tokio", "tokio-util", @@ -3238,9 +3219,9 @@ dependencies = [ [[package]] name = "h2" -version = "0.4.7" +version = "0.4.12" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "ccae279728d634d083c00f6099cb58f01cc99c145b84b8be2f6c74618d79922e" +checksum = "f3c0b69cfcb4e1b9f1bf2f53f95f766e4661169728ec61cd3fe5a0166f2d1386" dependencies = [ "atomic-waker", "bytes", @@ -3248,7 +3229,7 @@ dependencies = [ "futures-core", "futures-sink", "http 1.3.1", - "indexmap 2.9.0", + "indexmap 2.10.0", "slab", "tokio", "tokio-util", @@ -3281,15 +3262,15 @@ version = "0.14.5" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "e5274423e17b7c9fc20b6e7e208532f9b19825d82dfd615708b70edd83df41f1" dependencies = [ - "ahash 0.8.11", + "ahash 0.8.12", "allocator-api2", ] [[package]] name = "hashbrown" -version = "0.15.2" +version = "0.15.5" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "bf151400ff0baff5465007dd2f3e717f3fe502074ca563069ce3a6629d07b289" +checksum = "9229cfe53dfd69f0609a49f65461bd93001ea1ef889cd5529dd176593f5338a1" dependencies = [ "allocator-api2", "equivalent", @@ -3302,7 +3283,7 @@ version = "0.10.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "7382cf6263419f2d8df38c55d7da83da5c18aef87fc7a7fc1fb1e344edfe14c1" dependencies = [ - "hashbrown 0.15.2", + "hashbrown 0.15.5", ] [[package]] @@ -3313,15 +3294,9 @@ checksum = "2304e00983f87ffb38b55b444b5e3b60a884b5d30c0fca7d82fe33449bbe55ea" [[package]] name = "hermit-abi" -version = "0.3.9" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d231dfb89cfffdbc30e7fc41579ed6066ad03abda9e567ccafae602b97ec5024" - -[[package]] -name = "hermit-abi" -version = "0.4.0" +version = "0.5.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "fbf6a919d6cf397374f7dfeeea91d974c7c0a7221d0d0f4f20d859d329e53fcc" +checksum = "fc0fef456e4baa96da950455cd02c081ca953b141298e41db3fc7e36b1da849c" [[package]] name = "hex" @@ -3331,9 +3306,9 @@ checksum = "7f24254aa9a54b5c858eaee2f5bccdb46aaf0e486a595ed5fd8f86ba55232a70" [[package]] name = "hive_metastore" -version = "0.1.0" +version = "0.2.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "35f502759a3b4517dc44d06d8cdaddb942b7930bb81ebf7d645e209cfc7b7e43" +checksum = "a699b5fcbc7f4614c4fcd8154bedcaab8b741e524e5f1b7bf1e9b1dca9917d8a" dependencies = [ "anyhow", "pilota", @@ -3413,12 +3388,12 @@ dependencies = [ [[package]] name = "http-body-util" -version = "0.1.2" +version = "0.1.3" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "793429d76616a256bcb62c2a2ec2bed781c8307e797e2598c50010f2bee2544f" +checksum = "b021d93e26becf5dc7e1b75b1bed1fd93124b374ceb73f43d4d4eafec896a64a" dependencies = [ "bytes", - "futures-util", + "futures-core", "http 1.3.1", "http-body 1.0.1", "pin-project-lite", @@ -3426,9 +3401,9 @@ dependencies = [ [[package]] name = "httparse" -version = "1.10.0" +version = "1.10.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "f2d708df4e7140240a16cd6ab0ab65c972d7433ab77819ea693fde9c43811e2a" +checksum = "6dbf3de79e51f3d586ab4cb9d5c3e2c14aa28ed23d180cf89b4df0454a69cc87" [[package]] name = "httpdate" @@ -3447,9 +3422,9 @@ dependencies = [ [[package]] name = "humantime" -version = "2.1.0" +version = "2.2.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "9a3a5bfb195931eeb336b2a7b4d761daec841b97f947d34394601737a7bba5e4" +checksum = "9b112acc8b3adf4b107a8ec20977da0273a8c386765a3ec0229bd500a1443f9f" [[package]] name = "hyper" @@ -3461,14 +3436,14 @@ dependencies = [ "futures-channel", "futures-core", "futures-util", - "h2 0.3.26", + "h2 0.3.27", "http 0.2.12", "http-body 0.4.6", "httparse", "httpdate", "itoa", "pin-project-lite", - "socket2 0.5.8", + "socket2 0.5.10", "tokio", "tower-service", "tracing", @@ -3477,20 +3452,22 @@ dependencies = [ [[package]] name = "hyper" -version = "1.6.0" +version = "1.7.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "cc2b571658e38e0c01b1fdca3bbbe93c00d3d71693ff2770043f8c29bc7d6f80" +checksum = "eb3aa54a13a0dfe7fbe3a59e0c76093041720fdc77b110cc0fc260fafb4dc51e" dependencies = [ + "atomic-waker", "bytes", "futures-channel", - "futures-util", - "h2 0.4.7", + "futures-core", + "h2 0.4.12", "http 1.3.1", "http-body 1.0.1", "httparse", "httpdate", "itoa", "pin-project-lite", + "pin-utils", "smallvec", "tokio", "want", @@ -3514,21 +3491,20 @@ dependencies = [ [[package]] name = "hyper-rustls" -version = "0.27.5" +version = "0.27.7" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "2d191583f3da1305256f22463b9bb0471acad48a4e534a5218b9963e9c1f59b2" +checksum = "e3c93eb611681b207e1fe55d5a71ecf91572ec8a6705cdb6857f7d8d5242cf58" dependencies = [ - "futures-util", "http 1.3.1", - "hyper 1.6.0", + "hyper 1.7.0", "hyper-util", - "rustls 0.23.23", + "rustls 0.23.31", "rustls-native-certs 0.8.1", "rustls-pki-types", "tokio", - "tokio-rustls 0.26.1", + "tokio-rustls 0.26.2", "tower-service", - "webpki-roots", + "webpki-roots 1.0.2", ] [[package]] @@ -3545,18 +3521,23 @@ dependencies = [ [[package]] name = "hyper-util" -version = "0.1.10" +version = "0.1.16" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "df2dcfbe0677734ab2f3ffa7fa7bfd4706bfdc1ef393f2ee30184aed67e631b4" +checksum = "8d9b05277c7e8da2c93a568989bb6207bef0112e8d17df7a6eda4a3cf143bc5e" dependencies = [ + "base64 0.22.1", "bytes", "futures-channel", + "futures-core", "futures-util", "http 1.3.1", "http-body 1.0.1", - "hyper 1.6.0", + "hyper 1.7.0", + "ipnet", + "libc", + "percent-encoding", "pin-project-lite", - "socket2 0.5.8", + "socket2 0.6.0", "tokio", "tower-service", "tracing", @@ -3564,16 +3545,17 @@ dependencies = [ [[package]] name = "iana-time-zone" -version = "0.1.61" +version = "0.1.63" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "235e081f3925a06703c2d0117ea8b91f042756fd6e7a6e5d901e8ca1a996b220" +checksum = "b0c919e5debc312ad217002b8048a17b7d83f80703865bbfcfebb0458b0b27d8" dependencies = [ "android_system_properties", "core-foundation-sys", "iana-time-zone-haiku", "js-sys", + "log", "wasm-bindgen", - "windows-core 0.52.0", + "windows-core", ] [[package]] @@ -3636,7 +3618,7 @@ dependencies = [ "serde_json", "serde_repr", "serde_with", - "strum 0.27.1", + "strum 0.27.2", "tempfile", "tera", "thrift", @@ -3670,7 +3652,6 @@ dependencies = [ "serde_json", "tokio", "tracing", - "typed-builder 0.20.1", ] [[package]] @@ -3706,6 +3687,7 @@ dependencies = [ "iceberg", "iceberg-catalog-glue", "iceberg-catalog-rest", + "iceberg-catalog-s3tables", "tokio", ] @@ -3736,7 +3718,6 @@ dependencies = [ name = "iceberg-catalog-s3tables" version = "0.6.0" dependencies = [ - "anyhow", "async-trait", "aws-config", "aws-sdk-s3tables", @@ -3744,7 +3725,6 @@ dependencies = [ "iceberg_test_utils", "itertools 0.13.0", "tokio", - "typed-builder 0.20.1", ] [[package]] @@ -3868,21 +3848,22 @@ dependencies = [ [[package]] name = "icu_collections" -version = "1.5.0" +version = "2.0.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "db2fa452206ebee18c4b5c2274dbf1de17008e874b4dc4f0aea9d01ca79e4526" +checksum = "200072f5d0e3614556f94a9930d5dc3e0662a652823904c3a75dc3b0af7fee47" dependencies = [ "displaydoc", + "potential_utf", "yoke", "zerofrom", "zerovec", ] [[package]] -name = "icu_locid" -version = "1.5.0" +name = "icu_locale_core" +version = "2.0.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "13acbb8371917fc971be86fc8057c41a64b521c184808a698c02acc242dbf637" +checksum = "0cde2700ccaed3872079a65fb1a78f6c0a36c91570f28755dda67bc8f7d9f00a" dependencies = [ "displaydoc", "litemap", @@ -3891,31 +3872,11 @@ dependencies = [ "zerovec", ] -[[package]] -name = "icu_locid_transform" -version = "1.5.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "01d11ac35de8e40fdeda00d9e1e9d92525f3f9d887cdd7aa81d727596788b54e" -dependencies = [ - "displaydoc", - "icu_locid", - "icu_locid_transform_data", - "icu_provider", - "tinystr", - "zerovec", -] - -[[package]] -name = "icu_locid_transform_data" -version = "1.5.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "fdc8ff3388f852bede6b579ad4e978ab004f139284d7b28715f773507b946f6e" - [[package]] name = "icu_normalizer" -version = "1.5.0" +version = "2.0.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "19ce3e0da2ec68599d193c93d088142efd7f9c5d6fc9b803774855747dc6a84f" +checksum = "436880e8e18df4d7bbc06d58432329d6458cc84531f7ac5f024e93deadb37979" dependencies = [ "displaydoc", "icu_collections", @@ -3923,67 +3884,54 @@ dependencies = [ "icu_properties", "icu_provider", "smallvec", - "utf16_iter", - "utf8_iter", - "write16", "zerovec", ] [[package]] name = "icu_normalizer_data" -version = "1.5.0" +version = "2.0.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "f8cafbf7aa791e9b22bec55a167906f9e1215fd475cd22adfcf660e03e989516" +checksum = "00210d6893afc98edb752b664b8890f0ef174c8adbb8d0be9710fa66fbbf72d3" [[package]] name = "icu_properties" -version = "1.5.1" +version = "2.0.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "93d6020766cfc6302c15dbbc9c8778c37e62c14427cb7f6e601d849e092aeef5" +checksum = "016c619c1eeb94efb86809b015c58f479963de65bdb6253345c1a1276f22e32b" dependencies = [ "displaydoc", "icu_collections", - "icu_locid_transform", + "icu_locale_core", "icu_properties_data", "icu_provider", - "tinystr", + "potential_utf", + "zerotrie", "zerovec", ] [[package]] name = "icu_properties_data" -version = "1.5.0" +version = "2.0.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "67a8effbc3dd3e4ba1afa8ad918d5684b8868b3b26500753effea8d2eed19569" +checksum = "298459143998310acd25ffe6810ed544932242d3f07083eee1084d83a71bd632" [[package]] name = "icu_provider" -version = "1.5.0" +version = "2.0.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "6ed421c8a8ef78d3e2dbc98a973be2f3770cb42b606e3ab18d6237c4dfde68d9" +checksum = "03c80da27b5f4187909049ee2d72f276f0d9f99a42c306bd0131ecfe04d8e5af" dependencies = [ "displaydoc", - "icu_locid", - "icu_provider_macros", + "icu_locale_core", "stable_deref_trait", "tinystr", "writeable", "yoke", "zerofrom", + "zerotrie", "zerovec", ] -[[package]] -name = "icu_provider_macros" -version = "1.5.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "1ec89e9337638ecdc08744df490b221a7399bf8d164eb52a665454e60e075ad6" -dependencies = [ - "proc-macro2", - "quote", - "syn 2.0.101", -] - [[package]] name = "ident_case" version = "1.0.1" @@ -4003,9 +3951,9 @@ dependencies = [ [[package]] name = "idna_adapter" -version = "1.2.0" +version = "1.2.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "daca1df1c957320b2cf139ac61e7bd64fed304c5040df000a745aa1de3b4ef71" +checksum = "3acae9609540aa318d1bc588455225fb2085b9ed0c4f6bd0d9d5bcd86f1a0344" dependencies = [ "icu_normalizer", "icu_properties", @@ -4040,12 +3988,12 @@ dependencies = [ [[package]] name = "indexmap" -version = "2.9.0" +version = "2.10.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "cea70ddb795996207ad57735b50c5982d8844f38ba9ee5f1aedcfb708a2aa11e" +checksum = "fe4cd85333e22411419a0bcae1297d25e58c9443848b11dc6a86fefe8c78a661" dependencies = [ "equivalent", - "hashbrown 0.15.2", + "hashbrown 0.15.5", "serde", ] @@ -4058,15 +4006,15 @@ dependencies = [ "console", "number_prefix", "portable-atomic", - "unicode-width 0.2.0", + "unicode-width 0.2.1", "web-time", ] [[package]] name = "inout" -version = "0.1.3" +version = "0.1.4" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a0c10553d664a4d0bcff9f4215d0aac67a639cc68ef660840afe309b807bc9f5" +checksum = "879f10e63c20629ecabbb64a8010319738c66a5cd0c29b02d63d272b03751d01" dependencies = [ "block-padding", "generic-array", @@ -4090,11 +4038,11 @@ dependencies = [ [[package]] name = "io-uring" -version = "0.7.8" +version = "0.7.9" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "b86e202f00093dcba4275d4636b93ef9dd75d025ae560d2521b45ea28ab49013" +checksum = "d93587f37623a1a17d94ef2bc9ada592f5465fe7732084ab7beefabe5c77c0c4" dependencies = [ - "bitflags 2.8.0", + "bitflags 2.9.2", "cfg-if", "libc", ] @@ -4105,6 +4053,16 @@ version = "2.11.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "469fb0b9cefa57e3ef31275ee7cacb78f2fdca44e4765491884a2b119d4eb130" +[[package]] +name = "iri-string" +version = "0.7.8" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "dbc5ebe9c3a1a7a5127f920a418f7585e9e758e911d0466ed004f393b0e380b2" +dependencies = [ + "memchr", + "serde", +] + [[package]] name = "is_terminal_polyfill" version = "1.70.1" @@ -4140,15 +4098,15 @@ dependencies = [ [[package]] name = "itoa" -version = "1.0.14" +version = "1.0.15" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d75a2a4b1b190afb6f5425f10f6a8f959d2ea0b9c2b1d79553551850539e4674" +checksum = "4a5f13b858c8d314ee3e8f639011f7ccefe71f97f96e50151fb991f267928e2c" [[package]] name = "jiff" -version = "0.2.6" +version = "0.2.15" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "1f33145a5cbea837164362c7bd596106eb7c5198f97d1ba6f6ebb3223952e488" +checksum = "be1f93b8b1eb69c77f24bbb0afdf66f54b632ee39af40ca21c4365a1d7347e49" dependencies = [ "jiff-static", "log", @@ -4159,21 +4117,22 @@ dependencies = [ [[package]] name = "jiff-static" -version = "0.2.6" +version = "0.2.15" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "43ce13c40ec6956157a3635d97a1ee2df323b263f09ea14165131289cb0f5c19" +checksum = "03343451ff899767262ec32146f6d559dd759fdadf42ff0e227c7c48f72594b4" dependencies = [ "proc-macro2", "quote", - "syn 2.0.101", + "syn 2.0.106", ] [[package]] name = "jobserver" -version = "0.1.32" +version = "0.1.33" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "48d1dbcbbeb6a7fec7e059840aa538bd62aaccf972c7346c4d9d2059312853d0" +checksum = "38f262f097c174adebe41eb73d66ae9c06b2844fb0da69969647bbddd9b0538a" dependencies = [ + "getrandom 0.3.3", "libc", ] @@ -4292,9 +4251,9 @@ dependencies = [ [[package]] name = "libc" -version = "0.2.174" +version = "0.2.175" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "1171693293099992e19cddea4e8b849964e9846f4acee11b3948bcc337be8776" +checksum = "6a82ae493e598baaea5209805c49bbf2ea7de956d50d7da0da1164f9c6d28543" [[package]] name = "libflate" @@ -4322,25 +4281,25 @@ dependencies = [ [[package]] name = "libloading" -version = "0.8.6" +version = "0.8.8" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "fc2f4eb4bc735547cfed7c0a4922cbd04a4655978c09b54f1f7b228750664c34" +checksum = "07033963ba89ebaf1584d767badaa2e8fcec21aedea6b8c0346d487d49c28667" dependencies = [ "cfg-if", - "windows-targets 0.52.6", + "windows-targets 0.53.3", ] [[package]] name = "libm" -version = "0.2.11" +version = "0.2.15" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "8355be11b20d696c8f18f6cc018c4e372165b1fa8126cef092399c9951984ffa" +checksum = "f9fbbcab51052fe104eb5e5d351cf728d30a5be1fe14d9be8a3b097481fb97de" [[package]] name = "libmimalloc-sys" -version = "0.1.42" +version = "0.1.43" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "ec9d6fac27761dabcd4ee73571cdb06b7022dc99089acbe5435691edffaac0f4" +checksum = "bf88cd67e9de251c1781dbe2f641a1a3ad66eaae831b8a2c38fbdc5ddae16d4d" dependencies = [ "cc", "libc", @@ -4348,12 +4307,13 @@ dependencies = [ [[package]] name = "libredox" -version = "0.1.3" +version = "0.1.9" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c0ff37bd590ca25063e35af745c343cb7a0271906fb7b37e4813e8f79f00268d" +checksum = "391290121bad3d37fbddad76d8f5d1c1c314cfc646d143d7e07a3086ddff0ce3" dependencies = [ - "bitflags 2.8.0", + "bitflags 2.9.2", "libc", + "redox_syscall", ] [[package]] @@ -4393,9 +4353,9 @@ dependencies = [ [[package]] name = "libz-rs-sys" -version = "0.5.0" +version = "0.5.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "6489ca9bd760fe9642d7644e827b0c9add07df89857b0416ee15c1cc1a3b8c5a" +checksum = "172a788537a2221661b480fee8dc5f96c580eb34fa88764d3205dc356c7e4221" dependencies = [ "zlib-rs", ] @@ -4408,9 +4368,9 @@ checksum = "0717cef1bc8b636c6e1c1bbdefc09e6322da8a9321966e8928ef80d20f7f770f" [[package]] name = "linkedbytes" -version = "0.1.8" +version = "0.1.16" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "487640b2e3f554987c1345fc71c3eda34d6d750fba63a39147205ccc93f920cd" +checksum = "80dd357febf29a3b1e37d0ff3509b035077769e5f4af161ff32edc3e97a78548" dependencies = [ "bytes", "faststr", @@ -4425,21 +4385,21 @@ checksum = "d26c52dbd32dccf2d10cac7725f8eae5296885fb5703b261f7d0a0739ec807ab" [[package]] name = "linux-raw-sys" -version = "0.9.2" +version = "0.9.4" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "6db9c683daf087dc577b7506e9695b3d556a9f3849903fa28186283afd6809e9" +checksum = "cd945864f07fe9f5371a27ad7b52a172b4b499999f1d97574c9fa68373937e12" [[package]] name = "litemap" -version = "0.7.4" +version = "0.8.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "4ee93343901ab17bd981295f2cf0026d4ad018c7c31ba84549a4ddbb47a45104" +checksum = "241eaef5fd12c88705a01fc1066c48c4b36e0dd4377dcdc7ec3942cea7a69956" [[package]] name = "lock_api" -version = "0.4.12" +version = "0.4.13" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "07af8b9cdd281b7915f413fa73f29ebd5d55d0d3f0155584dade1ff18cea1b17" +checksum = "96936507f153605bddfcda068dd804796c84324ed2510809e5b2a624c81da765" dependencies = [ "autocfg", "scopeguard", @@ -4447,9 +4407,9 @@ dependencies = [ [[package]] name = "log" -version = "0.4.25" +version = "0.4.27" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "04cbf5b083de1c7e0222a7a51dbfdba1cbe1c6ab0b15e29fff3f6c077fd9cd9f" +checksum = "13dc2df351e3202783a1fe0d44375f7295ffb4049267b0f3018346dc122a1d94" dependencies = [ "value-bag", ] @@ -4467,13 +4427,19 @@ dependencies = [ "tracing-subscriber", ] +[[package]] +name = "lru-slab" +version = "0.1.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "112b39cec0b298b6c1999fee3e31427f74f676e4cb9879ed1a121b43661a4154" + [[package]] name = "lz4_flex" -version = "0.11.3" +version = "0.11.5" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "75761162ae2b0e580d7e7c390558127e5f01b4194debd6221fd8c207fc80e3f5" +checksum = "08ab2867e3eeeca90e844d1940eab391c9dc5228783db2ed999acbc0a9ed375a" dependencies = [ - "twox-hash 1.6.3", + "twox-hash", ] [[package]] @@ -4514,9 +4480,9 @@ dependencies = [ [[package]] name = "memchr" -version = "2.7.4" +version = "2.7.5" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "78ca9ab1a0babb1e7d5695e3530886289c18cf2f87ec19a575a0abdce112e3a3" +checksum = "32a282da65faaf38286cf3be983213fcf1d2e2a58700e808f83f4ea9a4804bc0" [[package]] name = "memoffset" @@ -4533,7 +4499,7 @@ version = "0.7.14" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "cdc67a1d6ef0340a7f5152b9fd34ad7477b4d518920f7557267f1fc6e5a62641" dependencies = [ - "ahash 0.8.11", + "ahash 0.8.12", "faststr", "paste", "rustc-hash 2.1.1", @@ -4542,9 +4508,9 @@ dependencies = [ [[package]] name = "mimalloc" -version = "0.1.46" +version = "0.1.47" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "995942f432bbb4822a7e9c3faa87a695185b0d09273ba85f097b54f4e458f2af" +checksum = "b1791cbe101e95af5764f06f20f6760521f7158f69dbf9d6baf941ee1bf6bc40" dependencies = [ "libmimalloc-sys", ] @@ -4563,22 +4529,22 @@ checksum = "68354c5c6bd36d73ff3feceb05efa59b6acb7626617f4962be322a825e61f79a" [[package]] name = "miniz_oxide" -version = "0.8.8" +version = "0.8.9" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "3be647b768db090acb35d5ec5db2b0e1f1de11133ca123b9eacf5137868f892a" +checksum = "1fa76a2c86f704bdb222d66965fb3d63269ce38518b83cb0575fca855ebb6316" dependencies = [ "adler2", ] [[package]] name = "mio" -version = "1.0.3" +version = "1.0.4" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "2886843bf800fba2e3377cff24abf6379b4c4d5c6681eaf9ea5b0d15090450bd" +checksum = "78bed444cc8a2160f01cbcf811ef18cac863ad68ae8ca62092e8db51d51c761c" dependencies = [ "libc", - "wasi 0.11.0+wasi-snapshot-preview1", - "windows-sys 0.52.0", + "wasi 0.11.1+wasi-snapshot-preview1", + "windows-sys 0.59.0", ] [[package]] @@ -4604,14 +4570,14 @@ dependencies = [ "cfg-if", "proc-macro2", "quote", - "syn 2.0.101", + "syn 2.0.106", ] [[package]] name = "mockito" -version = "1.6.1" +version = "1.7.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "652cd6d169a36eaf9d1e6bce1a221130439a966d7f27858af66a33a66e9c4ee2" +checksum = "7760e0e418d9b7e5777c0374009ca4c93861b9066f18cb334a20ce50ab63aa48" dependencies = [ "assert-json-diff", "bytes", @@ -4620,10 +4586,10 @@ dependencies = [ "http 1.3.1", "http-body 1.0.1", "http-body-util", - "hyper 1.6.0", + "hyper 1.7.0", "hyper-util", "log", - "rand 0.8.5", + "rand 0.9.2", "regex", "serde_json", "serde_urlencoded", @@ -4641,7 +4607,7 @@ dependencies = [ "crossbeam-channel", "crossbeam-epoch", "crossbeam-utils", - "event-listener 5.4.0", + "event-listener 5.4.1", "futures-util", "loom", "parking_lot", @@ -4673,7 +4639,7 @@ checksum = "b40e46c845ac234bcba19db7ab252bc2778cbadd516a466d2f12b1580852d136" dependencies = [ "proc-macro2", "quote", - "syn 2.0.101", + "syn 2.0.106", ] [[package]] @@ -4684,22 +4650,22 @@ checksum = "1d87ecb2933e8aeadb3e3a02b828fed80a7528047e68b4f424523a0981a3a084" [[package]] name = "munge" -version = "0.4.1" +version = "0.4.6" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "64142d38c84badf60abf06ff9bd80ad2174306a5b11bd4706535090a30a419df" +checksum = "d7feb0b48aa0a25f9fe0899482c6e1379ee7a11b24a53073eacdecb9adb6dc60" dependencies = [ "munge_macro", ] [[package]] name = "munge_macro" -version = "0.4.1" +version = "0.4.6" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "1bb5c1d8184f13f7d0ccbeeca0def2f9a181bce2624302793005f5ca8aa62e5e" +checksum = "f2e3795a5d2da581a8b252fec6022eee01aea10161a4d1bf237d4cbe47f7e988" dependencies = [ "proc-macro2", "quote", - "syn 2.0.101", + "syn 2.0.106", ] [[package]] @@ -4729,7 +4695,7 @@ version = "0.29.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "71e2746dc3a24dd78b3cfcb7be93368c6de9963d30f43a6a73998a9cf4b17b46" dependencies = [ - "bitflags 2.8.0", + "bitflags 2.9.2", "cfg-if", "cfg_aliases", "libc", @@ -4856,33 +4822,34 @@ dependencies = [ [[package]] name = "num_cpus" -version = "1.16.0" +version = "1.17.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "4161fcb6d602d4d2081af7c3a45852d875a03dd337a6bfdd6e06407b61342a43" +checksum = "91df4bbde75afed763b708b7eee1e8e7651e02d97f6d5dd763e89367e957b23b" dependencies = [ - "hermit-abi 0.3.9", + "hermit-abi", "libc", ] [[package]] name = "num_enum" -version = "0.7.3" +version = "0.7.4" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "4e613fc340b2220f734a8595782c551f1250e969d87d3be1ae0579e8d4065179" +checksum = "a973b4e44ce6cad84ce69d797acf9a044532e4184c4f267913d1b546a0727b7a" dependencies = [ "num_enum_derive", + "rustversion", ] [[package]] name = "num_enum_derive" -version = "0.7.3" +version = "0.7.4" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "af1844ef2428cc3e1cb900be36181049ef3d3193c63e43026cfe202983b27a56" +checksum = "77e878c846a8abae00dd069496dbe8751b16ac1c3d6bd2a7283a938e8228f90d" dependencies = [ "proc-macro-crate", "proc-macro2", "quote", - "syn 2.0.101", + "syn 2.0.106", ] [[package]] @@ -4902,9 +4869,9 @@ dependencies = [ [[package]] name = "object_store" -version = "0.12.0" +version = "0.12.3" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "e9ce831b09395f933addbc56d894d889e4b226eba304d4e7adbab591e26daf1e" +checksum = "efc4f07659e11cd45a341cd24d71e683e3be65d9ff1f8150061678fe60437496" dependencies = [ "async-trait", "base64 0.22.1", @@ -4915,31 +4882,39 @@ dependencies = [ "http 1.3.1", "http-body-util", "humantime", - "hyper 1.6.0", + "hyper 1.7.0", "itertools 0.14.0", "md-5", "parking_lot", "percent-encoding", - "quick-xml", - "rand 0.8.5", + "quick-xml 0.38.1", + "rand 0.9.2", "reqwest", "ring", "rustls-pemfile 2.2.0", "serde", "serde_json", "serde_urlencoded", - "thiserror 2.0.12", + "thiserror 2.0.15", "tokio", "tracing", "url", "walkdir", + "wasm-bindgen-futures", + "web-time", ] [[package]] name = "once_cell" -version = "1.21.1" +version = "1.21.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "42f5e15c9953c5e4ccceeb2e7382a716482c34515315f7b03532b8b4e8393d2d" + +[[package]] +name = "once_cell_polyfill" +version = "1.70.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d75b0bedcc4fe52caa0e03d9f1151a323e4aa5e2d78ba3580400cd3c9e2bc4bc" +checksum = "a4895175b425cb1f87721b59f0f286c2092bd4af812243672510e1ac53e2e0ad" [[package]] name = "opendal" @@ -4954,13 +4929,13 @@ dependencies = [ "chrono", "crc32c", "futures", - "getrandom 0.2.15", + "getrandom 0.2.16", "http 1.3.1", "http-body 1.0.1", "log", "md-5", "percent-encoding", - "quick-xml", + "quick-xml 0.37.5", "reqsign", "reqwest", "serde", @@ -5025,9 +5000,9 @@ checksum = "b15813163c1d831bf4a13c3610c05c0d03b39feb07f7e09fa234dac9b15aaf39" [[package]] name = "owo-colors" -version = "4.2.0" +version = "4.2.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "1036865bb9422d3300cf723f657c2851d0e9ab12567854b1f4eba3d77decf564" +checksum = "48dd4f4a2c8405440fd0462561f0e5806bd0f77e86f51c761481bdd4018b545e" [[package]] name = "parking" @@ -5037,9 +5012,9 @@ checksum = "f38d5652c16fde515bb1ecef450ab0f6a219d619a7274976324d5e377f7dceba" [[package]] name = "parking_lot" -version = "0.12.3" +version = "0.12.4" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "f1bf18183cf54e8d6059647fc3063646a1801cf30896933ec2311622cc4b9a27" +checksum = "70d58bf43669b5795d1576d0641cfb6fbb2057bf629506267a92807158584a13" dependencies = [ "lock_api", "parking_lot_core", @@ -5047,9 +5022,9 @@ dependencies = [ [[package]] name = "parking_lot_core" -version = "0.9.10" +version = "0.9.11" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "1e401f977ab385c9e4e3ab30627d6f26d00e2c73eef317493c4ec6d468726cf8" +checksum = "bc838d2a56b5b1a6c25f55575dfc605fabb63bb2365f6c2353ef9159aa69e4a5" dependencies = [ "cfg-if", "libc", @@ -5064,7 +5039,7 @@ version = "55.2.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "b17da4150748086bd43352bc77372efa9b6e3dbd06a04831d2a98c041c225cfa" dependencies = [ - "ahash 0.8.11", + "ahash 0.8.12", "arrow-array", "arrow-buffer", "arrow-cast", @@ -5079,7 +5054,7 @@ dependencies = [ "flate2", "futures", "half", - "hashbrown 0.15.2", + "hashbrown 0.15.5", "lz4_flex", "num", "num-bigint", @@ -5090,7 +5065,7 @@ dependencies = [ "snap", "thrift", "tokio", - "twox-hash 2.1.0", + "twox-hash", "zstd", ] @@ -5121,9 +5096,9 @@ dependencies = [ [[package]] name = "pem" -version = "3.0.4" +version = "3.0.5" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "8e459365e590736a54c3fa561947c84837534b8e9af6fc5bf781307e82658fae" +checksum = "38af38e8470ac9dee3ce1bae1af9c1671fffc44ddfd8bd1d0a3445bf349a8ef3" dependencies = [ "base64 0.22.1", "serde", @@ -5146,20 +5121,20 @@ checksum = "e3148f5046208a5d56bcfc03053e3ca6334e51da8dfb19b6cdc8b306fae3283e" [[package]] name = "pest" -version = "2.7.15" +version = "2.8.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "8b7cafe60d6cf8e62e1b9b2ea516a089c008945bb5a275416789e7db0bc199dc" +checksum = "1db05f56d34358a8b1066f67cbb203ee3e7ed2ba674a6263a1d5ec6db2204323" dependencies = [ "memchr", - "thiserror 2.0.12", + "thiserror 2.0.15", "ucd-trie", ] [[package]] name = "pest_derive" -version = "2.7.15" +version = "2.8.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "816518421cfc6887a0d62bf441b6ffb4536fcc926395a69e1a85852d4363f57e" +checksum = "bb056d9e8ea77922845ec74a1c4e8fb17e7c218cc4fc11a15c5d25e189aa40bc" dependencies = [ "pest", "pest_generator", @@ -5167,24 +5142,23 @@ dependencies = [ [[package]] name = "pest_generator" -version = "2.7.15" +version = "2.8.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "7d1396fd3a870fc7838768d171b4616d5c91f6cc25e377b673d714567d99377b" +checksum = "87e404e638f781eb3202dc82db6760c8ae8a1eeef7fb3fa8264b2ef280504966" dependencies = [ "pest", "pest_meta", "proc-macro2", "quote", - "syn 2.0.101", + "syn 2.0.106", ] [[package]] name = "pest_meta" -version = "2.7.15" +version = "2.8.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "e1e58089ea25d717bfd31fb534e4f3afcc2cc569c70de3e239778991ea3b7dea" +checksum = "edd1101f170f5903fde0914f899bb503d9ff5271d7ba76bbb70bea63690cc0d5" dependencies = [ - "once_cell", "pest", "sha2", ] @@ -5196,7 +5170,7 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "b4c5cc86750666a3ed20bdaf5ca2a0344f9c67674cae0515bec2da16fbaa47db" dependencies = [ "fixedbitset 0.4.2", - "indexmap 2.9.0", + "indexmap 2.10.0", ] [[package]] @@ -5206,7 +5180,7 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "3672b37090dbd86368a4145bc067582552b29c27377cad4e0a306c97f9bd7772" dependencies = [ "fixedbitset 0.5.7", - "indexmap 2.9.0", + "indexmap 2.10.0", ] [[package]] @@ -5215,7 +5189,16 @@ version = "0.11.3" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "1fd6780a80ae0c52cc120a26a1a42c1ae51b247a253e4e06113d23d2c2edd078" dependencies = [ - "phf_shared", + "phf_shared 0.11.3", +] + +[[package]] +name = "phf" +version = "0.12.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "913273894cec178f401a31ec4b656318d95473527be05c0752cc41cdc32be8b7" +dependencies = [ + "phf_shared 0.12.1", ] [[package]] @@ -5225,7 +5208,7 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "aef8048c789fa5e851558d709946d6d79a8ff88c0440c587967f8e94bfb1216a" dependencies = [ "phf_generator", - "phf_shared", + "phf_shared 0.11.3", ] [[package]] @@ -5234,7 +5217,7 @@ version = "0.11.3" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "3c80231409c20246a13fddb31776fb942c38553c51e871f8cbd687a4cfb5843d" dependencies = [ - "phf_shared", + "phf_shared 0.11.3", "rand 0.8.5", ] @@ -5248,16 +5231,24 @@ dependencies = [ ] [[package]] -name = "pilota" -version = "0.11.8" +name = "phf_shared" +version = "0.12.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d18f400b02a75df232f315ad31c6d15b1b649237151ebe8b242eee1657330137" +checksum = "06005508882fb681fd97892ecff4b7fd0fee13ef1aa569f8695dae7ab9099981" dependencies = [ - "ahash 0.8.11", - "anyhow", - "async-recursion", - "bytes", - "derivative", + "siphasher", +] + +[[package]] +name = "pilota" +version = "0.11.10" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "520e32b295867834617b8a352b6fd4837e578f328f4e48813db4904ebc0d70b2" +dependencies = [ + "ahash 0.8.12", + "anyhow", + "async-recursion", + "bytes", "faststr", "integer-encoding 4.0.2", "lazy_static", @@ -5272,22 +5263,22 @@ dependencies = [ [[package]] name = "pin-project" -version = "1.1.9" +version = "1.1.10" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "dfe2e71e1471fe07709406bf725f710b02927c9c54b2b5b2ec0e8087d97c327d" +checksum = "677f1add503faace112b9f1373e43e9e054bfdd22ff1a63c1bc485eaec6a6a8a" dependencies = [ "pin-project-internal", ] [[package]] name = "pin-project-internal" -version = "1.1.9" +version = "1.1.10" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "f6e859e6e5bd50440ab63c47e3ebabc90f26251f7c73c3d3e837b74a1cc3fa67" +checksum = "6e918e4ff8c4549eb882f14b3a4bc8c8bc93de829416eacf579f1207a8fbf861" dependencies = [ "proc-macro2", "quote", - "syn 2.0.101", + "syn 2.0.106", ] [[package]] @@ -5353,23 +5344,22 @@ dependencies = [ [[package]] name = "pkg-config" -version = "0.3.31" +version = "0.3.32" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "953ec861398dccce10c670dfeaf3ec4911ca479e9c02154b3a215178c5f566f2" +checksum = "7edddbd0b52d732b21ad9a5fab5c704c14cd949e5e9a1ec5929a24fded1b904c" [[package]] name = "polling" -version = "3.7.4" +version = "3.10.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a604568c3202727d1507653cb121dbd627a58684eb09a820fd746bee38b4442f" +checksum = "b5bd19146350fe804f7cb2669c851c03d69da628803dab0d98018142aaa5d829" dependencies = [ "cfg-if", "concurrent-queue", - "hermit-abi 0.4.0", + "hermit-abi", "pin-project-lite", - "rustix 0.38.44", - "tracing", - "windows-sys 0.59.0", + "rustix 1.0.8", + "windows-sys 0.60.2", ] [[package]] @@ -5380,9 +5370,9 @@ checksum = "325a6d2ac5dee293c3b2612d4993b98aec1dff096b0a2dae70ed7d95784a05da" [[package]] name = "portable-atomic" -version = "1.10.0" +version = "1.11.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "280dc24453071f1b63954171985a0b0d30058d287960968b9b2aca264c8d4ee6" +checksum = "f84267b20a16ea918e43c6a88433c2d54fa145c92a811b5b047ccbe153674483" [[package]] name = "portable-atomic-util" @@ -5406,7 +5396,7 @@ dependencies = [ "hmac", "md-5", "memchr", - "rand 0.9.0", + "rand 0.9.2", "sha2", "stringprep", ] @@ -5422,6 +5412,15 @@ dependencies = [ "postgres-protocol", ] +[[package]] +name = "potential_utf" +version = "0.1.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "e5a7c30837279ca13e7c867e9e40053bc68740f988cb07f7ca6df43cc734b585" +dependencies = [ + "zerovec", +] + [[package]] name = "powerfmt" version = "0.2.0" @@ -5430,11 +5429,11 @@ checksum = "439ee305def115ba05938db6eb1644ff94165c5ab5e9420d1c1bcedbba909391" [[package]] name = "ppv-lite86" -version = "0.2.20" +version = "0.2.21" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "77957b295656769bb8ad2b6a6b09d897d94f05c41b069aede1fcdaa675eaea04" +checksum = "85eae3c4ed2f50dcfe72643da4befc30deadb458a9b590d720cde2f2b1e97da9" dependencies = [ - "zerocopy 0.7.35", + "zerocopy", ] [[package]] @@ -5475,28 +5474,28 @@ dependencies = [ [[package]] name = "prettyplease" -version = "0.2.32" +version = "0.2.37" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "664ec5419c51e34154eec046ebcba56312d5a2fc3b09a06da188e1ad21afadf6" +checksum = "479ca8adacdd7ce8f1fb39ce9ecccbfe93a3f1344b3d0d97f20bc0196208f62b" dependencies = [ "proc-macro2", - "syn 2.0.101", + "syn 2.0.106", ] [[package]] name = "proc-macro-crate" -version = "3.2.0" +version = "3.3.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "8ecf48c7ca261d60b74ab1a7b20da18bede46776b2e55535cb958eb595c5fa7b" +checksum = "edce586971a4dfaa28950c6f18ed55e0406c1ab88bbce2c6f6293a7aaba73d35" dependencies = [ "toml_edit", ] [[package]] name = "proc-macro2" -version = "1.0.93" +version = "1.0.101" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "60946a68e5f9d28b0dc1c21bb8a97ee7d018a8b322fa57838ba31cc878e22d99" +checksum = "89ae43fd86e4158d6db51ad8e2b80f313af9cc74f5c0e03ccb87de09998732de" dependencies = [ "unicode-ident", ] @@ -5528,7 +5527,7 @@ dependencies = [ "prost", "prost-types", "regex", - "syn 2.0.101", + "syn 2.0.106", "tempfile", ] @@ -5542,7 +5541,7 @@ dependencies = [ "itertools 0.12.1", "proc-macro2", "quote", - "syn 2.0.101", + "syn 2.0.106", ] [[package]] @@ -5556,9 +5555,9 @@ dependencies = [ [[package]] name = "psm" -version = "0.1.25" +version = "0.1.26" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "f58e5423e24c18cc840e1c98370b3993c6649cd1678b4d24318bcf0a083cbe88" +checksum = "6e944464ec8536cd1beb0bbfd96987eb5e3b72f2ecdafdc5c769a37f1fa2ae1f" dependencies = [ "cc", ] @@ -5600,7 +5599,7 @@ checksum = "ca414edb151b4c8d125c12566ab0d74dc9cdba36fb80eb7b848c15f495fd32d1" dependencies = [ "proc-macro2", "quote", - "syn 2.0.101", + "syn 2.0.106", ] [[package]] @@ -5611,9 +5610,19 @@ checksum = "5a651516ddc9168ebd67b24afd085a718be02f8858fe406591b013d101ce2f40" [[package]] name = "quick-xml" -version = "0.37.4" +version = "0.37.5" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "331e97a1af0bf59823e6eadffe373d7b27f485be8748f71471c662c1f269b7fb" +dependencies = [ + "memchr", + "serde", +] + +[[package]] +name = "quick-xml" +version = "0.38.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a4ce8c88de324ff838700f36fb6ab86c96df0e3c4ab6ef3a9b2044465cce1369" +checksum = "9845d9dccf565065824e69f9f235fafba1587031eda353c1f1561cd6a6be78f4" dependencies = [ "memchr", "serde", @@ -5621,37 +5630,40 @@ dependencies = [ [[package]] name = "quinn" -version = "0.11.6" +version = "0.11.8" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "62e96808277ec6f97351a2380e6c25114bc9e67037775464979f3037c92d05ef" +checksum = "626214629cda6781b6dc1d316ba307189c85ba657213ce642d9c77670f8202c8" dependencies = [ "bytes", + "cfg_aliases", "pin-project-lite", "quinn-proto", "quinn-udp", "rustc-hash 2.1.1", - "rustls 0.23.23", - "socket2 0.5.8", - "thiserror 2.0.12", + "rustls 0.23.31", + "socket2 0.5.10", + "thiserror 2.0.15", "tokio", "tracing", + "web-time", ] [[package]] name = "quinn-proto" -version = "0.11.9" +version = "0.11.12" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a2fe5ef3495d7d2e377ff17b1a8ce2ee2ec2a18cde8b6ad6619d65d0701c135d" +checksum = "49df843a9161c85bb8aae55f101bc0bac8bcafd637a620d9122fd7e0b2f7422e" dependencies = [ "bytes", - "getrandom 0.2.15", - "rand 0.8.5", + "getrandom 0.3.3", + "lru-slab", + "rand 0.9.2", "ring", "rustc-hash 2.1.1", - "rustls 0.23.23", + "rustls 0.23.31", "rustls-pki-types", "slab", - "thiserror 2.0.12", + "thiserror 2.0.15", "tinyvec", "tracing", "web-time", @@ -5659,14 +5671,14 @@ dependencies = [ [[package]] name = "quinn-udp" -version = "0.5.10" +version = "0.5.13" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "e46f3055866785f6b92bc6164b76be02ca8f2eb4b002c0354b28cf4c119e5944" +checksum = "fcebb1209ee276352ef14ff8732e24cc2b02bbac986cd74a4c81bcb2f9881970" dependencies = [ "cfg_aliases", "libc", "once_cell", - "socket2 0.5.8", + "socket2 0.5.10", "tracing", "windows-sys 0.59.0", ] @@ -5680,6 +5692,12 @@ dependencies = [ "proc-macro2", ] +[[package]] +name = "r-efi" +version = "5.3.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "69cdb34c158ceb288df11e18b4bd39de994f6657d83847bdffdbd7f346754b0f" + [[package]] name = "radium" version = "0.7.0" @@ -5719,13 +5737,12 @@ dependencies = [ [[package]] name = "rand" -version = "0.9.0" +version = "0.9.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "3779b94aeb87e8bd4e834cee3650289ee9e0d5677f976ecdb6d219e5f4f6cd94" +checksum = "6db2770f06117d490610c7488547d543617b21bfa07796d7a12f6f1bd53850d1" dependencies = [ "rand_chacha 0.9.0", - "rand_core 0.9.1", - "zerocopy 0.8.18", + "rand_core 0.9.3", ] [[package]] @@ -5745,7 +5762,7 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "d3022b5f1df60f26e1ffddd6c66e8aa15de382ae63b3a0c1bfc0e4d3e3f325cb" dependencies = [ "ppv-lite86", - "rand_core 0.9.1", + "rand_core 0.9.3", ] [[package]] @@ -5754,18 +5771,17 @@ version = "0.6.4" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "ec0be4795e2f6a28069bec0b5ff3e2ac9bafc99e6a9a7dc3547996c5c816922c" dependencies = [ - "getrandom 0.2.15", + "getrandom 0.2.16", "serde", ] [[package]] name = "rand_core" -version = "0.9.1" +version = "0.9.3" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a88e0da7a2c97baa202165137c158d0a2e824ac465d13d81046727b34cb247d3" +checksum = "99d9a13982dcf210057a8a78572b2217b667c3beacbf3a0d8b454f6f82837d38" dependencies = [ - "getrandom 0.3.1", - "zerocopy 0.8.18", + "getrandom 0.3.3", ] [[package]] @@ -5785,47 +5801,47 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "76009fbe0614077fc1a2ce255e3a1881a2e3a3527097d5dc6d8212c585e7e38b" dependencies = [ "quote", - "syn 2.0.101", + "syn 2.0.106", ] [[package]] name = "redox_syscall" -version = "0.5.8" +version = "0.5.17" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "03a862b389f93e68874fbf580b9de08dd02facb9a788ebadaf4a3fd33cf58834" +checksum = "5407465600fb0548f1442edf71dd20683c6ed326200ace4b1ef0763521bb3b77" dependencies = [ - "bitflags 2.8.0", + "bitflags 2.9.2", ] [[package]] name = "redox_users" -version = "0.5.0" +version = "0.5.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "dd6f9d3d47bdd2ad6945c5015a226ec6155d0bcdfd8f7cd29f86b71f8de99d2b" +checksum = "a4e608c6638b9c18977b00b475ac1f28d14e84b27d8d42f70e0bf1e3dec127ac" dependencies = [ - "getrandom 0.2.15", + "getrandom 0.2.16", "libredox", - "thiserror 2.0.12", + "thiserror 2.0.15", ] [[package]] name = "ref-cast" -version = "1.0.23" +version = "1.0.24" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "ccf0a6f84d5f1d581da8b41b47ec8600871962f2a528115b542b362d4b744931" +checksum = "4a0ae411dbe946a674d89546582cea4ba2bb8defac896622d6496f14c23ba5cf" dependencies = [ "ref-cast-impl", ] [[package]] name = "ref-cast-impl" -version = "1.0.23" +version = "1.0.24" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "bcc303e793d3734489387d205e9b186fac9c6cfacedd98cbb2e8a5943595f3e6" +checksum = "1165225c21bff1f3bbce98f5a1f889949bc902d3575308cc7b0de30b4f6d27c7" dependencies = [ "proc-macro2", "quote", - "syn 2.0.101", + "syn 2.0.106", ] [[package]] @@ -5895,16 +5911,16 @@ checksum = "a35e8a6bf28cd121053a66aa2e6a2e3eaffad4a60012179f0e864aa5ffeff215" [[package]] name = "reqsign" -version = "0.16.3" +version = "0.16.5" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "9323c0afb30e54f793f4705b10c890395bccc87c6e6ea62c4e7e82d09a380dc6" +checksum = "43451dbf3590a7590684c25fb8d12ecdcc90ed3ac123433e500447c7d77ed701" dependencies = [ "anyhow", "async-trait", "base64 0.22.1", "chrono", "form_urlencoded", - "getrandom 0.2.15", + "getrandom 0.2.16", "hex", "hmac", "home", @@ -5913,7 +5929,7 @@ dependencies = [ "log", "once_cell", "percent-encoding", - "quick-xml", + "quick-xml 0.37.5", "rand 0.8.5", "reqwest", "rsa", @@ -5922,64 +5938,61 @@ dependencies = [ "serde_json", "sha1", "sha2", + "tokio", ] [[package]] name = "reqwest" -version = "0.12.12" +version = "0.12.23" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "43e734407157c3c2034e0258f5e4473ddb361b1e85f95a66690d67264d7cd1da" +checksum = "d429f34c8092b2d42c7c93cec323bb4adeb7c67698f70839adec842ec10c7ceb" dependencies = [ "base64 0.22.1", "bytes", "futures-core", "futures-util", - "h2 0.4.7", + "h2 0.4.12", "http 1.3.1", "http-body 1.0.1", "http-body-util", - "hyper 1.6.0", - "hyper-rustls 0.27.5", + "hyper 1.7.0", + "hyper-rustls 0.27.7", "hyper-util", - "ipnet", "js-sys", "log", - "mime", - "once_cell", "percent-encoding", "pin-project-lite", "quinn", - "rustls 0.23.23", + "rustls 0.23.31", "rustls-native-certs 0.8.1", - "rustls-pemfile 2.2.0", "rustls-pki-types", "serde", "serde_json", "serde_urlencoded", "sync_wrapper 1.0.2", "tokio", - "tokio-rustls 0.26.1", + "tokio-rustls 0.26.2", "tokio-util", "tower 0.5.2", + "tower-http", "tower-service", "url", "wasm-bindgen", "wasm-bindgen-futures", "wasm-streams", "web-sys", - "webpki-roots", - "windows-registry", + "webpki-roots 1.0.2", ] [[package]] name = "ring" -version = "0.17.13" +version = "0.17.14" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "70ac5d832aa16abd7d1def883a8545280c20a60f523a370aa3a9617c2b8550ee" +checksum = "a4689e6c2294d81e88dc6261c768b63bc4fcdb852be6d1352498b114f61383b7" dependencies = [ "cc", "cfg-if", - "getrandom 0.2.15", + "getrandom 0.2.16", "libc", "untrusted", "windows-sys 0.52.0", @@ -6005,18 +6018,18 @@ dependencies = [ [[package]] name = "rkyv" -version = "0.8.10" +version = "0.8.11" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "1e147371c75553e1e2fcdb483944a8540b8438c31426279553b9a8182a9b7b65" +checksum = "19f5c3e5da784cd8c69d32cdc84673f3204536ca56e1fa01be31a74b92c932ac" dependencies = [ "bytes", - "hashbrown 0.15.2", - "indexmap 2.9.0", + "hashbrown 0.15.5", + "indexmap 2.10.0", "munge", "ptr_meta 0.3.0", "rancor", "rend 0.5.2", - "rkyv_derive 0.8.10", + "rkyv_derive 0.8.11", "tinyvec", "uuid", ] @@ -6034,13 +6047,13 @@ dependencies = [ [[package]] name = "rkyv_derive" -version = "0.8.10" +version = "0.8.11" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "246b40ac189af6c675d124b802e8ef6d5246c53e17367ce9501f8f66a81abb7a" +checksum = "4270433626cffc9c4c1d3707dd681f2a2718d3d7b09ad754bec137acecda8d22" dependencies = [ "proc-macro2", "quote", - "syn 2.0.101", + "syn 2.0.106", ] [[package]] @@ -6061,9 +6074,9 @@ dependencies = [ [[package]] name = "rsa" -version = "0.9.7" +version = "0.9.8" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "47c75d7c5c6b673e58bf54d8544a9f432e3a925b0e80f7cd3602ab5c50c55519" +checksum = "78928ac1ed176a5ca1d17e578a1825f3d81ca54cf41053a592584b020cfd691b" dependencies = [ "const-oid", "digest", @@ -6093,9 +6106,9 @@ dependencies = [ [[package]] name = "rust_decimal" -version = "1.37.1" +version = "1.37.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "faa7de2ba56ac291bd90c6b9bece784a52ae1411f9506544b3eae36dd2356d50" +checksum = "b203a6425500a03e0919c42d3c47caca51e79f1132046626d2c8871c5092035d" dependencies = [ "arrayvec", "borsh", @@ -6110,9 +6123,9 @@ dependencies = [ [[package]] name = "rustc-demangle" -version = "0.1.24" +version = "0.1.26" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "719b953e2095829ee67db738b3bfa9fa368c94900df327b3f07fe6e794d2fe1f" +checksum = "56f7d92ca342cea22a06f2121d944b4fd82af56988c270852495420f961d4ace" [[package]] name = "rustc-hash" @@ -6144,7 +6157,7 @@ version = "0.38.44" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "fdb5bc1ae2baa591800df16c9ca78619bf65c0488b41b96ccec5d11220d8c154" dependencies = [ - "bitflags 2.8.0", + "bitflags 2.9.2", "errno", "libc", "linux-raw-sys 0.4.15", @@ -6153,15 +6166,15 @@ dependencies = [ [[package]] name = "rustix" -version = "1.0.1" +version = "1.0.8" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "dade4812df5c384711475be5fcd8c162555352945401aed22a35bffeab61f657" +checksum = "11181fbabf243db407ef8df94a6ce0b2f9a733bd8be4ad02b4eda9602296cac8" dependencies = [ - "bitflags 2.8.0", + "bitflags 2.9.2", "errno", "libc", - "linux-raw-sys 0.9.2", - "windows-sys 0.59.0", + "linux-raw-sys 0.9.4", + "windows-sys 0.60.2", ] [[package]] @@ -6178,15 +6191,15 @@ dependencies = [ [[package]] name = "rustls" -version = "0.23.23" +version = "0.23.31" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "47796c98c480fce5406ef69d1c76378375492c3b0a0de587be0c1d9feb12f395" +checksum = "c0ebcbd2f03de0fc1122ad9bb24b127a5a6cd51d72604a3f3c50ac459762b6cc" dependencies = [ "aws-lc-rs", "once_cell", "ring", "rustls-pki-types", - "rustls-webpki 0.102.8", + "rustls-webpki 0.103.4", "subtle", "zeroize", ] @@ -6212,7 +6225,7 @@ dependencies = [ "openssl-probe", "rustls-pki-types", "schannel", - "security-framework 3.2.0", + "security-framework 3.3.0", ] [[package]] @@ -6235,11 +6248,12 @@ dependencies = [ [[package]] name = "rustls-pki-types" -version = "1.11.0" +version = "1.12.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "917ce264624a4b4db1c364dcc35bfca9ded014d0a958cd47ad3e960e988ea51c" +checksum = "229a4a4c221013e7e1f1a043678c5cc39fe5171437c88fb47151a21e6f5b5c79" dependencies = [ "web-time", + "zeroize", ] [[package]] @@ -6254,9 +6268,9 @@ dependencies = [ [[package]] name = "rustls-webpki" -version = "0.102.8" +version = "0.103.4" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "64ca1bc8749bd4cf37b5ce386cc146580777b4e8572c7b97baf22c83f444bee9" +checksum = "0a17884ae0c1b773f1ccd2bd4a8c72f16da897310a98b0e84bf349ad5ead92fc" dependencies = [ "aws-lc-rs", "ring", @@ -6266,9 +6280,9 @@ dependencies = [ [[package]] name = "rustversion" -version = "1.0.19" +version = "1.0.22" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "f7c45b9784283f1b2e7fb61b42047c2fd678ef0960d4f6f1eba131594cc369d4" +checksum = "b39cdef0fa800fc44525c84ccb54a029961a8215f9619753635a9c0d2538d46d" [[package]] name = "rustyline" @@ -6276,7 +6290,7 @@ version = "15.0.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "2ee1e066dc922e513bda599c6ccb5f3bb2b0ea5870a579448f2622993f0a9a2f" dependencies = [ - "bitflags 2.8.0", + "bitflags 2.9.2", "cfg-if", "clipboard-win", "fd-lock", @@ -6287,16 +6301,16 @@ dependencies = [ "nix", "radix_trie", "unicode-segmentation", - "unicode-width 0.2.0", + "unicode-width 0.2.1", "utf8parse", "windows-sys 0.59.0", ] [[package]] name = "ryu" -version = "1.0.19" +version = "1.0.20" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "6ea1a2d0a644769cc99faa24c3ad26b379b786fe7c36fd3c546254801650e6dd" +checksum = "28d3b2b1366ec20994f1fd18c3c594f05c5dd4bc44d8bb0c1c632c8d6829481f" [[package]] name = "salsa20" @@ -6394,7 +6408,7 @@ version = "2.11.1" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "897b2245f0b511c87893af39b033e5ca9cce68824c4d7e7630b5a1d339658d02" dependencies = [ - "bitflags 2.8.0", + "bitflags 2.9.2", "core-foundation 0.9.4", "core-foundation-sys", "libc", @@ -6403,12 +6417,12 @@ dependencies = [ [[package]] name = "security-framework" -version = "3.2.0" +version = "3.3.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "271720403f46ca04f7ba6f55d438f8bd878d6b8ca0a1046e8228c4145bcbb316" +checksum = "80fb1d92c5028aa318b4b8bd7302a5bfcf48be96a37fc6fc790f806b0004ee0c" dependencies = [ - "bitflags 2.8.0", - "core-foundation 0.10.0", + "bitflags 2.9.2", + "core-foundation 0.10.1", "core-foundation-sys", "libc", "security-framework-sys", @@ -6426,50 +6440,50 @@ dependencies = [ [[package]] name = "semver" -version = "1.0.25" +version = "1.0.26" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "f79dfe2d285b0488816f30e700a7438c5a73d816b5b7d3ac72fbc48b0d185e03" +checksum = "56e6fa9c48d24d85fb3de5ad847117517440f6beceb7798af16b4a87d616b8d0" [[package]] name = "seq-macro" -version = "0.3.5" +version = "0.3.6" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a3f0bf26fd526d2a95683cd0f87bf103b8539e2ca1ef48ce002d67aad59aa0b4" +checksum = "1bc711410fbe7399f390ca1c3b60ad0f53f80e95c5eb935e52268a0e2cd49acc" [[package]] name = "serde" -version = "1.0.217" +version = "1.0.219" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "02fc4265df13d6fa1d00ecff087228cc0a2b5f3c0e87e258d8b94a156e984c70" +checksum = "5f0e2c6ed6606019b4e29e69dbaba95b11854410e5347d525002456dbbb786b6" dependencies = [ "serde_derive", ] [[package]] name = "serde_bytes" -version = "0.11.15" +version = "0.11.17" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "387cc504cb06bb40a96c8e04e951fe01854cf6bc921053c954e4a606d9675c6a" +checksum = "8437fd221bde2d4ca316d61b90e337e9e702b3820b87d63caa9ba6c02bd06d96" dependencies = [ "serde", ] [[package]] name = "serde_derive" -version = "1.0.217" +version = "1.0.219" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "5a9bf7cf98d04a2b28aead066b7496853d4779c9cc183c440dbac457641e19a0" +checksum = "5b0276cf7f2c73365f7157c8123c21cd9a50fbbd844757af28ca1f5925fc2a00" dependencies = [ "proc-macro2", "quote", - "syn 2.0.101", + "syn 2.0.106", ] [[package]] name = "serde_json" -version = "1.0.138" +version = "1.0.143" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d434192e7da787e94a6ea7e9670b26a036d0ca41e0b7efb2676dd32bae872949" +checksum = "d401abef1d108fbd9cbaebc3e46611f4b1021f714a0597a71f41ee463f5f4a5a" dependencies = [ "itoa", "memchr", @@ -6479,20 +6493,20 @@ dependencies = [ [[package]] name = "serde_repr" -version = "0.1.19" +version = "0.1.20" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "6c64451ba24fc7a6a2d60fc75dd9c83c90903b19028d4eff35e88fc1e86564e9" +checksum = "175ee3e80ae9982737ca543e96133087cbd9a485eecc3bc4de9c1a37b47ea59c" dependencies = [ "proc-macro2", "quote", - "syn 2.0.101", + "syn 2.0.106", ] [[package]] name = "serde_spanned" -version = "0.6.8" +version = "0.6.9" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "87607cb1398ed59d48732e575a4c28a7a8ebf2454b964fe3f224f2afc07909e1" +checksum = "bf41e0cfaf7226dca15e8197172c295a782857fcb97fad1808a166870dee75a3" dependencies = [ "serde", ] @@ -6519,7 +6533,7 @@ dependencies = [ "chrono", "hex", "indexmap 1.9.3", - "indexmap 2.9.0", + "indexmap 2.10.0", "schemars 0.9.0", "schemars 1.0.4", "serde", @@ -6538,7 +6552,7 @@ dependencies = [ "darling", "proc-macro2", "quote", - "syn 2.0.101", + "syn 2.0.106", ] [[package]] @@ -6554,9 +6568,9 @@ dependencies = [ [[package]] name = "sha2" -version = "0.10.8" +version = "0.10.9" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "793db75ad2bcafc3ffa7c68b215fee268f537982cd901d132f89c6343f3a3dc8" +checksum = "a7507d819769d01a365ab707794a4084392c824f54a7a6a7862f8c3d0892b283" dependencies = [ "cfg-if", "cpufeatures", @@ -6580,9 +6594,9 @@ checksum = "0fda2ff0d084019ba4d7c6f371c95d8fd75ce3524c3cb8fb653a3023f6323e64" [[package]] name = "signal-hook-registry" -version = "1.4.2" +version = "1.4.6" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a9e9e0b4211b72e7b8b6e85c807d36c212bdb33ea8587f7569562a84df5465b1" +checksum = "b2a4719bff48cee6b39d12c020eeb490953ad2443b7055bd0b21fca26bd8c28b" dependencies = [ "libc", ] @@ -6617,7 +6631,7 @@ checksum = "297f631f50729c8c99b84667867963997ec0b50f32b2a7dbcab828ef0541e8bb" dependencies = [ "num-bigint", "num-traits", - "thiserror 2.0.12", + "thiserror 2.0.15", "time", ] @@ -6629,12 +6643,9 @@ checksum = "56199f7ddabf13fe5074ce809e7d3f42b42ae711800501b5b16ea82ad029c39d" [[package]] name = "slab" -version = "0.4.9" +version = "0.4.11" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "8f92a496fb766b417c996b9c5e57daf2f7ad3b0bebe1ccfca4856390e3d3bb67" -dependencies = [ - "autocfg", -] +checksum = "7a2ae44ef20feb57a68b23d846850f861394c2e02dc425a50098ae8c90267589" [[package]] name = "slug" @@ -6648,9 +6659,9 @@ dependencies = [ [[package]] name = "smallvec" -version = "1.14.0" +version = "1.15.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "7fcf8323ef1faaee30a44a340193b1ac6814fd9b7b4e88e9d4519a3e4abe1cfd" +checksum = "67b1b7a3b5fe4f1376887184045fcf45c69e92af734b7aaddc05fb777b6fbd03" dependencies = [ "serde", ] @@ -6663,9 +6674,9 @@ checksum = "1b6b67fb9a61334225b5b790716f609cd58395f895b3fe8b328786812a40bc3b" [[package]] name = "socket2" -version = "0.5.8" +version = "0.5.10" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c970269d99b64e60ec3bd6ad27270092a5394c4e309314b18ae3fe575695fbe8" +checksum = "e22376abed350d73dd1cd119b57ffccad95b4e585a7cda43e286245ce23c0678" dependencies = [ "libc", "windows-sys 0.52.0", @@ -6707,14 +6718,14 @@ dependencies = [ "simdutf8", "sonic-number", "sonic-simd", - "thiserror 2.0.12", + "thiserror 2.0.15", ] [[package]] name = "sonic-simd" -version = "0.1.0" +version = "0.1.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "940a24e82c9a97483ef66cef06b92160a8fa5cd74042c57c10b24d99d169d2fc" +checksum = "b421f7b6aa4a5de8f685aaf398dfaa828346ee639d2b1c1061ab43d40baa6223" dependencies = [ "cfg-if", ] @@ -6731,10 +6742,10 @@ dependencies = [ "http-body 0.4.6", "prost", "prost-types", - "rand 0.9.0", + "rand 0.9.2", "regex", "serde_json", - "thiserror 2.0.12", + "thiserror 2.0.15", "tokio", "tonic", "tonic-build", @@ -6764,9 +6775,9 @@ dependencies = [ [[package]] name = "sqllogictest" -version = "0.28.1" +version = "0.28.3" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "ee6199c1e008acc669b1e5873c138bf3ad4f8709ccd5c5d88913e664ae4f75de" +checksum = "9fcbf91368a8d6807093d94f274fa4d0978cd78a310fee1d20368c545a606f7a" dependencies = [ "async-trait", "educe", @@ -6783,7 +6794,7 @@ dependencies = [ "similar", "subst", "tempfile", - "thiserror 2.0.12", + "thiserror 2.0.15", "tracing", ] @@ -6806,14 +6817,14 @@ checksum = "da5fc6819faabb412da764b99d3b713bb55083c11e7e0c00144d386cd6a1939c" dependencies = [ "proc-macro2", "quote", - "syn 2.0.101", + "syn 2.0.106", ] [[package]] name = "sqlx" -version = "0.8.3" +version = "0.8.6" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "4410e73b3c0d8442c5f99b425d7a435b5ee0ae4167b3196771dd3f7a01be745f" +checksum = "1fefb893899429669dcdd979aff487bd78f4064e5e7907e4269081e0ef7d97dc" dependencies = [ "sqlx-core", "sqlx-macros", @@ -6824,58 +6835,58 @@ dependencies = [ [[package]] name = "sqlx-core" -version = "0.8.3" +version = "0.8.6" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "6a007b6936676aa9ab40207cde35daab0a04b823be8ae004368c0793b96a61e0" +checksum = "ee6798b1838b6a0f69c007c133b8df5866302197e404e8b6ee8ed3e3a5e68dc6" dependencies = [ + "base64 0.22.1", "bytes", "crc", "crossbeam-queue", "either", - "event-listener 5.4.0", + "event-listener 5.4.1", "futures-core", "futures-intrusive", "futures-io", "futures-util", - "hashbrown 0.15.2", + "hashbrown 0.15.5", "hashlink", - "indexmap 2.9.0", + "indexmap 2.10.0", "log", "memchr", "once_cell", "percent-encoding", - "rustls 0.23.23", - "rustls-pemfile 2.2.0", + "rustls 0.23.31", "serde", "serde_json", "sha2", "smallvec", - "thiserror 2.0.12", + "thiserror 2.0.15", "tokio", "tokio-stream", "tracing", "url", - "webpki-roots", + "webpki-roots 0.26.11", ] [[package]] name = "sqlx-macros" -version = "0.8.3" +version = "0.8.6" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "3112e2ad78643fef903618d78cf0aec1cb3134b019730edb039b69eaf531f310" +checksum = "a2d452988ccaacfbf5e0bdbc348fb91d7c8af5bee192173ac3636b5fb6e6715d" dependencies = [ "proc-macro2", "quote", "sqlx-core", "sqlx-macros-core", - "syn 2.0.101", + "syn 2.0.106", ] [[package]] name = "sqlx-macros-core" -version = "0.8.3" +version = "0.8.6" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "4e9f90acc5ab146a99bf5061a7eb4976b573f560bc898ef3bf8435448dd5e7ad" +checksum = "19a9c1841124ac5a61741f96e1d9e2ec77424bf323962dd894bdb93f37d5219b" dependencies = [ "dotenvy", "either", @@ -6889,21 +6900,20 @@ dependencies = [ "sha2", "sqlx-core", "sqlx-sqlite", - "syn 2.0.101", - "tempfile", + "syn 2.0.106", "tokio", "url", ] [[package]] name = "sqlx-mysql" -version = "0.8.3" +version = "0.8.6" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "4560278f0e00ce64938540546f59f590d60beee33fffbd3b9cd47851e5fff233" +checksum = "aa003f0038df784eb8fecbbac13affe3da23b45194bd57dba231c8f48199c526" dependencies = [ "atoi", "base64 0.22.1", - "bitflags 2.8.0", + "bitflags 2.9.2", "byteorder", "bytes", "crc", @@ -6931,20 +6941,20 @@ dependencies = [ "smallvec", "sqlx-core", "stringprep", - "thiserror 2.0.12", + "thiserror 2.0.15", "tracing", "whoami", ] [[package]] name = "sqlx-postgres" -version = "0.8.3" +version = "0.8.6" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c5b98a57f363ed6764d5b3a12bfedf62f07aa16e1856a7ddc2a0bb190a959613" +checksum = "db58fcd5a53cf07c184b154801ff91347e4c30d17a3562a635ff028ad5deda46" dependencies = [ "atoi", "base64 0.22.1", - "bitflags 2.8.0", + "bitflags 2.9.2", "byteorder", "crc", "dotenvy", @@ -6968,16 +6978,16 @@ dependencies = [ "smallvec", "sqlx-core", "stringprep", - "thiserror 2.0.12", + "thiserror 2.0.15", "tracing", "whoami", ] [[package]] name = "sqlx-sqlite" -version = "0.8.3" +version = "0.8.6" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "f85ca71d3a5b24e64e1d08dd8fe36c6c95c339a896cc33068148906784620540" +checksum = "c2d12fe70b2c1b4401038055f90f151b78208de1f9f89a7dbfd41587a10c3eea" dependencies = [ "atoi", "flume", @@ -6992,6 +7002,7 @@ dependencies = [ "serde", "serde_urlencoded", "sqlx-core", + "thiserror 2.0.15", "tracing", "url", ] @@ -7004,9 +7015,9 @@ checksum = "a8f112729512f8e442d81f95a8a7ddf2b7c6b8a1a6f509a95864142b30cab2d3" [[package]] name = "stacker" -version = "0.1.20" +version = "0.1.21" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "601f9201feb9b09c00266478bf459952b9ef9a6b94edb2f21eba14ab681a60a9" +checksum = "cddb07e32ddb770749da91081d8d0ac3a16f1a569a18b20348cd371f5dead06b" dependencies = [ "cc", "cfg-if", @@ -7046,11 +7057,11 @@ checksum = "8fec0f0aef304996cf250b31b5a10dee7980c85da9d759361292b8bca5a18f06" [[package]] name = "strum" -version = "0.27.1" +version = "0.27.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "f64def088c51c9510a8579e3c5d67c65349dcf755e5479ad3d010aa6454e2c32" +checksum = "af23d6f6c1a224baef9d3f61e287d2761385a5b88fdab4eb4c6f11aeb54c4bcf" dependencies = [ - "strum_macros 0.27.1", + "strum_macros 0.27.2", ] [[package]] @@ -7063,27 +7074,26 @@ dependencies = [ "proc-macro2", "quote", "rustversion", - "syn 2.0.101", + "syn 2.0.106", ] [[package]] name = "strum_macros" -version = "0.27.1" +version = "0.27.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c77a8c5abcaf0f9ce05d62342b7d298c346515365c36b673df4ebe3ced01fde8" +checksum = "7695ce3845ea4b33927c055a39dc438a45b059f7c1b3d91d38d10355fb8cbca7" dependencies = [ "heck", "proc-macro2", "quote", - "rustversion", - "syn 2.0.101", + "syn 2.0.106", ] [[package]] name = "subst" -version = "0.3.7" +version = "0.3.8" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "33e7942675ea19db01ef8cf15a1e6443007208e6c74568bd64162da26d40160d" +checksum = "0a9a86e5144f63c2d18334698269a8bfae6eece345c70b64821ea5b35054ec99" dependencies = [ "memchr", "unicode-width 0.1.14", @@ -7108,9 +7118,9 @@ dependencies = [ [[package]] name = "syn" -version = "2.0.101" +version = "2.0.106" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "8ce2b7fc941b3a24138a0a7cf8e858bfc6a992e7978a068a5c760deb0ed43caf" +checksum = "ede7c438028d4436d71104916910f5bb611972c5cfd7f89b8300a8186e6fada6" dependencies = [ "proc-macro2", "quote", @@ -7134,13 +7144,13 @@ dependencies = [ [[package]] name = "synstructure" -version = "0.13.1" +version = "0.13.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c8af7666ab7b6390ab78131fb5b0fce11d6b7a6951602017c35fa82800708971" +checksum = "728a70f3dbaf5bab7f0c4b1ac8d7ae5ea60a4b5549c8a5914361c99147a709d2" dependencies = [ "proc-macro2", "quote", - "syn 2.0.101", + "syn 2.0.106", ] [[package]] @@ -7162,9 +7172,9 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "e8a64e3985349f2441a1a9ef0b853f869006c3855f2cda6862a94d26ebb9d6a1" dependencies = [ "fastrand", - "getrandom 0.3.1", + "getrandom 0.3.3", "once_cell", - "rustix 1.0.1", + "rustix 1.0.8", "windows-sys 0.59.0", ] @@ -7216,11 +7226,11 @@ dependencies = [ [[package]] name = "thiserror" -version = "2.0.12" +version = "2.0.15" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "567b8a2dae586314f7be2a752ec7474332959c6460e02bde30d702a66d488708" +checksum = "80d76d3f064b981389ecb4b6b7f45a0bf9fdac1d5b9204c7bd6714fecc302850" dependencies = [ - "thiserror-impl 2.0.12", + "thiserror-impl 2.0.15", ] [[package]] @@ -7231,28 +7241,27 @@ checksum = "4fee6c4efc90059e10f81e6d42c60a18f76588c3d74cb83a0b242a2b6c7504c1" dependencies = [ "proc-macro2", "quote", - "syn 2.0.101", + "syn 2.0.106", ] [[package]] name = "thiserror-impl" -version = "2.0.12" +version = "2.0.15" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "7f7cf42b4507d8ea322120659672cf1b9dbb93f8f2d4ecfd6e51350ff5b17a1d" +checksum = "44d29feb33e986b6ea906bd9c3559a856983f92371b3eaa5e83782a351623de0" dependencies = [ "proc-macro2", "quote", - "syn 2.0.101", + "syn 2.0.106", ] [[package]] name = "thread_local" -version = "1.1.8" +version = "1.1.9" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "8b9ef9bad013ada3808854ceac7b46812a6465ba368859a37e2100283d2d719c" +checksum = "f60246a4944f24f6e018aa17cdeffb7818b76356965d03b07d6a9886e8962185" dependencies = [ "cfg-if", - "once_cell", ] [[package]] @@ -7279,9 +7288,9 @@ dependencies = [ [[package]] name = "time" -version = "0.3.37" +version = "0.3.41" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "35e7868883861bd0e56d9ac6efcaaca0d6d5d82a2a7ec8209ff492c07cf37b21" +checksum = "8a7619e19bc266e0f9c5e6686659d394bc57973859340060a69221e57dbc0c40" dependencies = [ "deranged", "itoa", @@ -7294,15 +7303,15 @@ dependencies = [ [[package]] name = "time-core" -version = "0.1.2" +version = "0.1.4" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "ef927ca75afb808a4d64dd374f00a2adf8d0fcff8e7b184af886c3c87ec4a3f3" +checksum = "c9e9a38711f559d9e3ce1cdb06dd7c5b8ea546bc90052da6d06bb76da74bb07c" [[package]] name = "time-macros" -version = "0.2.19" +version = "0.2.22" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "2834e6017e3e5e4b9834939793b282bc03b37a3336245fa820e35e233e2a85de" +checksum = "3526739392ec93fd8b359c8e98514cb3e8e021beb4e5f597b00a0221f8ed8a49" dependencies = [ "num-conv", "time-core", @@ -7319,9 +7328,9 @@ dependencies = [ [[package]] name = "tinystr" -version = "0.7.6" +version = "0.8.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "9117f5d4db391c1cf6927e7bea3db74b9a1c1add8f7eda9ffd5364f40f57b82f" +checksum = "5d4f6d1145dcb577acf783d4e601bc1d76a13337bb54e6233add580b07344c8b" dependencies = [ "displaydoc", "zerovec", @@ -7329,9 +7338,9 @@ dependencies = [ [[package]] name = "tinyvec" -version = "1.8.1" +version = "1.10.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "022db8904dfa342efe721985167e9fcd16c29b226db4397ed752a761cfce81e8" +checksum = "bfa5fdc3bce6191a1dbc8c02d5c8bffcf557bafa17c124c5264a458f1b0613fa" dependencies = [ "tinyvec_macros", ] @@ -7344,9 +7353,9 @@ checksum = "1f3ccbac311fea05f86f61904b462b55fb3df8837a366dfc601a0161d0532f20" [[package]] name = "tokio" -version = "1.47.0" +version = "1.47.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "43864ed400b6043a4757a25c7a64a8efde741aed79a056a2fb348a406701bb35" +checksum = "89e49afdadebb872d3145a5638b59eb0691ea23e46ca484037cfab3b76b95038" dependencies = [ "backtrace", "bytes", @@ -7380,7 +7389,7 @@ checksum = "6e06d43f1345a3bcd39f6a56dbb7dcab2ba47e68e8ac134855e7e2bdbaf8cab8" dependencies = [ "proc-macro2", "quote", - "syn 2.0.101", + "syn 2.0.106", ] [[package]] @@ -7395,11 +7404,11 @@ dependencies = [ [[package]] name = "tokio-rustls" -version = "0.26.1" +version = "0.26.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "5f6d0975eaace0cf0fcadee4e4aaa5da15b5c079146f2cffb67c113be122bf37" +checksum = "8e727b36a1a0e8b74c376ac2211e40c2c8af09fb4013c60d910495810f008e9b" dependencies = [ - "rustls 0.23.23", + "rustls 0.23.31", "tokio", ] @@ -7416,9 +7425,9 @@ dependencies = [ [[package]] name = "tokio-util" -version = "0.7.15" +version = "0.7.16" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "66a539a9ad6d5d281510d5bd368c973d636c02dbf8a67300bfb6b950696ad7df" +checksum = "14307c986784f72ef81c89db7d9e28d6ac26d16213b109ea501696195e6e3ce5" dependencies = [ "bytes", "futures-core", @@ -7429,9 +7438,9 @@ dependencies = [ [[package]] name = "toml" -version = "0.8.20" +version = "0.8.23" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "cd87a5cdd6ffab733b2f74bc4fd7ee5fff6634124999ac278c35fc78c6120148" +checksum = "dc1beb996b9d83529a9e75c17a1686767d148d70663143c7854d8b4a09ced362" dependencies = [ "serde", "serde_spanned", @@ -7441,26 +7450,33 @@ dependencies = [ [[package]] name = "toml_datetime" -version = "0.6.8" +version = "0.6.11" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "0dd7358ecb8fc2f8d014bf86f6f638ce72ba252a2c3a2572f2a795f1d23efb41" +checksum = "22cddaf88f4fbc13c51aebbf5f8eceb5c7c5a9da2ac40a13519eb5b0a0e8f11c" dependencies = [ "serde", ] [[package]] name = "toml_edit" -version = "0.22.24" +version = "0.22.27" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "17b4795ff5edd201c7cd6dca065ae59972ce77d1b80fa0a84d94950ece7d1474" +checksum = "41fe8c660ae4257887cf66394862d21dbca4a6ddd26f04a3560410406a2f819a" dependencies = [ - "indexmap 2.9.0", + "indexmap 2.10.0", "serde", "serde_spanned", "toml_datetime", + "toml_write", "winnow", ] +[[package]] +name = "toml_write" +version = "0.1.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "5d99f8c9a7727884afe522e9bd5edbfc91a3312b36a77b5fb8926e4c31a41801" + [[package]] name = "tonic" version = "0.11.0" @@ -7472,7 +7488,7 @@ dependencies = [ "axum", "base64 0.21.7", "bytes", - "h2 0.3.26", + "h2 0.3.27", "http 0.2.12", "http-body 0.4.6", "hyper 0.14.32", @@ -7498,7 +7514,7 @@ dependencies = [ "proc-macro2", "prost-build", "quote", - "syn 2.0.101", + "syn 2.0.106", ] [[package]] @@ -7536,6 +7552,24 @@ dependencies = [ "tower-service", ] +[[package]] +name = "tower-http" +version = "0.6.6" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "adc82fd73de2a9722ac5da747f12383d2bfdb93591ee6c58486e0097890f05f2" +dependencies = [ + "bitflags 2.9.2", + "bytes", + "futures-util", + "http 1.3.1", + "http-body 1.0.1", + "iri-string", + "pin-project-lite", + "tower 0.5.2", + "tower-layer", + "tower-service", +] + [[package]] name = "tower-layer" version = "0.3.3" @@ -7562,20 +7596,20 @@ dependencies = [ [[package]] name = "tracing-attributes" -version = "0.1.28" +version = "0.1.30" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "395ae124c09f9e6918a2310af6038fba074bcf474ac352496d5910dd59a2226d" +checksum = "81383ab64e72a7a8b8e13130c49e3dab29def6d0c7d76a03087b3cf71c5c6903" dependencies = [ "proc-macro2", "quote", - "syn 2.0.101", + "syn 2.0.106", ] [[package]] name = "tracing-core" -version = "0.1.33" +version = "0.1.34" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "e672c95779cf947c5311f83787af4fa8fffd12fb27e4993211a84bdfd9610f9c" +checksum = "b9d12581f227e93f094d3af2ae690a574abb8a2b9b7a96e7cfe9647b2b617678" dependencies = [ "once_cell", "valuable", @@ -7624,19 +7658,9 @@ checksum = "e421abadd41a4225275504ea4d6566923418b7f05506fbc9c0fe86ba7396114b" [[package]] name = "twox-hash" -version = "1.6.3" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "97fee6b57c6a41524a810daee9286c02d7752c4253064d0b05472833a438f675" -dependencies = [ - "cfg-if", - "static_assertions", -] - -[[package]] -name = "twox-hash" -version = "2.1.0" +version = "2.1.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "e7b17f197b3050ba473acf9181f7b1d3b66d1cf7356c6cc57886662276e65908" +checksum = "8b907da542cbced5261bd3256de1b3a1bf340a3d37f93425a07362a1d687de56" [[package]] name = "typed-builder" @@ -7664,7 +7688,7 @@ checksum = "f9534daa9fd3ed0bd911d462a37f172228077e7abf18c18a5f67199d959205f8" dependencies = [ "proc-macro2", "quote", - "syn 2.0.101", + "syn 2.0.106", ] [[package]] @@ -7675,14 +7699,14 @@ checksum = "3c36781cc0e46a83726d9879608e4cf6c2505237e263a8eb8c24502989cfdb28" dependencies = [ "proc-macro2", "quote", - "syn 2.0.101", + "syn 2.0.106", ] [[package]] name = "typenum" -version = "1.17.0" +version = "1.18.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "42ff0bf0c66b8238c6f3b578df37d0b7848e55df8577b3f74f92a69acceeb825" +checksum = "1dccffe3ce07af9386bfd29e80c0ab1a8205a2fc34e4bcd40364df902cfa8f3f" [[package]] name = "ucd-trie" @@ -7748,9 +7772,9 @@ checksum = "5c1cb5db39152898a79168971543b1cb5020dff7fe43c8dc468b0885f5e29df5" [[package]] name = "unicode-ident" -version = "1.0.16" +version = "1.0.18" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a210d160f08b701c8721ba1c726c11662f877ea6b7094007e1ca9a1041945034" +checksum = "5a5f39404a5da50712a4c1eecf25e90dd62b613502b7e925fd4e4d19b5c96512" [[package]] name = "unicode-normalization" @@ -7781,9 +7805,9 @@ checksum = "7dd6e30e90baa6f72411720665d41d89b9a3d039dc45b8faea1ddd07f617f6af" [[package]] name = "unicode-width" -version = "0.2.0" +version = "0.2.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "1fc81956842c57dac11422a97c3b8195a1ff727f06e85c84ed2e8aa277c9a0fd" +checksum = "4a1a07cc7db3810833284e8d372ccdc6da29741639ecc70c9ec107df0fa6154c" [[package]] name = "untrusted" @@ -7808,12 +7832,6 @@ version = "2.1.3" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "daf8dba3b7eb870caf1ddeed7bc9d2a049f3cfdfae7cb521b087cc33ae4c49da" -[[package]] -name = "utf16_iter" -version = "1.0.5" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c8232dd3cdaed5356e0f716d285e4b40b932ac434100fe9b7e0e8e935b9e6246" - [[package]] name = "utf8_iter" version = "1.0.4" @@ -7828,11 +7846,11 @@ checksum = "06abde3611657adf66d383f00b093d7faecc7fa57071cce2578660c9f1010821" [[package]] name = "uuid" -version = "1.17.0" +version = "1.18.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "3cf4199d1e5d15ddd86a694e4d0dffa9c323ce759fea589f00fef9d81cc1931d" +checksum = "f33196643e165781c20a5ead5582283a7dacbb87855d867fbc2df3f81eddc1be" dependencies = [ - "getrandom 0.3.1", + "getrandom 0.3.3", "js-sys", "serde", "wasm-bindgen", @@ -7846,9 +7864,9 @@ checksum = "ba73ea9cf16a25df0c8caa16c51acb937d5712a8429db78a3ee29d5dcacd3a65" [[package]] name = "value-bag" -version = "1.10.0" +version = "1.11.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "3ef4c4aa54d5d05a279399bfa921ec387b7aba77caf7a682ae8d86785b8fdad2" +checksum = "943ce29a8a743eb10d6082545d861b24f9d1b160b7d741e0f2cdf726bec909c5" [[package]] name = "vcpkg" @@ -7879,9 +7897,9 @@ dependencies = [ "nix", "once_cell", "pin-project", - "rand 0.9.0", - "socket2 0.5.8", - "thiserror 2.0.12", + "rand 0.9.2", + "socket2 0.5.10", + "thiserror 2.0.15", "tokio", "tokio-stream", "tower 0.5.2", @@ -7894,7 +7912,7 @@ version = "0.10.8" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "a4a41a936651c3b8206339898466fd4f39596cfedf789e5ee6a8d02a45feb45f" dependencies = [ - "ahash 0.8.11", + "ahash 0.8.12", "anyhow", "bytes", "chrono", @@ -7912,7 +7930,7 @@ dependencies = [ "rustc-hash 2.1.1", "scopeguard", "sonic-rs", - "thiserror 2.0.12", + "thiserror 2.0.15", "tokio", "tracing", "volo", @@ -7945,15 +7963,15 @@ dependencies = [ [[package]] name = "wasi" -version = "0.11.0+wasi-snapshot-preview1" +version = "0.11.1+wasi-snapshot-preview1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "9c8d87e72b64a3b4db28d11ce29237c246188f4f51057d65a7eab63b7987e423" +checksum = "ccf3ec651a847eb01de73ccad15eb7d99f80485de043efb2f370cd654f4ea44b" [[package]] name = "wasi" -version = "0.13.3+wasi-0.2.2" +version = "0.14.2+wasi-0.2.4" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "26816d2e1a4a36a2940b96c5296ce403917633dff8f3440e9b236ed6f6bacad2" +checksum = "9683f9a5a998d873c0d21fcbe3c083009670149a8fab228644b8bd36b2c48cb3" dependencies = [ "wit-bindgen-rt", ] @@ -7986,7 +8004,7 @@ dependencies = [ "log", "proc-macro2", "quote", - "syn 2.0.101", + "syn 2.0.106", "wasm-bindgen-shared", ] @@ -8021,7 +8039,7 @@ checksum = "8ae87ea40c9f689fc23f209965b6fb8a99ad69aeeb0231408be24920604395de" dependencies = [ "proc-macro2", "quote", - "syn 2.0.101", + "syn 2.0.106", "wasm-bindgen-backend", "wasm-bindgen-shared", ] @@ -8070,9 +8088,18 @@ dependencies = [ [[package]] name = "webpki-roots" -version = "0.26.8" +version = "0.26.11" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "521bc38abb08001b01866da9f51eb7c5d647a19260e00054a8c7fd5f9e57f7a9" +dependencies = [ + "webpki-roots 1.0.2", +] + +[[package]] +name = "webpki-roots" +version = "1.0.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "2210b291f7ea53617fbafcc4939f10914214ec15aace5ba62293a668f322c5c9" +checksum = "7e8983c3ab33d6fb807cfcdad2491c4ea8cbc8ed839181c7dfd9c67c83e261b2" dependencies = [ "rustls-pki-types", ] @@ -8091,11 +8118,11 @@ dependencies = [ [[package]] name = "whoami" -version = "1.5.2" +version = "1.6.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "372d5b87f58ec45c384ba03563b03544dc5fadc3983e434b286913f5b4a9bb6d" +checksum = "5d4a4db5077702ca3015d3d02d74974948aba2ad9e12ab7df718ee64ccd7e97d" dependencies = [ - "redox_syscall", + "libredox", "wasite", ] @@ -8132,92 +8159,104 @@ checksum = "712e227841d057c1ee1cd2fb22fa7e5a5461ae8e48fa2ca79ec42cfc1931183f" [[package]] name = "windows" -version = "0.58.0" +version = "0.61.3" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "dd04d41d93c4992d421894c18c8b43496aa748dd4c081bac0dc93eb0489272b6" +checksum = "9babd3a767a4c1aef6900409f85f5d53ce2544ccdfaa86dad48c91782c6d6893" dependencies = [ - "windows-core 0.58.0", - "windows-targets 0.52.6", + "windows-collections", + "windows-core", + "windows-future", + "windows-link", + "windows-numerics", ] [[package]] -name = "windows-core" -version = "0.52.0" +name = "windows-collections" +version = "0.2.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "33ab640c8d7e35bf8ba19b884ba838ceb4fba93a4e8c65a9059d08afcfc683d9" +checksum = "3beeceb5e5cfd9eb1d76b381630e82c4241ccd0d27f1a39ed41b2760b255c5e8" dependencies = [ - "windows-targets 0.52.6", + "windows-core", ] [[package]] name = "windows-core" -version = "0.58.0" +version = "0.61.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "6ba6d44ec8c2591c134257ce647b7ea6b20335bf6379a27dac5f1641fcf59f99" +checksum = "c0fdd3ddb90610c7638aa2b3a3ab2904fb9e5cdbecc643ddb3647212781c4ae3" dependencies = [ "windows-implement", "windows-interface", + "windows-link", "windows-result", "windows-strings", - "windows-targets 0.52.6", +] + +[[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", + "windows-threading", ] [[package]] name = "windows-implement" -version = "0.58.0" +version = "0.60.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "2bbd5b46c938e506ecbce286b6628a02171d56153ba733b6c741fc627ec9579b" +checksum = "a47fddd13af08290e67f4acabf4b459f647552718f683a7b415d290ac744a836" dependencies = [ "proc-macro2", "quote", - "syn 2.0.101", + "syn 2.0.106", ] [[package]] name = "windows-interface" -version = "0.58.0" +version = "0.59.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "053c4c462dc91d3b1504c6fe5a726dd15e216ba718e84a0e46a88fbe5ded3515" +checksum = "bd9211b69f8dcdfa817bfd14bf1c97c9188afa36f4750130fcdf3f400eca9fa8" dependencies = [ "proc-macro2", "quote", - "syn 2.0.101", + "syn 2.0.106", ] [[package]] name = "windows-link" -version = "0.1.1" +version = "0.1.3" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "76840935b766e1b0a05c0066835fb9ec80071d4c09a16f6bd5f7e655e3c14c38" +checksum = "5e6ad25900d524eaabdbbb96d20b4311e1e7ae1699af4fb28c17ae66c80d798a" [[package]] -name = "windows-registry" +name = "windows-numerics" version = "0.2.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "e400001bb720a623c1c69032f8e3e4cf09984deec740f007dd2b03ec864804b0" +checksum = "9150af68066c4c5c07ddc0ce30421554771e528bde427614c61038bc2c92c2b1" dependencies = [ - "windows-result", - "windows-strings", - "windows-targets 0.52.6", + "windows-core", + "windows-link", ] [[package]] name = "windows-result" -version = "0.2.0" +version = "0.3.4" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "1d1043d8214f791817bab27572aaa8af63732e11bf84aa21a45a78d6c317ae0e" +checksum = "56f42bd332cc6c8eac5af113fc0c1fd6a8fd2aa08a0119358686e5160d0586c6" dependencies = [ - "windows-targets 0.52.6", + "windows-link", ] [[package]] name = "windows-strings" -version = "0.1.0" +version = "0.4.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "4cd9b125c486025df0eabcb585e62173c6c9eddcec5d117d3b6e8c30e2ee4d10" +checksum = "56e6c93f3a0c3b36176cb1327a4958a0353d5d166c2a35cb268ace15e91d3b57" dependencies = [ - "windows-result", - "windows-targets 0.52.6", + "windows-link", ] [[package]] @@ -8247,6 +8286,15 @@ dependencies = [ "windows-targets 0.52.6", ] +[[package]] +name = "windows-sys" +version = "0.60.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "f2f500e4d28234f72040990ec9d39e3a6b950f9f22d3dba18416c35882612bcb" +dependencies = [ + "windows-targets 0.53.3", +] + [[package]] name = "windows-targets" version = "0.48.5" @@ -8271,13 +8319,39 @@ dependencies = [ "windows_aarch64_gnullvm 0.52.6", "windows_aarch64_msvc 0.52.6", "windows_i686_gnu 0.52.6", - "windows_i686_gnullvm", + "windows_i686_gnullvm 0.52.6", "windows_i686_msvc 0.52.6", "windows_x86_64_gnu 0.52.6", "windows_x86_64_gnullvm 0.52.6", "windows_x86_64_msvc 0.52.6", ] +[[package]] +name = "windows-targets" +version = "0.53.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "d5fe6031c4041849d7c496a8ded650796e7b6ecc19df1a431c1a363342e5dc91" +dependencies = [ + "windows-link", + "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", +] + [[package]] name = "windows_aarch64_gnullvm" version = "0.48.5" @@ -8290,6 +8364,12 @@ version = "0.52.6" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "32a4622180e7a0ec044bb555404c800bc9fd9ec262ec147edd5989ccd0c02cd3" +[[package]] +name = "windows_aarch64_gnullvm" +version = "0.53.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "86b8d5f90ddd19cb4a147a5fa63ca848db3df085e25fee3cc10b39b6eebae764" + [[package]] name = "windows_aarch64_msvc" version = "0.48.5" @@ -8302,6 +8382,12 @@ version = "0.52.6" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "09ec2a7bb152e2252b53fa7803150007879548bc709c039df7627cabbd05d469" +[[package]] +name = "windows_aarch64_msvc" +version = "0.53.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "c7651a1f62a11b8cbd5e0d42526e55f2c99886c77e007179efff86c2b137e66c" + [[package]] name = "windows_i686_gnu" version = "0.48.5" @@ -8314,12 +8400,24 @@ version = "0.52.6" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "8e9b5ad5ab802e97eb8e295ac6720e509ee4c243f69d781394014ebfe8bbfa0b" +[[package]] +name = "windows_i686_gnu" +version = "0.53.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "c1dc67659d35f387f5f6c479dc4e28f1d4bb90ddd1a5d3da2e5d97b42d6272c3" + [[package]] name = "windows_i686_gnullvm" version = "0.52.6" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "0eee52d38c090b3caa76c563b86c3a4bd71ef1a819287c19d586d7334ae8ed66" +[[package]] +name = "windows_i686_gnullvm" +version = "0.53.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "9ce6ccbdedbf6d6354471319e781c0dfef054c81fbc7cf83f338a4296c0cae11" + [[package]] name = "windows_i686_msvc" version = "0.48.5" @@ -8332,6 +8430,12 @@ version = "0.52.6" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "240948bc05c5e7c6dabba28bf89d89ffce3e303022809e73deaefe4f6ec56c66" +[[package]] +name = "windows_i686_msvc" +version = "0.53.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "581fee95406bb13382d2f65cd4a908ca7b1e4c2f1917f143ba16efe98a589b5d" + [[package]] name = "windows_x86_64_gnu" version = "0.48.5" @@ -8344,6 +8448,12 @@ version = "0.52.6" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "147a5c80aabfbf0c7d901cb5895d1de30ef2907eb21fbbab29ca94c5b08b1a78" +[[package]] +name = "windows_x86_64_gnu" +version = "0.53.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "2e55b5ac9ea33f2fc1716d1742db15574fd6fc8dadc51caab1c16a3d3b4190ba" + [[package]] name = "windows_x86_64_gnullvm" version = "0.48.5" @@ -8356,6 +8466,12 @@ version = "0.52.6" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "24d5b23dc417412679681396f2b49f3de8c1473deb516bd34410872eff51ed0d" +[[package]] +name = "windows_x86_64_gnullvm" +version = "0.53.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "0a6e035dd0599267ce1ee132e51c27dd29437f63325753051e71dd9e42406c57" + [[package]] name = "windows_x86_64_msvc" version = "0.48.5" @@ -8368,35 +8484,35 @@ version = "0.52.6" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "589f6da84c646204747d1270a2a5661ea66ed1cced2631d546fdfb155959f9ec" +[[package]] +name = "windows_x86_64_msvc" +version = "0.53.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "271414315aff87387382ec3d271b52d7ae78726f5d44ac98b4f4030c91880486" + [[package]] name = "winnow" -version = "0.7.2" +version = "0.7.12" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "59690dea168f2198d1a3b0cac23b8063efcd11012f10ae4698f284808c8ef603" +checksum = "f3edebf492c8125044983378ecb5766203ad3b4c2f7a922bd7dd207f6d443e95" dependencies = [ "memchr", ] [[package]] name = "wit-bindgen-rt" -version = "0.33.0" +version = "0.39.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "3268f3d866458b787f390cf61f4bbb563b922d091359f9608842999eaee3943c" +checksum = "6f42320e61fe2cfd34354ecb597f86f413484a798ba44a8ca1165c58d42da6c1" dependencies = [ - "bitflags 2.8.0", + "bitflags 2.9.2", ] -[[package]] -name = "write16" -version = "1.0.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d1890f4022759daae28ed4fe62859b1236caebfc61ede2f63ed4e695f3f6d936" - [[package]] name = "writeable" -version = "0.5.5" +version = "0.6.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "1e9df38ee2d2c3c5948ea468a8406ff0db0b29ae1ffde1bcf20ef305bcc95c51" +checksum = "ea2f10b9bb0928dfb1b42b65e1f9e36f7f54dbdf08457afefb38afcdec4fa2bb" [[package]] name = "wyz" @@ -8430,9 +8546,9 @@ checksum = "cfe53a6657fd280eaa890a3bc59152892ffa3e30101319d168b781ed6529b049" [[package]] name = "yoke" -version = "0.7.5" +version = "0.8.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "120e6aef9aa629e3d4f52dc8cc43a015c7724194c97dfaf45180d2daf2b77f40" +checksum = "5f41bb01b8226ef4bfd589436a297c53d118f65921786300e427be8d487695cc" dependencies = [ "serde", "stable_deref_trait", @@ -8442,75 +8558,54 @@ dependencies = [ [[package]] name = "yoke-derive" -version = "0.7.5" +version = "0.8.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "2380878cad4ac9aac1e2435f3eb4020e8374b5f13c296cb75b4620ff8e229154" +checksum = "38da3c9736e16c5d3c8c597a9aaa5d1fa565d0532ae05e27c24aa62fb32c0ab6" dependencies = [ "proc-macro2", "quote", - "syn 2.0.101", + "syn 2.0.106", "synstructure", ] [[package]] name = "zerocopy" -version = "0.7.35" +version = "0.8.26" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "1b9b4fd18abc82b8136838da5d50bae7bdea537c574d8dc1a34ed098d6c166f0" +checksum = "1039dd0d3c310cf05de012d8a39ff557cb0d23087fd44cad61df08fc31907a2f" dependencies = [ - "byteorder", - "zerocopy-derive 0.7.35", -] - -[[package]] -name = "zerocopy" -version = "0.8.18" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "79386d31a42a4996e3336b0919ddb90f81112af416270cff95b5f5af22b839c2" -dependencies = [ - "zerocopy-derive 0.8.18", + "zerocopy-derive", ] [[package]] name = "zerocopy-derive" -version = "0.7.35" +version = "0.8.26" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "fa4f8080344d4671fb4e831a13ad1e68092748387dfc4f55e356242fae12ce3e" +checksum = "9ecf5b4cc5364572d7f4c329661bcc82724222973f2cab6f050a4e5c22f75181" dependencies = [ "proc-macro2", "quote", - "syn 2.0.101", -] - -[[package]] -name = "zerocopy-derive" -version = "0.8.18" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "76331675d372f91bf8d17e13afbd5fe639200b73d01f0fc748bb059f9cca2db7" -dependencies = [ - "proc-macro2", - "quote", - "syn 2.0.101", + "syn 2.0.106", ] [[package]] name = "zerofrom" -version = "0.1.5" +version = "0.1.6" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "cff3ee08c995dee1859d998dea82f7374f2826091dd9cd47def953cae446cd2e" +checksum = "50cc42e0333e05660c3587f3bf9d0478688e15d870fab3346451ce7f8c9fbea5" dependencies = [ "zerofrom-derive", ] [[package]] name = "zerofrom-derive" -version = "0.1.5" +version = "0.1.6" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "595eed982f7d355beb85837f651fa22e90b3c044842dc7f2c2842c086f295808" +checksum = "d71e5d6e06ab090c67b5e44993ec16b72dcbaabc526db883a360057678b48502" dependencies = [ "proc-macro2", "quote", - "syn 2.0.101", + "syn 2.0.106", "synstructure", ] @@ -8520,11 +8615,22 @@ version = "1.8.1" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "ced3678a2879b30306d323f4542626697a464a97c0a07c9aebf7ebca65cd4dde" +[[package]] +name = "zerotrie" +version = "0.2.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "36f0bbd478583f79edad978b407914f61b2972f5af6fa089686016be8f9af595" +dependencies = [ + "displaydoc", + "yoke", + "zerofrom", +] + [[package]] name = "zerovec" -version = "0.10.4" +version = "0.11.4" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "aa2b893d79df23bfb12d5461018d408ea19dfafe76c2c7ef6d4eba614f8ff079" +checksum = "e7aa2bd55086f1ab526693ecbe444205da57e25f4489879da80635a46d90e73b" dependencies = [ "yoke", "zerofrom", @@ -8533,44 +8639,44 @@ dependencies = [ [[package]] name = "zerovec-derive" -version = "0.10.3" +version = "0.11.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "6eafa6dfb17584ea3e2bd6e76e0cc15ad7af12b09abdd1ca55961bed9b1063c6" +checksum = "5b96237efa0c878c64bd89c436f661be4e46b2f3eff1ebb976f7ef2321d2f58f" dependencies = [ "proc-macro2", "quote", - "syn 2.0.101", + "syn 2.0.106", ] [[package]] name = "zlib-rs" -version = "0.5.0" +version = "0.5.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "868b928d7949e09af2f6086dfc1e01936064cc7a819253bce650d4e2a2d63ba8" +checksum = "626bd9fa9734751fc50d6060752170984d7053f5a39061f524cda68023d4db8a" [[package]] name = "zstd" -version = "0.13.2" +version = "0.13.3" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "fcf2b778a664581e31e389454a7072dab1647606d44f7feea22cd5abb9c9f3f9" +checksum = "e91ee311a569c327171651566e07972200e76fcfe2242a4fa446149a3881c08a" dependencies = [ "zstd-safe", ] [[package]] name = "zstd-safe" -version = "7.2.1" +version = "7.2.4" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "54a3ab4db68cea366acc5c897c7b4d4d1b8994a9cd6e6f841f8964566a419059" +checksum = "8f49c4d5f0abb602a93fb8736af2a4f4dd9512e36f7f570d66e65ff867ed3b9d" dependencies = [ "zstd-sys", ] [[package]] name = "zstd-sys" -version = "2.0.13+zstd.1.5.6" +version = "2.0.15+zstd.1.5.7" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "38ff0f21cfee8f97d94cef41359e0c89aa6113028ab0291aa8ca0038995a95aa" +checksum = "eb81183ddd97d0c74cedf1d50d85c8d08c1b8b68ee863bdee9e706eedba1a237" dependencies = [ "cc", "pkg-config", From f776bf05bfb1cc1283e10bedfa4e7e288749e065 Mon Sep 17 00:00:00 2001 From: Leon Lin Date: Thu, 21 Aug 2025 09:47:01 -0700 Subject: [PATCH 4/9] Update doc --- crates/sqllogictest/README.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/crates/sqllogictest/README.md b/crates/sqllogictest/README.md index ddcfe851c5..72b592bfa0 100644 --- a/crates/sqllogictest/README.md +++ b/crates/sqllogictest/README.md @@ -24,7 +24,7 @@ This crate contains a suite of [sqllogictest](https://crates.io/crates/sqllogict Just run the following command: ```bash -cargo test +cargo test --test sqllogictests ``` ## Sql Engines From 1a530cc164e2b43e042bb55577f4ee3fd4032842 Mon Sep 17 00:00:00 2001 From: Leon Lin Date: Thu, 21 Aug 2025 09:49:22 -0700 Subject: [PATCH 5/9] clean up --- crates/sqllogictests/Cargo.toml | 25 -- crates/sqllogictests/src/engine/conversion.rs | 101 ------- .../src/engine/datafusion/error.rs | 50 ---- .../src/engine/datafusion/mod.rs | 26 -- .../src/engine/datafusion/normalize.rs | 279 ------------------ .../src/engine/datafusion/runner.rs | 87 ------ crates/sqllogictests/src/engine/mod.rs | 16 - crates/sqllogictests/src/engine/output.rs | 57 ---- .../sqllogictests/src/engine/sparksql/mod.rs | 30 -- crates/sqllogictests/src/lib.rs | 21 -- crates/sqllogictests/src/schedule.rs | 38 --- 11 files changed, 730 deletions(-) delete mode 100644 crates/sqllogictests/Cargo.toml delete mode 100644 crates/sqllogictests/src/engine/conversion.rs delete mode 100644 crates/sqllogictests/src/engine/datafusion/error.rs delete mode 100644 crates/sqllogictests/src/engine/datafusion/mod.rs delete mode 100644 crates/sqllogictests/src/engine/datafusion/normalize.rs delete mode 100644 crates/sqllogictests/src/engine/datafusion/runner.rs delete mode 100644 crates/sqllogictests/src/engine/mod.rs delete mode 100644 crates/sqllogictests/src/engine/output.rs delete mode 100644 crates/sqllogictests/src/engine/sparksql/mod.rs delete mode 100644 crates/sqllogictests/src/lib.rs delete mode 100644 crates/sqllogictests/src/schedule.rs diff --git a/crates/sqllogictests/Cargo.toml b/crates/sqllogictests/Cargo.toml deleted file mode 100644 index f5667e24db..0000000000 --- a/crates/sqllogictests/Cargo.toml +++ /dev/null @@ -1,25 +0,0 @@ -[package] -name = "sqllogictests" -version.workspace = true -edition.workspace = true -homepage.workspace = true -repository.workspace = true -license.workspace = true -rust-version.workspace = true - -[dependencies] -arrow-schema = { workspace = true } -arrow-array= { workspace = true } -async-trait = { workspace = true } -sqllogictest = "0.21.0" -datafusion = { workspace = true, default-features = true} -datafusion-common = { workspace = true, default-features = true} -thiserror = "1.0.63" -sqlparser = {workspace = true} -itertools = "0.13.0" -half = "2.4.1" -bigdecimal = "0.4.1" -rust_decimal = { version = "1.27.0" } -tempfile = { workspace = true } -log = "0.4.22" -tokio = "1.38.0" \ No newline at end of file diff --git a/crates/sqllogictests/src/engine/conversion.rs b/crates/sqllogictests/src/engine/conversion.rs deleted file mode 100644 index 937f43520b..0000000000 --- a/crates/sqllogictests/src/engine/conversion.rs +++ /dev/null @@ -1,101 +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 arrow_array::types::{Decimal128Type, Decimal256Type, DecimalType}; -use bigdecimal::BigDecimal; -use datafusion_common::arrow::datatypes::i256; -use half::f16; -use rust_decimal::prelude::*; - -/// Represents a constant for NULL string in your database. -pub const NULL_STR: &str = "NULL"; - -pub(crate) fn bool_to_str(value: bool) -> String { - if value { - "true".to_string() - } else { - "false".to_string() - } -} - -pub(crate) fn varchar_to_str(value: &str) -> String { - if value.is_empty() { - "(empty)".to_string() - } else { - value.trim_end_matches('\n').to_string() - } -} - -pub(crate) fn f16_to_str(value: f16) -> String { - if value.is_nan() { - // The sign of NaN can be different depending on platform. - // So the string representation of NaN ignores the sign. - "NaN".to_string() - } else if value == f16::INFINITY { - "Infinity".to_string() - } else if value == f16::NEG_INFINITY { - "-Infinity".to_string() - } else { - big_decimal_to_str(BigDecimal::from_str(&value.to_string()).unwrap()) - } -} - -pub(crate) fn f32_to_str(value: f32) -> String { - if value.is_nan() { - // The sign of NaN can be different depending on platform. - // So the string representation of NaN ignores the sign. - "NaN".to_string() - } else if value == f32::INFINITY { - "Infinity".to_string() - } else if value == f32::NEG_INFINITY { - "-Infinity".to_string() - } else { - big_decimal_to_str(BigDecimal::from_str(&value.to_string()).unwrap()) - } -} - -pub(crate) fn f64_to_str(value: f64) -> String { - if value.is_nan() { - // The sign of NaN can be different depending on platform. - // So the string representation of NaN ignores the sign. - "NaN".to_string() - } else if value == f64::INFINITY { - "Infinity".to_string() - } else if value == f64::NEG_INFINITY { - "-Infinity".to_string() - } else { - big_decimal_to_str(BigDecimal::from_str(&value.to_string()).unwrap()) - } -} - -pub(crate) fn i128_to_str(value: i128, precision: &u8, scale: &i8) -> String { - big_decimal_to_str( - BigDecimal::from_str(&Decimal128Type::format_decimal(value, *precision, *scale)) - .unwrap(), - ) -} - -pub(crate) fn i256_to_str(value: i256, precision: &u8, scale: &i8) -> String { - big_decimal_to_str( - BigDecimal::from_str(&Decimal256Type::format_decimal(value, *precision, *scale)) - .unwrap(), - ) -} - -pub(crate) fn big_decimal_to_str(value: BigDecimal) -> String { - value.round(12).normalized().to_string() -} \ No newline at end of file diff --git a/crates/sqllogictests/src/engine/datafusion/error.rs b/crates/sqllogictests/src/engine/datafusion/error.rs deleted file mode 100644 index ec479516cd..0000000000 --- a/crates/sqllogictests/src/engine/datafusion/error.rs +++ /dev/null @@ -1,50 +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 arrow_schema::ArrowError; -use datafusion_common::DataFusionError; -use sqllogictest::TestError; -use sqlparser::parser::ParserError; -use thiserror::Error; - -pub type Result = std::result::Result; - -/// DataFusion sql-logicaltest error -#[derive(Debug, Error)] -pub enum DFSqlLogicTestError { - /// Error from sqllogictest-rs - #[error("SqlLogicTest error(from sqllogictest-rs crate): {0}")] - SqlLogicTest(#[from] TestError), - /// Error from datafusion - #[error("DataFusion error: {0}")] - DataFusion(#[from] DataFusionError), - /// Error returned when SQL is syntactically incorrect. - #[error("SQL Parser error: {0}")] - Sql(#[from] ParserError), - /// Error from arrow-rs - #[error("Arrow error: {0}")] - Arrow(#[from] ArrowError), - /// Generic error - #[error("Other Error: {0}")] - Other(String), -} - -impl From for DFSqlLogicTestError { - fn from(value: String) -> Self { - DFSqlLogicTestError::Other(value) - } -} \ No newline at end of file diff --git a/crates/sqllogictests/src/engine/datafusion/mod.rs b/crates/sqllogictests/src/engine/datafusion/mod.rs deleted file mode 100644 index da2c218585..0000000000 --- a/crates/sqllogictests/src/engine/datafusion/mod.rs +++ /dev/null @@ -1,26 +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. - - -/// DataFusion engine implementation for sqllogictest. - -mod error; -mod normalize; -mod runner; - -pub use error::*; -pub use runner::*; \ No newline at end of file diff --git a/crates/sqllogictests/src/engine/datafusion/normalize.rs b/crates/sqllogictests/src/engine/datafusion/normalize.rs deleted file mode 100644 index d4e883d21d..0000000000 --- a/crates/sqllogictests/src/engine/datafusion/normalize.rs +++ /dev/null @@ -1,279 +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 arrow_schema::Fields; -use datafusion_common::format::DEFAULT_FORMAT_OPTIONS; -use datafusion_common::DataFusionError; -use std::path::PathBuf; -use std::sync::OnceLock; -use arrow_array::{ArrayRef, RecordBatch}; -use crate::engines::output::DFColumnType; - -use super::super::conversion::*; -use super::error::{DFSqlLogicTestError, Result}; - -/// Converts `batches` to a result as expected by sqllogicteset. -pub(crate) fn convert_batches(batches: Vec) -> Result>> { - if batches.is_empty() { - Ok(vec![]) - } else { - let schema = batches[0].schema(); - let mut rows = vec![]; - for batch in batches { - // Verify schema - if !schema.contains(&batch.schema()) { - return Err(DFSqlLogicTestError::DataFusion(DataFusionError::Internal( - format!( - "Schema mismatch. Previously had\n{:#?}\n\nGot:\n{:#?}", - &schema, - batch.schema() - ), - ))); - } - - let new_rows = convert_batch(batch)? - .into_iter() - .flat_map(expand_row) - .map(normalize_paths); - rows.extend(new_rows); - } - Ok(rows) - } -} - -/// special case rows that have newlines in them (like explain plans) -// -/// Transform inputs like: -/// ```text -/// [ -/// "logical_plan", -/// "Sort: d.b ASC NULLS LAST\n Projection: d.b, MAX(d.a) AS max_a", -/// ] -/// ``` -/// -/// Into one cell per line, adding lines if necessary -/// ```text -/// [ -/// "logical_plan", -/// ] -/// [ -/// "Sort: d.b ASC NULLS LAST", -/// ] -/// [ <--- newly added row -/// "|-- Projection: d.b, MAX(d.a) AS max_a", -/// ] -/// ``` -fn expand_row(mut row: Vec) -> impl Iterator> { - use itertools::Either; - use std::iter::once; - - // check last cell - if let Some(cell) = row.pop() { - let lines: Vec<_> = cell.split('\n').collect(); - - // no newlines in last cell - if lines.len() < 2 { - row.push(cell); - return Either::Left(once(row)); - } - - // form new rows with each additional line - let new_lines: Vec<_> = lines - .into_iter() - .enumerate() - .map(|(idx, l)| { - // replace any leading spaces with '-' as - // `sqllogictest` ignores whitespace differences - // - // See https://github.com/apache/datafusion/issues/6328 - let content = l.trim_start(); - let new_prefix = "-".repeat(l.len() - content.len()); - // maintain for each line a number, so - // reviewing explain result changes is easier - let line_num = idx + 1; - vec![format!("{line_num:02}){new_prefix}{content}")] - }) - .collect(); - - Either::Right(once(row).chain(new_lines)) - } else { - Either::Left(once(row)) - } -} - -/// normalize path references -/// -/// ```text -/// CsvExec: files={1 group: [[path/to/datafusion/testing/data/csv/aggregate_test_100.csv]]}, ... -/// ``` -/// -/// into: -/// -/// ```text -/// CsvExec: files={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, ... -/// ``` -fn normalize_paths(mut row: Vec) -> Vec { - row.iter_mut().for_each(|s| { - let workspace_root: &str = workspace_root().as_ref(); - if s.contains(workspace_root) { - *s = s.replace(workspace_root, "WORKSPACE_ROOT"); - } - }); - row -} - -/// return the location of the datafusion checkout -fn workspace_root() -> &'static object_store::path::Path { - static WORKSPACE_ROOT_LOCK: OnceLock = OnceLock::new(); - WORKSPACE_ROOT_LOCK.get_or_init(|| { - // e.g. /Software/datafusion/datafusion/core - let dir = PathBuf::from(env!("CARGO_MANIFEST_DIR")); - - // e.g. /Software/datafusion/datafusion - let workspace_root = dir - .parent() - .expect("Can not find parent of datafusion/core") - // e.g. /Software/datafusion - .parent() - .expect("parent of datafusion") - .to_string_lossy(); - - let sanitized_workplace_root = if cfg!(windows) { - // Object store paths are delimited with `/`, e.g. `/datafusion/datafusion/testing/data/csv/aggregate_test_100.csv`. - // The default windows delimiter is `\`, so the workplace path is `datafusion\datafusion`. - workspace_root - .replace(std::path::MAIN_SEPARATOR, object_store::path::DELIMITER) - } else { - workspace_root.to_string() - }; - - object_store::path::Path::parse(sanitized_workplace_root).unwrap() - }) -} - -/// Convert a single batch to a `Vec>` for comparison -fn convert_batch(batch: RecordBatch) -> Result>> { - (0..batch.num_rows()) - .map(|row| { - batch - .columns() - .iter() - .map(|col| cell_to_string(col, row)) - .collect::>>() - }) - .collect() -} - -macro_rules! get_row_value { - ($array_type:ty, $column: ident, $row: ident) => {{ - let array = $column.as_any().downcast_ref::<$array_type>().unwrap(); - - array.value($row) - }}; -} - -/// Normalizes the content of a single cell in RecordBatch prior to printing. -/// -/// This is to make the output comparable to the semi-standard .slt format -/// -/// Normalizations applied to [NULL Values and empty strings] -/// -/// [NULL Values and empty strings]: https://duckdb.org/dev/sqllogictest/result_verification#null-values-and-empty-strings -/// -/// Floating numbers are rounded to have a consistent representation with the Postgres runner. -/// -pub fn cell_to_string(col: &ArrayRef, row: usize) -> Result { - if !col.is_valid(row) { - // represent any null value with the string "NULL" - Ok(NULL_STR.to_string()) - } else { - match col.data_type() { - DataType::Null => Ok(NULL_STR.to_string()), - DataType::Boolean => { - Ok(bool_to_str(get_row_value!(array::BooleanArray, col, row))) - } - DataType::Float16 => { - Ok(f16_to_str(get_row_value!(array::Float16Array, col, row))) - } - DataType::Float32 => { - Ok(f32_to_str(get_row_value!(array::Float32Array, col, row))) - } - DataType::Float64 => { - Ok(f64_to_str(get_row_value!(array::Float64Array, col, row))) - } - DataType::Decimal128(precision, scale) => { - let value = get_row_value!(array::Decimal128Array, col, row); - Ok(i128_to_str(value, precision, scale)) - } - DataType::Decimal256(precision, scale) => { - let value = get_row_value!(array::Decimal256Array, col, row); - Ok(i256_to_str(value, precision, scale)) - } - DataType::LargeUtf8 => Ok(varchar_to_str(get_row_value!( - array::LargeStringArray, - col, - row - ))), - DataType::Utf8 => { - Ok(varchar_to_str(get_row_value!(array::StringArray, col, row))) - } - DataType::Utf8View => Ok(varchar_to_str(get_row_value!( - array::StringViewArray, - col, - row - ))), - _ => { - let f = ArrayFormatter::try_new(col.as_ref(), &DEFAULT_FORMAT_OPTIONS); - Ok(f.unwrap().value(row).to_string()) - } - } - .map_err(DFSqlLogicTestError::Arrow) - } -} - -/// Converts columns to a result as expected by sqllogicteset. -pub(crate) fn convert_schema_to_types(columns: &Fields) -> Vec { - columns - .iter() - .map(|f| f.data_type()) - .map(|data_type| match data_type { - DataType::Boolean => DFColumnType::Boolean, - DataType::Int8 - | DataType::Int16 - | DataType::Int32 - | DataType::Int64 - | DataType::UInt8 - | DataType::UInt16 - | DataType::UInt32 - | DataType::UInt64 => DFColumnType::Integer, - DataType::Float16 - | DataType::Float32 - | DataType::Float64 - | DataType::Decimal128(_, _) - | DataType::Decimal256(_, _) => DFColumnType::Float, - DataType::Utf8 | DataType::LargeUtf8 | DataType::Utf8View => { - DFColumnType::Text - } - DataType::Date32 - | DataType::Date64 - | DataType::Time32(_) - | DataType::Time64(_) => DFColumnType::DateTime, - DataType::Timestamp(_, _) => DFColumnType::Timestamp, - _ => DFColumnType::Another, - }) - .collect() -} \ No newline at end of file diff --git a/crates/sqllogictests/src/engine/datafusion/runner.rs b/crates/sqllogictests/src/engine/datafusion/runner.rs deleted file mode 100644 index e39f62b1be..0000000000 --- a/crates/sqllogictests/src/engine/datafusion/runner.rs +++ /dev/null @@ -1,87 +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::sync::Arc; -use std::{path::PathBuf, time::Duration}; -use arrow_array::RecordBatch; -use async_trait::async_trait; -use datafusion::physical_plan::common::collect; -use datafusion::physical_plan::execute_stream; -use datafusion::prelude::SessionContext; -use log::info; -use sqllogictest::DBOutput; - -use super::{error::Result, normalize, DFSqlLogicTestError}; - -use crate::engine::output::{DFColumnType, DFOutput}; - -pub struct DataFusionEngine { - ctx: SessionContext, - relative_path: PathBuf, -} - -impl DataFusionEngine { - pub fn new(ctx: SessionContext, relative_path: PathBuf) -> Self { - Self { ctx, relative_path } - } -} - -#[async_trait] -impl sqllogictest::AsyncDB for DataFusionEngine { - type Error = DFSqlLogicTestError; - type ColumnType = DFColumnType; - - async fn run(&mut self, sql: &str) -> Result { - info!( - "[{}] Running query: \"{}\"", - self.relative_path.display(), - sql - ); - run_query(&self.ctx, sql).await - } - - /// Engine name of current database. - fn engine_name(&self) -> &str { - "DataFusion" - } - - /// [`DataFusionEngine`] calls this function to perform sleep. - /// - /// The default implementation is `std::thread::sleep`, which is universal to any async runtime - /// but would block the current thread. If you are running in tokio runtime, you should override - /// this by `tokio::time::sleep`. - async fn sleep(dur: Duration) { - tokio::time::sleep(dur).await; - } -} - -async fn run_query(ctx: &SessionContext, sql: impl Into) -> Result { - let df = ctx.sql(sql.into().as_str()).await?; - let task_ctx = Arc::new(df.task_ctx()); - let plan = df.create_physical_plan().await?; - - let stream = execute_stream(plan, task_ctx)?; - let types = normalize::convert_schema_to_types(stream.schema().fields()); - let results: Vec = collect(stream).await?; - let rows = normalize::convert_batches(results)?; - - if rows.is_empty() && types.is_empty() { - Ok(DBOutput::StatementComplete(0)) - } else { - Ok(DBOutput::Rows { types, rows }) - } -} \ No newline at end of file diff --git a/crates/sqllogictests/src/engine/mod.rs b/crates/sqllogictests/src/engine/mod.rs deleted file mode 100644 index ba2a1f4151..0000000000 --- a/crates/sqllogictests/src/engine/mod.rs +++ /dev/null @@ -1,16 +0,0 @@ -mod datafusion; - -use std::sync::Arc; -pub use datafusion::*; - -mod sparksql; -mod conversion; -mod output; - -pub use sparksql::*; - - -pub enum Engine { - DataFusion(Arc), - SparkSQL(Arc), -} \ No newline at end of file diff --git a/crates/sqllogictests/src/engine/output.rs b/crates/sqllogictests/src/engine/output.rs deleted file mode 100644 index ae1030ca4a..0000000000 --- a/crates/sqllogictests/src/engine/output.rs +++ /dev/null @@ -1,57 +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 sqllogictest::{ColumnType, DBOutput}; - -#[derive(Debug, PartialEq, Eq, Clone)] -pub enum DFColumnType { - Boolean, - DateTime, - Integer, - Float, - Text, - Timestamp, - Another, -} - -impl ColumnType for DFColumnType { - fn from_char(value: char) -> Option { - match value { - 'B' => Some(Self::Boolean), - 'D' => Some(Self::DateTime), - 'I' => Some(Self::Integer), - 'P' => Some(Self::Timestamp), - 'R' => Some(Self::Float), - 'T' => Some(Self::Text), - _ => Some(Self::Another), - } - } - - fn to_char(&self) -> char { - match self { - Self::Boolean => 'B', - Self::DateTime => 'D', - Self::Integer => 'I', - Self::Timestamp => 'P', - Self::Float => 'R', - Self::Text => 'T', - Self::Another => '?', - } - } -} - -pub(crate) type DFOutput = DBOutput; \ No newline at end of file diff --git a/crates/sqllogictests/src/engine/sparksql/mod.rs b/crates/sqllogictests/src/engine/sparksql/mod.rs deleted file mode 100644 index aeff2616e8..0000000000 --- a/crates/sqllogictests/src/engine/sparksql/mod.rs +++ /dev/null @@ -1,30 +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 sqllogictest::{AsyncDB, DBOutput}; - -/// SparkSql engine implementation for sqllogictest. -pub struct SparkSqlEngine; - -impl AsyncDB for SparkSqlEngine { - type Error = (); - type ColumnType = (); - - async fn run(&mut self, sql: &str) -> Result, Self::Error> { - todo!() - } -} diff --git a/crates/sqllogictests/src/lib.rs b/crates/sqllogictests/src/lib.rs deleted file mode 100644 index d01802f61d..0000000000 --- a/crates/sqllogictests/src/lib.rs +++ /dev/null @@ -1,21 +0,0 @@ -// Licensed to the Apache Software Foundation (ASF) under one -// or more contributor license agreements. See the NOTICE file -// distributed with this work for additional information -// regarding copyright ownership. The ASF licenses this file -// to you under the Apache License, Version 2.0 (the -// "License"); you may not use this file except in compliance -// with the License. You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, -// software distributed under the License is distributed on an -// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -// KIND, either express or implied. See the License for the -// specific language governing permissions and limitations -// under the License. - -// This lib contains codes copied from -// [Apache Datafusion](https://github.com/apache/datafusion/tree/main/datafusion/sqllogictest) -mod engine; -mod schedule; diff --git a/crates/sqllogictests/src/schedule.rs b/crates/sqllogictests/src/schedule.rs deleted file mode 100644 index 26e789bb09..0000000000 --- a/crates/sqllogictests/src/schedule.rs +++ /dev/null @@ -1,38 +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::path::Path; -use crate::engine::Engine; - -/// Schedule of engines to run tests. -pub struct Schedule { - /// Map of engine names to engine instances. - engines: HashMap, - /// List of steps to run, each step is a sql file. - steps: Vec, -} - -impl Schedule { - pub async fn parse>(_schedule_def_file: P) -> Self { - todo!() - } - - pub async fn run(mut self) { - todo!() - } -} From 4ca69d3e80087ed2532d30cdb25e8c22b290cc8c Mon Sep 17 00:00:00 2001 From: Leon Lin Date: Thu, 21 Aug 2025 09:52:26 -0700 Subject: [PATCH 6/9] add license headers --- crates/sqllogictest/src/display/conversion.rs | 17 +++++++++++++++++ crates/sqllogictest/src/display/mod.rs | 17 +++++++++++++++++ crates/sqllogictest/src/engine/spark.rs | 19 ++++++++++++++++++- 3 files changed, 52 insertions(+), 1 deletion(-) diff --git a/crates/sqllogictest/src/display/conversion.rs b/crates/sqllogictest/src/display/conversion.rs index d68afaaf00..83935184b0 100644 --- a/crates/sqllogictest/src/display/conversion.rs +++ b/crates/sqllogictest/src/display/conversion.rs @@ -1,3 +1,20 @@ +// 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 arrow::array::types::{Decimal128Type, Decimal256Type, DecimalType}; use arrow::datatypes::i256; use bigdecimal::BigDecimal; diff --git a/crates/sqllogictest/src/display/mod.rs b/crates/sqllogictest/src/display/mod.rs index f237773c30..a90ab3d3be 100644 --- a/crates/sqllogictest/src/display/mod.rs +++ b/crates/sqllogictest/src/display/mod.rs @@ -1 +1,18 @@ +// 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. + mod conversion; diff --git a/crates/sqllogictest/src/engine/spark.rs b/crates/sqllogictest/src/engine/spark.rs index 1473c07ea0..29294738b0 100644 --- a/crates/sqllogictest/src/engine/spark.rs +++ b/crates/sqllogictest/src/engine/spark.rs @@ -1,8 +1,25 @@ +// 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::path::Path; use std::time::Duration; use anyhow::{Context, anyhow}; -use datafusion_sqllogictest::{DFColumnType, DFOutput, convert_schema_to_types, convert_batches}; +use datafusion_sqllogictest::{DFColumnType, DFOutput, convert_batches, convert_schema_to_types}; use spark_connect_rs::{SparkSession, SparkSessionBuilder}; use sqllogictest::{AsyncDB, DBOutput, Record, parse_file}; use toml::Table as TomlTable; From 509a6981dc52af2992f20d0dfbe46cf6142697d4 Mon Sep 17 00:00:00 2001 From: Leon Lin Date: Thu, 21 Aug 2025 11:19:30 -0700 Subject: [PATCH 7/9] install protobuf due to spark rs compile --- .github/workflows/ci.yml | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/.github/workflows/ci.yml b/.github/workflows/ci.yml index 7d94370fe5..20c91ffbd7 100644 --- a/.github/workflows/ci.yml +++ b/.github/workflows/ci.yml @@ -53,6 +53,10 @@ jobs: uses: taiki-e/install-action@v2 with: tool: taplo-cli@0.9.3 + + - name: Install protobuf + run: brew install protobuf + - name: Check toml format run: make check-toml From 7f5499b207b1d5cad076b3cfffd175ba5585a570 Mon Sep 17 00:00:00 2001 From: Leon Lin Date: Thu, 21 Aug 2025 11:37:39 -0700 Subject: [PATCH 8/9] Fixing license check for slt files --- .github/workflows/ci.yml | 6 ++++++ .licenserc.yaml | 1 + .../testdata/docker/docker-compose.yml | 17 +++++++++++++++++ .../docker/spark/spark-connect-server.sh | 18 ++++++++++++++++++ .../sqllogictest/testdata/schedules/test.toml | 17 +++++++++++++++++ 5 files changed, 59 insertions(+) diff --git a/.github/workflows/ci.yml b/.github/workflows/ci.yml index 20c91ffbd7..87af272dca 100644 --- a/.github/workflows/ci.yml +++ b/.github/workflows/ci.yml @@ -93,6 +93,9 @@ jobs: - name: Cache Rust artifacts uses: Swatinem/rust-cache@v2 + - name: Install protobuf + run: brew install protobuf + - name: Build run: make build @@ -113,6 +116,9 @@ jobs: - name: Cache Rust artifacts uses: Swatinem/rust-cache@v2 + - name: Install protobuf + run: brew install protobuf + - name: Build run: cargo build -p iceberg --no-default-features diff --git a/.licenserc.yaml b/.licenserc.yaml index 01625e32cb..8c050828b0 100644 --- a/.licenserc.yaml +++ b/.licenserc.yaml @@ -33,4 +33,5 @@ header: - 'dist/*' - 'Cargo.lock' - '.github/PULL_REQUEST_TEMPLATE.md' + - '**/*.slt' comment: on-failure diff --git a/crates/sqllogictest/testdata/docker/docker-compose.yml b/crates/sqllogictest/testdata/docker/docker-compose.yml index 5b30aa8f99..b4335af423 100644 --- a/crates/sqllogictest/testdata/docker/docker-compose.yml +++ b/crates/sqllogictest/testdata/docker/docker-compose.yml @@ -1,3 +1,20 @@ +# 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. + networks: rest_bridge: diff --git a/crates/sqllogictest/testdata/docker/spark/spark-connect-server.sh b/crates/sqllogictest/testdata/docker/spark/spark-connect-server.sh index 6bc2877d27..404599099f 100755 --- a/crates/sqllogictest/testdata/docker/spark/spark-connect-server.sh +++ b/crates/sqllogictest/testdata/docker/spark/spark-connect-server.sh @@ -1,4 +1,22 @@ #!/bin/bash +# +# 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. +# set -ex diff --git a/crates/sqllogictest/testdata/schedules/test.toml b/crates/sqllogictest/testdata/schedules/test.toml index 109c37e7b6..10c9577e83 100644 --- a/crates/sqllogictest/testdata/schedules/test.toml +++ b/crates/sqllogictest/testdata/schedules/test.toml @@ -1,3 +1,20 @@ +# 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] sc = { type = "spark-connect", url = "sc://localhost:15002" } df = { type = "datafusion", url = "http://localhost:8181" } From 0d5903d675c1b97c36b2c9c7114c86792dc8be09 Mon Sep 17 00:00:00 2001 From: Leon Lin Date: Thu, 21 Aug 2025 12:07:57 -0700 Subject: [PATCH 9/9] Fix issues due to spark connect rs compile --- .github/workflows/bindings_python_ci.yml | 1 + .github/workflows/ci.yml | 39 +++++++- .github/workflows/ci_typos.yml | 1 + .github/workflows/website.yml | 6 ++ Cargo.lock | 99 +++++++++---------- Cargo.toml | 2 +- Makefile | 15 ++- .../integrations/datafusion/src/table/mod.rs | 2 +- crates/sqllogictest/Cargo.toml | 8 +- crates/sqllogictest/src/display/conversion.rs | 99 ------------------- crates/sqllogictest/src/display/mod.rs | 18 ---- crates/sqllogictest/src/engine/datafusion.rs | 12 +-- crates/sqllogictest/src/engine/mod.rs | 6 +- crates/sqllogictest/src/engine/spark.rs | 4 +- crates/sqllogictest/src/lib.rs | 1 - crates/sqllogictest/src/schedule.rs | 14 +-- .../testdata/slts/demo/verify.slt | 2 +- crates/sqllogictest/tests/sqllogictests.rs | 2 - 18 files changed, 114 insertions(+), 217 deletions(-) delete mode 100644 crates/sqllogictest/src/display/conversion.rs delete mode 100644 crates/sqllogictest/src/display/mod.rs diff --git a/.github/workflows/bindings_python_ci.yml b/.github/workflows/bindings_python_ci.yml index eabc3144d9..d21e98f5b8 100644 --- a/.github/workflows/bindings_python_ci.yml +++ b/.github/workflows/bindings_python_ci.yml @@ -24,6 +24,7 @@ on: pull_request: branches: - main + - 'sqllogic-test' concurrency: group: ${{ github.workflow }}-${{ github.ref }}-${{ github.event_name }} diff --git a/.github/workflows/ci.yml b/.github/workflows/ci.yml index 87af272dca..b61d09e50f 100644 --- a/.github/workflows/ci.yml +++ b/.github/workflows/ci.yml @@ -21,9 +21,11 @@ on: push: branches: - main + - 'sqllogic-test' pull_request: branches: - main + - 'sqllogic-test' concurrency: group: ${{ github.workflow }}-${{ github.ref }}-${{ github.event_name }} @@ -55,7 +57,15 @@ jobs: tool: taplo-cli@0.9.3 - name: Install protobuf - run: brew install protobuf + run: | + if [[ "$RUNNER_OS" == "Linux" ]]; then + sudo apt-get update && sudo apt-get install -y protobuf-compiler + elif [[ "$RUNNER_OS" == "macOS" ]]; then + brew install protobuf + elif [[ "$RUNNER_OS" == "Windows" ]]; then + choco install protoc + fi + shell: bash - name: Check toml format run: make check-toml @@ -94,7 +104,15 @@ jobs: uses: Swatinem/rust-cache@v2 - name: Install protobuf - run: brew install protobuf + run: | + if [[ "$RUNNER_OS" == "Linux" ]]; then + sudo apt-get update && sudo apt-get install -y protobuf-compiler + elif [[ "$RUNNER_OS" == "macOS" ]]; then + brew install protobuf + elif [[ "$RUNNER_OS" == "Windows" ]]; then + choco install protoc + fi + shell: bash - name: Build run: make build @@ -117,7 +135,15 @@ jobs: uses: Swatinem/rust-cache@v2 - name: Install protobuf - run: brew install protobuf + run: | + if [[ "$RUNNER_OS" == "Linux" ]]; then + sudo apt-get update && sudo apt-get install -y protobuf-compiler + elif [[ "$RUNNER_OS" == "macOS" ]]; then + brew install protobuf + elif [[ "$RUNNER_OS" == "Windows" ]]; then + choco install protoc + fi + shell: bash - name: Build run: cargo build -p iceberg --no-default-features @@ -144,6 +170,9 @@ jobs: - name: Cache Rust artifacts uses: Swatinem/rust-cache@v2 + - name: Install protobuf + run: sudo apt-get update && sudo apt-get install -y protobuf-compiler + - name: Test run: cargo test --no-fail-fast --all-targets --all-features --workspace @@ -160,6 +189,10 @@ jobs: - uses: actions/checkout@v5 - name: Setup Nightly Rust toolchain uses: ./.github/actions/setup-builder + + - name: Install protobuf + run: sudo apt-get update && sudo apt-get install -y protobuf-compiler libprotobuf-dev + - name: Generate minimal versions lockfile run: | cargo generate-lockfile -Z direct-minimal-versions -Z minimal-versions diff --git a/.github/workflows/ci_typos.yml b/.github/workflows/ci_typos.yml index e79e0a0acd..2699aeae94 100644 --- a/.github/workflows/ci_typos.yml +++ b/.github/workflows/ci_typos.yml @@ -24,6 +24,7 @@ on: pull_request: branches: - main + - 'sqllogic-test' concurrency: group: ${{ github.workflow }}-${{ github.ref }}-${{ github.event_name }} diff --git a/.github/workflows/website.yml b/.github/workflows/website.yml index 7f17192af8..ed8c4f308a 100644 --- a/.github/workflows/website.yml +++ b/.github/workflows/website.yml @@ -24,6 +24,7 @@ on: pull_request: branches: - main + - 'sqllogic-test' concurrency: group: ${{ github.workflow }}-${{ github.ref }}-${{ github.event_name }} @@ -49,6 +50,11 @@ jobs: - name: Copy asf file run: cp .asf.yaml ./website/book/.asf.yaml + - name: Install protobuf + run: | + sudo apt-get update && sudo apt-get install -y protobuf-compiler + shell: bash + - name: Build API docs run: | cargo doc --no-deps --workspace --all-features diff --git a/Cargo.lock b/Cargo.lock index 38464a945d..0ecb6be7ea 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -743,9 +743,9 @@ dependencies = [ [[package]] name = "aws-sdk-glue" -version = "1.116.0" +version = "1.117.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "b4b44ba32edf94e0723dfa1ee340170925858012ac0c981f4ee220c7455014bf" +checksum = "0f68344fb124bf37061b07186ec9e5aaa4560097c916091ae039021dde64cce9" dependencies = [ "aws-credential-types", "aws-runtime", @@ -831,9 +831,9 @@ dependencies = [ [[package]] name = "aws-sdk-sts" -version = "1.83.0" +version = "1.84.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a5468593c47efc31fdbe6c902d1a5fde8d9c82f78a3f8ccfe907b1e9434748cb" +checksum = "91abcdbfb48c38a0419eb75e0eac772a4783a96750392680e4f3c25a8a0535b9" dependencies = [ "aws-credential-types", "aws-runtime", @@ -1448,9 +1448,9 @@ dependencies = [ [[package]] name = "cfg-if" -version = "1.0.1" +version = "1.0.3" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "9555578bc9e57714c812a1f84e4fc5b4d21fcb063490c624de019f7464c91268" +checksum = "2fd1289c04a9ea8cb22300a459a72a385d7c73d3259e2ed7dcb2af674838cfa9" [[package]] name = "cfg_aliases" @@ -2556,7 +2556,7 @@ dependencies = [ "sqllogictest", "sqlparser", "tempfile", - "thiserror 2.0.15", + "thiserror 2.0.16", "tokio", ] @@ -2952,9 +2952,9 @@ checksum = "d9c4f5dac5e15c24eb999c26181a6ca40b39fe946cbe4c263c7209467bc83af2" [[package]] name = "form_urlencoded" -version = "1.2.1" +version = "1.2.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "e13624c2627564efccf4934284bdd98cbaa14e79b0b5a141218e507b3a823456" +checksum = "cb4cb245038516f5f85277875cdaa4f7d2c9a0fa0468de06ed190163b1581fcf" dependencies = [ "percent-encoding", ] @@ -3101,9 +3101,9 @@ dependencies = [ [[package]] name = "generator" -version = "0.8.5" +version = "0.8.7" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d18470a76cb7f8ff746cf1f7470914f900252ec36bbc40b569d74b1258446827" +checksum = "605183a538e3e2a9c1038635cc5c2d194e2ee8fd0d1b66b8349fad7dbacce5a2" dependencies = [ "cc", "cfg-if", @@ -3813,15 +3813,12 @@ name = "iceberg-sqllogictest" version = "0.6.0" dependencies = [ "anyhow", - "arrow", + "async-stream", "async-trait", - "bigdecimal", "datafusion", - "datafusion-common", "datafusion-sqllogictest", "enum-ordinalize", "env_logger", - "half", "iceberg", "iceberg-catalog-rest", "iceberg-datafusion", @@ -3830,10 +3827,8 @@ dependencies = [ "itertools 0.13.0", "libtest-mimic 0.7.3", "log", - "rust_decimal", "spark-connect-rs", "sqllogictest", - "tempfile", "tokio", "toml", ] @@ -3940,9 +3935,9 @@ checksum = "b9e0384b61958566e926dc50660321d12159025e767c18e043daf26b70104c39" [[package]] name = "idna" -version = "1.0.3" +version = "1.1.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "686f825264d630750a544639377bae737628043f20d38bbc029e8f29ea968a7e" +checksum = "3b0875f23caa03898994f6ddc501886a45c7d3d62d04d2d90788d47be1b1e4de" dependencies = [ "idna_adapter", "smallvec", @@ -4887,7 +4882,7 @@ dependencies = [ "md-5", "parking_lot", "percent-encoding", - "quick-xml 0.38.1", + "quick-xml 0.38.2", "rand 0.9.2", "reqwest", "ring", @@ -4895,7 +4890,7 @@ dependencies = [ "serde", "serde_json", "serde_urlencoded", - "thiserror 2.0.15", + "thiserror 2.0.16", "tokio", "tracing", "url", @@ -5115,9 +5110,9 @@ dependencies = [ [[package]] name = "percent-encoding" -version = "2.3.1" +version = "2.3.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "e3148f5046208a5d56bcfc03053e3ca6334e51da8dfb19b6cdc8b306fae3283e" +checksum = "9b4f627cb1b25917193a259e49bdad08f671f8d9708acfd5fe0a8c1455d87220" [[package]] name = "pest" @@ -5126,7 +5121,7 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "1db05f56d34358a8b1066f67cbb203ee3e7ed2ba674a6263a1d5ec6db2204323" dependencies = [ "memchr", - "thiserror 2.0.15", + "thiserror 2.0.16", "ucd-trie", ] @@ -5620,9 +5615,9 @@ dependencies = [ [[package]] name = "quick-xml" -version = "0.38.1" +version = "0.38.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "9845d9dccf565065824e69f9f235fafba1587031eda353c1f1561cd6a6be78f4" +checksum = "d200a41a7797e6461bd04e4e95c3347053a731c32c87f066f2f0dda22dbdbba8" dependencies = [ "memchr", "serde", @@ -5642,7 +5637,7 @@ dependencies = [ "rustc-hash 2.1.1", "rustls 0.23.31", "socket2 0.5.10", - "thiserror 2.0.15", + "thiserror 2.0.16", "tokio", "tracing", "web-time", @@ -5663,7 +5658,7 @@ dependencies = [ "rustls 0.23.31", "rustls-pki-types", "slab", - "thiserror 2.0.15", + "thiserror 2.0.16", "tinyvec", "tracing", "web-time", @@ -5821,7 +5816,7 @@ checksum = "a4e608c6638b9c18977b00b475ac1f28d14e84b27d8d42f70e0bf1e3dec127ac" dependencies = [ "getrandom 0.2.16", "libredox", - "thiserror 2.0.15", + "thiserror 2.0.16", ] [[package]] @@ -6631,7 +6626,7 @@ checksum = "297f631f50729c8c99b84667867963997ec0b50f32b2a7dbcab828ef0541e8bb" dependencies = [ "num-bigint", "num-traits", - "thiserror 2.0.15", + "thiserror 2.0.16", "time", ] @@ -6718,7 +6713,7 @@ dependencies = [ "simdutf8", "sonic-number", "sonic-simd", - "thiserror 2.0.15", + "thiserror 2.0.16", ] [[package]] @@ -6745,7 +6740,7 @@ dependencies = [ "rand 0.9.2", "regex", "serde_json", - "thiserror 2.0.15", + "thiserror 2.0.16", "tokio", "tonic", "tonic-build", @@ -6794,7 +6789,7 @@ dependencies = [ "similar", "subst", "tempfile", - "thiserror 2.0.15", + "thiserror 2.0.16", "tracing", ] @@ -6861,7 +6856,7 @@ dependencies = [ "serde_json", "sha2", "smallvec", - "thiserror 2.0.15", + "thiserror 2.0.16", "tokio", "tokio-stream", "tracing", @@ -6941,7 +6936,7 @@ dependencies = [ "smallvec", "sqlx-core", "stringprep", - "thiserror 2.0.15", + "thiserror 2.0.16", "tracing", "whoami", ] @@ -6978,7 +6973,7 @@ dependencies = [ "smallvec", "sqlx-core", "stringprep", - "thiserror 2.0.15", + "thiserror 2.0.16", "tracing", "whoami", ] @@ -7002,7 +6997,7 @@ dependencies = [ "serde", "serde_urlencoded", "sqlx-core", - "thiserror 2.0.15", + "thiserror 2.0.16", "tracing", "url", ] @@ -7167,15 +7162,15 @@ checksum = "55937e1799185b12863d447f42597ed69d9928686b8d88a1df17376a097d8369" [[package]] name = "tempfile" -version = "3.20.0" +version = "3.21.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "e8a64e3985349f2441a1a9ef0b853f869006c3855f2cda6862a94d26ebb9d6a1" +checksum = "15b61f8f20e3a6f7e0649d825294eaf317edce30f82cf6026e7e4cb9222a7d1e" dependencies = [ "fastrand", "getrandom 0.3.3", "once_cell", "rustix 1.0.8", - "windows-sys 0.59.0", + "windows-sys 0.60.2", ] [[package]] @@ -7226,11 +7221,11 @@ dependencies = [ [[package]] name = "thiserror" -version = "2.0.15" +version = "2.0.16" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "80d76d3f064b981389ecb4b6b7f45a0bf9fdac1d5b9204c7bd6714fecc302850" +checksum = "3467d614147380f2e4e374161426ff399c91084acd2363eaf549172b3d5e60c0" dependencies = [ - "thiserror-impl 2.0.15", + "thiserror-impl 2.0.16", ] [[package]] @@ -7246,9 +7241,9 @@ dependencies = [ [[package]] name = "thiserror-impl" -version = "2.0.15" +version = "2.0.16" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "44d29feb33e986b6ea906bd9c3559a856983f92371b3eaa5e83782a351623de0" +checksum = "6c5e1be1c48b9172ee610da68fd9cd2770e7a4056cb3fc98710ee6906f0c7960" dependencies = [ "proc-macro2", "quote", @@ -7817,9 +7812,9 @@ checksum = "8ecb6da28b8a351d773b68d5825ac39017e680750f980f3a1a85cd8dd28a47c1" [[package]] name = "url" -version = "2.5.4" +version = "2.5.6" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "32f8b686cadd1473f4bd0117a5d28d36b1ade384ea9b5069a1c40aefed7fda60" +checksum = "137a3c834eaf7139b73688502f3f1141a0337c5d8e4d9b536f9b8c796e26a7c4" dependencies = [ "form_urlencoded", "idna", @@ -7899,7 +7894,7 @@ dependencies = [ "pin-project", "rand 0.9.2", "socket2 0.5.10", - "thiserror 2.0.15", + "thiserror 2.0.16", "tokio", "tokio-stream", "tower 0.5.2", @@ -7930,7 +7925,7 @@ dependencies = [ "rustc-hash 2.1.1", "scopeguard", "sonic-rs", - "thiserror 2.0.15", + "thiserror 2.0.16", "tokio", "tracing", "volo", @@ -8144,11 +8139,11 @@ checksum = "ac3b87c63620426dd9b991e5ce0329eff545bccbbb34f3be09ff6fb6ab51b7b6" [[package]] name = "winapi-util" -version = "0.1.9" +version = "0.1.10" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "cf221c93e13a30d793f7645a0e7762c55d169dbb0a49671918a2319d289b10bb" +checksum = "0978bf7171b3d90bac376700cb56d606feb40f251a475a5d6634613564460b22" dependencies = [ - "windows-sys 0.59.0", + "windows-sys 0.60.2", ] [[package]] diff --git a/Cargo.toml b/Cargo.toml index f7494ec878..4f249d99b8 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -127,4 +127,4 @@ url = "2.5.4" uuid = { version = "1.16", features = ["v7"] } volo = "0.10.6" volo-thrift = "0.10.8" -zstd = "0.13.2" +zstd = "0.13.2" \ No newline at end of file diff --git a/Makefile b/Makefile index 2cc847d7b3..67283d3e62 100644 --- a/Makefile +++ b/Makefile @@ -17,14 +17,16 @@ .EXPORT_ALL_VARIABLES: +SQL_LOGIC_TEST := iceberg-sqllogictest + build: - cargo build --all-targets --all-features --workspace + cargo build --all-targets --all-features --workspace --exclude $(SQL_LOGIC_TEST) check-fmt: cargo fmt --all -- --check check-clippy: - cargo clippy --all-targets --all-features --workspace -- -D warnings + cargo clippy --all-targets --all-features --workspace --exclude $(SQL_LOGIC_TEST) -- -D warnings install-cargo-machete: cargo install cargo-machete@0.7.0 @@ -44,13 +46,16 @@ check-toml: install-taplo-cli check: check-fmt check-clippy check-toml cargo-machete doc-test: - cargo test --no-fail-fast --doc --all-features --workspace + cargo test --no-fail-fast --doc --all-features --workspace --exclude $(SQL_LOGIC_TEST) unit-test: doc-test - cargo test --no-fail-fast --lib --all-features --workspace + cargo test --no-fail-fast --lib --all-features --workspace --exclude $(SQL_LOGIC_TEST) test: doc-test - cargo test --no-fail-fast --all-targets --all-features --workspace + cargo test --no-fail-fast --all-targets --all-features --workspace --exclude $(SQL_LOGIC_TEST) + +sqllogictest: + cargo test -p iceberg-sqllogictest --no-fail-fast clean: cargo clean diff --git a/crates/integrations/datafusion/src/table/mod.rs b/crates/integrations/datafusion/src/table/mod.rs index ed87afdcdd..7f741a534a 100644 --- a/crates/integrations/datafusion/src/table/mod.rs +++ b/crates/integrations/datafusion/src/table/mod.rs @@ -172,7 +172,7 @@ mod tests { async fn get_test_table_from_metadata_file() -> Table { let metadata_file_name = "TableMetadataV2Valid.json"; let metadata_file_path = format!( - "{}/tests/testdata/{}", + "{}/tests/test_data/{}", env!("CARGO_MANIFEST_DIR"), metadata_file_name ); diff --git a/crates/sqllogictest/Cargo.toml b/crates/sqllogictest/Cargo.toml index 963be88339..8e0fe4b9e8 100644 --- a/crates/sqllogictest/Cargo.toml +++ b/crates/sqllogictest/Cargo.toml @@ -25,11 +25,9 @@ rust-version = { workspace = true } version = { workspace = true } [dependencies] -arrow = { workspace = true } anyhow = { workspace = true } async-trait = { workspace = true } datafusion = { workspace = true } -datafusion-common = { workspace = true, default-features = true } datafusion-sqllogictest = { workspace = true } enum-ordinalize = { workspace = true } indicatif = { workspace = true } @@ -43,14 +41,12 @@ tokio = { workspace = true } env_logger = "0.11.8" log = "0.4.25" itertools = "0.13.0" -tempfile = { workspace = true } spark-connect-rs = { git = "https://github.com/apache/spark-connect-rust.git", rev = "061cb3ecb187b039141f20c722c7984e915f3b9d" } -half = "2.6.0" -bigdecimal = "0.4.1" -rust_decimal = { version = "1.27.0" } +#spark-connect-rs = "0.0.2" [dev-dependencies] libtest-mimic = "0.7.3" +async-stream = ">=0.3.5, <0.4" [[test]] harness = false diff --git a/crates/sqllogictest/src/display/conversion.rs b/crates/sqllogictest/src/display/conversion.rs deleted file mode 100644 index 83935184b0..0000000000 --- a/crates/sqllogictest/src/display/conversion.rs +++ /dev/null @@ -1,99 +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 arrow::array::types::{Decimal128Type, Decimal256Type, DecimalType}; -use arrow::datatypes::i256; -use bigdecimal::BigDecimal; -use half::f16; -use rust_decimal::prelude::*; - -/// Represents a constant for NULL string in your database. -pub const NULL_STR: &str = "NULL"; - -pub(crate) fn bool_to_str(value: bool) -> String { - if value { - "true".to_string() - } else { - "false".to_string() - } -} - -pub(crate) fn varchar_to_str(value: &str) -> String { - if value.is_empty() { - "(empty)".to_string() - } else { - value.trim_end_matches('\n').to_string() - } -} - -pub(crate) fn f16_to_str(value: f16) -> String { - if value.is_nan() { - // The sign of NaN can be different depending on platform. - // So the string representation of NaN ignores the sign. - "NaN".to_string() - } else if value == f16::INFINITY { - "Infinity".to_string() - } else if value == f16::NEG_INFINITY { - "-Infinity".to_string() - } else { - big_decimal_to_str(BigDecimal::from_str(&value.to_string()).unwrap()) - } -} - -pub(crate) fn f32_to_str(value: f32) -> String { - if value.is_nan() { - // The sign of NaN can be different depending on platform. - // So the string representation of NaN ignores the sign. - "NaN".to_string() - } else if value == f32::INFINITY { - "Infinity".to_string() - } else if value == f32::NEG_INFINITY { - "-Infinity".to_string() - } else { - big_decimal_to_str(BigDecimal::from_str(&value.to_string()).unwrap()) - } -} - -pub(crate) fn f64_to_str(value: f64) -> String { - if value.is_nan() { - // The sign of NaN can be different depending on platform. - // So the string representation of NaN ignores the sign. - "NaN".to_string() - } else if value == f64::INFINITY { - "Infinity".to_string() - } else if value == f64::NEG_INFINITY { - "-Infinity".to_string() - } else { - big_decimal_to_str(BigDecimal::from_str(&value.to_string()).unwrap()) - } -} - -pub(crate) fn i128_to_str(value: i128, precision: &u8, scale: &i8) -> String { - big_decimal_to_str( - BigDecimal::from_str(&Decimal128Type::format_decimal(value, *precision, *scale)).unwrap(), - ) -} - -pub(crate) fn i256_to_str(value: i256, precision: &u8, scale: &i8) -> String { - big_decimal_to_str( - BigDecimal::from_str(&Decimal256Type::format_decimal(value, *precision, *scale)).unwrap(), - ) -} - -pub(crate) fn big_decimal_to_str(value: BigDecimal) -> String { - value.round(12).normalized().to_string() -} diff --git a/crates/sqllogictest/src/display/mod.rs b/crates/sqllogictest/src/display/mod.rs deleted file mode 100644 index a90ab3d3be..0000000000 --- a/crates/sqllogictest/src/display/mod.rs +++ /dev/null @@ -1,18 +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. - -mod conversion; diff --git a/crates/sqllogictest/src/engine/datafusion.rs b/crates/sqllogictest/src/engine/datafusion.rs index 6c3216f6d3..669f120df6 100644 --- a/crates/sqllogictest/src/engine/datafusion.rs +++ b/crates/sqllogictest/src/engine/datafusion.rs @@ -35,7 +35,6 @@ use crate::engine::EngineRunner; use crate::error::{Error, Result}; pub struct DataFusionEngine { - ctx: SessionContext, relative_path: PathBuf, pb: ProgressBar, config: TomlTable, @@ -44,12 +43,9 @@ pub struct DataFusionEngine { #[async_trait::async_trait] impl EngineRunner for DataFusionEngine { async fn run_slt_file(&mut self, path: &Path) -> Result<()> { - let path_dir = path.to_str().unwrap(); - println!("engine running slt file on path: {path_dir}"); - let session_config = SessionConfig::new().with_target_partitions(4); let ctx = SessionContext::new_with_config(session_config); - ctx.register_catalog("demo", Self::create_catalog(&self.config).await?); + ctx.register_catalog("default", Self::create_catalog(&self.config).await?); let runner = sqllogictest::Runner::new(|| async { Ok(DataFusion::new( @@ -68,11 +64,7 @@ impl EngineRunner for DataFusionEngine { impl DataFusionEngine { pub async fn new(config: TomlTable) -> Result { - let session_config = SessionConfig::new().with_target_partitions(4); - let ctx = SessionContext::new_with_config(session_config); - ctx.register_catalog("demo", Self::create_catalog(&config).await?); Ok(Self { - ctx, relative_path: PathBuf::from("testdata"), pb: ProgressBar::new(100), config, @@ -111,7 +103,7 @@ impl DataFusionEngine { ) -> Result<()> { println!("run file in runner"); - let records = parse_file(&path).context("Failed to parse slt file")?; + let records = parse_file(path).context("Failed to parse slt file")?; let mut errs = vec![]; for record in records.into_iter() { diff --git a/crates/sqllogictest/src/engine/mod.rs b/crates/sqllogictest/src/engine/mod.rs index 35faa7eff7..c07ca8fb6f 100644 --- a/crates/sqllogictest/src/engine/mod.rs +++ b/crates/sqllogictest/src/engine/mod.rs @@ -31,10 +31,10 @@ pub trait EngineRunner { async fn run_slt_file(&mut self, path: &Path) -> Result<()>; } -pub async fn load_engine(typ: &str, cfg: TomlTable) -> Result> { - match typ { +pub async fn load_engine(engine_type: &str, cfg: TomlTable) -> Result> { + match engine_type { "datafusion" => Ok(Box::new(DataFusionEngine::new(cfg).await?)), "spark-connect" => Ok(Box::new(SparkEngine::new(cfg).await?)), - _ => Err(anyhow::anyhow!("Unsupported engine type: {}", typ).into()), + _ => Err(anyhow::anyhow!("Unsupported engine type: {}", engine_type).into()), } } diff --git a/crates/sqllogictest/src/engine/spark.rs b/crates/sqllogictest/src/engine/spark.rs index 29294738b0..26b177bb6b 100644 --- a/crates/sqllogictest/src/engine/spark.rs +++ b/crates/sqllogictest/src/engine/spark.rs @@ -27,8 +27,6 @@ use toml::Table as TomlTable; use crate::engine::EngineRunner; use crate::error::{Error, Result}; -pub type SparkOutput = DBOutput; - pub struct SparkEngine { session: SparkSession, } @@ -109,7 +107,7 @@ impl SparkEngine { { println!("run file in runner"); - let records = parse_file(&path).context("Failed to parse slt file")?; + let records = parse_file(path).context("Failed to parse slt file")?; let mut errs = vec![]; for record in records.into_iter() { diff --git a/crates/sqllogictest/src/lib.rs b/crates/sqllogictest/src/lib.rs index 82cd668aeb..34a3c654aa 100644 --- a/crates/sqllogictest/src/lib.rs +++ b/crates/sqllogictest/src/lib.rs @@ -18,7 +18,6 @@ // This lib contains codes copied from // [Apache Datafusion](https://github.com/apache/datafusion/tree/main/datafusion/sqllogictest) -mod display; mod engine; pub mod error; pub mod schedule; diff --git a/crates/sqllogictest/src/schedule.rs b/crates/sqllogictest/src/schedule.rs index 4f8566c0bb..67bd8038ec 100644 --- a/crates/sqllogictest/src/schedule.rs +++ b/crates/sqllogictest/src/schedule.rs @@ -20,8 +20,6 @@ use std::fs::read_to_string; use std::path::{Path, PathBuf}; use anyhow::anyhow; -use iceberg::{Catalog, CatalogBuilder}; -use iceberg_catalog_rest::RestCatalogBuilder; use itertools::Itertools; use toml::{Table, Value}; @@ -77,13 +75,13 @@ impl Schedule { println!("name {name}, engine config {engine_configs}"); - let typ = engine_configs + let engine_type = engine_configs .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(typ, engine_configs.clone()).await?; + let engine = load_engine(engine_type, engine_configs.clone()).await?; result.insert(name.clone(), engine); } @@ -91,14 +89,6 @@ impl Schedule { Ok(result) } - async fn parse_catalog() -> anyhow::Result> { - let catalog = RestCatalogBuilder::default() - .load("rest", HashMap::from([])) - .await?; - - Ok(Box::new(catalog)) - } - async fn parse_steps(table: &Table) -> anyhow::Result> { let steps = table .get("steps") diff --git a/crates/sqllogictest/testdata/slts/demo/verify.slt b/crates/sqllogictest/testdata/slts/demo/verify.slt index f8744e2120..7f371d70ce 100644 --- a/crates/sqllogictest/testdata/slts/demo/verify.slt +++ b/crates/sqllogictest/testdata/slts/demo/verify.slt @@ -1,5 +1,5 @@ query I -SELECT * FROM demo.s1.t1 ORDER BY id; +SELECT * FROM default.s1.t1 ORDER BY id; ---- 1 2 diff --git a/crates/sqllogictest/tests/sqllogictests.rs b/crates/sqllogictest/tests/sqllogictests.rs index 4d9353dab8..cda9fb089f 100644 --- a/crates/sqllogictest/tests/sqllogictests.rs +++ b/crates/sqllogictest/tests/sqllogictests.rs @@ -19,7 +19,6 @@ use std::fs; use std::path::PathBuf; use std::sync::RwLock; -use iceberg_sqllogictest::schedule; use iceberg_sqllogictest::schedule::Schedule; use iceberg_test_utils::docker::DockerCompose; use iceberg_test_utils::normalize_test_name; @@ -104,7 +103,6 @@ pub(crate) fn collect_schedule_files() -> anyhow::Result> { } pub(crate) async fn run_schedule(schedule_file: PathBuf) -> anyhow::Result<()> { - let schedule_file_name = schedule_file.file_name().unwrap().to_string_lossy(); let schedules = Schedule::parse(schedule_file).await?; schedules.run().await?;