From aeefb6c2370d00deb9ea44f764daef5f2ba6beb3 Mon Sep 17 00:00:00 2001 From: Anthony Alaribe Date: Sat, 31 Jan 2026 12:18:32 -0800 Subject: [PATCH 1/8] Unify data directory config and add Variant INSERT support - Consolidate WALRUS_DATA_DIR and FOYER_CACHE_DIR into single TIMEFUSION_DATA_DIR with derived subdirs (wal/, cache/) - Add VariantConversionExec to convert string columns to Variant during INSERT - Add VariantInsertRewriter analyzer rule to rewrite string literals for Variant columns - Add is_schema_compatible_for_insert() for flexible INSERT type checking - Split optimizers.rs into optimizers/ module directory - Improve query sanitization for INSERT and long queries --- .env.example | 3 + .env.minio | 2 +- .env.test | 2 +- Makefile | 7 +- src/buffered_write_layer.rs | 8 +- src/config.rs | 18 +- src/database.rs | 152 ++++++++++++++++- src/main.rs | 4 +- src/object_store_cache.rs | 29 ++-- src/{optimizers.rs => optimizers/mod.rs} | 4 + src/optimizers/variant_insert_rewriter.rs | 196 ++++++++++++++++++++++ src/pgwire_handlers.rs | 15 +- src/schema_loader.rs | 18 ++ src/test_utils.rs | 2 +- 14 files changed, 413 insertions(+), 47 deletions(-) rename src/{optimizers.rs => optimizers/mod.rs} (97%) create mode 100644 src/optimizers/variant_insert_rewriter.rs diff --git a/.env.example b/.env.example index cf146bc..a546509 100644 --- a/.env.example +++ b/.env.example @@ -51,3 +51,6 @@ OTEL_EXPORTER_OTLP_PROTOCOL=grpc OTEL_EXPORTER_OTLP_HEADERS= # Optional: Enable/disable tracing (default: true) OTEL_SDK_DISABLED=false + +# Data Directory (WAL stored in {dir}/wal, cache in {dir}/cache) +TIMEFUSION_DATA_DIR=./data diff --git a/.env.minio b/.env.minio index e045e70..de15966 100644 --- a/.env.minio +++ b/.env.minio @@ -24,7 +24,7 @@ MAX_PG_CONNECTIONS=100 AWS_S3_LOCKING_PROVIDER="" # WAL storage directory for walrus-rust -WALRUS_DATA_DIR=/tmp/walrus-wal +TIMEFUSION_DATA_DIR=./data/minio # Foyer cache configuration for tests TIMEFUSION_FOYER_MEMORY_MB=256 diff --git a/.env.test b/.env.test index ffa9e16..46ccbda 100644 --- a/.env.test +++ b/.env.test @@ -39,7 +39,7 @@ TIMEFUSION_VACUUM_RETENTION_HOURS=1 TIMEFUSION_FOYER_MEMORY_MB=64 TIMEFUSION_FOYER_DISK_GB=1 TIMEFUSION_FOYER_TTL_SECONDS=60 -TIMEFUSION_FOYER_CACHE_DIR=/tmp/timefusion_test_cache +TIMEFUSION_DATA_DIR=./data/test TIMEFUSION_FOYER_SHARDS=4 TIMEFUSION_FOYER_FILE_SIZE_MB=8 TIMEFUSION_FOYER_STATS=true diff --git a/Makefile b/Makefile index 3890bf7..3429a30 100644 --- a/Makefile +++ b/Makefile @@ -1,4 +1,4 @@ -.PHONY: test test-all test-ovh test-minio test-minio-all test-prod test-integration test-integration-minio run-prod build-prod minio-start minio-stop minio-clean +.PHONY: test test-all test-ovh test-minio test-minio-all test-prod test-integration test-integration-minio run-prod run-minio build-prod minio-start minio-stop minio-clean # Default test (fast, excludes slow integration tests) test: @@ -40,6 +40,11 @@ build-prod: @echo "Building release with PRODUCTION configuration..." @export $$(cat .env.prod | grep -v '^#' | xargs) && cargo build --release +# Run with MinIO configuration (local development with prod-like settings) +run-minio: + @echo "Running with MinIO configuration..." + @export $$(cat .env.minio.prod | grep -v '^#' | xargs) && cargo run + # Start MinIO server minio-start: @mkdir -p /tmp/minio-data diff --git a/src/buffered_write_layer.rs b/src/buffered_write_layer.rs index 7646048..3d612cd 100644 --- a/src/buffered_write_layer.rs +++ b/src/buffered_write_layer.rs @@ -67,7 +67,7 @@ impl std::fmt::Debug for BufferedWriteLayer { impl BufferedWriteLayer { /// Create a new BufferedWriteLayer with explicit config. pub fn with_config(cfg: Arc) -> anyhow::Result { - let wal = Arc::new(WalManager::new(cfg.core.walrus_data_dir.clone())?); + let wal = Arc::new(WalManager::new(cfg.core.wal_dir())?); let mem_buffer = Arc::new(MemBuffer::new()); Ok(Self { @@ -569,9 +569,9 @@ mod tests { use std::path::PathBuf; use tempfile::tempdir; - fn create_test_config(wal_dir: PathBuf) -> Arc { + fn create_test_config(data_dir: PathBuf) -> Arc { let mut cfg = AppConfig::default(); - cfg.core.walrus_data_dir = wal_dir; + cfg.core.timefusion_data_dir = data_dir; Arc::new(cfg) } @@ -613,7 +613,7 @@ mod tests { // SAFETY: walrus-rust reads WALRUS_DATA_DIR from environment. We use #[serial] // to prevent concurrent access to this process-global state. - unsafe { std::env::set_var("WALRUS_DATA_DIR", &cfg.core.walrus_data_dir) }; + unsafe { std::env::set_var("WALRUS_DATA_DIR", cfg.core.wal_dir()) }; // Use unique but short project/table names (walrus has metadata size limit) let test_id = &uuid::Uuid::new_v4().to_string()[..4]; diff --git a/src/config.rs b/src/config.rs index cca0d6b..7d23072 100644 --- a/src/config.rs +++ b/src/config.rs @@ -89,7 +89,7 @@ macro_rules! const_default { // All default value functions using the macro const_default!(d_true: bool = true); const_default!(d_s3_endpoint: String = "https://s3.amazonaws.com"); -const_default!(d_wal_dir: PathBuf = "/var/lib/timefusion/wal"); +const_default!(d_data_dir: PathBuf = "./data"); const_default!(d_pgwire_port: u16 = 5432); const_default!(d_table_prefix: String = "timefusion"); const_default!(d_batch_queue_capacity: usize = 100_000_000); @@ -104,7 +104,6 @@ const_default!(d_flush_parallelism: usize = 4); const_default!(d_foyer_memory_mb: usize = 512); const_default!(d_foyer_disk_gb: usize = 100); const_default!(d_foyer_ttl: u64 = 604_800); // 7 days -const_default!(d_cache_dir: PathBuf = "/tmp/timefusion_cache"); const_default!(d_foyer_shards: usize = 8); const_default!(d_foyer_file_size_mb: usize = 32); const_default!(d_foyer_stats: String = "true"); @@ -219,8 +218,8 @@ impl AwsConfig { #[derive(Debug, Clone, Deserialize)] pub struct CoreConfig { - #[serde(default = "d_wal_dir")] - pub walrus_data_dir: PathBuf, + #[serde(default = "d_data_dir")] + pub timefusion_data_dir: PathBuf, #[serde(default = "d_pgwire_port")] pub pgwire_port: u16, #[serde(default = "d_table_prefix")] @@ -237,6 +236,15 @@ pub struct CoreConfig { pub pgwire_password: Option, } +impl CoreConfig { + pub fn wal_dir(&self) -> PathBuf { + self.timefusion_data_dir.join("wal") + } + pub fn cache_dir(&self) -> PathBuf { + self.timefusion_data_dir.join("cache") + } +} + #[derive(Debug, Clone, Deserialize)] pub struct BufferConfig { #[serde(default = "d_flush_interval")] @@ -295,8 +303,6 @@ pub struct CacheConfig { pub timefusion_foyer_disk_gb: usize, #[serde(default = "d_foyer_ttl")] pub timefusion_foyer_ttl_seconds: u64, - #[serde(default = "d_cache_dir")] - pub timefusion_foyer_cache_dir: PathBuf, #[serde(default = "d_foyer_shards")] pub timefusion_foyer_shards: usize, #[serde(default = "d_foyer_file_size_mb")] diff --git a/src/database.rs b/src/database.rs index 3434c7c..77c606a 100644 --- a/src/database.rs +++ b/src/database.rs @@ -1,6 +1,6 @@ use crate::config::{self, AppConfig}; use crate::object_store_cache::{FoyerCacheConfig, FoyerObjectStoreCache, SharedFoyerCache}; -use crate::schema_loader::{get_default_schema, get_schema, is_variant_type}; +use crate::schema_loader::{create_insert_compatible_schema, get_default_schema, get_schema, is_variant_type}; use crate::statistics::DeltaStatisticsExtractor; use anyhow::Result; use arrow_schema::{Schema, SchemaRef}; @@ -14,8 +14,11 @@ use datafusion::execution::TaskContext; use datafusion::execution::context::SessionContext; use datafusion::logical_expr::{Expr, Operator, TableProviderFilterPushDown}; use datafusion::physical_expr::expressions::{CastExpr, Column as PhysicalColumn}; +use datafusion::physical_plan::stream::RecordBatchStreamAdapter; use datafusion::physical_plan::DisplayAs; use datafusion::physical_plan::projection::ProjectionExec; +use datafusion::physical_plan::{ExecutionPlanProperties, PlanProperties}; +use datafusion::physical_plan::execution_plan::Boundedness; use datafusion::scalar::ScalarValue; use datafusion::{ catalog::Session, @@ -161,6 +164,114 @@ fn json_strings_to_variant<'a>(iter: impl Iterator>) -> D Ok(builder.build().into()) } +/// Check if input schema is compatible with target schema for INSERT operations. +/// This allows string types (Utf8, Utf8View, LargeUtf8) to be inserted into Variant columns, +/// since convert_variant_columns() will handle the conversion in write_all(). +fn is_schema_compatible_for_insert(input_schema: &SchemaRef, target_schema: &SchemaRef) -> DFResult<()> { + use datafusion::arrow::datatypes::DataType; + + if input_schema.fields().len() != target_schema.fields().len() { + return Err(DataFusionError::Plan(format!( + "Schema field count mismatch: input has {} fields, target has {} fields", + input_schema.fields().len(), + target_schema.fields().len() + ))); + } + + for (input_field, target_field) in input_schema.fields().iter().zip(target_schema.fields()) { + let input_type = input_field.data_type(); + let target_type = target_field.data_type(); + + // Same type is always compatible + if input_type == target_type { + continue; + } + + // Allow string types to be inserted into Variant columns + // (convert_variant_columns will handle the conversion) + let is_string_to_variant = matches!( + input_type, + DataType::Utf8 | DataType::Utf8View | DataType::LargeUtf8 + ) && is_variant_type(target_type); + + if is_string_to_variant { + continue; + } + + // Check logical equivalence for other types + if !input_type.equals_datatype(target_type) { + return Err(DataFusionError::Plan(format!( + "Schema mismatch for field '{}': input type {:?} is not compatible with target type {:?}", + input_field.name(), + input_type, + target_type + ))); + } + } + + Ok(()) +} + +/// Custom execution plan that converts string columns to Variant type. +/// This wraps an input plan and transforms string columns to Variant in the output. +#[derive(Debug)] +struct VariantConversionExec { + input: Arc, + target_schema: SchemaRef, + properties: PlanProperties, +} + +impl VariantConversionExec { + fn new(input: Arc, target_schema: SchemaRef) -> Self { + let properties = PlanProperties::new( + datafusion::physical_expr::EquivalenceProperties::new(target_schema.clone()), + input.output_partitioning().clone(), + input.pipeline_behavior(), + Boundedness::Bounded, + ); + Self { input, target_schema, properties } + } +} + +impl DisplayAs for VariantConversionExec { + fn fmt_as(&self, _t: DisplayFormatType, f: &mut fmt::Formatter) -> fmt::Result { + write!(f, "VariantConversionExec") + } +} + +impl ExecutionPlan for VariantConversionExec { + fn name(&self) -> &str { + "VariantConversionExec" + } + + fn as_any(&self) -> &dyn Any { + self + } + + fn properties(&self) -> &PlanProperties { + &self.properties + } + + fn children(&self) -> Vec<&Arc> { + vec![&self.input] + } + + fn with_new_children(self: Arc, children: Vec>) -> DFResult> { + Ok(Arc::new(VariantConversionExec::new(children[0].clone(), self.target_schema.clone()))) + } + + fn execute(&self, partition: usize, context: Arc) -> DFResult { + let input_stream = self.input.execute(partition, context)?; + let target_schema = self.target_schema.clone(); + + let converted_stream = input_stream.map(move |batch_result| { + batch_result.and_then(|batch| convert_variant_columns(batch, &target_schema)) + }); + + Ok(Box::pin(RecordBatchStreamAdapter::new(self.target_schema.clone(), converted_stream))) + } +} + // Compression level for parquet files - kept for WriterProperties fallback const ZSTD_COMPRESSION_LEVEL: i32 = 3; @@ -344,7 +455,7 @@ impl Database { return None; } - let foyer_config = FoyerCacheConfig::from(&cfg.cache); + let foyer_config = FoyerCacheConfig::from_app_config(&cfg); info!( "Initializing shared Foyer hybrid cache (memory: {}MB, disk: {}GB, TTL: {}s)", foyer_config.memory_size_bytes / 1024 / 1024, @@ -747,10 +858,19 @@ impl Database { ); // Create session state with tracing rule and DML support + // IMPORTANT: VariantInsertRewriter must run BEFORE TypeCoercion to rewrite + // string literals into json_to_variant() calls before type checking happens + let analyzer_rules: Vec> = vec![ + Arc::new(datafusion::optimizer::analyzer::resolve_grouping_function::ResolveGroupingFunction::new()), + Arc::new(crate::optimizers::VariantInsertRewriter), + Arc::new(datafusion::optimizer::analyzer::type_coercion::TypeCoercion::new()), + ]; + let session_state = SessionStateBuilder::new() .with_config(options.into()) .with_runtime_env(runtime_env) .with_default_features() + .with_analyzer_rules(analyzer_rules) .with_physical_optimizer_rule(instrument_rule) .with_query_planner(Arc::new({ let planner = DmlQueryPlanner::new(self.clone()); @@ -1652,6 +1772,14 @@ impl ProjectRoutingTable { } fn schema(&self) -> SchemaRef { + // Return INSERT-compatible schema where Variant columns appear as Utf8View. + // This allows INSERT statements with JSON strings to pass DataFusion's type validation. + // VariantConversionExec handles the actual string->Variant conversion during write. + create_insert_compatible_schema(&self.schema) + } + + /// Return the actual schema with Variant types (for internal use) + fn real_schema(&self) -> SchemaRef { self.schema.clone() } @@ -2039,10 +2167,10 @@ impl TableProvider for ProjectRoutingTable { } async fn insert_into(&self, _state: &dyn Session, input: Arc, insert_op: InsertOp) -> DFResult> { - // Create a physical plan from the logical plan. - // Check that the schema of the plan matches the schema of this table. - match self.schema().logically_equivalent_names_and_types(&input.schema()) { - Ok(_) => debug!("insert_into; Schema validation passed"), + // Check that the schema of the plan is compatible with this table. + // Use custom compatibility check that allows string -> Variant conversion. + match is_schema_compatible_for_insert(&input.schema(), &self.schema()) { + Ok(_) => debug!("insert_into; Schema validation passed (with Variant compatibility)"), Err(e) => { error!("Schema validation failed: {}", e); return Err(e); @@ -2054,8 +2182,14 @@ impl TableProvider for ProjectRoutingTable { return not_impl_err!("{insert_op} not implemented for MemoryTable yet"); } - // Create sink executor but with additional logging - let sink = DataSinkExec::new(input, Arc::new(self.clone()), None); + // Wrap input with VariantConversionExec to convert string columns to Variant + // before they reach the sink. This prevents DataFusion from trying to cast + // Utf8 -> Struct(Variant) which would fail. + // Use real_schema() to get the actual Variant types for proper conversion. + let converted_input: Arc = Arc::new(VariantConversionExec::new(input, self.real_schema())); + + // Create sink executor with the converted input + let sink = DataSinkExec::new(converted_input, Arc::new(self.clone()), None); Ok(Arc::new(sink)) } @@ -2223,7 +2357,7 @@ mod tests { cfg.aws.aws_allow_http = Some("true".to_string()); // Core settings - unique per test cfg.core.timefusion_table_prefix = format!("test-{}", test_id); - cfg.core.walrus_data_dir = PathBuf::from(format!("/tmp/walrus-db-{}", test_id)); + cfg.core.timefusion_data_dir = PathBuf::from(format!("/tmp/timefusion-db-{}", test_id)); // Disable Foyer cache for tests cfg.cache.timefusion_foyer_disabled = true; Arc::new(cfg) diff --git a/src/main.rs b/src/main.rs index 44ff29f..37095d4 100644 --- a/src/main.rs +++ b/src/main.rs @@ -20,7 +20,7 @@ fn main() -> anyhow::Result<()> { // Set WALRUS_DATA_DIR before Tokio runtime starts (required by walrus-rust) // SAFETY: No threads exist yet - we're before tokio::runtime::Builder - unsafe { std::env::set_var("WALRUS_DATA_DIR", &cfg.core.walrus_data_dir) }; + unsafe { std::env::set_var("WALRUS_DATA_DIR", cfg.core.wal_dir()) }; // Build and run Tokio runtime after env vars are set tokio::runtime::Builder::new_multi_thread().enable_all().build()?.block_on(async_main(cfg)) @@ -42,7 +42,7 @@ async fn async_main(cfg: &'static AppConfig) -> anyhow::Result<()> { // Initialize BufferedWriteLayer with explicit config info!( "BufferedWriteLayer config: wal_dir={:?}, flush_interval={}s, retention={}min", - cfg.core.walrus_data_dir, + cfg.core.wal_dir(), cfg.buffer.flush_interval_secs(), cfg.buffer.retention_mins() ); diff --git a/src/object_store_cache.rs b/src/object_store_cache.rs index 67df578..1506284 100644 --- a/src/object_store_cache.rs +++ b/src/object_store_cache.rs @@ -14,7 +14,6 @@ use std::time::{Duration, SystemTime, UNIX_EPOCH}; use tracing::field::Empty; use tracing::{Instrument, debug, info, instrument}; -use crate::config::CacheConfig; use foyer::{BlockEngineBuilder, DeviceBuilder, FsDeviceBuilder, HybridCache, HybridCacheBuilder, HybridCachePolicy, IoEngineBuilder, PsyncIoEngineBuilder}; use serde::{Deserialize, Serialize}; use tokio::sync::{Mutex, RwLock}; @@ -129,25 +128,23 @@ impl Default for FoyerCacheConfig { } } -impl From<&CacheConfig> for FoyerCacheConfig { - fn from(cfg: &CacheConfig) -> Self { +impl FoyerCacheConfig { + pub fn from_app_config(cfg: &crate::config::AppConfig) -> Self { Self { - memory_size_bytes: cfg.memory_size_bytes(), - disk_size_bytes: cfg.disk_size_bytes(), - ttl: cfg.ttl(), - cache_dir: cfg.timefusion_foyer_cache_dir.clone(), - shards: cfg.timefusion_foyer_shards, - file_size_bytes: cfg.file_size_bytes(), - enable_stats: cfg.stats_enabled(), - parquet_metadata_size_hint: cfg.timefusion_parquet_metadata_size_hint, - metadata_memory_size_bytes: cfg.metadata_memory_size_bytes(), - metadata_disk_size_bytes: cfg.metadata_disk_size_bytes(), - metadata_shards: cfg.timefusion_foyer_metadata_shards, + memory_size_bytes: cfg.cache.memory_size_bytes(), + disk_size_bytes: cfg.cache.disk_size_bytes(), + ttl: cfg.cache.ttl(), + cache_dir: cfg.core.cache_dir(), + shards: cfg.cache.timefusion_foyer_shards, + file_size_bytes: cfg.cache.file_size_bytes(), + enable_stats: cfg.cache.stats_enabled(), + parquet_metadata_size_hint: cfg.cache.timefusion_parquet_metadata_size_hint, + metadata_memory_size_bytes: cfg.cache.metadata_memory_size_bytes(), + metadata_disk_size_bytes: cfg.cache.metadata_disk_size_bytes(), + metadata_shards: cfg.cache.timefusion_foyer_metadata_shards, } } -} -impl FoyerCacheConfig { /// Create a test configuration with sensible defaults for testing /// The name parameter is used to create unique cache directories pub fn test_config(name: &str) -> Self { diff --git a/src/optimizers.rs b/src/optimizers/mod.rs similarity index 97% rename from src/optimizers.rs rename to src/optimizers/mod.rs index ea04e54..af47243 100644 --- a/src/optimizers.rs +++ b/src/optimizers/mod.rs @@ -1,3 +1,7 @@ +mod variant_insert_rewriter; + +pub use variant_insert_rewriter::VariantInsertRewriter; + use datafusion::logical_expr::{BinaryExpr, Expr, Operator}; use datafusion::scalar::ScalarValue; diff --git a/src/optimizers/variant_insert_rewriter.rs b/src/optimizers/variant_insert_rewriter.rs new file mode 100644 index 0000000..0acfb52 --- /dev/null +++ b/src/optimizers/variant_insert_rewriter.rs @@ -0,0 +1,196 @@ +use std::collections::HashSet; +use std::sync::Arc; + +use datafusion::{ + common::{Result, tree_node::{Transformed, TreeNode}}, + config::ConfigOptions, + logical_expr::{ + DmlStatement, Expr, LogicalPlan, Projection, Values, WriteOp, + expr::ScalarFunction, + }, + optimizer::AnalyzerRule, + scalar::ScalarValue, +}; +use datafusion_variant::JsonToVariantUdf; +use tracing::debug; + +use crate::schema_loader::is_variant_type; + +/// AnalyzerRule that rewrites INSERT statements to wrap Utf8 expressions +/// going into Variant columns with `json_to_variant()`. +/// +/// This is necessary because DataFusion's type checker rejects Utf8 -> Variant(Struct) +/// casts before our custom VariantConversionExec can run. +#[derive(Debug, Default)] +pub struct VariantInsertRewriter; + +impl AnalyzerRule for VariantInsertRewriter { + fn name(&self) -> &str { + "variant_insert_rewriter" + } + + fn analyze(&self, plan: LogicalPlan, _config: &ConfigOptions) -> Result { + plan.transform_up(|node| rewrite_insert_node(node)).map(|t| t.data) + } +} + +fn rewrite_insert_node(plan: LogicalPlan) -> Result> { + if let LogicalPlan::Dml(dml) = &plan { + if !matches!(dml.op, WriteOp::Insert(_)) { + return Ok(Transformed::no(plan)); + } + + debug!("VariantInsertRewriter: INSERT into {}", dml.table_name); + + // Get target table schema to find variant column names + let target_schema = dml.target.schema(); + let variant_column_names: HashSet = target_schema + .fields() + .iter() + .filter(|f| is_variant_type(f.data_type())) + .map(|f| f.name().clone()) + .collect(); + + if variant_column_names.is_empty() { + return Ok(Transformed::no(plan)); + } + + // Get input schema to find which positions correspond to variant columns + let input_schema = dml.input.schema(); + + + let variant_indices: Vec = input_schema + .fields() + .iter() + .enumerate() + .filter(|(_, f)| variant_column_names.contains(f.name())) + .map(|(i, _)| i) + .collect(); + + + if variant_indices.is_empty() { + return Ok(Transformed::no(plan)); + } + + debug!( + "VariantInsertRewriter: Found {} variant columns in INSERT: {:?}", + variant_indices.len(), + input_schema.fields().iter().enumerate() + .filter(|(i, _)| variant_indices.contains(i)) + .map(|(_, f)| f.name()) + .collect::>() + ); + + let new_input = rewrite_input_for_variant(&dml.input, &variant_indices)?; + + if let Some(new_input) = new_input { + let new_dml = LogicalPlan::Dml(DmlStatement { + op: dml.op.clone(), + table_name: dml.table_name.clone(), + target: dml.target.clone(), + input: Arc::new(new_input), + output_schema: dml.output_schema.clone(), + }); + return Ok(Transformed::yes(new_dml)); + } + } + Ok(Transformed::no(plan)) +} + +fn rewrite_input_for_variant(input: &LogicalPlan, variant_indices: &[usize]) -> Result> { + match input { + LogicalPlan::Values(values) => rewrite_values_for_variant(values, variant_indices), + LogicalPlan::Projection(proj) => rewrite_projection_for_variant(proj, variant_indices), + _ => { + if let Some(child) = input.inputs().first() { + if let Some(new_child) = rewrite_input_for_variant(child, variant_indices)? { + let new_inputs = vec![new_child]; + Ok(Some(input.with_new_exprs(input.expressions(), new_inputs)?)) + } else { + Ok(None) + } + } else { + Ok(None) + } + } + } +} + +fn rewrite_values_for_variant(values: &Values, variant_indices: &[usize]) -> Result> { + let json_to_variant_udf = Arc::new(datafusion::logical_expr::ScalarUDF::from(JsonToVariantUdf::default())); + let mut modified = false; + + let new_rows: Vec> = values + .values + .iter() + .map(|row| { + row.iter() + .enumerate() + .map(|(idx, expr)| { + if variant_indices.contains(&idx) && is_utf8_expr(expr) { + modified = true; + wrap_with_json_to_variant(expr, &json_to_variant_udf) + } else { + expr.clone() + } + }) + .collect() + }) + .collect(); + + if modified { + Ok(Some(LogicalPlan::Values(Values { + schema: values.schema.clone(), + values: new_rows, + }))) + } else { + Ok(None) + } +} + +fn rewrite_projection_for_variant(proj: &Projection, variant_indices: &[usize]) -> Result> { + let json_to_variant_udf = Arc::new(datafusion::logical_expr::ScalarUDF::from(JsonToVariantUdf::default())); + let mut modified = false; + + let new_exprs: Vec = proj + .expr + .iter() + .enumerate() + .map(|(idx, expr)| { + if variant_indices.contains(&idx) && is_utf8_expr(expr) { + modified = true; + wrap_with_json_to_variant(expr, &json_to_variant_udf) + } else { + expr.clone() + } + }) + .collect(); + + if modified { + let new_input = rewrite_input_for_variant(&proj.input, variant_indices)?; + let input = new_input.map(Arc::new).unwrap_or_else(|| proj.input.clone()); + Ok(Some(LogicalPlan::Projection(Projection::try_new(new_exprs, input)?))) + } else { + let new_input = rewrite_input_for_variant(&proj.input, variant_indices)?; + if let Some(new_input) = new_input { + Ok(Some(LogicalPlan::Projection(Projection::try_new(proj.expr.clone(), Arc::new(new_input))?))) + } else { + Ok(None) + } + } +} + +fn is_utf8_expr(expr: &Expr) -> bool { + match expr { + Expr::Literal(ScalarValue::Utf8(_), _) | Expr::Literal(ScalarValue::Utf8View(_), _) | Expr::Literal(ScalarValue::LargeUtf8(_), _) => true, + Expr::Cast(cast) => is_utf8_expr(&cast.expr), + _ => false, + } +} + +fn wrap_with_json_to_variant(expr: &Expr, udf: &Arc) -> Expr { + Expr::ScalarFunction(ScalarFunction { + func: udf.clone(), + args: vec![expr.clone()], + }) +} diff --git a/src/pgwire_handlers.rs b/src/pgwire_handlers.rs index a578431..5eb41c0 100644 --- a/src/pgwire_handlers.rs +++ b/src/pgwire_handlers.rs @@ -141,11 +141,16 @@ fn classify_query(query: &str) -> (&'static str, &'static str) { } fn sanitize_query(query: &str, operation: &str) -> String { + const MAX_LEN: usize = 120; let lower = query.to_lowercase(); match operation { - "INSERT" => lower.find(" values").map(|i| format!("{} VALUES ...", &query[..i])).unwrap_or_else(|| query.into()), - "UPDATE" => lower.find(" set").map(|i| format!("{} SET ...", &query[..i])).unwrap_or_else(|| query.into()), - _ => query.into(), + "INSERT" => { + let table_end = lower.find('(').or_else(|| lower.find("values")).unwrap_or(lower.len()); + let table_part = query[..table_end].trim_end(); + format!("{} (...) VALUES ...", table_part) + } + "UPDATE" => lower.find(" set ").map(|i| format!("{} SET ...", &query[..i])).unwrap_or_else(|| query.into()), + _ => if query.len() > MAX_LEN { format!("{}...", &query[..MAX_LEN]) } else { query.into() }, } } @@ -181,9 +186,7 @@ pub struct LoggingExtendedQueryHandler { impl LoggingExtendedQueryHandler { pub fn new(session_context: Arc) -> Self { - Self { - inner: DfSessionService::new(session_context), - } + Self { inner: DfSessionService::new(session_context) } } } diff --git a/src/schema_loader.rs b/src/schema_loader.rs index 9a051fe..f7b32de 100644 --- a/src/schema_loader.rs +++ b/src/schema_loader.rs @@ -206,3 +206,21 @@ pub fn is_variant_type(data_type: &ArrowDataType) -> bool { pub fn get_variant_column_indices(schema: &SchemaRef) -> Vec { schema.fields().iter().enumerate().filter(|(_, f)| is_variant_type(f.data_type())).map(|(i, _)| i).collect() } + +/// Create an INSERT-compatible schema where Variant columns are presented as Utf8View. +/// This allows INSERT statements with JSON strings to pass DataFusion's type validation. +/// The actual conversion from Utf8View to Variant happens in VariantConversionExec during write. +pub fn create_insert_compatible_schema(schema: &SchemaRef) -> SchemaRef { + let new_fields: Vec = schema + .fields() + .iter() + .map(|f| { + if is_variant_type(f.data_type()) { + Arc::new(Field::new(f.name(), ArrowDataType::Utf8View, f.is_nullable())) + } else { + f.clone() + } + }) + .collect(); + Arc::new(Schema::new(new_fields)) +} diff --git a/src/test_utils.rs b/src/test_utils.rs index f7aa816..aed2781 100644 --- a/src/test_utils.rs +++ b/src/test_utils.rs @@ -54,7 +54,7 @@ pub mod test_helpers { cfg.aws.aws_default_region = Some("us-east-1".to_string()); cfg.aws.aws_allow_http = Some("true".to_string()); cfg.core.timefusion_table_prefix = format!("test-{}-{}", self.test_name, uuid); - cfg.core.walrus_data_dir = PathBuf::from(format!("/tmp/walrus-{}-{}", self.test_name, uuid)); + cfg.core.timefusion_data_dir = PathBuf::from(format!("/tmp/timefusion-{}-{}", self.test_name, uuid)); cfg.cache.timefusion_foyer_disabled = true; cfg.buffer.timefusion_flush_immediately = self.buffer_mode == BufferMode::FlushImmediately; Arc::new(cfg) From c8eb41c608541ece3a4387755e50e4ce9f176c8a Mon Sep 17 00:00:00 2001 From: Anthony Alaribe Date: Sun, 1 Feb 2026 13:12:51 -0800 Subject: [PATCH 2/8] Add Variant SELECT rewriter and comprehensive architecture docs - Add VariantSelectRewriter analyzer rule to wrap Variant columns with variant_to_json() in SELECT projections for PostgreSQL wire protocol - Add comprehensive documentation: - docs/ARCHITECTURE.md: Full system architecture overview - docs/VARIANT_TYPE_SYSTEM.md: Variant type implementation details - docs/WAL.md: Write-ahead log implementation and recovery - Update database.rs with unified table storage model improvements - Update DML operations with buffered layer integration - Align otel_logs_and_spans schema with monoscope - Fix test configurations for new architecture --- docs/ARCHITECTURE.md | 353 ++++++++++++ docs/VARIANT_TYPE_SYSTEM.md | 237 ++++++++ docs/WAL.md | 322 +++++++++++ schemas/otel_logs_and_spans.yaml | 1 + src/database.rs | 673 ++++++++++++++-------- src/dml.rs | 17 +- src/optimizers/mod.rs | 4 + src/optimizers/variant_insert_rewriter.rs | 33 +- src/optimizers/variant_select_rewriter.rs | 80 +++ tests/buffer_consistency_test.rs | 2 +- tests/integration_test.rs | 11 +- tests/slt/json_functions.slt | 6 +- tests/test_dml_operations.rs | 2 +- 13 files changed, 1475 insertions(+), 266 deletions(-) create mode 100644 docs/ARCHITECTURE.md create mode 100644 docs/VARIANT_TYPE_SYSTEM.md create mode 100644 docs/WAL.md create mode 100644 src/optimizers/variant_select_rewriter.rs diff --git a/docs/ARCHITECTURE.md b/docs/ARCHITECTURE.md new file mode 100644 index 0000000..e119fa4 --- /dev/null +++ b/docs/ARCHITECTURE.md @@ -0,0 +1,353 @@ +# TimeFusion Architecture + +This document provides a comprehensive overview of TimeFusion's architecture, covering all major subsystems and their interactions. + +## System Overview + +TimeFusion is a time-series database that combines: +- **Apache DataFusion**: Vectorized SQL query engine +- **Delta Lake**: ACID transactional storage on S3 +- **PostgreSQL Wire Protocol**: Client compatibility via `datafusion-postgres` +- **Buffered Write Layer**: Sub-second write latency with WAL + MemBuffer + +``` +┌─────────────────────────────────────────────────────────────────────────────┐ +│ PostgreSQL Clients │ +│ (psql, pgAdmin, any PostgreSQL driver) │ +└─────────────────────────────────────────────────────────────────────────────┘ + │ + ▼ +┌─────────────────────────────────────────────────────────────────────────────┐ +│ PGWire Protocol Layer │ +│ (datafusion-postgres crate) │ +└─────────────────────────────────────────────────────────────────────────────┘ + │ + ▼ +┌─────────────────────────────────────────────────────────────────────────────┐ +│ DataFusion Query Engine │ +│ ┌─────────────────┐ ┌─────────────────┐ ┌─────────────────────────────┐ │ +│ │AnalyzerRules │ │PhysicalPlanner │ │ExpressionPlanner │ │ +│ │• VariantInsert │ │• DmlQueryPlanner│ │• VariantAwareExprPlanner │ │ +│ │• VariantSelect │ │ │ │ (-> and ->> operators) │ │ +│ └─────────────────┘ └─────────────────┘ └─────────────────────────────┘ │ +└─────────────────────────────────────────────────────────────────────────────┘ + │ + ┌──────────────────────────┼──────────────────────────┐ + ▼ ▼ ▼ +┌─────────────────────┐ ┌─────────────────────┐ ┌─────────────────────────┐ +│ Buffered Write Layer│ │ Object Store Cache │ │ Delta Lake │ +│ ┌───────────────┐ │ │ (Foyer) │ │ on S3 │ +│ │ WAL │ │ │ ┌─────────────┐ │ │ ┌─────────────────┐ │ +│ │ (walrus- │ │ │ │ L1: Memory │ │ │ │ Parquet Files │ │ +│ │ rust) │ │ │ │ (512MB) │ │ │ │ + Delta Log │ │ +│ └───────────────┘ │ │ └─────────────┘ │ │ └─────────────────┘ │ +│ ┌───────────────┐ │ │ ┌─────────────┐ │ │ │ +│ │ MemBuffer │ │ │ │ L2: Disk │ │ │ Partitioned by: │ +│ │ (10-min │ │ │ │ (100GB) │ │ │ • project_id │ +│ │ buckets) │ │ │ └─────────────┘ │ │ • date │ +│ └───────────────┘ │ │ │ │ │ +└─────────────────────┘ └─────────────────────┘ └─────────────────────────┘ +``` + +## Module Structure + +``` +src/ +├── main.rs # Entry point, server startup +├── lib.rs # Module exports +├── config.rs # OnceLock singleton +├── database.rs # Core DB engine (~2600 lines) +├── buffered_write_layer.rs # Orchestrates WAL + MemBuffer +├── mem_buffer.rs # In-memory storage with time buckets +├── wal.rs # Write-ahead log (walrus-rust) +├── object_store_cache.rs # Foyer L1/L2 hybrid cache +├── dml.rs # UPDATE/DELETE interception +├── functions.rs # Custom SQL functions + VariantAwareExprPlanner +├── schema_loader.rs # YAML schema registry (compile-time embedded) +├── pgwire_handlers.rs # PostgreSQL protocol handlers +├── batch_queue.rs # Queue for batch insert operations +├── statistics.rs # Delta statistics extraction +├── telemetry.rs # OpenTelemetry integration +├── test_utils.rs # Testing utilities +└── optimizers/ + ├── mod.rs # Optimizer utilities + partition pruning + ├── variant_insert_rewriter.rs # INSERT: Utf8 → json_to_variant() + └── variant_select_rewriter.rs # SELECT: Variant → variant_to_json() +``` + +## Data Flow + +### Insert Path + +``` +Client INSERT + │ + ▼ +┌─────────────────────────────────────────────────────────────────┐ +│ 1. PGWire parses SQL │ +│ 2. DataFusion analyzes query │ +│ └── VariantInsertRewriter wraps Utf8→json_to_variant() │ +│ 3. Execute INSERT │ +└─────────────────────────────────────────────────────────────────┘ + │ + ▼ +┌─────────────────────────────────────────────────────────────────┐ +│ BufferedWriteLayer.insert() │ +│ 1. Check memory pressure → early flush if needed │ +│ 2. try_reserve_memory() → atomic CAS with backoff │ +│ 3. WAL.append_batch() → durable write (fsync every 200ms) │ +│ 4. MemBuffer.insert() → fast in-memory write │ +│ 5. release_reservation() │ +└─────────────────────────────────────────────────────────────────┘ + │ + ▼ +Response to client (sub-second latency) +``` + +### Select Path + +``` +Client SELECT + │ + ▼ +┌─────────────────────────────────────────────────────────────────┐ +│ 1. PGWire parses SQL │ +│ 2. DataFusion analyzes query │ +│ └── VariantSelectRewriter wraps Variant→variant_to_json() │ +│ 3. VariantAwareExprPlanner handles -> and ->> operators │ +│ 4. Physical planning │ +└─────────────────────────────────────────────────────────────────┘ + │ + ▼ +┌─────────────────────────────────────────────────────────────────┐ +│ ProjectRoutingTable.scan() │ +│ 1. Extract project_id from WHERE clause (mandatory) │ +│ 2. Get MemBuffer time range │ +│ 3. Determine data sources: │ +│ • Query entirely in MemBuffer? → MemBuffer only │ +│ • Query spans both? → UnionExec(MemBuffer + Delta) │ +│ • No MemBuffer data? → Delta only │ +│ 4. Add time-range exclusion filter for Delta │ +└─────────────────────────────────────────────────────────────────┘ + │ + ▼ +┌─────────────────────────────────────────────────────────────────┐ +│ Execution │ +│ • MemBuffer: query_partitioned() → parallel by time bucket │ +│ • Delta: Parquet scan with partition pruning │ +│ • Object Store Cache: L1/L2 caching of parquet files │ +└─────────────────────────────────────────────────────────────────┘ + │ + ▼ +Result stream → PGWire encoding → Client +``` + +### Flush Path (Background) + +``` +Every 10 minutes (flush_interval_secs) + │ + ▼ +┌─────────────────────────────────────────────────────────────────┐ +│ BufferedWriteLayer.flush_completed_buckets() │ +│ 1. Acquire flush lock │ +│ 2. Get flushable buckets (bucket_id < current_bucket) │ +│ 3. For each bucket (parallel with bounded concurrency): │ +│ a. DeltaWriteCallback → write to Delta Lake │ +│ b. WAL.checkpoint() → mark entries as consumed │ +│ c. MemBuffer.drain_bucket() → free memory │ +└─────────────────────────────────────────────────────────────────┘ +``` + +## Multi-Tenant Storage Model + +### Two Table Types + +1. **Unified Tables**: Default projects share one Delta table per schema + - Partitioned by `[project_id, date]` + - Path: `s3://bucket/timefusion/default/{table_name}/` + +2. **Custom Project Tables**: Isolated tables for specific projects + - Own S3 bucket/path configuration + - Path: `s3://bucket/timefusion/projects/{project_id}/{table_name}/` + +### Routing + +- `WHERE project_id = 'xxx'` is **mandatory** in all queries +- `ProjectIdPushdown` utility validates filters contain project_id +- MemBuffer uses composite key: `(Arc, Arc)` for (project_id, table_name) + +## Key Data Structures + +### MemBuffer Hierarchy + +``` +MemBuffer + └── tables: DashMap> + │ + └── TableBuffer + ├── schema: SchemaRef (immutable) + ├── project_id: Arc + ├── table_name: Arc + └── buckets: DashMap + │ + └── TimeBucket + ├── batches: RwLock> + ├── row_count: AtomicUsize + ├── memory_bytes: AtomicUsize + ├── min_timestamp: AtomicI64 + └── max_timestamp: AtomicI64 +``` + +**Key type:** `TableKey = (Arc, Arc)` - (project_id, table_name) + +**Bucket ID calculation:** `bucket_id = timestamp_micros / (10 * 60 * 1_000_000)` + +### WAL Entry Format + +``` +┌─────────────────────────────────────────────────────────────────┐ +│ WAL_MAGIC: 4 bytes [0x57, 0x41, 0x4C, 0x32] ("WAL2") │ +│ VERSION: 1 byte (128) │ +│ OPERATION: 1 byte (0=Insert, 1=Delete, 2=Update) │ +│ BINCODE_PAYLOAD: WalEntry │ +│ ├── timestamp_micros: i64 │ +│ ├── project_id: String │ +│ ├── table_name: String │ +│ ├── operation: WalOperation │ +│ └── data: Vec │ +│ ├── Insert: CompactBatch (Arrow data without schema) │ +│ ├── Delete: DeletePayload { predicate_sql } │ +│ └── Update: UpdatePayload { predicate_sql, assignments } │ +└─────────────────────────────────────────────────────────────────┘ +``` + +### Configuration (AppConfig) + +```rust +AppConfig { + aws: AwsConfig, // S3/DynamoDB credentials and endpoints + core: CoreConfig, // Data directory, PGWire port, table prefix + buffer: BufferConfig, // Flush intervals, memory limits, WAL settings + cache: CacheConfig, // Foyer cache sizes, TTL + parquet: ParquetConfig, // Compression, row groups, page limits + maintenance: MaintenanceConfig, // Optimize, vacuum schedules + memory: MemoryConfig, // Memory limits and spill settings + telemetry: TelemetryConfig, // OTLP endpoint, service name/version +} +``` + +## Query Transformation Pipeline + +### Analyzer Rules (Before Type Checking) + +1. **VariantInsertRewriter** (`src/optimizers/variant_insert_rewriter.rs`) + - Intercepts `LogicalPlan::Dml` with `WriteOp::Insert` + - Finds columns where target schema has Variant type + - Wraps Utf8/Utf8View literals with `json_to_variant()` UDF + - Applies recursively to Values and Projection nodes + +2. **VariantSelectRewriter** (`src/optimizers/variant_select_rewriter.rs`) + - Intercepts `LogicalPlan::Projection` + - Checks if expression result type is Variant (via `is_variant_type()`) + - Wraps with `variant_to_json()` for PostgreSQL wire protocol + - Preserves column aliases + +### Physical Planner + +- **DmlQueryPlanner** (`src/dml.rs`) + - Intercepts UPDATE/DELETE logical plans + - Extracts table_name, project_id, predicate, assignments + - Creates `DmlExec` physical plan + - Logs to WAL and applies to MemBuffer + +### Expression Planner + +- **VariantAwareExprPlanner** (`src/functions.rs`) + - Handles `->` (get JSON object) and `->>` (get JSON as text) operators + - Converts to `variant_get(col, "path.to.field")` calls + - Builds dot-path strings from nested access patterns + +## Caching Architecture + +### Foyer Hybrid Cache + +``` +┌─────────────────────────────────────────────────────────────────┐ +│ FoyerObjectStoreCache │ +├─────────────────────────────────────────────────────────────────┤ +│ Main Cache (Parquet data files) │ +│ ├── L1: Memory (512MB default) │ +│ └── L2: Disk (100GB default) │ +├─────────────────────────────────────────────────────────────────┤ +│ Metadata Cache (Parquet footers) │ +│ ├── L1: Memory (512MB) │ +│ └── L2: Disk (5GB) │ +├─────────────────────────────────────────────────────────────────┤ +│ Features: │ +│ • TTL-based expiration (7 days default) │ +│ • Implements ObjectStore trait transparently │ +│ • Statistics tracking (hits, misses, expirations) │ +└─────────────────────────────────────────────────────────────────┘ +``` + +## Safety and Durability + +### Memory Management + +- **Reservation system**: Atomic CAS prevents race conditions +- **20% overhead multiplier**: Accounts for Arrow alignment/metadata +- **Hard limit**: `max_bytes + max_bytes/5 = 120%` headroom +- **Exponential backoff**: Reduces CPU thrashing under contention + +### WAL Durability + +- **Fsync schedule**: Every 200ms (configurable) +- **Size limits**: `MAX_BATCH_SIZE = 100MB` prevents unbounded allocation +- **Version detection**: Byte 4 > 2 distinguishes from legacy format +- **Recovery**: On startup, replays entries within retention window + +### Crash Recovery + +``` +Startup + │ + ▼ +┌─────────────────────────────────────────────────────────────────┐ +│ BufferedWriteLayer.recover_from_wal() │ +│ 1. Calculate cutoff = now - retention_mins │ +│ 2. Read all WAL entries (sorted by timestamp) │ +│ 3. For each entry within retention: │ +│ • Insert: Replay to MemBuffer │ +│ • Delete: Apply delete to MemBuffer │ +│ • Update: Apply update to MemBuffer │ +│ 4. Report recovery stats │ +└─────────────────────────────────────────────────────────────────┘ +``` + +## Key Constants + +```rust +// MemBuffer +BUCKET_DURATION_MICROS = 10 * 60 * 1_000_000 // 10 minutes + +// BufferedWriteLayer +MEMORY_OVERHEAD_MULTIPLIER = 1.2 // 20% overhead +HARD_LIMIT_MULTIPLIER = 5 // max + max/5 = 120% +MAX_CAS_RETRIES = 100 +CAS_BACKOFF_BASE_MICROS = 1 + +// WAL +WAL_MAGIC = [0x57, 0x41, 0x4C, 0x32] // "WAL2" +WAL_VERSION = 128 +MAX_BATCH_SIZE = 100 * 1024 * 1024 // 100MB +FSYNC_SCHEDULE_MS = 200 +``` + +## Related Documentation + +- [Buffered Write Layer](buffered-write-layer.md) - Detailed WAL and MemBuffer internals +- [Multi-Table Architecture](MULTI_TABLE_ARCHITECTURE.md) - Multi-tenant table organization +- [Caching](CACHING.md) - Foyer cache configuration +- [Tracing](TRACING.md) - OpenTelemetry integration +- [Delta Checkpoint Handling](DELTA_CHECKPOINT_HANDLING.md) - Delta Lake internals diff --git a/docs/VARIANT_TYPE_SYSTEM.md b/docs/VARIANT_TYPE_SYSTEM.md new file mode 100644 index 0000000..e1355e6 --- /dev/null +++ b/docs/VARIANT_TYPE_SYSTEM.md @@ -0,0 +1,237 @@ +# Variant Type System + +TimeFusion supports Snowflake-style Variant columns for semi-structured JSON data. This document explains how Variant types are implemented and used. + +## Overview + +Variant columns allow storing arbitrary JSON structures without a predefined schema. They're useful for: +- Dynamic attributes that vary between records +- Nested JSON objects from external APIs +- Schema-less data that evolves over time + +## Representation + +Variant is represented as an Arrow Struct with two BinaryView fields: + +``` +Struct { + metadata: BinaryView, // Type information + value: BinaryView, // Serialized data +} +``` + +### Detection + +The `is_variant_type()` function in `schema_loader.rs` identifies Variant columns: + +```rust +pub fn is_variant_type(dt: &DataType) -> bool { + matches!(dt, DataType::Struct(fields) + if fields.len() == 2 + && fields.iter().any(|f| f.name() == "metadata") + && fields.iter().any(|f| f.name() == "value")) +} +``` + +## Schema Definition + +In YAML schema files, Variant columns are defined with type `Variant`: + +```yaml +# schemas/otel_logs_and_spans.yaml +fields: + - name: attributes + type: Variant + nullable: true + - name: resource_attributes + type: Variant + nullable: true +``` + +## Query Transformations + +### INSERT: Automatic Utf8 → Variant Conversion + +When inserting JSON strings into Variant columns, the `VariantInsertRewriter` automatically wraps them with `json_to_variant()`: + +**Before rewrite:** +```sql +INSERT INTO otel_logs_and_spans (project_id, attributes) +VALUES ('proj-1', '{"user": "alice", "action": "login"}'); +``` + +**After rewrite (internal):** +```sql +INSERT INTO otel_logs_and_spans (project_id, attributes) +VALUES ('proj-1', json_to_variant('{"user": "alice", "action": "login"}')); +``` + +The rewriter: +1. Intercepts INSERT DML statements +2. Identifies columns with Variant target type +3. Wraps Utf8/Utf8View literals with `json_to_variant()` UDF +4. Applies recursively to Values and Projection nodes + +### SELECT: Automatic Variant → JSON Conversion + +When selecting Variant columns, the `VariantSelectRewriter` wraps them with `variant_to_json()` for PostgreSQL wire protocol compatibility: + +**Before rewrite:** +```sql +SELECT attributes FROM otel_logs_and_spans WHERE project_id = 'proj-1'; +``` + +**After rewrite (internal):** +```sql +SELECT variant_to_json(attributes) AS attributes FROM otel_logs_and_spans WHERE project_id = 'proj-1'; +``` + +The rewriter: +1. Intercepts Projection nodes +2. Checks if expression result type is Variant +3. Wraps with `variant_to_json()` to output JSON string +4. Preserves original column aliases + +## JSON Path Access Operators + +TimeFusion supports PostgreSQL-style JSON operators for accessing nested values: + +| Operator | Description | Example | +|----------|-------------|---------| +| `->` | Get JSON object at key | `attributes->'user'` | +| `->>` | Get JSON value as text | `attributes->>'user_id'` | + +### Implementation + +The `VariantAwareExprPlanner` (in `functions.rs`) intercepts these operators: + +```rust +// Example: attributes->'user'->'id' becomes: +variant_get(attributes, "user.id") + +// Example: attributes->>'user_id' becomes: +variant_to_json(variant_get(attributes, "user_id")) +``` + +### Usage Examples + +```sql +-- Get nested object +SELECT attributes->'http'->'request' +FROM otel_logs_and_spans +WHERE project_id = 'proj-1'; + +-- Get text value for filtering +SELECT * FROM otel_logs_and_spans +WHERE project_id = 'proj-1' + AND attributes->>'user_id' = 'u_123'; + +-- Access array elements +SELECT attributes->'items'->0 +FROM otel_logs_and_spans +WHERE project_id = 'proj-1'; +``` + +## Variant UDFs + +### json_to_variant(utf8) → Variant + +Converts a JSON string to Variant type: + +```sql +SELECT json_to_variant('{"key": "value"}'); +``` + +### variant_to_json(variant) → Utf8 + +Converts Variant back to JSON string: + +```sql +SELECT variant_to_json(attributes) FROM otel_logs_and_spans; +``` + +### variant_get(variant, path) → Variant + +Extracts a sub-value using dot-notation path: + +```sql +-- Get nested value +SELECT variant_get(attributes, 'user.profile.name'); + +-- Get array element +SELECT variant_get(attributes, 'items[0]'); +``` + +## WAL and Recovery + +Variant data is stored in WAL entries as serialized Arrow data: +- INSERT: `CompactBatch` contains the raw Variant struct data +- No special handling needed - Variant is just a Struct type to Arrow + +On recovery, Variant columns are reconstructed from the WAL entry's schema. + +## Schema Evolution + +Variant columns naturally support schema evolution: +- New JSON fields can be added without schema changes +- Old fields can be removed from new records +- Different records can have different JSON structures + +## Performance Considerations + +### Storage +- Variant data is stored as binary, typically more compact than string JSON +- Parquet compression applies to the underlying BinaryView + +### Query Performance +- `->` and `->>` operators are converted to `variant_get()` calls +- Path access involves parsing and traversing the Variant structure +- For frequently-accessed fields, consider promoting to top-level columns + +### Best Practices +1. Use Variant for truly dynamic data +2. Promote frequently-queried fields to dedicated columns +3. Use `->>` for text comparisons in WHERE clauses +4. Index on top-level columns, not Variant paths + +## Files + +| File | Purpose | +|------|---------| +| `src/schema_loader.rs` | `is_variant_type()` detection, schema parsing | +| `src/optimizers/variant_insert_rewriter.rs` | INSERT Utf8 → Variant rewriting | +| `src/optimizers/variant_select_rewriter.rs` | SELECT Variant → JSON rewriting | +| `src/functions.rs` | `VariantAwareExprPlanner` for `->` and `->>` | +| `datafusion-variant` crate | UDF implementations | + +## Example Session + +```sql +-- Create data with Variant attributes +INSERT INTO otel_logs_and_spans ( + project_id, name, id, timestamp, date, hashes, + attributes +) VALUES ( + 'proj-1', + 'api.request', + '550e8400-e29b-41d4-a716-446655440000', + '2025-01-17 14:25:00', + '2025-01-17', + ARRAY[]::text[], + '{"http": {"method": "POST", "status": 200}, "user": {"id": "u_123", "role": "admin"}}' +); + +-- Query with path access +SELECT + name, + attributes->>'http'->>'method' as http_method, + attributes->>'user'->>'id' as user_id +FROM otel_logs_and_spans +WHERE project_id = 'proj-1' + AND attributes->'http'->>'status' = '200'; + +-- Filter on nested values +SELECT * FROM otel_logs_and_spans +WHERE project_id = 'proj-1' + AND attributes->'user'->>'role' = 'admin'; +``` diff --git a/docs/WAL.md b/docs/WAL.md new file mode 100644 index 0000000..d6f74eb --- /dev/null +++ b/docs/WAL.md @@ -0,0 +1,322 @@ +# Write-Ahead Log (WAL) + +TimeFusion uses a Write-Ahead Log for durability, ensuring data is never lost even if the server crashes before flushing to Delta Lake. + +## Overview + +The WAL is implemented using [walrus-rust](https://github.com/nubskr/walrus/), a topic-based logging library. Every write operation is logged before being applied to the in-memory buffer. + +``` +Client INSERT + │ + ▼ +┌─────────────────┐ ┌─────────────────┐ ┌─────────────────┐ +│ WAL.append() │───▶│ MemBuffer.insert│───▶│ Response │ +│ (durable) │ │ (fast) │ │ to client │ +└─────────────────┘ └─────────────────┘ └─────────────────┘ + │ + │ (async, every 10 min) + ▼ +┌─────────────────┐ ┌─────────────────┐ +│ Delta Lake │───▶│ WAL.checkpoint()│ +│ write │ │ (mark consumed) │ +└─────────────────┘ └─────────────────┘ +``` + +## Entry Format + +### Wire Format + +``` +┌──────────────────────────────────────────────────────────────┐ +│ Byte 0-3: WAL_MAGIC [0x57, 0x41, 0x4C, 0x32] ("WAL2") │ +│ Byte 4: VERSION (128) │ +│ Byte 5: OPERATION (0=Insert, 1=Delete, 2=Update) │ +│ Byte 6+: BINCODE_PAYLOAD (WalEntry) │ +└──────────────────────────────────────────────────────────────┘ +``` + +### WalEntry Structure + +```rust +#[derive(Debug, Encode, Decode)] +pub struct WalEntry { + pub timestamp_micros: i64, + pub project_id: String, + pub table_name: String, + pub operation: WalOperation, + pub data: Vec, +} + +#[derive(Debug, Clone, Copy, PartialEq, Eq, Encode, Decode)] +pub enum WalOperation { + Insert = 0, + Delete = 1, + Update = 2, +} +``` + +### Data Payloads + +**Insert**: `CompactBatch` (Arrow data without schema) +```rust +struct CompactBatch { + num_rows: usize, + columns: Vec, +} + +struct CompactColumn { + null_bitmap: Option>, + buffers: Vec>, + children: Vec, + null_count: usize, + child_lens: Vec, +} +``` + +**Delete**: +```rust +struct DeletePayload { + predicate_sql: Option, +} +``` + +**Update**: +```rust +struct UpdatePayload { + predicate_sql: Option, + assignments: Vec<(String, String)>, // (column, value_sql) +} +``` + +## Topic Partitioning + +Each (project_id, table_name) combination gets its own WAL topic: + +- **Human-readable topic**: `{project_id}:{table_name}` +- **Walrus key**: 16-character hex hash (walrus has 62-byte metadata limit) + +```rust +fn walrus_topic_key(project_id: &str, table_name: &str) -> String { + let mut hasher = AHasher::default(); + project_id.hash(&mut hasher); + table_name.hash(&mut hasher); + format!("{:016x}", hasher.finish()) +} +``` + +Topics are persisted to `.timefusion_meta/topics` for discovery on startup. + +## Operations + +### Append + +```rust +// Single batch +wal.append(project_id, table_name, &batch)?; + +// Multiple batches (more efficient) +wal.append_batch(project_id, table_name, &batches)?; + +// DML operations +wal.append_delete(project_id, table_name, predicate_sql)?; +wal.append_update(project_id, table_name, predicate_sql, &assignments)?; +``` + +### Read + +```rust +// Read entries for a specific table +let (entries, error_count) = wal.read_entries_raw( + project_id, + table_name, + Some(cutoff_timestamp), // Filter old entries + checkpoint, // Mark as consumed? +)?; + +// Read all entries across all tables +let (entries, error_count) = wal.read_all_entries_raw( + Some(cutoff_timestamp), + checkpoint, +)?; +``` + +### Checkpoint + +After successful Delta Lake flush, mark WAL entries as consumed: + +```rust +wal.checkpoint(project_id, table_name)?; +``` + +This removes the entries from the WAL, preventing replay on next startup. + +## Recovery + +On startup, the system replays WAL entries within the retention window: + +```rust +pub async fn recover_from_wal(&self) -> anyhow::Result { + let retention_micros = (retention_mins as i64) * 60 * 1_000_000; + let cutoff = now() - retention_micros; + + let (entries, error_count) = self.wal.read_all_entries_raw(Some(cutoff), true)?; + + // Fail if corruption exceeds threshold + if corruption_threshold > 0 && error_count >= corruption_threshold { + anyhow::bail!("WAL corruption threshold exceeded"); + } + + for entry in entries { + match entry.operation { + WalOperation::Insert => { + let batch = WalManager::deserialize_batch(&entry.data, &entry.table_name)?; + self.mem_buffer.insert(&entry.project_id, &entry.table_name, batch, entry.timestamp_micros)?; + } + WalOperation::Delete => { + let payload = deserialize_delete_payload(&entry.data)?; + self.mem_buffer.delete_by_sql(&entry.project_id, &entry.table_name, payload.predicate_sql.as_deref())?; + } + WalOperation::Update => { + let payload = deserialize_update_payload(&entry.data)?; + self.mem_buffer.update_by_sql(&entry.project_id, &entry.table_name, payload.predicate_sql.as_deref(), &payload.assignments)?; + } + } + } + + Ok(RecoveryStats { ... }) +} +``` + +## Safety Features + +### Size Limits + +```rust +const MAX_BATCH_SIZE: usize = 100 * 1024 * 1024; // 100MB +``` + +Prevents unbounded memory allocation from corrupted or malicious WAL data. + +### Version Detection + +The version byte (128) is greater than any valid operation byte (0-2), allowing safe format detection: + +```rust +fn deserialize_wal_entry(data: &[u8]) -> Result { + if data[0..4] == WAL_MAGIC { + if data[4] > 2 { + // New format: version byte + operation byte + let version = data[4]; + let operation = data[5]; + // ... + } else { + // Legacy v0: magic + operation byte only + let operation = data[4]; + // ... + } + } else { + // Ancient format: no magic header + // ... + } +} +``` + +### Fsync Schedule + +```rust +const FSYNC_SCHEDULE_MS: u64 = 200; + +Walrus::with_consistency_and_schedule( + ReadConsistency::StrictlyAtOnce, + FsyncSchedule::Milliseconds(FSYNC_SCHEDULE_MS) +)?; +``` + +Balances durability (200ms max data loss window) with performance. + +### Corruption Threshold + +The `wal_corruption_threshold` config controls failure behavior: +- `0`: Disabled (continue despite corruption) +- `>0`: Fail if error_count >= threshold + +## Configuration + +| Environment Variable | Default | Description | +|---------------------|---------|-------------| +| `TIMEFUSION_DATA_DIR` | `./data` | Base directory containing WAL | +| `TIMEFUSION_BUFFER_RETENTION_MINS` | `70` | Entries older than this are skipped on recovery | +| `TIMEFUSION_WAL_CORRUPTION_THRESHOLD` | `0` | Max errors before failing recovery | + +WAL directory: `{TIMEFUSION_DATA_DIR}/wal` + +## File Structure + +``` +data/ +└── wal/ + ├── {walrus_topic_key_1}/ + │ └── ... (walrus internal files) + ├── {walrus_topic_key_2}/ + │ └── ... + └── .timefusion_meta/ + └── topics # Line-separated topic names +``` + +## Performance Characteristics + +| Operation | Latency | Notes | +|-----------|---------|-------| +| `append()` | ~1ms | Includes fsync if schedule triggers | +| `append_batch()` | ~1ms total | Amortizes fsync across batches | +| `read_entries_raw()` | O(n) | Reads all entries for topic | +| `checkpoint()` | O(n) | Marks all entries as consumed | + +## Best Practices + +1. **Use batch append**: Reduces fsync overhead +2. **Set appropriate retention**: Balance recovery time vs. disk usage +3. **Monitor corruption**: Set threshold > 0 in production +4. **Regular checkpointing**: Happens automatically after Delta flush + +## Tradeoffs + +### Chosen: Topic-per-table + +**Pros:** +- Parallel read/write per table +- Independent checkpointing +- Smaller recovery scope per table + +**Cons:** +- More files on disk +- Topic discovery overhead on startup + +### Chosen: 200ms Fsync Schedule + +**Pros:** +- Good balance of durability and performance +- Max 200ms data loss on crash +- Batches multiple writes into single fsync + +**Cons:** +- Not immediately durable (fsync not per-write) +- Some data loss possible on crash + +### Chosen: CompactBatch (No Schema) + +**Pros:** +- Smaller WAL entries +- Schema reconstructed from registry + +**Cons:** +- Requires schema registry at recovery time +- Schema changes need careful handling + +## Files + +| File | Purpose | +|------|---------| +| `src/wal.rs` | WalManager implementation | +| `src/buffered_write_layer.rs` | WAL integration with buffer | diff --git a/schemas/otel_logs_and_spans.yaml b/schemas/otel_logs_and_spans.yaml index 4b381e4..f3ebe86 100644 --- a/schemas/otel_logs_and_spans.yaml +++ b/schemas/otel_logs_and_spans.yaml @@ -1,5 +1,6 @@ table_name: otel_logs_and_spans partitions: + - project_id - date sorting_columns: [] z_order_columns: diff --git a/src/database.rs b/src/database.rs index 77c606a..91cc7d0 100644 --- a/src/database.rs +++ b/src/database.rs @@ -8,7 +8,7 @@ use async_trait::async_trait; use chrono::Utc; use datafusion::arrow::array::Array; use datafusion::common::not_impl_err; -use datafusion::common::{SchemaExt, Statistics}; +use datafusion::common::Statistics; use datafusion::datasource::sink::{DataSink, DataSinkExec}; use datafusion::execution::TaskContext; use datafusion::execution::context::SessionContext; @@ -59,13 +59,22 @@ fn env_mutex() -> &'static Mutex<()> { ENV_MUTEX.get_or_init(|| Mutex::new(())) } -// Changed to support multiple tables per project: (project_id, table_name) -> DeltaTable -pub type ProjectConfigs = Arc>>>>; +// Unified tables: one Delta table per schema (table_name -> DeltaTable) +// All default projects share the same table, with project_id as a partition column +pub type UnifiedTables = Arc>>>>; -/// Get a Delta table by project_id and table_name -pub async fn get_delta_table(project_configs: &ProjectConfigs, project_id: &str, table_name: &str) -> Option>> { - let table_key = (project_id.to_string(), table_name.to_string()); - project_configs.read().await.get(&table_key).cloned() +// Custom project tables: projects with their own S3 bucket get isolated tables +// Key: (project_id, table_name) -> DeltaTable +pub type CustomProjectTables = Arc>>>>; + +/// Get a Delta table from custom project tables by project_id and table_name +pub async fn get_custom_delta_table(custom_tables: &CustomProjectTables, project_id: &str, table_name: &str) -> Option>> { + custom_tables.read().await.get(&(project_id.to_string(), table_name.to_string())).cloned() +} + +/// Get a Delta table from unified tables by table_name +pub async fn get_unified_delta_table(unified_tables: &UnifiedTables, table_name: &str) -> Option>> { + unified_tables.read().await.get(table_name).cloned() } // Helper function to extract project_id from a batch @@ -164,6 +173,128 @@ fn json_strings_to_variant<'a>(iter: impl Iterator>) -> D Ok(builder.build().into()) } +/// Convert Variant columns to JSON strings for SELECT output. +/// This enables pgwire to properly encode Variant data as JSON text. +pub fn variant_columns_to_json(batch: RecordBatch, real_schema: &SchemaRef) -> DFResult { + use datafusion::arrow::array::{ArrayRef, StructArray}; + use datafusion::arrow::datatypes::{DataType, Field}; + + let batch_schema = batch.schema(); + let mut columns: Vec = batch.columns().to_vec(); + let mut new_fields: Vec> = batch_schema.fields().iter().cloned().collect(); + + // Iterate over batch columns (which may be projected) and look up by name in real schema + for (idx, batch_field) in batch_schema.fields().iter().enumerate() { + let is_variant = real_schema + .column_with_name(batch_field.name()) + .is_some_and(|(_, f)| is_variant_type(f.data_type())); + if !is_variant { + continue; + } + + let col = &columns[idx]; + if let Some(struct_arr) = col.as_any().downcast_ref::() { + let json_arr = variant_struct_to_json(struct_arr)?; + columns[idx] = Arc::new(json_arr); + new_fields[idx] = Arc::new(Field::new(batch_field.name(), DataType::Utf8, batch_field.is_nullable())); + } + } + + let new_schema = Arc::new(Schema::new(new_fields)); + RecordBatch::try_new(new_schema, columns).map_err(|e| DataFusionError::ArrowError(Box::new(e), None)) +} + +/// Convert a Variant StructArray to a StringArray of JSON values. +fn variant_struct_to_json(arr: &datafusion::arrow::array::StructArray) -> DFResult { + use datafusion::arrow::array::StringBuilder; + use parquet_variant_compute::VariantArray; + use parquet_variant_json::VariantToJson; + + let variant_arr = VariantArray::try_new(arr) + .map_err(|e| DataFusionError::Execution(format!("Failed to create VariantArray: {}", e)))?; + + let mut builder = StringBuilder::new(); + for i in 0..variant_arr.len() { + if variant_arr.is_null(i) { + builder.append_null(); + } else { + let variant = variant_arr.value(i); + let json = variant.to_json_string() + .map_err(|e| DataFusionError::Execution(format!("Failed to convert variant to JSON: {}", e)))?; + builder.append_value(&json); + } + } + Ok(builder.finish()) +} + +/// Custom execution plan that converts Variant columns to JSON strings for SELECT. +#[derive(Debug)] +struct VariantToJsonExec { + input: Arc, + real_schema: SchemaRef, + output_schema: SchemaRef, + properties: PlanProperties, +} + +impl VariantToJsonExec { + fn new(input: Arc, real_schema: SchemaRef) -> Self { + use datafusion::arrow::datatypes::{DataType, Field}; + // Output schema: for each column in input, convert Variant to Utf8 + let input_schema = input.schema(); + let output_fields: Vec> = input_schema + .fields() + .iter() + .map(|f| { + let is_variant = real_schema + .column_with_name(f.name()) + .is_some_and(|(_, rf)| is_variant_type(rf.data_type())); + if is_variant { + Arc::new(Field::new(f.name(), DataType::Utf8, f.is_nullable())) + } else { + f.clone() + } + }) + .collect(); + let output_schema = Arc::new(Schema::new(output_fields)); + let properties = PlanProperties::new( + datafusion::physical_expr::EquivalenceProperties::new(output_schema.clone()), + input.output_partitioning().clone(), + input.pipeline_behavior(), + Boundedness::Bounded, + ); + Self { input, real_schema, output_schema, properties } + } +} + +impl DisplayAs for VariantToJsonExec { + fn fmt_as(&self, _t: DisplayFormatType, f: &mut fmt::Formatter) -> fmt::Result { + write!(f, "VariantToJsonExec") + } +} + +impl ExecutionPlan for VariantToJsonExec { + fn name(&self) -> &str { "VariantToJsonExec" } + fn as_any(&self) -> &dyn Any { self } + fn properties(&self) -> &PlanProperties { &self.properties } + fn children(&self) -> Vec<&Arc> { vec![&self.input] } + + fn with_new_children(self: Arc, children: Vec>) -> DFResult> { + Ok(Arc::new(VariantToJsonExec::new(children[0].clone(), self.real_schema.clone()))) + } + + fn execute(&self, partition: usize, context: Arc) -> DFResult { + let input_stream = self.input.execute(partition, context)?; + let real_schema = self.real_schema.clone(); + let output_schema = self.output_schema.clone(); + + let converted_stream = input_stream.map(move |batch_result| { + batch_result.and_then(|batch| variant_columns_to_json(batch, &real_schema)) + }); + + Ok(Box::pin(RecordBatchStreamAdapter::new(output_schema, converted_stream))) + } +} + /// Check if input schema is compatible with target schema for INSERT operations. /// This allows string types (Utf8, Utf8View, LargeUtf8) to be inserted into Variant columns, /// since convert_variant_columns() will handle the conversion in write_all(). @@ -178,33 +309,42 @@ fn is_schema_compatible_for_insert(input_schema: &SchemaRef, target_schema: &Sch ))); } - for (input_field, target_field) in input_schema.fields().iter().zip(target_schema.fields()) { - let input_type = input_field.data_type(); - let target_type = target_field.data_type(); + fn is_string_type(dt: &DataType) -> bool { + matches!(dt, DataType::Utf8 | DataType::Utf8View | DataType::LargeUtf8) + } - // Same type is always compatible - if input_type == target_type { - continue; + fn types_compatible(input: &DataType, target: &DataType) -> bool { + if input == target { + return true; } - - // Allow string types to be inserted into Variant columns - // (convert_variant_columns will handle the conversion) - let is_string_to_variant = matches!( - input_type, - DataType::Utf8 | DataType::Utf8View | DataType::LargeUtf8 - ) && is_variant_type(target_type); - - if is_string_to_variant { - continue; + if is_string_type(input) && is_string_type(target) { + return true; + } + // String -> Variant (string will be converted to variant) + if is_string_type(input) && is_variant_type(target) { + return true; } + // Variant -> Utf8View (INSERT-compatible schema uses Utf8View for Variant cols) + if is_variant_type(input) && is_string_type(target) { + return true; + } + // List types with compatible element types + if let (DataType::List(in_f), DataType::List(tgt_f)) = (input, target) { + return types_compatible(in_f.data_type(), tgt_f.data_type()); + } + if let (DataType::LargeList(in_f), DataType::LargeList(tgt_f)) = (input, target) { + return types_compatible(in_f.data_type(), tgt_f.data_type()); + } + input.equals_datatype(target) + } - // Check logical equivalence for other types - if !input_type.equals_datatype(target_type) { + for (input_field, target_field) in input_schema.fields().iter().zip(target_schema.fields()) { + if !types_compatible(input_field.data_type(), target_field.data_type()) { return Err(DataFusionError::Plan(format!( "Schema mismatch for field '{}': input type {:?} is not compatible with target type {:?}", input_field.name(), - input_type, - target_type + input_field.data_type(), + target_field.data_type() ))); } } @@ -290,7 +430,10 @@ struct StorageConfig { #[derive(Debug, Clone)] pub struct Database { config: Arc, - project_configs: ProjectConfigs, + /// Unified tables: one Delta table per schema, partitioned by [project_id, date] + unified_tables: UnifiedTables, + /// Custom project tables: isolated tables for projects with their own S3 bucket + custom_project_tables: CustomProjectTables, batch_queue: Option>, maintenance_shutdown: Arc, config_pool: Option, @@ -310,9 +453,14 @@ impl Database { &self.config } - /// Get the project configs for direct access - pub fn project_configs(&self) -> &ProjectConfigs { - &self.project_configs + /// Get the unified tables cache for direct access + pub fn unified_tables(&self) -> &UnifiedTables { + &self.unified_tables + } + + /// Get the custom project tables cache for direct access + pub fn custom_project_tables(&self) -> &CustomProjectTables { + &self.custom_project_tables } /// Perform a Delta table UPDATE operation @@ -534,8 +682,6 @@ impl Database { None => (None, HashMap::new()), }; - let project_configs = HashMap::new(); - // Initialize object store cache BEFORE creating any tables // This ensures all tables benefit from caching let object_store_cache = Self::initialize_cache_with_retry(&cfg).await; @@ -547,7 +693,8 @@ impl Database { let db = Self { config: cfg, - project_configs: Arc::new(RwLock::new(project_configs)), + unified_tables: Arc::new(RwLock::new(HashMap::new())), + custom_project_tables: Arc::new(RwLock::new(HashMap::new())), batch_queue: None, maintenance_shutdown: Arc::new(CancellationToken::new()), config_pool, @@ -628,14 +775,18 @@ impl Database { let db = db.clone(); Box::pin(async move { info!("Running scheduled light optimize on recent small files"); - for ((project_id, table_name), table) in db.project_configs.read().await.iter() { + // Optimize unified tables + for (table_name, table) in db.unified_tables.read().await.iter() { match db.optimize_table_light(table, table_name).await { - Ok(_) => { - info!("Light optimize completed for project '{}' table '{}'", project_id, table_name); - } - Err(e) => { - error!("Light optimize failed for project '{}' table '{}': {}", project_id, table_name, e); - } + Ok(_) => info!("Light optimize completed for unified table '{}'", table_name), + Err(e) => error!("Light optimize failed for unified table '{}': {}", table_name, e), + } + } + // Optimize custom project tables + for ((project_id, table_name), table) in db.custom_project_tables.read().await.iter() { + match db.optimize_table_light(table, table_name).await { + Ok(_) => info!("Light optimize completed for custom project '{}' table '{}'", project_id, table_name), + Err(e) => error!("Light optimize failed for custom project '{}' table '{}': {}", project_id, table_name, e), } } }) @@ -662,9 +813,16 @@ impl Database { let db = db.clone(); Box::pin(async move { info!("Running scheduled optimize on all tables"); - for ((project_id, table_name), table) in db.project_configs.read().await.iter() { + // Optimize unified tables + for (table_name, table) in db.unified_tables.read().await.iter() { + if let Err(e) = db.optimize_table(table, table_name, None).await { + error!("Optimize failed for unified table '{}': {}", table_name, e); + } + } + // Optimize custom project tables + for ((project_id, table_name), table) in db.custom_project_tables.read().await.iter() { if let Err(e) = db.optimize_table(table, table_name, None).await { - error!("Optimize failed for project '{}' table '{}': {}", project_id, table_name, e); + error!("Optimize failed for custom project '{}' table '{}': {}", project_id, table_name, e); } } }) @@ -691,8 +849,14 @@ impl Database { info!("Running scheduled vacuum on all tables"); let retention_hours = vacuum_retention; - for ((project_id, table_name), table) in db.project_configs.read().await.iter() { - info!("Vacuuming project '{}' table '{}' (retention: {}h)", project_id, table_name, retention_hours); + // Vacuum unified tables + for (table_name, table) in db.unified_tables.read().await.iter() { + info!("Vacuuming unified table '{}' (retention: {}h)", table_name, retention_hours); + db.vacuum_table(table, retention_hours).await; + } + // Vacuum custom project tables + for ((project_id, table_name), table) in db.custom_project_tables.read().await.iter() { + info!("Vacuuming custom project '{}' table '{}' (retention: {}h)", project_id, table_name, retention_hours); db.vacuum_table(table, retention_hours).await; } }) @@ -733,12 +897,23 @@ impl Database { info!("Refreshing Delta Lake statistics cache"); db.statistics_extractor.clear_cache().await; - // Pre-warm cache for active tables - for ((project_id, table_name), table) in db.project_configs.read().await.iter() { + // Pre-warm cache for unified tables + for (table_name, table) in db.unified_tables.read().await.iter() { + let table = table.read().await; + let current_version = table.version().unwrap_or(0); + let schema_def = get_schema(table_name).unwrap_or_else(get_default_schema); + let schema = schema_def.schema_ref(); + // Use empty string for project_id since unified tables are shared + if let Err(e) = db.statistics_extractor.extract_statistics(&table, "", table_name, &schema).await { + error!("Failed to refresh statistics for unified table '{}': {}", table_name, e); + } else { + debug!("Refreshed statistics for unified table '{}' (version {})", table_name, current_version); + } + } + // Pre-warm cache for custom project tables + for ((project_id, table_name), table) in db.custom_project_tables.read().await.iter() { let table = table.read().await; let current_version = table.version().unwrap_or(0); - - // Always refresh statistics after clearing cache let schema_def = get_schema(table_name).unwrap_or_else(get_default_schema); let schema = schema_def.schema_ref(); if let Err(e) = db.statistics_extractor.extract_statistics(&table, project_id, table_name, &schema).await { @@ -858,12 +1033,13 @@ impl Database { ); // Create session state with tracing rule and DML support - // IMPORTANT: VariantInsertRewriter must run BEFORE TypeCoercion to rewrite - // string literals into json_to_variant() calls before type checking happens + // Rule ordering: VariantInsertRewriter runs BEFORE TypeCoercion (rewrites string->json_to_variant) + // VariantSelectRewriter runs AFTER TypeCoercion (wraps Variant cols with variant_to_json) let analyzer_rules: Vec> = vec![ Arc::new(datafusion::optimizer::analyzer::resolve_grouping_function::ResolveGroupingFunction::new()), Arc::new(crate::optimizers::VariantInsertRewriter), Arc::new(datafusion::optimizer::analyzer::type_coercion::TypeCoercion::new()), + Arc::new(crate::optimizers::VariantSelectRewriter), ]; let session_state = SessionStateBuilder::new() @@ -997,6 +1173,11 @@ impl Database { info!("Registered JSON functions with SessionContext"); } + /// Check if a project has custom storage configuration (their own S3 bucket) + async fn has_custom_storage(&self, project_id: &str, table_name: &str) -> bool { + self.storage_configs.read().await.contains_key(&(project_id.to_string(), table_name.to_string())) + } + #[instrument( name = "database.resolve_table", skip(self), @@ -1004,217 +1185,247 @@ impl Database { project_id = %project_id, table.name = %table_name, cache_hit = Empty, + is_custom = Empty, ) )] pub async fn resolve_table(&self, project_id: &str, table_name: &str) -> DFResult>> { let span = tracing::Span::current(); - // First check if table already exists + + // Try to reload custom configs from database if we have a pool (lazy loading) + if let Some(ref pool) = self.config_pool + && let Ok(new_configs) = Self::load_storage_configs(pool).await { - let project_configs = self.project_configs.read().await; - debug!( - "Checking cache for project '{}' table '{}', cache contains {} entries", - project_id, - table_name, - project_configs.len() - ); - if let Some(table) = project_configs.get(&(project_id.to_string(), table_name.to_string())) { - debug!("Found table in cache for project '{}' table '{}'", project_id, table_name); - span.record("cache_hit", true); - // Check if we have a recent write that might not be visible yet + let mut configs = self.storage_configs.write().await; + *configs = new_configs; + } + + // Check if project has custom storage config → use isolated table + if self.has_custom_storage(project_id, table_name).await { + span.record("is_custom", true); + return self.resolve_custom_table(project_id, table_name).await; + } + + span.record("is_custom", false); + // Default: use unified table (all projects share the same table, partitioned by project_id) + self.resolve_unified_table(table_name).await + } + + /// Resolve a unified table (shared by all default projects, partitioned by project_id) + async fn resolve_unified_table(&self, table_name: &str) -> DFResult>> { + // Check unified_tables cache first + { + let tables = self.unified_tables.read().await; + if let Some(table) = tables.get(table_name) { + debug!("Found unified table '{}' in cache", table_name); + // For unified tables, we use table_name as the key for version tracking let last_written_version = { let versions = self.last_written_versions.read().await; - versions.get(&(project_id.to_string(), table_name.to_string())).cloned() + // Use empty string for project_id since unified tables aren't project-specific + versions.get(&("".to_string(), table_name.to_string())).cloned() }; - // Check current version without holding the lock too long let current_version = table.read().await.version(); + let should_update = match (current_version, last_written_version) { + (Some(current), Some(last)) => current < last, + (Some(_), None) => true, + _ => false, + }; + + if should_update { + self.update_table(table, "", table_name) + .await + .map_err(|e| DataFusionError::Execution(format!("Failed to update table: {}", e)))?; + } + + return Ok(Arc::clone(table)); + } + } + + // Not in cache, create/load it + self.get_or_create_unified_table(table_name) + .await + .map_err(|e| DataFusionError::Execution(format!("Failed to get or create unified table: {}", e))) + } + + /// Resolve a custom project table (isolated table for projects with their own S3 bucket) + async fn resolve_custom_table(&self, project_id: &str, table_name: &str) -> DFResult>> { + // Check custom_project_tables cache first + { + let tables = self.custom_project_tables.read().await; + if let Some(table) = tables.get(&(project_id.to_string(), table_name.to_string())) { + debug!("Found custom table for project '{}' table '{}' in cache", project_id, table_name); + let last_written_version = { + let versions = self.last_written_versions.read().await; + versions.get(&(project_id.to_string(), table_name.to_string())).cloned() + }; - // Only update if we don't have a recent write or if the table version is behind + let current_version = table.read().await.version(); let should_update = match (current_version, last_written_version) { - (Some(current), Some(last)) => { - let needs_update = current < last; - debug!( - "Version check for {}/{}: current={}, last_written={}, needs_update={}", - project_id, table_name, current, last, needs_update - ); - needs_update - } - (None, Some(last)) => { - debug!( - "No current version for {}/{}, but last_written={}, will skip update", - project_id, table_name, last - ); - // If we have a last written version but no current version, it means - // we just wrote to a new table and it hasn't been loaded yet - false - } - (Some(current), None) => { - debug!("Current version {} for {}/{}, no last written, will update", current, project_id, table_name); - true - } - (None, None) => { - debug!("No version info for {}/{}, will update", project_id, table_name); - true - } + (Some(current), Some(last)) => current < last, + (Some(_), None) => true, + _ => false, }; if should_update { self.update_table(table, project_id, table_name) .await .map_err(|e| DataFusionError::Execution(format!("Failed to update table: {}", e)))?; - } else { - debug!("Skipping update for {}/{} - using cached version", project_id, table_name); } return Ok(Arc::clone(table)); } } - // Table doesn't exist, try to create it - debug!("Table not found in cache for project '{}' table '{}', creating/loading", project_id, table_name); - span.record("cache_hit", false); - self.get_or_create_table(project_id, table_name) + // Not in cache, create/load it + self.get_or_create_custom_table(project_id, table_name) .await - .map_err(|e| DataFusionError::Execution(format!("Failed to get or create table: {}", e))) + .map_err(|e| DataFusionError::Execution(format!("Failed to get or create custom table: {}", e))) } #[instrument( - name = "database.get_or_create_table", + name = "database.get_or_create_unified_table", skip(self), - fields( - project_id = %project_id, - table.name = %table_name, - ) + fields(table.name = %table_name) )] - pub async fn get_or_create_table(&self, project_id: &str, table_name: &str) -> Result>> { - // Check if table already exists before trying to create + pub async fn get_or_create_unified_table(&self, table_name: &str) -> Result>> { + // Check cache first { - let configs = self.project_configs.read().await; - if let Some(table) = configs.get(&(project_id.to_string(), table_name.to_string())) { + let tables = self.unified_tables.read().await; + if let Some(table) = tables.get(table_name) { return Ok(Arc::clone(table)); } } - // Try to reload configs from database if we have a pool (lazy loading) - if let Some(ref pool) = self.config_pool - && let Ok(new_configs) = Self::load_storage_configs(pool).await - { - let mut configs = self.storage_configs.write().await; - *configs = new_configs; + + let Some(ref bucket) = self.default_s3_bucket else { + return Err(anyhow::anyhow!("No default S3 bucket configured for unified table '{}'", table_name)); + }; + + let prefix = self.default_s3_prefix.as_ref().unwrap(); + let endpoint = self.default_s3_endpoint.as_ref().unwrap(); + // Unified table path: s3://{bucket}/{prefix}/{table_name}/ (NO project_id subdirectory) + let storage_uri = format!("s3://{}/{}/{}/?endpoint={}", bucket, prefix, table_name, endpoint); + let storage_options = self.build_storage_options(); + + info!("Creating or loading unified table '{}' at: {}", table_name, storage_uri); + + // Hold write lock during table creation + let mut tables = self.unified_tables.write().await; + + // Double-check after acquiring write lock + if let Some(table) = tables.get(table_name) { + return Ok(Arc::clone(table)); } - // Check if we have specific config for this project - let configs = self.storage_configs.read().await; - let (storage_uri, storage_options) = if let Some(config) = configs.get(&(project_id.to_string(), table_name.to_string())) { - // Use project-specific S3 settings - let storage_uri = format!( - "s3://{}/{}/?endpoint={}", - config.s3_bucket, - config.s3_prefix, - config - .s3_endpoint - .as_ref() - .unwrap_or(&self.default_s3_endpoint.clone().unwrap_or_else(|| "https://s3.amazonaws.com".to_string())) - ); + let table = self.create_delta_table_internal(&storage_uri, &storage_options, table_name).await?; + let table_arc = Arc::new(RwLock::new(table)); + tables.insert(table_name.to_string(), Arc::clone(&table_arc)); + info!("Cached unified table '{}', cache now contains {} entries", table_name, tables.len()); - let mut storage_options = HashMap::new(); - storage_options.insert("AWS_ACCESS_KEY_ID".to_string(), config.s3_access_key_id.clone()); - storage_options.insert("AWS_SECRET_ACCESS_KEY".to_string(), config.s3_secret_access_key.clone()); - storage_options.insert("AWS_REGION".to_string(), config.s3_region.clone()); - if let Some(ref endpoint) = config.s3_endpoint { - storage_options.insert("AWS_ENDPOINT_URL".to_string(), endpoint.clone()); - } + Ok(table_arc) + } - // Add DynamoDB locking configuration if enabled (even for project-specific configs) - if self.config.aws.is_dynamodb_locking_enabled() { - storage_options.insert("AWS_S3_LOCKING_PROVIDER".to_string(), "dynamodb".to_string()); - if let Some(ref table) = self.config.aws.dynamodb.delta_dynamo_table_name { - storage_options.insert("DELTA_DYNAMO_TABLE_NAME".to_string(), table.clone()); - } - if let Some(ref key) = self.config.aws.dynamodb.aws_access_key_id_dynamodb { - storage_options.insert("AWS_ACCESS_KEY_ID_DYNAMODB".to_string(), key.clone()); - } - if let Some(ref secret) = self.config.aws.dynamodb.aws_secret_access_key_dynamodb { - storage_options.insert("AWS_SECRET_ACCESS_KEY_DYNAMODB".to_string(), secret.clone()); - } - if let Some(ref region) = self.config.aws.dynamodb.aws_region_dynamodb { - storage_options.insert("AWS_REGION_DYNAMODB".to_string(), region.clone()); - } - if let Some(ref endpoint) = self.config.aws.dynamodb.aws_endpoint_url_dynamodb { - storage_options.insert("AWS_ENDPOINT_URL_DYNAMODB".to_string(), endpoint.clone()); - } + #[instrument( + name = "database.get_or_create_custom_table", + skip(self), + fields(project_id = %project_id, table.name = %table_name) + )] + pub async fn get_or_create_custom_table(&self, project_id: &str, table_name: &str) -> Result>> { + // Check cache first + { + let tables = self.custom_project_tables.read().await; + if let Some(table) = tables.get(&(project_id.to_string(), table_name.to_string())) { + return Ok(Arc::clone(table)); } + } - (storage_uri, storage_options) - } else if let Some(ref bucket) = self.default_s3_bucket { - // No specific config, use default bucket with environment credentials - let prefix = self.default_s3_prefix.as_ref().unwrap(); - let endpoint = self.default_s3_endpoint.as_ref().unwrap(); - let storage_uri = format!("s3://{}/{}/projects/{}/{}/?endpoint={}", bucket, prefix, project_id, table_name, endpoint); + // Get custom storage config for this project + let configs = self.storage_configs.read().await; + let config = configs.get(&(project_id.to_string(), table_name.to_string())) + .ok_or_else(|| anyhow::anyhow!("No storage config found for project '{}' table '{}'", project_id, table_name))? + .clone(); + drop(configs); + + let storage_uri = format!( + "s3://{}/{}/?endpoint={}", + config.s3_bucket, + config.s3_prefix, + config.s3_endpoint.as_ref().unwrap_or(&self.default_s3_endpoint.clone().unwrap_or_else(|| "https://s3.amazonaws.com".to_string())) + ); - // Populate storage options with AWS credentials and DynamoDB locking if enabled - let storage_options = self.build_storage_options(); + let mut storage_options = HashMap::new(); + storage_options.insert("AWS_ACCESS_KEY_ID".to_string(), config.s3_access_key_id.clone()); + storage_options.insert("AWS_SECRET_ACCESS_KEY".to_string(), config.s3_secret_access_key.clone()); + storage_options.insert("AWS_REGION".to_string(), config.s3_region.clone()); + if let Some(ref endpoint) = config.s3_endpoint { + storage_options.insert("AWS_ENDPOINT_URL".to_string(), endpoint.clone()); + } - (storage_uri, storage_options) - } else { - return Err(anyhow::anyhow!( - "No configuration for project '{}' table '{}' and no default S3 bucket set", - project_id, - table_name - )); - }; + // Add DynamoDB locking configuration if enabled + if self.config.aws.is_dynamodb_locking_enabled() { + storage_options.insert("AWS_S3_LOCKING_PROVIDER".to_string(), "dynamodb".to_string()); + if let Some(ref table) = self.config.aws.dynamodb.delta_dynamo_table_name { + storage_options.insert("DELTA_DYNAMO_TABLE_NAME".to_string(), table.clone()); + } + if let Some(ref key) = self.config.aws.dynamodb.aws_access_key_id_dynamodb { + storage_options.insert("AWS_ACCESS_KEY_ID_DYNAMODB".to_string(), key.clone()); + } + if let Some(ref secret) = self.config.aws.dynamodb.aws_secret_access_key_dynamodb { + storage_options.insert("AWS_SECRET_ACCESS_KEY_DYNAMODB".to_string(), secret.clone()); + } + if let Some(ref region) = self.config.aws.dynamodb.aws_region_dynamodb { + storage_options.insert("AWS_REGION_DYNAMODB".to_string(), region.clone()); + } + if let Some(ref endpoint) = self.config.aws.dynamodb.aws_endpoint_url_dynamodb { + storage_options.insert("AWS_ENDPOINT_URL_DYNAMODB".to_string(), endpoint.clone()); + } + } - info!( - "Creating or loading table for project '{}' table '{}' at: {}", - project_id, table_name, storage_uri - ); + info!("Creating or loading custom table for project '{}' table '{}' at: {}", project_id, table_name, storage_uri); - // Hold a write lock during table creation to prevent concurrent creation - let mut configs = self.project_configs.write().await; + // Hold write lock during table creation + let mut tables = self.custom_project_tables.write().await; // Double-check after acquiring write lock - if let Some(table) = configs.get(&(project_id.to_string(), table_name.to_string())) { + if let Some(table) = tables.get(&(project_id.to_string(), table_name.to_string())) { return Ok(Arc::clone(table)); } - // Create the base S3 object store - let base_store = self.create_object_store(&storage_uri, &storage_options).instrument(tracing::trace_span!("create_object_store")).await?; + let table = self.create_delta_table_internal(&storage_uri, &storage_options, table_name).await?; + let table_arc = Arc::new(RwLock::new(table)); + tables.insert((project_id.to_string(), table_name.to_string()), Arc::clone(&table_arc)); + info!("Cached custom table for project '{}' table '{}', cache now contains {} entries", project_id, table_name, tables.len()); + + Ok(table_arc) + } - // Wrap with instrumentation for tracing + /// Internal helper to create/load a Delta table with caching and retry logic + async fn create_delta_table_internal(&self, storage_uri: &str, storage_options: &HashMap, table_name: &str) -> Result { + // Create the base S3 object store + let base_store = self.create_object_store(storage_uri, storage_options).instrument(tracing::trace_span!("create_object_store")).await?; let instrumented_store = instrument_object_store(base_store, "s3"); - // Wrap with the shared Foyer cache if available, otherwise use base store let cached_store = if let Some(ref shared_cache) = self.object_store_cache { - // Create a new wrapper around the instrumented store using our shared cache - // This allows the same cache to be used across all tables - // Note: We don't double-instrument with instrument_object_store here since FoyerObjectStoreCache - // already has its own instrumentation that properly propagates parent spans Arc::new(FoyerObjectStoreCache::new_with_shared_cache(instrumented_store.clone(), shared_cache)) as Arc } else { warn!("Shared Foyer cache not initialized, using uncached object store"); instrumented_store }; - // Try to load or create the table with the cached object store - let table = match self.create_or_load_delta_table(&storage_uri, storage_options.clone(), cached_store.clone()).await { + // Try to load existing table + match self.create_or_load_delta_table(storage_uri, storage_options.clone(), cached_store.clone()).await { Ok(table) => { - info!("Loaded existing table for project '{}' table '{}'", project_id, table_name); - table + info!("Loaded existing table '{}'", table_name); + Ok(table) } Err(load_err) => { - info!( - "Table doesn't exist for project '{}' table '{}', creating new table. err: {:?}", - project_id, table_name, load_err - ); + info!("Table '{}' doesn't exist, creating new table. err: {:?}", table_name, load_err); let schema = get_schema(table_name).unwrap_or_else(get_default_schema); - - // Try to create the table with retry logic for concurrent creation let mut create_attempts = 0; + loop { create_attempts += 1; - let commit_properties = CommitProperties::default().with_create_checkpoint(true).with_cleanup_expired_logs(Some(true)); - let checkpoint_interval = self.config.parquet.timefusion_checkpoint_interval.to_string(); let mut config = HashMap::new(); @@ -1222,7 +1433,7 @@ impl Database { config.insert("delta.checkpointPolicy".to_string(), Some("v2".to_string())); match CreateBuilder::new() - .with_location(&storage_uri) + .with_location(storage_uri) .with_columns(schema.columns().unwrap_or_default()) .with_partition_columns(schema.partitions.clone()) .with_storage_options(storage_options.clone()) @@ -1230,50 +1441,46 @@ impl Database { .with_configuration(config) .await { - Ok(table) => break table, + Ok(table) => break Ok(table), Err(create_err) => { let err_str = create_err.to_string(); if (err_str.contains("already exists") || err_str.contains("version 0") || err_str.contains("ConditionalCheckFailedException")) && create_attempts < 3 { - // Table was created by another process or DynamoDB lock conflict, try to load it - debug!( - "Table creation conflict (possibly DynamoDB lock), attempting to load existing table (attempt {})", - create_attempts - ); - // Exponential backoff + debug!("Table creation conflict, attempting to load existing table (attempt {})", create_attempts); let backoff_ms = 100 * (2_u64.pow(create_attempts.min(5))); tokio::time::sleep(tokio::time::Duration::from_millis(backoff_ms)).await; - // Try to load the table that was just created - match self.create_or_load_delta_table(&storage_uri, storage_options.clone(), cached_store.clone()).await { - Ok(table) => break table, + match self.create_or_load_delta_table(storage_uri, storage_options.clone(), cached_store.clone()).await { + Ok(table) => break Ok(table), Err(reload_err) => { debug!("Failed to load table after creation conflict: {:?}", reload_err); continue; } } } else { - return Err(anyhow::anyhow!("Failed to create table: {}", create_err)); + break Err(anyhow::anyhow!("Failed to create table: {}", create_err)); } } } } } - }; - - let table_arc = Arc::new(RwLock::new(table)); - - // Store in cache (we already have the write lock) - configs.insert((project_id.to_string(), table_name.to_string()), Arc::clone(&table_arc)); - info!( - "Cached table for project '{}' table '{}', cache now contains {} entries", - project_id, - table_name, - configs.len() - ); + } + } - Ok(table_arc) + /// Legacy method for backward compatibility - routes to unified or custom table + #[instrument( + name = "database.get_or_create_table", + skip(self), + fields(project_id = %project_id, table.name = %table_name) + )] + pub async fn get_or_create_table(&self, project_id: &str, table_name: &str) -> Result>> { + // Route to appropriate table based on whether project has custom storage + if self.has_custom_storage(project_id, table_name).await { + self.get_or_create_custom_table(project_id, table_name).await + } else { + self.get_or_create_unified_table(table_name).await + } } /// Create an object store for the given URI and storage options @@ -1774,7 +1981,8 @@ impl ProjectRoutingTable { fn schema(&self) -> SchemaRef { // Return INSERT-compatible schema where Variant columns appear as Utf8View. // This allows INSERT statements with JSON strings to pass DataFusion's type validation. - // VariantConversionExec handles the actual string->Variant conversion during write. + // VariantConversionExec handles string->Variant conversion during write. + // The pgwire layer handles Variant->JSON conversion during read via VariantJsonExec. create_insert_compatible_schema(&self.schema) } @@ -2182,10 +2390,7 @@ impl TableProvider for ProjectRoutingTable { return not_impl_err!("{insert_op} not implemented for MemoryTable yet"); } - // Wrap input with VariantConversionExec to convert string columns to Variant - // before they reach the sink. This prevents DataFusion from trying to cast - // Utf8 -> Struct(Variant) which would fail. - // Use real_schema() to get the actual Variant types for proper conversion. + // Wrap input with VariantConversionExec to convert string columns to Variant. let converted_input: Arc = Arc::new(VariantConversionExec::new(input, self.real_schema())); // Create sink executor with the converted input @@ -2232,10 +2437,19 @@ impl TableProvider for ProjectRoutingTable { let project_id = self.extract_project_id_from_filters(&optimized_filters).unwrap_or_else(|| self.default_project.clone()); span.record("table.project_id", project_id.as_str()); + // Helper to wrap result with VariantToJsonExec for proper pgwire encoding + let wrap_result = |plan: Arc| -> DFResult> { + Ok(Arc::new(VariantToJsonExec::new(plan, self.real_schema()))) + }; + // Check if buffered layer is configured + let has_layer = self.database.buffered_layer().is_some(); + debug!("ProjectRoutingTable::scan - buffered_layer present: {}, project_id: {}", has_layer, project_id); let Some(layer) = self.database.buffered_layer() else { // No buffered layer, query Delta directly - return self.scan_delta_only(state, &project_id, projection, &optimized_filters, limit).await; + debug!("No buffered layer, querying Delta only"); + let plan = self.scan_delta_only(state, &project_id, projection, &optimized_filters, limit).await?; + return wrap_result(plan); }; span.record("scan.uses_mem_buffer", true); @@ -2265,8 +2479,11 @@ impl TableProvider for ProjectRoutingTable { }; // If no mem buffer data, query Delta only + debug!("MemBuffer partitions count: {} for {}/{}", mem_partitions.len(), project_id, self.table_name); if mem_partitions.is_empty() { - return self.scan_delta_only(state, &project_id, projection, &optimized_filters, limit).await; + debug!("No MemBuffer data, querying Delta only for {}/{}", project_id, self.table_name); + let plan = self.scan_delta_only(state, &project_id, projection, &optimized_filters, limit).await?; + return wrap_result(plan); } // Create MemorySourceConfig with multiple partitions for parallel execution @@ -2279,7 +2496,7 @@ impl TableProvider for ProjectRoutingTable { "Skipping Delta scan - query time range entirely within MemBuffer for {}/{}", project_id, self.table_name ); - return Ok(mem_plan); + return wrap_result(mem_plan); } // Get oldest timestamp from MemBuffer for time-based exclusion @@ -2306,7 +2523,7 @@ impl TableProvider for ProjectRoutingTable { let delta_plan = self.scan_delta_table(&table, state, projection, &delta_filters, limit).await?; // Union both plans (mem data first for recency, then Delta for historical) - UnionExec::try_new(vec![mem_plan, delta_plan]) + wrap_result(UnionExec::try_new(vec![mem_plan, delta_plan])?) } fn statistics(&self) -> Option { diff --git a/src/dml.rs b/src/dml.rs index 1d6de76..d7c7ff5 100644 --- a/src/dml.rs +++ b/src/dml.rs @@ -376,7 +376,13 @@ impl<'a> DmlContext<'a> { total_rows += mem_op(layer, self.predicate.as_ref())?; } - let has_committed = self.database.project_configs().read().await.contains_key(&(self.project_id.to_string(), self.table_name.to_string())); + // Check if there's committed data: either in custom project tables or unified tables + let has_committed = { + let custom_tables = self.database.custom_project_tables().read().await; + let unified_tables = self.database.unified_tables().read().await; + custom_tables.contains_key(&(self.project_id.to_string(), self.table_name.to_string())) + || unified_tables.contains_key(self.table_name) + }; if has_committed { total_rows += delta_op.await?; @@ -511,14 +517,11 @@ where F: FnOnce(deltalake::DeltaTable) -> Fut, Fut: std::future::Future>, { - let table_key = (project_id.to_string(), table_name.to_string()); + // Use resolve_table which routes to unified or custom table based on storage config let table_lock = database - .project_configs() - .read() + .resolve_table(project_id, table_name) .await - .get(&table_key) - .ok_or_else(|| DataFusionError::Execution(format!("Table not found: {} for project {}", table_name, project_id)))? - .clone(); + .map_err(|e| DataFusionError::Execution(format!("Table not found: {} for project {}: {}", table_name, project_id, e)))?; let delta_table = table_lock.write().await; let (new_table, rows_affected) = operation(delta_table.clone()).await?; diff --git a/src/optimizers/mod.rs b/src/optimizers/mod.rs index af47243..d8dec7f 100644 --- a/src/optimizers/mod.rs +++ b/src/optimizers/mod.rs @@ -1,6 +1,10 @@ mod variant_insert_rewriter; +mod variant_select_rewriter; pub use variant_insert_rewriter::VariantInsertRewriter; +pub use variant_select_rewriter::VariantSelectRewriter; + +// Remove unused imports warning - these are used by the submodules indirectly use datafusion::logical_expr::{BinaryExpr, Expr, Operator}; use datafusion::scalar::ScalarValue; diff --git a/src/optimizers/variant_insert_rewriter.rs b/src/optimizers/variant_insert_rewriter.rs index 0acfb52..b4c60ff 100644 --- a/src/optimizers/variant_insert_rewriter.rs +++ b/src/optimizers/variant_insert_rewriter.rs @@ -1,4 +1,3 @@ -use std::collections::HashSet; use std::sync::Arc; use datafusion::{ @@ -42,43 +41,33 @@ fn rewrite_insert_node(plan: LogicalPlan) -> Result> { debug!("VariantInsertRewriter: INSERT into {}", dml.table_name); - // Get target table schema to find variant column names let target_schema = dml.target.schema(); - let variant_column_names: HashSet = target_schema - .fields() - .iter() - .filter(|f| is_variant_type(f.data_type())) - .map(|f| f.name().clone()) - .collect(); - - if variant_column_names.is_empty() { - return Ok(Transformed::no(plan)); - } - - // Get input schema to find which positions correspond to variant columns let input_schema = dml.input.schema(); - + // For each input field, check if the TARGET column (by name) is Variant let variant_indices: Vec = input_schema .fields() .iter() .enumerate() - .filter(|(_, f)| variant_column_names.contains(f.name())) + .filter(|(_, input_field)| { + // Look up the target column by name and check if it's Variant + target_schema + .column_with_name(input_field.name()) + .map(|(_, f)| is_variant_type(f.data_type())) + .unwrap_or(false) + }) .map(|(i, _)| i) .collect(); - if variant_indices.is_empty() { return Ok(Transformed::no(plan)); } debug!( - "VariantInsertRewriter: Found {} variant columns in INSERT: {:?}", + "VariantInsertRewriter: Found {} variant columns at positions {:?} (names: {:?})", variant_indices.len(), - input_schema.fields().iter().enumerate() - .filter(|(i, _)| variant_indices.contains(i)) - .map(|(_, f)| f.name()) - .collect::>() + variant_indices, + variant_indices.iter().filter_map(|i| input_schema.fields().get(*i).map(|f| f.name())).collect::>() ); let new_input = rewrite_input_for_variant(&dml.input, &variant_indices)?; diff --git a/src/optimizers/variant_select_rewriter.rs b/src/optimizers/variant_select_rewriter.rs new file mode 100644 index 0000000..ecdab8a --- /dev/null +++ b/src/optimizers/variant_select_rewriter.rs @@ -0,0 +1,80 @@ +use std::sync::Arc; + +use datafusion::{ + common::{DFSchema, Result, tree_node::{Transformed, TreeNode}}, + config::ConfigOptions, + logical_expr::{Expr, ExprSchemable, LogicalPlan, Projection, expr::ScalarFunction}, + optimizer::AnalyzerRule, +}; +use datafusion_variant::VariantToJsonUdf; +use tracing::debug; + +use crate::schema_loader::is_variant_type; + +/// AnalyzerRule that rewrites SELECT queries to wrap Variant columns with `variant_to_json()`. +/// This ensures Variant data is serialized as JSON strings for PostgreSQL wire protocol. +#[derive(Debug, Default)] +pub struct VariantSelectRewriter; + +impl AnalyzerRule for VariantSelectRewriter { + fn name(&self) -> &str { + "variant_select_rewriter" + } + + fn analyze(&self, plan: LogicalPlan, _config: &ConfigOptions) -> Result { + plan.transform_up(rewrite_select_node).map(|t| t.data) + } +} + +fn rewrite_select_node(plan: LogicalPlan) -> Result> { + if let LogicalPlan::Projection(proj) = &plan { + let input_schema = proj.input.schema(); + let variant_to_json = Arc::new(datafusion::logical_expr::ScalarUDF::from(VariantToJsonUdf::default())); + let mut modified = false; + + let new_exprs: Vec = proj.expr.iter().map(|expr| { + if is_variant_expr(expr, input_schema) { + modified = true; + wrap_with_variant_to_json(expr, &variant_to_json) + } else { + expr.clone() + } + }).collect(); + + if modified { + debug!("VariantSelectRewriter: Wrapped {} Variant columns with variant_to_json()", + new_exprs.iter().filter(|e| matches!(e, Expr::ScalarFunction(_))).count()); + return Ok(Transformed::yes(LogicalPlan::Projection(Projection::try_new(new_exprs, proj.input.clone())?))); + } + } + Ok(Transformed::no(plan)) +} + +fn is_variant_expr(expr: &Expr, schema: &DFSchema) -> bool { + // Already wrapped - don't double-wrap + if let Expr::ScalarFunction(sf) = expr { + if sf.func.name() == "variant_to_json" { + return false; + } + } + // Check if expression's result type is Variant + expr.get_type(schema).map(|dt| is_variant_type(&dt)).unwrap_or(false) +} + +fn wrap_with_variant_to_json(expr: &Expr, udf: &Arc) -> Expr { + // Preserve the alias if there is one + let (inner, alias) = match expr { + Expr::Alias(a) => (a.expr.as_ref().clone(), Some(a.name.clone())), + _ => (expr.clone(), None), + }; + + let wrapped = Expr::ScalarFunction(ScalarFunction { + func: udf.clone(), + args: vec![inner], + }); + + match alias { + Some(name) => wrapped.alias(name), + None => wrapped, + } +} diff --git a/tests/buffer_consistency_test.rs b/tests/buffer_consistency_test.rs index 4d4d056..8837ed1 100644 --- a/tests/buffer_consistency_test.rs +++ b/tests/buffer_consistency_test.rs @@ -18,7 +18,7 @@ async fn setup_db_with_buffer(mode: BufferMode) -> Result<(Arc, Arc Date: Mon, 2 Feb 2026 23:02:53 +0100 Subject: [PATCH 3/8] Fix COUNT(*) and aggregation queries failing on empty projections variant_columns_to_json() was using RecordBatch::try_new() which fails when creating batches with 0 columns (empty projections used by COUNT(*)) because Arrow requires either columns or an explicit row count. Changed to try_new_with_options() to preserve the original batch's row count, fixing queries like SELECT COUNT(*) that don't need any columns. --- src/database.rs | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) diff --git a/src/database.rs b/src/database.rs index 91cc7d0..bd58694 100644 --- a/src/database.rs +++ b/src/database.rs @@ -178,8 +178,10 @@ fn json_strings_to_variant<'a>(iter: impl Iterator>) -> D pub fn variant_columns_to_json(batch: RecordBatch, real_schema: &SchemaRef) -> DFResult { use datafusion::arrow::array::{ArrayRef, StructArray}; use datafusion::arrow::datatypes::{DataType, Field}; + use datafusion::arrow::record_batch::RecordBatchOptions; let batch_schema = batch.schema(); + let row_count = batch.num_rows(); let mut columns: Vec = batch.columns().to_vec(); let mut new_fields: Vec> = batch_schema.fields().iter().cloned().collect(); @@ -201,7 +203,9 @@ pub fn variant_columns_to_json(batch: RecordBatch, real_schema: &SchemaRef) -> D } let new_schema = Arc::new(Schema::new(new_fields)); - RecordBatch::try_new(new_schema, columns).map_err(|e| DataFusionError::ArrowError(Box::new(e), None)) + // Use try_new_with_options to preserve row count for empty-column batches (e.g., COUNT(*) queries) + RecordBatch::try_new_with_options(new_schema, columns, &RecordBatchOptions::new().with_row_count(Some(row_count))) + .map_err(|e| DataFusionError::ArrowError(Box::new(e), None)) } /// Convert a Variant StructArray to a StringArray of JSON values. From bf29ef3d79ae0f7f00cff99ebb97738ce04ec40b Mon Sep 17 00:00:00 2001 From: Anthony Alaribe Date: Mon, 2 Feb 2026 23:37:03 +0100 Subject: [PATCH 4/8] Add retry and timeout config to S3 object store Transient network errors like "error sending request" were failing immediately with no retries. Added: - RetryConfig: 5 retries with exponential backoff (100ms-15s) - ClientOptions: 30s connect timeout, 5min request timeout This should resolve intermittent flush failures to R2/S3. --- src/database.rs | 23 ++++++++++++++++++++++- 1 file changed, 22 insertions(+), 1 deletion(-) diff --git a/src/database.rs b/src/database.rs index bd58694..fba1038 100644 --- a/src/database.rs +++ b/src/database.rs @@ -1490,13 +1490,34 @@ impl Database { /// Create an object store for the given URI and storage options async fn create_object_store(&self, storage_uri: &str, storage_options: &HashMap) -> Result> { use object_store::aws::AmazonS3Builder; + use object_store::{ClientOptions, RetryConfig, BackoffConfig}; + use std::time::Duration; // Parse the S3 URI to extract bucket and prefix let url = Url::parse(storage_uri)?; let bucket = url.host_str().ok_or_else(|| anyhow::anyhow!("Invalid S3 URI: missing bucket"))?; + // Configure retry with exponential backoff for transient network errors + let retry_config = RetryConfig { + max_retries: 5, + retry_timeout: Duration::from_secs(180), + backoff: BackoffConfig { + init_backoff: Duration::from_millis(100), + max_backoff: Duration::from_secs(15), + base: 2.0, + }, + }; + + // Configure HTTP client with reasonable timeouts + let client_options = ClientOptions::new() + .with_connect_timeout(Duration::from_secs(30)) + .with_timeout(Duration::from_secs(300)); + // Build S3 configuration - let mut builder = AmazonS3Builder::new().with_bucket_name(bucket); + let mut builder = AmazonS3Builder::new() + .with_bucket_name(bucket) + .with_retry(retry_config) + .with_client_options(client_options); // Apply storage options if let Some(access_key) = storage_options.get("AWS_ACCESS_KEY_ID") { From f68ab0f4225d52483bab45dd5948986597826a90 Mon Sep 17 00:00:00 2001 From: Anthony Alaribe Date: Mon, 16 Feb 2026 13:46:45 +0100 Subject: [PATCH 5/8] Fix correctness bugs and add performance optimizations - Replace blocking std::thread::sleep with tokio::time::sleep in CAS retry loop to avoid starving the Tokio executor under contention - Fix DML memory tracking: recalculate bucket memory_bytes after DELETE/UPDATE operations to prevent premature flush triggers - Improve WAL recovery resilience: catch schema-incompatible entries instead of aborting recovery, add empty batch skip - Add timestamp range filtering to MemBuffer queries: extract bounds from filter expressions and skip non-overlapping time buckets - Switch from GreedyMemoryPool to FairSpillPool for per-query memory fairness and automatic spill-to-disk under pressure - Make WAL fsync interval configurable via TIMEFUSION_WAL_FSYNC_MS env var (default 200ms) --- src/buffered_write_layer.rs | 29 ++++++------ src/config.rs | 6 +++ src/database.rs | 7 +-- src/mem_buffer.rs | 92 +++++++++++++++++++++++++++++++++---- src/wal.rs | 6 ++- 5 files changed, 112 insertions(+), 28 deletions(-) diff --git a/src/buffered_write_layer.rs b/src/buffered_write_layer.rs index 3d612cd..6eea8f7 100644 --- a/src/buffered_write_layer.rs +++ b/src/buffered_write_layer.rs @@ -67,7 +67,7 @@ impl std::fmt::Debug for BufferedWriteLayer { impl BufferedWriteLayer { /// Create a new BufferedWriteLayer with explicit config. pub fn with_config(cfg: Arc) -> anyhow::Result { - let wal = Arc::new(WalManager::new(cfg.core.wal_dir())?); + let wal = Arc::new(WalManager::with_fsync_ms(cfg.core.wal_dir(), cfg.buffer.wal_fsync_ms())?); let mem_buffer = Arc::new(MemBuffer::new()); Ok(Self { @@ -109,7 +109,7 @@ impl BufferedWriteLayer { /// Try to reserve memory atomically before a write. /// Returns estimated batch size on success, or error if hard limit exceeded. /// Uses exponential backoff to reduce CPU thrashing under contention. - fn try_reserve_memory(&self, batches: &[RecordBatch]) -> anyhow::Result { + async fn try_reserve_memory(&self, batches: &[RecordBatch]) -> anyhow::Result { let batch_size: usize = batches.iter().map(estimate_batch_size).sum(); let estimated_size = (batch_size as f64 * MEMORY_OVERHEAD_MULTIPLIER) as usize; @@ -138,16 +138,11 @@ impl BufferedWriteLayer { return Ok(estimated_size); } - // Exponential backoff: spin_loop for first few attempts, then brief sleep. - // Note: Using std::thread::sleep in this sync function called from async context. - // This is acceptable because: (1) max sleep is ~1ms, (2) only under high contention, - // (3) converting to async would require spawn_blocking which adds more overhead. if attempt < 5 { std::hint::spin_loop(); } else { - // Max backoff = 1μs << 10 = 1024μs ≈ 1ms let backoff_micros = CAS_BACKOFF_BASE_MICROS << attempt.min(CAS_BACKOFF_MAX_EXPONENT); - std::thread::sleep(std::time::Duration::from_micros(backoff_micros)); + tokio::time::sleep(std::time::Duration::from_micros(backoff_micros)).await; } } anyhow::bail!("Failed to reserve memory after {} retries due to contention", MAX_CAS_RETRIES) @@ -172,7 +167,7 @@ impl BufferedWriteLayer { } // Reserve memory atomically before writing - prevents race condition - let reserved_size = self.try_reserve_memory(&batches)?; + let reserved_size = self.try_reserve_memory(&batches).await?; // Write WAL and MemBuffer, ensuring reservation is released regardless of outcome. // Reservation covers the window between WAL write and MemBuffer insert; @@ -236,11 +231,17 @@ impl BufferedWriteLayer { match entry.operation { WalOperation::Insert => match WalManager::deserialize_batch(&entry.data, &entry.table_name) { Ok(batch) => { - self.mem_buffer.insert(&entry.project_id, &entry.table_name, batch, entry.timestamp_micros)?; - entries_replayed += 1; + if batch.num_rows() == 0 { + warn!("Skipping empty batch during WAL recovery for {}.{}", entry.project_id, entry.table_name); + continue; + } + match self.mem_buffer.insert(&entry.project_id, &entry.table_name, batch, entry.timestamp_micros) { + Ok(()) => entries_replayed += 1, + Err(e) => warn!("Skipping incompatible WAL entry for {}.{}: {}", entry.project_id, entry.table_name, e), + } } Err(e) => { - warn!("Skipping corrupted INSERT batch: {}", e); + warn!("Skipping corrupted INSERT batch for {}.{}: {}", entry.project_id, entry.table_name, e); } }, WalOperation::Delete => match deserialize_delete_payload(&entry.data) { @@ -522,8 +523,8 @@ impl BufferedWriteLayer { /// Query and return partitioned data - one partition per time bucket. /// This enables parallel execution across time buckets in DataFusion. - pub fn query_partitioned(&self, project_id: &str, table_name: &str) -> anyhow::Result>> { - self.mem_buffer.query_partitioned(project_id, table_name) + pub fn query_partitioned(&self, project_id: &str, table_name: &str, filters: &[datafusion::logical_expr::Expr]) -> anyhow::Result>> { + self.mem_buffer.query_partitioned(project_id, table_name, filters) } /// Check if a table exists in the memory buffer. diff --git a/src/config.rs b/src/config.rs index 7d23072..d71dd9b 100644 --- a/src/config.rs +++ b/src/config.rs @@ -101,6 +101,7 @@ const_default!(d_buffer_max_memory: usize = 4096); const_default!(d_shutdown_timeout: u64 = 5); const_default!(d_wal_corruption_threshold: usize = 10); const_default!(d_flush_parallelism: usize = 4); +const_default!(d_wal_fsync_ms: u64 = 200); const_default!(d_foyer_memory_mb: usize = 512); const_default!(d_foyer_disk_gb: usize = 100); const_default!(d_foyer_ttl: u64 = 604_800); // 7 days @@ -263,6 +264,8 @@ pub struct BufferConfig { pub timefusion_flush_parallelism: usize, #[serde(default)] pub timefusion_flush_immediately: bool, + #[serde(default = "d_wal_fsync_ms")] + pub timefusion_wal_fsync_ms: u64, } impl BufferConfig { @@ -287,6 +290,9 @@ impl BufferConfig { pub fn flush_immediately(&self) -> bool { self.timefusion_flush_immediately } + pub fn wal_fsync_ms(&self) -> u64 { + self.timefusion_wal_fsync_ms.max(1) + } pub fn compute_shutdown_timeout(&self, current_memory_mb: usize) -> Duration { Duration::from_secs((self.timefusion_shutdown_timeout_secs.max(1) + (current_memory_mb / 100) as u64).min(300)) diff --git a/src/database.rs b/src/database.rs index fba1038..34caf38 100644 --- a/src/database.rs +++ b/src/database.rs @@ -1018,9 +1018,10 @@ impl Database { let _ = options.set("datafusion.execution.memory_fraction", &memory_fraction.to_string()); let _ = options.set("datafusion.execution.sort_spill_reservation_bytes", &sort_spill_reservation_bytes.to_string()); - // Create runtime environment with memory limit + // Create runtime environment with FairSpillPool for per-query memory fairness + let pool_size = (memory_limit_bytes as f64 * memory_fraction) as usize; let runtime_env = RuntimeEnvBuilder::new() - .with_memory_limit(memory_limit_bytes, memory_fraction) + .with_memory_pool(Arc::new(datafusion::execution::memory_pool::FairSpillPool::new(pool_size))) .build() .expect("Failed to create runtime environment"); @@ -2495,7 +2496,7 @@ impl TableProvider for ProjectRoutingTable { }; // Query MemBuffer with partitioned data for parallel execution - let mem_partitions = match layer.query_partitioned(&project_id, &self.table_name) { + let mem_partitions = match layer.query_partitioned(&project_id, &self.table_name, &optimized_filters) { Ok(partitions) => partitions, Err(e) => { warn!("Failed to query mem buffer: {}", e); diff --git a/src/mem_buffer.rs b/src/mem_buffer.rs index adbc791..f86ccad 100644 --- a/src/mem_buffer.rs +++ b/src/mem_buffer.rs @@ -217,6 +217,60 @@ impl datafusion::sql::planner::ContextProvider for EmptyContextProvider { } } +/// Extract min/max timestamp bounds from filter expressions for bucket pruning. +fn extract_timestamp_range(filters: &[Expr]) -> (Option, Option) { + let (mut min_ts, mut max_ts) = (None, None); + for filter in filters { + if let Expr::BinaryExpr(datafusion::logical_expr::BinaryExpr { left, op, right }) = filter { + let is_ts = matches!(left.as_ref(), Expr::Column(c) if c.name == "timestamp"); + if !is_ts { + continue; + } + let ts = match right.as_ref() { + Expr::Literal(datafusion::scalar::ScalarValue::TimestampMicrosecond(Some(ts), _), _) => Some(*ts), + Expr::Literal(datafusion::scalar::ScalarValue::TimestampNanosecond(Some(ts), _), _) => Some(*ts / 1000), + Expr::Literal(datafusion::scalar::ScalarValue::TimestampMillisecond(Some(ts), _), _) => Some(*ts * 1000), + Expr::Literal(datafusion::scalar::ScalarValue::TimestampSecond(Some(ts), _), _) => Some(*ts * 1_000_000), + _ => None, + }; + if let Some(ts) = ts { + match op { + datafusion::logical_expr::Operator::Gt | datafusion::logical_expr::Operator::GtEq => { + min_ts = Some(min_ts.map_or(ts, |m: i64| m.max(ts))); + } + datafusion::logical_expr::Operator::Lt | datafusion::logical_expr::Operator::LtEq => { + max_ts = Some(max_ts.map_or(ts, |m: i64| m.min(ts))); + } + datafusion::logical_expr::Operator::Eq => { + min_ts = Some(ts); + max_ts = Some(ts); + } + _ => {} + } + } + } + } + (min_ts, max_ts) +} + +/// Check if a bucket's time range overlaps with the query range. +fn bucket_overlaps_range(bucket: &TimeBucket, range: &(Option, Option)) -> bool { + let (min_filter, max_filter) = range; + if let Some(max) = max_filter { + let bucket_min = bucket.min_timestamp.load(Ordering::Relaxed); + if bucket_min != i64::MAX && bucket_min > *max { + return false; + } + } + if let Some(min) = min_filter { + let bucket_max = bucket.max_timestamp.load(Ordering::Relaxed); + if bucket_max != i64::MIN && bucket_max < *min { + return false; + } + } + true +} + impl MemBuffer { pub fn new() -> Self { Self { @@ -323,16 +377,18 @@ impl MemBuffer { Ok(()) } - #[instrument(skip(self, _filters), fields(project_id, table_name))] - pub fn query(&self, project_id: &str, table_name: &str, _filters: &[Expr]) -> anyhow::Result> { + #[instrument(skip(self, filters), fields(project_id, table_name))] + pub fn query(&self, project_id: &str, table_name: &str, filters: &[Expr]) -> anyhow::Result> { let mut results = Vec::new(); + let ts_range = extract_timestamp_range(filters); if let Some(table) = self.get_table(project_id, table_name) { for bucket_entry in table.buckets.iter() { - if let Ok(batches) = bucket_entry.batches.read() { - // RecordBatch clone is cheap: Arc + Vec> - // Only clones pointers (~100 bytes/batch), NOT the underlying data - // A 4GB buffer query adds ~1MB overhead, not 4GB + let bucket = bucket_entry.value(); + if !bucket_overlaps_range(bucket, &ts_range) { + continue; + } + if let Ok(batches) = bucket.batches.read() { results.extend(batches.iter().cloned()); } } @@ -344,21 +400,22 @@ impl MemBuffer { /// Query and return partitioned data - one partition per time bucket. /// This enables parallel execution across time buckets. - #[instrument(skip(self), fields(project_id, table_name))] - pub fn query_partitioned(&self, project_id: &str, table_name: &str) -> anyhow::Result>> { + /// Optional filters enable timestamp-based bucket pruning. + #[instrument(skip(self, filters), fields(project_id, table_name))] + pub fn query_partitioned(&self, project_id: &str, table_name: &str, filters: &[Expr]) -> anyhow::Result>> { let mut partitions = Vec::new(); + let ts_range = extract_timestamp_range(filters); if let Some(table) = self.get_table(project_id, table_name) { - // Sort buckets by bucket_id for consistent ordering let mut bucket_ids: Vec = table.buckets.iter().map(|b| *b.key()).collect(); bucket_ids.sort(); for bucket_id in bucket_ids { if let Some(bucket) = table.buckets.get(&bucket_id) + && bucket_overlaps_range(&bucket, &ts_range) && let Ok(batches) = bucket.batches.read() && !batches.is_empty() { - // RecordBatch clone is cheap (~100 bytes/batch), data is Arc-shared partitions.push(batches.clone()); } } @@ -554,6 +611,8 @@ impl MemBuffer { *batches = new_batches; let new_row_count: usize = batches.iter().map(|b| b.num_rows()).sum(); bucket.row_count.store(new_row_count, Ordering::Relaxed); + let new_memory: usize = batches.iter().map(|b| estimate_batch_size(b)).sum(); + bucket.memory_bytes.store(new_memory, Ordering::Relaxed); } if memory_freed > 0 { @@ -593,11 +652,13 @@ impl MemBuffer { .collect::>>()?; let mut total_updated = 0u64; + let mut memory_delta = 0i64; for mut bucket_entry in table.buckets.iter_mut() { let bucket = bucket_entry.value_mut(); let mut batches = bucket.batches.write().map_err(|e| datafusion::error::DataFusionError::Execution(format!("Lock error: {}", e)))?; + let old_memory: usize = batches.iter().map(|b| estimate_batch_size(b)).sum(); let new_batches: Vec = batches .drain(..) .map(|batch| { @@ -646,6 +707,17 @@ impl MemBuffer { .collect::>>()?; *batches = new_batches; + let new_memory: usize = batches.iter().map(|b| estimate_batch_size(b)).sum(); + bucket.memory_bytes.store(new_memory, Ordering::Relaxed); + memory_delta += new_memory as i64 - old_memory as i64; + } + + if memory_delta != 0 { + if memory_delta > 0 { + self.estimated_bytes.fetch_add(memory_delta as usize, Ordering::Relaxed); + } else { + self.estimated_bytes.fetch_sub((-memory_delta) as usize, Ordering::Relaxed); + } } debug!("MemBuffer update: project={}, table={}, rows_updated={}", project_id, table_name, total_updated); diff --git a/src/wal.rs b/src/wal.rs index 836e0ce..ad015d0 100644 --- a/src/wal.rs +++ b/src/wal.rs @@ -184,9 +184,13 @@ pub struct WalManager { impl WalManager { pub fn new(data_dir: PathBuf) -> Result { + Self::with_fsync_ms(data_dir, FSYNC_SCHEDULE_MS) + } + + pub fn with_fsync_ms(data_dir: PathBuf, fsync_ms: u64) -> Result { std::fs::create_dir_all(&data_dir)?; - let wal = Walrus::with_consistency_and_schedule(ReadConsistency::StrictlyAtOnce, FsyncSchedule::Milliseconds(FSYNC_SCHEDULE_MS))?; + let wal = Walrus::with_consistency_and_schedule(ReadConsistency::StrictlyAtOnce, FsyncSchedule::Milliseconds(fsync_ms))?; // Load known topics from index file let meta_dir = data_dir.join(".timefusion_meta"); From 81cf1ffef13242b5314c87005d532c5d4580f955 Mon Sep 17 00:00:00 2001 From: Anthony Alaribe Date: Mon, 16 Feb 2026 15:02:54 +0100 Subject: [PATCH 6/8] Switch WAL to Arrow IPC format, add flush compaction and configurable optimization - WAL serialization now uses Arrow IPC (v129) instead of custom CompactBatch, with automatic fallback to legacy v128 format for existing WAL entries - MemBuffer compacts multiple small batches into a single RecordBatch before flush to reduce small file writes - Optimization window, min file threshold, and light optimize target size are now configurable via maintenance config --- Cargo.lock | 1 + Cargo.toml | 1 + src/config.rs | 9 ++++++ src/database.rs | 66 ++++++++++++++++++-------------------------- src/mem_buffer.rs | 32 +++++++++++++++++++++- src/wal.rs | 70 +++++++++++++++++++++++++++++++++-------------- 6 files changed, 119 insertions(+), 60 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index 77369b9..7622271 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -6871,6 +6871,7 @@ dependencies = [ "ahash 0.8.12", "anyhow", "arrow", + "arrow-ipc", "arrow-json", "arrow-schema", "async-trait", diff --git a/Cargo.toml b/Cargo.toml index ffa18ab..4c2192e 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -8,6 +8,7 @@ tokio = { version = "1.48", features = ["full"] } datafusion = "52.1.0" datafusion-datasource = "52.1.0" arrow = "57.1.0" +arrow-ipc = "57.1.0" arrow-json = "57.1.0" uuid = { version = "1.17", features = ["v4", "serde"] } serde = { version = "1", features = ["derive"] } diff --git a/src/config.rs b/src/config.rs index d71dd9b..a316a5b 100644 --- a/src/config.rs +++ b/src/config.rs @@ -119,6 +119,9 @@ const_default!(d_checkpoint_interval: u64 = 10); const_default!(d_optimize_target: i64 = 128 * 1024 * 1024); const_default!(d_stats_cache_size: usize = 50); const_default!(d_vacuum_retention: u64 = 72); +const_default!(d_optimize_window_hours: u64 = 48); +const_default!(d_compact_min_files: usize = 5); +const_default!(d_light_optimize_target: i64 = 16 * 1024 * 1024); const_default!(d_light_schedule: String = "0 */5 * * * *"); const_default!(d_optimize_schedule: String = "0 */30 * * * *"); const_default!(d_vacuum_schedule: String = "0 0 2 * * *"); @@ -377,6 +380,12 @@ pub struct ParquetConfig { pub struct MaintenanceConfig { #[serde(default = "d_vacuum_retention")] pub timefusion_vacuum_retention_hours: u64, + #[serde(default = "d_optimize_window_hours")] + pub timefusion_optimize_window_hours: u64, + #[serde(default = "d_compact_min_files")] + pub timefusion_compact_min_files: usize, + #[serde(default = "d_light_optimize_target")] + pub timefusion_light_optimize_target_size: i64, #[serde(default = "d_light_schedule")] pub timefusion_light_optimize_schedule: String, #[serde(default = "d_optimize_schedule")] diff --git a/src/database.rs b/src/database.rs index 34caf38..329acd7 100644 --- a/src/database.rs +++ b/src/database.rs @@ -1758,31 +1758,28 @@ impl Database { /// Optimize the Delta table using Z-ordering on timestamp and id columns /// This improves query performance for time-based queries pub async fn optimize_table(&self, table_ref: &Arc>, table_name: &str, _target_size: Option) -> Result<()> { - // Log the start of the optimization operation let start_time = std::time::Instant::now(); - info!("Starting Delta table optimization with Z-ordering (last 28 hours only)"); + let window_hours = self.config.maintenance.timefusion_optimize_window_hours.max(1); + info!("Starting Delta table optimization with Z-ordering (last {} hours)", window_hours); - // Get a clone of the table to avoid holding the lock during the operation let table_clone = { let table = table_ref.read().await; table.clone() }; - // Get configurable target size let target_size = self.config.parquet.timefusion_optimize_target_size; - // Calculate dates for filtering - last 2 days (today and yesterday) - let today = Utc::now().date_naive(); - let yesterday = (Utc::now() - chrono::Duration::days(1)).date_naive(); - info!("Optimizing files from dates: {} and {}", yesterday, today); - - // Create partition filters for the last 2 days - let partition_filters = vec![ - PartitionFilter::try_from(("date", "=", today.to_string().as_str()))?, - PartitionFilter::try_from(("date", "=", yesterday.to_string().as_str()))?, - ]; + // Generate partition filters for each date in the configurable window + let now = Utc::now(); + let num_days = (window_hours / 24).max(1); + let partition_filters: Vec = (0..=num_days) + .filter_map(|days_ago| { + let date = (now - chrono::Duration::days(days_ago as i64)).date_naive(); + PartitionFilter::try_from(("date", "=", date.to_string().as_str())).ok() + }) + .collect(); + info!("Optimizing files from {} date partitions", partition_filters.len()); - // Z-order files for better query performance on timestamp and service_name filters let schema = get_schema(table_name).unwrap_or_else(get_default_schema); let writer_properties = self.create_writer_properties(schema.sorting_columns()); @@ -1797,27 +1794,22 @@ impl Database { match optimize_result { Ok((new_table, metrics)) => { + let min_files = self.config.maintenance.timefusion_compact_min_files; + if metrics.total_considered_files < min_files { + debug!("Skipping optimization commit: {} files < min threshold {}", metrics.total_considered_files, min_files); + return Ok(()); + } let duration = start_time.elapsed(); info!( "Optimization completed in {:?}: {} files removed, {} files added, {} partitions optimized, {} total files considered, {} files skipped", - duration, - metrics.num_files_removed, - metrics.num_files_added, - metrics.partitions_optimized, - metrics.total_considered_files, - metrics.total_files_skipped + duration, metrics.num_files_removed, metrics.num_files_added, metrics.partitions_optimized, metrics.total_considered_files, metrics.total_files_skipped ); - - // Log performance metrics for monitoring if metrics.num_files_removed > 0 { let compression_ratio = metrics.num_files_removed as f64 / metrics.num_files_added as f64; info!("Optimization compression ratio: {:.2}x", compression_ratio); } - - // Update the table reference with the optimized version let mut table = table_ref.write().await; *table = new_table; - Ok(()) } Err(e) => { @@ -1827,11 +1819,8 @@ impl Database { } } - /// Light optimization for small recent files - /// Targets files < 10MB from today's partition only pub async fn optimize_table_light(&self, table_ref: &Arc>, table_name: &str) -> Result<()> { let start_time = std::time::Instant::now(); - // Get a clone of the table to avoid holding the lock during the operation let table_clone = { let table = table_ref.read().await; table.clone() @@ -1840,31 +1829,30 @@ impl Database { let today = Utc::now().date_naive(); info!("Light optimizing files from date: {}", today); - // Create partition filter for today only let partition_filters = vec![PartitionFilter::try_from(("date", "=", today.to_string().as_str()))?]; + let target_size = self.config.maintenance.timefusion_light_optimize_target_size; let schema = get_schema(table_name).unwrap_or_else(get_default_schema); let optimize_result = table_clone .optimize() .with_filters(&partition_filters) .with_type(deltalake::operations::optimize::OptimizeType::Compact) - .with_target_size(16 * 1024 * 1024) + .with_target_size(target_size as u64) .with_writer_properties(self.create_writer_properties(schema.sorting_columns())) - .with_min_commit_interval(tokio::time::Duration::from_secs(30)) // 1 minute min interval + .with_min_commit_interval(tokio::time::Duration::from_secs(30)) .await; match optimize_result { Ok((new_table, metrics)) => { + let min_files = self.config.maintenance.timefusion_compact_min_files; + if metrics.total_considered_files < min_files { + debug!("Skipping light optimization commit: {} files < min threshold {}", metrics.total_considered_files, min_files); + return Ok(()); + } let duration = start_time.elapsed(); - info!( - "Light optimization completed in {:?}: {} files removed, {} files added", - duration, metrics.num_files_removed, metrics.num_files_added, - ); - - // Update the table reference with the optimized version + info!("Light optimization completed in {:?}: {} files removed, {} files added", duration, metrics.num_files_removed, metrics.num_files_added); let mut table = table_ref.write().await; *table = new_table; - Ok(()) } Err(e) => { diff --git a/src/mem_buffer.rs b/src/mem_buffer.rs index f86ccad..94f9473 100644 --- a/src/mem_buffer.rs +++ b/src/mem_buffer.rs @@ -505,11 +505,17 @@ impl MemBuffer { && let Ok(batches) = bucket.batches.read() && !batches.is_empty() { + // Compact multiple small batches into one before flush + let compacted = if batches.len() > 1 { + arrow::compute::concat_batches(&table.schema, &*batches).map_or_else(|_| batches.clone(), |single| vec![single]) + } else { + batches.clone() + }; result.push(FlushableBucket { project_id: project_id.to_string(), table_name: table_name.to_string(), bucket_id, - batches: batches.clone(), + batches: compacted, row_count: bucket.row_count.load(Ordering::Relaxed), }); } @@ -1081,6 +1087,30 @@ mod tests { assert_eq!(results.len(), 10, "All 10 inserts should succeed"); } + #[test] + fn test_batch_compaction_on_flush() { + let buffer = MemBuffer::new(); + let ts = chrono::Utc::now().timestamp_micros(); + + // Insert 10 small batches into the same bucket + let total_rows = 10; + for i in 0..total_rows { + let batch = create_multi_row_batch(vec![i as i64], vec!["test"]); + buffer.insert("project1", "table1", batch, ts).unwrap(); + } + + let stats = buffer.get_stats(); + assert_eq!(stats.total_batches, total_rows); + + // get_flushable_buckets should compact into 1 batch + let cutoff = MemBuffer::compute_bucket_id(ts) + 1; + let flushable = buffer.get_flushable_buckets(cutoff); + assert_eq!(flushable.len(), 1); + assert_eq!(flushable[0].batches.len(), 1); + assert_eq!(flushable[0].row_count, total_rows); + assert_eq!(flushable[0].batches[0].num_rows(), total_rows); + } + #[test] fn test_negative_bucket_ids_pre_1970() { // Integer division truncates toward zero: -1 / N = 0, -N / N = -1 diff --git a/src/wal.rs b/src/wal.rs index ad015d0..5fbc9d9 100644 --- a/src/wal.rs +++ b/src/wal.rs @@ -2,6 +2,8 @@ use crate::schema_loader::{get_default_schema, get_schema}; use arrow::array::{Array, ArrayRef, RecordBatch, make_array}; use arrow::buffer::{Buffer, NullBuffer}; use arrow::datatypes::{DataType, SchemaRef}; +use arrow_ipc::reader::StreamReader; +use arrow_ipc::writer::{IpcWriteOptions, StreamWriter}; use bincode::{Decode, Encode}; use dashmap::DashSet; use std::path::PathBuf; @@ -35,6 +37,8 @@ pub enum WalError { const WAL_MAGIC: [u8; 4] = [0x57, 0x41, 0x4C, 0x32]; // "WAL2" /// Version byte must be > 2 to distinguish from legacy operation bytes (0=Insert, 1=Delete, 2=Update) const WAL_VERSION: u8 = 128; +/// Version 129: Arrow IPC format - embeds schema, handles all Arrow types automatically +const WAL_VERSION_IPC: u8 = 129; const BINCODE_CONFIG: bincode::config::Configuration = bincode::config::standard(); /// Maximum size for a single record batch (100MB) - prevents unbounded memory allocation from malicious/corrupted WAL const MAX_BATCH_SIZE: usize = 100 * 1024 * 1024; @@ -112,6 +116,7 @@ struct CompactBatch { columns: Vec, } +#[allow(dead_code)] // Kept for legacy WAL v128 test coverage impl CompactColumn { fn from_array(array: &dyn Array) -> Self { let data = array.to_data(); @@ -382,8 +387,11 @@ impl WalManager { } pub fn deserialize_batch(data: &[u8], table_name: &str) -> Result { - let schema = get_schema(table_name).map(|s| s.schema_ref()).unwrap_or_else(|| get_default_schema().schema_ref()); - deserialize_record_batch(data, &schema) + // Try IPC first (v129+), fall back to legacy CompactBatch (v128) + deserialize_record_batch_ipc(data).or_else(|_| { + let schema = get_schema(table_name).map(|s| s.schema_ref()).unwrap_or_else(|| get_default_schema().schema_ref()); + deserialize_record_batch_legacy(data, &schema) + }) } pub fn list_topics(&self) -> Result, WalError> { @@ -417,36 +425,44 @@ impl WalManager { } fn serialize_record_batch(batch: &RecordBatch) -> Result, WalError> { - let compact = CompactBatch { - num_rows: batch.num_rows(), - columns: batch.columns().iter().map(|c| CompactColumn::from_array(c.as_ref())).collect(), - }; - bincode::encode_to_vec(&compact, BINCODE_CONFIG).map_err(WalError::BincodeEncode) + let mut buf = Vec::new(); + let options = IpcWriteOptions::default(); + let mut writer = StreamWriter::try_new_with_options(&mut buf, &batch.schema(), options)?; + writer.write(batch)?; + writer.finish()?; + drop(writer); + Ok(buf) } -fn deserialize_record_batch(data: &[u8], schema: &SchemaRef) -> Result { +fn deserialize_record_batch_ipc(data: &[u8]) -> Result { if data.len() > MAX_BATCH_SIZE { - return Err(WalError::BatchTooLarge { - size: data.len(), - max: MAX_BATCH_SIZE, - }); + return Err(WalError::BatchTooLarge { size: data.len(), max: MAX_BATCH_SIZE }); } + let reader = StreamReader::try_new(std::io::Cursor::new(data), None)?; + for batch in reader { + return Ok(batch?); + } + Err(WalError::EmptyBatch) +} +/// Legacy CompactBatch deserialization for WAL version 128 +fn deserialize_record_batch_legacy(data: &[u8], schema: &SchemaRef) -> Result { + if data.len() > MAX_BATCH_SIZE { + return Err(WalError::BatchTooLarge { size: data.len(), max: MAX_BATCH_SIZE }); + } let (compact, _): (CompactBatch, _) = bincode::decode_from_slice(data, BINCODE_CONFIG)?; - let arrays: Result, WalError> = compact .columns .iter() .zip(schema.fields()) .map(|(col, field)| Ok(make_array(col.to_array_data(field.data_type(), compact.num_rows)?))) .collect(); - RecordBatch::try_new(schema.clone(), arrays?).map_err(WalError::ArrowIpc) } fn serialize_wal_entry(entry: &WalEntry) -> Result, WalError> { let mut buffer = WAL_MAGIC.to_vec(); - buffer.push(WAL_VERSION); + buffer.push(WAL_VERSION_IPC); buffer.push(entry.operation as u8); buffer.extend(bincode::encode_to_vec(entry, BINCODE_CONFIG)?); Ok(buffer) @@ -467,10 +483,10 @@ fn deserialize_wal_entry(data: &[u8]) -> Result { if data.len() < 6 { return Err(WalError::TooShort { len: data.len() }); } - if data[4] != WAL_VERSION { + if data[4] != WAL_VERSION && data[4] != WAL_VERSION_IPC { return Err(WalError::UnsupportedVersion { version: data[4], - expected: WAL_VERSION, + expected: WAL_VERSION_IPC, }); } WalOperation::try_from(data[5])?; @@ -520,11 +536,25 @@ mod tests { } #[test] - fn test_record_batch_serialization() { + fn test_record_batch_ipc_serialization() { let batch = create_test_batch(); - let schema = batch.schema(); let serialized = serialize_record_batch(&batch).unwrap(); - let deserialized = deserialize_record_batch(&serialized, &schema).unwrap(); + let deserialized = deserialize_record_batch_ipc(&serialized).unwrap(); + assert_eq!(batch.num_rows(), deserialized.num_rows()); + assert_eq!(batch.num_columns(), deserialized.num_columns()); + } + + #[test] + fn test_record_batch_legacy_serialization() { + let batch = create_test_batch(); + let schema = batch.schema(); + // Serialize using legacy CompactBatch format + let compact = CompactBatch { + num_rows: batch.num_rows(), + columns: batch.columns().iter().map(|c| CompactColumn::from_array(c.as_ref())).collect(), + }; + let serialized = bincode::encode_to_vec(&compact, BINCODE_CONFIG).unwrap(); + let deserialized = deserialize_record_batch_legacy(&serialized, &schema).unwrap(); assert_eq!(batch.num_rows(), deserialized.num_rows()); assert_eq!(batch.num_columns(), deserialized.num_columns()); } From be46a8fcd3366de74a7070ff543391c565db557e Mon Sep 17 00:00:00 2001 From: Anthony Alaribe Date: Mon, 16 Feb 2026 19:48:08 +0100 Subject: [PATCH 7/8] Optimize read perf 70-89%, fix write regressions, add benchmarks - Switch TimeBucket from RwLock to parking_lot::Mutex for lower overhead - Add compact-on-read in query()/query_partitioned(): first read compacts batches in-place, subsequent reads get pre-compacted single batch - Remove insert-time compaction that caused +64% batch_api write regression - Revert WAL serialization from Arrow IPC back to bincode CompactBatch (IPC schema preamble overhead caused +18-20% SQL insert regression) - Keep IPC deserialization as fallback for backward compatibility - Skip VariantToJsonExec wrapper for tables with no Variant columns - Add bloom filter config (timefusion_bloom_filter_disabled), enabled by default - Add WAL file monitoring and emergency flush on file count threshold - Add criterion benchmarks for write, read, S3 flush, and S3 read paths --- Cargo.lock | 181 +++++++++++++++++ Cargo.toml | 5 + benches/core_benchmarks.rs | 334 +++++++++++++++++++++++++++++++ schemas/otel_logs_and_spans.yaml | 18 +- src/buffered_write_layer.rs | 10 + src/config.rs | 8 + src/database.rs | 41 +--- src/mem_buffer.rs | 94 +++++---- src/wal.rs | 63 +++--- 9 files changed, 643 insertions(+), 111 deletions(-) create mode 100644 benches/core_benchmarks.rs diff --git a/Cargo.lock b/Cargo.lock index 7622271..93dc150 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -81,6 +81,12 @@ dependencies = [ "libc", ] +[[package]] +name = "anes" +version = "0.1.6" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "4b46cbb362ab8752921c97e041f5e366ee6297bd428a31275b9fcf1e380f7299" + [[package]] name = "anstream" version = "0.6.21" @@ -1226,6 +1232,12 @@ dependencies = [ "libbz2-rs-sys", ] +[[package]] +name = "cast" +version = "0.3.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "37b2a672a2cb129a2e41c10b1224bb368f9f37a2b16b612598138befd7b37eb5" + [[package]] name = "cc" version = "1.2.50" @@ -1274,6 +1286,33 @@ dependencies = [ "phf 0.12.1", ] +[[package]] +name = "ciborium" +version = "0.2.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "42e69ffd6f0917f5c029256a24d0161db17cea3997d185db0d35926308770f0e" +dependencies = [ + "ciborium-io", + "ciborium-ll", + "serde", +] + +[[package]] +name = "ciborium-io" +version = "0.2.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "05afea1e0a06c9be33d539b876f1ce3692f4afea2cb41f740e7743225ed1c757" + +[[package]] +name = "ciborium-ll" +version = "0.2.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "57663b653d948a338bfb3eeba9bb2fd5fcfaecb9e199e87e1eda4d9e8b240fd9" +dependencies = [ + "ciborium-io", + "half", +] + [[package]] name = "clap" version = "4.5.53" @@ -1517,6 +1556,44 @@ dependencies = [ "cfg-if", ] +[[package]] +name = "criterion" +version = "0.5.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "f2b12d017a929603d80db1831cd3a24082f8137ce19c69e6447f54f5fc8d692f" +dependencies = [ + "anes", + "cast", + "ciborium", + "clap", + "criterion-plot", + "futures", + "is-terminal", + "itertools 0.10.5", + "num-traits", + "once_cell", + "oorandom", + "plotters", + "rayon", + "regex", + "serde", + "serde_derive", + "serde_json", + "tinytemplate", + "tokio", + "walkdir", +] + +[[package]] +name = "criterion-plot" +version = "0.5.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "6b50826342786a51a89e2da3a28f1c32b06e387201bc2d19791f622c673706b1" +dependencies = [ + "cast", + "itertools 0.10.5", +] + [[package]] name = "croner" version = "3.0.1" @@ -1528,6 +1605,25 @@ dependencies = [ "strum", ] +[[package]] +name = "crossbeam-deque" +version = "0.8.6" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "9dd111b7b7f7d55b72c0a6ae361660ee5853c9af73f70c3c2ef6858b950e2e51" +dependencies = [ + "crossbeam-epoch", + "crossbeam-utils", +] + +[[package]] +name = "crossbeam-epoch" +version = "0.9.18" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "5b82ac4a3c2ca9c3460964f020e1402edd5753411d7737aa39c3714ad1b5420e" +dependencies = [ + "crossbeam-utils", +] + [[package]] name = "crossbeam-queue" version = "0.3.12" @@ -3993,12 +4089,32 @@ dependencies = [ "serde", ] +[[package]] +name = "is-terminal" +version = "0.4.17" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "3640c1c38b8e4e43584d8df18be5fc6b0aa314ce6ebf51b53313d4306cca8e46" +dependencies = [ + "hermit-abi", + "libc", + "windows-sys 0.61.2", +] + [[package]] name = "is_terminal_polyfill" version = "1.70.2" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "a6cb138bb79a146c1bd460005623e142ef0181e3d0219cb493e02f7d08a35695" +[[package]] +name = "itertools" +version = "0.10.5" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "b0fd2260e829bddf4cb6ea802289de2f86d6a7a690192fbe91b3f46e0f2c8473" +dependencies = [ + "either", +] + [[package]] name = "itertools" version = "0.13.0" @@ -4662,6 +4778,12 @@ version = "1.70.2" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "384b8ab6d37215f3c5301a95a4accb5d64aa607f1fcb26a11b5303878451b4fe" +[[package]] +name = "oorandom" +version = "11.1.5" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "d6790f58c7ff633d8771f42965289203411a5e5c68388703c06e14f24770b41e" + [[package]] name = "openssl-probe" version = "0.1.6" @@ -5097,6 +5219,34 @@ version = "0.3.32" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "7edddbd0b52d732b21ad9a5fab5c704c14cd949e5e9a1ec5929a24fded1b904c" +[[package]] +name = "plotters" +version = "0.3.7" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "5aeb6f403d7a4911efb1e33402027fc44f29b5bf6def3effcc22d7bb75f2b747" +dependencies = [ + "num-traits", + "plotters-backend", + "plotters-svg", + "wasm-bindgen", + "web-sys", +] + +[[package]] +name = "plotters-backend" +version = "0.3.7" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "df42e13c12958a16b3f7f4386b9ab1f3e7933914ecea48da7139435263a4172a" + +[[package]] +name = "plotters-svg" +version = "0.3.7" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "51bae2ac328883f7acdfea3d66a7c35751187f870bc81f94563733a154d7a670" +dependencies = [ + "plotters-backend", +] + [[package]] name = "portable-atomic" version = "1.12.0" @@ -5470,6 +5620,26 @@ dependencies = [ "rand_core 0.6.4", ] +[[package]] +name = "rayon" +version = "1.11.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "368f01d005bf8fd9b1206fb6fa653e6c4a81ceb1466406b81792d87c5677a58f" +dependencies = [ + "either", + "rayon-core", +] + +[[package]] +name = "rayon-core" +version = "1.13.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "22e18b0f0062d30d4230b2e85ff77fdfe4326feb054b9783a3460d8435c8ab91" +dependencies = [ + "crossbeam-deque", + "crossbeam-utils", +] + [[package]] name = "recursive" version = "0.1.1" @@ -6885,6 +7055,7 @@ dependencies = [ "chrono", "chrono-tz", "color-eyre", + "criterion", "dashmap", "datafusion", "datafusion-common", @@ -6962,6 +7133,16 @@ dependencies = [ "zerovec", ] +[[package]] +name = "tinytemplate" +version = "1.2.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "be4d6b5f19ff7664e8c98d03e2139cb510db9b0a60b55f8e8709b689d939b6bc" +dependencies = [ + "serde", + "serde_json", +] + [[package]] name = "tinyvec" version = "1.10.0" diff --git a/Cargo.toml b/Cargo.toml index 4c2192e..9082f14 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -92,6 +92,11 @@ scopeguard = "1.2.0" rand = "0.9.2" tempfile = "3" test-case = "3.3" +criterion = { version = "0.5", features = ["html_reports", "async_tokio"] } + +[[bench]] +name = "core_benchmarks" +harness = false [features] default = [] diff --git a/benches/core_benchmarks.rs b/benches/core_benchmarks.rs new file mode 100644 index 0000000..58d91d7 --- /dev/null +++ b/benches/core_benchmarks.rs @@ -0,0 +1,334 @@ +use criterion::{Criterion, criterion_group, criterion_main}; +use std::path::PathBuf; +use std::sync::Arc; +use timefusion::buffered_write_layer::BufferedWriteLayer; +use timefusion::config::AppConfig; +use timefusion::database::Database; +use timefusion::test_utils::test_helpers::{json_to_batch, test_span}; + +use datafusion::execution::context::SessionContext; + +fn bench_config(name: &str) -> Arc { + let uuid = &uuid::Uuid::new_v4().to_string()[..8].to_string(); + let mut cfg = AppConfig::default(); + cfg.cache.timefusion_foyer_disabled = true; + cfg.core.timefusion_table_prefix = format!("bench-{}-{}", name, uuid); + cfg.core.timefusion_data_dir = PathBuf::from(format!("/tmp/timefusion-bench-{}-{}", name, uuid)); + Arc::new(cfg) +} + +fn minio_config(name: &str) -> Arc { + let uuid = &uuid::Uuid::new_v4().to_string()[..8].to_string(); + let mut cfg = AppConfig::default(); + cfg.aws.aws_s3_bucket = Some("timefusion-tests".to_string()); + cfg.aws.aws_access_key_id = Some("minioadmin".to_string()); + cfg.aws.aws_secret_access_key = Some("minioadmin".to_string()); + cfg.aws.aws_s3_endpoint = "http://127.0.0.1:9000".to_string(); + cfg.aws.aws_default_region = Some("us-east-1".to_string()); + cfg.aws.aws_allow_http = Some("true".to_string()); + cfg.cache.timefusion_foyer_disabled = true; + cfg.core.timefusion_table_prefix = format!("bench-{}-{}", name, uuid); + cfg.core.timefusion_data_dir = PathBuf::from(format!("/tmp/timefusion-bench-{}-{}", name, uuid)); + Arc::new(cfg) +} + +fn minio_flush_config(name: &str) -> Arc { + let mut cfg = (*minio_config(name)).clone(); + cfg.buffer.timefusion_flush_immediately = true; + Arc::new(cfg) +} + + +fn is_minio_available() -> bool { + std::net::TcpStream::connect("127.0.0.1:9000").is_ok() +} + +/// Setup for in-memory write benchmarks (no S3 needed). +async fn setup_write_bench(name: &str) -> (SessionContext, Arc, String) { + let cfg = bench_config(name); + unsafe { std::env::set_var("WALRUS_DATA_DIR", cfg.core.wal_dir()) }; + let layer = Arc::new(BufferedWriteLayer::with_config(Arc::clone(&cfg)).unwrap()); + let db = Arc::new(Database::with_config(Arc::clone(&cfg)).await.unwrap().with_buffered_layer(Arc::clone(&layer))); + let mut ctx = db.clone().create_session_context(); + db.setup_session_context(&mut ctx).unwrap(); + let pid = format!("bench_{}", &uuid::Uuid::new_v4().to_string()[..8]); + (ctx, db, pid) +} + +/// Setup for read benchmarks (requires MinIO). Pre-inserts data to MemBuffer + registers tables. +async fn setup_read_bench(name: &str, pre_insert: usize) -> (SessionContext, Arc, String) { + let cfg = minio_config(name); + unsafe { std::env::set_var("WALRUS_DATA_DIR", cfg.core.wal_dir()) }; + let layer = Arc::new(BufferedWriteLayer::with_config(Arc::clone(&cfg)).unwrap()); + let db = Arc::new(Database::with_config(Arc::clone(&cfg)).await.unwrap().with_buffered_layer(Arc::clone(&layer))); + let mut ctx = db.clone().create_session_context(); + db.setup_session_context(&mut ctx).unwrap(); + + let pid = format!("bench_{}", &uuid::Uuid::new_v4().to_string()[..8]); + for i in 0..pre_insert { + let batch = json_to_batch(vec![test_span(&format!("id_{i}"), &format!("span_{i}"), &pid)]).unwrap(); + db.insert_records_batch(&pid, "otel_logs_and_spans", vec![batch], false).await.unwrap(); + } + (ctx, db, pid) +} + +/// Setup for S3 flush benchmarks (requires MinIO, flush_immediately=true). +async fn setup_s3_bench(name: &str) -> (SessionContext, Arc, String) { + let cfg = minio_flush_config(name); + unsafe { std::env::set_var("WALRUS_DATA_DIR", cfg.core.wal_dir()) }; + + let db_for_cb = Database::with_config(Arc::clone(&cfg)).await.unwrap(); + let db_clone = db_for_cb.clone(); + let delta_cb: timefusion::buffered_write_layer::DeltaWriteCallback = + Arc::new(move |project_id, table_name, batches| { + let db = db_clone.clone(); + Box::pin(async move { db.insert_records_batch(&project_id, &table_name, batches, true).await }) + }); + let layer = Arc::new(BufferedWriteLayer::with_config(Arc::clone(&cfg)).unwrap().with_delta_writer(delta_cb)); + let db = db_for_cb.with_buffered_layer(Arc::clone(&layer)); + + let pid = format!("bench_{}", &uuid::Uuid::new_v4().to_string()[..8]); + db.get_or_create_table(&pid, "otel_logs_and_spans").await.unwrap(); + + let db = Arc::new(db); + let mut ctx = db.clone().create_session_context(); + db.setup_session_context(&mut ctx).unwrap(); + (ctx, db, pid) +} + +fn now_ts() -> String { + chrono::Utc::now().format("%Y-%m-%dT%H:%M:%S").to_string() +} + +fn today() -> String { + chrono::Utc::now().format("%Y-%m-%d").to_string() +} + +fn insert_sql(project_id: &str, n: usize) -> String { + let date = today(); + let values: Vec = (0..n) + .map(|i| { + let ts = now_ts(); + format!( + "('{}', '{}', TIMESTAMP '{}', 'id_{i}', 'bench_span', 'INFO', ARRAY[]::varchar[], ARRAY['summary'])", + project_id, date, ts + ) + }) + .collect(); + format!( + "INSERT INTO otel_logs_and_spans (project_id, date, timestamp, id, name, level, hashes, summary) VALUES {}", + values.join(", ") + ) +} + +// ============================================================================= +// Group 1: In-Memory Write Throughput (no S3 needed) +// ============================================================================= + +fn bench_inmemory_writes(c: &mut Criterion) { + let rt = tokio::runtime::Runtime::new().unwrap(); + let mut group = c.benchmark_group("inmemory_write"); + + { + let (ctx, _db, pid) = rt.block_on(setup_write_bench("w1")); + let sql = insert_sql(&pid, 1); + group.bench_function("sql_insert_1_row", |b| { + b.to_async(&rt).iter(|| { + let (sql, ctx) = (sql.clone(), ctx.clone()); + async move { ctx.sql(&sql).await.unwrap().collect().await.unwrap() } + }) + }); + } + + { + let (ctx, _db, pid) = rt.block_on(setup_write_bench("w100")); + let sql = insert_sql(&pid, 100); + group.bench_function("sql_insert_100_rows", |b| { + b.to_async(&rt).iter(|| { + let (sql, ctx) = (sql.clone(), ctx.clone()); + async move { ctx.sql(&sql).await.unwrap().collect().await.unwrap() } + }) + }); + } + + // Direct batch API (bypasses SQL parsing) + { + let cfg = bench_config("wapi"); + unsafe { std::env::set_var("WALRUS_DATA_DIR", cfg.core.wal_dir()) }; + let layer = rt.block_on(async { Arc::new(BufferedWriteLayer::with_config(Arc::clone(&cfg)).unwrap()) }); + let db = rt.block_on(async { Arc::new(Database::with_config(cfg).await.unwrap().with_buffered_layer(layer)) }); + let pid = format!("bench_{}", &uuid::Uuid::new_v4().to_string()[..8]); + let batches: Vec<_> = (0..10).map(|i| json_to_batch(vec![test_span(&format!("id_{i}"), "span", &pid)]).unwrap()).collect(); + group.bench_function("batch_api_insert_10_rows", |b| { + let (db, pid, batches) = (db.clone(), pid.clone(), batches.clone()); + b.to_async(&rt).iter(|| { + let (db, pid, batches) = (db.clone(), pid.clone(), batches.clone()); + async move { db.insert_records_batch(&pid, "otel_logs_and_spans", batches, false).await.unwrap() } + }) + }); + } + + // 4 concurrent INSERTs + { + let (ctx, _db, pid) = rt.block_on(setup_write_bench("wconc")); + let sqls: Vec<_> = (0..4).map(|_| insert_sql(&pid, 1)).collect(); + group.bench_function("sql_insert_concurrent_4", |b| { + b.to_async(&rt).iter(|| { + let (ctx, sqls) = (ctx.clone(), sqls.clone()); + async move { + futures::future::join_all(sqls.iter().map(|s| { + let (ctx, s) = (ctx.clone(), s.clone()); + async move { ctx.sql(&s).await.unwrap().collect().await.unwrap() } + })).await; + } + }) + }); + } + + group.finish(); +} + +// ============================================================================= +// Group 2: Read Throughput (requires MinIO — reads from MemBuffer + Delta union) +// ============================================================================= + +fn bench_reads(c: &mut Criterion) { + let rt = tokio::runtime::Runtime::new().unwrap(); + let mut group = c.benchmark_group("read"); + + if !is_minio_available() { + eprintln!("MinIO not available at 127.0.0.1:9000, skipping read benchmarks"); + group.finish(); + return; + } + + let (ctx, _db, pid) = rt.block_on(setup_read_bench("read", 1000)); + + group.bench_function("sql_select_count", |b| { + let (ctx, sql) = (ctx.clone(), format!( + "SELECT COUNT(*) FROM otel_logs_and_spans WHERE project_id = '{pid}'" + )); + b.to_async(&rt).iter(|| { + let (ctx, sql) = (ctx.clone(), sql.clone()); + async move { ctx.sql(&sql).await.unwrap().collect().await.unwrap() } + }) + }); + + group.bench_function("sql_select_filter_level", |b| { + let (ctx, sql) = (ctx.clone(), format!( + "SELECT id, name FROM otel_logs_and_spans WHERE project_id = '{pid}' AND level = 'ERROR'" + )); + b.to_async(&rt).iter(|| { + let (ctx, sql) = (ctx.clone(), sql.clone()); + async move { ctx.sql(&sql).await.unwrap().collect().await.unwrap() } + }) + }); + + group.bench_function("sql_select_time_range", |b| { + let now = chrono::Utc::now().format("%Y-%m-%dT%H:%M:%S").to_string(); + let (ctx, sql) = (ctx.clone(), format!( + "SELECT id, name, timestamp FROM otel_logs_and_spans WHERE project_id = '{pid}' AND timestamp <= TIMESTAMP '{now}' LIMIT 100" + )); + b.to_async(&rt).iter(|| { + let (ctx, sql) = (ctx.clone(), sql.clone()); + async move { ctx.sql(&sql).await.unwrap().collect().await.unwrap() } + }) + }); + + group.bench_function("sql_select_aggregation", |b| { + let (ctx, sql) = (ctx.clone(), format!( + "SELECT level, COUNT(*) as cnt FROM otel_logs_and_spans WHERE project_id = '{pid}' GROUP BY level" + )); + b.to_async(&rt).iter(|| { + let (ctx, sql) = (ctx.clone(), sql.clone()); + async move { ctx.sql(&sql).await.unwrap().collect().await.unwrap() } + }) + }); + + group.finish(); +} + +// ============================================================================= +// Group 3: S3 Write Throughput (requires MinIO, flush_immediately=true) +// ============================================================================= + +fn bench_s3_writes(c: &mut Criterion) { + let rt = tokio::runtime::Runtime::new().unwrap(); + let mut group = c.benchmark_group("s3_write"); + group.sample_size(10); + + if !is_minio_available() { + eprintln!("MinIO not available at 127.0.0.1:9000, skipping S3 write benchmarks"); + group.finish(); + return; + } + + let (ctx, _db, pid) = rt.block_on(setup_s3_bench("s3w")); + let sql = insert_sql(&pid, 100); + group.bench_function("s3_insert_and_flush_100", |b| { + b.to_async(&rt).iter(|| { + let (ctx, sql) = (ctx.clone(), sql.clone()); + async move { ctx.sql(&sql).await.unwrap().collect().await.unwrap() } + }) + }); + + group.finish(); +} + +// ============================================================================= +// Group 4: S3 Read Throughput (requires MinIO, data flushed to Delta) +// ============================================================================= + +fn bench_s3_reads(c: &mut Criterion) { + let rt = tokio::runtime::Runtime::new().unwrap(); + let mut group = c.benchmark_group("s3_read"); + group.sample_size(10); + + if !is_minio_available() { + eprintln!("MinIO not available at 127.0.0.1:9000, skipping S3 read benchmarks"); + group.finish(); + return; + } + + let (ctx, _db, pid) = rt.block_on(setup_s3_bench("s3r")); + + // Pre-populate with data flushed to Delta (flush_immediately=true) + let insert = insert_sql(&pid, 100); + rt.block_on(async { ctx.sql(&insert).await.unwrap().collect().await.unwrap() }); + + group.bench_function("s3_select_count", |b| { + let (ctx, sql) = (ctx.clone(), format!( + "SELECT COUNT(*) FROM otel_logs_and_spans WHERE project_id = '{pid}'" + )); + b.to_async(&rt).iter(|| { + let (ctx, sql) = (ctx.clone(), sql.clone()); + async move { ctx.sql(&sql).await.unwrap().collect().await.unwrap() } + }) + }); + + group.bench_function("s3_select_filter", |b| { + let (ctx, sql) = (ctx.clone(), format!( + "SELECT id, name FROM otel_logs_and_spans WHERE project_id = '{pid}' AND level = 'INFO'" + )); + b.to_async(&rt).iter(|| { + let (ctx, sql) = (ctx.clone(), sql.clone()); + async move { ctx.sql(&sql).await.unwrap().collect().await.unwrap() } + }) + }); + + group.bench_function("s3_select_time_range", |b| { + let now = chrono::Utc::now().format("%Y-%m-%dT%H:%M:%S").to_string(); + let (ctx, sql) = (ctx.clone(), format!( + "SELECT id, name, timestamp FROM otel_logs_and_spans WHERE project_id = '{pid}' AND timestamp <= TIMESTAMP '{now}' LIMIT 100" + )); + b.to_async(&rt).iter(|| { + let (ctx, sql) = (ctx.clone(), sql.clone()); + async move { ctx.sql(&sql).await.unwrap().collect().await.unwrap() } + }) + }); + + group.finish(); +} + +criterion_group!(benches, bench_inmemory_writes, bench_reads, bench_s3_writes, bench_s3_reads); +criterion_main!(benches); diff --git a/schemas/otel_logs_and_spans.yaml b/schemas/otel_logs_and_spans.yaml index f3ebe86..1b383da 100644 --- a/schemas/otel_logs_and_spans.yaml +++ b/schemas/otel_logs_and_spans.yaml @@ -2,10 +2,20 @@ table_name: otel_logs_and_spans partitions: - project_id - date -sorting_columns: [] -z_order_columns: - - timestamp - - resource___service___name +sorting_columns: + - name: level + descending: false + nulls_first: false + - name: status_code + descending: false + nulls_first: false + - name: resource___service___name + descending: false + nulls_first: false + - name: timestamp + descending: false + nulls_first: false +z_order_columns: [] fields: - name: date data_type: Date32 diff --git a/src/buffered_write_layer.rs b/src/buffered_write_layer.rs index 6eea8f7..448665b 100644 --- a/src/buffered_write_layer.rs +++ b/src/buffered_write_layer.rs @@ -327,6 +327,16 @@ impl BufferedWriteLayer { if let Err(e) = self.flush_completed_buckets().await { error!("Flush task error: {}", e); } + // WAL monitoring: check file accumulation + let (file_count, total_bytes) = self.wal.wal_stats(); + info!("WAL stats: {} files, {}MB", file_count, total_bytes / (1024 * 1024)); + let max_files = self.config.buffer.wal_max_file_count(); + if max_files > 0 && file_count > max_files { + warn!("WAL file count {} exceeds threshold {}, triggering emergency flush", file_count, max_files); + if let Err(e) = self.flush_all_now().await { + error!("Emergency WAL flush failed: {}", e); + } + } } _ = self.shutdown.cancelled() => { info!("Flush task shutting down"); diff --git a/src/config.rs b/src/config.rs index a316a5b..691381d 100644 --- a/src/config.rs +++ b/src/config.rs @@ -102,6 +102,7 @@ const_default!(d_shutdown_timeout: u64 = 5); const_default!(d_wal_corruption_threshold: usize = 10); const_default!(d_flush_parallelism: usize = 4); const_default!(d_wal_fsync_ms: u64 = 200); +const_default!(d_wal_max_files: usize = 200); const_default!(d_foyer_memory_mb: usize = 512); const_default!(d_foyer_disk_gb: usize = 100); const_default!(d_foyer_ttl: u64 = 604_800); // 7 days @@ -269,6 +270,8 @@ pub struct BufferConfig { pub timefusion_flush_immediately: bool, #[serde(default = "d_wal_fsync_ms")] pub timefusion_wal_fsync_ms: u64, + #[serde(default = "d_wal_max_files")] + pub timefusion_wal_max_file_count: usize, } impl BufferConfig { @@ -296,6 +299,9 @@ impl BufferConfig { pub fn wal_fsync_ms(&self) -> u64 { self.timefusion_wal_fsync_ms.max(1) } + pub fn wal_max_file_count(&self) -> usize { + self.timefusion_wal_max_file_count + } pub fn compute_shutdown_timeout(&self, current_memory_mb: usize) -> Duration { Duration::from_secs((self.timefusion_shutdown_timeout_secs.max(1) + (current_memory_mb / 100) as u64).min(300)) @@ -374,6 +380,8 @@ pub struct ParquetConfig { pub timefusion_optimize_target_size: i64, #[serde(default = "d_stats_cache_size")] pub timefusion_stats_cache_size: usize, + #[serde(default)] + pub timefusion_bloom_filter_disabled: bool, } #[derive(Debug, Clone, Deserialize)] diff --git a/src/database.rs b/src/database.rs index 329acd7..760db8d 100644 --- a/src/database.rs +++ b/src/database.rs @@ -42,8 +42,6 @@ use instrumented_object_store::instrument_object_store; use serde::{Deserialize, Serialize}; use sqlx::{PgPool, postgres::PgPoolOptions}; use std::fmt; -use std::sync::Mutex; -use std::sync::OnceLock; use std::{any::Any, collections::HashMap, sync::Arc}; use tokio::sync::RwLock; use tokio_util::sync::CancellationToken; @@ -51,14 +49,6 @@ use tracing::field::Empty; use tracing::{Instrument, debug, error, info, instrument, warn}; use url::Url; -/// Mutex to serialize access to environment variable modifications. -/// Required because delta-rs uses std::env::var() for AWS credential resolution, -/// and std::env::set_var is unsafe in multi-threaded contexts. -static ENV_MUTEX: OnceLock> = OnceLock::new(); -fn env_mutex() -> &'static Mutex<()> { - ENV_MUTEX.get_or_init(|| Mutex::new(())) -} - // Unified tables: one Delta table per schema (table_name -> DeltaTable) // All default projects share the same table, with project_id as a partition column pub type UnifiedTables = Arc>>>>; @@ -513,6 +503,10 @@ impl Database { .set_dictionary_page_size_limit(8388608) // 8MB // Enable statistics for better query optimization .set_statistics_enabled(EnabledStatistics::Page) + // Enable bloom filters for predicate pushdown (read-side already enabled) + .set_bloom_filter_enabled(!self.config.parquet.timefusion_bloom_filter_disabled) + .set_bloom_filter_fpp(0.01) + .set_bloom_filter_ndv(100_000) // Set page row count limit for better compression .set_data_page_row_count_limit(page_row_count_limit) // Set sorting columns for better query performance on sorted data @@ -1577,26 +1571,9 @@ impl Database { } /// Creates or loads a DeltaTable with proper configuration. - /// Sets environment variables from storage_options to ensure delta-rs credential resolution works. async fn create_or_load_delta_table( &self, storage_uri: &str, storage_options: HashMap, cached_store: Arc, ) -> Result { - // delta-rs uses std::env::var() for AWS credential resolution. - // We serialize access with ENV_MUTEX to prevent data races from concurrent set_var calls. - { - let _guard = env_mutex().lock(); - for (key, value) in &storage_options { - if key.starts_with("AWS_") { - // SAFETY: Protected by ENV_MUTEX. set_var is only unsafe due to potential - // concurrent reads, which we prevent by holding the mutex during the entire - // block. The mutex ensures only one thread modifies env vars at a time. - unsafe { - std::env::set_var(key, value); - } - } - } - } - DeltaTableBuilder::from_url(Url::parse(storage_uri)?)? .with_storage_backend(cached_store.clone(), Url::parse(storage_uri)?) .with_storage_options(storage_options.clone()) @@ -1786,7 +1763,11 @@ impl Database { let optimize_result = table_clone .optimize() .with_filters(&partition_filters) - .with_type(deltalake::operations::optimize::OptimizeType::ZOrder(schema.z_order_columns.clone())) + .with_type(if schema.z_order_columns.is_empty() { + deltalake::operations::optimize::OptimizeType::Compact + } else { + deltalake::operations::optimize::OptimizeType::ZOrder(schema.z_order_columns.clone()) + }) .with_target_size(target_size as u64) .with_writer_properties(writer_properties) .with_min_commit_interval(tokio::time::Duration::from_secs(10 * 60)) @@ -2451,9 +2432,9 @@ impl TableProvider for ProjectRoutingTable { let project_id = self.extract_project_id_from_filters(&optimized_filters).unwrap_or_else(|| self.default_project.clone()); span.record("table.project_id", project_id.as_str()); - // Helper to wrap result with VariantToJsonExec for proper pgwire encoding + let has_variant_columns = self.real_schema().fields().iter().any(|f| is_variant_type(f.data_type())); let wrap_result = |plan: Arc| -> DFResult> { - Ok(Arc::new(VariantToJsonExec::new(plan, self.real_schema()))) + if has_variant_columns { Ok(Arc::new(VariantToJsonExec::new(plan, self.real_schema()))) } else { Ok(plan) } }; // Check if buffered layer is configured diff --git a/src/mem_buffer.rs b/src/mem_buffer.rs index 94f9473..e280b3d 100644 --- a/src/mem_buffer.rs +++ b/src/mem_buffer.rs @@ -10,8 +10,9 @@ use datafusion::physical_expr::execution_props::ExecutionProps; use datafusion::sql::planner::SqlToRel; use datafusion::sql::sqlparser::dialect::GenericDialect; use datafusion::sql::sqlparser::parser::Parser as SqlParser; +use parking_lot::Mutex; use std::sync::atomic::{AtomicI64, AtomicUsize, Ordering}; -use std::sync::{Arc, RwLock}; +use std::sync::Arc; use tracing::{debug, info, instrument, warn}; // 10-minute buckets balance flush granularity vs overhead. Shorter = more flushes, @@ -20,6 +21,7 @@ use tracing::{debug, info, instrument, warn}; // which is supported but may result in unexpected ordering if mixed with post-1970 data. const BUCKET_DURATION_MICROS: i64 = 10 * 60 * 1_000_000; + /// Check if two schemas are compatible for merge. /// Compatible means: all existing fields must be present in incoming schema with same type, /// incoming schema may have additional nullable fields. @@ -123,7 +125,7 @@ pub struct TableBuffer { } pub struct TimeBucket { - batches: RwLock>, + batches: Mutex>, row_count: AtomicUsize, memory_bytes: AtomicUsize, min_timestamp: AtomicI64, @@ -388,9 +390,14 @@ impl MemBuffer { if !bucket_overlaps_range(bucket, &ts_range) { continue; } - if let Ok(batches) = bucket.batches.read() { - results.extend(batches.iter().cloned()); + let mut batches = bucket.batches.lock(); + if batches.len() > 1 { + if let Ok(single) = arrow::compute::concat_batches(&table.schema, &*batches) { + batches.clear(); + batches.push(single); + } } + results.extend(batches.iter().cloned()); } } @@ -413,10 +420,17 @@ impl MemBuffer { for bucket_id in bucket_ids { if let Some(bucket) = table.buckets.get(&bucket_id) && bucket_overlaps_range(&bucket, &ts_range) - && let Ok(batches) = bucket.batches.read() - && !batches.is_empty() { - partitions.push(batches.clone()); + let mut batches = bucket.batches.lock(); + if !batches.is_empty() { + if batches.len() > 1 { + if let Ok(single) = arrow::compute::concat_batches(&table.schema, &*batches) { + batches.clear(); + batches.push(single); + } + } + partitions.push(batches.clone()); + } } } } @@ -469,17 +483,12 @@ impl MemBuffer { { let freed_bytes = bucket.memory_bytes.load(Ordering::Relaxed); self.estimated_bytes.fetch_sub(freed_bytes, Ordering::Relaxed); - if let Ok(batches) = bucket.batches.into_inner() { - debug!( - "MemBuffer drain: project={}, table={}, bucket={}, batches={}, freed_bytes={}", - project_id, - table_name, - bucket_id, - batches.len(), - freed_bytes - ); - return Some(batches); - } + let batches = bucket.batches.into_inner(); + debug!( + "MemBuffer drain: project={}, table={}, bucket={}, batches={}, freed_bytes={}", + project_id, table_name, bucket_id, batches.len(), freed_bytes + ); + return Some(batches); } None } @@ -501,23 +510,22 @@ impl MemBuffer { let table = table_entry.value(); for bucket in table.buckets.iter() { let bucket_id = *bucket.key(); - if filter(bucket_id) - && let Ok(batches) = bucket.batches.read() - && !batches.is_empty() - { - // Compact multiple small batches into one before flush - let compacted = if batches.len() > 1 { - arrow::compute::concat_batches(&table.schema, &*batches).map_or_else(|_| batches.clone(), |single| vec![single]) - } else { - batches.clone() - }; - result.push(FlushableBucket { - project_id: project_id.to_string(), - table_name: table_name.to_string(), - bucket_id, - batches: compacted, - row_count: bucket.row_count.load(Ordering::Relaxed), - }); + if filter(bucket_id) { + let batches = bucket.batches.lock(); + if !batches.is_empty() { + let compacted = if batches.len() > 1 { + arrow::compute::concat_batches(&table.schema, &*batches).map_or_else(|_| batches.clone(), |single| vec![single]) + } else { + batches.clone() + }; + result.push(FlushableBucket { + project_id: project_id.to_string(), + table_name: table_name.to_string(), + bucket_id, + batches: compacted, + row_count: bucket.row_count.load(Ordering::Relaxed), + }); + } } } } @@ -580,7 +588,7 @@ impl MemBuffer { for mut bucket_entry in table.buckets.iter_mut() { let bucket = bucket_entry.value_mut(); - let mut batches = bucket.batches.write().map_err(|e| datafusion::error::DataFusionError::Execution(format!("Lock error: {}", e)))?; + let mut batches = bucket.batches.lock(); let mut new_batches = Vec::with_capacity(batches.len()); for batch in batches.drain(..) { @@ -662,7 +670,7 @@ impl MemBuffer { for mut bucket_entry in table.buckets.iter_mut() { let bucket = bucket_entry.value_mut(); - let mut batches = bucket.batches.write().map_err(|e| datafusion::error::DataFusionError::Execution(format!("Lock error: {}", e)))?; + let mut batches = bucket.batches.lock(); let old_memory: usize = batches.iter().map(|b| estimate_batch_size(b)).sum(); let new_batches: Vec = batches @@ -762,7 +770,7 @@ impl MemBuffer { total_buckets += table.buckets.len(); for bucket in table.buckets.iter() { total_rows += bucket.row_count.load(Ordering::Relaxed); - total_batches += bucket.batches.read().map(|b| b.len()).unwrap_or(0); + total_batches += bucket.batches.lock().len(); } } MemBufferStats { @@ -814,10 +822,7 @@ impl TableBuffer { let bucket = self.buckets.entry(bucket_id).or_insert_with(TimeBucket::new); - { - let mut batches = bucket.batches.write().map_err(|e| anyhow::anyhow!("Failed to acquire write lock on bucket: {}", e))?; - batches.push(batch); - } + bucket.batches.lock().push(batch); bucket.row_count.fetch_add(row_count, Ordering::Relaxed); bucket.memory_bytes.fetch_add(batch_size, Ordering::Relaxed); @@ -834,7 +839,7 @@ impl TableBuffer { impl TimeBucket { fn new() -> Self { Self { - batches: RwLock::new(Vec::new()), + batches: Mutex::new(Vec::new()), row_count: AtomicUsize::new(0), memory_bytes: AtomicUsize::new(0), min_timestamp: AtomicI64::new(i64::MAX), @@ -1084,7 +1089,8 @@ mod tests { } let results = buffer.query("project1", "table1", &[]).unwrap(); - assert_eq!(results.len(), 10, "All 10 inserts should succeed"); + let total_rows: usize = results.iter().map(|b| b.num_rows()).sum(); + assert_eq!(total_rows, 10, "All 10 inserts should succeed"); } #[test] diff --git a/src/wal.rs b/src/wal.rs index 5fbc9d9..3d9f89c 100644 --- a/src/wal.rs +++ b/src/wal.rs @@ -3,7 +3,6 @@ use arrow::array::{Array, ArrayRef, RecordBatch, make_array}; use arrow::buffer::{Buffer, NullBuffer}; use arrow::datatypes::{DataType, SchemaRef}; use arrow_ipc::reader::StreamReader; -use arrow_ipc::writer::{IpcWriteOptions, StreamWriter}; use bincode::{Decode, Encode}; use dashmap::DashSet; use std::path::PathBuf; @@ -116,7 +115,6 @@ struct CompactBatch { columns: Vec, } -#[allow(dead_code)] // Kept for legacy WAL v128 test coverage impl CompactColumn { fn from_array(array: &dyn Array) -> Self { let data = array.to_data(); @@ -387,11 +385,9 @@ impl WalManager { } pub fn deserialize_batch(data: &[u8], table_name: &str) -> Result { - // Try IPC first (v129+), fall back to legacy CompactBatch (v128) - deserialize_record_batch_ipc(data).or_else(|_| { - let schema = get_schema(table_name).map(|s| s.schema_ref()).unwrap_or_else(|| get_default_schema().schema_ref()); - deserialize_record_batch_legacy(data, &schema) - }) + let schema = get_schema(table_name).map(|s| s.schema_ref()).unwrap_or_else(|| get_default_schema().schema_ref()); + // Try CompactBatch (v128) first, fall back to IPC (v129) for backward compat + deserialize_record_batch(data, &schema).or_else(|_| deserialize_record_batch_ipc(data)) } pub fn list_topics(&self) -> Result, WalError> { @@ -422,16 +418,31 @@ impl WalManager { pub fn data_dir(&self) -> &PathBuf { &self.data_dir } + + /// Returns WAL file count and total size in bytes by scanning the data directory. + pub fn wal_stats(&self) -> (usize, u64) { + let mut file_count = 0usize; + let mut total_bytes = 0u64; + if let Ok(entries) = std::fs::read_dir(&self.data_dir) { + for entry in entries.flatten() { + if let Ok(meta) = entry.metadata() { + if meta.is_file() { + file_count += 1; + total_bytes += meta.len(); + } + } + } + } + (file_count, total_bytes) + } } fn serialize_record_batch(batch: &RecordBatch) -> Result, WalError> { - let mut buf = Vec::new(); - let options = IpcWriteOptions::default(); - let mut writer = StreamWriter::try_new_with_options(&mut buf, &batch.schema(), options)?; - writer.write(batch)?; - writer.finish()?; - drop(writer); - Ok(buf) + let compact = CompactBatch { + num_rows: batch.num_rows(), + columns: batch.columns().iter().map(|c| CompactColumn::from_array(c.as_ref())).collect(), + }; + bincode::encode_to_vec(&compact, BINCODE_CONFIG).map_err(WalError::BincodeEncode) } fn deserialize_record_batch_ipc(data: &[u8]) -> Result { @@ -446,7 +457,7 @@ fn deserialize_record_batch_ipc(data: &[u8]) -> Result { } /// Legacy CompactBatch deserialization for WAL version 128 -fn deserialize_record_batch_legacy(data: &[u8], schema: &SchemaRef) -> Result { +fn deserialize_record_batch(data: &[u8], schema: &SchemaRef) -> Result { if data.len() > MAX_BATCH_SIZE { return Err(WalError::BatchTooLarge { size: data.len(), max: MAX_BATCH_SIZE }); } @@ -462,7 +473,7 @@ fn deserialize_record_batch_legacy(data: &[u8], schema: &SchemaRef) -> Result Result, WalError> { let mut buffer = WAL_MAGIC.to_vec(); - buffer.push(WAL_VERSION_IPC); + buffer.push(WAL_VERSION); buffer.push(entry.operation as u8); buffer.extend(bincode::encode_to_vec(entry, BINCODE_CONFIG)?); Ok(buffer) @@ -536,25 +547,11 @@ mod tests { } #[test] - fn test_record_batch_ipc_serialization() { - let batch = create_test_batch(); - let serialized = serialize_record_batch(&batch).unwrap(); - let deserialized = deserialize_record_batch_ipc(&serialized).unwrap(); - assert_eq!(batch.num_rows(), deserialized.num_rows()); - assert_eq!(batch.num_columns(), deserialized.num_columns()); - } - - #[test] - fn test_record_batch_legacy_serialization() { + fn test_record_batch_serialization() { let batch = create_test_batch(); let schema = batch.schema(); - // Serialize using legacy CompactBatch format - let compact = CompactBatch { - num_rows: batch.num_rows(), - columns: batch.columns().iter().map(|c| CompactColumn::from_array(c.as_ref())).collect(), - }; - let serialized = bincode::encode_to_vec(&compact, BINCODE_CONFIG).unwrap(); - let deserialized = deserialize_record_batch_legacy(&serialized, &schema).unwrap(); + let serialized = serialize_record_batch(&batch).unwrap(); + let deserialized = deserialize_record_batch(&serialized, &schema).unwrap(); assert_eq!(batch.num_rows(), deserialized.num_rows()); assert_eq!(batch.num_columns(), deserialized.num_columns()); } From 057b262d1fd541dc35cb0a687368bf0a21c2fbf1 Mon Sep 17 00:00:00 2001 From: Anthony Alaribe Date: Mon, 16 Feb 2026 20:51:28 +0100 Subject: [PATCH 8/8] x --- Cargo.lock | 1564 +++++++++++++++-------------- Cargo.toml | 10 +- src/database.rs | 21 +- src/dml.rs | 70 +- src/object_store_cache.rs | 10 +- tests/connection_pressure_test.rs | 2 +- tests/integration_test.rs | 2 +- 7 files changed, 887 insertions(+), 792 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index 93dc150..cd407e0 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -23,7 +23,7 @@ version = "0.7.8" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "891477e0c6a8957309ee5c45a6368af3ae14bb510732d2684ffa19af310920f9" dependencies = [ - "getrandom 0.2.16", + "getrandom 0.2.17", "once_cell", "version_check", ] @@ -66,6 +66,15 @@ dependencies = [ "alloc-no-stdlib", ] +[[package]] +name = "alloca" +version = "0.4.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "e5a7d05ea6aea7e9e64d25b9156ba2fee3fdd659e34e41063cd2fc7cd020d7f4" +dependencies = [ + "cc", +] + [[package]] name = "allocator-api2" version = "0.2.21" @@ -139,25 +148,19 @@ dependencies = [ [[package]] name = "anyhow" -version = "1.0.100" +version = "1.0.101" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a23eb6b1614318a8071c9b2521f36b424b2c83db5eb3a0fead4a6c0809af6e61" +checksum = "5f0e0fee31ef5ed1ba1316088939cea399010ed7731dba877ed44aeb407a75ea" [[package]] name = "ar_archive_writer" -version = "0.2.0" +version = "0.5.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "f0c269894b6fe5e9d7ada0cf69b5bf847ff35bc25fc271f08e1d080fce80339a" +checksum = "7eb93bbb63b9c227414f6eb3a0adfddca591a8ce1e9b60661bb08969b87e340b" dependencies = [ - "object 0.32.2", + "object", ] -[[package]] -name = "arc-swap" -version = "1.7.1" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "69f7f8c3906b62b754cd5326047894316021dcfe5a194c8ea52bdd94934a3457" - [[package]] name = "array-init" version = "2.1.0" @@ -178,9 +181,9 @@ checksum = "7c02d123df017efcdfbd739ef81735b36c5ba83ec3c59c80a9d7ecc718f92e50" [[package]] name = "arrow" -version = "57.2.0" +version = "57.3.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "2a2b10dcb159faf30d3f81f6d56c1211a5bea2ca424eabe477648a44b993320e" +checksum = "e4754a624e5ae42081f464514be454b39711daae0458906dacde5f4c632f33a8" dependencies = [ "arrow-arith", "arrow-array", @@ -199,9 +202,9 @@ dependencies = [ [[package]] name = "arrow-arith" -version = "57.2.0" +version = "57.3.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "288015089e7931843c80ed4032c5274f02b37bcb720c4a42096d50b390e70372" +checksum = "f7b3141e0ec5145a22d8694ea8b6d6f69305971c4fa1c1a13ef0195aef2d678b" dependencies = [ "arrow-array", "arrow-buffer", @@ -213,9 +216,9 @@ dependencies = [ [[package]] name = "arrow-array" -version = "57.2.0" +version = "57.3.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "65ca404ea6191e06bf30956394173337fa9c35f445bd447fe6c21ab944e1a23c" +checksum = "4c8955af33b25f3b175ee10af580577280b4bd01f7e823d94c7cdef7cf8c9aef" dependencies = [ "ahash 0.8.12", "arrow-buffer", @@ -232,9 +235,9 @@ dependencies = [ [[package]] name = "arrow-buffer" -version = "57.2.0" +version = "57.3.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "36356383099be0151dacc4245309895f16ba7917d79bdb71a7148659c9206c56" +checksum = "c697ddca96183182f35b3a18e50b9110b11e916d7b7799cbfd4d34662f2c56c2" dependencies = [ "bytes", "half", @@ -244,9 +247,9 @@ dependencies = [ [[package]] name = "arrow-cast" -version = "57.2.0" +version = "57.3.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "9c8e372ed52bd4ee88cc1e6c3859aa7ecea204158ac640b10e187936e7e87074" +checksum = "646bbb821e86fd57189c10b4fcdaa941deaf4181924917b0daa92735baa6ada5" dependencies = [ "arrow-array", "arrow-buffer", @@ -266,9 +269,9 @@ dependencies = [ [[package]] name = "arrow-csv" -version = "57.2.0" +version = "57.3.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "8e4100b729fe656f2e4fb32bc5884f14acf9118d4ad532b7b33c1132e4dce896" +checksum = "8da746f4180004e3ce7b83c977daf6394d768332349d3d913998b10a120b790a" dependencies = [ "arrow-array", "arrow-cast", @@ -281,9 +284,9 @@ dependencies = [ [[package]] name = "arrow-data" -version = "57.2.0" +version = "57.3.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "bf87f4ff5fc13290aa47e499a8b669a82c5977c6a1fedce22c7f542c1fd5a597" +checksum = "1fdd994a9d28e6365aa78e15da3f3950c0fdcea6b963a12fa1c391afb637b304" dependencies = [ "arrow-buffer", "arrow-schema", @@ -294,9 +297,9 @@ dependencies = [ [[package]] name = "arrow-ipc" -version = "57.2.0" +version = "57.3.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "eb3ca63edd2073fcb42ba112f8ae165df1de935627ead6e203d07c99445f2081" +checksum = "abf7df950701ab528bf7c0cf7eeadc0445d03ef5d6ffc151eaae6b38a58feff1" dependencies = [ "arrow-array", "arrow-buffer", @@ -310,9 +313,9 @@ dependencies = [ [[package]] name = "arrow-json" -version = "57.2.0" +version = "57.3.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a36b2332559d3310ebe3e173f75b29989b4412df4029a26a30cc3f7da0869297" +checksum = "0ff8357658bedc49792b13e2e862b80df908171275f8e6e075c460da5ee4bf86" dependencies = [ "arrow-array", "arrow-buffer", @@ -321,7 +324,7 @@ dependencies = [ "arrow-schema", "chrono", "half", - "indexmap 2.12.1", + "indexmap 2.13.0", "itoa", "lexical-core", "memchr", @@ -334,9 +337,9 @@ dependencies = [ [[package]] name = "arrow-ord" -version = "57.2.0" +version = "57.3.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "13c4e0530272ca755d6814218dffd04425c5b7854b87fa741d5ff848bf50aa39" +checksum = "f7d8f1870e03d4cbed632959498bcc84083b5a24bded52905ae1695bd29da45b" dependencies = [ "arrow-array", "arrow-buffer", @@ -347,14 +350,16 @@ dependencies = [ [[package]] name = "arrow-pg" -version = "0.10.0" +version = "0.12.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "88ce1ffbf30cd0198a53f1f838226337aa136c2eb58530253ed8796b97c05e2e" +checksum = "648178d89ddfc58dec82298e8b419ad201a6807190cf92b324ea2b17a9a668d9" dependencies = [ + "arrow-schema", "bytes", "chrono", "datafusion", "futures", + "pg_interval_2", "pgwire", "postgres-types", "rust_decimal", @@ -362,9 +367,9 @@ dependencies = [ [[package]] name = "arrow-row" -version = "57.2.0" +version = "57.3.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "b07f52788744cc71c4628567ad834cadbaeb9f09026ff1d7a4120f69edf7abd3" +checksum = "18228633bad92bff92a95746bbeb16e5fc318e8382b75619dec26db79e4de4c0" dependencies = [ "arrow-array", "arrow-buffer", @@ -375,9 +380,9 @@ dependencies = [ [[package]] name = "arrow-schema" -version = "57.2.0" +version = "57.3.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "6bb63203e8e0e54b288d0d8043ca8fa1013820822a27692ef1b78a977d879f2c" +checksum = "8c872d36b7bf2a6a6a2b40de9156265f0242910791db366a2c17476ba8330d68" dependencies = [ "bitflags", "serde", @@ -387,9 +392,9 @@ dependencies = [ [[package]] name = "arrow-select" -version = "57.2.0" +version = "57.3.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c96d8a1c180b44ecf2e66c9a2f2bbcb8b1b6f14e165ce46ac8bde211a363411b" +checksum = "68bf3e3efbd1278f770d67e5dc410257300b161b93baedb3aae836144edcaf4b" dependencies = [ "ahash 0.8.12", "arrow-array", @@ -401,9 +406,9 @@ dependencies = [ [[package]] name = "arrow-string" -version = "57.2.0" +version = "57.3.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a8ad6a81add9d3ea30bf8374ee8329992c7fd246ffd8b7e2f48a3cea5aa0cc9a" +checksum = "85e968097061b3c0e9fe3079cf2e703e487890700546b5b0647f60fca1b5a8d8" dependencies = [ "arrow-array", "arrow-buffer", @@ -416,23 +421,11 @@ dependencies = [ "regex-syntax", ] -[[package]] -name = "async-channel" -version = "2.5.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "924ed96dd52d1b75e9c1a3e6275715fd320f5f9439fb5a4a11fa51f4221158d2" -dependencies = [ - "concurrent-queue", - "event-listener-strategy", - "futures-core", - "pin-project-lite", -] - [[package]] name = "async-compression" -version = "0.4.37" +version = "0.4.39" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d10e4f991a553474232bc0a31799f6d24b034a84c0971d80d2e2f78b2e576e40" +checksum = "68650b7df54f0293fd061972a0fb05aaf4fc0879d3b3d21a638a182c5c543b9f" dependencies = [ "compression-codecs", "compression-core", @@ -440,34 +433,6 @@ dependencies = [ "tokio", ] -[[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.114", -] - -[[package]] -name = "async-task" -version = "4.7.1" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "8b75356056920673b02621b35afd0f7dda9306d03c79a30f5c56c44cf256e3de" - [[package]] name = "async-trait" version = "0.1.89" @@ -476,7 +441,7 @@ checksum = "9035ad2d096bed7955a320ee7e2230574d28fd3c3a0f186cbea1ff3c7eed5dbb" dependencies = [ "proc-macro2", "quote", - "syn 2.0.114", + "syn 2.0.116", ] [[package]] @@ -502,9 +467,9 @@ checksum = "c08606f8c3cbf4ce6ec8e28fb0014a2c086708fe954eaa885384a6165172e7e8" [[package]] name = "aws-config" -version = "1.8.12" +version = "1.8.13" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "96571e6996817bf3d58f6b569e4b9fd2e9d2fcf9f7424eed07b2ce9bb87535e5" +checksum = "c456581cb3c77fafcc8c67204a70680d40b61112d6da78c77bd31d945b65f1b5" dependencies = [ "aws-credential-types", "aws-runtime", @@ -512,8 +477,8 @@ dependencies = [ "aws-sdk-ssooidc", "aws-sdk-sts", "aws-smithy-async", - "aws-smithy-http", - "aws-smithy-json", + "aws-smithy-http 0.63.3", + "aws-smithy-json 0.62.3", "aws-smithy-runtime", "aws-smithy-runtime-api", "aws-smithy-types", @@ -544,9 +509,9 @@ dependencies = [ [[package]] name = "aws-lc-rs" -version = "1.15.2" +version = "1.15.4" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "6a88aab2464f1f25453baa7a07c84c5b7684e274054ba06817f382357f77a288" +checksum = "7b7b6141e96a8c160799cc2d5adecd5cbbe5054cb8c7c4af53da0f83bb7ad256" dependencies = [ "aws-lc-sys", "zeroize", @@ -554,9 +519,9 @@ dependencies = [ [[package]] name = "aws-lc-sys" -version = "0.35.0" +version = "0.37.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "b45afffdee1e7c9126814751f88dddc747f41d91da16c9551a0f1e8a11e788a1" +checksum = "b092fe214090261288111db7a2b2c2118e5a7f30dc2569f1732c4069a6840549" dependencies = [ "cc", "cmake", @@ -566,15 +531,15 @@ dependencies = [ [[package]] name = "aws-runtime" -version = "1.5.17" +version = "1.6.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d81b5b2898f6798ad58f484856768bca817e3cd9de0974c24ae0f1113fe88f1b" +checksum = "c635c2dc792cb4a11ce1a4f392a925340d1bdf499289b5ec1ec6810954eb43f5" dependencies = [ "aws-credential-types", "aws-sigv4", "aws-smithy-async", "aws-smithy-eventstream", - "aws-smithy-http", + "aws-smithy-http 0.63.3", "aws-smithy-runtime", "aws-smithy-runtime-api", "aws-smithy-types", @@ -582,7 +547,9 @@ dependencies = [ "bytes", "fastrand", "http 0.2.12", + "http 1.4.0", "http-body 0.4.6", + "http-body 1.0.1", "percent-encoding", "pin-project-lite", "tracing", @@ -591,15 +558,16 @@ dependencies = [ [[package]] name = "aws-sdk-dynamodb" -version = "1.101.0" +version = "1.104.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "b6f98cd9e5f2fc790aff1f393bc3c8680deea31c05d3c6f23b625cdc50b1b6b4" +checksum = "f04c47115cc8d46dcc94a9a81e7a3384cea859283c1a737729691d4221f11584" dependencies = [ "aws-credential-types", "aws-runtime", "aws-smithy-async", - "aws-smithy-http", - "aws-smithy-json", + "aws-smithy-http 0.63.3", + "aws-smithy-json 0.62.3", + "aws-smithy-observability", "aws-smithy-runtime", "aws-smithy-runtime-api", "aws-smithy-types", @@ -607,15 +575,16 @@ dependencies = [ "bytes", "fastrand", "http 0.2.12", + "http 1.4.0", "regex-lite", "tracing", ] [[package]] name = "aws-sdk-s3" -version = "1.118.0" +version = "1.119.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d3e6b7079f85d9ea9a70643c9f89f50db70f5ada868fa9cfe08c1ffdf51abc13" +checksum = "1d65fddc3844f902dfe1864acb8494db5f9342015ee3ab7890270d36fbd2e01c" dependencies = [ "aws-credential-types", "aws-runtime", @@ -623,8 +592,8 @@ dependencies = [ "aws-smithy-async", "aws-smithy-checksums", "aws-smithy-eventstream", - "aws-smithy-http", - "aws-smithy-json", + "aws-smithy-http 0.62.6", + "aws-smithy-json 0.61.9", "aws-smithy-runtime", "aws-smithy-runtime-api", "aws-smithy-types", @@ -647,15 +616,16 @@ dependencies = [ [[package]] name = "aws-sdk-sso" -version = "1.91.0" +version = "1.93.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "8ee6402a36f27b52fe67661c6732d684b2635152b676aa2babbfb5204f99115d" +checksum = "9dcb38bb33fc0a11f1ffc3e3e85669e0a11a37690b86f77e75306d8f369146a0" dependencies = [ "aws-credential-types", "aws-runtime", "aws-smithy-async", - "aws-smithy-http", - "aws-smithy-json", + "aws-smithy-http 0.63.3", + "aws-smithy-json 0.62.3", + "aws-smithy-observability", "aws-smithy-runtime", "aws-smithy-runtime-api", "aws-smithy-types", @@ -663,21 +633,23 @@ dependencies = [ "bytes", "fastrand", "http 0.2.12", + "http 1.4.0", "regex-lite", "tracing", ] [[package]] name = "aws-sdk-ssooidc" -version = "1.93.0" +version = "1.95.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a45a7f750bbd170ee3677671ad782d90b894548f4e4ae168302c57ec9de5cb3e" +checksum = "2ada8ffbea7bd1be1f53df1dadb0f8fdb04badb13185b3321b929d1ee3caad09" dependencies = [ "aws-credential-types", "aws-runtime", "aws-smithy-async", - "aws-smithy-http", - "aws-smithy-json", + "aws-smithy-http 0.63.3", + "aws-smithy-json 0.62.3", + "aws-smithy-observability", "aws-smithy-runtime", "aws-smithy-runtime-api", "aws-smithy-types", @@ -685,21 +657,23 @@ dependencies = [ "bytes", "fastrand", "http 0.2.12", + "http 1.4.0", "regex-lite", "tracing", ] [[package]] name = "aws-sdk-sts" -version = "1.95.0" +version = "1.97.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "55542378e419558e6b1f398ca70adb0b2088077e79ad9f14eb09441f2f7b2164" +checksum = "e6443ccadc777095d5ed13e21f5c364878c9f5bad4e35187a6cdbd863b0afcad" dependencies = [ "aws-credential-types", "aws-runtime", "aws-smithy-async", - "aws-smithy-http", - "aws-smithy-json", + "aws-smithy-http 0.63.3", + "aws-smithy-json 0.62.3", + "aws-smithy-observability", "aws-smithy-query", "aws-smithy-runtime", "aws-smithy-runtime-api", @@ -708,19 +682,20 @@ dependencies = [ "aws-types", "fastrand", "http 0.2.12", + "http 1.4.0", "regex-lite", "tracing", ] [[package]] name = "aws-sigv4" -version = "1.3.7" +version = "1.3.8" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "69e523e1c4e8e7e8ff219d732988e22bfeae8a1cafdbe6d9eca1546fa080be7c" +checksum = "efa49f3c607b92daae0c078d48a4571f599f966dce3caee5f1ea55c4d9073f99" dependencies = [ "aws-credential-types", "aws-smithy-eventstream", - "aws-smithy-http", + "aws-smithy-http 0.63.3", "aws-smithy-runtime-api", "aws-smithy-types", "bytes", @@ -742,9 +717,9 @@ dependencies = [ [[package]] name = "aws-smithy-async" -version = "1.2.7" +version = "1.2.11" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "9ee19095c7c4dda59f1697d028ce704c24b2d33c6718790c7f1d5a3015b4107c" +checksum = "52eec3db979d18cb807fc1070961cc51d87d069abe9ab57917769687368a8c6c" dependencies = [ "futures-util", "pin-project-lite", @@ -757,7 +732,7 @@ version = "0.63.12" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "87294a084b43d649d967efe58aa1f9e0adc260e13a6938eb904c0ae9b45824ae" dependencies = [ - "aws-smithy-http", + "aws-smithy-http 0.62.6", "aws-smithy-types", "bytes", "crc-fast", @@ -773,9 +748,9 @@ dependencies = [ [[package]] name = "aws-smithy-eventstream" -version = "0.60.14" +version = "0.60.18" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "dc12f8b310e38cad85cf3bef45ad236f470717393c613266ce0a89512286b650" +checksum = "35b9c7354a3b13c66f60fe4616d6d1969c9fd36b1b5333a5dfb3ee716b33c588" dependencies = [ "aws-smithy-types", "bytes", @@ -804,17 +779,38 @@ dependencies = [ "tracing", ] +[[package]] +name = "aws-smithy-http" +version = "0.63.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "630e67f2a31094ffa51b210ae030855cb8f3b7ee1329bdd8d085aaf61e8b97fc" +dependencies = [ + "aws-smithy-runtime-api", + "aws-smithy-types", + "bytes", + "bytes-utils", + "futures-core", + "futures-util", + "http 1.4.0", + "http-body 1.0.1", + "http-body-util", + "percent-encoding", + "pin-project-lite", + "pin-utils", + "tracing", +] + [[package]] name = "aws-smithy-http-client" -version = "1.1.5" +version = "1.1.9" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "59e62db736db19c488966c8d787f52e6270be565727236fd5579eaa301e7bc4a" +checksum = "12fb0abf49ff0cab20fd31ac1215ed7ce0ea92286ba09e2854b42ba5cabe7525" dependencies = [ "aws-smithy-async", "aws-smithy-runtime-api", "aws-smithy-types", "h2 0.3.27", - "h2 0.4.12", + "h2 0.4.13", "http 0.2.12", "http 1.4.0", "http-body 0.4.6", @@ -825,7 +821,7 @@ dependencies = [ "hyper-util", "pin-project-lite", "rustls 0.21.12", - "rustls 0.23.35", + "rustls 0.23.36", "rustls-native-certs", "rustls-pki-types", "tokio", @@ -843,20 +839,29 @@ dependencies = [ "aws-smithy-types", ] +[[package]] +name = "aws-smithy-json" +version = "0.62.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "3cb96aa208d62ee94104645f7b2ecaf77bf27edf161590b6224bfbac2832f979" +dependencies = [ + "aws-smithy-types", +] + [[package]] name = "aws-smithy-observability" -version = "0.1.5" +version = "0.2.4" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "17f616c3f2260612fe44cede278bafa18e73e6479c4e393e2c4518cf2a9a228a" +checksum = "c0a46543fbc94621080b3cf553eb4cbbdc41dd9780a30c4756400f0139440a1d" dependencies = [ "aws-smithy-runtime-api", ] [[package]] name = "aws-smithy-query" -version = "0.60.9" +version = "0.60.13" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "ae5d689cf437eae90460e944a58b5668530d433b4ff85789e69d2f2a556e057d" +checksum = "0cebbddb6f3a5bd81553643e9c7daf3cc3dc5b0b5f398ac668630e8a84e6fff0" dependencies = [ "aws-smithy-types", "urlencoding", @@ -864,12 +869,12 @@ dependencies = [ [[package]] name = "aws-smithy-runtime" -version = "1.9.6" +version = "1.10.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "65fda37911905ea4d3141a01364bc5509a0f32ae3f3b22d6e330c0abfb62d247" +checksum = "f3df87c14f0127a0d77eb261c3bc45d5b4833e2a1f63583ebfb728e4852134ee" dependencies = [ "aws-smithy-async", - "aws-smithy-http", + "aws-smithy-http 0.63.3", "aws-smithy-http-client", "aws-smithy-observability", "aws-smithy-runtime-api", @@ -880,6 +885,7 @@ dependencies = [ "http 1.4.0", "http-body 0.4.6", "http-body 1.0.1", + "http-body-util", "pin-project-lite", "pin-utils", "tokio", @@ -888,9 +894,9 @@ dependencies = [ [[package]] name = "aws-smithy-runtime-api" -version = "1.9.3" +version = "1.11.3" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "ab0d43d899f9e508300e587bf582ba54c27a452dd0a9ea294690669138ae14a2" +checksum = "49952c52f7eebb72ce2a754d3866cc0f87b97d2a46146b79f80f3a93fb2b3716" dependencies = [ "aws-smithy-async", "aws-smithy-types", @@ -905,9 +911,9 @@ dependencies = [ [[package]] name = "aws-smithy-types" -version = "1.3.5" +version = "1.4.3" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "905cb13a9895626d49cf2ced759b062d913834c7482c38e49557eac4e6193f01" +checksum = "3b3a26048eeab0ddeba4b4f9d51654c79af8c3b32357dc5f336cee85ab331c33" dependencies = [ "base64-simd", "bytes", @@ -972,7 +978,7 @@ dependencies = [ "cfg-if", "libc", "miniz_oxide", - "object 0.37.3", + "object", "rustc-demangle", "windows-link", ] @@ -1001,9 +1007,9 @@ dependencies = [ [[package]] name = "base64ct" -version = "1.8.1" +version = "1.8.3" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "0e050f626429857a27ddccb31e0aca21356bfa709c04041aefddac081a8f068a" +checksum = "2af50177e190e07a26ab74f8b1efbfe2ef87da2116221318cb1c2e82baf7de06" [[package]] name = "bcder" @@ -1017,9 +1023,9 @@ dependencies = [ [[package]] name = "bigdecimal" -version = "0.4.9" +version = "0.4.10" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "560f42649de9fa436b73517378a147ec21f6c997a546581df4b4b31677828934" +checksum = "4d6867f1565b3aad85681f1015055b087fcfd840d6aeee6eee7f2da317603695" dependencies = [ "autocfg", "libm", @@ -1059,9 +1065,9 @@ dependencies = [ [[package]] name = "bitflags" -version = "2.10.0" +version = "2.11.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "812e12b5285cc515a9c72a5c1d3b6d46a19dac5acfef5265968c166106e31dd3" +checksum = "843867be96c8daad0d758b57df9392b6d8d271134fce549de6ce169ff98a92af" dependencies = [ "serde_core", ] @@ -1089,15 +1095,16 @@ dependencies = [ [[package]] name = "blake3" -version = "1.8.2" +version = "1.8.3" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "3888aaa89e4b2a40fca9848e400f6a658a5a3978de7be858e209cafa8be9a4a0" +checksum = "2468ef7d57b3fb7e16b576e8377cdbde2320c60e1491e961d11da40fc4f02a2d" dependencies = [ "arrayref", "arrayvec", "cc", "cfg-if", "constant_time_eq", + "cpufeatures 0.2.17", ] [[package]] @@ -1129,7 +1136,7 @@ dependencies = [ "proc-macro-crate", "proc-macro2", "quote", - "syn 2.0.114", + "syn 2.0.116", ] [[package]] @@ -1183,9 +1190,9 @@ dependencies = [ [[package]] name = "bytemuck" -version = "1.24.0" +version = "1.25.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "1fbdf580320f38b612e485521afda1ee26d10cc9884efaaa750d383e13e3c5f4" +checksum = "c8efb64bd706a16a1bdde310ae86b351e4d21550d98d056f22f8a7f7a2183fec" dependencies = [ "bytemuck_derive", ] @@ -1198,7 +1205,7 @@ checksum = "f9abbd1bc6865053c427f7198e6af43bfdedc55ab791faed4fbd361d789575ff" dependencies = [ "proc-macro2", "quote", - "syn 2.0.114", + "syn 2.0.116", ] [[package]] @@ -1209,9 +1216,9 @@ checksum = "1fd0f2584146f6f2ef48085050886acf353beff7305ebd1ae69500e27c67f64b" [[package]] name = "bytes" -version = "1.11.0" +version = "1.11.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "b35204fbdc0b3f4446b89fc1ac2cf84a8a68971995d0bf2e925ec7cd960f9cb3" +checksum = "1e748733b7cbc798e1434b6ac524f0c1ff2ab456fe201501e6497c8417a4fc33" [[package]] name = "bytes-utils" @@ -1240,9 +1247,9 @@ checksum = "37b2a672a2cb129a2e41c10b1224bb368f9f37a2b16b612598138befd7b37eb5" [[package]] name = "cc" -version = "1.2.50" +version = "1.2.56" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "9f50d563227a1c37cc0a263f64eca3334388c01c5e4c4861a9def205c614383c" +checksum = "aebf35691d1bfb0ac386a69bac2fde4dd276fb618cf8bf4f5318fe285e821bb2" dependencies = [ "find-msvc-tools", "jobserver", @@ -1262,11 +1269,22 @@ version = "0.2.1" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "613afe47fcd5fac7ccf1db93babcb082c5994d996f20b8b159f2ad1658eb5724" +[[package]] +name = "chacha20" +version = "0.10.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "6f8d983286843e49675a4b7a2d174efe136dc93a18d69130dd18198a6c167601" +dependencies = [ + "cfg-if", + "cpufeatures 0.3.0", + "rand_core 0.10.0", +] + [[package]] name = "chrono" -version = "0.4.42" +version = "0.4.43" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "145052bdd345b87320e369255277e3fb5152762ad123a901ef5c262dd38fe8d2" +checksum = "fac4744fb15ae8337dc853fee7fb3f4e48c0fbaa23d0afe49c447b4fab126118" dependencies = [ "iana-time-zone", "js-sys", @@ -1315,9 +1333,9 @@ dependencies = [ [[package]] name = "clap" -version = "4.5.53" +version = "4.5.58" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c9e340e012a1bf4935f5282ed1436d1489548e8f72308207ea5df0e23d2d03f8" +checksum = "63be97961acde393029492ce0be7a1af7e323e6bae9511ebfac33751be5e6806" dependencies = [ "clap_builder", "clap_derive", @@ -1325,33 +1343,33 @@ dependencies = [ [[package]] name = "clap_builder" -version = "4.5.53" +version = "4.5.58" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d76b5d13eaa18c901fd2f7fca939fefe3a0727a953561fefdf3b2922b8569d00" +checksum = "7f13174bda5dfd69d7e947827e5af4b0f2f94a4a3ee92912fba07a66150f21e2" dependencies = [ "anstream", "anstyle", "clap_lex", - "strsim 0.11.1", + "strsim", ] [[package]] name = "clap_derive" -version = "4.5.49" +version = "4.5.55" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "2a0b5487afeab2deb2ff4e03a807ad1a03ac532ff5a2cee5d86884440c7f7671" +checksum = "a92793da1a46a5f2a02a6f4c46c6496b28c43638adea8306fcb0caa1634f24e5" dependencies = [ "heck", "proc-macro2", "quote", - "syn 2.0.114", + "syn 2.0.116", ] [[package]] name = "clap_lex" -version = "0.7.6" +version = "1.0.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a1d728cc89cf3aee9ff92b05e62b19ee65a02b5702cff7d5a377e32c6ae29d8d" +checksum = "3a822ea5bc7590f9d40f1ba12c0dc3c2760f3482c6984db1573ad11031420831" [[package]] name = "cmake" @@ -1403,9 +1421,9 @@ checksum = "b05b61dc5112cbb17e4b6cd61790d9845d13888356391624cbe7e41efeac1e75" [[package]] name = "comfy-table" -version = "7.2.1" +version = "7.2.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "b03b7db8e0b4b2fdad6c551e634134e99ec000e5c8c3b6856c65e8bbaded7a3b" +checksum = "958c5d6ecf1f214b4c2bbbbf6ab9523a864bd136dcf71a7e8904799acfe1ad47" dependencies = [ "crossterm", "unicode-segmentation", @@ -1463,16 +1481,16 @@ version = "0.1.16" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "f9d839f2a20b0aee515dc581a6172f2321f96cab76c1a38a4c584a194955390e" dependencies = [ - "getrandom 0.2.16", + "getrandom 0.2.17", "once_cell", "tiny-keccak", ] [[package]] name = "constant_time_eq" -version = "0.3.1" +version = "0.4.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "7c74b8349d32d297c9134b8c88677813a227df8f779daa29bfc29c183fe3dca6" +checksum = "3d52eff69cd5e647efe296129160853a42795992097e8af39800e1060caeea9b" [[package]] name = "convert_case" @@ -1519,6 +1537,15 @@ dependencies = [ "libc", ] +[[package]] +name = "cpufeatures" +version = "0.3.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "8b2a41393f66f16b0823bb79094d54ac5fbd34ab292ddafb9a0456ac9f87d201" +dependencies = [ + "libc", +] + [[package]] name = "crc" version = "3.4.0" @@ -1558,26 +1585,24 @@ dependencies = [ [[package]] name = "criterion" -version = "0.5.1" +version = "0.8.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "f2b12d017a929603d80db1831cd3a24082f8137ce19c69e6447f54f5fc8d692f" +checksum = "950046b2aa2492f9a536f5f4f9a3de7b9e2476e575e05bd6c333371add4d98f3" dependencies = [ + "alloca", "anes", "cast", "ciborium", "clap", "criterion-plot", - "futures", - "is-terminal", - "itertools 0.10.5", + "itertools 0.13.0", "num-traits", - "once_cell", "oorandom", + "page_size", "plotters", "rayon", "regex", "serde", - "serde_derive", "serde_json", "tinytemplate", "tokio", @@ -1586,12 +1611,12 @@ dependencies = [ [[package]] name = "criterion-plot" -version = "0.5.0" +version = "0.8.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "6b50826342786a51a89e2da3a28f1c32b06e387201bc2d19791f622c673706b1" +checksum = "d8d80a2f4f5b554395e47b5d8305bc3d27813bacb73493eb1001e8f76dae29ea" dependencies = [ "cast", - "itertools 0.10.5", + "itertools 0.13.0", ] [[package]] @@ -1737,16 +1762,6 @@ version = "0.0.7" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "52560adf09603e58c9a7ee1fe1dcb95a16927b17c127f0ac02d6e768a0e25bc1" -[[package]] -name = "darling" -version = "0.14.4" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "7b750cb3417fd1b327431a470f388520309479ab0bf5e323505daf0290cd3850" -dependencies = [ - "darling_core 0.14.4", - "darling_macro 0.14.4", -] - [[package]] name = "darling" version = "0.20.11" @@ -1767,20 +1782,6 @@ dependencies = [ "darling_macro 0.21.3", ] -[[package]] -name = "darling_core" -version = "0.14.4" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "109c1ca6e6b7f82cc233a97004ea8ed7ca123a9af07a8230878fcfda9b158bf0" -dependencies = [ - "fnv", - "ident_case", - "proc-macro2", - "quote", - "strsim 0.10.0", - "syn 1.0.109", -] - [[package]] name = "darling_core" version = "0.20.11" @@ -1791,8 +1792,8 @@ dependencies = [ "ident_case", "proc-macro2", "quote", - "strsim 0.11.1", - "syn 2.0.114", + "strsim", + "syn 2.0.116", ] [[package]] @@ -1805,19 +1806,8 @@ dependencies = [ "ident_case", "proc-macro2", "quote", - "strsim 0.11.1", - "syn 2.0.114", -] - -[[package]] -name = "darling_macro" -version = "0.14.4" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a4aab4dbc9f7611d8b55048a3a16d2d010c2c8334e46304b40ac1cc14bf3b48e" -dependencies = [ - "darling_core 0.14.4", - "quote", - "syn 1.0.109", + "strsim", + "syn 2.0.116", ] [[package]] @@ -1828,7 +1818,7 @@ checksum = "fc34b93ccb385b40dc71c6fceac4b2ad23662c7eeb248cf10d529b7e055b6ead" dependencies = [ "darling_core 0.20.11", "quote", - "syn 2.0.114", + "syn 2.0.116", ] [[package]] @@ -1839,7 +1829,7 @@ checksum = "d38308df82d1080de0afee5d069fa14b0326a88c14f15c5ccda35b4a6c414c81" dependencies = [ "darling_core 0.21.3", "quote", - "syn 2.0.114", + "syn 2.0.116", ] [[package]] @@ -1971,7 +1961,7 @@ dependencies = [ "chrono", "half", "hashbrown 0.16.1", - "indexmap 2.12.1", + "indexmap 2.13.0", "libc", "log", "object_store", @@ -2170,7 +2160,7 @@ dependencies = [ "datafusion-functions-aggregate-common", "datafusion-functions-window-common", "datafusion-physical-expr-common", - "indexmap 2.12.1", + "indexmap 2.13.0", "itertools 0.14.0", "paste", "recursive", @@ -2186,7 +2176,7 @@ checksum = "000c98206e3dd47d2939a94b6c67af4bfa6732dd668ac4fafdbde408fd9134ea" dependencies = [ "arrow", "datafusion-common", - "indexmap 2.12.1", + "indexmap 2.13.0", "itertools 0.14.0", "paste", ] @@ -2343,7 +2333,7 @@ checksum = "c4fe888aeb6a095c4bcbe8ac1874c4b9a4c7ffa2ba849db7922683ba20875aaf" dependencies = [ "datafusion-doc", "quote", - "syn 2.0.114", + "syn 2.0.116", ] [[package]] @@ -2358,7 +2348,7 @@ dependencies = [ "datafusion-expr", "datafusion-expr-common", "datafusion-physical-expr", - "indexmap 2.12.1", + "indexmap 2.13.0", "itertools 0.14.0", "log", "recursive", @@ -2368,9 +2358,9 @@ dependencies = [ [[package]] name = "datafusion-pg-catalog" -version = "0.14.0" +version = "0.15.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "daafc06d0478b70b13e8f3d906f2d47c49027efd3718263851137cf6d1d3e0a4" +checksum = "adc01bac56faeaef34a286872e9188647bf21fec47be06675739195faec18f4e" dependencies = [ "async-trait", "datafusion", @@ -2395,7 +2385,7 @@ dependencies = [ "datafusion-physical-expr-common", "half", "hashbrown 0.16.1", - "indexmap 2.12.1", + "indexmap 2.13.0", "itertools 0.14.0", "parking_lot", "paste", @@ -2431,7 +2421,7 @@ dependencies = [ "datafusion-common", "datafusion-expr-common", "hashbrown 0.16.1", - "indexmap 2.12.1", + "indexmap 2.13.0", "itertools 0.14.0", "parking_lot", ] @@ -2478,7 +2468,7 @@ dependencies = [ "futures", "half", "hashbrown 0.16.1", - "indexmap 2.12.1", + "indexmap 2.13.0", "itertools 0.14.0", "log", "parking_lot", @@ -2488,9 +2478,9 @@ dependencies = [ [[package]] name = "datafusion-postgres" -version = "0.14.0" +version = "0.15.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "12413f19af3af28a49fad42191b45d47941091dfeb5f58bb3791c976d3188be1" +checksum = "c7c2f1533ee3be7105e8769a773b57cd28a260d48736a5081fae536b356978ec" dependencies = [ "arrow-pg", "async-trait", @@ -2590,7 +2580,7 @@ dependencies = [ "chrono", "datafusion-common", "datafusion-expr", - "indexmap 2.12.1", + "indexmap 2.13.0", "log", "recursive", "regex", @@ -2635,14 +2625,14 @@ checksum = "780eb241654bf097afb00fc5f054a09b687dad862e485fdcf8399bb056565370" dependencies = [ "proc-macro2", "quote", - "syn 2.0.114", + "syn 2.0.116", ] [[package]] name = "delta_kernel" -version = "0.19.1" +version = "0.19.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "8d3d40b40819579c0ec4b58e8f256a8080a82f5540a42bfab9e0eb4b3f92de2a" +checksum = "06f7fc164b1557731fcc68a198e813811a000efade0f112d4f0a002e65042b83" dependencies = [ "arrow", "bytes", @@ -2651,7 +2641,7 @@ dependencies = [ "crc", "delta_kernel_derive", "futures", - "indexmap 2.12.1", + "indexmap 2.13.0", "itertools 0.14.0", "object_store", "parquet", @@ -2671,19 +2661,19 @@ dependencies = [ [[package]] name = "delta_kernel_derive" -version = "0.19.0" +version = "0.19.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c9e6474dabfc8e0b849ee2d68f8f13025230d1945b28c69695e9a21b9219ac8e" +checksum = "86815a2c475835751ffa9b8d9ac8ed86cf86294304c42bedd1103d54f25ecbfe" dependencies = [ "proc-macro2", "quote", - "syn 2.0.114", + "syn 2.0.116", ] [[package]] name = "deltalake" version = "0.30.1" -source = "git+https://github.com/tonyalaribe/delta-rs.git?rev=ba769136c5dd9b84a7335ea67e42b67884bfcce3#ba769136c5dd9b84a7335ea67e42b67884bfcce3" +source = "git+https://github.com/tonyalaribe/delta-rs.git?rev=c4d506da#c4d506daeace7c9298cb8a03dc417d611003e37a" dependencies = [ "ctor", "delta_kernel", @@ -2693,8 +2683,8 @@ dependencies = [ [[package]] name = "deltalake-aws" -version = "0.13.0" -source = "git+https://github.com/tonyalaribe/delta-rs.git?rev=ba769136c5dd9b84a7335ea67e42b67884bfcce3#ba769136c5dd9b84a7335ea67e42b67884bfcce3" +version = "0.13.1" +source = "git+https://github.com/tonyalaribe/delta-rs.git?rev=c4d506da#c4d506daeace7c9298cb8a03dc417d611003e37a" dependencies = [ "async-trait", "aws-config", @@ -2720,7 +2710,7 @@ dependencies = [ [[package]] name = "deltalake-core" version = "0.30.1" -source = "git+https://github.com/tonyalaribe/delta-rs.git?rev=ba769136c5dd9b84a7335ea67e42b67884bfcce3#ba769136c5dd9b84a7335ea67e42b67884bfcce3" +source = "git+https://github.com/tonyalaribe/delta-rs.git?rev=c4d506da#c4d506daeace7c9298cb8a03dc417d611003e37a" dependencies = [ "arrow", "arrow-arith", @@ -2740,6 +2730,7 @@ dependencies = [ "dashmap", "datafusion", "datafusion-datasource", + "datafusion-physical-expr-adapter", "datafusion-proto", "delta_kernel", "deltalake-derive", @@ -2747,7 +2738,7 @@ dependencies = [ "either", "futures", "humantime", - "indexmap 2.12.1", + "indexmap 2.13.0", "itertools 0.14.0", "num_cpus", "object_store", @@ -2773,13 +2764,13 @@ dependencies = [ [[package]] name = "deltalake-derive" version = "0.30.0" -source = "git+https://github.com/tonyalaribe/delta-rs.git?rev=ba769136c5dd9b84a7335ea67e42b67884bfcce3#ba769136c5dd9b84a7335ea67e42b67884bfcce3" +source = "git+https://github.com/tonyalaribe/delta-rs.git?rev=c4d506da#c4d506daeace7c9298cb8a03dc417d611003e37a" dependencies = [ "convert_case", "itertools 0.14.0", "proc-macro2", "quote", - "syn 2.0.114", + "syn 2.0.116", ] [[package]] @@ -2805,9 +2796,9 @@ dependencies = [ [[package]] name = "deranged" -version = "0.5.5" +version = "0.5.6" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "ececcb659e7ba858fb4f10388c250a7252eb0a27373f1a72b8748afdd248e587" +checksum = "cc3dc5ad92c2e2d1c193bbbbdf2ea477cb81331de4f3103f267ca18368b988c4" dependencies = [ "powerfmt", "serde_core", @@ -2821,7 +2812,7 @@ checksum = "2cdc8d50f426189eef89dac62fabfa0abb27d5cc008f25bf4156a0203325becc" dependencies = [ "proc-macro2", "quote", - "syn 2.0.114", + "syn 2.0.116", ] [[package]] @@ -2842,7 +2833,7 @@ dependencies = [ "darling 0.20.11", "proc-macro2", "quote", - "syn 2.0.114", + "syn 2.0.116", ] [[package]] @@ -2852,7 +2843,7 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "ab63b0e2bf4d5928aff72e83a7dace85d7bba5fe12dcc3c5a572d78caffd3f3c" dependencies = [ "derive_builder_core", - "syn 2.0.114", + "syn 2.0.116", ] [[package]] @@ -2896,7 +2887,7 @@ checksum = "97369cbbc041bc366949bc74d34658d6cda5621039731c6310521892a3a20ae0" dependencies = [ "proc-macro2", "quote", - "syn 2.0.114", + "syn 2.0.116", ] [[package]] @@ -2920,12 +2911,6 @@ version = "0.15.7" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "1aaf95b3e5c8f23aa320147307562d361db0ae0d51242340f558153b4eb2439b" -[[package]] -name = "downcast-rs" -version = "1.2.1" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "75b325c5dbd37f80359721ad39aca5a29fb04c89279657cffdda8736d0c0b9d2" - [[package]] name = "dtor" version = "0.1.1" @@ -2974,7 +2959,7 @@ dependencies = [ "enum-ordinalize", "proc-macro2", "quote", - "syn 2.0.114", + "syn 2.0.116", ] [[package]] @@ -3023,7 +3008,7 @@ checksum = "8ca9601fb2d62598ee17836250842873a413586e5d7ed88b356e38ddbb0ec631" dependencies = [ "proc-macro2", "quote", - "syn 2.0.114", + "syn 2.0.116", ] [[package]] @@ -3079,16 +3064,6 @@ dependencies = [ "pin-project-lite", ] -[[package]] -name = "event-listener-strategy" -version = "0.5.4" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "8be9f3dfaaffdae2972880079a491a1a8bb7cbed0b8dd7a347f668b4150a3b93" -dependencies = [ - "event-listener", - "pin-project-lite", -] - [[package]] name = "eyre" version = "0.6.12" @@ -3133,9 +3108,9 @@ dependencies = [ [[package]] name = "find-msvc-tools" -version = "0.1.5" +version = "0.1.9" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "3a3076410a55c90011c298b04d0cfa770b00fa04e1e3c97d3f6c9de105a03844" +checksum = "5baebc0774151f905a1a2cc41989300b1e6fbb29aff0ceffa1064fdd3088d582" [[package]] name = "fixedbitset" @@ -3155,13 +3130,13 @@ dependencies = [ [[package]] name = "flate2" -version = "1.1.5" +version = "1.1.9" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "bfe33edd8e85a12a67454e37f8c75e730830d83e313556ab9ebf9ee7fbeb3bfb" +checksum = "843fba2746e448b37e26a819579957415c8cef339bf08564fe8b7ddbd959573c" dependencies = [ "crc32fast", - "libz-rs-sys", "miniz_oxide", + "zlib-rs", ] [[package]] @@ -3172,7 +3147,6 @@ checksum = "da0e4dd2a88388a1f4ccc7c9ce104604dab68d9f408dc34cd45823d5a9069095" dependencies = [ "futures-core", "futures-sink", - "nanorand", "spin", ] @@ -3205,41 +3179,39 @@ dependencies = [ [[package]] name = "foyer" -version = "0.21.1" +version = "0.22.3" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "0a31f699ce88ac9a53677ca0b1f7a3a902bf3bfae0579e16e86ddf61dee569c0" +checksum = "3b0abc0b87814989efa711f9becd9f26969820e2d3905db27d10969c4bd45890" dependencies = [ "anyhow", "equivalent", "foyer-common", "foyer-memory", "foyer-storage", + "foyer-tokio", "futures-util", - "madsim-tokio", + "mea", "mixtrics", "pin-project", "serde", - "tokio", "tracing", ] [[package]] name = "foyer-common" -version = "0.21.1" +version = "0.22.3" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "e9ea2c266c9d93ea37c3960f2d0bb625981eefd38120eb06542804bf8169a187" +checksum = "a3db80d5dece93adb7ad709c84578794724a9cba342a7e566c3551c7ec626789" dependencies = [ "anyhow", "bincode 1.3.3", "bytes", "cfg-if", - "itertools 0.14.0", - "madsim-tokio", + "foyer-tokio", "mixtrics", "parking_lot", "pin-project", "serde", - "tokio", "twox-hash", ] @@ -3254,35 +3226,34 @@ dependencies = [ [[package]] name = "foyer-memory" -version = "0.21.1" +version = "0.22.3" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "09941796e5f8301e82e81e0c9e7514a8443524a461f9a2177500c7525aa73723" +checksum = "db907f40a527ca2aa2f40a5f68b32ea58aa70f050cd233518e9ffd402cfba6ce" dependencies = [ "anyhow", - "arc-swap", "bitflags", "cmsketch", "equivalent", "foyer-common", "foyer-intrusive-collections", + "foyer-tokio", "futures-util", "hashbrown 0.16.1", "itertools 0.14.0", - "madsim-tokio", + "mea", "mixtrics", "parking_lot", "paste", "pin-project", "serde", - "tokio", "tracing", ] [[package]] name = "foyer-storage" -version = "0.21.1" +version = "0.22.3" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "75fc3db8b685c3eb8b13f05847436933f1f68e91b68510c615d90e9a69541c84" +checksum = "1983f1db3d0710e9c9d5fc116d9202dccd41a2d1e032572224f1aff5520aa958" dependencies = [ "allocator-api2", "anyhow", @@ -3290,9 +3261,9 @@ dependencies = [ "core_affinity", "equivalent", "fastant", - "flume", "foyer-common", "foyer-memory", + "foyer-tokio", "fs4", "futures-core", "futures-util", @@ -3301,22 +3272,30 @@ dependencies = [ "itertools 0.14.0", "libc", "lz4", - "madsim-tokio", + "mea", "parking_lot", "pin-project", "rand 0.9.2", "serde", - "tokio", "tracing", "twox-hash", "zstd", ] +[[package]] +name = "foyer-tokio" +version = "0.22.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "f6577b05a7ffad0db555aedf00bfe52af818220fc4c1c3a7a12520896fc38627" +dependencies = [ + "tokio", +] + [[package]] name = "fs-err" -version = "3.2.1" +version = "3.3.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "824f08d01d0f496b3eca4f001a13cf17690a6ee930043d20817f547455fd98f8" +checksum = "73fde052dbfc920003cfd2c8e2c6e6d4cc7c1091538c3a24226cec0665ab08c0" dependencies = [ "autocfg", ] @@ -3345,9 +3324,9 @@ checksum = "e6d5a32815ae3f33302d95fdcb2ce17862f8c65363dcfd29360480ba1001fc9c" [[package]] name = "futures" -version = "0.3.31" +version = "0.3.32" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "65bc07b1a8bc7c85c5f2e110c476c7389b4554ba72af57d8445ea63a576b0876" +checksum = "8b147ee9d1f6d097cef9ce628cd2ee62288d963e16fb287bd9286455b241382d" dependencies = [ "futures-channel", "futures-core", @@ -3360,9 +3339,9 @@ dependencies = [ [[package]] name = "futures-channel" -version = "0.3.31" +version = "0.3.32" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "2dff15bf788c671c1934e366d07e30c1814a8ef514e1af724a602e8a2fbe1b10" +checksum = "07bbe89c50d7a535e539b8c17bc0b49bdb77747034daa8087407d655f3f7cc1d" dependencies = [ "futures-core", "futures-sink", @@ -3370,15 +3349,15 @@ dependencies = [ [[package]] name = "futures-core" -version = "0.3.31" +version = "0.3.32" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "05f29059c0c2090612e8d742178b0580d2dc940c837851ad723096f87af6663e" +checksum = "7e3450815272ef58cec6d564423f6e755e25379b217b0bc688e295ba24df6b1d" [[package]] name = "futures-executor" -version = "0.3.31" +version = "0.3.32" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "1e28d1d997f585e54aebc3f97d39e72338912123a67330d723fdbb564d646c9f" +checksum = "baf29c38818342a3b26b5b923639e7b1f4a61fc5e76102d4b1981c6dc7a7579d" dependencies = [ "futures-core", "futures-task", @@ -3398,38 +3377,38 @@ dependencies = [ [[package]] name = "futures-io" -version = "0.3.31" +version = "0.3.32" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "9e5c1b78ca4aae1ac06c48a526a655760685149f0d465d21f37abfe57ce075c6" +checksum = "cecba35d7ad927e23624b22ad55235f2239cfa44fd10428eecbeba6d6a717718" [[package]] name = "futures-macro" -version = "0.3.31" +version = "0.3.32" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "162ee34ebcb7c64a8abebc059ce0fee27c2262618d7b60ed8faf72fef13c3650" +checksum = "e835b70203e41293343137df5c0664546da5745f82ec9b84d40be8336958447b" dependencies = [ "proc-macro2", "quote", - "syn 2.0.114", + "syn 2.0.116", ] [[package]] name = "futures-sink" -version = "0.3.31" +version = "0.3.32" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "e575fab7d1e0dcb8d0c7bcf9a63ee213816ab51902e6d244a95819acacf1d4f7" +checksum = "c39754e157331b013978ec91992bde1ac089843443c49cbc7f46150b0fad0893" [[package]] name = "futures-task" -version = "0.3.31" +version = "0.3.32" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "f90f7dce0722e95104fcb095585910c0977252f286e354b5e3bd38902cd99988" +checksum = "037711b3d59c33004d3856fbdc83b99d4ff37a24768fa1be9ce3538a1cde4393" [[package]] name = "futures-util" -version = "0.3.31" +version = "0.3.32" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "9fa08315bb612088cc391249efdc3bc77536f16c91f6cf495e6fbe85b20a4a81" +checksum = "389ca41296e6190b48053de0321d02a77f32f8a5d2461dd38762c0593805c6d6" dependencies = [ "futures-channel", "futures-core", @@ -3439,7 +3418,6 @@ dependencies = [ "futures-task", "memchr", "pin-project-lite", - "pin-utils", "slab", ] @@ -3455,14 +3433,14 @@ dependencies = [ [[package]] name = "getrandom" -version = "0.2.16" +version = "0.2.17" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "335ff9f135e4384c8150d6f27c6daed433577f86b4750418338c01a1a2528592" +checksum = "ff2abc00be7fca6ebc474524697ae276ad847ad0a6b3faa4bcb027e9a4614ad0" dependencies = [ "cfg-if", "js-sys", "libc", - "wasi", + "wasi 0.11.1+wasi-snapshot-preview1", "wasm-bindgen", ] @@ -3480,6 +3458,20 @@ dependencies = [ "wasm-bindgen", ] +[[package]] +name = "getrandom" +version = "0.4.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "139ef39800118c7683f2fd3c98c1b23c09ae076556b435f8e9064ae108aaeeec" +dependencies = [ + "cfg-if", + "libc", + "r-efi", + "rand_core 0.10.0", + "wasip2", + "wasip3", +] + [[package]] name = "getset" version = "0.1.6" @@ -3489,7 +3481,7 @@ dependencies = [ "proc-macro-error2", "proc-macro2", "quote", - "syn 2.0.114", + "syn 2.0.116", ] [[package]] @@ -3527,7 +3519,7 @@ dependencies = [ "futures-sink", "futures-util", "http 0.2.12", - "indexmap 2.12.1", + "indexmap 2.13.0", "slab", "tokio", "tokio-util", @@ -3536,9 +3528,9 @@ dependencies = [ [[package]] name = "h2" -version = "0.4.12" +version = "0.4.13" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "f3c0b69cfcb4e1b9f1bf2f53f95f766e4661169728ec61cd3fe5a0166f2d1386" +checksum = "2f44da3a8150a6703ed5d34e164b875fd14c2cdab9af1252a9a1020bde2bdc54" dependencies = [ "atomic-waker", "bytes", @@ -3546,7 +3538,7 @@ dependencies = [ "futures-core", "futures-sink", "http 1.4.0", - "indexmap 2.12.1", + "indexmap 2.13.0", "slab", "tokio", "tokio-util", @@ -3764,7 +3756,7 @@ dependencies = [ "bytes", "futures-channel", "futures-core", - "h2 0.4.12", + "h2 0.4.13", "http 1.4.0", "http-body 1.0.1", "httparse", @@ -3800,7 +3792,7 @@ dependencies = [ "http 1.4.0", "hyper 1.8.1", "hyper-util", - "rustls 0.23.35", + "rustls 0.23.36", "rustls-native-certs", "rustls-pki-types", "tokio", @@ -3823,14 +3815,13 @@ dependencies = [ [[package]] name = "hyper-util" -version = "0.1.19" +version = "0.1.20" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "727805d60e7938b76b826a6ef209eb70eaa1812794f9424d4a4e2d740662df5f" +checksum = "96547c2556ec9d12fb1578c4eaf448b04993e7fb79cbaad930a656880a6bdfa0" dependencies = [ "base64", "bytes", "futures-channel", - "futures-core", "futures-util", "http 1.4.0", "http-body 1.0.1", @@ -3839,7 +3830,7 @@ dependencies = [ "libc", "percent-encoding", "pin-project-lite", - "socket2 0.6.1", + "socket2 0.6.2", "tokio", "tower-service", "tracing", @@ -3847,9 +3838,9 @@ dependencies = [ [[package]] name = "iana-time-zone" -version = "0.1.64" +version = "0.1.65" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "33e57f83510bb73707521ebaffa789ec8caf86f9657cad665b092b581d40e9fb" +checksum = "e31bc9ad994ba00e440a8aa5c9ef0ec67d5cb5e5cb0cc7f8b744a35b389cc470" dependencies = [ "android_system_properties", "core-foundation-sys", @@ -3950,6 +3941,12 @@ dependencies = [ "zerovec", ] +[[package]] +name = "id-arena" +version = "2.3.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "3d3067d79b975e8844ca9eb072e16b31c3c1c36928edf9c6789548c524d0d954" + [[package]] name = "ident_case" version = "1.0.1" @@ -4015,9 +4012,9 @@ dependencies = [ [[package]] name = "indexmap" -version = "2.12.1" +version = "2.13.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "0ad4bb2b565bca0645f4d68c5c9af97fba094e9791da685bf83cb5f3ce74acf2" +checksum = "7714e70437a7dc3ac8eb7e6f8df75fd8eb422675fc7678aff7364301092b1017" dependencies = [ "equivalent", "hashbrown 0.16.1", @@ -4081,40 +4078,20 @@ checksum = "469fb0b9cefa57e3ef31275ee7cacb78f2fdca44e4765491884a2b119d4eb130" [[package]] name = "iri-string" -version = "0.7.9" +version = "0.7.10" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "4f867b9d1d896b67beb18518eda36fdb77a32ea590de864f1325b294a6d14397" +checksum = "c91338f0783edbd6195decb37bae672fd3b165faffb89bf7b9e6942f8b1a731a" dependencies = [ "memchr", "serde", ] -[[package]] -name = "is-terminal" -version = "0.4.17" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "3640c1c38b8e4e43584d8df18be5fc6b0aa314ce6ebf51b53313d4306cca8e46" -dependencies = [ - "hermit-abi", - "libc", - "windows-sys 0.61.2", -] - [[package]] name = "is_terminal_polyfill" version = "1.70.2" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "a6cb138bb79a146c1bd460005623e142ef0181e3d0219cb493e02f7d08a35695" -[[package]] -name = "itertools" -version = "0.10.5" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "b0fd2260e829bddf4cb6ea802289de2f86d6a7a690192fbe91b3f46e0f2c8473" -dependencies = [ - "either", -] - [[package]] name = "itertools" version = "0.13.0" @@ -4135,9 +4112,9 @@ dependencies = [ [[package]] name = "itoa" -version = "1.0.16" +version = "1.0.17" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "7ee5b5339afb4c41626dde77b7a611bd4f2c202b897852b4bcf5d03eddc61010" +checksum = "92ecc6618181def0457392ccd0ee51198e065e016d1d527a7ac1b6dc7c1f09d2" [[package]] name = "jiter" @@ -4166,9 +4143,9 @@ dependencies = [ [[package]] name = "js-sys" -version = "0.3.83" +version = "0.3.85" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "464a3709c7f55f1f721e5389aa6ea4e3bc6aba669353300af094b29ffbdde1d8" +checksum = "8c942ebf8e95485ca0d52d97da7c5a2c387d0e7f0ba4c35e93bfcaee045955b3" dependencies = [ "once_cell", "wasm-bindgen", @@ -4176,9 +4153,9 @@ dependencies = [ [[package]] name = "lazy-regex" -version = "3.4.2" +version = "3.6.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "191898e17ddee19e60bccb3945aa02339e81edd4a8c50e21fd4d48cdecda7b29" +checksum = "6bae91019476d3ec7147de9aa291cadb6d870abf2f3015d2da73a90325ac1496" dependencies = [ "lazy-regex-proc_macros", "once_cell", @@ -4187,14 +4164,14 @@ dependencies = [ [[package]] name = "lazy-regex-proc_macros" -version = "3.4.2" +version = "3.6.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c35dc8b0da83d1a9507e12122c80dea71a9c7c613014347392483a83ea593e04" +checksum = "4de9c1e1439d8b7b3061b2d209809f447ca33241733d9a3c01eabf2dc8d94358" dependencies = [ "proc-macro2", "quote", "regex", - "syn 2.0.114", + "syn 2.0.116", ] [[package]] @@ -4206,6 +4183,12 @@ dependencies = [ "spin", ] +[[package]] +name = "leb128fmt" +version = "0.1.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "09edd9e8b54e49e587e4f6295a7d29c3ea94d469cb40ab8ca70b288248a81db2" + [[package]] name = "lexical-core" version = "1.0.6" @@ -4271,9 +4254,9 @@ checksum = "2c4a545a15244c7d945065b5d392b2d2d7f21526fba56ce51467b06ed445e8f7" [[package]] name = "libc" -version = "0.2.178" +version = "0.2.182" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "37c93d8daa9d8a012fd8ab92f088405fb202ea0b6ab73ee2482ae66af4f42091" +checksum = "6800badb6cb2082ffd7b6a67e6125bb39f18782f793520caee8cb8846be06112" [[package]] name = "liblzma" @@ -4297,19 +4280,19 @@ dependencies = [ [[package]] name = "libm" -version = "0.2.15" +version = "0.2.16" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "f9fbbcab51052fe104eb5e5d351cf728d30a5be1fe14d9be8a3b097481fb97de" +checksum = "b6d2cec3eae94f9f509c767b45932f1ada8350c4bdb85af2fcab4a3c14807981" [[package]] name = "libredox" -version = "0.1.11" +version = "0.1.12" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "df15f6eac291ed1cf25865b1ee60399f57e7c227e7f51bdbd4c5270396a9ed50" +checksum = "3d0b95e02c851351f877147b7deea7b1afb1df71b63aa5f8270716e0c5720616" dependencies = [ "bitflags", "libc", - "redox_syscall 0.6.0", + "redox_syscall 0.7.1", ] [[package]] @@ -4334,15 +4317,6 @@ dependencies = [ "escape8259", ] -[[package]] -name = "libz-rs-sys" -version = "0.5.5" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c10501e7805cee23da17c7790e59df2870c0d4043ec6d03f67d31e2b53e77415" -dependencies = [ - "zlib-rs", -] - [[package]] name = "linux-raw-sys" version = "0.11.0" @@ -4387,9 +4361,9 @@ dependencies = [ [[package]] name = "lru" -version = "0.16.2" +version = "0.16.3" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "96051b46fc183dc9cd4a223960ef37b9af631b55191852a8274bfef064cda20f" +checksum = "a1dc47f592c06f33f8e3aea9591776ec7c9f9e4124778ff8a3c3b87159f7e593" dependencies = [ "hashbrown 0.16.1", ] @@ -4429,65 +4403,10 @@ dependencies = [ ] [[package]] -name = "madsim" -version = "0.2.34" +name = "marrow" +version = "0.2.5" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "18351aac4194337d6ea9ffbd25b3d1540ecc0754142af1bff5ba7392d1f6f771" -dependencies = [ - "ahash 0.8.12", - "async-channel", - "async-stream", - "async-task", - "bincode 1.3.3", - "bytes", - "downcast-rs", - "errno", - "futures-util", - "lazy_static", - "libc", - "madsim-macros", - "naive-timer", - "panic-message", - "rand 0.8.5", - "rand_xoshiro", - "rustversion", - "serde", - "spin", - "tokio", - "tokio-util", - "toml", - "tracing", - "tracing-subscriber", -] - -[[package]] -name = "madsim-macros" -version = "0.2.12" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "f3d248e97b1a48826a12c3828d921e8548e714394bf17274dd0a93910dc946e1" -dependencies = [ - "darling 0.14.4", - "proc-macro2", - "quote", - "syn 1.0.109", -] - -[[package]] -name = "madsim-tokio" -version = "0.2.30" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "7d3eb2acc57c82d21d699119b859e2df70a91dbdb84734885a1e72be83bdecb5" -dependencies = [ - "madsim", - "spin", - "tokio", -] - -[[package]] -name = "marrow" -version = "0.2.5" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "ea734fcb7619dfcc47a396f7bf0c72571ccc8c18ae7236ae028d485b27424b74" +checksum = "ea734fcb7619dfcc47a396f7bf0c72571ccc8c18ae7236ae028d485b27424b74" dependencies = [ "arrow-array", "arrow-buffer", @@ -4523,17 +4442,26 @@ version = "0.8.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "ae960838283323069879657ca3de837e9f7bbb4c7bf6ea7f1b290d5e9476d2e0" +[[package]] +name = "mea" +version = "0.6.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "6747f54621d156e1b47eb6b25f39a941b9fc347f98f67d25d8881ff99e8ed832" +dependencies = [ + "slab", +] + [[package]] name = "memchr" -version = "2.7.6" +version = "2.8.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "f52b00d39961fc5b2736ea853c9cc86238e165017a493d1d5c8eac6bdc4cc273" +checksum = "f8ca58f447f06ed17d5fc4043ce1b10dd205e060fb3ce5b979b8ed8e59ff3f79" [[package]] name = "memmap2" -version = "0.9.9" +version = "0.9.10" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "744133e4a0e0a658e1374cf3bf8e415c4052a15a111acd372764c55b4177d490" +checksum = "714098028fe011992e1c3962653c96b2d578c4b4bce9036e15ff220319b1e0e3" dependencies = [ "libc", ] @@ -4570,7 +4498,7 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "a69bcab0ad47271a0234d9422b131806bf3968021e5dc9328caf2d4cd58557fc" dependencies = [ "libc", - "wasi", + "wasi 0.11.1+wasi-snapshot-preview1", "windows-sys 0.61.2", ] @@ -4584,21 +4512,6 @@ dependencies = [ "parking_lot", ] -[[package]] -name = "naive-timer" -version = "0.2.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "034a0ad7deebf0c2abcf2435950a6666c3c15ea9d8fad0c0f48efa8a7f843fed" - -[[package]] -name = "nanorand" -version = "0.7.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "6a51313c5820b0b02bd422f4b44776fbf47961755c74ce64afc73bfad10226c3" -dependencies = [ - "getrandom 0.2.16", -] - [[package]] name = "nom" version = "7.1.3" @@ -4655,9 +4568,9 @@ dependencies = [ [[package]] name = "num-conv" -version = "0.1.0" +version = "0.2.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "51d515d32fb182ee37cda2ccdcb92950d6a3c2893aa280e540671c2cd0f3b1d9" +checksum = "cf97ec579c3c42f953ef76dbf8d55ac91fb219dde70e49aa4a6b7d74e9919050" [[package]] name = "num-derive" @@ -4667,7 +4580,7 @@ checksum = "ed3955f1a9c7c0c15e092f9c887db08b1fc683305fdf6eb6684f22555355e202" dependencies = [ "proc-macro2", "quote", - "syn 2.0.114", + "syn 2.0.116", ] [[package]] @@ -4711,12 +4624,21 @@ dependencies = [ ] [[package]] -name = "object" -version = "0.32.2" +name = "objc2-core-foundation" +version = "0.3.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a6a622008b6e321afc04970976f62ee297fdbaa6f95318ca343e3eebb9648441" +checksum = "2a180dd8642fa45cdb7dd721cd4c11b1cadd4929ce112ebd8b9f5803cc79d536" dependencies = [ - "memchr", + "bitflags", +] + +[[package]] +name = "objc2-system-configuration" +version = "0.3.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "7216bd11cbda54ccabcab84d523dc93b858ec75ecfb3a7d89513fa22464da396" +dependencies = [ + "objc2-core-foundation", ] [[package]] @@ -4730,9 +4652,9 @@ dependencies = [ [[package]] name = "object_store" -version = "0.12.4" +version = "0.12.5" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "4c1be0c6c22ec0817cdc77d3842f721a17fd30ab6965001415b5402a74e6b740" +checksum = "fbfbfff40aeccab00ec8a910b57ca8ecf4319b335c542f2edcd19dd25a1e2a00" dependencies = [ "async-trait", "base64", @@ -4786,9 +4708,9 @@ checksum = "d6790f58c7ff633d8771f42965289203411a5e5c68388703c06e14f24770b41e" [[package]] name = "openssl-probe" -version = "0.1.6" +version = "0.2.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d05e27ee213611ffe7d6348b942e8f942b37114c00cc03cec254295a4a17852e" +checksum = "7c87def4c32ab89d880effc9e097653c8da5d6ef28e6b539d313baaacfbafcbe" [[package]] name = "opentelemetry" @@ -4905,10 +4827,14 @@ dependencies = [ ] [[package]] -name = "panic-message" -version = "0.3.0" +name = "page_size" +version = "0.6.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "384e52fd8fbd4cbe3c317e8216260c21a0f9134de108cea8a4dd4e7e152c472d" +checksum = "30d5b2194ed13191c1999ae0704b7839fb18384fa22e49b57eeaa97d79ce40da" +dependencies = [ + "libc", + "winapi", +] [[package]] name = "parking" @@ -4941,9 +4867,9 @@ dependencies = [ [[package]] name = "parquet" -version = "57.1.0" +version = "57.3.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "be3e4f6d320dd92bfa7d612e265d7d08bba0a240bab86af3425e1d255a511d89" +checksum = "6ee96b29972a257b855ff2341b37e61af5f12d6af1158b6dcdb5b31ea07bb3cb" dependencies = [ "ahash 0.8.12", "arrow-array", @@ -4978,29 +4904,29 @@ dependencies = [ [[package]] name = "parquet-variant" -version = "57.2.0" +version = "57.3.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c254fac16af78ad96aa442290cb6504951c4d484fdfcfe58f4588033d30e4c8f" +checksum = "a6c31f8f9bfefb9dbf67b0807e00fd918676954a7477c889be971ac904103184" dependencies = [ "arrow-schema", "chrono", "half", - "indexmap 2.12.1", + "indexmap 2.13.0", "simdutf8", "uuid", ] [[package]] name = "parquet-variant-compute" -version = "57.2.0" +version = "57.3.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "2178772f1c5ad7e5da8b569d986d3f5cbb4a4cee915925f28fdc700dbb2e80cf" +checksum = "196cd9f7178fed3ac8d5e6d2b51193818e896bbc3640aea3fde3440114a8f39c" dependencies = [ "arrow", "arrow-schema", "chrono", "half", - "indexmap 2.12.1", + "indexmap 2.13.0", "parquet-variant", "parquet-variant-json", "uuid", @@ -5008,9 +4934,9 @@ dependencies = [ [[package]] name = "parquet-variant-json" -version = "57.2.0" +version = "57.3.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "7a1510daa121c04848368f9c38d0be425b9418c70be610ecc0aa8071738c0ef3" +checksum = "ed23d7acc90ef60f7fdbcc473fa2fdaefa33542ed15b84388959346d52c839be" dependencies = [ "arrow-schema", "base64", @@ -5065,7 +4991,7 @@ checksum = "8701b58ea97060d5e5b155d383a69952a60943f0e6dfe30b04c287beb0b27455" dependencies = [ "fixedbitset", "hashbrown 0.15.5", - "indexmap 2.12.1", + "indexmap 2.13.0", "serde", ] @@ -5082,9 +5008,9 @@ dependencies = [ [[package]] name = "pgwire" -version = "0.37.3" +version = "0.38.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "6fcd410bc6990bd8d20b3fe3cd879a3c3ec250bdb1cb12537b528818823b02c9" +checksum = "89d5e5a60d3f6e40c91f6a2a7f8d09665e636272bd5611977253559b6651aabb" dependencies = [ "async-trait", "base64", @@ -5097,7 +5023,7 @@ dependencies = [ "md5", "pg_interval_2", "postgres-types", - "rand 0.9.2", + "rand 0.10.0", "ring", "rust_decimal", "rustls-pki-types", @@ -5167,7 +5093,7 @@ checksum = "6e918e4ff8c4549eb882f14b3a4bc8c8bc93de829416eacf579f1207a8fbf861" dependencies = [ "proc-macro2", "quote", - "syn 2.0.114", + "syn 2.0.116", ] [[package]] @@ -5249,15 +5175,15 @@ dependencies = [ [[package]] name = "portable-atomic" -version = "1.12.0" +version = "1.13.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "f59e70c4aef1e55797c2e8fd94a4f2a973fc972cfde0e0b05f683667b0cd39dd" +checksum = "c33a9471896f1c69cecef8d20cbe2f7accd12527ce60845ff44c153bb2a21b49" [[package]] name = "postgres-protocol" -version = "0.6.9" +version = "0.6.10" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "fbef655056b916eb868048276cfd5d6a7dea4f81560dfd047f97c8c6fe3fcfd4" +checksum = "3ee9dd5fe15055d2b6806f4736aa0c9637217074e224bbec46d4041b91bb9491" dependencies = [ "base64", "byteorder", @@ -5273,9 +5199,9 @@ dependencies = [ [[package]] name = "postgres-types" -version = "0.2.11" +version = "0.2.12" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "ef4605b7c057056dd35baeb6ac0c0338e4975b1f2bef0f65da953285eb007095" +checksum = "54b858f82211e84682fecd373f68e1ceae642d8d751a1ebd13f33de6257b3e20" dependencies = [ "array-init", "bytes", @@ -5310,6 +5236,16 @@ dependencies = [ "zerocopy", ] +[[package]] +name = "prettyplease" +version = "0.2.37" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "479ca8adacdd7ce8f1fb39ce9ecccbfe93a3f1344b3d0d97f20bc0196208f62b" +dependencies = [ + "proc-macro2", + "syn 2.0.116", +] + [[package]] name = "proc-macro-crate" version = "3.4.0" @@ -5338,23 +5274,23 @@ dependencies = [ "proc-macro-error-attr2", "proc-macro2", "quote", - "syn 2.0.114", + "syn 2.0.116", ] [[package]] name = "proc-macro2" -version = "1.0.103" +version = "1.0.106" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "5ee95bc4ef87b8d5ba32e8b7714ccc834865276eab0aed5c9958d00ec45f49e8" +checksum = "8fd00f0bb2e90d81d1044c2b32617f68fcb9fa3bb7640c23e9c748e53fb30934" dependencies = [ "unicode-ident", ] [[package]] name = "prost" -version = "0.14.1" +version = "0.14.3" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "7231bd9b3d3d33c86b58adbac74b5ec0ad9f496b19d22801d773636feaa95f3d" +checksum = "d2ea70524a2f82d518bce41317d0fae74151505651af45faf1ffbd6fd33f0568" dependencies = [ "bytes", "prost-derive", @@ -5362,22 +5298,22 @@ dependencies = [ [[package]] name = "prost-derive" -version = "0.14.1" +version = "0.14.3" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "9120690fafc389a67ba3803df527d0ec9cbbc9cc45e4cc20b332996dfb672425" +checksum = "27c6023962132f4b30eb4c172c91ce92d933da334c59c23cddee82358ddafb0b" dependencies = [ "anyhow", "itertools 0.14.0", "proc-macro2", "quote", - "syn 2.0.114", + "syn 2.0.116", ] [[package]] name = "psm" -version = "0.1.28" +version = "0.1.30" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d11f2fedc3b7dafdc2851bc52f277377c5473d378859be234bc7ebb593144d01" +checksum = "3852766467df634d74f0b2d7819bf8dc483a0eb2e3b0f50f756f9cfe8b0d18d8" dependencies = [ "ar_archive_writer", "cc", @@ -5450,7 +5386,7 @@ dependencies = [ "proc-macro2", "pyo3-macros-backend", "quote", - "syn 2.0.114", + "syn 2.0.116", ] [[package]] @@ -5463,7 +5399,7 @@ dependencies = [ "proc-macro2", "pyo3-build-config", "quote", - "syn 2.0.114", + "syn 2.0.116", ] [[package]] @@ -5488,8 +5424,8 @@ dependencies = [ "quinn-proto", "quinn-udp", "rustc-hash", - "rustls 0.23.35", - "socket2 0.6.1", + "rustls 0.23.36", + "socket2 0.6.2", "thiserror", "tokio", "tracing", @@ -5508,7 +5444,7 @@ dependencies = [ "rand 0.9.2", "ring", "rustc-hash", - "rustls 0.23.35", + "rustls 0.23.36", "rustls-pki-types", "slab", "thiserror", @@ -5526,16 +5462,16 @@ dependencies = [ "cfg_aliases", "libc", "once_cell", - "socket2 0.6.1", + "socket2 0.6.2", "tracing", "windows-sys 0.60.2", ] [[package]] name = "quote" -version = "1.0.42" +version = "1.0.44" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a338cc41d27e6cc6dce6cefc13a0729dfbb81c262b1f519331575dd80ef3067f" +checksum = "21b2ebcf727b7760c461f091f9f0f539b77b8e87f2fd88131e7f1b433b3cece4" dependencies = [ "proc-macro2", ] @@ -5570,7 +5506,18 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "6db2770f06117d490610c7488547d543617b21bfa07796d7a12f6f1bd53850d1" dependencies = [ "rand_chacha 0.9.0", - "rand_core 0.9.3", + "rand_core 0.9.5", +] + +[[package]] +name = "rand" +version = "0.10.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "bc266eb313df6c5c09c1c7b1fbe2510961e5bcd3add930c1e31f7ed9da0feff8" +dependencies = [ + "chacha20", + "getrandom 0.4.1", + "rand_core 0.10.0", ] [[package]] @@ -5590,7 +5537,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.5", ] [[package]] @@ -5599,26 +5546,23 @@ version = "0.6.4" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "ec0be4795e2f6a28069bec0b5ff3e2ac9bafc99e6a9a7dc3547996c5c816922c" dependencies = [ - "getrandom 0.2.16", + "getrandom 0.2.17", ] [[package]] name = "rand_core" -version = "0.9.3" +version = "0.9.5" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "99d9a13982dcf210057a8a78572b2217b667c3beacbf3a0d8b454f6f82837d38" +checksum = "76afc826de14238e6e8c374ddcc1fa19e374fd8dd986b0d2af0d02377261d83c" dependencies = [ "getrandom 0.3.4", ] [[package]] -name = "rand_xoshiro" -version = "0.6.0" +name = "rand_core" +version = "0.10.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "6f97cdb2a36ed4183de61b2f824cc45c9f1037f28afe0a322e9fff4c108b5aaa" -dependencies = [ - "rand_core 0.6.4", -] +checksum = "0c8d0fd677905edcbeedbf2edb6494d676f0e98d54d5cf9bda0b061cb8fb8aba" [[package]] name = "rayon" @@ -5657,7 +5601,7 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "76009fbe0614077fc1a2ce255e3a1881a2e3a3527097d5dc6d8212c585e7e38b" dependencies = [ "quote", - "syn 2.0.114", + "syn 2.0.116", ] [[package]] @@ -5671,9 +5615,9 @@ dependencies = [ [[package]] name = "redox_syscall" -version = "0.6.0" +version = "0.7.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "ec96166dafa0886eb81fe1c0a388bece180fbef2135f97c1e2cf8302e74b43b5" +checksum = "35985aa610addc02e24fc232012c86fd11f14111180f902b67e2d5331f8ebf2b" dependencies = [ "bitflags", ] @@ -5684,7 +5628,7 @@ version = "0.5.2" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "a4e608c6638b9c18977b00b475ac1f28d14e84b27d8d42f70e0bf1e3dec127ac" dependencies = [ - "getrandom 0.2.16", + "getrandom 0.2.17", "libredox", "thiserror", ] @@ -5706,14 +5650,14 @@ checksum = "b7186006dcb21920990093f30e3dea63b7d6e977bf1256be20c3563a5db070da" dependencies = [ "proc-macro2", "quote", - "syn 2.0.114", + "syn 2.0.116", ] [[package]] name = "regex" -version = "1.12.2" +version = "1.12.3" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "843bc0191f75f3e22651ae5f1e72939ab2f72a4bc30fa80a066bd66edefc24d4" +checksum = "e10754a14b9137dd7b1e3e5b0493cc9171fdd105e0ab477f51b72e7f3ac0e276" dependencies = [ "aho-corasick", "memchr", @@ -5723,9 +5667,9 @@ dependencies = [ [[package]] name = "regex-automata" -version = "0.4.13" +version = "0.4.14" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "5276caf25ac86c8d810222b3dbb938e512c55c6831a10f3e6ed1c93b84041f1c" +checksum = "6e1dd4122fc1595e8162618945476892eefca7b88c52820e74af6262213cae8f" dependencies = [ "aho-corasick", "memchr", @@ -5734,15 +5678,15 @@ dependencies = [ [[package]] name = "regex-lite" -version = "0.1.8" +version = "0.1.9" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "8d942b98df5e658f56f20d592c7f868833fe38115e65c33003d8cd224b0155da" +checksum = "cab834c73d247e67f4fae452806d17d3c7501756d98c8808d7c9c7aa7d18f973" [[package]] name = "regex-syntax" -version = "0.8.8" +version = "0.8.9" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "7a2d987857b319362043e95f5353c0535c1f58eec5336fdfcf626430af7def58" +checksum = "a96887878f22d7bad8a3b6dc5b7440e0ada9a245242924394987b21cf2210a4c" [[package]] name = "rend" @@ -5764,7 +5708,7 @@ dependencies = [ "futures-channel", "futures-core", "futures-util", - "h2 0.4.12", + "h2 0.4.13", "http 1.4.0", "http-body 1.0.1", "http-body-util", @@ -5776,7 +5720,7 @@ dependencies = [ "percent-encoding", "pin-project-lite", "quinn", - "rustls 0.23.35", + "rustls 0.23.36", "rustls-native-certs", "rustls-pki-types", "serde", @@ -5815,7 +5759,7 @@ checksum = "a4689e6c2294d81e88dc6261c768b63bc4fcdb852be6d1352498b114f61383b7" dependencies = [ "cc", "cfg-if", - "getrandom 0.2.16", + "getrandom 0.2.17", "libc", "untrusted", "windows-sys 0.52.0", @@ -5823,9 +5767,9 @@ dependencies = [ [[package]] name = "rkyv" -version = "0.7.45" +version = "0.7.46" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "9008cd6385b9e161d8229e1f6549dd23c3d022f132a2ea37ac3a10ac4935779b" +checksum = "2297bf9c81a3f0dc96bc9521370b88f054168c29826a75e89c55ff196e7ed6a1" dependencies = [ "bitvec", "bytecheck", @@ -5841,9 +5785,9 @@ dependencies = [ [[package]] name = "rkyv_derive" -version = "0.7.45" +version = "0.7.46" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "503d1d27590a2b0a3a4ca4c94755aa2875657196ecbf401a42eff41d7de532c0" +checksum = "84d7b42d4b8d06048d3ac8db0eb31bcb942cbeb709f0b5f2b2ebde398d3038f5" dependencies = [ "proc-macro2", "quote", @@ -5862,9 +5806,9 @@ dependencies = [ [[package]] name = "rsa" -version = "0.9.9" +version = "0.9.10" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "40a0376c50d0358279d9d643e4bf7b7be212f1f4ff1da9070a7b54d22ef75c88" +checksum = "b8573f03f5883dcaebdfcf4725caa1ecb9c15b2ef50c43a07b816e06799bb12d" dependencies = [ "const-oid", "digest", @@ -5882,9 +5826,9 @@ dependencies = [ [[package]] name = "rust_decimal" -version = "1.39.0" +version = "1.40.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "35affe401787a9bd846712274d97654355d21b2a2c092a3139aabe31e9022282" +checksum = "61f703d19852dbf87cbc513643fa81428361eb6940f1ac14fd58155d295a3eb0" dependencies = [ "arrayvec", "borsh", @@ -5899,9 +5843,9 @@ dependencies = [ [[package]] name = "rustc-demangle" -version = "0.1.26" +version = "0.1.27" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "56f7d92ca342cea22a06f2121d944b4fd82af56988c270852495420f961d4ace" +checksum = "b50b8869d9fc858ce7266cce0194bd74df58b9d0e3f6df3a9fc8eb470d95c09d" [[package]] name = "rustc-hash" @@ -5920,9 +5864,9 @@ dependencies = [ [[package]] name = "rustix" -version = "1.1.2" +version = "1.1.3" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "cd15f8a2c5551a84d56efdc1cd049089e409ac19a3072d5037a17fd70719ff3e" +checksum = "146c9e247ccc180c1f61615433868c99f3de3ae256a30a43b49f67c2d9171f34" dependencies = [ "bitflags", "errno", @@ -5945,25 +5889,25 @@ dependencies = [ [[package]] name = "rustls" -version = "0.23.35" +version = "0.23.36" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "533f54bc6a7d4f647e46ad909549eda97bf5afc1585190ef692b4286b198bd8f" +checksum = "c665f33d38cea657d9614f766881e4d510e0eda4239891eea56b4cadcf01801b" dependencies = [ "aws-lc-rs", "log", "once_cell", "ring", "rustls-pki-types", - "rustls-webpki 0.103.8", + "rustls-webpki 0.103.9", "subtle", "zeroize", ] [[package]] name = "rustls-native-certs" -version = "0.8.2" +version = "0.8.3" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "9980d917ebb0c0536119ba501e90834767bffc3d60641457fd84a1f3fd337923" +checksum = "612460d5f7bea540c490b2b6395d8e34a953e52b491accd6c86c8164c5932a63" dependencies = [ "openssl-probe", "rustls-pki-types", @@ -5982,9 +5926,9 @@ dependencies = [ [[package]] name = "rustls-pki-types" -version = "1.13.2" +version = "1.14.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "21e6f2ab2928ca4291b86736a8bd920a277a399bba1589409d72154ff87c1282" +checksum = "be040f8b0a225e40375822a563fa9524378b9d63112f53e19ffff34df5d33fdd" dependencies = [ "web-time", "zeroize", @@ -6002,9 +5946,9 @@ dependencies = [ [[package]] name = "rustls-webpki" -version = "0.103.8" +version = "0.103.9" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "2ffdfa2f5286e2247234e03f680868ac2815974dc39e00ea15adc445d0aafe52" +checksum = "d7df23109aa6c1567d1c575b9952556388da57401e4ace1d15f79eedad0d8f53" dependencies = [ "aws-lc-rs", "ring", @@ -6020,9 +5964,9 @@ checksum = "b39cdef0fa800fc44525c84ccb54a029961a8215f9619753635a9c0d2538d46d" [[package]] name = "ryu" -version = "1.0.21" +version = "1.0.23" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "62049b2877bf12821e8f9ad256ee38fdc31db7387ec2d3b3f403024de2034aea" +checksum = "9774ba4a74de5f7b1c1451ed6cd5285a32eddb5cccb8cc655a4e50009e06477f" [[package]] name = "same-file" @@ -6065,9 +6009,9 @@ dependencies = [ [[package]] name = "schemars" -version = "1.1.0" +version = "1.2.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "9558e172d4e8533736ba97870c4b2cd63f84b382a3d6eb063da41b91cce17289" +checksum = "a2b42f36aa1cd011945615b92222f6bf73c599a102a300334cd7f8dbeec726cc" dependencies = [ "dyn-clone", "ref-cast", @@ -6119,9 +6063,9 @@ dependencies = [ [[package]] name = "security-framework" -version = "3.5.1" +version = "3.6.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "b3297343eaf830f66ede390ea39da1d462b6b0c1b000f420d0a83f898bbbe6ef" +checksum = "d17b898a6d6948c3a8ee4372c17cb384f90d2e6e912ef00895b14fd7ab54ec38" dependencies = [ "bitflags", "core-foundation", @@ -6132,9 +6076,9 @@ dependencies = [ [[package]] name = "security-framework-sys" -version = "2.15.0" +version = "2.16.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "cc1f0cbffaac4852523ce30d8bd3c5cdc873501d96ff467ca09b6767bb8cd5c0" +checksum = "321c8673b092a9a42605034a9879d73cb79101ed5fd117bc9a597b89b4e9e61a" dependencies = [ "core-foundation-sys", "libc", @@ -6204,20 +6148,20 @@ checksum = "d540f220d3187173da220f885ab66608367b6574e925011a9353e4badda91d79" dependencies = [ "proc-macro2", "quote", - "syn 2.0.114", + "syn 2.0.116", ] [[package]] name = "serde_json" -version = "1.0.146" +version = "1.0.149" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "217ca874ae0207aac254aa02c957ded05585a90892cc8d87f9e5fa49669dadd8" +checksum = "83fc039473c5595ace860d8c4fafa220ff474b3fc6bfdb4293327f1a37e94d86" dependencies = [ "itoa", "memchr", - "ryu", "serde", "serde_core", + "zmij", ] [[package]] @@ -6267,16 +6211,7 @@ checksum = "aafbefbe175fa9bf03ca83ef89beecff7d2a95aaacd5732325b90ac8c3bd7b90" dependencies = [ "proc-macro2", "quote", - "syn 2.0.114", -] - -[[package]] -name = "serde_spanned" -version = "1.0.4" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "f8bbf91e5a4d6315eee45e704372590b30e260ee83af6639d64557f51b067776" -dependencies = [ - "serde_core", + "syn 2.0.116", ] [[package]] @@ -6301,9 +6236,9 @@ dependencies = [ "chrono", "hex", "indexmap 1.9.3", - "indexmap 2.12.1", + "indexmap 2.13.0", "schemars 0.9.0", - "schemars 1.1.0", + "schemars 1.2.1", "serde_core", "serde_json", "serde_with_macros", @@ -6319,7 +6254,7 @@ dependencies = [ "darling 0.21.3", "proc-macro2", "quote", - "syn 2.0.114", + "syn 2.0.116", ] [[package]] @@ -6328,7 +6263,7 @@ version = "0.9.34+deprecated" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "6a8b1a1a2ebf674015cc02edccce75287f1a0130d394307b36743c2f5d504b47" dependencies = [ - "indexmap 2.12.1", + "indexmap 2.13.0", "itoa", "ryu", "serde", @@ -6337,11 +6272,12 @@ dependencies = [ [[package]] name = "serial_test" -version = "3.2.0" +version = "3.3.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "1b258109f244e1d6891bf1053a55d63a5cd4f8f4c30cf9a1280989f80e7a1fa9" +checksum = "0d0b343e184fc3b7bb44dff0705fffcf4b3756ba6aff420dddd8b24ca145e555" dependencies = [ - "futures", + "futures-executor", + "futures-util", "log", "once_cell", "parking_lot", @@ -6351,13 +6287,13 @@ dependencies = [ [[package]] name = "serial_test_derive" -version = "3.2.0" +version = "3.3.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "5d69265a08751de7844521fd15003ae0a888e035773ba05695c5c759a6f89eef" +checksum = "6f50427f258fb77356e4cd4aa0e87e2bd2c66dbcee41dc405282cae2bfc26c83" dependencies = [ "proc-macro2", "quote", - "syn 2.0.114", + "syn 2.0.116", ] [[package]] @@ -6367,7 +6303,7 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "e3bf829a2d51ab4a5ddf1352d8470c140cadc8301b2ae1789db023f01cedd6ba" dependencies = [ "cfg-if", - "cpufeatures", + "cpufeatures 0.2.17", "digest", ] @@ -6378,7 +6314,7 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "a7507d819769d01a365ab707794a4084392c824f54a7a6a7862f8c3d0892b283" dependencies = [ "cfg-if", - "cpufeatures", + "cpufeatures 0.2.17", "digest", ] @@ -6399,10 +6335,11 @@ checksum = "0fda2ff0d084019ba4d7c6f371c95d8fd75ce3524c3cb8fb653a3023f6323e64" [[package]] name = "signal-hook-registry" -version = "1.4.7" +version = "1.4.8" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "7664a098b8e616bdfcc2dc0e9ac44eb231eedf41db4e9fe95d8d32ec728dedad" +checksum = "c4db69cba1110affc0e9f7bcd48bbf87b3f4fc7c61fc9155afd4c469eb3d6c1b" dependencies = [ + "errno", "libc", ] @@ -6446,15 +6383,15 @@ checksum = "bbbb5d9659141646ae647b42fe094daf6c6192d1620870b449d9557f748b2daa" [[package]] name = "siphasher" -version = "1.0.1" +version = "1.0.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "56199f7ddabf13fe5074ce809e7d3f42b42ae711800501b5b16ea82ad029c39d" +checksum = "b2aa850e253778c88a04c3d7323b043aeda9d3e30d5971937c1855769763678e" [[package]] name = "slab" -version = "0.4.11" +version = "0.4.12" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "7a2ae44ef20feb57a68b23d846850f861394c2e02dc425a50098ae8c90267589" +checksum = "0c790de23124f9ab44544d7ac05d60440adc586479ce501c1d6d7da3cd8c9cf5" [[package]] name = "small_ctor" @@ -6499,9 +6436,9 @@ dependencies = [ [[package]] name = "socket2" -version = "0.6.1" +version = "0.6.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "17129e116933cf371d018bb80ae557e889637989d8638274fb25622827b03881" +checksum = "86f4aa3ad99f2088c990dfa82d367e19cb29268ed67c574d10d0a4bfe71f07e0" dependencies = [ "libc", "windows-sys 0.60.2", @@ -6538,8 +6475,8 @@ dependencies = [ [[package]] name = "sqllogictest" -version = "0.29.0" -source = "git+https://github.com/risinglightdb/sqllogictest-rs.git#492c9e3e7b844682c705ec37b1d34d32808f6acd" +version = "0.29.1" +source = "git+https://github.com/risinglightdb/sqllogictest-rs.git#ebab8dae6d6655e86a4793c70246df6fbaa80ecb" dependencies = [ "async-trait", "educe", @@ -6579,7 +6516,7 @@ checksum = "da5fc6819faabb412da764b99d3b713bb55083c11e7e0c00144d386cd6a1939c" dependencies = [ "proc-macro2", "quote", - "syn 2.0.114", + "syn 2.0.116", ] [[package]] @@ -6614,7 +6551,7 @@ dependencies = [ "futures-util", "hashbrown 0.15.5", "hashlink", - "indexmap 2.12.1", + "indexmap 2.13.0", "log", "memchr", "once_cell", @@ -6641,7 +6578,7 @@ dependencies = [ "quote", "sqlx-core", "sqlx-macros-core", - "syn 2.0.114", + "syn 2.0.116", ] [[package]] @@ -6664,7 +6601,7 @@ dependencies = [ "sqlx-mysql", "sqlx-postgres", "sqlx-sqlite", - "syn 2.0.114", + "syn 2.0.116", "tokio", "url", ] @@ -6710,7 +6647,7 @@ dependencies = [ "thiserror", "tracing", "uuid", - "whoami", + "whoami 1.6.1", ] [[package]] @@ -6749,7 +6686,7 @@ dependencies = [ "thiserror", "tracing", "uuid", - "whoami", + "whoami 1.6.1", ] [[package]] @@ -6786,9 +6723,9 @@ checksum = "6ce2be8dc25455e1f91df71bfa12ad37d7af1092ae736f3a6cd0e37bc7810596" [[package]] name = "stacker" -version = "0.1.22" +version = "0.1.23" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "e1f8b29fb42aafcea4edeeb6b2f2d7ecd0d969c48b4cf0d2e64aafc471dd6e59" +checksum = "08d74a23609d509411d10e2176dc2a4346e3b4aea2e7b1869f19fdedbc71c013" dependencies = [ "cc", "cfg-if", @@ -6808,12 +6745,6 @@ dependencies = [ "unicode-properties", ] -[[package]] -name = "strsim" -version = "0.10.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "73473c0e59e6d5812c5dfe2a064a6444949f089e20eec9a2e5506596494e4623" - [[package]] name = "strsim" version = "0.11.1" @@ -6838,7 +6769,7 @@ dependencies = [ "heck", "proc-macro2", "quote", - "syn 2.0.114", + "syn 2.0.116", ] [[package]] @@ -6870,9 +6801,9 @@ dependencies = [ [[package]] name = "syn" -version = "2.0.114" +version = "2.0.116" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d4d107df263a3013ef9b1879b0df87d706ff80f65a86ea879bd9c31f9b307c2a" +checksum = "3df424c70518695237746f84cede799c9c58fcb37450d7b23716568cc8bc69cb" dependencies = [ "proc-macro2", "quote", @@ -6896,7 +6827,7 @@ checksum = "728a70f3dbaf5bab7f0c4b1ac8d7ae5ea60a4b5549c8a5914361c99147a709d2" dependencies = [ "proc-macro2", "quote", - "syn 2.0.114", + "syn 2.0.116", ] [[package]] @@ -6907,9 +6838,9 @@ checksum = "55937e1799185b12863d447f42597ed69d9928686b8d88a1df17376a097d8369" [[package]] name = "target-lexicon" -version = "0.13.4" +version = "0.13.5" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "b1dd07eb858a2067e2f3c7155d54e929265c264e6f37efe3ee7a8d1b5a1dd0ba" +checksum = "adb6935a6f5c20170eeceb1a3835a49e12e19d792f6dd344ccc76a985ca5a6ca" [[package]] name = "tdigests" @@ -6919,12 +6850,12 @@ checksum = "a8cc794f115de9eb67bb1bf4e8de08ac1b3d2f43bfdbec083636450da72a0986" [[package]] name = "tempfile" -version = "3.23.0" +version = "3.25.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "2d31c77bdf42a745371d260a26ca7163f1e0924b64afa0b688e61b5a9fa02f16" +checksum = "0136791f7c95b1f6dd99f9cc786b91bb81c3800b639b3478e561ddb7be95e5f1" dependencies = [ "fastrand", - "getrandom 0.3.4", + "getrandom 0.4.1", "once_cell", "rustix", "windows-sys 0.61.2", @@ -6948,7 +6879,7 @@ dependencies = [ "cfg-if", "proc-macro2", "quote", - "syn 2.0.114", + "syn 2.0.116", ] [[package]] @@ -6959,28 +6890,28 @@ checksum = "5c89e72a01ed4c579669add59014b9a524d609c0c88c6a585ce37485879f6ffb" dependencies = [ "proc-macro2", "quote", - "syn 2.0.114", + "syn 2.0.116", "test-case-core", ] [[package]] name = "thiserror" -version = "2.0.17" +version = "2.0.18" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "f63587ca0f12b72a0600bcba1d40081f830876000bb46dd2337a3051618f4fc8" +checksum = "4288b5bcbc7920c07a1149a35cf9590a2aa808e0bc1eafaade0b80947865fbc4" dependencies = [ "thiserror-impl", ] [[package]] name = "thiserror-impl" -version = "2.0.17" +version = "2.0.18" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "3ff15c8ecd7de3849db632e14d18d2571fa09dfc5ed93479bc4485c7a517c913" +checksum = "ebc4ee7f67670e9b64d05fa4253e753e016c6c95ff35b89b7941d6b856dec1d5" dependencies = [ "proc-macro2", "quote", - "syn 2.0.114", + "syn 2.0.116", ] [[package]] @@ -7005,30 +6936,30 @@ dependencies = [ [[package]] name = "time" -version = "0.3.44" +version = "0.3.47" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "91e7d9e3bb61134e77bde20dd4825b97c010155709965fedf0f49bb138e52a9d" +checksum = "743bd48c283afc0388f9b8827b976905fb217ad9e647fae3a379a9283c4def2c" dependencies = [ "deranged", "itoa", "num-conv", "powerfmt", - "serde", + "serde_core", "time-core", "time-macros", ] [[package]] name = "time-core" -version = "0.1.6" +version = "0.1.8" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "40868e7c1d2f0b8d73e4a8c7f0ff63af4f6d19be117e90bd73eb1d62cf831c6b" +checksum = "7694e1cfe791f8d31026952abf09c69ca6f6fa4e1a1229e18988f06a04a12dca" [[package]] name = "time-macros" -version = "0.2.24" +version = "0.2.27" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "30cfb0125f12d9c277f35663a0a33f8c30190f4e4574868a330595412d34ebf3" +checksum = "2e70e4c5a0e0a8a4823ad65dfe1a6930e4f4d756dcd9dd7939022b5e8c501215" dependencies = [ "num-conv", "time-core", @@ -7073,7 +7004,7 @@ dependencies = [ "include_dir", "instrumented-object-store", "log", - "lru 0.16.2", + "lru 0.16.3", "object_store", "opentelemetry", "opentelemetry-otlp", @@ -7082,7 +7013,7 @@ dependencies = [ "parquet-variant", "parquet-variant-compute", "parquet-variant-json", - "rand 0.9.2", + "rand 0.10.0", "regex", "scopeguard", "serde", @@ -7170,7 +7101,7 @@ dependencies = [ "parking_lot", "pin-project-lite", "signal-hook-registry", - "socket2 0.6.1", + "socket2 0.6.2", "tokio-macros", "windows-sys 0.61.2", ] @@ -7199,14 +7130,14 @@ checksum = "af407857209536a95c8e56f8231ef2c2e2aff839b22e07a1ffcbc617e9db9fa5" dependencies = [ "proc-macro2", "quote", - "syn 2.0.114", + "syn 2.0.116", ] [[package]] name = "tokio-postgres" -version = "0.7.15" +version = "0.7.16" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "2b40d66d9b2cfe04b628173409368e58247e8eddbbd3b0e6c6ba1d09f20f6c9e" +checksum = "dcea47c8f71744367793f16c2db1f11cb859d28f436bdb4ca9193eb1f787ee42" dependencies = [ "async-trait", "byteorder", @@ -7222,10 +7153,10 @@ dependencies = [ "postgres-protocol", "postgres-types", "rand 0.9.2", - "socket2 0.6.1", + "socket2 0.6.2", "tokio", "tokio-util", - "whoami", + "whoami 2.1.1", ] [[package]] @@ -7244,15 +7175,15 @@ version = "0.26.4" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "1729aa945f29d91ba541258c8df89027d5792d85a8841fb65e8bf0f4ede4ef61" dependencies = [ - "rustls 0.23.35", + "rustls 0.23.36", "tokio", ] [[package]] name = "tokio-stream" -version = "0.1.17" +version = "0.1.18" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "eca58d7bba4a75707817a2c44174253f9236b2d5fbd055602e9d5c07c139a047" +checksum = "32da49809aab5c3bc678af03902d4ccddea2a87d028d86392a4b1560c6906c70" dependencies = [ "futures-core", "pin-project-lite", @@ -7261,9 +7192,9 @@ dependencies = [ [[package]] name = "tokio-util" -version = "0.7.17" +version = "0.7.18" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "2efa149fe76073d6e8fd97ef4f4eca7b67f599660115591483572e406e165594" +checksum = "9ae9cec805b01e8fc3fd2fe289f89149a9b66dd16786abd8b19cfa7b48cb0098" dependencies = [ "bytes", "futures-core", @@ -7272,21 +7203,6 @@ dependencies = [ "tokio", ] -[[package]] -name = "toml" -version = "0.9.10+spec-1.1.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "0825052159284a1a8b4d6c0c86cbc801f2da5afd2b225fa548c72f2e74002f48" -dependencies = [ - "indexmap 2.12.1", - "serde_core", - "serde_spanned", - "toml_datetime", - "toml_parser", - "toml_writer", - "winnow", -] - [[package]] name = "toml_datetime" version = "0.7.5+spec-1.1.0" @@ -7302,7 +7218,7 @@ version = "0.23.10+spec-1.0.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "84c8b9f757e028cee9fa244aea147aab2a9ec09d5325a9b01e0a49730c2b5269" dependencies = [ - "indexmap 2.12.1", + "indexmap 2.13.0", "toml_datetime", "toml_parser", "winnow", @@ -7310,24 +7226,18 @@ dependencies = [ [[package]] name = "toml_parser" -version = "1.0.6+spec-1.1.0" +version = "1.0.9+spec-1.1.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a3198b4b0a8e11f09dd03e133c0280504d0801269e9afa46362ffde1cbeebf44" +checksum = "702d4415e08923e7e1ef96cd5727c0dfed80b4d2fa25db9647fe5eb6f7c5a4c4" dependencies = [ "winnow", ] -[[package]] -name = "toml_writer" -version = "1.0.6+spec-1.1.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "ab16f14aed21ee8bfd8ec22513f7287cd4a91aa92e44edfe2c17ddd004e92607" - [[package]] name = "tonic" -version = "0.14.2" +version = "0.14.4" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "eb7613188ce9f7df5bfe185db26c5814347d110db17920415cf2fbcad85e7203" +checksum = "7f32a6f80051a4111560201420c7885d0082ba9efe2ab61875c587bb6b18b9a0" dependencies = [ "async-trait", "base64", @@ -7351,9 +7261,9 @@ dependencies = [ [[package]] name = "tonic-prost" -version = "0.14.2" +version = "0.14.4" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "66bd50ad6ce1252d87ef024b3d64fe4c3cf54a86fb9ef4c631fdd0ded7aeaa67" +checksum = "9f86539c0089bfd09b1f8c0ab0239d80392af74c21bc9e0f15e1b4aca4c1647f" dependencies = [ "bytes", "prost", @@ -7362,13 +7272,13 @@ dependencies = [ [[package]] name = "tower" -version = "0.5.2" +version = "0.5.3" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d039ad9159c98b70ecfd540b2573b97f7f52c3e8d9f8ad57a24b916a536975f9" +checksum = "ebe5ef63511595f1344e2d5cfa636d973292adc0eec1f0ad45fae9f0851ab1d4" dependencies = [ "futures-core", "futures-util", - "indexmap 2.12.1", + "indexmap 2.13.0", "pin-project-lite", "slab", "sync_wrapper", @@ -7429,7 +7339,7 @@ checksum = "7490cfa5ec963746568740651ac6781f701c9c5ea257c58e057f3ba8cf69e8da" dependencies = [ "proc-macro2", "quote", - "syn 2.0.114", + "syn 2.0.116", ] [[package]] @@ -7477,16 +7387,13 @@ dependencies = [ [[package]] name = "tracing-opentelemetry" -version = "0.32.0" +version = "0.32.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "1e6e5658463dd88089aba75c7791e1d3120633b1bfde22478b28f625a9bb1b8e" +checksum = "1ac28f2d093c6c477eaa76b23525478f38de514fa9aeb1285738d4b97a9552fc" dependencies = [ "js-sys", "opentelemetry", - "opentelemetry_sdk", - "rustversion", "smallvec", - "thiserror", "tracing", "tracing-core", "tracing-log", @@ -7557,7 +7464,7 @@ checksum = "076a02dc54dd46795c2e9c8282ed40bcfb1e22747e955de9389a1de28190fb26" dependencies = [ "proc-macro2", "quote", - "syn 2.0.114", + "syn 2.0.116", ] [[package]] @@ -7574,9 +7481,9 @@ checksum = "5c1cb5db39152898a79168971543b1cb5020dff7fe43c8dc468b0885f5e29df5" [[package]] name = "unicode-ident" -version = "1.0.22" +version = "1.0.24" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "9312f7c4f6ff9069b165498234ce8be658059c6728633667c526e27dc2cf1df5" +checksum = "e6e4313cd5fcd3dad5cafa179702e2b244f760991f45397d14d4ebf38247da75" [[package]] name = "unicode-normalization" @@ -7611,6 +7518,12 @@ version = "0.2.2" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "b4ac048d71ede7ee76d585517add45da530660ef4390e49b098733c6e897f254" +[[package]] +name = "unicode-xid" +version = "0.2.6" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "ebc1c04c71510c7f702b52b7c350734c9ff1295c464a03335b00bb84fc54f853" + [[package]] name = "unindent" version = "0.2.4" @@ -7637,14 +7550,15 @@ checksum = "6d49784317cd0d1ee7ec5c716dd598ec5b4483ea832a2dced265471cc0f690ae" [[package]] name = "url" -version = "2.5.7" +version = "2.5.8" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "08bc136a29a3d1758e07a9cca267be308aeebf5cfd5a10f3f67ab2097683ef5b" +checksum = "ff67a8a4397373c3ef660812acab3268222035010ab8680ec4215f38ba3d0eed" dependencies = [ "form_urlencoded", "idna", "percent-encoding", "serde", + "serde_derive", ] [[package]] @@ -7667,11 +7581,11 @@ checksum = "06abde3611657adf66d383f00b093d7faecc7fa57071cce2578660c9f1010821" [[package]] name = "uuid" -version = "1.19.0" +version = "1.21.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "e2e054861b4bd027cd373e18e8d8d8e6548085000e41290d95ce0c373a654b4a" +checksum = "b672338555252d43fd2240c714dc444b8c6fb0a5c5335e65a07bba7742735ddb" dependencies = [ - "getrandom 0.3.4", + "getrandom 0.4.1", "js-sys", "rand 0.9.2", "serde_core", @@ -7705,7 +7619,7 @@ dependencies = [ "proc-macro-error2", "proc-macro2", "quote", - "syn 2.0.114", + "syn 2.0.116", ] [[package]] @@ -7776,11 +7690,29 @@ version = "0.11.1+wasi-snapshot-preview1" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "ccf3ec651a847eb01de73ccad15eb7d99f80485de043efb2f370cd654f4ea44b" +[[package]] +name = "wasi" +version = "0.14.7+wasi-0.2.4" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "883478de20367e224c0090af9cf5f9fa85bed63a95c1abf3afc5c083ebc06e8c" +dependencies = [ + "wasip2", +] + [[package]] name = "wasip2" -version = "1.0.1+wasi-0.2.4" +version = "1.0.2+wasi-0.2.9" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "0562428422c63773dad2c345a1882263bbf4d65cf3f42e90921f787ef5ad58e7" +checksum = "9517f9239f02c069db75e65f174b3da828fe5f5b945c4dd26bd25d89c03ebcf5" +dependencies = [ + "wit-bindgen", +] + +[[package]] +name = "wasip3" +version = "0.4.0+wasi-0.3.0-rc-2026-01-06" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "5428f8bf88ea5ddc08faddef2ac4a67e390b88186c703ce6dbd955e1c145aca5" dependencies = [ "wit-bindgen", ] @@ -7791,11 +7723,20 @@ version = "0.1.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "b8dad83b4f25e74f184f64c43b150b91efe7647395b42289f38e50566d82855b" +[[package]] +name = "wasite" +version = "1.0.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "66fe902b4a6b8028a753d5424909b764ccf79b7a209eac9bf97e59cda9f71a42" +dependencies = [ + "wasi 0.14.7+wasi-0.2.4", +] + [[package]] name = "wasm-bindgen" -version = "0.2.106" +version = "0.2.108" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "0d759f433fa64a2d763d1340820e46e111a7a5ab75f993d1852d70b03dbb80fd" +checksum = "64024a30ec1e37399cf85a7ffefebdb72205ca1c972291c51512360d90bd8566" dependencies = [ "cfg-if", "once_cell", @@ -7806,11 +7747,12 @@ dependencies = [ [[package]] name = "wasm-bindgen-futures" -version = "0.4.56" +version = "0.4.58" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "836d9622d604feee9e5de25ac10e3ea5f2d65b41eac0d9ce72eb5deae707ce7c" +checksum = "70a6e77fd0ae8029c9ea0063f87c46fde723e7d887703d74ad2616d792e51e6f" dependencies = [ "cfg-if", + "futures-util", "js-sys", "once_cell", "wasm-bindgen", @@ -7819,9 +7761,9 @@ dependencies = [ [[package]] name = "wasm-bindgen-macro" -version = "0.2.106" +version = "0.2.108" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "48cb0d2638f8baedbc542ed444afc0644a29166f1595371af4fecf8ce1e7eeb3" +checksum = "008b239d9c740232e71bd39e8ef6429d27097518b6b30bdf9086833bd5b6d608" dependencies = [ "quote", "wasm-bindgen-macro-support", @@ -7829,26 +7771,48 @@ dependencies = [ [[package]] name = "wasm-bindgen-macro-support" -version = "0.2.106" +version = "0.2.108" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "cefb59d5cd5f92d9dcf80e4683949f15ca4b511f4ac0a6e14d4e1ac60c6ecd40" +checksum = "5256bae2d58f54820e6490f9839c49780dff84c65aeab9e772f15d5f0e913a55" dependencies = [ "bumpalo", "proc-macro2", "quote", - "syn 2.0.114", + "syn 2.0.116", "wasm-bindgen-shared", ] [[package]] name = "wasm-bindgen-shared" -version = "0.2.106" +version = "0.2.108" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "cbc538057e648b67f72a982e708d485b2efa771e1ac05fec311f9f63e5800db4" +checksum = "1f01b580c9ac74c8d8f0c0e4afb04eeef2acf145458e52c03845ee9cd23e3d12" dependencies = [ "unicode-ident", ] +[[package]] +name = "wasm-encoder" +version = "0.244.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "990065f2fe63003fe337b932cfb5e3b80e0b4d0f5ff650e6985b1048f62c8319" +dependencies = [ + "leb128fmt", + "wasmparser", +] + +[[package]] +name = "wasm-metadata" +version = "0.244.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "bb0e353e6a2fbdc176932bbaab493762eb1255a7900fe0fea1a2f96c296cc909" +dependencies = [ + "anyhow", + "indexmap 2.13.0", + "wasm-encoder", + "wasmparser", +] + [[package]] name = "wasm-streams" version = "0.4.2" @@ -7862,11 +7826,23 @@ dependencies = [ "web-sys", ] +[[package]] +name = "wasmparser" +version = "0.244.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "47b807c72e1bac69382b3a6fb3dbe8ea4c0ed87ff5629b8685ae6b9a611028fe" +dependencies = [ + "bitflags", + "hashbrown 0.15.5", + "indexmap 2.13.0", + "semver", +] + [[package]] name = "web-sys" -version = "0.3.83" +version = "0.3.85" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "9b32828d774c412041098d182a8b38b16ea816958e07cf40eec2bc080ae137ac" +checksum = "312e32e551d92129218ea9a2452120f4aabc03529ef03e4d0d82fb2780608598" dependencies = [ "js-sys", "wasm-bindgen", @@ -7889,7 +7865,19 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "5d4a4db5077702ca3015d3d02d74974948aba2ad9e12ab7df718ee64ccd7e97d" dependencies = [ "libredox", - "wasite", + "wasite 0.1.0", +] + +[[package]] +name = "whoami" +version = "2.1.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "d6a5b12f9df4f978d2cfdb1bd3bac52433f44393342d7ee9c25f5a1c14c0f45d" +dependencies = [ + "libc", + "libredox", + "objc2-system-configuration", + "wasite 1.0.2", "web-sys", ] @@ -7945,7 +7933,7 @@ checksum = "053e2e040ab57b9dc951b72c264860db7eb3b0200ba345b4e4c3b14f67855ddf" dependencies = [ "proc-macro2", "quote", - "syn 2.0.114", + "syn 2.0.116", ] [[package]] @@ -7956,7 +7944,7 @@ checksum = "3f316c4a2570ba26bbec722032c4099d8c8bc095efccdc15688708623367e358" dependencies = [ "proc-macro2", "quote", - "syn 2.0.114", + "syn 2.0.116", ] [[package]] @@ -8225,9 +8213,91 @@ dependencies = [ [[package]] name = "wit-bindgen" -version = "0.46.0" +version = "0.51.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "d7249219f66ced02969388cf2bb044a09756a083d0fab1e566056b04d9fbcaa5" +dependencies = [ + "wit-bindgen-rust-macro", +] + +[[package]] +name = "wit-bindgen-core" +version = "0.51.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "ea61de684c3ea68cb082b7a88508a8b27fcc8b797d738bfc99a82facf1d752dc" +dependencies = [ + "anyhow", + "heck", + "wit-parser", +] + +[[package]] +name = "wit-bindgen-rust" +version = "0.51.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "b7c566e0f4b284dd6561c786d9cb0142da491f46a9fbed79ea69cdad5db17f21" +dependencies = [ + "anyhow", + "heck", + "indexmap 2.13.0", + "prettyplease", + "syn 2.0.116", + "wasm-metadata", + "wit-bindgen-core", + "wit-component", +] + +[[package]] +name = "wit-bindgen-rust-macro" +version = "0.51.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "f17a85883d4e6d00e8a97c586de764dabcc06133f7f1d55dce5cdc070ad7fe59" +checksum = "0c0f9bfd77e6a48eccf51359e3ae77140a7f50b1e2ebfe62422d8afdaffab17a" +dependencies = [ + "anyhow", + "prettyplease", + "proc-macro2", + "quote", + "syn 2.0.116", + "wit-bindgen-core", + "wit-bindgen-rust", +] + +[[package]] +name = "wit-component" +version = "0.244.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "9d66ea20e9553b30172b5e831994e35fbde2d165325bec84fc43dbf6f4eb9cb2" +dependencies = [ + "anyhow", + "bitflags", + "indexmap 2.13.0", + "log", + "serde", + "serde_derive", + "serde_json", + "wasm-encoder", + "wasm-metadata", + "wasmparser", + "wit-parser", +] + +[[package]] +name = "wit-parser" +version = "0.244.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "ecc8ac4bc1dc3381b7f59c34f00b67e18f910c2c0f50015669dde7def656a736" +dependencies = [ + "anyhow", + "id-arena", + "indexmap 2.13.0", + "log", + "semver", + "serde", + "serde_derive", + "serde_json", + "unicode-xid", + "wasmparser", +] [[package]] name = "writeable" @@ -8288,34 +8358,34 @@ checksum = "b659052874eb698efe5b9e8cf382204678a0086ebf46982b79d6ca3182927e5d" dependencies = [ "proc-macro2", "quote", - "syn 2.0.114", + "syn 2.0.116", "synstructure", ] [[package]] name = "z85" -version = "3.0.6" +version = "3.0.7" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "9b3a41ce106832b4da1c065baa4c31cf640cf965fa1483816402b7f6b96f0a64" +checksum = "c6e61e59a957b7ccee15d2049f86e8bfd6f66968fcd88f018950662d9b86e675" [[package]] name = "zerocopy" -version = "0.8.31" +version = "0.8.39" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "fd74ec98b9250adb3ca554bdde269adf631549f51d8a8f8f0a10b50f1cb298c3" +checksum = "db6d35d663eadb6c932438e763b262fe1a70987f9ae936e60158176d710cae4a" dependencies = [ "zerocopy-derive", ] [[package]] name = "zerocopy-derive" -version = "0.8.31" +version = "0.8.39" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d8a8d209fdf45cf5138cbb5a506f6b52522a25afccc534d1475dad8e31105c6a" +checksum = "4122cd3169e94605190e77839c9a40d40ed048d305bfdc146e7df40ab0f3e517" dependencies = [ "proc-macro2", "quote", - "syn 2.0.114", + "syn 2.0.116", ] [[package]] @@ -8335,7 +8405,7 @@ checksum = "d71e5d6e06ab090c67b5e44993ec16b72dcbaabc526db883a360057678b48502" dependencies = [ "proc-macro2", "quote", - "syn 2.0.114", + "syn 2.0.116", "synstructure", ] @@ -8350,13 +8420,13 @@ dependencies = [ [[package]] name = "zeroize_derive" -version = "1.4.2" +version = "1.4.3" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "ce36e65b0d2999d2aafac989fb249189a141aee1f53c612c1f37d72631959f69" +checksum = "85a5b4158499876c763cb03bc4e49185d3cccbabb15b33c627f7884f43db852e" dependencies = [ "proc-macro2", "quote", - "syn 2.0.114", + "syn 2.0.116", ] [[package]] @@ -8389,14 +8459,20 @@ checksum = "eadce39539ca5cb3985590102671f2567e659fca9666581ad3411d59207951f3" dependencies = [ "proc-macro2", "quote", - "syn 2.0.114", + "syn 2.0.116", ] [[package]] name = "zlib-rs" -version = "0.5.5" +version = "0.6.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "a7948af682ccbc3342b6e9420e8c51c1fe5d7bf7756002b4a3c6cabfe96a7e3c" + +[[package]] +name = "zmij" +version = "1.0.21" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "40990edd51aae2c2b6907af74ffb635029d5788228222c4bb811e9351c0caad3" +checksum = "b8848ee67ecc8aedbaf3e4122217aff892639231befc6a1b58d29fff4c2cabaa" [[package]] name = "zstd" diff --git a/Cargo.toml b/Cargo.toml index 9082f14..e448730 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -22,7 +22,7 @@ color-eyre = "0.6.5" arrow-schema = "57.1.0" regex = "1.11.1" # Using fork with VariantType support until upstream merges the feature -deltalake = { git = "https://github.com/tonyalaribe/delta-rs.git", rev = "ba769136c5dd9b84a7335ea67e42b67884bfcce3", features = [ +deltalake = { git = "https://github.com/tonyalaribe/delta-rs.git", rev = "c4d506da", features = [ "datafusion", "s3", ] } @@ -42,7 +42,7 @@ sqlx = { version = "0.8", features = [ futures = { version = "0.3.31", features = ["alloc"] } bytes = "1.4" tokio-rustls = "0.26.1" -datafusion-postgres = "0.14.0" +datafusion-postgres = "0.15.0" datafusion-functions-json = "0.52.0" anyhow = "1.0.100" tokio-util = "0.7.17" @@ -64,7 +64,7 @@ aws-sdk-dynamodb = "1.3.0" url = "2.5.4" tokio-cron-scheduler = "0.15" object_store = "0.12.4" -foyer = { version = "0.21.1", features = ["serde"] } +foyer = { version = "0.22.3", features = ["serde"] } ahash = "0.8" lru = "0.16.1" serde_bytes = "0.11.19" @@ -89,10 +89,10 @@ serial_test = "3.2.0" datafusion-common = "52.1.0" tokio-postgres = { version = "0.7.10", features = ["with-chrono-0_4"] } scopeguard = "1.2.0" -rand = "0.9.2" +rand = "0.10.0" tempfile = "3" test-case = "3.3" -criterion = { version = "0.5", features = ["html_reports", "async_tokio"] } +criterion = { version = "0.8", features = ["html_reports", "async_tokio"] } [[bench]] name = "core_benchmarks" diff --git a/src/database.rs b/src/database.rs index 760db8d..95c1549 100644 --- a/src/database.rs +++ b/src/database.rs @@ -98,9 +98,8 @@ pub fn convert_variant_columns(batch: RecordBatch, target_schema: &SchemaRef) -> if !is_variant_type(target_field.data_type()) { continue; } - // Skip columns beyond batch length - this is normal for INSERT with fewer columns than table schema - // (e.g., columns with defaults or nullable columns omitted from INSERT) if idx >= columns.len() { + debug!("Column index {} exceeds batch length {}, skipping", idx, columns.len()); continue; } @@ -460,16 +459,17 @@ impl Database { /// Perform a Delta table UPDATE operation pub async fn perform_delta_update( &self, table_name: &str, project_id: &str, predicate: Option, - assignments: Vec<(String, datafusion::logical_expr::Expr)>, + assignments: Vec<(String, datafusion::logical_expr::Expr)>, session: Arc, ) -> Result { - crate::dml::perform_delta_update(self, table_name, project_id, predicate, assignments).await + crate::dml::perform_delta_update(self, table_name, project_id, predicate, assignments, session).await } /// Perform a Delta table DELETE operation pub async fn perform_delta_delete( &self, table_name: &str, project_id: &str, predicate: Option, + session: Arc, ) -> Result { - crate::dml::perform_delta_delete(self, table_name, project_id, predicate).await + crate::dml::perform_delta_delete(self, table_name, project_id, predicate, session).await } /// Build storage options with consistent configuration including DynamoDB locking if enabled @@ -2132,16 +2132,7 @@ impl ProjectRoutingTable { async fn scan_delta_table( &self, table: &DeltaTable, state: &dyn Session, projection: Option<&Vec>, filters: &[Expr], limit: Option, ) -> DFResult> { - // Register the object store with DataFusion's runtime so table_provider().scan() can access it - let log_store = table.log_store(); - let root_store = log_store.root_object_store(None); - let bucket_url = { - let table_url = table.table_url(); - let scheme = table_url.scheme(); - let bucket = table_url.host_str().unwrap_or(""); - Url::parse(&format!("{}://{}/", scheme, bucket)).expect("valid bucket URL") - }; - state.runtime_env().register_object_store(&bucket_url, root_store); + table.update_datafusion_session(state).map_err(|e| DataFusionError::External(Box::new(e)))?; let provider = table.table_provider().await.map_err(|e| DataFusionError::External(Box::new(e)))?; diff --git a/src/dml.rs b/src/dml.rs index d7c7ff5..11858c0 100644 --- a/src/dml.rs +++ b/src/dml.rs @@ -7,10 +7,11 @@ use datafusion::{ array::RecordBatch, datatypes::{DataType, Field, Schema}, }, + catalog::Session, common::{Column, Result}, error::DataFusionError, execution::{ - SendableRecordBatchStream, TaskContext, + SendableRecordBatchStream, SessionStateBuilder, TaskContext, context::{QueryPlanner, SessionState}, }, logical_expr::{BinaryExpr, Expr, LogicalPlan, Operator, WriteOp}, @@ -23,6 +24,19 @@ use tracing::{Instrument, error, info, instrument}; use crate::buffered_write_layer::BufferedWriteLayer; use crate::database::Database; +/// Build a clean SessionState with config + runtime from the given session but with +/// delta-rs's DeltaPlanner instead of our custom DmlQueryPlanner. +fn delta_session_from(session: &SessionState) -> Arc { + Arc::new( + SessionStateBuilder::new() + .with_config(session.config().clone()) + .with_runtime_env(session.runtime_env().clone()) + .with_default_features() + .with_query_planner(deltalake::delta_datafusion::planner::DeltaPlanner::new()) + .build(), + ) +} + /// Type alias for DML information extracted from logical plan type DmlInfo = (String, String, Option, Option>); @@ -79,12 +93,13 @@ impl QueryPlanner for DmlQueryPlanner { span.record("table.name", table_name.as_str()); span.record("project_id", project_id.as_str()); + let session = delta_session_from(session_state); let exec = if is_update { - DmlExec::update(table_name, project_id, input_exec, self.database.clone()) + DmlExec::update(table_name, project_id, input_exec, self.database.clone(), session) .predicate(predicate) .assignments(assignments.unwrap_or_default()) } else { - DmlExec::delete(table_name, project_id, input_exec, self.database.clone()).predicate(predicate) + DmlExec::delete(table_name, project_id, input_exec, self.database.clone(), session).predicate(predicate) }; Ok(Arc::new(exec.buffered_layer(self.buffered_layer.clone()))) } @@ -193,6 +208,8 @@ pub struct DmlExec { input: Arc, database: Arc, buffered_layer: Option>, + session: Arc, + properties: PlanProperties, } impl std::fmt::Debug for DmlExec { @@ -223,7 +240,13 @@ impl DmlOperation { } impl DmlExec { - fn new(op_type: DmlOperation, table_name: String, project_id: String, input: Arc, database: Arc) -> Self { + fn new(op_type: DmlOperation, table_name: String, project_id: String, input: Arc, database: Arc, session: Arc) -> Self { + let properties = PlanProperties::new( + datafusion::physical_expr::EquivalenceProperties::new(input.schema()), + datafusion::physical_plan::Partitioning::UnknownPartitioning(1), + input.properties().emission_type, + input.properties().boundedness, + ); Self { op_type, table_name, @@ -233,15 +256,17 @@ impl DmlExec { input, database, buffered_layer: None, + session, + properties, } } - pub fn update(table_name: String, project_id: String, input: Arc, database: Arc) -> Self { - Self::new(DmlOperation::Update, table_name, project_id, input, database) + pub fn update(table_name: String, project_id: String, input: Arc, database: Arc, session: Arc) -> Self { + Self::new(DmlOperation::Update, table_name, project_id, input, database, session) } - pub fn delete(table_name: String, project_id: String, input: Arc, database: Arc) -> Self { - Self::new(DmlOperation::Delete, table_name, project_id, input, database) + pub fn delete(table_name: String, project_id: String, input: Arc, database: Arc, session: Arc) -> Self { + Self::new(DmlOperation::Delete, table_name, project_id, input, database, session) } pub fn predicate(mut self, predicate: Option) -> Self { @@ -294,7 +319,7 @@ impl ExecutionPlan for DmlExec { } fn properties(&self) -> &PlanProperties { - self.input.properties() + &self.properties } fn required_input_distribution(&self) -> Vec { @@ -327,13 +352,14 @@ impl ExecutionPlan for DmlExec { let predicate = self.predicate.clone(); let database = self.database.clone(); let buffered_layer = self.buffered_layer.clone(); + let session = self.session.clone(); let future = async move { let result = match op_type { DmlOperation::Update => { - perform_update_with_buffer(&database, buffered_layer.as_ref(), &table_name, &project_id, predicate, assignments, &span).await + perform_update_with_buffer(&database, buffered_layer.as_ref(), &table_name, &project_id, predicate, assignments, session, &span).await } - DmlOperation::Delete => perform_delete_with_buffer(&database, buffered_layer.as_ref(), &table_name, &project_id, predicate, &span).await, + DmlOperation::Delete => perform_delete_with_buffer(&database, buffered_layer.as_ref(), &table_name, &project_id, predicate, session, &span).await, }; if let Ok(rows) = &result { @@ -394,7 +420,7 @@ impl<'a> DmlContext<'a> { async fn perform_update_with_buffer( database: &Database, buffered_layer: Option<&Arc>, table_name: &str, project_id: &str, predicate: Option, - assignments: Vec<(String, Expr)>, span: &tracing::Span, + assignments: Vec<(String, Expr)>, session: Arc, span: &tracing::Span, ) -> Result { let assignments_clone = assignments.clone(); let update_span = tracing::trace_span!(parent: span, "delta.update"); @@ -407,13 +433,14 @@ async fn perform_update_with_buffer( } .execute( |layer, pred| layer.update(project_id, table_name, pred, &assignments_clone), - perform_delta_update(database, table_name, project_id, predicate, assignments).instrument(update_span), + perform_delta_update(database, table_name, project_id, predicate, assignments, session).instrument(update_span), ) .await } async fn perform_delete_with_buffer( - database: &Database, buffered_layer: Option<&Arc>, table_name: &str, project_id: &str, predicate: Option, span: &tracing::Span, + database: &Database, buffered_layer: Option<&Arc>, table_name: &str, project_id: &str, predicate: Option, + session: Arc, span: &tracing::Span, ) -> Result { let delete_span = tracing::trace_span!(parent: span, "delta.delete"); DmlContext { @@ -425,7 +452,7 @@ async fn perform_delete_with_buffer( } .execute( |layer, pred| layer.delete(project_id, table_name, pred), - perform_delta_delete(database, table_name, project_id, predicate).instrument(delete_span), + perform_delta_delete(database, table_name, project_id, predicate, session).instrument(delete_span), ) .await } @@ -444,13 +471,13 @@ async fn perform_delete_with_buffer( )] pub async fn perform_delta_update( database: &Database, table_name: &str, project_id: &str, predicate: Option, assignments: Vec<(String, Expr)>, + session: Arc, ) -> Result { info!("Performing Delta UPDATE on table {} for project {}", table_name, project_id); let span = tracing::Span::current(); let result = perform_delta_operation(database, table_name, project_id, |delta_table| async move { - // delta-rs handles Utf8View automatically with schema_force_view_types=true (default in DF52+) - let mut builder = delta_table.update(); + let mut builder = delta_table.update().with_session_state(session); if let Some(pred) = predicate { builder = builder.with_predicate(convert_expr_to_delta(&pred)?); @@ -485,13 +512,12 @@ pub async fn perform_delta_update( rows.deleted = Empty, ) )] -pub async fn perform_delta_delete(database: &Database, table_name: &str, project_id: &str, predicate: Option) -> Result { +pub async fn perform_delta_delete(database: &Database, table_name: &str, project_id: &str, predicate: Option, session: Arc) -> Result { info!("Performing Delta DELETE on table {} for project {}", table_name, project_id); let span = tracing::Span::current(); let result = perform_delta_operation(database, table_name, project_id, |delta_table| async move { - // delta-rs handles Utf8View automatically with schema_force_view_types=true (default in DF52+) - let mut builder = delta_table.delete(); + let mut builder = delta_table.delete().with_session_state(session); if let Some(pred) = predicate { builder = builder.with_predicate(convert_expr_to_delta(&pred)?); @@ -523,7 +549,9 @@ where .await .map_err(|e| DataFusionError::Execution(format!("Table not found: {} for project {}: {}", table_name, project_id, e)))?; - let delta_table = table_lock.write().await; + let mut delta_table = table_lock.write().await; + // Refresh snapshot so DML sees the latest committed version + delta_table.update_state().await.map_err(|e| DataFusionError::Execution(format!("Failed to refresh table state: {}", e)))?; let (new_table, rows_affected) = operation(delta_table.clone()).await?; drop(delta_table); diff --git a/src/object_store_cache.rs b/src/object_store_cache.rs index 1506284..a5cc66f 100644 --- a/src/object_store_cache.rs +++ b/src/object_store_cache.rs @@ -14,7 +14,7 @@ use std::time::{Duration, SystemTime, UNIX_EPOCH}; use tracing::field::Empty; use tracing::{Instrument, debug, info, instrument}; -use foyer::{BlockEngineBuilder, DeviceBuilder, FsDeviceBuilder, HybridCache, HybridCacheBuilder, HybridCachePolicy, IoEngineBuilder, PsyncIoEngineBuilder}; +use foyer::{BlockEngineConfig, DeviceBuilder, FsDeviceBuilder, HybridCache, HybridCacheBuilder, HybridCachePolicy, PsyncIoEngineConfig}; use serde::{Deserialize, Serialize}; use tokio::sync::{Mutex, RwLock}; use tokio::task::JoinSet; @@ -243,9 +243,9 @@ impl SharedFoyerCache { .with_shards(config.shards) .with_weighter(|_key: &String, value: &CacheValue| value.data.len()) .storage() - .with_io_engine(PsyncIoEngineBuilder::new().build().await?) + .with_io_engine_config(PsyncIoEngineConfig::new()) .with_engine_config( - BlockEngineBuilder::new(FsDeviceBuilder::new(&config.cache_dir).with_capacity(config.disk_size_bytes).build()?) + BlockEngineConfig::new(FsDeviceBuilder::new(&config.cache_dir).with_capacity(config.disk_size_bytes).build()?) .with_block_size(config.file_size_bytes), ) .build() @@ -257,9 +257,9 @@ impl SharedFoyerCache { .with_shards(config.metadata_shards) .with_weighter(|_key: &String, value: &CacheValue| value.data.len()) .storage() - .with_io_engine(PsyncIoEngineBuilder::new().build().await?) + .with_io_engine_config(PsyncIoEngineConfig::new()) .with_engine_config( - BlockEngineBuilder::new(FsDeviceBuilder::new(&metadata_cache_dir).with_capacity(config.metadata_disk_size_bytes).build()?) + BlockEngineConfig::new(FsDeviceBuilder::new(&metadata_cache_dir).with_capacity(config.metadata_disk_size_bytes).build()?) .with_block_size(config.file_size_bytes), ) .build() diff --git a/tests/connection_pressure_test.rs b/tests/connection_pressure_test.rs index 021804a..bca49bf 100644 --- a/tests/connection_pressure_test.rs +++ b/tests/connection_pressure_test.rs @@ -7,7 +7,7 @@ mod connection_pressure { use anyhow::Result; use datafusion_postgres::ServerOptions; use dotenv::dotenv; - use rand::Rng; + use rand::{Rng, RngExt}; use serial_test::serial; use std::sync::Arc; use std::sync::atomic::{AtomicUsize, Ordering}; diff --git a/tests/integration_test.rs b/tests/integration_test.rs index 52b229a..163fcf2 100644 --- a/tests/integration_test.rs +++ b/tests/integration_test.rs @@ -2,7 +2,7 @@ mod integration { use anyhow::Result; use datafusion_postgres::ServerOptions; - use rand::Rng; + use rand::{Rng, RngExt}; use serial_test::serial; use std::path::PathBuf; use std::sync::Arc;