diff --git a/rust/experimental/quiver/README.md b/rust/experimental/quiver/README.md deleted file mode 100644 index cc31fe2cd..000000000 --- a/rust/experimental/quiver/README.md +++ /dev/null @@ -1,36 +0,0 @@ -# Quiver (Experimental) - Arrow-Based Persistence for OTAP Dataflow - README - -The source for the experimental crate now lives under -`rust/otap-dataflow/crates/quiver`. This directory keeps the architecture notes -and high level design documents, while the shared `otap-dataflow` workspace -hosts the actual Rust code so the project inherits the same lint, fmt, and test -infrastructure. - -The crate currently exposes configuration scaffolding, placeholder engine APIs, -and Criterion bench harness stubs. No bytes are persisted yet; every ingest -call intentionally returns `QuiverError::Unimplemented` until Phase 1 wires in -the WAL. - -Integration with the `otap-df` binary is opt-in via the Cargo feature -`quiver-persistence`. The feature is *disabled by default*, so release builds -never pull in the experimental persistence code path unless the flag is -explicitly enabled. - -## Status - -**Not production-ready** This crate is being developed based on the -specifications in `ARCHITECTURE.md` (which may be updated as development -proceeds). It is not yet stable or suitable for taking a dependency on. - -## Quick start - -```bash -cd rust/otap-dataflow -cargo test -p otap-df-quiver # unit tests + doc tests -cargo bench -p otap-df-quiver # opt-in Criterion bench stub -# Enable the downstream integration (still a stub) when needed -cargo test -p otap-df --features quiver-persistence -``` - -The bench currently measures the placeholder ingest path so we have a home for -future perf instrumentation once real I/O lands. diff --git a/rust/otap-dataflow/Cargo.toml b/rust/otap-dataflow/Cargo.toml index 1df533859..fa0a5a32f 100644 --- a/rust/otap-dataflow/Cargo.toml +++ b/rust/otap-dataflow/Cargo.toml @@ -63,6 +63,7 @@ base64 = "0.22.1" bitflags = "2.10" bytes = "1.11" bytemuck = "1.24" +blake3 = "1.5.5" chrono = { version = "0.4", features = ["serde"] } ciborium = "0.2.2" clap = { version = "4.5.42", features = ["derive"] } @@ -83,7 +84,7 @@ local-sync = "0.1.1" log = "0.4" miette = { version="7.6.0", features = ["fancy"] } mimalloc-rust = "0.2.1" -nix = { version = "0.30.1", features = ["process", "signal"] } +nix = { version = "0.30.1", features = ["process", "signal", "fs"] } num_enum = "0.7" object_store = "0.12.3" once_cell = "1.20.2" @@ -111,6 +112,7 @@ serde_cbor = "0.11.2" serde_json = { version = "1.0.145" } serde_with = { version = "3.16.0", features = ["std", "macros", "json"] } serde_yaml = "0.9.34-deprecated" # Deprecated, but no good alternative yet +crc32fast = "1.4.2" replace_with = "0.1.8" simdutf8 = "0.1.5" slotmap = "1.0.7" diff --git a/rust/otap-dataflow/crates/quiver/ARCHITECTURE.md b/rust/otap-dataflow/crates/quiver/ARCHITECTURE.md index 54ca621bc..13572f5ba 100644 --- a/rust/otap-dataflow/crates/quiver/ARCHITECTURE.md +++ b/rust/otap-dataflow/crates/quiver/ARCHITECTURE.md @@ -139,7 +139,7 @@ WAL entries belonging to: } [u8;payload_len] arrow_payload; // streaming IPC bytes for the slot } - u32 crc32c; // trailer; covers EntryHeader..payloads + u32 crc32; // trailer; covers EntryHeader..payloads ``` - Arrow payload blobs are serialized in the **streaming** IPC format. For each @@ -150,8 +150,9 @@ WAL entries belonging to: [metadata1][payload1]...` Absent slots (bitmap bit cleared) contribute neither metadata nor bytes; they are implicitly `None` when reconstructing the bundle. - - Every entry ends with a 4-byte little-endian CRC32C checksum that covers the - entry header, bitmap, metadata blocks, and payload bytes (everything except + - Every entry ends with a 4-byte little-endian CRC32 (IEEE polynomial) + checksum that covers the entry header, bitmap, metadata blocks, and payload + bytes (everything except the leading length field and the checksum itself). Replay verifies the CRC before decoding Arrow IPC bytes; a mismatch marks the WAL as corrupted and triggers truncation back to the last known-good offset. @@ -171,78 +172,69 @@ WAL entries belonging to: introduce new entry types (e.g., periodic checkpoints) or swap serialization without breaking older data; unknown entry types are skipped using the recorded length. A checkpoint entry (`entry_type = 1`) would embed the current - open-segment manifest, `truncate_offset`, and high-water sequence numbers so + open-segment manifest, `logical_offset`, and high-water sequence numbers so recovery can jump directly to the latest checkpoint instead of replaying the entire log. -##### Truncation & rotation mechanics - -- **Track truncate progress**: After a segment finalizes and its metadata + file - are flushed, we advance a `truncate_offset` pointer to the first byte belonging - to the next entry in the open segment. Think of `truncate_offset` as "the - earliest WAL byte still needed for crash recovery." We persist that `u64` - (plus a monotonically increasing rotation generation) into a tiny sidecar file - (e.g., `wal/truncate.offset`) immediately after advancing it and fsync the - sidecar so crash recovery can seek straight to that logical offset without - rescanning finalized entries. -- **Truncate sidecar format**: The sidecar is a fixed 32-byte struct written in +##### Checkpointing & rotation mechanics + +- **Track checkpoint progress**: After a segment finalizes and its metadata + file + are flushed, we advance a *logical* cursor that counts data bytes (headers + excluded) across the entire WAL stream. This `global_data_offset` represents "the + earliest WAL data still needed for crash recovery." We persist that `u64` into + a tiny sidecar file (e.g., `wal/checkpoint.offset`) immediately after advancing + it and fsync the sidecar so crash recovery can resume from that logical offset + without rescanning finalized entries. +- **Checkpoint sidecar format**: The sidecar is a fixed 24-byte struct written in little-endian order: ```text - TruncateSidecar { + CheckpointSidecar { [u8; 8] magic = b"QUIVER\0T"; // distinguishes from WAL proper u16 version = 1; // bump if layout changes u16 reserved = 0; - u64 truncate_offset; // first byte still needed in wal/quiver.wal - u64 rotation_generation; // increments each WAL rotation - u32 crc32c; // covers magic..rotation_generation + u64 global_data_offset; // logical data bytes still required + u32 crc32; // covers magic..global_data_offset } ``` - We write updates via `truncate.offset.tmp`: encode the struct, compute the CRC, + We write updates via `checkpoint.offset.tmp`: encode the struct, compute the CRC, `pwrite`+`fdatasync`, then `renameat` over the live file so readers see either the old or new offset. On startup we verify the magic, version, and checksum before trusting the recorded offsets; failure falls back to scanning from the beginning of `quiver.wal`. -- **Probe prefix reclamation**: On startup we test whether the active filesystem - supports punching holes out of the current WAL file. Linux builds attempt - `fallocate(FALLOC_FL_PUNCH_HOLE)` against a temporary WAL stub while Windows - builds issue `FSCTL_SET_ZERO_DATA` on a sparse scratch file. If the probe - fails with `EOPNOTSUPP`/`ERROR_INVALID_FUNCTION` we mark the capability as - disabled and fall back to rewriting until the process restarts. -- **Drop reclaimed prefixes**: When support exists, or when the pointer crosses a - configurable threshold, we invoke `fallocate(FALLOC_FL_PUNCH_HOLE)` in-place to - discard bytes `[header_len, truncate_offset)`, leaving the fixed header (30 - bytes: magic + version + reserved + cfg hash) intact. On filesystems where hole - punching is not supported, we instead perform a rewrite: copy every byte from - `truncate_offset` through the current end-of-file into `quiver.wal.new` (using - `copy_file_range`/`CopyFile2` where available) while the writer stays paused, - then reopen the new file and stream the fixed header back to the front so the - layout matches a freshly created WAL. Once the copy completes we fsync the new - file, atomically rename it over the original, and resume appends at offset - `(header_len) + (old_len - truncate_offset)`. Windows builds rely on - `CopyFile2` plus `SetFileInformationByHandle` for the same sequence. This - rewrite path preserves the header bytes verbatim, so replay cannot distinguish - a rewritten WAL from one that was never hole-punched. +- **Record prefix reclamation**: When higher layers report a safe cursor, we + translate it into the logical coordinate space and persist it via the sidecar + while leaving the active WAL file unchanged. The pointer simply marks how far + readers have progressed through the concatenated stream. +- **Drop reclaimed prefixes**: Once the active file grows beyond + `rotation_target_bytes` we rotate it to `quiver.wal.N` (where N is a + monotonically increasing rotation id), start a fresh WAL, and remember the + byte span covered by the retired file. When the persisted consumer checkpoint + fully covers a rotated file we delete the file outright. Until a rotation + occurs the reclaimed bytes remain in the active file even though they are + logically safe to discard. + *Note:* We may revisit direct hole punching in the future as a disk-space + optimization if production workloads show that waiting for rotation leaves too + much reclaimed data stranded in the active file. - **Rotate on size**: `wal.max_size` caps the *aggregate* footprint of the active WAL plus every still-referenced rotated sibling. We keep a running total of the active file and the byte spans tracked for `quiver.wal.N`; when the next append - would push the aggregate over the configured cap we rotate immediately: shift - older suffixes up, close and rename `wal/quiver.wal` to `quiver.wal.1`, and - reopen a fresh `quiver.wal`. Each rotation records the byte span covered by the - retired chunk so cleanup can later delete `quiver.wal.N` only after the - persisted `truncate_offset` exceeds that span's upper bound. We never - hole-punch rotated files; they are deleted wholesale once fully covered by the - durability pointer, avoiding rewrites of large historical blobs while keeping - the total WAL footprint bounded by `wal.max_size`. To keep rename churn and - per-core directory fan-out predictable we retain at most `wal.max_chunks` - files (default `10`, counting the active WAL plus rotated siblings). Operators - can override the default. Hitting the chunk cap is treated like hitting the - byte cap: the next append that *would* require a rotation instead trips - backpressure (or `drop_oldest`, if selected) until either truncation reclaims - an older chunk or the limit is raised. We never create an eleventh file in the - background because doing so would undermine the predictive bound the knob is - meant to provide. + would push the aggregate over the configured cap we rotate immediately: rename + `wal/quiver.wal` to `quiver.wal.{next_rotation_id}` and reopen a fresh + `quiver.wal`. Each rotation records the byte span covered by the retired file + so cleanup can later delete `quiver.wal.N` only after the persisted logical + cursor exceeds that span's upper bound. We never rewrite rotated files; they + are deleted wholesale once fully covered by the durability pointer, avoiding + rewrites of large historical blobs while keeping the total WAL footprint + bounded by `wal.max_size`. To keep per-core directory fan-out predictable we + retain at most `wal.max_rotated_files` files (default `10`, counting the + active WAL plus rotated siblings). Operators can override the default. Hitting + the rotated file cap is treated like hitting the byte cap: the next append + that *would* require a rotation instead trips backpressure (or `drop_oldest`, + if selected) until either compaction reclaims an older rotated file or the + limit is raised. We never create more files in the background because doing so + would undermine the predictive bound the knob is meant to provide. - **Durability-only dependency**: Because WAL truncation depends solely on segment durability, exporter ACK lag never blocks WAL cleanup; segments themselves remain on disk until subscribers advance, but the WAL only needs to cover the diff --git a/rust/otap-dataflow/crates/quiver/Cargo.toml b/rust/otap-dataflow/crates/quiver/Cargo.toml index a6d26d5f3..af3a46cd5 100644 --- a/rust/otap-dataflow/crates/quiver/Cargo.toml +++ b/rust/otap-dataflow/crates/quiver/Cargo.toml @@ -23,11 +23,19 @@ serde = ["dep:serde"] [dependencies] arrow-array.workspace = true arrow-schema.workspace = true +arrow-ipc.workspace = true +crc32fast.workspace = true +blake3.workspace = true +parking_lot.workspace = true serde = { workspace = true, optional = true } thiserror.workspace = true +[target.'cfg(unix)'.dependencies] +nix.workspace = true + [dev-dependencies] criterion.workspace = true +tempfile.workspace = true [lints] workspace = true diff --git a/rust/otap-dataflow/crates/quiver/README.md b/rust/otap-dataflow/crates/quiver/README.md index fbc53a3cc..a8190fdd9 100644 --- a/rust/otap-dataflow/crates/quiver/README.md +++ b/rust/otap-dataflow/crates/quiver/README.md @@ -15,9 +15,10 @@ explicitly enabled. ## Status -**Not production-ready** This crate is being prototyped based on the -specifications in `ARCHITECTURE.md` (which may be updated as development proceeds). -*It is not yet complete, stable or suitable for taking a dependency on.* +**Under Development, Not production-ready** This crate is +being actively developed based on the specifications in `ARCHITECTURE.md` +(which may be updated as development proceeds). *It is not yet complete, +stable or suitable for taking a dependency on.* ## Quick start diff --git a/rust/otap-dataflow/crates/quiver/src/config.rs b/rust/otap-dataflow/crates/quiver/src/config.rs index 016f25934..98acb2c42 100644 --- a/rust/otap-dataflow/crates/quiver/src/config.rs +++ b/rust/otap-dataflow/crates/quiver/src/config.rs @@ -123,19 +123,26 @@ impl QuiverConfigBuilder { #[derive(Clone, Debug, PartialEq, Eq)] #[cfg_attr(feature = "serde", derive(serde::Serialize, serde::Deserialize))] pub struct WalConfig { - /// Maximum on-disk footprint (across active + rotated chunks). + /// Maximum on-disk footprint (across active + rotated files). pub max_size_bytes: NonZeroU64, - /// Maximum number of chunk files retained during rotation. - pub max_chunk_count: u16, + /// Maximum number of rotated WAL files retained during rotation. + pub max_rotated_files: u16, + /// Target data size to keep in the active WAL file before rotating. + pub rotation_target_bytes: NonZeroU64, /// Preferred fsync cadence for durability vs. latency. pub flush_interval: Duration, } impl WalConfig { fn validate(&self) -> Result<()> { - if self.max_chunk_count == 0 { + if self.max_rotated_files == 0 { return Err(QuiverError::invalid_config( - "max_chunk_count must be at least 1", + "max_rotated_files must be at least 1", + )); + } + if self.rotation_target_bytes > self.max_size_bytes { + return Err(QuiverError::invalid_config( + "rotation_target_bytes must not exceed max_size_bytes", )); } Ok(()) @@ -146,7 +153,8 @@ impl Default for WalConfig { fn default() -> Self { Self { max_size_bytes: NonZeroU64::new(4 * 1024 * 1024 * 1024).expect("non-zero"), - max_chunk_count: 10, + max_rotated_files: 10, + rotation_target_bytes: NonZeroU64::new(64 * 1024 * 1024).expect("non-zero"), flush_interval: Duration::from_millis(25), } } @@ -279,7 +287,8 @@ mod tests { fn builder_overrides_sub_configs() { let wal = WalConfig { max_size_bytes: NonZeroU64::new(1).unwrap(), - max_chunk_count: 1, + max_rotated_files: 1, + rotation_target_bytes: NonZeroU64::new(1).unwrap(), flush_interval: Duration::from_millis(1), }; let segment = SegmentConfig { @@ -316,10 +325,23 @@ mod tests { } #[test] - fn wal_validate_rejects_zero_chunk_count() { + fn wal_validate_rejects_zero_rotated_files() { let wal = WalConfig { max_size_bytes: NonZeroU64::new(1).unwrap(), - max_chunk_count: 0, + max_rotated_files: 0, + rotation_target_bytes: NonZeroU64::new(1).unwrap(), + flush_interval: Duration::from_millis(1), + }; + + assert!(wal.validate().is_err()); + } + + #[test] + fn wal_validate_rejects_rotation_target_exceeding_cap() { + let wal = WalConfig { + max_size_bytes: NonZeroU64::new(1024).unwrap(), + max_rotated_files: 1, + rotation_target_bytes: NonZeroU64::new(2048).unwrap(), flush_interval: Duration::from_millis(1), }; diff --git a/rust/otap-dataflow/crates/quiver/src/engine.rs b/rust/otap-dataflow/crates/quiver/src/engine.rs index c7615487d..06b8e706c 100644 --- a/rust/otap-dataflow/crates/quiver/src/engine.rs +++ b/rust/otap-dataflow/crates/quiver/src/engine.rs @@ -3,25 +3,34 @@ //! Placeholder engine entry-point so other crates can begin wiring dependencies. +use std::path::PathBuf; + +use parking_lot::Mutex; + use crate::config::QuiverConfig; use crate::error::{QuiverError, Result}; use crate::record_bundle::RecordBundle; use crate::telemetry::PersistenceMetrics; +use crate::wal::{WalWriter, WalWriterOptions}; /// Primary entry point for the persistence engine. #[derive(Debug)] pub struct QuiverEngine { config: QuiverConfig, metrics: PersistenceMetrics, + wal_writer: Mutex, } impl QuiverEngine { /// Validates configuration and returns a placeholder engine instance. pub fn new(config: QuiverConfig) -> Result { config.validate()?; + let wal_writer = initialize_wal_writer(&config)?; + Ok(Self { config, metrics: PersistenceMetrics::new(), + wal_writer: Mutex::new(wal_writer), }) } @@ -39,6 +48,11 @@ impl QuiverEngine { pub fn ingest(&self, bundle: &B) -> Result<()> { self.metrics.record_ingest_attempt(); + { + let mut writer = self.wal_writer.lock(); + let _wal_offset = writer.append_bundle(bundle)?; + } + let descriptor = bundle.descriptor(); let _ingestion_time = bundle.ingestion_time(); @@ -52,14 +66,59 @@ impl QuiverEngine { } } +fn initialize_wal_writer(config: &QuiverConfig) -> Result { + use crate::wal::FlushPolicy; + + let wal_path = wal_path(config); + let flush_policy = if config.wal.flush_interval.is_zero() { + FlushPolicy::Immediate + } else { + FlushPolicy::EveryDuration(config.wal.flush_interval) + }; + let options = WalWriterOptions::new(wal_path, segment_cfg_hash(config), flush_policy) + .with_max_wal_size(config.wal.max_size_bytes.get()) + .with_max_rotated_files(config.wal.max_rotated_files as usize) + .with_rotation_target(config.wal.rotation_target_bytes.get()); + Ok(WalWriter::open(options)?) +} + +fn wal_path(config: &QuiverConfig) -> PathBuf { + config.data_dir.join("wal").join("quiver.wal") +} + +fn segment_cfg_hash(_config: &QuiverConfig) -> [u8; 16] { + // Placeholder: the segment_cfg_hash should be derived from adapter-owned + // layout contracts (slot id → payload mappings, per-slot ordering, checksum + // policy toggles) once available. Operational knobs like segment.target_size, + // flush cadence, or retention caps are intentionally excluded so that tuning + // never invalidates an otherwise healthy WAL. + // + // For now we return a fixed placeholder until adapter metadata is implemented. + // + // Future implementation might look like: + // ``` + // let mut hasher = Hasher::new(); + // hasher.update(&adapter.slot_layout_fingerprint()); + // hasher.update(&adapter.checksum_policy().to_le_bytes()); + // // ... other adapter-specific layout settings + // let digest = hasher.finalize(); + // let mut hash = [0u8; 16]; + // hash.copy_from_slice(&digest.as_bytes()[..16]); + // hash + // ``` + *b"QUIVER_SEGCFG\0\0\0" +} + #[cfg(test)] mod tests { use super::*; use crate::record_bundle::{ BundleDescriptor, PayloadRef, RecordBundle, SlotDescriptor, SlotId, }; + use crate::wal::WalReader; use arrow_schema::{DataType, Field, Schema}; use std::sync::Arc; + use tempfile::tempdir; struct DummyBundle { descriptor: BundleDescriptor, @@ -106,7 +165,9 @@ mod tests { #[test] fn ingest_is_currently_unimplemented() { - let engine = QuiverEngine::new(QuiverConfig::default()).expect("config valid"); + let temp_dir = tempdir().expect("tempdir"); + let config = QuiverConfig::default().with_data_dir(temp_dir.path()); + let engine = QuiverEngine::new(config).expect("config valid"); let bundle = DummyBundle::new(); let err = engine.ingest(&bundle).expect_err("not implemented"); assert!(matches!(err, QuiverError::Unimplemented { .. })); @@ -115,8 +176,9 @@ mod tests { #[test] fn config_returns_engine_configuration() { + let temp_dir = tempdir().expect("tempdir"); let config = QuiverConfig::builder() - .data_dir("./config_return_test") + .data_dir(temp_dir.path()) .build() .expect("builder should produce valid config"); let engine = QuiverEngine::new(config.clone()).expect("config valid"); @@ -124,6 +186,30 @@ mod tests { assert_eq!(engine.config(), &config); } + #[test] + fn ingest_appends_to_wal_before_placeholder_error() { + let temp_dir = tempdir().expect("tempdir"); + let config = QuiverConfig::default().with_data_dir(temp_dir.path()); + let engine = QuiverEngine::new(config).expect("config valid"); + let bundle = DummyBundle::new(); + + let err = engine + .ingest(&bundle) + .expect_err("segment still unimplemented"); + assert!(matches!(err, QuiverError::Unimplemented { .. })); + + drop(engine); + + let wal_path = temp_dir.path().join("wal").join("quiver.wal"); + let mut reader = WalReader::open(&wal_path).expect("wal opens"); + let mut iter = reader.iter_from(0).expect("iterator"); + let entry = iter.next().expect("entry exists").expect("entry decodes"); + + assert_eq!(entry.sequence, 0); + assert_eq!(entry.slots.len(), 1); + assert_eq!(entry.slot_bitmap.count_ones(), 1); + } + #[test] fn dummy_bundle_payload_handles_missing_slot() { let bundle = DummyBundle::new(); diff --git a/rust/otap-dataflow/crates/quiver/src/error.rs b/rust/otap-dataflow/crates/quiver/src/error.rs index dd3eb89b6..a6139bfb5 100644 --- a/rust/otap-dataflow/crates/quiver/src/error.rs +++ b/rust/otap-dataflow/crates/quiver/src/error.rs @@ -9,7 +9,7 @@ use std::borrow::Cow; pub type Result = std::result::Result; /// Errors that can be produced by Quiver APIs. -#[derive(Debug, thiserror::Error, PartialEq, Eq)] +#[derive(Debug, thiserror::Error)] pub enum QuiverError { /// Raised when a caller provides an invalid configuration. #[error("invalid configuration: {message}")] @@ -23,6 +23,13 @@ pub enum QuiverError { /// Context string identifying the missing component. context: &'static str, }, + /// Wrapper for WAL-specific failures. + #[error("wal error: {source}")] + Wal { + /// Underlying WAL error. + #[from] + source: crate::wal::WalError, + }, } impl QuiverError { diff --git a/rust/otap-dataflow/crates/quiver/src/lib.rs b/rust/otap-dataflow/crates/quiver/src/lib.rs index df33a1363..b8c96df62 100644 --- a/rust/otap-dataflow/crates/quiver/src/lib.rs +++ b/rust/otap-dataflow/crates/quiver/src/lib.rs @@ -11,9 +11,13 @@ //! # Example //! ``` //! use quiver::{engine::QuiverEngine, config::QuiverConfig}; +//! use tempfile::tempdir; //! //! # fn main() -> Result<(), Box> { -//! let engine = QuiverEngine::new(QuiverConfig::default())?; +//! let tmp = tempdir()?; +//! let cfg = QuiverConfig::default().with_data_dir(tmp.path()); +//! let engine = QuiverEngine::new(cfg)?; +//! assert_eq!(engine.config().data_dir, tmp.path()); //! assert_eq!(engine.config().segment.target_size_bytes.get(), 32 * 1024 * 1024); //! # Ok(()) //! # } @@ -24,7 +28,9 @@ pub mod engine; pub mod error; pub mod record_bundle; pub mod telemetry; +pub(crate) mod wal; pub use config::{QuiverConfig, RetentionConfig, SegmentConfig, WalConfig}; pub use engine::QuiverEngine; pub use error::{QuiverError, Result}; +pub use wal::WalError; diff --git a/rust/otap-dataflow/crates/quiver/src/wal/checkpoint_sidecar.rs b/rust/otap-dataflow/crates/quiver/src/wal/checkpoint_sidecar.rs new file mode 100644 index 000000000..97544f299 --- /dev/null +++ b/rust/otap-dataflow/crates/quiver/src/wal/checkpoint_sidecar.rs @@ -0,0 +1,211 @@ +// Copyright The OpenTelemetry Authors +// SPDX-License-Identifier: Apache-2.0 + +//! Crash-safe checkpoint offset persistence. +//! +//! The checkpoint sidecar is a tiny file (`checkpoint.offset`) that tracks how +//! much of the WAL has been durably consumed. It survives crashes so restarts +//! can resume from the last known safe offset without rescanning the entire log. +//! +//! # Format (24 bytes) +//! +//! ```text +//! ┌────────────┬─────────┬──────────┬─────────────────────┬──────────┐ +//! │ magic (8) │ ver (2) │ rsv (2) │ global_data_off (8) │ crc (4) │ +//! └────────────┴─────────┴──────────┴─────────────────────┴──────────┘ +//! ``` +//! +//! Writes use atomic rename (`write_to` → `.tmp` → `rename`) plus parent +//! directory fsync to ensure durability across power loss. + +use std::fs::OpenOptions; +use std::io::{Read, Write}; +use std::path::{Path, PathBuf}; + +use crc32fast::Hasher; + +use super::writer::sync_parent_dir; +use super::{WalError, WalResult}; + +#[cfg(test)] +use super::writer::test_support::{self as writer_test_support, CrashInjection}; + +pub(crate) const CHECKPOINT_SIDECAR_MAGIC: &[u8; 8] = b"QUIVER\0T"; +pub(crate) const CHECKPOINT_SIDECAR_VERSION: u16 = 1; +pub(crate) const CHECKPOINT_SIDECAR_LEN: usize = 8 + 2 + 2 + 8 + 4; + +/// On-disk metadata describing the consumer checkpoint in the logical stream. +#[derive(Debug, Clone, Copy, PartialEq, Eq)] +pub(crate) struct CheckpointSidecar { + /// Global data offset (excluding WAL headers) that has been durably + /// consumed by readers. + pub global_data_offset: u64, +} + +impl CheckpointSidecar { + pub fn new(global_data_offset: u64) -> Self { + Self { global_data_offset } + } + + pub fn encode(&self) -> [u8; CHECKPOINT_SIDECAR_LEN] { + let mut buf = [0u8; CHECKPOINT_SIDECAR_LEN]; + let mut cursor = 0; + buf[cursor..cursor + CHECKPOINT_SIDECAR_MAGIC.len()] + .copy_from_slice(CHECKPOINT_SIDECAR_MAGIC); + cursor += CHECKPOINT_SIDECAR_MAGIC.len(); + + buf[cursor..cursor + 2].copy_from_slice(&CHECKPOINT_SIDECAR_VERSION.to_le_bytes()); + cursor += 2; + + buf[cursor..cursor + 2].copy_from_slice(&0u16.to_le_bytes()); + cursor += 2; + + buf[cursor..cursor + 8].copy_from_slice(&self.global_data_offset.to_le_bytes()); + cursor += 8; + + let crc = compute_crc(&buf[..cursor]); + buf[cursor..cursor + 4].copy_from_slice(&crc.to_le_bytes()); + buf + } + + pub fn decode(buf: &[u8]) -> WalResult { + if buf.len() < CHECKPOINT_SIDECAR_LEN { + return Err(WalError::InvalidCheckpointSidecar("buffer too short")); + } + let mut cursor = 0; + if &buf[cursor..cursor + CHECKPOINT_SIDECAR_MAGIC.len()] != CHECKPOINT_SIDECAR_MAGIC { + return Err(WalError::InvalidCheckpointSidecar("magic mismatch")); + } + cursor += CHECKPOINT_SIDECAR_MAGIC.len(); + + let version = u16::from_le_bytes([buf[cursor], buf[cursor + 1]]); + cursor += 2; + if version != CHECKPOINT_SIDECAR_VERSION { + return Err(WalError::InvalidCheckpointSidecar("unsupported version")); + } + + if buf[cursor] != 0 || buf[cursor + 1] != 0 { + return Err(WalError::InvalidCheckpointSidecar("reserved bits non-zero")); + } + cursor += 2; + + let global_data_offset = u64::from_le_bytes([ + buf[cursor], + buf[cursor + 1], + buf[cursor + 2], + buf[cursor + 3], + buf[cursor + 4], + buf[cursor + 5], + buf[cursor + 6], + buf[cursor + 7], + ]); + cursor += 8; + + let stored_crc = u32::from_le_bytes([ + buf[cursor], + buf[cursor + 1], + buf[cursor + 2], + buf[cursor + 3], + ]); + let computed_crc = compute_crc(&buf[..cursor]); + if stored_crc != computed_crc { + return Err(WalError::InvalidCheckpointSidecar("crc mismatch")); + } + + Ok(Self { global_data_offset }) + } + + pub fn read_from(path: &Path) -> WalResult { + let mut file = OpenOptions::new().read(true).open(path)?; + let mut buf = [0u8; CHECKPOINT_SIDECAR_LEN]; + file.read_exact(&mut buf)?; + Self::decode(&buf) + } + + pub fn write_to(path: &Path, value: &Self) -> WalResult<()> { + let tmp_path = temporary_path(path); + { + let mut file = OpenOptions::new() + .create(true) + .write(true) + .truncate(true) + .open(&tmp_path)?; + let encoded = value.encode(); + file.write_all(&encoded)?; + file.flush()?; + file.sync_data()?; + } + #[cfg(test)] + if writer_test_support::take_crash(CrashInjection::BeforeSidecarRename) { + return Err(WalError::InjectedCrash( + "crash injected before checkpoint sidecar rename", + )); + } + std::fs::rename(&tmp_path, path)?; + sync_parent_dir(path)?; + Ok(()) + } +} + +fn compute_crc(buf: &[u8]) -> u32 { + let mut hasher = Hasher::new(); + hasher.update(buf); + hasher.finalize() +} + +fn temporary_path(path: &Path) -> PathBuf { + let mut tmp = path.as_os_str().to_owned(); + tmp.push(".tmp"); + PathBuf::from(tmp) +} + +#[cfg(test)] +mod tests { + use super::*; + use tempfile::tempdir; + + fn sample_sidecar() -> CheckpointSidecar { + CheckpointSidecar::new(128) + } + + #[test] + fn encode_decode_roundtrip() { + let value = sample_sidecar(); + let encoded = value.encode(); + let decoded = CheckpointSidecar::decode(&encoded).expect("decode"); + assert_eq!(decoded, value); + } + + #[test] + fn decode_rejects_magic_mismatch() { + let mut encoded = sample_sidecar().encode(); + encoded[0] ^= 0xFF; + let err = CheckpointSidecar::decode(&encoded).unwrap_err(); + assert!(matches!( + err, + WalError::InvalidCheckpointSidecar("magic mismatch") + )); + } + + #[test] + fn decode_rejects_crc_mismatch() { + let mut encoded = sample_sidecar().encode(); + let last = encoded.len() - 1; + encoded[last] ^= 0xFF; + let err = CheckpointSidecar::decode(&encoded).unwrap_err(); + assert!(matches!( + err, + WalError::InvalidCheckpointSidecar("crc mismatch") + )); + } + + #[test] + fn write_and_read_sidecar() { + let dir = tempdir().expect("tempdir"); + let path = dir.path().join("checkpoint.offset"); + let value = sample_sidecar(); + CheckpointSidecar::write_to(&path, &value).expect("write"); + let loaded = CheckpointSidecar::read_from(&path).expect("read"); + assert_eq!(loaded, value); + } +} diff --git a/rust/otap-dataflow/crates/quiver/src/wal/header.rs b/rust/otap-dataflow/crates/quiver/src/wal/header.rs new file mode 100644 index 000000000..e66a0a34a --- /dev/null +++ b/rust/otap-dataflow/crates/quiver/src/wal/header.rs @@ -0,0 +1,156 @@ +// Copyright The OpenTelemetry Authors +// SPDX-License-Identifier: Apache-2.0 + +//! WAL file header encoding and validation. +//! +//! Every WAL file starts with a fixed 30-byte header: +//! +//! ```text +//! ┌────────────┬─────────┬──────────┬──────────────────┐ +//! │ magic (10) │ ver (2) │ rsv (2) │ segment_hash (16)│ +//! └────────────┴─────────┴──────────┴──────────────────┘ +//! ``` +//! +//! - **magic**: `b"QUIVER\0WAL"` identifies the file type +//! - **version**: Format version (currently 1) +//! - **reserved**: Zero padding for future use +//! - **segment_hash**: MD5 of segment configuration; mismatches reject the file + +use std::io::{Read, Seek, SeekFrom, Write}; + +use super::{WAL_MAGIC, WalError}; + +pub(crate) const WAL_VERSION: u16 = 1; +pub(crate) const WAL_HEADER_LEN: usize = WAL_MAGIC.len() + 2 + 2 + 16; + +#[derive(Clone, Copy, Debug, PartialEq, Eq)] +pub(crate) struct WalHeader { + pub segment_cfg_hash: [u8; 16], +} + +impl WalHeader { + pub fn new(segment_cfg_hash: [u8; 16]) -> Self { + Self { segment_cfg_hash } + } + + pub fn encode(&self) -> [u8; WAL_HEADER_LEN] { + let mut buf = [0u8; WAL_HEADER_LEN]; + let mut cursor = 0; + buf[cursor..cursor + WAL_MAGIC.len()].copy_from_slice(WAL_MAGIC); + cursor += WAL_MAGIC.len(); + + buf[cursor..cursor + 2].copy_from_slice(&WAL_VERSION.to_le_bytes()); + cursor += 2; + + buf[cursor..cursor + 2].copy_from_slice(&0u16.to_le_bytes()); + cursor += 2; + + buf[cursor..cursor + 16].copy_from_slice(&self.segment_cfg_hash); + buf + } + + pub fn write_to(&self, file: &mut (impl Write + Seek)) -> Result<(), WalError> { + let _ = file.seek(SeekFrom::Start(0))?; + file.write_all(&self.encode())?; + file.flush()?; + Ok(()) + } + + pub fn read_from(file: &mut (impl Read + Seek)) -> Result { + let _ = file.seek(SeekFrom::Start(0))?; + let mut buf = [0u8; WAL_HEADER_LEN]; + file.read_exact(&mut buf)?; + Self::decode(&buf) + } + + pub fn decode(buf: &[u8]) -> Result { + if buf.len() < WAL_HEADER_LEN { + return Err(WalError::InvalidHeader("buffer too short")); + } + let mut cursor = 0; + if &buf[cursor..cursor + WAL_MAGIC.len()] != WAL_MAGIC { + return Err(WalError::InvalidHeader("magic mismatch")); + } + cursor += WAL_MAGIC.len(); + + let version = u16::from_le_bytes([buf[cursor], buf[cursor + 1]]); + cursor += 2; + if version != WAL_VERSION { + return Err(WalError::InvalidHeader("unsupported version")); + } + + // Reserved field; must be zero today. + if buf[cursor] != 0 || buf[cursor + 1] != 0 { + return Err(WalError::InvalidHeader("reserved bits non-zero")); + } + cursor += 2; + + let mut segment_cfg_hash = [0u8; 16]; + segment_cfg_hash.copy_from_slice(&buf[cursor..cursor + 16]); + + Ok(Self { segment_cfg_hash }) + } +} + +#[cfg(test)] +mod tests { + use super::*; + + fn sample_hash() -> [u8; 16] { + let mut hash = [0u8; 16]; + for (idx, byte) in hash.iter_mut().enumerate() { + *byte = (idx as u8).wrapping_mul(3).wrapping_add(1); + } + hash + } + + #[test] + fn encode_decode_roundtrip_preserves_segment_hash() { + let header = WalHeader::new(sample_hash()); + let encoded = header.encode(); + let decoded = WalHeader::decode(&encoded).expect("decode should succeed"); + assert_eq!(decoded.segment_cfg_hash, sample_hash()); + } + + #[test] + fn decode_rejects_magic_mismatch() { + let header = WalHeader::new(sample_hash()); + let mut encoded = header.encode(); + encoded[0] ^= 0xFF; + let err = WalHeader::decode(&encoded).unwrap_err(); + assert!(matches!(err, WalError::InvalidHeader("magic mismatch"))); + } + + #[test] + fn decode_rejects_reserved_bits() { + let header = WalHeader::new(sample_hash()); + let mut encoded = header.encode(); + let reserved_start = WAL_MAGIC.len() + 2; + encoded[reserved_start] = 1; + let err = WalHeader::decode(&encoded).unwrap_err(); + assert!(matches!( + err, + WalError::InvalidHeader("reserved bits non-zero") + )); + } + + #[test] + fn decode_rejects_short_buffer() { + let mut buf = vec![0u8; WAL_HEADER_LEN - 1]; + buf.copy_from_slice(&WalHeader::new(sample_hash()).encode()[..WAL_HEADER_LEN - 1]); + let err = WalHeader::decode(&buf).unwrap_err(); + assert!(matches!(err, WalError::InvalidHeader("buffer too short"))); + } + + #[test] + fn decode_rejects_unsupported_version() { + let mut encoded = WalHeader::new(sample_hash()).encode(); + let version_idx = WAL_MAGIC.len(); + encoded[version_idx..version_idx + 2].copy_from_slice(&2u16.to_le_bytes()); + let err = WalHeader::decode(&encoded).unwrap_err(); + assert!(matches!( + err, + WalError::InvalidHeader("unsupported version") + )); + } +} diff --git a/rust/otap-dataflow/crates/quiver/src/wal/mod.rs b/rust/otap-dataflow/crates/quiver/src/wal/mod.rs new file mode 100644 index 000000000..25364e1b0 --- /dev/null +++ b/rust/otap-dataflow/crates/quiver/src/wal/mod.rs @@ -0,0 +1,195 @@ +// Copyright The OpenTelemetry Authors +// SPDX-License-Identifier: Apache-2.0 + +//! Write-Ahead Log (WAL) for Quiver crash recovery. +//! +//! This module provides durable, append-only storage for Arrow record batches. +//! On crash, the WAL replays uncommitted data to restore in-memory state. +//! +//! # Quick Start +//! +//! ```ignore +//! // Writing +//! let mut writer = WalWriter::open(options)?; +//! let offset = writer.append_bundle(&bundle)?; +//! +//! // Reading (for replay) +//! let mut reader = WalReader::open(&path)?; +//! let mut cursor = WalConsumerCheckpoint::default(); +//! for entry in reader.iter_from(0)? { +//! let bundle = entry?; +//! // ... rebuild state from bundle ... +//! cursor.increment(&bundle); // in-memory only +//! } +//! +//! // Checkpointing (after downstream confirms durability) +//! writer.checkpoint_cursor(&cursor)?; // persists + enables cleanup +//! ``` +//! +//! # Module Organization +//! +//! | File | Purpose | +//! |-------------------------|--------------------------------------------------| +//! | `writer.rs` | Append entries, rotate files, manage checkpoints | +//! | `reader.rs` | Iterate entries, decode payloads, track progress | +//! | `header.rs` | WAL file header format (magic, version, config) | +//! | `checkpoint_sidecar.rs` | Crash-safe checkpoint offset persistence | +//! | `tests.rs` | Integration tests and crash simulation | +//! +//! # On-Disk Layout +//! +//! ```text +//! wal/ +//! ├── quiver.wal # Active WAL file (append target) +//! ├── quiver.wal.1 # Rotated file (oldest) +//! ├── quiver.wal.2 # Rotated file +//! └── checkpoint.offset # Consumer progress (24 bytes, CRC-protected) +//! ``` +//! +//! # Key Concepts +//! +//! - **Entry**: One [`RecordBundle`] serialized with CRC32 integrity check +//! - **Rotation**: When the active file exceeds `rotation_target_bytes`, it's +//! renamed to `quiver.wal.N` and a fresh file starts +//! - **Checkpoint**: Consumers call [`WalConsumerCheckpoint::increment()`] while +//! iterating (in-memory), then [`WalWriter::checkpoint_cursor()`] to persist +//! - **Purge**: Rotated files are deleted once fully covered by the checkpoint +//! +//! See [`writer`] module docs for detailed lifecycle documentation. + +use std::io; + +use arrow_schema::ArrowError; +use thiserror::Error; + +use crate::record_bundle::SlotId; + +mod checkpoint_sidecar; +mod header; +mod reader; +#[cfg(test)] +mod tests; +mod writer; + +// Keep reader exports visible even though only tests consume them today. +#[allow(unused_imports)] +pub(crate) use reader::{DecodedWalSlot, WalConsumerCheckpoint, WalReader, WalRecordBundle}; +// Writer is used broadly soon; suppress warnings while integration lands. +#[allow(unused_imports)] +pub(crate) use writer::{FlushPolicy, WalOffset, WalWriter, WalWriterOptions}; + +// ───────────────────────────────────────────────────────────────────────────── +// WAL Format Constants +// +// See ARCHITECTURE.md § "Write-Ahead Log" for the full on-disk layout. +// ───────────────────────────────────────────────────────────────────────────── + +/// Magic bytes identifying a Quiver WAL file. +/// +/// The file header starts with these 10 bytes: `b"QUIVER\0WAL"`. +/// See ARCHITECTURE.md: "File header: fixed-width preamble (`b\"QUIVER\\0WAL\"`)" +pub(crate) const WAL_MAGIC: &[u8; 10] = b"QUIVER\0WAL"; + +/// Entry type marker for a serialized [`RecordBundle`]. +/// +/// Currently the only defined entry type. Future versions may add additional +/// types (e.g., for schema evolution or control records). +/// See ARCHITECTURE.md: "Entry header (`u8 entry_type`, currently `0 = RecordBundle`)" +pub(crate) const ENTRY_TYPE_RECORD_BUNDLE: u8 = 0; + +/// Size of the entry header in bytes: `entry_type(1) + timestamp(8) + sequence(8) + bitmap(8)`. +/// +/// Layout: `{ u8 entry_type, i64 ingestion_ts_nanos, u64 per_core_sequence, u64 slot_bitmap }` +/// See ARCHITECTURE.md § "Framed entries" for the complete entry structure. +pub(crate) const ENTRY_HEADER_LEN: usize = 1 + 8 + 8 + 8; + +/// Size of a schema fingerprint (BLAKE3 truncated to 256 bits). +pub(crate) const SCHEMA_FINGERPRINT_LEN: usize = 32; + +/// Size of per-slot metadata: `payload_type_id(2) + fingerprint(32) + row_count(4) + payload_len(4)`. +/// +/// Layout: `{ u16 payload_type_id, [u8;32] schema_fingerprint, u32 row_count, u32 payload_len }` +/// See ARCHITECTURE.md § "Framed entries" → SlotMeta block. +pub(crate) const SLOT_HEADER_LEN: usize = 2 + SCHEMA_FINGERPRINT_LEN + 4 + 4; + +/// Maximum allowed rotation target size (256 MiB). +/// +/// This bounds the maximum size of any single WAL file and, by extension, the +/// maximum size of any single entry. Both writer and reader enforce this limit: +/// - Writer clamps `rotation_target_bytes` to this value +/// - Reader rejects entries exceeding this size +/// +/// 256 MiB is generous for telemetry workloads while preventing excessive +/// memory allocation from corrupted or malicious WAL files. +pub(crate) const MAX_ROTATION_TARGET_BYTES: u64 = 256 * 1024 * 1024; + +pub(crate) type WalResult = Result; + +/// Errors produced while reading or writing WAL data. +/// +/// Most variants include context about where the failure occurred. +/// [`WalError::Io`] wraps underlying filesystem errors. +#[derive(Error, Debug)] +pub enum WalError { + /// Underlying filesystem failure. + #[error("wal io error: {0}")] + Io(#[from] io::Error), + /// File header contained unexpected bytes. + #[error("invalid wal header: {0}")] + InvalidHeader(&'static str), + /// Slot id exceeded the current bitmap encoding. + #[error("slot id {0:?} is out of supported bitmap range (>= 64)")] + SlotOutOfRange(SlotId), + /// Payload row count cannot be encoded as `u32`. + #[error("row count {0} exceeds u32::MAX")] + RowCountOverflow(usize), + /// Serialized payload exceeds allowed size. + #[error("payload length {0} exceeds u32::MAX")] + PayloadTooLarge(usize), + /// Entry body is larger than the framing supports. + #[error("entry body length {0} exceeds u32::MAX")] + EntryTooLarge(usize), + /// Ingestion timestamp could not be normalized. + #[error("invalid ingestion timestamp")] + InvalidTimestamp, + /// Encountered an unexpected EOF while parsing. + #[error("wal truncated while reading {0}")] + UnexpectedEof(&'static str), + /// CRC mismatch detected during validation. + #[error("wal crc mismatch: stored {stored:#010x} computed {computed:#010x}")] + CrcMismatch { + /// CRC value persisted alongside the entry. + stored: u32, + /// CRC recomputed from the decoded entry. + computed: u32, + }, + /// Entry type not supported by this build. + #[error("unsupported wal entry type {0}")] + UnsupportedEntry(u8), + /// Entry body failed structural validation. + #[error("invalid wal entry: {0}")] + InvalidEntry(&'static str), + /// Existing WAL header does not match expected segment configuration. + #[error("segment config mismatch: expected {expected:02x?}, found {found:02x?}")] + SegmentConfigMismatch { + /// The hash the caller expected. + expected: [u8; 16], + /// The hash stored in the WAL header. + found: [u8; 16], + }, + /// Checkpoint sidecar contains invalid or corrupted bytes. + #[error("invalid checkpoint sidecar: {0}")] + InvalidCheckpointSidecar(&'static str), + /// Consumer checkpoint failed validation. + #[error("invalid consumer checkpoint: {0}")] + InvalidConsumerCheckpoint(&'static str), + /// Arrow serialization/deserialization failure. + #[error("arrow serialization error: {0}")] + Arrow(#[from] ArrowError), + /// Writer cannot proceed because configured capacity limits were reached. + #[error("wal at capacity: {0}")] + WalAtCapacity(&'static str), + /// Test-only failure that simulates a crash at a specific point. + #[error("wal crash injected: {0}")] + InjectedCrash(&'static str), +} diff --git a/rust/otap-dataflow/crates/quiver/src/wal/reader.rs b/rust/otap-dataflow/crates/quiver/src/wal/reader.rs new file mode 100644 index 000000000..9b46ea464 --- /dev/null +++ b/rust/otap-dataflow/crates/quiver/src/wal/reader.rs @@ -0,0 +1,486 @@ +// Copyright The OpenTelemetry Authors +// SPDX-License-Identifier: Apache-2.0 + +//! Read-side companion to the WAL writer. +//! +//! The reader validates headers, streams entries starting at arbitrary offsets, +//! and exposes helper types for tracking replay progress. +//! +//! # Entry Format +//! +//! Each WAL entry has this layout: +//! +//! ```text +//! ┌──────────┬────────────────┬─────────────────┬──────────┐ +//! │ len (4) │ entry_hdr (25) │ slot_data (var) │ crc (4) │ +//! └──────────┴────────────────┴─────────────────┴──────────┘ +//! ``` +//! +//! - **len**: Size of `entry_hdr + slot_data` (excludes len and crc fields) +//! - **entry_hdr**: Type (1), timestamp (8), sequence (8), slot_bitmap (8) +//! - **slot_data**: For each set bit: slot_id (2), fingerprint (32), rows (4), +//! payload_len (4), Arrow IPC bytes (payload_len) +//! - **crc**: CRC32 over `entry_hdr + slot_data` +//! +//! # Usage +//! +//! ```ignore +//! let mut reader = WalReader::open("wal/quiver.wal")?; +//! let mut cursor = WalConsumerCheckpoint::default(); +//! +//! for result in reader.iter_from(0)? { +//! let bundle = result?; +//! println!("seq={} slots={}", bundle.sequence, bundle.slots.len()); +//! cursor.increment(&bundle); // in-memory only +//! } +//! // cursor now points past the last entry +//! // call writer.checkpoint_cursor(&cursor) to persist +//! ``` +#![allow(dead_code)] + +#[cfg(test)] +use self::test_support::ReadFailure; +use std::fs::File; +use std::io::{ErrorKind, Read, Seek, SeekFrom}; +use std::path::{Path, PathBuf}; + +use crc32fast::Hasher; + +use crate::record_bundle::{SchemaFingerprint, SlotId}; + +use super::header::{WAL_HEADER_LEN, WalHeader}; +use super::{ + ENTRY_HEADER_LEN, ENTRY_TYPE_RECORD_BUNDLE, MAX_ROTATION_TARGET_BYTES, SCHEMA_FINGERPRINT_LEN, + SLOT_HEADER_LEN, WalError, WalOffset, WalResult, +}; + +/// Maximum allowed entry size, derived from [`MAX_ROTATION_TARGET_BYTES`]. +/// +/// Since entries cannot span files and files are capped at 256 MiB, no valid +/// entry can exceed this size. The reader rejects larger entries to guard +/// against corrupted or malicious WAL files causing excessive allocation. +const MAX_ENTRY_SIZE: usize = MAX_ROTATION_TARGET_BYTES as usize; + +/// Sequential reader that validates the WAL header before exposing iterators +/// over decoded entries. +#[derive(Debug)] +pub(crate) struct WalReader { + file: File, + path: PathBuf, + segment_cfg_hash: [u8; 16], +} + +impl WalReader { + pub fn open(path: impl Into) -> WalResult { + let path = path.into(); + let mut file = File::open(&path)?; + let header = WalHeader::read_from(&mut file)?; + let _ = file.seek(SeekFrom::Start(WAL_HEADER_LEN as u64))?; + + Ok(Self { + file, + path, + segment_cfg_hash: header.segment_cfg_hash, + }) + } + + pub fn path(&self) -> &Path { + &self.path + } + + pub fn segment_cfg_hash(&self) -> [u8; 16] { + self.segment_cfg_hash + } + + /// Returns an iterator that starts at `offset`, clamped to the minimum + /// position right after the WAL header. + pub fn iter_from(&mut self, offset: u64) -> WalResult> { + let start = offset.max(WAL_HEADER_LEN as u64); + let file_len = self.file.metadata()?.len(); + let _ = self.file.seek(SeekFrom::Start(start))?; + Ok(WalEntryIter::new(&mut self.file, start, file_len)) + } + + /// Seeks back to the entry immediately after the header so a fresh scan can + /// start from the beginning. + pub fn rewind(&mut self) -> WalResult<()> { + let _ = self.file.seek(SeekFrom::Start(WAL_HEADER_LEN as u64))?; + Ok(()) + } +} + +/// Iterator that yields decoded [`WalRecordBundle`] instances while keeping +/// track of the next byte offset so callers can build consumer checkpoints. +pub(crate) struct WalEntryIter<'a> { + file: &'a mut File, + buffer: Vec, + next_offset: u64, + /// Known file size at iterator creation, used to reject impossibly large entries. + file_len: u64, + finished: bool, +} + +impl<'a> WalEntryIter<'a> { + fn new(file: &'a mut File, offset: u64, file_len: u64) -> Self { + Self { + file, + buffer: Vec::new(), + next_offset: offset, + file_len, + finished: false, + } + } +} + +impl<'a> Iterator for WalEntryIter<'a> { + type Item = WalResult; + + fn next(&mut self) -> Option { + if self.finished { + return None; + } + + let entry_start = self.next_offset; + let mut len_buf = [0u8; 4]; + match read_exact_or_eof(self.file, &mut len_buf) { + Ok(ReadStatus::Eof) => { + self.finished = true; + return None; + } + Ok(ReadStatus::Filled) => {} + Err(err) => { + self.finished = true; + return Some(Err(err)); + } + } + + let entry_len = u32::from_le_bytes(len_buf) as usize; + + // Guard against malicious or corrupted length values. Check the hard + // cap first (avoids allocation bombs even with huge files), then verify + // against actual remaining bytes. + if entry_len > MAX_ENTRY_SIZE { + self.finished = true; + return Some(Err(WalError::InvalidEntry( + "entry length exceeds maximum allowed size", + ))); + } + + let remaining_bytes = self.file_len.saturating_sub(entry_start + 4) as usize; + if entry_len > remaining_bytes { + self.finished = true; + return Some(Err(WalError::InvalidEntry( + "entry length exceeds remaining file size", + ))); + } + + self.buffer.resize(entry_len, 0); + if let Err(err) = read_entry_body(self.file, &mut self.buffer) { + self.finished = true; + let wal_err = if err.kind() == ErrorKind::UnexpectedEof { + WalError::UnexpectedEof("entry body") + } else { + err.into() + }; + return Some(Err(wal_err)); + } + + let mut crc_buf = [0u8; 4]; + if let Err(err) = read_entry_crc(self.file, &mut crc_buf) { + self.finished = true; + let wal_err = if err.kind() == ErrorKind::UnexpectedEof { + WalError::UnexpectedEof("entry crc") + } else { + err.into() + }; + return Some(Err(wal_err)); + } + + let stored_crc = u32::from_le_bytes(crc_buf); + let mut hasher = Hasher::new(); + hasher.update(&self.buffer); + let computed_crc = hasher.finalize(); + if stored_crc != computed_crc { + self.finished = true; + return Some(Err(WalError::CrcMismatch { + stored: stored_crc, + computed: computed_crc, + })); + } + + let next_offset = entry_start + .saturating_add(4) + .saturating_add(entry_len as u64) + .saturating_add(4); + + self.next_offset = next_offset; + + match decode_entry(entry_start, next_offset, &self.buffer) { + Ok(entry) => Some(Ok(entry)), + Err(err) => { + self.finished = true; + Some(Err(err)) + } + } + } +} + +/// Fully decoded WAL entry that the engine can replay without touching the raw +/// on-disk representation. +#[derive(Debug, Clone)] +pub(crate) struct WalRecordBundle { + pub offset: WalOffset, + pub next_offset: u64, + pub ingestion_ts_nanos: i64, + pub sequence: u64, + pub slot_bitmap: u64, + pub slots: Vec, +} + +/// Arrow payload captured for a single slot inside a WAL entry. +#[derive(Debug, Clone)] +pub(crate) struct DecodedWalSlot { + pub slot_id: SlotId, + pub schema_fingerprint: SchemaFingerprint, + pub row_count: u32, + pub payload_len: u32, + pub payload: Vec, +} + +/// Opaque cursor describing how much of the WAL has been seen. +/// +/// This is an **in-memory only** cursor. Updating it has no durability effect. +/// To persist progress and allow WAL cleanup, pass the cursor to +/// [`WalWriter::checkpoint_cursor()`]. +/// +/// Usage: +/// 1. Start with `WalConsumerCheckpoint::default()` (beginning of WAL) +/// 2. Call `cursor.increment(&bundle)` after processing each entry (in-memory) +/// 3. Call `writer.checkpoint_cursor(&cursor)` to persist and enable cleanup +/// +/// The writer validates that checkpoints land on entry boundaries and rejects +/// stale or regressed values. +#[derive(Debug, Clone, Copy, Default)] +pub(crate) struct WalConsumerCheckpoint { + /// Internal: byte position within the current WAL file. + pub(super) safe_offset: u64, + /// Internal: monotonic sequence number for ordering. + pub(super) safe_sequence: u64, +} + +impl WalConsumerCheckpoint { + /// Creates a checkpoint positioned immediately after the given bundle. + /// + /// This is equivalent to `WalConsumerCheckpoint::default()` followed by + /// `increment(bundle)`, but clearer when you only need to checkpoint a + /// single entry. + pub fn after(bundle: &WalRecordBundle) -> Self { + Self { + safe_offset: bundle.next_offset, + safe_sequence: bundle.sequence, + } + } + + /// Moves the cursor past the provided bundle (in-memory only). + /// + /// This does **not** persist the checkpoint or trigger WAL cleanup. + /// Call [`WalWriter::checkpoint_cursor()`] to make progress durable. + /// + /// Typical usage in a replay loop: + /// ```ignore + /// let mut cursor = WalConsumerCheckpoint::default(); + /// for bundle in reader.iter_from(0)? { + /// process(&bundle?); + /// cursor.increment(&bundle); // in-memory only + /// } + /// writer.checkpoint_cursor(&cursor)?; // persist + cleanup + /// ``` + pub fn increment(&mut self, bundle: &WalRecordBundle) { + self.safe_offset = bundle.next_offset; + self.safe_sequence = bundle.sequence; + } +} + +enum ReadStatus { + Filled, + Eof, +} + +fn read_exact_or_eof(file: &mut File, buf: &mut [u8]) -> WalResult { + let mut read = 0; + while read < buf.len() { + match read_length_chunk(file, &mut buf[read..]) { + Ok(0) if read == 0 => return Ok(ReadStatus::Eof), + Ok(0) => return Err(WalError::UnexpectedEof("entry length")), + Ok(n) => read += n, + Err(err) if err.kind() == ErrorKind::Interrupted => continue, + Err(err) => return Err(err.into()), + } + } + Ok(ReadStatus::Filled) +} + +fn read_length_chunk(file: &mut File, buf: &mut [u8]) -> std::io::Result { + #[cfg(test)] + if let Some(err) = test_support::take_failure(ReadFailure::Length) { + return Err(err); + } + file.read(buf) +} + +fn read_entry_body(file: &mut File, buffer: &mut [u8]) -> std::io::Result<()> { + #[cfg(test)] + if let Some(err) = test_support::take_failure(ReadFailure::Body) { + return Err(err); + } + file.read_exact(buffer) +} + +fn read_entry_crc(file: &mut File, buffer: &mut [u8; 4]) -> std::io::Result<()> { + #[cfg(test)] + if let Some(err) = test_support::take_failure(ReadFailure::Crc) { + return Err(err); + } + file.read_exact(buffer) +} + +fn decode_entry(entry_start: u64, next_offset: u64, body: &[u8]) -> WalResult { + if body.len() < ENTRY_HEADER_LEN { + return Err(WalError::InvalidEntry("body shorter than header")); + } + + let mut cursor = 0; + let entry_type = body[cursor]; + cursor += 1; + if entry_type != ENTRY_TYPE_RECORD_BUNDLE { + return Err(WalError::UnsupportedEntry(entry_type)); + } + + let ingestion_ts_nanos = read_i64(body, &mut cursor, "ingestion timestamp")?; + let sequence = read_u64(body, &mut cursor, "sequence")?; + let slot_bitmap = read_u64(body, &mut cursor, "slot bitmap")?; + + let expected_slots = slot_bitmap.count_ones() as usize; + let mut slots = Vec::with_capacity(expected_slots); + + for _ in 0..expected_slots { + if cursor + SLOT_HEADER_LEN > body.len() { + return Err(WalError::InvalidEntry("slot header truncated")); + } + + let slot_id = SlotId(read_u16(body, &mut cursor, "slot id")?); + + let mut schema_fingerprint = [0u8; SCHEMA_FINGERPRINT_LEN]; + schema_fingerprint.copy_from_slice(slice_bytes( + body, + &mut cursor, + SCHEMA_FINGERPRINT_LEN, + "schema fingerprint", + )?); + + let row_count = read_u32(body, &mut cursor, "row count")?; + let payload_len = read_u32(body, &mut cursor, "payload length")?; + let payload_len_usize = usize::try_from(payload_len) + .map_err(|_| WalError::InvalidEntry("payload length overflow"))?; + + let payload = slice_bytes(body, &mut cursor, payload_len_usize, "slot payload")?.to_vec(); + + slots.push(DecodedWalSlot { + slot_id, + schema_fingerprint, + row_count, + payload_len, + payload, + }); + } + + if cursor != body.len() { + return Err(WalError::InvalidEntry("unexpected trailing bytes")); + } + + Ok(WalRecordBundle { + offset: WalOffset { + position: entry_start, + sequence, + }, + next_offset, + ingestion_ts_nanos, + sequence, + slot_bitmap, + slots, + }) +} + +fn slice_bytes<'a>( + body: &'a [u8], + cursor: &mut usize, + len: usize, + ctx: &'static str, +) -> WalResult<&'a [u8]> { + if *cursor + len > body.len() { + return Err(WalError::InvalidEntry(ctx)); + } + let slice = &body[*cursor..*cursor + len]; + *cursor += len; + Ok(slice) +} + +fn read_u16(body: &[u8], cursor: &mut usize, ctx: &'static str) -> WalResult { + let mut bytes = [0u8; 2]; + bytes.copy_from_slice(slice_bytes(body, cursor, 2, ctx)?); + Ok(u16::from_le_bytes(bytes)) +} + +fn read_u32(body: &[u8], cursor: &mut usize, ctx: &'static str) -> WalResult { + let mut bytes = [0u8; 4]; + bytes.copy_from_slice(slice_bytes(body, cursor, 4, ctx)?); + Ok(u32::from_le_bytes(bytes)) +} + +fn read_u64(body: &[u8], cursor: &mut usize, ctx: &'static str) -> WalResult { + let mut bytes = [0u8; 8]; + bytes.copy_from_slice(slice_bytes(body, cursor, 8, ctx)?); + Ok(u64::from_le_bytes(bytes)) +} + +fn read_i64(body: &[u8], cursor: &mut usize, ctx: &'static str) -> WalResult { + let mut bytes = [0u8; 8]; + bytes.copy_from_slice(slice_bytes(body, cursor, 8, ctx)?); + Ok(i64::from_le_bytes(bytes)) +} + +#[cfg(test)] +pub(super) mod test_support { + use std::cell::Cell; + use std::io::Error; + + #[derive(Clone, Copy, Debug, PartialEq, Eq)] + pub enum ReadFailure { + Length, + Body, + Crc, + } + + thread_local! { + static NEXT_FAILURE: Cell> = const { Cell::new(None) }; + } + + pub fn fail_next_read(stage: ReadFailure) { + NEXT_FAILURE.with(|slot| slot.set(Some(stage))); + } + + pub fn take_failure(stage: ReadFailure) -> Option { + NEXT_FAILURE.with(|slot| { + if slot.get() == Some(stage) { + slot.set(None); + Some(Error::other("wal reader injected read failure")) + } else { + None + } + }) + } + + pub fn reset_failures() { + NEXT_FAILURE.with(|slot| slot.set(None)); + } +} diff --git a/rust/otap-dataflow/crates/quiver/src/wal/tests.rs b/rust/otap-dataflow/crates/quiver/src/wal/tests.rs new file mode 100644 index 000000000..95a3a85c9 --- /dev/null +++ b/rust/otap-dataflow/crates/quiver/src/wal/tests.rs @@ -0,0 +1,2466 @@ +// Copyright The OpenTelemetry Authors +// SPDX-License-Identifier: Apache-2.0 + +//! Cross-cutting WAL tests live here so shared fixtures can touch writer, reader, +//! and helper plumbing without sprinkling large #[cfg(test)] blocks in each file. + +use std::cmp; +use std::io::{Cursor, Read, Seek, SeekFrom, Write}; +use std::path::{Path, PathBuf}; +use std::sync::Arc; +use std::time::{Duration, Instant, SystemTime, UNIX_EPOCH}; + +use arrow_array::{Int64Array, RecordBatch, builder::StringBuilder}; +use arrow_ipc::reader::StreamReader; +use arrow_schema::{DataType, Field, Schema}; +use crc32fast::Hasher; +use tempfile::tempdir; + +use crate::record_bundle::{ + BundleDescriptor, PayloadRef, RecordBundle, SchemaFingerprint, SlotDescriptor, SlotId, +}; + +use super::checkpoint_sidecar::{CHECKPOINT_SIDECAR_LEN, CheckpointSidecar}; +use super::header::{WAL_HEADER_LEN, WalHeader}; +use super::reader::test_support::{self, ReadFailure}; +use super::writer::FlushPolicy; +use super::writer::test_support as writer_test_support; +use super::{ + ENTRY_HEADER_LEN, ENTRY_TYPE_RECORD_BUNDLE, SCHEMA_FINGERPRINT_LEN, WalConsumerCheckpoint, + WalError, WalReader, WalWriter, WalWriterOptions, +}; + +struct FixtureSlot { + id: SlotId, + fingerprint: SchemaFingerprint, + batch: RecordBatch, +} + +impl FixtureSlot { + fn new(id: SlotId, fingerprint_seed: u8, values: &[i64]) -> Self { + let fingerprint = [fingerprint_seed; 32]; + let batch = build_batch(values); + Self { + id, + fingerprint, + batch, + } + } + + fn with_batch(id: SlotId, fingerprint_seed: u8, batch: RecordBatch) -> Self { + let fingerprint = [fingerprint_seed; 32]; + Self { + id, + fingerprint, + batch, + } + } +} + +struct FixtureBundle { + descriptor: BundleDescriptor, + ingestion_time: SystemTime, + slots: Vec, +} + +impl FixtureBundle { + fn new(descriptor: BundleDescriptor, slots: Vec) -> Self { + Self { + descriptor, + ingestion_time: UNIX_EPOCH + Duration::from_secs(42), + slots, + } + } + + fn with_ingestion_time(mut self, ts: SystemTime) -> Self { + self.ingestion_time = ts; + self + } +} + +impl RecordBundle for FixtureBundle { + fn descriptor(&self) -> &BundleDescriptor { + &self.descriptor + } + + fn ingestion_time(&self) -> SystemTime { + self.ingestion_time + } + + fn payload(&self, slot: SlotId) -> Option> { + self.slots + .iter() + .find(|s| s.id == slot) + .map(|slot| PayloadRef { + schema_fingerprint: slot.fingerprint, + batch: &slot.batch, + }) + } +} + +fn build_batch(values: &[i64]) -> RecordBatch { + let schema = Arc::new(Schema::new(vec![Field::new( + "value", + DataType::Int64, + false, + )])); + let array = Int64Array::from(values.to_vec()); + RecordBatch::try_new(schema, vec![Arc::new(array)]).expect("valid batch") +} + +fn read_batch(bytes: &[u8]) -> RecordBatch { + let cursor = Cursor::new(bytes); + let mut reader = StreamReader::try_new(cursor, None).expect("ipc reader"); + reader + .next() + .expect("one message") + .expect("record batch present") +} + +fn build_complex_batch(rows: usize, prefix: &str, payload_repeat: usize) -> RecordBatch { + let schema = Arc::new(Schema::new(vec![ + Field::new("value", DataType::Int64, false), + Field::new("message", DataType::Utf8, false), + ])); + + let values: Vec = (0..rows).map(|idx| idx as i64).collect(); + let mut builder = StringBuilder::new(); + let chunk = "x".repeat(payload_repeat); + for idx in 0..rows { + builder.append_value(format!("{prefix}-{idx:05}-{}", chunk)); + } + + RecordBatch::try_new( + schema, + vec![ + Arc::new(Int64Array::from(values)), + Arc::new(builder.finish()), + ], + ) + .expect("complex batch") +} + +fn slot_descriptor(id: u16, label: &'static str) -> SlotDescriptor { + SlotDescriptor::new(SlotId::new(id), label) +} + +fn descriptor_with_all_slots() -> BundleDescriptor { + let slots = (0u16..64) + .map(|id| { + let leaked = Box::leak(format!("Slot{id}").into_boxed_str()); + let label: &'static str = leaked; + SlotDescriptor::new(SlotId::new(id), label) + }) + .collect(); + BundleDescriptor::new(slots) +} + +fn encode_entry_header(entry_type: u8, slot_bitmap: u64) -> Vec { + let mut buf = vec![0u8; ENTRY_HEADER_LEN]; + let mut cursor = 0; + buf[cursor] = entry_type; + cursor += 1; + buf[cursor..cursor + 8].copy_from_slice(&42i64.to_le_bytes()); + cursor += 8; + buf[cursor..cursor + 8].copy_from_slice(&7u64.to_le_bytes()); + cursor += 8; + buf[cursor..cursor + 8].copy_from_slice(&slot_bitmap.to_le_bytes()); + buf +} + +fn write_single_entry(body: &[u8]) -> (tempfile::TempDir, PathBuf) { + let dir = tempdir().expect("tempdir"); + let path = dir.path().join("single.wal"); + let mut file = std::fs::OpenOptions::new() + .create(true) + .read(true) + .write(true) + .truncate(true) + .open(&path) + .expect("create wal file"); + let header = WalHeader::new([0xEE; 16]); + header.write_to(&mut file).expect("write header"); + let len = u32::try_from(body.len()).expect("body fits u32"); + let _ = file.seek(SeekFrom::End(0)).expect("seek end"); + file.write_all(&len.to_le_bytes()).expect("write len"); + file.write_all(body).expect("write body"); + let mut hasher = Hasher::new(); + hasher.update(body); + let crc = hasher.finalize(); + file.write_all(&crc.to_le_bytes()).expect("write crc"); + (dir, path) +} + +fn truncate_file_from_end(path: &Path, bytes: u64) { + let metadata = std::fs::metadata(path).expect("metadata"); + assert!( + metadata.len() > bytes, + "file must be larger than truncation" + ); + let new_len = metadata.len() - bytes; + let file = std::fs::OpenOptions::new() + .write(true) + .open(path) + .expect("open for truncate"); + file.set_len(new_len).expect("truncate file"); +} + +fn rotated_path_for(base: &Path, index: usize) -> PathBuf { + let mut name = base.as_os_str().to_os_string(); + name.push(format!(".{index}")); + PathBuf::from(name) +} + +fn total_logical_bytes(path: &Path) -> u64 { + let header = WAL_HEADER_LEN as u64; + let mut total = std::fs::metadata(path) + .expect("active metadata") + .len() + .saturating_sub(header); + let mut index = 1; + loop { + let rotated = rotated_path_for(path, index); + if !rotated.exists() { + break; + } + let len = std::fs::metadata(&rotated) + .expect("rotated metadata") + .len() + .saturating_sub(header); + total = total.saturating_add(len); + index += 1; + } + total +} + +fn temp_wal(file_name: &str) -> (tempfile::TempDir, PathBuf) { + let dir = tempdir().expect("tempdir"); + let path = dir.path().join(file_name); + (dir, path) +} + +fn logs_descriptor() -> BundleDescriptor { + BundleDescriptor::new(vec![slot_descriptor(0, "Logs")]) +} + +fn single_slot_bundle( + descriptor: &BundleDescriptor, + fingerprint_seed: u8, + values: &[i64], +) -> FixtureBundle { + FixtureBundle::new( + descriptor.clone(), + vec![FixtureSlot::new(SlotId::new(0), fingerprint_seed, values)], + ) +} + +/// Creates a WalWriter with default options for tests that just need basic functionality. +fn open_test_writer(path: PathBuf, hash: [u8; 16]) -> WalWriter { + WalWriter::open(WalWriterOptions::new(path, hash, FlushPolicy::Immediate)).expect("writer") +} + +/// Creates a WalWriter with custom options builder. +fn open_test_writer_with(path: PathBuf, hash: [u8; 16], configure: F) -> WalWriter +where + F: FnOnce(WalWriterOptions) -> WalWriterOptions, +{ + let options = WalWriterOptions::new(path, hash, FlushPolicy::Immediate); + WalWriter::open(configure(options)).expect("writer") +} + +/// Reads all entries from a WAL file starting at the header. +fn read_all_entries(path: &Path) -> Vec { + let mut reader = WalReader::open(path).expect("reader"); + let iter = reader.iter_from(0).expect("iter"); + iter.map(|r| r.expect("entry")).collect() +} + +/// Reads the first N entries from a WAL file. +fn read_entries(path: &Path, count: usize) -> Vec { + let mut reader = WalReader::open(path).expect("reader"); + let iter = reader.iter_from(0).expect("iter"); + iter.take(count).map(|r| r.expect("entry")).collect() +} + +fn measure_bundle_data_bytes(mut build_bundle: impl FnMut() -> FixtureBundle) -> u64 { + let (_dir, wal_path) = temp_wal("measure_bundle.wal"); + let mut writer = WalWriter::open(WalWriterOptions::new( + wal_path.clone(), + [0xFE; 16], + FlushPolicy::Immediate, + )) + .expect("writer"); + let bundle = build_bundle(); + let _ = writer.append_bundle(&bundle).expect("append bundle"); + drop(writer); + std::fs::metadata(&wal_path) + .expect("metadata") + .len() + .saturating_sub(WAL_HEADER_LEN as u64) +} + +struct FailureGuard; + +impl FailureGuard { + fn new() -> Self { + test_support::reset_failures(); + Self + } +} + +impl Drop for FailureGuard { + fn drop(&mut self) { + test_support::reset_failures(); + } +} + +#[test] +fn wal_writer_reader_roundtrip_recovers_payloads() { + let (_dir, wal_path) = temp_wal("roundtrip.wal"); + let hash = [0xAB; 16]; + + let descriptor = BundleDescriptor::new(vec![ + slot_descriptor(0, "Logs"), + slot_descriptor(1, "LogsAttrs"), + slot_descriptor(2, "ScopeAttrs"), + ]); + + let bundle = FixtureBundle::new( + descriptor, + vec![ + FixtureSlot::new(SlotId::new(0), 0x11, &[1, 2, 3]), + FixtureSlot::new(SlotId::new(2), 0x33, &[99]), + ], + ); + + let options = WalWriterOptions::new(wal_path.clone(), hash, FlushPolicy::Immediate); + let mut writer = WalWriter::open(options).expect("writer"); + let offset = writer.append_bundle(&bundle).expect("append succeeds"); + assert_eq!(offset.position, WAL_HEADER_LEN as u64); + assert_eq!(offset.sequence, 0); + drop(writer); + + let mut reader = WalReader::open(&wal_path).expect("reader"); + assert_eq!(reader.segment_cfg_hash(), hash); + assert_eq!(reader.path(), wal_path.as_path()); + + let mut iter = reader.iter_from(0).expect("iterator"); + let record = iter.next().expect("entry present").expect("entry ok"); + assert!(iter.next().is_none(), "only one entry expected"); + + let expected_bitmap = (1u64 << 0) | (1u64 << 2); + assert_eq!(record.slot_bitmap, expected_bitmap); + assert_eq!(record.sequence, 0); + assert_eq!(record.offset.position, WAL_HEADER_LEN as u64); + assert_eq!(record.slots.len(), 2); + + let slot0 = record + .slots + .iter() + .find(|slot| slot.slot_id == SlotId::new(0)) + .expect("slot 0 present"); + assert_eq!(slot0.row_count, 3); + assert_eq!(slot0.schema_fingerprint, [0x11; 32]); + let decoded0 = read_batch(&slot0.payload); + let values0 = decoded0 + .column(0) + .as_any() + .downcast_ref::() + .expect("int64 values"); + let collected0: Vec = values0 + .iter() + .map(|value| value.expect("non-null")) + .collect(); + assert_eq!(collected0, vec![1, 2, 3]); + + let slot2 = record + .slots + .iter() + .find(|slot| slot.slot_id == SlotId::new(2)) + .expect("slot 2 present"); + assert_eq!(slot2.row_count, 1); + assert_eq!(slot2.schema_fingerprint, [0x33; 32]); + let decoded2 = read_batch(&slot2.payload); + let values2 = decoded2 + .column(0) + .as_any() + .downcast_ref::() + .expect("int64 values"); + let collected2: Vec = values2 + .iter() + .map(|value| value.expect("non-null")) + .collect(); + assert_eq!(collected2, vec![99]); +} + +#[test] +fn wal_writer_rejects_slot_ids_outside_bitmap() { + let (_dir, wal_path) = temp_wal("slot_range.wal"); + let mut writer = open_test_writer(wal_path, [0; 16]); + + let descriptor = BundleDescriptor::new(vec![slot_descriptor(65, "Overflow")]); + let bundle = FixtureBundle::new( + descriptor, + vec![FixtureSlot::new(SlotId::new(65), 0xAA, &[1])], + ); + + let err = writer.append_bundle(&bundle).expect_err("slot validation"); + assert!(matches!(err, WalError::SlotOutOfRange(slot) if slot == SlotId::new(65))); +} + +#[test] +fn wal_writer_rejects_pre_epoch_timestamp() { + let (_dir, wal_path) = temp_wal("pre_epoch.wal"); + let mut writer = open_test_writer(wal_path, [0; 16]); + + let descriptor = logs_descriptor(); + let bundle = FixtureBundle::new(descriptor, vec![]) + .with_ingestion_time(UNIX_EPOCH - Duration::from_secs(1)); + + let err = writer + .append_bundle(&bundle) + .expect_err("timestamp validation"); + assert!(matches!(err, WalError::InvalidTimestamp)); +} + +#[test] +fn wal_writer_rejects_truncated_existing_file() { + let (_dir, wal_path) = temp_wal("truncated.wal"); + { + let mut file = std::fs::File::create(&wal_path).expect("create file"); + file.write_all(&[0u8; WAL_HEADER_LEN - 1]) + .expect("truncate header"); + } + + let options = WalWriterOptions::new(wal_path, [0; 16], FlushPolicy::Immediate); + let err = WalWriter::open(options).expect_err("should reject truncated file"); + assert!(matches!( + err, + WalError::InvalidHeader("file smaller than header") + )); +} + +#[test] +fn wal_writer_reopens_with_matching_header() { + let (_dir, wal_path) = temp_wal("existing.wal"); + let original_hash = [0xAA; 16]; + { + let mut file = std::fs::OpenOptions::new() + .create(true) + .write(true) + .truncate(true) + .open(&wal_path) + .expect("create file"); + WalHeader::new(original_hash) + .write_to(&mut file) + .expect("write header"); + file.flush().expect("flush"); + } + + // Reopen with the same hash—should succeed and preserve the header. + let options = WalWriterOptions::new(wal_path.clone(), original_hash, FlushPolicy::Immediate); + let _writer = WalWriter::open(options).expect("open succeeds"); + drop(_writer); + + let mut file = std::fs::OpenOptions::new() + .read(true) + .open(&wal_path) + .expect("open for read"); + let header = WalHeader::read_from(&mut file).expect("read header"); + assert_eq!(header.segment_cfg_hash, original_hash); +} + +#[test] +fn wal_writer_flushes_after_interval_elapsed() { + let (_dir, wal_path) = temp_wal("flush.wal"); + + let descriptor = logs_descriptor(); + let mut writer = WalWriter::open(WalWriterOptions::new( + wal_path, + [0; 16], + FlushPolicy::EveryDuration(Duration::from_millis(10)), + )) + .expect("writer"); + + let bundle = FixtureBundle::new( + descriptor, + vec![FixtureSlot::new(SlotId::new(0), 0x42, &[1])], + ); + + let before = writer.test_last_flush(); + writer.test_set_last_flush(Instant::now() - Duration::from_secs(1)); + let _offset = writer + .append_bundle(&bundle) + .expect("append triggers flush"); + assert!(writer.test_last_flush() > before); +} + +#[test] +fn wal_writer_flush_syncs_file_data() { + writer_test_support::reset_flush_notifications(); + + let (_dir, wal_path) = temp_wal("flush_sync.wal"); + + let descriptor = logs_descriptor(); + let mut writer = WalWriter::open(WalWriterOptions::new( + wal_path, + [0; 16], + FlushPolicy::Immediate, + )) + .expect("writer"); + + let bundle = FixtureBundle::new( + descriptor, + vec![FixtureSlot::new(SlotId::new(0), 0xAA, &[7])], + ); + + assert!(!writer_test_support::take_sync_data_notification()); + let _offset = writer.append_bundle(&bundle).expect("append flush"); + assert!(writer_test_support::take_sync_data_notification()); +} + +#[test] +fn wal_writer_records_cursor_without_truncating() { + let (_dir, wal_path) = temp_wal("record_cursor.wal"); + + let descriptor = logs_descriptor(); + let mut writer = open_test_writer(wal_path.clone(), [0x20; 16]); + + let _ = writer + .append_bundle(&single_slot_bundle(&descriptor, 0x01, &[1, 2, 3])) + .expect("first append"); + let _ = writer + .append_bundle(&single_slot_bundle(&descriptor, 0x02, &[4, 5, 6])) + .expect("second append"); + + let len_before = std::fs::metadata(&wal_path).expect("metadata").len(); + + let entries = read_entries(&wal_path, 1); + let first_entry = &entries[0]; + + let cursor = WalConsumerCheckpoint { + safe_offset: first_entry.next_offset, + ..WalConsumerCheckpoint::default() + }; + writer.checkpoint_cursor(&cursor).expect("record cursor"); + drop(writer); + + let len_after = std::fs::metadata(&wal_path).expect("metadata").len(); + assert_eq!( + len_after, len_before, + "recording a safe cursor no longer mutates the active wal immediately" + ); + + let sidecar_path = wal_path.parent().unwrap().join("checkpoint.offset"); + let sidecar = CheckpointSidecar::read_from(&sidecar_path).expect("sidecar"); + assert_eq!( + sidecar.global_data_offset, + first_entry.next_offset - WAL_HEADER_LEN as u64 + ); + + let mut reader = WalReader::open(&wal_path).expect("reader"); + let mut iter = reader.iter_from(0).expect("iter"); + let entry_one = iter.next().expect("entry").expect("ok"); + let entry_two = iter.next().expect("entry").expect("ok"); + assert_eq!(entry_one.sequence, first_entry.sequence); + assert_eq!(entry_two.sequence, first_entry.sequence + 1); + assert!(iter.next().is_none()); +} + +#[test] +fn wal_writer_enforces_safe_offset_boundaries() { + let (_dir, wal_path) = temp_wal("safe_offset.wal"); + + let descriptor = logs_descriptor(); + let mut writer = WalWriter::open(WalWriterOptions::new( + wal_path.clone(), + [0x42; 16], + FlushPolicy::Immediate, + )) + .expect("writer"); + + let first_bundle = FixtureBundle::new( + descriptor.clone(), + vec![FixtureSlot::new(SlotId::new(0), 0x01, &[11, 12, 13])], + ); + let _ = writer.append_bundle(&first_bundle).expect("first append"); + + let second_bundle = FixtureBundle::new( + descriptor, + vec![FixtureSlot::new(SlotId::new(0), 0x02, &[21, 22, 23])], + ); + let _ = writer.append_bundle(&second_bundle).expect("second append"); + + let mut reader = WalReader::open(&wal_path).expect("reader"); + let mut iter = reader.iter_from(0).expect("iter"); + let first_entry = iter.next().expect("entry").expect("ok"); + + let mut cursor = WalConsumerCheckpoint { + safe_offset: first_entry.offset.position + 4, + safe_sequence: first_entry.sequence, + }; + + match writer.checkpoint_cursor(&cursor) { + Err(WalError::InvalidConsumerCheckpoint(message)) => { + assert_eq!(message, "safe offset splits entry boundary") + } + other => panic!("expected invalid cursor error, got {other:?}"), + } + + cursor.increment(&first_entry); + writer + .checkpoint_cursor(&cursor) + .expect("record succeeds with aligned cursor"); + drop(writer); + + let sidecar_path = wal_path.parent().unwrap().join("checkpoint.offset"); + let sidecar = CheckpointSidecar::read_from(&sidecar_path).expect("sidecar"); + assert_eq!( + sidecar.global_data_offset, + first_entry.next_offset - WAL_HEADER_LEN as u64 + ); +} + +#[test] +fn wal_writer_persists_consumer_checkpoint_sidecar() { + let (_dir, wal_path) = temp_wal("checkpoint_sidecar.wal"); + let descriptor = logs_descriptor(); + + let mut writer = open_test_writer(wal_path.clone(), [0x99; 16]); + + let _ = writer + .append_bundle(&single_slot_bundle(&descriptor, 0x01, &[1, 2])) + .expect("append"); + + let file_len = std::fs::metadata(&wal_path).expect("metadata").len(); + let cursor = WalConsumerCheckpoint { + safe_offset: file_len, + ..WalConsumerCheckpoint::default() + }; + writer.checkpoint_cursor(&cursor).expect("record cursor"); + drop(writer); + + let sidecar_path = wal_path.parent().expect("dir").join("checkpoint.offset"); + let state = CheckpointSidecar::read_from(&sidecar_path).expect("sidecar"); + assert_eq!( + state.global_data_offset, + file_len.saturating_sub(WAL_HEADER_LEN as u64) + ); +} + +#[test] +fn wal_writer_rotates_when_target_exceeded() { + let (_dir, wal_path) = temp_wal("force_rotate.wal"); + + let descriptor = logs_descriptor(); + let mut writer = open_test_writer_with(wal_path.clone(), [0x51; 16], |opts| { + opts.with_rotation_target(1).with_max_rotated_files(4) + }); + + let _ = writer + .append_bundle(&single_slot_bundle(&descriptor, 0x01, &[1, 2, 3, 4])) + .expect("append triggers rotation"); + drop(writer); + + let rotated_path = rotated_path_for(&wal_path, 1); + assert!( + rotated_path.exists(), + "rotated file missing at {:?}", + rotated_path + ); + let rotated_len = std::fs::metadata(&rotated_path) + .expect("rotated metadata") + .len(); + assert!(rotated_len > WAL_HEADER_LEN as u64); + + let active_len = std::fs::metadata(&wal_path).expect("active metadata").len(); + assert_eq!(active_len, WAL_HEADER_LEN as u64); + + let sidecar_path = wal_path.parent().unwrap().join("checkpoint.offset"); + // Sidecar should exist after rotation (even if no checkpoint has been recorded yet) + assert!(sidecar_path.exists(), "sidecar should exist after rotation"); + let sidecar = CheckpointSidecar::read_from(&sidecar_path).expect("sidecar should be readable"); + // global_data_offset is 0 because no consumer checkpoint has been recorded yet + assert_eq!(sidecar.global_data_offset, 0); +} + +#[test] +fn wal_writer_reloads_rotated_files_on_restart() { + let (_dir, wal_path) = temp_wal("replay_rotations.wal"); + + let descriptor = logs_descriptor(); + let options = WalWriterOptions::new(wal_path.clone(), [0x54; 16], FlushPolicy::Immediate) + .with_rotation_target(1) + .with_max_rotated_files(4); + + { + let mut writer = WalWriter::open(options.clone()).expect("first writer"); + let bundle = single_slot_bundle(&descriptor, 0x01, &[1, 2, 3, 4]); + let _ = writer + .append_bundle(&bundle) + .expect("first append triggers rotation"); + } + + assert!( + rotated_path_for(&wal_path, 1).exists(), + "expected initial rotation" + ); + + { + let mut writer = WalWriter::open(options).expect("reopen writer"); + let bundle = single_slot_bundle(&descriptor, 0x02, &[5, 6, 7, 8]); + let _ = writer + .append_bundle(&bundle) + .expect("rotation should succeed after restart"); + } + + assert!( + rotated_path_for(&wal_path, 2).exists(), + "existing rotation should be shifted during recovery" + ); +} + +#[test] +fn wal_writer_errors_when_rotated_file_cap_reached() { + let (_dir, wal_path) = temp_wal("rotated_cap.wal"); + + let descriptor = logs_descriptor(); + let mut writer = open_test_writer_with(wal_path.clone(), [0x52; 16], |opts| { + opts.with_rotation_target(1).with_max_rotated_files(1) + }); + + let payload = [10, 11, 12]; + let first_bundle = single_slot_bundle(&descriptor, 0x02, &payload); + let _ = writer + .append_bundle(&first_bundle) + .expect("first append rotates"); + assert!( + rotated_path_for(&wal_path, 1).exists(), + "expected rotated file to exist", + ); + + let err = writer + .append_bundle(&single_slot_bundle(&descriptor, 0x03, &payload)) + .expect_err("second rotation should hit rotated file cap"); + match err { + WalError::WalAtCapacity(message) => { + assert!( + message.contains("rotated wal file cap"), + "unexpected error message: {message}", + ); + } + other => panic!("expected WalAtCapacity, got {other:?}"), + } +} + +#[test] +fn wal_writer_enforces_size_cap_and_purges_rotations() { + let (_dir, wal_path) = temp_wal("size_cap.wal"); + + let descriptor = logs_descriptor(); + let payload: Vec = (0..64).collect(); + let entry_bytes = + measure_bundle_data_bytes(|| single_slot_bundle(&descriptor, 0x07, payload.as_slice())); + let header_len = WAL_HEADER_LEN as u64; + let chunk_file_len = header_len + entry_bytes; + let slack = cmp::max(1, entry_bytes / 2); + let max_wal_size = chunk_file_len + header_len + slack; + + let mut writer = WalWriter::open( + WalWriterOptions::new(wal_path.clone(), [0x53; 16], FlushPolicy::Immediate) + .with_rotation_target(1) + .with_max_rotated_files(4) + .with_max_wal_size(max_wal_size), + ) + .expect("writer"); + + let first_bundle = single_slot_bundle(&descriptor, 0x07, payload.as_slice()); + let _ = writer + .append_bundle(&first_bundle) + .expect("first append rotates under cap"); + assert!(rotated_path_for(&wal_path, 1).exists()); + + let second_bundle = single_slot_bundle(&descriptor, 0x08, payload.as_slice()); + let err = writer + .append_bundle(&second_bundle) + .expect_err("second rotation should exceed size cap"); + match err { + WalError::WalAtCapacity(message) => { + assert!( + message.contains("size cap"), + "unexpected error message: {message}", + ); + } + other => panic!("expected WalAtCapacity, got {other:?}"), + } + + let cursor = WalConsumerCheckpoint { + safe_offset: WAL_HEADER_LEN as u64, + ..WalConsumerCheckpoint::default() + }; + writer + .checkpoint_cursor(&cursor) + .expect("record cursor purges rotated chunks"); + + assert!( + !rotated_path_for(&wal_path, 1).exists(), + "all rotated chunks should be purged", + ); + assert!( + !rotated_path_for(&wal_path, 2).exists(), + "purge removes even shifted chunks", + ); + + let third_bundle = single_slot_bundle(&descriptor, 0x09, payload.as_slice()); + let _ = writer + .append_bundle(&third_bundle) + .expect("append succeeds once space is reclaimed"); +} + +#[test] +fn wal_writer_ignores_invalid_checkpoint_sidecar() { + let (_dir, wal_path) = temp_wal("bad_sidecar.wal"); + + // Create the WAL header so the file exists. + { + let _writer = WalWriter::open(WalWriterOptions::new( + wal_path.clone(), + [0x11; 16], + FlushPolicy::Immediate, + )) + .expect("writer"); + } + + let sidecar_path = wal_path.parent().expect("dir").join("checkpoint.offset"); + std::fs::write(&sidecar_path, vec![0u8; CHECKPOINT_SIDECAR_LEN - 4]).expect("write corrupt"); + + let mut writer = WalWriter::open(WalWriterOptions::new( + wal_path.clone(), + [0x11; 16], + FlushPolicy::Immediate, + )) + .expect("reopen"); + + let descriptor = logs_descriptor(); + let bundle = FixtureBundle::new( + descriptor, + vec![FixtureSlot::new(SlotId::new(0), 0x02, &[7])], + ); + let _ = writer.append_bundle(&bundle).expect("append"); + let file_len = std::fs::metadata(&wal_path).expect("metadata").len(); + + let cursor = WalConsumerCheckpoint { + safe_offset: file_len, + ..WalConsumerCheckpoint::default() + }; + writer.checkpoint_cursor(&cursor).expect("record cursor"); + drop(writer); + + let state = CheckpointSidecar::read_from(&sidecar_path).expect("sidecar"); + assert_eq!( + state.global_data_offset, + file_len.saturating_sub(WAL_HEADER_LEN as u64) + ); +} + +#[test] +fn wal_writer_flushes_after_unflushed_byte_threshold() { + let (_dir, wal_path) = temp_wal("flush_bytes.wal"); + + let descriptor = logs_descriptor(); + let mut writer = WalWriter::open(WalWriterOptions::new( + wal_path, + [0; 16], + FlushPolicy::EveryNBytes(1), + )) + .expect("writer"); + + let bundle = FixtureBundle::new( + descriptor, + vec![FixtureSlot::new(SlotId::new(0), 0x99, &[1, 2, 3])], + ); + + writer.test_set_last_flush(Instant::now()); + let before = writer.test_last_flush(); + let _offset = writer + .append_bundle(&bundle) + .expect("append triggers flush"); + assert!(writer.test_last_flush() > before); +} + +#[test] +fn wal_writer_flushes_pending_bytes_on_drop() { + writer_test_support::reset_flush_notifications(); + + let (_dir, wal_path) = temp_wal("flush_drop.wal"); + + let descriptor = logs_descriptor(); + // Use a duration-based flush with a very long interval so we don't flush during the test + let writer = WalWriter::open(WalWriterOptions::new( + wal_path, + [0; 16], + FlushPolicy::EveryDuration(Duration::from_secs(3600)), + )) + .expect("writer"); + + { + let mut writer = writer; + let bundle = FixtureBundle::new( + descriptor, + vec![FixtureSlot::new(SlotId::new(0), 0x55, &[42])], + ); + let _ = writer.append_bundle(&bundle).expect("append"); + assert!(!writer_test_support::take_drop_flush_notification()); + } + + assert!(writer_test_support::take_drop_flush_notification()); +} + +#[test] +fn wal_reader_rewind_allows_replay_from_start() { + let (_dir, wal_path) = temp_wal("rewind.wal"); + let descriptor = logs_descriptor(); + + let mut writer = WalWriter::open(WalWriterOptions::new( + wal_path.clone(), + [0x10; 16], + FlushPolicy::Immediate, + )) + .expect("writer"); + + let first_bundle = FixtureBundle::new( + descriptor.clone(), + vec![FixtureSlot::new(SlotId::new(0), 0x01, &[1])], + ); + let _ = writer.append_bundle(&first_bundle).expect("first append"); + + let second_bundle = FixtureBundle::new( + descriptor, + vec![FixtureSlot::new(SlotId::new(0), 0x02, &[2])], + ); + let _ = writer.append_bundle(&second_bundle).expect("second append"); + drop(writer); + + let mut reader = WalReader::open(&wal_path).expect("reader"); + { + let mut iter = reader.iter_from(0).expect("iterator"); + assert!(iter.next().is_some()); + assert!(iter.next().is_some()); + assert!(iter.next().is_none()); + } + + reader.rewind().expect("rewind succeeds"); + + let mut iter = reader.iter_from(0).expect("iterator after rewind"); + let entry = iter.next().expect("entry present").expect("entry ok"); + assert_eq!(entry.sequence, 0); +} + +#[test] +fn wal_reader_iterator_stays_finished_after_eof() { + let (_dir, wal_path) = temp_wal("empty.wal"); + { + let mut file = std::fs::OpenOptions::new() + .create(true) + .write(true) + .truncate(true) + .open(&wal_path) + .expect("create wal"); + WalHeader::new([0x44; 16]) + .write_to(&mut file) + .expect("header"); + } + + let mut reader = WalReader::open(&wal_path).expect("reader"); + let mut iter = reader.iter_from(0).expect("iterator"); + assert!(iter.next().is_none(), "no entries present"); + assert!(iter.next().is_none(), "iterator remains finished"); +} + +#[test] +fn wal_writer_restores_sequence_after_restart() { + let (_dir, wal_path) = temp_wal("sequence_resume.wal"); + let descriptor = logs_descriptor(); + let bundle = FixtureBundle::new( + descriptor.clone(), + vec![FixtureSlot::new(SlotId::new(0), 0x01, &[1])], + ); + + { + let mut writer = WalWriter::open(WalWriterOptions::new( + wal_path.clone(), + [0xAA; 16], + FlushPolicy::Immediate, + )) + .expect("writer"); + + let _ = writer.append_bundle(&bundle).expect("first append"); + let _ = writer.append_bundle(&bundle).expect("second append"); + } + + let mut writer = WalWriter::open(WalWriterOptions::new( + wal_path.clone(), + [0xAA; 16], + FlushPolicy::Immediate, + )) + .expect("writer reopen"); + + let third = writer.append_bundle(&bundle).expect("third append"); + assert_eq!(third.sequence, 2, "sequence should continue across restart"); +} + +#[test] +fn wal_writer_preflight_rejects_when_size_cap_hit() { + let (_dir, wal_path) = temp_wal("size_cap.wal"); + let descriptor = logs_descriptor(); + let bundle = FixtureBundle::new( + descriptor.clone(), + vec![FixtureSlot::new(SlotId::new(0), 0x02, &[4, 5, 6])], + ); + + let hash = [0x33; 16]; + { + let mut writer = WalWriter::open(WalWriterOptions::new( + wal_path.clone(), + hash, + FlushPolicy::Immediate, + )) + .expect("writer"); + let _ = writer.append_bundle(&bundle).expect("first append"); + } + + let wal_cap = std::fs::metadata(&wal_path).expect("metadata").len(); + + let mut writer = WalWriter::open( + WalWriterOptions::new(wal_path.clone(), hash, FlushPolicy::Immediate) + .with_max_wal_size(wal_cap), + ) + .expect("writer with cap"); + let err = writer.append_bundle(&bundle).expect_err("cap hit"); + assert!(matches!(err, WalError::WalAtCapacity(_))); + + // Verify failed append did not persist + let mut reader = WalReader::open(&wal_path).expect("reader"); + let mut iter = reader.iter_from(0).expect("iter"); + let only = iter.next().expect("entry").expect("ok"); + assert!(iter.next().is_none(), "failed append must not persist"); + drop(iter); + drop(reader); + drop(writer); + + // Reopening with higher cap allows appending + let mut writer = WalWriter::open( + WalWriterOptions::new(wal_path.clone(), hash, FlushPolicy::Immediate) + .with_max_wal_size(u64::MAX), + ) + .expect("writer after cap removed"); + let retry = writer.append_bundle(&bundle).expect("retry append"); + assert_eq!(retry.sequence, only.sequence + 1); +} + +#[test] +fn wal_writer_preflight_rejects_when_rotated_file_cap_hit() { + let (_dir, wal_path) = temp_wal("rotated_file_cap.wal"); + let descriptor = logs_descriptor(); + let bundle = FixtureBundle::new( + descriptor, + vec![FixtureSlot::new(SlotId::new(0), 0x03, &[7, 8])], + ); + + let hash = [0x77; 16]; + let constrained_opts = WalWriterOptions::new(wal_path.clone(), hash, FlushPolicy::Immediate) + .with_rotation_target(1) + .with_max_rotated_files(1); + + { + let mut writer = WalWriter::open(constrained_opts.clone()).expect("writer"); + let first = writer.append_bundle(&bundle).expect("first append"); + assert_eq!(first.sequence, 0); + } + + let rotated_path = rotated_path_for(&wal_path, 1); + assert!( + rotated_path.exists(), + "rotation should have produced rotated file" + ); + + { + let mut writer = WalWriter::open(constrained_opts).expect("writer with cap"); + let err = writer + .append_bundle(&bundle) + .expect_err("rotated file cap hit"); + assert!(matches!(err, WalError::WalAtCapacity(_))); + } + + let len = std::fs::metadata(&wal_path).expect("metadata").len(); + assert!( + len <= WAL_HEADER_LEN as u64, + "active wal should contain at most header bytes" + ); + + let mut writer = WalWriter::open( + WalWriterOptions::new(wal_path.clone(), hash, FlushPolicy::Immediate) + .with_rotation_target(1) + .with_max_rotated_files(2), + ) + .expect("writer with higher rotated file cap"); + let retry = writer + .append_bundle(&bundle) + .expect("retry append succeeds once cap raised"); + assert_eq!(retry.sequence, 1); +} + +#[test] +fn wal_reader_errors_on_truncated_entry_length() { + let (_dir, wal_path) = temp_wal("length_trunc.wal"); + { + let mut file = std::fs::OpenOptions::new() + .create(true) + .write(true) + .truncate(true) + .open(&wal_path) + .expect("create wal"); + WalHeader::new([0x55; 16]) + .write_to(&mut file) + .expect("header"); + file.write_all(&[0xAA, 0xBB]) + .expect("write partial entry length"); + } + + let mut reader = WalReader::open(&wal_path).expect("reader"); + let mut iter = reader.iter_from(0).expect("iterator"); + match iter.next() { + Some(Err(WalError::UnexpectedEof("entry length"))) => {} + other => panic!("expected entry length eof, got {:?}", other), + } +} + +#[test] +fn wal_reader_reports_crc_mismatch() { + let (_dir, wal_path) = temp_wal("crc.wal"); + let descriptor = logs_descriptor(); + let bundle = FixtureBundle::new( + descriptor, + vec![FixtureSlot::new(SlotId::new(0), 0x55, &[7, 8])], + ); + + let options = WalWriterOptions::new(wal_path.clone(), [1; 16], FlushPolicy::Immediate); + let mut writer = WalWriter::open(options).expect("writer"); + let _offset = writer.append_bundle(&bundle).expect("append"); + drop(writer); + + let mut file = std::fs::OpenOptions::new() + .read(true) + .write(true) + .open(&wal_path) + .expect("open file"); + let _ = file.seek(SeekFrom::End(-1)).expect("seek to crc"); + let mut byte = [0u8; 1]; + file.read_exact(&mut byte).expect("read crc"); + byte[0] ^= 0xFF; + let _ = file.seek(SeekFrom::End(-1)).expect("rewind"); + file.write_all(&byte).expect("overwrite crc"); + + let mut reader = WalReader::open(&wal_path).expect("reader"); + let mut iter = reader.iter_from(0).expect("iter"); + match iter.next() { + Some(Err(WalError::CrcMismatch { .. })) => {} + other => panic!("expected crc mismatch, got {:?}", other), + } +} + +#[test] +fn wal_reader_rejects_unsupported_entry_type() { + let body = encode_entry_header(0xAA, 0); + let (_dir, wal_path) = write_single_entry(&body); + let mut reader = WalReader::open(&wal_path).expect("reader"); + let mut iter = reader.iter_from(0).expect("iterator"); + + match iter.next() { + Some(Err(WalError::UnsupportedEntry(ty))) => assert_eq!(ty, 0xAA), + other => panic!("expected unsupported entry, got {:?}", other), + } +} + +#[test] +fn wal_reader_errors_on_truncated_slot_header() { + let mut body = encode_entry_header(ENTRY_TYPE_RECORD_BUNDLE, 1); + body.extend_from_slice(&SlotId::new(0).0.to_le_bytes()); + let (_dir, wal_path) = write_single_entry(&body); + + let mut reader = WalReader::open(&wal_path).expect("reader"); + let mut iter = reader.iter_from(0).expect("iterator"); + match iter.next() { + Some(Err(WalError::InvalidEntry(message))) => { + assert_eq!(message, "slot header truncated") + } + other => panic!("expected truncated slot header error, got {:?}", other), + } +} + +#[test] +fn wal_reader_errors_on_truncated_slot_payload() { + let mut body = encode_entry_header(ENTRY_TYPE_RECORD_BUNDLE, 1); + body.extend_from_slice(&SlotId::new(0).0.to_le_bytes()); + body.extend_from_slice(&[0x7Au8; SCHEMA_FINGERPRINT_LEN]); + body.extend_from_slice(&1u32.to_le_bytes()); + body.extend_from_slice(&4u32.to_le_bytes()); + body.extend_from_slice(&[0x01, 0x02]); + + let (_dir, wal_path) = write_single_entry(&body); + let mut reader = WalReader::open(&wal_path).expect("reader"); + let mut iter = reader.iter_from(0).expect("iterator"); + match iter.next() { + Some(Err(WalError::InvalidEntry(message))) => assert_eq!(message, "slot payload"), + other => panic!("expected slot payload error, got {:?}", other), + } +} + +#[test] +fn wal_reader_errors_on_entry_header_underflow() { + let body = vec![0u8; ENTRY_HEADER_LEN - 1]; + let (_dir, wal_path) = write_single_entry(&body); + let mut reader = WalReader::open(&wal_path).expect("reader"); + let mut iter = reader.iter_from(0).expect("iterator"); + match iter.next() { + Some(Err(WalError::InvalidEntry(message))) => { + assert_eq!(message, "body shorter than header") + } + other => panic!("expected header underflow error, got {:?}", other), + } +} + +#[test] +fn wal_reader_errors_on_unexpected_trailing_bytes() { + let mut body = encode_entry_header(ENTRY_TYPE_RECORD_BUNDLE, 0); + body.push(0xFF); + let (_dir, wal_path) = write_single_entry(&body); + let mut reader = WalReader::open(&wal_path).expect("reader"); + let mut iter = reader.iter_from(0).expect("iterator"); + match iter.next() { + Some(Err(WalError::InvalidEntry(message))) => { + assert_eq!(message, "unexpected trailing bytes") + } + other => panic!("expected trailing bytes error, got {:?}", other), + } +} + +#[test] +fn wal_reader_errors_on_truncated_entry_body() { + let body = encode_entry_header(ENTRY_TYPE_RECORD_BUNDLE, 0); + let (_dir, wal_path) = write_single_entry(&body); + truncate_file_from_end(&wal_path, 6); + + let mut reader = WalReader::open(&wal_path).expect("reader"); + let mut iter = reader.iter_from(0).expect("iterator"); + // The reader rejects entries whose declared length exceeds remaining file bytes. + // This guards against corrupted/malicious length values causing excessive allocation. + match iter.next() { + Some(Err(WalError::InvalidEntry("entry length exceeds remaining file size"))) => {} + other => panic!("expected entry length overflow error, got {:?}", other), + } +} + +#[test] +fn wal_reader_errors_on_truncated_entry_crc() { + let body = encode_entry_header(ENTRY_TYPE_RECORD_BUNDLE, 0); + let (_dir, wal_path) = write_single_entry(&body); + truncate_file_from_end(&wal_path, 2); + + let mut reader = WalReader::open(&wal_path).expect("reader"); + let mut iter = reader.iter_from(0).expect("iterator"); + match iter.next() { + Some(Err(WalError::UnexpectedEof("entry crc"))) => {} + other => panic!("expected entry crc EOF, got {:?}", other), + } +} + +#[test] +fn wal_reader_reports_io_error_during_entry_length_read() { + let _guard = FailureGuard::new(); + let body = encode_entry_header(ENTRY_TYPE_RECORD_BUNDLE, 0); + let (_dir, wal_path) = write_single_entry(&body); + + test_support::fail_next_read(ReadFailure::Length); + + let mut reader = WalReader::open(&wal_path).expect("reader"); + let mut iter = reader.iter_from(0).expect("iterator"); + match iter.next() { + Some(Err(WalError::Io(_))) => {} + other => panic!("expected io error for entry length read, got {:?}", other), + } +} + +#[test] +fn wal_reader_reports_io_error_during_entry_body_read() { + let _guard = FailureGuard::new(); + let body = encode_entry_header(ENTRY_TYPE_RECORD_BUNDLE, 0); + let (_dir, wal_path) = write_single_entry(&body); + + test_support::fail_next_read(ReadFailure::Body); + + let mut reader = WalReader::open(&wal_path).expect("reader"); + let mut iter = reader.iter_from(0).expect("iterator"); + match iter.next() { + Some(Err(WalError::Io(_))) => {} + other => panic!("expected io error for entry body read, got {:?}", other), + } +} + +#[test] +fn wal_reader_reports_io_error_during_entry_crc_read() { + let _guard = FailureGuard::new(); + let body = encode_entry_header(ENTRY_TYPE_RECORD_BUNDLE, 0); + let (_dir, wal_path) = write_single_entry(&body); + + test_support::fail_next_read(ReadFailure::Crc); + + let mut reader = WalReader::open(&wal_path).expect("reader"); + let mut iter = reader.iter_from(0).expect("iterator"); + match iter.next() { + Some(Err(WalError::Io(_))) => {} + other => panic!("expected io error for entry crc read, got {:?}", other), + } +} + +#[test] +fn wal_reader_iter_from_respects_offsets() { + let (_dir, wal_path) = temp_wal("offsets.wal"); + let descriptor = logs_descriptor(); + + let mut writer = WalWriter::open(WalWriterOptions::new( + wal_path.clone(), + [0xCC; 16], + FlushPolicy::Immediate, + )) + .expect("writer"); + + let first_bundle = FixtureBundle::new( + descriptor.clone(), + vec![FixtureSlot::new(SlotId::new(0), 0x01, &[1])], + ); + let first_offset = writer.append_bundle(&first_bundle).expect("first append"); + + let second_bundle = FixtureBundle::new( + descriptor, + vec![FixtureSlot::new(SlotId::new(0), 0x02, &[2])], + ); + let second_offset = writer.append_bundle(&second_bundle).expect("second append"); + drop(writer); + + let mut reader = WalReader::open(&wal_path).expect("reader"); + let mut iter = reader.iter_from(0).expect("iterator"); + let entry_one = iter.next().expect("first entry").expect("ok"); + let entry_two = iter.next().expect("second entry").expect("ok"); + assert_eq!(entry_one.sequence, 0); + assert_eq!(entry_two.sequence, 1); + assert_eq!(entry_one.next_offset, entry_two.offset.position); + + let mut cursor = WalConsumerCheckpoint::default(); + cursor.increment(&entry_one); + assert_eq!(cursor.safe_offset, entry_one.next_offset); + assert_eq!(cursor.safe_sequence, entry_one.sequence); + + let mut reader_from_offset = WalReader::open(&wal_path).expect("reader"); + let mut iter_from_second = reader_from_offset + .iter_from(second_offset.position) + .expect("iter from offset"); + let entry = iter_from_second.next().expect("entry").expect("ok"); + assert_eq!(entry.sequence, 1); + assert!(iter_from_second.next().is_none()); + assert_eq!(first_offset.sequence, 0); + assert_eq!(second_offset.sequence, 1); +} + +#[test] +fn wal_reader_iter_from_partial_length_reports_error() { + let (_dir, wal_path) = temp_wal("partial_offset.wal"); + let descriptor = logs_descriptor(); + + { + let mut writer = WalWriter::open(WalWriterOptions::new( + wal_path.clone(), + [0x44; 16], + FlushPolicy::Immediate, + )) + .expect("writer"); + + let bundle = FixtureBundle::new( + descriptor.clone(), + vec![FixtureSlot::new(SlotId::new(0), 0xAA, &[1, 2])], + ); + let _ = writer.append_bundle(&bundle).expect("append"); + } + + let metadata_len = std::fs::metadata(&wal_path).expect("metadata").len(); + let misaligned_offset = metadata_len.saturating_sub(2); + + let mut reader = WalReader::open(&wal_path).expect("reader"); + let mut iter = reader + .iter_from(misaligned_offset) + .expect("iterator from misaligned offset"); + match iter.next() { + Some(Err(WalError::UnexpectedEof("entry length"))) => {} + other => panic!("expected entry length eof, got {:?}", other), + } +} + +#[test] +fn wal_reader_iter_from_offset_past_file_returns_none() { + let (_dir, wal_path) = temp_wal("past_end_offset.wal"); + let descriptor = logs_descriptor(); + + { + let mut writer = WalWriter::open(WalWriterOptions::new( + wal_path.clone(), + [0x55; 16], + FlushPolicy::Immediate, + )) + .expect("writer"); + + let bundle = FixtureBundle::new( + descriptor, + vec![FixtureSlot::new(SlotId::new(0), 0xCC, &[3, 4])], + ); + let _ = writer.append_bundle(&bundle).expect("append"); + } + + let metadata_len = std::fs::metadata(&wal_path).expect("metadata").len(); + let offset_beyond_file = metadata_len + 128; + + let mut reader = WalReader::open(&wal_path).expect("reader"); + let mut iter = reader + .iter_from(offset_beyond_file) + .expect("iterator past end"); + assert!(iter.next().is_none()); +} + +#[test] +fn wal_writer_reader_handles_all_bitmap_slots() { + let (_dir, wal_path) = temp_wal("all_slots.wal"); + let descriptor = descriptor_with_all_slots(); + + let slots: Vec<_> = (0u16..64) + .map(|id| { + let values = [id as i64, (id as i64) * 2 + 1]; + FixtureSlot::new(SlotId::new(id), id as u8, &values) + }) + .collect(); + let bundle = FixtureBundle::new(descriptor.clone(), slots); + + let mut writer = WalWriter::open(WalWriterOptions::new( + wal_path.clone(), + [0xAA; 16], + FlushPolicy::Immediate, + )) + .expect("writer"); + let _ = writer.append_bundle(&bundle).expect("append"); + drop(writer); + + let mut reader = WalReader::open(&wal_path).expect("reader"); + let mut iter = reader.iter_from(0).expect("iter"); + let entry = iter.next().expect("entry present").expect("ok"); + assert_eq!(entry.slot_bitmap, u64::MAX, "all 64 slots set"); + assert_eq!(entry.slots.len(), 64); + + for slot in entry.slots { + assert_eq!(slot.payload_len, slot.payload.len() as u32); + assert_eq!(slot.row_count, 2); + } + + assert!(iter.next().is_none()); +} + +#[test] +fn wal_writer_handles_large_payload_batches() { + let (_dir, wal_path) = temp_wal("large_payload.wal"); + let descriptor = BundleDescriptor::new(vec![ + slot_descriptor(0, "Logs"), + slot_descriptor(1, "LogsAttrs"), + slot_descriptor(2, "ScopeAttrs"), + ]); + + let slot_specs = [ + (SlotId::new(0), 0x10, 6_000usize, "alpha", 256usize), + (SlotId::new(1), 0x20, 5_000usize, "beta", 512usize), + (SlotId::new(2), 0x30, 4_000usize, "gamma", 768usize), + ]; + + let slots: Vec<_> = slot_specs + .iter() + .map(|(id, seed, rows, prefix, repeat)| { + FixtureSlot::with_batch(*id, *seed, build_complex_batch(*rows, prefix, *repeat)) + }) + .collect(); + + let bundle = FixtureBundle::new(descriptor.clone(), slots); + + let mut writer = WalWriter::open(WalWriterOptions::new( + wal_path.clone(), + [0xBB; 16], + FlushPolicy::Immediate, + )) + .expect("writer"); + + let _ = writer.append_bundle(&bundle).expect("append large payload"); + drop(writer); + + let mut reader = WalReader::open(&wal_path).expect("reader"); + let mut iter = reader.iter_from(0).expect("iter"); + let entry = iter.next().expect("entry present").expect("ok"); + + let expected_bitmap = (1u64 << 0) | (1u64 << 1) | (1u64 << 2); + assert_eq!(entry.slot_bitmap, expected_bitmap); + assert_eq!(entry.slots.len(), 3); + + for (slot, (_, _, expected_rows, _, repeat)) in entry.slots.iter().zip(slot_specs.iter()) { + assert!(slot.payload_len as usize >= expected_rows * repeat); + let decoded = read_batch(&slot.payload); + assert_eq!(decoded.num_rows(), *expected_rows); + assert_eq!(slot.row_count as usize, *expected_rows); + } + + assert!(iter.next().is_none()); +} + +/// Verifies that `WalWriter::open` automatically truncates trailing garbage +/// when the file has been truncated mid-entry (simulating a crash where bytes +/// were lost). This complements `wal_writer_auto_truncates_trailing_garbage_on_open` +/// which tests garbage appended at the end. +#[test] +fn wal_writer_auto_truncates_after_mid_entry_truncation() { + let (_dir, wal_path) = temp_wal("recovery.wal"); + let descriptor = logs_descriptor(); + let hash = [0x99; 16]; + + // Write two valid entries + let mut writer = WalWriter::open(WalWriterOptions::new( + wal_path.clone(), + hash, + FlushPolicy::Immediate, + )) + .expect("writer"); + + let first_bundle = FixtureBundle::new( + descriptor.clone(), + vec![FixtureSlot::new(SlotId::new(0), 0x01, &[1])], + ); + let _ = writer.append_bundle(&first_bundle).expect("first append"); + + let second_bundle = FixtureBundle::new( + descriptor.clone(), + vec![FixtureSlot::new(SlotId::new(0), 0x02, &[2])], + ); + let _ = writer.append_bundle(&second_bundle).expect("second append"); + drop(writer); + + // Record valid file length after first entry for later comparison + let mut reader = WalReader::open(&wal_path).expect("reader"); + let mut iter = reader.iter_from(0).expect("iterator"); + let first_entry = iter.next().expect("first entry").expect("ok"); + let first_entry_end = first_entry.next_offset; + drop(reader); + + // Simulate a crash that truncates the file in the middle of the second + // entry, losing bytes (not appending garbage) + { + let file = std::fs::OpenOptions::new() + .write(true) + .open(&wal_path) + .expect("open for truncation"); + // Truncate one byte into the second entry + file.set_len(first_entry_end + 1) + .expect("truncate inside entry"); + } + + // Reopen the writer - it should auto-truncate to end of first entry + let mut writer = WalWriter::open(WalWriterOptions::new( + wal_path.clone(), + hash, + FlushPolicy::Immediate, + )) + .expect("writer reopens and auto-truncates"); + + // Verify file was truncated to first entry boundary + let file_len = std::fs::metadata(&wal_path).expect("metadata").len(); + assert_eq!( + file_len, first_entry_end, + "file should be truncated to end of first valid entry" + ); + + // Append a recovery entry - should continue from sequence 1 + let recovery_bundle = FixtureBundle::new( + descriptor, + vec![FixtureSlot::new(SlotId::new(0), 0x03, &[3])], + ); + let recovery_offset = writer + .append_bundle(&recovery_bundle) + .expect("append recovery entry"); + assert_eq!(recovery_offset.position, first_entry_end); + assert_eq!( + recovery_offset.sequence, 1, + "sequence continues from last valid" + ); + drop(writer); + + // Verify the WAL now has exactly two entries: first + recovery + let mut reader = WalReader::open(&wal_path).expect("reader after recovery"); + let mut iter = reader.iter_from(0).expect("iterator"); + let first = iter.next().expect("first entry").expect("ok"); + assert_eq!(first.sequence, 0); + let recovery = iter.next().expect("recovery entry").expect("ok"); + assert_eq!(recovery.sequence, 1); + assert_eq!( + recovery.slots.first().expect("slot").schema_fingerprint, + [0x03; 32] + ); + assert!(iter.next().is_none(), "only two entries should exist"); +} + +/// Verifies that `WalWriter::open` automatically truncates trailing garbage +/// left by a crash that interrupted a write. This test simulates the scenario +/// where a process crashes mid-write, leaving a partial entry at the end of +/// the WAL file. On reopen, the writer should automatically detect and remove +/// this garbage so that subsequent appends resume from a clean boundary. +#[test] +fn wal_writer_auto_truncates_trailing_garbage_on_open() { + let (_dir, wal_path) = temp_wal("auto_truncate.wal"); + let descriptor = logs_descriptor(); + let hash = [0xAA; 16]; + + // Write two valid entries to establish a baseline. + let mut writer = WalWriter::open(WalWriterOptions::new( + wal_path.clone(), + hash, + FlushPolicy::Immediate, + )) + .expect("writer"); + + let bundle1 = FixtureBundle::new( + descriptor.clone(), + vec![FixtureSlot::new(SlotId::new(0), 0x01, &[1])], + ); + let _ = writer.append_bundle(&bundle1).expect("first append"); + + let bundle2 = FixtureBundle::new( + descriptor.clone(), + vec![FixtureSlot::new(SlotId::new(0), 0x02, &[2])], + ); + let _ = writer.append_bundle(&bundle2).expect("second append"); + drop(writer); + + // Get the file length before corruption + let valid_len = std::fs::metadata(&wal_path).expect("metadata").len(); + + // Simulate a crash mid-write by appending garbage bytes to the WAL file + // (this simulates a partial entry header or body that was interrupted). + { + use std::io::Write; + let mut file = std::fs::OpenOptions::new() + .append(true) + .open(&wal_path) + .expect("open for corruption"); + // Write partial "entry length" field plus some garbage + file.write_all(&[0xFF, 0x00, 0x10, 0x00, 0xDE, 0xAD, 0xBE, 0xEF]) + .expect("append garbage"); + } + + let corrupted_len = std::fs::metadata(&wal_path).expect("metadata").len(); + assert!(corrupted_len > valid_len, "garbage should extend file"); + + // Reopen the writer - it should automatically truncate the garbage + let mut writer = WalWriter::open(WalWriterOptions::new( + wal_path.clone(), + hash, + FlushPolicy::Immediate, + )) + .expect("writer reopens and truncates garbage"); + + // File should be truncated back to valid length + let after_open_len = std::fs::metadata(&wal_path).expect("metadata").len(); + assert_eq!( + after_open_len, valid_len, + "garbage should be truncated on open" + ); + + // Append a third entry - it should continue from the correct position + let bundle3 = FixtureBundle::new( + descriptor.clone(), + vec![FixtureSlot::new(SlotId::new(0), 0x03, &[3])], + ); + let offset3 = writer.append_bundle(&bundle3).expect("third append"); + + // The third entry should have sequence 2 (continuing from 0, 1) + assert_eq!(offset3.sequence, 2, "sequence should continue correctly"); + drop(writer); + + // Verify all three entries are readable and valid + let mut reader = WalReader::open(&wal_path).expect("reader"); + let mut iter = reader.iter_from(0).expect("iterator"); + + let e1 = iter.next().expect("first entry").expect("ok"); + assert_eq!(e1.sequence, 0); + assert_eq!(e1.slots.first().unwrap().schema_fingerprint, [0x01; 32]); + + let e2 = iter.next().expect("second entry").expect("ok"); + assert_eq!(e2.sequence, 1); + assert_eq!(e2.slots.first().unwrap().schema_fingerprint, [0x02; 32]); + + let e3 = iter.next().expect("third entry").expect("ok"); + assert_eq!(e3.sequence, 2); + assert_eq!(e3.slots.first().unwrap().schema_fingerprint, [0x03; 32]); + + assert!(iter.next().is_none(), "only three entries should exist"); +} + +#[test] +fn wal_writer_rejects_segment_config_mismatch() { + let (_dir, wal_path) = temp_wal("mismatch.wal"); + let original_hash = [0xAA; 16]; + + // Create a WAL with one config hash. + { + let options = + WalWriterOptions::new(wal_path.clone(), original_hash, FlushPolicy::Immediate); + let _writer = WalWriter::open(options).expect("initial open"); + } + + // Attempt to reopen with a different hash. + let different_hash = [0xBB; 16]; + let options = WalWriterOptions::new(wal_path, different_hash, FlushPolicy::Immediate); + match WalWriter::open(options) { + Err(WalError::SegmentConfigMismatch { expected, found }) => { + assert_eq!(expected, different_hash); + assert_eq!(found, original_hash); + } + other => panic!("expected segment config mismatch, got {:?}", other), + } +} + +#[test] +fn wal_reader_detects_unexpected_segment_config() { + let (_dir, wal_path) = temp_wal("reader_mismatch.wal"); + let stored_hash = [0xDD; 16]; + + // Write a WAL with a known config hash. + { + let options = WalWriterOptions::new(wal_path.clone(), stored_hash, FlushPolicy::Immediate); + let _writer = WalWriter::open(options).expect("writer"); + } + + // Reader opens successfully and exposes the stored hash for the caller to verify. + let reader = WalReader::open(&wal_path).expect("reader opens"); + assert_eq!(reader.segment_cfg_hash(), stored_hash); + + // Caller can decide what to do if it doesn't match expectations. + let expected_hash = [0xEE; 16]; + assert_ne!( + reader.segment_cfg_hash(), + expected_hash, + "caller detects mismatch" + ); +} + +#[test] +fn wal_reader_fails_on_corrupt_header_version() { + let (_dir, wal_path) = temp_wal("bad_version.wal"); + + // Create a valid WAL first. + { + let options = WalWriterOptions::new(wal_path.clone(), [0x11; 16], FlushPolicy::Immediate); + let _writer = WalWriter::open(options).expect("writer"); + } + + // Corrupt the version field in the header. + { + let mut file = std::fs::OpenOptions::new() + .read(true) + .write(true) + .open(&wal_path) + .expect("open"); + // Version is at offset WAL_MAGIC.len() (10 bytes). + let _ = file.seek(SeekFrom::Start(10)).expect("seek"); + file.write_all(&99u16.to_le_bytes()) + .expect("corrupt version"); + } + + match WalReader::open(&wal_path) { + Err(WalError::InvalidHeader("unsupported version")) => {} + other => panic!("expected unsupported version error, got {:?}", other), + } +} + +#[derive(Clone, Copy)] +struct CrashCase { + name: &'static str, + injection: writer_test_support::CrashInjection, +} + +#[test] +fn wal_writer_recovers_from_crash_resilience_scenarios() { + let cases = [CrashCase { + name: "sidecar_pre_rename", + injection: writer_test_support::CrashInjection::BeforeSidecarRename, + }]; + + for case in cases { + run_crash_case(case); + } +} + +fn run_crash_case(case: CrashCase) { + writer_test_support::reset_flush_notifications(); + + let dir = tempdir().expect("tempdir"); + let wal_path = dir.path().join(format!("crash_{}.wal", case.name)); + let descriptor = logs_descriptor(); + let options = WalWriterOptions::new(wal_path.clone(), [0xC7; 16], FlushPolicy::Immediate) + .with_rotation_target(32 * 1024) + .with_max_rotated_files(4); + + let mut writer = WalWriter::open(options.clone()).expect("writer"); + for value in 0..4 { + let bundle = FixtureBundle::new( + descriptor.clone(), + vec![FixtureSlot::with_batch( + SlotId::new(0), + 0x60 + value as u8, + build_complex_batch(256, "crash", 1024), + )], + ); + let _ = writer.append_bundle(&bundle).expect("append bundle"); + } + + for value in 0..4 { + let bundle = FixtureBundle::new( + descriptor.clone(), + vec![FixtureSlot::new( + SlotId::new(0), + 0x40 + value as u8, + &[(value + 1) as i64], + )], + ); + let _ = writer.append_bundle(&bundle).expect("append bundle"); + } + + let cursor = wal_cursor_after_entries(&wal_path, 2); + assert!( + cursor.safe_offset > WAL_HEADER_LEN as u64, + "{}: cursor safe offset must exceed header", + case.name + ); + writer_test_support::inject_crash(case.injection); + let err = match writer.checkpoint_cursor(&cursor) { + Ok(_) => panic!("{}: crash injection did not trigger", case.name), + Err(err) => err, + }; + assert!( + matches!(err, WalError::InjectedCrash(_)), + "unexpected error: {:?}", + err + ); + + writer.test_force_crash(); + + assert_crash_recovery(&options, &descriptor, case.name, &cursor); + writer_test_support::reset_flush_notifications(); +} + +fn wal_cursor_after_entries(path: &Path, entry_count: usize) -> WalConsumerCheckpoint { + let mut reader = WalReader::open(path).expect("reader for cursor"); + let mut iter = reader.iter_from(0).expect("cursor iterator"); + let mut cursor = WalConsumerCheckpoint::default(); + for idx in 0..entry_count { + let bundle = iter + .next() + .unwrap_or_else(|| { + panic!( + "not enough entries for cursor (wanted {}, stopped at {})", + entry_count, idx + ) + }) + .expect("entry ok while building cursor"); + cursor.increment(&bundle); + } + cursor +} + +fn assert_crash_recovery( + options: &WalWriterOptions, + descriptor: &BundleDescriptor, + case_name: &str, + cursor: &WalConsumerCheckpoint, +) { + assert_reader_clean(&options.path, cursor.safe_offset, case_name); + + let mut writer = WalWriter::open(options.clone()).expect("writer reopen"); + let repair_bundle = FixtureBundle::new( + descriptor.clone(), + vec![FixtureSlot::new(SlotId::new(0), 0xF0, &[99])], + ); + let _ = writer + .append_bundle(&repair_bundle) + .expect("append after crash"); + drop(writer); + + assert_reader_clean(&options.path, cursor.safe_offset, case_name); + + let sidecar_path = options + .path + .parent() + .expect("wal dir") + .join("checkpoint.offset"); + if sidecar_path.exists() { + let sidecar = CheckpointSidecar::read_from(&sidecar_path).expect("sidecar readable"); + let total_logical = total_logical_bytes(&options.path); + assert!( + sidecar.global_data_offset <= total_logical, + "{case_name}: logical cursor must stay within logical stream" + ); + } +} + +fn assert_reader_clean(path: &Path, offset: u64, case_name: &str) { + let mut reader = WalReader::open(path) + .unwrap_or_else(|err| panic!("{}: reader open failed: {:?}", case_name, err)); + let iter = reader + .iter_from(offset) + .unwrap_or_else(|err| panic!("{}: iterator init failed: {:?}", case_name, err)); + for entry in iter { + let _ = entry.unwrap_or_else(|err| panic!("{}: wal entry error {:?}", case_name, err)); + } +} + +#[test] +fn wal_writer_flushes_with_bytes_or_duration_policy_on_bytes() { + // Test that BytesOrDuration flushes when byte threshold is exceeded + let (_dir, wal_path) = temp_wal("flush_bytes_or_duration_bytes.wal"); + + let descriptor = logs_descriptor(); + // Set a very small byte threshold and very long duration + let mut writer = WalWriter::open(WalWriterOptions::new( + wal_path, + [0; 16], + FlushPolicy::BytesOrDuration { + bytes: 1, + duration: Duration::from_secs(3600), + }, + )) + .expect("writer"); + + let bundle = FixtureBundle::new( + descriptor, + vec![FixtureSlot::new(SlotId::new(0), 0x99, &[1, 2, 3])], + ); + + let before = writer.test_last_flush(); + let _offset = writer + .append_bundle(&bundle) + .expect("append triggers flush via bytes threshold"); + assert!( + writer.test_last_flush() > before, + "flush should occur when bytes threshold exceeded" + ); +} + +#[test] +fn wal_writer_flushes_with_bytes_or_duration_policy_on_duration() { + // Test that BytesOrDuration flushes when duration threshold is exceeded + let (_dir, wal_path) = temp_wal("flush_bytes_or_duration_time.wal"); + + let descriptor = logs_descriptor(); + // Set a very large byte threshold and very short duration + let mut writer = WalWriter::open(WalWriterOptions::new( + wal_path, + [0; 16], + FlushPolicy::BytesOrDuration { + bytes: u64::MAX, + duration: Duration::from_millis(1), + }, + )) + .expect("writer"); + + let bundle = FixtureBundle::new( + descriptor, + vec![FixtureSlot::new(SlotId::new(0), 0x99, &[1, 2, 3])], + ); + + // Force the last_flush to be in the past + writer.test_set_last_flush(Instant::now() - Duration::from_secs(1)); + let before = writer.test_last_flush(); + + let _offset = writer + .append_bundle(&bundle) + .expect("append triggers flush via duration threshold"); + assert!( + writer.test_last_flush() > before, + "flush should occur when duration threshold exceeded" + ); +} + +#[test] +fn wal_writer_skips_flush_when_neither_threshold_met() { + // Test that BytesOrDuration does NOT flush when neither threshold is met + writer_test_support::reset_flush_notifications(); + + let (_dir, wal_path) = temp_wal("flush_bytes_or_duration_skip.wal"); + + let descriptor = logs_descriptor(); + // Set very large byte threshold and very long duration + let mut writer = WalWriter::open(WalWriterOptions::new( + wal_path, + [0; 16], + FlushPolicy::BytesOrDuration { + bytes: u64::MAX, + duration: Duration::from_secs(3600), + }, + )) + .expect("writer"); + + let bundle = FixtureBundle::new( + descriptor, + vec![FixtureSlot::new(SlotId::new(0), 0x99, &[1, 2, 3])], + ); + + // Ensure last_flush is recent + writer.test_set_last_flush(Instant::now()); + let before = writer.test_last_flush(); + + let _offset = writer.append_bundle(&bundle).expect("append without flush"); + + // last_flush should not have changed (no flush occurred during append) + assert_eq!( + writer.test_last_flush(), + before, + "flush should NOT occur when neither threshold is met" + ); +} + +#[test] +fn wal_writer_appends_empty_bundle_with_no_slots() { + // Test that a bundle with zero populated slots can be appended + let (_dir, wal_path) = temp_wal("empty_bundle.wal"); + + let descriptor = logs_descriptor(); + let mut writer = open_test_writer(wal_path.clone(), [0x11; 16]); + + // Create a bundle with no slots populated (empty slots vec) + let empty_bundle = FixtureBundle::new(descriptor.clone(), vec![]); + let offset = writer + .append_bundle(&empty_bundle) + .expect("empty bundle should append successfully"); + assert_eq!(offset.sequence, 0); + + // Append a normal bundle after to verify the writer is still functional + let offset2 = writer + .append_bundle(&single_slot_bundle(&descriptor, 0x22, &[1, 2, 3])) + .expect("normal bundle after empty"); + assert_eq!(offset2.sequence, 1); + drop(writer); + + // Verify both entries can be read back + let entries = read_all_entries(&wal_path); + assert_eq!(entries.len(), 2); + + assert_eq!(entries[0].sequence, 0); + assert_eq!( + entries[0].slot_bitmap, 0, + "empty bundle should have zero bitmap" + ); + assert!( + entries[0].slots.is_empty(), + "empty bundle should have no slots" + ); + + assert_eq!(entries[1].sequence, 1); + assert_eq!(entries[1].slots.len(), 1); +} + +#[test] +fn wal_writer_rejects_checkpoint_sequence_regression() { + // Test that advancing checkpoint with a lower sequence number fails + let (_dir, wal_path) = temp_wal("sequence_regression.wal"); + + let descriptor = logs_descriptor(); + let mut writer = open_test_writer(wal_path.clone(), [0x33; 16]); + + // Append three bundles + for i in 0..3 { + let _ = writer + .append_bundle(&single_slot_bundle(&descriptor, i, &[i as i64])) + .expect("append"); + } + + // Read entries to get offsets + let entries = read_entries(&wal_path, 2); + + // First, advance to the second entry (sequence=1) + let cursor_at_second = WalConsumerCheckpoint { + safe_offset: entries[1].next_offset, + safe_sequence: entries[1].sequence, + }; + writer + .checkpoint_cursor(&cursor_at_second) + .expect("advance to second entry"); + + // Now try to regress to the first entry (sequence=0) + let cursor_at_first = WalConsumerCheckpoint { + safe_offset: entries[0].next_offset, + safe_sequence: entries[0].sequence, // sequence=0, which is less than 1 + }; + + match writer.checkpoint_cursor(&cursor_at_first) { + Err(WalError::InvalidConsumerCheckpoint(msg)) => { + assert!( + msg.contains("regressed"), + "expected regression error, got: {msg}" + ); + } + other => panic!("expected sequence regression error, got {other:?}"), + } +} + +#[test] +fn wal_writer_rejects_checkpoint_offset_regression() { + // Test that advancing checkpoint with a lower offset fails + let (_dir, wal_path) = temp_wal("offset_regression.wal"); + + let descriptor = logs_descriptor(); + let mut writer = open_test_writer(wal_path.clone(), [0x44; 16]); + + // Append two bundles + for i in 0..2 { + let _ = writer + .append_bundle(&single_slot_bundle(&descriptor, i, &[i as i64])) + .expect("append"); + } + + // Read entries to get offsets + let entries = read_entries(&wal_path, 2); + + // Advance to the second entry + let cursor_at_second = WalConsumerCheckpoint { + safe_offset: entries[1].next_offset, + safe_sequence: entries[1].sequence, + }; + writer + .checkpoint_cursor(&cursor_at_second) + .expect("advance to second entry"); + + // Now try to advance with a higher sequence but lower offset + // This simulates a malformed checkpoint + let bad_cursor = WalConsumerCheckpoint { + safe_offset: entries[0].next_offset, // lower offset than before + safe_sequence: entries[1].sequence + 1, // higher sequence to pass that check + }; + + match writer.checkpoint_cursor(&bad_cursor) { + Err(WalError::InvalidConsumerCheckpoint(msg)) => { + assert!( + msg.contains("regressed"), + "expected offset regression error, got: {msg}" + ); + } + other => panic!("expected offset regression error, got {other:?}"), + } +} + +#[test] +fn wal_writer_handles_bundle_with_unpopulated_descriptor_slots() { + // Test a bundle where the descriptor has slots but payload() returns None for some + let (_dir, wal_path) = temp_wal("sparse_bundle.wal"); + + // Descriptor declares 3 slots + let descriptor = BundleDescriptor::new(vec![ + slot_descriptor(0, "Logs"), + slot_descriptor(1, "LogAttrs"), + slot_descriptor(2, "ScopeAttrs"), + ]); + + let mut writer = WalWriter::open(WalWriterOptions::new( + wal_path.clone(), + [0x55; 16], + FlushPolicy::Immediate, + )) + .expect("writer"); + + // Only populate slot 1 (middle slot), leaving 0 and 2 empty + let sparse_bundle = FixtureBundle::new( + descriptor.clone(), + vec![FixtureSlot::new(SlotId::new(1), 0xBB, &[100, 200])], + ); + + let offset = writer + .append_bundle(&sparse_bundle) + .expect("sparse bundle appends"); + assert_eq!(offset.sequence, 0); + drop(writer); + + // Verify the entry + let mut reader = WalReader::open(&wal_path).expect("reader"); + let mut iter = reader.iter_from(0).expect("iter"); + let entry = iter.next().expect("entry").expect("ok"); + + // Only bit 1 should be set in the bitmap + assert_eq!(entry.slot_bitmap, 1u64 << 1); + assert_eq!(entry.slots.len(), 1); + assert_eq!(entry.slots[0].slot_id, SlotId::new(1)); + assert_eq!(entry.slots[0].row_count, 2); +} + +#[test] +fn wal_recovery_clamps_stale_sidecar_offset() { + // Test that recovery handles a sidecar with global_data_offset beyond actual WAL data + let (_dir, wal_path) = temp_wal("stale_sidecar.wal"); + let sidecar_path = wal_path.parent().unwrap().join("checkpoint.offset"); + + let descriptor = logs_descriptor(); + + // First, create a WAL with some data + { + let mut writer = WalWriter::open(WalWriterOptions::new( + wal_path.clone(), + [0x66; 16], + FlushPolicy::Immediate, + )) + .expect("writer"); + + let bundle = single_slot_bundle(&descriptor, 0x01, &[1, 2, 3]); + let _ = writer.append_bundle(&bundle).expect("append"); + } + + // Now write a sidecar with an absurdly large offset + let stale_sidecar = CheckpointSidecar::new(u64::MAX / 2); + CheckpointSidecar::write_to(&sidecar_path, &stale_sidecar).expect("write stale sidecar"); + + // Reopen the writer - it should clamp the offset internally and not panic + let mut writer = WalWriter::open(WalWriterOptions::new( + wal_path.clone(), + [0x66; 16], + FlushPolicy::Immediate, + )) + .expect("writer should recover from stale sidecar"); + + // Writer should still be functional + let bundle = single_slot_bundle(&descriptor, 0x02, &[4, 5, 6]); + let offset = writer + .append_bundle(&bundle) + .expect("append after recovery"); + assert_eq!(offset.sequence, 1, "sequence should continue from WAL scan"); + + // Advance the checkpoint to the END of the WAL (after the second entry) + // The clamped global_data_offset equals total_logical_bytes from before the second append. + // After appending, we can checkpoint to the new end, which is beyond the clamped value. + let wal_len = std::fs::metadata(&wal_path).expect("metadata").len(); + let cursor = WalConsumerCheckpoint { + safe_offset: wal_len, + safe_sequence: offset.sequence, + }; + writer + .checkpoint_cursor(&cursor) + .expect("advance checkpoint to end of WAL"); + drop(writer); + + // Verify the sidecar now has a valid offset + let recovered_sidecar = CheckpointSidecar::read_from(&sidecar_path).expect("read sidecar"); + let max_logical = wal_len.saturating_sub(WAL_HEADER_LEN as u64); + assert!( + recovered_sidecar.global_data_offset <= max_logical, + "sidecar offset {} should be within actual WAL data {}", + recovered_sidecar.global_data_offset, + max_logical + ); +} + +#[test] +fn wal_recovery_handles_rotated_files_with_gaps_in_ids() { + // Test that recovery handles rotated files with non-contiguous IDs (e.g., wal.1, wal.5, wal.12) + let (_dir, wal_path) = temp_wal("rotation_gaps.wal"); + + let descriptor = logs_descriptor(); + + // Manually create rotated files with gaps in their IDs + // We'll create wal.2 and wal.7 (skipping 1, 3-6) + for rotation_id in [2u64, 7u64] { + let rotated_path = rotated_path_for(&wal_path, rotation_id as usize); + let mut file = std::fs::OpenOptions::new() + .create(true) + .write(true) + .truncate(true) + .open(&rotated_path) + .expect("create rotated file"); + WalHeader::new([0x77; 16]) + .write_to(&mut file) + .expect("write header"); + file.flush().expect("flush"); + } + + // Create the active WAL file + { + let mut writer = WalWriter::open(WalWriterOptions::new( + wal_path.clone(), + [0x77; 16], + FlushPolicy::Immediate, + )) + .expect("writer"); + + let bundle = single_slot_bundle(&descriptor, 0x01, &[1]); + let _ = writer.append_bundle(&bundle).expect("append"); + } + + // Reopen and verify the writer picks up where it left off + let mut writer = WalWriter::open( + WalWriterOptions::new(wal_path.clone(), [0x77; 16], FlushPolicy::Immediate) + .with_rotation_target(1) + .with_max_rotated_files(10), + ) + .expect("writer with gap-id rotations"); + + // Append and trigger a rotation + let bundle = single_slot_bundle(&descriptor, 0x02, &[2, 3, 4, 5]); + let _ = writer + .append_bundle(&bundle) + .expect("append triggers rotation"); + drop(writer); + + // The new rotation should use ID 8 (max existing + 1 = 7 + 1) + let expected_new_rotation = rotated_path_for(&wal_path, 8); + assert!( + expected_new_rotation.exists(), + "new rotation should use ID 8 (after existing max of 7)" + ); + + // Verify wal.2 and wal.7 still exist + assert!(rotated_path_for(&wal_path, 2).exists()); + assert!(rotated_path_for(&wal_path, 7).exists()); +} + +#[test] +#[ignore] // Run manually: cargo test wal_recovery_scan_benchmark --release -- --ignored --nocapture +fn wal_recovery_scan_benchmark() { + use std::time::Instant; + + let (_dir, wal_path) = temp_wal("benchmark.wal"); + let descriptor = logs_descriptor(); + let hash = [0xBE; 16]; + + // Create entries with realistic payload sizes + // Typical telemetry batch might be ~1-4 KB + let payload: Vec = (0..256).collect(); // ~2KB payload + + // Test 1: Single file (no rotation) + println!("\n=== WAL Recovery Benchmark ===\n"); + println!("--- Test 1: Single file (64 MB, no rotation) ---"); + + let mut writer = WalWriter::open( + WalWriterOptions::new(wal_path.clone(), hash, FlushPolicy::Immediate) + .with_rotation_target(512 * 1024 * 1024) // 512 MB - no rotation + .with_max_wal_size(512 * 1024 * 1024), + ) + .expect("writer"); + + let target_bytes = 64 * 1024 * 1024u64; + let mut total_bytes = 0u64; + let mut entry_count = 0u64; + + while total_bytes < target_bytes { + let bundle = single_slot_bundle(&descriptor, 0x01, payload.as_slice()); + let offset = writer.append_bundle(&bundle).expect("append"); + total_bytes = offset.position; + entry_count += 1; + } + drop(writer); + + println!( + "Total data: {} MB, {} entries", + total_bytes / 1024 / 1024, + entry_count + ); + + let recovery_start = Instant::now(); + let _writer = WalWriter::open(WalWriterOptions::new( + wal_path.clone(), + hash, + FlushPolicy::Immediate, + )) + .expect("reopen"); + let recovery_elapsed = recovery_start.elapsed(); + + println!("Recovery time: {:?}", recovery_elapsed); + println!( + "Scan rate: {:.2} MB/s", + (total_bytes as f64 / 1024.0 / 1024.0) / recovery_elapsed.as_secs_f64() + ); + + // Test 2: Multiple rotated files + let (_dir2, wal_path2) = temp_wal("benchmark_rotated.wal"); + println!("\n--- Test 2: With rotation (8 MB per file, ~8 rotated files) ---"); + + let mut writer = WalWriter::open( + WalWriterOptions::new(wal_path2.clone(), hash, FlushPolicy::Immediate) + .with_rotation_target(8 * 1024 * 1024) // 8 MB per file + .with_max_rotated_files(16) // Allow enough rotated files + .with_max_wal_size(512 * 1024 * 1024), + ) + .expect("writer"); + + entry_count = 0; + // Write a fixed number of entries to get ~64 MB total + let entries_for_64mb = 26000u64; + for _ in 0..entries_for_64mb { + let bundle = single_slot_bundle(&descriptor, 0x01, payload.as_slice()); + let _ = writer.append_bundle(&bundle).expect("append"); + entry_count += 1; + } + drop(writer); + + // Count rotated files + let rotated_count = std::fs::read_dir(wal_path2.parent().unwrap()) + .unwrap() + .filter(|e| { + e.as_ref() + .unwrap() + .file_name() + .to_str() + .unwrap() + .contains(".wal.") + }) + .count(); + println!( + "Total entries: {}, {} rotated files", + entry_count, rotated_count + ); + + // Calculate total size from all files + let total_size: u64 = std::fs::read_dir(wal_path2.parent().unwrap()) + .unwrap() + .filter_map(|e| e.ok()) + .filter(|e| { + e.file_name() + .to_str() + .unwrap() + .contains("benchmark_rotated") + }) + .map(|e| e.metadata().unwrap().len()) + .sum(); + println!( + "Total data across all files: {} MB", + total_size / 1024 / 1024 + ); + + let recovery_start = Instant::now(); + let _writer = WalWriter::open( + WalWriterOptions::new(wal_path2.clone(), hash, FlushPolicy::Immediate) + .with_rotation_target(8 * 1024 * 1024) + .with_max_rotated_files(16), + ) + .expect("reopen"); + let recovery_elapsed = recovery_start.elapsed(); + + println!("Recovery time: {:?}", recovery_elapsed); + println!( + "Note: Only scans active file (~8 MB), not {} rotated files", + rotated_count + ); + println!("\n===============================\n"); +} diff --git a/rust/otap-dataflow/crates/quiver/src/wal/writer.rs b/rust/otap-dataflow/crates/quiver/src/wal/writer.rs new file mode 100644 index 000000000..764335a6d --- /dev/null +++ b/rust/otap-dataflow/crates/quiver/src/wal/writer.rs @@ -0,0 +1,1187 @@ +// Copyright The OpenTelemetry Authors +// SPDX-License-Identifier: Apache-2.0 + +//! Write-ahead-log (WAL) writer for Quiver. +//! +//! The writer is responsible for appending Arrow payloads, rotating WAL +//! files when size thresholds are exceeded, and reclaiming durable space once +//! downstream consumers advance the consumer checkpoint. Safe offsets are always +//! validated against real entry boundaries so we never expose partially written +//! frames to readers. +//! +//! # WAL lifecycle at a glance +//! +//! ```text +//! ingest thread wal writer +//! ─────────────────────────────────────────────────────────────────────────── +//! RecordBundle -> encode slots -> entry header -> fs writes -> fsync +//! | | +//! v v +//! consumer checkpoint <── sidecar flush +//! ``` +//! +//! * **Append path** – Every [`RecordBundle`] is serialized slot-by-slot into an +//! in-memory buffer, hashed (CRC32), and written as one atomic entry. We track +//! [`WalOffset`] so readers can correlate persisted bytes back to sequences. +//! * **Flush loop** – `flush_interval` and `max_unflushed_bytes` control when we +//! call `flush()`/`sync_data()`. Tests can inspect thread-local flags to assert +//! that durability barriers occurred. +//! * **Rotation** – When `rotation_target_bytes` is exceeded the active WAL file +//! is renamed to `wal.n` (oldest files slide toward higher numbers) and a new +//! header is seeded in-place. [`RotatedWalFile`] metadata keeps track of logical +//! data ranges so stale files can be deleted after the checkpoint advances. +//! * **Checkpoint + sidecar** – Readers compute `WalConsumerCheckpoint` values. The +//! writer validates checkpoint boundaries, updates the checkpoint sidecar on disk, +//! and relies on file rotation to eventually drop fully consumed bytes. The +//! sidecar survives crashes so restarts resume from the last known safe offset +//! even if a rotation has not run yet. +//! * **Global cap enforcement** – `max_wal_size` is measured across the active +//! file plus rotated files. Exceeding the cap yields `WalAtCapacity` errors +//! until downstream consumers free space. +//! +//! ## Offset coordinate systems (internal) +//! +//! > **Note for operators:** You don't need to understand this section. +//! > Just use `WalConsumerCheckpoint::advance()` to track progress. +//! +//! Internally, checkpoint metadata uses two coordinate spaces: +//! +//! | Coordinate | Measures | Stored in | +//! |-----------------------|-----------------------------------|------------------------------------| +//! | `global_data_offset` | Total data bytes consumed across | `checkpoint.offset` sidecar | +//! | | the entire WAL (headers excluded) | (stable across rotations) | +//! | per-file offset | Byte position within the active | Derived at runtime via | +//! | | WAL file (includes header) | `to_active_file_offset()` | +//! +//! Each [`RotatedWalFile`] stores a `cumulative_data_offset` representing the +//! global data offset at the *end* of that file. This lets `purge_rotated_files` +//! delete a file once `global_data_offset` exceeds its cumulative boundary. +//! +//! **Conversion:** `to_global_offset(file_offset)` adds data bytes from rotated +//! files to the per-file data position. `to_active_file_offset(global)` reverses +//! the transformation, clamping to the active file's bounds. +//! +//! ## Lifecycle events +//! +//! | Event | Trigger | +//! |---------------------|------------------------------------------------------| +//! | **Rotation** | Active file exceeds `rotation_target_bytes` | +//! | **Purge** | `global_data_offset` passes a rotated file's cumulative end | +//! | **Backpressure** | `aggregate_bytes > max_wal_size` or rotated file cap hit | +//! +//! ## Internal structure +//! +//! * [`WalWriter`] – the façade exposed to call sites. It orchestrates appends, +//! compaction, and recovery by delegating to the pieces below. +//! * [`ActiveWalFile`] – owns the on-disk file handle plus the transient payload +//! buffer. All direct I/O (seek, write, flush/fsync) flows through this type +//! so we only mutate the OS file descriptor in one place. +//! * [`WalCoordinator`] – tracks policy decisions (rotation, global caps, and +//! checkpoint sidecar state). It never touches raw I/O directly; instead it +//! inspects `ActiveWalFile` lengths and requests actions. +//! +//! Splitting responsibilities this way keeps the append path easier to reason +//! about: `WalWriter` asks the coordinator for admission, streams bytes via the +//! active file, then records whatever book-keeping the coordinator dictates. +//! +//! ## Startup and recovery +//! +//! On [`WalWriter::open`]: +//! +//! 1. Read `checkpoint.offset` sidecar → recover `global_data_offset`. +//! 2. Scan for rotated files (`wal.N`) and rebuild the `rotated_files` queue +//! with cumulative offsets, sorted by rotation id (oldest first). +//! 3. Convert `global_data_offset` → per-file offset inside the active WAL. +//! 4. Detect the highest sequence number across all files; resume from +//! `highest + 1`. +//! 5. Position the file cursor at EOF and accept new appends. +//! +//! If the sidecar is missing or corrupt, we fall back to starting from offset +//! zero and scanning all entries. +//! +//! ## Testing hooks +//! +//! * Failpoints in `test_support` simulate crashes before the checkpoint sidecar +//! rename so we can verify idempotent recovery. +//! * `test_force_crash` skips drop-time flushes to model abrupt exits. +//! +//! See the reader module for how [`WalConsumerCheckpoint`] values are derived +//! from WAL entries. +#![allow(dead_code)] + +use std::collections::VecDeque; +use std::fs::{File, OpenOptions}; +use std::io::{ErrorKind, Read, Seek, SeekFrom, Write}; +use std::path::{Path, PathBuf}; +use std::time::{Duration, Instant, SystemTime, UNIX_EPOCH}; + +use arrow_array::RecordBatch; +use arrow_ipc::writer::StreamWriter; +use crc32fast::Hasher; + +use crate::record_bundle::{PayloadRef, RecordBundle, SlotId}; + +use super::checkpoint_sidecar::CheckpointSidecar; +use super::header::{WAL_HEADER_LEN, WalHeader}; +use super::reader::WalReader; +use super::{ + ENTRY_HEADER_LEN, ENTRY_TYPE_RECORD_BUNDLE, MAX_ROTATION_TARGET_BYTES, WalConsumerCheckpoint, + WalError, WalResult, +}; + +// --------------------------------------------------------------------------- +// Default configuration constants +// +// These defaults balance durability, I/O efficiency, and recovery time. See +// ARCHITECTURE.md § "Configuration defaults and rationale" for background. +// --------------------------------------------------------------------------- + +/// Default maximum aggregate size of active + rotated WAL files. +/// +/// `u64::MAX` means unlimited—backpressure is only applied by +/// `max_rotated_files`. +pub const DEFAULT_MAX_WAL_SIZE: u64 = u64::MAX; + +/// Default maximum number of rotated WAL files to retain. +/// +/// New rotations are blocked when this limit is reached; the writer returns +/// `WalAtCapacity` until the checkpoint advances and older files are purged. +/// Eight rotated files is a reasonable trade-off: it keeps enough history for +/// slow consumers while limiting disk footprint and recovery scan time. +pub const DEFAULT_MAX_ROTATED_FILES: usize = 8; + +/// Default size threshold (in bytes) that triggers WAL file rotation. +/// +/// 64 MiB keeps individual files manageable for sequential scans while +/// amortizing rotation overhead (header writes, renames). +pub const DEFAULT_ROTATION_TARGET_BYTES: u64 = 64 * 1024 * 1024; + +/// Controls when the WAL flushes data to disk. +/// +/// Flushing calls `fsync()` to ensure data reaches stable storage. More frequent +/// flushes improve durability but reduce throughput. +#[derive(Debug, Clone, Default)] +pub(crate) enum FlushPolicy { + /// Flush after every write. Safest but slowest. + #[default] + Immediate, + /// Flush when unflushed bytes exceed the threshold. + EveryNBytes(u64), + /// Flush when elapsed time since last flush exceeds the duration. + EveryDuration(Duration), + /// Flush when either bytes or duration threshold is exceeded. + /// This is the recommended policy for production. + BytesOrDuration { bytes: u64, duration: Duration }, +} + +/// Low-level tunables for the WAL writer. +/// +/// Most users should use [`WalWriterOptions::new()`] with defaults, then +/// customize via the builder methods: +/// +/// ```ignore +/// let options = WalWriterOptions::new(path, hash, FlushPolicy::Immediate) +/// .with_max_wal_size(1024 * 1024 * 1024) // 1 GB cap +/// .with_rotation_target_bytes(64 * 1024 * 1024); // 64 MB per file +/// ``` +#[derive(Debug, Clone)] +pub(crate) struct WalWriterOptions { + /// Path to the active WAL file (e.g., `wal/quiver.wal`). + pub path: PathBuf, + /// Hash of segment configuration; mismatches reject existing files. + pub segment_cfg_hash: [u8; 16], + /// When to call `fsync()` for durability. + pub flush_policy: FlushPolicy, + /// Maximum aggregate size of active + rotated files (default: unlimited). + pub max_wal_size: u64, + /// Maximum number of rotated files to keep (default: 8). + pub max_rotated_files: usize, + /// Rotate the active file when it exceeds this size (default: 64 MB). + pub rotation_target_bytes: u64, +} + +impl WalWriterOptions { + pub fn new(path: PathBuf, segment_cfg_hash: [u8; 16], flush_policy: FlushPolicy) -> Self { + Self { + path, + segment_cfg_hash, + flush_policy, + max_wal_size: DEFAULT_MAX_WAL_SIZE, + max_rotated_files: DEFAULT_MAX_ROTATED_FILES, + rotation_target_bytes: DEFAULT_ROTATION_TARGET_BYTES, + } + } + + pub fn with_flush_policy(mut self, policy: FlushPolicy) -> Self { + self.flush_policy = policy; + self + } + + pub fn with_max_wal_size(mut self, max_bytes: u64) -> Self { + self.max_wal_size = max_bytes; + self + } + + pub fn with_max_rotated_files(mut self, max_files: usize) -> Self { + self.max_rotated_files = max_files.max(1); + self + } + + pub fn with_rotation_target(mut self, target_bytes: u64) -> Self { + self.rotation_target_bytes = target_bytes.clamp(1, MAX_ROTATION_TARGET_BYTES); + self + } +} + +/// Stateful writer that maintains append position, rotation metadata, and +/// persisted consumer checkpoints. It tracks both the *current* WAL file and +/// any rotated files so the total on-disk footprint (`aggregate_bytes`) can be +/// compared against caps before admitting new entries. +#[derive(Debug)] +pub(crate) struct WalWriter { + /// Encapsulates the active WAL file handle plus its in-memory staging + /// buffer. All raw I/O flows through this helper. + active_file: ActiveWalFile, + /// Manages policy, checkpoint bookkeeping, and rotation metadata. + coordinator: WalCoordinator, + /// Next sequence number to assign to an appended entry. + next_sequence: u64, + #[cfg(test)] + /// When true we skip drop-time flushes to simulate a crash. + test_crashed: bool, +} + +#[derive(Debug)] +struct ActiveWalFile { + /// Active WAL file descriptor. + file: File, + /// Scratch buffer used to serialize slot payloads before writing. + payload_buffer: Vec, + /// Timestamp of the most recent flush. + last_flush: Instant, + /// Bytes written since the last flush. + unflushed_bytes: u64, + /// Current logical file length (includes header and data bytes). + current_len: u64, +} + +#[derive(Debug)] +struct WalCoordinator { + /// User-provided tuning knobs for the writer. + options: WalWriterOptions, + /// Path to the checkpoint sidecar file on disk. + sidecar_path: PathBuf, + /// Cached copy of the persisted checkpoint offset + rotation generation. + checkpoint_state: CheckpointSidecar, + /// Total bytes across the active WAL plus all rotated files. + aggregate_bytes: u64, + /// Metadata describing each rotated `wal.N` file on disk, ordered oldest-to-newest. + rotated_files: VecDeque, + /// Next rotation id to use when rotating (initialized to max existing + 1). + next_rotation_id: u64, + /// Most recent offset validated to be on an entry boundary in the active file. + active_file_checkpoint_offset: u64, + /// Sequence number associated with the last committed consumer checkpoint. + last_checkpoint_sequence: Option, +} + +/// Opaque marker returned after an append. +/// +/// Contains the byte position and sequence number of the written entry. +/// Useful for correlating writer positions with reader cursors during testing +/// or debugging. Operators typically don't need to inspect these values. +#[derive(Debug, Clone, Copy, PartialEq, Eq)] +pub(crate) struct WalOffset { + /// Byte offset where the entry starts in the active WAL file. + pub position: u64, + /// Monotonically increasing sequence number assigned to this entry. + pub sequence: u64, +} + +impl WalWriter { + pub fn open(options: WalWriterOptions) -> WalResult { + if let Some(parent) = options.path.parent() { + std::fs::create_dir_all(parent)?; + } + + let mut file = OpenOptions::new() + .create(true) + .read(true) + .write(true) + .truncate(false) + .open(&options.path)?; + + let metadata = file.metadata()?; + let is_new_file = metadata.len() == 0; + + if is_new_file { + let header = WalHeader::new(options.segment_cfg_hash); + header.write_to(&mut file)?; + file.flush()?; + } else if metadata.len() < WAL_HEADER_LEN as u64 { + return Err(WalError::InvalidHeader("file smaller than header")); + } else { + let header = WalHeader::read_from(&mut file)?; + if header.segment_cfg_hash != options.segment_cfg_hash { + return Err(WalError::SegmentConfigMismatch { + expected: options.segment_cfg_hash, + found: header.segment_cfg_hash, + }); + } + } + + let sidecar_path = checkpoint_sidecar_path(&options.path); + let checkpoint_state = load_checkpoint_state(&sidecar_path)?; + + // Create coordinator first to scan for valid entries + let mut coordinator = WalCoordinator::new( + options, + sidecar_path, + checkpoint_state, + WAL_HEADER_LEN as u64, + ); + coordinator.reload_rotated_files(metadata.len())?; + + // Scan to find the last valid entry and truncate any trailing garbage + let (next_sequence, valid_offset) = coordinator.detect_next_sequence()?; + let current_file_len = file.metadata()?.len(); + + if valid_offset < current_file_len { + // Truncate trailing garbage from a partial write (e.g., crash mid-write) + file.set_len(valid_offset)?; + file.sync_all()?; + } + + // Position at the end of valid data + let _ = file.seek(SeekFrom::Start(valid_offset))?; + + let active_file = ActiveWalFile::new(file, valid_offset); + coordinator.restore_checkpoint_offsets(active_file.len()); + coordinator.recalculate_aggregate_bytes(active_file.len()); + + Ok(Self { + active_file, + coordinator, + next_sequence, + #[cfg(test)] + test_crashed: false, + }) + } + + /// Serializes a [`RecordBundle`] into the active WAL file and returns the + /// byte offset + sequence number associated with the entry. The writer keeps + /// internal counters so the next call knows when to flush, rotate, or apply + /// global caps. + pub fn append_bundle(&mut self, bundle: &B) -> WalResult { + let descriptor = bundle.descriptor(); + let ingestion_time = bundle.ingestion_time(); + let ingestion_ts_nanos = system_time_to_nanos(ingestion_time)?; + + self.active_file.payload_buffer.clear(); + + let sequence = self.next_sequence; + + let mut slot_bitmap = 0u64; + + // Single pass: validate slots and write directly to payload_buffer. + // The buffer is reused across calls, so after a few appends it stabilizes + // at a capacity that fits typical bundles without reallocation. + for slot in &descriptor.slots { + let slot_index = slot.id.0 as usize; + if slot_index >= 64 { + return Err(WalError::SlotOutOfRange(slot.id)); + } + if let Some(payload) = bundle.payload(slot.id) { + slot_bitmap |= 1u64 << slot_index; + self.encode_slot_into_buffer(slot.id, payload)?; + } + } + + let mut entry_header = [0u8; ENTRY_HEADER_LEN]; + let mut cursor = 0; + entry_header[cursor] = ENTRY_TYPE_RECORD_BUNDLE; + cursor += 1; + entry_header[cursor..cursor + 8].copy_from_slice(&ingestion_ts_nanos.to_le_bytes()); + cursor += 8; + entry_header[cursor..cursor + 8].copy_from_slice(&sequence.to_le_bytes()); + cursor += 8; + entry_header[cursor..cursor + 8].copy_from_slice(&slot_bitmap.to_le_bytes()); + + let entry_body_len = ENTRY_HEADER_LEN + self.active_file.payload_buffer.len(); + let entry_len = + u32::try_from(entry_body_len).map_err(|_| WalError::EntryTooLarge(entry_body_len))?; + + let mut hasher = Hasher::new(); + hasher.update(&entry_header); + hasher.update(&self.active_file.payload_buffer); + let crc = hasher.finalize(); + + let entry_total_bytes = 4u64 + u64::from(entry_len) + 4; + self.coordinator + .preflight_append(&self.active_file, entry_total_bytes)?; + + let mut payload_bytes = std::mem::take(&mut self.active_file.payload_buffer); + let entry_start = + self.active_file + .write_entry(entry_len, &entry_header, &payload_bytes, crc)?; + payload_bytes.clear(); + self.active_file.payload_buffer = payload_bytes; + + self.next_sequence = self.next_sequence.wrapping_add(1); + + self.active_file.current_len = self + .active_file + .current_len + .saturating_add(entry_total_bytes); + self.coordinator.record_append(entry_total_bytes); + self.active_file + .maybe_flush(&self.coordinator.options().flush_policy, entry_total_bytes)?; + + self.coordinator + .maybe_rotate_after_append(&mut self.active_file)?; + + Ok(WalOffset { + position: entry_start, + sequence, + }) + } + + /// Persists the cursor position and enables cleanup of consumed WAL data. + /// + /// This validates the cursor, writes to the checkpoint sidecar with fsync, + /// and purges any rotated files fully covered by the new position. + /// + /// Call this after downstream has confirmed durability (e.g., segment flush). + pub(crate) fn checkpoint_cursor( + &mut self, + checkpoint: &WalConsumerCheckpoint, + ) -> WalResult<()> { + self.coordinator + .checkpoint_cursor(&mut self.active_file, checkpoint) + } + + /// Encodes a slot directly into `payload_buffer`, avoiding intermediate allocations. + /// + /// Writes the slot header (id, fingerprint, row_count, payload_len) followed by + /// the Arrow IPC-encoded payload bytes. + fn encode_slot_into_buffer( + &mut self, + slot_id: SlotId, + payload: PayloadRef<'_>, + ) -> WalResult<()> { + let row_count = u32::try_from(payload.batch.num_rows()) + .map_err(|_| WalError::RowCountOverflow(payload.batch.num_rows()))?; + let payload_bytes = encode_record_batch(payload.batch)?; + let payload_len = u32::try_from(payload_bytes.len()) + .map_err(|_| WalError::PayloadTooLarge(payload_bytes.len()))?; + + let buf = &mut self.active_file.payload_buffer; + + // Write slot header + buf.extend_from_slice(&slot_id.0.to_le_bytes()); + buf.extend_from_slice(&payload.schema_fingerprint); + buf.extend_from_slice(&row_count.to_le_bytes()); + buf.extend_from_slice(&payload_len.to_le_bytes()); + + // Write payload + buf.extend_from_slice(&payload_bytes); + + Ok(()) + } +} + +impl ActiveWalFile { + fn new(file: File, current_len: u64) -> Self { + Self { + file, + payload_buffer: Vec::new(), + last_flush: Instant::now(), + unflushed_bytes: 0, + current_len, + } + } + + fn len(&self) -> u64 { + self.current_len + } + + fn seek_to_end(&mut self) -> WalResult { + let pos = self.file.seek(SeekFrom::End(0))?; + Ok(pos) + } + + fn file_mut(&mut self) -> &mut File { + &mut self.file + } + + fn set_len(&mut self, len: u64) { + self.current_len = len; + } + + fn replace_file(&mut self, file: File, new_len: u64) { + self.file = file; + self.current_len = new_len; + self.last_flush = Instant::now(); + self.unflushed_bytes = 0; + } + + /// Writes a complete WAL entry to disk and returns the starting byte offset. + /// + /// Entry layout: + /// ```text + /// ┌──────────┬──────────────┬─────────────────┬──────────┐ + /// │ u32 len │ entry header │ payload bytes │ u32 crc │ + /// └──────────┴──────────────┴─────────────────┴──────────┘ + /// ``` + fn write_entry( + &mut self, + entry_len: u32, + entry_header: &[u8; ENTRY_HEADER_LEN], + payload: &[u8], + crc: u32, + ) -> WalResult { + let entry_start = self.seek_to_end()?; + self.file.write_all(&entry_len.to_le_bytes())?; + self.file.write_all(entry_header)?; + self.file.write_all(payload)?; + self.file.write_all(&crc.to_le_bytes())?; + Ok(entry_start) + } + + fn maybe_flush(&mut self, policy: &FlushPolicy, bytes_written: u64) -> WalResult<()> { + self.unflushed_bytes = self.unflushed_bytes.saturating_add(bytes_written); + + match policy { + FlushPolicy::Immediate => self.flush_now(), + FlushPolicy::EveryNBytes(threshold) => { + if self.unflushed_bytes >= *threshold { + self.flush_now() + } else { + Ok(()) + } + } + FlushPolicy::EveryDuration(interval) => { + if self.last_flush.elapsed() >= *interval { + self.flush_now() + } else { + Ok(()) + } + } + FlushPolicy::BytesOrDuration { bytes, duration } => { + if self.unflushed_bytes >= *bytes || self.last_flush.elapsed() >= *duration { + self.flush_now() + } else { + Ok(()) + } + } + } + } + + fn flush_now(&mut self) -> WalResult<()> { + self.file.flush()?; + sync_file_data(&self.file)?; + self.last_flush = Instant::now(); + self.unflushed_bytes = 0; + Ok(()) + } +} + +impl WalCoordinator { + fn new( + options: WalWriterOptions, + sidecar_path: PathBuf, + checkpoint_state: CheckpointSidecar, + current_len: u64, + ) -> Self { + Self { + options, + sidecar_path, + checkpoint_state, + aggregate_bytes: current_len, + rotated_files: VecDeque::new(), + next_rotation_id: 1, + active_file_checkpoint_offset: WAL_HEADER_LEN as u64, + last_checkpoint_sequence: None, + } + } + + /// Returns the total data bytes (excluding headers) across all rotated files. + /// Derived from the last rotated file's cumulative offset. + fn rotated_data_bytes(&self) -> u64 { + self.rotated_files + .back() + .map_or(0, |f| f.cumulative_data_offset) + } + + fn options(&self) -> &WalWriterOptions { + &self.options + } + + fn reload_rotated_files(&mut self, active_len: u64) -> WalResult<()> { + let discovered = discover_rotated_wal_files(&self.options.path)?; + if discovered.is_empty() { + self.aggregate_bytes = active_len; + self.rotated_files.clear(); + self.next_rotation_id = 1; + return Ok(()); + } + + // Files are returned sorted oldest-to-newest by rotation_id + let mut files = VecDeque::with_capacity(discovered.len()); + let mut aggregate = active_len; + let mut cumulative_data = 0u64; + let mut max_rotation_id = 0u64; + + for (rotation_id, len) in &discovered { + aggregate = aggregate.saturating_add(*len); + let data_bytes = len.saturating_sub(WAL_HEADER_LEN as u64); + cumulative_data = cumulative_data.saturating_add(data_bytes); + files.push_back(RotatedWalFile { + path: rotated_wal_path(&self.options.path, *rotation_id), + rotation_id: *rotation_id, + file_bytes: *len, + cumulative_data_offset: cumulative_data, + }); + max_rotation_id = max_rotation_id.max(*rotation_id); + } + + self.rotated_files = files; + self.aggregate_bytes = aggregate; + self.next_rotation_id = max_rotation_id.saturating_add(1); + Ok(()) + } + + fn restore_checkpoint_offsets(&mut self, active_len: u64) { + let header = WAL_HEADER_LEN as u64; + let active_data = active_len.saturating_sub(header); + let total_logical = self.rotated_data_bytes().saturating_add(active_data); + + if self.checkpoint_state.global_data_offset > total_logical { + self.checkpoint_state.global_data_offset = total_logical; + } + + self.active_file_checkpoint_offset = + self.to_active_file_offset(self.checkpoint_state.global_data_offset, active_len); + } + + fn record_append(&mut self, entry_total_bytes: u64) { + self.aggregate_bytes = self.aggregate_bytes.saturating_add(entry_total_bytes); + } + + fn preflight_append( + &mut self, + active_file: &ActiveWalFile, + entry_total_bytes: u64, + ) -> WalResult<()> { + let will_rotate = active_file + .len() + .saturating_add(entry_total_bytes) + .saturating_sub(WAL_HEADER_LEN as u64) + > self.options.rotation_target_bytes; + + if will_rotate && self.rotated_files.len() >= self.options.max_rotated_files { + return Err(WalError::WalAtCapacity( + "rotated wal file cap reached; advance checkpoint before rotating", + )); + } + + let mut projected = self.aggregate_bytes.saturating_add(entry_total_bytes); + if will_rotate { + projected = projected.saturating_add(WAL_HEADER_LEN as u64); + } + + if projected > self.options.max_wal_size { + return Err(WalError::WalAtCapacity( + "wal size cap exceeded; advance checkpoint to reclaim space", + )); + } + + Ok(()) + } + + fn checkpoint_cursor( + &mut self, + active_file: &mut ActiveWalFile, + checkpoint: &WalConsumerCheckpoint, + ) -> WalResult<()> { + let safe_offset = self.resolve_consumer_checkpoint(active_file, checkpoint)?; + self.persist_checkpoint(checkpoint, safe_offset) + } + + fn resolve_consumer_checkpoint( + &mut self, + active_file: &mut ActiveWalFile, + checkpoint: &WalConsumerCheckpoint, + ) -> WalResult { + let requested_offset = checkpoint.safe_offset.max(WAL_HEADER_LEN as u64); + let file_len = active_file.file_mut().metadata()?.len(); + if requested_offset > file_len { + return Err(WalError::InvalidConsumerCheckpoint( + "safe offset beyond wal tail", + )); + } + if let Some(last_seq) = self.last_checkpoint_sequence { + if checkpoint.safe_sequence < last_seq { + return Err(WalError::InvalidConsumerCheckpoint( + "safe sequence regressed", + )); + } + } + self.ensure_entry_boundary(active_file, requested_offset)?; + Ok(requested_offset) + } + + fn ensure_entry_boundary( + &mut self, + active_file: &mut ActiveWalFile, + target: u64, + ) -> WalResult<()> { + if target == self.active_file_checkpoint_offset { + return Ok(()); + } + if target < self.active_file_checkpoint_offset { + return Err(WalError::InvalidConsumerCheckpoint("safe offset regressed")); + } + + let original_pos = active_file.file_mut().stream_position()?; + let mut cursor = self.active_file_checkpoint_offset; + let _ = active_file.file_mut().seek(SeekFrom::Start(cursor))?; + while cursor < target { + let mut len_buf = [0u8; 4]; + active_file.file_mut().read_exact(&mut len_buf)?; + let entry_len = u32::from_le_bytes(len_buf) as u64; + let entry_total = 4u64 + .checked_add(entry_len) + .and_then(|val| val.checked_add(4)) + .ok_or(WalError::InvalidConsumerCheckpoint("entry length overflow"))?; + cursor = cursor + .checked_add(entry_total) + .ok_or(WalError::InvalidConsumerCheckpoint("safe offset overflow"))?; + if cursor > target { + let _ = active_file.file_mut().seek(SeekFrom::Start(original_pos))?; + return Err(WalError::InvalidConsumerCheckpoint( + "safe offset splits entry boundary", + )); + } + let _ = active_file + .file_mut() + .seek(SeekFrom::Current(entry_len as i64 + 4))?; + } + let _ = active_file.file_mut().seek(SeekFrom::Start(original_pos))?; + Ok(()) + } + + fn persist_checkpoint( + &mut self, + checkpoint: &WalConsumerCheckpoint, + recorded_offset: u64, + ) -> WalResult<()> { + let global_offset = self.to_global_offset(recorded_offset); + if global_offset < self.checkpoint_state.global_data_offset { + return Err(WalError::InvalidConsumerCheckpoint("safe offset regressed")); + } + self.record_global_offset(global_offset)?; + self.active_file_checkpoint_offset = recorded_offset; + self.last_checkpoint_sequence = Some(checkpoint.safe_sequence); + self.purge_rotated_files() + } + + fn record_global_offset(&mut self, global_offset: u64) -> WalResult<()> { + if self.checkpoint_state.global_data_offset == global_offset && self.sidecar_path.exists() { + return Ok(()); + } + self.checkpoint_state.global_data_offset = global_offset; + CheckpointSidecar::write_to(&self.sidecar_path, &self.checkpoint_state) + } + + fn maybe_rotate_after_append(&mut self, active_file: &mut ActiveWalFile) -> WalResult<()> { + let active_data_bytes = active_file.len().saturating_sub(WAL_HEADER_LEN as u64); + if active_data_bytes <= self.options.rotation_target_bytes { + return Ok(()); + } + self.rotate_active_file(active_file) + } + + fn rotate_active_file(&mut self, active_file: &mut ActiveWalFile) -> WalResult<()> { + if self.rotated_files.len() >= self.options.max_rotated_files { + return Err(WalError::WalAtCapacity( + "rotated wal file cap reached; advance checkpoint before rotating", + )); + } + + active_file.flush_now()?; + let old_len = active_file.len(); + if old_len <= WAL_HEADER_LEN as u64 { + return Ok(()); + } + self.aggregate_bytes = self.aggregate_bytes.saturating_sub(old_len); + + // Use monotonic naming: rename to wal.{next_rotation_id} + let rotation_id = self.next_rotation_id; + self.next_rotation_id = self.next_rotation_id.saturating_add(1); + + let new_rotated_path = rotated_wal_path(&self.options.path, rotation_id); + std::fs::rename(&self.options.path, &new_rotated_path)?; + sync_parent_dir(&self.options.path)?; + + let data_bytes = old_len.saturating_sub(WAL_HEADER_LEN as u64); + let cumulative_data_offset = self.rotated_data_bytes().saturating_add(data_bytes); + self.rotated_files.push_back(RotatedWalFile { + path: new_rotated_path, + rotation_id, + file_bytes: old_len, + cumulative_data_offset, + }); + self.aggregate_bytes = self.aggregate_bytes.saturating_add(old_len); + + let mut file = reopen_wal_file(&self.options.path, self.options.segment_cfg_hash)?; + let _ = file.seek(SeekFrom::End(0))?; // ensure positioned at end + active_file.replace_file(file, WAL_HEADER_LEN as u64); + self.aggregate_bytes = self.aggregate_bytes.saturating_add(active_file.len()); + self.active_file_checkpoint_offset = WAL_HEADER_LEN as u64; + + CheckpointSidecar::write_to(&self.sidecar_path, &self.checkpoint_state)?; + Ok(()) + } + + fn purge_rotated_files(&mut self) -> WalResult<()> { + while let Some(front) = self.rotated_files.front() { + if front.cumulative_data_offset <= self.checkpoint_state.global_data_offset { + std::fs::remove_file(&front.path)?; + self.aggregate_bytes = self.aggregate_bytes.saturating_sub(front.total_bytes()); + let _ = self.rotated_files.pop_front(); + } else { + break; + } + } + Ok(()) + } + + fn recalculate_aggregate_bytes(&mut self, active_len: u64) { + let rotated_total: u64 = self.rotated_files.iter().map(|f| f.total_bytes()).sum(); + self.aggregate_bytes = rotated_total.saturating_add(active_len); + } + + fn to_global_offset(&self, file_offset: u64) -> u64 { + let data_offset = file_offset.saturating_sub(WAL_HEADER_LEN as u64); + self.rotated_data_bytes().saturating_add(data_offset) + } + + fn to_active_file_offset(&self, global_offset: u64, active_len: u64) -> u64 { + let header = WAL_HEADER_LEN as u64; + let active_data_len = active_len.saturating_sub(header); + let rotated_data = self.rotated_data_bytes(); + + if global_offset <= rotated_data { + return header; + } + + let data_within_active = global_offset + .saturating_sub(rotated_data) + .min(active_data_len); + header.saturating_add(data_within_active) + } + + /// Determines the next sequence number and last valid offset in the active file. + /// + /// Since sequence numbers are monotonically increasing, the highest sequence + /// is always in the active file (or the most recent rotated file if the active + /// file is empty after a rotation). Returns (next_sequence, active_file_valid_offset). + fn detect_next_sequence(&self) -> WalResult<(u64, u64)> { + let (active_seq, active_valid_offset) = self.scan_file_last_sequence(&self.options.path)?; + + // If active file has entries, use its sequence + if let Some(seq) = active_seq { + return Ok((seq.wrapping_add(1), active_valid_offset)); + } + + // Active file is empty - check the most recent rotated file (highest rotation_id) + if let Some(most_recent) = self.rotated_files.iter().max_by_key(|f| f.rotation_id) { + let (seq, _) = self.scan_file_last_sequence(&most_recent.path)?; + if let Some(s) = seq { + return Ok((s.wrapping_add(1), active_valid_offset)); + } + } + + // No entries anywhere - start at sequence 0 + Ok((0, active_valid_offset)) + } + + /// Scans a WAL file and returns the last valid sequence number and the + /// byte offset immediately after the last valid entry (i.e., where new + /// writes should begin). + fn scan_file_last_sequence(&self, path: &Path) -> WalResult<(Option, u64)> { + if !path.exists() { + return Ok((None, 0)); + } + let mut reader = WalReader::open(path)?; + let iter = reader.iter_from(0)?; + let mut last_seq = None; + let mut last_valid_offset = WAL_HEADER_LEN as u64; + for entry in iter { + match entry { + Ok(bundle) => { + last_seq = Some(bundle.sequence); + last_valid_offset = bundle.next_offset; + } + Err(WalError::UnexpectedEof(_)) | Err(WalError::InvalidEntry(_)) => { + // Partial or corrupted entry - stop here + break; + } + Err(err) => return Err(err), + } + } + Ok((last_seq, last_valid_offset)) + } +} + +impl Drop for WalWriter { + fn drop(&mut self) { + #[cfg(test)] + if self.test_crashed { + return; + } + if self.active_file.unflushed_bytes == 0 { + return; + } + + let _ = self.active_file.flush_now(); + #[cfg(test)] + test_support::record_drop_flush(); + } +} + +#[cfg(test)] +impl WalWriter { + pub(crate) fn test_set_last_flush(&mut self, instant: Instant) { + self.active_file.last_flush = instant; + } + + pub(crate) fn test_last_flush(&self) -> Instant { + self.active_file.last_flush + } + + /// Simulates a process crash by flagging Drop to skip flush/fsync logic. + #[cfg(test)] + pub(crate) fn test_force_crash(mut self) { + self.test_crashed = true; + } +} + +fn system_time_to_nanos(ts: SystemTime) -> WalResult { + let duration = ts + .duration_since(UNIX_EPOCH) + .map_err(|_| WalError::InvalidTimestamp)?; + i64::try_from(duration.as_nanos()).map_err(|_| WalError::InvalidTimestamp) +} + +fn encode_record_batch(batch: &RecordBatch) -> WalResult> { + let schema = batch.schema(); + let mut buffer = Vec::new(); + { + let mut writer = StreamWriter::try_new(&mut buffer, &schema).map_err(WalError::Arrow)?; + writer.write(batch).map_err(WalError::Arrow)?; + writer.finish().map_err(WalError::Arrow)?; + } + Ok(buffer) +} + +fn sync_file_data(file: &File) -> WalResult<()> { + #[cfg(test)] + test_support::record_sync_data(); + file.sync_data()?; + Ok(()) +} + +/// Syncs the parent directory to ensure rename durability. +/// +/// On POSIX systems, `rename()` is atomic but not necessarily durable until the +/// parent directory is fsynced. This matters on filesystems without automatic +/// rename barriers (older ext3, NFS, non-default mount options). +/// +/// On non-Unix platforms this is a no-op since directory sync semantics differ. +pub(super) fn sync_parent_dir(path: &Path) -> WalResult<()> { + #[cfg(unix)] + { + if let Some(parent) = path.parent() { + let dir = File::open(parent)?; + dir.sync_all()?; + } + } + #[cfg(not(unix))] + let _ = path; // silence unused warning + Ok(()) +} + +fn checkpoint_sidecar_path(wal_path: &Path) -> PathBuf { + wal_path + .parent() + .map(|parent| parent.join("checkpoint.offset")) + .unwrap_or_else(|| PathBuf::from("checkpoint.offset")) +} + +fn load_checkpoint_state(path: &Path) -> WalResult { + match CheckpointSidecar::read_from(path) { + Ok(state) => Ok(state), + Err(WalError::InvalidCheckpointSidecar(_)) => Ok(default_checkpoint_state()), + Err(WalError::Io(err)) + if matches!(err.kind(), ErrorKind::NotFound | ErrorKind::UnexpectedEof) => + { + Ok(default_checkpoint_state()) + } + Err(err) => Err(err), + } +} + +fn default_checkpoint_state() -> CheckpointSidecar { + CheckpointSidecar::new(0) +} + +fn rotated_wal_path(base_path: &Path, rotation_id: u64) -> PathBuf { + let mut name = base_path.as_os_str().to_os_string(); + name.push(format!(".{rotation_id}")); + PathBuf::from(name) +} + +/// Discovers rotated WAL files by scanning the directory for files matching +/// `.N` pattern. Returns a list of (rotation_id, file_size) tuples +/// sorted by rotation_id (oldest first). +fn discover_rotated_wal_files(base_path: &Path) -> WalResult> { + let parent = base_path.parent().ok_or_else(|| { + WalError::Io(std::io::Error::new( + ErrorKind::InvalidInput, + "WAL path has no parent directory", + )) + })?; + let base_name = base_path + .file_name() + .and_then(|n| n.to_str()) + .ok_or_else(|| { + WalError::Io(std::io::Error::new( + ErrorKind::InvalidInput, + "WAL path has invalid filename", + )) + })?; + + let prefix = format!("{base_name}."); + let mut discovered = Vec::new(); + + let entries = match std::fs::read_dir(parent) { + Ok(entries) => entries, + Err(err) if err.kind() == ErrorKind::NotFound => return Ok(discovered), + Err(err) => return Err(WalError::Io(err)), + }; + + for entry in entries { + let entry = entry?; + let file_name = entry.file_name(); + let Some(name_str) = file_name.to_str() else { + continue; + }; + let Some(suffix) = name_str.strip_prefix(&prefix) else { + continue; + }; + let Ok(rotation_id) = suffix.parse::() else { + continue; + }; + let metadata = entry.metadata()?; + discovered.push((rotation_id, metadata.len())); + } + + // Sort by rotation_id (oldest first) + discovered.sort_by_key(|(id, _)| *id); + Ok(discovered) +} + +fn reopen_wal_file(path: &Path, segment_hash: [u8; 16]) -> WalResult { + let mut file = OpenOptions::new() + .create(true) + .read(true) + .write(true) + .truncate(true) + .open(path)?; + WalHeader::new(segment_hash).write_to(&mut file)?; + Ok(file) +} + +/// Metadata describing an on-disk rotated WAL file. We retain enough information +/// to decide when the file can be deleted once readers have safely advanced +/// past its logical range. +#[derive(Clone, Debug)] +struct RotatedWalFile { + path: PathBuf, + /// Monotonic rotation id for this file (extracted from the suffix, e.g., wal.5 has id=5). + rotation_id: u64, + file_bytes: u64, + /// Global data offset at the *end* of this file (cumulative across older + /// rotated files). When `global_data_offset >= cumulative_data_offset` the + /// file is fully consumed and can be deleted. + cumulative_data_offset: u64, +} + +impl RotatedWalFile { + fn total_bytes(&self) -> u64 { + self.file_bytes + } +} + +#[cfg(test)] +pub(super) mod test_support { + use std::cell::Cell; + + thread_local! { + static DROP_FLUSH_NOTIFIED: Cell = const { Cell::new(false) }; + static SYNC_DATA_NOTIFIED: Cell = const { Cell::new(false) }; + static NEXT_CRASH: Cell> = const { Cell::new(None) }; + } + + #[derive(Clone, Copy, Debug, PartialEq, Eq)] + pub(crate) enum CrashInjection { + BeforeSidecarRename, + } + + pub fn record_drop_flush() { + DROP_FLUSH_NOTIFIED.with(|cell| cell.set(true)); + } + + pub fn take_drop_flush_notification() -> bool { + DROP_FLUSH_NOTIFIED.with(|cell| { + let notified = cell.get(); + cell.set(false); + notified + }) + } + + pub fn reset_flush_notifications() { + DROP_FLUSH_NOTIFIED.with(|cell| cell.set(false)); + SYNC_DATA_NOTIFIED.with(|cell| cell.set(false)); + NEXT_CRASH.with(|cell| cell.set(None)); + } + + pub fn record_sync_data() { + SYNC_DATA_NOTIFIED.with(|cell| cell.set(true)); + } + + pub fn take_sync_data_notification() -> bool { + SYNC_DATA_NOTIFIED.with(|cell| { + let notified = cell.get(); + cell.set(false); + notified + }) + } + + pub fn inject_crash(point: CrashInjection) { + NEXT_CRASH.with(|cell| cell.set(Some(point))); + } + + pub fn take_crash(point: CrashInjection) -> bool { + NEXT_CRASH.with(|cell| { + if cell.get() == Some(point) { + cell.set(None); + true + } else { + false + } + }) + } +} diff --git a/rust/otap-dataflow/deny.toml b/rust/otap-dataflow/deny.toml index 25d52a855..628babf74 100644 --- a/rust/otap-dataflow/deny.toml +++ b/rust/otap-dataflow/deny.toml @@ -103,6 +103,7 @@ allow = [ "MIT-0", "Apache-2.0", "Unicode-3.0", + "BSD-2-Clause", "BSD-3-Clause", "Zlib", "MPL-2.0",