From 7b25cc66f4f6eac8bd243506decfede4b67f5177 Mon Sep 17 00:00:00 2001 From: CGodiksen <36046286+CGodiksen@users.noreply.github.com> Date: Sun, 14 Dec 2025 13:45:08 +0100 Subject: [PATCH 001/116] Added basic structure for write-ahead-log --- crates/modelardb_storage/src/lib.rs | 1 + .../modelardb_storage/src/write_ahead_log.rs | 32 +++++++++++++++++++ 2 files changed, 33 insertions(+) create mode 100644 crates/modelardb_storage/src/write_ahead_log.rs diff --git a/crates/modelardb_storage/src/lib.rs b/crates/modelardb_storage/src/lib.rs index ad1cd3e7..98d95336 100644 --- a/crates/modelardb_storage/src/lib.rs +++ b/crates/modelardb_storage/src/lib.rs @@ -21,6 +21,7 @@ pub mod error; mod optimizer; pub mod parser; mod query; +mod write_ahead_log; use std::result::Result as StdResult; use std::sync::Arc; diff --git a/crates/modelardb_storage/src/write_ahead_log.rs b/crates/modelardb_storage/src/write_ahead_log.rs new file mode 100644 index 00000000..f1e5c381 --- /dev/null +++ b/crates/modelardb_storage/src/write_ahead_log.rs @@ -0,0 +1,32 @@ +/* Copyright 2025 The ModelarDB Contributors + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +//! Implementation of types that provide a write-ahead-log for ModelarDB that can be used to +//! efficiently persist data and operations on disk to avoid data loss and enable crash recovery. + +use std::collections::HashMap; +use std::fs::File; + +/// Write-ahead-log that logs data on a per-table level and operations separately. +struct WriteAheadLog { + table_logs: HashMap, + operation_log: WriteAheadLogFile, +} + +/// Wrapper around a [`File`] that enforces that [`sync_all()`](File::sync_all) is called +/// immediately after writing to ensure that all data is on disk before returning. +struct WriteAheadLogFile { + file: File, +} From 2401f864eac853f36363cfd5cff5682228ebf6f3 Mon Sep 17 00:00:00 2001 From: CGodiksen <36046286+CGodiksen@users.noreply.github.com> Date: Sun, 14 Dec 2025 17:47:57 +0100 Subject: [PATCH 002/116] Add try_new method to WriteAheadLogFile --- crates/modelardb_storage/src/write_ahead_log.rs | 14 ++++++++++++-- 1 file changed, 12 insertions(+), 2 deletions(-) diff --git a/crates/modelardb_storage/src/write_ahead_log.rs b/crates/modelardb_storage/src/write_ahead_log.rs index f1e5c381..208620fe 100644 --- a/crates/modelardb_storage/src/write_ahead_log.rs +++ b/crates/modelardb_storage/src/write_ahead_log.rs @@ -26,7 +26,17 @@ struct WriteAheadLog { } /// Wrapper around a [`File`] that enforces that [`sync_all()`](File::sync_all) is called -/// immediately after writing to ensure that all data is on disk before returning. -struct WriteAheadLogFile { +/// immediately after writing to ensure that all data is on disk before returning. Note that +/// an exclusive lock is held on the file while it is being written to. +pub struct WriteAheadLogFile { file: File, } + +impl WriteAheadLogFile { + /// Create a new [`WriteAheadLogFile`] that writes to the file at `file_path`. If the file could + /// not be created, return [`ModelarDbStorageError`](crate::error::ModelarDbStorageError). + pub fn try_new(file_path: PathBuf) -> Result { + let file = File::create(file_path)?; + Ok(Self { file }) + } +} From f17390e1fa0f240473bcf6a8264b4610813f8d4e Mon Sep 17 00:00:00 2001 From: CGodiksen <36046286+CGodiksen@users.noreply.github.com> Date: Sun, 14 Dec 2025 17:50:31 +0100 Subject: [PATCH 003/116] Add try_new method to WriteAheadLog --- .../modelardb_storage/src/write_ahead_log.rs | 32 +++++++++++++++++-- 1 file changed, 29 insertions(+), 3 deletions(-) diff --git a/crates/modelardb_storage/src/write_ahead_log.rs b/crates/modelardb_storage/src/write_ahead_log.rs index 208620fe..cd904926 100644 --- a/crates/modelardb_storage/src/write_ahead_log.rs +++ b/crates/modelardb_storage/src/write_ahead_log.rs @@ -13,18 +13,44 @@ * limitations under the License. */ -//! Implementation of types that provide a write-ahead-log for ModelarDB that can be used to +//! Implementation of types that provide a write-ahead log for ModelarDB that can be used to //! efficiently persist data and operations on disk to avoid data loss and enable crash recovery. use std::collections::HashMap; use std::fs::File; +use std::path::PathBuf; -/// Write-ahead-log that logs data on a per-table level and operations separately. -struct WriteAheadLog { +use crate::WRITE_AHEAD_LOG_FOLDER; +use crate::data_folder::DataFolder; +use crate::error::Result; + + +/// Write-ahead log that logs data on a per-table level and operations separately. +pub struct WriteAheadLog { + log_folder_path: PathBuf, table_logs: HashMap, operation_log: WriteAheadLogFile, } +impl WriteAheadLog { + /// Create a new [`WriteAheadLog`] that stores the log in the root of `local_data_folder` in + /// the [`WRITE_AHEAD_LOG_FOLDER`] folder. If the folder does not exist, it is created. If the + /// log could not be created, return [`ModelarDbStorageError`](crate::error::ModelarDbStorageError). + pub fn try_new(local_data_folder: DataFolder) -> Result { + // Create the folder for the write-ahead log if it does not exist. + let location = local_data_folder.location(); + let log_folder_path = PathBuf::from(format!("{location}/{WRITE_AHEAD_LOG_FOLDER}")); + + std::fs::create_dir(log_folder_path.clone())?; + + Ok(Self { + log_folder_path: log_folder_path.clone(), + table_logs: HashMap::new(), + operation_log: WriteAheadLogFile::try_new(log_folder_path.join("operations.wal"))?, + }) + } +} + /// Wrapper around a [`File`] that enforces that [`sync_all()`](File::sync_all) is called /// immediately after writing to ensure that all data is on disk before returning. Note that /// an exclusive lock is held on the file while it is being written to. From b7e688a23d58ca7c24f16d832848caf4b0e4f40f Mon Sep 17 00:00:00 2001 From: CGodiksen <36046286+CGodiksen@users.noreply.github.com> Date: Sun, 14 Dec 2025 18:02:32 +0100 Subject: [PATCH 004/116] Add method to get location of DataFolder --- crates/modelardb_storage/src/data_folder/mod.rs | 8 ++++++++ 1 file changed, 8 insertions(+) diff --git a/crates/modelardb_storage/src/data_folder/mod.rs b/crates/modelardb_storage/src/data_folder/mod.rs index 2d4ec8cd..207848c5 100644 --- a/crates/modelardb_storage/src/data_folder/mod.rs +++ b/crates/modelardb_storage/src/data_folder/mod.rs @@ -373,6 +373,14 @@ impl DataFolder { &self.session_context } + /// Return the location of the Delta Lake. This is `memory:///modelardb` if the Delta Lake is + /// in memory, the local path if the Delta Lake is stored on disk, `az://container-name` + /// if the Delta Lake is stored in Azure Blob Storage, or `s3://bucket-name` if the Delta Lake + /// is stored in Amazon S3. + pub fn location(&self) -> &str { + &self.location + } + /// Return an [`ObjectStore`] to access the root of the Delta Lake. pub fn object_store(&self) -> Arc { self.object_store.clone() From ab01fb649e05f94990bc4f816435ef8d9c1a9d15 Mon Sep 17 00:00:00 2001 From: CGodiksen <36046286+CGodiksen@users.noreply.github.com> Date: Sun, 14 Dec 2025 18:03:37 +0100 Subject: [PATCH 005/116] Add write-ahead log to context --- crates/modelardb_server/src/context.rs | 6 ++++++ crates/modelardb_storage/src/lib.rs | 5 ++++- crates/modelardb_storage/src/write_ahead_log.rs | 6 +++--- 3 files changed, 13 insertions(+), 4 deletions(-) diff --git a/crates/modelardb_server/src/context.rs b/crates/modelardb_server/src/context.rs index f21e7439..f1a013bb 100644 --- a/crates/modelardb_server/src/context.rs +++ b/crates/modelardb_server/src/context.rs @@ -20,6 +20,7 @@ use std::sync::Arc; use datafusion::arrow::datatypes::Schema; use datafusion::catalog::{SchemaProvider, TableProvider}; +use modelardb_storage::write_ahead_log::WriteAheadLog; use modelardb_types::types::TimeSeriesTableMetadata; use tokio::sync::RwLock; use tracing::info; @@ -38,6 +39,8 @@ pub struct Context { pub configuration_manager: Arc>, /// Manages all uncompressed and compressed data in the system. pub storage_engine: Arc>, + /// Write-ahead log for persisting data and operations. + pub write_ahead_log: WriteAheadLog, } impl Context { @@ -54,10 +57,13 @@ impl Context { StorageEngine::try_new(data_folders.clone(), &configuration_manager).await?, )); + let write_ahead_log = WriteAheadLog::try_new(&data_folders.local_data_folder)?; + Ok(Context { data_folders, configuration_manager, storage_engine, + write_ahead_log, }) } diff --git a/crates/modelardb_storage/src/lib.rs b/crates/modelardb_storage/src/lib.rs index 98d95336..a7e5104d 100644 --- a/crates/modelardb_storage/src/lib.rs +++ b/crates/modelardb_storage/src/lib.rs @@ -21,7 +21,7 @@ pub mod error; mod optimizer; pub mod parser; mod query; -mod write_ahead_log; +pub mod write_ahead_log; use std::result::Result as StdResult; use std::sync::Arc; @@ -63,6 +63,9 @@ const TABLE_FOLDER: &str = "tables"; /// The folder storing metadata in the data folders. const METADATA_FOLDER: &str = "metadata"; +/// The folder storing the write-ahead log in the data folders. +const WRITE_AHEAD_LOG_FOLDER: &str = "_modelardb_log"; + /// Create a new [`SessionContext`] for interacting with Apache DataFusion. The [`SessionContext`] /// is constructed with the default configuration, default resource managers, and additional /// optimizer rules that rewrite simple aggregate queries to be executed directly on the segments diff --git a/crates/modelardb_storage/src/write_ahead_log.rs b/crates/modelardb_storage/src/write_ahead_log.rs index cd904926..6813e91e 100644 --- a/crates/modelardb_storage/src/write_ahead_log.rs +++ b/crates/modelardb_storage/src/write_ahead_log.rs @@ -36,7 +36,7 @@ impl WriteAheadLog { /// Create a new [`WriteAheadLog`] that stores the log in the root of `local_data_folder` in /// the [`WRITE_AHEAD_LOG_FOLDER`] folder. If the folder does not exist, it is created. If the /// log could not be created, return [`ModelarDbStorageError`](crate::error::ModelarDbStorageError). - pub fn try_new(local_data_folder: DataFolder) -> Result { + pub fn try_new(local_data_folder: &DataFolder) -> Result { // Create the folder for the write-ahead log if it does not exist. let location = local_data_folder.location(); let log_folder_path = PathBuf::from(format!("{location}/{WRITE_AHEAD_LOG_FOLDER}")); @@ -54,14 +54,14 @@ impl WriteAheadLog { /// Wrapper around a [`File`] that enforces that [`sync_all()`](File::sync_all) is called /// immediately after writing to ensure that all data is on disk before returning. Note that /// an exclusive lock is held on the file while it is being written to. -pub struct WriteAheadLogFile { +struct WriteAheadLogFile { file: File, } impl WriteAheadLogFile { /// Create a new [`WriteAheadLogFile`] that writes to the file at `file_path`. If the file could /// not be created, return [`ModelarDbStorageError`](crate::error::ModelarDbStorageError). - pub fn try_new(file_path: PathBuf) -> Result { + fn try_new(file_path: PathBuf) -> Result { let file = File::create(file_path)?; Ok(Self { file }) } From fa57835c76b875514983d5ecaa6bb971ed59bc88 Mon Sep 17 00:00:00 2001 From: CGodiksen <36046286+CGodiksen@users.noreply.github.com> Date: Sun, 14 Dec 2025 18:19:40 +0100 Subject: [PATCH 006/116] Fix bug causing an error if the log folder already exists and ensuring we append --- crates/modelardb_storage/src/write_ahead_log.rs | 15 ++++++++++----- 1 file changed, 10 insertions(+), 5 deletions(-) diff --git a/crates/modelardb_storage/src/write_ahead_log.rs b/crates/modelardb_storage/src/write_ahead_log.rs index 6813e91e..dd60830f 100644 --- a/crates/modelardb_storage/src/write_ahead_log.rs +++ b/crates/modelardb_storage/src/write_ahead_log.rs @@ -17,7 +17,7 @@ //! efficiently persist data and operations on disk to avoid data loss and enable crash recovery. use std::collections::HashMap; -use std::fs::File; +use std::fs::{File, OpenOptions}; use std::path::PathBuf; use crate::WRITE_AHEAD_LOG_FOLDER; @@ -41,7 +41,7 @@ impl WriteAheadLog { let location = local_data_folder.location(); let log_folder_path = PathBuf::from(format!("{location}/{WRITE_AHEAD_LOG_FOLDER}")); - std::fs::create_dir(log_folder_path.clone())?; + std::fs::create_dir_all(log_folder_path.clone())?; Ok(Self { log_folder_path: log_folder_path.clone(), @@ -59,10 +59,15 @@ struct WriteAheadLogFile { } impl WriteAheadLogFile { - /// Create a new [`WriteAheadLogFile`] that writes to the file at `file_path`. If the file could - /// not be created, return [`ModelarDbStorageError`](crate::error::ModelarDbStorageError). + /// Create a new [`WriteAheadLogFile`] that appends to the file at `file_path`. If the file does + /// not exist, it is created. If the file could not be created, return + /// [`ModelarDbStorageError`](crate::error::ModelarDbStorageError). fn try_new(file_path: PathBuf) -> Result { - let file = File::create(file_path)?; + let file = OpenOptions::new() + .create(true) + .append(true) + .open(file_path)?; + Ok(Self { file }) } } From cd152237ac2fc218b9a9c5258b41d3a7b8a122bf Mon Sep 17 00:00:00 2001 From: CGodiksen <36046286+CGodiksen@users.noreply.github.com> Date: Sun, 14 Dec 2025 20:41:39 +0100 Subject: [PATCH 007/116] Add InvalidState to ModelarDbStoreError --- crates/modelardb_storage/src/error.rs | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/crates/modelardb_storage/src/error.rs b/crates/modelardb_storage/src/error.rs index cd39ca8e..b3ef4fa9 100644 --- a/crates/modelardb_storage/src/error.rs +++ b/crates/modelardb_storage/src/error.rs @@ -47,6 +47,8 @@ pub enum ModelarDbStorageError { EnvironmentVar(VarError), /// Error returned when an invalid argument was passed. InvalidArgument(String), + /// Error returned when an invalid state is encountered. + InvalidState(String), /// Error returned from IO operations. Io(IoError), /// Error returned by ObjectStore. @@ -71,6 +73,7 @@ impl Display for ModelarDbStorageError { Self::DeltaLake(reason) => write!(f, "Delta Lake Error: {reason}"), Self::EnvironmentVar(reason) => write!(f, "Environment Variable Error: {reason}"), Self::InvalidArgument(reason) => write!(f, "Invalid Argument Error: {reason}"), + Self::InvalidState(reason) => write!(f, "Invalid State Error: {reason}"), Self::Io(reason) => write!(f, "Io Error: {reason}"), Self::ObjectStore(reason) => write!(f, "Object Store Error: {reason}"), Self::ObjectStorePath(reason) => write!(f, "Object Store Path Error: {reason}"), @@ -91,6 +94,7 @@ impl Error for ModelarDbStorageError { Self::DeltaLake(reason) => Some(reason), Self::EnvironmentVar(reason) => Some(reason), Self::InvalidArgument(_reason) => None, + Self::InvalidState(_reason) => None, Self::Io(reason) => Some(reason), Self::ObjectStore(reason) => Some(reason), Self::ObjectStorePath(reason) => Some(reason), From ecd00ad10ccd2ef2f9a33a1452da3155e275260c Mon Sep 17 00:00:00 2001 From: CGodiksen <36046286+CGodiksen@users.noreply.github.com> Date: Sun, 14 Dec 2025 20:44:14 +0100 Subject: [PATCH 008/116] Add method for creating a table log in the WAL --- .../modelardb_storage/src/write_ahead_log.rs | 26 ++++++++++++++++--- 1 file changed, 22 insertions(+), 4 deletions(-) diff --git a/crates/modelardb_storage/src/write_ahead_log.rs b/crates/modelardb_storage/src/write_ahead_log.rs index dd60830f..0f0e5af8 100644 --- a/crates/modelardb_storage/src/write_ahead_log.rs +++ b/crates/modelardb_storage/src/write_ahead_log.rs @@ -22,7 +22,7 @@ use std::path::PathBuf; use crate::WRITE_AHEAD_LOG_FOLDER; use crate::data_folder::DataFolder; -use crate::error::Result; +use crate::error::{ModelarDbStorageError, Result}; /// Write-ahead log that logs data on a per-table level and operations separately. @@ -35,7 +35,7 @@ pub struct WriteAheadLog { impl WriteAheadLog { /// Create a new [`WriteAheadLog`] that stores the log in the root of `local_data_folder` in /// the [`WRITE_AHEAD_LOG_FOLDER`] folder. If the folder does not exist, it is created. If the - /// log could not be created, return [`ModelarDbStorageError`](crate::error::ModelarDbStorageError). + /// log could not be created, return [`ModelarDbStorageError`]. pub fn try_new(local_data_folder: &DataFolder) -> Result { // Create the folder for the write-ahead log if it does not exist. let location = local_data_folder.location(); @@ -49,6 +49,25 @@ impl WriteAheadLog { operation_log: WriteAheadLogFile::try_new(log_folder_path.join("operations.wal"))?, }) } + + /// Create a new [`WriteAheadLogFile`] for the table with the given name. If a log already + /// exists or the log file could not be created, return [`ModelarDbStorageError`]. + pub fn create_table_log(&mut self, table_name: &str) -> Result<()> { + if !self.table_logs.contains_key(table_name) { + let table_log_path = self.log_folder_path.join(format!("{}.wal", table_name)); + + self.table_logs.insert( + table_name.to_owned(), + WriteAheadLogFile::try_new(table_log_path)?, + ); + + Ok(()) + } else { + Err(ModelarDbStorageError::InvalidState(format!( + "Table log for table '{table_name}' already exists", + ))) + } + } } /// Wrapper around a [`File`] that enforces that [`sync_all()`](File::sync_all) is called @@ -60,8 +79,7 @@ struct WriteAheadLogFile { impl WriteAheadLogFile { /// Create a new [`WriteAheadLogFile`] that appends to the file at `file_path`. If the file does - /// not exist, it is created. If the file could not be created, return - /// [`ModelarDbStorageError`](crate::error::ModelarDbStorageError). + /// not exist, it is created. If the file could not be created, return [`ModelarDbStorageError`]. fn try_new(file_path: PathBuf) -> Result { let file = OpenOptions::new() .create(true) From a3d14b6324b1e36429f58ceee423729b1595179c Mon Sep 17 00:00:00 2001 From: CGodiksen <36046286+CGodiksen@users.noreply.github.com> Date: Sun, 14 Dec 2025 20:50:08 +0100 Subject: [PATCH 009/116] Add table logs when creating the write ahead log --- crates/modelardb_server/src/context.rs | 2 +- crates/modelardb_storage/src/write_ahead_log.rs | 17 +++++++++++++---- 2 files changed, 14 insertions(+), 5 deletions(-) diff --git a/crates/modelardb_server/src/context.rs b/crates/modelardb_server/src/context.rs index f1a013bb..a10b2e79 100644 --- a/crates/modelardb_server/src/context.rs +++ b/crates/modelardb_server/src/context.rs @@ -57,7 +57,7 @@ impl Context { StorageEngine::try_new(data_folders.clone(), &configuration_manager).await?, )); - let write_ahead_log = WriteAheadLog::try_new(&data_folders.local_data_folder)?; + let write_ahead_log = WriteAheadLog::try_new(&data_folders.local_data_folder).await?; Ok(Context { data_folders, diff --git a/crates/modelardb_storage/src/write_ahead_log.rs b/crates/modelardb_storage/src/write_ahead_log.rs index 0f0e5af8..3828746d 100644 --- a/crates/modelardb_storage/src/write_ahead_log.rs +++ b/crates/modelardb_storage/src/write_ahead_log.rs @@ -36,22 +36,31 @@ impl WriteAheadLog { /// Create a new [`WriteAheadLog`] that stores the log in the root of `local_data_folder` in /// the [`WRITE_AHEAD_LOG_FOLDER`] folder. If the folder does not exist, it is created. If the /// log could not be created, return [`ModelarDbStorageError`]. - pub fn try_new(local_data_folder: &DataFolder) -> Result { + pub async fn try_new(local_data_folder: &DataFolder) -> Result { // Create the folder for the write-ahead log if it does not exist. let location = local_data_folder.location(); let log_folder_path = PathBuf::from(format!("{location}/{WRITE_AHEAD_LOG_FOLDER}")); std::fs::create_dir_all(log_folder_path.clone())?; - Ok(Self { + let mut write_ahead_log = Self { log_folder_path: log_folder_path.clone(), table_logs: HashMap::new(), operation_log: WriteAheadLogFile::try_new(log_folder_path.join("operations.wal"))?, - }) + }; + + // For each time series table, create a log file if it does not already exist. + for table_name in local_data_folder.time_series_table_names().await? { + write_ahead_log.create_table_log(&table_name)?; + } + + Ok(write_ahead_log) } /// Create a new [`WriteAheadLogFile`] for the table with the given name. If a log already - /// exists or the log file could not be created, return [`ModelarDbStorageError`]. + /// exists in the map or the log file could not be created, return [`ModelarDbStorageError`]. + /// Note that if the log file already exists, but it is not present in the map, the existing + /// log file will be added to the map. pub fn create_table_log(&mut self, table_name: &str) -> Result<()> { if !self.table_logs.contains_key(table_name) { let table_log_path = self.log_folder_path.join(format!("{}.wal", table_name)); From 6c1582b8923cfe881d88750bf6001da12195d9e2 Mon Sep 17 00:00:00 2001 From: CGodiksen <36046286+CGodiksen@users.noreply.github.com> Date: Sun, 14 Dec 2025 21:11:16 +0100 Subject: [PATCH 010/116] Add file path to log file struct --- .../modelardb_storage/src/write_ahead_log.rs | 19 ++++++++++++++----- 1 file changed, 14 insertions(+), 5 deletions(-) diff --git a/crates/modelardb_storage/src/write_ahead_log.rs b/crates/modelardb_storage/src/write_ahead_log.rs index 3828746d..3c9390d4 100644 --- a/crates/modelardb_storage/src/write_ahead_log.rs +++ b/crates/modelardb_storage/src/write_ahead_log.rs @@ -27,8 +27,11 @@ use crate::error::{ModelarDbStorageError, Result}; /// Write-ahead log that logs data on a per-table level and operations separately. pub struct WriteAheadLog { - log_folder_path: PathBuf, + /// Path to the folder that contains the write-ahead log. + folder_path: PathBuf, + /// Logs for each table. The key is the table name, and the value is the log file for that table. table_logs: HashMap, + /// Log file for operations that are not associated with a specific table. operation_log: WriteAheadLogFile, } @@ -44,7 +47,7 @@ impl WriteAheadLog { std::fs::create_dir_all(log_folder_path.clone())?; let mut write_ahead_log = Self { - log_folder_path: log_folder_path.clone(), + folder_path: log_folder_path.clone(), table_logs: HashMap::new(), operation_log: WriteAheadLogFile::try_new(log_folder_path.join("operations.wal"))?, }; @@ -63,7 +66,7 @@ impl WriteAheadLog { /// log file will be added to the map. pub fn create_table_log(&mut self, table_name: &str) -> Result<()> { if !self.table_logs.contains_key(table_name) { - let table_log_path = self.log_folder_path.join(format!("{}.wal", table_name)); + let table_log_path = self.folder_path.join(format!("{}.wal", table_name)); self.table_logs.insert( table_name.to_owned(), @@ -83,6 +86,9 @@ impl WriteAheadLog { /// immediately after writing to ensure that all data is on disk before returning. Note that /// an exclusive lock is held on the file while it is being written to. struct WriteAheadLogFile { + /// Path to the file that the log is written to. + path: PathBuf, + /// File that the log is written to. file: File, } @@ -93,8 +99,11 @@ impl WriteAheadLogFile { let file = OpenOptions::new() .create(true) .append(true) - .open(file_path)?; + .open(file_path.clone())?; - Ok(Self { file }) + Ok(Self { + path: file_path, + file, + }) } } From a4ed7c3db5f65b8d24a64f109f9d2e27b00fd6e4 Mon Sep 17 00:00:00 2001 From: CGodiksen <36046286+CGodiksen@users.noreply.github.com> Date: Sun, 14 Dec 2025 21:25:29 +0100 Subject: [PATCH 011/116] Add method to remove table log --- .../modelardb_storage/src/write_ahead_log.rs | 21 +++++++++++++++++++ 1 file changed, 21 insertions(+) diff --git a/crates/modelardb_storage/src/write_ahead_log.rs b/crates/modelardb_storage/src/write_ahead_log.rs index 3c9390d4..2ee3acdf 100644 --- a/crates/modelardb_storage/src/write_ahead_log.rs +++ b/crates/modelardb_storage/src/write_ahead_log.rs @@ -80,6 +80,27 @@ impl WriteAheadLog { ))) } } + + /// Remove the log file for the table with the given name. If the log file does not exist or + /// could not be removed, return [`ModelarDbStorageError`]. + pub fn remove_table_log(&mut self, table_name: &str) -> Result<()> { + let log_path; + + if let Some(log_file) = self.table_logs.remove(table_name) { + log_path = log_file.path; + // log_file is dropped here as it goes out of scope which automatically closes its + // internal file handle. + } else { + return Err(ModelarDbStorageError::InvalidState(format!( + "Table log for table '{table_name}' does not exist", + ))); + } + + // Now that the file handle is closed, the file can be removed. + std::fs::remove_file(log_path)?; + + Ok(()) + } } /// Wrapper around a [`File`] that enforces that [`sync_all()`](File::sync_all) is called From 4bcb746f137b88618edc5e615187453fa3e4892a Mon Sep 17 00:00:00 2001 From: CGodiksen <36046286+CGodiksen@users.noreply.github.com> Date: Mon, 15 Dec 2025 14:03:17 +0100 Subject: [PATCH 012/116] Add StreamWriter to WriteAheadLogFile --- .../modelardb_storage/src/write_ahead_log.rs | 48 ++++++++++++++----- 1 file changed, 35 insertions(+), 13 deletions(-) diff --git a/crates/modelardb_storage/src/write_ahead_log.rs b/crates/modelardb_storage/src/write_ahead_log.rs index 2ee3acdf..1941e870 100644 --- a/crates/modelardb_storage/src/write_ahead_log.rs +++ b/crates/modelardb_storage/src/write_ahead_log.rs @@ -20,10 +20,15 @@ use std::collections::HashMap; use std::fs::{File, OpenOptions}; use std::path::PathBuf; +use arrow::datatypes::{DataType, Field, Schema}; +use arrow::ipc::writer::StreamWriter; +use modelardb_types::types::TimeSeriesTableMetadata; + use crate::WRITE_AHEAD_LOG_FOLDER; use crate::data_folder::DataFolder; use crate::error::{ModelarDbStorageError, Result}; +const OPERATIONS_LOG_FILE: &str = "operations.wal"; /// Write-ahead log that logs data on a per-table level and operations separately. pub struct WriteAheadLog { @@ -50,27 +55,36 @@ impl WriteAheadLog { folder_path: log_folder_path.clone(), table_logs: HashMap::new(), operation_log: WriteAheadLogFile::try_new(log_folder_path.join("operations.wal"))?, + operation_log: WriteAheadLogFile::try_new( + log_folder_path.join(OPERATIONS_LOG_FILE), + &operations_log_schema(), + )?, }; // For each time series table, create a log file if it does not already exist. - for table_name in local_data_folder.time_series_table_names().await? { - write_ahead_log.create_table_log(&table_name)?; + for metadata in local_data_folder.time_series_table_metadata().await? { + write_ahead_log.create_table_log(&metadata)?; } Ok(write_ahead_log) } - /// Create a new [`WriteAheadLogFile`] for the table with the given name. If a log already + /// Create a new [`WriteAheadLogFile`] for the table with the given metadata. If a log already /// exists in the map or the log file could not be created, return [`ModelarDbStorageError`]. /// Note that if the log file already exists, but it is not present in the map, the existing /// log file will be added to the map. - pub fn create_table_log(&mut self, table_name: &str) -> Result<()> { - if !self.table_logs.contains_key(table_name) { + pub fn create_table_log( + &mut self, + time_series_table_metadata: &TimeSeriesTableMetadata, + ) -> Result<()> { + let table_name = time_series_table_metadata.name.clone(); + + if !self.table_logs.contains_key(&table_name) { let table_log_path = self.folder_path.join(format!("{}.wal", table_name)); self.table_logs.insert( - table_name.to_owned(), - WriteAheadLogFile::try_new(table_log_path)?, + table_name, + WriteAheadLogFile::try_new(table_log_path, &time_series_table_metadata.schema)?, ); Ok(()) @@ -109,22 +123,30 @@ impl WriteAheadLog { struct WriteAheadLogFile { /// Path to the file that the log is written to. path: PathBuf, - /// File that the log is written to. - file: File, + /// Writer to write data in IPC streaming format to the log file. + writer: StreamWriter, } impl WriteAheadLogFile { - /// Create a new [`WriteAheadLogFile`] that appends to the file at `file_path`. If the file does - /// not exist, it is created. If the file could not be created, return [`ModelarDbStorageError`]. - fn try_new(file_path: PathBuf) -> Result { + /// Create a new [`WriteAheadLogFile`] that appends data with `schema` to the file at + /// `file_path`. If the file does not exist, it is created. If the file could not be created, + /// return [`ModelarDbStorageError`]. + fn try_new(file_path: PathBuf, schema: &Schema) -> Result { let file = OpenOptions::new() .create(true) .append(true) .open(file_path.clone())?; + let writer = StreamWriter::try_new(file, schema)?; + Ok(Self { path: file_path, - file, + writer, }) } } + +/// Return the schema for the operations log that is stored in [`OPERATIONS_LOG_FILE`]. +fn operations_log_schema() -> Schema { + Schema::new(vec![Field::new("operation", DataType::Utf8, false)]) +} From 16aa8a4fc2acf68fe93be0a43a621d7b5da8d37d Mon Sep 17 00:00:00 2001 From: CGodiksen <36046286+CGodiksen@users.noreply.github.com> Date: Mon, 15 Dec 2025 14:11:54 +0100 Subject: [PATCH 013/116] Add method to append and sync to WriteAheadLogFile --- crates/modelardb_storage/src/write_ahead_log.rs | 17 ++++++++++++++++- 1 file changed, 16 insertions(+), 1 deletion(-) diff --git a/crates/modelardb_storage/src/write_ahead_log.rs b/crates/modelardb_storage/src/write_ahead_log.rs index 1941e870..65a3fa22 100644 --- a/crates/modelardb_storage/src/write_ahead_log.rs +++ b/crates/modelardb_storage/src/write_ahead_log.rs @@ -22,6 +22,7 @@ use std::path::PathBuf; use arrow::datatypes::{DataType, Field, Schema}; use arrow::ipc::writer::StreamWriter; +use arrow::record_batch::RecordBatch; use modelardb_types::types::TimeSeriesTableMetadata; use crate::WRITE_AHEAD_LOG_FOLDER; @@ -54,7 +55,6 @@ impl WriteAheadLog { let mut write_ahead_log = Self { folder_path: log_folder_path.clone(), table_logs: HashMap::new(), - operation_log: WriteAheadLogFile::try_new(log_folder_path.join("operations.wal"))?, operation_log: WriteAheadLogFile::try_new( log_folder_path.join(OPERATIONS_LOG_FILE), &operations_log_schema(), @@ -144,6 +144,21 @@ impl WriteAheadLogFile { writer, }) } + + /// Append the given data to the log file and sync the file to ensure that all data is on disk. + /// If the data could not be appended or the file could not be synced, return + /// [`ModelarDbStorageError`]. + fn append_and_sync(&mut self, data: &RecordBatch) -> Result<()> { + self.writer.write(data)?; + + // Flush the writer's internal buffers to the file. + self.writer.flush()?; + + // Get a reference to the underlying file handle and sync to disk. + self.writer.get_ref().sync_all()?; + + Ok(()) + } } /// Return the schema for the operations log that is stored in [`OPERATIONS_LOG_FILE`]. From c7f3771cd2ce57e24efd2bdb8e5666a3037c9e02 Mon Sep 17 00:00:00 2001 From: CGodiksen <36046286+CGodiksen@users.noreply.github.com> Date: Mon, 15 Dec 2025 14:20:18 +0100 Subject: [PATCH 014/116] Add simple method to read all data from a log file --- .../modelardb_storage/src/write_ahead_log.rs | 21 +++++++++++++++++++ 1 file changed, 21 insertions(+) diff --git a/crates/modelardb_storage/src/write_ahead_log.rs b/crates/modelardb_storage/src/write_ahead_log.rs index 65a3fa22..91ccef7a 100644 --- a/crates/modelardb_storage/src/write_ahead_log.rs +++ b/crates/modelardb_storage/src/write_ahead_log.rs @@ -21,6 +21,7 @@ use std::fs::{File, OpenOptions}; use std::path::PathBuf; use arrow::datatypes::{DataType, Field, Schema}; +use arrow::ipc::reader::StreamReader; use arrow::ipc::writer::StreamWriter; use arrow::record_batch::RecordBatch; use modelardb_types::types::TimeSeriesTableMetadata; @@ -159,6 +160,26 @@ impl WriteAheadLogFile { Ok(()) } + + /// Read all data from the log file. This can be called even if the [`StreamWriter`] has not + /// been finished, meaning the log file is missing the end-of-stream bytes. If the file + /// could not be read, return [`ModelarDbStorageError`]. + fn read_all(file_path: &PathBuf) -> Result> { + // TODO: Maybe reuse the file handle instead of opening a new one. + let file = File::open(file_path)?; + let reader = StreamReader::try_new(file, None)?; + + let mut batches = Vec::new(); + for maybe_batch in reader { + match maybe_batch { + Ok(batch) => batches.push(batch), + // TODO: Maybe handle the specific error for end of file. + Err(_) => break, + } + } + + Ok(batches) + } } /// Return the schema for the operations log that is stored in [`OPERATIONS_LOG_FILE`]. From 0219a57e3526e7e2f173a97d2c7a69efa3b3cfc3 Mon Sep 17 00:00:00 2001 From: CGodiksen <36046286+CGodiksen@users.noreply.github.com> Date: Mon, 15 Dec 2025 14:24:16 +0100 Subject: [PATCH 015/116] Lock the file when we append and sync --- crates/modelardb_storage/src/write_ahead_log.rs | 8 ++++++++ 1 file changed, 8 insertions(+) diff --git a/crates/modelardb_storage/src/write_ahead_log.rs b/crates/modelardb_storage/src/write_ahead_log.rs index 91ccef7a..0a20b1c4 100644 --- a/crates/modelardb_storage/src/write_ahead_log.rs +++ b/crates/modelardb_storage/src/write_ahead_log.rs @@ -135,6 +135,7 @@ impl WriteAheadLogFile { fn try_new(file_path: PathBuf, schema: &Schema) -> Result { let file = OpenOptions::new() .create(true) + .read(true) .append(true) .open(file_path.clone())?; @@ -150,6 +151,9 @@ impl WriteAheadLogFile { /// If the data could not be appended or the file could not be synced, return /// [`ModelarDbStorageError`]. fn append_and_sync(&mut self, data: &RecordBatch) -> Result<()> { + // Lock the file handle so that no other process can write to it while we are writing. + let _lock = self.writer.get_mut().lock()?; + self.writer.write(data)?; // Flush the writer's internal buffers to the file. @@ -158,6 +162,10 @@ impl WriteAheadLogFile { // Get a reference to the underlying file handle and sync to disk. self.writer.get_ref().sync_all()?; + // Unlock the file manually since the lock is only dropped when the file handle goes out of + // scope. + self.writer.get_mut().unlock()?; + Ok(()) } From 1c68123bbd30305f0578488ab0b6a73b3f904703 Mon Sep 17 00:00:00 2001 From: CGodiksen <36046286+CGodiksen@users.noreply.github.com> Date: Sat, 31 Jan 2026 20:53:36 +0100 Subject: [PATCH 016/116] No longer allow WriteAheadLog to be created with non-local data folder --- crates/modelardb_storage/src/write_ahead_log.rs | 8 ++++++++ 1 file changed, 8 insertions(+) diff --git a/crates/modelardb_storage/src/write_ahead_log.rs b/crates/modelardb_storage/src/write_ahead_log.rs index 0a20b1c4..413bcc1f 100644 --- a/crates/modelardb_storage/src/write_ahead_log.rs +++ b/crates/modelardb_storage/src/write_ahead_log.rs @@ -49,6 +49,14 @@ impl WriteAheadLog { pub async fn try_new(local_data_folder: &DataFolder) -> Result { // Create the folder for the write-ahead log if it does not exist. let location = local_data_folder.location(); + + // Since the std:fs API is used, the location must be a local path. + if location.contains("://") { + return Err(ModelarDbStorageError::InvalidState(format!( + "Write-ahead log location '{location}' is not a local path." + ))); + } + let log_folder_path = PathBuf::from(format!("{location}/{WRITE_AHEAD_LOG_FOLDER}")); std::fs::create_dir_all(log_folder_path.clone())?; From 418373428b0be3bdc666bf54433ab9f2629bfd90 Mon Sep 17 00:00:00 2001 From: CGodiksen <36046286+CGodiksen@users.noreply.github.com> Date: Sat, 31 Jan 2026 21:08:25 +0100 Subject: [PATCH 017/116] Use path saved in field when reading log file --- crates/modelardb_storage/src/write_ahead_log.rs | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/crates/modelardb_storage/src/write_ahead_log.rs b/crates/modelardb_storage/src/write_ahead_log.rs index 413bcc1f..d52fc557 100644 --- a/crates/modelardb_storage/src/write_ahead_log.rs +++ b/crates/modelardb_storage/src/write_ahead_log.rs @@ -180,9 +180,9 @@ impl WriteAheadLogFile { /// Read all data from the log file. This can be called even if the [`StreamWriter`] has not /// been finished, meaning the log file is missing the end-of-stream bytes. If the file /// could not be read, return [`ModelarDbStorageError`]. - fn read_all(file_path: &PathBuf) -> Result> { + fn read_all(&self) -> Result> { // TODO: Maybe reuse the file handle instead of opening a new one. - let file = File::open(file_path)?; + let file = File::open(&self.path)?; let reader = StreamReader::try_new(file, None)?; let mut batches = Vec::new(); From 484cc5675fcf2454c678569ceaa3541be82ca20e Mon Sep 17 00:00:00 2001 From: CGodiksen <36046286+CGodiksen@users.noreply.github.com> Date: Sun, 1 Feb 2026 13:43:51 +0100 Subject: [PATCH 018/116] Use Mutex instead of locking the file itself --- .../modelardb_storage/src/write_ahead_log.rs | 21 ++++++++----------- 1 file changed, 9 insertions(+), 12 deletions(-) diff --git a/crates/modelardb_storage/src/write_ahead_log.rs b/crates/modelardb_storage/src/write_ahead_log.rs index d52fc557..ad47e353 100644 --- a/crates/modelardb_storage/src/write_ahead_log.rs +++ b/crates/modelardb_storage/src/write_ahead_log.rs @@ -19,6 +19,7 @@ use std::collections::HashMap; use std::fs::{File, OpenOptions}; use std::path::PathBuf; +use std::sync::Mutex; use arrow::datatypes::{DataType, Field, Schema}; use arrow::ipc::reader::StreamReader; @@ -133,7 +134,7 @@ struct WriteAheadLogFile { /// Path to the file that the log is written to. path: PathBuf, /// Writer to write data in IPC streaming format to the log file. - writer: StreamWriter, + writer: Mutex>, } impl WriteAheadLogFile { @@ -151,28 +152,24 @@ impl WriteAheadLogFile { Ok(Self { path: file_path, - writer, + writer: Mutex::new(writer), }) } /// Append the given data to the log file and sync the file to ensure that all data is on disk. /// If the data could not be appended or the file could not be synced, return /// [`ModelarDbStorageError`]. - fn append_and_sync(&mut self, data: &RecordBatch) -> Result<()> { - // Lock the file handle so that no other process can write to it while we are writing. - let _lock = self.writer.get_mut().lock()?; + fn append_and_sync(&self, data: &RecordBatch) -> Result<()> { + // Acquire the mutex to ensure only one thread can write at a time. + let mut writer = self.writer.lock().expect("Mutex should not be poisoned."); - self.writer.write(data)?; + writer.write(data)?; // Flush the writer's internal buffers to the file. - self.writer.flush()?; + writer.flush()?; // Get a reference to the underlying file handle and sync to disk. - self.writer.get_ref().sync_all()?; - - // Unlock the file manually since the lock is only dropped when the file handle goes out of - // scope. - self.writer.get_mut().unlock()?; + writer.get_ref().sync_all()?; Ok(()) } From c4448ba1b624c77ee17c478220ccbc3e5abff0cd Mon Sep 17 00:00:00 2001 From: CGodiksen <36046286+CGodiksen@users.noreply.github.com> Date: Sun, 1 Feb 2026 13:51:53 +0100 Subject: [PATCH 019/116] Acquire mutex during reading for safety --- crates/modelardb_storage/src/write_ahead_log.rs | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/crates/modelardb_storage/src/write_ahead_log.rs b/crates/modelardb_storage/src/write_ahead_log.rs index ad47e353..30436909 100644 --- a/crates/modelardb_storage/src/write_ahead_log.rs +++ b/crates/modelardb_storage/src/write_ahead_log.rs @@ -178,6 +178,11 @@ impl WriteAheadLogFile { /// been finished, meaning the log file is missing the end-of-stream bytes. If the file /// could not be read, return [`ModelarDbStorageError`]. fn read_all(&self) -> Result> { + // Acquire the mutex to ensure data is not being written while reading. Note that reading + // should only occur during recovery, which should make concurrent writes improbable. + // However, since performance is not critical during recovery, the mutex is held anyway. + let _writer = self.writer.lock().unwrap(); + // TODO: Maybe reuse the file handle instead of opening a new one. let file = File::open(&self.path)?; let reader = StreamReader::try_new(file, None)?; From cc581cea8f700abd604bc52553bffd276acc906c Mon Sep 17 00:00:00 2001 From: CGodiksen <36046286+CGodiksen@users.noreply.github.com> Date: Sun, 1 Feb 2026 14:05:21 +0100 Subject: [PATCH 020/116] Handle unexpected EOF error directly instead of allowing all errors --- crates/modelardb_storage/src/write_ahead_log.rs | 13 ++++++++++--- 1 file changed, 10 insertions(+), 3 deletions(-) diff --git a/crates/modelardb_storage/src/write_ahead_log.rs b/crates/modelardb_storage/src/write_ahead_log.rs index 30436909..afce5059 100644 --- a/crates/modelardb_storage/src/write_ahead_log.rs +++ b/crates/modelardb_storage/src/write_ahead_log.rs @@ -22,6 +22,7 @@ use std::path::PathBuf; use std::sync::Mutex; use arrow::datatypes::{DataType, Field, Schema}; +use arrow::error::ArrowError::IpcError; use arrow::ipc::reader::StreamReader; use arrow::ipc::writer::StreamWriter; use arrow::record_batch::RecordBatch; @@ -183,7 +184,6 @@ impl WriteAheadLogFile { // However, since performance is not critical during recovery, the mutex is held anyway. let _writer = self.writer.lock().unwrap(); - // TODO: Maybe reuse the file handle instead of opening a new one. let file = File::open(&self.path)?; let reader = StreamReader::try_new(file, None)?; @@ -191,8 +191,15 @@ impl WriteAheadLogFile { for maybe_batch in reader { match maybe_batch { Ok(batch) => batches.push(batch), - // TODO: Maybe handle the specific error for end of file. - Err(_) => break, + Err(IpcError(msg)) => { + // Check if it is an UnexpectedEof error, which is expected when reading + // an incomplete stream without the end-of-stream marker. + if msg.contains("UnexpectedEof") || msg.contains("unexpected end of file") { + break; + } + return Err(IpcError(msg).into()); + } + Err(e) => return Err(e.into()), } } From 5b9ecbdf48eca0507ece37fa36b4ba671cd5862f Mon Sep 17 00:00:00 2001 From: CGodiksen <36046286+CGodiksen@users.noreply.github.com> Date: Sun, 1 Feb 2026 18:37:26 +0100 Subject: [PATCH 021/116] Create table log file when time series table is created --- crates/modelardb_server/src/context.rs | 9 +++++++-- 1 file changed, 7 insertions(+), 2 deletions(-) diff --git a/crates/modelardb_server/src/context.rs b/crates/modelardb_server/src/context.rs index a10b2e79..b2b767db 100644 --- a/crates/modelardb_server/src/context.rs +++ b/crates/modelardb_server/src/context.rs @@ -40,7 +40,7 @@ pub struct Context { /// Manages all uncompressed and compressed data in the system. pub storage_engine: Arc>, /// Write-ahead log for persisting data and operations. - pub write_ahead_log: WriteAheadLog, + pub write_ahead_log: RwLock, } impl Context { @@ -57,7 +57,8 @@ impl Context { StorageEngine::try_new(data_folders.clone(), &configuration_manager).await?, )); - let write_ahead_log = WriteAheadLog::try_new(&data_folders.local_data_folder).await?; + let write_ahead_log = + RwLock::new(WriteAheadLog::try_new(&data_folders.local_data_folder).await?); Ok(Context { data_folders, @@ -115,6 +116,10 @@ impl Context { self.register_and_save_time_series_table(time_series_table_metadata) .await?; + // Create a file in the write-ahead log to log uncompressed data for the table. + let mut write_ahead_log = self.write_ahead_log.write().await; + write_ahead_log.create_table_log(time_series_table_metadata)?; + Ok(()) } From 99269e91af52e79f169b74f09dc3c82824a68b6a Mon Sep 17 00:00:00 2001 From: CGodiksen <36046286+CGodiksen@users.noreply.github.com> Date: Sun, 1 Feb 2026 18:45:18 +0100 Subject: [PATCH 022/116] Delete the table log file when dropping time series tables --- crates/modelardb_server/src/context.rs | 18 ++++++++++-------- 1 file changed, 10 insertions(+), 8 deletions(-) diff --git a/crates/modelardb_server/src/context.rs b/crates/modelardb_server/src/context.rs index b2b767db..a373af63 100644 --- a/crates/modelardb_server/src/context.rs +++ b/crates/modelardb_server/src/context.rs @@ -277,17 +277,19 @@ impl Context { self.drop_table_from_storage_engine(table_name).await?; + let local_data_folder = &self.data_folders.local_data_folder; + + // If the table is a time series table, delete the table log file from the write-ahead log. + if local_data_folder.is_time_series_table(table_name).await? { + let mut write_ahead_log = self.write_ahead_log.write().await; + write_ahead_log.remove_table_log(table_name)?; + } + // Drop the table metadata from the Delta Lake. - self.data_folders - .local_data_folder - .drop_table_metadata(table_name) - .await?; + local_data_folder.drop_table_metadata(table_name).await?; // Drop the table from the Delta Lake. - self.data_folders - .local_data_folder - .drop_table(table_name) - .await?; + local_data_folder.drop_table(table_name).await?; Ok(()) } From b2dd2b01b2695b1cdbc636f5399929f3956d85af Mon Sep 17 00:00:00 2001 From: CGodiksen <36046286+CGodiksen@users.noreply.github.com> Date: Sun, 1 Feb 2026 19:57:31 +0100 Subject: [PATCH 023/116] Pass write-ahead log to storage engine as well --- crates/modelardb_server/src/configuration.rs | 9 +++++++-- crates/modelardb_server/src/context.rs | 16 +++++++++++----- crates/modelardb_server/src/storage/mod.rs | 2 ++ 3 files changed, 20 insertions(+), 7 deletions(-) diff --git a/crates/modelardb_server/src/configuration.rs b/crates/modelardb_server/src/configuration.rs index dbdcc5db..af59264d 100644 --- a/crates/modelardb_server/src/configuration.rs +++ b/crates/modelardb_server/src/configuration.rs @@ -382,6 +382,7 @@ mod tests { use std::sync::Arc; use modelardb_storage::data_folder::DataFolder; + use modelardb_storage::write_ahead_log::WriteAheadLog; use modelardb_types::types::{Node, ServerMode}; use tempfile::TempDir; use tokio::sync::RwLock; @@ -691,15 +692,19 @@ mod tests { let configuration_manager = Arc::new(RwLock::new( ConfigurationManager::try_new( - local_data_folder, + local_data_folder.clone(), ClusterMode::MultiNode(Box::new(cluster)), ) .await .unwrap(), )); + let write_ahead_log = Arc::new(RwLock::new( + WriteAheadLog::try_new(&local_data_folder).await.unwrap(), + )); + let storage_engine = Arc::new(RwLock::new( - StorageEngine::try_new(data_folders, &configuration_manager) + StorageEngine::try_new(data_folders, write_ahead_log, &configuration_manager) .await .unwrap(), )); diff --git a/crates/modelardb_server/src/context.rs b/crates/modelardb_server/src/context.rs index a373af63..1bafda6e 100644 --- a/crates/modelardb_server/src/context.rs +++ b/crates/modelardb_server/src/context.rs @@ -40,7 +40,7 @@ pub struct Context { /// Manages all uncompressed and compressed data in the system. pub storage_engine: Arc>, /// Write-ahead log for persisting data and operations. - pub write_ahead_log: RwLock, + pub write_ahead_log: Arc>, } impl Context { @@ -53,12 +53,18 @@ impl Context { .await?, )); - let storage_engine = Arc::new(RwLock::new( - StorageEngine::try_new(data_folders.clone(), &configuration_manager).await?, + let write_ahead_log = Arc::new(RwLock::new( + WriteAheadLog::try_new(&data_folders.local_data_folder).await?, )); - let write_ahead_log = - RwLock::new(WriteAheadLog::try_new(&data_folders.local_data_folder).await?); + let storage_engine = Arc::new(RwLock::new( + StorageEngine::try_new( + data_folders.clone(), + write_ahead_log.clone(), + &configuration_manager, + ) + .await?, + )); Ok(Context { data_folders, diff --git a/crates/modelardb_server/src/storage/mod.rs b/crates/modelardb_server/src/storage/mod.rs index f7488f10..13d80127 100644 --- a/crates/modelardb_server/src/storage/mod.rs +++ b/crates/modelardb_server/src/storage/mod.rs @@ -34,6 +34,7 @@ use std::sync::{Arc, LazyLock}; use std::thread::{self, JoinHandle}; use datafusion::arrow::record_batch::RecordBatch; +use modelardb_storage::write_ahead_log::WriteAheadLog; use modelardb_types::types::TimeSeriesTableMetadata; use tokio::runtime::Handle; use tokio::sync::RwLock; @@ -91,6 +92,7 @@ impl StorageEngine { /// created. pub(super) async fn try_new( data_folders: DataFolders, + write_ahead_log: Arc>, configuration_manager: &Arc>, ) -> Result { // Create shared memory pool. From 748662a63a4d5bbfe6b61b858899e11690b54774 Mon Sep 17 00:00:00 2001 From: CGodiksen <36046286+CGodiksen@users.noreply.github.com> Date: Sun, 1 Feb 2026 20:15:05 +0100 Subject: [PATCH 024/116] Append to write-ahead log in storage engine when inserting data points --- crates/modelardb_server/src/context.rs | 2 +- crates/modelardb_server/src/storage/mod.rs | 10 +++++++++- crates/modelardb_storage/src/write_ahead_log.rs | 16 +++++++++++++++- 3 files changed, 25 insertions(+), 3 deletions(-) diff --git a/crates/modelardb_server/src/context.rs b/crates/modelardb_server/src/context.rs index 1bafda6e..85a5795c 100644 --- a/crates/modelardb_server/src/context.rs +++ b/crates/modelardb_server/src/context.rs @@ -40,7 +40,7 @@ pub struct Context { /// Manages all uncompressed and compressed data in the system. pub storage_engine: Arc>, /// Write-ahead log for persisting data and operations. - pub write_ahead_log: Arc>, + write_ahead_log: Arc>, } impl Context { diff --git a/crates/modelardb_server/src/storage/mod.rs b/crates/modelardb_server/src/storage/mod.rs index 13d80127..569d923b 100644 --- a/crates/modelardb_server/src/storage/mod.rs +++ b/crates/modelardb_server/src/storage/mod.rs @@ -83,6 +83,8 @@ pub struct StorageEngine { join_handles: Vec>, /// Unbounded channels used by the threads to communicate. channels: Arc, + /// Write-ahead log for persisting data and operations. + write_ahead_log: Arc>, } impl StorageEngine { @@ -200,6 +202,7 @@ impl StorageEngine { memory_pool, join_handles, channels, + write_ahead_log, }; // Start the task that transfers data periodically if a remote data folder is given and @@ -261,7 +264,12 @@ impl StorageEngine { time_series_table_metadata: Arc, multivariate_data_points: RecordBatch, ) -> Result<()> { - // TODO: write to a WAL and use it to ensure termination never duplicates or loses data. + // Write to the write-ahead log to ensure termination never duplicates or loses data. We use + // a read lock since the specific log file is locked internally before writing. + let write_ahead_log = self.write_ahead_log.read().await; + write_ahead_log + .append_to_table_log(&time_series_table_metadata.name, &multivariate_data_points)?; + self.memory_pool .wait_for_ingested_memory(multivariate_data_points.get_array_memory_size() as u64); diff --git a/crates/modelardb_storage/src/write_ahead_log.rs b/crates/modelardb_storage/src/write_ahead_log.rs index afce5059..cb47c6aa 100644 --- a/crates/modelardb_storage/src/write_ahead_log.rs +++ b/crates/modelardb_storage/src/write_ahead_log.rs @@ -117,7 +117,7 @@ impl WriteAheadLog { // internal file handle. } else { return Err(ModelarDbStorageError::InvalidState(format!( - "Table log for table '{table_name}' does not exist", + "Table log for table '{table_name}' does not exist.", ))); } @@ -126,6 +126,20 @@ impl WriteAheadLog { Ok(()) } + + /// Append data to the log for the given table and sync the file to ensure that all data is on + /// disk. Only requires read access to the log since the internal Mutex handles write + /// synchronization. If a table log does not exist or the data could not be appended, return + /// [`ModelarDbStorageError`]. + pub fn append_to_table_log(&self, table_name: &str, data: &RecordBatch) -> Result<()> { + let log_file = self.table_logs.get(table_name).ok_or_else(|| { + ModelarDbStorageError::InvalidState(format!( + "Table log for table '{table_name}' does not exist." + )) + })?; + + log_file.append_and_sync(data) + } } /// Wrapper around a [`File`] that enforces that [`sync_all()`](File::sync_all) is called From cf77ef70b97e2fa91dd6a53c9716e5a35e97a15a Mon Sep 17 00:00:00 2001 From: CGodiksen <36046286+CGodiksen@users.noreply.github.com> Date: Tue, 3 Feb 2026 19:10:11 +0100 Subject: [PATCH 025/116] Use sync_data() instead of sync_all() --- crates/modelardb_storage/src/write_ahead_log.rs | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/crates/modelardb_storage/src/write_ahead_log.rs b/crates/modelardb_storage/src/write_ahead_log.rs index cb47c6aa..8773cd15 100644 --- a/crates/modelardb_storage/src/write_ahead_log.rs +++ b/crates/modelardb_storage/src/write_ahead_log.rs @@ -183,8 +183,9 @@ impl WriteAheadLogFile { // Flush the writer's internal buffers to the file. writer.flush()?; - // Get a reference to the underlying file handle and sync to disk. - writer.get_ref().sync_all()?; + // Get a reference to the underlying file handle and sync to disk. Note that file metadata + // such as modification timestamps and permissions are not updated since we only sync data. + writer.get_ref().sync_data()?; Ok(()) } From 41f315163f82f8618ada60af06facc5dc34f1d37 Mon Sep 17 00:00:00 2001 From: CGodiksen <36046286+CGodiksen@users.noreply.github.com> Date: Tue, 10 Mar 2026 09:38:35 +0100 Subject: [PATCH 026/116] Prevent duplicate WAL schema header on open --- crates/modelardb_storage/src/write_ahead_log.rs | 8 ++++++++ 1 file changed, 8 insertions(+) diff --git a/crates/modelardb_storage/src/write_ahead_log.rs b/crates/modelardb_storage/src/write_ahead_log.rs index 8773cd15..ae9c6bf9 100644 --- a/crates/modelardb_storage/src/write_ahead_log.rs +++ b/crates/modelardb_storage/src/write_ahead_log.rs @@ -163,8 +163,16 @@ impl WriteAheadLogFile { .append(true) .open(file_path.clone())?; + let file_len = file.metadata()?.len(); + let writer = StreamWriter::try_new(file, schema)?; + // If the file already had data, the StreamWriter wrote a duplicate schema header. + // Truncate back to the original length to remove it. + if file_len > 0 { + writer.get_ref().set_len(file_len)?; + } + Ok(Self { path: file_path, writer: Mutex::new(writer), From 7dc9a47f5342045deafe7b2084b42cfdb6a29986 Mon Sep 17 00:00:00 2001 From: CGodiksen <36046286+CGodiksen@users.noreply.github.com> Date: Tue, 10 Mar 2026 10:33:03 +0100 Subject: [PATCH 027/116] Fix windows issue with append mode and set_len --- crates/modelardb_storage/src/write_ahead_log.rs | 7 +++++-- 1 file changed, 5 insertions(+), 2 deletions(-) diff --git a/crates/modelardb_storage/src/write_ahead_log.rs b/crates/modelardb_storage/src/write_ahead_log.rs index ae9c6bf9..ac064728 100644 --- a/crates/modelardb_storage/src/write_ahead_log.rs +++ b/crates/modelardb_storage/src/write_ahead_log.rs @@ -18,6 +18,7 @@ use std::collections::HashMap; use std::fs::{File, OpenOptions}; +use std::io::{Seek, SeekFrom}; use std::path::PathBuf; use std::sync::Mutex; @@ -160,7 +161,7 @@ impl WriteAheadLogFile { let file = OpenOptions::new() .create(true) .read(true) - .append(true) + .write(true) .open(file_path.clone())?; let file_len = file.metadata()?.len(); @@ -168,9 +169,11 @@ impl WriteAheadLogFile { let writer = StreamWriter::try_new(file, schema)?; // If the file already had data, the StreamWriter wrote a duplicate schema header. - // Truncate back to the original length to remove it. + // Truncate back to the original length to remove it, then seek to the end so + // subsequent writes append correctly. if file_len > 0 { writer.get_ref().set_len(file_len)?; + writer.get_ref().seek(SeekFrom::End(0))?; } Ok(Self { From 842e87e02406c32ff20134daeb3f306a954f652d Mon Sep 17 00:00:00 2001 From: CGodiksen <36046286+CGodiksen@users.noreply.github.com> Date: Tue, 10 Mar 2026 10:39:17 +0100 Subject: [PATCH 028/116] Add unit tests for WriteAheadLogFile --- .../modelardb_storage/src/write_ahead_log.rs | 150 ++++++++++++++++++ 1 file changed, 150 insertions(+) diff --git a/crates/modelardb_storage/src/write_ahead_log.rs b/crates/modelardb_storage/src/write_ahead_log.rs index ac064728..b8582469 100644 --- a/crates/modelardb_storage/src/write_ahead_log.rs +++ b/crates/modelardb_storage/src/write_ahead_log.rs @@ -237,3 +237,153 @@ impl WriteAheadLogFile { fn operations_log_schema() -> Schema { Schema::new(vec![Field::new("operation", DataType::Utf8, false)]) } + +#[cfg(test)] +mod tests { + use super::*; + + use modelardb_test::table::{ + TIME_SERIES_TABLE_NAME, time_series_table_metadata, + uncompressed_time_series_table_record_batch, + }; + + #[test] + fn test_try_new_creates_file() { + let temp_dir = tempfile::tempdir().unwrap(); + let file_path = temp_dir + .path() + .join(format!("{}.wal", TIME_SERIES_TABLE_NAME)); + + let metadata = time_series_table_metadata(); + let wal_file = WriteAheadLogFile::try_new(file_path.clone(), &metadata.schema).unwrap(); + + assert!(file_path.exists()); + assert_eq!(wal_file.path, file_path); + } + + #[test] + fn test_read_all_empty_file() { + let temp_dir = tempfile::tempdir().unwrap(); + let file_path = temp_dir + .path() + .join(format!("{}.wal", TIME_SERIES_TABLE_NAME)); + + let metadata = time_series_table_metadata(); + let wal_file = WriteAheadLogFile::try_new(file_path, &metadata.schema).unwrap(); + + let batches = wal_file.read_all().unwrap(); + assert!(batches.is_empty()); + } + + #[test] + fn test_append_and_read_single_batch() { + let temp_dir = tempfile::tempdir().unwrap(); + let file_path = temp_dir + .path() + .join(format!("{}.wal", TIME_SERIES_TABLE_NAME)); + + let metadata = time_series_table_metadata(); + let wal_file = WriteAheadLogFile::try_new(file_path, &metadata.schema).unwrap(); + let batch = uncompressed_time_series_table_record_batch(5); + + wal_file.append_and_sync(&batch).unwrap(); + + let batches = wal_file.read_all().unwrap(); + assert_eq!(batches.len(), 1); + assert_eq!(batches[0], batch); + } + + #[test] + fn test_append_and_read_multiple_batches() { + let temp_dir = tempfile::tempdir().unwrap(); + let file_path = temp_dir + .path() + .join(format!("{}.wal", TIME_SERIES_TABLE_NAME)); + + let metadata = time_series_table_metadata(); + let wal_file = WriteAheadLogFile::try_new(file_path, &metadata.schema).unwrap(); + + let batch_1 = uncompressed_time_series_table_record_batch(10); + let batch_2 = uncompressed_time_series_table_record_batch(20); + let batch_3 = uncompressed_time_series_table_record_batch(30); + + wal_file.append_and_sync(&batch_1).unwrap(); + wal_file.append_and_sync(&batch_2).unwrap(); + wal_file.append_and_sync(&batch_3).unwrap(); + + let batches = wal_file.read_all().unwrap(); + assert_eq!(batches.len(), 3); + assert_eq!(batches[0], batch_1); + assert_eq!(batches[1], batch_2); + assert_eq!(batches[2], batch_3); + } + + #[test] + fn test_reopen_existing_file_and_append() { + let temp_dir = tempfile::tempdir().unwrap(); + let file_path = temp_dir + .path() + .join(format!("{}.wal", TIME_SERIES_TABLE_NAME)); + + let metadata = time_series_table_metadata(); + let batch_1 = uncompressed_time_series_table_record_batch(10); + { + let wal_file = WriteAheadLogFile::try_new(file_path.clone(), &metadata.schema).unwrap(); + wal_file.append_and_sync(&batch_1).unwrap(); + } + + let batch_2 = uncompressed_time_series_table_record_batch(20); + let wal_file = WriteAheadLogFile::try_new(file_path, &metadata.schema).unwrap(); + wal_file.append_and_sync(&batch_2).unwrap(); + + let batches = wal_file.read_all().unwrap(); + assert_eq!(batches.len(), 2); + assert_eq!(batches[0], batch_1); + assert_eq!(batches[1], batch_2); + } + + #[test] + fn test_reopen_existing_file_and_read_without_append() { + let temp_dir = tempfile::tempdir().unwrap(); + let file_path = temp_dir + .path() + .join(format!("{}.wal", TIME_SERIES_TABLE_NAME)); + + let metadata = time_series_table_metadata(); + let batch = uncompressed_time_series_table_record_batch(10); + { + let wal_file = WriteAheadLogFile::try_new(file_path.clone(), &metadata.schema).unwrap(); + wal_file.append_and_sync(&batch).unwrap(); + } + + let wal_file = WriteAheadLogFile::try_new(file_path, &metadata.schema).unwrap(); + let batches = wal_file.read_all().unwrap(); + assert_eq!(batches.len(), 1); + assert_eq!(batches[0], batch); + } + + #[test] + fn test_file_size_not_changed_on_reopen() { + let temp_dir = tempfile::tempdir().unwrap(); + let file_path = temp_dir + .path() + .join(format!("{}.wal", TIME_SERIES_TABLE_NAME)); + + let metadata = time_series_table_metadata(); + let batch = uncompressed_time_series_table_record_batch(10); + { + let wal_file = WriteAheadLogFile::try_new(file_path.clone(), &metadata.schema).unwrap(); + wal_file.append_and_sync(&batch).unwrap(); + } + + let size_before = std::fs::metadata(&file_path).unwrap().len(); + + { + let _wal_file = + WriteAheadLogFile::try_new(file_path.clone(), &metadata.schema).unwrap(); + } + + let size_after = std::fs::metadata(&file_path).unwrap().len(); + assert_eq!(size_before, size_after); + } +} From 80965a4520fae9c629ade912109fb84f8a9851eb Mon Sep 17 00:00:00 2001 From: CGodiksen <36046286+CGodiksen@users.noreply.github.com> Date: Tue, 10 Mar 2026 11:33:47 +0100 Subject: [PATCH 029/116] Import module for test function imports --- .../modelardb_storage/src/write_ahead_log.rs | 36 +++++++++---------- 1 file changed, 17 insertions(+), 19 deletions(-) diff --git a/crates/modelardb_storage/src/write_ahead_log.rs b/crates/modelardb_storage/src/write_ahead_log.rs index b8582469..1f26f482 100644 --- a/crates/modelardb_storage/src/write_ahead_log.rs +++ b/crates/modelardb_storage/src/write_ahead_log.rs @@ -242,10 +242,8 @@ fn operations_log_schema() -> Schema { mod tests { use super::*; - use modelardb_test::table::{ - TIME_SERIES_TABLE_NAME, time_series_table_metadata, - uncompressed_time_series_table_record_batch, - }; + use modelardb_test::table; + use modelardb_test::table::TIME_SERIES_TABLE_NAME; #[test] fn test_try_new_creates_file() { @@ -254,7 +252,7 @@ mod tests { .path() .join(format!("{}.wal", TIME_SERIES_TABLE_NAME)); - let metadata = time_series_table_metadata(); + let metadata = table::time_series_table_metadata(); let wal_file = WriteAheadLogFile::try_new(file_path.clone(), &metadata.schema).unwrap(); assert!(file_path.exists()); @@ -268,7 +266,7 @@ mod tests { .path() .join(format!("{}.wal", TIME_SERIES_TABLE_NAME)); - let metadata = time_series_table_metadata(); + let metadata = table::time_series_table_metadata(); let wal_file = WriteAheadLogFile::try_new(file_path, &metadata.schema).unwrap(); let batches = wal_file.read_all().unwrap(); @@ -282,9 +280,9 @@ mod tests { .path() .join(format!("{}.wal", TIME_SERIES_TABLE_NAME)); - let metadata = time_series_table_metadata(); + let metadata = table::time_series_table_metadata(); let wal_file = WriteAheadLogFile::try_new(file_path, &metadata.schema).unwrap(); - let batch = uncompressed_time_series_table_record_batch(5); + let batch = table::uncompressed_time_series_table_record_batch(5); wal_file.append_and_sync(&batch).unwrap(); @@ -300,12 +298,12 @@ mod tests { .path() .join(format!("{}.wal", TIME_SERIES_TABLE_NAME)); - let metadata = time_series_table_metadata(); + let metadata = table::time_series_table_metadata(); let wal_file = WriteAheadLogFile::try_new(file_path, &metadata.schema).unwrap(); - let batch_1 = uncompressed_time_series_table_record_batch(10); - let batch_2 = uncompressed_time_series_table_record_batch(20); - let batch_3 = uncompressed_time_series_table_record_batch(30); + let batch_1 = table::uncompressed_time_series_table_record_batch(10); + let batch_2 = table::uncompressed_time_series_table_record_batch(20); + let batch_3 = table::uncompressed_time_series_table_record_batch(30); wal_file.append_and_sync(&batch_1).unwrap(); wal_file.append_and_sync(&batch_2).unwrap(); @@ -325,14 +323,14 @@ mod tests { .path() .join(format!("{}.wal", TIME_SERIES_TABLE_NAME)); - let metadata = time_series_table_metadata(); - let batch_1 = uncompressed_time_series_table_record_batch(10); + let metadata = table::time_series_table_metadata(); + let batch_1 = table::uncompressed_time_series_table_record_batch(10); { let wal_file = WriteAheadLogFile::try_new(file_path.clone(), &metadata.schema).unwrap(); wal_file.append_and_sync(&batch_1).unwrap(); } - let batch_2 = uncompressed_time_series_table_record_batch(20); + let batch_2 = table::uncompressed_time_series_table_record_batch(20); let wal_file = WriteAheadLogFile::try_new(file_path, &metadata.schema).unwrap(); wal_file.append_and_sync(&batch_2).unwrap(); @@ -349,8 +347,8 @@ mod tests { .path() .join(format!("{}.wal", TIME_SERIES_TABLE_NAME)); - let metadata = time_series_table_metadata(); - let batch = uncompressed_time_series_table_record_batch(10); + let metadata = table::time_series_table_metadata(); + let batch = table::uncompressed_time_series_table_record_batch(10); { let wal_file = WriteAheadLogFile::try_new(file_path.clone(), &metadata.schema).unwrap(); wal_file.append_and_sync(&batch).unwrap(); @@ -369,8 +367,8 @@ mod tests { .path() .join(format!("{}.wal", TIME_SERIES_TABLE_NAME)); - let metadata = time_series_table_metadata(); - let batch = uncompressed_time_series_table_record_batch(10); + let metadata = table::time_series_table_metadata(); + let batch = table::uncompressed_time_series_table_record_batch(10); { let wal_file = WriteAheadLogFile::try_new(file_path.clone(), &metadata.schema).unwrap(); wal_file.append_and_sync(&batch).unwrap(); From 2152dd2e780e1e5fe54dac90ca011d7e5f21e039 Mon Sep 17 00:00:00 2001 From: CGodiksen <36046286+CGodiksen@users.noreply.github.com> Date: Tue, 10 Mar 2026 12:10:13 +0100 Subject: [PATCH 030/116] Install serde_json for commit metadata --- Cargo.lock | 662 +++++++++++++--------------- Cargo.toml | 1 + crates/modelardb_storage/Cargo.toml | 1 + 3 files changed, 299 insertions(+), 365 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index 899f7205..6e1f1f1c 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -52,6 +52,12 @@ version = "0.2.21" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "683d7910e743518b0e34f1186f92494becacb047c7b6bf616c96772180fef923" +[[package]] +name = "android-tzdata" +version = "0.1.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "e999941b234f3131b00bc13c22d06e8c5ff726d1b6318ac7eb276997bbb4fef0" + [[package]] name = "android_system_properties" version = "0.1.5" @@ -81,9 +87,9 @@ checksum = "7c02d123df017efcdfbd739ef81735b36c5ba83ec3c59c80a9d7ecc718f92e50" [[package]] name = "arrow" -version = "57.1.0" +version = "56.2.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "cb372a7cbcac02a35d3fb7b3fc1f969ec078e871f9bb899bf00a2e1809bec8a3" +checksum = "6e833808ff2d94ed40d9379848a950d995043c7fb3e81a30b383f4c6033821cc" dependencies = [ "arrow-arith", "arrow-array", @@ -102,23 +108,23 @@ dependencies = [ [[package]] name = "arrow-arith" -version = "57.1.0" +version = "56.2.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "0f377dcd19e440174596d83deb49cd724886d91060c07fec4f67014ef9d54049" +checksum = "ad08897b81588f60ba983e3ca39bda2b179bdd84dced378e7df81a5313802ef8" dependencies = [ "arrow-array", "arrow-buffer", "arrow-data", "arrow-schema", "chrono", - "num-traits", + "num", ] [[package]] name = "arrow-array" -version = "57.1.0" +version = "56.2.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a23eaff85a44e9fa914660fb0d0bb00b79c4a3d888b5334adb3ea4330c84f002" +checksum = "8548ca7c070d8db9ce7aa43f37393e4bfcf3f2d3681df278490772fd1673d08d" dependencies = [ "ahash", "arrow-buffer", @@ -128,33 +134,29 @@ dependencies = [ "chrono-tz", "half", "hashbrown 0.16.0", - "num-complex", - "num-integer", - "num-traits", + "num", ] [[package]] name = "arrow-buffer" -version = "57.1.0" +version = "56.2.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a2819d893750cb3380ab31ebdc8c68874dd4429f90fd09180f3c93538bd21626" +checksum = "e003216336f70446457e280807a73899dd822feaf02087d31febca1363e2fccc" dependencies = [ "bytes", "half", - "num-bigint", - "num-traits", + "num", ] [[package]] name = "arrow-cast" -version = "57.1.0" +version = "56.2.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "e3d131abb183f80c450d4591dc784f8d7750c50c6e2bc3fcaad148afc8361271" +checksum = "919418a0681298d3a77d1a315f625916cb5678ad0d74b9c60108eb15fd083023" dependencies = [ "arrow-array", "arrow-buffer", "arrow-data", - "arrow-ord", "arrow-schema", "arrow-select", "atoi", @@ -163,15 +165,15 @@ dependencies = [ "comfy-table", "half", "lexical-core", - "num-traits", + "num", "ryu", ] [[package]] name = "arrow-csv" -version = "57.1.0" +version = "56.2.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "2275877a0e5e7e7c76954669366c2aa1a829e340ab1f612e647507860906fb6b" +checksum = "bfa9bf02705b5cf762b6f764c65f04ae9082c7cfc4e96e0c33548ee3f67012eb" dependencies = [ "arrow-array", "arrow-cast", @@ -184,22 +186,21 @@ dependencies = [ [[package]] name = "arrow-data" -version = "57.1.0" +version = "56.2.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "05738f3d42cb922b9096f7786f606fcb8669260c2640df8490533bb2fa38c9d3" +checksum = "a5c64fff1d142f833d78897a772f2e5b55b36cb3e6320376f0961ab0db7bd6d0" dependencies = [ "arrow-buffer", "arrow-schema", "half", - "num-integer", - "num-traits", + "num", ] [[package]] name = "arrow-flight" -version = "57.1.0" +version = "56.2.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "8b5f57c3d39d1b1b7c1376a772ea86a131e7da310aed54ebea9363124bb885e3" +checksum = "8c8b0ba0784d56bc6266b79f5de7a24b47024e7b3a0045d2ad4df3d9b686099f" dependencies = [ "arrow-array", "arrow-buffer", @@ -212,14 +213,13 @@ dependencies = [ "prost", "prost-types", "tonic", - "tonic-prost", ] [[package]] name = "arrow-ipc" -version = "57.1.0" +version = "56.2.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "3d09446e8076c4b3f235603d9ea7c5494e73d441b01cd61fb33d7254c11964b3" +checksum = "1d3594dcddccc7f20fd069bc8e9828ce37220372680ff638c5e00dea427d88f5" dependencies = [ "arrow-array", "arrow-buffer", @@ -233,9 +233,9 @@ dependencies = [ [[package]] name = "arrow-json" -version = "57.1.0" +version = "56.2.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "371ffd66fa77f71d7628c63f209c9ca5341081051aa32f9c8020feb0def787c0" +checksum = "88cf36502b64a127dc659e3b305f1d993a544eab0d48cce704424e62074dc04b" dependencies = [ "arrow-array", "arrow-buffer", @@ -245,21 +245,19 @@ dependencies = [ "chrono", "half", "indexmap", - "itoa", "lexical-core", "memchr", - "num-traits", - "ryu", - "serde_core", + "num", + "serde", "serde_json", "simdutf8", ] [[package]] name = "arrow-ord" -version = "57.1.0" +version = "56.2.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "cbc94fc7adec5d1ba9e8cd1b1e8d6f72423b33fe978bf1f46d970fafab787521" +checksum = "3c8f82583eb4f8d84d4ee55fd1cb306720cddead7596edce95b50ee418edf66f" dependencies = [ "arrow-array", "arrow-buffer", @@ -270,9 +268,9 @@ dependencies = [ [[package]] name = "arrow-row" -version = "57.1.0" +version = "56.2.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "169676f317157dc079cc5def6354d16db63d8861d61046d2f3883268ced6f99f" +checksum = "9d07ba24522229d9085031df6b94605e0f4b26e099fb7cdeec37abd941a73753" dependencies = [ "arrow-array", "arrow-buffer", @@ -283,35 +281,34 @@ dependencies = [ [[package]] name = "arrow-schema" -version = "57.1.0" +version = "56.2.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d27609cd7dd45f006abae27995c2729ef6f4b9361cde1ddd019dc31a5aa017e0" +checksum = "b3aa9e59c611ebc291c28582077ef25c97f1975383f1479b12f3b9ffee2ffabe" dependencies = [ "bitflags", "serde", - "serde_core", "serde_json", ] [[package]] name = "arrow-select" -version = "57.1.0" +version = "56.2.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "ae980d021879ea119dd6e2a13912d81e64abed372d53163e804dfe84639d8010" +checksum = "8c41dbbd1e97bfcaee4fcb30e29105fb2c75e4d82ae4de70b792a5d3f66b2e7a" dependencies = [ "ahash", "arrow-array", "arrow-buffer", "arrow-data", "arrow-schema", - "num-traits", + "num", ] [[package]] name = "arrow-string" -version = "57.1.0" +version = "56.2.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "cf35e8ef49dcf0c5f6d175edee6b8af7b45611805333129c541a8b89a0fc0534" +checksum = "53f5183c150fbc619eede22b861ea7c0eebed8eaac0333eaa7f6da5205fd504d" dependencies = [ "arrow-array", "arrow-buffer", @@ -319,7 +316,7 @@ dependencies = [ "arrow-schema", "arrow-select", "memchr", - "num-traits", + "num", "regex", "regex-syntax", ] @@ -953,9 +950,9 @@ checksum = "1fd0f2584146f6f2ef48085050886acf353beff7305ebd1ae69500e27c67f64b" [[package]] name = "bytes" -version = "1.11.0" +version = "1.10.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "b35204fbdc0b3f4446b89fc1ac2cf84a8a68971995d0bf2e925ec7cd960f9cb3" +checksum = "d71b6127be86fdcfddb610f7182ac57211d4b18a3e9c82eb2d17662f2227ad6a" [[package]] name = "bytes-utils" @@ -1030,16 +1027,17 @@ checksum = "613afe47fcd5fac7ccf1db93babcb082c5994d996f20b8b159f2ad1658eb5724" [[package]] name = "chrono" -version = "0.4.42" +version = "0.4.41" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "145052bdd345b87320e369255277e3fb5152762ad123a901ef5c262dd38fe8d2" +checksum = "c469d952047f47f91b68d1cba3f10d63c11d73e4636f24f08daf0278abf01c4d" dependencies = [ + "android-tzdata", "iana-time-zone", "js-sys", "num-traits", "serde", "wasm-bindgen", - "windows-link 0.2.1", + "windows-link 0.1.1", ] [[package]] @@ -1133,9 +1131,9 @@ checksum = "7c74b8349d32d297c9134b8c88677813a227df8f779daa29bfc29c183fe3dca6" [[package]] name = "convert_case" -version = "0.9.0" +version = "0.8.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "db05ffb6856bf0ecdf6367558a76a0e8a77b1713044eb92845c692100ed50190" +checksum = "baaaa0ecca5b51987b9423ccdc971514dd8b0bb7b4060b983d3664dad3f1f89f" dependencies = [ "unicode-segmentation", ] @@ -1175,21 +1173,6 @@ dependencies = [ "libc", ] -[[package]] -name = "crc" -version = "3.4.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "5eb8a2a1cd12ab0d987a5d5e825195d372001a4094a0376319d5a0ad71c1ba0d" -dependencies = [ - "crc-catalog", -] - -[[package]] -name = "crc-catalog" -version = "2.4.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "19d374276b40fb8bbdee95aef7c7fa6b5316ec764510eb64b8dd0e2ed0d7e7f5" - [[package]] name = "crc32fast" version = "1.4.2" @@ -1360,11 +1343,12 @@ dependencies = [ [[package]] name = "datafusion" -version = "51.0.0" +version = "50.3.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "8ba7cb113e9c0bedf9e9765926031e132fa05a1b09ba6e93a6d1a4d7044457b8" +checksum = "2af15bb3c6ffa33011ef579f6b0bcbe7c26584688bd6c994f548e44df67f011a" dependencies = [ "arrow", + "arrow-ipc", "arrow-schema", "async-trait", "bytes", @@ -1375,7 +1359,6 @@ dependencies = [ "datafusion-common", "datafusion-common-runtime", "datafusion-datasource", - "datafusion-datasource-arrow", "datafusion-datasource-csv", "datafusion-datasource-json", "datafusion-datasource-parquet", @@ -1404,8 +1387,7 @@ dependencies = [ "parquet", "rand 0.9.2", "regex", - "rstest", - "sqlparser", + "sqlparser 0.58.0", "tempfile", "tokio", "url", @@ -1416,9 +1398,9 @@ dependencies = [ [[package]] name = "datafusion-catalog" -version = "51.0.0" +version = "50.3.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "66a3a799f914a59b1ea343906a0486f17061f39509af74e874a866428951130d" +checksum = "187622262ad8f7d16d3be9202b4c1e0116f1c9aa387e5074245538b755261621" dependencies = [ "arrow", "async-trait", @@ -1431,6 +1413,7 @@ dependencies = [ "datafusion-physical-expr", "datafusion-physical-plan", "datafusion-session", + "datafusion-sql", "futures", "itertools 0.14.0", "log", @@ -1441,9 +1424,9 @@ dependencies = [ [[package]] name = "datafusion-catalog-listing" -version = "51.0.0" +version = "50.3.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "6db1b113c80d7a0febcd901476a57aef378e717c54517a163ed51417d87621b0" +checksum = "9657314f0a32efd0382b9a46fdeb2d233273ece64baa68a7c45f5a192daf0f83" dependencies = [ "arrow", "async-trait", @@ -1453,11 +1436,10 @@ dependencies = [ "datafusion-execution", "datafusion-expr", "datafusion-physical-expr", - "datafusion-physical-expr-adapter", "datafusion-physical-expr-common", "datafusion-physical-plan", + "datafusion-session", "futures", - "itertools 0.14.0", "log", "object_store", "tokio", @@ -1465,13 +1447,14 @@ dependencies = [ [[package]] name = "datafusion-common" -version = "51.0.0" +version = "50.3.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "7c10f7659e96127d25e8366be7c8be4109595d6a2c3eac70421f380a7006a1b0" +checksum = "5a83760d9a13122d025fbdb1d5d5aaf93dd9ada5e90ea229add92aa30898b2d1" dependencies = [ "ahash", "arrow", "arrow-ipc", + "base64 0.22.1", "chrono", "half", "hashbrown 0.14.5", @@ -1482,16 +1465,16 @@ dependencies = [ "parquet", "paste", "recursive", - "sqlparser", + "sqlparser 0.58.0", "tokio", "web-time", ] [[package]] name = "datafusion-common-runtime" -version = "51.0.0" +version = "50.3.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "b92065bbc6532c6651e2f7dd30b55cba0c7a14f860c7e1d15f165c41a1868d95" +checksum = "5b6234a6c7173fe5db1c6c35c01a12b2aa0f803a3007feee53483218817f8b1e" dependencies = [ "futures", "log", @@ -1500,9 +1483,9 @@ dependencies = [ [[package]] name = "datafusion-datasource" -version = "51.0.0" +version = "50.3.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "fde13794244bc7581cd82f6fff217068ed79cdc344cafe4ab2c3a1c3510b38d6" +checksum = "7256c9cb27a78709dd42d0c80f0178494637209cac6e29d5c93edd09b6721b86" dependencies = [ "arrow", "async-compression", @@ -1525,7 +1508,9 @@ dependencies = [ "itertools 0.14.0", "log", "object_store", + "parquet", "rand 0.9.2", + "tempfile", "tokio", "tokio-util", "url", @@ -1533,44 +1518,22 @@ dependencies = [ "zstd", ] -[[package]] -name = "datafusion-datasource-arrow" -version = "51.0.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "804fa9b4ecf3157982021770617200ef7c1b2979d57bec9044748314775a9aea" -dependencies = [ - "arrow", - "arrow-ipc", - "async-trait", - "bytes", - "datafusion-common", - "datafusion-common-runtime", - "datafusion-datasource", - "datafusion-execution", - "datafusion-expr", - "datafusion-physical-expr-common", - "datafusion-physical-plan", - "datafusion-session", - "futures", - "itertools 0.14.0", - "object_store", - "tokio", -] - [[package]] name = "datafusion-datasource-csv" -version = "51.0.0" +version = "50.3.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "61a1641a40b259bab38131c5e6f48fac0717bedb7dc93690e604142a849e0568" +checksum = "64533a90f78e1684bfb113d200b540f18f268134622d7c96bbebc91354d04825" dependencies = [ "arrow", "async-trait", "bytes", + "datafusion-catalog", "datafusion-common", "datafusion-common-runtime", "datafusion-datasource", "datafusion-execution", "datafusion-expr", + "datafusion-physical-expr", "datafusion-physical-expr-common", "datafusion-physical-plan", "datafusion-session", @@ -1582,44 +1545,49 @@ dependencies = [ [[package]] name = "datafusion-datasource-json" -version = "51.0.0" +version = "50.3.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "adeacdb00c1d37271176f8fb6a1d8ce096baba16ea7a4b2671840c5c9c64fe85" +checksum = "8d7ebeb12c77df0aacad26f21b0d033aeede423a64b2b352f53048a75bf1d6e6" dependencies = [ "arrow", "async-trait", "bytes", + "datafusion-catalog", "datafusion-common", "datafusion-common-runtime", "datafusion-datasource", "datafusion-execution", "datafusion-expr", + "datafusion-physical-expr", "datafusion-physical-expr-common", "datafusion-physical-plan", "datafusion-session", "futures", "object_store", + "serde_json", "tokio", ] [[package]] name = "datafusion-datasource-parquet" -version = "51.0.0" +version = "50.3.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "43d0b60ffd66f28bfb026565d62b0a6cbc416da09814766a3797bba7d85a3cd9" +checksum = "09e783c4c7d7faa1199af2df4761c68530634521b176a8d1331ddbc5a5c75133" dependencies = [ "arrow", "async-trait", "bytes", + "datafusion-catalog", "datafusion-common", "datafusion-common-runtime", "datafusion-datasource", "datafusion-execution", "datafusion-expr", - "datafusion-functions-aggregate-common", + "datafusion-functions-aggregate", "datafusion-physical-expr", "datafusion-physical-expr-adapter", "datafusion-physical-expr-common", + "datafusion-physical-optimizer", "datafusion-physical-plan", "datafusion-pruning", "datafusion-session", @@ -1629,20 +1597,21 @@ dependencies = [ "object_store", "parking_lot", "parquet", + "rand 0.9.2", "tokio", ] [[package]] name = "datafusion-doc" -version = "51.0.0" +version = "50.3.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "2b99e13947667b36ad713549237362afb054b2d8f8cc447751e23ec61202db07" +checksum = "99ee6b1d9a80d13f9deb2291f45c07044b8e62fb540dbde2453a18be17a36429" [[package]] name = "datafusion-execution" -version = "51.0.0" +version = "50.3.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "63695643190679037bc946ad46a263b62016931547bf119859c511f7ff2f5178" +checksum = "a4cec0a57653bec7b933fb248d3ffa3fa3ab3bd33bd140dc917f714ac036f531" dependencies = [ "arrow", "async-trait", @@ -1660,9 +1629,9 @@ dependencies = [ [[package]] name = "datafusion-expr" -version = "51.0.0" +version = "50.3.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "f9a4787cbf5feb1ab351f789063398f67654a6df75c4d37d7f637dc96f951a91" +checksum = "ef76910bdca909722586389156d0aa4da4020e1631994d50fadd8ad4b1aa05fe" dependencies = [ "arrow", "async-trait", @@ -1674,18 +1643,17 @@ dependencies = [ "datafusion-functions-window-common", "datafusion-physical-expr-common", "indexmap", - "itertools 0.14.0", "paste", "recursive", "serde_json", - "sqlparser", + "sqlparser 0.58.0", ] [[package]] name = "datafusion-expr-common" -version = "51.0.0" +version = "50.3.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "5ce2fb1b8c15c9ac45b0863c30b268c69dc9ee7a1ee13ecf5d067738338173dc" +checksum = "6d155ccbda29591ca71a1344dd6bed26c65a4438072b400df9db59447f590bb6" dependencies = [ "arrow", "datafusion-common", @@ -1696,9 +1664,9 @@ dependencies = [ [[package]] name = "datafusion-functions" -version = "51.0.0" +version = "50.3.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "794a9db7f7b96b3346fc007ff25e994f09b8f0511b4cf7dff651fadfe3ebb28f" +checksum = "7de2782136bd6014670fd84fe3b0ca3b3e4106c96403c3ae05c0598577139977" dependencies = [ "arrow", "arrow-buffer", @@ -1716,7 +1684,6 @@ dependencies = [ "itertools 0.14.0", "log", "md-5", - "num-traits", "rand 0.9.2", "regex", "sha2", @@ -1726,9 +1693,9 @@ dependencies = [ [[package]] name = "datafusion-functions-aggregate" -version = "51.0.0" +version = "50.3.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "1c25210520a9dcf9c2b2cbbce31ebd4131ef5af7fc60ee92b266dc7d159cb305" +checksum = "07331fc13603a9da97b74fd8a273f4238222943dffdbbed1c4c6f862a30105bf" dependencies = [ "ahash", "arrow", @@ -1747,9 +1714,9 @@ dependencies = [ [[package]] name = "datafusion-functions-aggregate-common" -version = "51.0.0" +version = "50.3.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "62f4a66f3b87300bb70f4124b55434d2ae3fe80455f3574701d0348da040b55d" +checksum = "b5951e572a8610b89968a09b5420515a121fbc305c0258651f318dc07c97ab17" dependencies = [ "ahash", "arrow", @@ -1760,9 +1727,9 @@ dependencies = [ [[package]] name = "datafusion-functions-nested" -version = "51.0.0" +version = "50.3.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "ae5c06eed03918dc7fe7a9f082a284050f0e9ecf95d72f57712d1496da03b8c4" +checksum = "fdacca9302c3d8fc03f3e94f338767e786a88a33f5ebad6ffc0e7b50364b9ea3" dependencies = [ "arrow", "arrow-ord", @@ -1770,7 +1737,6 @@ dependencies = [ "datafusion-doc", "datafusion-execution", "datafusion-expr", - "datafusion-expr-common", "datafusion-functions", "datafusion-functions-aggregate", "datafusion-functions-aggregate-common", @@ -1783,9 +1749,9 @@ dependencies = [ [[package]] name = "datafusion-functions-table" -version = "51.0.0" +version = "50.3.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "db4fed1d71738fbe22e2712d71396db04c25de4111f1ec252b8f4c6d3b25d7f5" +checksum = "8c37ff8a99434fbbad604a7e0669717c58c7c4f14c472d45067c4b016621d981" dependencies = [ "arrow", "async-trait", @@ -1799,9 +1765,9 @@ dependencies = [ [[package]] name = "datafusion-functions-window" -version = "51.0.0" +version = "50.3.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "1d92206aa5ae21892f1552b4d61758a862a70956e6fd7a95cb85db1de74bc6d1" +checksum = "48e2aea7c79c926cffabb13dc27309d4eaeb130f4a21c8ba91cdd241c813652b" dependencies = [ "arrow", "datafusion-common", @@ -1817,9 +1783,9 @@ dependencies = [ [[package]] name = "datafusion-functions-window-common" -version = "51.0.0" +version = "50.3.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "53ae9bcc39800820d53a22d758b3b8726ff84a5a3e24cecef04ef4e5fdf1c7cc" +checksum = "0fead257ab5fd2ffc3b40fda64da307e20de0040fe43d49197241d9de82a487f" dependencies = [ "datafusion-common", "datafusion-physical-expr-common", @@ -1827,20 +1793,20 @@ dependencies = [ [[package]] name = "datafusion-macros" -version = "51.0.0" +version = "50.3.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "1063ad4c9e094b3f798acee16d9a47bd7372d9699be2de21b05c3bd3f34ab848" +checksum = "ec6f637bce95efac05cdfb9b6c19579ed4aa5f6b94d951cfa5bb054b7bb4f730" dependencies = [ - "datafusion-doc", + "datafusion-expr", "quote", "syn", ] [[package]] name = "datafusion-optimizer" -version = "51.0.0" +version = "50.3.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "9f35f9ec5d08b87fd1893a30c2929f2559c2f9806ca072d8fefca5009dc0f06a" +checksum = "c6583ef666ae000a613a837e69e456681a9faa96347bf3877661e9e89e141d8a" dependencies = [ "arrow", "chrono", @@ -1858,9 +1824,9 @@ dependencies = [ [[package]] name = "datafusion-physical-expr" -version = "51.0.0" +version = "50.3.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c30cc8012e9eedcb48bbe112c6eff4ae5ed19cf3003cb0f505662e88b7014c5d" +checksum = "c8668103361a272cbbe3a61f72eca60c9b7c706e87cc3565bcf21e2b277b84f6" dependencies = [ "ahash", "arrow", @@ -1873,6 +1839,7 @@ dependencies = [ "hashbrown 0.14.5", "indexmap", "itertools 0.14.0", + "log", "parking_lot", "paste", "petgraph 0.8.3", @@ -1880,9 +1847,9 @@ dependencies = [ [[package]] name = "datafusion-physical-expr-adapter" -version = "51.0.0" +version = "50.3.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "7f9ff2dbd476221b1f67337699eff432781c4e6e1713d2aefdaa517dfbf79768" +checksum = "815acced725d30601b397e39958e0e55630e0a10d66ef7769c14ae6597298bb0" dependencies = [ "arrow", "datafusion-common", @@ -1895,9 +1862,9 @@ dependencies = [ [[package]] name = "datafusion-physical-expr-common" -version = "51.0.0" +version = "50.3.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "90da43e1ec550b172f34c87ec68161986ced70fd05c8d2a2add66eef9c276f03" +checksum = "6652fe7b5bf87e85ed175f571745305565da2c0b599d98e697bcbedc7baa47c3" dependencies = [ "ahash", "arrow", @@ -1909,9 +1876,9 @@ dependencies = [ [[package]] name = "datafusion-physical-optimizer" -version = "51.0.0" +version = "50.3.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "ce9804f799acd7daef3be7aaffe77c0033768ed8fdbf5fb82fc4c5f2e6bc14e6" +checksum = "49b7d623eb6162a3332b564a0907ba00895c505d101b99af78345f1acf929b5c" dependencies = [ "arrow", "datafusion-common", @@ -1923,14 +1890,15 @@ dependencies = [ "datafusion-physical-plan", "datafusion-pruning", "itertools 0.14.0", + "log", "recursive", ] [[package]] name = "datafusion-physical-plan" -version = "51.0.0" +version = "50.3.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "0acf0ad6b6924c6b1aa7d213b181e012e2d3ec0a64ff5b10ee6282ab0f8532ac" +checksum = "e2f7f778a1a838dec124efb96eae6144237d546945587557c9e6936b3414558c" dependencies = [ "ahash", "arrow", @@ -1959,26 +1927,15 @@ dependencies = [ [[package]] name = "datafusion-proto" -version = "51.0.0" +version = "50.3.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d368093a98a17d1449b1083ac22ed16b7128e4c67789991869480d8c4a40ecb9" +checksum = "a7df9f606892e6af45763d94d210634eec69b9bb6ced5353381682ff090028a3" dependencies = [ "arrow", "chrono", - "datafusion-catalog", - "datafusion-catalog-listing", + "datafusion", "datafusion-common", - "datafusion-datasource", - "datafusion-datasource-arrow", - "datafusion-datasource-csv", - "datafusion-datasource-json", - "datafusion-datasource-parquet", - "datafusion-execution", "datafusion-expr", - "datafusion-functions-table", - "datafusion-physical-expr", - "datafusion-physical-expr-common", - "datafusion-physical-plan", "datafusion-proto-common", "object_store", "prost", @@ -1986,9 +1943,9 @@ dependencies = [ [[package]] name = "datafusion-proto-common" -version = "51.0.0" +version = "50.3.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "3b6aef3d5e5c1d2bc3114c4876730cb76a9bdc5a8df31ef1b6db48f0c1671895" +checksum = "b4b14f288ca4ef77743d9672cafecf3adfffff0b9b04af9af79ecbeaaf736901" dependencies = [ "arrow", "datafusion-common", @@ -1997,11 +1954,12 @@ dependencies = [ [[package]] name = "datafusion-pruning" -version = "51.0.0" +version = "50.3.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "ac2c2498a1f134a9e11a9f5ed202a2a7d7e9774bd9249295593053ea3be999db" +checksum = "cd1e59e2ca14fe3c30f141600b10ad8815e2856caa59ebbd0e3e07cd3d127a65" dependencies = [ "arrow", + "arrow-schema", "datafusion-common", "datafusion-datasource", "datafusion-expr-common", @@ -2014,47 +1972,55 @@ dependencies = [ [[package]] name = "datafusion-session" -version = "51.0.0" +version = "50.3.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "8f96eebd17555386f459037c65ab73aae8df09f464524c709d6a3134ad4f4776" +checksum = "21ef8e2745583619bd7a49474e8f45fbe98ebb31a133f27802217125a7b3d58d" dependencies = [ + "arrow", "async-trait", + "dashmap", "datafusion-common", + "datafusion-common-runtime", "datafusion-execution", "datafusion-expr", + "datafusion-physical-expr", "datafusion-physical-plan", + "datafusion-sql", + "futures", + "itertools 0.14.0", + "log", + "object_store", "parking_lot", + "tokio", ] [[package]] name = "datafusion-sql" -version = "51.0.0" +version = "50.3.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "3fc195fe60634b2c6ccfd131b487de46dc30eccae8a3c35a13f136e7f440414f" +checksum = "89abd9868770386fede29e5a4b14f49c0bf48d652c3b9d7a8a0332329b87d50b" dependencies = [ "arrow", "bigdecimal", - "chrono", "datafusion-common", "datafusion-expr", "indexmap", "log", "recursive", "regex", - "sqlparser", + "sqlparser 0.58.0", ] [[package]] name = "delta_kernel" -version = "0.19.2" +version = "0.16.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "06f7fc164b1557731fcc68a198e813811a000efade0f112d4f0a002e65042b83" +checksum = "cb6b80fa39021744edf13509bbdd7caef94c1bf101e384990210332dbddddf44" dependencies = [ "arrow", "bytes", "chrono", "comfy-table", - "crc", "delta_kernel_derive", "futures", "indexmap", @@ -2077,9 +2043,9 @@ dependencies = [ [[package]] name = "delta_kernel_derive" -version = "0.19.0" +version = "0.16.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c9e6474dabfc8e0b849ee2d68f8f13025230d1945b28c69695e9a21b9219ac8e" +checksum = "ae1d02d9f5d886ae8bb7fc3f7a3cb8f1b75cd0f5c95f9b5f45bba308f1a0aa58" dependencies = [ "proc-macro2", "quote", @@ -2088,9 +2054,9 @@ dependencies = [ [[package]] name = "deltalake" -version = "0.30.2" +version = "0.29.4" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d5ace194fd6a5db14d4b4973c5780cf4569650716594ffd25297343be2e7cb0c" +checksum = "09169ef5ecf35911f5f1c3117844a4e00da1edcce58fe8593a237761525f6e3a" dependencies = [ "ctor", "delta_kernel", @@ -2100,9 +2066,9 @@ dependencies = [ [[package]] name = "deltalake-aws" -version = "0.13.0" +version = "0.12.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "63b470ec0212b5a704424db8a7f44ae90d8c2b4fc96246860dea2b90f80fe1ee" +checksum = "c0210d644f4ab27e6d477da99e4b4bf0c7d739fd399ac38c005b6d0dfa4fe132" dependencies = [ "async-trait", "aws-config", @@ -2120,16 +2086,15 @@ dependencies = [ "thiserror", "tokio", "tracing", - "typed-builder", "url", "uuid", ] [[package]] name = "deltalake-core" -version = "0.30.0" +version = "0.29.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "0fd42250f1dc45510e9745f5f747201ed9de72c13911ca5c11dd2cc27fe207e3" +checksum = "058a672565db45b857617c925f1b9a006bba4d339a87d2f9b1845e183b2ff4a9" dependencies = [ "arrow", "arrow-arith", @@ -2168,7 +2133,7 @@ dependencies = [ "regex", "serde", "serde_json", - "sqlparser", + "sqlparser 0.59.0", "strum 0.27.1", "thiserror", "tokio", @@ -2180,9 +2145,9 @@ dependencies = [ [[package]] name = "deltalake-derive" -version = "0.30.0" +version = "0.29.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "3963d9fe965af7b1dea433271389e1e39c6a97ffdbc2e81d808f5b329e4577b3" +checksum = "1a785b4702c2d1b6ff286075f375fb2fd52dfbb2fadf17b9233f4d5eea35c6ec" dependencies = [ "convert_case", "itertools 0.14.0", @@ -2311,7 +2276,7 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "0ce92ff622d6dadf7349484f42c93271a0d49b7cc4d466a936405bacbe10aa78" dependencies = [ "cfg-if", - "rustix 1.1.3", + "rustix 1.0.2", "windows-sys 0.59.0", ] @@ -2446,12 +2411,6 @@ version = "0.3.31" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "f90f7dce0722e95104fcb095585910c0977252f286e354b5e3bd38902cd99988" -[[package]] -name = "futures-timer" -version = "3.0.3" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "f288b0a4f20f9a56b5d1da57e2227c661b7b16168e2f72365f57b63326e29b24" - [[package]] name = "futures-util" version = "0.3.31" @@ -2551,14 +2510,13 @@ dependencies = [ [[package]] name = "half" -version = "2.7.1" +version = "2.6.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "6ea2d84b969582b4b1864a92dc5d27cd2b77b622a8d79306834f1be5ba20d84b" +checksum = "459196ed295495a68f7d7fe1d84f6c4b7ff0e21fe3017b2f283c6fac3ad803c9" dependencies = [ "cfg-if", "crunchy", "num-traits", - "zerocopy 0.8.31", ] [[package]] @@ -3180,9 +3138,9 @@ checksum = "d26c52dbd32dccf2d10cac7725f8eae5296885fb5703b261f7d0a0739ec807ab" [[package]] name = "linux-raw-sys" -version = "0.11.0" +version = "0.9.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "df1d3c3b53da64cf5760482273a98e575c651a67eec7f77df96b5b642de8f039" +checksum = "6db9c683daf087dc577b7506e9695b3d556a9f3849903fa28186283afd6809e9" [[package]] name = "litemap" @@ -3202,17 +3160,17 @@ dependencies = [ [[package]] name = "log" -version = "0.4.29" +version = "0.4.28" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "5e5032e24019045c762d3c0f28f5b6b8bbf38563a65908389bf7978758920897" +checksum = "34080505efa8e45a4b816c349525ebe327ceaa8559756f0356cba97ef3bf7432" [[package]] name = "lz4_flex" -version = "0.12.0" +version = "0.11.3" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "ab6473172471198271ff72e9379150e9dfd70d8e533e0752a27e515b48dd375e" +checksum = "75761162ae2b0e580d7e7c390558127e5f01b4194debd6221fd8c207fc80e3f5" dependencies = [ - "twox-hash", + "twox-hash 1.6.3", ] [[package]] @@ -3390,7 +3348,8 @@ dependencies = [ "modelardb_test", "modelardb_types", "object_store", - "sqlparser", + "serde_json", + "sqlparser 0.58.0", "tempfile", "tokio", "tonic", @@ -3476,6 +3435,20 @@ dependencies = [ "windows-sys 0.61.2", ] +[[package]] +name = "num" +version = "0.4.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "35bd024e8b2ff75562e5f34e7f4905839deb4b22955ef5e73d2fea1b9813cb23" +dependencies = [ + "num-bigint", + "num-complex", + "num-integer", + "num-iter", + "num-rational", + "num-traits", +] + [[package]] name = "num-bigint" version = "0.4.6" @@ -3510,6 +3483,28 @@ dependencies = [ "num-traits", ] +[[package]] +name = "num-iter" +version = "0.1.45" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "1429034a0490724d0075ebb2bc9e875d6503c3cf69e235a8941aa757d83ef5bf" +dependencies = [ + "autocfg", + "num-integer", + "num-traits", +] + +[[package]] +name = "num-rational" +version = "0.4.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "f83d14da390562dca69fc84082e73e548e1ad308d24accdedd2720017cb37824" +dependencies = [ + "num-bigint", + "num-integer", + "num-traits", +] + [[package]] name = "num-traits" version = "0.2.19" @@ -3645,9 +3640,9 @@ dependencies = [ [[package]] name = "parquet" -version = "57.1.0" +version = "56.2.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "be3e4f6d320dd92bfa7d612e265d7d08bba0a240bab86af3425e1d255a511d89" +checksum = "f0dbd48ad52d7dccf8ea1b90a3ddbfaea4f69878dd7683e51c507d4bc52b5b27" dependencies = [ "ahash", "arrow-array", @@ -3666,17 +3661,17 @@ dependencies = [ "half", "hashbrown 0.16.0", "lz4_flex", + "num", "num-bigint", - "num-integer", - "num-traits", "object_store", "paste", + "ring", "seq-macro", "simdutf8", "snap", "thrift", "tokio", - "twox-hash", + "twox-hash 2.1.0", "zstd", ] @@ -3817,7 +3812,7 @@ version = "0.2.21" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "85eae3c4ed2f50dcfe72643da4befc30deadb458a9b590d720cde2f2b1e97da9" dependencies = [ - "zerocopy 0.8.31", + "zerocopy 0.8.23", ] [[package]] @@ -3830,15 +3825,6 @@ dependencies = [ "syn", ] -[[package]] -name = "proc-macro-crate" -version = "3.4.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "219cb19e96be00ab2e37d6e299658a0cfa83e52429179969b0f0121b4ac46983" -dependencies = [ - "toml_edit", -] - [[package]] name = "proc-macro-error-attr2" version = "2.0.0" @@ -3891,9 +3877,9 @@ dependencies = [ [[package]] name = "prost" -version = "0.14.1" +version = "0.13.5" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "7231bd9b3d3d33c86b58adbac74b5ec0ad9f496b19d22801d773636feaa95f3d" +checksum = "2796faa41db3ec313a31f7624d9286acf277b52de526150b7e69f3debf891ee5" dependencies = [ "bytes", "prost-derive", @@ -3901,9 +3887,9 @@ dependencies = [ [[package]] name = "prost-build" -version = "0.14.1" +version = "0.13.5" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "ac6c3320f9abac597dcbc668774ef006702672474aad53c6d596b62e487b40b1" +checksum = "be769465445e8c1474e9c5dac2018218498557af32d9ed057325ec9a41ae81bf" dependencies = [ "heck", "itertools 0.14.0", @@ -3921,9 +3907,9 @@ dependencies = [ [[package]] name = "prost-derive" -version = "0.14.1" +version = "0.13.5" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "9120690fafc389a67ba3803df527d0ec9cbbc9cc45e4cc20b332996dfb672425" +checksum = "8a56d757972c98b346a9b766e3f02746cde6dd1cd1d1d563472929fdd74bec4d" dependencies = [ "anyhow", "itertools 0.14.0", @@ -3934,9 +3920,9 @@ dependencies = [ [[package]] name = "prost-types" -version = "0.14.1" +version = "0.13.5" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "b9b4db3d6da204ed77bb26ba83b6122a73aeb2e87e25fbf7ad2e84c4ccbf8f72" +checksum = "52c2c1bf36ddb1a1c396b3601a3cec27c2462e45f07c386894ec3ccf5332bd16" dependencies = [ "prost", ] @@ -4020,9 +4006,9 @@ dependencies = [ [[package]] name = "quote" -version = "1.0.42" +version = "1.0.40" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a338cc41d27e6cc6dce6cefc13a0729dfbb81c262b1f519331575dd80ef3067f" +checksum = "1885c039570dc00dcb4ff087a89e185fd56bae234ddc7f056a945bf36467248d" dependencies = [ "proc-macro2", ] @@ -4147,9 +4133,9 @@ dependencies = [ [[package]] name = "regex" -version = "1.12.2" +version = "1.11.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "843bc0191f75f3e22651ae5f1e72939ab2f72a4bc30fa80a066bd66edefc24d4" +checksum = "b544ef1b4eac5dc2db33ea63606ae9ffcfac26c1416a2806ae0bf5f56b201191" dependencies = [ "aho-corasick", "memchr", @@ -4159,9 +4145,9 @@ dependencies = [ [[package]] name = "regex-automata" -version = "0.4.13" +version = "0.4.9" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "5276caf25ac86c8d810222b3dbb938e512c55c6831a10f3e6ed1c93b84041f1c" +checksum = "809e8dc61f6de73b46c85f4c96486310fe304c434cfa43669d7b40f711150908" dependencies = [ "aho-corasick", "memchr", @@ -4180,12 +4166,6 @@ version = "0.8.8" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "7a2d987857b319362043e95f5353c0535c1f58eec5336fdfcf626430af7def58" -[[package]] -name = "relative-path" -version = "1.9.3" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "ba39f3699c378cd8970968dcbff9c43159ea4cfbd88d43c00b22f2ef10a435d2" - [[package]] name = "reqwest" version = "0.12.24" @@ -4252,35 +4232,6 @@ dependencies = [ "byteorder", ] -[[package]] -name = "rstest" -version = "0.26.1" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "f5a3193c063baaa2a95a33f03035c8a72b83d97a54916055ba22d35ed3839d49" -dependencies = [ - "futures-timer", - "futures-util", - "rstest_macros", -] - -[[package]] -name = "rstest_macros" -version = "0.26.1" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "9c845311f0ff7951c5506121a9ad75aec44d083c31583b2ea5a30bcb0b0abba0" -dependencies = [ - "cfg-if", - "glob", - "proc-macro-crate", - "proc-macro2", - "quote", - "regex", - "relative-path", - "rustc_version", - "syn", - "unicode-ident", -] - [[package]] name = "rustc-hash" version = "2.1.1" @@ -4311,15 +4262,15 @@ dependencies = [ [[package]] name = "rustix" -version = "1.1.3" +version = "1.0.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "146c9e247ccc180c1f61615433868c99f3de3ae256a30a43b49f67c2d9171f34" +checksum = "f7178faa4b75a30e269c71e61c353ce2748cf3d76f0c44c393f4e60abf49b825" dependencies = [ "bitflags", "errno", "libc", - "linux-raw-sys 0.11.0", - "windows-sys 0.61.2", + "linux-raw-sys 0.9.2", + "windows-sys 0.59.0", ] [[package]] @@ -4583,21 +4534,22 @@ dependencies = [ [[package]] name = "serde_json" -version = "1.0.140" +version = "1.0.149" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "20068b6e96dc6c9bd23e01df8827e6c7e1f2fddd43c21810382803c136b99373" +checksum = "83fc039473c5595ace860d8c4fafa220ff474b3fc6bfdb4293327f1a37e94d86" dependencies = [ "itoa", "memchr", - "ryu", "serde", + "serde_core", + "zmij", ] [[package]] name = "serde_spanned" -version = "1.0.4" +version = "1.0.3" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "f8bbf91e5a4d6315eee45e704372590b30e260ee83af6639d64557f51b067776" +checksum = "e24345aa0fe688594e73770a5f6d1b216508b4f93484c0026d521acd30134392" dependencies = [ "serde_core", ] @@ -4727,6 +4679,17 @@ dependencies = [ "windows-sys 0.60.2", ] +[[package]] +name = "sqlparser" +version = "0.58.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "ec4b661c54b1e4b603b37873a18c59920e4c51ea8ea2cf527d925424dbd4437c" +dependencies = [ + "log", + "recursive", + "sqlparser_derive", +] + [[package]] name = "sqlparser" version = "0.59.0" @@ -4735,7 +4698,6 @@ checksum = "4591acadbcf52f0af60eafbb2c003232b2b4cd8de5f0e9437cb8b1b59046cc0f" dependencies = [ "log", "recursive", - "sqlparser_derive", ] [[package]] @@ -4874,14 +4836,14 @@ dependencies = [ [[package]] name = "tempfile" -version = "3.24.0" +version = "3.23.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "655da9c7eb6305c55742045d5a8d2037996d61d8de95806335c7c86ce0f82e9c" +checksum = "2d31c77bdf42a745371d260a26ca7163f1e0924b64afa0b688e61b5a9fa02f16" dependencies = [ "fastrand", "getrandom 0.3.1", "once_cell", - "rustix 1.1.3", + "rustix 1.0.2", "windows-sys 0.61.2", ] @@ -5064,9 +5026,9 @@ dependencies = [ [[package]] name = "toml" -version = "0.9.10+spec-1.1.0" +version = "0.9.8" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "0825052159284a1a8b4d6c0c86cbc801f2da5afd2b225fa548c72f2e74002f48" +checksum = "f0dc8b1fb61449e27716ec0e1bdf0f6b8f3e8f6b05391e8497b8b6d7804ea6d8" dependencies = [ "indexmap", "serde_core", @@ -5079,45 +5041,33 @@ dependencies = [ [[package]] name = "toml_datetime" -version = "0.7.5+spec-1.1.0" +version = "0.7.3" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "92e1cfed4a3038bc5a127e35a2d360f145e1f4b971b551a2ba5fd7aedf7e1347" +checksum = "f2cdb639ebbc97961c51720f858597f7f24c4fc295327923af55b74c3c724533" dependencies = [ "serde_core", ] -[[package]] -name = "toml_edit" -version = "0.23.10+spec-1.0.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "84c8b9f757e028cee9fa244aea147aab2a9ec09d5325a9b01e0a49730c2b5269" -dependencies = [ - "indexmap", - "toml_datetime", - "toml_parser", - "winnow", -] - [[package]] name = "toml_parser" -version = "1.0.6+spec-1.1.0" +version = "1.0.4" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a3198b4b0a8e11f09dd03e133c0280504d0801269e9afa46362ffde1cbeebf44" +checksum = "c0cbe268d35bdb4bb5a56a2de88d0ad0eb70af5384a99d648cd4b3d04039800e" dependencies = [ "winnow", ] [[package]] name = "toml_writer" -version = "1.0.6+spec-1.1.0" +version = "1.0.4" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "ab16f14aed21ee8bfd8ec22513f7287cd4a91aa92e44edfe2c17ddd004e92607" +checksum = "df8b2b54733674ad286d16267dcfc7a71ed5c776e4ac7aa3c3e2561f7c637bf2" [[package]] name = "tonic" -version = "0.14.2" +version = "0.13.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "eb7613188ce9f7df5bfe185db26c5814347d110db17920415cf2fbcad85e7203" +checksum = "7e581ba15a835f4d9ea06c55ab1bd4dce26fc53752c69a04aac00703bfb49ba9" dependencies = [ "async-trait", "axum", @@ -5132,8 +5082,8 @@ dependencies = [ "hyper-util", "percent-encoding", "pin-project", - "socket2 0.6.1", - "sync_wrapper", + "prost", + "socket2 0.5.8", "tokio", "tokio-stream", "tower", @@ -5142,17 +5092,6 @@ dependencies = [ "tracing", ] -[[package]] -name = "tonic-prost" -version = "0.14.2" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "66bd50ad6ce1252d87ef024b3d64fe4c3cf54a86fb9ef4c631fdd0ded7aeaa67" -dependencies = [ - "bytes", - "prost", - "tonic", -] - [[package]] name = "tower" version = "0.5.2" @@ -5204,9 +5143,9 @@ checksum = "8df9b6e13f2d32c91b9bd719c00d1958837bc7dec474d94952798cc8e69eeec3" [[package]] name = "tracing" -version = "0.1.44" +version = "0.1.41" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "63e71662fa4b2a2c3a26f570f037eb95bb1f85397f3cd8076caed2f026a6d100" +checksum = "784e0ac535deb450455cbfa28a6f0df145ea1bb7ae51b821cf5e7927fdcfbdd0" dependencies = [ "log", "pin-project-lite", @@ -5216,9 +5155,9 @@ dependencies = [ [[package]] name = "tracing-attributes" -version = "0.1.31" +version = "0.1.28" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "7490cfa5ec963746568740651ac6781f701c9c5ea257c58e057f3ba8cf69e8da" +checksum = "395ae124c09f9e6918a2310af6038fba074bcf474ac352496d5910dd59a2226d" dependencies = [ "proc-macro2", "quote", @@ -5227,9 +5166,9 @@ dependencies = [ [[package]] name = "tracing-core" -version = "0.1.36" +version = "0.1.33" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "db97caf9d906fbde555dd62fa95ddba9eecfd14cb388e4f491a66d74cd5fb79a" +checksum = "e672c95779cf947c5311f83787af4fa8fffd12fb27e4993211a84bdfd9610f9c" dependencies = [ "once_cell", "valuable", @@ -5248,9 +5187,9 @@ dependencies = [ [[package]] name = "tracing-subscriber" -version = "0.3.22" +version = "0.3.20" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "2f30143827ddab0d256fd843b7a66d164e9f271cfa0dde49142c5ca0ca291f1e" +checksum = "2054a14f5307d601f88daf0553e1cbf472acc4f2c51afab632431cdcd72124d5" dependencies = [ "nu-ansi-term", "sharded-slab", @@ -5268,29 +5207,19 @@ checksum = "e421abadd41a4225275504ea4d6566923418b7f05506fbc9c0fe86ba7396114b" [[package]] name = "twox-hash" -version = "2.1.0" +version = "1.6.3" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "e7b17f197b3050ba473acf9181f7b1d3b66d1cf7356c6cc57886662276e65908" - -[[package]] -name = "typed-builder" -version = "0.23.2" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "31aa81521b70f94402501d848ccc0ecaa8f93c8eb6999eb9747e72287757ffda" +checksum = "97fee6b57c6a41524a810daee9286c02d7752c4253064d0b05472833a438f675" dependencies = [ - "typed-builder-macro", + "cfg-if", + "static_assertions", ] [[package]] -name = "typed-builder-macro" -version = "0.23.2" +name = "twox-hash" +version = "2.1.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "076a02dc54dd46795c2e9c8282ed40bcfb1e22747e955de9389a1de28190fb26" -dependencies = [ - "proc-macro2", - "quote", - "syn", -] +checksum = "e7b17f197b3050ba473acf9181f7b1d3b66d1cf7356c6cc57886662276e65908" [[package]] name = "typenum" @@ -5366,14 +5295,14 @@ checksum = "06abde3611657adf66d383f00b093d7faecc7fa57071cce2578660c9f1010821" [[package]] name = "uuid" -version = "1.19.0" +version = "1.18.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "e2e054861b4bd027cd373e18e8d8d8e6548085000e41290d95ce0c373a654b4a" +checksum = "2f87b8aa10b915a06587d0dec516c282ff295b475d94abf425d62b57710070a2" dependencies = [ "getrandom 0.3.1", "js-sys", "rand 0.9.2", - "serde_core", + "serde", "wasm-bindgen", ] @@ -5899,9 +5828,6 @@ name = "winnow" version = "0.7.14" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "5a5364e9d77fcdeeaa6062ced926ee3381faa2ee02d3eb83a5c27a8825540829" -dependencies = [ - "memchr", -] [[package]] name = "wit-bindgen-rt" @@ -5980,11 +5906,11 @@ dependencies = [ [[package]] name = "zerocopy" -version = "0.8.31" +version = "0.8.23" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "fd74ec98b9250adb3ca554bdde269adf631549f51d8a8f8f0a10b50f1cb298c3" +checksum = "fd97444d05a4328b90e75e503a34bad781f14e28a823ad3557f0750df1ebcbc6" dependencies = [ - "zerocopy-derive 0.8.31", + "zerocopy-derive 0.8.23", ] [[package]] @@ -6000,9 +5926,9 @@ dependencies = [ [[package]] name = "zerocopy-derive" -version = "0.8.31" +version = "0.8.23" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d8a8d209fdf45cf5138cbb5a506f6b52522a25afccc534d1475dad8e31105c6a" +checksum = "6352c01d0edd5db859a63e2605f4ea3183ddbd15e2c4a9e7d32184df75e4f154" dependencies = [ "proc-macro2", "quote", @@ -6064,6 +5990,12 @@ version = "0.5.2" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "2f06ae92f42f5e5c42443fd094f245eb656abf56dd7cce9b8b263236565e00f2" +[[package]] +name = "zmij" +version = "1.0.21" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "b8848ee67ecc8aedbaf3e4122217aff892639231befc6a1b58d29fff4c2cabaa" + [[package]] name = "zstd" version = "0.13.3" diff --git a/Cargo.toml b/Cargo.toml index 8da56f28..aaa68c81 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -39,6 +39,7 @@ prost-build = "0.14.1" rand = "0.9.2" rustyline = "17.0.2" serde = { version = "1.0.228", features = ["derive"] } +serde_json = "1.0.149" snmalloc-rs = "0.3.8" sqlparser = "0.59.0" sysinfo = "0.37.2" diff --git a/crates/modelardb_storage/Cargo.toml b/crates/modelardb_storage/Cargo.toml index 9e027ac4..d3381d04 100644 --- a/crates/modelardb_storage/Cargo.toml +++ b/crates/modelardb_storage/Cargo.toml @@ -33,6 +33,7 @@ futures.workspace = true modelardb_compression = { path = "../modelardb_compression" } modelardb_types = { path = "../modelardb_types" } object_store = { workspace = true, features = ["aws", "azure"] } +serde_json.workspace = true sqlparser.workspace = true tonic.workspace = true url.workspace = true From 5bfaf0ca8448101f4535ea91e11bcbfe8daccc3a Mon Sep 17 00:00:00 2001 From: CGodiksen <36046286+CGodiksen@users.noreply.github.com> Date: Tue, 10 Mar 2026 12:41:57 +0100 Subject: [PATCH 031/116] Add batch ids to DeltaTableWriter --- .../src/operations/data_folder.rs | 20 +++++++++++--- crates/modelardb_server/src/context.rs | 1 + .../src/storage/compressed_data_manager.rs | 2 +- .../src/storage/data_transfer.rs | 3 ++- .../modelardb_storage/src/data_folder/mod.rs | 27 +++++++++++++++++-- 5 files changed, 45 insertions(+), 8 deletions(-) diff --git a/crates/modelardb_embedded/src/operations/data_folder.rs b/crates/modelardb_embedded/src/operations/data_folder.rs index cf056765..1f5f3375 100644 --- a/crates/modelardb_embedded/src/operations/data_folder.rs +++ b/crates/modelardb_embedded/src/operations/data_folder.rs @@ -221,8 +221,12 @@ impl Operations for DataFolder { &uncompressed_data, )?; - self.write_compressed_segments_to_time_series_table(table_name, compressed_data) - .await?; + self.write_compressed_segments_to_time_series_table( + table_name, + compressed_data, + vec![], + ) + .await?; } else if let Some(normal_table_schema) = self.normal_table_schema(table_name).await { // Normal table. if !schemas_are_compatible(&uncompressed_data.schema(), &normal_table_schema) { @@ -413,7 +417,11 @@ impl Operations for DataFolder { // Write read data to target_table_name in target. target_data_folder - .write_compressed_segments_to_time_series_table(target_table_name, record_batches) + .write_compressed_segments_to_time_series_table( + target_table_name, + record_batches, + vec![], + ) .await?; Ok(()) @@ -461,7 +469,11 @@ impl Operations for DataFolder { let record_batches: Vec = stream.try_collect().await?; target_data_folder - .write_compressed_segments_to_time_series_table(target_table_name, record_batches) + .write_compressed_segments_to_time_series_table( + target_table_name, + record_batches, + vec![], + ) .await?; } else if let (Some(source_normal_table_schema), Some(target_normal_table_schema)) = ( self.normal_table_schema(source_table_name).await, diff --git a/crates/modelardb_server/src/context.rs b/crates/modelardb_server/src/context.rs index 85a5795c..9407a821 100644 --- a/crates/modelardb_server/src/context.rs +++ b/crates/modelardb_server/src/context.rs @@ -858,6 +858,7 @@ mod tests { .write_compressed_segments_to_time_series_table( TIME_SERIES_TABLE_NAME, vec![table::compressed_segments_record_batch()], + vec![], ) .await .unwrap(); diff --git a/crates/modelardb_server/src/storage/compressed_data_manager.rs b/crates/modelardb_server/src/storage/compressed_data_manager.rs index e8667cab..bdda20f8 100644 --- a/crates/modelardb_server/src/storage/compressed_data_manager.rs +++ b/crates/modelardb_server/src/storage/compressed_data_manager.rs @@ -246,7 +246,7 @@ impl CompressedDataManager { let compressed_data_buffer_size_in_bytes = compressed_data_buffer.size_in_bytes; let compressed_segments = compressed_data_buffer.record_batches(); self.local_data_folder - .write_compressed_segments_to_time_series_table(table_name, compressed_segments) + .write_compressed_segments_to_time_series_table(table_name, compressed_segments, vec![]) .await?; // Inform the data transfer component about the new data if a remote data folder was diff --git a/crates/modelardb_server/src/storage/data_transfer.rs b/crates/modelardb_server/src/storage/data_transfer.rs index b61b1557..b0d3d808 100644 --- a/crates/modelardb_server/src/storage/data_transfer.rs +++ b/crates/modelardb_server/src/storage/data_transfer.rs @@ -250,7 +250,7 @@ impl DataTransfer { .await? { self.remote_data_folder - .write_compressed_segments_to_time_series_table(table_name, record_batches) + .write_compressed_segments_to_time_series_table(table_name, record_batches, vec![]) .await?; } else { self.remote_data_folder @@ -515,6 +515,7 @@ mod tests { .write_compressed_segments_to_time_series_table( TIME_SERIES_TABLE_NAME, vec![table::compressed_segments_record_batch()], + vec![], ) .await .unwrap(); diff --git a/crates/modelardb_storage/src/data_folder/mod.rs b/crates/modelardb_storage/src/data_folder/mod.rs index 207848c5..af508d1a 100644 --- a/crates/modelardb_storage/src/data_folder/mod.rs +++ b/crates/modelardb_storage/src/data_folder/mod.rs @@ -58,6 +58,7 @@ use object_store::aws::AmazonS3Builder; use object_store::local::LocalFileSystem; use object_store::memory::InMemory; use object_store::path::Path; +use serde_json::json; use url::Url; use uuid::Uuid; @@ -873,15 +874,22 @@ impl DataFolder { } /// Write `compressed_segments` to a Delta Lake table for a time series table with `table_name`. + /// The `batch_ids` from the WAL are included in the commit metadata for checkpointing. /// Returns an updated [`DeltaTable`] if the file was written successfully, otherwise returns /// [`ModelarDbStorageError`]. pub async fn write_compressed_segments_to_time_series_table( &self, table_name: &str, compressed_segments: Vec, + batch_ids: Vec, ) -> Result { let delta_table = self.delta_table(table_name).await?; - let delta_table_writer = self.time_series_table_writer(delta_table).await?; + + let delta_table_writer = self + .time_series_table_writer(delta_table) + .await? + .with_batch_ids(batch_ids); + self.write_record_batches_to_table(delta_table_writer, compressed_segments) .await } @@ -1139,6 +1147,8 @@ pub struct DeltaTableWriter { operation_id: Uuid, /// Writes record batches to the Delta table as Apache Parquet files. delta_writer: DeltaWriter, + /// Batch IDs from the WAL to include in the commit metadata for checkpointing. + batch_ids: Vec, } impl DeltaTableWriter { @@ -1191,9 +1201,16 @@ impl DeltaTableWriter { delta_operation, operation_id, delta_writer, + batch_ids: vec![], }) } + /// Add batch IDs from the WAL that are included in the commit metadata for checkpointing. + pub fn with_batch_ids(mut self, batch_ids: Vec) -> Self { + self.batch_ids = batch_ids; + self + } + /// Write `record_batch` to the Delta table. Returns a [`ModelarDbStorageError`] if the /// [`RecordBatches`](RecordBatch) does not match the schema of the Delta table or if the /// writing fails. @@ -1231,7 +1248,13 @@ impl DeltaTableWriter { // Prepare all inputs to the commit. let object_store = self.delta_table.object_store(); - let commit_properties = CommitProperties::default(); + + let mut commit_properties = CommitProperties::default(); + if !self.batch_ids.is_empty() { + commit_properties = commit_properties + .with_metadata(vec![("batchIds".to_string(), json!(self.batch_ids))]); + } + let table_data = match self.delta_table.snapshot() { Ok(table_data) => table_data, Err(delta_table_error) => { From ed74f3bffbee2cc8ec95c9b3d4ee407241e88446 Mon Sep 17 00:00:00 2001 From: CGodiksen <36046286+CGodiksen@users.noreply.github.com> Date: Tue, 10 Mar 2026 13:25:14 +0100 Subject: [PATCH 032/116] Add WAL batch id to IngestedDataBuffer so it is passed to uncompressed data manager --- crates/modelardb_server/src/storage/mod.rs | 3 ++- .../src/storage/uncompressed_data_buffer.rs | 4 ++++ crates/modelardb_storage/src/data_folder/mod.rs | 4 ++-- crates/modelardb_storage/src/write_ahead_log.rs | 14 +++++++------- 4 files changed, 15 insertions(+), 10 deletions(-) diff --git a/crates/modelardb_server/src/storage/mod.rs b/crates/modelardb_server/src/storage/mod.rs index 569d923b..6f78ccf3 100644 --- a/crates/modelardb_server/src/storage/mod.rs +++ b/crates/modelardb_server/src/storage/mod.rs @@ -267,7 +267,7 @@ impl StorageEngine { // Write to the write-ahead log to ensure termination never duplicates or loses data. We use // a read lock since the specific log file is locked internally before writing. let write_ahead_log = self.write_ahead_log.read().await; - write_ahead_log + let batch_id = write_ahead_log .append_to_table_log(&time_series_table_metadata.name, &multivariate_data_points)?; self.memory_pool @@ -278,6 +278,7 @@ impl StorageEngine { .send(Message::Data(IngestedDataBuffer::new( time_series_table_metadata, multivariate_data_points, + batch_id, ))) .map_err(|error| error.into()) } diff --git a/crates/modelardb_server/src/storage/uncompressed_data_buffer.rs b/crates/modelardb_server/src/storage/uncompressed_data_buffer.rs index 92e7a547..693d8d5e 100644 --- a/crates/modelardb_server/src/storage/uncompressed_data_buffer.rs +++ b/crates/modelardb_server/src/storage/uncompressed_data_buffer.rs @@ -45,16 +45,20 @@ pub(super) struct IngestedDataBuffer { pub(super) time_series_table_metadata: Arc, /// Uncompressed data points to insert. pub(super) data_points: RecordBatch, + /// The id given to the [`RecordBatch`] by the WAL. + pub(super) batch_id: u64, } impl IngestedDataBuffer { pub(super) fn new( time_series_table_metadata: Arc, data_points: RecordBatch, + batch_id: u64, ) -> Self { Self { time_series_table_metadata, data_points, + batch_id, } } } diff --git a/crates/modelardb_storage/src/data_folder/mod.rs b/crates/modelardb_storage/src/data_folder/mod.rs index af508d1a..2f40dc0c 100644 --- a/crates/modelardb_storage/src/data_folder/mod.rs +++ b/crates/modelardb_storage/src/data_folder/mod.rs @@ -1147,7 +1147,7 @@ pub struct DeltaTableWriter { operation_id: Uuid, /// Writes record batches to the Delta table as Apache Parquet files. delta_writer: DeltaWriter, - /// Batch IDs from the WAL to include in the commit metadata for checkpointing. + /// Batch ids from the WAL to include in the commit metadata for checkpointing. batch_ids: Vec, } @@ -1205,7 +1205,7 @@ impl DeltaTableWriter { }) } - /// Add batch IDs from the WAL that are included in the commit metadata for checkpointing. + /// Add batch ids from the WAL that are included in the commit metadata for checkpointing. pub fn with_batch_ids(mut self, batch_ids: Vec) -> Self { self.batch_ids = batch_ids; self diff --git a/crates/modelardb_storage/src/write_ahead_log.rs b/crates/modelardb_storage/src/write_ahead_log.rs index 1f26f482..6687748e 100644 --- a/crates/modelardb_storage/src/write_ahead_log.rs +++ b/crates/modelardb_storage/src/write_ahead_log.rs @@ -130,9 +130,9 @@ impl WriteAheadLog { /// Append data to the log for the given table and sync the file to ensure that all data is on /// disk. Only requires read access to the log since the internal Mutex handles write - /// synchronization. If a table log does not exist or the data could not be appended, return - /// [`ModelarDbStorageError`]. - pub fn append_to_table_log(&self, table_name: &str, data: &RecordBatch) -> Result<()> { + /// synchronization. Return the batch id given to the appended data. If a table log does not + /// exist or the data could not be appended, return [`ModelarDbStorageError`]. + pub fn append_to_table_log(&self, table_name: &str, data: &RecordBatch) -> Result { let log_file = self.table_logs.get(table_name).ok_or_else(|| { ModelarDbStorageError::InvalidState(format!( "Table log for table '{table_name}' does not exist." @@ -183,9 +183,9 @@ impl WriteAheadLogFile { } /// Append the given data to the log file and sync the file to ensure that all data is on disk. - /// If the data could not be appended or the file could not be synced, return - /// [`ModelarDbStorageError`]. - fn append_and_sync(&self, data: &RecordBatch) -> Result<()> { + /// Return the batch id given to the appended data. If the data could not be appended or the + /// file could not be synced, return [`ModelarDbStorageError`]. + fn append_and_sync(&self, data: &RecordBatch) -> Result { // Acquire the mutex to ensure only one thread can write at a time. let mut writer = self.writer.lock().expect("Mutex should not be poisoned."); @@ -198,7 +198,7 @@ impl WriteAheadLogFile { // such as modification timestamps and permissions are not updated since we only sync data. writer.get_ref().sync_data()?; - Ok(()) + Ok(0) } /// Read all data from the log file. This can be called even if the [`StreamWriter`] has not From 92e576f1bf53d8c501d38ac5acabc610fd665df3 Mon Sep 17 00:00:00 2001 From: CGodiksen <36046286+CGodiksen@users.noreply.github.com> Date: Tue, 10 Mar 2026 14:48:36 +0100 Subject: [PATCH 033/116] No longer read spilled buffers back into memory on startup --- crates/modelardb_server/src/main.rs | 8 - crates/modelardb_server/src/storage/mod.rs | 21 +-- .../src/storage/uncompressed_data_buffer.rs | 22 --- .../src/storage/uncompressed_data_manager.rs | 145 +++--------------- 4 files changed, 31 insertions(+), 165 deletions(-) diff --git a/crates/modelardb_server/src/main.rs b/crates/modelardb_server/src/main.rs index be10e170..d2eb9644 100644 --- a/crates/modelardb_server/src/main.rs +++ b/crates/modelardb_server/src/main.rs @@ -86,14 +86,6 @@ async fn main() -> Result<()> { // Setup CTRL+C handler. setup_ctrl_c_handler(&context); - // Initialize storage engine with spilled buffers. - context - .storage_engine - .read() - .await - .initialize(&context) - .await?; - // Start the Apache Arrow Flight interface. remote::start_apache_arrow_flight_server(context, *PORT).await?; diff --git a/crates/modelardb_server/src/storage/mod.rs b/crates/modelardb_server/src/storage/mod.rs index 6f78ccf3..c8633340 100644 --- a/crates/modelardb_server/src/storage/mod.rs +++ b/crates/modelardb_server/src/storage/mod.rs @@ -41,7 +41,6 @@ use tokio::sync::RwLock; use tracing::error; use crate::configuration::ConfigurationManager; -use crate::context::Context; use crate::data_folders::DataFolders; use crate::error::{ModelarDbServerError, Result}; use crate::storage::compressed_data_manager::CompressedDataManager; @@ -113,11 +112,14 @@ impl StorageEngine { let channels = Arc::new(Channels::new()); // Create the uncompressed data manager. - let uncompressed_data_manager = Arc::new(UncompressedDataManager::new( - data_folders.local_data_folder.clone(), - memory_pool.clone(), - channels.clone(), - )); + let uncompressed_data_manager = Arc::new( + UncompressedDataManager::try_new( + data_folders.local_data_folder.clone(), + memory_pool.clone(), + channels.clone(), + ) + .await?, + ); { let runtime_handle = runtime_handle.clone(); @@ -238,13 +240,6 @@ impl StorageEngine { Ok(()) } - /// Add references to the - /// [`UncompressedDataBuffers`](uncompressed_data_buffer::UncompressedDataBuffer) currently on - /// disk to [`UncompressedDataManager`] which immediately will start compressing them. - pub(super) async fn initialize(&self, context: &Context) -> Result<()> { - self.uncompressed_data_manager.initialize(context).await - } - /// Pass `record_batch` to [`CompressedDataManager`]. Return [`Ok`] if `record_batch` was /// successfully written to an Apache Parquet file, otherwise return [`ModelarDbServerError`]. pub(super) async fn insert_record_batch( diff --git a/crates/modelardb_server/src/storage/uncompressed_data_buffer.rs b/crates/modelardb_server/src/storage/uncompressed_data_buffer.rs index 693d8d5e..2eec5a1a 100644 --- a/crates/modelardb_server/src/storage/uncompressed_data_buffer.rs +++ b/crates/modelardb_server/src/storage/uncompressed_data_buffer.rs @@ -298,28 +298,6 @@ impl UncompressedOnDiskDataBuffer { }) } - /// Return an [`UncompressedOnDiskDataBuffer`] with the data points for `tag_hash` in - /// `file_path` if a file at `file_path` exists, otherwise - /// [`ModelarDbServerError`](crate::error::ModelarDbServerError) is returned. - pub(super) fn try_new( - tag_hash: u64, - time_series_table_metadata: Arc, - updated_by_batch_index: u64, - local_data_folder: Arc, - file_name: &str, - ) -> Result { - let file_path = - spilled_buffer_file_path(&time_series_table_metadata.name, tag_hash, file_name); - - Ok(Self { - tag_hash, - time_series_table_metadata, - updated_by_batch_index, - local_data_folder, - file_path, - }) - } - /// Read the data from the Apache Parquet file, delete the Apache Parquet file, and return the /// data as a [`RecordBatch`] sorted by time. Return /// [`ModelarDbServerError`](crate::error::ModelarDbServerError) if the Apache Parquet file diff --git a/crates/modelardb_server/src/storage/uncompressed_data_manager.rs b/crates/modelardb_server/src/storage/uncompressed_data_manager.rs index a7274e81..a78b3259 100644 --- a/crates/modelardb_server/src/storage/uncompressed_data_manager.rs +++ b/crates/modelardb_server/src/storage/uncompressed_data_manager.rs @@ -23,14 +23,13 @@ use std::sync::Arc; use std::sync::atomic::{AtomicU64, Ordering}; use dashmap::DashMap; -use futures::StreamExt; +use futures::{StreamExt, TryStreamExt}; use modelardb_storage::data_folder::DataFolder; use modelardb_types::types::{TimeSeriesTableMetadata, Timestamp, Value}; use object_store::path::{Path, PathPart}; use tokio::runtime::Handle; use tracing::{debug, error, warn}; -use crate::context::Context; use crate::error::Result; use crate::storage::UNCOMPRESSED_DATA_FOLDER; use crate::storage::compressed_data_buffer::CompressedSegmentBatch; @@ -68,68 +67,34 @@ pub(super) struct UncompressedDataManager { } impl UncompressedDataManager { - pub(super) fn new( + /// Create a new [`UncompressedDataManager`] and delete all existing spilled buffers if + /// necessary. If the existing buffers could not be deleted, return + /// [`ModelarDbServerError`](crate::error::ModelarDbServerError). + pub(super) async fn try_new( local_data_folder: DataFolder, memory_pool: Arc, channels: Arc, - ) -> Self { - Self { + ) -> Result { + // Delete the previously spilled on disk data buffers if they exist. + let object_store = local_data_folder.object_store(); + let spilled_buffers = object_store + .list(Some(&Path::from(UNCOMPRESSED_DATA_FOLDER))) + .map_ok(|object_meta| object_meta.location) + .boxed(); + + object_store + .delete_stream(spilled_buffers) + .try_collect::>() + .await?; + + Ok(Self { local_data_folder, current_batch_index: AtomicU64::new(0), uncompressed_in_memory_data_buffers: DashMap::new(), uncompressed_on_disk_data_buffers: DashMap::new(), channels, memory_pool, - } - } - - /// Add references to the [`UncompressedDataBuffers`](UncompressedDataBuffer) currently on disk - /// to [`UncompressedDataManager`] which immediately will start compressing them. - pub(super) async fn initialize(&self, context: &Context) -> Result<()> { - let local_data_folder = self.local_data_folder.object_store(); - let mut spilled_buffers = - local_data_folder.list(Some(&Path::from(UNCOMPRESSED_DATA_FOLDER))); - - while let Some(maybe_spilled_buffer) = spilled_buffers.next().await { - let spilled_buffer = maybe_spilled_buffer?; - let path_parts: Vec = spilled_buffer.location.parts().collect(); - - let table_name = path_parts - .get(1) - .expect("The spilled buffers should be partitioned by their table name.") - .as_ref(); - - let tag_hash = path_parts - .get(2) - .expect("The spilled buffers should be partitioned by their tag hash.") - .as_ref() - .parse::() - .unwrap(); - - let file_name = path_parts - .get(3) - .expect("The spilled buffers should have an auto-generated file name.") - .as_ref(); - - let time_series_table_metadata = context - .time_series_table_metadata_from_default_database_schema(table_name) - .await? - .expect("The time series table for the spilled buffer should exist."); - - let buffer = UncompressedOnDiskDataBuffer::try_new( - tag_hash, - time_series_table_metadata, - self.current_batch_index.load(Ordering::Relaxed), - local_data_folder.clone(), - file_name, - )?; - - self.channels - .uncompressed_data_sender - .send(Message::Data(UncompressedDataBuffer::OnDisk(buffer)))?; - } - - Ok(()) + }) } /// Read and process messages received from the [`StorageEngine`](super::StorageEngine) to @@ -670,79 +635,13 @@ mod tests { use object_store::local::LocalFileSystem; use tempfile::TempDir; use tokio::runtime::Runtime; - use tokio::time::{Duration, sleep}; use crate::storage::UNCOMPRESSED_DATA_BUFFER_CAPACITY; - use crate::{ClusterMode, DataFolders}; const TAG_VALUE: &str = "tag"; const TAG_HASH: u64 = 14957893031159457585; // Tests for UncompressedDataManager. - #[tokio::test] - async fn test_can_compress_existing_on_disk_data_buffers_when_initializing() { - let temp_dir = tempfile::tempdir().unwrap(); - let temp_dir_url = temp_dir.path().to_str().unwrap(); - let local_data_folder = DataFolder::open_local_url(temp_dir_url).await.unwrap(); - - // Create a context with a storage engine. - let context = Arc::new( - Context::try_new( - DataFolders::new(local_data_folder.clone(), None, local_data_folder), - ClusterMode::SingleNode, - ) - .await - .unwrap(), - ); - - // Create a time series table in the context. - let time_series_table_metadata = Arc::new(table::time_series_table_metadata()); - context - .create_time_series_table(&time_series_table_metadata) - .await - .unwrap(); - - // Ingest a single data point and sleep to allow the ingestion thread to finish. - let mut storage_engine = context.storage_engine.write().await; - let data = table::uncompressed_time_series_table_record_batch(1); - - storage_engine - .insert_data_points(time_series_table_metadata, data) - .await - .unwrap(); - - sleep(Duration::from_millis(500)).await; - - storage_engine - .uncompressed_data_manager - .spill_in_memory_data_buffer() - .await - .unwrap(); - - // Compress the spilled buffer and sleep to allow the compression thread to finish. - assert!(storage_engine.initialize(&context).await.is_ok()); - sleep(Duration::from_millis(500)).await; - - // The spilled buffer should be deleted and the content should be compressed. - let spilled_buffers = storage_engine - .uncompressed_data_manager - .local_data_folder - .object_store() - .list(Some(&Path::from(UNCOMPRESSED_DATA_FOLDER))) - .collect::>() - .await; - - assert_eq!(spilled_buffers.len(), 0); - - assert_eq!( - storage_engine - .compressed_data_manager - .compressed_data_buffers - .len(), - 1 - ); - } - #[tokio::test] async fn test_can_insert_record_batch() { let temp_dir = tempfile::tempdir().unwrap(); @@ -1298,7 +1197,9 @@ mod tests { let channels = Arc::new(Channels::new()); let uncompressed_data_manager = - UncompressedDataManager::new(local_data_folder, memory_pool, channels); + UncompressedDataManager::try_new(local_data_folder, memory_pool, channels) + .await + .unwrap(); ( uncompressed_data_manager, From 3a6f06e083c3229bf363e9b71a4db2769a08e719 Mon Sep 17 00:00:00 2001 From: CGodiksen <36046286+CGodiksen@users.noreply.github.com> Date: Tue, 10 Mar 2026 15:21:55 +0100 Subject: [PATCH 034/116] Pass batch ids through uncompressed data manager --- .../src/storage/uncompressed_data_buffer.rs | 35 ++++++++++++++++++- .../src/storage/uncompressed_data_manager.rs | 28 +++++++++++---- 2 files changed, 56 insertions(+), 7 deletions(-) diff --git a/crates/modelardb_server/src/storage/uncompressed_data_buffer.rs b/crates/modelardb_server/src/storage/uncompressed_data_buffer.rs index 2eec5a1a..92aa59b5 100644 --- a/crates/modelardb_server/src/storage/uncompressed_data_buffer.rs +++ b/crates/modelardb_server/src/storage/uncompressed_data_buffer.rs @@ -18,6 +18,7 @@ //! supports inserting and storing data in-memory, while [`UncompressedOnDiskDataBuffer`] provides //! support for storing uncompressed data points in Apache Parquet files on disk. +use std::collections::HashSet; use std::fmt::{Debug, Formatter, Result as FmtResult}; use std::sync::Arc; use std::{iter, mem}; @@ -45,7 +46,7 @@ pub(super) struct IngestedDataBuffer { pub(super) time_series_table_metadata: Arc, /// Uncompressed data points to insert. pub(super) data_points: RecordBatch, - /// The id given to the [`RecordBatch`] by the WAL. + /// The id given to the data by the WAL. pub(super) batch_id: u64, } @@ -87,6 +88,8 @@ pub(super) struct UncompressedInMemoryDataBuffer { values: Vec, /// The tag values for the time series the buffer stores data points for. tag_values: Vec, + /// The ids given to the data by the WAL. + batch_ids: HashSet, } impl UncompressedInMemoryDataBuffer { @@ -95,6 +98,7 @@ impl UncompressedInMemoryDataBuffer { tag_values: Vec, time_series_table_metadata: Arc, current_batch_index: u64, + batch_ids: HashSet, ) -> Self { let timestamps = TimestampBuilder::with_capacity(*UNCOMPRESSED_DATA_BUFFER_CAPACITY); let values = (0..time_series_table_metadata.field_column_indices.len()) @@ -108,6 +112,7 @@ impl UncompressedInMemoryDataBuffer { timestamps, values, tag_values, + batch_ids, } } @@ -152,6 +157,11 @@ impl UncompressedInMemoryDataBuffer { debug!("Inserted data point into {:?}.", self) } + /// Add `batch_id` to the batch ids given to the data by the WAL. + pub(super) fn insert_batch_id(&mut self, batch_id: u64) { + self.batch_ids.insert(batch_id); + } + /// Return how many data points the [`UncompressedInMemoryDataBuffer`] can contain. pub(super) fn capacity(&self) -> usize { // The capacity is always the same for both builders. @@ -219,6 +229,7 @@ impl UncompressedInMemoryDataBuffer { self.updated_by_batch_index, local_data_folder, data_points, + self.batch_ids.clone(), ) .await } @@ -258,6 +269,8 @@ pub(super) struct UncompressedOnDiskDataBuffer { /// Path to the Apache Parquet file containing the uncompressed data in the /// [`UncompressedOnDiskDataBuffer`]. file_path: Path, + /// The ids given to the data by the WAL. + batch_ids: HashSet, } impl UncompressedOnDiskDataBuffer { @@ -271,6 +284,7 @@ impl UncompressedOnDiskDataBuffer { updated_by_batch_index: u64, local_data_folder: Arc, data_points: RecordBatch, + batch_ids: HashSet, ) -> Result { // Create a path that uses the first timestamp as the filename. let timestamp_index = time_series_table_metadata.timestamp_column_index; @@ -295,6 +309,7 @@ impl UncompressedOnDiskDataBuffer { updated_by_batch_index, local_data_folder, file_path, + batch_ids, }) } @@ -350,6 +365,7 @@ impl UncompressedOnDiskDataBuffer { tag_values, self.time_series_table_metadata.clone(), current_batch_index, + self.batch_ids.clone(), ); for index in 0..data_points.num_rows() { @@ -398,6 +414,7 @@ mod tests { use tokio::runtime::Runtime; const CURRENT_BATCH_INDEX: u64 = 1; + const BATCH_ID: u64 = 0; const TAG_VALUE: &str = "tag"; const TAG_HASH: u64 = 15537859409877038916; @@ -409,6 +426,7 @@ mod tests { vec![TAG_VALUE.to_owned()], table::time_series_table_metadata_arc(), CURRENT_BATCH_INDEX, + HashSet::new(), ); assert_eq!( @@ -438,6 +456,7 @@ mod tests { vec![TAG_VALUE.to_owned()], table::time_series_table_metadata_arc(), CURRENT_BATCH_INDEX, + HashSet::new(), ); assert_eq!(uncompressed_buffer.len(), 0); @@ -450,6 +469,7 @@ mod tests { vec![TAG_VALUE.to_owned()], table::time_series_table_metadata_arc(), CURRENT_BATCH_INDEX, + HashSet::new(), ); insert_data_points(1, &mut uncompressed_buffer); @@ -463,6 +483,7 @@ mod tests { vec![TAG_VALUE.to_owned()], table::time_series_table_metadata_arc(), CURRENT_BATCH_INDEX - 1, + HashSet::new(), ); assert!(!uncompressed_buffer.is_unused(CURRENT_BATCH_INDEX - 1)); @@ -482,6 +503,7 @@ mod tests { vec![TAG_VALUE.to_owned()], table::time_series_table_metadata_arc(), CURRENT_BATCH_INDEX, + HashSet::new(), ); insert_data_points(uncompressed_buffer.capacity(), &mut uncompressed_buffer); @@ -495,6 +517,7 @@ mod tests { vec![TAG_VALUE.to_owned()], table::time_series_table_metadata_arc(), CURRENT_BATCH_INDEX, + HashSet::new(), ); assert!(!uncompressed_buffer.is_full()); @@ -509,6 +532,7 @@ mod tests { vec![TAG_VALUE.to_owned()], table::time_series_table_metadata_arc(), CURRENT_BATCH_INDEX, + HashSet::new(), ); insert_data_points(uncompressed_buffer.capacity() + 1, &mut uncompressed_buffer); @@ -522,6 +546,7 @@ mod tests { vec![TAG_VALUE.to_owned()], time_series_table_metadata.clone(), CURRENT_BATCH_INDEX, + HashSet::new(), ); insert_data_points(uncompressed_buffer.capacity(), &mut uncompressed_buffer); @@ -544,6 +569,7 @@ mod tests { vec![TAG_VALUE.to_owned()], time_series_table_metadata.clone(), CURRENT_BATCH_INDEX, + HashSet::new(), ); // u64 is generated and then cast to i64 to ensure only positive values are generated. @@ -566,6 +592,7 @@ mod tests { vec![TAG_VALUE.to_owned()], table::time_series_table_metadata_arc(), CURRENT_BATCH_INDEX, + HashSet::new(), ); insert_data_points(1, &mut uncompressed_buffer); assert!(!uncompressed_buffer.is_full()); @@ -591,6 +618,7 @@ mod tests { vec![TAG_VALUE.to_owned()], table::time_series_table_metadata_arc(), CURRENT_BATCH_INDEX, + HashSet::new(), ); insert_data_points(uncompressed_buffer.capacity(), &mut uncompressed_buffer); assert!(uncompressed_buffer.is_full()); @@ -642,6 +670,7 @@ mod tests { vec![TAG_VALUE.to_owned()], time_series_table_metadata.clone(), CURRENT_BATCH_INDEX, + HashSet::new(), ); // u64 is generated and then cast to i64 to ensure only positive values are generated. @@ -695,6 +724,7 @@ mod tests { vec![TAG_VALUE.to_owned()], table::time_series_table_metadata_arc(), CURRENT_BATCH_INDEX, + HashSet::new(), ); insert_data_points( @@ -731,6 +761,7 @@ mod tests { vec![TAG_VALUE.to_owned()], table::time_series_table_metadata_arc(), CURRENT_BATCH_INDEX, + HashSet::new(), ); insert_data_points( @@ -756,5 +787,7 @@ mod tests { &mut values.iter().copied(), ); } + + uncompressed_buffer.insert_batch_id(BATCH_ID); } } diff --git a/crates/modelardb_server/src/storage/uncompressed_data_manager.rs b/crates/modelardb_server/src/storage/uncompressed_data_manager.rs index a78b3259..8091895b 100644 --- a/crates/modelardb_server/src/storage/uncompressed_data_manager.rs +++ b/crates/modelardb_server/src/storage/uncompressed_data_manager.rs @@ -16,6 +16,7 @@ //! Support for managing all uncompressed data that is ingested into the //! [`StorageEngine`](crate::storage::StorageEngine). +use std::collections::HashSet; use std::hash::{DefaultHasher, Hasher}; use std::io::{Error as IOError, ErrorKind as IOErrorKind}; use std::mem; @@ -26,7 +27,7 @@ use dashmap::DashMap; use futures::{StreamExt, TryStreamExt}; use modelardb_storage::data_folder::DataFolder; use modelardb_types::types::{TimeSeriesTableMetadata, Timestamp, Value}; -use object_store::path::{Path, PathPart}; +use object_store::path::Path; use tokio::runtime::Handle; use tracing::{debug, error, warn}; @@ -166,6 +167,7 @@ impl UncompressedDataManager { &mut values, time_series_table_metadata.clone(), current_batch_index, + ingested_data_buffer.batch_id, ) .await?; } @@ -200,6 +202,7 @@ impl UncompressedDataManager { values: &mut dyn Iterator, time_series_table_metadata: Arc, current_batch_index: u64, + batch_id: u64, ) -> Result { let tag_hash = calculate_tag_hash(&time_series_table_metadata.name, &tag_values); @@ -224,6 +227,9 @@ impl UncompressedDataManager { timestamp, values, ); + + uncompressed_in_memory_data_buffer.insert_batch_id(batch_id); + buffer_is_full = uncompressed_in_memory_data_buffer.is_full(); true } else { @@ -257,6 +263,8 @@ impl UncompressedDataManager { values, ); + uncompressed_in_memory_data_buffer.insert_batch_id(batch_id); + buffer_is_full = uncompressed_in_memory_data_buffer.is_full(); // The read-only reference must be dropped before the map can be modified. @@ -272,6 +280,7 @@ impl UncompressedDataManager { tag_values, time_series_table_metadata, current_batch_index, + HashSet::from_iter(vec![batch_id]), ); debug!( @@ -622,6 +631,7 @@ fn calculate_tag_hash(table_name: &str, tag_values: &[String]) -> u64 { mod tests { use super::*; + use std::collections::HashSet; use std::sync::Arc; use datafusion::arrow::array::StringBuilder; @@ -640,6 +650,7 @@ mod tests { const TAG_VALUE: &str = "tag"; const TAG_HASH: u64 = 14957893031159457585; + const BATCH_ID: u64 = 0; // Tests for UncompressedDataManager. #[tokio::test] @@ -648,7 +659,8 @@ mod tests { let (data_manager, time_series_table_metadata) = create_managers(&temp_dir).await; let data = table::uncompressed_time_series_table_record_batch(1); - let ingested_data_buffer = IngestedDataBuffer::new(time_series_table_metadata, data); + let ingested_data_buffer = + IngestedDataBuffer::new(time_series_table_metadata, data, BATCH_ID); data_manager .insert_data_points(ingested_data_buffer) @@ -665,7 +677,8 @@ mod tests { let (data_manager, time_series_table_metadata) = create_managers(&temp_dir).await; let data = table::uncompressed_time_series_table_record_batch(2); - let ingested_data_buffer = IngestedDataBuffer::new(time_series_table_metadata, data); + let ingested_data_buffer = + IngestedDataBuffer::new(time_series_table_metadata, data, BATCH_ID); data_manager .insert_data_points(ingested_data_buffer) @@ -689,7 +702,8 @@ mod tests { .memory_pool .remaining_ingested_memory_in_bytes(); - let ingested_data_buffer = IngestedDataBuffer::new(time_series_table_metadata, data); + let ingested_data_buffer = + IngestedDataBuffer::new(time_series_table_metadata, data, BATCH_ID); data_manager .insert_data_points(ingested_data_buffer) @@ -818,7 +832,7 @@ mod tests { .unwrap(); let ingested_data_buffer = - IngestedDataBuffer::new(time_series_table_metadata.clone(), data); + IngestedDataBuffer::new(time_series_table_metadata.clone(), data, BATCH_ID); data_manager .insert_data_points(ingested_data_buffer) .await @@ -838,7 +852,7 @@ mod tests { // Insert using insert_data_points() to finish unused buffers. let empty_record_batch = RecordBatch::new_empty(time_series_table_metadata.schema.clone()); let ingested_data_buffer = - IngestedDataBuffer::new(time_series_table_metadata, empty_record_batch); + IngestedDataBuffer::new(time_series_table_metadata, empty_record_batch, BATCH_ID); data_manager .insert_data_points(ingested_data_buffer) @@ -1050,6 +1064,7 @@ mod tests { 0, object_store, uncompressed_data, + HashSet::new(), )) .unwrap(); @@ -1166,6 +1181,7 @@ mod tests { &mut values.iter().copied(), time_series_table_metadata.clone(), current_batch_index, + BATCH_ID, ) .await .unwrap(); From 3ab29e6804bbabf646c01a3d6914ddfbf3ee43db Mon Sep 17 00:00:00 2001 From: CGodiksen <36046286+CGodiksen@users.noreply.github.com> Date: Tue, 10 Mar 2026 15:48:43 +0100 Subject: [PATCH 035/116] Pass batch ids through compressed data manager --- .../src/storage/compressed_data_buffer.rs | 35 ++++++++++++++----- .../src/storage/compressed_data_manager.rs | 15 +++++--- .../src/storage/uncompressed_data_buffer.rs | 10 ++++++ .../src/storage/uncompressed_data_manager.rs | 7 ++-- 4 files changed, 52 insertions(+), 15 deletions(-) diff --git a/crates/modelardb_server/src/storage/compressed_data_buffer.rs b/crates/modelardb_server/src/storage/compressed_data_buffer.rs index 187609a5..2fd1ca63 100644 --- a/crates/modelardb_server/src/storage/compressed_data_buffer.rs +++ b/crates/modelardb_server/src/storage/compressed_data_buffer.rs @@ -15,6 +15,7 @@ //! Buffer for compressed segments from the same time series table. +use std::collections::HashSet; use std::sync::Arc; use datafusion::arrow::record_batch::RecordBatch; @@ -30,16 +31,20 @@ pub(super) struct CompressedSegmentBatch { pub(super) time_series_table_metadata: Arc, /// Compressed segments representing the data points to insert. pub(super) compressed_segments: Vec, + /// The ids given to the data by the WAL. + pub(super) batch_ids: HashSet, } impl CompressedSegmentBatch { pub(super) fn new( time_series_table_metadata: Arc, compressed_segments: Vec, + batch_ids: HashSet, ) -> Self { Self { time_series_table_metadata, compressed_segments, + batch_ids, } } @@ -59,6 +64,8 @@ pub(super) struct CompressedDataBuffer { compressed_segments: Vec, /// Continuously updated total sum of the size of the compressed segments. pub(super) size_in_bytes: u64, + /// The ids given to the data by the WAL. + batch_ids: HashSet, } impl CompressedDataBuffer { @@ -67,6 +74,7 @@ impl CompressedDataBuffer { time_series_table_metadata, compressed_segments: vec![], size_in_bytes: 0, + batch_ids: HashSet::new(), } } @@ -76,6 +84,7 @@ impl CompressedDataBuffer { pub(super) fn append_compressed_segments( &mut self, mut compressed_segments: Vec, + batch_ids: HashSet, ) -> Result { if compressed_segments.iter().any(|compressed_segments| { compressed_segments.schema() != self.time_series_table_metadata.compressed_schema @@ -94,6 +103,8 @@ impl CompressedDataBuffer { self.size_in_bytes += compressed_segments_size; } + self.batch_ids.extend(batch_ids); + Ok(compressed_segments_size) } @@ -133,10 +144,13 @@ mod tests { CompressedDataBuffer::new(table::time_series_table_metadata_arc()); compressed_data_buffer - .append_compressed_segments(vec![ - table::compressed_segments_record_batch(), - table::compressed_segments_record_batch(), - ]) + .append_compressed_segments( + vec![ + table::compressed_segments_record_batch(), + table::compressed_segments_record_batch(), + ], + HashSet::from([0, 1, 2]), + ) .unwrap(); assert_eq!(compressed_data_buffer.compressed_segments.len(), 2); @@ -150,10 +164,13 @@ mod tests { CompressedDataBuffer::new(table::time_series_table_metadata_arc()); compressed_data_buffer - .append_compressed_segments(vec![ - table::compressed_segments_record_batch(), - table::compressed_segments_record_batch(), - ]) + .append_compressed_segments( + vec![ + table::compressed_segments_record_batch(), + table::compressed_segments_record_batch(), + ], + HashSet::from([0, 1, 2]), + ) .unwrap(); assert!(compressed_data_buffer.size_in_bytes > 0); @@ -169,7 +186,7 @@ mod tests { table::compressed_segments_record_batch(), ]; compressed_data_buffer - .append_compressed_segments(compressed_segments) + .append_compressed_segments(compressed_segments, HashSet::from([0, 1, 2])) .unwrap(); let record_batches = compressed_data_buffer.record_batches(); diff --git a/crates/modelardb_server/src/storage/compressed_data_manager.rs b/crates/modelardb_server/src/storage/compressed_data_manager.rs index bdda20f8..9dcda355 100644 --- a/crates/modelardb_server/src/storage/compressed_data_manager.rs +++ b/crates/modelardb_server/src/storage/compressed_data_manager.rs @@ -147,8 +147,10 @@ impl CompressedDataManager { { debug!("Found existing compressed data buffer for table '{time_series_table_name}'.",); - compressed_data_buffer - .append_compressed_segments(compressed_segment_batch.compressed_segments) + compressed_data_buffer.append_compressed_segments( + compressed_segment_batch.compressed_segments, + compressed_segment_batch.batch_ids, + ) } else { // A String is created as two copies are required for compressed_data_buffer and // compressed_queue anyway and compressed_segments cannot be moved out of @@ -160,8 +162,10 @@ impl CompressedDataManager { let mut compressed_data_buffer = CompressedDataBuffer::new(compressed_segment_batch.time_series_table_metadata); - let segment_size = compressed_data_buffer - .append_compressed_segments(compressed_segment_batch.compressed_segments); + let segment_size = compressed_data_buffer.append_compressed_segments( + compressed_segment_batch.compressed_segments, + compressed_segment_batch.batch_ids, + ); self.compressed_data_buffers .insert(time_series_table_name.clone(), compressed_data_buffer); @@ -290,6 +294,8 @@ impl CompressedDataManager { mod tests { use super::*; + use std::collections::HashSet; + use datafusion::arrow::array::{Array, Int8Array}; use datafusion::arrow::datatypes::{DataType, Field, Schema}; use modelardb_test::table::{self, NORMAL_TABLE_NAME, TIME_SERIES_TABLE_NAME}; @@ -594,6 +600,7 @@ mod tests { offset, ), ], + HashSet::from([0, 1]), ) } } diff --git a/crates/modelardb_server/src/storage/uncompressed_data_buffer.rs b/crates/modelardb_server/src/storage/uncompressed_data_buffer.rs index 92aa59b5..f16e2c22 100644 --- a/crates/modelardb_server/src/storage/uncompressed_data_buffer.rs +++ b/crates/modelardb_server/src/storage/uncompressed_data_buffer.rs @@ -215,6 +215,11 @@ impl UncompressedInMemoryDataBuffer { compute_memory_size(self.values.len()) } + /// Return the ids given to the data by the WAL. + pub(super) fn batch_ids(&self) -> &HashSet { + &self.batch_ids + } + /// Spill the in-memory [`UncompressedInMemoryDataBuffer`] to an Apache Parquet file and return /// an [`UncompressedOnDiskDataBuffer`] when finished. pub(super) async fn spill_to_apache_parquet( @@ -334,6 +339,11 @@ impl UncompressedOnDiskDataBuffer { &self.time_series_table_metadata } + /// Return the ids given to the data by the WAL. + pub(super) fn batch_ids(&self) -> &HashSet { + &self.batch_ids + } + /// Return [`true`] if all the data points in the [`UncompressedOnDiskDataBuffer`] are from /// [`RecordBatches`](`RecordBatch`) that are [`RECORD_BATCH_OFFSET_REQUIRED_FOR_UNUSED`] older /// than the [`RecordBatch`] with index `current_batch_index` ingested by the current process. diff --git a/crates/modelardb_server/src/storage/uncompressed_data_manager.rs b/crates/modelardb_server/src/storage/uncompressed_data_manager.rs index 8091895b..00d69d27 100644 --- a/crates/modelardb_server/src/storage/uncompressed_data_manager.rs +++ b/crates/modelardb_server/src/storage/uncompressed_data_manager.rs @@ -280,7 +280,7 @@ impl UncompressedDataManager { tag_values, time_series_table_metadata, current_batch_index, - HashSet::from_iter(vec![batch_id]), + HashSet::from([batch_id]), ); debug!( @@ -532,7 +532,7 @@ impl UncompressedDataManager { &self, uncompressed_data_buffer: UncompressedDataBuffer, ) -> Result<()> { - let (memory_use, maybe_data_points, time_series_table_metadata) = + let (memory_use, maybe_data_points, time_series_table_metadata, batch_ids) = match uncompressed_data_buffer { UncompressedDataBuffer::InMemory(mut uncompressed_in_memory_data_buffer) => ( uncompressed_in_memory_data_buffer.memory_size(), @@ -540,6 +540,7 @@ impl UncompressedDataManager { uncompressed_in_memory_data_buffer .time_series_table_metadata() .clone(), + uncompressed_in_memory_data_buffer.batch_ids().clone(), ), UncompressedDataBuffer::OnDisk(uncompressed_on_disk_data_buffer) => ( 0, @@ -547,6 +548,7 @@ impl UncompressedDataManager { uncompressed_on_disk_data_buffer .time_series_table_metadata() .clone(), + uncompressed_on_disk_data_buffer.batch_ids().clone(), ), }; @@ -584,6 +586,7 @@ impl UncompressedDataManager { .send(Message::Data(CompressedSegmentBatch::new( time_series_table_metadata.clone(), compressed_segments, + batch_ids, )))?; // Add the size of the uncompressed buffer back to the remaining reserved bytes. From 01f672bf7a3536cfb4c67729f1073b03b8c70aab Mon Sep 17 00:00:00 2001 From: CGodiksen <36046286+CGodiksen@users.noreply.github.com> Date: Tue, 10 Mar 2026 16:03:22 +0100 Subject: [PATCH 036/116] Pass batch ids when writing compressed segments --- .../modelardb_embedded/src/operations/data_folder.rs | 8 ++++---- crates/modelardb_server/src/context.rs | 4 +++- .../src/storage/compressed_data_buffer.rs | 5 +++++ .../src/storage/compressed_data_manager.rs | 8 +++++++- crates/modelardb_server/src/storage/data_transfer.rs | 8 ++++++-- crates/modelardb_storage/src/data_folder/mod.rs | 10 +++++----- 6 files changed, 30 insertions(+), 13 deletions(-) diff --git a/crates/modelardb_embedded/src/operations/data_folder.rs b/crates/modelardb_embedded/src/operations/data_folder.rs index 1f5f3375..a9a1abde 100644 --- a/crates/modelardb_embedded/src/operations/data_folder.rs +++ b/crates/modelardb_embedded/src/operations/data_folder.rs @@ -16,7 +16,7 @@ //! Operations for reading from and writing to ModelarDB data folders. use std::any::Any; -use std::collections::HashMap; +use std::collections::{HashMap, HashSet}; use std::fmt::{Debug, Formatter, Result as FmtResult}; use std::pin::Pin; use std::result::Result as StdResult; @@ -224,7 +224,7 @@ impl Operations for DataFolder { self.write_compressed_segments_to_time_series_table( table_name, compressed_data, - vec![], + HashSet::new(), ) .await?; } else if let Some(normal_table_schema) = self.normal_table_schema(table_name).await { @@ -420,7 +420,7 @@ impl Operations for DataFolder { .write_compressed_segments_to_time_series_table( target_table_name, record_batches, - vec![], + HashSet::new(), ) .await?; @@ -472,7 +472,7 @@ impl Operations for DataFolder { .write_compressed_segments_to_time_series_table( target_table_name, record_batches, - vec![], + HashSet::new(), ) .await?; } else if let (Some(source_normal_table_schema), Some(target_normal_table_schema)) = ( diff --git a/crates/modelardb_server/src/context.rs b/crates/modelardb_server/src/context.rs index 9407a821..ec5df4f6 100644 --- a/crates/modelardb_server/src/context.rs +++ b/crates/modelardb_server/src/context.rs @@ -435,6 +435,8 @@ fn table_does_not_exist_error(table_name: &str) -> ModelarDbServerError { mod tests { use super::*; + use std::collections::HashSet; + use modelardb_storage::data_folder::DataFolder; use modelardb_test::table::{self, NORMAL_TABLE_NAME, TIME_SERIES_TABLE_NAME}; use modelardb_types::types::MAX_RETENTION_PERIOD_IN_SECONDS; @@ -858,7 +860,7 @@ mod tests { .write_compressed_segments_to_time_series_table( TIME_SERIES_TABLE_NAME, vec![table::compressed_segments_record_batch()], - vec![], + HashSet::new(), ) .await .unwrap(); diff --git a/crates/modelardb_server/src/storage/compressed_data_buffer.rs b/crates/modelardb_server/src/storage/compressed_data_buffer.rs index 2fd1ca63..08c58ae1 100644 --- a/crates/modelardb_server/src/storage/compressed_data_buffer.rs +++ b/crates/modelardb_server/src/storage/compressed_data_buffer.rs @@ -113,6 +113,11 @@ impl CompressedDataBuffer { self.compressed_segments } + /// Return the ids given to the data by the WAL. + pub(super) fn batch_ids(&self) -> HashSet { + self.batch_ids.clone() + } + /// Return the size in bytes of `compressed_segments`. fn size_of_compressed_segments(compressed_segments: &RecordBatch) -> u64 { let mut total_size: u64 = 0; diff --git a/crates/modelardb_server/src/storage/compressed_data_manager.rs b/crates/modelardb_server/src/storage/compressed_data_manager.rs index 9dcda355..4f26b481 100644 --- a/crates/modelardb_server/src/storage/compressed_data_manager.rs +++ b/crates/modelardb_server/src/storage/compressed_data_manager.rs @@ -248,9 +248,15 @@ impl CompressedDataManager { // actual size is not computed as DeltaTable seems to have no support for listing the files // added in a version without iterating through all of the Add actions from file_actions(). let compressed_data_buffer_size_in_bytes = compressed_data_buffer.size_in_bytes; + let batch_ids = compressed_data_buffer.batch_ids(); let compressed_segments = compressed_data_buffer.record_batches(); + self.local_data_folder - .write_compressed_segments_to_time_series_table(table_name, compressed_segments, vec![]) + .write_compressed_segments_to_time_series_table( + table_name, + compressed_segments, + batch_ids, + ) .await?; // Inform the data transfer component about the new data if a remote data folder was diff --git a/crates/modelardb_server/src/storage/data_transfer.rs b/crates/modelardb_server/src/storage/data_transfer.rs index b0d3d808..715e52ed 100644 --- a/crates/modelardb_server/src/storage/data_transfer.rs +++ b/crates/modelardb_server/src/storage/data_transfer.rs @@ -250,7 +250,11 @@ impl DataTransfer { .await? { self.remote_data_folder - .write_compressed_segments_to_time_series_table(table_name, record_batches, vec![]) + .write_compressed_segments_to_time_series_table( + table_name, + record_batches, + HashSet::new(), + ) .await?; } else { self.remote_data_folder @@ -515,7 +519,7 @@ mod tests { .write_compressed_segments_to_time_series_table( TIME_SERIES_TABLE_NAME, vec![table::compressed_segments_record_batch()], - vec![], + HashSet::new(), ) .await .unwrap(); diff --git a/crates/modelardb_storage/src/data_folder/mod.rs b/crates/modelardb_storage/src/data_folder/mod.rs index 2f40dc0c..8c144c2f 100644 --- a/crates/modelardb_storage/src/data_folder/mod.rs +++ b/crates/modelardb_storage/src/data_folder/mod.rs @@ -17,7 +17,7 @@ pub mod cluster; -use std::collections::HashMap; +use std::collections::{HashMap, HashSet}; use std::path::Path as StdPath; use std::sync::Arc; use std::{env, fs}; @@ -881,7 +881,7 @@ impl DataFolder { &self, table_name: &str, compressed_segments: Vec, - batch_ids: Vec, + batch_ids: HashSet, ) -> Result { let delta_table = self.delta_table(table_name).await?; @@ -1148,7 +1148,7 @@ pub struct DeltaTableWriter { /// Writes record batches to the Delta table as Apache Parquet files. delta_writer: DeltaWriter, /// Batch ids from the WAL to include in the commit metadata for checkpointing. - batch_ids: Vec, + batch_ids: HashSet, } impl DeltaTableWriter { @@ -1201,12 +1201,12 @@ impl DeltaTableWriter { delta_operation, operation_id, delta_writer, - batch_ids: vec![], + batch_ids: HashSet::new(), }) } /// Add batch ids from the WAL that are included in the commit metadata for checkpointing. - pub fn with_batch_ids(mut self, batch_ids: Vec) -> Self { + pub fn with_batch_ids(mut self, batch_ids: HashSet) -> Self { self.batch_ids = batch_ids; self } From 766cf53408b190a7cc7c089485adfc228176c2c5 Mon Sep 17 00:00:00 2001 From: CGodiksen <36046286+CGodiksen@users.noreply.github.com> Date: Tue, 10 Mar 2026 16:53:17 +0100 Subject: [PATCH 037/116] Add batch id to WriteAheadLogFile --- crates/modelardb_storage/src/write_ahead_log.rs | 11 ++++++++++- 1 file changed, 10 insertions(+), 1 deletion(-) diff --git a/crates/modelardb_storage/src/write_ahead_log.rs b/crates/modelardb_storage/src/write_ahead_log.rs index 6687748e..c8f1e7d6 100644 --- a/crates/modelardb_storage/src/write_ahead_log.rs +++ b/crates/modelardb_storage/src/write_ahead_log.rs @@ -151,6 +151,9 @@ struct WriteAheadLogFile { path: PathBuf, /// Writer to write data in IPC streaming format to the log file. writer: Mutex>, + /// The batch id to give to the next batch of data appended to the log file. This is incremented + /// after each append, so the batch id given to data is monotonically increasing. + batch_id: Mutex, } impl WriteAheadLogFile { @@ -179,6 +182,7 @@ impl WriteAheadLogFile { Ok(Self { path: file_path, writer: Mutex::new(writer), + batch_id: Mutex::new(0), }) } @@ -188,6 +192,7 @@ impl WriteAheadLogFile { fn append_and_sync(&self, data: &RecordBatch) -> Result { // Acquire the mutex to ensure only one thread can write at a time. let mut writer = self.writer.lock().expect("Mutex should not be poisoned."); + let mut batch_id = self.batch_id.lock().expect("Mutex should not be poisoned."); writer.write(data)?; @@ -198,7 +203,11 @@ impl WriteAheadLogFile { // such as modification timestamps and permissions are not updated since we only sync data. writer.get_ref().sync_data()?; - Ok(0) + // Increment the batch id for the next batch of data. + let current_batch_id = *batch_id; + *batch_id += 1; + + Ok(current_batch_id) } /// Read all data from the log file. This can be called even if the [`StreamWriter`] has not From f3f114ad4ad377caa57120b51c085f8504701907 Mon Sep 17 00:00:00 2001 From: CGodiksen <36046286+CGodiksen@users.noreply.github.com> Date: Tue, 10 Mar 2026 17:22:42 +0100 Subject: [PATCH 038/116] Use folder path instead of file path when initializing WriteAheadLogFile --- .../modelardb_storage/src/write_ahead_log.rs | 90 +++++++++---------- 1 file changed, 43 insertions(+), 47 deletions(-) diff --git a/crates/modelardb_storage/src/write_ahead_log.rs b/crates/modelardb_storage/src/write_ahead_log.rs index c8f1e7d6..cf3ee29c 100644 --- a/crates/modelardb_storage/src/write_ahead_log.rs +++ b/crates/modelardb_storage/src/write_ahead_log.rs @@ -33,7 +33,7 @@ use crate::WRITE_AHEAD_LOG_FOLDER; use crate::data_folder::DataFolder; use crate::error::{ModelarDbStorageError, Result}; -const OPERATIONS_LOG_FILE: &str = "operations.wal"; +const OPERATIONS_LOG_FOLDER: &str = "operations"; /// Write-ahead log that logs data on a per-table level and operations separately. pub struct WriteAheadLog { @@ -68,7 +68,7 @@ impl WriteAheadLog { folder_path: log_folder_path.clone(), table_logs: HashMap::new(), operation_log: WriteAheadLogFile::try_new( - log_folder_path.join(OPERATIONS_LOG_FILE), + log_folder_path.join(OPERATIONS_LOG_FOLDER), &operations_log_schema(), )?, }; @@ -92,7 +92,7 @@ impl WriteAheadLog { let table_name = time_series_table_metadata.name.clone(); if !self.table_logs.contains_key(&table_name) { - let table_log_path = self.folder_path.join(format!("{}.wal", table_name)); + let table_log_path = self.folder_path.join(&table_name); self.table_logs.insert( table_name, @@ -157,10 +157,13 @@ struct WriteAheadLogFile { } impl WriteAheadLogFile { - /// Create a new [`WriteAheadLogFile`] that appends data with `schema` to the file at - /// `file_path`. If the file does not exist, it is created. If the file could not be created, + /// Create a new [`WriteAheadLogFile`] that appends data with `schema` to a file in + /// `folder_path`. If the file does not exist, it is created. If the file could not be created, /// return [`ModelarDbStorageError`]. - fn try_new(file_path: PathBuf, schema: &Schema) -> Result { + fn try_new(folder_path: PathBuf, schema: &Schema) -> Result { + std::fs::create_dir_all(folder_path.clone())?; + let file_path = folder_path.join("0.wal"); + let file = OpenOptions::new() .create(true) .read(true) @@ -242,7 +245,7 @@ impl WriteAheadLogFile { } } -/// Return the schema for the operations log that is stored in [`OPERATIONS_LOG_FILE`]. +/// Return the schema for the operations log that is stored in [`OPERATIONS_LOG_FOLDER`]. fn operations_log_schema() -> Schema { Schema::new(vec![Field::new("operation", DataType::Utf8, false)]) } @@ -257,26 +260,21 @@ mod tests { #[test] fn test_try_new_creates_file() { let temp_dir = tempfile::tempdir().unwrap(); - let file_path = temp_dir - .path() - .join(format!("{}.wal", TIME_SERIES_TABLE_NAME)); + let folder_path = temp_dir.path().join(TIME_SERIES_TABLE_NAME); let metadata = table::time_series_table_metadata(); - let wal_file = WriteAheadLogFile::try_new(file_path.clone(), &metadata.schema).unwrap(); + let wal_file = WriteAheadLogFile::try_new(folder_path.clone(), &metadata.schema).unwrap(); - assert!(file_path.exists()); - assert_eq!(wal_file.path, file_path); + assert!(wal_file.path.exists()); } #[test] fn test_read_all_empty_file() { let temp_dir = tempfile::tempdir().unwrap(); - let file_path = temp_dir - .path() - .join(format!("{}.wal", TIME_SERIES_TABLE_NAME)); + let folder_path = temp_dir.path().join(TIME_SERIES_TABLE_NAME); let metadata = table::time_series_table_metadata(); - let wal_file = WriteAheadLogFile::try_new(file_path, &metadata.schema).unwrap(); + let wal_file = WriteAheadLogFile::try_new(folder_path, &metadata.schema).unwrap(); let batches = wal_file.read_all().unwrap(); assert!(batches.is_empty()); @@ -285,12 +283,10 @@ mod tests { #[test] fn test_append_and_read_single_batch() { let temp_dir = tempfile::tempdir().unwrap(); - let file_path = temp_dir - .path() - .join(format!("{}.wal", TIME_SERIES_TABLE_NAME)); + let folder_path = temp_dir.path().join(TIME_SERIES_TABLE_NAME); let metadata = table::time_series_table_metadata(); - let wal_file = WriteAheadLogFile::try_new(file_path, &metadata.schema).unwrap(); + let wal_file = WriteAheadLogFile::try_new(folder_path, &metadata.schema).unwrap(); let batch = table::uncompressed_time_series_table_record_batch(5); wal_file.append_and_sync(&batch).unwrap(); @@ -303,12 +299,10 @@ mod tests { #[test] fn test_append_and_read_multiple_batches() { let temp_dir = tempfile::tempdir().unwrap(); - let file_path = temp_dir - .path() - .join(format!("{}.wal", TIME_SERIES_TABLE_NAME)); + let folder_path = temp_dir.path().join(TIME_SERIES_TABLE_NAME); let metadata = table::time_series_table_metadata(); - let wal_file = WriteAheadLogFile::try_new(file_path, &metadata.schema).unwrap(); + let wal_file = WriteAheadLogFile::try_new(folder_path, &metadata.schema).unwrap(); let batch_1 = table::uncompressed_time_series_table_record_batch(10); let batch_2 = table::uncompressed_time_series_table_record_batch(20); @@ -328,19 +322,18 @@ mod tests { #[test] fn test_reopen_existing_file_and_append() { let temp_dir = tempfile::tempdir().unwrap(); - let file_path = temp_dir - .path() - .join(format!("{}.wal", TIME_SERIES_TABLE_NAME)); + let folder_path = temp_dir.path().join(TIME_SERIES_TABLE_NAME); let metadata = table::time_series_table_metadata(); let batch_1 = table::uncompressed_time_series_table_record_batch(10); { - let wal_file = WriteAheadLogFile::try_new(file_path.clone(), &metadata.schema).unwrap(); + let wal_file = + WriteAheadLogFile::try_new(folder_path.clone(), &metadata.schema).unwrap(); wal_file.append_and_sync(&batch_1).unwrap(); } let batch_2 = table::uncompressed_time_series_table_record_batch(20); - let wal_file = WriteAheadLogFile::try_new(file_path, &metadata.schema).unwrap(); + let wal_file = WriteAheadLogFile::try_new(folder_path, &metadata.schema).unwrap(); wal_file.append_and_sync(&batch_2).unwrap(); let batches = wal_file.read_all().unwrap(); @@ -352,18 +345,17 @@ mod tests { #[test] fn test_reopen_existing_file_and_read_without_append() { let temp_dir = tempfile::tempdir().unwrap(); - let file_path = temp_dir - .path() - .join(format!("{}.wal", TIME_SERIES_TABLE_NAME)); + let folder_path = temp_dir.path().join(TIME_SERIES_TABLE_NAME); let metadata = table::time_series_table_metadata(); let batch = table::uncompressed_time_series_table_record_batch(10); { - let wal_file = WriteAheadLogFile::try_new(file_path.clone(), &metadata.schema).unwrap(); + let wal_file = + WriteAheadLogFile::try_new(folder_path.clone(), &metadata.schema).unwrap(); wal_file.append_and_sync(&batch).unwrap(); } - let wal_file = WriteAheadLogFile::try_new(file_path, &metadata.schema).unwrap(); + let wal_file = WriteAheadLogFile::try_new(folder_path, &metadata.schema).unwrap(); let batches = wal_file.read_all().unwrap(); assert_eq!(batches.len(), 1); assert_eq!(batches[0], batch); @@ -372,25 +364,29 @@ mod tests { #[test] fn test_file_size_not_changed_on_reopen() { let temp_dir = tempfile::tempdir().unwrap(); - let file_path = temp_dir - .path() - .join(format!("{}.wal", TIME_SERIES_TABLE_NAME)); + let folder_path = temp_dir.path().join(TIME_SERIES_TABLE_NAME); let metadata = table::time_series_table_metadata(); let batch = table::uncompressed_time_series_table_record_batch(10); - { - let wal_file = WriteAheadLogFile::try_new(file_path.clone(), &metadata.schema).unwrap(); + + let wal_file_path = { + let wal_file = + WriteAheadLogFile::try_new(folder_path.clone(), &metadata.schema).unwrap(); wal_file.append_and_sync(&batch).unwrap(); - } - let size_before = std::fs::metadata(&file_path).unwrap().len(); + wal_file.path.clone() + }; + + let size_before = std::fs::metadata(&wal_file_path).unwrap().len(); - { - let _wal_file = - WriteAheadLogFile::try_new(file_path.clone(), &metadata.schema).unwrap(); - } + let wal_file_path = { + let wal_file = + WriteAheadLogFile::try_new(folder_path.clone(), &metadata.schema).unwrap(); + + wal_file.path.clone() + }; - let size_after = std::fs::metadata(&file_path).unwrap().len(); + let size_after = std::fs::metadata(&wal_file_path).unwrap().len(); assert_eq!(size_before, size_after); } } From eb74385f8a36eab26012476163f9fd353e9aa7bd Mon Sep 17 00:00:00 2001 From: CGodiksen <36046286+CGodiksen@users.noreply.github.com> Date: Tue, 10 Mar 2026 17:36:37 +0100 Subject: [PATCH 039/116] Handle existing wal files and ensure batch id is initialized correctly --- .../modelardb_storage/src/write_ahead_log.rs | 28 +++++++++++++++++-- 1 file changed, 26 insertions(+), 2 deletions(-) diff --git a/crates/modelardb_storage/src/write_ahead_log.rs b/crates/modelardb_storage/src/write_ahead_log.rs index cf3ee29c..37dfed32 100644 --- a/crates/modelardb_storage/src/write_ahead_log.rs +++ b/crates/modelardb_storage/src/write_ahead_log.rs @@ -162,7 +162,9 @@ impl WriteAheadLogFile { /// return [`ModelarDbStorageError`]. fn try_new(folder_path: PathBuf, schema: &Schema) -> Result { std::fs::create_dir_all(folder_path.clone())?; - let file_path = folder_path.join("0.wal"); + + let (file_path, offset) = + find_existing_wal_file(&folder_path)?.unwrap_or_else(|| (folder_path.join("0.wal"), 0)); let file = OpenOptions::new() .create(true) @@ -182,10 +184,19 @@ impl WriteAheadLogFile { writer.get_ref().seek(SeekFrom::End(0))?; } + // Count existing batches to reconstruct the next batch ID. + let batch_count = if file_len > 0 { + let file = File::open(&file_path)?; + let reader = StreamReader::try_new(file, None)?; + reader.take_while(|r| r.is_ok()).count() as u64 + } else { + 0 + }; + Ok(Self { path: file_path, writer: Mutex::new(writer), - batch_id: Mutex::new(0), + batch_id: Mutex::new(offset + batch_count), }) } @@ -245,6 +256,19 @@ impl WriteAheadLogFile { } } +/// Find an existing WAL file in `folder_path` and return its path and the offset parsed from its +/// name if it exists, otherwise return `Ok(None)`. +fn find_existing_wal_file(folder_path: &PathBuf) -> Result> { + Ok(std::fs::read_dir(folder_path)? + .filter_map(|maybe_file| maybe_file.ok()) + .filter_map(|file| { + let path = file.path(); + let offset = path.file_stem()?.to_str()?.parse::().ok()?; + Some((path, offset)) + }) + .next()) +} + /// Return the schema for the operations log that is stored in [`OPERATIONS_LOG_FOLDER`]. fn operations_log_schema() -> Schema { Schema::new(vec![Field::new("operation", DataType::Utf8, false)]) From 676288cd228cebb30eadf95a1ca5d7fa2c565328 Mon Sep 17 00:00:00 2001 From: CGodiksen <36046286+CGodiksen@users.noreply.github.com> Date: Tue, 10 Mar 2026 17:49:10 +0100 Subject: [PATCH 040/116] Add checks the next batch id in unit tests --- .../modelardb_storage/src/write_ahead_log.rs | 19 ++++++++++++++----- 1 file changed, 14 insertions(+), 5 deletions(-) diff --git a/crates/modelardb_storage/src/write_ahead_log.rs b/crates/modelardb_storage/src/write_ahead_log.rs index 37dfed32..9d288f94 100644 --- a/crates/modelardb_storage/src/write_ahead_log.rs +++ b/crates/modelardb_storage/src/write_ahead_log.rs @@ -153,7 +153,7 @@ struct WriteAheadLogFile { writer: Mutex>, /// The batch id to give to the next batch of data appended to the log file. This is incremented /// after each append, so the batch id given to data is monotonically increasing. - batch_id: Mutex, + next_batch_id: Mutex, } impl WriteAheadLogFile { @@ -196,7 +196,7 @@ impl WriteAheadLogFile { Ok(Self { path: file_path, writer: Mutex::new(writer), - batch_id: Mutex::new(offset + batch_count), + next_batch_id: Mutex::new(offset + batch_count), }) } @@ -206,7 +206,10 @@ impl WriteAheadLogFile { fn append_and_sync(&self, data: &RecordBatch) -> Result { // Acquire the mutex to ensure only one thread can write at a time. let mut writer = self.writer.lock().expect("Mutex should not be poisoned."); - let mut batch_id = self.batch_id.lock().expect("Mutex should not be poisoned."); + let mut next_batch_id = self + .next_batch_id + .lock() + .expect("Mutex should not be poisoned."); writer.write(data)?; @@ -218,8 +221,8 @@ impl WriteAheadLogFile { writer.get_ref().sync_data()?; // Increment the batch id for the next batch of data. - let current_batch_id = *batch_id; - *batch_id += 1; + let current_batch_id = *next_batch_id; + *next_batch_id += 1; Ok(current_batch_id) } @@ -290,6 +293,7 @@ mod tests { let wal_file = WriteAheadLogFile::try_new(folder_path.clone(), &metadata.schema).unwrap(); assert!(wal_file.path.exists()); + assert_eq!(*wal_file.next_batch_id.lock().unwrap(), 0); } #[test] @@ -302,6 +306,7 @@ mod tests { let batches = wal_file.read_all().unwrap(); assert!(batches.is_empty()); + assert_eq!(*wal_file.next_batch_id.lock().unwrap(), 0); } #[test] @@ -318,6 +323,7 @@ mod tests { let batches = wal_file.read_all().unwrap(); assert_eq!(batches.len(), 1); assert_eq!(batches[0], batch); + assert_eq!(*wal_file.next_batch_id.lock().unwrap(), 1); } #[test] @@ -341,6 +347,7 @@ mod tests { assert_eq!(batches[0], batch_1); assert_eq!(batches[1], batch_2); assert_eq!(batches[2], batch_3); + assert_eq!(*wal_file.next_batch_id.lock().unwrap(), 3); } #[test] @@ -364,6 +371,7 @@ mod tests { assert_eq!(batches.len(), 2); assert_eq!(batches[0], batch_1); assert_eq!(batches[1], batch_2); + assert_eq!(*wal_file.next_batch_id.lock().unwrap(), 2); } #[test] @@ -383,6 +391,7 @@ mod tests { let batches = wal_file.read_all().unwrap(); assert_eq!(batches.len(), 1); assert_eq!(batches[0], batch); + assert_eq!(*wal_file.next_batch_id.lock().unwrap(), 1); } #[test] From bb9cbc59e37d1673eaedde2ef595d003459478dd Mon Sep 17 00:00:00 2001 From: CGodiksen <36046286+CGodiksen@users.noreply.github.com> Date: Wed, 11 Mar 2026 08:13:20 +0100 Subject: [PATCH 041/116] Add batch offset to WriteAheadLogFile struct fields --- crates/modelardb_storage/src/write_ahead_log.rs | 10 +++++++--- 1 file changed, 7 insertions(+), 3 deletions(-) diff --git a/crates/modelardb_storage/src/write_ahead_log.rs b/crates/modelardb_storage/src/write_ahead_log.rs index 9d288f94..5d8eb5d3 100644 --- a/crates/modelardb_storage/src/write_ahead_log.rs +++ b/crates/modelardb_storage/src/write_ahead_log.rs @@ -143,7 +143,7 @@ impl WriteAheadLog { } } -/// Wrapper around a [`File`] that enforces that [`sync_all()`](File::sync_all) is called +/// Wrapper around a [`File`] that enforces that [`sync_data()`](File::sync_data) is called /// immediately after writing to ensure that all data is on disk before returning. Note that /// an exclusive lock is held on the file while it is being written to. struct WriteAheadLogFile { @@ -151,6 +151,9 @@ struct WriteAheadLogFile { path: PathBuf, /// Writer to write data in IPC streaming format to the log file. writer: Mutex>, + /// The offset encoded in the WAL file name. This represents the number of batches that have + /// been removed from the start of the file across all previous truncations. + batch_offset: u64, /// The batch id to give to the next batch of data appended to the log file. This is incremented /// after each append, so the batch id given to data is monotonically increasing. next_batch_id: Mutex, @@ -163,7 +166,7 @@ impl WriteAheadLogFile { fn try_new(folder_path: PathBuf, schema: &Schema) -> Result { std::fs::create_dir_all(folder_path.clone())?; - let (file_path, offset) = + let (file_path, batch_offset) = find_existing_wal_file(&folder_path)?.unwrap_or_else(|| (folder_path.join("0.wal"), 0)); let file = OpenOptions::new() @@ -196,7 +199,8 @@ impl WriteAheadLogFile { Ok(Self { path: file_path, writer: Mutex::new(writer), - next_batch_id: Mutex::new(offset + batch_count), + batch_offset, + next_batch_id: Mutex::new(batch_offset + batch_count), }) } From 7d403c6bf43758427c260582e7edd8c7ab53843d Mon Sep 17 00:00:00 2001 From: CGodiksen <36046286+CGodiksen@users.noreply.github.com> Date: Wed, 11 Mar 2026 09:01:08 +0100 Subject: [PATCH 042/116] Add method to mark batches as persisted --- .../modelardb_storage/src/write_ahead_log.rs | 31 ++++++++++++++++++- 1 file changed, 30 insertions(+), 1 deletion(-) diff --git a/crates/modelardb_storage/src/write_ahead_log.rs b/crates/modelardb_storage/src/write_ahead_log.rs index 5d8eb5d3..e26e745a 100644 --- a/crates/modelardb_storage/src/write_ahead_log.rs +++ b/crates/modelardb_storage/src/write_ahead_log.rs @@ -16,7 +16,7 @@ //! Implementation of types that provide a write-ahead log for ModelarDB that can be used to //! efficiently persist data and operations on disk to avoid data loss and enable crash recovery. -use std::collections::HashMap; +use std::collections::{BTreeSet, HashMap, HashSet}; use std::fs::{File, OpenOptions}; use std::io::{Seek, SeekFrom}; use std::path::PathBuf; @@ -157,6 +157,9 @@ struct WriteAheadLogFile { /// The batch id to give to the next batch of data appended to the log file. This is incremented /// after each append, so the batch id given to data is monotonically increasing. next_batch_id: Mutex, + /// Batch ids that have been confirmed as saved to disk. Used to determine whether a + /// contiguous prefix of batches can be trimmed from the start of the log file. + persisted_batch_ids: Mutex>, } impl WriteAheadLogFile { @@ -201,6 +204,7 @@ impl WriteAheadLogFile { writer: Mutex::new(writer), batch_offset, next_batch_id: Mutex::new(batch_offset + batch_count), + persisted_batch_ids: Mutex::new(BTreeSet::new()), }) } @@ -231,6 +235,31 @@ impl WriteAheadLogFile { Ok(current_batch_id) } + /// Mark the given batch ids as saved to disk. Returns the new contiguous persisted watermark, + /// i.e., the highest batch id such that all ids from `batch_offset` up to and including it are + /// persisted. Returns `None` if no new contiguous prefix is available. + fn mark_batches_as_persisted(&self, batch_ids: HashSet) -> Option { + let mut persisted = self + .persisted_batch_ids + .lock() + .expect("Mutex should not be poisoned."); + + persisted.extend(batch_ids); + + // Walk forward from batch_offset to find the contiguous prefix watermark. + let mut watermark = self.batch_offset; + while persisted.contains(&watermark) { + watermark += 1; + } + + // If watermark advanced, we have a contiguous prefix ending at watermark - 1. + if watermark > self.batch_offset { + Some(watermark - 1) + } else { + None + } + } + /// Read all data from the log file. This can be called even if the [`StreamWriter`] has not /// been finished, meaning the log file is missing the end-of-stream bytes. If the file /// could not be read, return [`ModelarDbStorageError`]. From 8332e4a96bf7e42e3e4dc20ad59ff63159b7cb68 Mon Sep 17 00:00:00 2001 From: CGodiksen <36046286+CGodiksen@users.noreply.github.com> Date: Wed, 11 Mar 2026 09:11:03 +0100 Subject: [PATCH 043/116] Add write ahead log to compressed data manager --- .../src/storage/compressed_data_manager.rs | 10 ++++++++++ crates/modelardb_server/src/storage/mod.rs | 1 + 2 files changed, 11 insertions(+) diff --git a/crates/modelardb_server/src/storage/compressed_data_manager.rs b/crates/modelardb_server/src/storage/compressed_data_manager.rs index 4f26b481..810683bb 100644 --- a/crates/modelardb_server/src/storage/compressed_data_manager.rs +++ b/crates/modelardb_server/src/storage/compressed_data_manager.rs @@ -22,6 +22,7 @@ use crossbeam_queue::SegQueue; use dashmap::DashMap; use datafusion::arrow::record_batch::RecordBatch; use modelardb_storage::data_folder::DataFolder; +use modelardb_storage::write_ahead_log::WriteAheadLog; use tokio::runtime::Handle; use tokio::sync::RwLock; use tracing::{debug, error, info}; @@ -50,6 +51,8 @@ pub(super) struct CompressedDataManager { channels: Arc, /// Track how much memory is left for storing uncompressed and compressed data. memory_pool: Arc, + /// Write-ahead log for persisting data and operations. + write_ahead_log: Arc>, } impl CompressedDataManager { @@ -58,6 +61,7 @@ impl CompressedDataManager { local_data_folder: DataFolder, channels: Arc, memory_pool: Arc, + write_ahead_log: Arc>, ) -> Self { Self { data_transfer, @@ -66,6 +70,7 @@ impl CompressedDataManager { compressed_queue: SegQueue::new(), channels, memory_pool, + write_ahead_log, } } @@ -575,6 +580,10 @@ mod tests { .await .unwrap(); + let write_ahead_log = Arc::new(RwLock::new( + WriteAheadLog::try_new(&local_data_folder).await.unwrap(), + )); + ( temp_dir, CompressedDataManager::new( @@ -582,6 +591,7 @@ mod tests { local_data_folder, channels, memory_pool, + write_ahead_log, ), ) } diff --git a/crates/modelardb_server/src/storage/mod.rs b/crates/modelardb_server/src/storage/mod.rs index c8633340..b1446f17 100644 --- a/crates/modelardb_server/src/storage/mod.rs +++ b/crates/modelardb_server/src/storage/mod.rs @@ -178,6 +178,7 @@ impl StorageEngine { data_folders.local_data_folder, channels.clone(), memory_pool.clone(), + write_ahead_log.clone(), )); { From e39d1bfd764111b8aa02207bfe7afb7ff24d7ef3 Mon Sep 17 00:00:00 2001 From: CGodiksen <36046286+CGodiksen@users.noreply.github.com> Date: Wed, 11 Mar 2026 09:20:48 +0100 Subject: [PATCH 044/116] Add method to WriteAheadLog to mark batches as persisted in table log --- .../modelardb_storage/src/write_ahead_log.rs | 32 +++++++++++++++---- 1 file changed, 26 insertions(+), 6 deletions(-) diff --git a/crates/modelardb_storage/src/write_ahead_log.rs b/crates/modelardb_storage/src/write_ahead_log.rs index e26e745a..a23a60a0 100644 --- a/crates/modelardb_storage/src/write_ahead_log.rs +++ b/crates/modelardb_storage/src/write_ahead_log.rs @@ -141,6 +141,24 @@ impl WriteAheadLog { log_file.append_and_sync(data) } + + /// Mark the given batch ids as saved to disk in the corresponding table log. If a large enough + /// contiguous prefix of batches is marked as persisted, the log file is trimmed to remove + /// the persisted data. If a table log does not exist or the log file could not be trimmed, + /// return [`ModelarDbStorageError`]. + pub fn mark_batches_as_persisted_in_table_log( + &self, + table_name: &str, + batch_ids: HashSet, + ) -> Result<()> { + let log_file = self.table_logs.get(table_name).ok_or_else(|| { + ModelarDbStorageError::InvalidState(format!( + "Table log for table '{table_name}' does not exist." + )) + })?; + + log_file.mark_batches_as_persisted(batch_ids) + } } /// Wrapper around a [`File`] that enforces that [`sync_data()`](File::sync_data) is called @@ -235,10 +253,10 @@ impl WriteAheadLogFile { Ok(current_batch_id) } - /// Mark the given batch ids as saved to disk. Returns the new contiguous persisted watermark, - /// i.e., the highest batch id such that all ids from `batch_offset` up to and including it are - /// persisted. Returns `None` if no new contiguous prefix is available. - fn mark_batches_as_persisted(&self, batch_ids: HashSet) -> Option { + /// Mark the given batch ids as saved to disk. If a large enough contiguous prefix of batches + /// is marked as persisted, the log file is trimmed to remove the persisted data. If the + /// file could not be trimmed, return [`ModelarDbStorageError`]. + fn mark_batches_as_persisted(&self, batch_ids: HashSet) -> Result<()> { let mut persisted = self .persisted_batch_ids .lock() @@ -253,11 +271,13 @@ impl WriteAheadLogFile { } // If watermark advanced, we have a contiguous prefix ending at watermark - 1. - if watermark > self.batch_offset { + let max_prefix_batch_id = if watermark > self.batch_offset { Some(watermark - 1) } else { None - } + }; + + Ok(()) } /// Read all data from the log file. This can be called even if the [`StreamWriter`] has not From 3c50c93bdb282e3bc8bec6af74f19efe77898320 Mon Sep 17 00:00:00 2001 From: CGodiksen <36046286+CGodiksen@users.noreply.github.com> Date: Wed, 11 Mar 2026 09:23:19 +0100 Subject: [PATCH 045/116] Remove duplicated code to find table log --- .../modelardb_storage/src/write_ahead_log.rs | 20 +++++++++---------- 1 file changed, 10 insertions(+), 10 deletions(-) diff --git a/crates/modelardb_storage/src/write_ahead_log.rs b/crates/modelardb_storage/src/write_ahead_log.rs index a23a60a0..7cfe19fa 100644 --- a/crates/modelardb_storage/src/write_ahead_log.rs +++ b/crates/modelardb_storage/src/write_ahead_log.rs @@ -133,12 +133,7 @@ impl WriteAheadLog { /// synchronization. Return the batch id given to the appended data. If a table log does not /// exist or the data could not be appended, return [`ModelarDbStorageError`]. pub fn append_to_table_log(&self, table_name: &str, data: &RecordBatch) -> Result { - let log_file = self.table_logs.get(table_name).ok_or_else(|| { - ModelarDbStorageError::InvalidState(format!( - "Table log for table '{table_name}' does not exist." - )) - })?; - + let log_file = self.table_log(table_name)?; log_file.append_and_sync(data) } @@ -151,13 +146,18 @@ impl WriteAheadLog { table_name: &str, batch_ids: HashSet, ) -> Result<()> { - let log_file = self.table_logs.get(table_name).ok_or_else(|| { + let log_file = self.table_log(table_name)?; + log_file.mark_batches_as_persisted(batch_ids) + } + + /// Get the log file for the table with the given name. If the log file does not exist, return + /// [`ModelarDbStorageError`]. + fn table_log(&self, table_name: &str) -> Result<&WriteAheadLogFile> { + self.table_logs.get(table_name).ok_or_else(|| { ModelarDbStorageError::InvalidState(format!( "Table log for table '{table_name}' does not exist." )) - })?; - - log_file.mark_batches_as_persisted(batch_ids) + }) } } From 030fddb1fb66708ea38325a0a41f8c468d264281 Mon Sep 17 00:00:00 2001 From: CGodiksen <36046286+CGodiksen@users.noreply.github.com> Date: Wed, 11 Mar 2026 09:28:15 +0100 Subject: [PATCH 046/116] Mark batches as persisted when right after they are saved --- .../src/storage/compressed_data_manager.rs | 7 ++++++- 1 file changed, 6 insertions(+), 1 deletion(-) diff --git a/crates/modelardb_server/src/storage/compressed_data_manager.rs b/crates/modelardb_server/src/storage/compressed_data_manager.rs index 810683bb..083714af 100644 --- a/crates/modelardb_server/src/storage/compressed_data_manager.rs +++ b/crates/modelardb_server/src/storage/compressed_data_manager.rs @@ -260,10 +260,15 @@ impl CompressedDataManager { .write_compressed_segments_to_time_series_table( table_name, compressed_segments, - batch_ids, + batch_ids.clone(), ) .await?; + // Inform the write-ahead-log that data has been written to disk. We use a read lock since + // the specific log file is locked internally before being updated. + let write_ahead_log = self.write_ahead_log.read().await; + write_ahead_log.mark_batches_as_persisted_in_table_log(table_name, batch_ids)?; + // Inform the data transfer component about the new data if a remote data folder was // provided. If the total size of the data related to table_name has reached the transfer // threshold, all of the data is transferred to the remote object store. From 1be33dd7b371425e5ccc1ed8790d79ea97abbfb6 Mon Sep 17 00:00:00 2001 From: CGodiksen <36046286+CGodiksen@users.noreply.github.com> Date: Wed, 11 Mar 2026 09:47:17 +0100 Subject: [PATCH 047/116] Use AtomicU64 for next batch id instead of a Mutex --- .../modelardb_storage/src/write_ahead_log.rs | 24 ++++++++----------- 1 file changed, 10 insertions(+), 14 deletions(-) diff --git a/crates/modelardb_storage/src/write_ahead_log.rs b/crates/modelardb_storage/src/write_ahead_log.rs index 7cfe19fa..2d1f937c 100644 --- a/crates/modelardb_storage/src/write_ahead_log.rs +++ b/crates/modelardb_storage/src/write_ahead_log.rs @@ -21,6 +21,7 @@ use std::fs::{File, OpenOptions}; use std::io::{Seek, SeekFrom}; use std::path::PathBuf; use std::sync::Mutex; +use std::sync::atomic::{AtomicU64, Ordering}; use arrow::datatypes::{DataType, Field, Schema}; use arrow::error::ArrowError::IpcError; @@ -174,7 +175,7 @@ struct WriteAheadLogFile { batch_offset: u64, /// The batch id to give to the next batch of data appended to the log file. This is incremented /// after each append, so the batch id given to data is monotonically increasing. - next_batch_id: Mutex, + next_batch_id: AtomicU64, /// Batch ids that have been confirmed as saved to disk. Used to determine whether a /// contiguous prefix of batches can be trimmed from the start of the log file. persisted_batch_ids: Mutex>, @@ -221,7 +222,7 @@ impl WriteAheadLogFile { path: file_path, writer: Mutex::new(writer), batch_offset, - next_batch_id: Mutex::new(batch_offset + batch_count), + next_batch_id: AtomicU64::new(batch_offset + batch_count), persisted_batch_ids: Mutex::new(BTreeSet::new()), }) } @@ -232,10 +233,6 @@ impl WriteAheadLogFile { fn append_and_sync(&self, data: &RecordBatch) -> Result { // Acquire the mutex to ensure only one thread can write at a time. let mut writer = self.writer.lock().expect("Mutex should not be poisoned."); - let mut next_batch_id = self - .next_batch_id - .lock() - .expect("Mutex should not be poisoned."); writer.write(data)?; @@ -247,8 +244,7 @@ impl WriteAheadLogFile { writer.get_ref().sync_data()?; // Increment the batch id for the next batch of data. - let current_batch_id = *next_batch_id; - *next_batch_id += 1; + let current_batch_id = self.next_batch_id.fetch_add(1, Ordering::Relaxed); Ok(current_batch_id) } @@ -346,7 +342,7 @@ mod tests { let wal_file = WriteAheadLogFile::try_new(folder_path.clone(), &metadata.schema).unwrap(); assert!(wal_file.path.exists()); - assert_eq!(*wal_file.next_batch_id.lock().unwrap(), 0); + assert_eq!(wal_file.next_batch_id.load(Ordering::Relaxed), 0); } #[test] @@ -359,7 +355,7 @@ mod tests { let batches = wal_file.read_all().unwrap(); assert!(batches.is_empty()); - assert_eq!(*wal_file.next_batch_id.lock().unwrap(), 0); + assert_eq!(wal_file.next_batch_id.load(Ordering::Relaxed), 0); } #[test] @@ -376,7 +372,7 @@ mod tests { let batches = wal_file.read_all().unwrap(); assert_eq!(batches.len(), 1); assert_eq!(batches[0], batch); - assert_eq!(*wal_file.next_batch_id.lock().unwrap(), 1); + assert_eq!(wal_file.next_batch_id.load(Ordering::Relaxed), 1); } #[test] @@ -400,7 +396,7 @@ mod tests { assert_eq!(batches[0], batch_1); assert_eq!(batches[1], batch_2); assert_eq!(batches[2], batch_3); - assert_eq!(*wal_file.next_batch_id.lock().unwrap(), 3); + assert_eq!(wal_file.next_batch_id.load(Ordering::Relaxed), 3); } #[test] @@ -424,7 +420,7 @@ mod tests { assert_eq!(batches.len(), 2); assert_eq!(batches[0], batch_1); assert_eq!(batches[1], batch_2); - assert_eq!(*wal_file.next_batch_id.lock().unwrap(), 2); + assert_eq!(wal_file.next_batch_id.load(Ordering::Relaxed), 2); } #[test] @@ -444,7 +440,7 @@ mod tests { let batches = wal_file.read_all().unwrap(); assert_eq!(batches.len(), 1); assert_eq!(batches[0], batch); - assert_eq!(*wal_file.next_batch_id.lock().unwrap(), 1); + assert_eq!(wal_file.next_batch_id.load(Ordering::Relaxed), 1); } #[test] From 7238e4142ce9cc2664d1bfa824f56d202f60814f Mon Sep 17 00:00:00 2001 From: CGodiksen <36046286+CGodiksen@users.noreply.github.com> Date: Wed, 11 Mar 2026 12:05:56 +0100 Subject: [PATCH 048/116] Add treshold for when to rotate segments --- crates/modelardb_storage/src/write_ahead_log.rs | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/crates/modelardb_storage/src/write_ahead_log.rs b/crates/modelardb_storage/src/write_ahead_log.rs index 2d1f937c..7808ab29 100644 --- a/crates/modelardb_storage/src/write_ahead_log.rs +++ b/crates/modelardb_storage/src/write_ahead_log.rs @@ -34,8 +34,12 @@ use crate::WRITE_AHEAD_LOG_FOLDER; use crate::data_folder::DataFolder; use crate::error::{ModelarDbStorageError, Result}; +/// Folder containing the WAL files for the operations log. const OPERATIONS_LOG_FOLDER: &str = "operations"; +/// Number of batches to write to a single WAL segment file before rotating to a new one. +const SEGMENT_ROTATION_THRESHOLD: u64 = 100; + /// Write-ahead log that logs data on a per-table level and operations separately. pub struct WriteAheadLog { /// Path to the folder that contains the write-ahead log. From e3ab7bc18f241a5d2be4be9c97cc96f052f5d854 Mon Sep 17 00:00:00 2001 From: CGodiksen <36046286+CGodiksen@users.noreply.github.com> Date: Wed, 11 Mar 2026 12:07:38 +0100 Subject: [PATCH 049/116] Remove the entire folder when removing table log --- crates/modelardb_storage/src/write_ahead_log.rs | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/crates/modelardb_storage/src/write_ahead_log.rs b/crates/modelardb_storage/src/write_ahead_log.rs index 7808ab29..cd07a345 100644 --- a/crates/modelardb_storage/src/write_ahead_log.rs +++ b/crates/modelardb_storage/src/write_ahead_log.rs @@ -18,7 +18,7 @@ use std::collections::{BTreeSet, HashMap, HashSet}; use std::fs::{File, OpenOptions}; -use std::io::{Seek, SeekFrom}; +use std::io::Seek; use std::path::PathBuf; use std::sync::Mutex; use std::sync::atomic::{AtomicU64, Ordering}; @@ -112,13 +112,13 @@ impl WriteAheadLog { } } - /// Remove the log file for the table with the given name. If the log file does not exist or + /// Remove the log files for the table with the given name. If the log files do not exist or /// could not be removed, return [`ModelarDbStorageError`]. pub fn remove_table_log(&mut self, table_name: &str) -> Result<()> { let log_path; if let Some(log_file) = self.table_logs.remove(table_name) { - log_path = log_file.path; + log_path = log_file.folder_path; // log_file is dropped here as it goes out of scope which automatically closes its // internal file handle. } else { @@ -127,8 +127,8 @@ impl WriteAheadLog { ))); } - // Now that the file handle is closed, the file can be removed. - std::fs::remove_file(log_path)?; + // Now that the file handle is closed, the files can be removed. + std::fs::remove_dir_all(log_path)?; Ok(()) } From c2e109f66e677d95e35488e2b11c8699414c702a Mon Sep 17 00:00:00 2001 From: CGodiksen <36046286+CGodiksen@users.noreply.github.com> Date: Wed, 11 Mar 2026 12:11:21 +0100 Subject: [PATCH 050/116] Add struct for completed write ahead log segments --- .../modelardb_storage/src/write_ahead_log.rs | 18 ++++++++++++++++++ 1 file changed, 18 insertions(+) diff --git a/crates/modelardb_storage/src/write_ahead_log.rs b/crates/modelardb_storage/src/write_ahead_log.rs index cd07a345..90b127a4 100644 --- a/crates/modelardb_storage/src/write_ahead_log.rs +++ b/crates/modelardb_storage/src/write_ahead_log.rs @@ -166,6 +166,24 @@ impl WriteAheadLog { } } +/// A closed WAL segment file. The file contains all batches with ids in `[start_id, end_id]` +/// and will not be written to again. +struct WriteAheadLogSegment { + /// Path to the segment file on disk. + path: PathBuf, + /// Batch id of the first batch in this segment. + start_id: u64, + /// Batch id of the last batch in this segment (inclusive). + end_id: u64, +} + +impl WriteAheadLogSegment { + /// Return `true` if every batch id in this segment is present in `persisted`. + fn is_fully_persisted(&self, persisted: &BTreeSet) -> bool { + (self.start_id..=self.end_id).all(|id| persisted.contains(&id)) + } +} + /// Wrapper around a [`File`] that enforces that [`sync_data()`](File::sync_data) is called /// immediately after writing to ensure that all data is on disk before returning. Note that /// an exclusive lock is held on the file while it is being written to. From ef0e7616e5c63296a88aefb0bf135b64e912c6ff Mon Sep 17 00:00:00 2001 From: CGodiksen <36046286+CGodiksen@users.noreply.github.com> Date: Wed, 11 Mar 2026 12:29:27 +0100 Subject: [PATCH 051/116] Add struct for active segments --- crates/modelardb_storage/src/write_ahead_log.rs | 17 +++++++++++++++-- 1 file changed, 15 insertions(+), 2 deletions(-) diff --git a/crates/modelardb_storage/src/write_ahead_log.rs b/crates/modelardb_storage/src/write_ahead_log.rs index 90b127a4..4555402a 100644 --- a/crates/modelardb_storage/src/write_ahead_log.rs +++ b/crates/modelardb_storage/src/write_ahead_log.rs @@ -168,7 +168,7 @@ impl WriteAheadLog { /// A closed WAL segment file. The file contains all batches with ids in `[start_id, end_id]` /// and will not be written to again. -struct WriteAheadLogSegment { +struct ClosedSegment { /// Path to the segment file on disk. path: PathBuf, /// Batch id of the first batch in this segment. @@ -177,13 +177,26 @@ struct WriteAheadLogSegment { end_id: u64, } -impl WriteAheadLogSegment { +impl ClosedSegment { /// Return `true` if every batch id in this segment is present in `persisted`. fn is_fully_persisted(&self, persisted: &BTreeSet) -> bool { (self.start_id..=self.end_id).all(|id| persisted.contains(&id)) } } +/// The currently active WAL segment being written to. All fields are mutated together +/// during rotation and are protected by the mutex in [`WriteAheadLogFile`]. +struct ActiveSegment { + /// Path to the active segment file. + path: PathBuf, + /// Batch id of the first batch written to this segment. + start_id: u64, + /// Writer to write data in IPC streaming format to this segment file. + writer: StreamWriter, + /// The batch id to give to the next batch of data. Monotonically increasing across rotations. + next_batch_id: u64, +} + /// Wrapper around a [`File`] that enforces that [`sync_data()`](File::sync_data) is called /// immediately after writing to ensure that all data is on disk before returning. Note that /// an exclusive lock is held on the file while it is being written to. From 7cb50d6fcedfe36b33d839103c3dc8443a82536e Mon Sep 17 00:00:00 2001 From: CGodiksen <36046286+CGodiksen@users.noreply.github.com> Date: Wed, 11 Mar 2026 12:30:54 +0100 Subject: [PATCH 052/116] Update WriteAheadLogFile fields to use new closed and active segments --- .../modelardb_storage/src/write_ahead_log.rs | 23 ++++++++++--------- 1 file changed, 12 insertions(+), 11 deletions(-) diff --git a/crates/modelardb_storage/src/write_ahead_log.rs b/crates/modelardb_storage/src/write_ahead_log.rs index 4555402a..e104e60e 100644 --- a/crates/modelardb_storage/src/write_ahead_log.rs +++ b/crates/modelardb_storage/src/write_ahead_log.rs @@ -199,18 +199,19 @@ struct ActiveSegment { /// Wrapper around a [`File`] that enforces that [`sync_data()`](File::sync_data) is called /// immediately after writing to ensure that all data is on disk before returning. Note that -/// an exclusive lock is held on the file while it is being written to. +/// an exclusive lock is held on the file while it is being written to. At any point in time there +/// is exactly one active segment being written to plus zero or more closed segments that are +/// read-only. The active segment is rotated into the closed list once [`SEGMENT_ROTATION_THRESHOLD`] +/// batches have been written to it. struct WriteAheadLogFile { - /// Path to the file that the log is written to. - path: PathBuf, - /// Writer to write data in IPC streaming format to the log file. - writer: Mutex>, - /// The offset encoded in the WAL file name. This represents the number of batches that have - /// been removed from the start of the file across all previous truncations. - batch_offset: u64, - /// The batch id to give to the next batch of data appended to the log file. This is incremented - /// after each append, so the batch id given to data is monotonically increasing. - next_batch_id: AtomicU64, + /// Folder that contains all segment files for this log. + folder_path: PathBuf, + /// Arrow schema shared by every segment in this log. + schema: Schema, + /// The active segment currently being written to. + active_segment: Mutex, + /// Closed, read-only segment files ordered by `start_id`. + closed_segments: Mutex>, /// Batch ids that have been confirmed as saved to disk. Used to determine whether a /// contiguous prefix of batches can be trimmed from the start of the log file. persisted_batch_ids: Mutex>, From 72f30ecc5d5eb66ad19016656f21995b70bc6354 Mon Sep 17 00:00:00 2001 From: CGodiksen <36046286+CGodiksen@users.noreply.github.com> Date: Wed, 11 Mar 2026 12:37:29 +0100 Subject: [PATCH 053/116] Add util function for finding closed segments --- .../modelardb_storage/src/write_ahead_log.rs | 26 +++++++++++++++++++ 1 file changed, 26 insertions(+) diff --git a/crates/modelardb_storage/src/write_ahead_log.rs b/crates/modelardb_storage/src/write_ahead_log.rs index e104e60e..d6182341 100644 --- a/crates/modelardb_storage/src/write_ahead_log.rs +++ b/crates/modelardb_storage/src/write_ahead_log.rs @@ -355,6 +355,32 @@ fn find_existing_wal_file(folder_path: &PathBuf) -> Result Result> { + let mut segments = Vec::new(); + + for entry in std::fs::read_dir(folder_path)? { + let path = entry?.path(); + let stem = path + .file_stem() + .and_then(|s| s.to_str()) + .expect("WAL file should have a valid UTF-8 stem."); + + if let Some((start_id, end_id)) = stem + .split_once('-') + .and_then(|(s, e)| Some((s.parse::().ok()?, e.parse::().ok()?))) + { + segments.push(ClosedSegment { + path, + start_id, + end_id, + }); + } + } + + Ok(segments) +} } /// Return the schema for the operations log that is stored in [`OPERATIONS_LOG_FOLDER`]. From 39a017f1578f938cecdd0334a92f0fba3dec3fcc Mon Sep 17 00:00:00 2001 From: CGodiksen <36046286+CGodiksen@users.noreply.github.com> Date: Wed, 11 Mar 2026 12:55:52 +0100 Subject: [PATCH 054/116] Add util method to close leftover active segment --- .../modelardb_storage/src/write_ahead_log.rs | 41 ++++++++++++++++++- 1 file changed, 40 insertions(+), 1 deletion(-) diff --git a/crates/modelardb_storage/src/write_ahead_log.rs b/crates/modelardb_storage/src/write_ahead_log.rs index d6182341..1fc96362 100644 --- a/crates/modelardb_storage/src/write_ahead_log.rs +++ b/crates/modelardb_storage/src/write_ahead_log.rs @@ -353,8 +353,47 @@ fn find_existing_wal_file(folder_path: &PathBuf) -> Result().ok()?; Some((path, offset)) +/// If a leftover active segment (`{start_id}-.wal`) exists in `folder_path`, rename it to +/// its final `{start_id}-{end_id}.wal` name so it is picked up as a closed segment. If the +/// file contains no batches, it is removed instead. If the file could not be renamed or +/// removed, return [`ModelarDbStorageError`]. +fn close_leftover_active_segment(folder_path: &PathBuf) -> Result<()> { + let Some(active_path) = std::fs::read_dir(folder_path)? + .filter_map(|e| e.ok()) + .map(|e| e.path()) + .find(|p| { + p.file_stem() + .and_then(|s| s.to_str()) + .is_some_and(|stem| stem.ends_with('-')) }) - .next()) + else { + return Ok(()); + }; + + let stem = active_path + .file_stem() + .and_then(|s| s.to_str()) + .expect("Active WAL segment stem should be '{start_id}-'."); + + let start_id: u64 = stem[..stem.len() - 1] + .parse() + .expect("Active WAL segment stem should start with a valid u64."); + + let batches = read_batches_from_path(&active_path)?; + + if batches.is_empty() { + std::fs::remove_file(&active_path)?; + } else { + let end_id = start_id + batches.len() as u64 - 1; + std::fs::rename( + &active_path, + folder_path.join(format!("{start_id}-{end_id}.wal")), + )?; + } + + Ok(()) +} + /// Collect all closed segment files in `folder_path`. Closed segments have names of the form /// `{start_id}-{end_id}.wal` where both `start_id` and `end_id` are valid `u64` values. fn find_closed_segments(folder_path: &PathBuf) -> Result> { From 2bb6a39b057d5be75c98419aec59c832105cb15c Mon Sep 17 00:00:00 2001 From: CGodiksen <36046286+CGodiksen@users.noreply.github.com> Date: Wed, 11 Mar 2026 12:58:33 +0100 Subject: [PATCH 055/116] Add util method to read batches from path with tolerance for missing EOF --- .../modelardb_storage/src/write_ahead_log.rs | 23 +++++++++++++++++++ 1 file changed, 23 insertions(+) diff --git a/crates/modelardb_storage/src/write_ahead_log.rs b/crates/modelardb_storage/src/write_ahead_log.rs index 1fc96362..ade936e0 100644 --- a/crates/modelardb_storage/src/write_ahead_log.rs +++ b/crates/modelardb_storage/src/write_ahead_log.rs @@ -420,6 +420,29 @@ fn find_closed_segments(folder_path: &PathBuf) -> Result> { Ok(segments) } + +/// Read all [`RecordBatches`] from the file at `path`. Tolerates a missing end-of-stream +/// marker, which is normal for the active segment. If the file could not be read, return +/// [`ModelarDbStorageError`]. +fn read_batches_from_path(path: &PathBuf) -> Result> { + let file = File::open(path)?; + let reader = StreamReader::try_new(file, None)?; + + let mut batches = Vec::new(); + for maybe_batch in reader { + match maybe_batch { + Ok(batch) => batches.push(batch), + Err(IpcError(msg)) => { + if msg.contains("UnexpectedEof") || msg.contains("unexpected end of file") { + break; + } + return Err(IpcError(msg).into()); + } + Err(e) => return Err(e.into()), + } + } + + Ok(batches) } /// Return the schema for the operations log that is stored in [`OPERATIONS_LOG_FOLDER`]. From 2e21bf4499ec140868039a42ccb85fa6a27ccb67 Mon Sep 17 00:00:00 2001 From: CGodiksen <36046286+CGodiksen@users.noreply.github.com> Date: Wed, 11 Mar 2026 13:15:57 +0100 Subject: [PATCH 056/116] Add new try_new method that finds closed segments and uses new ActiveSegment struct --- .../modelardb_storage/src/write_ahead_log.rs | 60 +++++++++---------- 1 file changed, 29 insertions(+), 31 deletions(-) diff --git a/crates/modelardb_storage/src/write_ahead_log.rs b/crates/modelardb_storage/src/write_ahead_log.rs index ade936e0..7179ed1d 100644 --- a/crates/modelardb_storage/src/write_ahead_log.rs +++ b/crates/modelardb_storage/src/write_ahead_log.rs @@ -218,47 +218,45 @@ struct WriteAheadLogFile { } impl WriteAheadLogFile { - /// Create a new [`WriteAheadLogFile`] that appends data with `schema` to a file in - /// `folder_path`. If the file does not exist, it is created. If the file could not be created, - /// return [`ModelarDbStorageError`]. + /// Create a new [`WriteAheadLogFile`] that appends data with `schema` to segment files in + /// `folder_path`. Existing closed segment files are loaded into the closed-segment list. + /// A fresh active segment is always created on start-up. If the folder or file could not be + /// created, return [`ModelarDbStorageError`]. fn try_new(folder_path: PathBuf, schema: &Schema) -> Result { std::fs::create_dir_all(folder_path.clone())?; - let (file_path, batch_offset) = - find_existing_wal_file(&folder_path)?.unwrap_or_else(|| (folder_path.join("0.wal"), 0)); + close_leftover_active_segment(&folder_path)?; + + // Collect all closed segment files already on disk and sort them by start_id. + let mut closed_segments = find_closed_segments(&folder_path)?; + closed_segments.sort_by_key(|s| s.start_id); + + // The next batch id is one past the end of the last closed segment, or 0 if there are none. + let next_id = closed_segments.last().map(|s| s.end_id + 1).unwrap_or(0); - let file = OpenOptions::new() + // Always create a fresh active segment on startup to avoid writing into the middle of + // an existing IPC stream. + let active_path = folder_path.join(format!("{next_id}-.wal")); + let active_file = OpenOptions::new() .create(true) .read(true) .write(true) - .open(file_path.clone())?; - - let file_len = file.metadata()?.len(); - - let writer = StreamWriter::try_new(file, schema)?; + .truncate(true) + .open(&active_path)?; - // If the file already had data, the StreamWriter wrote a duplicate schema header. - // Truncate back to the original length to remove it, then seek to the end so - // subsequent writes append correctly. - if file_len > 0 { - writer.get_ref().set_len(file_len)?; - writer.get_ref().seek(SeekFrom::End(0))?; - } - - // Count existing batches to reconstruct the next batch ID. - let batch_count = if file_len > 0 { - let file = File::open(&file_path)?; - let reader = StreamReader::try_new(file, None)?; - reader.take_while(|r| r.is_ok()).count() as u64 - } else { - 0 - }; + let writer = StreamWriter::try_new(active_file, schema)?; Ok(Self { - path: file_path, - writer: Mutex::new(writer), - batch_offset, - next_batch_id: AtomicU64::new(batch_offset + batch_count), + folder_path, + schema: schema.clone(), + active_segment: Mutex::new(ActiveSegment { + path: active_path, + start_id: next_id, + writer, + next_batch_id: next_id, + }), + closed_segments: Mutex::new(closed_segments), + // TODO: This needs to be initialized with persisted batch ids from Delta Lake. persisted_batch_ids: Mutex::new(BTreeSet::new()), }) } From 222a74777b4692a1a6adc9fed2007219a827e7e4 Mon Sep 17 00:00:00 2001 From: CGodiksen <36046286+CGodiksen@users.noreply.github.com> Date: Wed, 11 Mar 2026 13:54:34 +0100 Subject: [PATCH 057/116] Refactor append_and_sync so it rotates segments if the treshold is hit --- .../modelardb_storage/src/write_ahead_log.rs | 29 ++++++++++++++----- 1 file changed, 21 insertions(+), 8 deletions(-) diff --git a/crates/modelardb_storage/src/write_ahead_log.rs b/crates/modelardb_storage/src/write_ahead_log.rs index 7179ed1d..f4007d18 100644 --- a/crates/modelardb_storage/src/write_ahead_log.rs +++ b/crates/modelardb_storage/src/write_ahead_log.rs @@ -261,24 +261,37 @@ impl WriteAheadLogFile { }) } - /// Append the given data to the log file and sync the file to ensure that all data is on disk. - /// Return the batch id given to the appended data. If the data could not be appended or the - /// file could not be synced, return [`ModelarDbStorageError`]. + /// Append the given data to the active segment and sync the file to ensure that all data is on + /// disk. Return the batch id given to the appended data. Rotates to a new segment file if + /// [`SEGMENT_ROTATION_THRESHOLD`] is reached. If the data could not be appended or the file + /// could not be synced, return [`ModelarDbStorageError`]. fn append_and_sync(&self, data: &RecordBatch) -> Result { // Acquire the mutex to ensure only one thread can write at a time. - let mut writer = self.writer.lock().expect("Mutex should not be poisoned."); + let mut active = self + .active_segment + .lock() + .expect("Mutex should not be poisoned."); - writer.write(data)?; + active.writer.write(data)?; // Flush the writer's internal buffers to the file. - writer.flush()?; + active.writer.flush()?; // Get a reference to the underlying file handle and sync to disk. Note that file metadata // such as modification timestamps and permissions are not updated since we only sync data. - writer.get_ref().sync_data()?; + active.writer.get_ref().sync_data()?; // Increment the batch id for the next batch of data. - let current_batch_id = self.next_batch_id.fetch_add(1, Ordering::Relaxed); + let current_batch_id = active.next_batch_id; + active.next_batch_id += 1; + + // Rotate to a new segment if the threshold has been reached. The number of batches in the + // active segment is the difference between the next batch id (post-increment) and the + // active start id. + let active_batch_count = active.next_batch_id - active.start_id; + if active_batch_count >= SEGMENT_ROTATION_THRESHOLD { + self.rotate_active_segment(&mut active)?; + } Ok(current_batch_id) } From affbb910c865c690e95aa17121f4fb1bd305e3cd Mon Sep 17 00:00:00 2001 From: CGodiksen <36046286+CGodiksen@users.noreply.github.com> Date: Wed, 11 Mar 2026 13:58:36 +0100 Subject: [PATCH 058/116] Add new method to rotate active segment --- .../modelardb_storage/src/write_ahead_log.rs | 40 +++++++++++++++++++ 1 file changed, 40 insertions(+) diff --git a/crates/modelardb_storage/src/write_ahead_log.rs b/crates/modelardb_storage/src/write_ahead_log.rs index f4007d18..126466ed 100644 --- a/crates/modelardb_storage/src/write_ahead_log.rs +++ b/crates/modelardb_storage/src/write_ahead_log.rs @@ -299,6 +299,46 @@ impl WriteAheadLogFile { /// Mark the given batch ids as saved to disk. If a large enough contiguous prefix of batches /// is marked as persisted, the log file is trimmed to remove the persisted data. If the /// file could not be trimmed, return [`ModelarDbStorageError`]. + /// Close the current active segment by renaming it to its final `{start_id}-{end_id}.wal` + /// name and open a fresh active segment. The caller must hold the `active_segment` lock. + fn rotate_active_segment(&self, active: &mut ActiveSegment) -> Result<()> { + let end_id = active.next_batch_id - 1; + + // Finish the current writer so the IPC end-of-stream marker is written. + active.writer.finish()?; + + // Rename the active file to its permanent name. + let closed_path = self + .folder_path + .join(format!("{}-{end_id}.wal", active.start_id)); + std::fs::rename(&active.path, &closed_path)?; + + self.closed_segments + .lock() + .expect("Mutex should not be poisoned.") + .push(ClosedSegment { + path: closed_path, + start_id: active.start_id, + end_id, + }); + + // Open a fresh active segment. + let next_id = end_id + 1; + let new_active_path = self.folder_path.join(format!("{next_id}-.wal")); + let new_file = OpenOptions::new() + .create(true) + .read(true) + .write(true) + .truncate(true) + .open(&new_active_path)?; + + active.writer = StreamWriter::try_new(new_file, &self.schema)?; + active.path = new_active_path; + active.start_id = next_id; + + Ok(()) + } + fn mark_batches_as_persisted(&self, batch_ids: HashSet) -> Result<()> { let mut persisted = self .persisted_batch_ids From 5d91b70dd74ea87cbf169f152b255e8540613818 Mon Sep 17 00:00:00 2001 From: CGodiksen <36046286+CGodiksen@users.noreply.github.com> Date: Wed, 11 Mar 2026 14:08:16 +0100 Subject: [PATCH 059/116] Use ActiveSegment initialization function to remove duplicated code --- .../modelardb_storage/src/write_ahead_log.rs | 53 ++++++++++--------- 1 file changed, 27 insertions(+), 26 deletions(-) diff --git a/crates/modelardb_storage/src/write_ahead_log.rs b/crates/modelardb_storage/src/write_ahead_log.rs index 126466ed..7c945ae6 100644 --- a/crates/modelardb_storage/src/write_ahead_log.rs +++ b/crates/modelardb_storage/src/write_ahead_log.rs @@ -197,6 +197,29 @@ struct ActiveSegment { next_batch_id: u64, } +impl ActiveSegment { + /// Create a new [`ActiveSegment`] in `folder_path` with the given `start_id` and `schema`. + /// If the file could not be created, return [`ModelarDbStorageError`]. + fn try_new(folder_path: PathBuf, schema: &Schema, start_id: u64) -> Result { + let path = folder_path.join(format!("{start_id}-.wal")); + let file = OpenOptions::new() + .create(true) + .read(true) + .write(true) + .truncate(true) + .open(&path)?; + + let writer = StreamWriter::try_new(file, schema)?; + + Ok(Self { + path, + start_id, + writer, + next_batch_id: start_id, + }) + } +} + /// Wrapper around a [`File`] that enforces that [`sync_data()`](File::sync_data) is called /// immediately after writing to ensure that all data is on disk before returning. Note that /// an exclusive lock is held on the file while it is being written to. At any point in time there @@ -236,25 +259,12 @@ impl WriteAheadLogFile { // Always create a fresh active segment on startup to avoid writing into the middle of // an existing IPC stream. - let active_path = folder_path.join(format!("{next_id}-.wal")); - let active_file = OpenOptions::new() - .create(true) - .read(true) - .write(true) - .truncate(true) - .open(&active_path)?; - - let writer = StreamWriter::try_new(active_file, schema)?; + let active_file = ActiveSegment::try_new(folder_path.clone(), schema, next_id)?; Ok(Self { folder_path, schema: schema.clone(), - active_segment: Mutex::new(ActiveSegment { - path: active_path, - start_id: next_id, - writer, - next_batch_id: next_id, - }), + active_segment: Mutex::new(active_file), closed_segments: Mutex::new(closed_segments), // TODO: This needs to be initialized with persisted batch ids from Delta Lake. persisted_batch_ids: Mutex::new(BTreeSet::new()), @@ -324,17 +334,8 @@ impl WriteAheadLogFile { // Open a fresh active segment. let next_id = end_id + 1; - let new_active_path = self.folder_path.join(format!("{next_id}-.wal")); - let new_file = OpenOptions::new() - .create(true) - .read(true) - .write(true) - .truncate(true) - .open(&new_active_path)?; - - active.writer = StreamWriter::try_new(new_file, &self.schema)?; - active.path = new_active_path; - active.start_id = next_id; + let new_file = ActiveSegment::try_new(self.folder_path.clone(), &self.schema, next_id)?; + *active = new_file; Ok(()) } From cd39c88c3a44bcd5f23d7e6f575cbfdcebdaafa8 Mon Sep 17 00:00:00 2001 From: CGodiksen <36046286+CGodiksen@users.noreply.github.com> Date: Wed, 11 Mar 2026 14:18:14 +0100 Subject: [PATCH 060/116] Now deleting fully persisted segments when marking batches as persisted --- .../modelardb_storage/src/write_ahead_log.rs | 43 +++++++++++-------- 1 file changed, 24 insertions(+), 19 deletions(-) diff --git a/crates/modelardb_storage/src/write_ahead_log.rs b/crates/modelardb_storage/src/write_ahead_log.rs index 7c945ae6..195af2c1 100644 --- a/crates/modelardb_storage/src/write_ahead_log.rs +++ b/crates/modelardb_storage/src/write_ahead_log.rs @@ -142,10 +142,9 @@ impl WriteAheadLog { log_file.append_and_sync(data) } - /// Mark the given batch ids as saved to disk in the corresponding table log. If a large enough - /// contiguous prefix of batches is marked as persisted, the log file is trimmed to remove - /// the persisted data. If a table log does not exist or the log file could not be trimmed, - /// return [`ModelarDbStorageError`]. + /// Mark the given batch ids as saved to disk in the corresponding table log. Fully persisted + /// segment files are deleted. If a table log does not exist or a segment file could not be + /// deleted, return [`ModelarDbStorageError`]. pub fn mark_batches_as_persisted_in_table_log( &self, table_name: &str, @@ -306,9 +305,6 @@ impl WriteAheadLogFile { Ok(current_batch_id) } - /// Mark the given batch ids as saved to disk. If a large enough contiguous prefix of batches - /// is marked as persisted, the log file is trimmed to remove the persisted data. If the - /// file could not be trimmed, return [`ModelarDbStorageError`]. /// Close the current active segment by renaming it to its final `{start_id}-{end_id}.wal` /// name and open a fresh active segment. The caller must hold the `active_segment` lock. fn rotate_active_segment(&self, active: &mut ActiveSegment) -> Result<()> { @@ -334,12 +330,14 @@ impl WriteAheadLogFile { // Open a fresh active segment. let next_id = end_id + 1; - let new_file = ActiveSegment::try_new(self.folder_path.clone(), &self.schema, next_id)?; - *active = new_file; + *active = ActiveSegment::try_new(self.folder_path.clone(), &self.schema, next_id)?; Ok(()) } + /// Mark the given batch ids as saved to disk. Any closed segment whose entire batch-id range + /// is now persisted is deleted from disk and removed from the in-memory list. If a segment file + /// could not be deleted, return [`ModelarDbStorageError`]. fn mark_batches_as_persisted(&self, batch_ids: HashSet) -> Result<()> { let mut persisted = self .persisted_batch_ids @@ -348,18 +346,25 @@ impl WriteAheadLogFile { persisted.extend(batch_ids); - // Walk forward from batch_offset to find the contiguous prefix watermark. - let mut watermark = self.batch_offset; - while persisted.contains(&watermark) { - watermark += 1; + let mut closed_segments = self + .closed_segments + .lock() + .expect("Mutex should not be poisoned."); + + // Identify and delete fully persisted segments. + let mut to_retain = Vec::new(); + for segment in closed_segments.drain(..) { + if segment.is_fully_persisted(&persisted) { + std::fs::remove_file(&segment.path)?; + + // Remove the persisted ids for this segment as they are no longer needed. + for id in segment.start_id..=segment.end_id { + persisted.remove(&id); + } + } else { + to_retain.push(segment); } - // If watermark advanced, we have a contiguous prefix ending at watermark - 1. - let max_prefix_batch_id = if watermark > self.batch_offset { - Some(watermark - 1) - } else { - None - }; Ok(()) } From 1f8d98d3acea11debca9f601881608af4cee9942 Mon Sep 17 00:00:00 2001 From: CGodiksen <36046286+CGodiksen@users.noreply.github.com> Date: Wed, 11 Mar 2026 14:20:08 +0100 Subject: [PATCH 061/116] Read both from closed and active segment in read_all --- .../modelardb_storage/src/write_ahead_log.rs | 56 +++++++------------ 1 file changed, 21 insertions(+), 35 deletions(-) diff --git a/crates/modelardb_storage/src/write_ahead_log.rs b/crates/modelardb_storage/src/write_ahead_log.rs index 195af2c1..4f4197eb 100644 --- a/crates/modelardb_storage/src/write_ahead_log.rs +++ b/crates/modelardb_storage/src/write_ahead_log.rs @@ -363,53 +363,39 @@ impl WriteAheadLogFile { } } else { to_retain.push(segment); + } } + *closed_segments = to_retain; Ok(()) } - /// Read all data from the log file. This can be called even if the [`StreamWriter`] has not - /// been finished, meaning the log file is missing the end-of-stream bytes. If the file - /// could not be read, return [`ModelarDbStorageError`]. + /// Read all data from all segment files (closed and active) in order. If any file could not + /// be read, return [`ModelarDbStorageError`]. fn read_all(&self) -> Result> { - // Acquire the mutex to ensure data is not being written while reading. Note that reading - // should only occur during recovery, which should make concurrent writes improbable. - // However, since performance is not critical during recovery, the mutex is held anyway. - let _writer = self.writer.lock().unwrap(); - - let file = File::open(&self.path)?; - let reader = StreamReader::try_new(file, None)?; - - let mut batches = Vec::new(); - for maybe_batch in reader { - match maybe_batch { - Ok(batch) => batches.push(batch), - Err(IpcError(msg)) => { - // Check if it is an UnexpectedEof error, which is expected when reading - // an incomplete stream without the end-of-stream marker. - if msg.contains("UnexpectedEof") || msg.contains("unexpected end of file") { - break; - } - return Err(IpcError(msg).into()); - } - Err(e) => return Err(e.into()), - } + // Acquire the mutex to ensure data is not being written while reading. + let active = self + .active_segment + .lock() + .expect("Mutex should not be poisoned."); + + let closed_segments = self + .closed_segments + .lock() + .expect("Mutex should not be poisoned."); + + let mut all_batches = Vec::new(); + for segment in closed_segments.iter() { + all_batches.extend(read_batches_from_path(&segment.path)?); } - Ok(batches) + all_batches.extend(read_batches_from_path(&active.path)?); + + Ok(all_batches) } } -/// Find an existing WAL file in `folder_path` and return its path and the offset parsed from its -/// name if it exists, otherwise return `Ok(None)`. -fn find_existing_wal_file(folder_path: &PathBuf) -> Result> { - Ok(std::fs::read_dir(folder_path)? - .filter_map(|maybe_file| maybe_file.ok()) - .filter_map(|file| { - let path = file.path(); - let offset = path.file_stem()?.to_str()?.parse::().ok()?; - Some((path, offset)) /// If a leftover active segment (`{start_id}-.wal`) exists in `folder_path`, rename it to /// its final `{start_id}-{end_id}.wal` name so it is picked up as a closed segment. If the /// file contains no batches, it is removed instead. If the file could not be renamed or From 8b329b3a602e74dc7fcc5d37452e2829ff306abc Mon Sep 17 00:00:00 2001 From: CGodiksen <36046286+CGodiksen@users.noreply.github.com> Date: Wed, 11 Mar 2026 14:46:31 +0100 Subject: [PATCH 062/116] Update tests to match new segment implementation --- .../modelardb_storage/src/write_ahead_log.rs | 152 ++++++++++-------- 1 file changed, 88 insertions(+), 64 deletions(-) diff --git a/crates/modelardb_storage/src/write_ahead_log.rs b/crates/modelardb_storage/src/write_ahead_log.rs index 4f4197eb..2328fc2c 100644 --- a/crates/modelardb_storage/src/write_ahead_log.rs +++ b/crates/modelardb_storage/src/write_ahead_log.rs @@ -18,10 +18,8 @@ use std::collections::{BTreeSet, HashMap, HashSet}; use std::fs::{File, OpenOptions}; -use std::io::Seek; use std::path::PathBuf; use std::sync::Mutex; -use std::sync::atomic::{AtomicU64, Ordering}; use arrow::datatypes::{DataType, Field, Schema}; use arrow::error::ArrowError::IpcError; @@ -37,6 +35,7 @@ use crate::error::{ModelarDbStorageError, Result}; /// Folder containing the WAL files for the operations log. const OPERATIONS_LOG_FOLDER: &str = "operations"; +// TODO: Look into using a byte size based threshold instead of a number of batches. /// Number of batches to write to a single WAL segment file before rotating to a new one. const SEGMENT_ROTATION_THRESHOLD: u64 = 100; @@ -499,56 +498,48 @@ mod tests { use modelardb_test::table; use modelardb_test::table::TIME_SERIES_TABLE_NAME; + use tempfile::TempDir; #[test] - fn test_try_new_creates_file() { - let temp_dir = tempfile::tempdir().unwrap(); - let folder_path = temp_dir.path().join(TIME_SERIES_TABLE_NAME); + fn test_try_new_creates_active_segment() { + let (_temp_dir, wal_file) = new_wal_file(); - let metadata = table::time_series_table_metadata(); - let wal_file = WriteAheadLogFile::try_new(folder_path.clone(), &metadata.schema).unwrap(); + let active = wal_file.active_segment.lock().unwrap(); + assert!(active.path.exists()); + assert_eq!(active.next_batch_id, 0); - assert!(wal_file.path.exists()); - assert_eq!(wal_file.next_batch_id.load(Ordering::Relaxed), 0); + assert!(wal_file.closed_segments.lock().unwrap().is_empty()); } #[test] fn test_read_all_empty_file() { - let temp_dir = tempfile::tempdir().unwrap(); - let folder_path = temp_dir.path().join(TIME_SERIES_TABLE_NAME); - - let metadata = table::time_series_table_metadata(); - let wal_file = WriteAheadLogFile::try_new(folder_path, &metadata.schema).unwrap(); + let (_temp_dir, wal_file) = new_wal_file(); let batches = wal_file.read_all().unwrap(); assert!(batches.is_empty()); - assert_eq!(wal_file.next_batch_id.load(Ordering::Relaxed), 0); + + let active = wal_file.active_segment.lock().unwrap(); + assert_eq!(active.next_batch_id, 0); } #[test] fn test_append_and_read_single_batch() { - let temp_dir = tempfile::tempdir().unwrap(); - let folder_path = temp_dir.path().join(TIME_SERIES_TABLE_NAME); + let (_temp_dir, wal_file) = new_wal_file(); - let metadata = table::time_series_table_metadata(); - let wal_file = WriteAheadLogFile::try_new(folder_path, &metadata.schema).unwrap(); let batch = table::uncompressed_time_series_table_record_batch(5); - wal_file.append_and_sync(&batch).unwrap(); let batches = wal_file.read_all().unwrap(); assert_eq!(batches.len(), 1); assert_eq!(batches[0], batch); - assert_eq!(wal_file.next_batch_id.load(Ordering::Relaxed), 1); + + let active = wal_file.active_segment.lock().unwrap(); + assert_eq!(active.next_batch_id, 1); } #[test] fn test_append_and_read_multiple_batches() { - let temp_dir = tempfile::tempdir().unwrap(); - let folder_path = temp_dir.path().join(TIME_SERIES_TABLE_NAME); - - let metadata = table::time_series_table_metadata(); - let wal_file = WriteAheadLogFile::try_new(folder_path, &metadata.schema).unwrap(); + let (_temp_dir, wal_file) = new_wal_file(); let batch_1 = table::uncompressed_time_series_table_record_batch(10); let batch_2 = table::uncompressed_time_series_table_record_batch(20); @@ -563,79 +554,112 @@ mod tests { assert_eq!(batches[0], batch_1); assert_eq!(batches[1], batch_2); assert_eq!(batches[2], batch_3); - assert_eq!(wal_file.next_batch_id.load(Ordering::Relaxed), 3); + + let active = wal_file.active_segment.lock().unwrap(); + assert_eq!(active.next_batch_id, 3); } #[test] - fn test_reopen_existing_file_and_append() { + fn test_segment_rotates_at_threshold() { + let (_temp_dir, wal_file) = new_wal_file(); + + let batch = table::uncompressed_time_series_table_record_batch(5); + + for _ in 0..SEGMENT_ROTATION_THRESHOLD { + wal_file.append_and_sync(&batch).unwrap(); + } + + let closed = wal_file.closed_segments.lock().unwrap(); + assert_eq!(closed.len(), 1); + assert_eq!(closed[0].start_id, 0); + assert_eq!(closed[0].end_id, SEGMENT_ROTATION_THRESHOLD - 1); + + let active = wal_file.active_segment.lock().unwrap(); + assert_eq!(active.start_id, SEGMENT_ROTATION_THRESHOLD); + } + + #[test] + fn test_reopen_loads_closed_segments() { let temp_dir = tempfile::tempdir().unwrap(); let folder_path = temp_dir.path().join(TIME_SERIES_TABLE_NAME); - let metadata = table::time_series_table_metadata(); - let batch_1 = table::uncompressed_time_series_table_record_batch(10); + + let batch = table::uncompressed_time_series_table_record_batch(10); + + // Write enough batches to trigger a rotation, then drop. { let wal_file = WriteAheadLogFile::try_new(folder_path.clone(), &metadata.schema).unwrap(); - wal_file.append_and_sync(&batch_1).unwrap(); + for _ in 0..SEGMENT_ROTATION_THRESHOLD { + wal_file.append_and_sync(&batch).unwrap(); + } } - let batch_2 = table::uncompressed_time_series_table_record_batch(20); - let wal_file = WriteAheadLogFile::try_new(folder_path, &metadata.schema).unwrap(); - wal_file.append_and_sync(&batch_2).unwrap(); + // The closed segment should be detected and the next id should continue. + let wal_file = WriteAheadLogFile::try_new(folder_path.clone(), &metadata.schema).unwrap(); - let batches = wal_file.read_all().unwrap(); - assert_eq!(batches.len(), 2); - assert_eq!(batches[0], batch_1); - assert_eq!(batches[1], batch_2); - assert_eq!(wal_file.next_batch_id.load(Ordering::Relaxed), 2); + let active = wal_file.active_segment.lock().unwrap(); + assert_eq!(active.next_batch_id, SEGMENT_ROTATION_THRESHOLD); + assert_eq!(wal_file.closed_segments.lock().unwrap().len(), 1); } #[test] - fn test_reopen_existing_file_and_read_without_append() { + fn test_reopen_and_append_continues_batch_ids() { let temp_dir = tempfile::tempdir().unwrap(); let folder_path = temp_dir.path().join(TIME_SERIES_TABLE_NAME); - let metadata = table::time_series_table_metadata(); + let batch = table::uncompressed_time_series_table_record_batch(10); + + // Fill and rotate one segment. { let wal_file = WriteAheadLogFile::try_new(folder_path.clone(), &metadata.schema).unwrap(); - wal_file.append_and_sync(&batch).unwrap(); + for _ in 0..SEGMENT_ROTATION_THRESHOLD { + wal_file.append_and_sync(&batch).unwrap(); + } } - let wal_file = WriteAheadLogFile::try_new(folder_path, &metadata.schema).unwrap(); + let wal_file = WriteAheadLogFile::try_new(folder_path.clone(), &metadata.schema).unwrap(); + wal_file.append_and_sync(&batch).unwrap(); + let batches = wal_file.read_all().unwrap(); - assert_eq!(batches.len(), 1); - assert_eq!(batches[0], batch); - assert_eq!(wal_file.next_batch_id.load(Ordering::Relaxed), 1); + assert_eq!(batches.len(), SEGMENT_ROTATION_THRESHOLD as usize + 1); + + let active = wal_file.active_segment.lock().unwrap(); + assert_eq!(active.next_batch_id, SEGMENT_ROTATION_THRESHOLD + 1); } #[test] - fn test_file_size_not_changed_on_reopen() { - let temp_dir = tempfile::tempdir().unwrap(); - let folder_path = temp_dir.path().join(TIME_SERIES_TABLE_NAME); + fn test_mark_batches_as_persisted_deletes_fully_persisted_segment() { + let (_temp_dir, wal_file) = new_wal_file(); - let metadata = table::time_series_table_metadata(); - let batch = table::uncompressed_time_series_table_record_batch(10); + let batch = table::uncompressed_time_series_table_record_batch(5); - let wal_file_path = { - let wal_file = - WriteAheadLogFile::try_new(folder_path.clone(), &metadata.schema).unwrap(); + // Fill and rotate one full segment. + for _ in 0..SEGMENT_ROTATION_THRESHOLD { wal_file.append_and_sync(&batch).unwrap(); + } - wal_file.path.clone() - }; + let segment_path = wal_file.closed_segments.lock().unwrap()[0].path.clone(); + assert!(segment_path.exists()); - let size_before = std::fs::metadata(&wal_file_path).unwrap().len(); + let ids: HashSet = (0..SEGMENT_ROTATION_THRESHOLD).collect(); + wal_file.mark_batches_as_persisted(ids).unwrap(); - let wal_file_path = { - let wal_file = - WriteAheadLogFile::try_new(folder_path.clone(), &metadata.schema).unwrap(); + assert!(!segment_path.exists()); + assert!(wal_file.closed_segments.lock().unwrap().is_empty()); + } - wal_file.path.clone() - }; + fn new_wal_file() -> (TempDir, WriteAheadLogFile) { + let temp_dir = tempfile::tempdir().unwrap(); + let folder_path = temp_dir.path().join(TIME_SERIES_TABLE_NAME); + + let metadata = table::time_series_table_metadata(); - let size_after = std::fs::metadata(&wal_file_path).unwrap().len(); - assert_eq!(size_before, size_after); + ( + temp_dir, + WriteAheadLogFile::try_new(folder_path, &metadata.schema).unwrap(), + ) } } From 623e78c28e1424b2af03f6458cf7f4f8b70c3fee Mon Sep 17 00:00:00 2001 From: CGodiksen <36046286+CGodiksen@users.noreply.github.com> Date: Wed, 11 Mar 2026 15:10:07 +0100 Subject: [PATCH 063/116] Add tests for closing leftover active segments --- .../modelardb_storage/src/write_ahead_log.rs | 53 +++++++++++++++++++ 1 file changed, 53 insertions(+) diff --git a/crates/modelardb_storage/src/write_ahead_log.rs b/crates/modelardb_storage/src/write_ahead_log.rs index 2328fc2c..d8859ea0 100644 --- a/crates/modelardb_storage/src/write_ahead_log.rs +++ b/crates/modelardb_storage/src/write_ahead_log.rs @@ -630,6 +630,59 @@ mod tests { assert_eq!(active.next_batch_id, SEGMENT_ROTATION_THRESHOLD + 1); } + #[test] + fn test_close_leftover_active_segment_on_reopen() { + let temp_dir = tempfile::tempdir().unwrap(); + let folder_path = temp_dir.path().join(TIME_SERIES_TABLE_NAME); + let metadata = table::time_series_table_metadata(); + + let batch = table::uncompressed_time_series_table_record_batch(5); + + // Write some batches but do not rotate, so the active segment is leftover as "{start_id}-.wal". + { + let wal_file = + WriteAheadLogFile::try_new(folder_path.clone(), &metadata.schema).unwrap(); + wal_file.append_and_sync(&batch).unwrap(); + wal_file.append_and_sync(&batch).unwrap(); + } + + // On re-open, close_leftover_active_segment should rename it to "0-1.wal". + let wal_file = WriteAheadLogFile::try_new(folder_path.clone(), &metadata.schema).unwrap(); + + let closed = wal_file.closed_segments.lock().unwrap(); + assert_eq!(closed.len(), 1); + assert_eq!(closed[0].start_id, 0); + assert_eq!(closed[0].end_id, 1); + + let active = wal_file.active_segment.lock().unwrap(); + assert_eq!(active.next_batch_id, 2); + } + + #[test] + fn test_delete_leftover_empty_active_segment_on_reopen() { + let temp_dir = tempfile::tempdir().unwrap(); + let folder_path = temp_dir.path().join(TIME_SERIES_TABLE_NAME); + let metadata = table::time_series_table_metadata(); + + // Create a WAL file and immediately drop it without writing anything. + // This leaves an empty "{start_id}-.wal" active segment. + { + WriteAheadLogFile::try_new(folder_path.clone(), &metadata.schema).unwrap(); + } + + // On re-open, the empty leftover active segment should be removed. + let wal_file = WriteAheadLogFile::try_new(folder_path.clone(), &metadata.schema).unwrap(); + + assert!(wal_file.closed_segments.lock().unwrap().is_empty()); + let active = wal_file.active_segment.lock().unwrap(); + assert_eq!(active.next_batch_id, 0); + assert!(active.path.exists()); + + // Only the new active segment file should exist. + let file_count = std::fs::read_dir(&folder_path).unwrap().count(); + assert_eq!(file_count, 1); + } + #[test] fn test_mark_batches_as_persisted_deletes_fully_persisted_segment() { let (_temp_dir, wal_file) = new_wal_file(); From 9102239a76b993d02bbda06482890e9a107c7878 Mon Sep 17 00:00:00 2001 From: CGodiksen <36046286+CGodiksen@users.noreply.github.com> Date: Wed, 11 Mar 2026 15:17:28 +0100 Subject: [PATCH 064/116] Add more unit testing for mark_batches_as_persisted --- .../modelardb_storage/src/write_ahead_log.rs | 50 ++++++++++++++++++- 1 file changed, 49 insertions(+), 1 deletion(-) diff --git a/crates/modelardb_storage/src/write_ahead_log.rs b/crates/modelardb_storage/src/write_ahead_log.rs index d8859ea0..a5cf8d28 100644 --- a/crates/modelardb_storage/src/write_ahead_log.rs +++ b/crates/modelardb_storage/src/write_ahead_log.rs @@ -577,7 +577,7 @@ mod tests { let active = wal_file.active_segment.lock().unwrap(); assert_eq!(active.start_id, SEGMENT_ROTATION_THRESHOLD); } - + #[test] fn test_reopen_loads_closed_segments() { let temp_dir = tempfile::tempdir().unwrap(); @@ -704,6 +704,54 @@ mod tests { assert!(wal_file.closed_segments.lock().unwrap().is_empty()); } + #[test] + fn test_mark_batches_as_persisted_retains_partially_persisted_segment() { + let (_temp_dir, wal_file) = new_wal_file(); + + let batch = table::uncompressed_time_series_table_record_batch(5); + + for _ in 0..SEGMENT_ROTATION_THRESHOLD { + wal_file.append_and_sync(&batch).unwrap(); + } + + let segment_path = wal_file.closed_segments.lock().unwrap()[0].path.clone(); + + // Only persist a subset of the batch ids in the closed segment. + let partial_ids: HashSet = (0..SEGMENT_ROTATION_THRESHOLD - 1).collect(); + wal_file.mark_batches_as_persisted(partial_ids).unwrap(); + + // Segment should still exist since not all ids are persisted. + assert!(segment_path.exists()); + assert_eq!(wal_file.closed_segments.lock().unwrap().len(), 1); + + // When persisting the last batch, the segment should be deleted. + wal_file + .mark_batches_as_persisted(HashSet::from([SEGMENT_ROTATION_THRESHOLD - 1])) + .unwrap(); + + assert!(!segment_path.exists()); + assert!(wal_file.closed_segments.lock().unwrap().is_empty()); + } + + #[test] + fn test_multiple_fully_persisted_segments_all_deleted() { + let (_temp_dir, wal_file) = new_wal_file(); + + let batch = table::uncompressed_time_series_table_record_batch(5); + + // Trigger five full rotations. + for _ in 0..SEGMENT_ROTATION_THRESHOLD * 5 { + wal_file.append_and_sync(&batch).unwrap(); + } + + assert_eq!(wal_file.closed_segments.lock().unwrap().len(), 5); + + let ids: HashSet = (0..SEGMENT_ROTATION_THRESHOLD * 5).collect(); + wal_file.mark_batches_as_persisted(ids).unwrap(); + + assert!(wal_file.closed_segments.lock().unwrap().is_empty()); + } + fn new_wal_file() -> (TempDir, WriteAheadLogFile) { let temp_dir = tempfile::tempdir().unwrap(); let folder_path = temp_dir.path().join(TIME_SERIES_TABLE_NAME); From 58099db07c3f65261d54dbcfec4d1ce26e8a48a6 Mon Sep 17 00:00:00 2001 From: CGodiksen <36046286+CGodiksen@users.noreply.github.com> Date: Thu, 12 Mar 2026 09:51:06 +0100 Subject: [PATCH 065/116] Add method to storage engine to insert data that already has a WAL batch id --- crates/modelardb_server/src/storage/mod.rs | 24 +++++++++++++++++++--- 1 file changed, 21 insertions(+), 3 deletions(-) diff --git a/crates/modelardb_server/src/storage/mod.rs b/crates/modelardb_server/src/storage/mod.rs index b1446f17..77e2e055 100644 --- a/crates/modelardb_server/src/storage/mod.rs +++ b/crates/modelardb_server/src/storage/mod.rs @@ -262,10 +262,28 @@ impl StorageEngine { ) -> Result<()> { // Write to the write-ahead log to ensure termination never duplicates or loses data. We use // a read lock since the specific log file is locked internally before writing. - let write_ahead_log = self.write_ahead_log.read().await; - let batch_id = write_ahead_log - .append_to_table_log(&time_series_table_metadata.name, &multivariate_data_points)?; + let batch_id = { + let write_ahead_log = self.write_ahead_log.read().await; + write_ahead_log + .append_to_table_log(&time_series_table_metadata.name, &multivariate_data_points)? + }; + + self.insert_data_points_with_batch_id( + time_series_table_metadata, + multivariate_data_points, + batch_id, + ) + } + /// Pass `data_points` to [`UncompressedDataManager`] with a batch id given to the data by the + /// WAL. Return [`Ok`] if all of the data points were successfully inserted, otherwise return + /// [`ModelarDbServerError`]. + pub(super) fn insert_data_points_with_batch_id( + &mut self, + time_series_table_metadata: Arc, + multivariate_data_points: RecordBatch, + batch_id: u64, + ) -> Result<()> { self.memory_pool .wait_for_ingested_memory(multivariate_data_points.get_array_memory_size() as u64); From 39681d51819c43e57b1b5efa1c729b5e130796b4 Mon Sep 17 00:00:00 2001 From: CGodiksen <36046286+CGodiksen@users.noreply.github.com> Date: Thu, 12 Mar 2026 10:07:52 +0100 Subject: [PATCH 066/116] Update read_all so it now returns both record batches and batch id --- crates/modelardb_storage/src/write_ahead_log.rs | 13 ++++++++----- 1 file changed, 8 insertions(+), 5 deletions(-) diff --git a/crates/modelardb_storage/src/write_ahead_log.rs b/crates/modelardb_storage/src/write_ahead_log.rs index a5cf8d28..0a507cf6 100644 --- a/crates/modelardb_storage/src/write_ahead_log.rs +++ b/crates/modelardb_storage/src/write_ahead_log.rs @@ -370,9 +370,9 @@ impl WriteAheadLogFile { Ok(()) } - /// Read all data from all segment files (closed and active) in order. If any file could not - /// be read, return [`ModelarDbStorageError`]. - fn read_all(&self) -> Result> { + /// Read all data from all segment files (closed and active) in order and return them as pairs + /// of (batch_id, batch). If any file could not be read, return [`ModelarDbStorageError`]. + fn read_all(&self) -> Result> { // Acquire the mutex to ensure data is not being written while reading. let active = self .active_segment @@ -386,10 +386,13 @@ impl WriteAheadLogFile { let mut all_batches = Vec::new(); for segment in closed_segments.iter() { - all_batches.extend(read_batches_from_path(&segment.path)?); + let batches = read_batches_from_path(&segment.path)?; + all_batches.extend((segment.start_id..=segment.end_id).zip(batches)); } - all_batches.extend(read_batches_from_path(&active.path)?); + // Add the active segment's batches to the end of the list. + let active_batches = read_batches_from_path(&active.path)?; + all_batches.extend((active.start_id..=active.next_batch_id - 1).zip(active_batches)); Ok(all_batches) } From 4b72b48a8ade0da5da122e88b92d365c811ac93e Mon Sep 17 00:00:00 2001 From: CGodiksen <36046286+CGodiksen@users.noreply.github.com> Date: Thu, 12 Mar 2026 10:17:01 +0100 Subject: [PATCH 067/116] Fix tests and a small underflow bug caused by reading empty file --- crates/modelardb_storage/src/write_ahead_log.rs | 12 +++++++----- 1 file changed, 7 insertions(+), 5 deletions(-) diff --git a/crates/modelardb_storage/src/write_ahead_log.rs b/crates/modelardb_storage/src/write_ahead_log.rs index 0a507cf6..360f52a4 100644 --- a/crates/modelardb_storage/src/write_ahead_log.rs +++ b/crates/modelardb_storage/src/write_ahead_log.rs @@ -392,7 +392,9 @@ impl WriteAheadLogFile { // Add the active segment's batches to the end of the list. let active_batches = read_batches_from_path(&active.path)?; - all_batches.extend((active.start_id..=active.next_batch_id - 1).zip(active_batches)); + if !active_batches.is_empty() { + all_batches.extend((active.start_id..=active.next_batch_id - 1).zip(active_batches)); + } Ok(all_batches) } @@ -534,7 +536,7 @@ mod tests { let batches = wal_file.read_all().unwrap(); assert_eq!(batches.len(), 1); - assert_eq!(batches[0], batch); + assert_eq!(batches[0], (0, batch)); let active = wal_file.active_segment.lock().unwrap(); assert_eq!(active.next_batch_id, 1); @@ -554,9 +556,9 @@ mod tests { let batches = wal_file.read_all().unwrap(); assert_eq!(batches.len(), 3); - assert_eq!(batches[0], batch_1); - assert_eq!(batches[1], batch_2); - assert_eq!(batches[2], batch_3); + assert_eq!(batches[0], (0, batch_1)); + assert_eq!(batches[1], (1, batch_2)); + assert_eq!(batches[2], (2, batch_3)); let active = wal_file.active_segment.lock().unwrap(); assert_eq!(active.next_batch_id, 3); From d7de30fa202bfc45ba7318322f62118698e81b8f Mon Sep 17 00:00:00 2001 From: CGodiksen <36046286+CGodiksen@users.noreply.github.com> Date: Thu, 12 Mar 2026 10:23:31 +0100 Subject: [PATCH 068/116] Add method to find all unpersisted batches --- .../modelardb_storage/src/write_ahead_log.rs | 42 +++++++++++++++++++ 1 file changed, 42 insertions(+) diff --git a/crates/modelardb_storage/src/write_ahead_log.rs b/crates/modelardb_storage/src/write_ahead_log.rs index 360f52a4..6775f8a5 100644 --- a/crates/modelardb_storage/src/write_ahead_log.rs +++ b/crates/modelardb_storage/src/write_ahead_log.rs @@ -26,6 +26,7 @@ use arrow::error::ArrowError::IpcError; use arrow::ipc::reader::StreamReader; use arrow::ipc::writer::StreamWriter; use arrow::record_batch::RecordBatch; +use deltalake::DeltaTable; use modelardb_types::types::TimeSeriesTableMetadata; use crate::WRITE_AHEAD_LOG_FOLDER; @@ -370,6 +371,47 @@ impl WriteAheadLogFile { Ok(()) } + /// Return pairs of (batch_id, batch) for all batches in the log that have not yet been + /// persisted. The persisted batch ids are retrieved from the commit history of `delta_table` + /// to ensure duplicated data is avoided. If the commit history could not be read or the + /// batches could not be read from the WAL files, return [`ModelarDbStorageError`]. + async fn unpersisted_batches( + &self, + delta_table: DeltaTable, + ) -> Result> { + let mut persisted_batch_ids = HashSet::new(); + + // For each commit in the history, extract the custom batch id metadata. + let history = delta_table.history(None).await?; + for commit in history.into_iter() { + if let Some(batch_ids) = commit.info.get("batchIds") { + let batch_ids: Vec = serde_json::from_value(batch_ids.clone()).expect( + "The batchIds field in the commit metadata should be a JSON array of u64 values.", + ); + + persisted_batch_ids.extend(batch_ids); + } + } + + // Add the persisted batch ids to the in-memory set and delete any fully persisted segments. + self.mark_batches_as_persisted(persisted_batch_ids)?; + + let persisted = self + .persisted_batch_ids + .lock() + .expect("Mutex should not be poisoned."); + + // Collect all batches that have not yet been persisted. + let mut unpersisted = Vec::new(); + for (batch_id, batch) in self.read_all()? { + if !persisted.contains(&batch_id) { + unpersisted.push((batch_id, batch)); + } + } + + Ok(unpersisted) + } + /// Read all data from all segment files (closed and active) in order and return them as pairs /// of (batch_id, batch). If any file could not be read, return [`ModelarDbStorageError`]. fn read_all(&self) -> Result> { From f456aa5dc6445314f8a5ddd999f98d011c1e2c59 Mon Sep 17 00:00:00 2001 From: CGodiksen <36046286+CGodiksen@users.noreply.github.com> Date: Thu, 12 Mar 2026 10:33:19 +0100 Subject: [PATCH 069/116] Add separate method to load persisted batches from delta table --- .../modelardb_storage/src/write_ahead_log.rs | 20 +++++++++++++++++++ 1 file changed, 20 insertions(+) diff --git a/crates/modelardb_storage/src/write_ahead_log.rs b/crates/modelardb_storage/src/write_ahead_log.rs index 6775f8a5..764f3b2a 100644 --- a/crates/modelardb_storage/src/write_ahead_log.rs +++ b/crates/modelardb_storage/src/write_ahead_log.rs @@ -371,6 +371,26 @@ impl WriteAheadLogFile { Ok(()) } + /// Update the in-memory set of persisted batch ids from the commit history of `delta_table` + /// and delete any fully persisted closed segment files. If the commit history could not be + /// read or a segment file could not be deleted, return [`ModelarDbStorageError`]. + async fn load_persisted_batches_from_delta_table(&self, delta_table: DeltaTable) -> Result<()> { + let mut persisted_batch_ids = HashSet::new(); + + let history = delta_table.history(None).await?; + for commit in history.into_iter() { + if let Some(batch_ids) = commit.info.get("batchIds") { + let batch_ids: Vec = serde_json::from_value(batch_ids.clone()).expect( + "The batchIds field in the commit metadata should be a JSON array of u64 values.", + ); + + persisted_batch_ids.extend(batch_ids); + } + } + + self.mark_batches_as_persisted(persisted_batch_ids) + } + /// Return pairs of (batch_id, batch) for all batches in the log that have not yet been /// persisted. The persisted batch ids are retrieved from the commit history of `delta_table` /// to ensure duplicated data is avoided. If the commit history could not be read or the From 4c578aa946bc38ec6ed74e9b7a9bd8b8261a15c1 Mon Sep 17 00:00:00 2001 From: CGodiksen <36046286+CGodiksen@users.noreply.github.com> Date: Thu, 12 Mar 2026 10:34:51 +0100 Subject: [PATCH 070/116] Add separate method to get unpersisted batches --- .../modelardb_storage/src/write_ahead_log.rs | 41 ++++--------------- 1 file changed, 8 insertions(+), 33 deletions(-) diff --git a/crates/modelardb_storage/src/write_ahead_log.rs b/crates/modelardb_storage/src/write_ahead_log.rs index 764f3b2a..cf0d06cc 100644 --- a/crates/modelardb_storage/src/write_ahead_log.rs +++ b/crates/modelardb_storage/src/write_ahead_log.rs @@ -392,44 +392,19 @@ impl WriteAheadLogFile { } /// Return pairs of (batch_id, batch) for all batches in the log that have not yet been - /// persisted. The persisted batch ids are retrieved from the commit history of `delta_table` - /// to ensure duplicated data is avoided. If the commit history could not be read or the - /// batches could not be read from the WAL files, return [`ModelarDbStorageError`]. - async fn unpersisted_batches( - &self, - delta_table: DeltaTable, - ) -> Result> { - let mut persisted_batch_ids = HashSet::new(); - - // For each commit in the history, extract the custom batch id metadata. - let history = delta_table.history(None).await?; - for commit in history.into_iter() { - if let Some(batch_ids) = commit.info.get("batchIds") { - let batch_ids: Vec = serde_json::from_value(batch_ids.clone()).expect( - "The batchIds field in the commit metadata should be a JSON array of u64 values.", - ); - - persisted_batch_ids.extend(batch_ids); - } - } - - // Add the persisted batch ids to the in-memory set and delete any fully persisted segments. - self.mark_batches_as_persisted(persisted_batch_ids)?; - + /// persisted according to the current in-memory `persisted_batch_ids` set. If the batches + /// could not be read from the WAL files, return [`ModelarDbStorageError`]. + fn unpersisted_batches(&self) -> Result> { let persisted = self .persisted_batch_ids .lock() .expect("Mutex should not be poisoned."); - // Collect all batches that have not yet been persisted. - let mut unpersisted = Vec::new(); - for (batch_id, batch) in self.read_all()? { - if !persisted.contains(&batch_id) { - unpersisted.push((batch_id, batch)); - } - } - - Ok(unpersisted) + Ok(self + .read_all()? + .into_iter() + .filter(|(batch_id, _)| !persisted.contains(batch_id)) + .collect()) } /// Read all data from all segment files (closed and active) in order and return them as pairs From 9e323cb0107b1d97165580b90510b87191b01108 Mon Sep 17 00:00:00 2001 From: CGodiksen <36046286+CGodiksen@users.noreply.github.com> Date: Thu, 12 Mar 2026 10:50:25 +0100 Subject: [PATCH 071/116] Now initializing log files with persisted batches from commit history --- .../modelardb_storage/src/write_ahead_log.rs | 30 ++++++++++++------- 1 file changed, 20 insertions(+), 10 deletions(-) diff --git a/crates/modelardb_storage/src/write_ahead_log.rs b/crates/modelardb_storage/src/write_ahead_log.rs index cf0d06cc..622a8411 100644 --- a/crates/modelardb_storage/src/write_ahead_log.rs +++ b/crates/modelardb_storage/src/write_ahead_log.rs @@ -80,29 +80,39 @@ impl WriteAheadLog { // For each time series table, create a log file if it does not already exist. for metadata in local_data_folder.time_series_table_metadata().await? { - write_ahead_log.create_table_log(&metadata)?; + let delta_table = local_data_folder.delta_table(&metadata.name).await?; + write_ahead_log + .create_table_log(&metadata, Some(delta_table)) + .await?; } Ok(write_ahead_log) } - /// Create a new [`WriteAheadLogFile`] for the table with the given metadata. If a log already - /// exists in the map or the log file could not be created, return [`ModelarDbStorageError`]. - /// Note that if the log file already exists, but it is not present in the map, the existing - /// log file will be added to the map. - pub fn create_table_log( + /// Create a new [`WriteAheadLogFile`] for the table with the given metadata. If a delta table + /// is provided, the log file will be initialized with the persisted batch ids from the commit + /// history of the delta table. If a log already exists in the map or the log file could not be + /// created, return [`ModelarDbStorageError`]. Note that if the log file already exists, but it + /// is not present in the map, the existing log file will be added to the map. + pub async fn create_table_log( &mut self, time_series_table_metadata: &TimeSeriesTableMetadata, + delta_table: Option, ) -> Result<()> { let table_name = time_series_table_metadata.name.clone(); if !self.table_logs.contains_key(&table_name) { let table_log_path = self.folder_path.join(&table_name); + let log_file = + WriteAheadLogFile::try_new(table_log_path, &time_series_table_metadata.schema)?; - self.table_logs.insert( - table_name, - WriteAheadLogFile::try_new(table_log_path, &time_series_table_metadata.schema)?, - ); + if let Some(delta_table) = delta_table { + log_file + .load_persisted_batches_from_delta_table(delta_table) + .await?; + } + + self.table_logs.insert(table_name, log_file); Ok(()) } else { From cff29c6031b372dbdee3e36830d25ae30e64f34b Mon Sep 17 00:00:00 2001 From: CGodiksen <36046286+CGodiksen@users.noreply.github.com> Date: Thu, 12 Mar 2026 11:01:34 +0100 Subject: [PATCH 072/116] No longer create temp dir in util method --- .../modelardb_storage/src/write_ahead_log.rs | 35 ++++++++++++------- 1 file changed, 22 insertions(+), 13 deletions(-) diff --git a/crates/modelardb_storage/src/write_ahead_log.rs b/crates/modelardb_storage/src/write_ahead_log.rs index 622a8411..eb5f1faa 100644 --- a/crates/modelardb_storage/src/write_ahead_log.rs +++ b/crates/modelardb_storage/src/write_ahead_log.rs @@ -554,7 +554,8 @@ mod tests { #[test] fn test_try_new_creates_active_segment() { - let (_temp_dir, wal_file) = new_wal_file(); + let temp_dir = tempfile::tempdir().unwrap(); + let (_folder_path, wal_file) = new_wal_file(&temp_dir); let active = wal_file.active_segment.lock().unwrap(); assert!(active.path.exists()); @@ -565,7 +566,8 @@ mod tests { #[test] fn test_read_all_empty_file() { - let (_temp_dir, wal_file) = new_wal_file(); + let temp_dir = tempfile::tempdir().unwrap(); + let (_folder_path, wal_file) = new_wal_file(&temp_dir); let batches = wal_file.read_all().unwrap(); assert!(batches.is_empty()); @@ -576,7 +578,8 @@ mod tests { #[test] fn test_append_and_read_single_batch() { - let (_temp_dir, wal_file) = new_wal_file(); + let temp_dir = tempfile::tempdir().unwrap(); + let (_folder_path, wal_file) = new_wal_file(&temp_dir); let batch = table::uncompressed_time_series_table_record_batch(5); wal_file.append_and_sync(&batch).unwrap(); @@ -591,7 +594,8 @@ mod tests { #[test] fn test_append_and_read_multiple_batches() { - let (_temp_dir, wal_file) = new_wal_file(); + let temp_dir = tempfile::tempdir().unwrap(); + let (_folder_path, wal_file) = new_wal_file(&temp_dir); let batch_1 = table::uncompressed_time_series_table_record_batch(10); let batch_2 = table::uncompressed_time_series_table_record_batch(20); @@ -613,7 +617,8 @@ mod tests { #[test] fn test_segment_rotates_at_threshold() { - let (_temp_dir, wal_file) = new_wal_file(); + let temp_dir = tempfile::tempdir().unwrap(); + let (_folder_path, wal_file) = new_wal_file(&temp_dir); let batch = table::uncompressed_time_series_table_record_batch(5); @@ -737,7 +742,8 @@ mod tests { #[test] fn test_mark_batches_as_persisted_deletes_fully_persisted_segment() { - let (_temp_dir, wal_file) = new_wal_file(); + let temp_dir = tempfile::tempdir().unwrap(); + let (_folder_path, wal_file) = new_wal_file(&temp_dir); let batch = table::uncompressed_time_series_table_record_batch(5); @@ -758,7 +764,8 @@ mod tests { #[test] fn test_mark_batches_as_persisted_retains_partially_persisted_segment() { - let (_temp_dir, wal_file) = new_wal_file(); + let temp_dir = tempfile::tempdir().unwrap(); + let (_folder_path, wal_file) = new_wal_file(&temp_dir); let batch = table::uncompressed_time_series_table_record_batch(5); @@ -787,7 +794,8 @@ mod tests { #[test] fn test_multiple_fully_persisted_segments_all_deleted() { - let (_temp_dir, wal_file) = new_wal_file(); + let temp_dir = tempfile::tempdir().unwrap(); + let (_folder_path, wal_file) = new_wal_file(&temp_dir); let batch = table::uncompressed_time_series_table_record_batch(5); @@ -804,15 +812,16 @@ mod tests { assert!(wal_file.closed_segments.lock().unwrap().is_empty()); } - fn new_wal_file() -> (TempDir, WriteAheadLogFile) { let temp_dir = tempfile::tempdir().unwrap(); let folder_path = temp_dir.path().join(TIME_SERIES_TABLE_NAME); + fn new_wal_file(temp_dir: &TempDir) -> (PathBuf, WriteAheadLogFile) { + let folder_path = temp_dir.path().join(TIME_SERIES_TABLE_NAME); let metadata = table::time_series_table_metadata(); - ( - temp_dir, - WriteAheadLogFile::try_new(folder_path, &metadata.schema).unwrap(), - ) + let wal_file = + WriteAheadLogFile::try_new(folder_path.clone(), &metadata.schema).unwrap(); + + (folder_path, wal_file) } } From 562342747cb6da7b1a94208eaf8fe82c9433b4b3 Mon Sep 17 00:00:00 2001 From: CGodiksen <36046286+CGodiksen@users.noreply.github.com> Date: Thu, 12 Mar 2026 11:06:21 +0100 Subject: [PATCH 073/116] Add test for checking that commit history with no batch ids changes nothing --- .../modelardb_storage/src/write_ahead_log.rs | 49 +++++++++++++++++-- 1 file changed, 46 insertions(+), 3 deletions(-) diff --git a/crates/modelardb_storage/src/write_ahead_log.rs b/crates/modelardb_storage/src/write_ahead_log.rs index eb5f1faa..aecbace6 100644 --- a/crates/modelardb_storage/src/write_ahead_log.rs +++ b/crates/modelardb_storage/src/write_ahead_log.rs @@ -812,15 +812,58 @@ mod tests { assert!(wal_file.closed_segments.lock().unwrap().is_empty()); } + #[tokio::test] + async fn test_no_batch_ids_in_history_leaves_persisted_set_empty() { + let (temp_dir, data_folder) = create_data_folder_with_time_series_table().await; + let (_wal_dir, wal_file) = new_wal_file(&temp_dir); + + let delta_table = data_folder + .delta_table(TIME_SERIES_TABLE_NAME) + .await + .unwrap(); + + wal_file + .load_persisted_batches_from_delta_table(delta_table) + .await + .unwrap(); + + assert!(wal_file.persisted_batch_ids.lock().unwrap().is_empty()); + } + + async fn create_data_folder_with_time_series_table() -> (TempDir, DataFolder) { let temp_dir = tempfile::tempdir().unwrap(); - let folder_path = temp_dir.path().join(TIME_SERIES_TABLE_NAME); + let data_folder = DataFolder::open_local(temp_dir.path()).await.unwrap(); + let metadata = table::time_series_table_metadata(); + + data_folder + .create_time_series_table(&metadata) + .await + .unwrap(); + + (temp_dir, data_folder) + } + + async fn write_compressed_segments_with_batch_ids( + data_folder: &DataFolder, + batch_ids: HashSet, + ) -> DeltaTable { + let compressed_segments = table::compressed_segments_record_batch(); + + data_folder + .write_compressed_segments_to_time_series_table( + TIME_SERIES_TABLE_NAME, + vec![compressed_segments], + batch_ids, + ) + .await + .unwrap() + } fn new_wal_file(temp_dir: &TempDir) -> (PathBuf, WriteAheadLogFile) { let folder_path = temp_dir.path().join(TIME_SERIES_TABLE_NAME); let metadata = table::time_series_table_metadata(); - let wal_file = - WriteAheadLogFile::try_new(folder_path.clone(), &metadata.schema).unwrap(); + let wal_file = WriteAheadLogFile::try_new(folder_path.clone(), &metadata.schema).unwrap(); (folder_path, wal_file) } From 53ceffd99c8e710839c107d3ecccb46fd28d17ab Mon Sep 17 00:00:00 2001 From: CGodiksen <36046286+CGodiksen@users.noreply.github.com> Date: Thu, 12 Mar 2026 11:13:30 +0100 Subject: [PATCH 074/116] Add tests for checking that commit history updates persisted batch ids --- .../modelardb_storage/src/write_ahead_log.rs | 35 +++++++++++++++++++ 1 file changed, 35 insertions(+) diff --git a/crates/modelardb_storage/src/write_ahead_log.rs b/crates/modelardb_storage/src/write_ahead_log.rs index aecbace6..dad2f624 100644 --- a/crates/modelardb_storage/src/write_ahead_log.rs +++ b/crates/modelardb_storage/src/write_ahead_log.rs @@ -830,6 +830,41 @@ mod tests { assert!(wal_file.persisted_batch_ids.lock().unwrap().is_empty()); } + #[tokio::test] + async fn test_single_commit_populates_persisted_batch_ids() { + let (temp_dir, data_folder) = create_data_folder_with_time_series_table().await; + let (_wal_dir, wal_file) = new_wal_file(&temp_dir); + + let delta_table = + write_compressed_segments_with_batch_ids(&data_folder, HashSet::from([0, 1, 2])).await; + + wal_file + .load_persisted_batches_from_delta_table(delta_table) + .await + .unwrap(); + + let persisted = wal_file.persisted_batch_ids.lock().unwrap(); + assert_eq!(*persisted, BTreeSet::from([0, 1, 2])); + } + + #[tokio::test] + async fn test_multiple_commits_accumulate_persisted_batch_ids() { + let (temp_dir, data_folder) = create_data_folder_with_time_series_table().await; + let (_wal_dir, wal_file) = new_wal_file(&temp_dir); + + write_compressed_segments_with_batch_ids(&data_folder, HashSet::from([0, 1, 2])).await; + let delta_table = + write_compressed_segments_with_batch_ids(&data_folder, HashSet::from([2, 3, 4])).await; + + wal_file + .load_persisted_batches_from_delta_table(delta_table) + .await + .unwrap(); + + let persisted = wal_file.persisted_batch_ids.lock().unwrap(); + assert_eq!(*persisted, BTreeSet::from([0, 1, 2, 3, 4])); + } + async fn create_data_folder_with_time_series_table() -> (TempDir, DataFolder) { let temp_dir = tempfile::tempdir().unwrap(); let data_folder = DataFolder::open_local(temp_dir.path()).await.unwrap(); From 926c9513138f552623c4444506ed28bc9414cdd2 Mon Sep 17 00:00:00 2001 From: CGodiksen <36046286+CGodiksen@users.noreply.github.com> Date: Thu, 12 Mar 2026 11:21:38 +0100 Subject: [PATCH 075/116] Add test for checking that fully persisted closed segments are deleted --- .../modelardb_storage/src/write_ahead_log.rs | 30 +++++++++++++++++-- 1 file changed, 28 insertions(+), 2 deletions(-) diff --git a/crates/modelardb_storage/src/write_ahead_log.rs b/crates/modelardb_storage/src/write_ahead_log.rs index dad2f624..ce93762c 100644 --- a/crates/modelardb_storage/src/write_ahead_log.rs +++ b/crates/modelardb_storage/src/write_ahead_log.rs @@ -831,7 +831,7 @@ mod tests { } #[tokio::test] - async fn test_single_commit_populates_persisted_batch_ids() { + async fn test_load_persisted_batches_loads_single_commit() { let (temp_dir, data_folder) = create_data_folder_with_time_series_table().await; let (_wal_dir, wal_file) = new_wal_file(&temp_dir); @@ -848,7 +848,7 @@ mod tests { } #[tokio::test] - async fn test_multiple_commits_accumulate_persisted_batch_ids() { + async fn test_load_persisted_batches_loads_multiple_commits() { let (temp_dir, data_folder) = create_data_folder_with_time_series_table().await; let (_wal_dir, wal_file) = new_wal_file(&temp_dir); @@ -865,6 +865,32 @@ mod tests { assert_eq!(*persisted, BTreeSet::from([0, 1, 2, 3, 4])); } + #[tokio::test] + async fn test_load_persisted_batches_deletes_fully_persisted_closed_segment() { + let (temp_dir, data_folder) = create_data_folder_with_time_series_table().await; + let (_wal_dir, wal_file) = new_wal_file(&temp_dir); + + let batch = table::uncompressed_time_series_table_record_batch(5); + for _ in 0..SEGMENT_ROTATION_THRESHOLD { + wal_file.append_and_sync(&batch).unwrap(); + } + + let segment_path = wal_file.closed_segments.lock().unwrap()[0].path.clone(); + assert!(segment_path.exists()); + + let all_ids: HashSet = (0..SEGMENT_ROTATION_THRESHOLD).collect(); + let delta_table = write_compressed_segments_with_batch_ids(&data_folder, all_ids).await; + + wal_file + .load_persisted_batches_from_delta_table(delta_table) + .await + .unwrap(); + + assert!(!segment_path.exists()); + assert!(wal_file.closed_segments.lock().unwrap().is_empty()); + assert_eq!(wal_file.persisted_batch_ids.lock().unwrap().len(), 0); + } + async fn create_data_folder_with_time_series_table() -> (TempDir, DataFolder) { let temp_dir = tempfile::tempdir().unwrap(); let data_folder = DataFolder::open_local(temp_dir.path()).await.unwrap(); From ae9afa372faa25f7a22e7cd81ea5e955f763c98f Mon Sep 17 00:00:00 2001 From: CGodiksen <36046286+CGodiksen@users.noreply.github.com> Date: Thu, 12 Mar 2026 12:28:40 +0100 Subject: [PATCH 076/116] Add test for checking that partially persisted closed segments are retained --- .../modelardb_storage/src/write_ahead_log.rs | 28 +++++++++++++++++++ 1 file changed, 28 insertions(+) diff --git a/crates/modelardb_storage/src/write_ahead_log.rs b/crates/modelardb_storage/src/write_ahead_log.rs index ce93762c..8573d45f 100644 --- a/crates/modelardb_storage/src/write_ahead_log.rs +++ b/crates/modelardb_storage/src/write_ahead_log.rs @@ -891,6 +891,34 @@ mod tests { assert_eq!(wal_file.persisted_batch_ids.lock().unwrap().len(), 0); } + #[tokio::test] + async fn test_load_persisted_batches_retains_partially_persisted_closed_segment() { + let (temp_dir, data_folder) = create_data_folder_with_time_series_table().await; + let (_wal_dir, wal_file) = new_wal_file(&temp_dir); + + let batch = table::uncompressed_time_series_table_record_batch(5); + for _ in 0..SEGMENT_ROTATION_THRESHOLD { + wal_file.append_and_sync(&batch).unwrap(); + } + + let segment_path = wal_file.closed_segments.lock().unwrap()[0].path.clone(); + + let partial_ids: HashSet = (0..SEGMENT_ROTATION_THRESHOLD - 1).collect(); + let delta_table = write_compressed_segments_with_batch_ids(&data_folder, partial_ids).await; + + wal_file + .load_persisted_batches_from_delta_table(delta_table) + .await + .unwrap(); + + assert!(segment_path.exists()); + assert_eq!(wal_file.closed_segments.lock().unwrap().len(), 1); + assert_eq!( + wal_file.persisted_batch_ids.lock().unwrap().len() as u64, + SEGMENT_ROTATION_THRESHOLD - 1 + ); + } + async fn create_data_folder_with_time_series_table() -> (TempDir, DataFolder) { let temp_dir = tempfile::tempdir().unwrap(); let data_folder = DataFolder::open_local(temp_dir.path()).await.unwrap(); From d4517007917d79cc10dbfa9cdad8ecf21b5c3435 Mon Sep 17 00:00:00 2001 From: CGodiksen <36046286+CGodiksen@users.noreply.github.com> Date: Thu, 12 Mar 2026 12:43:49 +0100 Subject: [PATCH 077/116] Add tests for unpersisted_batches() --- .../modelardb_storage/src/write_ahead_log.rs | 63 +++++++++++++++++++ 1 file changed, 63 insertions(+) diff --git a/crates/modelardb_storage/src/write_ahead_log.rs b/crates/modelardb_storage/src/write_ahead_log.rs index 8573d45f..b4b461a0 100644 --- a/crates/modelardb_storage/src/write_ahead_log.rs +++ b/crates/modelardb_storage/src/write_ahead_log.rs @@ -948,6 +948,69 @@ mod tests { .unwrap() } + #[test] + fn test_unpersisted_batches_returns_all_when_none_persisted() { + let temp_dir = tempfile::tempdir().unwrap(); + let (_folder_path, wal_file) = new_wal_file(&temp_dir); + + let batch_1 = table::uncompressed_time_series_table_record_batch(10); + let batch_2 = table::uncompressed_time_series_table_record_batch(20); + wal_file.append_and_sync(&batch_1).unwrap(); + wal_file.append_and_sync(&batch_2).unwrap(); + + let unpersisted = wal_file.unpersisted_batches().unwrap(); + assert_eq!(unpersisted.len(), 2); + assert_eq!(unpersisted[0], (0, batch_1)); + assert_eq!(unpersisted[1], (1, batch_2)); + } + + #[test] + fn test_unpersisted_batches_returns_empty_when_all_persisted() { + let temp_dir = tempfile::tempdir().unwrap(); + let (_folder_path, wal_file) = new_wal_file(&temp_dir); + + let batch = table::uncompressed_time_series_table_record_batch(10); + wal_file.append_and_sync(&batch).unwrap(); + wal_file.append_and_sync(&batch).unwrap(); + + wal_file + .mark_batches_as_persisted(HashSet::from([0, 1])) + .unwrap(); + + let unpersisted = wal_file.unpersisted_batches().unwrap(); + assert!(unpersisted.is_empty()); + } + + #[test] + fn test_unpersisted_batches_filters_persisted_batch_ids() { + let temp_dir = tempfile::tempdir().unwrap(); + let (_folder_path, wal_file) = new_wal_file(&temp_dir); + + let batch_1 = table::uncompressed_time_series_table_record_batch(10); + let batch_2 = table::uncompressed_time_series_table_record_batch(20); + wal_file.append_and_sync(&batch_1).unwrap(); + wal_file.append_and_sync(&batch_1).unwrap(); + wal_file.append_and_sync(&batch_2).unwrap(); + + wal_file + .mark_batches_as_persisted(HashSet::from([1])) + .unwrap(); + + let unpersisted = wal_file.unpersisted_batches().unwrap(); + assert_eq!(unpersisted.len(), 2); + assert_eq!(unpersisted[0], (0, batch_1)); + assert_eq!(unpersisted[1], (2, batch_2)); + } + + #[test] + fn test_unpersisted_batches_returns_empty_when_no_batches_written() { + let temp_dir = tempfile::tempdir().unwrap(); + let (_folder_path, wal_file) = new_wal_file(&temp_dir); + + let unpersisted = wal_file.unpersisted_batches().unwrap(); + assert!(unpersisted.is_empty()); + } + fn new_wal_file(temp_dir: &TempDir) -> (PathBuf, WriteAheadLogFile) { let folder_path = temp_dir.path().join(TIME_SERIES_TABLE_NAME); let metadata = table::time_series_table_metadata(); From 3a97ef03e9ec24f010fd919f10033993fcd5d64d Mon Sep 17 00:00:00 2001 From: CGodiksen <36046286+CGodiksen@users.noreply.github.com> Date: Thu, 12 Mar 2026 13:16:49 +0100 Subject: [PATCH 078/116] Add test for ensuring that unpersisted_batches includes closed and active segments --- .../modelardb_storage/src/write_ahead_log.rs | 28 +++++++++++++++++++ 1 file changed, 28 insertions(+) diff --git a/crates/modelardb_storage/src/write_ahead_log.rs b/crates/modelardb_storage/src/write_ahead_log.rs index b4b461a0..34b0add0 100644 --- a/crates/modelardb_storage/src/write_ahead_log.rs +++ b/crates/modelardb_storage/src/write_ahead_log.rs @@ -1002,6 +1002,34 @@ mod tests { assert_eq!(unpersisted[1], (2, batch_2)); } + #[test] + fn test_unpersisted_batches_returns_batches_across_closed_and_active_segments() { + let temp_dir = tempfile::tempdir().unwrap(); + let (_folder_path, wal_file) = new_wal_file(&temp_dir); + + let batch = table::uncompressed_time_series_table_record_batch(5); + + // Fill one full segment (triggers rotation) and write two more into the active segment. + for _ in 0..SEGMENT_ROTATION_THRESHOLD + 2 { + wal_file.append_and_sync(&batch).unwrap(); + } + + // Persist one batch id in the closed segment and one in the active segment. + wal_file + .mark_batches_as_persisted(HashSet::from([0, SEGMENT_ROTATION_THRESHOLD + 1])) + .unwrap(); + + assert_eq!(wal_file.closed_segments.lock().unwrap().len(), 1); + + let unpersisted = wal_file.unpersisted_batches().unwrap(); + assert_eq!(unpersisted.len() as u64, SEGMENT_ROTATION_THRESHOLD); + assert_eq!(unpersisted.first().unwrap(), &(1, batch.clone())); + assert_eq!( + unpersisted.last().unwrap(), + &(SEGMENT_ROTATION_THRESHOLD, batch) + ); + } + #[test] fn test_unpersisted_batches_returns_empty_when_no_batches_written() { let temp_dir = tempfile::tempdir().unwrap(); From f6d2420bf153ff54770451b236157f4f2bf39245 Mon Sep 17 00:00:00 2001 From: CGodiksen <36046286+CGodiksen@users.noreply.github.com> Date: Thu, 12 Mar 2026 13:17:00 +0100 Subject: [PATCH 079/116] Add method to get unpersisted batches in table log --- crates/modelardb_storage/src/write_ahead_log.rs | 11 +++++++++++ 1 file changed, 11 insertions(+) diff --git a/crates/modelardb_storage/src/write_ahead_log.rs b/crates/modelardb_storage/src/write_ahead_log.rs index 34b0add0..2bc48f8b 100644 --- a/crates/modelardb_storage/src/write_ahead_log.rs +++ b/crates/modelardb_storage/src/write_ahead_log.rs @@ -164,6 +164,17 @@ impl WriteAheadLog { log_file.mark_batches_as_persisted(batch_ids) } + /// Return pairs of (batch_id, batch) for all batches that have not yet been persisted in the + /// corresponding table log. If the log file does not exist or the batches could not be read + /// from the WAL files, return [`ModelarDbStorageError`]. + pub fn unpersisted_batches_in_table_log( + &self, + table_name: &str, + ) -> Result> { + let log_file = self.table_log(table_name)?; + log_file.unpersisted_batches() + } + /// Get the log file for the table with the given name. If the log file does not exist, return /// [`ModelarDbStorageError`]. fn table_log(&self, table_name: &str) -> Result<&WriteAheadLogFile> { From 253b594c5ba83836105b3bf7560e39c5db45a239 Mon Sep 17 00:00:00 2001 From: CGodiksen <36046286+CGodiksen@users.noreply.github.com> Date: Thu, 12 Mar 2026 13:20:38 +0100 Subject: [PATCH 080/116] Add method to context to replay write ahead log --- crates/modelardb_server/src/context.rs | 30 +++++++++++++++++++++++++- 1 file changed, 29 insertions(+), 1 deletion(-) diff --git a/crates/modelardb_server/src/context.rs b/crates/modelardb_server/src/context.rs index ec5df4f6..e6141fe9 100644 --- a/crates/modelardb_server/src/context.rs +++ b/crates/modelardb_server/src/context.rs @@ -124,7 +124,9 @@ impl Context { // Create a file in the write-ahead log to log uncompressed data for the table. let mut write_ahead_log = self.write_ahead_log.write().await; - write_ahead_log.create_table_log(time_series_table_metadata)?; + write_ahead_log + .create_table_log(time_series_table_metadata, None) + .await?; Ok(()) } @@ -266,6 +268,32 @@ impl Context { Ok(()) } + /// For each time series table in the local data folder, use the write-ahead-log to replay any + /// data that was written to the storage engine but not compressed and saved to disk. Note that + /// this method should only be called before the storage engine starts ingesting data to avoid + /// replaying data that is currently in memory. + pub(super) async fn replay_write_ahead_log(&self) -> Result<()> { + let local_data_folder = &self.data_folders.local_data_folder; + + let write_ahead_log = self.write_ahead_log.write().await; + let mut storage_engine = self.storage_engine.write().await; + + for metadata in local_data_folder.time_series_table_metadata().await? { + let unpersisted_batches = + write_ahead_log.unpersisted_batches_in_table_log(&metadata.name)?; + + for (batch_id, batch) in unpersisted_batches { + storage_engine.insert_data_points_with_batch_id( + metadata.clone(), + batch, + batch_id, + )?; + } + } + + Ok(()) + } + /// Drop the table with `table_name` if it exists. The table is deregistered from the Apache /// Arrow Datafusion session context and deleted from the storage engine and Delta Lake. If the /// table does not exist or if it could not be dropped, [`ModelarDbServerError`] is returned. From ecae75b4d8eed433a101bcc002e215faca7a8299 Mon Sep 17 00:00:00 2001 From: CGodiksen <36046286+CGodiksen@users.noreply.github.com> Date: Thu, 12 Mar 2026 13:28:27 +0100 Subject: [PATCH 081/116] Calling replay write ahead log in main --- crates/modelardb_server/src/main.rs | 3 +++ crates/modelardb_storage/src/write_ahead_log.rs | 5 ++--- 2 files changed, 5 insertions(+), 3 deletions(-) diff --git a/crates/modelardb_server/src/main.rs b/crates/modelardb_server/src/main.rs index d2eb9644..ab509982 100644 --- a/crates/modelardb_server/src/main.rs +++ b/crates/modelardb_server/src/main.rs @@ -83,6 +83,9 @@ async fn main() -> Result<()> { cluster.retrieve_and_create_tables(&context).await?; } + // Replay any data that was written to the storage engine but not compressed and saved to disk. + context.replay_write_ahead_log().await?; + // Setup CTRL+C handler. setup_ctrl_c_handler(&context); diff --git a/crates/modelardb_storage/src/write_ahead_log.rs b/crates/modelardb_storage/src/write_ahead_log.rs index 2bc48f8b..4638f620 100644 --- a/crates/modelardb_storage/src/write_ahead_log.rs +++ b/crates/modelardb_storage/src/write_ahead_log.rs @@ -47,7 +47,7 @@ pub struct WriteAheadLog { /// Logs for each table. The key is the table name, and the value is the log file for that table. table_logs: HashMap, /// Log file for operations that are not associated with a specific table. - operation_log: WriteAheadLogFile, + _operation_log: WriteAheadLogFile, } impl WriteAheadLog { @@ -72,7 +72,7 @@ impl WriteAheadLog { let mut write_ahead_log = Self { folder_path: log_folder_path.clone(), table_logs: HashMap::new(), - operation_log: WriteAheadLogFile::try_new( + _operation_log: WriteAheadLogFile::try_new( log_folder_path.join(OPERATIONS_LOG_FOLDER), &operations_log_schema(), )?, @@ -286,7 +286,6 @@ impl WriteAheadLogFile { schema: schema.clone(), active_segment: Mutex::new(active_file), closed_segments: Mutex::new(closed_segments), - // TODO: This needs to be initialized with persisted batch ids from Delta Lake. persisted_batch_ids: Mutex::new(BTreeSet::new()), }) } From c574c8bd7c0ad2f2f1621d2f5b311d8f6b4e6819 Mon Sep 17 00:00:00 2001 From: CGodiksen <36046286+CGodiksen@users.noreply.github.com> Date: Thu, 12 Mar 2026 13:41:22 +0100 Subject: [PATCH 082/116] Create time series table in compressed data manager tests --- .../src/storage/compressed_data_manager.rs | 18 ++++++------------ 1 file changed, 6 insertions(+), 12 deletions(-) diff --git a/crates/modelardb_server/src/storage/compressed_data_manager.rs b/crates/modelardb_server/src/storage/compressed_data_manager.rs index 083714af..0fcc58e5 100644 --- a/crates/modelardb_server/src/storage/compressed_data_manager.rs +++ b/crates/modelardb_server/src/storage/compressed_data_manager.rs @@ -410,7 +410,7 @@ mod tests { let local_data_folder = data_manager.local_data_folder.clone(); let mut delta_table = local_data_folder - .create_time_series_table(&table::time_series_table_metadata()) + .delta_table(TIME_SERIES_TABLE_NAME) .await .unwrap(); @@ -465,14 +465,8 @@ mod tests { #[tokio::test] async fn test_remaining_memory_incremented_when_saving_compressed_segments() { let (_temp_dir, data_manager) = create_compressed_data_manager().await; - let local_data_folder = data_manager.local_data_folder.clone(); let segments = compressed_segments_record_batch(); - local_data_folder - .create_time_series_table(&segments.time_series_table_metadata) - .await - .unwrap(); - data_manager .insert_compressed_segments(segments.clone()) .await @@ -519,14 +513,9 @@ mod tests { #[tokio::test] async fn test_decrease_compressed_remaining_memory_in_bytes() { let (_temp_dir, data_manager) = create_compressed_data_manager().await; - let local_data_folder = data_manager.local_data_folder.clone(); // Insert data that should be saved when the remaining memory is decreased. let segments = compressed_segments_record_batch(); - local_data_folder - .create_time_series_table(&segments.time_series_table_metadata) - .await - .unwrap(); data_manager .insert_compressed_segments(segments) .await @@ -580,6 +569,11 @@ mod tests { let local_data_folder = DataFolder::open_local_url(temp_dir_url).await.unwrap(); let time_series_table_metadata = table::time_series_table_metadata(); + local_data_folder + .create_time_series_table(&time_series_table_metadata) + .await + .unwrap(); + local_data_folder .save_time_series_table_metadata(&time_series_table_metadata) .await From 08afca48263de709124c351478f065f84ac056f6 Mon Sep 17 00:00:00 2001 From: CGodiksen <36046286+CGodiksen@users.noreply.github.com> Date: Thu, 12 Mar 2026 13:44:45 +0100 Subject: [PATCH 083/116] Fix doc issues --- crates/modelardb_storage/src/lib.rs | 2 +- crates/modelardb_storage/src/write_ahead_log.rs | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/crates/modelardb_storage/src/lib.rs b/crates/modelardb_storage/src/lib.rs index a7e5104d..1cd81afc 100644 --- a/crates/modelardb_storage/src/lib.rs +++ b/crates/modelardb_storage/src/lib.rs @@ -64,7 +64,7 @@ const TABLE_FOLDER: &str = "tables"; const METADATA_FOLDER: &str = "metadata"; /// The folder storing the write-ahead log in the data folders. -const WRITE_AHEAD_LOG_FOLDER: &str = "_modelardb_log"; +pub const WRITE_AHEAD_LOG_FOLDER: &str = "_modelardb_log"; /// Create a new [`SessionContext`] for interacting with Apache DataFusion. The [`SessionContext`] /// is constructed with the default configuration, default resource managers, and additional diff --git a/crates/modelardb_storage/src/write_ahead_log.rs b/crates/modelardb_storage/src/write_ahead_log.rs index 4638f620..f476420e 100644 --- a/crates/modelardb_storage/src/write_ahead_log.rs +++ b/crates/modelardb_storage/src/write_ahead_log.rs @@ -89,7 +89,7 @@ impl WriteAheadLog { Ok(write_ahead_log) } - /// Create a new [`WriteAheadLogFile`] for the table with the given metadata. If a delta table + /// Create a new write-ahead-log file for the table with the given metadata. If a delta table /// is provided, the log file will be initialized with the persisted batch ids from the commit /// history of the delta table. If a log already exists in the map or the log file could not be /// created, return [`ModelarDbStorageError`]. Note that if the log file already exists, but it From 25d12cc2365111bbd63a0edde045a89e9c9c35d1 Mon Sep 17 00:00:00 2001 From: CGodiksen <36046286+CGodiksen@users.noreply.github.com> Date: Thu, 12 Mar 2026 14:48:48 +0100 Subject: [PATCH 084/116] Fix dependency issue after rebase --- Cargo.lock | 1141 +++++++++++++++++++++++++++++----------------------- 1 file changed, 638 insertions(+), 503 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index 6e1f1f1c..1ae3eeca 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -52,12 +52,6 @@ version = "0.2.21" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "683d7910e743518b0e34f1186f92494becacb047c7b6bf616c96772180fef923" -[[package]] -name = "android-tzdata" -version = "0.1.1" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "e999941b234f3131b00bc13c22d06e8c5ff726d1b6318ac7eb276997bbb4fef0" - [[package]] name = "android_system_properties" version = "0.1.5" @@ -87,9 +81,9 @@ checksum = "7c02d123df017efcdfbd739ef81735b36c5ba83ec3c59c80a9d7ecc718f92e50" [[package]] name = "arrow" -version = "56.2.0" +version = "57.3.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "6e833808ff2d94ed40d9379848a950d995043c7fb3e81a30b383f4c6033821cc" +checksum = "e4754a624e5ae42081f464514be454b39711daae0458906dacde5f4c632f33a8" dependencies = [ "arrow-arith", "arrow-array", @@ -108,23 +102,23 @@ dependencies = [ [[package]] name = "arrow-arith" -version = "56.2.0" +version = "57.3.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "ad08897b81588f60ba983e3ca39bda2b179bdd84dced378e7df81a5313802ef8" +checksum = "f7b3141e0ec5145a22d8694ea8b6d6f69305971c4fa1c1a13ef0195aef2d678b" dependencies = [ "arrow-array", "arrow-buffer", "arrow-data", "arrow-schema", "chrono", - "num", + "num-traits", ] [[package]] name = "arrow-array" -version = "56.2.0" +version = "57.3.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "8548ca7c070d8db9ce7aa43f37393e4bfcf3f2d3681df278490772fd1673d08d" +checksum = "4c8955af33b25f3b175ee10af580577280b4bd01f7e823d94c7cdef7cf8c9aef" dependencies = [ "ahash", "arrow-buffer", @@ -134,46 +128,50 @@ dependencies = [ "chrono-tz", "half", "hashbrown 0.16.0", - "num", + "num-complex", + "num-integer", + "num-traits", ] [[package]] name = "arrow-buffer" -version = "56.2.0" +version = "57.3.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "e003216336f70446457e280807a73899dd822feaf02087d31febca1363e2fccc" +checksum = "c697ddca96183182f35b3a18e50b9110b11e916d7b7799cbfd4d34662f2c56c2" dependencies = [ "bytes", "half", - "num", + "num-bigint", + "num-traits", ] [[package]] name = "arrow-cast" -version = "56.2.0" +version = "57.3.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "919418a0681298d3a77d1a315f625916cb5678ad0d74b9c60108eb15fd083023" +checksum = "646bbb821e86fd57189c10b4fcdaa941deaf4181924917b0daa92735baa6ada5" dependencies = [ "arrow-array", "arrow-buffer", "arrow-data", + "arrow-ord", "arrow-schema", "arrow-select", "atoi", - "base64 0.22.1", + "base64", "chrono", "comfy-table", "half", "lexical-core", - "num", + "num-traits", "ryu", ] [[package]] name = "arrow-csv" -version = "56.2.0" +version = "57.3.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "bfa9bf02705b5cf762b6f764c65f04ae9082c7cfc4e96e0c33548ee3f67012eb" +checksum = "8da746f4180004e3ce7b83c977daf6394d768332349d3d913998b10a120b790a" dependencies = [ "arrow-array", "arrow-cast", @@ -186,40 +184,42 @@ dependencies = [ [[package]] name = "arrow-data" -version = "56.2.0" +version = "57.3.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a5c64fff1d142f833d78897a772f2e5b55b36cb3e6320376f0961ab0db7bd6d0" +checksum = "1fdd994a9d28e6365aa78e15da3f3950c0fdcea6b963a12fa1c391afb637b304" dependencies = [ "arrow-buffer", "arrow-schema", "half", - "num", + "num-integer", + "num-traits", ] [[package]] name = "arrow-flight" -version = "56.2.0" +version = "57.3.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "8c8b0ba0784d56bc6266b79f5de7a24b47024e7b3a0045d2ad4df3d9b686099f" +checksum = "58c5b083668e6230eae3eab2fc4b5fb989974c845d0aa538dde61a4327c78675" dependencies = [ "arrow-array", "arrow-buffer", "arrow-cast", "arrow-ipc", "arrow-schema", - "base64 0.22.1", + "base64", "bytes", "futures", "prost", "prost-types", "tonic", + "tonic-prost", ] [[package]] name = "arrow-ipc" -version = "56.2.0" +version = "57.3.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "1d3594dcddccc7f20fd069bc8e9828ce37220372680ff638c5e00dea427d88f5" +checksum = "abf7df950701ab528bf7c0cf7eeadc0445d03ef5d6ffc151eaae6b38a58feff1" dependencies = [ "arrow-array", "arrow-buffer", @@ -233,9 +233,9 @@ dependencies = [ [[package]] name = "arrow-json" -version = "56.2.0" +version = "57.3.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "88cf36502b64a127dc659e3b305f1d993a544eab0d48cce704424e62074dc04b" +checksum = "0ff8357658bedc49792b13e2e862b80df908171275f8e6e075c460da5ee4bf86" dependencies = [ "arrow-array", "arrow-buffer", @@ -245,19 +245,21 @@ dependencies = [ "chrono", "half", "indexmap", + "itoa", "lexical-core", "memchr", - "num", - "serde", + "num-traits", + "ryu", + "serde_core", "serde_json", "simdutf8", ] [[package]] name = "arrow-ord" -version = "56.2.0" +version = "57.3.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "3c8f82583eb4f8d84d4ee55fd1cb306720cddead7596edce95b50ee418edf66f" +checksum = "f7d8f1870e03d4cbed632959498bcc84083b5a24bded52905ae1695bd29da45b" dependencies = [ "arrow-array", "arrow-buffer", @@ -268,9 +270,9 @@ dependencies = [ [[package]] name = "arrow-row" -version = "56.2.0" +version = "57.3.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "9d07ba24522229d9085031df6b94605e0f4b26e099fb7cdeec37abd941a73753" +checksum = "18228633bad92bff92a95746bbeb16e5fc318e8382b75619dec26db79e4de4c0" dependencies = [ "arrow-array", "arrow-buffer", @@ -281,34 +283,35 @@ dependencies = [ [[package]] name = "arrow-schema" -version = "56.2.0" +version = "57.3.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "b3aa9e59c611ebc291c28582077ef25c97f1975383f1479b12f3b9ffee2ffabe" +checksum = "8c872d36b7bf2a6a6a2b40de9156265f0242910791db366a2c17476ba8330d68" dependencies = [ "bitflags", "serde", + "serde_core", "serde_json", ] [[package]] name = "arrow-select" -version = "56.2.0" +version = "57.3.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "8c41dbbd1e97bfcaee4fcb30e29105fb2c75e4d82ae4de70b792a5d3f66b2e7a" +checksum = "68bf3e3efbd1278f770d67e5dc410257300b161b93baedb3aae836144edcaf4b" dependencies = [ "ahash", "arrow-array", "arrow-buffer", "arrow-data", "arrow-schema", - "num", + "num-traits", ] [[package]] name = "arrow-string" -version = "56.2.0" +version = "57.3.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "53f5183c150fbc619eede22b861ea7c0eebed8eaac0333eaa7f6da5205fd504d" +checksum = "85e968097061b3c0e9fe3079cf2e703e487890700546b5b0647f60fca1b5a8d8" dependencies = [ "arrow-array", "arrow-buffer", @@ -316,7 +319,7 @@ dependencies = [ "arrow-schema", "arrow-select", "memchr", - "num", + "num-traits", "regex", "regex-syntax", ] @@ -610,23 +613,17 @@ dependencies = [ "aws-smithy-async", "aws-smithy-runtime-api", "aws-smithy-types", - "h2 0.3.26", - "h2 0.4.12", - "http 0.2.12", + "h2", "http 1.2.0", - "http-body 0.4.6", - "hyper 0.14.32", - "hyper 1.6.0", - "hyper-rustls 0.24.2", - "hyper-rustls 0.27.5", + "hyper", + "hyper-rustls", "hyper-util", "pin-project-lite", - "rustls 0.21.12", - "rustls 0.23.34", - "rustls-native-certs 0.8.1", + "rustls", + "rustls-native-certs", "rustls-pki-types", "tokio", - "tokio-rustls 0.26.2", + "tokio-rustls", "tower", "tracing", ] @@ -802,12 +799,6 @@ dependencies = [ "tokio", ] -[[package]] -name = "base64" -version = "0.21.7" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "9d297deb1925b89f2ccc13d7635fa0714f12c87adce1c75356b39ca9b7178567" - [[package]] name = "base64" version = "0.22.1" @@ -950,9 +941,9 @@ checksum = "1fd0f2584146f6f2ef48085050886acf353beff7305ebd1ae69500e27c67f64b" [[package]] name = "bytes" -version = "1.10.1" +version = "1.11.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d71b6127be86fdcfddb610f7182ac57211d4b18a3e9c82eb2d17662f2227ad6a" +checksum = "1e748733b7cbc798e1434b6ac524f0c1ff2ab456fe201501e6497c8417a4fc33" [[package]] name = "bytes-utils" @@ -1025,19 +1016,29 @@ 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.41" +version = "0.4.44" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c469d952047f47f91b68d1cba3f10d63c11d73e4636f24f08daf0278abf01c4d" +checksum = "c673075a2e0e5f4a1dde27ce9dee1ea4558c7ffe648f576438a20ca1d2acc4b0" dependencies = [ - "android-tzdata", "iana-time-zone", "js-sys", "num-traits", "serde", "wasm-bindgen", - "windows-link 0.1.1", + "windows-link 0.2.1", ] [[package]] @@ -1131,23 +1132,13 @@ checksum = "7c74b8349d32d297c9134b8c88677813a227df8f779daa29bfc29c183fe3dca6" [[package]] name = "convert_case" -version = "0.8.0" +version = "0.9.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "baaaa0ecca5b51987b9423ccdc971514dd8b0bb7b4060b983d3664dad3f1f89f" +checksum = "db05ffb6856bf0ecdf6367558a76a0e8a77b1713044eb92845c692100ed50190" dependencies = [ "unicode-segmentation", ] -[[package]] -name = "core-foundation" -version = "0.9.4" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "91e195e091a93c46f7102ec7818a2aa394e1e1771c3ab4825963fa03e45afb8f" -dependencies = [ - "core-foundation-sys", - "libc", -] - [[package]] name = "core-foundation" version = "0.10.0" @@ -1173,6 +1164,30 @@ 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" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "5eb8a2a1cd12ab0d987a5d5e825195d372001a4094a0376319d5a0ad71c1ba0d" +dependencies = [ + "crc-catalog", +] + +[[package]] +name = "crc-catalog" +version = "2.4.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "19d374276b40fb8bbdee95aef7c7fa6b5316ec764510eb64b8dd0e2ed0d7e7f5" + [[package]] name = "crc32fast" version = "1.4.2" @@ -1343,12 +1358,11 @@ dependencies = [ [[package]] name = "datafusion" -version = "50.3.0" +version = "51.0.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "2af15bb3c6ffa33011ef579f6b0bcbe7c26584688bd6c994f548e44df67f011a" +checksum = "8ba7cb113e9c0bedf9e9765926031e132fa05a1b09ba6e93a6d1a4d7044457b8" dependencies = [ "arrow", - "arrow-ipc", "arrow-schema", "async-trait", "bytes", @@ -1359,6 +1373,7 @@ dependencies = [ "datafusion-common", "datafusion-common-runtime", "datafusion-datasource", + "datafusion-datasource-arrow", "datafusion-datasource-csv", "datafusion-datasource-json", "datafusion-datasource-parquet", @@ -1387,7 +1402,8 @@ dependencies = [ "parquet", "rand 0.9.2", "regex", - "sqlparser 0.58.0", + "rstest", + "sqlparser", "tempfile", "tokio", "url", @@ -1398,9 +1414,9 @@ dependencies = [ [[package]] name = "datafusion-catalog" -version = "50.3.0" +version = "51.0.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "187622262ad8f7d16d3be9202b4c1e0116f1c9aa387e5074245538b755261621" +checksum = "66a3a799f914a59b1ea343906a0486f17061f39509af74e874a866428951130d" dependencies = [ "arrow", "async-trait", @@ -1413,7 +1429,6 @@ dependencies = [ "datafusion-physical-expr", "datafusion-physical-plan", "datafusion-session", - "datafusion-sql", "futures", "itertools 0.14.0", "log", @@ -1424,9 +1439,9 @@ dependencies = [ [[package]] name = "datafusion-catalog-listing" -version = "50.3.0" +version = "51.0.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "9657314f0a32efd0382b9a46fdeb2d233273ece64baa68a7c45f5a192daf0f83" +checksum = "6db1b113c80d7a0febcd901476a57aef378e717c54517a163ed51417d87621b0" dependencies = [ "arrow", "async-trait", @@ -1436,10 +1451,11 @@ dependencies = [ "datafusion-execution", "datafusion-expr", "datafusion-physical-expr", + "datafusion-physical-expr-adapter", "datafusion-physical-expr-common", "datafusion-physical-plan", - "datafusion-session", "futures", + "itertools 0.14.0", "log", "object_store", "tokio", @@ -1447,14 +1463,13 @@ dependencies = [ [[package]] name = "datafusion-common" -version = "50.3.0" +version = "51.0.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "5a83760d9a13122d025fbdb1d5d5aaf93dd9ada5e90ea229add92aa30898b2d1" +checksum = "7c10f7659e96127d25e8366be7c8be4109595d6a2c3eac70421f380a7006a1b0" dependencies = [ "ahash", "arrow", "arrow-ipc", - "base64 0.22.1", "chrono", "half", "hashbrown 0.14.5", @@ -1465,16 +1480,16 @@ dependencies = [ "parquet", "paste", "recursive", - "sqlparser 0.58.0", + "sqlparser", "tokio", "web-time", ] [[package]] name = "datafusion-common-runtime" -version = "50.3.0" +version = "51.0.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "5b6234a6c7173fe5db1c6c35c01a12b2aa0f803a3007feee53483218817f8b1e" +checksum = "b92065bbc6532c6651e2f7dd30b55cba0c7a14f860c7e1d15f165c41a1868d95" dependencies = [ "futures", "log", @@ -1483,9 +1498,9 @@ dependencies = [ [[package]] name = "datafusion-datasource" -version = "50.3.0" +version = "51.0.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "7256c9cb27a78709dd42d0c80f0178494637209cac6e29d5c93edd09b6721b86" +checksum = "fde13794244bc7581cd82f6fff217068ed79cdc344cafe4ab2c3a1c3510b38d6" dependencies = [ "arrow", "async-compression", @@ -1508,9 +1523,7 @@ dependencies = [ "itertools 0.14.0", "log", "object_store", - "parquet", "rand 0.9.2", - "tempfile", "tokio", "tokio-util", "url", @@ -1518,22 +1531,44 @@ dependencies = [ "zstd", ] +[[package]] +name = "datafusion-datasource-arrow" +version = "51.0.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "804fa9b4ecf3157982021770617200ef7c1b2979d57bec9044748314775a9aea" +dependencies = [ + "arrow", + "arrow-ipc", + "async-trait", + "bytes", + "datafusion-common", + "datafusion-common-runtime", + "datafusion-datasource", + "datafusion-execution", + "datafusion-expr", + "datafusion-physical-expr-common", + "datafusion-physical-plan", + "datafusion-session", + "futures", + "itertools 0.14.0", + "object_store", + "tokio", +] + [[package]] name = "datafusion-datasource-csv" -version = "50.3.0" +version = "51.0.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "64533a90f78e1684bfb113d200b540f18f268134622d7c96bbebc91354d04825" +checksum = "61a1641a40b259bab38131c5e6f48fac0717bedb7dc93690e604142a849e0568" dependencies = [ "arrow", "async-trait", "bytes", - "datafusion-catalog", "datafusion-common", "datafusion-common-runtime", "datafusion-datasource", "datafusion-execution", "datafusion-expr", - "datafusion-physical-expr", "datafusion-physical-expr-common", "datafusion-physical-plan", "datafusion-session", @@ -1545,49 +1580,44 @@ dependencies = [ [[package]] name = "datafusion-datasource-json" -version = "50.3.0" +version = "51.0.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "8d7ebeb12c77df0aacad26f21b0d033aeede423a64b2b352f53048a75bf1d6e6" +checksum = "adeacdb00c1d37271176f8fb6a1d8ce096baba16ea7a4b2671840c5c9c64fe85" dependencies = [ "arrow", "async-trait", "bytes", - "datafusion-catalog", "datafusion-common", "datafusion-common-runtime", "datafusion-datasource", "datafusion-execution", "datafusion-expr", - "datafusion-physical-expr", "datafusion-physical-expr-common", "datafusion-physical-plan", "datafusion-session", "futures", "object_store", - "serde_json", "tokio", ] [[package]] name = "datafusion-datasource-parquet" -version = "50.3.0" +version = "51.0.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "09e783c4c7d7faa1199af2df4761c68530634521b176a8d1331ddbc5a5c75133" +checksum = "43d0b60ffd66f28bfb026565d62b0a6cbc416da09814766a3797bba7d85a3cd9" dependencies = [ "arrow", "async-trait", "bytes", - "datafusion-catalog", "datafusion-common", "datafusion-common-runtime", "datafusion-datasource", "datafusion-execution", "datafusion-expr", - "datafusion-functions-aggregate", + "datafusion-functions-aggregate-common", "datafusion-physical-expr", "datafusion-physical-expr-adapter", "datafusion-physical-expr-common", - "datafusion-physical-optimizer", "datafusion-physical-plan", "datafusion-pruning", "datafusion-session", @@ -1597,21 +1627,20 @@ dependencies = [ "object_store", "parking_lot", "parquet", - "rand 0.9.2", "tokio", ] [[package]] name = "datafusion-doc" -version = "50.3.0" +version = "51.0.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "99ee6b1d9a80d13f9deb2291f45c07044b8e62fb540dbde2453a18be17a36429" +checksum = "2b99e13947667b36ad713549237362afb054b2d8f8cc447751e23ec61202db07" [[package]] name = "datafusion-execution" -version = "50.3.0" +version = "51.0.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a4cec0a57653bec7b933fb248d3ffa3fa3ab3bd33bd140dc917f714ac036f531" +checksum = "63695643190679037bc946ad46a263b62016931547bf119859c511f7ff2f5178" dependencies = [ "arrow", "async-trait", @@ -1629,9 +1658,9 @@ dependencies = [ [[package]] name = "datafusion-expr" -version = "50.3.0" +version = "51.0.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "ef76910bdca909722586389156d0aa4da4020e1631994d50fadd8ad4b1aa05fe" +checksum = "f9a4787cbf5feb1ab351f789063398f67654a6df75c4d37d7f637dc96f951a91" dependencies = [ "arrow", "async-trait", @@ -1643,17 +1672,18 @@ dependencies = [ "datafusion-functions-window-common", "datafusion-physical-expr-common", "indexmap", + "itertools 0.14.0", "paste", "recursive", "serde_json", - "sqlparser 0.58.0", + "sqlparser", ] [[package]] name = "datafusion-expr-common" -version = "50.3.0" +version = "51.0.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "6d155ccbda29591ca71a1344dd6bed26c65a4438072b400df9db59447f590bb6" +checksum = "5ce2fb1b8c15c9ac45b0863c30b268c69dc9ee7a1ee13ecf5d067738338173dc" dependencies = [ "arrow", "datafusion-common", @@ -1664,13 +1694,13 @@ dependencies = [ [[package]] name = "datafusion-functions" -version = "50.3.0" +version = "51.0.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "7de2782136bd6014670fd84fe3b0ca3b3e4106c96403c3ae05c0598577139977" +checksum = "794a9db7f7b96b3346fc007ff25e994f09b8f0511b4cf7dff651fadfe3ebb28f" dependencies = [ "arrow", "arrow-buffer", - "base64 0.22.1", + "base64", "blake2", "blake3", "chrono", @@ -1684,6 +1714,7 @@ dependencies = [ "itertools 0.14.0", "log", "md-5", + "num-traits", "rand 0.9.2", "regex", "sha2", @@ -1693,9 +1724,9 @@ dependencies = [ [[package]] name = "datafusion-functions-aggregate" -version = "50.3.0" +version = "51.0.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "07331fc13603a9da97b74fd8a273f4238222943dffdbbed1c4c6f862a30105bf" +checksum = "1c25210520a9dcf9c2b2cbbce31ebd4131ef5af7fc60ee92b266dc7d159cb305" dependencies = [ "ahash", "arrow", @@ -1714,9 +1745,9 @@ dependencies = [ [[package]] name = "datafusion-functions-aggregate-common" -version = "50.3.0" +version = "51.0.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "b5951e572a8610b89968a09b5420515a121fbc305c0258651f318dc07c97ab17" +checksum = "62f4a66f3b87300bb70f4124b55434d2ae3fe80455f3574701d0348da040b55d" dependencies = [ "ahash", "arrow", @@ -1727,9 +1758,9 @@ dependencies = [ [[package]] name = "datafusion-functions-nested" -version = "50.3.0" +version = "51.0.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "fdacca9302c3d8fc03f3e94f338767e786a88a33f5ebad6ffc0e7b50364b9ea3" +checksum = "ae5c06eed03918dc7fe7a9f082a284050f0e9ecf95d72f57712d1496da03b8c4" dependencies = [ "arrow", "arrow-ord", @@ -1737,6 +1768,7 @@ dependencies = [ "datafusion-doc", "datafusion-execution", "datafusion-expr", + "datafusion-expr-common", "datafusion-functions", "datafusion-functions-aggregate", "datafusion-functions-aggregate-common", @@ -1749,9 +1781,9 @@ dependencies = [ [[package]] name = "datafusion-functions-table" -version = "50.3.0" +version = "51.0.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "8c37ff8a99434fbbad604a7e0669717c58c7c4f14c472d45067c4b016621d981" +checksum = "db4fed1d71738fbe22e2712d71396db04c25de4111f1ec252b8f4c6d3b25d7f5" dependencies = [ "arrow", "async-trait", @@ -1765,9 +1797,9 @@ dependencies = [ [[package]] name = "datafusion-functions-window" -version = "50.3.0" +version = "51.0.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "48e2aea7c79c926cffabb13dc27309d4eaeb130f4a21c8ba91cdd241c813652b" +checksum = "1d92206aa5ae21892f1552b4d61758a862a70956e6fd7a95cb85db1de74bc6d1" dependencies = [ "arrow", "datafusion-common", @@ -1783,9 +1815,9 @@ dependencies = [ [[package]] name = "datafusion-functions-window-common" -version = "50.3.0" +version = "51.0.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "0fead257ab5fd2ffc3b40fda64da307e20de0040fe43d49197241d9de82a487f" +checksum = "53ae9bcc39800820d53a22d758b3b8726ff84a5a3e24cecef04ef4e5fdf1c7cc" dependencies = [ "datafusion-common", "datafusion-physical-expr-common", @@ -1793,20 +1825,20 @@ dependencies = [ [[package]] name = "datafusion-macros" -version = "50.3.0" +version = "51.0.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "ec6f637bce95efac05cdfb9b6c19579ed4aa5f6b94d951cfa5bb054b7bb4f730" +checksum = "1063ad4c9e094b3f798acee16d9a47bd7372d9699be2de21b05c3bd3f34ab848" dependencies = [ - "datafusion-expr", + "datafusion-doc", "quote", "syn", ] [[package]] name = "datafusion-optimizer" -version = "50.3.0" +version = "51.0.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c6583ef666ae000a613a837e69e456681a9faa96347bf3877661e9e89e141d8a" +checksum = "9f35f9ec5d08b87fd1893a30c2929f2559c2f9806ca072d8fefca5009dc0f06a" dependencies = [ "arrow", "chrono", @@ -1824,9 +1856,9 @@ dependencies = [ [[package]] name = "datafusion-physical-expr" -version = "50.3.0" +version = "51.0.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c8668103361a272cbbe3a61f72eca60c9b7c706e87cc3565bcf21e2b277b84f6" +checksum = "c30cc8012e9eedcb48bbe112c6eff4ae5ed19cf3003cb0f505662e88b7014c5d" dependencies = [ "ahash", "arrow", @@ -1839,17 +1871,16 @@ dependencies = [ "hashbrown 0.14.5", "indexmap", "itertools 0.14.0", - "log", "parking_lot", "paste", - "petgraph 0.8.3", + "petgraph", ] [[package]] name = "datafusion-physical-expr-adapter" -version = "50.3.0" +version = "51.0.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "815acced725d30601b397e39958e0e55630e0a10d66ef7769c14ae6597298bb0" +checksum = "7f9ff2dbd476221b1f67337699eff432781c4e6e1713d2aefdaa517dfbf79768" dependencies = [ "arrow", "datafusion-common", @@ -1862,9 +1893,9 @@ dependencies = [ [[package]] name = "datafusion-physical-expr-common" -version = "50.3.0" +version = "51.0.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "6652fe7b5bf87e85ed175f571745305565da2c0b599d98e697bcbedc7baa47c3" +checksum = "90da43e1ec550b172f34c87ec68161986ced70fd05c8d2a2add66eef9c276f03" dependencies = [ "ahash", "arrow", @@ -1876,9 +1907,9 @@ dependencies = [ [[package]] name = "datafusion-physical-optimizer" -version = "50.3.0" +version = "51.0.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "49b7d623eb6162a3332b564a0907ba00895c505d101b99af78345f1acf929b5c" +checksum = "ce9804f799acd7daef3be7aaffe77c0033768ed8fdbf5fb82fc4c5f2e6bc14e6" dependencies = [ "arrow", "datafusion-common", @@ -1890,15 +1921,14 @@ dependencies = [ "datafusion-physical-plan", "datafusion-pruning", "itertools 0.14.0", - "log", "recursive", ] [[package]] name = "datafusion-physical-plan" -version = "50.3.0" +version = "51.0.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "e2f7f778a1a838dec124efb96eae6144237d546945587557c9e6936b3414558c" +checksum = "0acf0ad6b6924c6b1aa7d213b181e012e2d3ec0a64ff5b10ee6282ab0f8532ac" dependencies = [ "ahash", "arrow", @@ -1927,15 +1957,26 @@ dependencies = [ [[package]] name = "datafusion-proto" -version = "50.3.0" +version = "51.0.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a7df9f606892e6af45763d94d210634eec69b9bb6ced5353381682ff090028a3" +checksum = "d368093a98a17d1449b1083ac22ed16b7128e4c67789991869480d8c4a40ecb9" dependencies = [ "arrow", "chrono", - "datafusion", + "datafusion-catalog", + "datafusion-catalog-listing", "datafusion-common", + "datafusion-datasource", + "datafusion-datasource-arrow", + "datafusion-datasource-csv", + "datafusion-datasource-json", + "datafusion-datasource-parquet", + "datafusion-execution", "datafusion-expr", + "datafusion-functions-table", + "datafusion-physical-expr", + "datafusion-physical-expr-common", + "datafusion-physical-plan", "datafusion-proto-common", "object_store", "prost", @@ -1943,9 +1984,9 @@ dependencies = [ [[package]] name = "datafusion-proto-common" -version = "50.3.0" +version = "51.0.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "b4b14f288ca4ef77743d9672cafecf3adfffff0b9b04af9af79ecbeaaf736901" +checksum = "3b6aef3d5e5c1d2bc3114c4876730cb76a9bdc5a8df31ef1b6db48f0c1671895" dependencies = [ "arrow", "datafusion-common", @@ -1954,12 +1995,11 @@ dependencies = [ [[package]] name = "datafusion-pruning" -version = "50.3.0" +version = "51.0.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "cd1e59e2ca14fe3c30f141600b10ad8815e2856caa59ebbd0e3e07cd3d127a65" +checksum = "ac2c2498a1f134a9e11a9f5ed202a2a7d7e9774bd9249295593053ea3be999db" dependencies = [ "arrow", - "arrow-schema", "datafusion-common", "datafusion-datasource", "datafusion-expr-common", @@ -1972,55 +2012,47 @@ dependencies = [ [[package]] name = "datafusion-session" -version = "50.3.0" +version = "51.0.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "21ef8e2745583619bd7a49474e8f45fbe98ebb31a133f27802217125a7b3d58d" +checksum = "8f96eebd17555386f459037c65ab73aae8df09f464524c709d6a3134ad4f4776" dependencies = [ - "arrow", "async-trait", - "dashmap", "datafusion-common", - "datafusion-common-runtime", "datafusion-execution", "datafusion-expr", - "datafusion-physical-expr", "datafusion-physical-plan", - "datafusion-sql", - "futures", - "itertools 0.14.0", - "log", - "object_store", "parking_lot", - "tokio", ] [[package]] name = "datafusion-sql" -version = "50.3.0" +version = "51.0.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "89abd9868770386fede29e5a4b14f49c0bf48d652c3b9d7a8a0332329b87d50b" +checksum = "3fc195fe60634b2c6ccfd131b487de46dc30eccae8a3c35a13f136e7f440414f" dependencies = [ "arrow", "bigdecimal", + "chrono", "datafusion-common", "datafusion-expr", "indexmap", "log", "recursive", "regex", - "sqlparser 0.58.0", + "sqlparser", ] [[package]] name = "delta_kernel" -version = "0.16.0" +version = "0.19.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "cb6b80fa39021744edf13509bbdd7caef94c1bf101e384990210332dbddddf44" +checksum = "06f7fc164b1557731fcc68a198e813811a000efade0f112d4f0a002e65042b83" dependencies = [ "arrow", "bytes", "chrono", "comfy-table", + "crc", "delta_kernel_derive", "futures", "indexmap", @@ -2043,9 +2075,9 @@ dependencies = [ [[package]] name = "delta_kernel_derive" -version = "0.16.0" +version = "0.19.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "ae1d02d9f5d886ae8bb7fc3f7a3cb8f1b75cd0f5c95f9b5f45bba308f1a0aa58" +checksum = "86815a2c475835751ffa9b8d9ac8ed86cf86294304c42bedd1103d54f25ecbfe" dependencies = [ "proc-macro2", "quote", @@ -2054,9 +2086,9 @@ dependencies = [ [[package]] name = "deltalake" -version = "0.29.4" +version = "0.30.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "09169ef5ecf35911f5f1c3117844a4e00da1edcce58fe8593a237761525f6e3a" +checksum = "d5ace194fd6a5db14d4b4973c5780cf4569650716594ffd25297343be2e7cb0c" dependencies = [ "ctor", "delta_kernel", @@ -2066,9 +2098,9 @@ dependencies = [ [[package]] name = "deltalake-aws" -version = "0.12.1" +version = "0.13.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c0210d644f4ab27e6d477da99e4b4bf0c7d739fd399ac38c005b6d0dfa4fe132" +checksum = "b60353287c8dc49bc21caa77c62e6eca4141bdcaf967365553dc62b518c7d2f1" dependencies = [ "async-trait", "aws-config", @@ -2086,15 +2118,16 @@ dependencies = [ "thiserror", "tokio", "tracing", + "typed-builder", "url", "uuid", ] [[package]] name = "deltalake-core" -version = "0.29.2" +version = "0.30.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "058a672565db45b857617c925f1b9a006bba4d339a87d2f9b1845e183b2ff4a9" +checksum = "5b098d0ce09726f10a08b102c885a501ee18f06ea4aca864570508a9d5b620d1" dependencies = [ "arrow", "arrow-arith", @@ -2113,6 +2146,7 @@ dependencies = [ "chrono", "dashmap", "datafusion", + "datafusion-datasource", "datafusion-proto", "delta_kernel", "deltalake-derive", @@ -2133,7 +2167,7 @@ dependencies = [ "regex", "serde", "serde_json", - "sqlparser 0.59.0", + "sqlparser", "strum 0.27.1", "thiserror", "tokio", @@ -2145,9 +2179,9 @@ dependencies = [ [[package]] name = "deltalake-derive" -version = "0.29.0" +version = "0.30.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "1a785b4702c2d1b6ff286075f375fb2fd52dfbb2fadf17b9233f4d5eea35c6ec" +checksum = "3963d9fe965af7b1dea433271389e1e39c6a97ffdbc2e81d808f5b329e4577b3" dependencies = [ "convert_case", "itertools 0.14.0", @@ -2276,7 +2310,7 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "0ce92ff622d6dadf7349484f42c93271a0d49b7cc4d466a936405bacbe10aa78" dependencies = [ "cfg-if", - "rustix 1.0.2", + "rustix 1.1.4", "windows-sys 0.59.0", ] @@ -2411,6 +2445,12 @@ version = "0.3.31" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "f90f7dce0722e95104fcb095585910c0977252f286e354b5e3bd38902cd99988" +[[package]] +name = "futures-timer" +version = "3.0.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "f288b0a4f20f9a56b5d1da57e2227c661b7b16168e2f72365f57b63326e29b24" + [[package]] name = "futures-util" version = "0.3.31" @@ -2465,29 +2505,24 @@ dependencies = [ ] [[package]] -name = "glob" -version = "0.3.2" +name = "getrandom" +version = "0.4.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a8d1add55171497b4705a648c6b583acafb01d58050a51727785f0b2c8e0a2b2" +checksum = "0de51e6874e94e7bf76d726fc5d13ba782deca734ff60d5bb2fb2607c7406555" +dependencies = [ + "cfg-if", + "libc", + "r-efi", + "rand_core 0.10.0", + "wasip2", + "wasip3", +] [[package]] -name = "h2" -version = "0.3.26" +name = "glob" +version = "0.3.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "81fe527a889e1532da5c525686d96d4c2e74cdd345badf8dfef9f6b39dd5f5e8" -dependencies = [ - "bytes", - "fnv", - "futures-core", - "futures-sink", - "futures-util", - "http 0.2.12", - "indexmap", - "slab", - "tokio", - "tokio-util", - "tracing", -] +checksum = "a8d1add55171497b4705a648c6b583acafb01d58050a51727785f0b2c8e0a2b2" [[package]] name = "h2" @@ -2510,13 +2545,14 @@ dependencies = [ [[package]] name = "half" -version = "2.6.0" +version = "2.7.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "459196ed295495a68f7d7fe1d84f6c4b7ff0e21fe3017b2f283c6fac3ad803c9" +checksum = "6ea2d84b969582b4b1864a92dc5d27cd2b77b622a8d79306834f1be5ba20d84b" dependencies = [ "cfg-if", "crunchy", "num-traits", + "zerocopy 0.8.42", ] [[package]] @@ -2654,30 +2690,6 @@ version = "2.1.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "9a3a5bfb195931eeb336b2a7b4d761daec841b97f947d34394601737a7bba5e4" -[[package]] -name = "hyper" -version = "0.14.32" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "41dfc780fdec9373c01bae43289ea34c972e40ee3c9f6b3c8801a35f35586ce7" -dependencies = [ - "bytes", - "futures-channel", - "futures-core", - "futures-util", - "h2 0.3.26", - "http 0.2.12", - "http-body 0.4.6", - "httparse", - "httpdate", - "itoa", - "pin-project-lite", - "socket2 0.5.8", - "tokio", - "tower-service", - "tracing", - "want", -] - [[package]] name = "hyper" version = "1.6.0" @@ -2687,7 +2699,7 @@ dependencies = [ "bytes", "futures-channel", "futures-util", - "h2 0.4.12", + "h2", "http 1.2.0", "http-body 1.0.1", "httparse", @@ -2699,22 +2711,6 @@ dependencies = [ "want", ] -[[package]] -name = "hyper-rustls" -version = "0.24.2" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "ec3efd23720e2049821a693cbc7e65ea87c72f1c58ff2f9522ff332b1491e590" -dependencies = [ - "futures-util", - "http 0.2.12", - "hyper 0.14.32", - "log", - "rustls 0.21.12", - "rustls-native-certs 0.6.3", - "tokio", - "tokio-rustls 0.24.1", -] - [[package]] name = "hyper-rustls" version = "0.27.5" @@ -2723,13 +2719,13 @@ checksum = "2d191583f3da1305256f22463b9bb0471acad48a4e534a5218b9963e9c1f59b2" dependencies = [ "futures-util", "http 1.2.0", - "hyper 1.6.0", + "hyper", "hyper-util", - "rustls 0.23.34", - "rustls-native-certs 0.8.1", + "rustls", + "rustls-native-certs", "rustls-pki-types", "tokio", - "tokio-rustls 0.26.2", + "tokio-rustls", "tower-service", ] @@ -2739,7 +2735,7 @@ version = "0.5.2" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "2b90d566bffbce6a75bd8b09a05aa8c2cb1fabb6cb348f8840c9e4c90a0d83b0" dependencies = [ - "hyper 1.6.0", + "hyper", "hyper-util", "pin-project-lite", "tokio", @@ -2752,14 +2748,14 @@ version = "0.1.17" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "3c6995591a8f1380fcb4ba966a252a4b29188d51d2b89e3a252f5305be65aea8" dependencies = [ - "base64 0.22.1", + "base64", "bytes", "futures-channel", "futures-core", "futures-util", "http 1.2.0", "http-body 1.0.1", - "hyper 1.6.0", + "hyper", "ipnet", "libc", "percent-encoding", @@ -2911,6 +2907,12 @@ dependencies = [ "syn", ] +[[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" @@ -2946,6 +2948,8 @@ checksum = "6717a8d2a5a929a1a2eb43a12812498ed141a0bcfb7e8f7844fbdbe4303bba9f" dependencies = [ "equivalent", "hashbrown 0.16.0", + "serde", + "serde_core", ] [[package]] @@ -3019,6 +3023,12 @@ version = "1.5.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "bbd2bcb4c963f2ddae06a2efc7e9f3591312473c50c6685e1f298068316e66fe" +[[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.5" @@ -3091,9 +3101,9 @@ checksum = "2c4a545a15244c7d945065b5d392b2d2d7f21526fba56ce51467b06ed445e8f7" [[package]] name = "libc" -version = "0.2.177" +version = "0.2.183" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "2874a2af47a2325c2001a6e6fad9b16a53b802102b528163885171cf92b15976" +checksum = "b5b646652bf6661599e1da8901b3b9522896f01e736bad5f723fe7a3a27f899d" [[package]] name = "libloading" @@ -3138,9 +3148,9 @@ checksum = "d26c52dbd32dccf2d10cac7725f8eae5296885fb5703b261f7d0a0739ec807ab" [[package]] name = "linux-raw-sys" -version = "0.9.2" +version = "0.12.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "6db9c683daf087dc577b7506e9695b3d556a9f3849903fa28186283afd6809e9" +checksum = "32a66949e030da00e8c7d4434b251670a91556f4144941d37452769c25d58a53" [[package]] name = "litemap" @@ -3160,17 +3170,17 @@ dependencies = [ [[package]] name = "log" -version = "0.4.28" +version = "0.4.29" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "34080505efa8e45a4b816c349525ebe327ceaa8559756f0356cba97ef3bf7432" +checksum = "5e5032e24019045c762d3c0f28f5b6b8bbf38563a65908389bf7978758920897" [[package]] name = "lz4_flex" -version = "0.11.3" +version = "0.12.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "75761162ae2b0e580d7e7c390558127e5f01b4194debd6221fd8c207fc80e3f5" +checksum = "ab6473172471198271ff72e9379150e9dfd70d8e533e0752a27e515b48dd375e" dependencies = [ - "twox-hash 1.6.3", + "twox-hash", ] [[package]] @@ -3349,7 +3359,7 @@ dependencies = [ "modelardb_types", "object_store", "serde_json", - "sqlparser 0.58.0", + "sqlparser", "tempfile", "tokio", "tonic", @@ -3435,20 +3445,6 @@ dependencies = [ "windows-sys 0.61.2", ] -[[package]] -name = "num" -version = "0.4.3" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "35bd024e8b2ff75562e5f34e7f4905839deb4b22955ef5e73d2fea1b9813cb23" -dependencies = [ - "num-bigint", - "num-complex", - "num-integer", - "num-iter", - "num-rational", - "num-traits", -] - [[package]] name = "num-bigint" version = "0.4.6" @@ -3483,28 +3479,6 @@ dependencies = [ "num-traits", ] -[[package]] -name = "num-iter" -version = "0.1.45" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "1429034a0490724d0075ebb2bc9e875d6503c3cf69e235a8941aa757d83ef5bf" -dependencies = [ - "autocfg", - "num-integer", - "num-traits", -] - -[[package]] -name = "num-rational" -version = "0.4.2" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "f83d14da390562dca69fc84082e73e548e1ad308d24accdedd2720017cb37824" -dependencies = [ - "num-bigint", - "num-integer", - "num-traits", -] - [[package]] name = "num-traits" version = "0.2.19" @@ -3551,7 +3525,7 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "4c1be0c6c22ec0817cdc77d3842f721a17fd30ab6965001415b5402a74e6b740" dependencies = [ "async-trait", - "base64 0.22.1", + "base64", "bytes", "chrono", "form_urlencoded", @@ -3560,7 +3534,7 @@ dependencies = [ "http-body-util", "httparse", "humantime", - "hyper 1.6.0", + "hyper", "itertools 0.14.0", "md-5", "parking_lot", @@ -3569,7 +3543,7 @@ dependencies = [ "rand 0.9.2", "reqwest", "ring", - "rustls-pemfile 2.2.0", + "rustls-pemfile", "serde", "serde_json", "serde_urlencoded", @@ -3640,9 +3614,9 @@ dependencies = [ [[package]] name = "parquet" -version = "56.2.0" +version = "57.3.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "f0dbd48ad52d7dccf8ea1b90a3ddbfaea4f69878dd7683e51c507d4bc52b5b27" +checksum = "6ee96b29972a257b855ff2341b37e61af5f12d6af1158b6dcdb5b31ea07bb3cb" dependencies = [ "ahash", "arrow-array", @@ -3652,7 +3626,7 @@ dependencies = [ "arrow-ipc", "arrow-schema", "arrow-select", - "base64 0.22.1", + "base64", "brotli", "bytes", "chrono", @@ -3661,17 +3635,17 @@ dependencies = [ "half", "hashbrown 0.16.0", "lz4_flex", - "num", "num-bigint", + "num-integer", + "num-traits", "object_store", "paste", - "ring", "seq-macro", "simdutf8", "snap", "thrift", "tokio", - "twox-hash 2.1.0", + "twox-hash", "zstd", ] @@ -3702,16 +3676,6 @@ version = "0.1.3" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "3637c05577168127568a64e9dc5a6887da720efef07b3d9472d45f63ab191166" -[[package]] -name = "petgraph" -version = "0.7.1" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "3672b37090dbd86368a4145bc067582552b29c27377cad4e0a306c97f9bd7772" -dependencies = [ - "fixedbitset", - "indexmap", -] - [[package]] name = "petgraph" version = "0.8.3" @@ -3812,7 +3776,7 @@ version = "0.2.21" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "85eae3c4ed2f50dcfe72643da4befc30deadb458a9b590d720cde2f2b1e97da9" dependencies = [ - "zerocopy 0.8.23", + "zerocopy 0.8.42", ] [[package]] @@ -3826,12 +3790,21 @@ dependencies = [ ] [[package]] -name = "proc-macro-error-attr2" -version = "2.0.0" +name = "proc-macro-crate" +version = "3.5.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "96de42df36bb9bba5542fe9f1a054b8cc87e172759a1868aa05c1f3acc89dfc5" +checksum = "e67ba7e9b2b56446f1d419b1d807906278ffa1a658a8a5d8a39dcb1f5a78614f" dependencies = [ - "proc-macro2", + "toml_edit", +] + +[[package]] +name = "proc-macro-error-attr2" +version = "2.0.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "96de42df36bb9bba5542fe9f1a054b8cc87e172759a1868aa05c1f3acc89dfc5" +dependencies = [ + "proc-macro2", "quote", ] @@ -3877,9 +3850,9 @@ dependencies = [ [[package]] name = "prost" -version = "0.13.5" +version = "0.14.3" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "2796faa41db3ec313a31f7624d9286acf277b52de526150b7e69f3debf891ee5" +checksum = "d2ea70524a2f82d518bce41317d0fae74151505651af45faf1ffbd6fd33f0568" dependencies = [ "bytes", "prost-derive", @@ -3887,16 +3860,15 @@ dependencies = [ [[package]] name = "prost-build" -version = "0.13.5" +version = "0.14.3" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "be769465445e8c1474e9c5dac2018218498557af32d9ed057325ec9a41ae81bf" +checksum = "343d3bd7056eda839b03204e68deff7d1b13aba7af2b2fd16890697274262ee7" dependencies = [ "heck", "itertools 0.14.0", "log", "multimap", - "once_cell", - "petgraph 0.7.1", + "petgraph", "prettyplease", "prost", "prost-types", @@ -3907,9 +3879,9 @@ dependencies = [ [[package]] name = "prost-derive" -version = "0.13.5" +version = "0.14.3" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "8a56d757972c98b346a9b766e3f02746cde6dd1cd1d1d563472929fdd74bec4d" +checksum = "27c6023962132f4b30eb4c172c91ce92d933da334c59c23cddee82358ddafb0b" dependencies = [ "anyhow", "itertools 0.14.0", @@ -3920,9 +3892,9 @@ dependencies = [ [[package]] name = "prost-types" -version = "0.13.5" +version = "0.14.3" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "52c2c1bf36ddb1a1c396b3601a3cec27c2462e45f07c386894ec3ccf5332bd16" +checksum = "8991c4cbdb8bc5b11f0b074ffe286c30e523de90fee5ba8132f1399f23cb3dd7" dependencies = [ "prost", ] @@ -3963,7 +3935,7 @@ dependencies = [ "quinn-proto", "quinn-udp", "rustc-hash", - "rustls 0.23.34", + "rustls", "socket2 0.5.8", "thiserror", "tokio", @@ -3981,7 +3953,7 @@ dependencies = [ "rand 0.8.5", "ring", "rustc-hash", - "rustls 0.23.34", + "rustls", "rustls-pki-types", "slab", "thiserror", @@ -4006,13 +3978,19 @@ dependencies = [ [[package]] name = "quote" -version = "1.0.40" +version = "1.0.45" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "1885c039570dc00dcb4ff087a89e185fd56bae234ddc7f056a945bf36467248d" +checksum = "41f2619966050689382d2b44f664f4bc593e129785a36d6ee376ddf37259b924" dependencies = [ "proc-macro2", ] +[[package]] +name = "r-efi" +version = "6.0.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "f8dcc9c7d52a811697d2151c701e0d08956f92b0e24136cf4cf27b57a6a0d9bf" + [[package]] name = "radix_trie" version = "0.2.1" @@ -4044,6 +4022,17 @@ dependencies = [ "rand_core 0.9.3", ] +[[package]] +name = "rand" +version = "0.10.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "bc266eb313df6c5c09c1c7b1fbe2510961e5bcd3add930c1e31f7ed9da0feff8" +dependencies = [ + "chacha20", + "getrandom 0.4.2", + "rand_core 0.10.0", +] + [[package]] name = "rand_chacha" version = "0.3.1" @@ -4082,6 +4071,12 @@ dependencies = [ "getrandom 0.3.1", ] +[[package]] +name = "rand_core" +version = "0.10.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "0c8d0fd677905edcbeedbf2edb6494d676f0e98d54d5cf9bda0b061cb8fb8aba" + [[package]] name = "rand_xorshift" version = "0.4.0" @@ -4133,9 +4128,9 @@ dependencies = [ [[package]] name = "regex" -version = "1.11.1" +version = "1.12.3" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "b544ef1b4eac5dc2db33ea63606ae9ffcfac26c1416a2806ae0bf5f56b201191" +checksum = "e10754a14b9137dd7b1e3e5b0493cc9171fdd105e0ab477f51b72e7f3ac0e276" dependencies = [ "aho-corasick", "memchr", @@ -4145,9 +4140,9 @@ dependencies = [ [[package]] name = "regex-automata" -version = "0.4.9" +version = "0.4.14" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "809e8dc61f6de73b46c85f4c96486310fe304c434cfa43669d7b40f711150908" +checksum = "6e1dd4122fc1595e8162618945476892eefca7b88c52820e74af6262213cae8f" dependencies = [ "aho-corasick", "memchr", @@ -4166,37 +4161,43 @@ version = "0.8.8" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "7a2d987857b319362043e95f5353c0535c1f58eec5336fdfcf626430af7def58" +[[package]] +name = "relative-path" +version = "1.9.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "ba39f3699c378cd8970968dcbff9c43159ea4cfbd88d43c00b22f2ef10a435d2" + [[package]] name = "reqwest" version = "0.12.24" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "9d0946410b9f7b082a427e4ef5c8ff541a88b357bc6c637c40db3a68ac70a36f" dependencies = [ - "base64 0.22.1", + "base64", "bytes", "futures-core", "futures-util", - "h2 0.4.12", + "h2", "http 1.2.0", "http-body 1.0.1", "http-body-util", - "hyper 1.6.0", - "hyper-rustls 0.27.5", + "hyper", + "hyper-rustls", "hyper-util", "js-sys", "log", "percent-encoding", "pin-project-lite", "quinn", - "rustls 0.23.34", - "rustls-native-certs 0.8.1", + "rustls", + "rustls-native-certs", "rustls-pki-types", "serde", "serde_json", "serde_urlencoded", "sync_wrapper", "tokio", - "tokio-rustls 0.26.2", + "tokio-rustls", "tokio-util", "tower", "tower-http", @@ -4232,6 +4233,35 @@ dependencies = [ "byteorder", ] +[[package]] +name = "rstest" +version = "0.26.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "f5a3193c063baaa2a95a33f03035c8a72b83d97a54916055ba22d35ed3839d49" +dependencies = [ + "futures-timer", + "futures-util", + "rstest_macros", +] + +[[package]] +name = "rstest_macros" +version = "0.26.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "9c845311f0ff7951c5506121a9ad75aec44d083c31583b2ea5a30bcb0b0abba0" +dependencies = [ + "cfg-if", + "glob", + "proc-macro-crate", + "proc-macro2", + "quote", + "regex", + "relative-path", + "rustc_version", + "syn", + "unicode-ident", +] + [[package]] name = "rustc-hash" version = "2.1.1" @@ -4262,27 +4292,15 @@ dependencies = [ [[package]] name = "rustix" -version = "1.0.2" +version = "1.1.4" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "f7178faa4b75a30e269c71e61c353ce2748cf3d76f0c44c393f4e60abf49b825" +checksum = "b6fe4565b9518b83ef4f91bb47ce29620ca828bd32cb7e408f0062e9930ba190" dependencies = [ "bitflags", "errno", "libc", - "linux-raw-sys 0.9.2", - "windows-sys 0.59.0", -] - -[[package]] -name = "rustls" -version = "0.21.12" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "3f56a14d1f48b391359b22f731fd4bd7e43c97f3c50eee276f3aa09c94784d3e" -dependencies = [ - "log", - "ring", - "rustls-webpki 0.101.7", - "sct", + "linux-raw-sys 0.12.1", + "windows-sys 0.61.2", ] [[package]] @@ -4295,23 +4313,11 @@ dependencies = [ "once_cell", "ring", "rustls-pki-types", - "rustls-webpki 0.103.8", + "rustls-webpki", "subtle", "zeroize", ] -[[package]] -name = "rustls-native-certs" -version = "0.6.3" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a9aace74cb666635c918e9c12bc0d348266037aa8eb599b5cba565709a8dff00" -dependencies = [ - "openssl-probe", - "rustls-pemfile 1.0.4", - "schannel", - "security-framework 2.11.1", -] - [[package]] name = "rustls-native-certs" version = "0.8.1" @@ -4321,16 +4327,7 @@ dependencies = [ "openssl-probe", "rustls-pki-types", "schannel", - "security-framework 3.2.0", -] - -[[package]] -name = "rustls-pemfile" -version = "1.0.4" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "1c74cae0a4cf6ccbbf5f359f08efdf8ee7e1dc532573bf0db71968cb56b1448c" -dependencies = [ - "base64 0.21.7", + "security-framework", ] [[package]] @@ -4352,16 +4349,6 @@ dependencies = [ "zeroize", ] -[[package]] -name = "rustls-webpki" -version = "0.101.7" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "8b6275d1ee7a1cd780b64aca7726599a1dbc893b1e64144529e55c3c2f745765" -dependencies = [ - "ring", - "untrusted", -] - [[package]] name = "rustls-webpki" version = "0.103.8" @@ -4444,29 +4431,6 @@ version = "1.2.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "94143f37725109f92c262ed2cf5e59bce7498c01bcc1502d7b9afe439a4e9f49" -[[package]] -name = "sct" -version = "0.7.1" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "da046153aa2352493d6cb7da4b6e5c0c057d8a1d0a9aa8560baffdd945acd414" -dependencies = [ - "ring", - "untrusted", -] - -[[package]] -name = "security-framework" -version = "2.11.1" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "897b2245f0b511c87893af39b033e5ca9cce68824c4d7e7630b5a1d339658d02" -dependencies = [ - "bitflags", - "core-foundation 0.9.4", - "core-foundation-sys", - "libc", - "security-framework-sys", -] - [[package]] name = "security-framework" version = "3.2.0" @@ -4474,7 +4438,7 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "271720403f46ca04f7ba6f55d438f8bd878d6b8ca0a1046e8228c4145bcbb316" dependencies = [ "bitflags", - "core-foundation 0.10.0", + "core-foundation", "core-foundation-sys", "libc", "security-framework-sys", @@ -4547,9 +4511,9 @@ dependencies = [ [[package]] name = "serde_spanned" -version = "1.0.3" +version = "1.0.4" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "e24345aa0fe688594e73770a5f6d1b216508b4f93484c0026d521acd30134392" +checksum = "f8bbf91e5a4d6315eee45e704372590b30e260ee83af6639d64557f51b067776" dependencies = [ "serde_core", ] @@ -4573,7 +4537,7 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "a7507d819769d01a365ab707794a4084392c824f54a7a6a7862f8c3d0892b283" dependencies = [ "cfg-if", - "cpufeatures", + "cpufeatures 0.2.17", "digest", ] @@ -4679,17 +4643,6 @@ dependencies = [ "windows-sys 0.60.2", ] -[[package]] -name = "sqlparser" -version = "0.58.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "ec4b661c54b1e4b603b37873a18c59920e4c51ea8ea2cf527d925424dbd4437c" -dependencies = [ - "log", - "recursive", - "sqlparser_derive", -] - [[package]] name = "sqlparser" version = "0.59.0" @@ -4698,6 +4651,7 @@ checksum = "4591acadbcf52f0af60eafbb2c003232b2b4cd8de5f0e9437cb8b1b59046cc0f" dependencies = [ "log", "recursive", + "sqlparser_derive", ] [[package]] @@ -4836,14 +4790,14 @@ dependencies = [ [[package]] name = "tempfile" -version = "3.23.0" +version = "3.27.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "2d31c77bdf42a745371d260a26ca7163f1e0924b64afa0b688e61b5a9fa02f16" +checksum = "32497e9a4c7b38532efcdebeef879707aa9f794296a4f0244f6f69e9bc8574bd" dependencies = [ "fastrand", "getrandom 0.3.1", "once_cell", - "rustix 1.0.2", + "rustix 1.1.4", "windows-sys 0.61.2", ] @@ -4980,23 +4934,13 @@ dependencies = [ "syn", ] -[[package]] -name = "tokio-rustls" -version = "0.24.1" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c28327cf380ac148141087fbfb9de9d7bd4e84ab5d2c28fbc911d753de8a7081" -dependencies = [ - "rustls 0.21.12", - "tokio", -] - [[package]] name = "tokio-rustls" version = "0.26.2" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "8e727b36a1a0e8b74c376ac2211e40c2c8af09fb4013c60d910495810f008e9b" dependencies = [ - "rustls 0.23.34", + "rustls", "tokio", ] @@ -5026,14 +4970,14 @@ dependencies = [ [[package]] name = "toml" -version = "0.9.8" +version = "0.9.12+spec-1.1.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "f0dc8b1fb61449e27716ec0e1bdf0f6b8f3e8f6b05391e8497b8b6d7804ea6d8" +checksum = "cf92845e79fc2e2def6a5d828f0801e29a2f8acc037becc5ab08595c7d5e9863" dependencies = [ "indexmap", "serde_core", "serde_spanned", - "toml_datetime", + "toml_datetime 0.7.5+spec-1.1.0", "toml_parser", "toml_writer", "winnow", @@ -5041,49 +4985,70 @@ dependencies = [ [[package]] name = "toml_datetime" -version = "0.7.3" +version = "0.7.5+spec-1.1.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "92e1cfed4a3038bc5a127e35a2d360f145e1f4b971b551a2ba5fd7aedf7e1347" +dependencies = [ + "serde_core", +] + +[[package]] +name = "toml_datetime" +version = "1.0.0+spec-1.1.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "f2cdb639ebbc97961c51720f858597f7f24c4fc295327923af55b74c3c724533" +checksum = "32c2555c699578a4f59f0cc68e5116c8d7cabbd45e1409b989d4be085b53f13e" dependencies = [ "serde_core", ] +[[package]] +name = "toml_edit" +version = "0.25.4+spec-1.1.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "7193cbd0ce53dc966037f54351dbbcf0d5a642c7f0038c382ef9e677ce8c13f2" +dependencies = [ + "indexmap", + "toml_datetime 1.0.0+spec-1.1.0", + "toml_parser", + "winnow", +] + [[package]] name = "toml_parser" -version = "1.0.4" +version = "1.0.9+spec-1.1.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c0cbe268d35bdb4bb5a56a2de88d0ad0eb70af5384a99d648cd4b3d04039800e" +checksum = "702d4415e08923e7e1ef96cd5727c0dfed80b4d2fa25db9647fe5eb6f7c5a4c4" dependencies = [ "winnow", ] [[package]] name = "toml_writer" -version = "1.0.4" +version = "1.0.6+spec-1.1.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "df8b2b54733674ad286d16267dcfc7a71ed5c776e4ac7aa3c3e2561f7c637bf2" +checksum = "ab16f14aed21ee8bfd8ec22513f7287cd4a91aa92e44edfe2c17ddd004e92607" [[package]] name = "tonic" -version = "0.13.1" +version = "0.14.5" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "7e581ba15a835f4d9ea06c55ab1bd4dce26fc53752c69a04aac00703bfb49ba9" +checksum = "fec7c61a0695dc1887c1b53952990f3ad2e3a31453e1f49f10e75424943a93ec" dependencies = [ "async-trait", "axum", - "base64 0.22.1", + "base64", "bytes", - "h2 0.4.12", + "h2", "http 1.2.0", "http-body 1.0.1", "http-body-util", - "hyper 1.6.0", + "hyper", "hyper-timeout", "hyper-util", "percent-encoding", "pin-project", - "prost", - "socket2 0.5.8", + "socket2 0.6.1", + "sync_wrapper", "tokio", "tokio-stream", "tower", @@ -5092,6 +5057,17 @@ dependencies = [ "tracing", ] +[[package]] +name = "tonic-prost" +version = "0.14.5" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "a55376a0bbaa4975a3f10d009ad763d8f4108f067c7c2e74f3001fb49778d309" +dependencies = [ + "bytes", + "prost", + "tonic", +] + [[package]] name = "tower" version = "0.5.2" @@ -5143,9 +5119,9 @@ checksum = "8df9b6e13f2d32c91b9bd719c00d1958837bc7dec474d94952798cc8e69eeec3" [[package]] name = "tracing" -version = "0.1.41" +version = "0.1.44" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "784e0ac535deb450455cbfa28a6f0df145ea1bb7ae51b821cf5e7927fdcfbdd0" +checksum = "63e71662fa4b2a2c3a26f570f037eb95bb1f85397f3cd8076caed2f026a6d100" dependencies = [ "log", "pin-project-lite", @@ -5155,9 +5131,9 @@ dependencies = [ [[package]] name = "tracing-attributes" -version = "0.1.28" +version = "0.1.31" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "395ae124c09f9e6918a2310af6038fba074bcf474ac352496d5910dd59a2226d" +checksum = "7490cfa5ec963746568740651ac6781f701c9c5ea257c58e057f3ba8cf69e8da" dependencies = [ "proc-macro2", "quote", @@ -5166,9 +5142,9 @@ dependencies = [ [[package]] name = "tracing-core" -version = "0.1.33" +version = "0.1.36" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "e672c95779cf947c5311f83787af4fa8fffd12fb27e4993211a84bdfd9610f9c" +checksum = "db97caf9d906fbde555dd62fa95ddba9eecfd14cb388e4f491a66d74cd5fb79a" dependencies = [ "once_cell", "valuable", @@ -5187,9 +5163,9 @@ dependencies = [ [[package]] name = "tracing-subscriber" -version = "0.3.20" +version = "0.3.22" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "2054a14f5307d601f88daf0553e1cbf472acc4f2c51afab632431cdcd72124d5" +checksum = "2f30143827ddab0d256fd843b7a66d164e9f271cfa0dde49142c5ca0ca291f1e" dependencies = [ "nu-ansi-term", "sharded-slab", @@ -5207,19 +5183,29 @@ checksum = "e421abadd41a4225275504ea4d6566923418b7f05506fbc9c0fe86ba7396114b" [[package]] name = "twox-hash" -version = "1.6.3" +version = "2.1.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "97fee6b57c6a41524a810daee9286c02d7752c4253064d0b05472833a438f675" +checksum = "e7b17f197b3050ba473acf9181f7b1d3b66d1cf7356c6cc57886662276e65908" + +[[package]] +name = "typed-builder" +version = "0.23.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "31aa81521b70f94402501d848ccc0ecaa8f93c8eb6999eb9747e72287757ffda" dependencies = [ - "cfg-if", - "static_assertions", + "typed-builder-macro", ] [[package]] -name = "twox-hash" -version = "2.1.0" +name = "typed-builder-macro" +version = "0.23.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "e7b17f197b3050ba473acf9181f7b1d3b66d1cf7356c6cc57886662276e65908" +checksum = "076a02dc54dd46795c2e9c8282ed40bcfb1e22747e955de9389a1de28190fb26" +dependencies = [ + "proc-macro2", + "quote", + "syn", +] [[package]] name = "typenum" @@ -5251,6 +5237,12 @@ version = "0.2.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "1fc81956842c57dac11422a97c3b8195a1ff727f06e85c84ed2e8aa277c9a0fd" +[[package]] +name = "unicode-xid" +version = "0.2.6" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "ebc1c04c71510c7f702b52b7c350734c9ff1295c464a03335b00bb84fc54f853" + [[package]] name = "untrusted" version = "0.9.0" @@ -5295,14 +5287,14 @@ checksum = "06abde3611657adf66d383f00b093d7faecc7fa57071cce2578660c9f1010821" [[package]] name = "uuid" -version = "1.18.1" +version = "1.22.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "2f87b8aa10b915a06587d0dec516c282ff295b475d94abf425d62b57710070a2" +checksum = "a68d3c8f01c0cfa54a75291d83601161799e4a89a39e0929f4b0354d88757a37" dependencies = [ - "getrandom 0.3.1", + "getrandom 0.4.2", "js-sys", - "rand 0.9.2", - "serde", + "rand 0.10.0", + "serde_core", "wasm-bindgen", ] @@ -5397,6 +5389,24 @@ dependencies = [ "wit-bindgen-rt", ] +[[package]] +name = "wasip2" +version = "1.0.2+wasi-0.2.9" +source = "registry+https://github.com/rust-lang/crates.io-index" +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", +] + [[package]] name = "wasm-bindgen" version = "0.2.100" @@ -5468,6 +5478,28 @@ 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", + "wasm-encoder", + "wasmparser", +] + [[package]] name = "wasm-streams" version = "0.4.2" @@ -5481,6 +5513,18 @@ 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.2", + "indexmap", + "semver", +] + [[package]] name = "web-sys" version = "0.3.77" @@ -5828,6 +5872,29 @@ name = "winnow" version = "0.7.14" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "5a5364e9d77fcdeeaa6062ced926ee3381faa2ee02d3eb83a5c27a8825540829" +dependencies = [ + "memchr", +] + +[[package]] +name = "wit-bindgen" +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-rt" @@ -5838,6 +5905,74 @@ dependencies = [ "bitflags", ] +[[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", + "prettyplease", + "syn", + "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 = "0c0f9bfd77e6a48eccf51359e3ae77140a7f50b1e2ebfe62422d8afdaffab17a" +dependencies = [ + "anyhow", + "prettyplease", + "proc-macro2", + "quote", + "syn", + "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", + "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", + "log", + "semver", + "serde", + "serde_derive", + "serde_json", + "unicode-xid", + "wasmparser", +] + [[package]] name = "write16" version = "1.0.0" @@ -5906,11 +6041,11 @@ dependencies = [ [[package]] name = "zerocopy" -version = "0.8.23" +version = "0.8.42" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "fd97444d05a4328b90e75e503a34bad781f14e28a823ad3557f0750df1ebcbc6" +checksum = "f2578b716f8a7a858b7f02d5bd870c14bf4ddbbcf3a4c05414ba6503640505e3" dependencies = [ - "zerocopy-derive 0.8.23", + "zerocopy-derive 0.8.42", ] [[package]] @@ -5926,9 +6061,9 @@ dependencies = [ [[package]] name = "zerocopy-derive" -version = "0.8.23" +version = "0.8.42" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "6352c01d0edd5db859a63e2605f4ea3183ddbd15e2c4a9e7d32184df75e4f154" +checksum = "7e6cc098ea4d3bd6246687de65af3f920c430e236bee1e3bf2e441463f08a02f" dependencies = [ "proc-macro2", "quote", From a722bb822080eda9d6a74ebd70dce63c7957a32d Mon Sep 17 00:00:00 2001 From: CGodiksen <36046286+CGodiksen@users.noreply.github.com> Date: Fri, 13 Mar 2026 08:20:08 +0100 Subject: [PATCH 085/116] Add logging to WriteAheadLog struct --- Cargo.lock | 1 + crates/modelardb_storage/Cargo.toml | 1 + .../modelardb_storage/src/write_ahead_log.rs | 21 ++++++++++++++++++- 3 files changed, 22 insertions(+), 1 deletion(-) diff --git a/Cargo.lock b/Cargo.lock index 1ae3eeca..b6904a66 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -3363,6 +3363,7 @@ dependencies = [ "tempfile", "tokio", "tonic", + "tracing", "url", "uuid", ] diff --git a/crates/modelardb_storage/Cargo.toml b/crates/modelardb_storage/Cargo.toml index d3381d04..6f6a20e2 100644 --- a/crates/modelardb_storage/Cargo.toml +++ b/crates/modelardb_storage/Cargo.toml @@ -36,6 +36,7 @@ object_store = { workspace = true, features = ["aws", "azure"] } serde_json.workspace = true sqlparser.workspace = true tonic.workspace = true +tracing.workspace = true url.workspace = true uuid.workspace = true diff --git a/crates/modelardb_storage/src/write_ahead_log.rs b/crates/modelardb_storage/src/write_ahead_log.rs index f476420e..5a6e9caa 100644 --- a/crates/modelardb_storage/src/write_ahead_log.rs +++ b/crates/modelardb_storage/src/write_ahead_log.rs @@ -28,6 +28,7 @@ use arrow::ipc::writer::StreamWriter; use arrow::record_batch::RecordBatch; use deltalake::DeltaTable; use modelardb_types::types::TimeSeriesTableMetadata; +use tracing::{debug, info, warn}; use crate::WRITE_AHEAD_LOG_FOLDER; use crate::data_folder::DataFolder; @@ -86,6 +87,12 @@ impl WriteAheadLog { .await?; } + info!( + path = %log_folder_path.display(), + table_count = write_ahead_log.table_logs.len(), + "WAL initialized." + ); + Ok(write_ahead_log) } @@ -112,6 +119,12 @@ impl WriteAheadLog { .await?; } + debug!( + table = %table_name, + folder_path = %log_file.folder_path.display(), + "WAL table log created." + ); + self.table_logs.insert(table_name, log_file); Ok(()) @@ -138,7 +151,13 @@ impl WriteAheadLog { } // Now that the file handle is closed, the files can be removed. - std::fs::remove_dir_all(log_path)?; + std::fs::remove_dir_all(&log_path)?; + + debug!( + table = %table_name, + folder_path = %log_path.display(), + "WAL table log removed." + ); Ok(()) } From 709a03626bac5ab4f8a1f14ce5cb9b39a69608c0 Mon Sep 17 00:00:00 2001 From: CGodiksen <36046286+CGodiksen@users.noreply.github.com> Date: Fri, 13 Mar 2026 08:21:56 +0100 Subject: [PATCH 086/116] Add logging for creating WAL file and appending --- .../modelardb_storage/src/write_ahead_log.rs | 19 +++++++++++++++++++ 1 file changed, 19 insertions(+) diff --git a/crates/modelardb_storage/src/write_ahead_log.rs b/crates/modelardb_storage/src/write_ahead_log.rs index 5a6e9caa..e4cbfcc5 100644 --- a/crates/modelardb_storage/src/write_ahead_log.rs +++ b/crates/modelardb_storage/src/write_ahead_log.rs @@ -250,6 +250,11 @@ impl ActiveSegment { let writer = StreamWriter::try_new(file, schema)?; + debug!( + path = %path.display(), + "WAL file created." + ); + Ok(Self { path, start_id, @@ -296,6 +301,13 @@ impl WriteAheadLogFile { // The next batch id is one past the end of the last closed segment, or 0 if there are none. let next_id = closed_segments.last().map(|s| s.end_id + 1).unwrap_or(0); + debug!( + folder_path = %folder_path.display(), + closed_segment_count = closed_segments.len(), + next_batch_id = next_id, + "Found closed WAL segments." + ); + // Always create a fresh active segment on startup to avoid writing into the middle of // an existing IPC stream. let active_file = ActiveSegment::try_new(folder_path.clone(), schema, next_id)?; @@ -333,6 +345,13 @@ impl WriteAheadLogFile { let current_batch_id = active.next_batch_id; active.next_batch_id += 1; + debug!( + path = %active.path.display(), + batch_id = current_batch_id, + row_count = data.num_rows(), + "Appended batch to WAL file." + ); + // Rotate to a new segment if the threshold has been reached. The number of batches in the // active segment is the difference between the next batch id (post-increment) and the // active start id. From 6fdd510f110213cebf9072ea025e229b56e99434 Mon Sep 17 00:00:00 2001 From: CGodiksen <36046286+CGodiksen@users.noreply.github.com> Date: Fri, 13 Mar 2026 08:24:21 +0100 Subject: [PATCH 087/116] Add logging for rotating segments, deleting closed segments, and loading batch ids --- .../modelardb_storage/src/write_ahead_log.rs | 18 ++++++++++++++++++ 1 file changed, 18 insertions(+) diff --git a/crates/modelardb_storage/src/write_ahead_log.rs b/crates/modelardb_storage/src/write_ahead_log.rs index e4cbfcc5..be35973e 100644 --- a/crates/modelardb_storage/src/write_ahead_log.rs +++ b/crates/modelardb_storage/src/write_ahead_log.rs @@ -368,6 +368,13 @@ impl WriteAheadLogFile { fn rotate_active_segment(&self, active: &mut ActiveSegment) -> Result<()> { let end_id = active.next_batch_id - 1; + debug!( + path = %active.path.display(), + start_id = active.start_id, + end_id, + "Rotating WAL segment." + ); + // Finish the current writer so the IPC end-of-stream marker is written. active.writer.finish()?; @@ -413,6 +420,11 @@ impl WriteAheadLogFile { let mut to_retain = Vec::new(); for segment in closed_segments.drain(..) { if segment.is_fully_persisted(&persisted) { + debug!( + path = %segment.path.display(), + "Deleting fully persisted WAL segment." + ); + std::fs::remove_file(&segment.path)?; // Remove the persisted ids for this segment as they are no longer needed. @@ -446,6 +458,12 @@ impl WriteAheadLogFile { } } + debug!( + folder_path = %self.folder_path.display(), + batch_ids = ?persisted_batch_ids, + "Loaded persisted batch ids from Delta table commit history." + ); + self.mark_batches_as_persisted(persisted_batch_ids) } From 8006bac2a5688a483c05fa59af4ad52407703c63 Mon Sep 17 00:00:00 2001 From: CGodiksen <36046286+CGodiksen@users.noreply.github.com> Date: Fri, 13 Mar 2026 08:26:45 +0100 Subject: [PATCH 088/116] Add logging for reading WAL files and closing leftover active files --- .../modelardb_storage/src/write_ahead_log.rs | 22 +++++++++++++++---- 1 file changed, 18 insertions(+), 4 deletions(-) diff --git a/crates/modelardb_storage/src/write_ahead_log.rs b/crates/modelardb_storage/src/write_ahead_log.rs index be35973e..81e9d1e1 100644 --- a/crates/modelardb_storage/src/write_ahead_log.rs +++ b/crates/modelardb_storage/src/write_ahead_log.rs @@ -509,6 +509,13 @@ impl WriteAheadLogFile { all_batches.extend((active.start_id..=active.next_batch_id - 1).zip(active_batches)); } + debug!( + folder_path = %self.folder_path.display(), + closed_segment_count = closed_segments.len(), + batch_count = all_batches.len(), + "Read all batches from WAL files." + ); + Ok(all_batches) } } @@ -543,12 +550,19 @@ fn close_leftover_active_segment(folder_path: &PathBuf) -> Result<()> { if batches.is_empty() { std::fs::remove_file(&active_path)?; + debug!(path = %active_path.display(), "Removed empty leftover active WAL segment."); } else { let end_id = start_id + batches.len() as u64 - 1; - std::fs::rename( - &active_path, - folder_path.join(format!("{start_id}-{end_id}.wal")), - )?; + let closed_path = folder_path.join(format!("{start_id}-{end_id}.wal")); + + warn!( + path = %active_path.display(), + closed_path = %closed_path.display(), + batch_count = batches.len(), + "Closed leftover active WAL segment from unclean shutdown." + ); + + std::fs::rename(&active_path, closed_path)?; } Ok(()) From c1c93feaa48759449f25ed52b607794928e01f9a Mon Sep 17 00:00:00 2001 From: CGodiksen <36046286+CGodiksen@users.noreply.github.com> Date: Fri, 13 Mar 2026 08:52:29 +0100 Subject: [PATCH 089/116] Add warn message when replaying unpersisted batches --- crates/modelardb_server/src/context.rs | 10 +++++++++- 1 file changed, 9 insertions(+), 1 deletion(-) diff --git a/crates/modelardb_server/src/context.rs b/crates/modelardb_server/src/context.rs index e6141fe9..2ddbacd3 100644 --- a/crates/modelardb_server/src/context.rs +++ b/crates/modelardb_server/src/context.rs @@ -23,7 +23,7 @@ use datafusion::catalog::{SchemaProvider, TableProvider}; use modelardb_storage::write_ahead_log::WriteAheadLog; use modelardb_types::types::TimeSeriesTableMetadata; use tokio::sync::RwLock; -use tracing::info; +use tracing::{info, warn}; use crate::configuration::ConfigurationManager; use crate::error::{ModelarDbServerError, Result}; @@ -282,6 +282,14 @@ impl Context { let unpersisted_batches = write_ahead_log.unpersisted_batches_in_table_log(&metadata.name)?; + if !unpersisted_batches.is_empty() { + warn!( + table = %metadata.name, + batch_count = unpersisted_batches.len(), + "Replaying unpersisted batches for time series table." + ); + } + for (batch_id, batch) in unpersisted_batches { storage_engine.insert_data_points_with_batch_id( metadata.clone(), From e092e657e5a9f72fb334ed82ed9b527e2bb45bf3 Mon Sep 17 00:00:00 2001 From: CGodiksen <36046286+CGodiksen@users.noreply.github.com> Date: Fri, 13 Mar 2026 09:11:38 +0100 Subject: [PATCH 090/116] Add debug message for marking batches as persisted --- crates/modelardb_storage/src/write_ahead_log.rs | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/crates/modelardb_storage/src/write_ahead_log.rs b/crates/modelardb_storage/src/write_ahead_log.rs index 81e9d1e1..4dc4c43c 100644 --- a/crates/modelardb_storage/src/write_ahead_log.rs +++ b/crates/modelardb_storage/src/write_ahead_log.rs @@ -404,6 +404,12 @@ impl WriteAheadLogFile { /// is now persisted is deleted from disk and removed from the in-memory list. If a segment file /// could not be deleted, return [`ModelarDbStorageError`]. fn mark_batches_as_persisted(&self, batch_ids: HashSet) -> Result<()> { + debug!( + folder_path = %self.folder_path.display(), + batch_ids = ?batch_ids, + "Marking batches as persisted." + ); + let mut persisted = self .persisted_batch_ids .lock() From 417578a4ec02e3d330ca52f1b30983b42168e096 Mon Sep 17 00:00:00 2001 From: CGodiksen <36046286+CGodiksen@users.noreply.github.com> Date: Fri, 13 Mar 2026 09:32:05 +0100 Subject: [PATCH 091/116] Write-ahead log instead of write-ahead-log --- crates/modelardb_server/src/context.rs | 2 +- crates/modelardb_server/src/storage/compressed_data_manager.rs | 2 +- crates/modelardb_storage/src/write_ahead_log.rs | 2 +- 3 files changed, 3 insertions(+), 3 deletions(-) diff --git a/crates/modelardb_server/src/context.rs b/crates/modelardb_server/src/context.rs index 2ddbacd3..50ddafdf 100644 --- a/crates/modelardb_server/src/context.rs +++ b/crates/modelardb_server/src/context.rs @@ -268,7 +268,7 @@ impl Context { Ok(()) } - /// For each time series table in the local data folder, use the write-ahead-log to replay any + /// For each time series table in the local data folder, use the write-ahead log to replay any /// data that was written to the storage engine but not compressed and saved to disk. Note that /// this method should only be called before the storage engine starts ingesting data to avoid /// replaying data that is currently in memory. diff --git a/crates/modelardb_server/src/storage/compressed_data_manager.rs b/crates/modelardb_server/src/storage/compressed_data_manager.rs index 0fcc58e5..3ab0beb4 100644 --- a/crates/modelardb_server/src/storage/compressed_data_manager.rs +++ b/crates/modelardb_server/src/storage/compressed_data_manager.rs @@ -264,7 +264,7 @@ impl CompressedDataManager { ) .await?; - // Inform the write-ahead-log that data has been written to disk. We use a read lock since + // Inform the write-ahead log that data has been written to disk. We use a read lock since // the specific log file is locked internally before being updated. let write_ahead_log = self.write_ahead_log.read().await; write_ahead_log.mark_batches_as_persisted_in_table_log(table_name, batch_ids)?; diff --git a/crates/modelardb_storage/src/write_ahead_log.rs b/crates/modelardb_storage/src/write_ahead_log.rs index 4dc4c43c..d31e2b20 100644 --- a/crates/modelardb_storage/src/write_ahead_log.rs +++ b/crates/modelardb_storage/src/write_ahead_log.rs @@ -96,7 +96,7 @@ impl WriteAheadLog { Ok(write_ahead_log) } - /// Create a new write-ahead-log file for the table with the given metadata. If a delta table + /// Create a new write-ahead log file for the table with the given metadata. If a delta table /// is provided, the log file will be initialized with the persisted batch ids from the commit /// history of the delta table. If a log already exists in the map or the log file could not be /// created, return [`ModelarDbStorageError`]. Note that if the log file already exists, but it From a4fe666b623b8761be95d72fa519d8cf86018e1e Mon Sep 17 00:00:00 2001 From: CGodiksen <36046286+CGodiksen@users.noreply.github.com> Date: Fri, 13 Mar 2026 09:34:54 +0100 Subject: [PATCH 092/116] Add unit tests for WriteAheadLog try_new --- .../modelardb_storage/src/write_ahead_log.rs | 42 +++++++++++++++++++ 1 file changed, 42 insertions(+) diff --git a/crates/modelardb_storage/src/write_ahead_log.rs b/crates/modelardb_storage/src/write_ahead_log.rs index d31e2b20..a1b389e8 100644 --- a/crates/modelardb_storage/src/write_ahead_log.rs +++ b/crates/modelardb_storage/src/write_ahead_log.rs @@ -638,6 +638,43 @@ mod tests { use modelardb_test::table::TIME_SERIES_TABLE_NAME; use tempfile::TempDir; + // Tests for WriteAheadLog. + #[tokio::test] + async fn test_try_new_without_tables_creates_empty_wal() { + let (_temp_dir, wal) = new_empty_write_ahead_log().await; + + assert!(wal.table_logs.is_empty()); + } + + #[tokio::test] + async fn test_try_new_with_existing_table_creates_table_log() { + let (_temp_dir, data_folder) = create_data_folder_with_time_series_table().await; + let wal = WriteAheadLog::try_new(&data_folder).await.unwrap(); + + assert_eq!(wal.table_logs.len(), 1); + assert!(wal.table_logs.contains_key(TIME_SERIES_TABLE_NAME)); + } + + #[tokio::test] + async fn test_try_new_fails_for_non_local_data_folder() { + let data_folder = DataFolder::open_memory().await.unwrap(); + let result = WriteAheadLog::try_new(&data_folder).await; + + assert_eq!( + result.err().unwrap().to_string(), + "Invalid State Error: Write-ahead log location 'memory:///modelardb' is not a local path." + ); + } + + async fn new_empty_write_ahead_log() -> (TempDir, WriteAheadLog) { + let temp_dir = tempfile::tempdir().unwrap(); + let data_folder = DataFolder::open_local(temp_dir.path()).await.unwrap(); + let wal = WriteAheadLog::try_new(&data_folder).await.unwrap(); + + (temp_dir, wal) + } + + // Tests for WriteAheadLogFile. #[test] fn test_try_new_creates_active_segment() { let temp_dir = tempfile::tempdir().unwrap(); @@ -1015,6 +1052,11 @@ mod tests { .await .unwrap(); + data_folder + .save_time_series_table_metadata(&metadata) + .await + .unwrap(); + (temp_dir, data_folder) } From f14b88a86d12a838aa08837ce42aba98a600d789 Mon Sep 17 00:00:00 2001 From: CGodiksen <36046286+CGodiksen@users.noreply.github.com> Date: Fri, 13 Mar 2026 09:41:25 +0100 Subject: [PATCH 093/116] Add unit tests for creating table log --- .../modelardb_storage/src/write_ahead_log.rs | 28 ++++++++++++++++++- 1 file changed, 27 insertions(+), 1 deletion(-) diff --git a/crates/modelardb_storage/src/write_ahead_log.rs b/crates/modelardb_storage/src/write_ahead_log.rs index a1b389e8..f1ec3728 100644 --- a/crates/modelardb_storage/src/write_ahead_log.rs +++ b/crates/modelardb_storage/src/write_ahead_log.rs @@ -130,7 +130,7 @@ impl WriteAheadLog { Ok(()) } else { Err(ModelarDbStorageError::InvalidState(format!( - "Table log for table '{table_name}' already exists", + "Table log for table '{table_name}' already exists.", ))) } } @@ -666,6 +666,32 @@ mod tests { ); } + #[tokio::test] + async fn test_create_table_log_adds_log_for_table() { + let (_temp_dir, mut wal) = new_empty_write_ahead_log().await; + + let metadata = table::time_series_table_metadata(); + wal.create_table_log(&metadata, None).await.unwrap(); + + assert!(wal.table_logs.contains_key(TIME_SERIES_TABLE_NAME)); + } + + #[tokio::test] + async fn test_create_table_log_fails_if_table_log_already_exists() { + let (_temp_dir, mut wal) = new_empty_write_ahead_log().await; + let metadata = table::time_series_table_metadata(); + + wal.create_table_log(&metadata, None).await.unwrap(); + let result = wal.create_table_log(&metadata, None).await; + + assert_eq!( + result.err().unwrap().to_string(), + format!( + "Invalid State Error: Table log for table '{TIME_SERIES_TABLE_NAME}' already exists.", + ) + ); + } + async fn new_empty_write_ahead_log() -> (TempDir, WriteAheadLog) { let temp_dir = tempfile::tempdir().unwrap(); let data_folder = DataFolder::open_local(temp_dir.path()).await.unwrap(); From b6c0d7e7538df70a1ea921999ddcb86044b2c333 Mon Sep 17 00:00:00 2001 From: CGodiksen <36046286+CGodiksen@users.noreply.github.com> Date: Fri, 13 Mar 2026 09:43:43 +0100 Subject: [PATCH 094/116] Add unit tests for removing table log --- .../modelardb_storage/src/write_ahead_log.rs | 28 +++++++++++++++++++ 1 file changed, 28 insertions(+) diff --git a/crates/modelardb_storage/src/write_ahead_log.rs b/crates/modelardb_storage/src/write_ahead_log.rs index f1ec3728..b4d57ad1 100644 --- a/crates/modelardb_storage/src/write_ahead_log.rs +++ b/crates/modelardb_storage/src/write_ahead_log.rs @@ -692,6 +692,34 @@ mod tests { ); } + #[tokio::test] + async fn test_remove_table_log_removes_log_and_directory() { + let (_temp_dir, data_folder) = create_data_folder_with_time_series_table().await; + let mut wal = WriteAheadLog::try_new(&data_folder).await.unwrap(); + + let log_path = wal.table_logs[TIME_SERIES_TABLE_NAME].folder_path.clone(); + assert!(log_path.exists()); + + wal.remove_table_log(TIME_SERIES_TABLE_NAME).unwrap(); + + assert!(!wal.table_logs.contains_key(TIME_SERIES_TABLE_NAME)); + assert!(!log_path.exists()); + } + + #[tokio::test] + async fn test_remove_table_log_fails_if_table_log_does_not_exist() { + let (_temp_dir, mut wal) = new_empty_write_ahead_log().await; + + let result = wal.remove_table_log(TIME_SERIES_TABLE_NAME); + + assert_eq!( + result.err().unwrap().to_string(), + format!( + "Invalid State Error: Table log for table '{TIME_SERIES_TABLE_NAME}' does not exist.", + ) + ); + } + async fn new_empty_write_ahead_log() -> (TempDir, WriteAheadLog) { let temp_dir = tempfile::tempdir().unwrap(); let data_folder = DataFolder::open_local(temp_dir.path()).await.unwrap(); From 1b0d688dde8def2e65532834b33d9a19abb11736 Mon Sep 17 00:00:00 2001 From: CGodiksen <36046286+CGodiksen@users.noreply.github.com> Date: Fri, 13 Mar 2026 09:50:43 +0100 Subject: [PATCH 095/116] Add unit tests for append to table log --- .../modelardb_storage/src/write_ahead_log.rs | 39 +++++++++++++++++++ 1 file changed, 39 insertions(+) diff --git a/crates/modelardb_storage/src/write_ahead_log.rs b/crates/modelardb_storage/src/write_ahead_log.rs index b4d57ad1..be6c778f 100644 --- a/crates/modelardb_storage/src/write_ahead_log.rs +++ b/crates/modelardb_storage/src/write_ahead_log.rs @@ -720,6 +720,45 @@ mod tests { ); } + #[tokio::test] + async fn test_append_to_table_log_returns_incrementing_batch_ids() { + let (_temp_dir, data_folder) = create_data_folder_with_time_series_table().await; + let wal = WriteAheadLog::try_new(&data_folder).await.unwrap(); + + let batch = table::uncompressed_time_series_table_record_batch(5); + + assert_eq!( + wal.append_to_table_log(TIME_SERIES_TABLE_NAME, &batch) + .unwrap(), + 0 + ); + assert_eq!( + wal.append_to_table_log(TIME_SERIES_TABLE_NAME, &batch) + .unwrap(), + 1 + ); + assert_eq!( + wal.append_to_table_log(TIME_SERIES_TABLE_NAME, &batch) + .unwrap(), + 2 + ); + } + + #[tokio::test] + async fn test_append_to_table_log_fails_if_table_log_does_not_exist() { + let (_temp_dir, wal) = new_empty_write_ahead_log().await; + + let batch = table::uncompressed_time_series_table_record_batch(5); + let result = wal.append_to_table_log(TIME_SERIES_TABLE_NAME, &batch); + + assert_eq!( + result.err().unwrap().to_string(), + format!( + "Invalid State Error: Table log for table '{TIME_SERIES_TABLE_NAME}' does not exist.", + ) + ); + } + async fn new_empty_write_ahead_log() -> (TempDir, WriteAheadLog) { let temp_dir = tempfile::tempdir().unwrap(); let data_folder = DataFolder::open_local(temp_dir.path()).await.unwrap(); From 1ac6b357864019137a49682d1a0c182dcc8420cd Mon Sep 17 00:00:00 2001 From: CGodiksen <36046286+CGodiksen@users.noreply.github.com> Date: Fri, 13 Mar 2026 09:53:15 +0100 Subject: [PATCH 096/116] Add unit tests for mark batches as persisted --- .../modelardb_storage/src/write_ahead_log.rs | 36 +++++++++++++++++++ 1 file changed, 36 insertions(+) diff --git a/crates/modelardb_storage/src/write_ahead_log.rs b/crates/modelardb_storage/src/write_ahead_log.rs index be6c778f..8648d346 100644 --- a/crates/modelardb_storage/src/write_ahead_log.rs +++ b/crates/modelardb_storage/src/write_ahead_log.rs @@ -759,6 +759,42 @@ mod tests { ); } + #[tokio::test] + async fn test_mark_batches_as_persisted_in_table_log_removes_from_unpersisted() { + let (_temp_dir, data_folder) = create_data_folder_with_time_series_table().await; + let wal = WriteAheadLog::try_new(&data_folder).await.unwrap(); + + let batch = table::uncompressed_time_series_table_record_batch(5); + wal.append_to_table_log(TIME_SERIES_TABLE_NAME, &batch) + .unwrap(); + wal.append_to_table_log(TIME_SERIES_TABLE_NAME, &batch) + .unwrap(); + + wal.mark_batches_as_persisted_in_table_log(TIME_SERIES_TABLE_NAME, HashSet::from([0, 1])) + .unwrap(); + + let unpersisted = wal + .unpersisted_batches_in_table_log(TIME_SERIES_TABLE_NAME) + .unwrap(); + + assert!(unpersisted.is_empty()); + } + + #[tokio::test] + async fn test_mark_batches_as_persisted_in_table_log_fails_if_table_log_does_not_exist() { + let (_temp_dir, wal) = new_empty_write_ahead_log().await; + + let result = + wal.mark_batches_as_persisted_in_table_log(TIME_SERIES_TABLE_NAME, HashSet::new()); + + assert_eq!( + result.err().unwrap().to_string(), + format!( + "Invalid State Error: Table log for table '{TIME_SERIES_TABLE_NAME}' does not exist.", + ) + ); + } + async fn new_empty_write_ahead_log() -> (TempDir, WriteAheadLog) { let temp_dir = tempfile::tempdir().unwrap(); let data_folder = DataFolder::open_local(temp_dir.path()).await.unwrap(); From ac7eff9e518dc7e7604a68172a23704c9974d2b5 Mon Sep 17 00:00:00 2001 From: CGodiksen <36046286+CGodiksen@users.noreply.github.com> Date: Fri, 13 Mar 2026 09:54:25 +0100 Subject: [PATCH 097/116] Add unit tests for unpersisted batches --- .../modelardb_storage/src/write_ahead_log.rs | 32 +++++++++++++++++++ 1 file changed, 32 insertions(+) diff --git a/crates/modelardb_storage/src/write_ahead_log.rs b/crates/modelardb_storage/src/write_ahead_log.rs index 8648d346..95e56c90 100644 --- a/crates/modelardb_storage/src/write_ahead_log.rs +++ b/crates/modelardb_storage/src/write_ahead_log.rs @@ -795,6 +795,38 @@ mod tests { ); } + #[tokio::test] + async fn test_unpersisted_batches_in_table_log_returns_all_when_none_persisted() { + let (_temp_dir, data_folder) = create_data_folder_with_time_series_table().await; + let wal = WriteAheadLog::try_new(&data_folder).await.unwrap(); + + let batch = table::uncompressed_time_series_table_record_batch(5); + wal.append_to_table_log(TIME_SERIES_TABLE_NAME, &batch) + .unwrap(); + wal.append_to_table_log(TIME_SERIES_TABLE_NAME, &batch) + .unwrap(); + + let unpersisted = wal + .unpersisted_batches_in_table_log(TIME_SERIES_TABLE_NAME) + .unwrap(); + + assert_eq!(unpersisted.len(), 2); + } + + #[tokio::test] + async fn test_unpersisted_batches_in_table_log_fails_if_table_log_does_not_exist() { + let (_temp_dir, wal) = new_empty_write_ahead_log().await; + + let result = wal.unpersisted_batches_in_table_log(TIME_SERIES_TABLE_NAME); + + assert_eq!( + result.err().unwrap().to_string(), + format!( + "Invalid State Error: Table log for table '{TIME_SERIES_TABLE_NAME}' does not exist.", + ) + ); + } + async fn new_empty_write_ahead_log() -> (TempDir, WriteAheadLog) { let temp_dir = tempfile::tempdir().unwrap(); let data_folder = DataFolder::open_local(temp_dir.path()).await.unwrap(); From 7feda8ff7606fea3775e085a5a13c91df935fd74 Mon Sep 17 00:00:00 2001 From: CGodiksen <36046286+CGodiksen@users.noreply.github.com> Date: Fri, 13 Mar 2026 10:13:35 +0100 Subject: [PATCH 098/116] Add unit test for removing table log and appending again --- .../modelardb_storage/src/write_ahead_log.rs | 24 +++++++++++++++++++ 1 file changed, 24 insertions(+) diff --git a/crates/modelardb_storage/src/write_ahead_log.rs b/crates/modelardb_storage/src/write_ahead_log.rs index 95e56c90..4da92566 100644 --- a/crates/modelardb_storage/src/write_ahead_log.rs +++ b/crates/modelardb_storage/src/write_ahead_log.rs @@ -706,6 +706,30 @@ mod tests { assert!(!log_path.exists()); } + #[tokio::test] + async fn test_remove_and_recreate_table_log_resets_batch_ids() { + let (_temp_dir, mut wal) = new_empty_write_ahead_log().await; + + let metadata = table::time_series_table_metadata(); + let batch = table::uncompressed_time_series_table_record_batch(5); + + wal.create_table_log(&metadata, None).await.unwrap(); + + wal.append_to_table_log(TIME_SERIES_TABLE_NAME, &batch) + .unwrap(); + wal.append_to_table_log(TIME_SERIES_TABLE_NAME, &batch) + .unwrap(); + + wal.remove_table_log(TIME_SERIES_TABLE_NAME).unwrap(); + wal.create_table_log(&metadata, None).await.unwrap(); + + assert_eq!( + wal.append_to_table_log(TIME_SERIES_TABLE_NAME, &batch) + .unwrap(), + 0 + ); + } + #[tokio::test] async fn test_remove_table_log_fails_if_table_log_does_not_exist() { let (_temp_dir, mut wal) = new_empty_write_ahead_log().await; From 6c67bb4f7009e1cdbc79465f44540af1b721357a Mon Sep 17 00:00:00 2001 From: CGodiksen <36046286+CGodiksen@users.noreply.github.com> Date: Fri, 13 Mar 2026 10:21:31 +0100 Subject: [PATCH 099/116] Add unit test for loading persisted batch ids on try new --- crates/modelardb_storage/src/write_ahead_log.rs | 17 +++++++++++++++++ 1 file changed, 17 insertions(+) diff --git a/crates/modelardb_storage/src/write_ahead_log.rs b/crates/modelardb_storage/src/write_ahead_log.rs index 4da92566..742c4412 100644 --- a/crates/modelardb_storage/src/write_ahead_log.rs +++ b/crates/modelardb_storage/src/write_ahead_log.rs @@ -655,6 +655,23 @@ mod tests { assert!(wal.table_logs.contains_key(TIME_SERIES_TABLE_NAME)); } + #[tokio::test] + async fn test_try_new_loads_persisted_batch_ids() { + let (_temp_dir, data_folder) = create_data_folder_with_time_series_table().await; + + // Simulate a previous WAL session by committing batch ids to the delta table. + write_compressed_segments_with_batch_ids(&data_folder, HashSet::from([0, 1, 2])).await; + + let wal = WriteAheadLog::try_new(&data_folder).await.unwrap(); + + let persisted = wal.table_logs[TIME_SERIES_TABLE_NAME] + .persisted_batch_ids + .lock() + .unwrap(); + + assert_eq!(*persisted, BTreeSet::from([0, 1, 2])); + } + #[tokio::test] async fn test_try_new_fails_for_non_local_data_folder() { let data_folder = DataFolder::open_memory().await.unwrap(); From aaee538611fd4513f515106e0c99fae9b982ccb2 Mon Sep 17 00:00:00 2001 From: CGodiksen <36046286+CGodiksen@users.noreply.github.com> Date: Fri, 13 Mar 2026 10:56:52 +0100 Subject: [PATCH 100/116] Add more assertions to cover scenario with one closed segment and one active on re-open --- .../modelardb_storage/src/write_ahead_log.rs | 20 ++++++++++++------- 1 file changed, 13 insertions(+), 7 deletions(-) diff --git a/crates/modelardb_storage/src/write_ahead_log.rs b/crates/modelardb_storage/src/write_ahead_log.rs index 742c4412..4f53fca7 100644 --- a/crates/modelardb_storage/src/write_ahead_log.rs +++ b/crates/modelardb_storage/src/write_ahead_log.rs @@ -1020,24 +1020,29 @@ mod tests { let batch = table::uncompressed_time_series_table_record_batch(5); - // Write some batches but do not rotate, so the active segment is leftover as "{start_id}-.wal". + // Write enough batches to trigger a rotation and append to a new active segment. { let wal_file = WriteAheadLogFile::try_new(folder_path.clone(), &metadata.schema).unwrap(); - wal_file.append_and_sync(&batch).unwrap(); - wal_file.append_and_sync(&batch).unwrap(); + + for _ in 0..SEGMENT_ROTATION_THRESHOLD + 2 { + wal_file.append_and_sync(&batch).unwrap(); + } } - // On re-open, close_leftover_active_segment should rename it to "0-1.wal". + // On re-open the leftover active segment should be closed, leaving two closed segments + // and a fresh active segment starting after them. let wal_file = WriteAheadLogFile::try_new(folder_path.clone(), &metadata.schema).unwrap(); let closed = wal_file.closed_segments.lock().unwrap(); - assert_eq!(closed.len(), 1); + assert_eq!(closed.len(), 2); assert_eq!(closed[0].start_id, 0); - assert_eq!(closed[0].end_id, 1); + assert_eq!(closed[0].end_id, SEGMENT_ROTATION_THRESHOLD - 1); + assert_eq!(closed[1].start_id, SEGMENT_ROTATION_THRESHOLD); + assert_eq!(closed[1].end_id, SEGMENT_ROTATION_THRESHOLD + 1); let active = wal_file.active_segment.lock().unwrap(); - assert_eq!(active.next_batch_id, 2); + assert_eq!(active.next_batch_id, SEGMENT_ROTATION_THRESHOLD + 2); } #[test] @@ -1135,6 +1140,7 @@ mod tests { wal_file.mark_batches_as_persisted(ids).unwrap(); assert!(wal_file.closed_segments.lock().unwrap().is_empty()); + assert!(wal_file.persisted_batch_ids.lock().unwrap().is_empty()); } #[tokio::test] From c38839b2cb006d8e64ab5e69c53efe8f254572ae Mon Sep 17 00:00:00 2001 From: CGodiksen <36046286+CGodiksen@users.noreply.github.com> Date: Fri, 13 Mar 2026 11:57:03 +0100 Subject: [PATCH 101/116] Only log closed segments if there are any --- crates/modelardb_storage/src/write_ahead_log.rs | 14 ++++++++------ 1 file changed, 8 insertions(+), 6 deletions(-) diff --git a/crates/modelardb_storage/src/write_ahead_log.rs b/crates/modelardb_storage/src/write_ahead_log.rs index 4f53fca7..a75648a6 100644 --- a/crates/modelardb_storage/src/write_ahead_log.rs +++ b/crates/modelardb_storage/src/write_ahead_log.rs @@ -301,12 +301,14 @@ impl WriteAheadLogFile { // The next batch id is one past the end of the last closed segment, or 0 if there are none. let next_id = closed_segments.last().map(|s| s.end_id + 1).unwrap_or(0); - debug!( - folder_path = %folder_path.display(), - closed_segment_count = closed_segments.len(), - next_batch_id = next_id, - "Found closed WAL segments." - ); + if !closed_segments.is_empty() { + debug!( + folder_path = %folder_path.display(), + closed_segment_count = closed_segments.len(), + next_batch_id = next_id, + "Found closed WAL segments." + ); + } // Always create a fresh active segment on startup to avoid writing into the middle of // an existing IPC stream. From c4f240a5b7b129954c02624e62da41dde8f4917a Mon Sep 17 00:00:00 2001 From: CGodiksen <36046286+CGodiksen@users.noreply.github.com> Date: Fri, 13 Mar 2026 12:14:06 +0100 Subject: [PATCH 102/116] Minor type and documentation changes --- .../modelardb_storage/src/write_ahead_log.rs | 19 +++++++++---------- 1 file changed, 9 insertions(+), 10 deletions(-) diff --git a/crates/modelardb_storage/src/write_ahead_log.rs b/crates/modelardb_storage/src/write_ahead_log.rs index a75648a6..9b80faa6 100644 --- a/crates/modelardb_storage/src/write_ahead_log.rs +++ b/crates/modelardb_storage/src/write_ahead_log.rs @@ -18,7 +18,7 @@ use std::collections::{BTreeSet, HashMap, HashSet}; use std::fs::{File, OpenOptions}; -use std::path::PathBuf; +use std::path::{Path, PathBuf}; use std::sync::Mutex; use arrow::datatypes::{DataType, Field, Schema}; @@ -37,7 +37,6 @@ use crate::error::{ModelarDbStorageError, Result}; /// Folder containing the WAL files for the operations log. const OPERATIONS_LOG_FOLDER: &str = "operations"; -// TODO: Look into using a byte size based threshold instead of a number of batches. /// Number of batches to write to a single WAL segment file before rotating to a new one. const SEGMENT_ROTATION_THRESHOLD: u64 = 100; @@ -68,7 +67,7 @@ impl WriteAheadLog { let log_folder_path = PathBuf::from(format!("{location}/{WRITE_AHEAD_LOG_FOLDER}")); - std::fs::create_dir_all(log_folder_path.clone())?; + std::fs::create_dir_all(&log_folder_path)?; let mut write_ahead_log = Self { folder_path: log_folder_path.clone(), @@ -290,7 +289,7 @@ impl WriteAheadLogFile { /// A fresh active segment is always created on start-up. If the folder or file could not be /// created, return [`ModelarDbStorageError`]. fn try_new(folder_path: PathBuf, schema: &Schema) -> Result { - std::fs::create_dir_all(folder_path.clone())?; + std::fs::create_dir_all(&folder_path)?; close_leftover_active_segment(&folder_path)?; @@ -532,7 +531,7 @@ impl WriteAheadLogFile { /// its final `{start_id}-{end_id}.wal` name so it is picked up as a closed segment. If the /// file contains no batches, it is removed instead. If the file could not be renamed or /// removed, return [`ModelarDbStorageError`]. -fn close_leftover_active_segment(folder_path: &PathBuf) -> Result<()> { +fn close_leftover_active_segment(folder_path: &Path) -> Result<()> { let Some(active_path) = std::fs::read_dir(folder_path)? .filter_map(|e| e.ok()) .map(|e| e.path()) @@ -578,7 +577,7 @@ fn close_leftover_active_segment(folder_path: &PathBuf) -> Result<()> { /// Collect all closed segment files in `folder_path`. Closed segments have names of the form /// `{start_id}-{end_id}.wal` where both `start_id` and `end_id` are valid `u64` values. -fn find_closed_segments(folder_path: &PathBuf) -> Result> { +fn find_closed_segments(folder_path: &Path) -> Result> { let mut segments = Vec::new(); for entry in std::fs::read_dir(folder_path)? { @@ -603,10 +602,10 @@ fn find_closed_segments(folder_path: &PathBuf) -> Result> { Ok(segments) } -/// Read all [`RecordBatches`] from the file at `path`. Tolerates a missing end-of-stream -/// marker, which is normal for the active segment. If the file could not be read, return -/// [`ModelarDbStorageError`]. -fn read_batches_from_path(path: &PathBuf) -> Result> { +/// Read all [`RecordBatches`](RecordBatch) from the file at `path`. Tolerates a missing +/// end-of-stream marker, which is normal for the active segment. If the file could not be read, +/// return [`ModelarDbStorageError`]. +fn read_batches_from_path(path: &Path) -> Result> { let file = File::open(path)?; let reader = StreamReader::try_new(file, None)?; From d1d508d4f28ce66ca72b49c15d4e34209a1d521c Mon Sep 17 00:00:00 2001 From: CGodiksen <36046286+CGodiksen@users.noreply.github.com> Date: Fri, 13 Mar 2026 12:29:47 +0100 Subject: [PATCH 103/116] Locking closed segments immediately when rotating active segment --- .../modelardb_storage/src/write_ahead_log.rs | 18 ++++++++++-------- 1 file changed, 10 insertions(+), 8 deletions(-) diff --git a/crates/modelardb_storage/src/write_ahead_log.rs b/crates/modelardb_storage/src/write_ahead_log.rs index 9b80faa6..18cfa5a0 100644 --- a/crates/modelardb_storage/src/write_ahead_log.rs +++ b/crates/modelardb_storage/src/write_ahead_log.rs @@ -367,6 +367,11 @@ impl WriteAheadLogFile { /// Close the current active segment by renaming it to its final `{start_id}-{end_id}.wal` /// name and open a fresh active segment. The caller must hold the `active_segment` lock. fn rotate_active_segment(&self, active: &mut ActiveSegment) -> Result<()> { + let mut closed_segments = self + .closed_segments + .lock() + .expect("Mutex should not be poisoned."); + let end_id = active.next_batch_id - 1; debug!( @@ -385,14 +390,11 @@ impl WriteAheadLogFile { .join(format!("{}-{end_id}.wal", active.start_id)); std::fs::rename(&active.path, &closed_path)?; - self.closed_segments - .lock() - .expect("Mutex should not be poisoned.") - .push(ClosedSegment { - path: closed_path, - start_id: active.start_id, - end_id, - }); + closed_segments.push(ClosedSegment { + path: closed_path, + start_id: active.start_id, + end_id, + }); // Open a fresh active segment. let next_id = end_id + 1; From 13a1e6f19ff671d9e76b1c62c56d692fd5b3dbf0 Mon Sep 17 00:00:00 2001 From: CGodiksen <36046286+CGodiksen@users.noreply.github.com> Date: Fri, 13 Mar 2026 13:24:32 +0100 Subject: [PATCH 104/116] Minor refactoring to WAL code --- .../modelardb_storage/src/data_folder/mod.rs | 2 +- .../modelardb_storage/src/write_ahead_log.rs | 26 +++++++++---------- 2 files changed, 14 insertions(+), 14 deletions(-) diff --git a/crates/modelardb_storage/src/data_folder/mod.rs b/crates/modelardb_storage/src/data_folder/mod.rs index 8c144c2f..6bcd942d 100644 --- a/crates/modelardb_storage/src/data_folder/mod.rs +++ b/crates/modelardb_storage/src/data_folder/mod.rs @@ -1252,7 +1252,7 @@ impl DeltaTableWriter { let mut commit_properties = CommitProperties::default(); if !self.batch_ids.is_empty() { commit_properties = commit_properties - .with_metadata(vec![("batchIds".to_string(), json!(self.batch_ids))]); + .with_metadata(vec![("batchIds".to_owned(), json!(self.batch_ids))]); } let table_data = match self.delta_table.snapshot() { diff --git a/crates/modelardb_storage/src/write_ahead_log.rs b/crates/modelardb_storage/src/write_ahead_log.rs index 18cfa5a0..98161676 100644 --- a/crates/modelardb_storage/src/write_ahead_log.rs +++ b/crates/modelardb_storage/src/write_ahead_log.rs @@ -1,4 +1,4 @@ -/* Copyright 2025 The ModelarDB Contributors +/* Copyright 2026 The ModelarDB Contributors * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -65,7 +65,7 @@ impl WriteAheadLog { ))); } - let log_folder_path = PathBuf::from(format!("{location}/{WRITE_AHEAD_LOG_FOLDER}")); + let log_folder_path = PathBuf::from(location).join(WRITE_AHEAD_LOG_FOLDER); std::fs::create_dir_all(&log_folder_path)?; @@ -103,7 +103,7 @@ impl WriteAheadLog { pub async fn create_table_log( &mut self, time_series_table_metadata: &TimeSeriesTableMetadata, - delta_table: Option, + maybe_delta_table: Option, ) -> Result<()> { let table_name = time_series_table_metadata.name.clone(); @@ -112,7 +112,7 @@ impl WriteAheadLog { let log_file = WriteAheadLogFile::try_new(table_log_path, &time_series_table_metadata.schema)?; - if let Some(delta_table) = delta_table { + if let Some(delta_table) = maybe_delta_table { log_file .load_persisted_batches_from_delta_table(delta_table) .await?; @@ -278,8 +278,8 @@ struct WriteAheadLogFile { active_segment: Mutex, /// Closed, read-only segment files ordered by `start_id`. closed_segments: Mutex>, - /// Batch ids that have been confirmed as saved to disk. Used to determine whether a - /// contiguous prefix of batches can be trimmed from the start of the log file. + /// Batch ids that have been confirmed as saved to disk. Used to determine when closed segments + /// can be deleted. persisted_batch_ids: Mutex>, } @@ -535,11 +535,11 @@ impl WriteAheadLogFile { /// removed, return [`ModelarDbStorageError`]. fn close_leftover_active_segment(folder_path: &Path) -> Result<()> { let Some(active_path) = std::fs::read_dir(folder_path)? - .filter_map(|e| e.ok()) - .map(|e| e.path()) - .find(|p| { - p.file_stem() - .and_then(|s| s.to_str()) + .filter_map(|maybe_entry| maybe_entry.ok()) + .map(|entry| entry.path()) + .find(|path| { + path.file_stem() + .and_then(|stem| stem.to_str()) .is_some_and(|stem| stem.ends_with('-')) }) else { @@ -548,7 +548,7 @@ fn close_leftover_active_segment(folder_path: &Path) -> Result<()> { let stem = active_path .file_stem() - .and_then(|s| s.to_str()) + .and_then(|stem| stem.to_str()) .expect("Active WAL segment stem should be '{start_id}-'."); let start_id: u64 = stem[..stem.len() - 1] @@ -586,7 +586,7 @@ fn find_closed_segments(folder_path: &Path) -> Result> { let path = entry?.path(); let stem = path .file_stem() - .and_then(|s| s.to_str()) + .and_then(|stem| stem.to_str()) .expect("WAL file should have a valid UTF-8 stem."); if let Some((start_id, end_id)) = stem From 91fbb667cb0bbbe6430bfa914a3830c3304935cc Mon Sep 17 00:00:00 2001 From: CGodiksen <36046286+CGodiksen@users.noreply.github.com> Date: Fri, 13 Mar 2026 13:46:52 +0100 Subject: [PATCH 105/116] Minor test refactoring to WAL code --- crates/modelardb_storage/src/write_ahead_log.rs | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/crates/modelardb_storage/src/write_ahead_log.rs b/crates/modelardb_storage/src/write_ahead_log.rs index 98161676..c3179f9f 100644 --- a/crates/modelardb_storage/src/write_ahead_log.rs +++ b/crates/modelardb_storage/src/write_ahead_log.rs @@ -1009,7 +1009,7 @@ mod tests { wal_file.append_and_sync(&batch).unwrap(); let batches = wal_file.read_all().unwrap(); - assert_eq!(batches.len(), SEGMENT_ROTATION_THRESHOLD as usize + 1); + assert_eq!(batches.len() as u64, SEGMENT_ROTATION_THRESHOLD + 1); let active = wal_file.active_segment.lock().unwrap(); assert_eq!(active.next_batch_id, SEGMENT_ROTATION_THRESHOLD + 1); @@ -1222,7 +1222,7 @@ mod tests { assert!(!segment_path.exists()); assert!(wal_file.closed_segments.lock().unwrap().is_empty()); - assert_eq!(wal_file.persisted_batch_ids.lock().unwrap().len(), 0); + assert!(wal_file.persisted_batch_ids.lock().unwrap().is_empty()); } #[tokio::test] From b74bb0ad52ea19a346a34cfeeee616cc617b0b9c Mon Sep 17 00:00:00 2001 From: CGodiksen <36046286+CGodiksen@users.noreply.github.com> Date: Fri, 13 Mar 2026 14:11:11 +0100 Subject: [PATCH 106/116] Fix small issue with order or retain and delete when marking as persisted --- .../modelardb_storage/src/write_ahead_log.rs | 31 +++++++++---------- 1 file changed, 15 insertions(+), 16 deletions(-) diff --git a/crates/modelardb_storage/src/write_ahead_log.rs b/crates/modelardb_storage/src/write_ahead_log.rs index c3179f9f..0c2904a1 100644 --- a/crates/modelardb_storage/src/write_ahead_log.rs +++ b/crates/modelardb_storage/src/write_ahead_log.rs @@ -426,27 +426,26 @@ impl WriteAheadLogFile { .expect("Mutex should not be poisoned."); // Identify and delete fully persisted segments. - let mut to_retain = Vec::new(); - for segment in closed_segments.drain(..) { - if segment.is_fully_persisted(&persisted) { - debug!( - path = %segment.path.display(), - "Deleting fully persisted WAL segment." - ); + let (to_delete, to_retain): (Vec<_>, Vec<_>) = closed_segments + .drain(..) + .partition(|segment| segment.is_fully_persisted(&persisted)); - std::fs::remove_file(&segment.path)?; + *closed_segments = to_retain; - // Remove the persisted ids for this segment as they are no longer needed. - for id in segment.start_id..=segment.end_id { - persisted.remove(&id); - } - } else { - to_retain.push(segment); + for segment in to_delete { + debug!( + path = %segment.path.display(), + "Deleting fully persisted WAL segment." + ); + + std::fs::remove_file(&segment.path)?; + + // Remove the persisted ids for this segment as they are no longer needed. + for id in segment.start_id..=segment.end_id { + persisted.remove(&id); } } - *closed_segments = to_retain; - Ok(()) } From 73f93579f459e7090290d2b88292af73578421a6 Mon Sep 17 00:00:00 2001 From: CGodiksen <36046286+CGodiksen@users.noreply.github.com> Date: Tue, 24 Mar 2026 08:20:31 +0100 Subject: [PATCH 107/116] Expanded documentation for batch_ids field --- .../src/storage/compressed_data_buffer.rs | 8 +++++--- .../src/storage/uncompressed_data_buffer.rs | 8 ++++---- 2 files changed, 9 insertions(+), 7 deletions(-) diff --git a/crates/modelardb_server/src/storage/compressed_data_buffer.rs b/crates/modelardb_server/src/storage/compressed_data_buffer.rs index 08c58ae1..4949ac30 100644 --- a/crates/modelardb_server/src/storage/compressed_data_buffer.rs +++ b/crates/modelardb_server/src/storage/compressed_data_buffer.rs @@ -31,7 +31,8 @@ pub(super) struct CompressedSegmentBatch { pub(super) time_series_table_metadata: Arc, /// Compressed segments representing the data points to insert. pub(super) compressed_segments: Vec, - /// The ids given to the data by the WAL. + /// The ids of the uncompressed batches that correspond to the compressed segments. The ids are + /// assigned by the WAL and are used to delete uncompressed data when compressed data is saved. pub(super) batch_ids: HashSet, } @@ -64,7 +65,8 @@ pub(super) struct CompressedDataBuffer { compressed_segments: Vec, /// Continuously updated total sum of the size of the compressed segments. pub(super) size_in_bytes: u64, - /// The ids given to the data by the WAL. + /// The ids of the uncompressed batches that correspond to the compressed segments. The ids are + /// assigned by the WAL and are used to delete uncompressed data when compressed data is saved. batch_ids: HashSet, } @@ -113,7 +115,7 @@ impl CompressedDataBuffer { self.compressed_segments } - /// Return the ids given to the data by the WAL. + /// Return the ids given to the uncompressed batches by the WAL. pub(super) fn batch_ids(&self) -> HashSet { self.batch_ids.clone() } diff --git a/crates/modelardb_server/src/storage/uncompressed_data_buffer.rs b/crates/modelardb_server/src/storage/uncompressed_data_buffer.rs index f16e2c22..ad266f48 100644 --- a/crates/modelardb_server/src/storage/uncompressed_data_buffer.rs +++ b/crates/modelardb_server/src/storage/uncompressed_data_buffer.rs @@ -88,7 +88,7 @@ pub(super) struct UncompressedInMemoryDataBuffer { values: Vec, /// The tag values for the time series the buffer stores data points for. tag_values: Vec, - /// The ids given to the data by the WAL. + /// The ids given to the batches by the WAL. batch_ids: HashSet, } @@ -215,7 +215,7 @@ impl UncompressedInMemoryDataBuffer { compute_memory_size(self.values.len()) } - /// Return the ids given to the data by the WAL. + /// Return the ids given to the batches by the WAL. pub(super) fn batch_ids(&self) -> &HashSet { &self.batch_ids } @@ -274,7 +274,7 @@ pub(super) struct UncompressedOnDiskDataBuffer { /// Path to the Apache Parquet file containing the uncompressed data in the /// [`UncompressedOnDiskDataBuffer`]. file_path: Path, - /// The ids given to the data by the WAL. + /// The ids given to the batches by the WAL. batch_ids: HashSet, } @@ -339,7 +339,7 @@ impl UncompressedOnDiskDataBuffer { &self.time_series_table_metadata } - /// Return the ids given to the data by the WAL. + /// Return the ids given to the batches by the WAL. pub(super) fn batch_ids(&self) -> &HashSet { &self.batch_ids } From 081a9439c4aed8be6f9dd5ebe09fbaf146a8a3ab Mon Sep 17 00:00:00 2001 From: CGodiksen <36046286+CGodiksen@users.noreply.github.com> Date: Tue, 24 Mar 2026 08:34:24 +0100 Subject: [PATCH 108/116] Append CompressedSegmentBatch to CompressedDataBuffer directly --- .../src/storage/compressed_data_buffer.rs | 50 +++++++++---------- .../src/storage/compressed_data_manager.rs | 14 ++---- 2 files changed, 28 insertions(+), 36 deletions(-) diff --git a/crates/modelardb_server/src/storage/compressed_data_buffer.rs b/crates/modelardb_server/src/storage/compressed_data_buffer.rs index 4949ac30..3086e757 100644 --- a/crates/modelardb_server/src/storage/compressed_data_buffer.rs +++ b/crates/modelardb_server/src/storage/compressed_data_buffer.rs @@ -80,14 +80,15 @@ impl CompressedDataBuffer { } } - /// Append `compressed_segments` to the [`CompressedDataBuffer`] and return the size of - /// `compressed_segments` in bytes if their schema matches the time series table, otherwise - /// [`ModelarDbServerError`] is returned. - pub(super) fn append_compressed_segments( + /// Append the compressed segments in `compressed_segment_batch` to the [`CompressedDataBuffer`] + /// and return the size of the compressed segments in bytes if their schema matches the time + /// series table, otherwise [`ModelarDbServerError`] is returned. + pub(super) fn append_compressed_segment_batch( &mut self, - mut compressed_segments: Vec, - batch_ids: HashSet, + compressed_segment_batch: CompressedSegmentBatch, ) -> Result { + let mut compressed_segments = compressed_segment_batch.compressed_segments; + if compressed_segments.iter().any(|compressed_segments| { compressed_segments.schema() != self.time_series_table_metadata.compressed_schema }) { @@ -105,7 +106,7 @@ impl CompressedDataBuffer { self.size_in_bytes += compressed_segments_size; } - self.batch_ids.extend(batch_ids); + self.batch_ids.extend(compressed_segment_batch.batch_ids); Ok(compressed_segments_size) } @@ -146,18 +147,12 @@ mod tests { use modelardb_test::table; #[test] - fn test_can_append_valid_compressed_segments() { + fn test_can_append_valid_compressed_segment_batch() { let mut compressed_data_buffer = CompressedDataBuffer::new(table::time_series_table_metadata_arc()); compressed_data_buffer - .append_compressed_segments( - vec![ - table::compressed_segments_record_batch(), - table::compressed_segments_record_batch(), - ], - HashSet::from([0, 1, 2]), - ) + .append_compressed_segment_batch(compressed_segment_batch()) .unwrap(); assert_eq!(compressed_data_buffer.compressed_segments.len(), 2); @@ -171,13 +166,7 @@ mod tests { CompressedDataBuffer::new(table::time_series_table_metadata_arc()); compressed_data_buffer - .append_compressed_segments( - vec![ - table::compressed_segments_record_batch(), - table::compressed_segments_record_batch(), - ], - HashSet::from([0, 1, 2]), - ) + .append_compressed_segment_batch(compressed_segment_batch()) .unwrap(); assert!(compressed_data_buffer.size_in_bytes > 0); @@ -188,12 +177,8 @@ mod tests { let mut compressed_data_buffer = CompressedDataBuffer::new(table::time_series_table_metadata_arc()); - let compressed_segments = vec![ - table::compressed_segments_record_batch(), - table::compressed_segments_record_batch(), - ]; compressed_data_buffer - .append_compressed_segments(compressed_segments, HashSet::from([0, 1, 2])) + .append_compressed_segment_batch(compressed_segment_batch()) .unwrap(); let record_batches = compressed_data_buffer.record_batches(); @@ -203,6 +188,17 @@ mod tests { assert_eq!(record_batch.num_rows(), 6); } + fn compressed_segment_batch() -> CompressedSegmentBatch { + CompressedSegmentBatch::new( + table::time_series_table_metadata_arc(), + vec![ + table::compressed_segments_record_batch(), + table::compressed_segments_record_batch(), + ], + HashSet::from([0, 1, 2]), + ) + } + #[test] fn test_get_size_of_compressed_data_buffer() { let compressed_data_buffer = table::compressed_segments_record_batch(); diff --git a/crates/modelardb_server/src/storage/compressed_data_manager.rs b/crates/modelardb_server/src/storage/compressed_data_manager.rs index 3ab0beb4..4ea50d09 100644 --- a/crates/modelardb_server/src/storage/compressed_data_manager.rs +++ b/crates/modelardb_server/src/storage/compressed_data_manager.rs @@ -152,10 +152,7 @@ impl CompressedDataManager { { debug!("Found existing compressed data buffer for table '{time_series_table_name}'.",); - compressed_data_buffer.append_compressed_segments( - compressed_segment_batch.compressed_segments, - compressed_segment_batch.batch_ids, - ) + compressed_data_buffer.append_compressed_segment_batch(compressed_segment_batch) } else { // A String is created as two copies are required for compressed_data_buffer and // compressed_queue anyway and compressed_segments cannot be moved out of @@ -165,12 +162,11 @@ impl CompressedDataManager { "Creating compressed data buffer for table '{time_series_table_name}' as none exist.", ); - let mut compressed_data_buffer = - CompressedDataBuffer::new(compressed_segment_batch.time_series_table_metadata); - let segment_size = compressed_data_buffer.append_compressed_segments( - compressed_segment_batch.compressed_segments, - compressed_segment_batch.batch_ids, + let mut compressed_data_buffer = CompressedDataBuffer::new( + compressed_segment_batch.time_series_table_metadata.clone(), ); + let segment_size = + compressed_data_buffer.append_compressed_segment_batch(compressed_segment_batch); self.compressed_data_buffers .insert(time_series_table_name.clone(), compressed_data_buffer); From 55c6b11f1160de5fc8c38bcfbea09f849fe87317 Mon Sep 17 00:00:00 2001 From: CGodiksen <36046286+CGodiksen@users.noreply.github.com> Date: Tue, 24 Mar 2026 09:46:26 +0100 Subject: [PATCH 109/116] Add more WAL specific documentation and change WAL folder --- crates/modelardb_server/src/configuration.rs | 10 +++++----- crates/modelardb_server/src/main.rs | 6 +++--- .../src/storage/compressed_data_manager.rs | 7 ++++++- crates/modelardb_server/src/storage/mod.rs | 4 ++-- .../src/storage/uncompressed_data_manager.rs | 2 +- crates/modelardb_storage/src/data_folder/mod.rs | 12 ++++++++---- crates/modelardb_storage/src/lib.rs | 2 +- 7 files changed, 26 insertions(+), 17 deletions(-) diff --git a/crates/modelardb_server/src/configuration.rs b/crates/modelardb_server/src/configuration.rs index af59264d..90d9c796 100644 --- a/crates/modelardb_server/src/configuration.rs +++ b/crates/modelardb_server/src/configuration.rs @@ -690,19 +690,19 @@ mod tests { .await .unwrap(); + let write_ahead_log = Arc::new(RwLock::new( + WriteAheadLog::try_new(&local_data_folder).await.unwrap(), + )); + let configuration_manager = Arc::new(RwLock::new( ConfigurationManager::try_new( - local_data_folder.clone(), + local_data_folder, ClusterMode::MultiNode(Box::new(cluster)), ) .await .unwrap(), )); - let write_ahead_log = Arc::new(RwLock::new( - WriteAheadLog::try_new(&local_data_folder).await.unwrap(), - )); - let storage_engine = Arc::new(RwLock::new( StorageEngine::try_new(data_folders, write_ahead_log, &configuration_manager) .await diff --git a/crates/modelardb_server/src/main.rs b/crates/modelardb_server/src/main.rs index ab509982..ce5cfbd0 100644 --- a/crates/modelardb_server/src/main.rs +++ b/crates/modelardb_server/src/main.rs @@ -83,12 +83,12 @@ async fn main() -> Result<()> { cluster.retrieve_and_create_tables(&context).await?; } - // Replay any data that was written to the storage engine but not compressed and saved to disk. - context.replay_write_ahead_log().await?; - // Setup CTRL+C handler. setup_ctrl_c_handler(&context); + // Replay any data that was written to the storage engine but not compressed and saved to disk. + context.replay_write_ahead_log().await?; + // Start the Apache Arrow Flight interface. remote::start_apache_arrow_flight_server(context, *PORT).await?; diff --git a/crates/modelardb_server/src/storage/compressed_data_manager.rs b/crates/modelardb_server/src/storage/compressed_data_manager.rs index 4ea50d09..92efbeb4 100644 --- a/crates/modelardb_server/src/storage/compressed_data_manager.rs +++ b/crates/modelardb_server/src/storage/compressed_data_manager.rs @@ -252,6 +252,11 @@ impl CompressedDataManager { let batch_ids = compressed_data_buffer.batch_ids(); let compressed_segments = compressed_data_buffer.record_batches(); + // If a crash occurs between writing to the Delta Lake and updating the WAL, no data is + // lost or duplicated. The batch_ids are stored in the Delta Lake commit metadata, so on + // restart the WAL recovers which batches were persisted from the commit history and + // excludes them during replay. The WAL update is only an optimization to eagerly delete + // fully persisted WAL segment files. self.local_data_folder .write_compressed_segments_to_time_series_table( table_name, @@ -261,7 +266,7 @@ impl CompressedDataManager { .await?; // Inform the write-ahead log that data has been written to disk. We use a read lock since - // the specific log file is locked internally before being updated. + // the specific WAL file is locked internally before being updated. let write_ahead_log = self.write_ahead_log.read().await; write_ahead_log.mark_batches_as_persisted_in_table_log(table_name, batch_ids)?; diff --git a/crates/modelardb_server/src/storage/mod.rs b/crates/modelardb_server/src/storage/mod.rs index 77e2e055..a1aed2e8 100644 --- a/crates/modelardb_server/src/storage/mod.rs +++ b/crates/modelardb_server/src/storage/mod.rs @@ -260,8 +260,8 @@ impl StorageEngine { time_series_table_metadata: Arc, multivariate_data_points: RecordBatch, ) -> Result<()> { - // Write to the write-ahead log to ensure termination never duplicates or loses data. We use - // a read lock since the specific log file is locked internally before writing. + // Write to the write-ahead log to ensure termination never loses data. We use a read lock + // since the specific log file is locked internally before writing. let batch_id = { let write_ahead_log = self.write_ahead_log.read().await; write_ahead_log diff --git a/crates/modelardb_server/src/storage/uncompressed_data_manager.rs b/crates/modelardb_server/src/storage/uncompressed_data_manager.rs index 00d69d27..c1cef774 100644 --- a/crates/modelardb_server/src/storage/uncompressed_data_manager.rs +++ b/crates/modelardb_server/src/storage/uncompressed_data_manager.rs @@ -69,7 +69,7 @@ pub(super) struct UncompressedDataManager { impl UncompressedDataManager { /// Create a new [`UncompressedDataManager`] and delete all existing spilled buffers if - /// necessary. If the existing buffers could not be deleted, return + /// there are any. If the existing buffers could not be deleted, return /// [`ModelarDbServerError`](crate::error::ModelarDbServerError). pub(super) async fn try_new( local_data_folder: DataFolder, diff --git a/crates/modelardb_storage/src/data_folder/mod.rs b/crates/modelardb_storage/src/data_folder/mod.rs index 6bcd942d..b266f03c 100644 --- a/crates/modelardb_storage/src/data_folder/mod.rs +++ b/crates/modelardb_storage/src/data_folder/mod.rs @@ -874,8 +874,10 @@ impl DataFolder { } /// Write `compressed_segments` to a Delta Lake table for a time series table with `table_name`. - /// The `batch_ids` from the WAL are included in the commit metadata for checkpointing. - /// Returns an updated [`DeltaTable`] if the file was written successfully, otherwise returns + /// The `batch_ids` from the WAL are included in the commit metadata, so the uncompressed + /// batches that correspond to the compressed segments can be deleted from the WAL. If the + /// uncompressed batches were not written to the WAL, `batch_ids` can be left empty. Returns an + /// updated [`DeltaTable`] if the file was written successfully, otherwise returns /// [`ModelarDbStorageError`]. pub async fn write_compressed_segments_to_time_series_table( &self, @@ -1147,7 +1149,8 @@ pub struct DeltaTableWriter { operation_id: Uuid, /// Writes record batches to the Delta table as Apache Parquet files. delta_writer: DeltaWriter, - /// Batch ids from the WAL to include in the commit metadata for checkpointing. + /// Batch ids from the WAL to include in the commit metadata so the uncompressed batches that + /// correspond to the compressed segments can be deleted from the WAL. batch_ids: HashSet, } @@ -1205,7 +1208,8 @@ impl DeltaTableWriter { }) } - /// Add batch ids from the WAL that are included in the commit metadata for checkpointing. + /// Add batch ids from the WAL that are included in the commit metadata so the uncompressed + /// batches that correspond to the compressed segments can be deleted from the WAL. pub fn with_batch_ids(mut self, batch_ids: HashSet) -> Self { self.batch_ids = batch_ids; self diff --git a/crates/modelardb_storage/src/lib.rs b/crates/modelardb_storage/src/lib.rs index 1cd81afc..73b84b64 100644 --- a/crates/modelardb_storage/src/lib.rs +++ b/crates/modelardb_storage/src/lib.rs @@ -64,7 +64,7 @@ const TABLE_FOLDER: &str = "tables"; const METADATA_FOLDER: &str = "metadata"; /// The folder storing the write-ahead log in the data folders. -pub const WRITE_AHEAD_LOG_FOLDER: &str = "_modelardb_log"; +pub const WRITE_AHEAD_LOG_FOLDER: &str = "wal"; /// Create a new [`SessionContext`] for interacting with Apache DataFusion. The [`SessionContext`] /// is constructed with the default configuration, default resource managers, and additional From 369affc191557ae3d453eadc80968cb32fb7e9fc Mon Sep 17 00:00:00 2001 From: CGodiksen <36046286+CGodiksen@users.noreply.github.com> Date: Tue, 24 Mar 2026 09:57:54 +0100 Subject: [PATCH 110/116] Remove all mentions of operations from WAL --- .../modelardb_storage/src/write_ahead_log.rs | 20 +++---------------- 1 file changed, 3 insertions(+), 17 deletions(-) diff --git a/crates/modelardb_storage/src/write_ahead_log.rs b/crates/modelardb_storage/src/write_ahead_log.rs index 0c2904a1..e6c87857 100644 --- a/crates/modelardb_storage/src/write_ahead_log.rs +++ b/crates/modelardb_storage/src/write_ahead_log.rs @@ -14,14 +14,14 @@ */ //! Implementation of types that provide a write-ahead log for ModelarDB that can be used to -//! efficiently persist data and operations on disk to avoid data loss and enable crash recovery. +//! efficiently persist data on disk to avoid data loss and enable crash recovery. use std::collections::{BTreeSet, HashMap, HashSet}; use std::fs::{File, OpenOptions}; use std::path::{Path, PathBuf}; use std::sync::Mutex; -use arrow::datatypes::{DataType, Field, Schema}; +use arrow::datatypes::Schema; use arrow::error::ArrowError::IpcError; use arrow::ipc::reader::StreamReader; use arrow::ipc::writer::StreamWriter; @@ -34,20 +34,15 @@ use crate::WRITE_AHEAD_LOG_FOLDER; use crate::data_folder::DataFolder; use crate::error::{ModelarDbStorageError, Result}; -/// Folder containing the WAL files for the operations log. -const OPERATIONS_LOG_FOLDER: &str = "operations"; - /// Number of batches to write to a single WAL segment file before rotating to a new one. const SEGMENT_ROTATION_THRESHOLD: u64 = 100; -/// Write-ahead log that logs data on a per-table level and operations separately. +/// Write-ahead log that logs data on a per-table level. pub struct WriteAheadLog { /// Path to the folder that contains the write-ahead log. folder_path: PathBuf, /// Logs for each table. The key is the table name, and the value is the log file for that table. table_logs: HashMap, - /// Log file for operations that are not associated with a specific table. - _operation_log: WriteAheadLogFile, } impl WriteAheadLog { @@ -72,10 +67,6 @@ impl WriteAheadLog { let mut write_ahead_log = Self { folder_path: log_folder_path.clone(), table_logs: HashMap::new(), - _operation_log: WriteAheadLogFile::try_new( - log_folder_path.join(OPERATIONS_LOG_FOLDER), - &operations_log_schema(), - )?, }; // For each time series table, create a log file if it does not already exist. @@ -627,11 +618,6 @@ fn read_batches_from_path(path: &Path) -> Result> { Ok(batches) } -/// Return the schema for the operations log that is stored in [`OPERATIONS_LOG_FOLDER`]. -fn operations_log_schema() -> Schema { - Schema::new(vec![Field::new("operation", DataType::Utf8, false)]) -} - #[cfg(test)] mod tests { use super::*; From e6602a76dc3db321d34860c5bdc18364bb234570 Mon Sep 17 00:00:00 2001 From: CGodiksen <36046286+CGodiksen@users.noreply.github.com> Date: Tue, 24 Mar 2026 10:28:59 +0100 Subject: [PATCH 111/116] Remove DeltaTable from create_table_log arguments --- crates/modelardb_server/src/context.rs | 2 +- .../modelardb_storage/src/write_ahead_log.rs | 44 +++++++++---------- 2 files changed, 22 insertions(+), 24 deletions(-) diff --git a/crates/modelardb_server/src/context.rs b/crates/modelardb_server/src/context.rs index 50ddafdf..596c323a 100644 --- a/crates/modelardb_server/src/context.rs +++ b/crates/modelardb_server/src/context.rs @@ -125,7 +125,7 @@ impl Context { // Create a file in the write-ahead log to log uncompressed data for the table. let mut write_ahead_log = self.write_ahead_log.write().await; write_ahead_log - .create_table_log(time_series_table_metadata, None) + .create_table_log(time_series_table_metadata) .await?; Ok(()) diff --git a/crates/modelardb_storage/src/write_ahead_log.rs b/crates/modelardb_storage/src/write_ahead_log.rs index e6c87857..a5647e2d 100644 --- a/crates/modelardb_storage/src/write_ahead_log.rs +++ b/crates/modelardb_storage/src/write_ahead_log.rs @@ -46,14 +46,15 @@ pub struct WriteAheadLog { } impl WriteAheadLog { - /// Create a new [`WriteAheadLog`] that stores the log in the root of `local_data_folder` in + /// Create a new [`WriteAheadLog`] that stores the WAL in the root of `local_data_folder` in /// the [`WRITE_AHEAD_LOG_FOLDER`] folder. If the folder does not exist, it is created. If the - /// log could not be created, return [`ModelarDbStorageError`]. + /// WAL could not be created, return [`ModelarDbStorageError`]. pub async fn try_new(local_data_folder: &DataFolder) -> Result { // Create the folder for the write-ahead log if it does not exist. let location = local_data_folder.location(); - // Since the std:fs API is used, the location must be a local path. + // Since the std::fs API is used, the location must be a local path. We use std::fs to avoid + // the overhead of the ObjectStore API and to allow the use of File::sync_data(). if location.contains("://") { return Err(ModelarDbStorageError::InvalidState(format!( "Write-ahead log location '{location}' is not a local path." @@ -69,11 +70,16 @@ impl WriteAheadLog { table_logs: HashMap::new(), }; - // For each time series table, create a log file if it does not already exist. + // For each time series table, create a log folder if it does not already exist. for metadata in local_data_folder.time_series_table_metadata().await? { let delta_table = local_data_folder.delta_table(&metadata.name).await?; - write_ahead_log - .create_table_log(&metadata, Some(delta_table)) + write_ahead_log.create_table_log(&metadata).await?; + + // Load the persisted batch ids from the commit history of the delta table. This is + // only necessary when initializing the WAL for an existing table. + let table_log = write_ahead_log.table_log(&metadata.name)?; + table_log + .load_persisted_batches_from_delta_table(delta_table) .await?; } @@ -86,15 +92,13 @@ impl WriteAheadLog { Ok(write_ahead_log) } - /// Create a new write-ahead log file for the table with the given metadata. If a delta table - /// is provided, the log file will be initialized with the persisted batch ids from the commit - /// history of the delta table. If a log already exists in the map or the log file could not be - /// created, return [`ModelarDbStorageError`]. Note that if the log file already exists, but it - /// is not present in the map, the existing log file will be added to the map. + /// Create a new write-ahead log file for the table with the given metadata. If a log already + /// exists in the map or the log file could not be created, return [`ModelarDbStorageError`]. + /// Note that if the log file already exists, but it is not present in the map, the existing log + /// file will be added to the map. pub async fn create_table_log( &mut self, time_series_table_metadata: &TimeSeriesTableMetadata, - maybe_delta_table: Option, ) -> Result<()> { let table_name = time_series_table_metadata.name.clone(); @@ -103,12 +107,6 @@ impl WriteAheadLog { let log_file = WriteAheadLogFile::try_new(table_log_path, &time_series_table_metadata.schema)?; - if let Some(delta_table) = maybe_delta_table { - log_file - .load_persisted_batches_from_delta_table(delta_table) - .await?; - } - debug!( table = %table_name, folder_path = %log_file.folder_path.display(), @@ -676,7 +674,7 @@ mod tests { let (_temp_dir, mut wal) = new_empty_write_ahead_log().await; let metadata = table::time_series_table_metadata(); - wal.create_table_log(&metadata, None).await.unwrap(); + wal.create_table_log(&metadata).await.unwrap(); assert!(wal.table_logs.contains_key(TIME_SERIES_TABLE_NAME)); } @@ -686,8 +684,8 @@ mod tests { let (_temp_dir, mut wal) = new_empty_write_ahead_log().await; let metadata = table::time_series_table_metadata(); - wal.create_table_log(&metadata, None).await.unwrap(); - let result = wal.create_table_log(&metadata, None).await; + wal.create_table_log(&metadata).await.unwrap(); + let result = wal.create_table_log(&metadata).await; assert_eq!( result.err().unwrap().to_string(), @@ -718,7 +716,7 @@ mod tests { let metadata = table::time_series_table_metadata(); let batch = table::uncompressed_time_series_table_record_batch(5); - wal.create_table_log(&metadata, None).await.unwrap(); + wal.create_table_log(&metadata).await.unwrap(); wal.append_to_table_log(TIME_SERIES_TABLE_NAME, &batch) .unwrap(); @@ -726,7 +724,7 @@ mod tests { .unwrap(); wal.remove_table_log(TIME_SERIES_TABLE_NAME).unwrap(); - wal.create_table_log(&metadata, None).await.unwrap(); + wal.create_table_log(&metadata).await.unwrap(); assert_eq!( wal.append_to_table_log(TIME_SERIES_TABLE_NAME, &batch) From aa7b209ad3e023281c278fc9320ce254df73183a Mon Sep 17 00:00:00 2001 From: CGodiksen <36046286+CGodiksen@users.noreply.github.com> Date: Tue, 24 Mar 2026 10:59:15 +0100 Subject: [PATCH 112/116] Rename WriteAheadLogFile to SegmentedLog --- .../modelardb_storage/src/write_ahead_log.rs | 309 +++++++++--------- 1 file changed, 160 insertions(+), 149 deletions(-) diff --git a/crates/modelardb_storage/src/write_ahead_log.rs b/crates/modelardb_storage/src/write_ahead_log.rs index a5647e2d..edd120b3 100644 --- a/crates/modelardb_storage/src/write_ahead_log.rs +++ b/crates/modelardb_storage/src/write_ahead_log.rs @@ -41,8 +41,8 @@ const SEGMENT_ROTATION_THRESHOLD: u64 = 100; pub struct WriteAheadLog { /// Path to the folder that contains the write-ahead log. folder_path: PathBuf, - /// Logs for each table. The key is the table name, and the value is the log file for that table. - table_logs: HashMap, + /// Logs for each table. The key is the table name, and the value is the table log for that table. + table_logs: HashMap, } impl WriteAheadLog { @@ -70,7 +70,7 @@ impl WriteAheadLog { table_logs: HashMap::new(), }; - // For each time series table, create a log folder if it does not already exist. + // For each time series table, create a table log if it does not already exist. for metadata in local_data_folder.time_series_table_metadata().await? { let delta_table = local_data_folder.delta_table(&metadata.name).await?; write_ahead_log.create_table_log(&metadata).await?; @@ -92,10 +92,10 @@ impl WriteAheadLog { Ok(write_ahead_log) } - /// Create a new write-ahead log file for the table with the given metadata. If a log already - /// exists in the map or the log file could not be created, return [`ModelarDbStorageError`]. - /// Note that if the log file already exists, but it is not present in the map, the existing log - /// file will be added to the map. + /// Create a new segmented log for the table with the given metadata. If a table log already + /// exists in the map or the table log could not be created, return [`ModelarDbStorageError`]. + /// Note that if the table log already exists, but it is not present in the map, the existing + /// table log will be added to the map. pub async fn create_table_log( &mut self, time_series_table_metadata: &TimeSeriesTableMetadata, @@ -104,16 +104,16 @@ impl WriteAheadLog { if !self.table_logs.contains_key(&table_name) { let table_log_path = self.folder_path.join(&table_name); - let log_file = - WriteAheadLogFile::try_new(table_log_path, &time_series_table_metadata.schema)?; + let table_log = + SegmentedLog::try_new(table_log_path, &time_series_table_metadata.schema)?; debug!( table = %table_name, - folder_path = %log_file.folder_path.display(), + folder_path = %table_log.folder_path.display(), "WAL table log created." ); - self.table_logs.insert(table_name, log_file); + self.table_logs.insert(table_name, table_log); Ok(()) } else { @@ -123,14 +123,14 @@ impl WriteAheadLog { } } - /// Remove the log files for the table with the given name. If the log files do not exist or + /// Remove the table log for the table with the given name. If the table log does not exist or /// could not be removed, return [`ModelarDbStorageError`]. pub fn remove_table_log(&mut self, table_name: &str) -> Result<()> { let log_path; - if let Some(log_file) = self.table_logs.remove(table_name) { - log_path = log_file.folder_path; - // log_file is dropped here as it goes out of scope which automatically closes its + if let Some(table_log) = self.table_logs.remove(table_name) { + log_path = table_log.folder_path; + // table_log is dropped here as it goes out of scope which automatically closes its // internal file handle. } else { return Err(ModelarDbStorageError::InvalidState(format!( @@ -150,13 +150,13 @@ impl WriteAheadLog { Ok(()) } - /// Append data to the log for the given table and sync the file to ensure that all data is on - /// disk. Only requires read access to the log since the internal Mutex handles write - /// synchronization. Return the batch id given to the appended data. If a table log does not - /// exist or the data could not be appended, return [`ModelarDbStorageError`]. + /// Append data to the table log for the given table and sync the file to ensure that all data + /// is on disk. Only requires read access to the write-ahead log since the internal Mutex + /// handles write synchronization. Return the batch id given to the appended data. If a table + /// log does not exist or the data could not be appended, return [`ModelarDbStorageError`]. pub fn append_to_table_log(&self, table_name: &str, data: &RecordBatch) -> Result { - let log_file = self.table_log(table_name)?; - log_file.append_and_sync(data) + let table_log = self.table_log(table_name)?; + table_log.append_and_sync(data) } /// Mark the given batch ids as saved to disk in the corresponding table log. Fully persisted @@ -167,24 +167,24 @@ impl WriteAheadLog { table_name: &str, batch_ids: HashSet, ) -> Result<()> { - let log_file = self.table_log(table_name)?; - log_file.mark_batches_as_persisted(batch_ids) + let table_log = self.table_log(table_name)?; + table_log.mark_batches_as_persisted(batch_ids) } /// Return pairs of (batch_id, batch) for all batches that have not yet been persisted in the - /// corresponding table log. If the log file does not exist or the batches could not be read - /// from the WAL files, return [`ModelarDbStorageError`]. + /// corresponding table log. If the table log does not exist or the batches could not be read + /// from the table log, return [`ModelarDbStorageError`]. pub fn unpersisted_batches_in_table_log( &self, table_name: &str, ) -> Result> { - let log_file = self.table_log(table_name)?; - log_file.unpersisted_batches() + let table_log = self.table_log(table_name)?; + table_log.unpersisted_batches() } - /// Get the log file for the table with the given name. If the log file does not exist, return + /// Get the table log for the table with the given name. If the table log does not exist, return /// [`ModelarDbStorageError`]. - fn table_log(&self, table_name: &str) -> Result<&WriteAheadLogFile> { + fn table_log(&self, table_name: &str) -> Result<&SegmentedLog> { self.table_logs.get(table_name).ok_or_else(|| { ModelarDbStorageError::InvalidState(format!( "Table log for table '{table_name}' does not exist." @@ -212,7 +212,7 @@ impl ClosedSegment { } /// The currently active WAL segment being written to. All fields are mutated together -/// during rotation and are protected by the mutex in [`WriteAheadLogFile`]. +/// during rotation and are protected by the mutex in [`SegmentedLog`]. struct ActiveSegment { /// Path to the active segment file. path: PathBuf, @@ -252,13 +252,14 @@ impl ActiveSegment { } } -/// Wrapper around a [`File`] that enforces that [`sync_data()`](File::sync_data) is called -/// immediately after writing to ensure that all data is on disk before returning. Note that -/// an exclusive lock is held on the file while it is being written to. At any point in time there -/// is exactly one active segment being written to plus zero or more closed segments that are -/// read-only. The active segment is rotated into the closed list once [`SEGMENT_ROTATION_THRESHOLD`] -/// batches have been written to it. -struct WriteAheadLogFile { +/// Segmented log that appends data in Arrow IPC streaming format to segment files in a folder. +/// At any point in time there is exactly one active segment being written to plus zero or more +/// closed segments that are read-only. The active segment is rotated into the closed list once +/// [`SEGMENT_ROTATION_THRESHOLD`] batches have been written to it. Appending enforces that +/// [`sync_data()`](File::sync_data) is called immediately after writing to ensure that all data is +/// on disk before returning. Note that an exclusive lock is held on the file while it is being +/// written to. +struct SegmentedLog { /// Folder that contains all segment files for this log. folder_path: PathBuf, /// Arrow schema shared by every segment in this log. @@ -272,8 +273,8 @@ struct WriteAheadLogFile { persisted_batch_ids: Mutex>, } -impl WriteAheadLogFile { - /// Create a new [`WriteAheadLogFile`] that appends data with `schema` to segment files in +impl SegmentedLog { + /// Create a new [`SegmentedLog`] that appends data with `schema` to segment files in /// `folder_path`. Existing closed segment files are loaded into the closed-segment list. /// A fresh active segment is always created on start-up. If the folder or file could not be /// created, return [`ModelarDbStorageError`]. @@ -466,7 +467,7 @@ impl WriteAheadLogFile { /// Return pairs of (batch_id, batch) for all batches in the log that have not yet been /// persisted according to the current in-memory `persisted_batch_ids` set. If the batches - /// could not be read from the WAL files, return [`ModelarDbStorageError`]. + /// could not be read from the segment files, return [`ModelarDbStorageError`]. fn unpersisted_batches(&self) -> Result> { let persisted = self .persisted_batch_ids @@ -862,87 +863,87 @@ mod tests { (temp_dir, wal) } - // Tests for WriteAheadLogFile. + // Tests for SegmentedLog. #[test] fn test_try_new_creates_active_segment() { let temp_dir = tempfile::tempdir().unwrap(); - let (_folder_path, wal_file) = new_wal_file(&temp_dir); + let (_folder_path, segmented_log) = new_segmented_log(&temp_dir); - let active = wal_file.active_segment.lock().unwrap(); + let active = segmented_log.active_segment.lock().unwrap(); assert!(active.path.exists()); assert_eq!(active.next_batch_id, 0); - assert!(wal_file.closed_segments.lock().unwrap().is_empty()); + assert!(segmented_log.closed_segments.lock().unwrap().is_empty()); } #[test] fn test_read_all_empty_file() { let temp_dir = tempfile::tempdir().unwrap(); - let (_folder_path, wal_file) = new_wal_file(&temp_dir); + let (_folder_path, segmented_log) = new_segmented_log(&temp_dir); - let batches = wal_file.read_all().unwrap(); + let batches = segmented_log.read_all().unwrap(); assert!(batches.is_empty()); - let active = wal_file.active_segment.lock().unwrap(); + let active = segmented_log.active_segment.lock().unwrap(); assert_eq!(active.next_batch_id, 0); } #[test] fn test_append_and_read_single_batch() { let temp_dir = tempfile::tempdir().unwrap(); - let (_folder_path, wal_file) = new_wal_file(&temp_dir); + let (_folder_path, segmented_log) = new_segmented_log(&temp_dir); let batch = table::uncompressed_time_series_table_record_batch(5); - wal_file.append_and_sync(&batch).unwrap(); + segmented_log.append_and_sync(&batch).unwrap(); - let batches = wal_file.read_all().unwrap(); + let batches = segmented_log.read_all().unwrap(); assert_eq!(batches.len(), 1); assert_eq!(batches[0], (0, batch)); - let active = wal_file.active_segment.lock().unwrap(); + let active = segmented_log.active_segment.lock().unwrap(); assert_eq!(active.next_batch_id, 1); } #[test] fn test_append_and_read_multiple_batches() { let temp_dir = tempfile::tempdir().unwrap(); - let (_folder_path, wal_file) = new_wal_file(&temp_dir); + let (_folder_path, segmented_log) = new_segmented_log(&temp_dir); let batch_1 = table::uncompressed_time_series_table_record_batch(10); let batch_2 = table::uncompressed_time_series_table_record_batch(20); let batch_3 = table::uncompressed_time_series_table_record_batch(30); - wal_file.append_and_sync(&batch_1).unwrap(); - wal_file.append_and_sync(&batch_2).unwrap(); - wal_file.append_and_sync(&batch_3).unwrap(); + segmented_log.append_and_sync(&batch_1).unwrap(); + segmented_log.append_and_sync(&batch_2).unwrap(); + segmented_log.append_and_sync(&batch_3).unwrap(); - let batches = wal_file.read_all().unwrap(); + let batches = segmented_log.read_all().unwrap(); assert_eq!(batches.len(), 3); assert_eq!(batches[0], (0, batch_1)); assert_eq!(batches[1], (1, batch_2)); assert_eq!(batches[2], (2, batch_3)); - let active = wal_file.active_segment.lock().unwrap(); + let active = segmented_log.active_segment.lock().unwrap(); assert_eq!(active.next_batch_id, 3); } #[test] fn test_segment_rotates_at_threshold() { let temp_dir = tempfile::tempdir().unwrap(); - let (_folder_path, wal_file) = new_wal_file(&temp_dir); + let (_folder_path, segmented_log) = new_segmented_log(&temp_dir); let batch = table::uncompressed_time_series_table_record_batch(5); for _ in 0..SEGMENT_ROTATION_THRESHOLD { - wal_file.append_and_sync(&batch).unwrap(); + segmented_log.append_and_sync(&batch).unwrap(); } - let closed = wal_file.closed_segments.lock().unwrap(); + let closed = segmented_log.closed_segments.lock().unwrap(); assert_eq!(closed.len(), 1); assert_eq!(closed[0].start_id, 0); assert_eq!(closed[0].end_id, SEGMENT_ROTATION_THRESHOLD - 1); - let active = wal_file.active_segment.lock().unwrap(); + let active = segmented_log.active_segment.lock().unwrap(); assert_eq!(active.start_id, SEGMENT_ROTATION_THRESHOLD); } @@ -956,19 +957,19 @@ mod tests { // Write enough batches to trigger a rotation, then drop. { - let wal_file = - WriteAheadLogFile::try_new(folder_path.clone(), &metadata.schema).unwrap(); + let segmented_log = + SegmentedLog::try_new(folder_path.clone(), &metadata.schema).unwrap(); for _ in 0..SEGMENT_ROTATION_THRESHOLD { - wal_file.append_and_sync(&batch).unwrap(); + segmented_log.append_and_sync(&batch).unwrap(); } } // The closed segment should be detected and the next id should continue. - let wal_file = WriteAheadLogFile::try_new(folder_path.clone(), &metadata.schema).unwrap(); + let segmented_log = SegmentedLog::try_new(folder_path.clone(), &metadata.schema).unwrap(); - let active = wal_file.active_segment.lock().unwrap(); + let active = segmented_log.active_segment.lock().unwrap(); assert_eq!(active.next_batch_id, SEGMENT_ROTATION_THRESHOLD); - assert_eq!(wal_file.closed_segments.lock().unwrap().len(), 1); + assert_eq!(segmented_log.closed_segments.lock().unwrap().len(), 1); } #[test] @@ -981,20 +982,20 @@ mod tests { // Fill and rotate one segment. { - let wal_file = - WriteAheadLogFile::try_new(folder_path.clone(), &metadata.schema).unwrap(); + let segmented_log = + SegmentedLog::try_new(folder_path.clone(), &metadata.schema).unwrap(); for _ in 0..SEGMENT_ROTATION_THRESHOLD { - wal_file.append_and_sync(&batch).unwrap(); + segmented_log.append_and_sync(&batch).unwrap(); } } - let wal_file = WriteAheadLogFile::try_new(folder_path.clone(), &metadata.schema).unwrap(); - wal_file.append_and_sync(&batch).unwrap(); + let segmented_log = SegmentedLog::try_new(folder_path.clone(), &metadata.schema).unwrap(); + segmented_log.append_and_sync(&batch).unwrap(); - let batches = wal_file.read_all().unwrap(); + let batches = segmented_log.read_all().unwrap(); assert_eq!(batches.len() as u64, SEGMENT_ROTATION_THRESHOLD + 1); - let active = wal_file.active_segment.lock().unwrap(); + let active = segmented_log.active_segment.lock().unwrap(); assert_eq!(active.next_batch_id, SEGMENT_ROTATION_THRESHOLD + 1); } @@ -1008,26 +1009,26 @@ mod tests { // Write enough batches to trigger a rotation and append to a new active segment. { - let wal_file = - WriteAheadLogFile::try_new(folder_path.clone(), &metadata.schema).unwrap(); + let segmented_log = + SegmentedLog::try_new(folder_path.clone(), &metadata.schema).unwrap(); for _ in 0..SEGMENT_ROTATION_THRESHOLD + 2 { - wal_file.append_and_sync(&batch).unwrap(); + segmented_log.append_and_sync(&batch).unwrap(); } } // On re-open the leftover active segment should be closed, leaving two closed segments // and a fresh active segment starting after them. - let wal_file = WriteAheadLogFile::try_new(folder_path.clone(), &metadata.schema).unwrap(); + let segmented_log = SegmentedLog::try_new(folder_path.clone(), &metadata.schema).unwrap(); - let closed = wal_file.closed_segments.lock().unwrap(); + let closed = segmented_log.closed_segments.lock().unwrap(); assert_eq!(closed.len(), 2); assert_eq!(closed[0].start_id, 0); assert_eq!(closed[0].end_id, SEGMENT_ROTATION_THRESHOLD - 1); assert_eq!(closed[1].start_id, SEGMENT_ROTATION_THRESHOLD); assert_eq!(closed[1].end_id, SEGMENT_ROTATION_THRESHOLD + 1); - let active = wal_file.active_segment.lock().unwrap(); + let active = segmented_log.active_segment.lock().unwrap(); assert_eq!(active.next_batch_id, SEGMENT_ROTATION_THRESHOLD + 2); } @@ -1037,17 +1038,17 @@ mod tests { let folder_path = temp_dir.path().join(TIME_SERIES_TABLE_NAME); let metadata = table::time_series_table_metadata(); - // Create a WAL file and immediately drop it without writing anything. + // Create a segmented log and immediately drop it without writing anything. // This leaves an empty "{start_id}-.wal" active segment. { - WriteAheadLogFile::try_new(folder_path.clone(), &metadata.schema).unwrap(); + SegmentedLog::try_new(folder_path.clone(), &metadata.schema).unwrap(); } // On re-open, the empty leftover active segment should be removed. - let wal_file = WriteAheadLogFile::try_new(folder_path.clone(), &metadata.schema).unwrap(); + let segmented_log = SegmentedLog::try_new(folder_path.clone(), &metadata.schema).unwrap(); - assert!(wal_file.closed_segments.lock().unwrap().is_empty()); - let active = wal_file.active_segment.lock().unwrap(); + assert!(segmented_log.closed_segments.lock().unwrap().is_empty()); + let active = segmented_log.active_segment.lock().unwrap(); assert_eq!(active.next_batch_id, 0); assert!(active.path.exists()); @@ -1059,179 +1060,189 @@ mod tests { #[test] fn test_mark_batches_as_persisted_deletes_fully_persisted_segment() { let temp_dir = tempfile::tempdir().unwrap(); - let (_folder_path, wal_file) = new_wal_file(&temp_dir); + let (_folder_path, segmented_log) = new_segmented_log(&temp_dir); let batch = table::uncompressed_time_series_table_record_batch(5); // Fill and rotate one full segment. for _ in 0..SEGMENT_ROTATION_THRESHOLD { - wal_file.append_and_sync(&batch).unwrap(); + segmented_log.append_and_sync(&batch).unwrap(); } - let segment_path = wal_file.closed_segments.lock().unwrap()[0].path.clone(); + let segment_path = segmented_log.closed_segments.lock().unwrap()[0] + .path + .clone(); assert!(segment_path.exists()); let ids: HashSet = (0..SEGMENT_ROTATION_THRESHOLD).collect(); - wal_file.mark_batches_as_persisted(ids).unwrap(); + segmented_log.mark_batches_as_persisted(ids).unwrap(); assert!(!segment_path.exists()); - assert!(wal_file.closed_segments.lock().unwrap().is_empty()); + assert!(segmented_log.closed_segments.lock().unwrap().is_empty()); } #[test] fn test_mark_batches_as_persisted_retains_partially_persisted_segment() { let temp_dir = tempfile::tempdir().unwrap(); - let (_folder_path, wal_file) = new_wal_file(&temp_dir); + let (_folder_path, segmented_log) = new_segmented_log(&temp_dir); let batch = table::uncompressed_time_series_table_record_batch(5); for _ in 0..SEGMENT_ROTATION_THRESHOLD { - wal_file.append_and_sync(&batch).unwrap(); + segmented_log.append_and_sync(&batch).unwrap(); } - let segment_path = wal_file.closed_segments.lock().unwrap()[0].path.clone(); + let segment_path = segmented_log.closed_segments.lock().unwrap()[0] + .path + .clone(); // Only persist a subset of the batch ids in the closed segment. let partial_ids: HashSet = (0..SEGMENT_ROTATION_THRESHOLD - 1).collect(); - wal_file.mark_batches_as_persisted(partial_ids).unwrap(); + segmented_log + .mark_batches_as_persisted(partial_ids) + .unwrap(); // Segment should still exist since not all ids are persisted. assert!(segment_path.exists()); - assert_eq!(wal_file.closed_segments.lock().unwrap().len(), 1); + assert_eq!(segmented_log.closed_segments.lock().unwrap().len(), 1); // When persisting the last batch, the segment should be deleted. - wal_file + segmented_log .mark_batches_as_persisted(HashSet::from([SEGMENT_ROTATION_THRESHOLD - 1])) .unwrap(); assert!(!segment_path.exists()); - assert!(wal_file.closed_segments.lock().unwrap().is_empty()); + assert!(segmented_log.closed_segments.lock().unwrap().is_empty()); } #[test] fn test_multiple_fully_persisted_segments_all_deleted() { let temp_dir = tempfile::tempdir().unwrap(); - let (_folder_path, wal_file) = new_wal_file(&temp_dir); + let (_folder_path, segmented_log) = new_segmented_log(&temp_dir); let batch = table::uncompressed_time_series_table_record_batch(5); // Trigger five full rotations. for _ in 0..SEGMENT_ROTATION_THRESHOLD * 5 { - wal_file.append_and_sync(&batch).unwrap(); + segmented_log.append_and_sync(&batch).unwrap(); } - assert_eq!(wal_file.closed_segments.lock().unwrap().len(), 5); + assert_eq!(segmented_log.closed_segments.lock().unwrap().len(), 5); let ids: HashSet = (0..SEGMENT_ROTATION_THRESHOLD * 5).collect(); - wal_file.mark_batches_as_persisted(ids).unwrap(); + segmented_log.mark_batches_as_persisted(ids).unwrap(); - assert!(wal_file.closed_segments.lock().unwrap().is_empty()); - assert!(wal_file.persisted_batch_ids.lock().unwrap().is_empty()); + assert!(segmented_log.closed_segments.lock().unwrap().is_empty()); + assert!(segmented_log.persisted_batch_ids.lock().unwrap().is_empty()); } #[tokio::test] async fn test_no_batch_ids_in_history_leaves_persisted_set_empty() { let (temp_dir, data_folder) = create_data_folder_with_time_series_table().await; - let (_wal_dir, wal_file) = new_wal_file(&temp_dir); + let (_wal_dir, segmented_log) = new_segmented_log(&temp_dir); let delta_table = data_folder .delta_table(TIME_SERIES_TABLE_NAME) .await .unwrap(); - wal_file + segmented_log .load_persisted_batches_from_delta_table(delta_table) .await .unwrap(); - assert!(wal_file.persisted_batch_ids.lock().unwrap().is_empty()); + assert!(segmented_log.persisted_batch_ids.lock().unwrap().is_empty()); } #[tokio::test] async fn test_load_persisted_batches_loads_single_commit() { let (temp_dir, data_folder) = create_data_folder_with_time_series_table().await; - let (_wal_dir, wal_file) = new_wal_file(&temp_dir); + let (_wal_dir, segmented_log) = new_segmented_log(&temp_dir); let delta_table = write_compressed_segments_with_batch_ids(&data_folder, HashSet::from([0, 1, 2])).await; - wal_file + segmented_log .load_persisted_batches_from_delta_table(delta_table) .await .unwrap(); - let persisted = wal_file.persisted_batch_ids.lock().unwrap(); + let persisted = segmented_log.persisted_batch_ids.lock().unwrap(); assert_eq!(*persisted, BTreeSet::from([0, 1, 2])); } #[tokio::test] async fn test_load_persisted_batches_loads_multiple_commits() { let (temp_dir, data_folder) = create_data_folder_with_time_series_table().await; - let (_wal_dir, wal_file) = new_wal_file(&temp_dir); + let (_wal_dir, segmented_log) = new_segmented_log(&temp_dir); write_compressed_segments_with_batch_ids(&data_folder, HashSet::from([0, 1, 2])).await; let delta_table = write_compressed_segments_with_batch_ids(&data_folder, HashSet::from([2, 3, 4])).await; - wal_file + segmented_log .load_persisted_batches_from_delta_table(delta_table) .await .unwrap(); - let persisted = wal_file.persisted_batch_ids.lock().unwrap(); + let persisted = segmented_log.persisted_batch_ids.lock().unwrap(); assert_eq!(*persisted, BTreeSet::from([0, 1, 2, 3, 4])); } #[tokio::test] async fn test_load_persisted_batches_deletes_fully_persisted_closed_segment() { let (temp_dir, data_folder) = create_data_folder_with_time_series_table().await; - let (_wal_dir, wal_file) = new_wal_file(&temp_dir); + let (_wal_dir, segmented_log) = new_segmented_log(&temp_dir); let batch = table::uncompressed_time_series_table_record_batch(5); for _ in 0..SEGMENT_ROTATION_THRESHOLD { - wal_file.append_and_sync(&batch).unwrap(); + segmented_log.append_and_sync(&batch).unwrap(); } - let segment_path = wal_file.closed_segments.lock().unwrap()[0].path.clone(); + let segment_path = segmented_log.closed_segments.lock().unwrap()[0] + .path + .clone(); assert!(segment_path.exists()); let all_ids: HashSet = (0..SEGMENT_ROTATION_THRESHOLD).collect(); let delta_table = write_compressed_segments_with_batch_ids(&data_folder, all_ids).await; - wal_file + segmented_log .load_persisted_batches_from_delta_table(delta_table) .await .unwrap(); assert!(!segment_path.exists()); - assert!(wal_file.closed_segments.lock().unwrap().is_empty()); - assert!(wal_file.persisted_batch_ids.lock().unwrap().is_empty()); + assert!(segmented_log.closed_segments.lock().unwrap().is_empty()); + assert!(segmented_log.persisted_batch_ids.lock().unwrap().is_empty()); } #[tokio::test] async fn test_load_persisted_batches_retains_partially_persisted_closed_segment() { let (temp_dir, data_folder) = create_data_folder_with_time_series_table().await; - let (_wal_dir, wal_file) = new_wal_file(&temp_dir); + let (_wal_dir, segmented_log) = new_segmented_log(&temp_dir); let batch = table::uncompressed_time_series_table_record_batch(5); for _ in 0..SEGMENT_ROTATION_THRESHOLD { - wal_file.append_and_sync(&batch).unwrap(); + segmented_log.append_and_sync(&batch).unwrap(); } - let segment_path = wal_file.closed_segments.lock().unwrap()[0].path.clone(); + let segment_path = segmented_log.closed_segments.lock().unwrap()[0] + .path + .clone(); let partial_ids: HashSet = (0..SEGMENT_ROTATION_THRESHOLD - 1).collect(); let delta_table = write_compressed_segments_with_batch_ids(&data_folder, partial_ids).await; - wal_file + segmented_log .load_persisted_batches_from_delta_table(delta_table) .await .unwrap(); assert!(segment_path.exists()); - assert_eq!(wal_file.closed_segments.lock().unwrap().len(), 1); + assert_eq!(segmented_log.closed_segments.lock().unwrap().len(), 1); assert_eq!( - wal_file.persisted_batch_ids.lock().unwrap().len() as u64, + segmented_log.persisted_batch_ids.lock().unwrap().len() as u64, SEGMENT_ROTATION_THRESHOLD - 1 ); } @@ -1273,14 +1284,14 @@ mod tests { #[test] fn test_unpersisted_batches_returns_all_when_none_persisted() { let temp_dir = tempfile::tempdir().unwrap(); - let (_folder_path, wal_file) = new_wal_file(&temp_dir); + let (_folder_path, segmented_log) = new_segmented_log(&temp_dir); let batch_1 = table::uncompressed_time_series_table_record_batch(10); let batch_2 = table::uncompressed_time_series_table_record_batch(20); - wal_file.append_and_sync(&batch_1).unwrap(); - wal_file.append_and_sync(&batch_2).unwrap(); + segmented_log.append_and_sync(&batch_1).unwrap(); + segmented_log.append_and_sync(&batch_2).unwrap(); - let unpersisted = wal_file.unpersisted_batches().unwrap(); + let unpersisted = segmented_log.unpersisted_batches().unwrap(); assert_eq!(unpersisted.len(), 2); assert_eq!(unpersisted[0], (0, batch_1)); assert_eq!(unpersisted[1], (1, batch_2)); @@ -1289,36 +1300,36 @@ mod tests { #[test] fn test_unpersisted_batches_returns_empty_when_all_persisted() { let temp_dir = tempfile::tempdir().unwrap(); - let (_folder_path, wal_file) = new_wal_file(&temp_dir); + let (_folder_path, segmented_log) = new_segmented_log(&temp_dir); let batch = table::uncompressed_time_series_table_record_batch(10); - wal_file.append_and_sync(&batch).unwrap(); - wal_file.append_and_sync(&batch).unwrap(); + segmented_log.append_and_sync(&batch).unwrap(); + segmented_log.append_and_sync(&batch).unwrap(); - wal_file + segmented_log .mark_batches_as_persisted(HashSet::from([0, 1])) .unwrap(); - let unpersisted = wal_file.unpersisted_batches().unwrap(); + let unpersisted = segmented_log.unpersisted_batches().unwrap(); assert!(unpersisted.is_empty()); } #[test] fn test_unpersisted_batches_filters_persisted_batch_ids() { let temp_dir = tempfile::tempdir().unwrap(); - let (_folder_path, wal_file) = new_wal_file(&temp_dir); + let (_folder_path, segmented_log) = new_segmented_log(&temp_dir); let batch_1 = table::uncompressed_time_series_table_record_batch(10); let batch_2 = table::uncompressed_time_series_table_record_batch(20); - wal_file.append_and_sync(&batch_1).unwrap(); - wal_file.append_and_sync(&batch_1).unwrap(); - wal_file.append_and_sync(&batch_2).unwrap(); + segmented_log.append_and_sync(&batch_1).unwrap(); + segmented_log.append_and_sync(&batch_1).unwrap(); + segmented_log.append_and_sync(&batch_2).unwrap(); - wal_file + segmented_log .mark_batches_as_persisted(HashSet::from([1])) .unwrap(); - let unpersisted = wal_file.unpersisted_batches().unwrap(); + let unpersisted = segmented_log.unpersisted_batches().unwrap(); assert_eq!(unpersisted.len(), 2); assert_eq!(unpersisted[0], (0, batch_1)); assert_eq!(unpersisted[1], (2, batch_2)); @@ -1327,23 +1338,23 @@ mod tests { #[test] fn test_unpersisted_batches_returns_batches_across_closed_and_active_segments() { let temp_dir = tempfile::tempdir().unwrap(); - let (_folder_path, wal_file) = new_wal_file(&temp_dir); + let (_folder_path, segmented_log) = new_segmented_log(&temp_dir); let batch = table::uncompressed_time_series_table_record_batch(5); // Fill one full segment (triggers rotation) and write two more into the active segment. for _ in 0..SEGMENT_ROTATION_THRESHOLD + 2 { - wal_file.append_and_sync(&batch).unwrap(); + segmented_log.append_and_sync(&batch).unwrap(); } // Persist one batch id in the closed segment and one in the active segment. - wal_file + segmented_log .mark_batches_as_persisted(HashSet::from([0, SEGMENT_ROTATION_THRESHOLD + 1])) .unwrap(); - assert_eq!(wal_file.closed_segments.lock().unwrap().len(), 1); + assert_eq!(segmented_log.closed_segments.lock().unwrap().len(), 1); - let unpersisted = wal_file.unpersisted_batches().unwrap(); + let unpersisted = segmented_log.unpersisted_batches().unwrap(); assert_eq!(unpersisted.len() as u64, SEGMENT_ROTATION_THRESHOLD); assert_eq!(unpersisted.first().unwrap(), &(1, batch.clone())); assert_eq!( @@ -1355,18 +1366,18 @@ mod tests { #[test] fn test_unpersisted_batches_returns_empty_when_no_batches_written() { let temp_dir = tempfile::tempdir().unwrap(); - let (_folder_path, wal_file) = new_wal_file(&temp_dir); + let (_folder_path, segmented_log) = new_segmented_log(&temp_dir); - let unpersisted = wal_file.unpersisted_batches().unwrap(); + let unpersisted = segmented_log.unpersisted_batches().unwrap(); assert!(unpersisted.is_empty()); } - fn new_wal_file(temp_dir: &TempDir) -> (PathBuf, WriteAheadLogFile) { + fn new_segmented_log(temp_dir: &TempDir) -> (PathBuf, SegmentedLog) { let folder_path = temp_dir.path().join(TIME_SERIES_TABLE_NAME); let metadata = table::time_series_table_metadata(); - let wal_file = WriteAheadLogFile::try_new(folder_path.clone(), &metadata.schema).unwrap(); + let segmented_log = SegmentedLog::try_new(folder_path.clone(), &metadata.schema).unwrap(); - (folder_path, wal_file) + (folder_path, segmented_log) } } From a65df7a60140e280ed5235e55b00e6eb83150576 Mon Sep 17 00:00:00 2001 From: CGodiksen <36046286+CGodiksen@users.noreply.github.com> Date: Tue, 24 Mar 2026 11:44:12 +0100 Subject: [PATCH 113/116] No longer use "rotation" terminology in WAL --- .../modelardb_storage/src/write_ahead_log.rs | 110 +++++++++--------- 1 file changed, 57 insertions(+), 53 deletions(-) diff --git a/crates/modelardb_storage/src/write_ahead_log.rs b/crates/modelardb_storage/src/write_ahead_log.rs index edd120b3..8cce9343 100644 --- a/crates/modelardb_storage/src/write_ahead_log.rs +++ b/crates/modelardb_storage/src/write_ahead_log.rs @@ -14,7 +14,11 @@ */ //! Implementation of types that provide a write-ahead log for ModelarDB that can be used to -//! efficiently persist data on disk to avoid data loss and enable crash recovery. +//! efficiently persist data on disk to avoid data loss and enable crash recovery. Each table has +//! its own segmented log consisting of an active segment that is appended to and zero or more +//! closed segments that are read-only. The active segment is closed once a configured number of +//! batches have been written to it, and closed segments are deleted once all of their batches have +//! been persisted to the Delta Lake. use std::collections::{BTreeSet, HashMap, HashSet}; use std::fs::{File, OpenOptions}; @@ -34,8 +38,8 @@ use crate::WRITE_AHEAD_LOG_FOLDER; use crate::data_folder::DataFolder; use crate::error::{ModelarDbStorageError, Result}; -/// Number of batches to write to a single WAL segment file before rotating to a new one. -const SEGMENT_ROTATION_THRESHOLD: u64 = 100; +/// Number of batches to write to a single WAL segment file before closing it and starting a new one. +const SEGMENT_BATCH_COUNT_THRESHOLD: u64 = 100; /// Write-ahead log that logs data on a per-table level. pub struct WriteAheadLog { @@ -212,7 +216,7 @@ impl ClosedSegment { } /// The currently active WAL segment being written to. All fields are mutated together -/// during rotation and are protected by the mutex in [`SegmentedLog`]. +/// when closing the active segment and are protected by the mutex in [`SegmentedLog`]. struct ActiveSegment { /// Path to the active segment file. path: PathBuf, @@ -220,7 +224,7 @@ struct ActiveSegment { start_id: u64, /// Writer to write data in IPC streaming format to this segment file. writer: StreamWriter, - /// The batch id to give to the next batch of data. Monotonically increasing across rotations. + /// The batch id to give to the next batch of data. Monotonically increasing across segments. next_batch_id: u64, } @@ -254,8 +258,8 @@ impl ActiveSegment { /// Segmented log that appends data in Arrow IPC streaming format to segment files in a folder. /// At any point in time there is exactly one active segment being written to plus zero or more -/// closed segments that are read-only. The active segment is rotated into the closed list once -/// [`SEGMENT_ROTATION_THRESHOLD`] batches have been written to it. Appending enforces that +/// closed segments that are read-only. The active segment is closed once +/// [`SEGMENT_BATCH_COUNT_THRESHOLD`] batches have been written to it. Appending enforces that /// [`sync_data()`](File::sync_data) is called immediately after writing to ensure that all data is /// on disk before returning. Note that an exclusive lock is held on the file while it is being /// written to. @@ -313,9 +317,9 @@ impl SegmentedLog { } /// Append the given data to the active segment and sync the file to ensure that all data is on - /// disk. Return the batch id given to the appended data. Rotates to a new segment file if - /// [`SEGMENT_ROTATION_THRESHOLD`] is reached. If the data could not be appended or the file - /// could not be synced, return [`ModelarDbStorageError`]. + /// disk. Return the batch id given to the appended data. Close the active segment and start a + /// new one if [`SEGMENT_BATCH_COUNT_THRESHOLD`] is reached. If the data could not be appended + /// or the file could not be synced, return [`ModelarDbStorageError`]. fn append_and_sync(&self, data: &RecordBatch) -> Result { // Acquire the mutex to ensure only one thread can write at a time. let mut active = self @@ -343,12 +347,12 @@ impl SegmentedLog { "Appended batch to WAL file." ); - // Rotate to a new segment if the threshold has been reached. The number of batches in the - // active segment is the difference between the next batch id (post-increment) and the - // active start id. + // Close the active segment and start a new one if the threshold has been reached. The + // number of batches in the active segment is the difference between the next batch id + // (post-increment) and the active start id. let active_batch_count = active.next_batch_id - active.start_id; - if active_batch_count >= SEGMENT_ROTATION_THRESHOLD { - self.rotate_active_segment(&mut active)?; + if active_batch_count >= SEGMENT_BATCH_COUNT_THRESHOLD { + self.close_active_segment(&mut active)?; } Ok(current_batch_id) @@ -356,7 +360,7 @@ impl SegmentedLog { /// Close the current active segment by renaming it to its final `{start_id}-{end_id}.wal` /// name and open a fresh active segment. The caller must hold the `active_segment` lock. - fn rotate_active_segment(&self, active: &mut ActiveSegment) -> Result<()> { + fn close_active_segment(&self, active: &mut ActiveSegment) -> Result<()> { let mut closed_segments = self .closed_segments .lock() @@ -368,7 +372,7 @@ impl SegmentedLog { path = %active.path.display(), start_id = active.start_id, end_id, - "Rotating WAL segment." + "Closing active WAL segment." ); // Finish the current writer so the IPC end-of-stream marker is written. @@ -928,23 +932,23 @@ mod tests { } #[test] - fn test_segment_rotates_at_threshold() { + fn test_segment_closes_at_threshold() { let temp_dir = tempfile::tempdir().unwrap(); let (_folder_path, segmented_log) = new_segmented_log(&temp_dir); let batch = table::uncompressed_time_series_table_record_batch(5); - for _ in 0..SEGMENT_ROTATION_THRESHOLD { + for _ in 0..SEGMENT_BATCH_COUNT_THRESHOLD { segmented_log.append_and_sync(&batch).unwrap(); } let closed = segmented_log.closed_segments.lock().unwrap(); assert_eq!(closed.len(), 1); assert_eq!(closed[0].start_id, 0); - assert_eq!(closed[0].end_id, SEGMENT_ROTATION_THRESHOLD - 1); + assert_eq!(closed[0].end_id, SEGMENT_BATCH_COUNT_THRESHOLD - 1); let active = segmented_log.active_segment.lock().unwrap(); - assert_eq!(active.start_id, SEGMENT_ROTATION_THRESHOLD); + assert_eq!(active.start_id, SEGMENT_BATCH_COUNT_THRESHOLD); } #[test] @@ -955,11 +959,11 @@ mod tests { let batch = table::uncompressed_time_series_table_record_batch(10); - // Write enough batches to trigger a rotation, then drop. + // Write enough batches to close the active segment, then drop. { let segmented_log = SegmentedLog::try_new(folder_path.clone(), &metadata.schema).unwrap(); - for _ in 0..SEGMENT_ROTATION_THRESHOLD { + for _ in 0..SEGMENT_BATCH_COUNT_THRESHOLD { segmented_log.append_and_sync(&batch).unwrap(); } } @@ -968,7 +972,7 @@ mod tests { let segmented_log = SegmentedLog::try_new(folder_path.clone(), &metadata.schema).unwrap(); let active = segmented_log.active_segment.lock().unwrap(); - assert_eq!(active.next_batch_id, SEGMENT_ROTATION_THRESHOLD); + assert_eq!(active.next_batch_id, SEGMENT_BATCH_COUNT_THRESHOLD); assert_eq!(segmented_log.closed_segments.lock().unwrap().len(), 1); } @@ -980,11 +984,11 @@ mod tests { let batch = table::uncompressed_time_series_table_record_batch(10); - // Fill and rotate one segment. + // Fill and close one full segment. { let segmented_log = SegmentedLog::try_new(folder_path.clone(), &metadata.schema).unwrap(); - for _ in 0..SEGMENT_ROTATION_THRESHOLD { + for _ in 0..SEGMENT_BATCH_COUNT_THRESHOLD { segmented_log.append_and_sync(&batch).unwrap(); } } @@ -993,10 +997,10 @@ mod tests { segmented_log.append_and_sync(&batch).unwrap(); let batches = segmented_log.read_all().unwrap(); - assert_eq!(batches.len() as u64, SEGMENT_ROTATION_THRESHOLD + 1); + assert_eq!(batches.len() as u64, SEGMENT_BATCH_COUNT_THRESHOLD + 1); let active = segmented_log.active_segment.lock().unwrap(); - assert_eq!(active.next_batch_id, SEGMENT_ROTATION_THRESHOLD + 1); + assert_eq!(active.next_batch_id, SEGMENT_BATCH_COUNT_THRESHOLD + 1); } #[test] @@ -1007,12 +1011,12 @@ mod tests { let batch = table::uncompressed_time_series_table_record_batch(5); - // Write enough batches to trigger a rotation and append to a new active segment. + // Write enough batches to close the active segment and append to a new active segment. { let segmented_log = SegmentedLog::try_new(folder_path.clone(), &metadata.schema).unwrap(); - for _ in 0..SEGMENT_ROTATION_THRESHOLD + 2 { + for _ in 0..SEGMENT_BATCH_COUNT_THRESHOLD + 2 { segmented_log.append_and_sync(&batch).unwrap(); } } @@ -1024,12 +1028,12 @@ mod tests { let closed = segmented_log.closed_segments.lock().unwrap(); assert_eq!(closed.len(), 2); assert_eq!(closed[0].start_id, 0); - assert_eq!(closed[0].end_id, SEGMENT_ROTATION_THRESHOLD - 1); - assert_eq!(closed[1].start_id, SEGMENT_ROTATION_THRESHOLD); - assert_eq!(closed[1].end_id, SEGMENT_ROTATION_THRESHOLD + 1); + assert_eq!(closed[0].end_id, SEGMENT_BATCH_COUNT_THRESHOLD - 1); + assert_eq!(closed[1].start_id, SEGMENT_BATCH_COUNT_THRESHOLD); + assert_eq!(closed[1].end_id, SEGMENT_BATCH_COUNT_THRESHOLD + 1); let active = segmented_log.active_segment.lock().unwrap(); - assert_eq!(active.next_batch_id, SEGMENT_ROTATION_THRESHOLD + 2); + assert_eq!(active.next_batch_id, SEGMENT_BATCH_COUNT_THRESHOLD + 2); } #[test] @@ -1064,8 +1068,8 @@ mod tests { let batch = table::uncompressed_time_series_table_record_batch(5); - // Fill and rotate one full segment. - for _ in 0..SEGMENT_ROTATION_THRESHOLD { + // Fill and close one full segment. + for _ in 0..SEGMENT_BATCH_COUNT_THRESHOLD { segmented_log.append_and_sync(&batch).unwrap(); } @@ -1074,7 +1078,7 @@ mod tests { .clone(); assert!(segment_path.exists()); - let ids: HashSet = (0..SEGMENT_ROTATION_THRESHOLD).collect(); + let ids: HashSet = (0..SEGMENT_BATCH_COUNT_THRESHOLD).collect(); segmented_log.mark_batches_as_persisted(ids).unwrap(); assert!(!segment_path.exists()); @@ -1088,7 +1092,7 @@ mod tests { let batch = table::uncompressed_time_series_table_record_batch(5); - for _ in 0..SEGMENT_ROTATION_THRESHOLD { + for _ in 0..SEGMENT_BATCH_COUNT_THRESHOLD { segmented_log.append_and_sync(&batch).unwrap(); } @@ -1097,7 +1101,7 @@ mod tests { .clone(); // Only persist a subset of the batch ids in the closed segment. - let partial_ids: HashSet = (0..SEGMENT_ROTATION_THRESHOLD - 1).collect(); + let partial_ids: HashSet = (0..SEGMENT_BATCH_COUNT_THRESHOLD - 1).collect(); segmented_log .mark_batches_as_persisted(partial_ids) .unwrap(); @@ -1108,7 +1112,7 @@ mod tests { // When persisting the last batch, the segment should be deleted. segmented_log - .mark_batches_as_persisted(HashSet::from([SEGMENT_ROTATION_THRESHOLD - 1])) + .mark_batches_as_persisted(HashSet::from([SEGMENT_BATCH_COUNT_THRESHOLD - 1])) .unwrap(); assert!(!segment_path.exists()); @@ -1122,14 +1126,14 @@ mod tests { let batch = table::uncompressed_time_series_table_record_batch(5); - // Trigger five full rotations. - for _ in 0..SEGMENT_ROTATION_THRESHOLD * 5 { + // Close five full segments. + for _ in 0..SEGMENT_BATCH_COUNT_THRESHOLD * 5 { segmented_log.append_and_sync(&batch).unwrap(); } assert_eq!(segmented_log.closed_segments.lock().unwrap().len(), 5); - let ids: HashSet = (0..SEGMENT_ROTATION_THRESHOLD * 5).collect(); + let ids: HashSet = (0..SEGMENT_BATCH_COUNT_THRESHOLD * 5).collect(); segmented_log.mark_batches_as_persisted(ids).unwrap(); assert!(segmented_log.closed_segments.lock().unwrap().is_empty()); @@ -1195,7 +1199,7 @@ mod tests { let (_wal_dir, segmented_log) = new_segmented_log(&temp_dir); let batch = table::uncompressed_time_series_table_record_batch(5); - for _ in 0..SEGMENT_ROTATION_THRESHOLD { + for _ in 0..SEGMENT_BATCH_COUNT_THRESHOLD { segmented_log.append_and_sync(&batch).unwrap(); } @@ -1204,7 +1208,7 @@ mod tests { .clone(); assert!(segment_path.exists()); - let all_ids: HashSet = (0..SEGMENT_ROTATION_THRESHOLD).collect(); + let all_ids: HashSet = (0..SEGMENT_BATCH_COUNT_THRESHOLD).collect(); let delta_table = write_compressed_segments_with_batch_ids(&data_folder, all_ids).await; segmented_log @@ -1223,7 +1227,7 @@ mod tests { let (_wal_dir, segmented_log) = new_segmented_log(&temp_dir); let batch = table::uncompressed_time_series_table_record_batch(5); - for _ in 0..SEGMENT_ROTATION_THRESHOLD { + for _ in 0..SEGMENT_BATCH_COUNT_THRESHOLD { segmented_log.append_and_sync(&batch).unwrap(); } @@ -1231,7 +1235,7 @@ mod tests { .path .clone(); - let partial_ids: HashSet = (0..SEGMENT_ROTATION_THRESHOLD - 1).collect(); + let partial_ids: HashSet = (0..SEGMENT_BATCH_COUNT_THRESHOLD - 1).collect(); let delta_table = write_compressed_segments_with_batch_ids(&data_folder, partial_ids).await; segmented_log @@ -1243,7 +1247,7 @@ mod tests { assert_eq!(segmented_log.closed_segments.lock().unwrap().len(), 1); assert_eq!( segmented_log.persisted_batch_ids.lock().unwrap().len() as u64, - SEGMENT_ROTATION_THRESHOLD - 1 + SEGMENT_BATCH_COUNT_THRESHOLD - 1 ); } @@ -1342,24 +1346,24 @@ mod tests { let batch = table::uncompressed_time_series_table_record_batch(5); - // Fill one full segment (triggers rotation) and write two more into the active segment. - for _ in 0..SEGMENT_ROTATION_THRESHOLD + 2 { + // Fill one full segment and write two more into the active segment. + for _ in 0..SEGMENT_BATCH_COUNT_THRESHOLD + 2 { segmented_log.append_and_sync(&batch).unwrap(); } // Persist one batch id in the closed segment and one in the active segment. segmented_log - .mark_batches_as_persisted(HashSet::from([0, SEGMENT_ROTATION_THRESHOLD + 1])) + .mark_batches_as_persisted(HashSet::from([0, SEGMENT_BATCH_COUNT_THRESHOLD + 1])) .unwrap(); assert_eq!(segmented_log.closed_segments.lock().unwrap().len(), 1); let unpersisted = segmented_log.unpersisted_batches().unwrap(); - assert_eq!(unpersisted.len() as u64, SEGMENT_ROTATION_THRESHOLD); + assert_eq!(unpersisted.len() as u64, SEGMENT_BATCH_COUNT_THRESHOLD); assert_eq!(unpersisted.first().unwrap(), &(1, batch.clone())); assert_eq!( unpersisted.last().unwrap(), - &(SEGMENT_ROTATION_THRESHOLD, batch) + &(SEGMENT_BATCH_COUNT_THRESHOLD, batch) ); } From 9825bd9858f2883a49dd91f611db66b8d825b48f Mon Sep 17 00:00:00 2001 From: CGodiksen <36046286+CGodiksen@users.noreply.github.com> Date: Tue, 24 Mar 2026 13:34:18 +0100 Subject: [PATCH 114/116] Update based on comments from @skejserjensen --- .../modelardb_storage/src/write_ahead_log.rs | 116 +++++++++++------- 1 file changed, 71 insertions(+), 45 deletions(-) diff --git a/crates/modelardb_storage/src/write_ahead_log.rs b/crates/modelardb_storage/src/write_ahead_log.rs index 8cce9343..b917bb4a 100644 --- a/crates/modelardb_storage/src/write_ahead_log.rs +++ b/crates/modelardb_storage/src/write_ahead_log.rs @@ -222,7 +222,7 @@ struct ActiveSegment { path: PathBuf, /// Batch id of the first batch written to this segment. start_id: u64, - /// Writer to write data in IPC streaming format to this segment file. + /// Writer to write data in Apache Arrow IPC streaming format to this segment file. writer: StreamWriter, /// The batch id to give to the next batch of data. Monotonically increasing across segments. next_batch_id: u64, @@ -232,7 +232,7 @@ impl ActiveSegment { /// Create a new [`ActiveSegment`] in `folder_path` with the given `start_id` and `schema`. /// If the file could not be created, return [`ModelarDbStorageError`]. fn try_new(folder_path: PathBuf, schema: &Schema, start_id: u64) -> Result { - let path = folder_path.join(format!("{start_id}-.wal")); + let path = folder_path.join(format!("{start_id}-.arrows")); let file = OpenOptions::new() .create(true) .read(true) @@ -256,13 +256,13 @@ impl ActiveSegment { } } -/// Segmented log that appends data in Arrow IPC streaming format to segment files in a folder. -/// At any point in time there is exactly one active segment being written to plus zero or more -/// closed segments that are read-only. The active segment is closed once +/// Segmented log that appends data in Apache Arrow IPC streaming format to segment files in a +/// folder. At any point in time there is exactly one active segment being written to plus zero or +/// more closed segments that are read-only. The active segment is closed once /// [`SEGMENT_BATCH_COUNT_THRESHOLD`] batches have been written to it. Appending enforces that /// [`sync_data()`](File::sync_data) is called immediately after writing to ensure that all data is /// on disk before returning. Note that an exclusive lock is held on the file while it is being -/// written to. +/// written to, to ensure that no other thread can write to it. struct SegmentedLog { /// Folder that contains all segment files for this log. folder_path: PathBuf, @@ -279,7 +279,7 @@ struct SegmentedLog { impl SegmentedLog { /// Create a new [`SegmentedLog`] that appends data with `schema` to segment files in - /// `folder_path`. Existing closed segment files are loaded into the closed-segment list. + /// `folder_path`. Existing closed segment files are appended to the closed-segment list. /// A fresh active segment is always created on start-up. If the folder or file could not be /// created, return [`ModelarDbStorageError`]. fn try_new(folder_path: PathBuf, schema: &Schema) -> Result { @@ -304,7 +304,7 @@ impl SegmentedLog { } // Always create a fresh active segment on startup to avoid writing into the middle of - // an existing IPC stream. + // an existing Apache Arrow IPC stream. let active_file = ActiveSegment::try_new(folder_path.clone(), schema, next_id)?; Ok(Self { @@ -334,6 +334,7 @@ impl SegmentedLog { // Get a reference to the underlying file handle and sync to disk. Note that file metadata // such as modification timestamps and permissions are not updated since we only sync data. + // Only syncing data reduces disk operations and improves performance. active.writer.get_ref().sync_data()?; // Increment the batch id for the next batch of data. @@ -358,8 +359,10 @@ impl SegmentedLog { Ok(current_batch_id) } - /// Close the current active segment by renaming it to its final `{start_id}-{end_id}.wal` - /// name and open a fresh active segment. The caller must hold the `active_segment` lock. + /// Close the current active segment by renaming it to its final `{start_id}-{end_id}.arrows` + /// name and open a fresh active segment. The end id is added to the file name to avoid having + /// to read the entire file to determine the end id later. The caller must hold the + /// `active_segment` lock. fn close_active_segment(&self, active: &mut ActiveSegment) -> Result<()> { let mut closed_segments = self .closed_segments @@ -375,13 +378,13 @@ impl SegmentedLog { "Closing active WAL segment." ); - // Finish the current writer so the IPC end-of-stream marker is written. + // Finish the current writer so the Apache Arrow IPC end-of-stream marker is written. active.writer.finish()?; - // Rename the active file to its permanent name. + // Rename the active file to its permanent name that includes the end id. let closed_path = self .folder_path - .join(format!("{}-{end_id}.wal", active.start_id)); + .join(format!("{}-{end_id}.arrows", active.start_id)); std::fs::rename(&active.path, &closed_path)?; closed_segments.push(ClosedSegment { @@ -390,7 +393,7 @@ impl SegmentedLog { end_id, }); - // Open a fresh active segment. + // Open a new active segment. let next_id = end_id + 1; *active = ActiveSegment::try_new(self.folder_path.clone(), &self.schema, next_id)?; @@ -479,7 +482,7 @@ impl SegmentedLog { .expect("Mutex should not be poisoned."); Ok(self - .read_all()? + .all_batches()? .into_iter() .filter(|(batch_id, _)| !persisted.contains(batch_id)) .collect()) @@ -487,7 +490,7 @@ impl SegmentedLog { /// Read all data from all segment files (closed and active) in order and return them as pairs /// of (batch_id, batch). If any file could not be read, return [`ModelarDbStorageError`]. - fn read_all(&self) -> Result> { + fn all_batches(&self) -> Result> { // Acquire the mutex to ensure data is not being written while reading. let active = self .active_segment @@ -505,7 +508,7 @@ impl SegmentedLog { all_batches.extend((segment.start_id..=segment.end_id).zip(batches)); } - // Add the active segment's batches to the end of the list. + // Append the active segment's batches to the end of the list. let active_batches = read_batches_from_path(&active.path)?; if !active_batches.is_empty() { all_batches.extend((active.start_id..=active.next_batch_id - 1).zip(active_batches)); @@ -522,8 +525,8 @@ impl SegmentedLog { } } -/// If a leftover active segment (`{start_id}-.wal`) exists in `folder_path`, rename it to -/// its final `{start_id}-{end_id}.wal` name so it is picked up as a closed segment. If the +/// If a leftover active segment (`{start_id}-.arrows`) exists in `folder_path`, rename it to +/// its final `{start_id}-{end_id}.arrows` name so it is picked up as a closed segment. If the /// file contains no batches, it is removed instead. If the file could not be renamed or /// removed, return [`ModelarDbStorageError`]. fn close_leftover_active_segment(folder_path: &Path) -> Result<()> { @@ -555,7 +558,7 @@ fn close_leftover_active_segment(folder_path: &Path) -> Result<()> { debug!(path = %active_path.display(), "Removed empty leftover active WAL segment."); } else { let end_id = start_id + batches.len() as u64 - 1; - let closed_path = folder_path.join(format!("{start_id}-{end_id}.wal")); + let closed_path = folder_path.join(format!("{start_id}-{end_id}.arrows")); warn!( path = %active_path.display(), @@ -571,7 +574,7 @@ fn close_leftover_active_segment(folder_path: &Path) -> Result<()> { } /// Collect all closed segment files in `folder_path`. Closed segments have names of the form -/// `{start_id}-{end_id}.wal` where both `start_id` and `end_id` are valid `u64` values. +/// `{start_id}-{end_id}.arrows` where both `start_id` and `end_id` are valid `u64` values. fn find_closed_segments(folder_path: &Path) -> Result> { let mut segments = Vec::new(); @@ -591,6 +594,11 @@ fn find_closed_segments(folder_path: &Path) -> Result> { start_id, end_id, }); + } else { + return Err(ModelarDbStorageError::InvalidState(format!( + "Unexpected file found in WAL folder: {}.", + path.display() + ))); } } @@ -598,8 +606,8 @@ fn find_closed_segments(folder_path: &Path) -> Result> { } /// Read all [`RecordBatches`](RecordBatch) from the file at `path`. Tolerates a missing -/// end-of-stream marker, which is normal for the active segment. If the file could not be read, -/// return [`ModelarDbStorageError`]. +/// end-of-stream marker, which is normal for the active segment since [`StreamWriter::finish()`] +/// has not been called yet. If the file could not be read, return [`ModelarDbStorageError`]. fn read_batches_from_path(path: &Path) -> Result> { let file = File::open(path)?; let reader = StreamReader::try_new(file, None)?; @@ -650,7 +658,8 @@ mod tests { async fn test_try_new_loads_persisted_batch_ids() { let (_temp_dir, data_folder) = create_data_folder_with_time_series_table().await; - // Simulate a previous WAL session by committing batch ids to the delta table. + // Simulate that data was written to the table in a previous process to ensure that the + // WAL can load already persisted batch ids from the commit history. write_compressed_segments_with_batch_ids(&data_folder, HashSet::from([0, 1, 2])).await; let wal = WriteAheadLog::try_new(&data_folder).await.unwrap(); @@ -677,8 +686,10 @@ mod tests { #[tokio::test] async fn test_create_table_log_adds_log_for_table() { let (_temp_dir, mut wal) = new_empty_write_ahead_log().await; - let metadata = table::time_series_table_metadata(); + + assert!(wal.table_logs.is_empty()); + wal.create_table_log(&metadata).await.unwrap(); assert!(wal.table_logs.contains_key(TIME_SERIES_TABLE_NAME)); @@ -707,6 +718,7 @@ mod tests { let log_path = wal.table_logs[TIME_SERIES_TABLE_NAME].folder_path.clone(); assert!(log_path.exists()); + assert!(wal.table_logs.contains_key(TIME_SERIES_TABLE_NAME)); wal.remove_table_log(TIME_SERIES_TABLE_NAME).unwrap(); @@ -802,6 +814,11 @@ mod tests { wal.append_to_table_log(TIME_SERIES_TABLE_NAME, &batch) .unwrap(); + let unpersisted = wal + .unpersisted_batches_in_table_log(TIME_SERIES_TABLE_NAME) + .unwrap(); + assert_eq!(unpersisted.len(), 2); + wal.mark_batches_as_persisted_in_table_log(TIME_SERIES_TABLE_NAME, HashSet::from([0, 1])) .unwrap(); @@ -885,7 +902,7 @@ mod tests { let temp_dir = tempfile::tempdir().unwrap(); let (_folder_path, segmented_log) = new_segmented_log(&temp_dir); - let batches = segmented_log.read_all().unwrap(); + let batches = segmented_log.all_batches().unwrap(); assert!(batches.is_empty()); let active = segmented_log.active_segment.lock().unwrap(); @@ -900,7 +917,7 @@ mod tests { let batch = table::uncompressed_time_series_table_record_batch(5); segmented_log.append_and_sync(&batch).unwrap(); - let batches = segmented_log.read_all().unwrap(); + let batches = segmented_log.all_batches().unwrap(); assert_eq!(batches.len(), 1); assert_eq!(batches[0], (0, batch)); @@ -921,7 +938,7 @@ mod tests { segmented_log.append_and_sync(&batch_2).unwrap(); segmented_log.append_and_sync(&batch_3).unwrap(); - let batches = segmented_log.read_all().unwrap(); + let batches = segmented_log.all_batches().unwrap(); assert_eq!(batches.len(), 3); assert_eq!(batches[0], (0, batch_1)); assert_eq!(batches[1], (1, batch_2)); @@ -984,7 +1001,7 @@ mod tests { let batch = table::uncompressed_time_series_table_record_batch(10); - // Fill and close one full segment. + // Write enough batches to close the active segment, then drop. { let segmented_log = SegmentedLog::try_new(folder_path.clone(), &metadata.schema).unwrap(); @@ -996,7 +1013,7 @@ mod tests { let segmented_log = SegmentedLog::try_new(folder_path.clone(), &metadata.schema).unwrap(); segmented_log.append_and_sync(&batch).unwrap(); - let batches = segmented_log.read_all().unwrap(); + let batches = segmented_log.all_batches().unwrap(); assert_eq!(batches.len() as u64, SEGMENT_BATCH_COUNT_THRESHOLD + 1); let active = segmented_log.active_segment.lock().unwrap(); @@ -1011,7 +1028,8 @@ mod tests { let batch = table::uncompressed_time_series_table_record_batch(5); - // Write enough batches to close the active segment and append to a new active segment. + // Write enough batches to close the active segment and append to a new active segment, + // then drop. { let segmented_log = SegmentedLog::try_new(folder_path.clone(), &metadata.schema).unwrap(); @@ -1043,7 +1061,7 @@ mod tests { let metadata = table::time_series_table_metadata(); // Create a segmented log and immediately drop it without writing anything. - // This leaves an empty "{start_id}-.wal" active segment. + // This leaves an empty "{start_id}-.arrows" active segment. { SegmentedLog::try_new(folder_path.clone(), &metadata.schema).unwrap(); } @@ -1182,7 +1200,7 @@ mod tests { write_compressed_segments_with_batch_ids(&data_folder, HashSet::from([0, 1, 2])).await; let delta_table = - write_compressed_segments_with_batch_ids(&data_folder, HashSet::from([2, 3, 4])).await; + write_compressed_segments_with_batch_ids(&data_folder, HashSet::from([3, 4, 5])).await; segmented_log .load_persisted_batches_from_delta_table(delta_table) @@ -1190,7 +1208,7 @@ mod tests { .unwrap(); let persisted = segmented_log.persisted_batch_ids.lock().unwrap(); - assert_eq!(*persisted, BTreeSet::from([0, 1, 2, 3, 4])); + assert_eq!(*persisted, BTreeSet::from([0, 1, 2, 3, 4, 5])); } #[tokio::test] @@ -1285,6 +1303,15 @@ mod tests { .unwrap() } + #[test] + fn test_unpersisted_batches_returns_empty_when_no_batches_written() { + let temp_dir = tempfile::tempdir().unwrap(); + let (_folder_path, segmented_log) = new_segmented_log(&temp_dir); + + let unpersisted = segmented_log.unpersisted_batches().unwrap(); + assert!(unpersisted.is_empty()); + } + #[test] fn test_unpersisted_batches_returns_all_when_none_persisted() { let temp_dir = tempfile::tempdir().unwrap(); @@ -1310,6 +1337,9 @@ mod tests { segmented_log.append_and_sync(&batch).unwrap(); segmented_log.append_and_sync(&batch).unwrap(); + let unpersisted = segmented_log.unpersisted_batches().unwrap(); + assert_eq!(unpersisted.len(), 2); + segmented_log .mark_batches_as_persisted(HashSet::from([0, 1])) .unwrap(); @@ -1325,9 +1355,14 @@ mod tests { let batch_1 = table::uncompressed_time_series_table_record_batch(10); let batch_2 = table::uncompressed_time_series_table_record_batch(20); - segmented_log.append_and_sync(&batch_1).unwrap(); + let batch_3 = table::uncompressed_time_series_table_record_batch(30); segmented_log.append_and_sync(&batch_1).unwrap(); segmented_log.append_and_sync(&batch_2).unwrap(); + segmented_log.append_and_sync(&batch_3).unwrap(); + + let unpersisted = segmented_log.unpersisted_batches().unwrap(); + assert_eq!(unpersisted.len(), 3); + assert_eq!(unpersisted[1], (1, batch_2)); segmented_log .mark_batches_as_persisted(HashSet::from([1])) @@ -1336,7 +1371,7 @@ mod tests { let unpersisted = segmented_log.unpersisted_batches().unwrap(); assert_eq!(unpersisted.len(), 2); assert_eq!(unpersisted[0], (0, batch_1)); - assert_eq!(unpersisted[1], (2, batch_2)); + assert_eq!(unpersisted[1], (2, batch_3)); } #[test] @@ -1367,15 +1402,6 @@ mod tests { ); } - #[test] - fn test_unpersisted_batches_returns_empty_when_no_batches_written() { - let temp_dir = tempfile::tempdir().unwrap(); - let (_folder_path, segmented_log) = new_segmented_log(&temp_dir); - - let unpersisted = segmented_log.unpersisted_batches().unwrap(); - assert!(unpersisted.is_empty()); - } - fn new_segmented_log(temp_dir: &TempDir) -> (PathBuf, SegmentedLog) { let folder_path = temp_dir.path().join(TIME_SERIES_TABLE_NAME); let metadata = table::time_series_table_metadata(); From 9addf202cd44d9fcfb451854f65fffa428295ca1 Mon Sep 17 00:00:00 2001 From: CGodiksen <36046286+CGodiksen@users.noreply.github.com> Date: Tue, 24 Mar 2026 14:11:55 +0100 Subject: [PATCH 115/116] Fix writing of record batches after merge --- .../src/operations/data_folder.rs | 2 +- crates/modelardb_server/src/context.rs | 5 +-- .../src/storage/compressed_data_manager.rs | 6 +--- .../src/storage/data_transfer.rs | 1 - .../src/data_folder/delta_table_writer.rs | 21 ++++++++++++- .../modelardb_storage/src/data_folder/mod.rs | 31 ++++++++++++++++++- .../modelardb_storage/src/write_ahead_log.rs | 7 +---- 7 files changed, 54 insertions(+), 19 deletions(-) diff --git a/crates/modelardb_embedded/src/operations/data_folder.rs b/crates/modelardb_embedded/src/operations/data_folder.rs index 1f059f4f..92700c89 100644 --- a/crates/modelardb_embedded/src/operations/data_folder.rs +++ b/crates/modelardb_embedded/src/operations/data_folder.rs @@ -16,7 +16,7 @@ //! Operations for reading from and writing to ModelarDB data folders. use std::any::Any; -use std::collections::{HashMap, HashSet}; +use std::collections::HashMap; use std::fmt::{Debug, Formatter, Result as FmtResult}; use std::pin::Pin; use std::result::Result as StdResult; diff --git a/crates/modelardb_server/src/context.rs b/crates/modelardb_server/src/context.rs index 102bdba6..e1f5ba5d 100644 --- a/crates/modelardb_server/src/context.rs +++ b/crates/modelardb_server/src/context.rs @@ -306,7 +306,7 @@ impl Context { session_context.deregister_table(table_name)?; self.drop_table_from_storage_engine(table_name).await?; - + let local_data_folder = &self.data_folders.local_data_folder; // If the table is a time series table, delete the table log file from the write-ahead log. @@ -456,8 +456,6 @@ fn table_does_not_exist_error(table_name: &str) -> ModelarDbServerError { mod tests { use super::*; - use std::collections::HashSet; - use modelardb_storage::data_folder::DataFolder; use modelardb_test::table::{self, NORMAL_TABLE_NAME, TIME_SERIES_TABLE_NAME}; use modelardb_types::types::MAX_RETENTION_PERIOD_IN_SECONDS; @@ -878,7 +876,6 @@ mod tests { .write_record_batches( TIME_SERIES_TABLE_NAME, vec![table::compressed_segments_record_batch()], - HashSet::new(), ) .await .unwrap(); diff --git a/crates/modelardb_server/src/storage/compressed_data_manager.rs b/crates/modelardb_server/src/storage/compressed_data_manager.rs index 829d8dcd..a0ccd9a7 100644 --- a/crates/modelardb_server/src/storage/compressed_data_manager.rs +++ b/crates/modelardb_server/src/storage/compressed_data_manager.rs @@ -258,11 +258,7 @@ impl CompressedDataManager { // excludes them during replay. The WAL update is only an optimization to eagerly delete // fully persisted WAL segment files. self.local_data_folder - .write_compressed_segments_to_time_series_table( - table_name, - compressed_segments, - batch_ids.clone(), - ) + .write_record_batches_with_batch_ids(table_name, compressed_segments, batch_ids.clone()) .await?; // Inform the write-ahead log that data has been written to disk. We use a read lock since diff --git a/crates/modelardb_server/src/storage/data_transfer.rs b/crates/modelardb_server/src/storage/data_transfer.rs index d6909d96..cad3ff90 100644 --- a/crates/modelardb_server/src/storage/data_transfer.rs +++ b/crates/modelardb_server/src/storage/data_transfer.rs @@ -492,7 +492,6 @@ mod tests { .write_record_batches( TIME_SERIES_TABLE_NAME, vec![table::compressed_segments_record_batch()], - HashSet::new(), ) .await .unwrap(); diff --git a/crates/modelardb_storage/src/data_folder/delta_table_writer.rs b/crates/modelardb_storage/src/data_folder/delta_table_writer.rs index 8501c4f7..78395ba6 100644 --- a/crates/modelardb_storage/src/data_folder/delta_table_writer.rs +++ b/crates/modelardb_storage/src/data_folder/delta_table_writer.rs @@ -17,6 +17,7 @@ //! [`RecordBatches`](RecordBatch) to a Delta table stored in an object store. Writing can be //! committed or rolled back to ensure that the Delta table is always in a consistent state. +use std::collections::HashSet; use std::sync::Arc; use arrow::array::RecordBatch; @@ -34,6 +35,7 @@ use deltalake::protocol::{DeltaOperation, SaveMode}; use modelardb_types::schemas::{COMPRESSED_SCHEMA, FIELD_COLUMN}; use object_store::ObjectStore; use object_store::path::Path; +use serde_json::json; use uuid::Uuid; use crate::apache_parquet_writer_properties; @@ -52,6 +54,9 @@ pub struct DeltaTableWriter { operation_id: Uuid, /// Writes record batches to the Delta table as Apache Parquet files. delta_writer: DeltaWriter, + /// Batch ids from the WAL to include in the commit metadata so the uncompressed batches that + /// correspond to the written data can be deleted from the WAL. + batch_ids: HashSet, } impl DeltaTableWriter { @@ -140,9 +145,17 @@ impl DeltaTableWriter { delta_operation, operation_id, delta_writer, + batch_ids: HashSet::new(), }) } + /// Add batch ids from the WAL that are included in the commit metadata so the uncompressed + /// batches that correspond to the written data can be deleted from the WAL. + pub fn with_batch_ids(mut self, batch_ids: HashSet) -> Self { + self.batch_ids = batch_ids; + self + } + /// Write `record_batch` to the Delta table. Returns a [`ModelarDbStorageError`] if the /// [`RecordBatches`](RecordBatch) does not match the schema of the Delta table or if the /// writing fails. @@ -196,7 +209,13 @@ impl DeltaTableWriter { // Prepare all inputs to the commit. let object_store = self.delta_table.object_store(); - let commit_properties = CommitProperties::default(); + + let mut commit_properties = CommitProperties::default(); + if !self.batch_ids.is_empty() { + commit_properties = commit_properties + .with_metadata(vec![("batchIds".to_owned(), json!(self.batch_ids))]); + } + let table_data = match self.delta_table.snapshot() { Ok(table_data) => table_data, Err(delta_table_error) => { diff --git a/crates/modelardb_storage/src/data_folder/mod.rs b/crates/modelardb_storage/src/data_folder/mod.rs index 6d7bcaf2..a08b5e7c 100644 --- a/crates/modelardb_storage/src/data_folder/mod.rs +++ b/crates/modelardb_storage/src/data_folder/mod.rs @@ -53,7 +53,6 @@ use object_store::aws::AmazonS3Builder; use object_store::local::LocalFileSystem; use object_store::memory::InMemory; use object_store::path::Path; -use serde_json::json; use url::Url; use crate::data_folder::delta_table_writer::DeltaTableWriter; @@ -332,6 +331,14 @@ impl DataFolder { &self.session_context } + /// Return the location of the Delta Lake. This is `memory:///modelardb` if the Delta Lake is + /// in memory, the local path if the Delta Lake is stored on disk, `az://container-name` + /// if the Delta Lake is stored in Azure Blob Storage, or `s3://bucket-name` if the Delta Lake + /// is stored in Amazon S3. + pub fn location(&self) -> &str { + &self.location + } + /// Return an [`ObjectStore`] to access the root of the Delta Lake. pub fn object_store(&self) -> Arc { self.object_store.clone() @@ -691,6 +698,28 @@ impl DataFolder { .await } + /// Write `record_batches` to the table with `table_name` in the Delta Lake. The correct + /// writer is selected automatically based on the table type. The `batch_ids` from the WAL are + /// included in the commit metadata, so the uncompressed batches that correspond to + /// `record_batches` can be deleted from the WAL. If the uncompressed batches were not written + /// to the WAL, use [`Self::write_record_batches`] instead. Returns an updated [`DeltaTable`] + /// if the file was written successfully, otherwise returns [`ModelarDbStorageError`]. + pub async fn write_record_batches_with_batch_ids( + &self, + table_name: &str, + record_batches: Vec, + batch_ids: HashSet, + ) -> Result { + let delta_table_writer = self + .table_writer(table_name) + .await? + .with_batch_ids(batch_ids); + + delta_table_writer + .write_all_and_commit(&record_batches) + .await + } + /// Write `columns` to a Delta Lake table with `table_name`. Returns an updated [`DeltaTable`] /// version if the file was written successfully, otherwise returns [`ModelarDbStorageError`]. async fn write_columns_to_metadata_table( diff --git a/crates/modelardb_storage/src/write_ahead_log.rs b/crates/modelardb_storage/src/write_ahead_log.rs index b917bb4a..dc1e3dd7 100644 --- a/crates/modelardb_storage/src/write_ahead_log.rs +++ b/crates/modelardb_storage/src/write_ahead_log.rs @@ -1279,11 +1279,6 @@ mod tests { .await .unwrap(); - data_folder - .save_time_series_table_metadata(&metadata) - .await - .unwrap(); - (temp_dir, data_folder) } @@ -1294,7 +1289,7 @@ mod tests { let compressed_segments = table::compressed_segments_record_batch(); data_folder - .write_compressed_segments_to_time_series_table( + .write_record_batches_with_batch_ids( TIME_SERIES_TABLE_NAME, vec![compressed_segments], batch_ids, From 023b277f390da563252258619428a335e9f6bb4b Mon Sep 17 00:00:00 2001 From: CGodiksen <36046286+CGodiksen@users.noreply.github.com> Date: Fri, 27 Mar 2026 14:56:47 +0100 Subject: [PATCH 116/116] Update based on comments from @chrthomsen --- .../modelardb_storage/src/write_ahead_log.rs | 19 ++++++++++++------- 1 file changed, 12 insertions(+), 7 deletions(-) diff --git a/crates/modelardb_storage/src/write_ahead_log.rs b/crates/modelardb_storage/src/write_ahead_log.rs index dc1e3dd7..78e71471 100644 --- a/crates/modelardb_storage/src/write_ahead_log.rs +++ b/crates/modelardb_storage/src/write_ahead_log.rs @@ -51,14 +51,15 @@ pub struct WriteAheadLog { impl WriteAheadLog { /// Create a new [`WriteAheadLog`] that stores the WAL in the root of `local_data_folder` in - /// the [`WRITE_AHEAD_LOG_FOLDER`] folder. If the folder does not exist, it is created. If the - /// WAL could not be created, return [`ModelarDbStorageError`]. + /// the [`WRITE_AHEAD_LOG_FOLDER`] folder. `local_data_folder` must be in a local path since the + /// WAL uses the [`std::fs`] API to avoid the overhead of the `ObjectStore` API and to allow the + /// use of [`sync_data()`](File::sync_data). If the folder does not exist, it is created. If + /// `local_data_folder` is not in a local path or the WAL could not be created, return + /// [`ModelarDbStorageError`]. pub async fn try_new(local_data_folder: &DataFolder) -> Result { // Create the folder for the write-ahead log if it does not exist. let location = local_data_folder.location(); - // Since the std::fs API is used, the location must be a local path. We use std::fs to avoid - // the overhead of the ObjectStore API and to allow the use of File::sync_data(). if location.contains("://") { return Err(ModelarDbStorageError::InvalidState(format!( "Write-ahead log location '{location}' is not a local path." @@ -175,8 +176,8 @@ impl WriteAheadLog { table_log.mark_batches_as_persisted(batch_ids) } - /// Return pairs of (batch_id, batch) for all batches that have not yet been persisted in the - /// corresponding table log. If the table log does not exist or the batches could not be read + /// Return pairs of (batch_id, batch) for all batches in the corresponding table log that have + /// not yet been persisted. If the table log does not exist or the batches could not be read /// from the table log, return [`ModelarDbStorageError`]. pub fn unpersisted_batches_in_table_log( &self, @@ -211,7 +212,11 @@ struct ClosedSegment { impl ClosedSegment { /// Return `true` if every batch id in this segment is present in `persisted`. fn is_fully_persisted(&self, persisted: &BTreeSet) -> bool { - (self.start_id..=self.end_id).all(|id| persisted.contains(&id)) + // Iterate in reverse since newer (higher) ids are least likely to be persisted, allowing + // all() to short-circuit earlier for partially persisted segments. + (self.start_id..=self.end_id) + .rev() + .all(|id| persisted.contains(&id)) } }